From 453aeaf1a0e97042174ed99faa2aa91da33a0ead Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Fri, 9 Apr 2021 17:39:23 -0400 Subject: [PATCH 01/45] feat: Add tests for writing RB chunks to Object Store --- Cargo.lock | 1 + data_types/src/chunk.rs | 24 ++- .../influxdata/iox/management/v1/chunk.proto | 5 +- parquet_file/src/chunk.rs | 26 ++- parquet_file/src/table.rs | 13 +- server/Cargo.toml | 1 + server/src/db.rs | 149 +++++++++++++++++- server/src/db/chunk.rs | 30 ++-- server/src/query_tests/utils.rs | 10 ++ 9 files changed, 230 insertions(+), 29 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8ee0b04d3c..741fb6d354 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3274,6 +3274,7 @@ dependencies = [ "serde_json", "snafu", "snap", + "tempfile", "test_helpers", "tokio", "tokio-util", diff --git a/data_types/src/chunk.rs b/data_types/src/chunk.rs index 5945534900..31f61ac13b 100644 --- a/data_types/src/chunk.rs +++ b/data_types/src/chunk.rs @@ -21,8 +21,11 @@ pub enum ChunkStorage { /// The chunk is in the Read Buffer (where it can not be mutated) ReadBuffer, + /// The chunk is both in ReadBuffer and Object Store + ReadBufferAndObjectStore, + /// The chunk is stored in Object Storage (where it can not be mutated) - ObjectStore, + ObjectStoreOnly, } impl ChunkStorage { @@ -32,7 +35,8 @@ impl ChunkStorage { Self::OpenMutableBuffer => "OpenMutableBuffer", Self::ClosedMutableBuffer => "ClosedMutableBuffer", Self::ReadBuffer => "ReadBuffer", - Self::ObjectStore => "ObjectStore", + Self::ReadBufferAndObjectStore => "ReadBufferAndObjectStore", + Self::ObjectStoreOnly => "ObjectStoreOnly", } } } @@ -134,7 +138,8 @@ impl From for management::ChunkStorage { ChunkStorage::OpenMutableBuffer => Self::OpenMutableBuffer, ChunkStorage::ClosedMutableBuffer => Self::ClosedMutableBuffer, ChunkStorage::ReadBuffer => Self::ReadBuffer, - ChunkStorage::ObjectStore => Self::ObjectStore, + ChunkStorage::ReadBufferAndObjectStore => Self::ReadBufferAndObjectStore, + ChunkStorage::ObjectStoreOnly => Self::ObjectStoreOnly, } } } @@ -204,7 +209,10 @@ impl TryFrom for ChunkStorage { management::ChunkStorage::OpenMutableBuffer => Ok(Self::OpenMutableBuffer), management::ChunkStorage::ClosedMutableBuffer => Ok(Self::ClosedMutableBuffer), management::ChunkStorage::ReadBuffer => Ok(Self::ReadBuffer), - management::ChunkStorage::ObjectStore => Ok(Self::ObjectStore), + management::ChunkStorage::ReadBufferAndObjectStore => { + Ok(Self::ReadBufferAndObjectStore) + } + management::ChunkStorage::ObjectStoreOnly => Ok(Self::ObjectStoreOnly), management::ChunkStorage::Unspecified => Err(FieldViolation::required("")), } } @@ -220,7 +228,7 @@ mod test { partition_key: "foo".to_string(), id: 42, estimated_bytes: 1234, - storage: management::ChunkStorage::ObjectStore.into(), + storage: management::ChunkStorage::ObjectStoreOnly.into(), time_of_first_write: None, time_of_last_write: None, time_closing: None, @@ -231,7 +239,7 @@ mod test { partition_key: Arc::new("foo".to_string()), id: 42, estimated_bytes: 1234, - storage: ChunkStorage::ObjectStore, + storage: ChunkStorage::ObjectStoreOnly, time_of_first_write: None, time_of_last_write: None, time_closing: None, @@ -250,7 +258,7 @@ mod test { partition_key: Arc::new("foo".to_string()), id: 42, estimated_bytes: 1234, - storage: ChunkStorage::ObjectStore, + storage: ChunkStorage::ObjectStoreOnly, time_of_first_write: None, time_of_last_write: None, time_closing: None, @@ -262,7 +270,7 @@ mod test { partition_key: "foo".to_string(), id: 42, estimated_bytes: 1234, - storage: management::ChunkStorage::ObjectStore.into(), + storage: management::ChunkStorage::ObjectStoreOnly.into(), time_of_first_write: None, time_of_last_write: None, time_closing: None, diff --git a/generated_types/protos/influxdata/iox/management/v1/chunk.proto b/generated_types/protos/influxdata/iox/management/v1/chunk.proto index 9bd8a5f745..b7502822c0 100644 --- a/generated_types/protos/influxdata/iox/management/v1/chunk.proto +++ b/generated_types/protos/influxdata/iox/management/v1/chunk.proto @@ -17,8 +17,11 @@ enum ChunkStorage { // The chunk is in the Read Buffer (where it can not be mutated) CHUNK_STORAGE_READ_BUFFER = 3; + // The chunk is in the Read Buffer and Object Store + CHUNK_STORAGE_READ_BUFFER_AND_OBJECT_STORE = 4; + // The chunk is stored in Object Storage (where it can not be mutated) - CHUNK_STORAGE_OBJECT_STORE = 4; + CHUNK_STORAGE_OBJECT_STORE_ONLY = 5; } // `Chunk` represents part of a partition of data in a database. diff --git a/parquet_file/src/chunk.rs b/parquet_file/src/chunk.rs index fa6e0879b1..8f971924b7 100644 --- a/parquet_file/src/chunk.rs +++ b/parquet_file/src/chunk.rs @@ -10,13 +10,13 @@ use std::mem; #[derive(Debug)] pub struct Chunk { /// Partition this chunk belongs to - pub partition_key: String, + partition_key: String, /// The id for this chunk - pub id: u32, + id: u32, /// Tables of this chunk - pub tables: Vec, + tables: Vec
, /// Track memory used by this chunk memory_tracker: MemTracker, @@ -34,6 +34,26 @@ impl Chunk { chunk } + /// Return the chunk id + pub fn id(&self) -> u32 { + self.id + } + + /// Return the chunk's partition key + pub fn partition_key(&self) -> &str { + self.partition_key.as_ref() + } + + /// Return all paths of this chunks + pub fn all_paths(&self) -> Vec { + self.tables.iter().map(|t| t.path()).collect() + } + + /// Returns a vec of the summary statistics of the tables in this chunk + pub fn table_summaries(&self) -> Vec { + self.tables.iter().map(|t| t.table_summary()).collect() + } + /// Add a chunk's table and its summary pub fn add_table(&mut self, table_summary: TableSummary, file_location: Path) { self.tables.push(Table::new(table_summary, file_location)); diff --git a/parquet_file/src/table.rs b/parquet_file/src/table.rs index 48fedd3c9e..4f2776e8ea 100644 --- a/parquet_file/src/table.rs +++ b/parquet_file/src/table.rs @@ -7,12 +7,12 @@ use std::mem; #[derive(Debug, Clone)] pub struct Table { /// Meta data of the table - pub table_summary: TableSummary, + table_summary: TableSummary, /// Path in the object store. Format: /// //data///.parquet - pub object_store_path: Path, + object_store_path: Path, } impl Table { @@ -23,6 +23,10 @@ impl Table { } } + pub fn table_summary(&self) -> TableSummary { + self.table_summary.clone() + } + pub fn has_table(&self, table_name: &str) -> bool { self.table_summary.has_table(table_name) } @@ -38,4 +42,9 @@ impl Table { pub fn name(&self) -> String { self.table_summary.name.clone() } + + /// Return the object store path of this table + pub fn path(&self) -> Path { + self.object_store_path.clone() + } } diff --git a/server/Cargo.toml b/server/Cargo.toml index 6a677b113b..85d6ec8fad 100644 --- a/server/Cargo.toml +++ b/server/Cargo.toml @@ -29,6 +29,7 @@ serde = "1.0" serde_json = "1.0" snafu = "0.6" snap = "1.0.0" +tempfile = "3.1.0" tokio = { version = "1.0", features = ["macros", "time"] } tokio-util = { version = "0.6.3" } tracker = { path = "../tracker" } diff --git a/server/src/db.rs b/server/src/db.rs index cdbbb45e3d..5673bd0364 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -479,8 +479,6 @@ impl Db { let predicate = read_buffer::Predicate::default(); // Get RecordBatchStream of data from the read buffer chunk - // TODO: When we have the rb_chunk, the following code will be replaced with one - // line let stream = rb_chunk.read_filter() let read_results = rb_chunk .read_filter(stats.name.as_str(), predicate, Selection::All) .context(ReadBufferChunkError { chunk_id })?; @@ -719,8 +717,11 @@ impl CatalogProvider for Db { #[cfg(test)] mod tests { + use crate::query_tests::utils::{make_database, make_db}; use arrow_deps::{ - arrow::record_batch::RecordBatch, assert_table_eq, datafusion::physical_plan::collect, + arrow::record_batch::RecordBatch, + assert_table_eq, + datafusion::{self, execution::context, physical_plan::collect}, }; use chrono::Utc; use data_types::{ @@ -728,15 +729,24 @@ mod tests { database_rules::{Order, Sort, SortOrder}, partition_metadata::{ColumnSummary, StatValues, Statistics, TableSummary}, }; + use object_store::{ + disk::File, path::ObjectStorePath, path::Path, ObjectStore, ObjectStoreApi, + }; use query::{ exec::Executor, frontend::sql::SQLQueryPlanner, test::TestLPWriter, PartitionChunk, }; use test_helpers::assert_contains; - use crate::query_tests::utils::make_db; - use super::*; + use futures::stream; + use futures::{StreamExt, TryStreamExt}; + use std::iter::Iterator; use std::num::NonZeroUsize; + use std::str; + use tempfile::TempDir; + + type Error = Box; + type Result = std::result::Result; #[tokio::test] async fn write_no_mutable_buffer() { @@ -895,6 +905,120 @@ mod tests { // cpu").await; assert_table_eq!(expected, &batches); } + async fn flatten_list_stream( + storage: Arc, + prefix: Option<&Path>, + ) -> Result> { + storage + .list(prefix) + .await? + .map_ok(|v| stream::iter(v).map(Ok)) + .try_flatten() + .try_collect() + .await + } + + async fn datafusion_plan_and_collect( + ctx: &mut context::ExecutionContext, + sql: &str, + ) -> Result, datafusion::error::DataFusionError> { + let logical_plan = ctx.create_logical_plan(sql).expect("Create logical plan"); + let logical_plan = ctx.optimize(&logical_plan).expect("Optimize Logical Plan"); + let physical_plan = ctx + .create_physical_plan(&logical_plan) + .expect("Create Physical Plan"); + collect(physical_plan).await + } + + #[tokio::test] + async fn write_to_parquet_file() { + // Test that data can be written into parquet files + + // Create an object store with a specified location in a local disk + let root = TempDir::new().unwrap(); + let object_store = Arc::new(ObjectStore::new_file(File::new(root.path()))); + + // Create a DB given a server id, an object store and a db name + let server_id: NonZeroU32 = NonZeroU32::new(10).unwrap(); + let db_name = "parquet_test_db"; + let db = Arc::new(make_database(server_id, Arc::clone(&object_store), db_name)); + + // Write some line protocols in Mutable buffer of the DB + let mut writer = TestLPWriter::default(); + writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); + writer.write_lp_string(db.as_ref(), "cpu bar=2 20").unwrap(); + + //Now mark the MB chunk close + let partition_key = "1970-01-01T00"; + let mb_chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); + // Move that MB chunk to RB chunk and drop it from MB + let rb_chunk = db + .load_chunk_to_read_buffer(partition_key, mb_chunk.id()) + .await + .unwrap(); + // Write the RB chunk to Object Store but keep it in RB + let pq_chunk = db + .load_chunk_to_object_store(partition_key, mb_chunk.id()) + .await + .unwrap(); + + // it should be the same chunk! + assert_eq!(mb_chunk.id(), rb_chunk.id()); + assert_eq!(mb_chunk.id(), pq_chunk.id()); + + // we should have chunks in the mutable buffer, read buffer, and object store + // (Note the currently open chunk is not listed) + assert_eq!(mutable_chunk_ids(&db, partition_key), vec![1]); + assert_eq!(read_buffer_chunk_ids(&db, partition_key), vec![0]); + assert_eq!(read_parquet_file_chunk_ids(&db, partition_key), vec![0]); + + // Verify data written to the parquet file in object store + // First, there must be one path of object store in the catalog + let paths = pq_chunk.object_store_paths(); + assert_eq!(paths.len(), 1); + + // Check that the path must exist in the object store + let path_list = flatten_list_stream(Arc::clone(&object_store), Some(&paths[0])) + .await + .unwrap(); + println!("path_list: {:#?}", path_list); + assert_eq!(path_list.len(), 1); + assert_eq!(path_list, paths.clone()); + + // Get full string path + // Todo: it is better if we can get this full path from a function + // in object_store::path::Path (will talk with mkm (aka Marko)) + let root_path = format!("{:?}", root.path()); + let root_path = root_path.trim_matches('"'); + let path = format!("{}/{}", root_path, paths[0].display()); + println!("path: {}", path); + + // Create External table of this parquet file to get its content in a human + // readable form + let sql = format!( + "CREATE EXTERNAL TABLE parquet_table STORED AS PARQUET LOCATION '{}'", + path + ); + + let mut ctx = context::ExecutionContext::new(); + let df = ctx.sql(&sql).unwrap(); + df.collect().await.unwrap(); + + // Select data from that table + let sql = "SELECT * FROM parquet_table"; + let content = datafusion_plan_and_collect(&mut ctx, &sql).await.unwrap(); + println!("Content: {:?}", content); + let expected = vec![ + "+-----+------+", + "| bar | time |", + "+-----+------+", + "| 1 | 10 |", + "| 2 | 20 |", + "+-----+------+", + ]; + assert_table_eq!(expected, &content); + } + #[tokio::test] async fn write_updates_last_write_at() { let db = make_db(); @@ -1413,6 +1537,21 @@ mod tests { .into_iter() .filter_map(|chunk| match chunk.storage { ChunkStorage::ReadBuffer => Some(chunk.id), + ChunkStorage::ReadBufferAndObjectStore => Some(chunk.id), + _ => None, + }) + .collect(); + chunk_ids.sort_unstable(); + chunk_ids + } + + fn read_parquet_file_chunk_ids(db: &Db, partition_key: &str) -> Vec { + let mut chunk_ids: Vec = db + .partition_chunk_summaries(partition_key) + .into_iter() + .filter_map(|chunk| match chunk.storage { + ChunkStorage::ReadBufferAndObjectStore => Some(chunk.id), + ChunkStorage::ObjectStoreOnly => Some(chunk.id), _ => None, }) .collect(); diff --git a/server/src/db/chunk.rs b/server/src/db/chunk.rs index 6f5880f77a..a275f986fc 100644 --- a/server/src/db/chunk.rs +++ b/server/src/db/chunk.rs @@ -2,6 +2,7 @@ use arrow_deps::datafusion::physical_plan::SendableRecordBatchStream; use data_types::chunk::{ChunkStorage, ChunkSummary}; use internal_types::{schema::Schema, selection::Selection}; use mutable_buffer::chunk::Chunk as MBChunk; +use object_store::path::Path; use observability_deps::tracing::debug; use parquet_file::chunk::Chunk as ParquetChunk; use query::{exec::stringset::StringSet, predicate::Predicate, PartitionChunk}; @@ -165,9 +166,20 @@ impl DBChunk { estimated_bytes, ) } - Self::ParquetFile { .. } => { - unimplemented!("parquet file summary not implemented") - } + Self::ParquetFile { chunk } => ChunkSummary::new_without_timestamps( + Arc::clone(&Arc::new(chunk.partition_key().to_string())), + chunk.id(), + ChunkStorage::ReadBufferAndObjectStore, + chunk.size(), + ), + } + } + + /// Return object store paths + pub fn object_store_paths(&self) -> Vec { + match self { + Self::ParquetFile { chunk } => chunk.all_paths(), + _ => vec![], } } } @@ -179,7 +191,7 @@ impl PartitionChunk for DBChunk { match self { Self::MutableBuffer { chunk, .. } => chunk.id(), Self::ReadBuffer { chunk, .. } => chunk.id(), - Self::ParquetFile { .. } => unimplemented!("parquet file not implemented"), + Self::ParquetFile { chunk, .. } => chunk.id(), } } @@ -187,7 +199,7 @@ impl PartitionChunk for DBChunk { match self { Self::MutableBuffer { chunk, .. } => chunk.table_summaries(), Self::ReadBuffer { chunk, .. } => chunk.table_summaries(), - Self::ParquetFile { .. } => unimplemented!("parquet file not implemented"), + Self::ParquetFile { chunk } => chunk.table_summaries(), } } @@ -201,7 +213,7 @@ impl PartitionChunk for DBChunk { known_tables.insert(name); } } - Self::ParquetFile { .. } => unimplemented!("parquet file not implemented"), + Self::ParquetFile { chunk, .. } => chunk.all_table_names(known_tables), } } @@ -252,7 +264,7 @@ impl PartitionChunk for DBChunk { Some(chunk.table_names(&rb_predicate, &BTreeSet::new())) } Self::ParquetFile { .. } => { - unimplemented!("parquet file not implemented") + unimplemented!("parquet file not implemented for scan_data") } }; @@ -313,9 +325,7 @@ impl PartitionChunk for DBChunk { match self { Self::MutableBuffer { chunk, .. } => chunk.has_table(table_name), Self::ReadBuffer { chunk, .. } => chunk.has_table(table_name), - Self::ParquetFile { .. } => { - unimplemented!("parquet file not implemented for has_table") - } + Self::ParquetFile { chunk, .. } => chunk.has_table(table_name), } } diff --git a/server/src/query_tests/utils.rs b/server/src/query_tests/utils.rs index fcf1bf5178..46bda879d2 100644 --- a/server/src/query_tests/utils.rs +++ b/server/src/query_tests/utils.rs @@ -23,6 +23,16 @@ pub fn make_db() -> Db { ) } +pub fn make_database(server_id: NonZeroU32, object_store: Arc, db_name: &str) -> Db { + Db::new( + DatabaseRules::new(DatabaseName::new(db_name.to_string()).unwrap()), + server_id, + object_store, + None, // wal buffer + Arc::new(JobRegistry::new()), + ) +} + fn chunk_summary_iter(db: &Db) -> impl Iterator + '_ { db.partition_keys() .unwrap() From e86a02efa719eb17755e7629866070223225c403 Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Mon, 12 Apr 2021 11:46:31 -0400 Subject: [PATCH 02/45] refactor: Address comments and add more tests --- server/Cargo.toml | 2 +- server/src/db.rs | 132 ++++++++++++++++++++++++++++++++++++++++------ 2 files changed, 116 insertions(+), 18 deletions(-) diff --git a/server/Cargo.toml b/server/Cargo.toml index 85d6ec8fad..d27f1ac029 100644 --- a/server/Cargo.toml +++ b/server/Cargo.toml @@ -29,7 +29,6 @@ serde = "1.0" serde_json = "1.0" snafu = "0.6" snap = "1.0.0" -tempfile = "3.1.0" tokio = { version = "1.0", features = ["macros", "time"] } tokio-util = { version = "0.6.3" } tracker = { path = "../tracker" } @@ -37,3 +36,4 @@ uuid = { version = "0.8", features = ["serde", "v4"] } [dev-dependencies] # In alphabetical order test_helpers = { path = "../test_helpers" } +tempfile = "3.1.0" diff --git a/server/src/db.rs b/server/src/db.rs index 5673bd0364..d358c4766b 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -721,7 +721,7 @@ mod tests { use arrow_deps::{ arrow::record_batch::RecordBatch, assert_table_eq, - datafusion::{self, execution::context, physical_plan::collect}, + datafusion::{execution::context, physical_plan::collect}, }; use chrono::Utc; use data_types::{ @@ -918,20 +918,8 @@ mod tests { .await } - async fn datafusion_plan_and_collect( - ctx: &mut context::ExecutionContext, - sql: &str, - ) -> Result, datafusion::error::DataFusionError> { - let logical_plan = ctx.create_logical_plan(sql).expect("Create logical plan"); - let logical_plan = ctx.optimize(&logical_plan).expect("Optimize Logical Plan"); - let physical_plan = ctx - .create_physical_plan(&logical_plan) - .expect("Create Physical Plan"); - collect(physical_plan).await - } - #[tokio::test] - async fn write_to_parquet_file() { + async fn write_one_chunk_one_table_to_parquet_file() { // Test that data can be written into parquet files // Create an object store with a specified location in a local disk @@ -986,8 +974,6 @@ mod tests { assert_eq!(path_list, paths.clone()); // Get full string path - // Todo: it is better if we can get this full path from a function - // in object_store::path::Path (will talk with mkm (aka Marko)) let root_path = format!("{:?}", root.path()); let root_path = root_path.trim_matches('"'); let path = format!("{}/{}", root_path, paths[0].display()); @@ -995,6 +981,7 @@ mod tests { // Create External table of this parquet file to get its content in a human // readable form + // Note: We do not care about escaping quotes here because it is just a test let sql = format!( "CREATE EXTERNAL TABLE parquet_table STORED AS PARQUET LOCATION '{}'", path @@ -1006,7 +993,7 @@ mod tests { // Select data from that table let sql = "SELECT * FROM parquet_table"; - let content = datafusion_plan_and_collect(&mut ctx, &sql).await.unwrap(); + let content = ctx.sql(&sql).unwrap().collect().await.unwrap(); println!("Content: {:?}", content); let expected = vec![ "+-----+------+", @@ -1019,6 +1006,117 @@ mod tests { assert_table_eq!(expected, &content); } + #[tokio::test] + async fn write_one_chunk_many_tables_to_parquet_files() { + // Test that data can be written into parquet files + + // Create an object store with a specified location in a local disk + let root = TempDir::new().unwrap(); + let object_store = Arc::new(ObjectStore::new_file(File::new(root.path()))); + + // Create a DB given a server id, an object store and a db name + let server_id: NonZeroU32 = NonZeroU32::new(10).unwrap(); + let db_name = "parquet_test_db"; + let db = Arc::new(make_database(server_id, Arc::clone(&object_store), db_name)); + + // Write some line protocols in Mutable buffer of the DB + let mut writer = TestLPWriter::default(); + writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); + writer + .write_lp_string(db.as_ref(), "disk ops=1 20") + .unwrap(); + writer.write_lp_string(db.as_ref(), "cpu bar=2 20").unwrap(); + + //Now mark the MB chunk close + let partition_key = "1970-01-01T00"; + let mb_chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); + // Move that MB chunk to RB chunk and drop it from MB + let rb_chunk = db + .load_chunk_to_read_buffer(partition_key, mb_chunk.id()) + .await + .unwrap(); + // Write the RB chunk to Object Store but keep it in RB + let pq_chunk = db + .load_chunk_to_object_store(partition_key, mb_chunk.id()) + .await + .unwrap(); + + // it should be the same chunk! + assert_eq!(mb_chunk.id(), rb_chunk.id()); + assert_eq!(mb_chunk.id(), pq_chunk.id()); + + // we should have chunks in the mutable buffer, read buffer, and object store + // (Note the currently open chunk is not listed) + assert_eq!(mutable_chunk_ids(&db, partition_key), vec![1]); + assert_eq!(read_buffer_chunk_ids(&db, partition_key), vec![0]); + assert_eq!(read_parquet_file_chunk_ids(&db, partition_key), vec![0]); + + // Verify data written to the parquet files in object store + // First, there must be 2 paths of object store in the catalog + // that represents 2 files + let paths = pq_chunk.object_store_paths(); + assert_eq!(paths.len(), 2); + + // Check that the path must exist in the object store + let prefix = object_store.new_path(); + let path_list = flatten_list_stream(Arc::clone(&object_store), Some(&prefix)) + .await + .unwrap(); + println!("path_list: {:#?}", path_list); + assert_eq!(path_list.len(), 2); + assert_eq!(path_list, paths.clone()); + + // Check the content of each path + + // Root path + let root_path = format!("{:?}", root.path()); + let root_path = root_path.trim_matches('"'); + + let mut i = 0; + while i < 2 { + // Get full string path + let path = format!("{}/{}", root_path, paths[i].display()); + println!("path: {}", path); + + // Create External table of this parquet file to get its content in a human + // readable form + // Note: We do not care about escaping quotes here because it is just a test + let sql = format!( + "CREATE EXTERNAL TABLE parquet_table STORED AS PARQUET LOCATION '{}'", + path + ); + + let mut ctx = context::ExecutionContext::new(); + let df = ctx.sql(&sql).unwrap(); + df.collect().await.unwrap(); + + // Select data from that table + let sql = "SELECT * FROM parquet_table"; + let content = ctx.sql(&sql).unwrap().collect().await.unwrap(); + println!("Content: {:?}", content); + let mut expected = vec![ + "+-----+------+", + "| bar | time |", + "+-----+------+", + "| 1 | 10 |", + "| 2 | 20 |", + "+-----+------+", + ]; + if i == 1 { + expected = vec![ + "+-----+------+", + "| ops | time |", + "+-----+------+", + "| 1 | 20 |", + "+-----+------+", + ]; + } + + assert_table_eq!(expected, &content); + i += 1; + } + } + #[tokio::test] async fn write_updates_last_write_at() { let db = make_db(); From dad8d6bafd10901583e5a7e1598dbb03fad87602 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 9 Apr 2021 15:17:21 -0400 Subject: [PATCH 03/45] chore: Update Entry with test helpers --- internal_types/src/entry.rs | 120 +++++++++++++++++++++--------------- 1 file changed, 72 insertions(+), 48 deletions(-) diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index dac86f82e9..12a9594741 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -1180,10 +1180,81 @@ impl TryFrom> for SequencedEntry { } } +pub mod test_helpers { + use super::*; + use influxdb_line_protocol::parse_lines; + + /// Converts the line protocol to a vec of ShardedEntry with a single shard + /// and a single partition + pub fn lp_to_entry(lp: &str) -> Entry { + let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); + + lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)) + .unwrap() + .pop() + .unwrap() + .entry + } + + /// Returns a test sharder that will assign shard ids from [0, count) + /// incrementing for each line. + pub fn sharder(count: u16) -> TestSharder { + TestSharder { + count, + n: std::cell::RefCell::new(0), + } + } + + // For each line passed to shard returns a shard id from [0, count) in order + #[derive(Debug)] + pub struct TestSharder { + count: u16, + n: std::cell::RefCell, + } + + impl Sharder for TestSharder { + fn shard(&self, _line: &ParsedLine<'_>) -> Result { + let n = *self.n.borrow(); + self.n.replace(n + 1); + Ok(n % self.count) + } + } + + /// Returns a test partitioner that will assign partition keys in the form + /// key_# where # is replaced by a number [0, count) incrementing for + /// each line. + pub fn partitioner(count: u8) -> TestPartitioner { + TestPartitioner { + count, + n: std::cell::RefCell::new(0), + } + } + + // For each line passed to partition_key returns a key with a number from + // [0, count) + #[derive(Debug)] + pub struct TestPartitioner { + count: u8, + n: std::cell::RefCell, + } + + impl Partitioner for TestPartitioner { + fn partition_key( + &self, + _line: &ParsedLine<'_>, + _default_time: &DateTime, + ) -> data_types::database_rules::Result { + let n = *self.n.borrow(); + self.n.replace(n + 1); + Ok(format!("key_{}", n % self.count)) + } + } +} + #[cfg(test)] mod tests { + use super::test_helpers::*; use super::*; - use influxdb_line_protocol::parse_lines; #[test] @@ -1725,51 +1796,4 @@ mod tests { let values = col.values().f64_values().unwrap(); assert_eq!(&values, &[None, Some(23.2), None]); } - - fn sharder(count: u16) -> TestSharder { - TestSharder { - count, - n: std::cell::RefCell::new(0), - } - } - - // For each line passed to shard returns a shard id from [0, count) in order - struct TestSharder { - count: u16, - n: std::cell::RefCell, - } - - impl Sharder for TestSharder { - fn shard(&self, _line: &ParsedLine<'_>) -> Result { - let n = *self.n.borrow(); - self.n.replace(n + 1); - Ok(n % self.count) - } - } - - fn partitioner(count: u8) -> TestPartitioner { - TestPartitioner { - count, - n: std::cell::RefCell::new(0), - } - } - - // For each line passed to partition_key returns a key with a number from [0, - // count) - struct TestPartitioner { - count: u8, - n: std::cell::RefCell, - } - - impl Partitioner for TestPartitioner { - fn partition_key( - &self, - _line: &ParsedLine<'_>, - _default_time: &DateTime, - ) -> data_types::database_rules::Result { - let n = *self.n.borrow(); - self.n.replace(n + 1); - Ok(format!("key_{}", n % self.count)) - } - } } From 3f928ed374ed3ce8990d5b99a88556b5670b8f80 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 9 Apr 2021 15:19:17 -0400 Subject: [PATCH 04/45] refactor: Add ClockValue and WriterId to Entry --- internal_types/src/entry.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index 12a9594741..871a3d1dda 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -2,7 +2,7 @@ //! from line protocol and the `DatabaseRules` configuration. use crate::schema::TIME_COLUMN_NAME; -use data_types::database_rules::{Error as DataError, Partitioner, ShardId, Sharder}; +use data_types::database_rules::{Error as DataError, Partitioner, ShardId, Sharder, WriterId}; use generated_types::entry as entry_fb; use influxdb_line_protocol::{FieldValue, ParsedLine}; @@ -1087,6 +1087,8 @@ enum ColumnRaw<'a> { Bool(Vec), } +pub type ClockValue = u64; + #[self_referencing] #[derive(Debug)] pub struct SequencedEntry { @@ -1151,11 +1153,11 @@ impl SequencedEntry { } } - pub fn clock_value(&self) -> u64 { + pub fn clock_value(&self) -> ClockValue { self.fb().clock_value() } - pub fn writer_id(&self) -> u32 { + pub fn writer_id(&self) -> WriterId { self.fb().writer_id() } } From 0a3386f24a93e168bade11acee0e6a53cb71f80a Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 9 Apr 2021 15:33:26 -0400 Subject: [PATCH 05/45] refactor: Make PartitionWrite, Table, and Column return keys/names --- internal_types/src/entry.rs | 92 ++++++++++++++++++++----------------- 1 file changed, 51 insertions(+), 41 deletions(-) diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index 871a3d1dda..acc067d53c 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -52,6 +52,16 @@ pub enum ColumnError { pub type Result = std::result::Result; type ColumnResult = std::result::Result; +// The flatbuffers structs for partition write, table write batch, and column +// all have string values for their key/name. These values should always be +// present. They are if the flatbuffer was generated by code here. However, +// those are all returned as Option<&str> from the flatbuffer. So to keep us +// from having option unwraps all over the place when dealing with this, we'll +// use this default value to return in cases where the partition key, table +// name, or column name are not there. If you see this value anywhere, you know +// there's some illigitimate flatbuffer data somewhere. +const UNKNOWN_STRING_VALUE: &str = "unknown_from_flatbuffer"; + /// Converts parsed line protocol into a collection of ShardedEntry with the /// underlying flatbuffers bytes generated. pub fn lines_to_sharded_entries( @@ -337,8 +347,8 @@ pub struct PartitionWrite<'a> { } impl<'a> PartitionWrite<'a> { - pub fn key(&self) -> Option<&str> { - self.fb.key() + pub fn key(&self) -> &str { + self.fb.key().unwrap_or(UNKNOWN_STRING_VALUE) } pub fn table_batches(&self) -> Vec> { @@ -360,8 +370,8 @@ pub struct TableBatch<'a> { } impl<'a> TableBatch<'a> { - pub fn name(&self) -> Option<&str> { - self.fb.name() + pub fn name(&self) -> &str { + self.fb.name().unwrap_or(UNKNOWN_STRING_VALUE) } pub fn columns(&self) -> Vec> { @@ -424,8 +434,8 @@ pub struct Column<'a> { } impl<'a> Column<'a> { - pub fn name(&self) -> Option<&str> { - self.fb.name() + pub fn name(&self) -> &str { + self.fb.name().unwrap_or(UNKNOWN_STRING_VALUE) } pub fn logical_type(&self) -> entry_fb::LogicalColumnType { @@ -1292,8 +1302,8 @@ mod tests { let partition_writes = sharded_entries[0].entry.partition_writes().unwrap(); assert_eq!(partition_writes.len(), 2); - assert_eq!(partition_writes[0].key().unwrap(), "key_0"); - assert_eq!(partition_writes[1].key().unwrap(), "key_1"); + assert_eq!(partition_writes[0].key(), "key_0"); + assert_eq!(partition_writes[1].key(), "key_1"); } #[test] @@ -1315,9 +1325,9 @@ mod tests { let table_batches = partition_writes[0].table_batches(); assert_eq!(table_batches.len(), 3); - assert_eq!(table_batches[0].name().unwrap(), "cpu"); - assert_eq!(table_batches[1].name().unwrap(), "disk"); - assert_eq!(table_batches[2].name().unwrap(), "mem"); + assert_eq!(table_batches[0].name(), "cpu"); + assert_eq!(table_batches[1].name(), "disk"); + assert_eq!(table_batches[2].name(), "mem"); } #[test] @@ -1336,22 +1346,22 @@ mod tests { assert_eq!(columns.len(), 5); - assert_eq!(columns[0].name().unwrap(), "host"); + assert_eq!(columns[0].name(), "host"); assert_eq!(columns[0].logical_type(), entry_fb::LogicalColumnType::Tag); - assert_eq!(columns[1].name().unwrap(), "region"); + assert_eq!(columns[1].name(), "region"); assert_eq!(columns[1].logical_type(), entry_fb::LogicalColumnType::Tag); - assert_eq!(columns[2].name().unwrap(), "time"); + assert_eq!(columns[2].name(), "time"); assert_eq!(columns[2].logical_type(), entry_fb::LogicalColumnType::Time); - assert_eq!(columns[3].name().unwrap(), "val"); + assert_eq!(columns[3].name(), "val"); assert_eq!( columns[3].logical_type(), entry_fb::LogicalColumnType::Field ); - assert_eq!(columns[4].name().unwrap(), "val2"); + assert_eq!(columns[4].name(), "val2"); assert_eq!( columns[4].logical_type(), entry_fb::LogicalColumnType::Field @@ -1385,17 +1395,17 @@ mod tests { assert_eq!(columns.len(), 7); let col = columns.get(0).unwrap(); - assert_eq!(col.name().unwrap(), "bval"); + assert_eq!(col.name(), "bval"); let values = col.values().bool_values().unwrap(); assert_eq!(&values, &[Some(true), Some(false)]); let col = columns.get(1).unwrap(); - assert_eq!(col.name().unwrap(), "fval"); + assert_eq!(col.name(), "fval"); let values = col.values().f64_values().unwrap(); assert_eq!(&values, &[Some(1.2), Some(2.2)]); let col = columns.get(2).unwrap(); - assert_eq!(col.name().unwrap(), "host"); + assert_eq!(col.name(), "host"); let values = match col.values() { TypedValuesIterator::String(v) => v, _ => panic!("wrong type"), @@ -1404,12 +1414,12 @@ mod tests { assert_eq!(&values, &[Some("a"), Some("b")]); let col = columns.get(3).unwrap(); - assert_eq!(col.name().unwrap(), "ival"); + assert_eq!(col.name(), "ival"); let values = col.values().i64_values().unwrap(); assert_eq!(&values, &[Some(23), Some(22)]); let col = columns.get(4).unwrap(); - assert_eq!(col.name().unwrap(), "sval"); + assert_eq!(col.name(), "sval"); let values = match col.values() { TypedValuesIterator::String(v) => v, _ => panic!("wrong type"), @@ -1418,12 +1428,12 @@ mod tests { assert_eq!(&values, &[Some("hi"), Some("world")]); let col = columns.get(5).unwrap(); - assert_eq!(col.name().unwrap(), TIME_COLUMN_NAME); + assert_eq!(col.name(), TIME_COLUMN_NAME); let values = col.values().i64_values().unwrap(); assert_eq!(&values, &[Some(1), Some(2)]); let col = columns.get(6).unwrap(); - assert_eq!(col.name().unwrap(), "uval"); + assert_eq!(col.name(), "uval"); let values = col.values().u64_values().unwrap(); assert_eq!(&values, &[Some(7), Some(1)]); } @@ -1456,13 +1466,13 @@ mod tests { assert_eq!(columns.len(), 7); let col = columns.get(0).unwrap(); - assert_eq!(col.name().unwrap(), "bool"); + assert_eq!(col.name(), "bool"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = col.values().bool_values().unwrap(); assert_eq!(&values, &[None, None, Some(true)]); let col = columns.get(1).unwrap(); - assert_eq!(col.name().unwrap(), "host"); + assert_eq!(col.name(), "host"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Tag); let values = match col.values() { TypedValuesIterator::String(v) => v, @@ -1472,7 +1482,7 @@ mod tests { assert_eq!(&values, &[Some("a"), Some("a"), None]); let col = columns.get(2).unwrap(); - assert_eq!(col.name().unwrap(), "region"); + assert_eq!(col.name(), "region"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Tag); let values = match col.values() { TypedValuesIterator::String(v) => v, @@ -1482,7 +1492,7 @@ mod tests { assert_eq!(&values, &[None, Some("west"), None]); let col = columns.get(3).unwrap(); - assert_eq!(col.name().unwrap(), "string"); + assert_eq!(col.name(), "string"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = match col.values() { TypedValuesIterator::String(v) => v, @@ -1492,19 +1502,19 @@ mod tests { assert_eq!(&values, &[None, None, Some("hello")]); let col = columns.get(4).unwrap(); - assert_eq!(col.name().unwrap(), TIME_COLUMN_NAME); + assert_eq!(col.name(), TIME_COLUMN_NAME); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Time); let values = col.values().i64_values().unwrap(); assert_eq!(&values, &[Some(983), Some(2343), Some(222)]); let col = columns.get(5).unwrap(); - assert_eq!(col.name().unwrap(), "val"); + assert_eq!(col.name(), "val"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = col.values().i64_values().unwrap(); assert_eq!(&values, &[Some(23), None, Some(21)]); let col = columns.get(6).unwrap(); - assert_eq!(col.name().unwrap(), "val2"); + assert_eq!(col.name(), "val2"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = col.values().f64_values().unwrap(); assert_eq!(&values, &[None, Some(23.2), None]); @@ -1577,7 +1587,7 @@ mod tests { assert_eq!(batch.row_count(), 1); let col = columns.get(1).unwrap(); - assert_eq!(col.name().unwrap(), "val"); + assert_eq!(col.name(), "val"); let values = col.values().i64_values().unwrap(); assert_eq!(&values, &[Some(1)]); @@ -1608,7 +1618,7 @@ mod tests { assert_eq!(batch.row_count(), 8); let col = columns.get(1).unwrap(); - assert_eq!(col.name().unwrap(), "val"); + assert_eq!(col.name(), "val"); let values = col.values().i64_values().unwrap(); assert_eq!( &values, @@ -1652,7 +1662,7 @@ mod tests { assert_eq!(batch.row_count(), 9); let col = columns.get(1).unwrap(); - assert_eq!(col.name().unwrap(), "val"); + assert_eq!(col.name(), "val"); let values = col.values().i64_values().unwrap(); assert_eq!( &values, @@ -1691,7 +1701,7 @@ mod tests { let columns = batch.columns(); let col = columns.get(0).unwrap(); - assert_eq!(col.name().unwrap(), TIME_COLUMN_NAME); + assert_eq!(col.name(), TIME_COLUMN_NAME); let values = col.values().i64_values().unwrap(); assert!(values[0].unwrap() > t); assert_eq!(values[1], Some(123)); @@ -1745,13 +1755,13 @@ mod tests { assert_eq!(columns.len(), 7); let col = columns.get(0).unwrap(); - assert_eq!(col.name().unwrap(), "bool"); + assert_eq!(col.name(), "bool"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = col.values().bool_values().unwrap(); assert_eq!(&values, &[None, None, Some(true)]); let col = columns.get(1).unwrap(); - assert_eq!(col.name().unwrap(), "host"); + assert_eq!(col.name(), "host"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Tag); let values = match col.values() { TypedValuesIterator::String(v) => v, @@ -1761,7 +1771,7 @@ mod tests { assert_eq!(&values, &[Some("a"), Some("a"), None]); let col = columns.get(2).unwrap(); - assert_eq!(col.name().unwrap(), "region"); + assert_eq!(col.name(), "region"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Tag); let values = match col.values() { TypedValuesIterator::String(v) => v, @@ -1771,7 +1781,7 @@ mod tests { assert_eq!(&values, &[None, Some("west"), None]); let col = columns.get(3).unwrap(); - assert_eq!(col.name().unwrap(), "string"); + assert_eq!(col.name(), "string"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = match col.values() { TypedValuesIterator::String(v) => v, @@ -1781,19 +1791,19 @@ mod tests { assert_eq!(&values, &[None, None, Some("hello")]); let col = columns.get(4).unwrap(); - assert_eq!(col.name().unwrap(), TIME_COLUMN_NAME); + assert_eq!(col.name(), TIME_COLUMN_NAME); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Time); let values = col.values().i64_values().unwrap(); assert_eq!(&values, &[Some(983), Some(2343), Some(222)]); let col = columns.get(5).unwrap(); - assert_eq!(col.name().unwrap(), "val"); + assert_eq!(col.name(), "val"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = col.values().i64_values().unwrap(); assert_eq!(&values, &[Some(23), None, Some(21)]); let col = columns.get(6).unwrap(); - assert_eq!(col.name().unwrap(), "val2"); + assert_eq!(col.name(), "val2"); assert_eq!(col.logical_type(), entry_fb::LogicalColumnType::Field); let values = col.values().f64_values().unwrap(); assert_eq!(&values, &[None, Some(23.2), None]); From 31115742ecc2b6e23593024dc4fe9d9d45bd293e Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sat, 10 Apr 2021 13:38:03 -0400 Subject: [PATCH 06/45] feat: Add writing of Entry structures to MB Chunk This adds writing of Entry of a vec of TableWriteBatch to the Mutable Buffer Chunk. This is additional to the previous method of writing via ReplicatedWrite. The next step is to remove the old ReplicatedWrite bits. Test helpers for parsing line protocol into Entry and writing line protocol directly to Chunks have also been added. --- internal_types/src/entry.rs | 30 +++- mutable_buffer/src/chunk.rs | 155 +++++++++++++++++++- mutable_buffer/src/column.rs | 271 +++++++++++++++++++++++++++++++++++ mutable_buffer/src/table.rs | 250 ++++++++++++++++++++++++++++++++ 4 files changed, 701 insertions(+), 5 deletions(-) diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index acc067d53c..c5789dfcfe 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -430,7 +430,7 @@ impl<'a> TableBatch<'a> { #[derive(Debug)] pub struct Column<'a> { fb: entry_fb::Column<'a>, - row_count: usize, + pub row_count: usize, } impl<'a> Column<'a> { @@ -442,6 +442,18 @@ impl<'a> Column<'a> { self.fb.logical_column_type() } + pub fn is_tag(&self) -> bool { + self.fb.logical_column_type() == entry_fb::LogicalColumnType::Tag + } + + pub fn is_field(&self) -> bool { + self.fb.logical_column_type() == entry_fb::LogicalColumnType::Field + } + + pub fn is_time(&self) -> bool { + self.fb.logical_column_type() == entry_fb::LogicalColumnType::Time + } + pub fn values(&self) -> TypedValuesIterator<'a> { match self.fb.values_type() { entry_fb::ColumnValues::BoolValues => TypedValuesIterator::Bool(BoolIterator { @@ -564,12 +576,22 @@ impl<'a> TypedValuesIterator<'a> { _ => None, } } + + pub fn type_description(&self) -> &str { + match self { + Self::Bool(_) => "bool", + Self::I64(_) => "i64", + Self::F64(_) => "f64", + Self::U64(_) => "u64", + Self::String(_) => "String", + } + } } /// Iterator over the flatbuffers BoolValues #[derive(Debug)] pub struct BoolIterator<'a> { - row_count: usize, + pub row_count: usize, position: usize, null_mask: Option<&'a [u8]>, values: &'a [bool], @@ -599,7 +621,7 @@ impl<'a> Iterator for BoolIterator<'a> { /// Iterator over the flatbuffers I64Values, F64Values, and U64Values. #[derive(Debug)] pub struct ValIterator<'a, T: Follow<'a> + Follow<'a, Inner = T>> { - row_count: usize, + pub row_count: usize, position: usize, null_mask: Option<&'a [u8]>, values_iter: VectorIter<'a, T>, @@ -625,7 +647,7 @@ impl<'a, T: Follow<'a> + Follow<'a, Inner = T>> Iterator for ValIterator<'a, T> /// Iterator over the flatbuffers StringValues #[derive(Debug)] pub struct StringIterator<'a> { - row_count: usize, + pub row_count: usize, position: usize, null_mask: Option<&'a [u8]>, values: VectorIter<'a, ForwardsUOffset<&'a str>>, diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index 14bacacbfe..269c887c11 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -6,7 +6,7 @@ use generated_types::wal as wb; use std::collections::{BTreeSet, HashMap}; use data_types::partition_metadata::TableSummary; -use internal_types::{schema::Schema, selection::Selection}; +use internal_types::{entry::TableBatch, schema::Schema, selection::Selection}; use crate::{ column::Column, @@ -14,6 +14,8 @@ use crate::{ pred::{ChunkPredicate, ChunkPredicateBuilder}, table::Table, }; +use data_types::database_rules::WriterId; +use internal_types::entry::ClockValue; use snafu::{OptionExt, ResultExt, Snafu}; use tracker::{MemRegistry, MemTracker}; @@ -151,6 +153,30 @@ impl Chunk { chunk } + pub fn write_table_batches( + &mut self, + clock_value: ClockValue, + writer_id: WriterId, + batches: &[TableBatch<'_>], + ) -> Result<()> { + for batch in batches { + let table_name = batch.name(); + let table_id = self.dictionary.lookup_value_or_insert(table_name); + + let table = self + .tables + .entry(table_id) + .or_insert_with(|| Table::new(table_id)); + + let columns = batch.columns(); + table + .write_columns(&mut self.dictionary, clock_value, writer_id, columns) + .context(TableWrite { table_name })?; + } + + Ok(()) + } + pub fn write_entry(&mut self, entry: &wb::WriteBufferEntry<'_>) -> Result<()> { if let Some(table_batches) = entry.table_batches() { for batch in table_batches { @@ -486,3 +512,130 @@ impl Chunk { matches!(self.table(table_name), Ok(Some(_))) } } + +pub mod test_helpers { + use super::*; + use internal_types::entry::test_helpers::lp_to_entry; + + /// A helper that will write line protocol string to the passed in Chunk. + /// All data will be under a single partition with a clock value and + /// writer id of 0. + pub fn write_lp_to_chunk(lp: &str, chunk: &mut Chunk) -> Result<()> { + let entry = lp_to_entry(lp); + + for w in entry.partition_writes().unwrap() { + chunk.write_table_batches(0, 0, &w.table_batches())?; + } + + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use super::test_helpers::write_lp_to_chunk; + use super::*; + use arrow_deps::arrow::util::pretty::pretty_format_batches; + + #[test] + fn writes_table_batches() { + let mr = MemRegistry::new(); + let mut chunk = Chunk::new(1, &mr); + + let lp = vec![ + "cpu,host=a val=23 1", + "cpu,host=b val=2 1", + "mem,host=a val=23432i 1", + ] + .join("\n"); + + write_lp_to_chunk(&lp, &mut chunk).unwrap(); + + assert_table( + &chunk, + "cpu", + &[ + "+------+------+-----+", + "| host | time | val |", + "+------+------+-----+", + "| a | 1 | 23 |", + "| b | 1 | 2 |", + "+------+------+-----+\n", + ], + ); + + assert_table( + &chunk, + "mem", + &[ + "+------+------+-------+", + "| host | time | val |", + "+------+------+-------+", + "| a | 1 | 23432 |", + "+------+------+-------+\n", + ], + ); + + let lp = vec![ + "cpu,host=c val=11 1", + "mem sval=\"hi\" 2", + "disk val=true 1", + ] + .join("\n"); + + write_lp_to_chunk(&lp, &mut chunk).unwrap(); + + assert_table( + &chunk, + "cpu", + &[ + "+------+------+-----+", + "| host | time | val |", + "+------+------+-----+", + "| a | 1 | 23 |", + "| b | 1 | 2 |", + "| c | 1 | 11 |", + "+------+------+-----+\n", + ], + ); + + assert_table( + &chunk, + "disk", + &[ + "+------+------+", + "| time | val |", + "+------+------+", + "| 1 | true |", + "+------+------+\n", + ], + ); + + assert_table( + &chunk, + "mem", + &[ + "+------+------+------+-------+", + "| host | sval | time | val |", + "+------+------+------+-------+", + "| a | | 1 | 23432 |", + "| | hi | 2 | |", + "+------+------+------+-------+\n", + ], + ); + } + + fn assert_table(chunk: &Chunk, table: &str, data: &[&str]) { + let mut batches = vec![]; + chunk + .table_to_arrow(&mut batches, table, Selection::All) + .unwrap(); + let res = pretty_format_batches(&batches).unwrap(); + let data = data.join("\n"); + assert_eq!( + res, data, + "\n{} table results not as expected:\nEXPECTED:\n{}\nRECEIVED:\n{}", + table, data, res + ); + } +} diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index 2ba6a89ee9..360a8122fb 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -4,7 +4,9 @@ use snafu::Snafu; use crate::dictionary::Dictionary; use arrow_deps::arrow::datatypes::DataType as ArrowDataType; use data_types::partition_metadata::StatValues; +use generated_types::entry::LogicalColumnType; use internal_types::data::type_description; +use internal_types::entry::TypedValuesIterator; use std::mem; @@ -41,6 +43,275 @@ pub enum Column { } impl Column { + /// Initializes a new column from typed values, the column on a table write + /// batach on an Entry. Will initialize the stats with the first + /// non-null value and update with any other non-null values included. + pub fn new_from_typed_values( + dictionary: &mut Dictionary, + row_count: usize, + logical_type: LogicalColumnType, + values: TypedValuesIterator<'_>, + ) -> Self { + match values { + TypedValuesIterator::String(vals) => match logical_type { + LogicalColumnType::Tag => { + let mut tag_values = vec![None; row_count]; + let mut stats: Option> = None; + + for tag in vals { + let tag_id = match tag { + Some(tag) => { + match stats.as_mut() { + Some(s) => StatValues::update_string(s, tag), + None => { + stats = Some(StatValues::new(tag.to_string())); + } + } + + Some(dictionary.lookup_value_or_insert(tag)) + } + None => None, + }; + + tag_values.push(tag_id); + } + + Self::Tag( + tag_values, + stats.expect("can't insert tag column with no values"), + ) + } + LogicalColumnType::Field => { + let mut values = vec![None; row_count]; + let mut stats: Option> = None; + + for value in vals { + match value { + Some(v) => { + match stats.as_mut() { + Some(s) => StatValues::update_string(s, v), + None => stats = Some(StatValues::new(v.to_string())), + } + + values.push(Some(v.to_string())); + } + None => values.push(None), + } + } + + Self::String( + values, + stats.expect("can't insert string column with no values"), + ) + } + _ => panic!("unsupported!"), + }, + TypedValuesIterator::I64(vals) => { + let mut values = vec![None; row_count]; + let mut stats: Option> = None; + + for v in vals { + if let Some(val) = v { + match stats.as_mut() { + Some(s) => s.update(val), + None => stats = Some(StatValues::new(val)), + } + } + values.push(v); + } + + Self::I64( + values, + stats.expect("can't insert i64 column with no values"), + ) + } + TypedValuesIterator::F64(vals) => { + let mut values = vec![None; row_count]; + let mut stats: Option> = None; + + for v in vals { + if let Some(val) = v { + match stats.as_mut() { + Some(s) => s.update(val), + None => stats = Some(StatValues::new(val)), + } + } + values.push(v); + } + + Self::F64( + values, + stats.expect("can't insert f64 column with no values"), + ) + } + TypedValuesIterator::U64(vals) => { + let mut values = vec![None; row_count]; + let mut stats: Option> = None; + + for v in vals { + if let Some(val) = v { + match stats.as_mut() { + Some(s) => s.update(val), + None => stats = Some(StatValues::new(val)), + } + } + values.push(v); + } + + Self::U64( + values, + stats.expect("can't insert u64 column with no values"), + ) + } + TypedValuesIterator::Bool(vals) => { + let mut values = vec![None; row_count]; + let mut stats: Option> = None; + + for v in vals { + if let Some(val) = v { + match stats.as_mut() { + Some(s) => s.update(val), + None => stats = Some(StatValues::new(val)), + } + } + values.push(v); + } + + Self::Bool( + values, + stats.expect("can't insert bool column with no values"), + ) + } + } + } + + /// Pushes typed values, the column from a table write batch on an Entry. + /// Updates statsistics for any non-null values. + pub fn push_typed_values( + &mut self, + dictionary: &mut Dictionary, + logical_type: LogicalColumnType, + values: TypedValuesIterator<'_>, + ) -> Result<()> { + match (self, values) { + (Self::Bool(col, stats), TypedValuesIterator::Bool(values)) => { + for val in values { + if let Some(v) = val { + stats.update(v) + }; + col.push(val); + } + } + (Self::I64(col, stats), TypedValuesIterator::I64(values)) => { + for val in values { + if let Some(v) = val { + stats.update(v) + }; + col.push(val); + } + } + (Self::F64(col, stats), TypedValuesIterator::F64(values)) => { + for val in values { + if let Some(v) = val { + stats.update(v) + }; + col.push(val); + } + } + (Self::U64(col, stats), TypedValuesIterator::U64(values)) => { + for val in values { + if let Some(v) = val { + stats.update(v) + }; + col.push(val); + } + } + (Self::String(col, stats), TypedValuesIterator::String(values)) => { + if logical_type != LogicalColumnType::Field { + TypeMismatch { + existing_column_type: "String", + inserted_value_type: "tag", + } + .fail()?; + } + + for val in values { + match val { + Some(v) => { + StatValues::update_string(stats, v); + col.push(Some(v.to_string())); + } + None => col.push(None), + } + } + } + (Self::Tag(col, stats), TypedValuesIterator::String(values)) => { + if logical_type != LogicalColumnType::Tag { + TypeMismatch { + existing_column_type: "tag", + inserted_value_type: "String", + } + .fail()?; + } + + for val in values { + match val { + Some(v) => { + StatValues::update_string(stats, v); + let id = dictionary.lookup_value_or_insert(v); + col.push(Some(id)); + } + None => col.push(None), + } + } + } + (existing, values) => TypeMismatch { + existing_column_type: existing.type_description(), + inserted_value_type: values.type_description(), + } + .fail()?, + } + + Ok(()) + } + + /// Pushes None values onto the column until its len is equal to that passed + /// in + pub fn push_nulls_to_len(&mut self, len: usize) { + match self { + Self::Tag(vals, _) => { + while vals.len() < len { + vals.push(None); + } + } + Self::I64(vals, _) => { + while vals.len() < len { + vals.push(None); + } + } + Self::F64(vals, _) => { + while vals.len() < len { + vals.push(None); + } + } + Self::U64(vals, _) => { + while vals.len() < len { + vals.push(None); + } + } + Self::Bool(vals, _) => { + while vals.len() < len { + vals.push(None); + } + } + Self::String(vals, _) => { + while vals.len() < len { + vals.push(None); + } + } + } + } + pub fn with_value( dictionary: &mut Dictionary, capacity: usize, diff --git a/mutable_buffer/src/table.rs b/mutable_buffer/src/table.rs index fe35b2f1f8..8d2e9f837d 100644 --- a/mutable_buffer/src/table.rs +++ b/mutable_buffer/src/table.rs @@ -14,6 +14,7 @@ use crate::{ }; use data_types::partition_metadata::{ColumnSummary, Statistics}; use internal_types::{ + entry, schema::{builder::SchemaBuilder, Schema, TIME_COLUMN_NAME}, selection::Selection, }; @@ -30,6 +31,8 @@ use arrow_deps::{ record_batch::RecordBatch, }, }; +use data_types::database_rules::WriterId; +use internal_types::entry::ClockValue; #[derive(Debug, Snafu)] pub enum Error { @@ -237,6 +240,106 @@ impl Table { Ok(()) } + /// Validates the schema of the passed in columns, then adds their values to + /// the associated columns in the table and updates summary statistics. + pub fn write_columns( + &mut self, + dictionary: &mut Dictionary, + _clock_value: ClockValue, + _writer_id: WriterId, + columns: Vec>, + ) -> Result<()> { + // get the column ids and validate schema for those that already exist + let columns_with_inserts = columns + .into_iter() + .map(|insert_column| { + let column_id = dictionary.lookup_value_or_insert(insert_column.name()); + let values = insert_column.values(); + + if let Some(c) = self.columns.get(&column_id) { + match (&values, c) { + (entry::TypedValuesIterator::Bool(_), Column::Bool(_, _)) => (), + (entry::TypedValuesIterator::U64(_), Column::U64(_, _)) => (), + (entry::TypedValuesIterator::F64(_), Column::F64(_, _)) => (), + (entry::TypedValuesIterator::I64(_), Column::I64(_, _)) => (), + (entry::TypedValuesIterator::String(_), Column::String(_, _)) => { + if !insert_column.is_field() { + InternalColumnTypeMismatch { + column_id, + expected_column_type: c.type_description(), + actual_column_type: values.type_description(), + } + .fail()? + }; + } + (entry::TypedValuesIterator::String(_), Column::Tag(_, _)) => { + if !insert_column.is_tag() { + InternalColumnTypeMismatch { + column_id, + expected_column_type: c.type_description(), + actual_column_type: values.type_description(), + } + .fail()? + }; + } + _ => InternalColumnTypeMismatch { + column_id, + expected_column_type: c.type_description(), + actual_column_type: values.type_description(), + } + .fail()?, + } + } + + Ok((column_id, insert_column.logical_type(), values)) + }) + .collect::>>()?; + + let row_count_before_insert = self.row_count(); + + for (column_id, logical_type, values) in columns_with_inserts.into_iter() { + match self.columns.get_mut(&column_id) { + Some(c) => c + .push_typed_values(dictionary, logical_type, values) + .with_context(|| { + let column = dictionary.lookup_id(column_id).unwrap_or("unknown"); + ColumnError { column } + })?, + None => { + self.columns.insert( + column_id, + Column::new_from_typed_values( + dictionary, + row_count_before_insert, + logical_type, + values, + ), + ); + } + } + } + + // ensure all columns have the same number of rows as the one with the most. + // This adds nulls to the columns that weren't included in this write + let max_row_count = self + .columns + .values() + .fold(row_count_before_insert, |max, col| { + let len = col.len(); + if max < len { + len + } else { + max + } + }); + + for c in self.columns.values_mut() { + c.push_nulls_to_len(max_row_count); + } + + Ok(()) + } + /// Returns the column selection for all the columns in this table, orderd /// by table name fn all_columns_selection<'a>(&self, chunk: &'a Chunk) -> Result> { @@ -348,6 +451,7 @@ impl Table { for col in &selection.cols { let column = self.column(col.column_id)?; + println!("COLUMN: {:#?}", column); let array = match column { Column::String(vals, _) => { @@ -594,6 +698,7 @@ mod tests { use influxdb_line_protocol::{parse_lines, ParsedLine}; use internal_types::data::split_lines_into_write_entry_partitions; + use internal_types::entry::test_helpers::lp_to_entry; use super::*; use tracker::MemRegistry; @@ -802,6 +907,151 @@ mod tests { ); } + #[test] + fn write_columns_validates_schema() { + let mut dictionary = Dictionary::new(); + let mut table = Table::new(dictionary.lookup_value_or_insert("foo")); + + let lp = "foo,t1=asdf iv=1i,uv=1u,fv=1.0,bv=true,sv=\"hi\" 1"; + let entry = lp_to_entry(&lp); + table + .write_columns( + &mut dictionary, + 0, + 0, + entry + .partition_writes() + .unwrap() + .first() + .unwrap() + .table_batches() + .first() + .unwrap() + .columns(), + ) + .unwrap(); + + let lp = "foo t1=\"string\" 1"; + let entry = lp_to_entry(&lp); + let response = table + .write_columns( + &mut dictionary, + 0, + 0, + entry + .partition_writes() + .unwrap() + .first() + .unwrap() + .table_batches() + .first() + .unwrap() + .columns(), + ) + .err() + .unwrap(); + assert!( + matches!(&response, Error::InternalColumnTypeMismatch {expected_column_type, actual_column_type, ..} if expected_column_type == "tag" && actual_column_type == "String"), + format!("didn't match returned error: {:?}", response) + ); + + let lp = "foo iv=1u 1"; + let entry = lp_to_entry(&lp); + let response = table + .write_columns( + &mut dictionary, + 0, + 0, + entry + .partition_writes() + .unwrap() + .first() + .unwrap() + .table_batches() + .first() + .unwrap() + .columns(), + ) + .err() + .unwrap(); + assert!( + matches!(&response, Error::InternalColumnTypeMismatch {expected_column_type, actual_column_type, ..} if expected_column_type == "i64" && actual_column_type == "u64"), + format!("didn't match returned error: {:?}", response) + ); + + let lp = "foo fv=1i 1"; + let entry = lp_to_entry(&lp); + let response = table + .write_columns( + &mut dictionary, + 0, + 0, + entry + .partition_writes() + .unwrap() + .first() + .unwrap() + .table_batches() + .first() + .unwrap() + .columns(), + ) + .err() + .unwrap(); + assert!( + matches!(&response, Error::InternalColumnTypeMismatch {expected_column_type, actual_column_type, ..} if expected_column_type == "f64" && actual_column_type == "i64"), + format!("didn't match returned error: {:?}", response) + ); + + let lp = "foo bv=1 1"; + let entry = lp_to_entry(&lp); + let response = table + .write_columns( + &mut dictionary, + 0, + 0, + entry + .partition_writes() + .unwrap() + .first() + .unwrap() + .table_batches() + .first() + .unwrap() + .columns(), + ) + .err() + .unwrap(); + assert!( + matches!(&response, Error::InternalColumnTypeMismatch {expected_column_type, actual_column_type, ..} if expected_column_type == "bool" && actual_column_type == "f64"), + format!("didn't match returned error: {:?}", response) + ); + + let lp = "foo sv=true 1"; + let entry = lp_to_entry(&lp); + let response = table + .write_columns( + &mut dictionary, + 0, + 0, + entry + .partition_writes() + .unwrap() + .first() + .unwrap() + .table_batches() + .first() + .unwrap() + .columns(), + ) + .err() + .unwrap(); + assert!( + matches!(&response, Error::InternalColumnTypeMismatch {expected_column_type, actual_column_type, ..} if expected_column_type == "String" && actual_column_type == "bool"), + format!("didn't match returned error: {:?}", response) + ); + } + /// Insert the line protocol lines in `lp_lines` into this table fn write_lines_to_table(table: &mut Table, dictionary: &mut Dictionary, lp_lines: Vec<&str>) { let lp_data = lp_lines.join("\n"); From 5893c17905789f255ecd8d49c4fab46ea35fc24e Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 12 Apr 2021 13:49:23 -0400 Subject: [PATCH 07/45] refactor: PR feedback and change ClockValue to actual type. --- internal_types/benches/benchmark.rs | 9 +++-- internal_types/src/entry.rs | 51 +++++++++++++++++------------ mutable_buffer/src/chunk.rs | 2 +- mutable_buffer/src/column.rs | 20 +++++------ mutable_buffer/src/table.rs | 27 +++++++-------- 5 files changed, 56 insertions(+), 53 deletions(-) diff --git a/internal_types/benches/benchmark.rs b/internal_types/benches/benchmark.rs index bfc9ad462c..85c2d89d32 100644 --- a/internal_types/benches/benchmark.rs +++ b/internal_types/benches/benchmark.rs @@ -2,7 +2,7 @@ use chrono::{DateTime, Utc}; use criterion::{criterion_group, criterion_main, Criterion}; use data_types::database_rules::{Error as DataError, Partitioner, Sharder}; use influxdb_line_protocol::ParsedLine; -use internal_types::entry::{lines_to_sharded_entries, SequencedEntry}; +use internal_types::entry::{lines_to_sharded_entries, ClockValue, SequencedEntry}; static LINES: &str = include_str!("../../tests/fixtures/lineproto/prometheus.lp"); @@ -28,10 +28,13 @@ fn sequenced_entry(c: &mut Criterion) { 554 ); + let clock_value = ClockValue::new(23); + group.bench_function("new_from_entry_bytes", |b| { b.iter(|| { - let sequenced_entry = SequencedEntry::new_from_entry_bytes(23, 2, data).unwrap(); - assert_eq!(sequenced_entry.clock_value(), 23); + let sequenced_entry = + SequencedEntry::new_from_entry_bytes(clock_value, 2, data).unwrap(); + assert_eq!(sequenced_entry.clock_value(), clock_value); assert_eq!(sequenced_entry.writer_id(), 2); }) }); diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index c5789dfcfe..70b51d9d71 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -52,16 +52,6 @@ pub enum ColumnError { pub type Result = std::result::Result; type ColumnResult = std::result::Result; -// The flatbuffers structs for partition write, table write batch, and column -// all have string values for their key/name. These values should always be -// present. They are if the flatbuffer was generated by code here. However, -// those are all returned as Option<&str> from the flatbuffer. So to keep us -// from having option unwraps all over the place when dealing with this, we'll -// use this default value to return in cases where the partition key, table -// name, or column name are not there. If you see this value anywhere, you know -// there's some illigitimate flatbuffer data somewhere. -const UNKNOWN_STRING_VALUE: &str = "unknown_from_flatbuffer"; - /// Converts parsed line protocol into a collection of ShardedEntry with the /// underlying flatbuffers bytes generated. pub fn lines_to_sharded_entries( @@ -348,7 +338,9 @@ pub struct PartitionWrite<'a> { impl<'a> PartitionWrite<'a> { pub fn key(&self) -> &str { - self.fb.key().unwrap_or(UNKNOWN_STRING_VALUE) + self.fb + .key() + .expect("key must be present in the flatbuffer PartitionWrite") } pub fn table_batches(&self) -> Vec> { @@ -371,7 +363,9 @@ pub struct TableBatch<'a> { impl<'a> TableBatch<'a> { pub fn name(&self) -> &str { - self.fb.name().unwrap_or(UNKNOWN_STRING_VALUE) + self.fb + .name() + .expect("name must be present in flatbuffers TableWriteBatch") } pub fn columns(&self) -> Vec> { @@ -435,7 +429,9 @@ pub struct Column<'a> { impl<'a> Column<'a> { pub fn name(&self) -> &str { - self.fb.name().unwrap_or(UNKNOWN_STRING_VALUE) + self.fb + .name() + .expect("name must be present in flatbuffers Column") } pub fn logical_type(&self) -> entry_fb::LogicalColumnType { @@ -1119,7 +1115,18 @@ enum ColumnRaw<'a> { Bool(Vec), } -pub type ClockValue = u64; +#[derive(Debug, PartialOrd, PartialEq, Copy, Clone)] +pub struct ClockValue(u64); + +impl ClockValue { + pub fn get(&self) -> u64 { + self.0 + } + + pub fn new(v: u64) -> Self { + Self { 0: v } + } +} #[self_referencing] #[derive(Debug)] @@ -1135,7 +1142,7 @@ pub struct SequencedEntry { impl SequencedEntry { pub fn new_from_entry_bytes( - clock_value: u64, + clock_value: ClockValue, writer_id: u32, entry_bytes: &[u8], ) -> Result { @@ -1152,7 +1159,7 @@ impl SequencedEntry { let sequenced_entry = entry_fb::SequencedEntry::create( &mut fbb, &entry_fb::SequencedEntryArgs { - clock_value, + clock_value: clock_value.get(), writer_id, entry_bytes: Some(entry_bytes), }, @@ -1186,7 +1193,7 @@ impl SequencedEntry { } pub fn clock_value(&self) -> ClockValue { - self.fb().clock_value() + ClockValue::new(self.fb().clock_value()) } pub fn writer_id(&self) -> WriterId { @@ -1255,7 +1262,7 @@ pub mod test_helpers { } /// Returns a test partitioner that will assign partition keys in the form - /// key_# where # is replaced by a number [0, count) incrementing for + /// key_# where # is replaced by a number `[0, count)` incrementing for /// each line. pub fn partitioner(count: u8) -> TestPartitioner { TestPartitioner { @@ -1265,7 +1272,7 @@ pub mod test_helpers { } // For each line passed to partition_key returns a key with a number from - // [0, count) + // `[0, count)` #[derive(Debug)] pub struct TestPartitioner { count: u8, @@ -1763,8 +1770,10 @@ mod tests { lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); let entry_bytes = sharded_entries.first().unwrap().entry.data(); - let sequenced_entry = SequencedEntry::new_from_entry_bytes(23, 2, entry_bytes).unwrap(); - assert_eq!(sequenced_entry.clock_value(), 23); + let clock_value = ClockValue::new(23); + let sequenced_entry = + SequencedEntry::new_from_entry_bytes(clock_value, 2, entry_bytes).unwrap(); + assert_eq!(sequenced_entry.clock_value(), clock_value); assert_eq!(sequenced_entry.writer_id(), 2); let partition_writes = sequenced_entry.partition_writes().unwrap(); diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index 269c887c11..672666edfd 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -524,7 +524,7 @@ pub mod test_helpers { let entry = lp_to_entry(lp); for w in entry.partition_writes().unwrap() { - chunk.write_table_batches(0, 0, &w.table_batches())?; + chunk.write_table_batches(ClockValue::new(0), 0, &w.table_batches())?; } Ok(()) diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index 360a8122fb..151d1aec3f 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -44,7 +44,7 @@ pub enum Column { impl Column { /// Initializes a new column from typed values, the column on a table write - /// batach on an Entry. Will initialize the stats with the first + /// batch on an Entry. Will initialize the stats with the first /// non-null value and update with any other non-null values included. pub fn new_from_typed_values( dictionary: &mut Dictionary, @@ -55,12 +55,11 @@ impl Column { match values { TypedValuesIterator::String(vals) => match logical_type { LogicalColumnType::Tag => { - let mut tag_values = vec![None; row_count]; let mut stats: Option> = None; - for tag in vals { - let tag_id = match tag { - Some(tag) => { + let tag_values: Vec<_> = vals + .map(|tag| { + tag.map(|tag| { match stats.as_mut() { Some(s) => StatValues::update_string(s, tag), None => { @@ -68,13 +67,10 @@ impl Column { } } - Some(dictionary.lookup_value_or_insert(tag)) - } - None => None, - }; - - tag_values.push(tag_id); - } + dictionary.lookup_value_or_insert(tag) + }) + }) + .collect(); Self::Tag( tag_values, diff --git a/mutable_buffer/src/table.rs b/mutable_buffer/src/table.rs index 8d2e9f837d..e5dcda7d64 100644 --- a/mutable_buffer/src/table.rs +++ b/mutable_buffer/src/table.rs @@ -1,6 +1,7 @@ use generated_types::wal as wb; use std::{ + cmp, collections::{BTreeMap, BTreeSet}, sync::Arc, }; @@ -302,7 +303,9 @@ impl Table { Some(c) => c .push_typed_values(dictionary, logical_type, values) .with_context(|| { - let column = dictionary.lookup_id(column_id).unwrap_or("unknown"); + let column = dictionary + .lookup_id(column_id) + .expect("column name must be present in dictionary"); ColumnError { column } })?, None => { @@ -324,14 +327,7 @@ impl Table { let max_row_count = self .columns .values() - .fold(row_count_before_insert, |max, col| { - let len = col.len(); - if max < len { - len - } else { - max - } - }); + .fold(row_count_before_insert, |max, col| cmp::max(max, col.len())); for c in self.columns.values_mut() { c.push_nulls_to_len(max_row_count); @@ -451,7 +447,6 @@ impl Table { for col in &selection.cols { let column = self.column(col.column_id)?; - println!("COLUMN: {:#?}", column); let array = match column { Column::String(vals, _) => { @@ -917,7 +912,7 @@ mod tests { table .write_columns( &mut dictionary, - 0, + ClockValue::new(0), 0, entry .partition_writes() @@ -936,7 +931,7 @@ mod tests { let response = table .write_columns( &mut dictionary, - 0, + ClockValue::new(0), 0, entry .partition_writes() @@ -960,7 +955,7 @@ mod tests { let response = table .write_columns( &mut dictionary, - 0, + ClockValue::new(0), 0, entry .partition_writes() @@ -984,7 +979,7 @@ mod tests { let response = table .write_columns( &mut dictionary, - 0, + ClockValue::new(0), 0, entry .partition_writes() @@ -1008,7 +1003,7 @@ mod tests { let response = table .write_columns( &mut dictionary, - 0, + ClockValue::new(0), 0, entry .partition_writes() @@ -1032,7 +1027,7 @@ mod tests { let response = table .write_columns( &mut dictionary, - 0, + ClockValue::new(0), 0, entry .partition_writes() From d67739c8735cc6aa3622c19affabf99a4bb571e6 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 12 Apr 2021 14:32:21 -0400 Subject: [PATCH 08/45] chore: updates for pr feedback --- mutable_buffer/src/column.rs | 24 ++++++++++++------------ mutable_buffer/src/table.rs | 8 +++++++- 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index 151d1aec3f..b832e452df 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -276,33 +276,33 @@ impl Column { pub fn push_nulls_to_len(&mut self, len: usize) { match self { Self::Tag(vals, _) => { - while vals.len() < len { - vals.push(None); + if len > vals.len() { + vals.resize(len, None); } } Self::I64(vals, _) => { - while vals.len() < len { - vals.push(None); + if len > vals.len() { + vals.resize(len, None); } } Self::F64(vals, _) => { - while vals.len() < len { - vals.push(None); + if len > vals.len() { + vals.resize(len, None); } } Self::U64(vals, _) => { - while vals.len() < len { - vals.push(None); + if len > vals.len() { + vals.resize(len, None); } } Self::Bool(vals, _) => { - while vals.len() < len { - vals.push(None); + if len > vals.len() { + vals.resize(len, None); } } Self::String(vals, _) => { - while vals.len() < len { - vals.push(None); + if len > vals.len() { + vals.resize(len, None); } } } diff --git a/mutable_buffer/src/table.rs b/mutable_buffer/src/table.rs index e5dcda7d64..9e54e16aeb 100644 --- a/mutable_buffer/src/table.rs +++ b/mutable_buffer/src/table.rs @@ -946,7 +946,13 @@ mod tests { .err() .unwrap(); assert!( - matches!(&response, Error::InternalColumnTypeMismatch {expected_column_type, actual_column_type, ..} if expected_column_type == "tag" && actual_column_type == "String"), + matches!( + &response, + Error::InternalColumnTypeMismatch { + expected_column_type, + actual_column_type, + .. + } if expected_column_type == "tag" && actual_column_type == "String"), format!("didn't match returned error: {:?}", response) ); From 494480bc57ef22cc0326ec348088901ba4e25dd8 Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Mon, 12 Apr 2021 15:37:08 -0400 Subject: [PATCH 09/45] fix: check filename to have deterministic output --- server/src/db.rs | 40 ++++++++++++++++++++-------------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/server/src/db.rs b/server/src/db.rs index bad1875e1c..b4f5ca38f2 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -1078,26 +1078,24 @@ mod tests { .unwrap(); println!("path_list: {:#?}", path_list); assert_eq!(path_list.len(), 2); - assert_eq!(path_list, paths.clone()); // Check the content of each path - + // // Root path let root_path = format!("{:?}", root.path()); let root_path = root_path.trim_matches('"'); - let mut i = 0; - while i < 2 { + for path in path_list { // Get full string path - let path = format!("{}/{}", root_path, paths[i].display()); - println!("path: {}", path); + let path_string = format!("{}/{}", root_path, path.display()); + println!("path: {}", path_string); // Create External table of this parquet file to get its content in a human // readable form // Note: We do not care about escaping quotes here because it is just a test let sql = format!( "CREATE EXTERNAL TABLE parquet_table STORED AS PARQUET LOCATION '{}'", - path + path_string ); let mut ctx = context::ExecutionContext::new(); @@ -1108,26 +1106,28 @@ mod tests { let sql = "SELECT * FROM parquet_table"; let content = ctx.sql(&sql).unwrap().collect().await.unwrap(); println!("Content: {:?}", content); - let mut expected = vec![ - "+-----+------+", - "| bar | time |", - "+-----+------+", - "| 1 | 10 |", - "| 2 | 20 |", - "+-----+------+", - ]; - if i == 1 { - expected = vec![ + let expected = if path_string.contains("cpu") { + // file name: cpu.parquet + vec![ + "+-----+------+", + "| bar | time |", + "+-----+------+", + "| 1 | 10 |", + "| 2 | 20 |", + "+-----+------+", + ] + } else { + // file name: disk.parquet + vec![ "+-----+------+", "| ops | time |", "+-----+------+", "| 1 | 20 |", "+-----+------+", - ]; - } + ] + }; assert_table_eq!(expected, &content); - i += 1; } } From 0f53ba8d9a3351d3fc96b707df77e086f5b4f26c Mon Sep 17 00:00:00 2001 From: Edd Robinson Date: Thu, 8 Apr 2021 22:16:29 +0100 Subject: [PATCH 10/45] docs: improve flatc install --- docs/regenerating_flatbuffers.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/regenerating_flatbuffers.md b/docs/regenerating_flatbuffers.md index 1c5f8a07f5..f01bdede18 100644 --- a/docs/regenerating_flatbuffers.md +++ b/docs/regenerating_flatbuffers.md @@ -5,3 +5,6 @@ When updating the version of the [flatbuffers](https://crates.io/crates/flatbuff To update the generated code, edit `generated_types/regenerate-flatbuffers.sh` and set the `FB_COMMIT` variable at the top of the file to the commit SHA of the same commit in the [flatbuffers repository](https://github.com/google/flatbuffers) where the `flatbuffers` Rust crate version was updated. This ensures we'll be [using the same version of `flatc` that the crate was tested with](https://github.com/google/flatbuffers/issues/6199#issuecomment-714562121). Then run the `generated_types/regenerate-flatbuffers.sh` script and check in any changes. Check the whole project builds. + +`generated_types/regenerate-flatbuffers.sh` will build `flatc` from source if it cannot be found. +In order to do that your system will require `bazel`; you can likely install this with your favourite package manager. From 37b7bbc069c2e824e2eab2d5b1dab1b824c82ebf Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 9 Apr 2021 14:03:33 -0400 Subject: [PATCH 11/45] docs: Improve instructions for usage of the regenerate flatbuffers script --- generated_types/regenerate-flatbuffers.sh | 25 +++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/generated_types/regenerate-flatbuffers.sh b/generated_types/regenerate-flatbuffers.sh index 56f0db2ccd..da61ba4e56 100755 --- a/generated_types/regenerate-flatbuffers.sh +++ b/generated_types/regenerate-flatbuffers.sh @@ -1,11 +1,28 @@ #!/bin/bash -e -# The commit where the Rust `flatbuffers` crate version was changed to the version in `Cargo.lock` -# Update this, rerun this script, and check in the changes in the generated code when the -# `flatbuffers` crate version is updated. +# Instructions +# +# If you have changed some `*.fbs` files: +# +# - Run this script to regenerate the corresponding Rust code. +# - Run `cargo test` to make sure everything works as you would expect. +# - Check in the changes to the generated code along with your changes to the `*.fbs` files. +# - You should not need to edit this script. +# +# If you are updating the version of the `flatbuffers` crate in `Cargo.lock`: +# +# - The `flatbuffers` crate gets developed in sync with the `flatc` compiler in the same repo, +# so when updating the `flatbuffers` crate we also need to update the `flatc` compiler we're +# using. +# - Go to https://github.com/google/flatbuffers/blame/master/rust/flatbuffers/Cargo.toml and find +# the commit SHA where the `version` metadata was updated to the version of the `flatbuffers` +# crate we now want to have in our `Cargo.lock`. +# - Put that commit SHA in this variable: FB_COMMIT="86401e078d0746d2381735415f8c2dfe849f3f52" +# - Run this script to regenerate the corresponding Rust code. +# - Run `cargo test` to make sure everything works as you would expect. +# - Check in the changes to the generated code along with your changes to the `*.fbs` files. -# Change to the generated_types crate directory, where this script is located DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" pushd $DIR From eff6eb6b35844dc16a8679657cea50432d6a87d0 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 9 Apr 2021 14:15:34 -0400 Subject: [PATCH 12/45] docs: Fix instructions on what files are expected to change --- generated_types/regenerate-flatbuffers.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/generated_types/regenerate-flatbuffers.sh b/generated_types/regenerate-flatbuffers.sh index da61ba4e56..18031b19e3 100755 --- a/generated_types/regenerate-flatbuffers.sh +++ b/generated_types/regenerate-flatbuffers.sh @@ -21,7 +21,8 @@ FB_COMMIT="86401e078d0746d2381735415f8c2dfe849f3f52" # - Run this script to regenerate the corresponding Rust code. # - Run `cargo test` to make sure everything works as you would expect. -# - Check in the changes to the generated code along with your changes to the `*.fbs` files. +# - Check in the changes to the generated code along with your changes to the `Cargo.lock` file and +# this script. DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" pushd $DIR From 1997324344c5a9ef2e0c1df74582f454bfbdaa0f Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Tue, 13 Apr 2021 13:14:54 +0100 Subject: [PATCH 13/45] feat: mutable buffer snapshotting (#1179) * feat: mutable buffer snapshotting * chore: review feedback --- Cargo.toml | 2 +- data_types/src/timestamp.rs | 21 ++++ generated_types/Cargo.toml | 2 +- influxdb2_client/Cargo.toml | 2 +- internal_types/src/schema.rs | 99 ++++++++++++++- mutable_buffer/src/chunk.rs | 48 ++++--- mutable_buffer/src/chunk/snapshot.rs | 181 +++++++++++++++++++++++++++ mutable_buffer/src/column.rs | 7 ++ object_store/Cargo.toml | 2 +- parquet_file/Cargo.toml | 2 +- query/Cargo.toml | 2 +- server/Cargo.toml | 2 +- server/src/db/chunk.rs | 157 ++++++----------------- server/src/db/streams.rs | 40 ++++++ server/src/snapshot.rs | 4 +- tracker/Cargo.toml | 2 +- wal/Cargo.toml | 2 +- 17 files changed, 416 insertions(+), 159 deletions(-) create mode 100644 mutable_buffer/src/chunk/snapshot.rs diff --git a/Cargo.toml b/Cargo.toml index bee78699c3..d2056a3556 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -71,7 +71,7 @@ csv = "1.1" dirs = "3.0.1" dotenv = "0.15.0" flate2 = "1.0" -futures = "0.3.1" +futures = "0.3" http = "0.2.0" hyper = "0.14" once_cell = { version = "1.4.0", features = ["parking_lot"] } diff --git a/data_types/src/timestamp.rs b/data_types/src/timestamp.rs index 778e816144..b79101cfa8 100644 --- a/data_types/src/timestamp.rs +++ b/data_types/src/timestamp.rs @@ -12,6 +12,7 @@ pub struct TimestampRange { impl TimestampRange { pub fn new(start: i64, end: i64) -> Self { + debug_assert!(end > start); Self { start, end } } @@ -26,6 +27,12 @@ impl TimestampRange { pub fn contains_opt(&self, v: Option) -> bool { Some(true) == v.map(|ts| self.contains(ts)) } + + #[inline] + /// Returns if this range is disjoint w.r.t the provided range + pub fn disjoint(&self, other: &Self) -> bool { + self.end <= other.start || self.start >= other.end + } } #[cfg(test)] @@ -55,4 +62,18 @@ mod tests { assert!(!range.contains_opt(None)); } + + #[test] + fn test_disjoint() { + let r1 = TimestampRange::new(100, 200); + let r2 = TimestampRange::new(200, 300); + let r3 = TimestampRange::new(150, 250); + + assert!(r1.disjoint(&r2)); + assert!(r2.disjoint(&r1)); + assert!(!r1.disjoint(&r3)); + assert!(!r3.disjoint(&r1)); + assert!(!r2.disjoint(&r3)); + assert!(!r3.disjoint(&r2)); + } } diff --git a/generated_types/Cargo.toml b/generated_types/Cargo.toml index a8ee50d03f..5fb4c7395e 100644 --- a/generated_types/Cargo.toml +++ b/generated_types/Cargo.toml @@ -9,7 +9,7 @@ bytes = { version = "1.0", features = ["serde"] } # See docs/regenerating_flatbuffers.md about updating generated code when updating the # version of the flatbuffers crate flatbuffers = "0.8" -futures = "0.3.1" +futures = "0.3" prost = "0.7" prost-types = "0.7" tonic = "0.4" diff --git a/influxdb2_client/Cargo.toml b/influxdb2_client/Cargo.toml index ef7148fa3e..be1bf22965 100644 --- a/influxdb2_client/Cargo.toml +++ b/influxdb2_client/Cargo.toml @@ -6,7 +6,7 @@ edition = "2018" [dependencies] # In alphabetical order bytes = { version = "1.0", default-features = false } -futures = { version = "0.3.5", default-features = false } +futures = { version = "0.3", default-features = false } reqwest = { version = "0.11", features = ["stream", "json"] } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0.44" diff --git a/internal_types/src/schema.rs b/internal_types/src/schema.rs index c22979bb11..dec2ed51f3 100644 --- a/internal_types/src/schema.rs +++ b/internal_types/src/schema.rs @@ -94,11 +94,8 @@ pub enum Error { source: arrow_deps::arrow::error::ArrowError, }, - #[snafu(display("Schema Selection error while selecting '{}': {}", column_name, source))] - SelectingColumns { - column_name: String, - source: arrow_deps::arrow::error::ArrowError, - }, + #[snafu(display("Column not found '{}'", column_name))] + ColumnNotFound { column_name: String }, } fn nullable_to_str(nullability: bool) -> &'static str { @@ -470,6 +467,44 @@ impl Schema { } } } + + /// Returns the field indexes for a given selection + /// + /// Returns an error if a corresponding column isn't found + pub fn select(&self, columns: &[&str]) -> Result> { + columns + .iter() + .map(|column_name| { + self.find_index_of(column_name) + .ok_or_else(|| Error::ColumnNotFound { + column_name: column_name.to_string(), + }) + }) + .collect() + } + + /// Returns the schema for a given set of column projects + pub fn project(&self, projection: &[usize]) -> Self { + let mut metadata = HashMap::with_capacity(projection.len() + 1); + let mut fields = Vec::with_capacity(projection.len()); + let current_metadata = self.inner.metadata(); + for idx in projection { + let (_, field) = self.field(*idx); + fields.push(field.clone()); + + if let Some(value) = current_metadata.get(field.name()) { + metadata.insert(field.name().clone(), value.clone()); + } + } + + if let Some(measurement) = current_metadata.get(MEASUREMENT_METADATA_KEY).cloned() { + metadata.insert(MEASUREMENT_METADATA_KEY.to_string(), measurement); + } + + Self { + inner: Arc::new(ArrowSchema::new_with_metadata(fields, metadata)), + } + } } /// Valid types for InfluxDB data model, as defined in [the documentation] @@ -1180,4 +1215,58 @@ mod test { expected_schema, sorted_schema ); } + + #[test] + fn test_select() { + let schema1 = SchemaBuilder::new() + .influx_field("the_field", String) + .tag("the_tag") + .timestamp() + .measurement("the_measurement") + .build() + .unwrap(); + + let projection = schema1.select(&[TIME_COLUMN_NAME]).unwrap(); + + let schema2 = schema1.project(&projection); + let schema3 = Schema::try_from_arrow(Arc::clone(&schema2.inner)).unwrap(); + + assert_eq!(schema1.measurement(), schema2.measurement()); + assert_eq!(schema1.measurement(), schema3.measurement()); + + assert_eq!(schema1.len(), 3); + assert_eq!(schema2.len(), 1); + assert_eq!(schema3.len(), 1); + + assert_eq!(schema1.inner.fields().len(), 3); + assert_eq!(schema2.inner.fields().len(), 1); + assert_eq!(schema3.inner.fields().len(), 1); + + let get_type = |x: &Schema, field: &str| -> InfluxColumnType { + let idx = x.find_index_of(field).unwrap(); + x.field(idx).0.unwrap() + }; + + assert_eq!( + get_type(&schema1, TIME_COLUMN_NAME), + InfluxColumnType::Timestamp + ); + assert_eq!( + get_type(&schema2, TIME_COLUMN_NAME), + InfluxColumnType::Timestamp + ); + assert_eq!(get_type(&schema1, "the_tag"), InfluxColumnType::Tag); + assert_eq!( + get_type(&schema1, "the_field"), + InfluxColumnType::Field(InfluxFieldType::String) + ); + assert_eq!( + get_type(&schema2, TIME_COLUMN_NAME), + InfluxColumnType::Timestamp + ); + assert_eq!( + get_type(&schema3, TIME_COLUMN_NAME), + InfluxColumnType::Timestamp + ); + } } diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index 672666edfd..332fdca22d 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -1,23 +1,29 @@ //! Represents a Chunk of data (a collection of tables and their data within //! some chunk) in the mutable store. -use arrow_deps::{arrow::record_batch::RecordBatch, datafusion::logical_plan::Expr}; - -use generated_types::wal as wb; use std::collections::{BTreeSet, HashMap}; +use std::sync::Arc; -use data_types::partition_metadata::TableSummary; -use internal_types::{entry::TableBatch, schema::Schema, selection::Selection}; +use snafu::{OptionExt, ResultExt, Snafu}; +use arrow_deps::{arrow::record_batch::RecordBatch, datafusion::logical_plan::Expr}; +use data_types::{database_rules::WriterId, partition_metadata::TableSummary}; +use generated_types::wal as wb; +use internal_types::{ + entry::{ClockValue, TableBatch}, + schema::Schema, + selection::Selection, +}; +use tracker::{MemRegistry, MemTracker}; + +use crate::chunk::snapshot::ChunkSnapshot; use crate::{ column::Column, dictionary::{Dictionary, Error as DictionaryError}, pred::{ChunkPredicate, ChunkPredicateBuilder}, table::Table, }; -use data_types::database_rules::WriterId; -use internal_types::entry::ClockValue; -use snafu::{OptionExt, ResultExt, Snafu}; -use tracker::{MemRegistry, MemTracker}; + +pub mod snapshot; #[derive(Debug, Snafu)] pub enum Error { @@ -126,21 +132,6 @@ pub struct Chunk { tracker: MemTracker, } -impl Clone for Chunk { - fn clone(&self) -> Self { - // TODO: The performance of this is not great - (#635) - let mut ret = Self { - id: self.id, - dictionary: self.dictionary.clone(), - tables: self.tables.clone(), - tracker: self.tracker.clone_empty(), - }; - - ret.tracker.set_bytes(ret.size()); - ret - } -} - impl Chunk { pub fn new(id: u32, memory_registry: &MemRegistry) -> Self { let mut chunk = Self { @@ -207,7 +198,8 @@ impl Chunk { Ok(()) } - // Add all tables names in this chunk to `names` if they are not already present + /// Add all tables names in this chunk to `names` if they are not already + /// present pub fn all_table_names(&self, names: &mut BTreeSet) { for &table_id in self.tables.keys() { let table_name = self.dictionary.lookup_id(table_id).unwrap(); @@ -217,6 +209,12 @@ impl Chunk { } } + /// Returns a queryable snapshot of this chunk + pub fn snapshot(&self) -> Arc { + // TODO: Cache this + Arc::new(ChunkSnapshot::new(self)) + } + /// Return all the names of the tables names in this chunk that match /// chunk predicate pub fn table_names(&self, chunk_predicate: &ChunkPredicate) -> Result> { diff --git a/mutable_buffer/src/chunk/snapshot.rs b/mutable_buffer/src/chunk/snapshot.rs new file mode 100644 index 0000000000..901b8c5640 --- /dev/null +++ b/mutable_buffer/src/chunk/snapshot.rs @@ -0,0 +1,181 @@ +use std::collections::{BTreeSet, HashMap}; +use std::sync::Arc; + +use arrow_deps::arrow::record_batch::RecordBatch; +use data_types::timestamp::TimestampRange; +use internal_types::schema::{Schema, TIME_COLUMN_NAME}; +use internal_types::selection::Selection; +use snafu::{OptionExt, ResultExt, Snafu}; + +use super::Chunk; + +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("Table not found: {}", table_name))] + TableNotFound { table_name: String }, + + #[snafu(display("Failed to select columns: {}", source))] + SelectColumns { + source: internal_types::schema::Error, + }, +} + +pub type Result = std::result::Result; + +/// A queryable snapshot of a mutable buffer chunk +#[derive(Debug)] +pub struct ChunkSnapshot { + /// The ID of the chunk this is a snapshot of + chunk_id: u32, + + /// Maps table name to `TableSnapshot` + records: HashMap, + // TODO: Memory tracking +} + +#[derive(Debug)] +struct TableSnapshot { + schema: Schema, + batch: RecordBatch, + timestamp_range: Option, +} + +impl TableSnapshot { + fn matches_predicate(&self, timestamp_range: &Option) -> bool { + match (self.timestamp_range, timestamp_range) { + (Some(a), Some(b)) => !a.disjoint(b), + (None, Some(_)) => false, /* If this chunk doesn't have a time column it can't match */ + // the predicate + (_, None) => true, + } + } +} + +impl ChunkSnapshot { + pub fn new(chunk: &Chunk) -> Self { + let mut records: HashMap = Default::default(); + for (id, table) in &chunk.tables { + let schema = table.schema(chunk, Selection::All).unwrap(); + let batch = table.to_arrow(chunk, Selection::All).unwrap(); + let name = chunk.dictionary.lookup_id(*id).unwrap(); + + let timestamp_range = chunk + .dictionary + .lookup_value(TIME_COLUMN_NAME) + .ok() + .and_then(|column_id| { + table.column(column_id).ok().and_then(|column| { + // TimestampRange has an exclusive upper bound + column + .get_i64_stats() + .map(|x| TimestampRange::new(x.min, x.max + 1)) + }) + }); + + records.insert( + name.to_string(), + TableSnapshot { + batch, + schema, + timestamp_range, + }, + ); + } + + Self { + chunk_id: chunk.id, + records, + } + } + + /// return the ID of the chunk this is a snapshot of + pub fn chunk_id(&self) -> u32 { + self.chunk_id + } + + /// returns true if there is no data in this snapshot + pub fn is_empty(&self) -> bool { + self.records.is_empty() + } + + /// Return true if this snapshot has the specified table name + pub fn has_table(&self, table_name: &str) -> bool { + self.records.get(table_name).is_some() + } + + /// Return Schema for the specified table / columns + pub fn table_schema(&self, table_name: &str, selection: Selection<'_>) -> Result { + let table = self + .records + .get(table_name) + .context(TableNotFound { table_name })?; + + Ok(match selection { + Selection::All => table.schema.clone(), + Selection::Some(columns) => { + let columns = table.schema.select(columns).context(SelectColumns)?; + table.schema.project(&columns) + } + }) + } + + /// Returns a list of tables with writes matching the given timestamp_range + pub fn table_names( + &self, + timestamp_range: Option, + ) -> impl Iterator + '_ { + self.records + .iter() + .flat_map(move |(table_name, table_snapshot)| { + match table_snapshot.matches_predicate(×tamp_range) { + true => Some(table_name), + false => None, + } + }) + } + + /// Returns a RecordBatch with the given selection + pub fn read_filter(&self, table_name: &str, selection: Selection<'_>) -> Result { + let table = self + .records + .get(table_name) + .context(TableNotFound { table_name })?; + + Ok(match selection { + Selection::All => table.batch.clone(), + Selection::Some(columns) => { + let projection = table.schema.select(columns).context(SelectColumns)?; + let schema = table.schema.project(&projection).into(); + let columns = projection + .into_iter() + .map(|x| Arc::clone(table.batch.column(x))) + .collect(); + + RecordBatch::try_new(schema, columns).expect("failed to project record batch") + } + }) + } + + /// Returns a given selection of column names from a table + pub fn column_names( + &self, + table_name: &str, + selection: Selection<'_>, + ) -> Option> { + let table = self.records.get(table_name)?; + let fields = table.schema.inner().fields().iter(); + + Some(match selection { + Selection::Some(cols) => fields + .filter_map(|x| { + if cols.contains(&x.name().as_str()) { + Some(x.name().clone()) + } else { + None + } + }) + .collect(), + Selection::All => fields.map(|x| x.name().clone()).collect(), + }) + } +} diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index b832e452df..05f95af026 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -526,6 +526,13 @@ impl Column { } } + pub fn get_i64_stats(&self) -> Option> { + match self { + Self::I64(_, values) => Some(values.clone()), + _ => None, + } + } + /// Returns true if any rows are within the range [min_value, /// max_value). Inclusive of `start`, exclusive of `end` pub fn has_i64_range(&self, start: i64, end: i64) -> Result { diff --git a/object_store/Cargo.toml b/object_store/Cargo.toml index bcfd28a719..6bf1e8cbee 100644 --- a/object_store/Cargo.toml +++ b/object_store/Cargo.toml @@ -14,7 +14,7 @@ bytes = "1.0" chrono = "0.4" # Google Cloud Storage integration cloud-storage = "0.9.0" -futures = "0.3.5" +futures = "0.3" itertools = "0.9.0" percent-encoding = "2.1" # rusoto crates are for Amazon S3 integration diff --git a/parquet_file/Cargo.toml b/parquet_file/Cargo.toml index 015913d1d1..2750677be4 100644 --- a/parquet_file/Cargo.toml +++ b/parquet_file/Cargo.toml @@ -8,7 +8,7 @@ edition = "2018" arrow_deps = { path = "../arrow_deps" } bytes = "1.0" data_types = { path = "../data_types" } -futures = "0.3.7" +futures = "0.3" object_store = {path = "../object_store"} parking_lot = "0.11.1" snafu = "0.6" diff --git a/query/Cargo.toml b/query/Cargo.toml index 2f1bb79862..bfa7d705b7 100644 --- a/query/Cargo.toml +++ b/query/Cargo.toml @@ -19,7 +19,7 @@ async-trait = "0.1" chrono = "0.4" croaring = "0.4.5" data_types = { path = "../data_types" } -futures = "0.3.7" +futures = "0.3" influxdb_line_protocol = { path = "../influxdb_line_protocol" } internal_types = { path = "../internal_types" } parking_lot = "0.11.1" diff --git a/server/Cargo.toml b/server/Cargo.toml index d27f1ac029..0cc7e6de7d 100644 --- a/server/Cargo.toml +++ b/server/Cargo.toml @@ -14,7 +14,7 @@ data_types = { path = "../data_types" } # See docs/regenerating_flatbuffers.md about updating generated code when updating the # version of the flatbuffers crate flatbuffers = "0.8" -futures = "0.3.7" +futures = "0.3" generated_types = { path = "../generated_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } internal_types = { path = "../internal_types" } diff --git a/server/src/db/chunk.rs b/server/src/db/chunk.rs index 07c979a04f..99898df8f6 100644 --- a/server/src/db/chunk.rs +++ b/server/src/db/chunk.rs @@ -1,6 +1,6 @@ use arrow_deps::datafusion::physical_plan::SendableRecordBatchStream; use internal_types::{schema::Schema, selection::Selection}; -use mutable_buffer::chunk::Chunk as MBChunk; +use mutable_buffer::chunk::snapshot::ChunkSnapshot; use object_store::path::Path; use observability_deps::tracing::debug; use parquet_file::chunk::Chunk as ParquetChunk; @@ -11,15 +11,15 @@ use snafu::{ResultExt, Snafu}; use std::{collections::BTreeSet, sync::Arc}; use super::{ - pred::{to_mutable_buffer_predicate, to_read_buffer_predicate}, - streams::{MutableBufferChunkStream, ReadFilterResultsStream}, + pred::to_read_buffer_predicate, + streams::{MemoryStream, ReadFilterResultsStream}, }; #[derive(Debug, Snafu)] pub enum Error { #[snafu(display("Mutable Buffer Chunk Error: {}", source))] MutableBufferChunk { - source: mutable_buffer::chunk::Error, + source: mutable_buffer::chunk::snapshot::Error, }, #[snafu(display("Read Buffer Error in chunk {}: {}", chunk_id, source))] @@ -59,10 +59,7 @@ pub type Result = std::result::Result; #[derive(Debug)] pub enum DBChunk { MutableBuffer { - chunk: Arc, - partition_key: Arc, - /// is this chunk open for writing? - open: bool, + chunk: Arc, }, ReadBuffer { chunk: Arc, @@ -84,36 +81,12 @@ impl DBChunk { ChunkState::Invalid => { panic!("Invalid internal state"); } - ChunkState::Open(chunk) => { - // TODO the performance if cloning the chunk is terrible - // Proper performance is tracked in - // https://github.com/influxdata/influxdb_iox/issues/635 - let chunk = Arc::new(chunk.clone()); - Self::MutableBuffer { - chunk, - partition_key, - open: true, - } - } - ChunkState::Closing(chunk) => { - // TODO the performance if cloning the chunk is terrible - // Proper performance is tracked in - // https://github.com/influxdata/influxdb_iox/issues/635 - let chunk = Arc::new(chunk.clone()); - Self::MutableBuffer { - chunk, - partition_key, - open: false, - } - } - ChunkState::Moving(chunk) => { - let chunk = Arc::clone(chunk); - Self::MutableBuffer { - chunk, - partition_key, - open: false, - } - } + ChunkState::Open(chunk) | ChunkState::Closing(chunk) => Self::MutableBuffer { + chunk: chunk.snapshot(), + }, + ChunkState::Moving(chunk) => Self::MutableBuffer { + chunk: chunk.snapshot(), + }, ChunkState::Moved(chunk) => Self::ReadBuffer { chunk: Arc::clone(chunk), partition_key, @@ -144,7 +117,7 @@ impl PartitionChunk for DBChunk { fn id(&self) -> u32 { match self { - Self::MutableBuffer { chunk, .. } => chunk.id(), + Self::MutableBuffer { chunk, .. } => chunk.chunk_id(), Self::ReadBuffer { chunk, .. } => chunk.id(), Self::ParquetFile { chunk, .. } => chunk.id(), } @@ -152,7 +125,9 @@ impl PartitionChunk for DBChunk { fn all_table_names(&self, known_tables: &mut StringSet) { match self { - Self::MutableBuffer { chunk, .. } => chunk.all_table_names(known_tables), + Self::MutableBuffer { chunk, .. } => { + known_tables.extend(chunk.table_names(None).cloned()) + } Self::ReadBuffer { chunk, .. } => { // TODO - align APIs so they behave in the same way... let rb_names = chunk.all_table_names(known_tables); @@ -167,35 +142,15 @@ impl PartitionChunk for DBChunk { fn table_names( &self, predicate: &Predicate, - _known_tables: &StringSet, + _known_tables: &StringSet, // TODO: Should this be being used? ) -> Result, Self::Error> { let names = match self { Self::MutableBuffer { chunk, .. } => { - if chunk.is_empty() { - Some(StringSet::new()) - } else { - let chunk_predicate = match to_mutable_buffer_predicate(chunk, predicate) { - Ok(chunk_predicate) => chunk_predicate, - Err(e) => { - debug!(?predicate, %e, "mutable buffer predicate not supported for table_names, falling back"); - return Ok(None); - } - }; - - // we don't support arbitrary expressions in chunk predicate yet - if !chunk_predicate.chunk_exprs.is_empty() { - None - } else { - let names = chunk - .table_names(&chunk_predicate) - .context(MutableBufferChunk)? - .into_iter() - .map(|s| s.to_string()) - .collect::(); - - Some(names) - } + if predicate.has_exprs() { + // TODO: Support more predicates + return Ok(None); } + chunk.table_names(predicate.range).cloned().collect() } Self::ReadBuffer { chunk, .. } => { // If not supported, ReadBuffer can't answer with @@ -208,7 +163,7 @@ impl PartitionChunk for DBChunk { } }; - Some(chunk.table_names(&rb_predicate, &BTreeSet::new())) + chunk.table_names(&rb_predicate, &BTreeSet::new()) } Self::ParquetFile { .. } => { unimplemented!("parquet file not implemented for scan_data") @@ -217,17 +172,12 @@ impl PartitionChunk for DBChunk { // Prune out tables that should not be // present (based on additional table restrictions of the Predicate) - // - // This is needed because at time of writing, the ReadBuffer's - // table_names implementation doesn't include any way to - // further restrict the tables to a known set of tables - let names = names.map(|names| { + Ok(Some( names .into_iter() .filter(|table_name| predicate.should_include_table(table_name)) - .collect() - }); - Ok(names) + .collect(), + )) } fn table_schema( @@ -284,22 +234,17 @@ impl PartitionChunk for DBChunk { ) -> Result { match self { Self::MutableBuffer { chunk, .. } => { - // Note MutableBuffer doesn't support predicate - // pushdown (other than pruning out the entire chunk - // via `might_pass_predicate) if !predicate.is_empty() { return InternalPredicateNotSupported { predicate: predicate.clone(), } .fail(); } - let schema: Schema = self.table_schema(table_name, selection)?; + let batch = chunk + .read_filter(table_name, selection) + .context(MutableBufferChunk)?; - Ok(Box::pin(MutableBufferChunkStream::new( - Arc::clone(&chunk), - schema.as_arrow(), - table_name, - ))) + Ok(Box::pin(MemoryStream::new(batch))) } Self::ReadBuffer { chunk, .. } => { // Error converting to a rb_predicate needs to fail @@ -361,17 +306,11 @@ impl PartitionChunk for DBChunk { ) -> Result, Self::Error> { match self { Self::MutableBuffer { chunk, .. } => { - let chunk_predicate = match to_mutable_buffer_predicate(chunk, predicate) { - Ok(chunk_predicate) => chunk_predicate, - Err(e) => { - debug!(?predicate, %e, "mutable buffer predicate not supported for column_names, falling back"); - return Ok(None); - } - }; - - chunk - .column_names(table_name, &chunk_predicate, columns) - .context(MutableBufferChunk) + if !predicate.is_empty() { + // TODO: Support predicates + return Ok(None); + } + Ok(chunk.column_names(table_name, columns)) } Self::ReadBuffer { chunk, .. } => { let rb_predicate = match to_read_buffer_predicate(&predicate) { @@ -398,31 +337,15 @@ impl PartitionChunk for DBChunk { fn column_values( &self, - table_name: &str, - column_name: &str, - predicate: &Predicate, + _table_name: &str, + _column_name: &str, + _predicate: &Predicate, ) -> Result, Self::Error> { match self { - Self::MutableBuffer { chunk, .. } => { - use mutable_buffer::chunk::Error::UnsupportedColumnTypeForListingValues; - - let chunk_predicate = match to_mutable_buffer_predicate(chunk, predicate) { - Ok(chunk_predicate) => chunk_predicate, - Err(e) => { - debug!(?predicate, %e, "mutable buffer predicate not supported for column_values, falling back"); - return Ok(None); - } - }; - - let values = chunk.tag_column_values(table_name, column_name, &chunk_predicate); - - // if the mutable buffer doesn't support getting - // values for this kind of column, report back None - if let Err(UnsupportedColumnTypeForListingValues { .. }) = values { - Ok(None) - } else { - values.context(MutableBufferChunk) - } + Self::MutableBuffer { .. } => { + // There is no advantage to manually implementing this + // vs just letting DataFusion do its thing + Ok(None) } Self::ReadBuffer { .. } => { // TODO hook up read buffer API here when ready. Until diff --git a/server/src/db/streams.rs b/server/src/db/streams.rs index 5ea7d8cd17..b421c018a2 100644 --- a/server/src/db/streams.rs +++ b/server/src/db/streams.rs @@ -48,6 +48,7 @@ pub(crate) struct MutableBufferChunkStream { } impl MutableBufferChunkStream { + #[allow(dead_code)] pub fn new(chunk: Arc, schema: SchemaRef, table_name: impl Into) -> Self { Self { chunk, @@ -143,3 +144,42 @@ impl futures::Stream for ReadFilterResultsStream { // TODO is there a useful size_hint to pass? } + +/// A RecordBatchStream created from a single RecordBatch +/// +/// Unfortunately datafusion's MemoryStream is crate-local +#[derive(Debug)] +pub(crate) struct MemoryStream { + schema: SchemaRef, + batch: Option, +} + +impl MemoryStream { + pub fn new(batch: RecordBatch) -> Self { + Self { + schema: batch.schema(), + batch: Some(batch), + } + } +} + +impl RecordBatchStream for MemoryStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +impl futures::Stream for MemoryStream { + type Item = ArrowResult; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + _: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(self.batch.take().map(Ok)) + } + + fn size_hint(&self) -> (usize, Option) { + (1, Some(1)) + } +} diff --git a/server/src/snapshot.rs b/server/src/snapshot.rs index 86be67cadf..80f2737c5d 100644 --- a/server/src/snapshot.rs +++ b/server/src/snapshot.rs @@ -354,9 +354,7 @@ mem,host=A,region=west used=45 1 let registry = MemRegistry::new(); let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); let chunk = Arc::new(DBChunk::MutableBuffer { - chunk: Arc::new(ChunkWB::new(11, ®istry)), - partition_key: Arc::new("key".to_string()), - open: false, + chunk: ChunkWB::new(11, ®istry).snapshot(), }); let mut metadata_path = store.new_path(); metadata_path.push_dir("meta"); diff --git a/tracker/Cargo.toml b/tracker/Cargo.toml index be8adabf15..1fa013a874 100644 --- a/tracker/Cargo.toml +++ b/tracker/Cargo.toml @@ -7,7 +7,7 @@ description = "Utilities for tracking resource utilisation within IOx" [dependencies] -futures = "0.3.7" +futures = "0.3" hashbrown = "0.9.1" observability_deps = { path = "../observability_deps" } pin-project = "1.0" diff --git a/wal/Cargo.toml b/wal/Cargo.toml index f952f2c288..f18a7d2248 100644 --- a/wal/Cargo.toml +++ b/wal/Cargo.toml @@ -7,7 +7,7 @@ edition = "2018" [dependencies] # In alphabetical order byteorder = "1.3.4" crc32fast = "1.2.0" -futures = "0.3.4" +futures = "0.3" itertools = "0.9.0" once_cell = { version = "1.4.0", features = ["parking_lot"] } regex = "1.3.7" From 7e28f8ef66a06c4b4d70da6ef7cdca23200aa755 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 12 Apr 2021 13:17:28 -0400 Subject: [PATCH 14/45] feat: Implement Entry writing to Db This removes the old ReplicatedWrite structure and implements the writing of an Entry to the Db. I also call out in `server/lib.rs` and in the `Db` where sharding and replication might happen. I've also added helpers in various places to write line protocol to chunks, tables, and databases. That enabled removing a good amount of code from the test helpers crate. --- data_types/src/database_rules.rs | 16 ++ internal_types/src/entry.rs | 30 +- mutable_buffer/src/chunk.rs | 32 +-- mutable_buffer/src/column.rs | 144 ---------- mutable_buffer/src/table.rs | 99 ++----- query/src/lib.rs | 5 +- query/src/test.rs | 143 +--------- server/src/db.rs | 256 +++++++++--------- server/src/lib.rs | 201 +++++--------- .../influxrpc/read_window_aggregate.rs | 15 +- server/src/query_tests/scenarios.rs | 43 ++- server/src/query_tests/sql.rs | 6 +- server/src/snapshot.rs | 6 +- 13 files changed, 284 insertions(+), 712 deletions(-) diff --git a/data_types/src/database_rules.rs b/data_types/src/database_rules.rs index ec923618a5..4c61b432e1 100644 --- a/data_types/src/database_rules.rs +++ b/data_types/src/database_rules.rs @@ -743,12 +743,22 @@ impl TryFrom for TemplatePart { /// ShardId maps to a nodegroup that holds the the shard. pub type ShardId = u16; +const DEFAULT_SHARD_ID: u16 = 0; /// Assigns a given line to a specific shard id. pub trait Sharder { fn shard(&self, line: &ParsedLine<'_>) -> Result; } +impl Sharder for DatabaseRules { + fn shard(&self, line: &ParsedLine<'_>) -> Result { + match &self.shard_config { + Some(s) => s.shard(line), + None => Ok(DEFAULT_SHARD_ID), + } + } +} + /// ShardConfig defines rules for assigning a line/row to an individual /// host or a group of hosts. A shard /// is a logical concept, but the usage is meant to split data into @@ -776,6 +786,12 @@ pub struct ShardConfig { pub ignore_errors: bool, } +impl Sharder for ShardConfig { + fn shard(&self, _line: &ParsedLine<'_>) -> Result { + unimplemented!() // TODO: mkm to implement as part of #916 + } +} + /// Maps a matcher with specific target group. If the line/row matches /// it should be sent to the group. #[derive(Debug, Eq, PartialEq, Clone, Default)] diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index 70b51d9d71..315f8c5e63 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -1223,6 +1223,7 @@ impl TryFrom> for SequencedEntry { pub mod test_helpers { use super::*; + use chrono::TimeZone; use influxdb_line_protocol::parse_lines; /// Converts the line protocol to a vec of ShardedEntry with a single shard @@ -1230,7 +1231,7 @@ pub mod test_helpers { pub fn lp_to_entry(lp: &str) -> Entry { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)) + lines_to_sharded_entries(&lines, &sharder(1), &hour_partitioner()) .unwrap() .pop() .unwrap() @@ -1261,6 +1262,11 @@ pub mod test_helpers { } } + /// Returns a test partitioner that will partition data by the hour + pub fn hour_partitioner() -> HourPartitioner { + HourPartitioner {} + } + /// Returns a test partitioner that will assign partition keys in the form /// key_# where # is replaced by a number `[0, count)` incrementing for /// each line. @@ -1290,6 +1296,28 @@ pub mod test_helpers { Ok(format!("key_{}", n % self.count)) } } + + // Partitions by the hour + #[derive(Debug)] + pub struct HourPartitioner {} + + impl Partitioner for HourPartitioner { + fn partition_key( + &self, + line: &ParsedLine<'_>, + default_time: &DateTime, + ) -> data_types::database_rules::Result { + const HOUR_FORMAT: &str = "%Y-%m-%dT%H"; + + let key = match line.timestamp { + Some(t) => Utc.timestamp_nanos(t).format(HOUR_FORMAT), + None => default_time.format(HOUR_FORMAT), + } + .to_string(); + + Ok(key) + } + } } #[cfg(test)] diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index 332fdca22d..254c29c2d3 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -7,7 +7,6 @@ use snafu::{OptionExt, ResultExt, Snafu}; use arrow_deps::{arrow::record_batch::RecordBatch, datafusion::logical_plan::Expr}; use data_types::{database_rules::WriterId, partition_metadata::TableSummary}; -use generated_types::wal as wb; use internal_types::{ entry::{ClockValue, TableBatch}, schema::Schema, @@ -165,41 +164,12 @@ impl Chunk { .context(TableWrite { table_name })?; } - Ok(()) - } - - pub fn write_entry(&mut self, entry: &wb::WriteBufferEntry<'_>) -> Result<()> { - if let Some(table_batches) = entry.table_batches() { - for batch in table_batches { - self.write_table_batch(&batch)?; - } - } - self.tracker.set_bytes(self.size()); Ok(()) } - fn write_table_batch(&mut self, batch: &wb::TableWriteBatch<'_>) -> Result<()> { - let table_name = batch.name().context(TableWriteWithoutName)?; - let table_id = self.dictionary.lookup_value_or_insert(table_name); - - let table = self - .tables - .entry(table_id) - .or_insert_with(|| Table::new(table_id)); - - if let Some(rows) = batch.rows() { - table - .append_rows(&mut self.dictionary, &rows) - .context(TableWrite { table_name })?; - } - - Ok(()) - } - - /// Add all tables names in this chunk to `names` if they are not already - /// present + // Add all tables names in this chunk to `names` if they are not already present pub fn all_table_names(&self, names: &mut BTreeSet) { for &table_id in self.tables.keys() { let table_name = self.dictionary.lookup_id(table_id).unwrap(); diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index 05f95af026..ddd7ab9e12 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -1,11 +1,9 @@ -use generated_types::wal as wb; use snafu::Snafu; use crate::dictionary::Dictionary; use arrow_deps::arrow::datatypes::DataType as ArrowDataType; use data_types::partition_metadata::StatValues; use generated_types::entry::LogicalColumnType; -use internal_types::data::type_description; use internal_types::entry::TypedValuesIterator; use std::mem; @@ -308,78 +306,6 @@ impl Column { } } - pub fn with_value( - dictionary: &mut Dictionary, - capacity: usize, - value: wb::Value<'_>, - ) -> Result { - Ok(match value.value_type() { - wb::ColumnValue::F64Value => { - let val = value - .value_as_f64value() - .expect("f64 value should be present") - .value(); - let mut vals = vec![None; capacity]; - vals.push(Some(val)); - Self::F64(vals, StatValues::new(val)) - } - wb::ColumnValue::I64Value => { - let val = value - .value_as_i64value() - .expect("i64 value should be present") - .value(); - let mut vals = vec![None; capacity]; - vals.push(Some(val)); - Self::I64(vals, StatValues::new(val)) - } - wb::ColumnValue::U64Value => { - let val = value - .value_as_u64value() - .expect("u64 value should be present") - .value(); - let mut vals = vec![None; capacity]; - vals.push(Some(val)); - Self::U64(vals, StatValues::new(val)) - } - wb::ColumnValue::StringValue => { - let val = value - .value_as_string_value() - .expect("string value should be present") - .value() - .expect("string must be present"); - let mut vals = vec![None; capacity]; - vals.push(Some(val.to_string())); - Self::String(vals, StatValues::new(val.to_string())) - } - wb::ColumnValue::BoolValue => { - let val = value - .value_as_bool_value() - .expect("bool value should be present") - .value(); - let mut vals = vec![None; capacity]; - vals.push(Some(val)); - Self::Bool(vals, StatValues::new(val)) - } - wb::ColumnValue::TagValue => { - let val = value - .value_as_tag_value() - .expect("tag value should be present") - .value() - .expect("tag value must have string value"); - let mut vals = vec![None; capacity]; - let id = dictionary.lookup_value_or_insert(val); - vals.push(Some(id)); - Self::Tag(vals, StatValues::new(val.to_string())) - } - _ => { - return UnknownColumnType { - inserted_value_type: type_description(value.value_type()), - } - .fail() - } - }) - } - pub fn len(&self) -> usize { match self { Self::F64(v, _) => v.len(), @@ -418,76 +344,6 @@ impl Column { } } - pub fn push(&mut self, dictionary: &mut Dictionary, value: &wb::Value<'_>) -> Result<()> { - let inserted = match self { - Self::Tag(vals, stats) => match value.value_as_tag_value() { - Some(tag) => { - let tag_value = tag.value().expect("tag must have string value"); - let id = dictionary.lookup_value_or_insert(tag_value); - vals.push(Some(id)); - StatValues::update_string(stats, tag_value); - true - } - None => false, - }, - Self::String(vals, stats) => match value.value_as_string_value() { - Some(str_val) => { - let str_val = str_val.value().expect("string must have value"); - vals.push(Some(str_val.to_string())); - StatValues::update_string(stats, str_val); - true - } - None => false, - }, - Self::Bool(vals, stats) => match value.value_as_bool_value() { - Some(bool_val) => { - let bool_val = bool_val.value(); - vals.push(Some(bool_val)); - stats.update(bool_val); - true - } - None => false, - }, - Self::I64(vals, stats) => match value.value_as_i64value() { - Some(i64_val) => { - let i64_val = i64_val.value(); - vals.push(Some(i64_val)); - stats.update(i64_val); - true - } - None => false, - }, - Self::U64(vals, stats) => match value.value_as_u64value() { - Some(u64_val) => { - let u64_val = u64_val.value(); - vals.push(Some(u64_val)); - stats.update(u64_val); - true - } - None => false, - }, - Self::F64(vals, stats) => match value.value_as_f64value() { - Some(f64_val) => { - let f64_val = f64_val.value(); - vals.push(Some(f64_val)); - stats.update(f64_val); - true - } - None => false, - }, - }; - - if inserted { - Ok(()) - } else { - TypeMismatch { - existing_column_type: self.type_description(), - inserted_value_type: type_description(value.value_type()), - } - .fail() - } - } - // push_none_if_len_equal will add a None value to the end of the Vec of values // if the length is equal to the passed in value. This is used to ensure // columns are all the same length. diff --git a/mutable_buffer/src/table.rs b/mutable_buffer/src/table.rs index 9e54e16aeb..e522eeadba 100644 --- a/mutable_buffer/src/table.rs +++ b/mutable_buffer/src/table.rs @@ -1,5 +1,3 @@ -use generated_types::wal as wb; - use std::{ cmp, collections::{BTreeMap, BTreeSet}, @@ -13,9 +11,12 @@ use crate::{ dictionary::{Dictionary, Error as DictionaryError}, pred::{ChunkIdSet, ChunkPredicate}, }; -use data_types::partition_metadata::{ColumnSummary, Statistics}; +use data_types::{ + database_rules::WriterId, + partition_metadata::{ColumnSummary, Statistics}, +}; use internal_types::{ - entry, + entry::{self, ClockValue}, schema::{builder::SchemaBuilder, Schema, TIME_COLUMN_NAME}, selection::Selection, }; @@ -32,8 +33,6 @@ use arrow_deps::{ record_batch::RecordBatch, }, }; -use data_types::database_rules::WriterId; -use internal_types::entry::ClockValue; #[derive(Debug, Snafu)] pub enum Error { @@ -148,47 +147,6 @@ impl Table { } } - fn append_row( - &mut self, - dictionary: &mut Dictionary, - values: &flatbuffers::Vector<'_, flatbuffers::ForwardsUOffset>>, - ) -> Result<()> { - let row_count = self.row_count(); - - // insert new columns and validate existing ones - for value in values { - let column_name = value - .column() - .context(ColumnNameNotInRow { table: self.id })?; - let column_id = dictionary.lookup_value_or_insert(column_name); - - let column = match self.columns.get_mut(&column_id) { - Some(col) => col, - None => { - // Add the column and make all values for existing rows None - self.columns.insert( - column_id, - Column::with_value(dictionary, row_count, value) - .context(CreatingFromWal { column: column_id })?, - ); - - continue; - } - }; - - column.push(dictionary, &value).context(ColumnError { - column: column_name, - })?; - } - - // make sure all the columns are of the same length - for col in self.columns.values_mut() { - col.push_none_if_len_equal(row_count); - } - - Ok(()) - } - pub fn row_count(&self) -> usize { self.columns .values() @@ -227,20 +185,6 @@ impl Table { } } - pub fn append_rows( - &mut self, - dictionary: &mut Dictionary, - rows: &flatbuffers::Vector<'_, flatbuffers::ForwardsUOffset>>, - ) -> Result<()> { - for row in rows { - if let Some(values) = row.values() { - self.append_row(dictionary, &values)?; - } - } - - Ok(()) - } - /// Validates the schema of the passed in columns, then adds their values to /// the associated columns in the table and updates summary statistics. pub fn write_columns( @@ -690,9 +634,6 @@ impl<'a> TableColSelection<'a> { #[cfg(test)] mod tests { - - use influxdb_line_protocol::{parse_lines, ParsedLine}; - use internal_types::data::split_lines_into_write_entry_partitions; use internal_types::entry::test_helpers::lp_to_entry; use super::*; @@ -1056,26 +997,18 @@ mod tests { /// Insert the line protocol lines in `lp_lines` into this table fn write_lines_to_table(table: &mut Table, dictionary: &mut Dictionary, lp_lines: Vec<&str>) { let lp_data = lp_lines.join("\n"); + let entry = lp_to_entry(&lp_data); - let lines: Vec<_> = parse_lines(&lp_data).map(|l| l.unwrap()).collect(); - - let data = split_lines_into_write_entry_partitions(chunk_key_func, &lines); - - let batch = flatbuffers::root::>(&data).unwrap(); - let entries = batch.entries().expect("at least one entry"); - - for entry in entries { - let table_batches = entry.table_batches().expect("there were table batches"); - for batch in table_batches { - let rows = batch.rows().expect("Had rows in the batch"); - table - .append_rows(dictionary, &rows) - .expect("Appended the row"); - } + for batch in entry + .partition_writes() + .unwrap() + .first() + .unwrap() + .table_batches() + { + table + .write_columns(dictionary, ClockValue::new(0), 0, batch.columns()) + .unwrap(); } } - - fn chunk_key_func(_: &ParsedLine<'_>) -> String { - String::from("the_chunk_key") - } } diff --git a/query/src/lib.rs b/query/src/lib.rs index d240900626..c3d8aa5c85 100644 --- a/query/src/lib.rs +++ b/query/src/lib.rs @@ -10,7 +10,7 @@ use arrow_deps::datafusion::physical_plan::SendableRecordBatchStream; use async_trait::async_trait; use data_types::chunk::ChunkSummary; use exec::{stringset::StringSet, Executor}; -use internal_types::{data::ReplicatedWrite, schema::Schema, selection::Selection}; +use internal_types::{schema::Schema, selection::Selection}; use std::{fmt::Debug, sync::Arc}; @@ -39,9 +39,6 @@ pub trait Database: Debug + Send + Sync { type Error: std::error::Error + Send + Sync + 'static; type Chunk: PartitionChunk; - /// Stores the replicated write into the database. - fn store_replicated_write(&self, write: &ReplicatedWrite) -> Result<(), Self::Error>; - /// Return the partition keys for data in this DB fn partition_keys(&self) -> Result, Self::Error>; diff --git a/query/src/test.rs b/query/src/test.rs index 0f5e443dec..a4e357613c 100644 --- a/query/src/test.rs +++ b/query/src/test.rs @@ -18,10 +18,7 @@ use crate::{ Database, DatabaseStore, PartitionChunk, Predicate, }; -use data_types::database_rules::{PartitionTemplate, TemplatePart}; -use influxdb_line_protocol::{parse_lines, ParsedLine}; use internal_types::{ - data::{lines_to_replicated_write, ReplicatedWrite}, schema::{ builder::{SchemaBuilder, SchemaMerger}, Schema, @@ -30,10 +27,8 @@ use internal_types::{ }; use async_trait::async_trait; -use chrono::{DateTime, Utc}; -use data_types::database_rules::Partitioner; use parking_lot::Mutex; -use snafu::{OptionExt, ResultExt, Snafu}; +use snafu::{OptionExt, Snafu}; use std::{collections::BTreeMap, sync::Arc}; #[derive(Debug, Default)] @@ -43,12 +38,6 @@ pub struct TestDatabase { /// Value is map of chunk_id to chunk partitions: Mutex>>>, - /// Lines which have been written to this database, in order - saved_lines: Mutex>, - - /// Replicated writes which have been written to this database, in order - replicated_writes: Mutex>, - /// `column_names` to return upon next request column_names: Arc>>, } @@ -74,33 +63,6 @@ impl TestDatabase { Self::default() } - /// Get all lines written to this database - pub fn get_lines(&self) -> Vec { - self.saved_lines.lock().clone() - } - - /// Get all replicated writs to this database - pub fn get_writes(&self) -> Vec { - self.replicated_writes.lock().clone() - } - - /// Parse line protocol and add it as new lines to this - /// database - pub async fn add_lp_string(&self, lp_data: &str) { - let parsed_lines = parse_lines(&lp_data) - .collect::, _>>() - .unwrap_or_else(|_| panic!("parsing line protocol: {}", lp_data)); - - let mut writer = TestLPWriter::default(); - writer.write_lines(self, &parsed_lines).unwrap(); - - // Writes parsed lines into this database - let mut saved_lines = self.saved_lines.lock(); - for line in parsed_lines { - saved_lines.push(line.to_string()) - } - } - /// Add a test chunk to the database pub fn add_chunk(&self, partition_key: &str, chunk: Arc) { let mut partitions = self.partitions.lock(); @@ -132,12 +94,6 @@ impl Database for TestDatabase { type Error = TestError; type Chunk = TestChunk; - /// Adds the replicated write to this database - fn store_replicated_write(&self, write: &ReplicatedWrite) -> Result<(), Self::Error> { - self.replicated_writes.lock().push(write.clone()); - Ok(()) - } - /// Return the partition keys for data in this DB fn partition_keys(&self) -> Result, Self::Error> { let partitions = self.partitions.lock(); @@ -448,15 +404,6 @@ impl TestDatabaseStore { pub fn new() -> Self { Self::default() } - - /// Parse line protocol and add it as new lines to the `db_name` database - pub async fn add_lp_string(&self, db_name: &str, lp_data: &str) { - self.db_or_create(db_name) - .await - .expect("db_or_create suceeeds") - .add_lp_string(lp_data) - .await - } } impl Default for TestDatabaseStore { @@ -505,91 +452,3 @@ impl DatabaseStore for TestDatabaseStore { Arc::clone(&self.executor) } } - -/// Helper for writing line protocol data directly into test databases -/// (handles creating sequence numbers and writer ids -#[derive(Debug, Default)] -pub struct TestLPWriter { - pub writer_id: u32, - sequence_number: u64, -} - -impl TestLPWriter { - // writes data in LineProtocol format into a database - pub fn write_lines( - &mut self, - database: &D, - lines: &[ParsedLine<'_>], - ) -> Result<()> { - // partitions data in hourly segments - let partition_template = PartitionTemplate { - parts: vec![TemplatePart::TimeFormat("%Y-%m-%dT%H".to_string())], - }; - - let write = lines_to_replicated_write( - self.writer_id, - self.sequence_number, - &lines, - &partition_template, - ); - self.sequence_number += 1; - database - .store_replicated_write(&write) - .map_err(|e| TestError::DatabaseWrite { - source: Box::new(e), - }) - } - - /// Writes line protocol formatted data in lp_data to `database` - pub fn write_lp_string(&mut self, database: &D, lp_data: &str) -> Result<()> { - let lines = parse_lines(lp_data) - .collect::, _>>() - .map_err(|e| Box::new(e) as _) - .context(DatabaseWrite)?; - - self.write_lines(database, &lines) - } - - /// Writes line protocol formatted data to database and partition - pub fn write_lp_to_partition( - &mut self, - database: &D, - lp_data: &str, - paritition_key: impl Into, - ) { - let lines = parse_lines(lp_data).collect::, _>>().unwrap(); - self.write_lines_to_partition(database, paritition_key, &lines) - } - - /// Writes lines the the given partition - pub fn write_lines_to_partition( - &mut self, - database: &D, - partition_key: impl Into, - lines: &[ParsedLine<'_>], - ) { - let partitioner = TestPartitioner { - key: partition_key.into(), - }; - let write = - lines_to_replicated_write(self.writer_id, self.sequence_number, &lines, &partitioner); - self.sequence_number += 1; - database.store_replicated_write(&write).unwrap(); - } -} - -// Outputs a set partition key for testing. Used for parsing line protocol into -// ReplicatedWrite and setting an explicit partition key for all writes therein. -struct TestPartitioner { - key: String, -} - -impl Partitioner for TestPartitioner { - fn partition_key( - &self, - _line: &ParsedLine<'_>, - _default_time: &DateTime, - ) -> data_types::database_rules::Result { - Ok(self.key.clone()) - } -} diff --git a/server/src/db.rs b/server/src/db.rs index b4f5ca38f2..78df6c41ad 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -25,7 +25,7 @@ pub(crate) use chunk::DBChunk; use data_types::{ chunk::ChunkSummary, database_rules::DatabaseRules, partition_metadata::PartitionSummary, }; -use internal_types::{data::ReplicatedWrite, selection::Selection}; +use internal_types::selection::Selection; use object_store::ObjectStore; use parquet_file::{chunk::Chunk, storage::Storage}; use query::{Database, DEFAULT_SCHEMA}; @@ -36,6 +36,7 @@ use super::{buffer::Buffer, JobRegistry}; use data_types::job::Job; use data_types::partition_metadata::TableSummary; +use internal_types::entry::{self, Entry, SequencedEntry, ClockValue}; use lifecycle::LifecycleManager; use system_tables::{SystemSchemaProvider, SYSTEM_SCHEMA}; @@ -131,6 +132,9 @@ pub enum Error { chunk_id: u32, source: mutable_buffer::chunk::Error, }, + + #[snafu(display("Error building sequenced entry: {}", source))] + SequencedEntryError { source: entry::Error }, } pub type Result = std::result::Result; @@ -622,6 +626,75 @@ impl Db { info!("finished background worker"); } + + /// Stores an entry based on the configuration. The Entry will first be + /// converted into a Sequenced Entry with the logical clock assigned + /// from the database. If the write buffer is configured, the sequenced + /// entry is written into the buffer and replicated based on the + /// configured rules. If the mutable buffer is configured, the sequenced + /// entry is then written into the mutable buffer. + pub fn store_entry(&self, entry: Entry) -> Result<()> { + // TODO: build this based on either this or on the write buffer, if configured + let sequenced_entry = SequencedEntry::new_from_entry_bytes( + ClockValue::new(self.next_sequence()), + self.server_id.get(), + entry.data(), + ) + .context(SequencedEntryError)?; + + self.store_sequenced_entry(sequenced_entry) + } + + pub fn store_sequenced_entry(&self, sequenced_entry: SequencedEntry) -> Result<()> { + let rules = self.rules.read(); + let mutable_size_threshold = rules.lifecycle_rules.mutable_size_threshold; + if rules.lifecycle_rules.immutable { + return DatabaseNotWriteable {}.fail(); + } + std::mem::drop(rules); + + // TODO: Direct writes to closing chunks + + if let Some(partitioned_writes) = sequenced_entry.partition_writes() { + for write in partitioned_writes { + let partition_key = write.key(); + let partition = self.catalog.get_or_create_partition(partition_key); + let mut partition = partition.write(); + partition.update_last_write_at(); + + let chunk = partition.open_chunk().unwrap_or_else(|| { + partition.create_open_chunk(self.memory_registries.mutable_buffer.as_ref()) + }); + + let mut chunk = chunk.write(); + chunk.record_write(); + let chunk_id = chunk.id(); + + let mb_chunk = chunk.mutable_buffer().expect("cannot mutate open chunk"); + + mb_chunk + .write_table_batches( + sequenced_entry.clock_value(), + sequenced_entry.writer_id(), + &write.table_batches(), + ) + .context(WriteEntry { + partition_key, + chunk_id, + })?; + + let size = mb_chunk.size(); + + if let Some(threshold) = mutable_size_threshold { + if size > threshold.get() { + chunk.set_closing().expect("cannot close open chunk") + } + } + } + } + + Ok(()) + } } #[async_trait] @@ -650,54 +723,6 @@ impl Database for Db { .collect() } - fn store_replicated_write(&self, write: &ReplicatedWrite) -> Result<(), Self::Error> { - let rules = self.rules.read(); - let mutable_size_threshold = rules.lifecycle_rules.mutable_size_threshold; - if rules.lifecycle_rules.immutable { - return DatabaseNotWriteable {}.fail(); - } - std::mem::drop(rules); - - let entries = match write.write_buffer_batch().and_then(|batch| batch.entries()) { - Some(entries) => entries, - None => return Ok(()), - }; - - // TODO: Direct writes to closing chunks - - for entry in entries.into_iter() { - if let Some(partition_key) = entry.partition_key() { - let partition = self.catalog.get_or_create_partition(partition_key); - let mut partition = partition.write(); - partition.update_last_write_at(); - - let chunk = partition.open_chunk().unwrap_or_else(|| { - partition.create_open_chunk(self.memory_registries.mutable_buffer.as_ref()) - }); - - let mut chunk = chunk.write(); - chunk.record_write(); - let chunk_id = chunk.id(); - - let mb_chunk = chunk.mutable_buffer().expect("cannot mutate open chunk"); - - mb_chunk.write_entry(&entry).context(WriteEntry { - partition_key, - chunk_id, - })?; - - let size = mb_chunk.size(); - - if let Some(threshold) = mutable_size_threshold { - if size > threshold.get() { - chunk.set_closing().expect("cannot close open chunk") - } - } - } - } - Ok(()) - } - fn partition_keys(&self) -> Result, Self::Error> { Ok(self.catalog.partition_keys()) } @@ -729,9 +754,20 @@ impl CatalogProvider for Db { } } +pub mod test_helpers { + use super::*; + use internal_types::entry::test_helpers::lp_to_entry; + + pub fn write_lp(db: &Db, lp: &str) { + let entry = lp_to_entry(lp); + db.store_entry(entry).unwrap(); + } +} + #[cfg(test)] mod tests { use crate::query_tests::utils::{make_database, make_db}; + use ::test_helpers::assert_contains; use arrow_deps::{ arrow::record_batch::RecordBatch, assert_table_eq, @@ -746,15 +782,15 @@ mod tests { use object_store::{ disk::File, path::ObjectStorePath, path::Path, ObjectStore, ObjectStoreApi, }; - use query::{ - exec::Executor, frontend::sql::SQLQueryPlanner, test::TestLPWriter, PartitionChunk, - }; - use test_helpers::assert_contains; + use query::{exec::Executor, frontend::sql::SQLQueryPlanner, PartitionChunk}; use super::*; use futures::stream; use futures::{StreamExt, TryStreamExt}; use std::iter::Iterator; + + use super::test_helpers::write_lp; + use internal_types::entry::test_helpers::lp_to_entry; use std::num::NonZeroUsize; use std::str; use tempfile::TempDir; @@ -766,9 +802,9 @@ mod tests { async fn write_no_mutable_buffer() { // Validate that writes are rejected if there is no mutable buffer let db = make_db(); - let mut writer = TestLPWriter::default(); db.rules.write().lifecycle_rules.immutable = true; - let res = writer.write_lp_string(&db, "cpu bar=1 10"); + let entry = lp_to_entry("cpu bar=1 10"); + let res = db.store_entry(entry); assert_contains!( res.unwrap_err().to_string(), "Cannot write to this database: no mutable buffer configured" @@ -778,8 +814,7 @@ mod tests { #[tokio::test] async fn read_write() { let db = Arc::new(make_db()); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); + write_lp(db.as_ref(), "cpu bar=1 10"); let batches = run_query(db, "select * from cpu").await; @@ -796,9 +831,7 @@ mod tests { #[tokio::test] async fn write_with_rollover() { let db = Arc::new(make_db()); - let mut writer = TestLPWriter::default(); - //writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); - writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); + write_lp(db.as_ref(), "cpu bar=1 10"); assert_eq!(vec!["1970-01-01T00"], db.partition_keys().unwrap()); let mb_chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); @@ -815,7 +848,7 @@ mod tests { assert_table_eq!(expected, &batches); // add new data - writer.write_lp_string(db.as_ref(), "cpu bar=2 20").unwrap(); + write_lp(db.as_ref(), "cpu bar=2 20"); let expected = vec![ "+-----+------+", "| bar | time |", @@ -838,7 +871,6 @@ mod tests { #[tokio::test] async fn write_with_missing_tags_are_null() { let db = Arc::new(make_db()); - let mut writer = TestLPWriter::default(); // Note the `region` tag is introduced in the second line, so // the values in prior rows for the region column are // null. Likewise the `core` tag is introduced in the third @@ -849,9 +881,7 @@ mod tests { "cpu,core=one user=10.0 11", ]; - writer - .write_lp_string(db.as_ref(), &lines.join("\n")) - .unwrap(); + write_lp(db.as_ref(), &lines.join("\n")); assert_eq!(vec!["1970-01-01T00"], db.partition_keys().unwrap()); let mb_chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); @@ -874,9 +904,8 @@ mod tests { async fn read_from_read_buffer() { // Test that data can be loaded into the ReadBuffer let db = Arc::new(make_db()); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); - writer.write_lp_string(db.as_ref(), "cpu bar=2 20").unwrap(); + write_lp(db.as_ref(), "cpu bar=1 10"); + write_lp(db.as_ref(), "cpu bar=2 20"); let partition_key = "1970-01-01T00"; let mb_chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); @@ -946,9 +975,8 @@ mod tests { let db = Arc::new(make_database(server_id, Arc::clone(&object_store), db_name)); // Write some line protocols in Mutable buffer of the DB - let mut writer = TestLPWriter::default(); - writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); - writer.write_lp_string(db.as_ref(), "cpu bar=2 20").unwrap(); + write_lp(db.as_ref(), "cpu bar=1 10"); + write_lp(db.as_ref(), "cpu bar=2 20"); //Now mark the MB chunk close let partition_key = "1970-01-01T00"; @@ -1034,12 +1062,9 @@ mod tests { let db = Arc::new(make_database(server_id, Arc::clone(&object_store), db_name)); // Write some line protocols in Mutable buffer of the DB - let mut writer = TestLPWriter::default(); - writer.write_lp_string(db.as_ref(), "cpu bar=1 10").unwrap(); - writer - .write_lp_string(db.as_ref(), "disk ops=1 20") - .unwrap(); - writer.write_lp_string(db.as_ref(), "cpu bar=2 20").unwrap(); + write_lp(db.as_ref(), "cpu bar=1 10"); + write_lp(db.as_ref(), "disk ops=1 20"); + write_lp(db.as_ref(), "cpu bar=2 20"); //Now mark the MB chunk close let partition_key = "1970-01-01T00"; @@ -1137,8 +1162,7 @@ mod tests { let before_create = Utc::now(); let partition_key = "1970-01-01T00"; - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, "cpu bar=1 10").unwrap(); + write_lp(&db, "cpu bar=1 10"); let after_write = Utc::now(); let last_write_prev = { @@ -1151,7 +1175,7 @@ mod tests { partition.last_write_at() }; - writer.write_lp_string(&db, "cpu bar=1 20").unwrap(); + write_lp(&db, "cpu bar=1 20"); { let partition = db.catalog.valid_partition(partition_key).unwrap(); let partition = partition.read(); @@ -1165,8 +1189,7 @@ mod tests { let db = make_db(); // Given data loaded into two chunks - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, "cpu bar=1 10").unwrap(); + write_lp(&db, "cpu bar=1 10"); let after_data_load = Utc::now(); // When the chunk is rolled over @@ -1199,9 +1222,8 @@ mod tests { db.rules.write().lifecycle_rules.mutable_size_threshold = Some(NonZeroUsize::new(2).unwrap()); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, "cpu bar=1 10").unwrap(); - writer.write_lp_string(&db, "cpu bar=1 20").unwrap(); + write_lp(&db, "cpu bar=1 10"); + write_lp(&db, "cpu bar=1 20"); let partitions = db.catalog.partition_keys(); assert_eq!(partitions.len(), 1); @@ -1218,15 +1240,10 @@ mod tests { #[tokio::test] async fn chunks_sorted_by_times() { let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, "cpu val=1 1").unwrap(); - writer - .write_lp_string(&db, "mem val=2 400000000000001") - .unwrap(); - writer.write_lp_string(&db, "cpu val=1 2").unwrap(); - writer - .write_lp_string(&db, "mem val=2 400000000000002") - .unwrap(); + write_lp(&db, "cpu val=1 1"); + write_lp(&db, "mem val=2 400000000000001"); + write_lp(&db, "cpu val=1 2"); + write_lp(&db, "mem val=2 400000000000002"); let sort_rules = SortOrder { order: Order::Desc, @@ -1257,9 +1274,9 @@ mod tests { // Test that chunk id listing is hooked up let db = make_db(); let partition_key = "1970-01-01T00"; - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, "cpu bar=1 10").unwrap(); - writer.write_lp_string(&db, "cpu bar=1 20").unwrap(); + + write_lp(&db, "cpu bar=1 10"); + write_lp(&db, "cpu bar=1 20"); assert_eq!(mutable_chunk_ids(&db, partition_key), vec![0]); assert_eq!( @@ -1273,13 +1290,13 @@ mod tests { // add a new chunk in mutable buffer, and move chunk1 (but // not chunk 0) to read buffer - writer.write_lp_string(&db, "cpu bar=1 30").unwrap(); + write_lp(&db, "cpu bar=1 30"); let mb_chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); db.load_chunk_to_read_buffer(partition_key, mb_chunk.id()) .await .unwrap(); - writer.write_lp_string(&db, "cpu bar=1 40").unwrap(); + write_lp(&db, "cpu bar=1 40"); assert_eq!(mutable_chunk_ids(&db, partition_key), vec![0, 2]); assert_eq!(read_buffer_chunk_ids(&db, partition_key), vec![1]); @@ -1308,15 +1325,12 @@ mod tests { async fn partition_chunk_summaries() { // Test that chunk id listing is hooked up let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, "cpu bar=1 1").unwrap(); + write_lp(&db, "cpu bar=1 1"); db.rollover_partition("1970-01-01T00").await.unwrap(); // write into a separate partitiion - writer - .write_lp_string(&db, "cpu bar=1,baz2,frob=3 400000000000000") - .unwrap(); + write_lp(&db, "cpu bar=1,baz2,frob=3 400000000000000"); print!("Partitions: {:?}", db.partition_keys().unwrap()); @@ -1353,11 +1367,10 @@ mod tests { #[tokio::test] async fn partition_chunk_summaries_timestamp() { let db = make_db(); - let mut writer = TestLPWriter::default(); let start = Utc::now(); - writer.write_lp_string(&db, "cpu bar=1 1").unwrap(); + write_lp(&db, "cpu bar=1 1"); let after_first_write = Utc::now(); - writer.write_lp_string(&db, "cpu bar=2 2").unwrap(); + write_lp(&db, "cpu bar=2 2"); db.rollover_partition("1970-01-01T00").await.unwrap(); let after_close = Utc::now(); @@ -1405,17 +1418,13 @@ mod tests { async fn chunk_summaries() { // Test that chunk id listing is hooked up let db = make_db(); - let mut writer = TestLPWriter::default(); // get three chunks: one open, one closed in mb and one close in rb - writer.write_lp_string(&db, "cpu bar=1 1").unwrap(); + write_lp(&db, "cpu bar=1 1"); db.rollover_partition("1970-01-01T00").await.unwrap(); - writer.write_lp_string(&db, "cpu bar=1,baz=2 2").unwrap(); - - writer - .write_lp_string(&db, "cpu bar=1,baz=2,frob=3 400000000000000") - .unwrap(); + write_lp(&db, "cpu bar=1,baz=2 2"); + write_lp(&db, "cpu bar=1,baz=2,frob=3 400000000000000"); print!("Partitions: {:?}", db.partition_keys().unwrap()); @@ -1426,9 +1435,7 @@ mod tests { print!("Partitions2: {:?}", db.partition_keys().unwrap()); db.rollover_partition("1970-01-05T15").await.unwrap(); - writer - .write_lp_string(&db, "cpu bar=1,baz=3,blargh=3 400000000000000") - .unwrap(); + write_lp(&db, "cpu bar=1,baz=3,blargh=3 400000000000000"); fn to_arc(s: &str) -> Arc { Arc::new(s.to_string()) @@ -1478,12 +1485,11 @@ mod tests { async fn partition_summaries() { // Test that chunk id listing is hooked up let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, "cpu bar=1 1").unwrap(); + write_lp(&db, "cpu bar=1 1"); let chunk_id = db.rollover_partition("1970-01-01T00").await.unwrap().id(); - writer.write_lp_string(&db, "cpu bar=2,baz=3.0 2").unwrap(); - writer.write_lp_string(&db, "mem foo=1 1").unwrap(); + write_lp(&db, "cpu bar=2,baz=3.0 2"); + write_lp(&db, "mem foo=1 1"); // load a chunk to the read buffer db.load_chunk_to_read_buffer("1970-01-01T00", chunk_id) @@ -1491,12 +1497,8 @@ mod tests { .unwrap(); // write into a separate partitiion - writer - .write_lp_string(&db, "cpu bar=1 400000000000000") - .unwrap(); - writer - .write_lp_string(&db, "mem frob=3 400000000000001") - .unwrap(); + write_lp(&db, "cpu bar=1 400000000000000"); + write_lp(&db, "mem frob=3 400000000000001"); print!("Partitions: {:?}", db.partition_keys().unwrap()); diff --git a/server/src/lib.rs b/server/src/lib.rs index 5f981f5f4e..371a197e41 100644 --- a/server/src/lib.rs +++ b/server/src/lib.rs @@ -83,11 +83,11 @@ use data_types::{ }; use influxdb_line_protocol::ParsedLine; use internal_types::{ - data::{lines_to_replicated_write, ReplicatedWrite}, + entry::{self, lines_to_sharded_entries, Entry}, once::OnceNonZeroU32, }; use object_store::{path::ObjectStorePath, ObjectStore, ObjectStoreApi}; -use query::{exec::Executor, Database, DatabaseStore}; +use query::{exec::Executor, DatabaseStore}; use tracker::{TaskId, TaskRegistration, TaskRegistryWithHistory, TaskTracker, TrackedFutureExt}; use futures::{pin_mut, FutureExt}; @@ -98,6 +98,7 @@ use crate::{ }, db::Db, }; +use internal_types::entry::SequencedEntry; use std::num::NonZeroU32; pub mod buffer; @@ -147,6 +148,8 @@ pub enum Error { DatabaseAlreadyExists { db_name: String }, #[snafu(display("error appending to wal buffer: {}", source))] WalError { source: buffer::Error }, + #[snafu(display("error converting line protocol to flatbuffers: {}", source))] + LineConversion { source: entry::Error }, } pub type Result = std::result::Result; @@ -337,12 +340,12 @@ impl Server { Ok(()) } - /// `write_lines` takes in raw line protocol and converts it to a - /// `ReplicatedWrite`, which is then replicated to other servers based - /// on the configuration of the `db`. This is step #1 from the crate - /// level documentation. + /// `write_lines` takes in raw line protocol and converts it to a collection + /// of ShardedEntry which are then sent to other IOx servers based on + /// the ShardConfig or sent to the local database for buffering in the + /// WriteBuffer and/or the MutableBuffer if configured. pub async fn write_lines(&self, db_name: &str, lines: &[ParsedLine<'_>]) -> Result<()> { - let id = self.require_id()?.get(); + self.require_id()?; let db_name = DatabaseName::new(db_name).context(InvalidDatabaseName)?; let db = self @@ -350,62 +353,35 @@ impl Server { .db(&db_name) .context(DatabaseNotFound { db_name: &*db_name })?; - let sequence = db.next_sequence(); - let write = lines_to_replicated_write(id, sequence, lines, &*db.rules.read()); + let sharded_entries = lines_to_sharded_entries(lines, &*db.rules.read(), &*db.rules.read()) + .context(LineConversion)?; - self.handle_replicated_write(&db_name, &db, write).await?; + for e in sharded_entries { + // TODO: handle sending to shards based on ShardConfig + self.handle_write_entry(&db, e.entry).await?; + } Ok(()) } - pub async fn handle_replicated_write( + pub async fn handle_write_entry(&self, db: &Db, entry: Entry) -> Result<()> { + db.store_entry(entry) + .map_err(|e| Error::UnknownDatabaseError { + source: Box::new(e), + })?; + + Ok(()) + } + + pub async fn handle_sequenced_entry( &self, - db_name: &DatabaseName<'_>, db: &Db, - write: ReplicatedWrite, + sequenced_entry: SequencedEntry, ) -> Result<()> { - match db.store_replicated_write(&write) { - Err(db::Error::DatabaseNotWriteable {}) | Ok(_) => {} - Err(e) => { - return Err(Error::UnknownDatabaseError { - source: Box::new(e), - }) - } - } - - let write = Arc::new(write); - - if let Some(wal_buffer) = &db.wal_buffer { - let persist; - let segment = { - let mut wal_buffer = wal_buffer.lock(); - persist = wal_buffer.persist; - - // TODO: address this issue? - // the mutable buffer and the wal buffer have different locking mechanisms, - // which means that it's possible for a mutable buffer write to - // succeed while a WAL buffer write fails, which would then - // return an error. A single lock is probably undesirable, but - // we need to figure out what semantics we want. - wal_buffer.append(Arc::clone(&write)).context(WalError)? - }; - - if let Some(segment) = segment { - if persist { - let writer_id = self.require_id()?.get(); - let store = Arc::clone(&self.store); - - let (_, tracker) = self.jobs.register(Job::PersistSegment { - writer_id, - segment_id: segment.id, - }); - - segment - .persist_bytes_in_background(tracker, writer_id, db_name, store) - .context(WalError)?; - } - } - } + db.store_sequenced_entry(sequenced_entry) + .map_err(|e| Error::UnknownDatabaseError { + source: Box::new(e), + })?; Ok(()) } @@ -610,12 +586,17 @@ pub trait ConnectionManager { pub trait RemoteServer { type Error: std::error::Error + Send + Sync + 'static; - /// Sends a replicated write to a remote server. This is step #2 from the - /// diagram. - async fn replicate( + /// Sends an Entry to the remote server. An IOx server acting as a + /// router/sharder will call this method to send entries to remotes. + async fn write_entry(&self, db: &str, entry: Entry) -> Result<(), Self::Error>; + + /// Sends a SequencedEntry to the remote server. An IOx server acting as a + /// write buffer will call this method to replicate to other write + /// buffer servers or to send data to downstream subscribers. + async fn write_sequenced_entry( &self, db: &str, - replicated_write: &ReplicatedWrite, + sequenced_entry: SequencedEntry, ) -> Result<(), Self::Error>; } @@ -643,10 +624,19 @@ pub struct RemoteServerImpl {} impl RemoteServer for RemoteServerImpl { type Error = Error; - async fn replicate( + /// Sends an Entry to the remote server. An IOx server acting as a + /// router/sharder will call this method to send entries to remotes. + async fn write_entry(&self, _db: &str, _entry: Entry) -> Result<(), Self::Error> { + unimplemented!() + } + + /// Sends a SequencedEntry to the remote server. An IOx server acting as a + /// write buffer will call this method to replicate to other write + /// buffer servers or to send data to downstream subscribers. + async fn write_sequenced_entry( &self, _db: &str, - _replicated_write: &ReplicatedWrite, + _sequenced_entry: SequencedEntry, ) -> Result<(), Self::Error> { unimplemented!() } @@ -675,21 +665,16 @@ mod tests { use async_trait::async_trait; use futures::TryStreamExt; - use parking_lot::Mutex; use snafu::Snafu; use tokio::task::JoinHandle; use tokio_util::sync::CancellationToken; use arrow_deps::{assert_table_eq, datafusion::physical_plan::collect}; - use data_types::database_rules::{ - PartitionTemplate, TemplatePart, WalBufferConfig, WalBufferRollover, - }; + use data_types::database_rules::{PartitionTemplate, TemplatePart}; use influxdb_line_protocol::parse_lines; use object_store::{memory::InMemory, path::ObjectStorePath}; use query::{frontend::sql::SQLQueryPlanner, Database}; - use crate::buffer::Segment; - use super::*; #[tokio::test] @@ -945,66 +930,6 @@ mod tests { let _ = background_handle.await; } - #[tokio::test] - async fn segment_persisted_on_rollover() { - let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - - let server = Server::new(manager, Arc::clone(&store)); - server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); - let db_name = DatabaseName::new("my_db").unwrap(); - let rules = DatabaseRules { - name: db_name.clone(), - partition_template: Default::default(), - wal_buffer_config: Some(WalBufferConfig { - buffer_size: 500, - segment_size: 10, - buffer_rollover: WalBufferRollover::ReturnError, - store_segments: true, - close_segment_after: None, - }), - lifecycle_rules: Default::default(), - shard_config: None, - }; - server - .create_database( - rules, - server.require_id().unwrap(), - Arc::clone(&server.store), - ) - .await - .unwrap(); - - let lines = parsed_lines("disk,host=a used=10.1 12"); - server.write_lines(db_name.as_str(), &lines).await.unwrap(); - - // write lines should have caused a segment rollover and persist, wait - tokio::task::yield_now().await; - - let mut path = store.new_path(); - path.push_all_dirs(&["1", "my_db", "wal", "000", "000"]); - path.set_file_name("001.segment"); - - let data = store - .get(&path) - .await - .unwrap() - .map_ok(|b| bytes::BytesMut::from(&b[..])) - .try_concat() - .await - .unwrap(); - - let segment = Segment::from_file_bytes(&data).unwrap(); - assert_eq!(segment.writes.len(), 1); - let write = r#" -writer:1, sequence:1, checksum:2741956553 -partition_key: - table:disk - host:a used:10.1 time:12 -"#; - assert_eq!(segment.writes[0].to_string(), write); - } - #[tokio::test] async fn background_task_cleans_jobs() { let manager = TestConnectionManager::new(); @@ -1057,24 +982,22 @@ partition_key: } #[derive(Debug, Default)] - struct TestRemoteServer { - writes: Mutex>>, - } + struct TestRemoteServer {} #[async_trait] impl RemoteServer for TestRemoteServer { type Error = TestClusterError; - async fn replicate( - &self, - db: &str, - replicated_write: &ReplicatedWrite, - ) -> Result<(), Self::Error> { - let mut writes = self.writes.lock(); - let entries = writes.entry(db.to_string()).or_insert_with(Vec::new); - entries.push(replicated_write.clone()); + async fn write_entry(&self, _db: &str, _entry: Entry) -> Result<(), Self::Error> { + unimplemented!() + } - Ok(()) + async fn write_sequenced_entry( + &self, + _db: &str, + _sequenced_entry: SequencedEntry, + ) -> Result<(), Self::Error> { + unimplemented!() } } diff --git a/server/src/query_tests/influxrpc/read_window_aggregate.rs b/server/src/query_tests/influxrpc/read_window_aggregate.rs index 320a1dd0fa..8d3b728176 100644 --- a/server/src/query_tests/influxrpc/read_window_aggregate.rs +++ b/server/src/query_tests/influxrpc/read_window_aggregate.rs @@ -1,5 +1,8 @@ //! Tests for the Influx gRPC queries -use crate::query_tests::{scenarios::*, utils::make_db}; +use crate::{ + db::test_helpers::write_lp, + query_tests::{scenarios::*, utils::make_db}, +}; use arrow_deps::{arrow::util::pretty::pretty_format_batches, datafusion::prelude::*}; use async_trait::async_trait; @@ -8,7 +11,6 @@ use query::{ frontend::influxrpc::InfluxRPCPlanner, group_by::{Aggregate, WindowDuration}, predicate::{Predicate, PredicateBuilder}, - test::TestLPWriter, }; /// runs read_window_aggregate(predicate) and compares it to the expected @@ -162,18 +164,16 @@ impl DBSetup for MeasurementForWindowAggregateMonths { // "2020-04-02T00"] let db = make_db(); - let mut writer = TestLPWriter::default(); let data = lp_lines.join("\n"); - writer.write_lp_string(&db, &data).unwrap(); + write_lp(&db, &data); let scenario1 = DBScenario { scenario_name: "Data in 4 partitions, open chunks of mutable buffer".into(), db, }; let db = make_db(); - let mut writer = TestLPWriter::default(); let data = lp_lines.join("\n"); - writer.write_lp_string(&db, &data).unwrap(); + write_lp(&db, &data); db.rollover_partition("2020-03-01T00").await.unwrap(); db.rollover_partition("2020-03-02T00").await.unwrap(); let scenario2 = DBScenario { @@ -184,9 +184,8 @@ impl DBSetup for MeasurementForWindowAggregateMonths { }; let db = make_db(); - let mut writer = TestLPWriter::default(); let data = lp_lines.join("\n"); - writer.write_lp_string(&db, &data).unwrap(); + write_lp(&db, &data); rollover_and_load(&db, "2020-03-01T00").await; rollover_and_load(&db, "2020-03-02T00").await; rollover_and_load(&db, "2020-04-01T00").await; diff --git a/server/src/query_tests/scenarios.rs b/server/src/query_tests/scenarios.rs index cf051d85bd..4118556e4a 100644 --- a/server/src/query_tests/scenarios.rs +++ b/server/src/query_tests/scenarios.rs @@ -1,10 +1,10 @@ //! This module contains testing scenarios for Db -use query::{test::TestLPWriter, PartitionChunk}; +use query::PartitionChunk; use async_trait::async_trait; -use crate::db::Db; +use crate::db::{test_helpers::write_lp, Db}; use super::utils::{count_mutable_buffer_chunks, count_read_buffer_chunks, make_db}; @@ -47,8 +47,7 @@ impl DBSetup for NoData { let db = make_db(); let data = "cpu,region=west user=23.2 100"; - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data).unwrap(); + write_lp(&db, data); // move data out of open chunk assert_eq!(db.rollover_partition(partition_key).await.unwrap().id(), 0); @@ -174,7 +173,6 @@ pub struct TwoMeasurementsManyFieldsOneChunk {} impl DBSetup for TwoMeasurementsManyFieldsOneChunk { async fn make(&self) -> Vec { let db = make_db(); - let mut writer = TestLPWriter::default(); let lp_lines = vec![ "h2o,state=MA,city=Boston temp=70.4 50", @@ -184,7 +182,7 @@ impl DBSetup for TwoMeasurementsManyFieldsOneChunk { "o2,state=CA temp=79.0 300", ]; - writer.write_lp_string(&db, &lp_lines.join("\n")).unwrap(); + write_lp(&db, &lp_lines.join("\n")); vec![DBScenario { scenario_name: "Data in open chunk of mutable buffer".into(), db, @@ -231,9 +229,7 @@ impl DBSetup for EndToEndTest { let lp_data = lp_lines.join("\n"); let db = make_db(); - let mut writer = TestLPWriter::default(); - let res = writer.write_lp_string(&db, &lp_data); - assert!(res.is_ok(), "Error: {}", res.unwrap_err()); + write_lp(&db, &lp_data); let scenario1 = DBScenario { scenario_name: "Data in open chunk of mutable buffer".into(), @@ -251,16 +247,14 @@ impl DBSetup for EndToEndTest { /// Data in one only read buffer chunk pub(crate) async fn make_one_chunk_scenarios(partition_key: &str, data: &str) -> Vec { let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data).unwrap(); + write_lp(&db, data); let scenario1 = DBScenario { scenario_name: "Data in open chunk of mutable buffer".into(), db, }; let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data).unwrap(); + write_lp(&db, data); db.rollover_partition(partition_key).await.unwrap(); let scenario2 = DBScenario { scenario_name: "Data in closed chunk of mutable buffer".into(), @@ -268,8 +262,7 @@ pub(crate) async fn make_one_chunk_scenarios(partition_key: &str, data: &str) -> }; let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data).unwrap(); + write_lp(&db, data); db.rollover_partition(partition_key).await.unwrap(); db.load_chunk_to_read_buffer(partition_key, 0) .await @@ -294,9 +287,8 @@ pub async fn make_two_chunk_scenarios( data2: &str, ) -> Vec { let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data1).unwrap(); - writer.write_lp_string(&db, data2).unwrap(); + write_lp(&db, data1); + write_lp(&db, data2); let scenario1 = DBScenario { scenario_name: "Data in single open chunk of mutable buffer".into(), db, @@ -304,10 +296,9 @@ pub async fn make_two_chunk_scenarios( // spread across 2 mutable buffer chunks let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data1).unwrap(); + write_lp(&db, data1); db.rollover_partition(partition_key).await.unwrap(); - writer.write_lp_string(&db, data2).unwrap(); + write_lp(&db, data2); let scenario2 = DBScenario { scenario_name: "Data in one open chunk and one closed chunk of mutable buffer".into(), db, @@ -315,13 +306,12 @@ pub async fn make_two_chunk_scenarios( // spread across 1 mutable buffer, 1 read buffer chunks let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data1).unwrap(); + write_lp(&db, data1); db.rollover_partition(partition_key).await.unwrap(); db.load_chunk_to_read_buffer(partition_key, 0) .await .unwrap(); - writer.write_lp_string(&db, data2).unwrap(); + write_lp(&db, data2); let scenario3 = DBScenario { scenario_name: "Data in open chunk of mutable buffer, and one chunk of read buffer".into(), db, @@ -329,10 +319,9 @@ pub async fn make_two_chunk_scenarios( // in 2 read buffer chunks let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, data1).unwrap(); + write_lp(&db, data1); db.rollover_partition(partition_key).await.unwrap(); - writer.write_lp_string(&db, data2).unwrap(); + write_lp(&db, data2); db.rollover_partition(partition_key).await.unwrap(); db.load_chunk_to_read_buffer(partition_key, 0) diff --git a/server/src/query_tests/sql.rs b/server/src/query_tests/sql.rs index 5a69671c54..dfe3dc8f5e 100644 --- a/server/src/query_tests/sql.rs +++ b/server/src/query_tests/sql.rs @@ -291,13 +291,13 @@ async fn sql_select_from_system_tables() { "+---------------+------------+-------------+-------+", "| partition_key | table_name | column_name | count |", "+---------------+------------+-------------+-------+", - "| 1970-01-01T00 | h2o | state | 3 |", "| 1970-01-01T00 | h2o | city | 3 |", + "| 1970-01-01T00 | h2o | other_temp | 2 |", + "| 1970-01-01T00 | h2o | state | 3 |", "| 1970-01-01T00 | h2o | temp | 1 |", "| 1970-01-01T00 | h2o | time | 3 |", - "| 1970-01-01T00 | h2o | other_temp | 2 |", - "| 1970-01-01T00 | o2 | state | 2 |", "| 1970-01-01T00 | o2 | city | 1 |", + "| 1970-01-01T00 | o2 | state | 2 |", "| 1970-01-01T00 | o2 | temp | 2 |", "| 1970-01-01T00 | o2 | time | 2 |", "| 1970-01-01T00 | o2 | reading | 1 |", diff --git a/server/src/snapshot.rs b/server/src/snapshot.rs index 80f2737c5d..cc6136b1a6 100644 --- a/server/src/snapshot.rs +++ b/server/src/snapshot.rs @@ -273,12 +273,13 @@ mod tests { }; use super::*; + use crate::db::test_helpers::write_lp; use data_types::database_rules::DatabaseRules; use data_types::DatabaseName; use futures::TryStreamExt; use mutable_buffer::chunk::Chunk as ChunkWB; use object_store::memory::InMemory; - use query::{test::TestLPWriter, Database}; + use query::Database; use tracker::MemRegistry; #[tokio::test] @@ -291,8 +292,7 @@ mem,host=A,region=west used=45 1 "#; let db = make_db(); - let mut writer = TestLPWriter::default(); - writer.write_lp_string(&db, &lp).unwrap(); + write_lp(&db, &lp); let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); let (tx, rx) = tokio::sync::oneshot::channel(); From 4d22982b5f5664accc0c7196e8728d269b1dbf70 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 12 Apr 2021 16:00:31 -0400 Subject: [PATCH 15/45] fix: fix broken tests from column refactoring --- mutable_buffer/src/column.rs | 5 ++++- server/src/db.rs | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index ddd7ab9e12..7d822aa103 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -53,9 +53,10 @@ impl Column { match values { TypedValuesIterator::String(vals) => match logical_type { LogicalColumnType::Tag => { + let mut tag_values = vec![None; row_count]; let mut stats: Option> = None; - let tag_values: Vec<_> = vals + let mut added_tag_values: Vec<_> = vals .map(|tag| { tag.map(|tag| { match stats.as_mut() { @@ -70,6 +71,8 @@ impl Column { }) .collect(); + tag_values.append(&mut added_tag_values); + Self::Tag( tag_values, stats.expect("can't insert tag column with no values"), diff --git a/server/src/db.rs b/server/src/db.rs index 78df6c41ad..94d60de241 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -36,7 +36,7 @@ use super::{buffer::Buffer, JobRegistry}; use data_types::job::Job; use data_types::partition_metadata::TableSummary; -use internal_types::entry::{self, Entry, SequencedEntry, ClockValue}; +use internal_types::entry::{self, ClockValue, Entry, SequencedEntry}; use lifecycle::LifecycleManager; use system_tables::{SystemSchemaProvider, SYSTEM_SCHEMA}; From bd13c09bade06c86494d4c171d12bbba923a4c13 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 13 Apr 2021 08:28:48 -0400 Subject: [PATCH 16/45] refactor: make sharder optional when generating entry --- data_types/src/database_rules.rs | 15 ++---- internal_types/benches/benchmark.rs | 55 ++-------------------- internal_types/src/entry.rs | 73 +++++++++++++++++++---------- server/src/lib.rs | 8 +++- 4 files changed, 62 insertions(+), 89 deletions(-) diff --git a/data_types/src/database_rules.rs b/data_types/src/database_rules.rs index 4c61b432e1..c582d08fe3 100644 --- a/data_types/src/database_rules.rs +++ b/data_types/src/database_rules.rs @@ -743,22 +743,13 @@ impl TryFrom for TemplatePart { /// ShardId maps to a nodegroup that holds the the shard. pub type ShardId = u16; -const DEFAULT_SHARD_ID: u16 = 0; +pub const NO_SHARD_CONFIG: Option<&ShardConfig> = None; /// Assigns a given line to a specific shard id. pub trait Sharder { fn shard(&self, line: &ParsedLine<'_>) -> Result; } -impl Sharder for DatabaseRules { - fn shard(&self, line: &ParsedLine<'_>) -> Result { - match &self.shard_config { - Some(s) => s.shard(line), - None => Ok(DEFAULT_SHARD_ID), - } - } -} - /// ShardConfig defines rules for assigning a line/row to an individual /// host or a group of hosts. A shard /// is a logical concept, but the usage is meant to split data into @@ -787,8 +778,8 @@ pub struct ShardConfig { } impl Sharder for ShardConfig { - fn shard(&self, _line: &ParsedLine<'_>) -> Result { - unimplemented!() // TODO: mkm to implement as part of #916 + fn shard(&self, _line: &ParsedLine<'_>) -> Result { + todo!("mkm to implement as part of #916"); } } diff --git a/internal_types/benches/benchmark.rs b/internal_types/benches/benchmark.rs index 85c2d89d32..76a0ba520e 100644 --- a/internal_types/benches/benchmark.rs +++ b/internal_types/benches/benchmark.rs @@ -1,7 +1,6 @@ -use chrono::{DateTime, Utc}; use criterion::{criterion_group, criterion_main, Criterion}; -use data_types::database_rules::{Error as DataError, Partitioner, Sharder}; -use influxdb_line_protocol::ParsedLine; +use data_types::database_rules::ShardConfig; +use internal_types::entry::test_helpers::partitioner; use internal_types::entry::{lines_to_sharded_entries, ClockValue, SequencedEntry}; static LINES: &str = include_str!("../../tests/fixtures/lineproto/prometheus.lp"); @@ -12,7 +11,8 @@ fn sequenced_entry(c: &mut Criterion) { let lines = influxdb_line_protocol::parse_lines(LINES) .collect::, _>>() .unwrap(); - let sharded_entries = lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + let shard_config: Option<&ShardConfig> = None; + let sharded_entries = lines_to_sharded_entries(&lines, shard_config, &partitioner(1)).unwrap(); let entry = &sharded_entries.first().unwrap().entry; let data = entry.data(); assert_eq!( @@ -45,50 +45,3 @@ fn sequenced_entry(c: &mut Criterion) { criterion_group!(benches, sequenced_entry); criterion_main!(benches); - -fn sharder(count: u16) -> TestSharder { - TestSharder { - count, - n: std::cell::RefCell::new(0), - } -} - -// For each line passed to shard returns a shard id from [0, count) in order -struct TestSharder { - count: u16, - n: std::cell::RefCell, -} - -impl Sharder for TestSharder { - fn shard(&self, _line: &ParsedLine<'_>) -> Result { - let n = *self.n.borrow(); - self.n.replace(n + 1); - Ok(n % self.count) - } -} - -fn partitioner(count: u8) -> TestPartitioner { - TestPartitioner { - count, - n: std::cell::RefCell::new(0), - } -} - -// For each line passed to partition_key returns a key with a number from [0, -// count) -struct TestPartitioner { - count: u8, - n: std::cell::RefCell, -} - -impl Partitioner for TestPartitioner { - fn partition_key( - &self, - _line: &ParsedLine<'_>, - _default_time: &DateTime, - ) -> data_types::database_rules::Result { - let n = *self.n.borrow(); - self.n.replace(n + 1); - Ok(format!("key_{}", n % self.count)) - } -} diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index 315f8c5e63..6e5963690a 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -56,14 +56,17 @@ type ColumnResult = std::result::Result; /// underlying flatbuffers bytes generated. pub fn lines_to_sharded_entries( lines: &[ParsedLine<'_>], - sharder: &impl Sharder, + sharder: Option<&impl Sharder>, partitioner: &impl Partitioner, ) -> Result> { let default_time = Utc::now(); let mut sharded_lines = BTreeMap::new(); for line in lines { - let shard_id = sharder.shard(line).context(GeneratingShardId)?; + let shard_id = match &sharder { + Some(s) => Some(s.shard(line).context(GeneratingShardId)?), + None => None, + }; let partition_key = partitioner .partition_key(line, &default_time) .context(GeneratingPartitionKey)?; @@ -90,7 +93,7 @@ pub fn lines_to_sharded_entries( } fn build_sharded_entry( - shard_id: ShardId, + shard_id: Option, partitions: BTreeMap>>>, default_time: &DateTime, ) -> Result { @@ -277,10 +280,12 @@ fn build_table_write_batch<'a>( )) } -/// Holds a shard id to the associated entry +/// Holds a shard id to the associated entry. If there is no ShardId, then +/// everything goes to the same place. This means a single entry will be +/// generated from a batch of line protocol. #[derive(Debug)] pub struct ShardedEntry { - pub shard_id: ShardId, + pub shard_id: Option, pub entry: Entry, } @@ -1231,7 +1236,7 @@ pub mod test_helpers { pub fn lp_to_entry(lp: &str) -> Entry { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); - lines_to_sharded_entries(&lines, &sharder(1), &hour_partitioner()) + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &hour_partitioner()) .unwrap() .pop() .unwrap() @@ -1240,11 +1245,11 @@ pub mod test_helpers { /// Returns a test sharder that will assign shard ids from [0, count) /// incrementing for each line. - pub fn sharder(count: u16) -> TestSharder { - TestSharder { + pub fn sharder(count: u16) -> Option { + Some(TestSharder { count, n: std::cell::RefCell::new(0), - } + }) } // For each line passed to shard returns a shard id from [0, count) in order @@ -1324,6 +1329,7 @@ pub mod test_helpers { mod tests { use super::test_helpers::*; use super::*; + use data_types::database_rules::NO_SHARD_CONFIG; use influxdb_line_protocol::parse_lines; #[test] @@ -1337,11 +1343,28 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(2), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(2).as_ref(), &partitioner(1)).unwrap(); assert_eq!(sharded_entries.len(), 2); - assert_eq!(sharded_entries[0].shard_id, 0); - assert_eq!(sharded_entries[1].shard_id, 1); + assert_eq!(sharded_entries[0].shard_id, Some(0)); + assert_eq!(sharded_entries[1].shard_id, Some(1)); + } + + #[test] + fn no_shard_config() { + let lp = vec![ + "cpu,host=a,region=west user=23.1,system=66.1 123", + "mem,host=a,region=west used=23432 123", + "foo bar=true 21", + ] + .join("\n"); + let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); + + let sharded_entries = + lines_to_sharded_entries(&lines, NO_SHARD_CONFIG, &partitioner(1)).unwrap(); + + assert_eq!(sharded_entries.len(), 1); + assert_eq!(sharded_entries[0].shard_id, None); } #[test] @@ -1355,7 +1378,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(2)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(2)).unwrap(); let partition_writes = sharded_entries[0].entry.partition_writes().unwrap(); assert_eq!(partition_writes.len(), 2); @@ -1376,7 +1399,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries[0].entry.partition_writes().unwrap(); let table_batches = partition_writes[0].table_batches(); @@ -1393,7 +1416,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries[0].entry.partition_writes().unwrap(); let table_batches = partition_writes[0].table_batches(); @@ -1435,7 +1458,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries .first() @@ -1506,7 +1529,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries .first() @@ -1631,7 +1654,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries .first() .unwrap() @@ -1662,7 +1685,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries .first() .unwrap() @@ -1706,7 +1729,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries .first() .unwrap() @@ -1745,7 +1768,7 @@ mod tests { let t = Utc::now().timestamp_nanos(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let partition_writes = sharded_entries .first() @@ -1769,7 +1792,8 @@ mod tests { let lp = vec!["a val=1i 1", "a val=2.1 123"].join("\n"); let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); - let sharded_entries = lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)); + let sharded_entries = + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)); assert!(sharded_entries.is_err()); } @@ -1779,7 +1803,8 @@ mod tests { let lp = vec!["a,host=a val=1i 1", "a host=\"b\" 123"].join("\n"); let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); - let sharded_entries = lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)); + let sharded_entries = + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)); assert!(sharded_entries.is_err()); } @@ -1795,7 +1820,7 @@ mod tests { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); let sharded_entries = - lines_to_sharded_entries(&lines, &sharder(1), &partitioner(1)).unwrap(); + lines_to_sharded_entries(&lines, sharder(1).as_ref(), &partitioner(1)).unwrap(); let entry_bytes = sharded_entries.first().unwrap().entry.data(); let clock_value = ClockValue::new(23); diff --git a/server/src/lib.rs b/server/src/lib.rs index 371a197e41..fa25b149db 100644 --- a/server/src/lib.rs +++ b/server/src/lib.rs @@ -353,8 +353,12 @@ impl Server { .db(&db_name) .context(DatabaseNotFound { db_name: &*db_name })?; - let sharded_entries = lines_to_sharded_entries(lines, &*db.rules.read(), &*db.rules.read()) - .context(LineConversion)?; + let sharded_entries = lines_to_sharded_entries( + lines, + db.rules.read().shard_config.as_ref(), + &*db.rules.read(), + ) + .context(LineConversion)?; for e in sharded_entries { // TODO: handle sending to shards based on ShardConfig From 871de60bb7a48709c27f7b03b2a0fd86f341e21f Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 13 Apr 2021 08:43:24 -0400 Subject: [PATCH 17/45] chore: remove get_wal_meta test until it gets refactored --- src/influxdb_ioxd/http.rs | 116 +------------------------------------- 1 file changed, 2 insertions(+), 114 deletions(-) diff --git a/src/influxdb_ioxd/http.rs b/src/influxdb_ioxd/http.rs index 2462344a78..8ab7e23ff0 100644 --- a/src/influxdb_ioxd/http.rs +++ b/src/influxdb_ioxd/http.rs @@ -736,11 +736,7 @@ mod tests { use query::exec::Executor; use reqwest::{Client, Response}; - use data_types::{ - database_rules::{DatabaseRules, WalBufferConfig, WalBufferRollover}, - wal::WriterSummary, - DatabaseName, - }; + use data_types::{database_rules::DatabaseRules, DatabaseName}; use object_store::{memory::InMemory, ObjectStore}; use serde::de::DeserializeOwned; use server::{db::Db, ConnectionManagerImpl}; @@ -1103,115 +1099,6 @@ mod tests { .await; } - #[tokio::test] - async fn get_wal_meta() { - let server = Arc::new(AppServer::new( - ConnectionManagerImpl {}, - Arc::new(ObjectStore::new_in_memory(InMemory::new())), - )); - server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); - let server_url = test_server(Arc::clone(&server)); - - let database_name = "foo_bar"; - let rules = DatabaseRules { - name: DatabaseName::new(database_name).unwrap(), - partition_template: Default::default(), - wal_buffer_config: Some(WalBufferConfig { - buffer_size: 500, - segment_size: 10, - buffer_rollover: WalBufferRollover::ReturnError, - store_segments: true, - close_segment_after: None, - }), - lifecycle_rules: Default::default(), - shard_config: None, - }; - - server - .create_database( - rules, - server.require_id().unwrap(), - Arc::clone(&server.store), - ) - .await - .unwrap(); - - let base_url = format!( - "{}/iox/api/v1/databases/{}/wal/meta", - server_url, database_name - ); - - let client = Client::new(); - - let r1: WalMetadataResponse = check_json_response(&client, &base_url, StatusCode::OK).await; - - let lines: std::result::Result, _> = influxdb_line_protocol::parse_lines( - "cpu,host=A,region=west usage_system=64i 1590488773254420000", - ) - .collect(); - - server - .write_lines(database_name, &lines.unwrap()) - .await - .unwrap(); - - let r2: WalMetadataResponse = check_json_response(&client, &base_url, StatusCode::OK).await; - - let limit_1 = serde_urlencoded::to_string(&WalMetadataQuery { - limit: Some(1), - newer_than: None, - offset: None, - }) - .unwrap(); - let limit_url = format!("{}?{}", base_url, limit_1); - - let r3: WalMetadataResponse = - check_json_response(&client, &limit_url, StatusCode::OK).await; - - let limit_future = serde_urlencoded::to_string(&WalMetadataQuery { - limit: None, - offset: None, - newer_than: Some(chrono::Utc::now() + chrono::Duration::seconds(5)), - }) - .unwrap(); - let future_url = format!("{}?{}", base_url, limit_future); - - let r4: WalMetadataResponse = - check_json_response(&client, &future_url, StatusCode::OK).await; - - // No data written yet - expect no results - assert_eq!(r1.segments.len(), 1); - assert_eq!(r1.segments[0].size, 0); - assert_eq!(r1.segments[0].writers.len(), 0); - - // The WAL segment size is less than the line size - // We therefore expect an open and a closed segment in that order - // With the closed segment containing the written data - // And the open segment containing no data - assert_eq!(r2.segments.len(), 2); - assert_eq!(r2.segments[0].size, 0); - assert!(r2.segments[0].created_at >= r2.segments[1].created_at); - - assert!(r2.segments[1].persisted.is_none()); - assert_eq!(r2.segments[1].size, 368); - assert_eq!(r2.segments[1].writers.len(), 1); - assert_eq!( - r2.segments[1].writers.values().next().unwrap(), - &WriterSummary { - start_sequence: 1, - end_sequence: 1, - missing_sequence: false - } - ); - - // Query limited to a single segment - expect only the most recent segment - assert_eq!(r3.segments.len(), 1); - assert_eq!(r3.segments[0], r2.segments[0]); - - // Requesting segments from future - expect no results - assert_eq!(r4.segments.len(), 0); - } - fn get_content_type(response: &Result) -> String { if let Ok(response) = response { response @@ -1250,6 +1137,7 @@ mod tests { } } + #[allow(dead_code)] async fn check_json_response( client: &Client, url: &str, From ee46764a2deabab6da13202a62012ffd2f519147 Mon Sep 17 00:00:00 2001 From: Marko Mikulicic Date: Tue, 13 Apr 2021 16:56:18 +0200 Subject: [PATCH 18/45] fix: Use debug level to log requests Part of #1188 --- src/influxdb_ioxd/http.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/influxdb_ioxd/http.rs b/src/influxdb_ioxd/http.rs index 8ab7e23ff0..d41af8a518 100644 --- a/src/influxdb_ioxd/http.rs +++ b/src/influxdb_ioxd/http.rs @@ -32,7 +32,7 @@ use http::header::{CONTENT_ENCODING, CONTENT_TYPE}; use hyper::{Body, Method, Request, Response, StatusCode}; use observability_deps::{ opentelemetry::KeyValue, - tracing::{self, debug, error, info}, + tracing::{self, debug, error}, }; use routerify::{prelude::*, Middleware, RequestInfo, Router, RouterError, RouterService}; use serde::Deserialize; @@ -312,11 +312,11 @@ where Router::builder() .data(server) .middleware(Middleware::pre(|req| async move { - info!(request = ?req, "Processing request"); + debug!(request = ?req, "Processing request"); Ok(req) })) .middleware(Middleware::post(|res| async move { - info!(response = ?res, "Successfully processed request"); + debug!(response = ?res, "Successfully processed request"); Ok(res) })) // this endpoint is for API backward compatibility with InfluxDB 2.x .post("/api/v2/write", write::) From 55a77914b1dfd9ff4ff61093f4eaa5b082029db1 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Tue, 13 Apr 2021 18:10:28 +0100 Subject: [PATCH 19/45] feat: basic snapshot caching (#1184) --- Cargo.lock | 1 + mutable_buffer/Cargo.toml | 1 + mutable_buffer/src/chunk.rs | 50 +++++++++++++++++++++++++++++++++++-- 3 files changed, 50 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f78e995f8f..5c519f64dc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1885,6 +1885,7 @@ dependencies = [ "influxdb_line_protocol", "internal_types", "observability_deps", + "parking_lot", "snafu", "string-interner", "test_helpers", diff --git a/mutable_buffer/Cargo.toml b/mutable_buffer/Cargo.toml index 8f1a476c80..d3e70a9b53 100644 --- a/mutable_buffer/Cargo.toml +++ b/mutable_buffer/Cargo.toml @@ -24,6 +24,7 @@ generated_types = { path = "../generated_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } internal_types = { path = "../internal_types" } observability_deps = { path = "../observability_deps" } +parking_lot = "0.11.1" snafu = "0.6.2" string-interner = "0.12.2" tokio = { version = "1.0", features = ["macros"] } diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index 254c29c2d3..0c8bfccadf 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -21,6 +21,7 @@ use crate::{ pred::{ChunkPredicate, ChunkPredicateBuilder}, table::Table, }; +use parking_lot::Mutex; pub mod snapshot; @@ -129,6 +130,12 @@ pub struct Chunk { /// keep track of memory used by chunk tracker: MemTracker, + + /// Cached chunk snapshot + /// + /// Note: This is a mutex to allow mutation within + /// `Chunk::snapshot()` which only takes an immutable borrow + snapshot: Mutex>>, } impl Chunk { @@ -138,6 +145,7 @@ impl Chunk { dictionary: Dictionary::new(), tables: HashMap::new(), tracker: memory_registry.register(), + snapshot: Mutex::new(None), }; chunk.tracker.set_bytes(chunk.size()); chunk @@ -164,6 +172,12 @@ impl Chunk { .context(TableWrite { table_name })?; } + // Invalidate chunk snapshot + *self + .snapshot + .try_lock() + .expect("concurrent readers/writers to MBChunk") = None; + self.tracker.set_bytes(self.size()); Ok(()) @@ -181,8 +195,15 @@ impl Chunk { /// Returns a queryable snapshot of this chunk pub fn snapshot(&self) -> Arc { - // TODO: Cache this - Arc::new(ChunkSnapshot::new(self)) + let mut guard = self.snapshot.lock(); + if let Some(snapshot) = &*guard { + return Arc::clone(snapshot); + } + + // TODO: Incremental snapshot generation + let snapshot = Arc::new(ChunkSnapshot::new(self)); + *guard = Some(Arc::clone(&snapshot)); + snapshot } /// Return all the names of the tables names in this chunk that match @@ -593,6 +614,31 @@ mod tests { ); } + #[test] + fn test_snapshot() { + let mr = MemRegistry::new(); + let mut chunk = Chunk::new(1, &mr); + + let lp = vec![ + "cpu,host=a val=23 1", + "cpu,host=b val=2 1", + "mem,host=a val=23432i 1", + ] + .join("\n"); + + write_lp_to_chunk(&lp, &mut chunk).unwrap(); + let s1 = chunk.snapshot(); + let s2 = chunk.snapshot(); + + write_lp_to_chunk(&lp, &mut chunk).unwrap(); + let s3 = chunk.snapshot(); + let s4 = chunk.snapshot(); + + assert_eq!(Arc::as_ptr(&s1), Arc::as_ptr(&s2)); + assert_ne!(Arc::as_ptr(&s1), Arc::as_ptr(&s3)); + assert_eq!(Arc::as_ptr(&s3), Arc::as_ptr(&s4)); + } + fn assert_table(chunk: &Chunk, table: &str, data: &[&str]) { let mut batches = vec![]; chunk From a68137a661ddc33d77a4066e67bf6033b128077d Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Tue, 13 Apr 2021 13:47:10 -0400 Subject: [PATCH 20/45] feat: initial work for querying data from parquet file in object store --- Cargo.lock | 1 + parquet_file/Cargo.toml | 1 + parquet_file/src/chunk.rs | 41 +++++++++++++++++ parquet_file/src/table.rs | 94 ++++++++++++++++++++++++++++++++++++++- server/src/db/chunk.rs | 12 +++-- 5 files changed, 145 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f78e995f8f..f1961349d6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2331,6 +2331,7 @@ dependencies = [ "bytes", "data_types", "futures", + "internal_types", "object_store", "parking_lot", "snafu", diff --git a/parquet_file/Cargo.toml b/parquet_file/Cargo.toml index 015913d1d1..965056fb07 100644 --- a/parquet_file/Cargo.toml +++ b/parquet_file/Cargo.toml @@ -9,6 +9,7 @@ arrow_deps = { path = "../arrow_deps" } bytes = "1.0" data_types = { path = "../data_types" } futures = "0.3.7" +internal_types = {path = "../internal_types"} object_store = {path = "../object_store"} parking_lot = "0.11.1" snafu = "0.6" diff --git a/parquet_file/src/chunk.rs b/parquet_file/src/chunk.rs index 8f971924b7..d352cfd223 100644 --- a/parquet_file/src/chunk.rs +++ b/parquet_file/src/chunk.rs @@ -1,12 +1,37 @@ + +use snafu::{OptionExt, ResultExt, Snafu}; use std::collections::BTreeSet; use crate::table::Table; use data_types::partition_metadata::TableSummary; +use internal_types::{schema::Schema, selection::Selection}; use object_store::path::Path; use tracker::{MemRegistry, MemTracker}; + + use std::mem; +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("Error writing table '{}': {}", table_name, source))] + TableWrite { + table_name: String, + source: crate::table::Error, + }, + + #[snafu(display("Table Error in '{}': {}", table_name, source))] + NamedTableError { + table_name: String, + source: crate::table::Error, + }, + + #[snafu(display("Table '{}' not found in chunk {}", table_name, chunk_id))] + NamedTableNotFoundInChunk { table_name: String, chunk_id: u64 }, +} + +pub type Result = std::result::Result; + #[derive(Debug)] pub struct Chunk { /// Partition this chunk belongs to @@ -82,4 +107,20 @@ impl Chunk { size + self.partition_key.len() + mem::size_of::() + mem::size_of::() } + + /// Return Schema for the specified table / columns + pub fn table_schema(&self, table_name: &str, selection: Selection<'_>) -> Result { + let table = self + .tables + .iter() + .find(|t| t.has_table(table_name)) + .context(NamedTableNotFoundInChunk { + table_name, + chunk_id: self.id(), + })?; + + table + .schema(selection) + .context(NamedTableError { table_name }) + } } diff --git a/parquet_file/src/table.rs b/parquet_file/src/table.rs index 4f2776e8ea..f642c637ec 100644 --- a/parquet_file/src/table.rs +++ b/parquet_file/src/table.rs @@ -1,7 +1,33 @@ +use snafu::{ResultExt, Snafu}; +use std::mem; + use data_types::partition_metadata::TableSummary; +use internal_types::{schema::{builder::SchemaBuilder, Schema}, selection::Selection}; use object_store::path::Path; -use std::mem; +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("Error writing table '{}'", table_name))] + TableWrite { + table_name: String, + }, + + #[snafu(display("Table Error in '{}'", table_name))] + NamedTableError { + table_name: String, + }, + + #[snafu(display("Table '{}' not found in chunk {}", table_name, chunk_id))] + NamedTableNotFoundInChunk { table_name: String, chunk_id: u64 }, + + #[snafu(display("Internal error converting schema: {}", source))] + InternalSchema { + source: internal_types::schema::builder::Error, + }, +} + +pub type Result = std::result::Result; + /// Table that belongs to a chunk persisted in a parquet file in object store #[derive(Debug, Clone)] @@ -47,4 +73,70 @@ impl Table { pub fn path(&self) -> Path { self.object_store_path.clone() } + + + /// Return all columns of this table + // pub fn all_columns_selection(&self) -> Result> { + // // TODO + // let cols: Vec = vec![]; + // let selection = TableColSelection { cols }; + + // // sort so the columns always come out in a predictable name + // Ok(selection.sort_by_name()) + // } + + // /// Returns a column selection for just the specified columns + // fn specific_columns_selection<'a>( + // &self, + // columns: &'a [&'a str], + // ) -> Result> { + // // TODO + // let cols: Vec = vec![]; + + // Ok(TableColSelection { cols }) + // } + + pub fn schema(&self, selection: Selection<'_>) -> Result { + + let mut schema_builder = SchemaBuilder::new(); + + // // TODO: maybe just refactor MB's corresponding one + // for col in &selection.cols { + // let column_name = col.column_name; + // let column = self.column(col.column_id)?; + + // schema_builder = match column { + // Column::String(_, _) => schema_builder.field(column_name, ArrowDataType::Utf8), + // Column::Tag(_, _) => schema_builder.tag(column_name), + // Column::F64(_, _) => schema_builder.field(column_name, ArrowDataType::Float64), + // Column::I64(_, _) => { + // if column_name == TIME_COLUMN_NAME { + // schema_builder.timestamp() + // } else { + // schema_builder.field(column_name, ArrowDataType::Int64) + // } + // } + // Column::U64(_, _) => schema_builder.field(column_name, ArrowDataType::UInt64), + // Column::Bool(_, _) => schema_builder.field(column_name, ArrowDataType::Boolean), + // }; + // } + + schema_builder.build().context(InternalSchema) + + + // translate chunk selection into name/indexes: + // let selection = match selection { + // Selection::All => self.all_columns_selection(), + // Selection::Some(cols) => self.specific_columns_selection(cols), + // }?; + // self.schema_impl(&selection) + } + + // fn schema_impl(&self, selection: &TableColSelection<'_>) -> Result { + // let mut schema_builder = SchemaBuilder::new(); + + // // TODO: maybe just refactor MB's corresponding one + + // schema_builder.build().context(InternalSchema) + // } } diff --git a/server/src/db/chunk.rs b/server/src/db/chunk.rs index 07c979a04f..41efcde0be 100644 --- a/server/src/db/chunk.rs +++ b/server/src/db/chunk.rs @@ -28,6 +28,12 @@ pub enum Error { chunk_id: u32, }, + #[snafu(display("Parquet File Error in chunk {}: {}", chunk_id, source))] + ParquetFileChunkError { + source: parquet_file::chunk::Error, + chunk_id: u32, + }, + #[snafu(display("Internal error restricting schema: {}", source))] InternalSelectingSchema { source: internal_types::schema::Error, @@ -262,9 +268,9 @@ impl PartitionChunk for DBChunk { Ok(schema) } - Self::ParquetFile { .. } => { - unimplemented!("parquet file not implemented for table schema") - } + Self::ParquetFile { chunk, .. } => chunk + .table_schema(table_name, selection) + .context(ParquetFileChunkError{ chunk_id: chunk.id() }), } } From 4a6d6bd7addd3f554993c92e0802e5e662f64a21 Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Tue, 13 Apr 2021 13:47:10 -0400 Subject: [PATCH 21/45] feat: initial work for querying data from parquet file in object store --- Cargo.lock | 1 + parquet_file/Cargo.toml | 3 +- parquet_file/src/chunk.rs | 41 +++++++++++++++++ parquet_file/src/table.rs | 94 ++++++++++++++++++++++++++++++++++++++- server/src/db/chunk.rs | 12 +++-- 5 files changed, 146 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5c519f64dc..dabbf52f49 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2332,6 +2332,7 @@ dependencies = [ "bytes", "data_types", "futures", + "internal_types", "object_store", "parking_lot", "snafu", diff --git a/parquet_file/Cargo.toml b/parquet_file/Cargo.toml index 2750677be4..965056fb07 100644 --- a/parquet_file/Cargo.toml +++ b/parquet_file/Cargo.toml @@ -8,7 +8,8 @@ edition = "2018" arrow_deps = { path = "../arrow_deps" } bytes = "1.0" data_types = { path = "../data_types" } -futures = "0.3" +futures = "0.3.7" +internal_types = {path = "../internal_types"} object_store = {path = "../object_store"} parking_lot = "0.11.1" snafu = "0.6" diff --git a/parquet_file/src/chunk.rs b/parquet_file/src/chunk.rs index 8f971924b7..d352cfd223 100644 --- a/parquet_file/src/chunk.rs +++ b/parquet_file/src/chunk.rs @@ -1,12 +1,37 @@ + +use snafu::{OptionExt, ResultExt, Snafu}; use std::collections::BTreeSet; use crate::table::Table; use data_types::partition_metadata::TableSummary; +use internal_types::{schema::Schema, selection::Selection}; use object_store::path::Path; use tracker::{MemRegistry, MemTracker}; + + use std::mem; +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("Error writing table '{}': {}", table_name, source))] + TableWrite { + table_name: String, + source: crate::table::Error, + }, + + #[snafu(display("Table Error in '{}': {}", table_name, source))] + NamedTableError { + table_name: String, + source: crate::table::Error, + }, + + #[snafu(display("Table '{}' not found in chunk {}", table_name, chunk_id))] + NamedTableNotFoundInChunk { table_name: String, chunk_id: u64 }, +} + +pub type Result = std::result::Result; + #[derive(Debug)] pub struct Chunk { /// Partition this chunk belongs to @@ -82,4 +107,20 @@ impl Chunk { size + self.partition_key.len() + mem::size_of::() + mem::size_of::() } + + /// Return Schema for the specified table / columns + pub fn table_schema(&self, table_name: &str, selection: Selection<'_>) -> Result { + let table = self + .tables + .iter() + .find(|t| t.has_table(table_name)) + .context(NamedTableNotFoundInChunk { + table_name, + chunk_id: self.id(), + })?; + + table + .schema(selection) + .context(NamedTableError { table_name }) + } } diff --git a/parquet_file/src/table.rs b/parquet_file/src/table.rs index 4f2776e8ea..f642c637ec 100644 --- a/parquet_file/src/table.rs +++ b/parquet_file/src/table.rs @@ -1,7 +1,33 @@ +use snafu::{ResultExt, Snafu}; +use std::mem; + use data_types::partition_metadata::TableSummary; +use internal_types::{schema::{builder::SchemaBuilder, Schema}, selection::Selection}; use object_store::path::Path; -use std::mem; +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("Error writing table '{}'", table_name))] + TableWrite { + table_name: String, + }, + + #[snafu(display("Table Error in '{}'", table_name))] + NamedTableError { + table_name: String, + }, + + #[snafu(display("Table '{}' not found in chunk {}", table_name, chunk_id))] + NamedTableNotFoundInChunk { table_name: String, chunk_id: u64 }, + + #[snafu(display("Internal error converting schema: {}", source))] + InternalSchema { + source: internal_types::schema::builder::Error, + }, +} + +pub type Result = std::result::Result; + /// Table that belongs to a chunk persisted in a parquet file in object store #[derive(Debug, Clone)] @@ -47,4 +73,70 @@ impl Table { pub fn path(&self) -> Path { self.object_store_path.clone() } + + + /// Return all columns of this table + // pub fn all_columns_selection(&self) -> Result> { + // // TODO + // let cols: Vec = vec![]; + // let selection = TableColSelection { cols }; + + // // sort so the columns always come out in a predictable name + // Ok(selection.sort_by_name()) + // } + + // /// Returns a column selection for just the specified columns + // fn specific_columns_selection<'a>( + // &self, + // columns: &'a [&'a str], + // ) -> Result> { + // // TODO + // let cols: Vec = vec![]; + + // Ok(TableColSelection { cols }) + // } + + pub fn schema(&self, selection: Selection<'_>) -> Result { + + let mut schema_builder = SchemaBuilder::new(); + + // // TODO: maybe just refactor MB's corresponding one + // for col in &selection.cols { + // let column_name = col.column_name; + // let column = self.column(col.column_id)?; + + // schema_builder = match column { + // Column::String(_, _) => schema_builder.field(column_name, ArrowDataType::Utf8), + // Column::Tag(_, _) => schema_builder.tag(column_name), + // Column::F64(_, _) => schema_builder.field(column_name, ArrowDataType::Float64), + // Column::I64(_, _) => { + // if column_name == TIME_COLUMN_NAME { + // schema_builder.timestamp() + // } else { + // schema_builder.field(column_name, ArrowDataType::Int64) + // } + // } + // Column::U64(_, _) => schema_builder.field(column_name, ArrowDataType::UInt64), + // Column::Bool(_, _) => schema_builder.field(column_name, ArrowDataType::Boolean), + // }; + // } + + schema_builder.build().context(InternalSchema) + + + // translate chunk selection into name/indexes: + // let selection = match selection { + // Selection::All => self.all_columns_selection(), + // Selection::Some(cols) => self.specific_columns_selection(cols), + // }?; + // self.schema_impl(&selection) + } + + // fn schema_impl(&self, selection: &TableColSelection<'_>) -> Result { + // let mut schema_builder = SchemaBuilder::new(); + + // // TODO: maybe just refactor MB's corresponding one + + // schema_builder.build().context(InternalSchema) + // } } diff --git a/server/src/db/chunk.rs b/server/src/db/chunk.rs index 99898df8f6..f442360045 100644 --- a/server/src/db/chunk.rs +++ b/server/src/db/chunk.rs @@ -28,6 +28,12 @@ pub enum Error { chunk_id: u32, }, + #[snafu(display("Parquet File Error in chunk {}: {}", chunk_id, source))] + ParquetFileChunkError { + source: parquet_file::chunk::Error, + chunk_id: u32, + }, + #[snafu(display("Internal error restricting schema: {}", source))] InternalSelectingSchema { source: internal_types::schema::Error, @@ -212,9 +218,9 @@ impl PartitionChunk for DBChunk { Ok(schema) } - Self::ParquetFile { .. } => { - unimplemented!("parquet file not implemented for table schema") - } + Self::ParquetFile { chunk, .. } => chunk + .table_schema(table_name, selection) + .context(ParquetFileChunkError{ chunk_id: chunk.id() }), } } From 518df742dfd710e3cc4231140d36bcccc124fa2b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 13 Apr 2021 14:05:03 -0400 Subject: [PATCH 22/45] chore: update arrow deps (#1195) --- Cargo.lock | 76 +++++++++++++++++++++---------------------- arrow_deps/Cargo.toml | 10 +++--- 2 files changed, 43 insertions(+), 43 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5c519f64dc..6802e86d24 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -111,7 +111,7 @@ checksum = "23b62fc65de8e4e7f52534fb52b0f3ed04746ae267519eef2a83941e8085068b" [[package]] name = "arrow" version = "4.0.0-SNAPSHOT" -source = "git+https://github.com/apache/arrow.git?rev=e69478a890b1e4eee49b540b69b2711d170a0433#e69478a890b1e4eee49b540b69b2711d170a0433" +source = "git+https://github.com/apache/arrow.git?rev=00a443629c00079ea03c0b9f415d74669d2759a7#00a443629c00079ea03c0b9f415d74669d2759a7" dependencies = [ "cfg_aliases", "chrono", @@ -134,7 +134,7 @@ dependencies = [ [[package]] name = "arrow-flight" version = "4.0.0-SNAPSHOT" -source = "git+https://github.com/apache/arrow.git?rev=e69478a890b1e4eee49b540b69b2711d170a0433#e69478a890b1e4eee49b540b69b2711d170a0433" +source = "git+https://github.com/apache/arrow.git?rev=00a443629c00079ea03c0b9f415d74669d2759a7#00a443629c00079ea03c0b9f415d74669d2759a7" dependencies = [ "arrow", "bytes", @@ -429,9 +429,9 @@ dependencies = [ [[package]] name = "cast" -version = "0.2.3" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b9434b9a5aa1450faa3f9cb14ea0e8c53bb5d2b3c1bfd1ab4fc03e9f33fbfb0" +checksum = "cc38c385bfd7e444464011bb24820f40dd1c76bcdfa1b78611cb7c2e5cafab75" dependencies = [ "rustc_version", ] @@ -488,9 +488,9 @@ dependencies = [ [[package]] name = "clang-sys" -version = "1.1.1" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f54d78e30b388d4815220c8dd03fea5656b6c6d32adb59e89061552a102f8da1" +checksum = "853eda514c284c2287f4bf20ae614f8781f40a81d32ecda6e91449304dfe077c" dependencies = [ "glob", "libc", @@ -662,9 +662,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dca26ee1f8d361640700bde38b2c37d8c22b3ce2d360e1fc1c74ea4b0aa7d775" +checksum = "06ed27e177f16d65f0f0c22a213e17c696ace5dd64b14258b52f9417ccb52db4" dependencies = [ "cfg-if 1.0.0", "crossbeam-utils", @@ -787,7 +787,7 @@ dependencies = [ [[package]] name = "datafusion" version = "4.0.0-SNAPSHOT" -source = "git+https://github.com/apache/arrow.git?rev=e69478a890b1e4eee49b540b69b2711d170a0433#e69478a890b1e4eee49b540b69b2711d170a0433" +source = "git+https://github.com/apache/arrow.git?rev=00a443629c00079ea03c0b9f415d74669d2759a7#00a443629c00079ea03c0b9f415d74669d2759a7" dependencies = [ "ahash 0.7.2", "arrow", @@ -1044,9 +1044,9 @@ checksum = "2022715d62ab30faffd124d40b76f4134a550a87792276512b18d63272333394" [[package]] name = "futures" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f55667319111d593ba876406af7c409c0ebb44dc4be6132a783ccf163ea14c1" +checksum = "a9d5813545e459ad3ca1bff9915e9ad7f1a47dc6a91b627ce321d5863b7dd253" dependencies = [ "futures-channel", "futures-core", @@ -1059,9 +1059,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2dd2df839b57db9ab69c2c9d8f3e8c81984781937fe2807dc6dcf3b2ad2939" +checksum = "ce79c6a52a299137a6013061e0cf0e688fce5d7f1bc60125f520912fdb29ec25" dependencies = [ "futures-core", "futures-sink", @@ -1069,15 +1069,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15496a72fabf0e62bdc3df11a59a3787429221dd0710ba8ef163d6f7a9112c94" +checksum = "098cd1c6dda6ca01650f1a37a794245eb73181d0d4d4e955e2f3c37db7af1815" [[package]] name = "futures-executor" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "891a4b7b96d84d5940084b2a37632dd65deeae662c114ceaa2c879629c9c0ad1" +checksum = "10f6cb7042eda00f0049b1d2080aa4b93442997ee507eb3828e8bd7577f94c9d" dependencies = [ "futures-core", "futures-task", @@ -1086,15 +1086,15 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d71c2c65c57704c32f5241c1223167c2c3294fd34ac020c807ddbe6db287ba59" +checksum = "365a1a1fb30ea1c03a830fdb2158f5236833ac81fa0ad12fe35b29cddc35cb04" [[package]] name = "futures-macro" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea405816a5139fb39af82c2beb921d52143f556038378d6db21183a5c37fbfb7" +checksum = "668c6733a182cd7deb4f1de7ba3bf2120823835b3bcfbeacf7d2c4a773c1bb8b" dependencies = [ "proc-macro-hack", "proc-macro2", @@ -1104,21 +1104,21 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85754d98985841b7d4f5e8e6fbfa4a4ac847916893ec511a2917ccd8525b8bb3" +checksum = "5c5629433c555de3d82861a7a4e3794a4c40040390907cfbfd7143a92a426c23" [[package]] name = "futures-task" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa189ef211c15ee602667a6fcfe1c1fd9e07d42250d2156382820fba33c9df80" +checksum = "ba7aa51095076f3ba6d9a1f702f74bd05ec65f555d70d2033d55ba8d69f581bc" [[package]] name = "futures-test" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1fe5e51002528907757d5f1648101086f7197f792112db43ba23b06b09e6bce" +checksum = "e77baeade98824bc928c21b8ad39918b9d8a06745ebdb6e2c93fb7673fb7968d" dependencies = [ "futures-core", "futures-executor", @@ -1132,9 +1132,9 @@ dependencies = [ [[package]] name = "futures-util" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1812c7ab8aedf8d6f2701a43e1243acdbcc2b36ab26e2ad421eb99ac963d96d1" +checksum = "3c144ad54d60f23927f0a6b6d816e4271278b64f005ad65e4e35291d2de9c025" dependencies = [ "futures-channel", "futures-core", @@ -2299,7 +2299,7 @@ dependencies = [ [[package]] name = "parquet" version = "4.0.0-SNAPSHOT" -source = "git+https://github.com/apache/arrow.git?rev=e69478a890b1e4eee49b540b69b2711d170a0433#e69478a890b1e4eee49b540b69b2711d170a0433" +source = "git+https://github.com/apache/arrow.git?rev=00a443629c00079ea03c0b9f415d74669d2759a7#00a443629c00079ea03c0b9f415d74669d2759a7" dependencies = [ "arrow", "base64 0.12.3", @@ -2881,9 +2881,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf12057f289428dbf5c591c74bf10392e4a8003f993405a902f20117019022d4" +checksum = "2296f2fac53979e8ccbc4a1136b25dcefd37be9ed7e4a1f6b05a6029c84ff124" dependencies = [ "base64 0.13.0", "bytes", @@ -3118,9 +3118,9 @@ checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" [[package]] name = "sct" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3042af939fca8c3453b7af0f1c66e533a15a86169e39de2657310ade8f98d3c" +checksum = "b362b83898e0e69f38515b82ee15aa80636befe47c3b6d3d89a911e78fc228ce" dependencies = [ "ring", "untrusted", @@ -3758,9 +3758,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "134af885d758d645f0f0505c9a8b3f9bf8a348fd822e112ab5248138348f1722" +checksum = "83f0c8e7c0addab50b663055baf787d0af7f413a46e6e7fb9559a4e4db7137a5" dependencies = [ "autocfg", "bytes", @@ -3821,9 +3821,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.5" +version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5143d049e85af7fbc36f5454d990e62c2df705b3589f123b71f441b6b59f443f" +checksum = "940a12c99365c31ea8dd9ba04ec1be183ffe4920102bb7122c2f515437601e8e" dependencies = [ "bytes", "futures-core", diff --git a/arrow_deps/Cargo.toml b/arrow_deps/Cargo.toml index ba8263f2d6..5466195a9f 100644 --- a/arrow_deps/Cargo.toml +++ b/arrow_deps/Cargo.toml @@ -8,14 +8,14 @@ description = "Apache Arrow / Parquet / DataFusion dependencies for InfluxDB IOx [dependencies] # In alphabetical order # We are using development version of arrow/parquet/datafusion and the dependencies are at the same rev -# The version can be found here: https://github.com/apache/arrow/commit/e69478a890b1e4eee49b540b69b2711d170a0433 +# The version can be found here: https://github.com/apache/arrow/commit/00a443629c00079ea03c0b9f415d74669d2759a7 # -arrow = { git = "https://github.com/apache/arrow.git", rev = "e69478a890b1e4eee49b540b69b2711d170a0433" , features = ["simd"] } -arrow-flight = { git = "https://github.com/apache/arrow.git", rev = "e69478a890b1e4eee49b540b69b2711d170a0433" } +arrow = { git = "https://github.com/apache/arrow.git", rev = "00a443629c00079ea03c0b9f415d74669d2759a7" , features = ["simd"] } +arrow-flight = { git = "https://github.com/apache/arrow.git", rev = "00a443629c00079ea03c0b9f415d74669d2759a7" } # Turn off optional datafusion features (function packages) -datafusion = { git = "https://github.com/apache/arrow.git", rev = "e69478a890b1e4eee49b540b69b2711d170a0433", default-features = false } +datafusion = { git = "https://github.com/apache/arrow.git", rev = "00a443629c00079ea03c0b9f415d74669d2759a7", default-features = false } # Turn off the "arrow" feature; it currently has a bug that causes the crate to rebuild every time # and we're not currently using it anyway -parquet = { git = "https://github.com/apache/arrow.git", rev = "e69478a890b1e4eee49b540b69b2711d170a0433", default-features = false, features = ["snap", "brotli", "flate2", "lz4", "zstd"] } +parquet = { git = "https://github.com/apache/arrow.git", rev = "00a443629c00079ea03c0b9f415d74669d2759a7", default-features = false, features = ["snap", "brotli", "flate2", "lz4", "zstd"] } From 61cd745ab67a083f7a647fd1a27d62a95e7c80ec Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Tue, 13 Apr 2021 20:09:36 +0100 Subject: [PATCH 23/45] refactor: remove mutable buffer predicate logic (#1186) Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- mutable_buffer/src/chunk.rs | 279 +----------------------------------- mutable_buffer/src/lib.rs | 1 - mutable_buffer/src/table.rs | 240 +------------------------------ server/src/db/pred.rs | 30 ---- server/src/db/streams.rs | 102 +------------ 5 files changed, 3 insertions(+), 649 deletions(-) diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index 0c8bfccadf..4359a8c276 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -5,20 +5,17 @@ use std::sync::Arc; use snafu::{OptionExt, ResultExt, Snafu}; -use arrow_deps::{arrow::record_batch::RecordBatch, datafusion::logical_plan::Expr}; +use arrow_deps::arrow::record_batch::RecordBatch; use data_types::{database_rules::WriterId, partition_metadata::TableSummary}; use internal_types::{ entry::{ClockValue, TableBatch}, - schema::Schema, selection::Selection, }; use tracker::{MemRegistry, MemTracker}; use crate::chunk::snapshot::ChunkSnapshot; use crate::{ - column::Column, dictionary::{Dictionary, Error as DictionaryError}, - pred::{ChunkPredicate, ChunkPredicateBuilder}, table::Table, }; use parking_lot::Mutex; @@ -39,54 +36,9 @@ pub enum Error { source: crate::table::Error, }, - #[snafu(display("Error checking predicate in table {}: {}", table_id, source))] - PredicateCheck { - table_id: u32, - source: crate::table::Error, - }, - - #[snafu(display("Error checking predicate in table '{}': {}", table_name, source))] - NamedTablePredicateCheck { - table_name: String, - source: crate::table::Error, - }, - - #[snafu(display( - "Unsupported predicate when mutable buffer table names. Found a general expression: {:?}", - exprs - ))] - PredicateNotYetSupported { exprs: Vec }, - - #[snafu(display("Table ID {} not found in dictionary of chunk {}", table_id, chunk))] - TableIdNotFoundInDictionary { - table_id: u32, - chunk: u64, - source: DictionaryError, - }, - - #[snafu(display( - "Internal error: table {} not found in dictionary of chunk {}", - table_name, - chunk_id - ))] - InternalTableNotFoundInDictionary { table_name: String, chunk_id: u32 }, - #[snafu(display("Table {} not found in chunk {}", table, chunk))] TableNotFoundInChunk { table: u32, chunk: u64 }, - #[snafu(display("Table '{}' not found in chunk {}", table_name, chunk_id))] - NamedTableNotFoundInChunk { table_name: String, chunk_id: u64 }, - - #[snafu(display("Attempt to write table batch without a name"))] - TableWriteWithoutName, - - #[snafu(display("Value ID {} not found in dictionary of chunk {}", value_id, chunk_id))] - InternalColumnValueIdNotFoundInDictionary { - value_id: u32, - chunk_id: u64, - source: DictionaryError, - }, - #[snafu(display("Column ID {} not found in dictionary of chunk {}", column_id, chunk))] ColumnIdNotFoundInDictionary { column_id: u32, @@ -104,12 +56,6 @@ pub enum Error { chunk_id: u64, source: DictionaryError, }, - - #[snafu(display( - "Column '{}' is not a string tag column and thus can not list values", - column_name - ))] - UnsupportedColumnTypeForListingValues { column_name: String }, } pub type Result = std::result::Result; @@ -206,214 +152,6 @@ impl Chunk { snapshot } - /// Return all the names of the tables names in this chunk that match - /// chunk predicate - pub fn table_names(&self, chunk_predicate: &ChunkPredicate) -> Result> { - // we don't support arbitrary expressions in chunk predicate yet - if !chunk_predicate.chunk_exprs.is_empty() { - return PredicateNotYetSupported { - exprs: chunk_predicate.chunk_exprs.clone(), - } - .fail(); - } - - self.tables - .iter() - .filter_map(|(&table_id, table)| { - // could match is good enough for this metadata query - match table.could_match_predicate(chunk_predicate) { - Ok(true) => Some(self.dictionary.lookup_id(table_id).context( - TableIdNotFoundInDictionary { - table_id, - chunk: self.id, - }, - )), - Ok(false) => None, - Err(e) => Some(Err(e).context(PredicateCheck { table_id })), - } - }) - .collect() - } - - /// If the column names that match the predicate can be found - /// from the predicate entirely using metadata, return those - /// strings. - /// - /// If the predicate cannot be evaluated entirely with - /// metadata, return `Ok(None)`. - pub fn column_names( - &self, - table_name: &str, - chunk_predicate: &ChunkPredicate, - selection: Selection<'_>, - ) -> Result>> { - // No support for general purpose expressions - if !chunk_predicate.chunk_exprs.is_empty() { - return Ok(None); - } - - let table_name_id = self.table_name_id(table_name)?; - - let mut chunk_column_ids = BTreeSet::new(); - - // Is this table in the chunk? - if let Some(table) = self.tables.get(&table_name_id) { - for (&column_id, column) in &table.columns { - let column_matches_predicate = table - .column_matches_predicate(&column, chunk_predicate) - .context(NamedTableError { table_name })?; - - if column_matches_predicate { - chunk_column_ids.insert(column_id); - } - } - } - - // Only return subset of these selection_cols if not all_cols - let mut all_cols = true; - let selection_cols = match selection { - Selection::All => &[""], - Selection::Some(cols) => { - all_cols = false; - cols - } - }; - - let mut column_names = BTreeSet::new(); - for &column_id in &chunk_column_ids { - let column_name = - self.dictionary - .lookup_id(column_id) - .context(ColumnIdNotFoundInDictionary { - column_id, - chunk: self.id, - })?; - - if !column_names.contains(column_name) - && (all_cols || selection_cols.contains(&column_name)) - { - // only use columns in selection_cols - column_names.insert(column_name.to_string()); - } - } - - Ok(Some(column_names)) - } - - /// Return the id of the table in the chunk's dictionary - fn table_name_id(&self, table_name: &str) -> Result { - self.dictionary - .id(table_name) - .context(InternalTableNotFoundInDictionary { - table_name, - chunk_id: self.id(), - }) - } - - /// Returns the strings of the specified Tag column that satisfy - /// the predicate, if they can be determined entirely using metadata. - /// - /// If the predicate cannot be evaluated entirely with metadata, - /// return `Ok(None)`. - pub fn tag_column_values( - &self, - table_name: &str, - column_name: &str, - chunk_predicate: &ChunkPredicate, - ) -> Result>> { - // No support for general purpose expressions - if !chunk_predicate.chunk_exprs.is_empty() { - return Ok(None); - } - let chunk_id = self.id(); - - let table_name_id = self.table_name_id(table_name)?; - - // Is this table even in the chunk? - let table = self - .tables - .get(&table_name_id) - .context(NamedTableNotFoundInChunk { - table_name, - chunk_id, - })?; - - // See if we can rule out the table entire on metadata - let could_match = table - .could_match_predicate(chunk_predicate) - .context(NamedTablePredicateCheck { table_name })?; - - if !could_match { - // No columns could match, return empty set - return Ok(Default::default()); - } - - let column_id = - self.dictionary - .lookup_value(column_name) - .context(ColumnNameNotFoundInDictionary { - column_name, - chunk_id, - })?; - - let column = table - .column(column_id) - .context(NamedTableError { table_name })?; - - if let Column::Tag(column, _) = column { - // if we have a timestamp predicate, find all values - // where the timestamp is within range. Otherwise take - // all values. - - // Collect matching ids into BTreeSet to deduplicate on - // ids *before* looking up Strings - let column_value_ids: BTreeSet = match chunk_predicate.range { - None => { - // take all non-null values - column.iter().filter_map(|&s| s).collect() - } - Some(range) => { - // filter out all values that don't match the timestmap - let time_column = table - .column_i64(chunk_predicate.time_column_id) - .context(NamedTableError { table_name })?; - - column - .iter() - .zip(time_column.iter()) - .filter_map(|(&column_value_id, ×tamp_value)| { - if range.contains_opt(timestamp_value) { - column_value_id - } else { - None - } - }) - .collect() - } - }; - - // convert all the (deduplicated) ids to Strings - let column_values = column_value_ids - .into_iter() - .map(|value_id| { - let value = self.dictionary.lookup_id(value_id).context( - InternalColumnValueIdNotFoundInDictionary { value_id, chunk_id }, - )?; - Ok(value.to_string()) - }) - .collect::>>()?; - - Ok(Some(column_values)) - } else { - UnsupportedColumnTypeForListingValues { column_name }.fail() - } - } - - /// Return a builder suitable to create predicates for this Chunk - pub fn predicate_builder(&self) -> Result, crate::pred::Error> { - ChunkPredicateBuilder::new(&self.dictionary) - } - /// returns true if there is no data in this chunk pub fn is_empty(&self) -> bool { self.tables.is_empty() @@ -474,21 +212,6 @@ impl Chunk { Ok(table) } - /// Return Schema for the specified table / columns - pub fn table_schema(&self, table_name: &str, selection: Selection<'_>) -> Result { - let table = self - .table(table_name)? - // Option --> Result - .context(NamedTableNotFoundInChunk { - table_name, - chunk_id: self.id(), - })?; - - table - .schema(self, selection) - .context(NamedTableError { table_name }) - } - /// Return the approximate memory size of the chunk, in bytes including the /// dictionary, tables, and their rows. pub fn size(&self) -> usize { diff --git a/mutable_buffer/src/lib.rs b/mutable_buffer/src/lib.rs index d2e8b64af1..0fd90cc61d 100644 --- a/mutable_buffer/src/lib.rs +++ b/mutable_buffer/src/lib.rs @@ -60,5 +60,4 @@ pub mod chunk; mod column; mod dictionary; -pub mod pred; mod table; diff --git a/mutable_buffer/src/table.rs b/mutable_buffer/src/table.rs index e522eeadba..5439f9f054 100644 --- a/mutable_buffer/src/table.rs +++ b/mutable_buffer/src/table.rs @@ -1,15 +1,10 @@ -use std::{ - cmp, - collections::{BTreeMap, BTreeSet}, - sync::Arc, -}; +use std::{cmp, collections::BTreeMap, sync::Arc}; use crate::{ chunk::Chunk, column, column::Column, dictionary::{Dictionary, Error as DictionaryError}, - pred::{ChunkIdSet, ChunkPredicate}, }; use data_types::{ database_rules::WriterId, @@ -473,118 +468,6 @@ impl Table { RecordBatch::try_new(schema, columns).context(ArrowError {}) } - /// returns true if any row in this table could possible match the - /// predicate. true does not mean any rows will *actually* match, - /// just that the entire table can not be ruled out. - /// - /// false means that no rows in this table could possibly match - pub fn could_match_predicate(&self, chunk_predicate: &ChunkPredicate) -> Result { - Ok( - self.matches_column_name_predicate(chunk_predicate.field_name_predicate.as_ref()) - && self.matches_table_name_predicate(chunk_predicate.table_name_predicate.as_ref()) - && self.matches_timestamp_predicate(chunk_predicate)? - && self.has_columns(chunk_predicate.required_columns.as_ref()), - ) - } - - /// Returns true if the table contains any of the field columns - /// requested or there are no specific fields requested. - fn matches_column_name_predicate(&self, column_selection: Option<&BTreeSet>) -> bool { - match column_selection { - Some(column_selection) => { - for column_id in column_selection { - if let Some(column) = self.columns.get(column_id) { - if !column.is_tag() { - return true; - } - } - } - - // selection only had tag columns - false - } - None => true, // no specific selection - } - } - - fn matches_table_name_predicate(&self, table_name_predicate: Option<&BTreeSet>) -> bool { - match table_name_predicate { - Some(table_name_predicate) => table_name_predicate.contains(&self.id), - None => true, // no table predicate - } - } - - /// returns true if there are any timestamps in this table that - /// fall within the timestamp range - fn matches_timestamp_predicate(&self, chunk_predicate: &ChunkPredicate) -> Result { - match &chunk_predicate.range { - None => Ok(true), - Some(range) => { - let time_column_id = chunk_predicate.time_column_id; - let time_column = self.column(time_column_id)?; - time_column.has_i64_range(range.start, range.end).context( - ColumnPredicateEvaluation { - column: time_column_id, - }, - ) - } - } - } - - /// returns true if no columns are specified, or the table has all - /// columns specified - fn has_columns(&self, columns: Option<&ChunkIdSet>) -> bool { - if let Some(columns) = columns { - match columns { - ChunkIdSet::AtLeastOneMissing => return false, - ChunkIdSet::Present(symbols) => { - for symbol in symbols { - if !self.columns.contains_key(symbol) { - return false; - } - } - } - } - } - true - } - - /// returns true if there are any rows in column that are non-null - /// and within the timestamp range specified by pred - pub(crate) fn column_matches_predicate( - &self, - column: &Column, - chunk_predicate: &ChunkPredicate, - ) -> Result { - match column { - Column::F64(v, _) => self.column_value_matches_predicate(v, chunk_predicate), - Column::I64(v, _) => self.column_value_matches_predicate(v, chunk_predicate), - Column::U64(v, _) => self.column_value_matches_predicate(v, chunk_predicate), - Column::String(v, _) => self.column_value_matches_predicate(v, chunk_predicate), - Column::Bool(v, _) => self.column_value_matches_predicate(v, chunk_predicate), - Column::Tag(v, _) => self.column_value_matches_predicate(v, chunk_predicate), - } - } - - fn column_value_matches_predicate( - &self, - column_value: &[Option], - chunk_predicate: &ChunkPredicate, - ) -> Result { - match chunk_predicate.range { - None => Ok(true), - Some(range) => { - let time_column_id = chunk_predicate.time_column_id; - let time_column = self.column(time_column_id)?; - time_column - .has_non_null_i64_range(column_value, range.start, range.end) - .context(ColumnPredicateEvaluation { - column: time_column_id, - }) - } - } - } - pub fn stats(&self, chunk: &Chunk) -> Vec { self.columns .iter() @@ -639,49 +522,6 @@ mod tests { use super::*; use tracker::MemRegistry; - #[test] - fn test_has_columns() { - let registry = Arc::new(MemRegistry::new()); - let mut chunk = Chunk::new(42, registry.as_ref()); - let dictionary = &mut chunk.dictionary; - let mut table = Table::new(dictionary.lookup_value_or_insert("table_name")); - - let lp_lines = vec![ - "h2o,state=MA,city=Boston temp=70.4 100", - "h2o,state=MA,city=Boston temp=72.4 250", - ]; - - write_lines_to_table(&mut table, dictionary, lp_lines); - - let state_symbol = dictionary.id("state").unwrap(); - let new_symbol = dictionary.lookup_value_or_insert("not_a_columns"); - - assert!(table.has_columns(None)); - - let pred = ChunkIdSet::AtLeastOneMissing; - assert!(!table.has_columns(Some(&pred))); - - let set = BTreeSet::::new(); - let pred = ChunkIdSet::Present(set); - assert!(table.has_columns(Some(&pred))); - - let mut set = BTreeSet::new(); - set.insert(state_symbol); - let pred = ChunkIdSet::Present(set); - assert!(table.has_columns(Some(&pred))); - - let mut set = BTreeSet::new(); - set.insert(new_symbol); - let pred = ChunkIdSet::Present(set); - assert!(!table.has_columns(Some(&pred))); - - let mut set = BTreeSet::new(); - set.insert(state_symbol); - set.insert(new_symbol); - let pred = ChunkIdSet::Present(set); - assert!(!table.has_columns(Some(&pred))); - } - #[test] fn table_size() { let registry = Arc::new(MemRegistry::new()); @@ -706,84 +546,6 @@ mod tests { assert_eq!(320, table.size()); } - #[test] - fn test_matches_table_name_predicate() { - let registry = Arc::new(MemRegistry::new()); - let mut chunk = Chunk::new(42, registry.as_ref()); - let dictionary = &mut chunk.dictionary; - let mut table = Table::new(dictionary.lookup_value_or_insert("h2o")); - - let lp_lines = vec![ - "h2o,state=MA,city=Boston temp=70.4 100", - "h2o,state=MA,city=Boston temp=72.4 250", - ]; - write_lines_to_table(&mut table, dictionary, lp_lines); - - let h2o_symbol = dictionary.id("h2o").unwrap(); - - assert!(table.matches_table_name_predicate(None)); - - let set = BTreeSet::new(); - assert!(!table.matches_table_name_predicate(Some(&set))); - - let mut set = BTreeSet::new(); - set.insert(h2o_symbol); - assert!(table.matches_table_name_predicate(Some(&set))); - - // Some symbol that is not the same as h2o_symbol - assert_ne!(37377, h2o_symbol); - let mut set = BTreeSet::new(); - set.insert(37377); - assert!(!table.matches_table_name_predicate(Some(&set))); - } - - #[test] - fn test_matches_column_name_predicate() { - let registry = Arc::new(MemRegistry::new()); - let mut chunk = Chunk::new(42, registry.as_ref()); - let dictionary = &mut chunk.dictionary; - let mut table = Table::new(dictionary.lookup_value_or_insert("h2o")); - - let lp_lines = vec![ - "h2o,state=MA,city=Boston temp=70.4,awesomeness=1000 100", - "h2o,state=MA,city=Boston temp=72.4,awesomeness=2000 250", - ]; - write_lines_to_table(&mut table, dictionary, lp_lines); - - let state_symbol = dictionary.id("state").unwrap(); - let temp_symbol = dictionary.id("temp").unwrap(); - let awesomeness_symbol = dictionary.id("awesomeness").unwrap(); - - assert!(table.matches_column_name_predicate(None)); - - let set = BTreeSet::new(); - assert!(!table.matches_column_name_predicate(Some(&set))); - - // tag columns should not count - let mut set = BTreeSet::new(); - set.insert(state_symbol); - assert!(!table.matches_column_name_predicate(Some(&set))); - - let mut set = BTreeSet::new(); - set.insert(temp_symbol); - assert!(table.matches_column_name_predicate(Some(&set))); - - let mut set = BTreeSet::new(); - set.insert(temp_symbol); - set.insert(awesomeness_symbol); - assert!(table.matches_column_name_predicate(Some(&set))); - - let mut set = BTreeSet::new(); - set.insert(temp_symbol); - set.insert(awesomeness_symbol); - set.insert(1337); // some other symbol, but that is ok - assert!(table.matches_column_name_predicate(Some(&set))); - - let mut set = BTreeSet::new(); - set.insert(1337); - assert!(!table.matches_column_name_predicate(Some(&set))); - } - #[test] fn test_to_arrow_schema_all() { let registry = Arc::new(MemRegistry::new()); diff --git a/server/src/db/pred.rs b/server/src/db/pred.rs index 02276d0d31..ec8f469f80 100644 --- a/server/src/db/pred.rs +++ b/server/src/db/pred.rs @@ -3,7 +3,6 @@ use std::convert::TryFrom; -use mutable_buffer::{chunk::Chunk, pred::ChunkPredicate}; use query::predicate::Predicate; use snafu::Snafu; @@ -11,15 +10,6 @@ use snafu::Snafu; pub enum Error { #[snafu(display("Error translating predicate: {}", msg))] ReadBufferPredicate { msg: String, pred: Predicate }, - - #[snafu(display("Error building predicate for mutable buffer: {}", source))] - MutableBufferPredicate { source: mutable_buffer::pred::Error }, -} - -impl From for Error { - fn from(source: mutable_buffer::pred::Error) -> Self { - Self::MutableBufferPredicate { source } - } } pub type Result = std::result::Result; @@ -52,25 +42,6 @@ pub fn to_read_buffer_predicate(predicate: &Predicate) -> Result, - predicate: &Predicate, -) -> Result { - let predicate = chunk - .as_ref() - .predicate_builder()? - .table_names(predicate.table_names.as_ref())? - .field_names(predicate.field_columns.as_ref())? - .range(predicate.range)? - // it would be nice to avoid cloning all the exprs here. - .exprs(predicate.exprs.clone())? - .build(); - - Ok(predicate) -} - #[cfg(test)] pub mod test { use super::*; @@ -196,7 +167,6 @@ pub mod test { Error::ReadBufferPredicate { msg, pred: _ } => { assert_eq!(msg, exp.to_owned()); } - _ => panic!("Unexpected error type"), } } } diff --git a/server/src/db/streams.rs b/server/src/db/streams.rs index b421c018a2..8aa5243296 100644 --- a/server/src/db/streams.rs +++ b/server/src/db/streams.rs @@ -1,15 +1,9 @@ //! Adapter streams for different Chunk types that implement the interface //! needed by DataFusion use arrow_deps::{ - arrow::{ - datatypes::SchemaRef, - error::{ArrowError, Result as ArrowResult}, - record_batch::RecordBatch, - }, + arrow::{datatypes::SchemaRef, error::Result as ArrowResult, record_batch::RecordBatch}, datafusion::physical_plan::RecordBatchStream, }; -use internal_types::selection::Selection; -use mutable_buffer::chunk::Chunk as MBChunk; use read_buffer::ReadFilterResults; use std::{ @@ -17,100 +11,6 @@ use std::{ task::{Context, Poll}, }; -use snafu::{ResultExt, Snafu}; - -#[derive(Debug, Snafu)] -pub enum Error { - #[snafu(display( - "Error getting data for table '{}' chunk {}: {}", - table_name, - chunk_id, - source - ))] - GettingTableData { - table_name: String, - chunk_id: u32, - source: mutable_buffer::chunk::Error, - }, -} - -/// Adapter which will produce record batches from a mutable buffer -/// chunk on demand -pub(crate) struct MutableBufferChunkStream { - /// Requested output schema (includes selection) - schema: SchemaRef, - chunk: Arc, - table_name: Arc, - - /// Vector of record batches to send in reverse order (send data[len-1] - /// next) Is None until the first call to poll_next - data: Option>, -} - -impl MutableBufferChunkStream { - #[allow(dead_code)] - pub fn new(chunk: Arc, schema: SchemaRef, table_name: impl Into) -> Self { - Self { - chunk, - schema, - table_name: Arc::new(table_name.into()), - data: None, - } - } - - // gets the next batch, as needed - fn next_batch(&mut self) -> ArrowResult> { - if self.data.is_none() { - // Want all the columns in the schema. Note we don't - // use `Selection::All` here because the mutable buffer chunk would interpret it - // as "all columns in the table in that chunk" rather than - // all columns this query needs - let selected_cols = self - .schema - .fields() - .iter() - .map(|f| f.name() as &str) - .collect::>(); - let selection = Selection::Some(&selected_cols); - - let mut data = Vec::new(); - self.chunk - .table_to_arrow(&mut data, self.table_name.as_ref(), selection) - .context(GettingTableData { - table_name: self.table_name.as_ref(), - chunk_id: self.chunk.id(), - }) - .map_err(|e| ArrowError::ExternalError(Box::new(e)))?; - - // reverse the array so we can pop off the back - data.reverse(); - self.data = Some(data); - } - - // self.data was set to Some above - Ok(self.data.as_mut().unwrap().pop()) - } -} - -impl RecordBatchStream for MutableBufferChunkStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } -} - -impl futures::Stream for MutableBufferChunkStream { - type Item = ArrowResult; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - _: &mut Context<'_>, - ) -> Poll> { - Poll::Ready(self.next_batch().transpose()) - } - - // TODO is there a useful size_hint to pass? -} - /// Adapter which will take a ReadFilterResults and make it an async stream pub struct ReadFilterResultsStream { read_results: ReadFilterResults, From 7ef490694c7de3aadb5d2fbca7f1244f4230eb29 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Tue, 13 Apr 2021 21:59:41 +0100 Subject: [PATCH 24/45] refactor: reduce module coupling in mutable buffer (#1199) * refactor: reduce module coupling in mutable buffer * refactor: tweak visibility * chore: formatting --- mutable_buffer/src/chunk.rs | 10 +- mutable_buffer/src/chunk/snapshot.rs | 4 +- mutable_buffer/src/column.rs | 185 ----------------- mutable_buffer/src/pred.rs | 298 --------------------------- mutable_buffer/src/table.rs | 145 +++++-------- 5 files changed, 61 insertions(+), 581 deletions(-) delete mode 100644 mutable_buffer/src/pred.rs diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index 4359a8c276..b2711cb7f1 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -63,16 +63,16 @@ pub type Result = std::result::Result; #[derive(Debug)] pub struct Chunk { /// The id for this chunk - pub id: u32, + id: u32, /// `dictionary` maps &str -> u32. The u32s are used in place of String or /// str to avoid slow string operations. The same dictionary is used for /// table names, tag names, tag values, and column names. // TODO: intern string field values too? - pub dictionary: Dictionary, + dictionary: Dictionary, /// map of the dictionary ID for the table name to the table - pub tables: HashMap, + tables: HashMap, /// keep track of memory used by chunk tracker: MemTracker, @@ -173,7 +173,7 @@ impl Chunk { if let Some(table) = self.table(table_name)? { dst.push( table - .to_arrow(&self, selection) + .to_arrow(&self.dictionary, selection) .context(NamedTableError { table_name })?, ); } @@ -192,7 +192,7 @@ impl Chunk { TableSummary { name: name.to_string(), - columns: table.stats(&self), + columns: table.stats(&self.dictionary), } }) .collect() diff --git a/mutable_buffer/src/chunk/snapshot.rs b/mutable_buffer/src/chunk/snapshot.rs index 901b8c5640..4f6e1f3c15 100644 --- a/mutable_buffer/src/chunk/snapshot.rs +++ b/mutable_buffer/src/chunk/snapshot.rs @@ -55,8 +55,8 @@ impl ChunkSnapshot { pub fn new(chunk: &Chunk) -> Self { let mut records: HashMap = Default::default(); for (id, table) in &chunk.tables { - let schema = table.schema(chunk, Selection::All).unwrap(); - let batch = table.to_arrow(chunk, Selection::All).unwrap(); + let schema = table.schema(&chunk.dictionary, Selection::All).unwrap(); + let batch = table.to_arrow(&chunk.dictionary, Selection::All).unwrap(); let name = chunk.dictionary.lookup_id(*id).unwrap(); let timestamp_range = chunk diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index 7d822aa103..dfe9617d2b 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -1,7 +1,6 @@ use snafu::Snafu; use crate::dictionary::Dictionary; -use arrow_deps::arrow::datatypes::DataType as ArrowDataType; use data_types::partition_metadata::StatValues; use generated_types::entry::LogicalColumnType; use internal_types::entry::TypedValuesIterator; @@ -320,10 +319,6 @@ impl Column { } } - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - pub fn type_description(&self) -> &'static str { match self { Self::F64(_, _) => "f64", @@ -335,56 +330,6 @@ impl Column { } } - /// Return the arrow DataType for this column - pub fn data_type(&self) -> ArrowDataType { - match self { - Self::F64(..) => ArrowDataType::Float64, - Self::I64(..) => ArrowDataType::Int64, - Self::U64(..) => ArrowDataType::UInt64, - Self::String(..) => ArrowDataType::Utf8, - Self::Bool(..) => ArrowDataType::Boolean, - Self::Tag(..) => ArrowDataType::Utf8, - } - } - - // push_none_if_len_equal will add a None value to the end of the Vec of values - // if the length is equal to the passed in value. This is used to ensure - // columns are all the same length. - pub fn push_none_if_len_equal(&mut self, len: usize) { - match self { - Self::F64(v, _) => { - if v.len() == len { - v.push(None); - } - } - Self::I64(v, _) => { - if v.len() == len { - v.push(None); - } - } - Self::U64(v, _) => { - if v.len() == len { - v.push(None); - } - } - Self::String(v, _) => { - if v.len() == len { - v.push(None); - } - } - Self::Bool(v, _) => { - if v.len() == len { - v.push(None); - } - } - Self::Tag(v, _) => { - if v.len() == len { - v.push(None); - } - } - } - } - pub fn get_i64_stats(&self) -> Option> { match self { Self::I64(_, values) => Some(values.clone()), @@ -392,50 +337,6 @@ impl Column { } } - /// Returns true if any rows are within the range [min_value, - /// max_value). Inclusive of `start`, exclusive of `end` - pub fn has_i64_range(&self, start: i64, end: i64) -> Result { - match self { - Self::I64(_, stats) => { - if stats.max < start || stats.min >= end { - Ok(false) - } else { - Ok(true) - } - } - _ => InternalTypeMismatchForTimePredicate {}.fail(), - } - } - - /// Return true of this column's type is a Tag - pub fn is_tag(&self) -> bool { - matches!(self, Self::Tag(..)) - } - - /// Returns true if there exists at least one row idx where this - /// self[i] is within the range [min_value, max_value). Inclusive - /// of `start`, exclusive of `end` and where col[i] is non null - pub fn has_non_null_i64_range( - &self, - column: &[Option], - start: i64, - end: i64, - ) -> Result { - match self { - Self::I64(v, _) => { - for (index, val) in v.iter().enumerate() { - if let Some(val) = val { - if start <= *val && *val < end && column[index].is_some() { - return Ok(true); - } - } - } - Ok(false) - } - _ => InternalTypeMismatchForTimePredicate {}.fail(), - } - } - /// The approximate memory size of the data in the column. Note that /// the space taken for the tag string values is represented in /// the dictionary size in the chunk that holds the table that has this @@ -467,89 +368,3 @@ impl Column { } } } - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_has_i64_range() { - let mut stats = StatValues::new(1); - stats.update(2); - let col = Column::I64(vec![Some(1), None, Some(2)], stats.clone()); - assert!(!col.has_i64_range(-1, 0).unwrap()); - assert!(!col.has_i64_range(0, 1).unwrap()); - assert!(col.has_i64_range(1, 2).unwrap()); - assert!(col.has_i64_range(2, 3).unwrap()); - assert!(!col.has_i64_range(3, 4).unwrap()); - - let col = Column::I64(vec![Some(2), None, Some(1)], stats); - assert!(!col.has_i64_range(-1, 0).unwrap()); - assert!(!col.has_i64_range(0, 1).unwrap()); - assert!(col.has_i64_range(1, 2).unwrap()); - assert!(col.has_i64_range(2, 3).unwrap()); - assert!(!col.has_i64_range(3, 4).unwrap()); - } - - #[test] - fn test_has_i64_range_does_not_panic() { - // providing the wrong column type should get an internal error, not a panic - let col = Column::F64(vec![Some(1.2)], StatValues::new(1.2)); - let res = col.has_i64_range(-1, 0); - assert!(res.is_err()); - let res_string = format!("{:?}", res); - let expected = "InternalTypeMismatchForTimePredicate"; - assert!( - res_string.contains(expected), - "Did not find expected text '{}' in '{}'", - expected, - res_string - ); - } - - #[test] - fn test_has_non_null_i64_range_() { - let none_col: Vec> = vec![None, None, None]; - let some_col: Vec> = vec![Some(0), Some(0), Some(0)]; - - let mut stats = StatValues::new(1); - stats.update(2); - let col = Column::I64(vec![Some(1), None, Some(2)], stats); - - assert!(!col.has_non_null_i64_range(&some_col, -1, 0).unwrap()); - assert!(!col.has_non_null_i64_range(&some_col, 0, 1).unwrap()); - assert!(col.has_non_null_i64_range(&some_col, 1, 2).unwrap()); - assert!(col.has_non_null_i64_range(&some_col, 2, 3).unwrap()); - assert!(!col.has_non_null_i64_range(&some_col, 3, 4).unwrap()); - - assert!(!col.has_non_null_i64_range(&none_col, -1, 0).unwrap()); - assert!(!col.has_non_null_i64_range(&none_col, 0, 1).unwrap()); - assert!(!col.has_non_null_i64_range(&none_col, 1, 2).unwrap()); - assert!(!col.has_non_null_i64_range(&none_col, 2, 3).unwrap()); - assert!(!col.has_non_null_i64_range(&none_col, 3, 4).unwrap()); - } - - #[test] - fn column_size() { - let i64col = Column::I64(vec![Some(1), Some(1)], StatValues::new(1)); - assert_eq!(40, i64col.size()); - - let f64col = Column::F64(vec![Some(1.1), Some(1.1), Some(1.1)], StatValues::new(1.1)); - assert_eq!(56, f64col.size()); - - let boolcol = Column::Bool(vec![Some(true)], StatValues::new(true)); - assert_eq!(9, boolcol.size()); - - let tagcol = Column::Tag( - vec![Some(1), Some(1), Some(1), Some(1)], - StatValues::new("foo".to_string()), - ); - assert_eq!(40, tagcol.size()); - - let stringcol = Column::String( - vec![Some("foo".to_string()), Some("hello world".to_string())], - StatValues::new("foo".to_string()), - ); - assert_eq!(70, stringcol.size()); - } -} diff --git a/mutable_buffer/src/pred.rs b/mutable_buffer/src/pred.rs deleted file mode 100644 index 24a7db3074..0000000000 --- a/mutable_buffer/src/pred.rs +++ /dev/null @@ -1,298 +0,0 @@ -use std::collections::{BTreeSet, HashSet}; - -use crate::dictionary::{Dictionary, Error as DictionaryError}; - -use arrow_deps::{ - datafusion::{ - error::{DataFusionError, Result as DatafusionResult}, - logical_plan::{Expr, ExpressionVisitor, Operator, Recursion}, - optimizer::utils::expr_to_column_names, - }, - util::{make_range_expr, AndExprBuilder}, -}; -use data_types::timestamp::TimestampRange; -use internal_types::schema::TIME_COLUMN_NAME; - -//use snafu::{OptionExt, ResultExt, Snafu}; -use snafu::{ensure, ResultExt, Snafu}; - -#[derive(Debug, Snafu)] -pub enum Error { - #[snafu(display("Error writing table '{}': {}", table_name, source))] - TableWrite { - table_name: String, - source: crate::table::Error, - }, - - #[snafu(display("Time Column was not not found in dictionary: {}", source))] - TimeColumnNotFound { source: DictionaryError }, - - #[snafu(display("Unsupported predicate. Mutable buffer does not support: {}", source))] - UnsupportedPredicate { source: DataFusionError }, - - #[snafu(display( - "Internal error visiting expressions in ChunkPredicateBuilder: {}", - source - ))] - InternalVisitingExpressions { source: DataFusionError }, - - #[snafu(display("table_names has already been specified in ChunkPredicateBuilder"))] - TableNamesAlreadySet {}, - - #[snafu(display("field_names has already been specified in ChunkPredicateBuilder"))] - FieldNamesAlreadySet {}, - - #[snafu(display("range has already been specified in ChunkPredicateBuilder"))] - RangeAlreadySet {}, - - #[snafu(display("exprs has already been specified in ChunkPredicateBuilder"))] - ExprsAlreadySet {}, - - #[snafu(display("required_columns has already been specified in ChunkPredicateBuilder"))] - RequiredColumnsAlreadySet {}, -} - -pub type Result = std::result::Result; - -/// Describes the result of translating a set of strings into -/// chunk specific ids -#[derive(Debug, PartialEq, Eq)] -pub enum ChunkIdSet { - /// At least one of the strings was not present in the chunks' - /// dictionary. - /// - /// This is important when testing for the presence of all ids in - /// a set, as we know they can not all be present - AtLeastOneMissing, - - /// All strings existed in this chunk's dictionary - Present(BTreeSet), -} - -/// a 'Compiled' set of predicates / filters that can be evaluated on -/// this chunk (where strings have been translated to chunk -/// specific u32 ids) -#[derive(Debug, Default)] -pub struct ChunkPredicate { - /// If present, restrict the request to just those tables whose - /// names are in table_names. If present but empty, means there - /// was a predicate but no tables named that way exist in the - /// chunk (so no table can pass) - pub table_name_predicate: Option>, - - /// Optional column restriction. If present, further - /// restrict any field columns returned to only those named, and - /// skip tables entirely when querying metadata that do not have - /// *any* of the fields - pub field_name_predicate: Option>, - - /// General DataFusion expressions (arbitrary predicates) applied - /// as a filter using logical conjuction (aka are 'AND'ed - /// together). Only rows that evaluate to TRUE for all these - /// expressions should be returned. - /// - /// TODO these exprs should eventually be removed (when they are - /// all handled one layer up in the query layer) - pub chunk_exprs: Vec, - - /// If Some, then the table must contain all columns specified - /// to pass the predicate - pub required_columns: Option, - - /// The id of the "time" column in this chunk - pub time_column_id: u32, - - /// Timestamp range: only rows within this range should be considered - pub range: Option, -} - -impl ChunkPredicate { - /// Creates and adds a datafuson predicate representing the - /// combination of predicate and timestamp. - pub fn filter_expr(&self) -> Option { - // build up a list of expressions - let mut builder = - AndExprBuilder::default().append_opt(self.make_timestamp_predicate_expr()); - - for expr in &self.chunk_exprs { - builder = builder.append_expr(expr.clone()); - } - - builder.build() - } - - /// For plans which select a subset of fields, returns true if - /// the field should be included in the results - pub fn should_include_field(&self, field_id: u32) -> bool { - match &self.field_name_predicate { - None => true, - Some(field_restriction) => field_restriction.contains(&field_id), - } - } - - /// Return true if this column is the time column - pub fn is_time_column(&self, id: u32) -> bool { - self.time_column_id == id - } - - /// Creates a DataFusion predicate for appliying a timestamp range: - /// - /// range.start <= time and time < range.end` - fn make_timestamp_predicate_expr(&self) -> Option { - self.range - .map(|range| make_range_expr(range.start, range.end, TIME_COLUMN_NAME)) - } -} - -/// Builds ChunkPredicates -#[derive(Debug)] -pub struct ChunkPredicateBuilder<'a> { - inner: ChunkPredicate, - dictionary: &'a Dictionary, -} - -impl<'a> ChunkPredicateBuilder<'a> { - pub fn new(dictionary: &'a Dictionary) -> Result { - let time_column_id = dictionary - .lookup_value(TIME_COLUMN_NAME) - .context(TimeColumnNotFound)?; - - let inner = ChunkPredicate { - time_column_id, - ..Default::default() - }; - - Ok(Self { inner, dictionary }) - } - - /// Set table_name_predicate so only tables in `names` are returned - pub fn table_names(mut self, names: Option<&BTreeSet>) -> Result { - ensure!( - self.inner.table_name_predicate.is_none(), - TableNamesAlreadySet - ); - self.inner.table_name_predicate = self.compile_string_list(names); - Ok(self) - } - - /// Set field_name_predicate so only tables in `names` are returned - pub fn field_names(mut self, names: Option<&BTreeSet>) -> Result { - ensure!( - self.inner.field_name_predicate.is_none(), - FieldNamesAlreadySet - ); - self.inner.field_name_predicate = self.compile_string_list(names); - Ok(self) - } - - pub fn range(mut self, range: Option) -> Result { - ensure!(self.inner.range.is_none(), RangeAlreadySet); - self.inner.range = range; - Ok(self) - } - - /// Set the general purpose predicates - pub fn exprs(mut self, chunk_exprs: Vec) -> Result { - // In order to evaluate expressions in the table, all columns - // referenced in the expression must appear (I think, not sure - // about NOT, etc so panic if we see one of those); - let mut visitor = SupportVisitor {}; - let mut predicate_columns: HashSet = HashSet::new(); - for expr in &chunk_exprs { - visitor = expr.accept(visitor).context(UnsupportedPredicate)?; - expr_to_column_names(&expr, &mut predicate_columns) - .context(InternalVisitingExpressions)?; - } - - ensure!(self.inner.chunk_exprs.is_empty(), ExprsAlreadySet); - self.inner.chunk_exprs = chunk_exprs; - - // if there are any column references in the expression, ensure they appear in - // any table - if !predicate_columns.is_empty() { - ensure!( - self.inner.required_columns.is_none(), - RequiredColumnsAlreadySet - ); - self.inner.required_columns = Some(self.make_chunk_ids(predicate_columns.iter())); - } - Ok(self) - } - - /// Return the created chunk predicate, consuming self - pub fn build(self) -> ChunkPredicate { - self.inner - } - - /// Converts a Set of strings into a set of ids in terms of this - /// Chunk's dictionary. - /// - /// If there are no matching Strings in the chunks dictionary, - /// those strings are ignored and a (potentially empty) set is - /// returned. - fn compile_string_list(&self, names: Option<&BTreeSet>) -> Option> { - names.map(|names| { - names - .iter() - .filter_map(|name| self.dictionary.id(name)) - .collect::>() - }) - } - - /// Translate a bunch of strings into a set of ids from the dictionarythis - /// chunk - pub fn make_chunk_ids<'b, I>(&self, predicate_columns: I) -> ChunkIdSet - where - I: Iterator, - { - let mut symbols = BTreeSet::new(); - for column_name in predicate_columns { - if let Some(column_id) = self.dictionary.id(column_name) { - symbols.insert(column_id); - } else { - return ChunkIdSet::AtLeastOneMissing; - } - } - - ChunkIdSet::Present(symbols) - } -} - -/// Used to figure out if we know how to deal with this kind of -/// predicate in the write buffer -struct SupportVisitor {} - -impl ExpressionVisitor for SupportVisitor { - fn pre_visit(self, expr: &Expr) -> DatafusionResult> { - match expr { - Expr::Literal(..) => Ok(Recursion::Continue(self)), - Expr::Column(..) => Ok(Recursion::Continue(self)), - Expr::BinaryExpr { op, .. } => { - match op { - Operator::Eq - | Operator::Lt - | Operator::LtEq - | Operator::Gt - | Operator::GtEq - | Operator::Plus - | Operator::Minus - | Operator::Multiply - | Operator::Divide - | Operator::And - | Operator::Or => Ok(Recursion::Continue(self)), - // Unsupported (need to think about ramifications) - Operator::NotEq | Operator::Modulus | Operator::Like | Operator::NotLike => { - Err(DataFusionError::NotImplemented(format!( - "Operator {:?} not yet supported in IOx MutableBuffer", - op - ))) - } - } - } - _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported expression in mutable_buffer database: {:?}", - expr - ))), - } - } -} diff --git a/mutable_buffer/src/table.rs b/mutable_buffer/src/table.rs index 5439f9f054..687b5dcffa 100644 --- a/mutable_buffer/src/table.rs +++ b/mutable_buffer/src/table.rs @@ -1,7 +1,6 @@ use std::{cmp, collections::BTreeMap, sync::Arc}; use crate::{ - chunk::Chunk, column, column::Column, dictionary::{Dictionary, Error as DictionaryError}, @@ -31,12 +30,8 @@ use arrow_deps::{ #[derive(Debug, Snafu)] pub enum Error { - #[snafu(display("Tag value ID {} not found in dictionary of chunk {}", value, chunk))] - TagValueIdNotFoundInDictionary { - value: u32, - chunk: u64, - source: DictionaryError, - }, + #[snafu(display("Tag value ID {} not found in dictionary of chunk", value))] + TagValueIdNotFoundInDictionary { value: u32, source: DictionaryError }, #[snafu(display("Column error on column {}: {}", column, source))] ColumnError { @@ -59,21 +54,12 @@ pub enum Error { #[snafu(display("Internal error: unexpected aggregate request for None aggregate",))] InternalUnexpectedNoneAggregate {}, - #[snafu(display( - "Column name '{}' not found in dictionary of chunk {}", - column_name, - chunk - ))] - ColumnNameNotFoundInDictionary { column_name: String, chunk: u64 }, + #[snafu(display("Column name '{}' not found in dictionary of chunk", column_name,))] + ColumnNameNotFoundInDictionary { column_name: String }, - #[snafu(display( - "Internal: Column id '{}' not found in dictionary of chunk {}", - column_id, - chunk - ))] + #[snafu(display("Internal: Column id '{}' not found in dictionary", column_id,))] ColumnIdNotFoundInDictionary { column_id: u32, - chunk: u64, source: DictionaryError, }, @@ -165,21 +151,6 @@ impl Table { }) } - /// Returns a reference to the specified column as a slice of - /// i64s. Errors if the type is not i64 - pub fn column_i64(&self, column_id: u32) -> Result<&[Option]> { - let column = self.column(column_id)?; - match column { - Column::I64(vals, _) => Ok(vals), - _ => InternalColumnTypeMismatch { - column_id, - expected_column_type: "i64", - actual_column_type: column.type_description(), - } - .fail(), - } - } - /// Validates the schema of the passed in columns, then adds their values to /// the associated columns in the table and updates summary statistics. pub fn write_columns( @@ -277,17 +248,20 @@ impl Table { /// Returns the column selection for all the columns in this table, orderd /// by table name - fn all_columns_selection<'a>(&self, chunk: &'a Chunk) -> Result> { + fn all_columns_selection<'a>( + &self, + dictionary: &'a Dictionary, + ) -> Result> { let cols = self .columns .iter() .map(|(column_id, _)| { - let column_name = chunk.dictionary.lookup_id(*column_id).context( - ColumnIdNotFoundInDictionary { - column_id: *column_id, - chunk: chunk.id, - }, - )?; + let column_name = + dictionary + .lookup_id(*column_id) + .context(ColumnIdNotFoundInDictionary { + column_id: *column_id, + })?; Ok(ColSelection { column_name, column_id: *column_id, @@ -304,45 +278,45 @@ impl Table { /// Returns a column selection for just the specified columns fn specific_columns_selection<'a>( &self, - chunk: &'a Chunk, + dictionary: &'a Dictionary, columns: &'a [&'a str], ) -> Result> { - let cols = - columns - .iter() - .map(|&column_name| { - let column_id = chunk.dictionary.id(column_name).context( - ColumnNameNotFoundInDictionary { - column_name, - chunk: chunk.id, - }, - )?; + let cols = columns + .iter() + .map(|&column_name| { + let column_id = dictionary + .id(column_name) + .context(ColumnNameNotFoundInDictionary { column_name })?; - Ok(ColSelection { - column_name, - column_id, - }) + Ok(ColSelection { + column_name, + column_id, }) - .collect::>()?; + }) + .collect::>()?; Ok(TableColSelection { cols }) } /// Converts this table to an arrow record batch. - pub fn to_arrow(&self, chunk: &Chunk, selection: Selection<'_>) -> Result { + pub fn to_arrow( + &self, + dictionary: &Dictionary, + selection: Selection<'_>, + ) -> Result { // translate chunk selection into name/indexes: let selection = match selection { - Selection::All => self.all_columns_selection(chunk), - Selection::Some(cols) => self.specific_columns_selection(chunk, cols), + Selection::All => self.all_columns_selection(dictionary), + Selection::Some(cols) => self.specific_columns_selection(dictionary, cols), }?; - self.to_arrow_impl(chunk, &selection) + self.to_arrow_impl(dictionary, &selection) } - pub fn schema(&self, chunk: &Chunk, selection: Selection<'_>) -> Result { + pub fn schema(&self, dictionary: &Dictionary, selection: Selection<'_>) -> Result { // translate chunk selection into name/indexes: let selection = match selection { - Selection::All => self.all_columns_selection(chunk), - Selection::Some(cols) => self.specific_columns_selection(chunk, cols), + Selection::All => self.all_columns_selection(dictionary), + Selection::Some(cols) => self.specific_columns_selection(dictionary, cols), }?; self.schema_impl(&selection) } @@ -379,7 +353,7 @@ impl Table { /// requested columns with index are tuples of column_name, column_index fn to_arrow_impl( &self, - chunk: &Chunk, + dictionary: &Dictionary, selection: &TableColSelection<'_>, ) -> Result { let mut columns = Vec::with_capacity(selection.cols.len()); @@ -408,12 +382,9 @@ impl Table { match v { None => builder.append_null(), Some(value_id) => { - let tag_value = chunk.dictionary.lookup_id(*value_id).context( - TagValueIdNotFoundInDictionary { - value: *value_id, - chunk: chunk.id, - }, - )?; + let tag_value = dictionary + .lookup_id(*value_id) + .context(TagValueIdNotFoundInDictionary { value: *value_id })?; builder.append_value(tag_value) } } @@ -468,12 +439,11 @@ impl Table { RecordBatch::try_new(schema, columns).context(ArrowError {}) } - pub fn stats(&self, chunk: &Chunk) -> Vec { + pub fn stats(&self, dictionary: &Dictionary) -> Vec { self.columns .iter() .map(|(column_id, c)| { - let column_name = chunk - .dictionary + let column_name = dictionary .lookup_id(*column_id) .expect("column name in dictionary"); @@ -520,13 +490,10 @@ mod tests { use internal_types::entry::test_helpers::lp_to_entry; use super::*; - use tracker::MemRegistry; #[test] fn table_size() { - let registry = Arc::new(MemRegistry::new()); - let mut chunk = Chunk::new(42, registry.as_ref()); - let dictionary = &mut chunk.dictionary; + let mut dictionary = Dictionary::new(); let mut table = Table::new(dictionary.lookup_value_or_insert("table_name")); let lp_lines = vec![ @@ -534,33 +501,31 @@ mod tests { "h2o,state=MA,city=Boston temp=72.4 250", ]; - write_lines_to_table(&mut table, dictionary, lp_lines.clone()); + write_lines_to_table(&mut table, &mut dictionary, lp_lines.clone()); assert_eq!(128, table.size()); // doesn't double because of the stats overhead - write_lines_to_table(&mut table, dictionary, lp_lines.clone()); + write_lines_to_table(&mut table, &mut dictionary, lp_lines.clone()); assert_eq!(224, table.size()); // now make sure it increased by the same amount minus stats overhead - write_lines_to_table(&mut table, dictionary, lp_lines); + write_lines_to_table(&mut table, &mut dictionary, lp_lines); assert_eq!(320, table.size()); } #[test] fn test_to_arrow_schema_all() { - let registry = Arc::new(MemRegistry::new()); - let mut chunk = Chunk::new(42, registry.as_ref()); - let dictionary = &mut chunk.dictionary; + let mut dictionary = Dictionary::new(); let mut table = Table::new(dictionary.lookup_value_or_insert("table_name")); let lp_lines = vec![ "h2o,state=MA,city=Boston float_field=70.4,int_field=8i,uint_field=42u,bool_field=t,string_field=\"foo\" 100", ]; - write_lines_to_table(&mut table, dictionary, lp_lines); + write_lines_to_table(&mut table, &mut dictionary, lp_lines); let selection = Selection::All; - let actual_schema = table.schema(&chunk, selection).unwrap(); + let actual_schema = table.schema(&dictionary, selection).unwrap(); let expected_schema = SchemaBuilder::new() .field("bool_field", ArrowDataType::Boolean) .tag("city") @@ -582,17 +547,15 @@ mod tests { #[test] fn test_to_arrow_schema_subset() { - let registry = Arc::new(MemRegistry::new()); - let mut chunk = Chunk::new(42, registry.as_ref()); - let dictionary = &mut chunk.dictionary; + let mut dictionary = Dictionary::new(); let mut table = Table::new(dictionary.lookup_value_or_insert("table_name")); let lp_lines = vec!["h2o,state=MA,city=Boston float_field=70.4 100"]; - write_lines_to_table(&mut table, dictionary, lp_lines); + write_lines_to_table(&mut table, &mut dictionary, lp_lines); let selection = Selection::Some(&["float_field"]); - let actual_schema = table.schema(&chunk, selection).unwrap(); + let actual_schema = table.schema(&dictionary, selection).unwrap(); let expected_schema = SchemaBuilder::new() .field("float_field", ArrowDataType::Float64) .build() From 59c35e5a4fdabc240fd12ee0cd107a0e53de64aa Mon Sep 17 00:00:00 2001 From: Marko Mikulicic Date: Tue, 13 Apr 2021 16:27:52 +0200 Subject: [PATCH 25/45] feat: Implement WriteBuffer gRPC --- .../influxdata/iox/write/v1/service.proto | 21 +++++- server/src/db.rs | 4 ++ server/src/lib.rs | 69 ++++++++++++++++++- src/influxdb_ioxd/rpc/error.rs | 5 ++ src/influxdb_ioxd/rpc/write.rs | 17 +++++ 5 files changed, 114 insertions(+), 2 deletions(-) diff --git a/generated_types/protos/influxdata/iox/write/v1/service.proto b/generated_types/protos/influxdata/iox/write/v1/service.proto index c0fa0a9cbf..17e3a03a96 100644 --- a/generated_types/protos/influxdata/iox/write/v1/service.proto +++ b/generated_types/protos/influxdata/iox/write/v1/service.proto @@ -4,7 +4,12 @@ package influxdata.iox.write.v1; service WriteService { // write data into a specific Database - rpc Write(WriteRequest) returns (WriteResponse); + rpc Write(WriteRequest) returns (WriteResponse) { + option deprecated = true; + }; + + // write an entry into a Database + rpc WriteEntry(WriteEntryRequest) returns (WriteEntryResponse); } message WriteRequest { @@ -21,3 +26,17 @@ message WriteResponse { // how many lines were parsed and written into the database uint64 lines_written = 1; } + + +message WriteEntryRequest { + // name of database into which to write + string db_name = 1; + + // entry, in serialized flatbuffers [Entry] format + // + // [Entry](https://github.com/influxdata/influxdb_iox/blob/main/generated_types/protos/influxdata/iox/write/v1/entry.fbs) + bytes entry = 2; +} + +message WriteEntryResponse { +} diff --git a/server/src/db.rs b/server/src/db.rs index 94d60de241..c66fff5df1 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -642,6 +642,10 @@ impl Db { ) .context(SequencedEntryError)?; + if self.rules.read().wal_buffer_config.is_some() { + todo!("route to the Write Buffer. TODO: carols10cents #1157") + } + self.store_sequenced_entry(sequenced_entry) } diff --git a/server/src/lib.rs b/server/src/lib.rs index fa25b149db..5d6ce4dca5 100644 --- a/server/src/lib.rs +++ b/server/src/lib.rs @@ -67,6 +67,7 @@ clippy::clone_on_ref_ptr )] +use std::convert::TryInto; use std::sync::Arc; use async_trait::async_trait; @@ -150,6 +151,10 @@ pub enum Error { WalError { source: buffer::Error }, #[snafu(display("error converting line protocol to flatbuffers: {}", source))] LineConversion { source: entry::Error }, + #[snafu(display("error decoding entry flatbuffers: {}", source))] + DecodingEntry { + source: flatbuffers::InvalidFlatbuffer, + }, } pub type Result = std::result::Result; @@ -368,6 +373,19 @@ impl Server { Ok(()) } + pub async fn write_entry(&self, db_name: &str, entry_bytes: Vec) -> Result<()> { + self.require_id()?; + + let db_name = DatabaseName::new(db_name).context(InvalidDatabaseName)?; + let db = self + .config + .db(&db_name) + .context(DatabaseNotFound { db_name: &*db_name })?; + + let entry = entry_bytes.try_into().context(DecodingEntry)?; + self.handle_write_entry(&db, entry).await + } + pub async fn handle_write_entry(&self, db: &Db, entry: Entry) -> Result<()> { db.store_entry(entry) .map_err(|e| Error::UnknownDatabaseError { @@ -674,7 +692,7 @@ mod tests { use tokio_util::sync::CancellationToken; use arrow_deps::{assert_table_eq, datafusion::physical_plan::collect}; - use data_types::database_rules::{PartitionTemplate, TemplatePart}; + use data_types::database_rules::{PartitionTemplate, TemplatePart, NO_SHARD_CONFIG}; use influxdb_line_protocol::parse_lines; use object_store::{memory::InMemory, path::ObjectStorePath}; use query::{frontend::sql::SQLQueryPlanner, Database}; @@ -867,6 +885,55 @@ mod tests { assert_table_eq!(expected, &batches); } + #[tokio::test] + async fn write_entry_local() { + let manager = TestConnectionManager::new(); + let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); + let server = Server::new(manager, store); + server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); + + let name = DatabaseName::new("foo".to_string()).unwrap(); + server + .create_database( + DatabaseRules::new(name), + server.require_id().unwrap(), + Arc::clone(&server.store), + ) + .await + .unwrap(); + + let db_name = DatabaseName::new("foo").unwrap(); + let db = server.db(&db_name).unwrap(); + + let line = "cpu bar=1 10"; + let lines: Vec<_> = parse_lines(line).map(|l| l.unwrap()).collect(); + let sharded_entries = lines_to_sharded_entries(&lines, NO_SHARD_CONFIG, &*db.rules.read()) + .expect("sharded entries"); + + let entry = &sharded_entries[0].entry; + server + .write_entry("foo", entry.data().into()) + .await + .expect("write entry"); + + let planner = SQLQueryPlanner::default(); + let executor = server.executor(); + let physical_plan = planner + .query(db, "select * from cpu", executor.as_ref()) + .await + .unwrap(); + + let batches = collect(physical_plan).await.unwrap(); + let expected = vec![ + "+-----+------+", + "| bar | time |", + "+-----+------+", + "| 1 | 10 |", + "+-----+------+", + ]; + assert_table_eq!(expected, &batches); + } + #[tokio::test] async fn close_chunk() { test_helpers::maybe_start_logging(); diff --git a/src/influxdb_ioxd/rpc/error.rs b/src/influxdb_ioxd/rpc/error.rs index 20bdf63966..bd78779564 100644 --- a/src/influxdb_ioxd/rpc/error.rs +++ b/src/influxdb_ioxd/rpc/error.rs @@ -23,6 +23,11 @@ pub fn default_server_error_handler(error: server::Error) -> tonic::Status { description: source.to_string(), } .into(), + Error::DecodingEntry { source } => FieldViolation { + field: "entry".into(), + description: source.to_string(), + } + .into(), error => { error!(?error, "Unexpected error"); InternalError {}.into() diff --git a/src/influxdb_ioxd/rpc/write.rs b/src/influxdb_ioxd/rpc/write.rs index ff715620b7..a67e492dc5 100644 --- a/src/influxdb_ioxd/rpc/write.rs +++ b/src/influxdb_ioxd/rpc/write.rs @@ -47,6 +47,23 @@ where let lines_written = lp_line_count as u64; Ok(Response::new(WriteResponse { lines_written })) } + + async fn write_entry( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + let request = request.into_inner(); + if request.entry.is_empty() { + return Err(FieldViolation::required("entry").into()); + } + + self.server + .write_entry(&request.db_name, request.entry) + .await + .map_err(default_server_error_handler)?; + + Ok(Response::new(WriteEntryResponse {})) + } } /// Instantiate the write service From 6d0467277b4df9a40222e85244cdeeb5f8f02e05 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 13 Apr 2021 17:56:36 -0400 Subject: [PATCH 26/45] docs: update multi core design doc (#1197) Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- docs/multi_core_tasks.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/multi_core_tasks.md b/docs/multi_core_tasks.md index 57f3e0d148..605fada859 100644 --- a/docs/multi_core_tasks.md +++ b/docs/multi_core_tasks.md @@ -6,7 +6,7 @@ As discussed on https://github.com/influxdata/influxdb_iox/pull/221 and https:// 1. Use only async I/O via `tokio` for socket communication. It is ok to use either blocking (e.g. `std::fs::File`) or async APIs (e.g. `tokio::fs::File`) for local File I/O. -2. All CPU bound tasks should be scheduled on the separate application level `thread_pool` not with `tokio::task::spawn` nor `tokio::task::spawn_blocking` nor a new threadpool. +2. All CPU bound tasks should be scheduled on the separate application level `thread_pool` (which can be another tokio executor but should be separate from the executor that handles I/O). We will work, over time, to migrate the rest of the codebase to use these patterns. @@ -41,11 +41,11 @@ It is ok to use either blocking (e.g. `std::fs::File`) or async APIs for local This can not always be done (e.g. with a library such as parquet writer which is not `async`). In such cases, using `tokio::task::spawn_blocking` should be used to perform the file I/O. -### All CPU heavy work should be done on the single app level worker pool, separate from the tokio runtime +### All CPU heavy work should be done on the single app level worker pool, separate from the tokio runtime handling IO -**What**: All CPU heavy work should be done on the single app level worker pool. We provide a `thread_pool` interface that interacts nicely with async tasks (e.g. that allows an async task to `await` for a CPU heavy task to complete). +**What**: All CPU heavy work should be done on the app level worker pool. We provide a `thread_pool` interface that interacts nicely with async tasks (e.g. that allows an async task to `await` for a CPU heavy task to complete). -**Rationale**: A single app level worker pool gives us a single place to control work priority, eventually, so that tasks such as compaction of large data files can have lower precedence than incoming queries. By using a different pool than the tokio runtime, with a limited number of threads, we avoid over-saturating the CPU with OS threads and thereby starving the limited number tokio I/O threads. A separate, single app level pool also limits the number of underlying OS CPU threads which are spawned, even under heavy load, keeping thread context switching overhead low. +**Rationale**: A single app level worker pool gives us a single place to control work priority, eventually, so that tasks such as compaction of large data files can have lower precedence than incoming queries. By using a different pool than the main tokio runtime, with a limited number of threads, we avoid over-saturating the CPU with OS threads and thereby starving the limited number tokio I/O threads. A separate, single app level pool also limits the number of underlying OS CPU threads which are spawned, even under heavy load, keeping thread context switching overhead low. There will, of course, always be a judgment call to be made of where "CPU bound work" starts and "work acceptable for I/O processing" ends. A reasonable rule of thumb is if a job will *always* be completed in less than 100ms then that is probably fine for an I/O thread). This number may be revised as we tune the system. From 150ed4e1d9864bd548a2274f780d24c9494905f6 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 13 Apr 2021 18:17:19 -0400 Subject: [PATCH 27/45] refactor: Remove async from `InfluxRPCPlanner` (#1200) * refactor: Remove async from InfluxRPCPlanner * fix: make it compile Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- query/src/frontend/influxrpc.rs | 104 +++++++----------- .../query_tests/influxrpc/field_columns.rs | 2 - .../src/query_tests/influxrpc/read_filter.rs | 1 - .../src/query_tests/influxrpc/read_group.rs | 1 - .../influxrpc/read_window_aggregate.rs | 1 - .../src/query_tests/influxrpc/table_names.rs | 1 - server/src/query_tests/influxrpc/tag_keys.rs | 1 - .../src/query_tests/influxrpc/tag_values.rs | 3 +- src/influxdb_ioxd/rpc/storage/service.rs | 13 +-- 9 files changed, 45 insertions(+), 82 deletions(-) diff --git a/query/src/frontend/influxrpc.rs b/query/src/frontend/influxrpc.rs index 01aaa61d0f..cf6132ab9d 100644 --- a/query/src/frontend/influxrpc.rs +++ b/query/src/frontend/influxrpc.rs @@ -195,13 +195,13 @@ impl InfluxRPCPlanner { /// Returns a plan that lists the names of tables in this /// database that have at least one row that matches the /// conditions listed on `predicate` - pub async fn table_names(&self, database: &D, predicate: Predicate) -> Result + pub fn table_names(&self, database: &D, predicate: Predicate) -> Result where D: Database + 'static, { let mut builder = StringSetPlanBuilder::new(); - for chunk in self.filtered_chunks(database, &predicate).await? { + for chunk in self.filtered_chunks(database, &predicate)? { let new_table_names = chunk .table_names(&predicate, builder.known_strings()) .map_err(|e| Box::new(e) as _) @@ -227,7 +227,7 @@ impl InfluxRPCPlanner { /// columns (as defined in the InfluxDB Data model) names in this /// database that have more than zero rows which pass the /// conditions specified by `predicate`. - pub async fn tag_keys(&self, database: &D, predicate: Predicate) -> Result + pub fn tag_keys(&self, database: &D, predicate: Predicate) -> Result where D: Database + 'static, { @@ -246,9 +246,9 @@ impl InfluxRPCPlanner { let mut need_full_plans = BTreeMap::new(); let mut known_columns = BTreeSet::new(); - for chunk in self.filtered_chunks(database, &predicate).await? { + for chunk in self.filtered_chunks(database, &predicate)? { // try and get the table names that have rows that match the predicate - let table_names = self.chunk_table_names(chunk.as_ref(), &predicate).await?; + let table_names = self.chunk_table_names(chunk.as_ref(), &predicate)?; for table_name in table_names { debug!( @@ -308,7 +308,7 @@ impl InfluxRPCPlanner { // were already known to have data (based on the contents of known_columns) for (table_name, chunks) in need_full_plans.into_iter() { - let plan = self.tag_keys_plan(&table_name, &predicate, chunks).await?; + let plan = self.tag_keys_plan(&table_name, &predicate, chunks)?; if let Some(plan) = plan { builder = builder.append(plan) @@ -326,7 +326,7 @@ impl InfluxRPCPlanner { /// Returns a plan which finds the distinct, non-null tag values /// in the specified `tag_name` column of this database which pass /// the conditions specified by `predicate`. - pub async fn tag_values( + pub fn tag_values( &self, database: &D, tag_name: &str, @@ -351,8 +351,8 @@ impl InfluxRPCPlanner { let mut need_full_plans = BTreeMap::new(); let mut known_values = BTreeSet::new(); - for chunk in self.filtered_chunks(database, &predicate).await? { - let table_names = self.chunk_table_names(chunk.as_ref(), &predicate).await?; + for chunk in self.filtered_chunks(database, &predicate)? { + let table_names = self.chunk_table_names(chunk.as_ref(), &predicate)?; for table_name in table_names { debug!( @@ -426,9 +426,7 @@ impl InfluxRPCPlanner { // time in `known_columns`, and some tables in chunks that we // need to run a plan to find what values pass the predicate. for (table_name, chunks) in need_full_plans.into_iter() { - let scan_and_filter = self - .scan_and_filter(&table_name, &predicate, chunks) - .await?; + let scan_and_filter = self.scan_and_filter(&table_name, &predicate, chunks)?; // if we have any data to scan, make a plan! if let Some(TableScanAndFilter { @@ -471,11 +469,7 @@ impl InfluxRPCPlanner { /// datatypes (as defined in the data written via `write_lines`), /// and which have more than zero rows which pass the conditions /// specified by `predicate`. - pub async fn field_columns( - &self, - database: &D, - predicate: Predicate, - ) -> Result + pub fn field_columns(&self, database: &D, predicate: Predicate) -> Result where D: Database + 'static, { @@ -488,15 +482,12 @@ impl InfluxRPCPlanner { // values and stops the plan executing once it has them // map table -> Vec> - let chunks = self.filtered_chunks(database, &predicate).await?; - let table_chunks = self.group_chunks_by_table(&predicate, chunks).await?; + let chunks = self.filtered_chunks(database, &predicate)?; + let table_chunks = self.group_chunks_by_table(&predicate, chunks)?; let mut field_list_plan = FieldListPlan::new(); for (table_name, chunks) in table_chunks { - if let Some(plan) = self - .field_columns_plan(&table_name, &predicate, chunks) - .await? - { + if let Some(plan) = self.field_columns_plan(&table_name, &predicate, chunks)? { field_list_plan = field_list_plan.append(plan); } } @@ -523,7 +514,7 @@ impl InfluxRPCPlanner { /// rows for a particular series (groups where all tags are the /// same) occur together in the plan - pub async fn read_filter(&self, database: &D, predicate: Predicate) -> Result + pub fn read_filter(&self, database: &D, predicate: Predicate) -> Result where D: Database + 'static, { @@ -531,17 +522,15 @@ impl InfluxRPCPlanner { // group tables by chunk, pruning if possible // key is table name, values are chunks - let chunks = self.filtered_chunks(database, &predicate).await?; - let table_chunks = self.group_chunks_by_table(&predicate, chunks).await?; + let chunks = self.filtered_chunks(database, &predicate)?; + let table_chunks = self.group_chunks_by_table(&predicate, chunks)?; // now, build up plans for each table let mut ss_plans = Vec::with_capacity(table_chunks.len()); for (table_name, chunks) in table_chunks { let prefix_columns: Option<&[&str]> = None; - let ss_plan = self - .read_filter_plan(table_name, prefix_columns, &predicate, chunks) - .await?; + let ss_plan = self.read_filter_plan(table_name, prefix_columns, &predicate, chunks)?; // If we have to do real work, add it to the list of plans if let Some(ss_plan) = ss_plan { ss_plans.push(ss_plan); @@ -555,7 +544,7 @@ impl InfluxRPCPlanner { /// with rows grouped by an aggregate function. Note that we still /// group by all tags (so group within series) and the /// group_columns define the order of the result - pub async fn read_group( + pub fn read_group( &self, database: &D, predicate: Predicate, @@ -568,8 +557,8 @@ impl InfluxRPCPlanner { debug!(predicate=?predicate, agg=?agg, "planning read_group"); // group tables by chunk, pruning if possible - let chunks = self.filtered_chunks(database, &predicate).await?; - let table_chunks = self.group_chunks_by_table(&predicate, chunks).await?; + let chunks = self.filtered_chunks(database, &predicate)?; + let table_chunks = self.group_chunks_by_table(&predicate, chunks)?; let num_prefix_tag_group_columns = group_columns.len(); // now, build up plans for each table @@ -577,13 +566,9 @@ impl InfluxRPCPlanner { for (table_name, chunks) in table_chunks { let ss_plan = match agg { Aggregate::None => { - self.read_filter_plan(table_name, Some(group_columns), &predicate, chunks) - .await? - } - _ => { - self.read_group_plan(table_name, &predicate, agg, group_columns, chunks) - .await? + self.read_filter_plan(table_name, Some(group_columns), &predicate, chunks)? } + _ => self.read_group_plan(table_name, &predicate, agg, group_columns, chunks)?, }; // If we have to do real work, add it to the list of plans @@ -598,7 +583,7 @@ impl InfluxRPCPlanner { /// Creates a GroupedSeriesSet plan that produces an output table with rows /// that are grouped by window defintions - pub async fn read_window_aggregate( + pub fn read_window_aggregate( &self, database: &D, predicate: Predicate, @@ -612,15 +597,14 @@ impl InfluxRPCPlanner { debug!(predicate=?predicate, "planning read_window_aggregate"); // group tables by chunk, pruning if possible - let chunks = self.filtered_chunks(database, &predicate).await?; - let table_chunks = self.group_chunks_by_table(&predicate, chunks).await?; + let chunks = self.filtered_chunks(database, &predicate)?; + let table_chunks = self.group_chunks_by_table(&predicate, chunks)?; // now, build up plans for each table let mut ss_plans = Vec::with_capacity(table_chunks.len()); for (table_name, chunks) in table_chunks { let ss_plan = self - .read_window_aggregate_plan(table_name, &predicate, agg, &every, &offset, chunks) - .await?; + .read_window_aggregate_plan(table_name, &predicate, agg, &every, &offset, chunks)?; // If we have to do real work, add it to the list of plans if let Some(ss_plan) = ss_plan { ss_plans.push(ss_plan); @@ -631,7 +615,7 @@ impl InfluxRPCPlanner { } /// Creates a map of table_name --> Chunks that have that table - async fn group_chunks_by_table( + fn group_chunks_by_table( &self, predicate: &Predicate, chunks: Vec>, @@ -641,7 +625,7 @@ impl InfluxRPCPlanner { { let mut table_chunks = BTreeMap::new(); for chunk in chunks { - let table_names = self.chunk_table_names(chunk.as_ref(), &predicate).await?; + let table_names = self.chunk_table_names(chunk.as_ref(), &predicate)?; for table_name in table_names { table_chunks .entry(table_name) @@ -653,11 +637,7 @@ impl InfluxRPCPlanner { } /// Find all the table names in the specified chunk that pass the predicate - async fn chunk_table_names( - &self, - chunk: &C, - predicate: &Predicate, - ) -> Result> + fn chunk_table_names(&self, chunk: &C, predicate: &Predicate) -> Result> where C: PartitionChunk + 'static, { @@ -705,7 +685,7 @@ impl InfluxRPCPlanner { /// Filter(predicate) /// TableScan (of chunks) /// ``` - async fn tag_keys_plan( + fn tag_keys_plan( &self, table_name: &str, predicate: &Predicate, @@ -714,7 +694,7 @@ impl InfluxRPCPlanner { where C: PartitionChunk + 'static, { - let scan_and_filter = self.scan_and_filter(table_name, predicate, chunks).await?; + let scan_and_filter = self.scan_and_filter(table_name, predicate, chunks)?; let TableScanAndFilter { plan_builder, @@ -767,7 +747,7 @@ impl InfluxRPCPlanner { /// Filter(predicate) [optional] /// Scan /// ``` - async fn field_columns_plan( + fn field_columns_plan( &self, table_name: &str, predicate: &Predicate, @@ -776,7 +756,7 @@ impl InfluxRPCPlanner { where C: PartitionChunk + 'static, { - let scan_and_filter = self.scan_and_filter(table_name, predicate, chunks).await?; + let scan_and_filter = self.scan_and_filter(table_name, predicate, chunks)?; let TableScanAndFilter { plan_builder, schema, @@ -817,7 +797,7 @@ impl InfluxRPCPlanner { /// Order by (tag_columns, timestamp_column) /// Filter(predicate) /// Scan - async fn read_filter_plan( + fn read_filter_plan( &self, table_name: impl Into, prefix_columns: Option<&[impl AsRef]>, @@ -828,7 +808,7 @@ impl InfluxRPCPlanner { C: PartitionChunk + 'static, { let table_name = table_name.into(); - let scan_and_filter = self.scan_and_filter(&table_name, predicate, chunks).await?; + let scan_and_filter = self.scan_and_filter(&table_name, predicate, chunks)?; let TableScanAndFilter { plan_builder, @@ -937,7 +917,7 @@ impl InfluxRPCPlanner { /// GroupBy(gby cols, aggs, time cols) /// Filter(predicate) /// Scan - pub async fn read_group_plan( + pub fn read_group_plan( &self, table_name: impl Into, predicate: &Predicate, @@ -949,7 +929,7 @@ impl InfluxRPCPlanner { C: PartitionChunk + 'static, { let table_name = table_name.into(); - let scan_and_filter = self.scan_and_filter(&table_name, predicate, chunks).await?; + let scan_and_filter = self.scan_and_filter(&table_name, predicate, chunks)?; let TableScanAndFilter { plan_builder, @@ -1027,7 +1007,7 @@ impl InfluxRPCPlanner { /// GroupBy(gby: tag columns, window_function; agg: aggregate(field) /// Filter(predicate) /// Scan - pub async fn read_window_aggregate_plan( + pub fn read_window_aggregate_plan( &self, table_name: impl Into, predicate: &Predicate, @@ -1040,7 +1020,7 @@ impl InfluxRPCPlanner { C: PartitionChunk + 'static, { let table_name = table_name.into(); - let scan_and_filter = self.scan_and_filter(&table_name, predicate, chunks).await?; + let scan_and_filter = self.scan_and_filter(&table_name, predicate, chunks)?; let TableScanAndFilter { plan_builder, @@ -1114,7 +1094,7 @@ impl InfluxRPCPlanner { /// Filter(predicate) [optional] /// Scan /// ``` - async fn scan_and_filter( + fn scan_and_filter( &self, table_name: &str, predicate: &Predicate, @@ -1190,7 +1170,7 @@ impl InfluxRPCPlanner { /// Returns a list of chunks across all partitions which may /// contain data that pass the predicate - async fn filtered_chunks( + fn filtered_chunks( &self, database: &D, predicate: &Predicate, diff --git a/server/src/query_tests/influxrpc/field_columns.rs b/server/src/query_tests/influxrpc/field_columns.rs index dc6f27df34..f0751375b6 100644 --- a/server/src/query_tests/influxrpc/field_columns.rs +++ b/server/src/query_tests/influxrpc/field_columns.rs @@ -35,7 +35,6 @@ macro_rules! run_field_columns_test_case { let plan = planner .field_columns(&db, predicate.clone()) - .await .expect("built plan successfully"); let fields = executor .to_field_list(plan) @@ -137,7 +136,6 @@ async fn test_field_name_plan() { let plan = planner .field_columns(&db, predicate.clone()) - .await .expect("built plan successfully"); let mut plans = plan.plans; diff --git a/server/src/query_tests/influxrpc/read_filter.rs b/server/src/query_tests/influxrpc/read_filter.rs index b3831d39a4..b11b10f3e8 100644 --- a/server/src/query_tests/influxrpc/read_filter.rs +++ b/server/src/query_tests/influxrpc/read_filter.rs @@ -50,7 +50,6 @@ macro_rules! run_read_filter_test_case { let plan = planner .read_filter(&db, predicate.clone()) - .await .expect("built plan successfully"); let string_results = run_series_set_plan(executor, plan).await; diff --git a/server/src/query_tests/influxrpc/read_group.rs b/server/src/query_tests/influxrpc/read_group.rs index a614180183..aac4f408ee 100644 --- a/server/src/query_tests/influxrpc/read_group.rs +++ b/server/src/query_tests/influxrpc/read_group.rs @@ -30,7 +30,6 @@ macro_rules! run_read_group_test_case { let plans = planner .read_group(&db, predicate.clone(), agg, &group_columns) - .await .expect("built plan successfully"); let plans = plans.into_inner(); diff --git a/server/src/query_tests/influxrpc/read_window_aggregate.rs b/server/src/query_tests/influxrpc/read_window_aggregate.rs index 8d3b728176..42e64d219d 100644 --- a/server/src/query_tests/influxrpc/read_window_aggregate.rs +++ b/server/src/query_tests/influxrpc/read_window_aggregate.rs @@ -34,7 +34,6 @@ macro_rules! run_read_window_aggregate_test_case { let plans = planner .read_window_aggregate(&db, predicate.clone(), agg, every.clone(), offset.clone()) - .await .expect("built plan successfully"); let plans = plans.into_inner(); diff --git a/server/src/query_tests/influxrpc/table_names.rs b/server/src/query_tests/influxrpc/table_names.rs index 7ef7d041ef..da106479f8 100644 --- a/server/src/query_tests/influxrpc/table_names.rs +++ b/server/src/query_tests/influxrpc/table_names.rs @@ -27,7 +27,6 @@ macro_rules! run_table_names_test_case { let plan = planner .table_names(&db, predicate.clone()) - .await .expect("built plan successfully"); let names = executor .to_string_set(plan) diff --git a/server/src/query_tests/influxrpc/tag_keys.rs b/server/src/query_tests/influxrpc/tag_keys.rs index 39c940bf87..30e93f3117 100644 --- a/server/src/query_tests/influxrpc/tag_keys.rs +++ b/server/src/query_tests/influxrpc/tag_keys.rs @@ -31,7 +31,6 @@ macro_rules! run_tag_keys_test_case { let plan = planner .tag_keys(&db, predicate.clone()) - .await .expect("built plan successfully"); let names = executor .to_string_set(plan) diff --git a/server/src/query_tests/influxrpc/tag_values.rs b/server/src/query_tests/influxrpc/tag_values.rs index 484dc41ab3..09a08ec163 100644 --- a/server/src/query_tests/influxrpc/tag_values.rs +++ b/server/src/query_tests/influxrpc/tag_values.rs @@ -29,7 +29,6 @@ macro_rules! run_tag_values_test_case { let plan = planner .tag_values(&db, &tag_name, predicate.clone()) - .await .expect("built plan successfully"); let names = executor .to_string_set(plan) @@ -239,7 +238,7 @@ async fn list_tag_values_field_col() { // Test: temp is a field, not a tag let tag_name = "temp"; - let plan_result = planner.tag_values(&db, &tag_name, predicate.clone()).await; + let plan_result = planner.tag_values(&db, &tag_name, predicate.clone()); assert_eq!( plan_result.unwrap_err().to_string(), diff --git a/src/influxdb_ioxd/rpc/storage/service.rs b/src/influxdb_ioxd/rpc/storage/service.rs index 94ea311961..bf317869dd 100644 --- a/src/influxdb_ioxd/rpc/storage/service.rs +++ b/src/influxdb_ioxd/rpc/storage/service.rs @@ -714,7 +714,6 @@ where let plan = planner .table_names(db.as_ref(), predicate) - .await .map_err(|e| Box::new(e) as _) .context(ListingTables { db_name })?; let executor = db_store.executor(); @@ -765,7 +764,6 @@ where let tag_key_plan = planner .tag_keys(db.as_ref(), predicate) - .await .map_err(|e| Box::new(e) as _) .context(ListingColumns { db_name: db_name.as_str(), @@ -825,7 +823,6 @@ where let tag_value_plan = planner .tag_values(db.as_ref(), tag_name, predicate) - .await .map_err(|e| Box::new(e) as _) .context(ListingTagValues { db_name, tag_name })?; @@ -882,7 +879,6 @@ where let series_plan = planner .read_filter(db.as_ref(), predicate) - .await .map_err(|e| Box::new(e) as _) .context(PlanningFilteringSeries { db_name })?; @@ -968,14 +964,10 @@ where let grouped_series_set_plan = match gby_agg { GroupByAndAggregate::Columns { agg, group_columns } => { - planner - .read_group(db.as_ref(), predicate, agg, &group_columns) - .await + planner.read_group(db.as_ref(), predicate, agg, &group_columns) } GroupByAndAggregate::Window { agg, every, offset } => { - planner - .read_window_aggregate(db.as_ref(), predicate, agg, every, offset) - .await + planner.read_window_aggregate(db.as_ref(), predicate, agg, every, offset) } }; let grouped_series_set_plan = grouped_series_set_plan @@ -1039,7 +1031,6 @@ where let field_list_plan = planner .field_columns(db.as_ref(), predicate) - .await .map_err(|e| Box::new(e) as _) .context(ListingFields { db_name })?; From 05bf28ce85353ebb4018bef1df676c1f95fd7bad Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Tue, 13 Apr 2021 18:23:55 -0400 Subject: [PATCH 28/45] feat: Add 2 main functions table_schema and table_names for Parquet Chunk ato pay a foundation for querying it --- parquet_file/src/chunk.rs | 33 ++++++++--- parquet_file/src/table.rs | 118 ++++++++++++-------------------------- read_buffer/src/chunk.rs | 13 +++++ server/src/db.rs | 55 +++++++++++++++--- server/src/db/chunk.rs | 14 +++-- 5 files changed, 129 insertions(+), 104 deletions(-) diff --git a/parquet_file/src/chunk.rs b/parquet_file/src/chunk.rs index d352cfd223..5a884ac81f 100644 --- a/parquet_file/src/chunk.rs +++ b/parquet_file/src/chunk.rs @@ -1,15 +1,12 @@ - use snafu::{OptionExt, ResultExt, Snafu}; use std::collections::BTreeSet; use crate::table::Table; -use data_types::partition_metadata::TableSummary; +use data_types::{partition_metadata::TableSummary, timestamp::TimestampRange}; use internal_types::{schema::Schema, selection::Selection}; use object_store::path::Path; use tracker::{MemRegistry, MemTracker}; - - use std::mem; #[derive(Debug, Snafu)] @@ -80,8 +77,15 @@ impl Chunk { } /// Add a chunk's table and its summary - pub fn add_table(&mut self, table_summary: TableSummary, file_location: Path) { - self.tables.push(Table::new(table_summary, file_location)); + pub fn add_table( + &mut self, + table_summary: TableSummary, + file_location: Path, + schema: Schema, + range: Option, + ) { + self.tables + .push(Table::new(table_summary, file_location, schema, range)); } /// Return true if this chunk includes the given table @@ -108,8 +112,8 @@ impl Chunk { size + self.partition_key.len() + mem::size_of::() + mem::size_of::() } - /// Return Schema for the specified table / columns - pub fn table_schema(&self, table_name: &str, selection: Selection<'_>) -> Result { + /// Return Schema for the specified table / columns + pub fn table_schema(&self, table_name: &str, selection: Selection<'_>) -> Result { let table = self .tables .iter() @@ -123,4 +127,17 @@ impl Chunk { .schema(selection) .context(NamedTableError { table_name }) } + + pub fn table_names( + &self, + timestamp_range: Option, + ) -> impl Iterator + '_ { + self.tables.iter().flat_map(move |t| { + if t.matches_predicate(×tamp_range) { + Some(t.name()) + } else { + None + } + }) + } } diff --git a/parquet_file/src/table.rs b/parquet_file/src/table.rs index f642c637ec..5065df2d4e 100644 --- a/parquet_file/src/table.rs +++ b/parquet_file/src/table.rs @@ -1,34 +1,20 @@ use snafu::{ResultExt, Snafu}; use std::mem; -use data_types::partition_metadata::TableSummary; -use internal_types::{schema::{builder::SchemaBuilder, Schema}, selection::Selection}; +use data_types::{partition_metadata::TableSummary, timestamp::TimestampRange}; +use internal_types::{schema::Schema, selection::Selection}; use object_store::path::Path; #[derive(Debug, Snafu)] pub enum Error { - #[snafu(display("Error writing table '{}'", table_name))] - TableWrite { - table_name: String, - }, - - #[snafu(display("Table Error in '{}'", table_name))] - NamedTableError { - table_name: String, - }, - - #[snafu(display("Table '{}' not found in chunk {}", table_name, chunk_id))] - NamedTableNotFoundInChunk { table_name: String, chunk_id: u64 }, - - #[snafu(display("Internal error converting schema: {}", source))] - InternalSchema { - source: internal_types::schema::builder::Error, + #[snafu(display("Failed to select columns: {}", source))] + SelectColumns { + source: internal_types::schema::Error, }, } pub type Result = std::result::Result; - /// Table that belongs to a chunk persisted in a parquet file in object store #[derive(Debug, Clone)] pub struct Table { @@ -39,13 +25,26 @@ pub struct Table { /// //data///.parquet object_store_path: Path, + + /// Schema that goes with this table's parquet file + table_schema: Schema, + + /// Timestamp rang of this table's parquet file + timestamp_range: Option, } impl Table { - pub fn new(meta: TableSummary, path: Path) -> Self { + pub fn new( + meta: TableSummary, + path: Path, + schema: Schema, + range: Option, + ) -> Self { Self { table_summary: meta, object_store_path: path, + table_schema: schema, + timestamp_range: range, } } @@ -62,6 +61,7 @@ impl Table { mem::size_of::() + self.table_summary.size() + mem::size_of_val(&self.object_store_path) + + mem::size_of_val(&self.table_schema) } /// Return name of this table @@ -74,69 +74,23 @@ impl Table { self.object_store_path.clone() } - - /// Return all columns of this table - // pub fn all_columns_selection(&self) -> Result> { - // // TODO - // let cols: Vec = vec![]; - // let selection = TableColSelection { cols }; - - // // sort so the columns always come out in a predictable name - // Ok(selection.sort_by_name()) - // } - - // /// Returns a column selection for just the specified columns - // fn specific_columns_selection<'a>( - // &self, - // columns: &'a [&'a str], - // ) -> Result> { - // // TODO - // let cols: Vec = vec![]; - - // Ok(TableColSelection { cols }) - // } - + /// return schema of this table for specified selection columns pub fn schema(&self, selection: Selection<'_>) -> Result { - - let mut schema_builder = SchemaBuilder::new(); - - // // TODO: maybe just refactor MB's corresponding one - // for col in &selection.cols { - // let column_name = col.column_name; - // let column = self.column(col.column_id)?; - - // schema_builder = match column { - // Column::String(_, _) => schema_builder.field(column_name, ArrowDataType::Utf8), - // Column::Tag(_, _) => schema_builder.tag(column_name), - // Column::F64(_, _) => schema_builder.field(column_name, ArrowDataType::Float64), - // Column::I64(_, _) => { - // if column_name == TIME_COLUMN_NAME { - // schema_builder.timestamp() - // } else { - // schema_builder.field(column_name, ArrowDataType::Int64) - // } - // } - // Column::U64(_, _) => schema_builder.field(column_name, ArrowDataType::UInt64), - // Column::Bool(_, _) => schema_builder.field(column_name, ArrowDataType::Boolean), - // }; - // } - - schema_builder.build().context(InternalSchema) - - - // translate chunk selection into name/indexes: - // let selection = match selection { - // Selection::All => self.all_columns_selection(), - // Selection::Some(cols) => self.specific_columns_selection(cols), - // }?; - // self.schema_impl(&selection) + Ok(match selection { + Selection::All => self.table_schema.clone(), + Selection::Some(columns) => { + let columns = self.table_schema.select(columns).context(SelectColumns)?; + self.table_schema.project(&columns) + } + }) } - // fn schema_impl(&self, selection: &TableColSelection<'_>) -> Result { - // let mut schema_builder = SchemaBuilder::new(); - - // // TODO: maybe just refactor MB's corresponding one - - // schema_builder.build().context(InternalSchema) - // } + pub fn matches_predicate(&self, timestamp_range: &Option) -> bool { + match (self.timestamp_range, timestamp_range) { + (Some(a), Some(b)) => !a.disjoint(b), + (None, Some(_)) => false, /* If this chunk doesn't have a time column it can't match */ + // the predicate + (_, None) => true, + } + } } diff --git a/read_buffer/src/chunk.rs b/read_buffer/src/chunk.rs index 5e83c0e9ad..7ad1c7cd08 100644 --- a/read_buffer/src/chunk.rs +++ b/read_buffer/src/chunk.rs @@ -316,6 +316,19 @@ impl Chunk { Ok(table.read_filter(&select_columns, &predicate)) } + /// Returns timestamp range for specified table + pub fn read_time_range(&self, table_name: &str) -> Result> { + // read lock on chunk. + let chunk_data = self.chunk_data.read().unwrap(); + + let table = chunk_data + .data + .get(table_name) + .context(TableNotFound { table_name })?; + + Ok(table.time_range()) + } + /// Returns an iterable collection of data in group columns and aggregate /// columns, optionally filtered by the provided predicate. Results are /// merged across all row groups within the returned table. diff --git a/server/src/db.rs b/server/src/db.rs index 94d60de241..d7f9a64be6 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -3,6 +3,7 @@ use std::any::Any; use std::{ + convert::TryInto, num::NonZeroU32, sync::{ atomic::{AtomicU64, AtomicUsize, Ordering}, @@ -15,15 +16,19 @@ use observability_deps::tracing::{debug, info}; use parking_lot::{Mutex, RwLock}; use snafu::{ensure, OptionExt, ResultExt, Snafu}; -use arrow_deps::datafusion::{ - catalog::{catalog::CatalogProvider, schema::SchemaProvider}, - physical_plan::SendableRecordBatchStream, +use arrow_deps::{ + arrow::datatypes::SchemaRef as ArrowSchemaRef, + datafusion::{ + catalog::{catalog::CatalogProvider, schema::SchemaProvider}, + physical_plan::SendableRecordBatchStream, + }, }; use catalog::{chunk::ChunkState, Catalog}; pub(crate) use chunk::DBChunk; use data_types::{ chunk::ChunkSummary, database_rules::DatabaseRules, partition_metadata::PartitionSummary, + timestamp::TimestampRange, }; use internal_types::selection::Selection; use object_store::ObjectStore; @@ -115,6 +120,18 @@ pub enum Error { chunk_id: u32, }, + #[snafu(display("Read Buffer Schema Error in chunk {}: {}", chunk_id, source))] + ReadBufferChunkSchemaError { + source: read_buffer::Error, + chunk_id: u32, + }, + + #[snafu(display("Read Buffer Timestamp Error in chunk {}: {}", chunk_id, source))] + ReadBufferChunkTimestampError { + chunk_id: u32, + source: read_buffer::Error, + }, + #[snafu(display("Error writing to object store: {}", source))] WritingToObjectStore { source: parquet_file::storage::Error, @@ -135,6 +152,11 @@ pub enum Error { #[snafu(display("Error building sequenced entry: {}", source))] SequencedEntryError { source: entry::Error }, + + #[snafu(display("Error building sequenced entry: {}", source))] + SchemaConversion { + source: internal_types::schema::Error, + }, } pub type Result = std::result::Result; @@ -487,12 +509,16 @@ impl Db { let read_results = rb_chunk .read_filter(stats.name.as_str(), predicate, Selection::All) .context(ReadBufferChunkError { chunk_id })?; - let schema = rb_chunk + let arrow_schema: ArrowSchemaRef = rb_chunk .read_filter_table_schema(stats.name.as_str(), Selection::All) - .context(ReadBufferChunkError { chunk_id })? + .context(ReadBufferChunkSchemaError { chunk_id })? .into(); - let stream: SendableRecordBatchStream = - Box::pin(streams::ReadFilterResultsStream::new(read_results, schema)); + let time_range = rb_chunk + .read_time_range(stats.name.as_str()) + .context(ReadBufferChunkTimestampError { chunk_id })?; + let stream: SendableRecordBatchStream = Box::pin( + streams::ReadFilterResultsStream::new(read_results, Arc::clone(&arrow_schema)), + ); // Write this table data into the object store let path = storage @@ -506,7 +532,20 @@ impl Db { .context(WritingToObjectStore)?; // Now add the saved info into the parquet_chunk - parquet_chunk.add_table(stats, path); + let schema = Arc::clone(&arrow_schema) + .try_into() + .context(SchemaConversion)?; + let table_time_range = match time_range { + None => None, + Some((start, end)) => { + if start < end { + Some(TimestampRange::new(start, end)) + } else { + None + } + } + }; + parquet_chunk.add_table(stats, path, schema, table_time_range); } // Relock the chunk again (nothing else should have been able diff --git a/server/src/db/chunk.rs b/server/src/db/chunk.rs index f442360045..270961550d 100644 --- a/server/src/db/chunk.rs +++ b/server/src/db/chunk.rs @@ -171,9 +171,7 @@ impl PartitionChunk for DBChunk { chunk.table_names(&rb_predicate, &BTreeSet::new()) } - Self::ParquetFile { .. } => { - unimplemented!("parquet file not implemented for scan_data") - } + Self::ParquetFile { chunk, .. } => chunk.table_names(predicate.range).collect(), }; // Prune out tables that should not be @@ -218,9 +216,13 @@ impl PartitionChunk for DBChunk { Ok(schema) } - Self::ParquetFile { chunk, .. } => chunk - .table_schema(table_name, selection) - .context(ParquetFileChunkError{ chunk_id: chunk.id() }), + Self::ParquetFile { chunk, .. } => { + chunk + .table_schema(table_name, selection) + .context(ParquetFileChunkError { + chunk_id: chunk.id(), + }) + } } } From c9cbc7485759dfd2e630b1252e0f91c707595b62 Mon Sep 17 00:00:00 2001 From: Marko Mikulicic Date: Tue, 13 Apr 2021 15:10:39 +0200 Subject: [PATCH 29/45] feat: Use a DID newtype instead of u32 for dictionary ids Rationale --------- We use `u32` throughout the codebase to reference for interned dictionary strings. We also use `u32` for other reasons and it would be nice to get some help from the compiler to avoid mixing them up --- mutable_buffer/src/chunk.rs | 10 +++--- mutable_buffer/src/column.rs | 6 ++-- mutable_buffer/src/dictionary.rs | 46 ++++++++++++++++-------- mutable_buffer/src/table.rs | 36 +++++++++---------- server/src/query_tests/sql.rs | 2 +- tests/end_to_end_cases/management_api.rs | 6 ++-- tests/end_to_end_cases/management_cli.rs | 2 +- 7 files changed, 63 insertions(+), 45 deletions(-) diff --git a/mutable_buffer/src/chunk.rs b/mutable_buffer/src/chunk.rs index b2711cb7f1..d7cc928123 100644 --- a/mutable_buffer/src/chunk.rs +++ b/mutable_buffer/src/chunk.rs @@ -15,7 +15,7 @@ use tracker::{MemRegistry, MemTracker}; use crate::chunk::snapshot::ChunkSnapshot; use crate::{ - dictionary::{Dictionary, Error as DictionaryError}, + dictionary::{Dictionary, Error as DictionaryError, DID}, table::Table, }; use parking_lot::Mutex; @@ -37,11 +37,11 @@ pub enum Error { }, #[snafu(display("Table {} not found in chunk {}", table, chunk))] - TableNotFoundInChunk { table: u32, chunk: u64 }, + TableNotFoundInChunk { table: DID, chunk: u64 }, #[snafu(display("Column ID {} not found in dictionary of chunk {}", column_id, chunk))] ColumnIdNotFoundInDictionary { - column_id: u32, + column_id: DID, chunk: u64, source: DictionaryError, }, @@ -65,14 +65,14 @@ pub struct Chunk { /// The id for this chunk id: u32, - /// `dictionary` maps &str -> u32. The u32s are used in place of String or + /// `dictionary` maps &str -> DID. The DIDs are used in place of String or /// str to avoid slow string operations. The same dictionary is used for /// table names, tag names, tag values, and column names. // TODO: intern string field values too? dictionary: Dictionary, /// map of the dictionary ID for the table name to the table - tables: HashMap, + tables: HashMap, /// keep track of memory used by chunk tracker: MemTracker, diff --git a/mutable_buffer/src/column.rs b/mutable_buffer/src/column.rs index dfe9617d2b..df6f088235 100644 --- a/mutable_buffer/src/column.rs +++ b/mutable_buffer/src/column.rs @@ -1,6 +1,6 @@ use snafu::Snafu; -use crate::dictionary::Dictionary; +use crate::dictionary::{Dictionary, DID}; use data_types::partition_metadata::StatValues; use generated_types::entry::LogicalColumnType; use internal_types::entry::TypedValuesIterator; @@ -36,7 +36,7 @@ pub enum Column { U64(Vec>, StatValues), String(Vec>, StatValues), Bool(Vec>, StatValues), - Tag(Vec>, StatValues), + Tag(Vec>, StatValues), } impl Column { @@ -356,7 +356,7 @@ impl Column { mem::size_of::>() * v.len() + mem::size_of_val(&stats) } Self::Tag(v, stats) => { - mem::size_of::>() * v.len() + mem::size_of_val(&stats) + mem::size_of::>() * v.len() + mem::size_of_val(&stats) } Self::String(v, stats) => { let string_bytes_size = v diff --git a/mutable_buffer/src/dictionary.rs b/mutable_buffer/src/dictionary.rs index bd5f68fab6..b0920d5c5d 100644 --- a/mutable_buffer/src/dictionary.rs +++ b/mutable_buffer/src/dictionary.rs @@ -8,7 +8,7 @@ use string_interner::{ #[derive(Debug, Snafu)] pub enum Error { #[snafu(display("Dictionary lookup error on id {}", id))] - DictionaryIdLookupError { id: u32 }, + DictionaryIdLookupError { id: DID }, #[snafu(display("Dictionary lookup error for value {}", value))] DictionaryValueLookupError { value: String }, @@ -16,6 +16,30 @@ pub enum Error { pub type Result = std::result::Result; +/// A "dictionary ID" (DID) is a compact numeric representation of an interned +/// string in the dictionary. The same string always maps the same DID. DIDs can +/// be compared, hashed and cheaply copied around, just like small integers. +#[derive(Clone, Copy, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] +pub struct DID(DefaultSymbol); + +impl DID { + fn new(s: DefaultSymbol) -> Self { + Self(s) + } +} + +impl From for DefaultSymbol { + fn from(id: DID) -> Self { + id.0 + } +} + +impl std::fmt::Display for DID { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.to_usize()) + } +} + #[derive(Debug, Clone)] pub struct Dictionary { interner: StringInterner, DefaultHashBuilder>, @@ -39,43 +63,37 @@ impl Dictionary { /// Returns the id corresponding to value, adding an entry for the /// id if it is not yet present in the dictionary. - pub fn lookup_value_or_insert(&mut self, value: &str) -> u32 { + pub fn lookup_value_or_insert(&mut self, value: &str) -> DID { self.id(value).unwrap_or_else(|| { self.size += value.len(); self.size += std::mem::size_of::(); - symbol_to_u32(self.interner.get_or_intern(value)) + DID::new(self.interner.get_or_intern(value)) }) } /// Returns the ID in self.dictionary that corresponds to `value`, if any. /// Returns an error if no such value is found. Does not add the value /// to the dictionary. - pub fn lookup_value(&self, value: &str) -> Result { + pub fn lookup_value(&self, value: &str) -> Result { self.id(value).context(DictionaryValueLookupError { value }) } /// Returns the ID in self.dictionary that corresponds to `value`, /// if any. No error is returned to avoid an allocation when no value is /// present - pub fn id(&self, value: &str) -> Option { - self.interner.get(value).map(symbol_to_u32) + pub fn id(&self, value: &str) -> Option { + self.interner.get(value).map(DID::new) } /// Returns the str in self.dictionary that corresponds to `id`, /// if any. Returns an error if no such id is found - pub fn lookup_id(&self, id: u32) -> Result<&str> { - let symbol = - Symbol::try_from_usize(id as usize).expect("to be able to convert u32 to symbol"); + pub fn lookup_id(&self, id: DID) -> Result<&str> { self.interner - .resolve(symbol) + .resolve(id.into()) .context(DictionaryIdLookupError { id }) } } -fn symbol_to_u32(sym: DefaultSymbol) -> u32 { - sym.to_usize() as u32 -} - #[cfg(test)] mod test { use crate::dictionary::Dictionary; diff --git a/mutable_buffer/src/table.rs b/mutable_buffer/src/table.rs index 687b5dcffa..3ca97766b5 100644 --- a/mutable_buffer/src/table.rs +++ b/mutable_buffer/src/table.rs @@ -3,7 +3,7 @@ use std::{cmp, collections::BTreeMap, sync::Arc}; use crate::{ column, column::Column, - dictionary::{Dictionary, Error as DictionaryError}, + dictionary::{Dictionary, Error as DictionaryError, DID}, }; use data_types::{ database_rules::WriterId, @@ -31,7 +31,7 @@ use arrow_deps::{ #[derive(Debug, Snafu)] pub enum Error { #[snafu(display("Tag value ID {} not found in dictionary of chunk", value))] - TagValueIdNotFoundInDictionary { value: u32, source: DictionaryError }, + TagValueIdNotFoundInDictionary { value: DID, source: DictionaryError }, #[snafu(display("Column error on column {}: {}", column, source))] ColumnError { @@ -46,7 +46,7 @@ pub enum Error { actual_column_type ))] InternalColumnTypeMismatch { - column_id: u32, + column_id: DID, expected_column_type: String, actual_column_type: String, }, @@ -59,7 +59,7 @@ pub enum Error { #[snafu(display("Internal: Column id '{}' not found in dictionary", column_id,))] ColumnIdNotFoundInDictionary { - column_id: u32, + column_id: DID, source: DictionaryError, }, @@ -76,22 +76,22 @@ pub enum Error { column_name, column_id ))] - InternalNoColumnInIndex { column_name: String, column_id: u32 }, + InternalNoColumnInIndex { column_name: String, column_id: DID }, #[snafu(display("Error creating column from wal for column {}: {}", column, source))] CreatingFromWal { - column: u32, + column: DID, source: crate::column::Error, }, #[snafu(display("Error evaluating column predicate for column {}: {}", column, source))] ColumnPredicateEvaluation { - column: u32, + column: DID, source: crate::column::Error, }, #[snafu(display("Row insert to table {} missing column name", table))] - ColumnNameNotInRow { table: u32 }, + ColumnNameNotInRow { table: DID }, #[snafu(display( "Group column '{}' not found in tag columns: {}", @@ -107,21 +107,21 @@ pub enum Error { DuplicateGroupColumn { column_name: String }, #[snafu(display("Column {} not found in table {}", id, table_id))] - ColumnIdNotFound { id: u32, table_id: u32 }, + ColumnIdNotFound { id: DID, table_id: DID }, } pub type Result = std::result::Result; #[derive(Debug, Clone)] pub struct Table { - /// Name of the table as a u32 in the chunk dictionary - pub id: u32, + /// Name of the table as a DID in the chunk dictionary + pub id: DID, /// Map of column id from the chunk dictionary to the column - pub columns: BTreeMap, + pub columns: BTreeMap, } impl Table { - pub fn new(id: u32) -> Self { + pub fn new(id: DID) -> Self { Self { id, columns: BTreeMap::new(), @@ -144,7 +144,7 @@ impl Table { } /// Returns a reference to the specified column - pub(crate) fn column(&self, column_id: u32) -> Result<&Column> { + pub(crate) fn column(&self, column_id: DID) -> Result<&Column> { self.columns.get(&column_id).context(ColumnIdNotFound { id: column_id, table_id: self.id, @@ -468,7 +468,7 @@ impl Table { struct ColSelection<'a> { column_name: &'a str, - column_id: u32, + column_id: DID, } /// Represets a set of column_name, column_index pairs @@ -502,15 +502,15 @@ mod tests { ]; write_lines_to_table(&mut table, &mut dictionary, lp_lines.clone()); - assert_eq!(128, table.size()); + assert_eq!(112, table.size()); // doesn't double because of the stats overhead write_lines_to_table(&mut table, &mut dictionary, lp_lines.clone()); - assert_eq!(224, table.size()); + assert_eq!(192, table.size()); // now make sure it increased by the same amount minus stats overhead write_lines_to_table(&mut table, &mut dictionary, lp_lines); - assert_eq!(320, table.size()); + assert_eq!(272, table.size()); } #[test] diff --git a/server/src/query_tests/sql.rs b/server/src/query_tests/sql.rs index dfe3dc8f5e..eaad832418 100644 --- a/server/src/query_tests/sql.rs +++ b/server/src/query_tests/sql.rs @@ -278,7 +278,7 @@ async fn sql_select_from_system_tables() { "+----+---------------+-------------------+-----------------+", "| id | partition_key | storage | estimated_bytes |", "+----+---------------+-------------------+-----------------+", - "| 0 | 1970-01-01T00 | OpenMutableBuffer | 493 |", + "| 0 | 1970-01-01T00 | OpenMutableBuffer | 453 |", "+----+---------------+-------------------+-----------------+", ]; run_sql_test_case!( diff --git a/tests/end_to_end_cases/management_api.rs b/tests/end_to_end_cases/management_api.rs index 0ea2b48b14..6a9b699342 100644 --- a/tests/end_to_end_cases/management_api.rs +++ b/tests/end_to_end_cases/management_api.rs @@ -277,7 +277,7 @@ async fn test_chunk_get() { partition_key: "cpu".into(), id: 0, storage: ChunkStorage::OpenMutableBuffer as i32, - estimated_bytes: 145, + estimated_bytes: 137, time_of_first_write: None, time_of_last_write: None, time_closing: None, @@ -286,7 +286,7 @@ async fn test_chunk_get() { partition_key: "disk".into(), id: 0, storage: ChunkStorage::OpenMutableBuffer as i32, - estimated_bytes: 107, + estimated_bytes: 103, time_of_first_write: None, time_of_last_write: None, time_closing: None, @@ -452,7 +452,7 @@ async fn test_list_partition_chunks() { partition_key: "cpu".into(), id: 0, storage: ChunkStorage::OpenMutableBuffer as i32, - estimated_bytes: 145, + estimated_bytes: 137, time_of_first_write: None, time_of_last_write: None, time_closing: None, diff --git a/tests/end_to_end_cases/management_cli.rs b/tests/end_to_end_cases/management_cli.rs index 973bf98bc2..765ddb0132 100644 --- a/tests/end_to_end_cases/management_cli.rs +++ b/tests/end_to_end_cases/management_cli.rs @@ -191,7 +191,7 @@ async fn test_get_chunks() { .and(predicate::str::contains( r#""storage": "OpenMutableBuffer","#, )) - .and(predicate::str::contains(r#""estimated_bytes": 145"#)) + .and(predicate::str::contains(r#""estimated_bytes": 137"#)) // Check for a non empty timestamp such as // "time_of_first_write": "2021-03-30T17:11:10.723866Z", .and(predicate::str::contains(r#""time_of_first_write": "20"#)); From 3cfbfe8ebf178970220c865ef6a47961007c2ce1 Mon Sep 17 00:00:00 2001 From: Edd Robinson Date: Tue, 13 Apr 2021 11:18:36 +0100 Subject: [PATCH 30/45] refactor: expose query_tests to benches crate --- server/src/lib.rs | 8 ++++++-- server/src/query_tests/influxrpc.rs | 1 + server/src/query_tests/influxrpc/read_filter.rs | 2 ++ server/src/query_tests/scenarios.rs | 11 +++++++++++ server/src/query_tests/sql.rs | 2 ++ server/src/query_tests/table_schema.rs | 2 ++ 6 files changed, 24 insertions(+), 2 deletions(-) diff --git a/server/src/lib.rs b/server/src/lib.rs index 5d6ce4dca5..8d6f49235e 100644 --- a/server/src/lib.rs +++ b/server/src/lib.rs @@ -105,10 +105,14 @@ use std::num::NonZeroU32; pub mod buffer; mod config; pub mod db; +mod query_tests; pub mod snapshot; -#[cfg(test)] -mod query_tests; +// This module exposes `query_tests` outside of the crate so that it may be used +// in benchmarks. Do not import this module for non-benchmark purposes! +pub mod benchmarks { + pub use crate::query_tests::*; +} type DatabaseError = Box; diff --git a/server/src/query_tests/influxrpc.rs b/server/src/query_tests/influxrpc.rs index 971b033b69..7e031275b2 100644 --- a/server/src/query_tests/influxrpc.rs +++ b/server/src/query_tests/influxrpc.rs @@ -1,3 +1,4 @@ +#![allow(unused_imports, dead_code, unused_macros)] pub mod field_columns; pub mod read_filter; pub mod read_group; diff --git a/server/src/query_tests/influxrpc/read_filter.rs b/server/src/query_tests/influxrpc/read_filter.rs index b11b10f3e8..9bc289c331 100644 --- a/server/src/query_tests/influxrpc/read_filter.rs +++ b/server/src/query_tests/influxrpc/read_filter.rs @@ -9,6 +9,7 @@ use query::{ predicate::{Predicate, PredicateBuilder, EMPTY_PREDICATE}, }; +#[derive(Debug)] pub struct TwoMeasurementsMultiSeries {} #[async_trait] impl DBSetup for TwoMeasurementsMultiSeries { @@ -309,6 +310,7 @@ async fn test_read_filter_data_pred_unsupported_in_scan() { run_read_filter_test_case!(TwoMeasurementsMultiSeries {}, predicate, expected_results); } +#[derive(Debug)] pub struct MeasurementsSortableTags {} #[async_trait] impl DBSetup for MeasurementsSortableTags { diff --git a/server/src/query_tests/scenarios.rs b/server/src/query_tests/scenarios.rs index 4118556e4a..1d52e7d96e 100644 --- a/server/src/query_tests/scenarios.rs +++ b/server/src/query_tests/scenarios.rs @@ -1,5 +1,6 @@ //! This module contains testing scenarios for Db +#[allow(unused_imports, dead_code, unused_macros)] use query::PartitionChunk; use async_trait::async_trait; @@ -9,6 +10,7 @@ use crate::db::{test_helpers::write_lp, Db}; use super::utils::{count_mutable_buffer_chunks, count_read_buffer_chunks, make_db}; /// Holds a database and a description of how its data was configured +#[derive(Debug)] pub struct DBScenario { pub scenario_name: String, pub db: Db, @@ -22,6 +24,7 @@ pub trait DBSetup { } /// No data +#[derive(Debug)] pub struct NoData {} #[async_trait] impl DBSetup for NoData { @@ -76,6 +79,7 @@ impl DBSetup for NoData { } /// Two measurements data in a single mutable buffer chunk +#[derive(Debug)] pub struct TwoMeasurements {} #[async_trait] impl DBSetup for TwoMeasurements { @@ -91,6 +95,7 @@ impl DBSetup for TwoMeasurements { } } +#[derive(Debug)] pub struct TwoMeasurementsUnsignedType {} #[async_trait] impl DBSetup for TwoMeasurementsUnsignedType { @@ -109,6 +114,7 @@ impl DBSetup for TwoMeasurementsUnsignedType { /// Single measurement that has several different chunks with /// different (but compatible) schema +#[derive(Debug)] pub struct MultiChunkSchemaMerge {} #[async_trait] impl DBSetup for MultiChunkSchemaMerge { @@ -128,6 +134,7 @@ impl DBSetup for MultiChunkSchemaMerge { } /// Two measurements data with many null values +#[derive(Debug)] pub struct TwoMeasurementsManyNulls {} #[async_trait] impl DBSetup for TwoMeasurementsManyNulls { @@ -149,6 +156,7 @@ impl DBSetup for TwoMeasurementsManyNulls { } } +#[derive(Debug)] pub struct TwoMeasurementsManyFields {} #[async_trait] impl DBSetup for TwoMeasurementsManyFields { @@ -168,6 +176,7 @@ impl DBSetup for TwoMeasurementsManyFields { } } +#[derive(Debug)] pub struct TwoMeasurementsManyFieldsOneChunk {} #[async_trait] impl DBSetup for TwoMeasurementsManyFieldsOneChunk { @@ -190,6 +199,7 @@ impl DBSetup for TwoMeasurementsManyFieldsOneChunk { } } +#[derive(Debug)] pub struct OneMeasurementManyFields {} #[async_trait] impl DBSetup for OneMeasurementManyFields { @@ -210,6 +220,7 @@ impl DBSetup for OneMeasurementManyFields { } /// This data (from end to end test) +#[derive(Debug)] pub struct EndToEndTest {} #[async_trait] impl DBSetup for EndToEndTest { diff --git a/server/src/query_tests/sql.rs b/server/src/query_tests/sql.rs index eaad832418..4044cd9690 100644 --- a/server/src/query_tests/sql.rs +++ b/server/src/query_tests/sql.rs @@ -3,6 +3,8 @@ //! wired all the pieces together (as well as ensure any particularly //! important SQL does not regress) +#![allow(unused_imports, dead_code, unused_macros)] + use super::scenarios::*; use arrow_deps::{ arrow::record_batch::RecordBatch, assert_table_eq, datafusion::physical_plan::collect, diff --git a/server/src/query_tests/table_schema.rs b/server/src/query_tests/table_schema.rs index 22c3ed869b..def66f012e 100644 --- a/server/src/query_tests/table_schema.rs +++ b/server/src/query_tests/table_schema.rs @@ -1,5 +1,7 @@ //! Tests for the table_names implementation +#![allow(unused_imports, dead_code, unused_macros)] + use arrow_deps::arrow::datatypes::DataType; use internal_types::{schema::builder::SchemaBuilder, selection::Selection}; use query::{Database, PartitionChunk}; From 4db3a4b3b58c3a2c6a64634878836b576e5a9638 Mon Sep 17 00:00:00 2001 From: Edd Robinson Date: Tue, 13 Apr 2021 13:29:03 +0100 Subject: [PATCH 31/45] test: enable writer to split large batches --- internal_types/src/entry.rs | 27 +++++++++++++++++++++++++-- server/src/db.rs | 8 +++++--- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/internal_types/src/entry.rs b/internal_types/src/entry.rs index 6e5963690a..0b74946eb1 100644 --- a/internal_types/src/entry.rs +++ b/internal_types/src/entry.rs @@ -1231,8 +1231,13 @@ pub mod test_helpers { use chrono::TimeZone; use influxdb_line_protocol::parse_lines; - /// Converts the line protocol to a vec of ShardedEntry with a single shard - /// and a single partition + // An appropriate maximum size for batches of LP to be written into IOx. Using + // test fixtures containing more than this many lines of LP will result in them + // being written as multiple writes. + const LP_BATCH_SIZE: usize = 10000; + + /// Converts the line protocol to a single `Entry` with a single shard and + /// a single partition. pub fn lp_to_entry(lp: &str) -> Entry { let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); @@ -1243,6 +1248,24 @@ pub mod test_helpers { .entry } + /// Converts the line protocol to a collection of `Entry` with a single + /// shard and a single partition, which is useful for testing when `lp` is + /// large. Batches are sized according to LP_BATCH_SIZE. + pub fn lp_to_entries(lp: &str) -> Vec { + let lines: Vec<_> = parse_lines(&lp).map(|l| l.unwrap()).collect(); + + lines + .chunks(LP_BATCH_SIZE) + .map(|batch| { + lines_to_sharded_entries(batch, sharder(1).as_ref(), &hour_partitioner()) + .unwrap() + .pop() + .unwrap() + .entry + }) + .collect::>() + } + /// Returns a test sharder that will assign shard ids from [0, count) /// incrementing for each line. pub fn sharder(count: u16) -> Option { diff --git a/server/src/db.rs b/server/src/db.rs index a2faca2585..7ecf2ab820 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -799,11 +799,13 @@ impl CatalogProvider for Db { pub mod test_helpers { use super::*; - use internal_types::entry::test_helpers::lp_to_entry; + use internal_types::entry::test_helpers::lp_to_entries; pub fn write_lp(db: &Db, lp: &str) { - let entry = lp_to_entry(lp); - db.store_entry(entry).unwrap(); + let entries = lp_to_entries(lp); + for entry in entries { + db.store_entry(entry).unwrap(); + } } } From 9834c845dbb87c0cc11ef9157447402d0dcc4d1e Mon Sep 17 00:00:00 2001 From: Edd Robinson Date: Tue, 13 Apr 2021 14:56:15 +0100 Subject: [PATCH 32/45] test: add influxrpc tag_values benches MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The initial benchmarks look like this on my i9 MBP: ``` Data in one open chunk and one closed chunk of mutable buffer/tag0/no_pred 1.00 91.0±2.55ms ? ?/sec Data in one open chunk and one closed chunk of mutable buffer/tag0/with_pred 1.00 11.5±0.72ms ? ?/sec Data in one open chunk and one closed chunk of mutable buffer/tag1/no_pred 1.00 120.3±5.10ms ? ?/sec Data in one open chunk and one closed chunk of mutable buffer/tag1/with_pred 1.00 11.2±0.22ms ? ?/sec Data in one open chunk and one closed chunk of mutable buffer/tag2/no_pred 1.00 203.2±8.45ms ? ?/sec Data in one open chunk and one closed chunk of mutable buffer/tag2/with_pred 1.00 11.2±0.21ms ? ?/sec Data in open chunk of mutable buffer, and one chunk of read buffer/tag0/no_pred 1.00 100.3±3.73ms ? ?/sec Data in open chunk of mutable buffer, and one chunk of read buffer/tag0/with_pred 1.00 31.2±1.80ms ? ?/sec Data in open chunk of mutable buffer, and one chunk of read buffer/tag1/no_pred 1.00 126.7±2.29ms ? ?/sec Data in open chunk of mutable buffer, and one chunk of read buffer/tag1/with_pred 1.00 33.0±1.70ms ? ?/sec Data in open chunk of mutable buffer, and one chunk of read buffer/tag2/no_pred 1.00 212.0±6.86ms ? ?/sec Data in open chunk of mutable buffer, and one chunk of read buffer/tag2/with_pred 1.00 18.1±0.99ms ? ?/sec Data in single open chunk of mutable buffer/tag0/no_pred 1.00 98.7±6.08ms ? ?/sec Data in single open chunk of mutable buffer/tag0/with_pred 1.00 11.2±0.37ms ? ?/sec Data in single open chunk of mutable buffer/tag1/no_pred 1.00 118.9±3.97ms ? ?/sec Data in single open chunk of mutable buffer/tag1/with_pred 1.00 11.7±0.64ms ? ?/sec Data in single open chunk of mutable buffer/tag2/no_pred 1.00 202.1±8.49ms ? ?/sec Data in single open chunk of mutable buffer/tag2/with_pred 1.00 11.1±0.27ms ? ?/sec Data in two read buffer chunks/tag0/no_pred 1.00 109.2±5.20ms ? ?/sec Data in two read buffer chunks/tag0/with_pred 1.00 44.2±1.83ms ? ?/sec Data in two read buffer chunks/tag1/no_pred 1.00 132.9±3.79ms ? ?/sec Data in two read buffer chunks/tag1/with_pred 1.00 41.7±2.43ms ? ?/sec Data in two read buffer chunks/tag2/no_pred 1.00 222.4±7.00ms ? ?/sec Data in two read buffer chunks/tag2/with_pred 1.00 27.9±0.92ms ? ?/sec ``` --- Cargo.lock | 4 + server/Cargo.toml | 9 +- server/benches/influxrpc.rs | 8 ++ server/benches/tag_values.rs | 122 ++++++++++++++++++++++ tests/fixtures/lineproto/tag_values.lp.gz | Bin 0 -> 3509787 bytes 5 files changed, 142 insertions(+), 1 deletion(-) create mode 100644 server/benches/influxrpc.rs create mode 100644 server/benches/tag_values.rs create mode 100644 tests/fixtures/lineproto/tag_values.lp.gz diff --git a/Cargo.lock b/Cargo.lock index c0d99a7f6c..2cda0a48d8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -599,6 +599,7 @@ dependencies = [ "clap", "criterion-plot", "csv", + "futures", "itertools 0.10.0", "lazy_static", "num-traits", @@ -611,6 +612,7 @@ dependencies = [ "serde_derive", "serde_json", "tinytemplate", + "tokio", "walkdir", ] @@ -3262,8 +3264,10 @@ dependencies = [ "bytes", "chrono", "crc32fast", + "criterion", "data_types", "flatbuffers", + "flate2", "futures", "generated_types", "influxdb_line_protocol", diff --git a/server/Cargo.toml b/server/Cargo.toml index 0cc7e6de7d..80da4b5069 100644 --- a/server/Cargo.toml +++ b/server/Cargo.toml @@ -35,5 +35,12 @@ tracker = { path = "../tracker" } uuid = { version = "0.8", features = ["serde", "v4"] } [dev-dependencies] # In alphabetical order -test_helpers = { path = "../test_helpers" } +criterion = { version = "0.3.4", features = ["async_tokio"] } +flate2 = "1.0.20" tempfile = "3.1.0" +test_helpers = { path = "../test_helpers" } + +[[bench]] +name = "influxrpc" +harness = false + diff --git a/server/benches/influxrpc.rs b/server/benches/influxrpc.rs new file mode 100644 index 0000000000..a47faee05b --- /dev/null +++ b/server/benches/influxrpc.rs @@ -0,0 +1,8 @@ +mod tag_values; + +use criterion::{criterion_group, criterion_main}; + +use tag_values::benchmark_tag_values; + +criterion_group!(benches, benchmark_tag_values); +criterion_main!(benches); diff --git a/server/benches/tag_values.rs b/server/benches/tag_values.rs new file mode 100644 index 0000000000..e39257876b --- /dev/null +++ b/server/benches/tag_values.rs @@ -0,0 +1,122 @@ +use std::io::Read; + +use arrow_deps::datafusion::{logical_plan::Expr, scalar::ScalarValue}; +use criterion::{BenchmarkId, Criterion}; +// This is a struct that tells Criterion.rs to use the "futures" crate's +// current-thread executor +use flate2::read::GzDecoder; +use tokio::runtime::Runtime; + +use query::frontend::influxrpc::InfluxRPCPlanner; +use query::predicate::PredicateBuilder; +use query::{exec::Executor, predicate::Predicate}; +use server::{benchmarks::scenarios::DBScenario, db::Db}; + +// Uses the `query_tests` module to generate some chunk scenarios, specifically +// the scenarios where there are: +// +// - a single open mutable buffer chunk; +// - a closed mutable buffer chunk and another open one; +// - an open mutable buffer chunk and a closed read buffer chunk; +// - two closed read buffer chunks. +// +// The chunks are all fed the *same* line protocol, so these benchmarks are +// useful for assessig the differences in performance between querying the +// chunks held in different execution engines. +// +// These benchmarks use a synthetically generated set of line protocol using +// `inch`. Each point is a new series containing three tag keys. Those tag keys +// are: +// +// - tag0, cardinality 10. +// - tag1, cardinality 100. +// - tag2, cardinality 1,000. +// +// The timespan of the points in the line protocol is around 1m or wall-clock +// time. +async fn setup_scenarios() -> Vec { + let raw = include_bytes!("../../tests/fixtures/lineproto/tag_values.lp.gz"); + let mut gz = GzDecoder::new(&raw[..]); + let mut lp = String::new(); + gz.read_to_string(&mut lp).unwrap(); + + let db = + server::benchmarks::scenarios::make_two_chunk_scenarios("2021-04-12T17", &lp, &lp).await; + db +} + +// Run all benchmarks for `tag_values`. +pub fn benchmark_tag_values(c: &mut Criterion) { + let scenarios = Runtime::new().unwrap().block_on(setup_scenarios()); + + execute_benchmark_group(c, scenarios.as_slice()); +} + +// Runs an async criterion benchmark against the provided scenarios and +// predicate. +fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DBScenario]) { + let planner = InfluxRPCPlanner::new(); + let executor = Executor::new(); + + let predicates = vec![ + (PredicateBuilder::default().build(), "no_pred"), + ( + PredicateBuilder::default() + .add_expr( + Expr::Column("tag2".to_owned()).eq(Expr::Literal(ScalarValue::Utf8(Some( + "value321".to_owned(), + )))), + ) + .build(), + "with_pred", + ), + ]; + + // these tags have different cardinalities: 10, 100, 1000. + let tag_keys = &["tag0", "tag1", "tag2"]; + + for scenario in scenarios { + let DBScenario { scenario_name, db } = scenario; + let mut group = c.benchmark_group(scenario_name); + + for (predicate, pred_name) in &predicates { + for tag_key in tag_keys { + group.bench_with_input( + BenchmarkId::from_parameter(format!("{}/{}", tag_key, pred_name)), + tag_key, + |b, &tag_key| { + b.to_async(Runtime::new().unwrap()).iter(|| { + run_tag_values_query( + &planner, + &executor, + db, + tag_key, + predicate.clone(), + ) + }); + }, + ); + } + } + + group.finish(); + } +} + +// Plans and runs a tag_values query. +async fn run_tag_values_query( + planner: &InfluxRPCPlanner, + executor: &Executor, + db: &Db, + tag_key: &str, + predicate: Predicate, +) { + let plan = planner + .tag_values(db, &tag_key, predicate) + .expect("built plan successfully"); + let names = executor.to_string_set(plan).await.expect( + "converted plan to strings + successfully", + ); + assert!(names.len() > 0); +} diff --git a/tests/fixtures/lineproto/tag_values.lp.gz b/tests/fixtures/lineproto/tag_values.lp.gz new file mode 100644 index 0000000000000000000000000000000000000000..5124a7a57278baba67f6ddb9f5a35d9ec7e27f5a GIT binary patch literal 3509787 zcmce93v^UPw)W}H69NQK0RxkufS{nkA%b2>!l(!c8kJEZA_+Pg8PJ4bkU%Ht4vsPo zU<3qoz+@amRMbRqAZn6sWFRU^6jWe#c8(H}0Y$@W;Gjg(?n=yOdlXGvHohL0leDlX=$1j}zK-qeS z^mM=e#dm$-$bNdr(Z81dyD@I#_=d4Ryj$%U@=NQkUn>8$@Ah1CZ(mExR-Y%y=Q;26 zT}<+w@U@j&z0R{GK9jG^=gIP&KJPqeeRl4no}2e2lz9?PHy5P&n##=exw+1*btk-Y ztX6CK?j(0ZTibt%4keX4x1PN`$6RAgH@BS_Ww)1S`C1+;ce>m5nTt-hwY6R>v$uJ@ zUi?#*Zho}B0EqU>k;psx-5Iy+9GeB`Ii&%?uqH;WUUe8DMc3-Qzp~z%47kNF6 zC+h6Kc<|$F@uRiIj?$Y&oqV(JyuIZ7#WpYgY4LhbSd;dh_B7(+B44;YbZ2YJ`T0lN zsPt@GTb()mlAE5{r`%bGANz5$b84)<7H4fiVLCc6r?mz3HeDKLx1WhWv^hPm&ikdOafV!4`;BLr_|>WW`ip0q*OuyuaGxouN#CDTnPZ+- zSm*u4vr^Q{xD<2l0F_>7+o9IWI#cqan;W?7eM)&?eNqzFEwp|_Peq#zyFG7EE7U@v zXd9jT-W?-9joY2wVrgsjd>ivi^mTD}Nmtw@^^LUci0xtP+{4zzP_h3SW1nk{t!rN? z_`1luw!gWjp?Ur_oz^A!ej8~q_chwA-DYN=cit&A_r6wY?YlhsIo~~z(%L`l8h3W% z_022y+p<^1X0MiR-9O0K=Q?BNb<%xrza(vWN!r=T^3{JGXj@&oe zImGzd5UE$+vC@Dj+qd`huubY=yYE2Lo2H^ZHyAT-Furz!G;+(|q@910e(G#_4wJ*j>o^sap6BRv**$cDHWe*N|p4N(C4OJ68)`?##7S%n#Iovc}Pm$ zr}7O=Gh3&sw>+yn&L|$j&1ZjfrX{6jWUn~O8#}JG(4R58A~TZzq<+JC8R&0yYT3?x zUp?N#_PAmABg2fD^oM>dk?1eZvg7d{j>8kumX#knG9hB?n!(cO)Cmr6zR757xI&s< zmQt~x$Y}FiAtnAJrDAjDrGfCZ56+u9x1+IFnS zXxN(5*r_0+Wo4|8uw`STQ3y8wH_2S|U`B)8S9mj~3S{THYq}T{L;2AtAx_u*DB4iE z)9RZz&pU9W>+zxSG*V0cHCi5*QL;eCG;yso%Zbj;H5k_YFE|cM=)sfwjor;v2 z6yb9b=B2ACQQX>9x2bfl!lDgqWP?GuMvV)B&C(KaQ4v?F$b7jVA-%dP#&JxM3T~C9 zvLKl!`=5^PZ9Yjq$4Mc-?XjVGY00rKH#VL;mMVUi&{?rcYM9md z;G^W~Hv8|tqDHDE^FvXV^i3?Z))`kx-jDAwu7LnspIH@MRn;2{e`DjWT_!ZCOdH%I zRD@E54j-e!i4jUk3ca6SC9c8!t<3D1sap#juHom+q}LD{CFr^!AxrmGbc`Qpj{j}h zPV~L&Ayjh>zQ)kJ_7xt=tQR63b$5(Zgud%_EL6NAGzOj^mb>zsEcD&b2+A<&jKd0B zOJkGw{f&*zn19O4!B6&B4id(kXY4Vmv}Pw)B2DH$Mshu|-pk%^$ap*=xn>5|J1gfU zb^9ZDp$@F5ZeEcp#F@IxY%Tn<>@L!`EFEa(V1d1hCyq;LS-HQx$!z{hiq z{TqmUj%Tu%v1BZuh{`S=({x9`B;fJR{^%R%j-sTg=`V$qCUo~x=y`g|ZuETE>gX3) z_@VOO_07h{Bkj-Yx`KoxjIN)Aqv$pLhVu26lp@_Fp^AG@+0Ksj#bUGygBYp_I<4bL z$g5PI(we+fO@N}J<6z5ElQFX$ff^HiT_F0ZJI5oRO?H;^>QrPDMpt0{qK3};;Po1UiRV!>L{+9KHD%2Jq-^9I& z5y<}N5bRG?`*!Zc-e%U4I#QeI;9rjI-q;AcEu^FhFQC5lJlSoX5fZ~ggZM^911)CZ zA}R`hrh_dhw>51TX12UO{3eqsH1clQZ0pQ8wUVg8hQ0Sh<6b@adv52otfN-cA%vr% zdn@@ar0{Kzx14{nz~f1MevmWSd2UYKv~8)gHaJs1t4)10$~|dSjr-?BcbBUUZJxZP z%6%x&{r;;p?){1G*;gJK@xbN!iKas%?$4`BJ87yrGpFvq4tPrE>~YucZz!F)$2}eY z%vfZ4Zi3x3y~Olzr*l^iT41gEAgQo!;I`ByZ#h%nuT3>Yy1!Xo<9;vEy%hhvo#>v_ z^ZMAjd9=CTL z%0{7V(gtVSjjO8MPba#)L$eB<=}oKL)4yq`H~&1;S-qyV+Gm?$_c%?*s%x7n-C6Ea zCf}U$D)UFLI!$%=U9R7>vdVn(N+&_S)Z1S2oId~f0J}SJ-O;usFBI5K&*C}vj;}W* zmY95rZ#mm$y;9{KiRTO&Z}-gazrZw@%E%_j^BFSPf+BW4ICUrN=?k-K= z5tsc!Z1#(i>xSvlop)KxS3H<;pw8T*vHT8+K;djuaODCD?z^UI{#D5~Yj=ag{?`=8 zLsinquO2S>PrPl2fi9lvB$fXhHGEQJ#Uu&!EYDnPnS;kPe^9zL$+F;DOCdjV*p-G! zSMoF8;s=fxE8YI&rp4!Gx$*RgW2MPamMM+KK0}R}L#3*}^|nptwtG)ZU64+mD4rmz z-WwDjSSdQy==o3e-*}?q-Q?9k*O=%NvZ?1g9;|c3lX;@wF)8IA#We%Ls=wKrepkSC z_fS7%1Vs00*Q6gpZjJ25{YW4oE_t#JtAA5Izb2s%ep3NW&?qlwQ`)j5lDyt;^x}*G zQ6mBkS~d=b>BcBDM_Tp`=0Oku(D6`fX-30FZj+_O_>TqyE;#zb?l~Z0I^-fdi}V}e zOz|@f6r;b5IgMQ>IaT4NsKRX!@+LFfpQSLkNs-DRT$8_NSxiNcO4g)fee*wB6$24F zc1%r_L3{7#7YLXgc~kCjH|*c&zw;n%UZ4!pI=((AUsH`TF>Et!aSP>2qyNd3MgY34 zUY^mgX7L57>MiA^ZWcywV#F?!fOb@6*Gl@ol&E674~1VFW!ZQUV=w>?ckM-~J*Y?a z{HbJ{`NSqe{8&)H^;OI?<(x*ws{sId~fk~d4t$3WLkK^xGismsdAC-A=4Su#J&ShRWj z4N~iqt0Ku6SHT{z172M-^g4)+!8YczzSB$IN7+u;TQN=($dHRE?C%;6F9bxi^A+swdE;AzMr_(p(o2r17LrWUHShouR8gahQ-S^<7c_oaNSgx!LAL_?AH5h70! ze(xLnP6E%C`940UhYHxfvG)wnn?wq^ogqj+&CWPQOoITTdJSjXeLTcR9#{b>$&F_Y zo8rH?L$L)puMeS_a|A~;Q3L_B)tk^?WnPBC)@^7eUN_L8Huk4uvOCVPKfJr2zs$Nz z4zu;WwS5JwyR5ZEES9wxFC8^AFmK@kGkQV`>A#DcMRRgc;{9-OwuILdJc}wF;O0j- zh$ez{`ttVxMUum=Xamv~c?^B1Av4wMmIR>dYw=3Fr+c!^Y@Y zXT?J6;wkN_1at(EJQ+#&+0X!`%LY^%FQHA8n{1DN3cPhm^i}VLWV{7Ctvj738r*X^ zi2iBA%+@$E?0PCyAZ72%8yk~5n6tGAV#Uu>gYk-1*khxWCzecQ=FfOk`)Xkj!d&M6 z`cC*^yzF&Gp)ti9rL6ZAHd>Q`VH-@pw%=qlWU&eljzMeJXr^sbB{>e^g-jUF#~5$% ze0OEehmO*J?Qtg_ZYcd=k?HMw?4~&-rWZS(J8=D@^`>Wm)lf3K#5A(=xz~pjIya=A zyLw1vPT7U(oJ*BCZ=5NtyY-|g7jQ4-q$wHy4BwXeInZ2rZR(~-_tF(r>Hm8e@5KK$ z#(US(@_!HUZcM5GCSX7i*~1pi29ZyK)WlldDJy6uci`KvcE7eZvkPtt_DZe zV#aj7JHP-AV6?M;;G=1O%n}$x+8oKijM+W{W+QvpZbi$jtLqM~0RY)Qn4fn?58GY* zyt_N|!zM>oOx{xP%XES|@5W}o$4KajvC=GtdHWgT!TTNTJoeCNx*;66Yicx6KmAY` z!yuaXD{N;0YM|qvx65!T98TO?(A}zw5H$oy20sMb8Ml-$7Nkd7c6_`K#+noWU2tF= za8f2O2+%3cC*pT{o>dqF( zMz=2{ZjOb+Ex^iu402+Mv~;IJiOiFWQkUg^9z!^;YyP#6Gvm4`D%2Z-BDe(vo*g?L zPDGQ%y=Td=E_!;GNmtS4C#>;r6>BF9zx}bdm?lQ zy?!d82P--HU6$|z3gU8}QV6N35XWzc^r}p>>?5=%Rh26M?Iopp`I&J5;iU)ST=tef z+U{fq$~!r$)4KZq#JQ3@f3(d4>X@4)F;>ioCpgE$H&>DMX11P*Y4lptMTI1)rTs{3M-G zW?X5Fd4v{9{2kX-z?-s_{Uy?oD+oOERXr`~aR&tHgE^U#y3834M9V(gZqgEp09A)q z5g_;|uwHdsCmeL+jDyXkJKcn#6^d18DG`>IvX2nSVQ^99 z$Kp9Vc9UfyES%@!Wv(p1OCFfVU_|pZ;57!p6LmgwrOaVKiSZo*ymSsc%Mo)OctDJs zLN5hOUL{nt#w`%G&RP&48gm^eQF!BG-fOqYjwDBSWYSxgH^Nk6noEIKa@0}D0I&)0 zo3<=VSp>n8TUInVTm{-F7@la?N)1)%kRCE>zBSxrjm6TNE)e3eufr^CRZkH0C%g1E zSd2?DqZ~+oKq2xE0i~?aNokNf4zFLhS6>J9irx9>| zNK&m%w4FfK{&MvP*A!2)${b^O!D6}VtYvlOB6lcjmlEvhG)&hhIhDi{Z8D3vsCF$( zu&!Prdxqs+x2CbtZHk^)Z?6ZC`*e}%C&0EpmzcHyw%t(Z+yU5jLuJm!*$bSXrk;D` z`ah4D_qad!+l!iW+wn z=)RtZM%;7C`{^H6F0IK~;V2yo*!WXJY3d@=@$q)kLnWr2_-As7>FwBak6b&h-gJM7 z>ABc*uU}K>yams{rZVUG3)LGhROS?4s9t=b(%t)vr#`8o&ePjo?lhTeYMgBsFISlx z8!y+mx1RRYmwj07IlE)cX%FRenkR#=D+OQo&I)(w?uL5%)V2oKkPi{Muce zGq}+CzG6G8DMxTAQ`g zyBp6quY4WpEUllv68R!9OY6bzl6T(t2qKS??~8@Du$V=WNL+EC`dkrEmVe@ zqj`^P6b7CpqUKWESCVe7#k^rW$FALPcSxaD-s^^~(!@-uR_;PRHJU~YJ2cIzc$Yb}cYgKW)1z2RiQ z=Xb8@fxK54K(AF=DLo7~S#r5Zg4D77N0H?g&b~t{BZlvO^aD;yn~?3+!xwT(No~iJ zmNYf&-*OxEY2SLz#noJ55uewt;jy=MxsB6t!(b7SZmFiFg~fW{yuuho@e9FZ;d)A8 z(uvx@x9YZ{#Gaei-heqmE`8QR21ng{{Nm=!D2w&|6*kW0d{lO%7$?iwtg6qi0bLZK zA@EsYF)}6}*WjcvDr8lUpbyEy!gq3P;Mw$!>(EDUK$C0%bz^#U_3WN~`7O$0_iJ?D*hdhAbtddbWq8 zsn)Lm(&)@wENwZX8zKt!|B$FcI6(si>U7eI;NYJN=h_V%&f@3@kw9C>>7?z0Js@Ny zq$(?1O3K~>kkpeD*!6G%vX%ASbA~qJF zX;n&CEO#h`sA*xNU;bI1nw3(Upd#XXJ0O`ZrW8FB&d9R-d+mCxLyhn!#Kb2r3q;te z{5Eik>9EzG`;pJGk_Zmq%x9KcCvK#wBkIgPX1TTX06Uw|2k#)VX4!ktSb0^01_+vQ zHYYd{%c)1Naq_qbfyzrbxEAGTiuE3>bJS~?v`~#8uO6x-z+QvX-DP!EdiCeEsdJ;; z?;|tw7vy1Hb!fyxm+O}x5A*kVb=baDKi70<^MjY`pTIv4cjz z*QTESgVVGVDVa|Iv*Di^ApSa?J20rwxd0$>P-V`mN82uJDY}rlGSWQ^5U>Kdi9HUD z7>8U(E6v(wHx&)ORnI2(}t-o)oUr$i308sABDfOnC0sLMnuX0aENn?OZ{d~#8wfMX{1f~32&QFTV|?uzMk3vN zIP8BpI{8u)u#363V847LLE8O>NxRm#9+Gz%L%!^lvs>ONG+h->36J0OWOVUbZ1(Gn z=nmc`H4vruXj(3KvvzUbsW(l`W8Z)^5;KDk$!B`LswlF1{NPdM@urK{6Zdx zs&s?Ha1VZ?t!|R6jAVk4M*zLz=J5n%gAxpu38p|YkA{s2CfBf<;_aU+sjZ`zDeT&5 zxri9CS6GQ$p{7=@P%7O?6dYKF-lhPwgm2Gd;D#9L4w5MM2}P#vX&wlO|$r_6M%?ZvSLTWSU}1;)%C1Z zJE@vl9ZNx#1c7(9Ljn&LU5B&)fQ~#@!Vbc(%+Hk5g}F5qK`MR(r)Q>r>L>xRKjk5* zdGihY9!U=y9LbvoG680PScKaH(pOcH1xOJHx|%pMC8((*0hy*^Cpiqt_Wmjs&rV6I z?P0t3kx%N1x#kM8~{JbsNkQWqIR~-%6Q3ERjqZ=OEf%-CYK4?2s=`c$4B-1Tx zh(M4da?%j)Y5e($OI@p)Urd!_WXWky#IwJJU4K7Ml+!i;d+<3Z%T;zdkW(tQi)yZS zPs&d7M#GU~*aiD$Zr_0KAS_K;|2oD|P^`W^a!6$pN5=ddO`OBZzfXuz}(*j)JpaePp zr-gxmm6_g_nXLr$^mEh%vwI2VXk$_lK z;nEKHp%s}A`z`MP@kY~zZCWzB7;?bBN(MqGTkkpNU-2#A{q!fL!FKOQ&k-?3SV=L zu-buk(5VYs=Y{Z2^92kSI4fxJb|pfrBJ&&*VAR>Lp-ZrJv~c~1QK3dOoQ>yDX9IQb z9@|+pxrOs!<=j$Tn*vKFP7AZacf+>S^#h&rb5D=6kC>2G zcP)0*p)_$@>Rgb2@71P0fq&kuO`Qw?xO{*!^_|+(+p$Xz;OOtzrPuw?=HFeepA6(T z=5qa=3lDkXE)_cC_XF2CY2O{~##=~`FCUQaARu4HzI|=^jmIwIuog!>P6#lSoWSnA z_l{O?JYVT{|Ne4)+2pTGP7rq`?_Q|3ZdlA49-nmuoUm;U%o$~m*rM3% z=NS?Vy-OM)@+{@US$L-zWod$UYA;JuVXoTA(j@+?{VdH=AdU93G*5PzrAhtN&(b`` zurBTStY49($yiRy$`sg6yC(KL1Ykz4#tRAx5J{R~>6-+>DSHjUKyMdz&h6sNO_g;M zsEpM!nyR`TNe6OTYW_ljv-Qv&En7WOP;HHN10Lij1rvtQNw#lseh6>>+T0v@sbs#o z=P&_sWKUzCT!frba|ig2V%$K^$y=Jpb*{?XC5~Z8aXmg<@dJ;4+e$g5vV#(C58lEA z`=~nk5S9ejB7vYc7)pMD7n9&x3S~-%rEnH(2%#7Kob#m6m3Je-?eoE$emVL*CH?X- zk=zp}We|ObD!fUoy(CRidA@IY|C$3=YT`RYdFoLqI_QGcG&X+&G>6&#Mfi^mv51!H zxE53dlt;EYGy%@v{s{}IDsb6!$~Ot>PPWy75oFCSfx{B^UdQj{FgYLEf-3FWKTvlf zTSdf~Ra2`xr}f@dBEd}u;n9W>n~L!MKP9y&S#Du~HgS!*o_0XC5a-MVpi2?GXV=c< z?hh(|!2&zmaC6U|7HcQY%WQoJoAHM6W;_=9I$^bm@eh+eNu(P#o`j9jcUlTVj!aL? zavi@e0U5FVwsQARIwA}-LNYQNdq`yjs3;Ch1pqaP&Av!N@$1UcbecMAH z?;+L0_vbQXs`yQ9oV^zaSU{Xv`*x=z^~7-kfw2lPWwENAul+UsCH+|^ya_B>`$Gww zGt!gNLxbk9svewG%aw)#=B{Xkil|4oKY||lhkSyijgymN_Hjmx&U%vheXv&Ys?*&u z2UrUXmlO^@ckRc&;LMhe@H0T1ek@r&_3OdbJ(O4X zd$4slg5~~gskcA5%Tx#A?w*sT68v-bw$v*&IIBOYO^t|hpIKSs{syU;E32JX1Me*Z zRJ^Xx`2zASudB>ibD{dg#mbyjj?xEKxOW}A{Jncp&qJHXU#{N{%r-8st{)h&Q*-M2 zU<)7qxpG_T#G_xD;_=^Z+fqjW`Mq76Iv~C6SVs|C30{ZLc=Uf?$qa#8EvJz6~CT+U3Oy%UfD2)ODKpzNH$sMhGOOt}qnoLP&2{$cJV2(#PZ?MZ1hG94w64-5RZ1y_9j-pK`&N|1XQBGU?hpssHJx-N2Tx)rkTUvUR z;r*+)rE)?qL|s=|%YJj5($)0XIH5>?XmGndKg#E^p~HdnoObkzJ^sz}l(t8^{B%e_ z6cz1?X(SRYg#FXlKnlpojGlHH43f_Y4l+RG^EE-H8QR;QN9sZkv$4WQ^OVSm8cdv) zmJ0MOj^CuzPA51%3vqm)?Et)4JJ7K&tT%uh{TM@}jJ6`l)c-g2XQonVZ+cnjj#0y_ zmnhnJuPVZqMG@SNlf^8R(k~U;wMxnMO#B)}kV?VLt8&}hn^#%&x*eTqdYXnc?1`K3 zJemqr&x7C|1C+mtn|>k0hM=4N-n?Nd?nAsRgVHyp{NF3dnV*Yv&t6jLPmtGAgyjipXOOLhy=SHont*v`^ED*VAlf69JNu*QcbXG5DgEP$5d zxf1MK?LrD{UHZ8W(p#uEDsEA|-~gm39u;8igfUcLiFq_s z0TvK4?0g4?5p7aeagw0e5Ed*ZGbTD8-=; z`8lK_>Ux(|V0n)^BX-&*#C89ls*|4SfltAc%!y;y^$B%@28FzF=z?N8SQS zV1FS@Lo;Vpw^cXt@TvbHPcySQvB00cA{HvkFL8x@U>)%SWKyInkQ(uW-Xd`Yi;Z!! zHW35xfI#MCzp*02WWOmHrutc%ScT|G8LN)qd97V5HFTISuGDK6z@ILv-oTr-m8?x7 z$l3vZfXr$m0s2;5*~r=qQVI&etu7g21evrB9u)RA%3*eg!MbakJ;OzU z3HXJj|?qzeve&v*q8U-h3ej5_TF)nzVY#~>J!+Ix8Xu{ zE&f^WD18ZB-+vlPix-(Lf!cbi#Priw_3ouWeLHbLCjR*XCvskKX!FF&_2&WV?#Zhg z@|9=%{~pw<`QL_mpOyVTAiXAoX`w(2;m*}uk#PV!*$m#t#vAF#l=e2~$-L9>iM&N@ z#99i)&w^q$ffyZx>vrJ_| znSmKkP#Iuha~}1Rrg-4CI`AZ&4lyD;I->ck!U=}Qbi{s5B12LpiWpw8DGwp^z-!f+>;53jlrHANp_-}7d~WlU9Rkr((&o45TOu|&US@=y zZ#LQ@UPVvsB0)6tNqz@wE=;hFYCmg}NHRoj_qPM?00~U)_P6QDu0_@UufgYm%)2pO z#C>c_wv84MOyl;oE!CQL5q_IqTE|CgZohM(G<}a^?q3s*>4xI%ercPFJ0SHCYU>!? zZ65*m`}VA^`A-u@w3$y0E#CVmik?8xGdN=NTpUDU4yRdIZaJobkC`WTRl) zh=>d8s8K3X%)Wd&d4TI7BA%Fl>iG;RMW^|@C@Mln00!ER$cg+=8Dsj=pP2v3u-jjL zX563eSrggGc)l!i`GNO7NjZPjAIIPL;ZG|s{AN^LH`&`M5TS22KnSOs3y=z=0s;Ie zZ%tH+;OeB;kkf2%sH_xU!|ct>ZXwLVXEXpzg{y`;J#=q&m0rG$&P0n4IW|F0OCe#f_n4nxHiUK{T9b0KyFX5G zi)elRl}`PrQ*=m+{V%hrXRU++aR#R%l4Cf_tv9_l96q!uJHd3#aE6Nx;bb&&zk`L1 zPpk0AAyI3W=l0T_=|STt@z5I8p`Jf_nbpE0KnZ(F528d7T|>}l1)&04yjA_$LVUWK z3!`>nn(SE`##Btgg3+#!r7SeLjWEw1Ew6d`S5p#_W z=5krMxjd-2mS4Rk#oLV2DDWr>cIpZUT}k3;s?A)#Im%-O;WmccJAsxsX(FC{qd|;y zpf55IkK?)$3s0N44CqLnyH8+qtaC4lc=<3K zymYM4XdlwWHwfHOfQk@rkEjCCR8f&!Ay_sXb6}|w33q!B-=LAa?pLcf?-4?WBh&NN zf(1U9V7GZ~b25Kuh{dfwQ@aNbti@vRSI63ec=-72mfpkzg~%o)@3O2vp9D0Y!x!?R z)mg_6r!KQnJcB1;@FExW>F^dDG3kHvhGOcTqGPa#a0X0K7amk~5VZm0PLkn^mlurR zve#Cp$adl7yuyYBs{i)9zGbZ(W-wrnvUJx2Pf;Z*cQ)J;WcecB6VpqbOTkMR)9qtK zKjccCyus?;ky-7kAAk zweBug7T^O6&cdfkoR197G2?VjXW$la^a#&Isr*9Wv!m!E1x_NUc-FP`}~hiUbftL&bC;F)*2Kn+^mx1jhh z4%5i-X6NLE1^5($vv7Qgvn}pYTeaDiX7wF$pZ1u11%+m3R(YM%+}wECTaTT7cF)Gs zr#%gevp#fU6JB$@_1zVAPky4^1BTF4w!6XPE4|0;L>{QKa1@$*;s!J~agf#h{pcF^ z*Hg^S<>jsQz0Wt-Uy;>P?L5|49aD4JgfB9<(^l8Ir@K!%Q_rok`|_^GbIzcXKTas} zZGXMMj7(5>+HLPT>yn;5>M&-E^$ic-{x`@QZ zC|oS23j=!7s|5WWmzr&rA|fpS6vKZPS@Aom84yO~gHjPr?M%Mr;;V(FHNZ#K7X}V5 z05~$^*v@nu+ex7Ed9Z|I-!f!@j#}BT<(=)^?+Dg$+bAtd$76h-|+_~D$Tz~O}gMB?U(%-u+2lb%h2MEh(ua$T0cp!E2f zwFugG-iMgkWu%s(v>EuZ7SAszFxv2821v#NZm?N`-rIq|zn4S>Rqubl|24W}rokaM z(Mazh`#pl0$J-V~#*4LCX{yb07xqN5W^HH4zBgN=Jdr%f=g6ph2c4^CU%j%TDRzJ>AAP#z| zllj5iB%IwjNO1#_u$FSPl%(o!9NprKhl z;DzizdGK;d3_Ed@BC4Vc>M+AAx|b z!{gfA3>Y(V#PzPJ{%ic$hTfW8D z(*WKO&b^{Kd3vfdl+J_s{Ej2me^XaOv7n4wK*m|;Ugw<{85%~>mI<}FGz+v?Ue`-7$$N28QU{4P5F;2Ly101X}_3rR-O#t0E-$CwA()s{2~2CM~GD6D)P zSgA!;6xbf%OZp@Cip)|#<&I$^x%Hv)Q8E1L!Pv&7U~(Khf^G%E7Mkj19fNWJRGLPz zDGqeJc`#_x8IZl*c7b$5fMf&ST6fKB)y0|@3q`w6Rq>47(U)`{wcrW?UHHU^6&*Z8MWI2w4aJ1C{GlQu1RNF75WBio zxK&QBEfg_F-Ow5kvyAL}<2wjtfhtuKFHQ2C&zh4z^0QR3H|QrMoTO^Dx_6pi3b3xh zx6lD@x4yy)M&O`;ap$qg9EKxYtXco~!Frbny{Yq(Pv%o5)B!pu?bYjf5v!U+Ly$6h ze-f;qdu^q>_`(^(BC6H!b?X}bj&Pvb(FTaSb@Dj66{x6ZW^+uB^*^T_cs`x)oycLw zcEm1JLx2hp4zHNElvksKUI!JEr4DTiY8s2ZBXg*KFPuGA{=-)L%valeZKc~**zGl+ z*CL~|pl&Ll*`I-4ZfM57LA&SYMEi)}yHamI_NBA%S)j!0+p0^>Hm3HCvf~?mrmX)o zVC!HFKJeg9+gyvZ(#_VKb%2&1)q-sF*7v^DjL-Nv4-9Int~=d`Z#>vX+|%Sa3Az#6 z2VMA(pVd7Bz_7%D6G6?+O#p^{@bt+5pw~8+4*C;t+B>!QmR|wRiE$R*TjIR>n5 z=+vtKmSX^B_W%Gc0%pXqpumi6z>L^MX!qfS=U29R7ss`D>rVh!`+5Syjs}Kx_QypRT#Up;30*|vVmL06`?plr{oIJp!haca z<`_=OBv2#1`v6e$es{w+-MI+o##s5>7+@~njRT9%;#co1pvK!6HM){;K4|i_1Y)GM zN533*7Qgd-!wiJXM3N~rEk69Psd+%zR)S;|06e%Qjt>aM_3#Jzn!f9I?48BuT22tM zy2>!=DsC*ZhixjicZRQ#5jWV`n`Rhl1p-G6|D9?yd4WSY;u!)oBfBxw`(Z>ge<_9W zWCUSkO1IO%AzsV-PE6whf@l;u4QL0E6&bC__X063iCqNyH#bM$WLIBY%`z>Jzwb^?d^{Z#n&5g#Me6GLG4eP5 zirVsSu-oFd@&?1!Je7%zk5RtueyHYkKE>dBLiQ19>Ng2Unx6Ru_{<-u$YvZAdZI?+yw~(J^(%ksDWPfNW9~8)sW5k; zDp+7<^p`z{)sd^i8BzEsGFcLqmMIaQL+I}TE(*ga%{Xf@O0pY*A%vsQp5>&Y;5}4V z2eDu~OdZ=lnDdU}*g6jp_?}^T;hMA6}vz@{EMc0+44A!Vw68v8Wa?>72<6 zQZV+J!!rCAdN8LVa9=5A`mADM_Eb5 zVdJ}JFy8G(_4e`N53f*w%B?cnD!amgn1;Z6+hj84!fD{upkkW1G?9y_6%Vao4;%fZ8yaXj!%JUZ zxO;4r=Fp3%K{c5{6_%X)A+)K7PL$#4AB_LiYcQF#vSn`$uf}1Mg#FqOP{hODv3g+P z2EVWtkcLkubE*hV+q~y>qbGo?SwTx*m{KiRipAzO@0mw*y{gx>r^q2ZaEz_Z9(?$4 zrXy6e>h#<7J;lkw0c>(iw1`}&bFw-b!T0zeTrCfSTKiM(enfsQ6wLL0Ym&U*dLnnmTNlKGm9bfC`PG`;B89`37$^A>g;eZM;4A1)uuD>PRkZEV z8EnzFe!-8`)!Nh}Ey`ED;e~@a%~!Ve?%m0|GA=Uf=gF?5#|Ac+-_*TeX-4r? zb1#@$&dx39R8zar_R4a{l7h~YuI@bh>P~52Z?ygWP20NVj?Y`yXYI~SIzBo3`|+-s zX;CA)M%}$EGV8?T?4QQF3e%!q>>9QD+wXIK**p01mHS6t*=JPGf(;qP3+^l~9I$Qi z>C|BPxii48`IjeU>kmq*fy$FX%Y>|j-hU`l1Nsk)2`#|}*T_@Ul7T7b2m)%*6 z4iENTx&K!D^(;6hisoM0|5*6kg2OHZG-S!Ptmxi+xuXjDvVmx)*rNXy{U-Xvn8DWR$3tTnVk>Z(ZA| zn$(Cfa#towa#u!2qiDdkxf0%hYQ(K|P6?iBYFSBexh zs#|19#Qk{<WinmofEIX#d61OhR(D5;jh#5 zE5(n?9S@*r-2>(4%INpU==V<3dnrH8QR>(C?t^AJmEN_|ac@d&X&XBhPv+YCBc5Kxw3Zen<-_%D%A4bL?9K1~9h1pBTMjN8t6zDei;e z9?%^REM7ME;wI18Zw;N$0t<6FwID>O6n%R6$F@ty#boOyCfj%yU1Jt^`^!nHziw03 z;f2gQer~ESBH~V(-ZdltZ zF}K_OuH3_8=)t4zvwoId-|rRivNZF8-oMkcw0>Ej5!=GOf7>aqI!aSBLt3ClD=|(B zNc1s5gQ;CBHF)AG!z;#%D>e5{^-v?y7bHU8I~yZ??+uQmmvJt<#}#k7Ly_d}!{z5_ zte`%4U|6H@wOTaS4J~loLU)rQk%s^(KpvG(@>&_Ez=z& zj2jKwqP~N(4rh`643Xa6FOKjWgNzI2rFX-#X+_hhu!i7kRvJ`vUkz5cB0c?C%>9-1 zgxtz~ahOZ20_16r&Xnv8>51GqF^@-_|FP|=c4P-}Qk7e)Y0^i`;E5_vO#iu>14Zj< zN5qh!?aUN6i2!;;St0TaR@#jEIuKtc2BARW=xB%Txa?%ilKxhU-FhU9xa%e7EN zUc02rFO*@6yc@U zs4pm+9lloewWuK$gSOl?sl9Oa+hWw|Sh%8w+OF0gY6}Jh3Rqp#{_Fa_xD)CceYobc zGPADog#8nvv2IoQIeT40qkUjl{XmoV>nZNsQ)_(J4H65TjcN=pX}`gIq~(jF zTam(3!SwLcD6DYu8jf!AxXZGe#Ph;UVyd+6-{zI@;ci@~t7BofzMA58n?Hrc*Hg8! zydVK+5>0rjJhicD`mwEFxJU5nn7s}c-1wV@F6_(WF8xmvT~y89wZddEZ#&x@?cmGYk*Bb~lmYKXRM zjgB1tOk~CEm!n6g^tU_~VH*))%)G_e=N75z=Khum11w`BY!MO0{UZiT$Gb@7ak1Io zNUkjw>FuS`>Ml||u4`kncj6mIHw>4CKOR{zL&|?}j1+gb6cKH4M@A0MlJf5zBQ=bb z4j%iXZL-mrd6TivO;Xi>{+0vRT7ERz1{jU|adyNt{joPXf|78GEOPdGNem(;_)Dp% z?uTCi=;rLTUXph*PA!P)?nwVca@ikAa1BHUR&6lE8_~mlEhD|1 zC-v%7AMp>{({I}TxZIH{bw1OpbC=#+^yfDn)-6YewltqlgrfmSx7{SEkj`niUx9R0 zF=GJfj+$DA4u4``e!Rta=iQ6k(vaa#4ykyOuV=kzoJ&P77u=S#7{63J&Bbe9G_Dsl zXYuv+7md4)e>v*HoP}uP;ETov@ni0gid_CZZX7jRIj8BC!gci#uTWk6hMZREQB(I0u0^EGDM)IMDQtKOGxu8Qfu4evzLKSo&*e9bE&Chx+d9v9vnI0dhp zC!kBvNPO^ZZ4OvL!p7)pja&oL9;6xeA6+0diMev5pN~0n5V#ge%Qhkgv3NN9E zYNRiX9Me2w&=`o~i}>3O5Gu*F=(WzT@NmsM(m2%yccSEkku}9|Wu!(4xOR)UhQ3LP zk5R<`oZ=VOQPCKe1uwh%mnrVrD~vs>u2BR#!YBmWef1g{-Q5jXT+)fn;W~jdUXD!e ztcd6xxs_v&z$dwi2jWT6nxr@rdn*G`So}Q@dPKyPRWC(vyMjFHD@QN+>5EHWOHuLb zf9KjQMP;m?klnDfXr8?^LAmK)YC8momOH)zzJhS<9l{ONfIuf4@vMe%)x~|VPo9jm zExA1fm{4`AtZRj4%_fVPwCg-K99nc*UF#3+DM=?k4x#&v6Jaeh#=6&ZVmYT?@H}u$7A! znH#RlI&9tkw0G^RdLZvbJgs@_vqCb1QFF4%yY?@-jxE7nhIvUWcYW_yj2&tSsJO1z zO@RWH5z92;uB$*rwQ<_Wu)D6z<-kUGR9wjmfgIo-#FD$$JT7%m^OV$U$e}Uso4m-p z{|XfopM!p35Tbds!T!j6>u1V*(;pRBan#9Rw@gKKF9-R31SrYe%UX<_Y7J8gp90sRNQ;~L zUB29R06#e3I&(4u{=M#iX*vejkQRGjCu9(=0zcHSD-lNGX>!FnZjX@C~y{HI3F}vY?)&%{7fG*Z(#0*B#n7srJFQ65Q^9EorZLXn} zGP%3D$^xU1*>);TsGJ#_DRhdz{P!nN>;7q|hvmDS593rhgs>tY_9YkeC#nRdq*0h~?o2G7CPNl`Oic|%` z6G?V-X|7SY&9_mBw@syBS153-ph%hU?GzO0=QnVDD5di>?b?~Vt*5_l!*^Kzjbowa zjFBS7NuFrSiRj3RXCsF{E9K7`BkjFg+I9SC+noS)w@9N$^tU`T!16$Zt#5?!wG?A! ziZpt7e@p6IWA<*z^_fN5*+t3)$oo1r`zy&+VUeC&Dm^tRV$GDOXzA{y(#S4SM(5b< zZL!%l$yI8Rj^luhPO;flvDu$b>zaSh7-`p7X?v7qK7in(krh*={JX|TYojcKBawpe zEQmm+z4BG?p{l8Ai&yZq~@*9 zB)Fzoq~w1ZT>Ym;)OYUgxb2hZsw?~3o_JcSdf_KajxjR7Q&-F4?-H7443I2!Ia2e& zw~LdajYtQJZ2t3g#Vy_^3z35IxJzL(iRWk7Trf=sneHnUw!PoGU??sJvkzUahYJ-4C z2ppU_12~5wbNjctC3?6F$z)&J4;a2&G|R5rirOlB>>)7o%F=Ykd}BzcBX_@Eey%(` zh9fkwZba5$yKif0Ap9d#ry;_EbgY(gM+D)ye>krAd#`SIhN}nI&+qv;b}*vXbcKl5 z!944MMbg?bc)Sfu^R|H2{UGQ7fqkhJglF$dHYylvh_XP?a(m>{&HL5%YQ~ zTAy_|xvukg*N1Ellj9UHScz*kht4=|W%!tFYoG2u`ewR$+vcZlJT(34e>i5O#Ma}V znJJyPJ6kuZyxlX3pBN*Q@t=4OQb@c5Q)g!vc_!Iv-M}PF9ne#|elbvPhOS0eHN0w4 zS-R|@seG3l(**UV*k!$FEkv_hHSWQdojd_@#IqZKf#5pJ&PXsIf%H+72WO;7&lyoH z9ds!IEL6q!^|DCcH(kkTam61V?@B)WNd*}r>pa=iu<_-m=~^Oy3t^shDJY#bu75wB z1?lgOkae8K%Z1r#1<}Y*QT3q_eZF`iP_mk^xmMPNZ-^DoShyvR5#0|!bU!kpYn~6F zx0rz*JFh3;Fw6@VK9)V%bL=q4GSfjIV_>%PPHgfF4U7!ScdH{vk8+!HXz}1LxE(Qf z;SZFD0eD>$aY0!muxNf2gw}D0si;y!&Ha74sg>+qXzq&wvbDf*AkMCs^OWX!2Qtu_ zn~M082%7gl#18D6em<-W1&Dkz3FbUps3IF!y(5ID_{ z@BN~I<{S(akg?o|K298tAYmgpW5M2@c8yiXtp&_;*$HPv!sMHRYQo7@_~x=ls*A;l z2UH<=F|BC-n1Yax3!Ne069QS}8IQ>d^dwyG_GF71-+kfnJ@Xk^C3U6*Odc4qwxspp z6o`-}8)ta*Vpv|gFX6JDG8#5bQQfUVPdFM$mB1@7>3sDPk^KA5YKg>9+!7o;A=(Lp8tO&mK+(NwP#uu>i z6b400_bG;0L=I7a#F-?K(I3VzY?cw-RJ`nb`8aoDmgS|rwo5--r0AEWOI@Y9uCdvN zW3vxQuAe}|Et8PHXITt@H%rQYYK*k$ZfSkAWfWMwnUNJwOZnMjB-8Oo+d6||p24v6 zYN@_gciXEyY%2_o83x0dK89~zGw$>Z8v2x!{qNZ9?UL&)&~Hnn=q{2IIIc!={T&c+ ziL_Y`#@*Ct5u|GdTVfn+GX zbCzwQ!8rQGXxpQp^R63XS@WHgANxUZ(nzD!To7$3&jGUQFCh)rl9AK%R&!>I0f1$kKgKE-Rnidb;&pNY#rR2aYg!P<6T*g5wtT7J^qWk=}4-?r~&Di08%0ylTd_6 z$4s>#9kV0q?w3&)KvBu*m;^dRIwp!T;q4K@hICBqEEZ6V6))3Gtcztdw$c70_LVZT z14t#p1$@m(fC`?=mBD8SF~?yf0-rw-#}$hAn37}dPaIe16NXn)HzP=B`)>s+rvPRd z4+KzZv~1x$?B%+F5KaYI+uS3_s0*60;11Ym2siDY%N;`ny1PB0Q`%bqnU>A)x$&N) zA_z(f_z`eT&nkhuWG60=!4o2-fLgq9Oy=}R1wDmiW0Jj{NsuZwQKjn&RfrXf`%vSS z!(iz$JAtJuc5S~8aE`?Km!!94L^pN+^tfB$pP6oA3CF1UmLvFjihPay9n-Cff!nH9 zN)tB`EeEucn-M$S^`i2|b+M2D{wVKkf`Cyua3>p2+{`HqrIeD%4%%3iR3ozPY5|1X zW$u;}?wpX)ss66#9WU|SrDJCuaRpO%l!2DfRrF%K>px1z;)V7;iYXb8ffm+{xRrfX zB?hiq9@Q8j)y=FrV5-s59558wz63_d6QnbFYg&j@t&hGWkww<^ScE2~bWAn>jOP~_ zb)H{rl28Psmd09pfKKH22m>?%d~%po*CQb-hjdJcsk;&C?cUtSU}@|gG*nj{H}wUyRT;I3x2nMsNJ8jV5bl+iNG}e zzKrafkfWkxtBL;yc)(NlD_$91oLrw!(WrtDHHef6n=FW)z{O&AU-oqezi_^eXRPX7 zz@g^6G~JOFsX#f$8NCW8ONWCaVgprL2zA3T6Jc$D5z!8NdHL;0fHkmB8rLC&xAv}`eB_iyH)rm`^nJL$WGzjQd%i(9tgy0>NVNi zaNS@%J#sgc3>fb)%Ndfu-@}5U3PlL&5Kt>BQdYwk^3rFU_-!$iH2LLzNy`y}w$?@p z56V^ae(-SfHF0jJA^xwK9DgCJY}s4Gy=>+JCYG4M8msF?Fy&1b&z>zkTKT_0AfLLj$i9rm>p_XGVL8s35Mit~6m=O9Ry{)GARs7|6S zogqUJ(OqTR$LhAF4$QhY%Fp>)$Mv~9>U#g_i>`F1rR_)3`C5{OTvSqc)VB*z%J8IE z3jE5tYLaWn8${$dS@57#AnHT5Jj7yZD+Fss7ZNH~z_znx-+IqQ#&u_R7z|zd8P4>T zO8a)Veb&QPX>fdCFih%e7`Ar3As$$7pXB<+B0aoJy1T2C+a)%8Pi*%0lFMe1E`8I< za$AJ$w9(jSgj987f6Mg)EQ2C!+mTUuv+=c?yG|wMeU%NH4}o&e+)OEwR~vOMkUUBk_r6Tz_TjD1EO-4FOD(Wm+y{ob$XHiTK)yF`ac#dFEL@%Bx;QB!y-TEo)KAyRNygKQ z=OI;d-Au>gXYlV@?55k2;+O$e@KNmMdlw&(Gse1-rb^9Dz`34dEWWSzSYiwy7hf*w zt1wTRXu1ojv#Uk?p~{mE|Yx=F8tuUW#5Cr3n12be^)CC)IJD zh)%s5`_ZfHL{_GA^z**!u4$L9E12c$R@9vPV0!=$vAo-_p**y}9MPFaE)eYFj85!v zgAV%5pr-ULi^C*dvOfjV&N^ZWQp`wskeo!6M&{VKsHW^IgwXQ-F84g+X^~~Bm!b*v zar=%Fa*m6C8~(fl5<?qbu=L_*V(vTvSo!x#=uZhYb3ECZmhy)=&fyzs1@HMO78_(5u% zo;Ny)pC2LfZTfI;%xmh-JNPQ89kO;CbD{?3PhnB4iGo~MG}8oLgmj{a1&i-;zw5e$ z$c!$qp!d6mO^quF-ggIW*YK`_Bd7OOpyf6y>QqWjEgTWqaJLi96#hY&{a&IG*>0=$ zQ1ze%9OSIhK%ZE4-cyXM?^k1dvCl7GEyn_vSB7Uy#SO6N*hCEOF-$;vu+4@9VMv4Ke;I8Y8vg*=N zu4cy=Q1ST#1^!7MXQEMvSlP*?MjSL}K@^EFyS#By-_HY;7Yx-u86L!9VcBy@u+zfI zgI!t#$Yk;RdO_6`ze;ng6zOQf6$E{z#LvRyLcYoQn}uFv zL6Z4l|5bKj{RgAepJ*CG>pRHx#k6-JoYxT#4Yjd=r@s!;=cg<1I{XGOh=*F?wXdyb z6|~7kSl=p057r}?iO5$uZWBO8h&zBSQAY%fYmyQ)Q43`p$=9 zBI{B=dZtmDg7gwgJj~?PX{9@|OZ_zk0m)wluREB%1S7FAU#w)EbqkYq-n{hi?>>I( z%d5t2yRk6#ul+AsZ(cs)U#kwxx$f_)O7={hm$+j`2RvVfvP&#@=+x_t9dc2dJ${KO z{@_?MgP(^0XYIG{v{5{};Y^LXLp)qo?EbQXq1#krcqg|YTJst%AZ)H|(^qX$ zARJ^f9vbn{2BA>fUb61WpJ>x(C~ACeI_<~S?ore4wUw}7UY5ConeFR$*vWhJ7XQ!f z(aT%-Q1<^6&6k~j8_2Y=7G;l~j0Y)cvux{3+Y&&;IejJX7Lb5(-5q1G4ezd5wnstl zE&j=}2AP^&x>~kkx85WqbYiz&_o)%g@?E@IDl)xY93No-3^lf%nq=HZc(7;=%CJSR z^*>#tba{*3`%#vpf2BAck_<)P8Eti9hhDmxvuWQ6I;nb|(u-)_4+9gNr*+TS(&55yK3C%C00V?S_w=P5IkQ-BgFLsJ+*CJ(1~71v)OF*Lsek`H=@IFG(c z7Ut*y!Y_z8a$$v_qIQ2t`K6QCxi}&3*BhAsfCA{o5yrcFIud7}1U=ebwq_N!bPxW1 zVEz?d7Y*io%=}&-yl`Zdp>yKh&pX~QbS}MnCD_6R^UN=HkgloPYh4XVzJWNh0eQ1z z!F5@>3w{SO==;yMUzC7bu;;8^pA-ow@&eGZG_;W1rfNc^bV%g;Hp zjT9}+6G&5NR9$sS&H^l3MmWcW1w3-M`Q>xxnaXSW zA4}1UPW&IF@@C8X?MC5PmDv`_eB^CI3FtdF_Sne@n(5C**!~-*s}PfCO^dMQ)}^@Q z1kJ2Bu6HzhD!NIwWNf&b(O0r$fy>)?W7qsXV=QH$9M|K6;+^~3;z8xzHZh_!vb$rW zlA!tJ!RDdEA7QMu{srTvv3wfHqeF&IrobVAIbTNljW+_rEi54+fG-*3Hyj8o1oB9zDeSKu`~U+TFqu-p{e&VZ9uYUtP$ z-X16~%0M~|JH<9*TO@-*_$!sE9Nk5(zfNeup5`TS6&0*~ie zQ)8tW3wtPm;c@DIy54eN7{<7L^akwFGGn0hh0NJ$JM9Y8lT~IcxY2P2JEx{G)frER zWU_gDim%J%YuK*^9e_93^udvOhh>gdDLa7ZjyBpM>eUft2df@J*&Q4japmZx(c5mv zKg+Jed3*2ttc9>Z6)PF`bEA1_O7Uk5{glHOmF2;JbMvk!Uwxw zzryeB0}$z-lBqf$Y`0k2M}KS^9WFhFV{H3xog!o7aC8NHGgPSyOB!{*4>}$RoYWb` z{;BpVz#iIGv4OU|0)L+4hz6VmnhHrea*M@3-Pl z@?$-Z9)KH~H&u^#wHWXIeJZ(=PKBE863-{`b$m=nZ(ZPPm$G9!qV=3@Udx_HoQc{Y zHL9@#jqOdUNVPzvdMA7`4~W;*t*}*h#a;a2oApB57e;A8Oy&CGUDTsyu~HBSRAZ}u z4y z5cMWonqhGY>qE=r-yShFot~~K$D8M$`85mF)X>U%s$=>}4@}1Q_C(U~(#;-Q|LOl9 zj8o1pkh?awfGX{}q57`0y%F9FDfQ~sbtP>>sNXk1Bn+F2X`@`%e=E+!;I^sZOXjs+7c<%^!IBPrajMI9oavAII@ue z9W_085xq}+DY%mWJzWJntG*%kO@?Rph~t_Ya{tkL!Dw(+J`|kEYRSFDh2|Daq_-{v1qGc05aL6=frxI87ZR41@p`%MOQUvcj+L;>IfeRD!u69*!M2 zTmNuo^R78mmMBx-K!Q#{H;Uf!4|_xUMdEfX#Hu(^g^0WUYoqA}#%E1gCopyScQPvS zgmX$cx*rUFgWxCf&9xh^IHOZ^Durk95WMRvpE{`&-1ydRKsWD!mIXUjm+y`5pMo#; z?dkj)r|0)JK}WrUei!2utt=CkpMn-sI;dEVnOCVc2$^MF=g@%d|Eav0muDuz>s_V> zv+N=~Y9{(m(W%s|bc{Q&)Kze=)zNY%OXjW={N|7hFXB%q_!tx*EO-I_)PRhM#`bLNpt~qPW zAeFJTCZtu-h``po>y8-TM%H=7-it7=H=^JbYo2g7GZ|;M>Ron(my?y1A75A#jZQ4xn*M|GlDW)RnvyLtYFfVLNW$qexnMkwm>SqT#k zLuL^ylaaVv$S2M-eN1Y-e>#h$~rI1p*lp?-())3V@hI*POxX^ z^iLs*kONIZ?DJXD)7G-E7DFSDwf(Ee)t*)x-j-yIw#?JgRB^y7>^Z#i#p&;Zt=Px- zC(4o(a~9e@iea1F+k{rz@!_@&kg;8@vIo`tMVPwp#!057Zr~Rkbel?SSVN^=cRD35 ziMuEuh2^fpmHuOL8)8!0Cu-2Y&uZfm@3VMP@->bfo-p97KWY|N#8iJ+N}e-S`_G=~ zmisR>4j7ppD#{2gNC?kGSA!A3?jww{Vd8 zvb&yv0X>aA-BZJBM+Gvc8TohS7(Yl3ZB0tdygRYXGNG zE?e|y$|?7a!P_V zUX0v=!D*6tYS)7+=6c-z zWOQo9j%TS3&@H!A@=L^a;tCEIn5yHZK72d2k?BScYIA$Q1l-Ro#8 zp78j!&MLa4u~l@~0I#1+51G{&o0WA+gu{mkq*kIm(w(VQ%PZn4#JuMAMgg=lB`zZb zjtt~RrH?VfkdgNHOq4NdL}W1E3qiIL(ixyG?UI8*!|BuX6!~qAr%_zM&WkzK{mvs0L{!~^Cqqt9=<}e^IIF4{T?1w9dlSD3jEDO zWgBVhSPDQzy21G<`8hUSQ^m>@PJl<^3o1oITpxHL`~@Kg7R+0+N{-|p7xoPLw<{O=X;G1}v zOsHtZ&f+upxMt)t(0CCc&RKJNXlk9pC7Xp)dmOWrJKrs&tRYKp(uxR~C?dyBVkZ&58yG_L?o zg|S01ZEXd-;zH}#4744|9lP}?%O&xWtG`f=kUda^(2>0sOitd;F3AN22+&#}xW|Wk z93fH!jiUKwwK`P-ym7uy&QR2f>dbquIt^h|q9CGO&bi$Tf3-_)C_$KMfl-z6c|DrKz+oxL6aS7Z35 zB*iOzdGA6m&W&u(!r42W^Lu}hHFz>ITv4yReNs!;(PH~; z3l-e$FVI#EQSI z(N@hNMC>Et-gf#_sk;9_nF8{reg5a=vKSi>^PurUciNL~_mxtjakx)?T#belIZyM^$Lo`++7WN}$3D^?WDl5>!c1+5l&`F0idN{&sU zJZ{`T0yl)cCkp@cE;({=*sZueg1YpE5dk)qN4ibt&jtrK zmAN?h%WM^=YSF5$+YJ0`1O**tCAd?Xsic|&C<%l+F(gAb?5qrhX}k7uLBU4r)2a?v zq>Txt|4dq5PVJi`MtSGbKZ{P}Xpg%&3D~v#%5MC$ey*ss9(=q>9@7;*9w^9kYY|-f zIMulTo2_4`Isok_0ko-q-FKsZGe>BpYGJU0X{@l3l=t|zqkrdruX_C1(lR&ew(llW zMp-u8Wbb~?9%5Nvq3R@;w`>hQv7!w;X*1gPSb~Av{8T4EZX#TAvpmIJcZ=@0PxI!* zTacTO5F29T6>IL&Zt0497na|*YGVB-WT~Q2Q+7a57gb&9=zZ}PlaQ8QDaBMM#gV=( zoWqA<-!`{7eVd*+QYq&CFtxGLCwT&J{%~|I*wl#o8?;d`%szg3#gtFE3a)BXAc_!XiJ_lPN;itqAg}dYV*S_Pj|!Sr0))E&LF_u}G-{_e%MvHa4O$7LEsIfa`=+TvbV= zycE{jD%-3vl(>yJs(9XYy5-?}`7KW>fj3+hC%;?R(r*$ZSBS;6fPYT~ZV(wkaD87sfJi;^yw!Td7;~t?s{&r7a2qS{AgjG77WX;DCH0=BCWUFE zkNDVnK}%NFhi$~{NocvJ(nCN4gO;)Nr%rHr+~M=*bXh$4!tO@*D}ptdM%Lx&p`?sZ zAR#hD7!2@^vT7ckM189xjYaU&@Iren{lX9YIs(s@Q(tKPJzIzuEyF_cv(Il zD4k^FXOoMGu`DUHCjl;!_^)So>2ztfquY6XLLc-oI01LdI<%m>3l($dUr;`~I#dTA zlTPhJt@ihf-V!pZ)(wk1VFdb&PKXTLlALuxdic8OM%JQ(VQM~Jupl{9yLLuoSB^p5 z&da~KD*Wlxi%=B@r^{VvM^}#K;}b+Xy6l;O$XFE0U6>O}L+|mqq8r@=7@VY|vBJLn zTR)Ngtc&(vU3B1Lw%i}srTZWz$=<)ZV#4j8|4#u`dzEa3AsPK2&FC^j^Ma-5WAJ7Y z6F+yY*s=N?03Elyj@OHB$GQnVcy3BHdgBo`aYHb1YU7FH)C|DwNrG86aDad-ix)@< zC~b(kTSeZxy^$oKuKGH5tmO1Q7GAD`5gzoUOp&=Y|^QP!! zX9+ErpMsLoQ&f(fYQ_>hqyj2sqv|>i*OK!6e-i=JcoQjfQ>K;Q$Ej&TL(;jdKkrxj zOzC9SARKcb&=@4X&egqOYX04!g-ao;oEBQRasMUkky00Z(+otkTpVxO5+^9tq9Z5+ zgTo<*M^zRj+O5>JS9Y#D@7bDo+5#?S#4Tl2cG8b8Lg^dt*t{RRLM!n{5)(%uyv_02 zie9DC1gLO=$>*)mTmIc)2GnX#!`1kM`n}B(Di?eY-P1 zOo1?UC18D+Cx?mQ7;@_P4pGLLgUH)lN}RD45MP=Po4m#>&hnbWrkoSwu7=)v8Ukiu z>S1z_c5H&O z*r69sD!j^7nA~AeOu_Rh?2x31)}}gm`GDsD)ni-WRB@BmDR~pBHx_+2>_n1)9YDuc zeW(T-<5Wbf9h$vUU5_=Y%3O@Bj~CyFkN<0HvW8a*m6ebybB&oDiv@w|7@-F?bJNX0 zF8Fecd4Sp0WKX$q!N#Ja=MT4Kok!eO^OfMXo5IcK)z;*pwCsByj?@hlw*GyA*Is74 z*063_V)g^9DtvNH#)+ew!^;^Zcye(k?{99?u4i0R%9BfW_+htm;o@qk^|=<72`~TM zd)o@4o?x4|jIJ*Cf-uuUB!LZm)=&YC!=x)pGQun86OCw+C4<-mKC-_paXrsi=l zG?LQMax|V}Z_r@?kS+p@!@IsOdeA{s0wvFoVgczV8f z@!LM%;cUxcY|!`}%Yuu%*cD2}x^vG3l|5i_Qqi zSS)4$j2I6^mr^;X?aE61;#BUN=VIG+>V(h!c872)Qvp(JP>MYJ^Tz?7IcU=Sqb$^@ z`-{IZn*6AkppB5Y{PalPJNTnmG^f!~(jTH~;P90$h!cN__}2$p;8jqy99uwPnMn*A zP#`Bwk>8Egwy0y=MO@(yQUt6S{8L&bjv0&m;Tw}uW)xpd{EkI!c>-xbJlc5U z)e#{LOmUT5C7eh`pjrG;s`}DCi&;M=pz0Sg{{a( zzT?Ib0v*Sj!9Zn&9F-;r>6k^1DfmidYrd_+J@B<;pmDEzpp~NHDjRufh2?^;a1>W5 zz~BHH^B&0lzi~DpHMiy?6b?Z|f5g2$NH;-VZJ3H`-76}8Glgt$Vy3bFr&8_Yn* zJc4f|P;lqf)Ln5GafQdqz$!}9h0saewqpEDV?ioPpRt7&)2ye|t_pympxs3Ml{HoE z#aHGBX;HPgK=Z(zG;6Z$tAh1*Q^K3p(T$5eQ(F8g#%1S;BeY5|=U9DJ9iQVPF#%zQ zLRH1xT7S1XH7!oMY@_IXiFS0zVyWz+esGbUdq?#C}Yhr1__Lp(<7axCsMm+D!? zXeUs`?F4o$O9*x!8LSy$WQ|A9wYzu9OI@j#&yY|rAdjUKS<2{LC(K0wv*b&1aZv!SgSdwpCWI5H>|T@ z9+c@cE+XT}x5BSpzMMp$(g9_7<)czy?IBsDbr+X>QpZ?||6t)uz{ec0;*YAIM!Zr% zQ8Lk+4Hy2dYX?u>%~~RGiH+W4Z6qHT!$6GNwhIQzt+V8p`$#n1SQHPtkqy~KBH`k> zmmF!z>}>d(mn>*kmBd4qtB4J8Uz8m7bqNNp&Kzt_F#v zS_O>3SwvsxjdiIAK#TVyll=oWCCTb*<_(uAX6@@Z1p}0VdAH;&e2~bvOY(0Ow-~!{ zk0R%$kSg(HtcY^X{8-ivG?iD&@TQf2{np-xk8XRs-?k_IxAL*GdCNQP>KNQ}|KmMo ze}CbRCx7GV>!RFK+|psVxE?hsB4D+QB4RVZ*wUq(&KCJPI1qAF6qa);R+BeT$1*N( z);Y<&fnE-}4%Mz`ZIm)V5HWiE`nX2YuyKyV86|SY7c52-v2s*y*YMk3 ztqWv3@pbs|{SAvLL@KcL8~DFfpVVgWvl>fUJ6aL;_1f|^uljm=)o_rTo@cu6E*?dzy9K{47X}sG@f1$)@;nHGir(pWb7N z4y`~#G?Ry_-XuW%xn}Yo@(Jv)7!b1pA4|8|Nlr=JMY&;GQ{M3wj1kB)9*~9u5y2-< z%)*8oO6{v*B5=xN@vSkiUU9*qVV#u@4ECXO#DMs&g5PwU<#kx5H8z@3Hai}y=58N? zjLALns@17O_ZQKo2&nC&0d(9%?<0CH4xz%qNUQu*G#!|)Ezu@z-bU<2G;e=C%V*x= z^_|f4Nj#Y)`r~Dy;^NAld;OP-c9*xrwE!dy>HL~o@KTzuskug?-?Si?3Fgw5x)cjf zFbxGDj}`I{|(_R*<}rO5Qx|lS7Y}& zH@{F%a7nKY>~^@8n6)!bSZ%%OLQkonnghMb^aO;PepEhaFkHfVDrY`TVLXytQX0~k zYm=>f_|eOvPH7UQNfp>y1UQmsc4%i`A~r78<|B*nP5e`t^mgk48;0JwN1EpTxV*Vz z=6yYDCihMahdYIL1R~D^+w}@8?`0eU*4xqvy~lb+Uk@2SX>U}am-vB{+&7Kr?vQcW zLq^y3Mn{~gDmK=jo%o>i&=m>cx1pi@MsU(MjI06ap^Gv?JywMqPW>gE6zY@_YM&5Z z6%0obbyFqLD1zHi1 z*0T9$kY_?`xwObI26PQH|M;?^Q&c?HJZD2u+H(P;;h7#r^RrI`Iwqt<^Wa-~ld!O) zuHp|0v1Pf5`!e#5Y!3gBk(EbR@n>Ot!oL13D4(OgY0WQPv#{`f=8Bs4<42RT1M&l` zV+=Q10J^*TmEO(AGnWMC(y0|Yil>ksFiunP^?C0x=`_1v=Dqy8rcMfAdE`n7#XE`i82eRksNCN80?~d_!Wz+^^rJgOCVk!u_f~C*~=5rD*VzM z#1K0z!^#4y5yAhF^9fXZ`rhGNGF!?n5uxc^?1E?Wl0-F@S06dEx+@Os z(nKa`$0?MZP3T3CJ(s^`)9Viq~_&;iRFbeT}hZyIGTNG#DF zwO3arLtd)`P+F}va9wAP4_S(;#UVkNdRUwW763A7p?BN{E#xFugR7WVOTel)L|$^} zbp$sns#JAp7A(EMuKHw{S|EKgYBQ$(bT0K1)OH!S9)+JJlG%SxGu5n4d$59`OqW+d zGG#@%xt@e4j+BEL2h7bi`|nwcfV2%Y*WKYHndN2NfnCHu^HY*;pz?**>{gt7*ullL zt9CY@@`(H$e6>nVSgR>cm5O~&gIyFCin-tGN1UtlxGKE+A9jjZEuoIT4%F3&7{w`1 zA`Yb_d?tEJVOx*Utz)gXnuyliR8RI9XuZYaZvQ4`mpZyEPU}wh%c{=E*IB*1;&tYy zhDcSs#j%eHST)agQ4m+C`&ybJ{^;SAXg$!_LSoB3=Z~-JKKa6Jetg}%;hO;I1_iqh zGRiX2L;tuqbRZbc2nK5g7|?lzUOH*y{?&*Upzv&&(Y39yvQVtZwPyWvGMu#lg^4l*@H^Dlj-HCEC5Gk%w}* zM~%Sbs}mw$+mc*RlVU`wrWp+%Jepk4J;ey$)hRMzTIWgGPbL`Ar9;{`QR{&cv>wrbm=N~6sF}8M}gj2yduq}@eN}LXIH;y=crg{CiUhycqKhOLeeIGTL zOr6^J@$t=~t2YB15w3wbiGI*3;lHsjgR7@h1q`)ka^JtmUK3C zLMyFn_zN{%gde|4KTf;zUHG%{@_;Wcvtv#`2LK20CKd@^C!BnuvaTH5F^DQ>??&Jh zLi=`-Yh3!Uk&-PXmzML1)d)Nbjle1MH3*=*)xuVanH$2#6Xh|*(D7Tz0l{UjP5o7* zy_vRJ+es9=6lE#^hy5u(ej9MyrDIi_xLv^SKqTI$`yV3L4^3IyojxLR>z1w?Vtgy> zu+N?LEtQjveY~LsbtI;O0aHIKy8gLa*4>Tpi;k`XyY&W_7U1P`%0Q zTZQL)^0 zBgo?W50NkAe3G_Q#h_Zg3;h+rc2-(*0LC3{TRD7S&sX3T~- za2N3;W8)AtiJ8_nM@1j$0?5!!;C4UGIr*%3Zn>>0%$$-jWSh7|^O~rMUI_}087b?& z>TVUsky!(!`a?JRJ0E}^0%B|XYhP;P_ zj4-_WQU4(LRV5~iovmti6`T4NS}V*Vt=MpvN~^sIHiF)C8_s#@=s-ETcpV`!9ayYQ z#vr4D%mZTod0OOUCE9!vXEP3;JQjYHQ*<7a(i~u{;TUp{ix_(86K5S*oy|Kuq!;uc z;{!O;#W!ZcV(y9QXUaou(6}fX_72MW^Ol@JhplasW)~#nimf)Rx<;$g_i}=p?!fOR{Ie3^U&3v1}tN-vhOZ zgSnf+*$wqV&~nS=(?TH2)v*ws&~Iq-fL}OY_PuQ)4d&2s6Er5PK2ME~`FeQw>kvww zGj;DZH>IK9z?}U1a*TwrMq_fQ4lb1UCT8Ai48s>c}nj!h4bP6*ZvGqQ%Hhi=XY z-Ix&W1s&$l;H05OR%UwW%DKVZN+Y^GWbAHdk&bqQ40lk1R8mv^7qoq~t!Al8esT zzX%z_+Zq#a{MRgF(g33@jgVUCrC|67b8V9bw7)}^%E`uY<#Vb-V-pe^7W6U-4s8ew zMlcJ{^~k-7XQBF(tZjNb7h0bM%IG&@4RkXg4quo#O~|MNbSIm_;PdlshhI=qo%< z($D~WBL5iv-%+1WZkGH~<`0=KyJ<=AcEOk3(=T%xPDS3^cZS521A4NsaKpOrpP27^ zRUDVD1B3ec67EPadvwu#8H5nmjObD`f<>E)`50NV>(eCOl(P%p$%)T)Y$6&F+ci6%XE+{X6)UUV9Z&yj5;E|Xqr z@+aXJmyhMC$cbsRON(dJ)cSe>p|zL|`!EAP9(@}SRL5jiNxq|ES95|l%d9dFjj`*wV5 z;ut$={Y9yk!bGRJta}9)&FC@h=Jfp0rc!d6)L*6-o4p5~$Bw(;sh5Q=WbFm!kY8g$ zzCIV?6klXOK$mie4j+!(t^T>2`5Oz0z-ZNuI6x{J2aX%Gx!n!X%$|uS@V}wIOB`PL z+llojhAe2N_Mf~*{?mK{9R{WlqM0LoE7 z?v`I#ESxZ#*X|B{(X3*)z$*=zrlN=t26jAsf}Hjvk;w$ZhDcc?D#r>7_t@CuW3%cd zc6T+8nS`Sqyx{1Vg(c7P_bYf0b)(xU^Br1PyIuTUqA1&EcsIs}aTr|?9U=qj0jI-X zUTP9uww5>XP$}Vd>OLUbfuF#SHqR4k>Z=7`)&l|f$}vm`bkWM4e35l~^|1Ilw!*%02wp4|I$2YS6{04HoxknZY;2}asf+aTiOaulN;5gi z;@@?^Khlf#dF!at|Ic2)CF?nwuY600T|Lw9put-D7;oWDUNI@&+T}D%udpx;5^kn` zc!74*5VEA(hgXLFl0tQNkgYQI0GU$7seCC!XpLoqALEy7reXG8wsXp{&`scobF#b@ z6-$orScc?Gm7v!HlURyiuy_k~3Ra**T~n#Jmln@{lLUIT?T)r$G$qfaC=fNq);zvP zs72!tp5i5EQcH`bFfegIGylaz>>&!5DGlA*@Z5-2_$Gw4C=2ae%+;|LG|La5SBIPH zfyC0t+^ctrWEoNemDm);fo27er=C0q;t0Bx*d}tqP}I6DZo}Q;Wwuy~vE?H@XX0q` zyL%Xsc_qpa*9qh;mJc}~iVu}A3~@_0K7NCpWV+y9paDBG~l_#px=6? zhsn9})bRHCSHMtmYap@}CYE;qx;-9xlx zxqaNW;PXb8u;kBMQQ3F%`%_nb^nNdHM;u=BIjF*^vl zpk>#4Eo3>I{3$_b@*wNDGSFw_lfsD0s&{ju!D#&us{F9v-AX{XhK!!AoBubNtLwgn zAAdzZR#IF}5K+3CD9EUe$-Mn4h25%KS<{ zW3g7p6fCPC%1ZQ%TxpppsU*dSFC#bDpOKs5V&tCFFmj*yGIDaUj}tyf<;mw?LU{tG z{B!U)cpxUY5MdoZT#iA>LC_s9Id(W)ve{30A_zCRS((txxtWE*+tp|P!^_tXTysU< zJO4hg;P(VhC*FHWgLJT5iXWeNtm`0bveDZGPe-lzCfaBl-cpugNrcEE|Ml zU^1onlksZoRpOG^v-aa5NGvs1ncjmI<`6^A$LG>63Yo6P!jHHQjv5Cm$QA4~5lepk zxJV)@r6*>IVQj=a?}1vHEzU=?aj~W5Wam#|And@o7KgiziMv|LoVXFKI_0n!O#r2( znSac3c>*&=dnE7F{;mTU;|iv8V5lf7i zbSrBr^=Y(pBBnW6_@q|?OhFD(LrOCzEegRBN0c1D^A?=QA$LOxC1fUpTDkv+&Pdks zi-m*PPLDb)VatqlS**I&?>SZx{KfjETU~{vTR>;~;v^RtLNOpa+2BNq7t}1YmQtJo zaHE@OC++w0XVN=RGxCL}KQ2*&q)0Y3Xe4K^fqpB|b1~&-_~CZgj__yMKd#kI0#Sm? zi|qdMrbbqzIdcK_KX9XLhv~N6QH!JfCgy7#3ykSc@=on?c3*y7=d$1!-@20ZjZN9% zzt4!A{8z|GUSWKOk~j2^dp9Nb9V2=$WNdiIc;vPHp`~EzW*YhT)j_nS`PUk4&n;f~Th@amf6dpEba!>{qc2%p9S z^9+6Cf&4Ourg8u(=ui?@4;|&a$CAssr-xSp$h{39Hz+66ZNHJ9`cxnfTFTzHB}@j0 zyJDzu=<1o_A%S4oqU7-C>5xozf@HGFh`t5+WM-;Ca?O&;OEL>4wSv4d@I|rP>DfOs znY7#bmEOkm%?pczyRIQXka=gnhPNKs@oS>Eh?5&f{~rwV%2jRI@_r?L+{8cTzSMic z;1d5va$KnuuaR8s>#GS>c*8an!bz(Yt=JJOVD*Ea63g}fk_8GHTPmzKt9XcXHp2Ij zF+(izMwjdyx3OX&CQ|ER8)v?s}9Ae7!1ry36CMa4e;m5BGb3{LbOD5z=uE3p_vE&s~Z zZv`v)G6sC!V#DFE|^YACHa8#g3IBalxeEY~L>uSxWO|#gvzn01|yBXEDs(cDNSXJ?Pl_*UEz8o{$3YKh>AYTP{S(^5puI~cgt{tO&#$L`R={A_06D8SY* zaKuyE7`Qxh>vxa^?tNW9Za}RfB`#CRrhH6oe=SBUVu@;qYfX#wSpqG@7!w6iGjb|?6LUZ&@kS#(gN5zVL=eHVVkd+#q zIi^&+QI@m9vmz1L4Gf2!MMnF{7%*qnZfIT-e&oYR!6>CjS~VW91>ha^yraU+PjsLG zQ=RStrddR1j?l~`>R&4Y1QdQKigA)CPaVT)Nia}WMU29Be61{=)aX<#mQ%#)Hb85= z(XtsANk&ZZmHItrnJm7mfZ4Ickve`~r%2mUN)uIxTbXjh8?OF3`PZty2!A9fw;ew7 z7MTlmlLWneC{b=&5_+r(cqP&0iR{d`df**}JfkdxZ-23|*-ELH8iv^Mqy5;N)s6*M zrhFH3bZydP8Cpjh5dq6?(nzNxys*#|Y@Fsu9|?e+k(l-5Y79E>~# z?-8n;6i-R>i0Bk}thZuTah-?BX0e{zzl*#f-JLfs`6#t10kO5wJs@a>up2M7W{fcJ zlyXpwmvi!BlNCbWSV*mo)fQ@Bo)Ux$5?%CP4-;+1%hZ0l(Vu!B@OUUW;#ot$9wt+m zbenbFNVjVvXUE=pMK@#2a$|TqV?vvh+=`Ukun}DwGLE%1HiK&`OUZrVo%Z?vlVj|= z#@Lk70AZ+3i%Cq0V!Vc_y+F!67Kls{?c!SCT$%ms(8$dwe_IFoZQ6|RD$sBJ zuS@8Yh_5>8`%|Xi~eEZwEaNS-@sOoQSq&WE?Ey<1|AiA z?))0kjY)Nht`Ybu=q&%NbYmVMgS^8YJTu~QY>M%E>=agwIu*wn4Uqs@u+-;_lqy(f z;!efZeIXHDb&8GHG&?HqqAaIWFwBvpR1pKoDFun4$wH=V<>aDZt2jV)R|tqiBP>GV z7i={=^qAzl+!P$y(<=&2Y~v!h@WIMz>zBK{+W77FsNTttIQE&5p7Kj-O;}HvcBpqK z(_c^N6R-)!Nm%crEC8lDnfeS3n3 ziB>C!W~656$LtJ(gb_FejD_5aCP|}Ck$ke-eaOWYpCWFVdTn9jkiDMLEHCE z5uEZ*e3EAvaIQ4d-bOuHIINV2``O3b6mrvLT&$_`2b#nyy_e?8s$O5sqAPcDl1sR2 zUTJcocp_%i)hr6)3bmM9--kXLs4BB-LZN_&nOVp8i4{#R!9zNbIkEvOVuX?gj-$QO{z6s@s zjatKlT-6$@GcVAO3kqpP;>tESCbN>Y7yc@8C(9KkcQuFTKC>q7P`7PJpJ!#}R6I64 zHR}qZF)N;#K5!iB;AjV^G?FH)zgB8Fdz!!ftohr0gb-z~kcWWb5L`wq6i{(5t>Z93 zoaD9ta2_lqTr)Jm15QzEYLR7A$upVHRL8i98ERW`hqxObv(03!lH*AohchlJ#!63H zfVMT!zz;>YtaFr3Vk_mbbdn$@3sim|pznymO)tZJ^8Sp9s(1LCX8$;&V(`0L=8y(d zv2cO_)+_SF>wNO~j*d<$Cw2rJG?Rw`!nSiN{}v&fCQRfg?!SH}g!W=bU%nXs^)t~# zVR1^c&byz2u-$-*SwTCb<@Q9ytN0##?Ew+Jw3rM?K>ms%b2rPvj9_f zjogC=$U9B*%PL!h?2sU&JTBt$H+^|ltDS(ltnDFLqvsMVuo~c-1+P`Y&3Z;S1y9Pe z&FGTza?zVFKi+Tq6N^GAceV7I`NrBUH~(e+8-M%v!&UtbeAIY3A)>G)sq8KDa_XKW zy*9rXJvGOJ6i+~0A%51tR-BUeKo(*H?}HkMz-FG_@>PtmY>MZm_bohQzDb&Cz>gW? zNES^?`>Qv?#;eWqo7V?$GGv>O;%HZna_lo48x z5T2V5oHQcXeS}dqEIl+MBXsxNp^aAXg9ZZ zR3LMjk$-27@q^^hR`i;?JF)bKM*gH6<6C2lAsdRSHw6=nj4?*9Fxb8@8DC_ix-wWclx(YCxJe)Q5#1UT-oY`%SRUtWX;~BAzjuCfyQsn>7z|_hprDX3s`+a?wd>u z-Xo4{Zpi&d?**eF)ciyExpm=f!V;xdn-4|5xt%8V*f2r1$`*SJ*P$^gz^&=j_}T+h zT}R1A7YY=O@^-)9wox|i)|#WB+Aq9+>t}kQrt-nW3Pj`L{1#iH{Gr^7jvEPNOWmoL z!V=;VQL5%8u&gb?2bqoaCQgT)Hzb^^(O_VpkX+W|$7|?EI6qR`J4&trYRa^)^B7i@ z!Wl)uJ_`xb=L-$W&y9h_X>khUAoXhUsf?3uUh%vtrQ?Zw`@JgT_QE;L$E?!y0Pm$* z#$^!lBCXVEvqAXFWIz_bHqN}w3*3pWY=F2&M#T~Hwq=9aN__2iRyl1>mc$gtO~d%~ zUh9YN04;p*olxK0w#_SpECK`Cyxms!a<@9Cyt`kD4Zv7s%ppb7NXuB8>g*?^HhbB} zw$h$r*4zT8sNKVnxEd7 zSZHfC09MnF+CY4u9+YN?XH5NB*{C`G#@BP}4lKZx=KN>iuCRNCP$;noTLZ|(w-vJ% z@x;mFS9S1(&t!2RqC5{F6-2C}KehE?8gG-Upd2-GLj)DAoC(L>!z*9i)CkK^$xWy~ z#u2SS5_d$X))t9^6gNB|M|}B(weJBxUsc#OYnU~O<$Ky7f!e=^=j&7fFAz;CQKuoyTi zuLJ&MdR^TbSOo+Xb8U~k<#Hs+4J>0?r{M8zPn7Q?hcx^*4_ocoj2D~8RLe|*xrob3 zgxV+4I@bvyhql%W3w8vXZT+(e%|gv0on`FhS?U89JiM5F!#szX?sTAEGbArgJ&65p z1p`;R`lANwDLI4nqPEvoX{C{3E8m=VQna|EmcaC6tS2E}4sM2db*9!#N&N;5LH}bI zCz(Umt`k6Es)kTaPAdn2Qy<1DM9iVRn&F;>$JItSRw;pTDwo^$aJHFW4E0$nl$-(| zG{yZM*swGQYuLQ$6QnDF3kd_u={Yv_AjB2o9#&zqgApaorCtP}mKDrYT03=5kQZ)f zF6-VaU?4Z?y!DN%4j;L>DB#mJZlYh?xLzJ@<32jd;>%{)Hg51cLnHg4f85dDC_E$# zEUEp#7p#@rj9R(3ty;Owm0ji(Qn6g58=*U>E~h|sxl!~VD1p)D`bUz>+N6g&qHo-& z>k`%`rbafRZFnjaphsVRyPoVJsE$qF8m0CuqS)8d6iC$ zJb8cf**p5&#bsh#jYr|_Da zESZL!n>Vqg;E%<^ACW0E&K7Vr9jQ^4Fz7qEv-#|YR~D`kzt4o2T8e@@CxUa5QV@g{kU{k%^HGpzo|{7P#(;#5&2fW#$R7PLly zm}-UtJtSF{vU0UFv>XG2aUF!YM9{xfOSb<6udJF$oD>%6^$# zaRE0$&02PF=Vz##>$v{ginm)%eeK$txAriv^k0xga_#b0QSxr`#?A#?f@g*tj(E)+ ztJjJiX6FyJOUA^zv5ryX1k>(iJVw+VSy<8*SN?zuOTEB{1rfOT zP=m`X6vkyvCc3+WhuRt*F=9Y>*v2(?u`r`~g>_%GuUhC1!M?%m_6py;X<_;C)`Xy% z&`7S?)*1XA5s6do#)-Eqh$8y%i4f0<5>&|7VbS}q6(JQJ7ePZm3`YP~cRDh}B1qu~ zge znn2*|xCRH%1PYJ!oEmpHB_IDyKkrY?Z zBybnUVTHgB%{C&$U)j{b8rg)nLfzL|T?Dg1eMOWTqMa3$2{Y0KV*ba}yv2?!?gHt6 z%xlD^%|xC$0?YWS8^4ryFcxpTFVeA6^VcGU< zcaH~C{Vi5Vy!3uPUTDuw=3`z67>kGfCSDC{Nkr;~&m5HT&9TnnFY-&K6|VTWu7n!N z9eybgXxBUNVJ~BCuhj7NPT_5V$WtKRdIbhNwm#4iMksF?(N{ypEe{!E+8cRj{QX8s z?tUX04jG@l+B!5UA^ZWFd7}~d!1U1N8KFxP!n@E0?uy`=E6^_B!tQ>iC&PS9Po59x zO;5JTMsl^_;+_E)Cp{}`w+14z19H0lx6i*UC$#!?BR}P-qO{>bqv6vYM!|e^`%Vr*X_?q?;L*fm9X~LPrhlP7!7Ox? zn~7cpTSN!&XSx~%Z@v+l2%pMtw>7e5`P^@;Msnc5;xv*g_~#oFPm<$)0s4P4HRnmm z0e$d|JQ3XI_rp>@rzpPb2^FK|zI{V3;xb^%W9^eZ6O7$qJ_aJF#uegcjzo2ooG55x zP$RMg=0#gd%}xX#`FA3qwt4gp*sz@a80#2rViL;IwDQ`ig?v3@!J3iNq>$Rlk#U@5 zp28)yE$Md)2!X#?>@+k8>Qs3)y>HRB!sBG~>rj`0KAVCz>RYEXh^$@zaHQJ|{1`#* zxxc3qCHr~R;S%olVDm&#NRRO}znA{DCp-MG8^8YvhnN1=pp7Z}!$d5ww8l0}{dx{B zsqngeHe{;~uTY~&4GD@O2gEU|8#;DWK9xT)kzy5{%87;~>Vk7@Vjj6y;=k`kmy3Z! z2hHmwrAGAB9>Q^+wR2y z6(k_VVA>!GOMTrAgt%8$vb&S>_4>O)g+f_HWcU&Z1J{Mibc6m~mU zS5nUDV=J~2Bet!%;DHvO__wll^T#bJe;Et2`MCM|qIc`WnwH_arPkFZ`#$sHeLUm`p(@N8`&Z3rVkUFya z489U}dyC7lCYUNRw2eaS5U4?=cikkbtuRah2|5M%n&ym{&Ehb18G`e{Oo*tEE+%O) zoJzAW@AddgB!CokCi~ZGA}>dQS8TW}U3yQln93Eb zP$YvoMFgDwMOW|stycK1%C_vlh0+K3AmSU`%ZZA{&$Yhm^_`y(nAR~0BD=Lx5oMC@2dNejYE38bxOykBpcIvICQ7XeQ+!tiu@36TOeENmU6W60aorKJ zpoyJ)__BJE+FI?+D>M(7FRB^YGHUzE?7(H?6W%^=4Z%B)%s!7mZlrXtJl{aBlFH{A zUP7Djnq(vL);&hU3|O5cr$(j?G@AbOL|}6;CHiwRa+?oIn{i#4JDa{P{39lT zKJ412T{k}U<8@&6=3SZ%IpYmP#}R+k@XKY1+4r;eegPT}Ot_PJeFx6Cox}h4s=TL$^#Z(otoF)tu$j~~* z@r5nvCN@&LYHRYwDPkr(E1~O3_+(!z1yjV$(w&$K*Ov1F6Qta-;tGK9RB2D^AB68+7Ap8rw}Hk`SNci zX=8%vO{JaF5#1eL`g_816oF)qOTs#h{{X3}Zf?PRzdqnQuI!EE53dk!ndqIJDP2Ub6m1hwReffs)_a3ubEhIyB3HTBNxTr#m=Z#$|h7O4HLDM?j*bX zXORa62!r9CACx0ioja()Db98?l=G9EkliLllJ5P_0p%!^)Id3k>Ke?x17g?qi66wi zu6BBBV-lzA_o&{f6^~GP;+NE&M7v$dk-RaLn7sJ%O4^p=X{fOKD|n(}0ta$0f`mg(+|i%m`ekc=W8x4KNlX;??_$V3of1+$J|St?3x zHyPjNMEM7fSOXKduw~K}tgO86`Q{& zBg-w-G!%H&Tn@1ou|f|(O$bagJjrB8t!0cE>}6GnS(6#_yU4R-Orc~^GG^|0PRya^ zGNN}w|9-0-4_Oa84;(vReUfU#OOvxdgJ26{i}>0)Hu)(puD*cxFC(9F4#9tHl&TW`K*g$G7q3 z(ec6+jDN^%R;hL_rvMNLx3=TyPjv5qHFxPwCm&{cmL?w}c02fwv=n^j`?$!Qtm-ee zCLdq)$Rn&U)mqLin7^H_XID^;O2zCX)nsi2Fl_z7QR-(_2InoL0JbK&msK*VOR zQZuo`i3g(me5m|WLt~oCP?rF}K9jlohi<*%i-g$nELF{Id-42L&Aopf#g<3E60zka z(I)CDH&}L+D@Rwk3ButrduAXq7P89=(Z6ns5uJ-}a;?+D6K?GpO{Zpf`?r3wNA^40 ze|6D;i@D%xV3+QL7#zHRb;X3+KmQ+e0S_%K*}A1PyC4E6hi>5ybFEx-G$fbDgY-L- zt{7Xp4;5LA!A1*Ac6YzhyZLzLlHgoAwPHtc9sxi2=o1&!K52|#-wyKelJ-fDyar>G zGoRoepKG7ATm0jp8**#;-@CGD2&V5aD#wgv&Uj;zWZJ3+KjoXaKpExR+6uiS)RioI zP+bHhRU)Z20!kTqADOn}PAZN86@qP*qD1XMU&kth$GH6uz!yuFALL#B%dLIQtnD8X=E@qGSTA4Hy1SVC+kKwk}vEPF;x~AEqD4>E9c#rj(mxlz$)vUfOEfTJ_$WJ;e_=0yw4i`Jt4Jm*LRIY zX}6FI3Siy^fO!Pb4ln()Kzgtk-<(v8P5w$hePy{nzT4V2@e)m&@Nt8s2?|7y6a4j+ zr~Ns)(_tgY0q#m+ZCSBvdTQB_oQmq{@UsNk+~?bwK*m0Cwmgml%-+5#9c*p;_VC&f z5Gyt2Iko%sQ=vS~F}fz2P^YLSjv{(DB7^j`>!EU1p-d#lz<(g?Kp}ufK!-;J)m7Z#(4Nxzscihp63MLc==q6ih6N2I`yckiag zZ@P<*1bpPMqv@I|ytWIo-~R{i)baI|L{Nnh74d+($DB-RWKvu^&eW}oTsSGA?KZMe zAu;ro=hT`@Bc-x+-(zhUC)qeM)=pwGEAEw^AS^2Iq6$YOZ*9fbtR6EnK#W`z$m0QK z1AR$oYbO{&VnvlD5XQEX4JwvGzcM&|N|mNUXcDoBtH?l~f8B8wx3i9wkkbw;SJ;3e zt-p|3@%yC_b znJ*1*+a&K3>IF4X9Q=Y(k9o5jLW1Inlc}T3InZ*PHH$lQMGHvHY?CRob*P;a`fd9p zi@jbn_=syJobSF%`a90WhR_^k&`ciB@r>vrzfdjLZPc?)y8&X_$tp_i497zuPR>84 z=H`=aXqs!_^g!9#+OYW0O>DfR`nL{TeaXHls_UQimOCB*m_4DLRKIM0-zol*zA1Qp z>~ByJy~=VKC#JsEmy`Pf>9oF!g+$fTFNmyLxRM)hXclV~WdKK?DD3hA^$7O33}J2d z*!so^pxM<1Famu>CqxErNzS?;J$&7CBWuyYaK~N&qv`W@M){T1 z;f@IbW72?N)A4b^xBg>!WqFxsGu?e)mq`N|vb}qC(P2h$YX)_JN^|qSQIplLVcvCZ z$u1)fDJLb1;a@SgH?BvUS)x?#ck9B%I2BqyxQ^~nI-P;cV@rbT3DNb-oXN+#mIU`5 zd#~5$L{Y_oZl{MW_^GHf^Csn}cOC640F};iH)O{MV2z(rgFlHrRZHCwo3~=evxAw9 zixp|AW9*(ZF3xWOy*Ga-K}*r+`xD7jDzT&uC}Hmgav?z`y`D^{@m6N%)S8;d;k}oN z((W)0;Luh+^Yq821ly}F?g4mt`iLw4w5gFzg_kf> zhE=vo$}Dq6z9`#)4>=C#+vft_zDJoU;>lgi9sWj~D!vilnPTENj+WiwV%Emr^W;sq zJ!K8UL*;RvX$q#YM)GrBMS4fhZJ$%$D38>b*d=A&+HJZa5grNxu{mkD5fzeP8jIy# zu_e}0&KC_QWQOdkijFk6Yc9OdD8$q6a4Zuk7KSGAp&K}C4h$Uoo{F58F66dybi3wP z9!e}cmP~S>hS30VXa@cW{cI!U&Ji11A+H(|d5Ux7lJtc6Dg~>K@DA8(4*qTyX2ssc zI+i(dT?oz>F)0U%%U~D(_^1s#w;6ysw|D{M2(_sPrFlx#J0HAjN%>x|San&Din{Ti z!h^*q_o&VO-K%IwBbF;>0~OR*vFN4U>)0E+{?%6mJvB$9uOxG17H45!M=0HVl=AO- z_(1-y1Y04Y!1MEjZ|-Qq47~*c#)DY}12rvkiUprX8s>|f*YXIjL_B)Wfv^iF4vw zf1UZ>T4&tr1Kxf4cw+wzizy*!viN5l&4Xm~Sh0>pDWw*`6qVvPYDS8N9BSfNGw!yL z*(OMZ@kv2JysQ}F3crq%SenT@Bt~tWa9AU$?C4w%EWS^q8yr@^Uf4LBZh~PVStwUq z$Q3|g0pJ6_rUd!tls6ltUAs9#oTMKn>U3U{zLBV;jN!c%>5e^E^%B=8YKC~WmzR~K*{LF#*ItS75vR4!eFI$tteZ) z5+)xE2EN8%V9htW)Clv+VWW#i(W$YMEBE-xDo-xCYCI7`aVq2{SMNc=9-$Js>^Z41 z#0q;~_xc~F{b`BZbyRZ_y`WEu`_`8J`Umdr{Rvov}zdG&{f9CB-Y<1k!`LVJ}sKquW zo8iN3GdBHdv<&hRj6wdWJ^sq!r$aCed*7L zK6vG&4ZGHgaYL*@44D^jULn|=ci0WZTSAoz%WtR`CvZsSof_7bYsfgI^)mG4Ag%|{ znlwhvXtbE2HP0-rv@`=JTR3asVzwoed5zfN6Tu4F4WeAraTDn%s=^g0Iw`Vsehs^v zcovyY!dW3+P6AXMxs8&fk4M-FQWcJ+@;K!w7&~<0sxV)o$!y4{gR6il3$w#7d0}O1 z_Fg-}eAg}hT}8V%g@CtqWg=b}wrvv^nPz!{=2F;}~C;0O3p94jlT%@HVry{oNy5VBDBt8QIK7j@hzWuYjV z)8lJ+sKlr5YeoVklKq2w#gSi*r!odBg;1b&MZ~BJo=W8%?9V~2#C(Of=pCCl;X5$T z*b!nEZ|m5+Li5O5?Y#fUcXN+q*@9E=`4kg2lf_L`BI$XGSfD z6esfIrLlo`$^mh$YO<$R%h+v7D$2xr{-w-MxI3Qq1tJV6*tt*phW|bzH~+pIBVnx3 zm>jB0PAt7QG4oy{e`bzRJ;vB~YGJdk$xMy6#>WN9oKApqw@a`NQzUyx(xqp2J zG?`JBmL7WP;?PUM@R0y|6yU%1?|en<+^MAx7)HT;FhqF{wR6wH9Oa*HgqFhbs9At_Yy%+ScSMxfx8H$trfz@000JcN3g}4uUm3+br+NywQ_<+^$y}xY*>q$L{>z z&XpTCzmF>fvBnD?P?4NJgs5<85Isi#Q_#|cP`!JRBl#X*}8&5@TDGEWsTSV4sO zL4`+P@yeL!x9C*xDnq;I(t9syxNd|10eSDred|l9*WC9Q-=?t&e}x;fkX*FFMeI4? z$-D<@3**9(=EuoLsiNfcEKZikBk^!Aa7Xr#YpmVVfHqZdU8szT1NbVlhgTw)h~NE> zFR3qN*{nU_k&jltgK`M_zwLV7Kov7DIY zR-CMAX1(HM3dcRb9`kyXJ2Mv<3sR^RhECzT;u79aolB9r3Tb*_JfWgvMw3kF@hZZ0 zlK6womsH2%lO>8zq5__j)=i|WEq43?-AH(CLmcucs$~S0D54z9T0GQxqDB^1NR`MsY1o()-T`FUh@|5^I+dwpSp3g zI&%iJ;41U$;8QVs&27h3z$HZ{9|~W8w>m&Q4nW1)xQp*BsV7x4NX{txB(s|A=}PE` z#A2Eie3!V3<`#4c`qv3Q>w-w4xfGaojAu`gX=V)bcNjO$p?i<4S{JNBXa^81!fQ`! zpnFMHiU~7GZF%@uG7Wku&~fgq{R>*UfxnBMKY$mgYwUaL6z@;1u>Q_2z#g26h1LO9bGx;!Ay_-M*#XC09 zJ?N!o_KhdGB;bUpKPfQtOqa|KF}5d?#_PRcCh=Tu9cKdliDt;aS8xKzJUMV2Q-Dkf z4zEO>SaOX{&W@2ApMofo^rDo4!7XG`(ax{20#kacEJRxZ2HIM4O9Ta^FU9`piSFp- zcd9d$V3UD1(fflt26JlSm(6`P6iaT0#FeTU!ByrGJRuPa2OvR9W@jU=wdJ0f0d&Ai z^Cj=neAJf!C`WW{35vU}$X0c8W!uvVedL$qc|wj8PxAl{R-TsI%qZ(b+vTJb>%0Gw z{0rJI?8^*Ye%GfK@9Lk>C2M(d#kcUkhc07-l>)z(b9Tv@W^d>CL0X-~ZO3C(cBxg> zifD0)o8-yY#7SrynA@=~!lU5o4iSXA zv!OEnpxQaPW4rqj2wVRTSd_{6b?NA>jh-;VW@fyVQ2uEi17Be(F!gmF=u4mvqOHdu ztjxU0B%n8zy08&=bSwzEO-#^L)PDM#NXPkXhe2_EB`7pcj4a{`_?aN}9S6ES$6Q^W zPY{6*5~YwDJTCY-N^hCkGrMrtR_W=LKer-c|JqN}!o@~~UVFId6h$SB9OjVflR9AX zDH+pY{ZU^@;tQh4N*25Us)>VtduV2bT996rx4w>z2jY98Ox|jB9L5zXo)R*QxB2QSG~ftvNjuCDxp-KP0TtWfc=Lbx5eUbn^TWGg znP?4){jHS6RUnt*F-dayw9`@MiW*ZsD=x9aX4w%4)iCUC8v{*u9_r?g$J4hyK1(tG)3%TTnnWF@Uki92T`<|%wU*`>^# zIKb3QFXrKZ29{c1iWl4(F7?9GDY_3(UhI6fACo(&khwS*LrGWDCd@-iu zrtGD{H!7hc@+(dWoosC#uY`{75U+&pSx{qKv=e*HH3mDu?V!uX| zO1yRY#OAB~8_YF-19~go_M|I{1-%CLW`wu5ZDajUll>aZ9r|Z^w*odNk41#)m}D)R zf5oOsPBd3+AxXj5UH8kTWrsz59lxM1&^8yl|NPf}D;<-xU`tSTFOGZ-G8!W2`VSr0 zj{fh#*|ki5sV?t(lyRo97xu$M8Cu7XL3VUvJ`N@L6w|C!Z2w<76t9*AU$q^INQ`L$ zI(v1rLTHZYqq|wK;W*354ygzfyKmebPo%;90G*cwd2!c|vpIUZ0}9fs_P^v~5J-~F zO?h#H;ld-AfeT-ncn1AvHNcJ{(?S#2A!)wgv0s}WX;#M29%o<+t$+R|)}yL4c`VFB zcv0K!$JA7l#;v%kzR?cs@m$BIl(cobL9i7MJZJ;t7JyODv(?U{+q^ zV9P=bJ6YQ12(_mf;CY5k7+3~Qp}Hr!6rNo6@EE>-~0X5N;<_*+*^te?lVU$HBL zm7(w>=+Fw2?OfcnzMb~~yz}Pg7&-vnr76tofOm7w{+0H7{D*p|!8~AUlEu!>|EvT| zcJHum27Fh`-wJDy;_Q&GR<*xhqy+3sb>{&X73f^edpT*1ZF1NeUo@F_HYK7>d}@fnZdxzk;&8wt~tx6z00pY4q7*)zhspu2oBDU|l?(8$fuUA`U+ zSHactl8}*o_l)q*4M;@C*6ucylU@aOkYKV=KKjp*DAAjqD==Z=51DnlX-V*QLB!qD zFLN4BMc&(YhVBr*YXOs0vqu-*$B=H#h%Pk(A492CR%YNTJaFn ztM2xhS*@lc@qpSw>+?RM763#qJHeqkrnSM%|4p6mtgSO?K#_rTVJIT-^sXknOtbou zUVK4(Hnq84AZ)FxC1u434hn^<@JMnTQz0M2YYXD>KFFw#o*6q+t!x#zt)?&zKZjSl z`BZ#@j_Ry2bOn=I|B7u=#)pL2%A&LlWORxD;6!*X;!A9z3-U+ei}9wpRn^&D#cM<> z>L-Ub0jQ|RsOY8I5M&^@MKGIi__V3Md>NXvD=*`icdmI^SwC@EM#XwMl>xh^mcPM- zbRr|))k(NDucLQ})H1+u5`9#~s_C7bt;3_St;5T=rl4U5ZmG7uFWR#2jdD!B$j^l0r(_TWKn-sF8EHXarGTftQNINsqv zy_NXvaFXkC0~2$HV|H!=ZWo;hN#&3cyiKQVe>gJuUD0Dp8CzO0wr&d&sDk!nFV6_^ zCSc^LD1vH}SWPCgmf`^ss{vIhkpMRONTO!7cj7^7f7PTy*d7J@I{JhV2#wn`ywD%D znU07$0(5-a+P_~^xXO}f^K)^+1`e0s!6hv>ub@}Rf`lrx;fLW}$>Fp{IkrK(xL&}@ zCjiE17Lg{*nO!%rw4$0oW0iQ)EHp_(+XC&}m4PRsEwm%xZn(KeJo;8+X?@flNbBP? z0HQQ2mbF0$>vh4JPcL=+0xvcB9I(kAnTmx$osp{ovOWXJk&xpw%uW%f#b$E^*fyKv z#TQYzvq`8NnyW}q?ouF~2;Fc?Ah0LBlx1rAkpNTOBxjNqAWUMG%A|p`L#uS&nuje$ z0&gT3!bRe}wI_Ad5CQf2XB@EJ(PF2SUvgJ$DFt<}Fe+mN1)FGVr&e5R5tbPTl+Q{J z7U#&0Z*%1AdL96j=if=)bcj;2Fjg#v`*8dALZv9yEYdiY`__T87dWS+%Nqr^Dfgtf z#c*tV`r{I}+|iNSkvEA4pKqF2Kf%oSbRa61_eAx=(<%CsWLV%_T`ahL{PwHG_b*cwfsGyN-#RilWU{h?~dU1TQ9jcRcjUVc(6(w^f^bEpR*?}E8&pT0$N zP=hI--d83(0o?qu!joCWg%`cMvL!gKIQfxotr9cmB$nnG`S<1+AB-`MB!{*pCuYu0 zgd{IHYiN4t&V=xggy5uM!S2J1vMbU<*JgySNeCyPvTjIU<5)XmGb-m^HKNajj19|; zN7@-3al9ubclXfJ7r&lqWZsonI@QSkUXIarjFFHWs!vSJoMPl(pJN=k#(4YakHR+v zgOdgYyALwTGSWl;xHxnm7|uZZ@c{dz8zh|mmUnQ z21D1beP}Cmmg7RkVCXDUZ$oqNL?ikEnuC9aM&a!e4SY;=(~zXzh5sQp_wU(F7p@t= zESOhsf|LJK(8Q&4TSePyuM)bl)#bq zHxXh>3)k%wu#Rf&!n+}#G`r;4m@x}B;dKT%CPa_1z>D8|eA|LTR0WxioKRREj^}Yn zZS_brVV7vyj^~;VDi#69d#KW75_CBGS-P?4{cFskwLY>RK~dyRQ^C63uX*^Cck4Q4 z1mYsNNt4@-gdi--uQ9=tKB{Q$Bl9b*?T7%SC=wuJ%KZS+I)!ELl(%j6C~y0;odZg~ z(zi5j$Ca5m6^~6%&ANiHU&S-i2WCy~@rTINp(%&(zZpYY^S9aK+n7LlGgRmbt1$bJ z8o(8pSC$7Z-~f66kR5j+|GC=gX5WV519 z%F9sb&3FqD4OhN+#9tT?x63&`;ghO&RLs5ewVkxsK z)`SVbWwrb>HRdz(H9XX6XsLEi#nQ3yu^1;(De6gS1@!D0gJ zvP->&4?7+hxU(hQ?mw-wR#lIM+87k}8FXB9$BSo!(sHKN69uba02#ZOsAF=}Wc2Qx zhTzCNr0=eU00L86M%`XBRAhP(my3770uL5UTMPg!GhFVwSeJ>HJ05?`W=I+xV24F) zAxujIq|xGMHRDv;KxQhy<(2p((rSdR)V_kG)E0i~cy9DbLhWsb8uG(zNZH4qmApzZ z#;RGR#UlwgEwHp$b37Ij@PT@`ZC`<@Gk|jf%e(PK!z;y(t7eYerc}}j6DVFczwvEP zl-x~j1<+3xwhX|uiL*=-wN*c{3|bQV0E_3P9v1b9ue~n8CBN>lNLc&OH)V?W#IP>@ zu1U+T$Bj%pZ0$ZOb4*r86OT6dXif0PwM^deZ!$!Ro#xgtGIB3g?jt>l=5u(=(Q)pB z_X?Vcl*>GiwwL?9l&*;~|#7z%G>ISsW@0=4V`PDO~oHz-QkmK)}J^E5-Yyv<&;YHc8uS&fB;MUrUjDwbAmMs zzkU8RI4qV6Yw#9Sp(J%L1%34^QBtRTPLzNW3roVt&ku5+Jr^#teGXieoL~>z?f15| zFp0ho$ST1F9dvT0tF*@h} zVef6AqpGep;5(Ba0wE+ILI{+kf}#~nEC|>}la>lv1+=z;h$OUCKu|!cMv-KKAZeHq$wGoVp|bJRInf!UPMGuMxZ82NWN$9bMCo6bMC#^O4s+T?_G*~!m2EQj@QNua*~hY zzd=t3Q_he4n~UQssr>WGMK8OADP>t6eJS@k|N=Fs&%Gg%KcZ5n^7+RcYoy9=hvFy+r?B+E?E)Z`rRjt0A3RJD9 z`$iK(YknsVRoO4%fwa-TK=_+jw@iR->ov&0qQH^G8IwJz&=``6-FxEJ9cM0 z`0l|{qs&w_Nk+A4JZq^T$1=ag0rS@Gpy&XrStR@BiF*W1IYlPT6Qz)iikg!8yHNRS z&gS8gDuvN$9;rWVGZ`(lOHRu`lx72`dfc0e*xI;Y(|n3G&n$~eWcjjVEl@k*>Gikv z6-fB_hgu*HEhGp++6Kj%ZNT*Jha*Tc1S<&Bijy@HN zRfa-G9zV;Nl^WTVl3Z}BQFc?Yas6=P>eR@Bl;pY}B^UgtOY_|CHT*Rc>D(uhkrZ7D zR8|nqE->o*Uh2nYRlJ<4Tid~9623_-_KbmaR&wnd230z(KjL3r5a+)Xh0b;8O z#nw*3&v^#1eOf&aFAIntn}#}&P7^2y~B45IYFvMpW3;dyXXvpjB-^>#B2BbORtO+$`dj| zIh=kHqDD54^wC#R%^-ybPQ0dGBU}(?@X{ksS~VYbC$i%KlXHN<>22bs`P6IDSMu-<^y5X1~aTt%nPLNY7l`w;d<`l@(EA@X=*cEEooA+uS8W8~J-qf= z8Jhu9sczJc=s`sa1l~A8)`A7zen^BV)7V;!M}G+^Cjm+Cyk+Z_k)jMlqp&0u%G`## zWL(4rRT9^#fXe|e5=*Y!FwTArS&rr2tAuPcV4EAQ-664Q@;1lx93n6F0&iY{Vz?i0 zFl7`^c#F+HDdvu{1e}qIOPC@>wb$yb;AbIyRQ$7VqLMR+b69 zf>~8%#i@zI3&p5!>H$`+dL85Gn z$)BrkCJagPP3s5ExKX|m`bjGc)#eQV@Zh!}L*T4CoI@|K3T)QsyFyGO5GXQ09!VhH z`(spGDF*ol3XLF!5Gq;C5=D=Tur*PGV*f@- z6Ipdfye9~EOM&OioQyeR7Dk>1u+i940yFD-KAFm34D)sFV<`7G0}V2Iv~LCMxCY04beBzJ};D zzx5OkK29m3jaa*S$bLM>DT!rf9IeM(?T&OvbpO-rp4Ry|MfSdQ-e+rLby}vX@p2eQ znolc##Y9|&T?hD1nWsntBWqfVn+B-KK1H^L*PW&?&v2{?wvz-6Cu-wnpSkt~JI^2u z_sV|Y8!;fG?Cj!5?iwRL3yvr4^P;0s zB9{lI@Rc2FKanUVI(d2YHV*TTbnDQuJEORtERU@kFz4JxqTdb;fV;OZ4TrL;AneA) zTR!ax`b>ZSPz^yD@BRe+Y7eue5kn+??g@?YJeG&5>Q~oP&_~tbvMKwySsYr=u`3uO zKP8J~@=UG>?Pk3DtXdwNNKu`S<)JC}4c2biKMCvxL6u?Nb17VvTp$wa;w+0voKi&f ze6&FL&~Rho>7v(M?Q3foM{C>9k=wr(DJs1s^p$h;NRgsy3_oROE-y?o_8;?UPk`-u zGP(6GpBg&HM924|CuW+D%C4U#DXJ)YD zj6}uiC-UlId|qO=cb7#um)3EmhMx?9S-=!+>qtG zqM*XD4MAZ@>J{w<2M0ZlLa5_h&<-aAn2t+}9njq2xZB(kvQ^NbH|%m2dSlq?{@#wA z2BeViHPeT+berneFOjg7DMPEcxAURe9F=vwW5GwU_pi44xPiD-UvZa?rcuAfF( zZU1g_o+w#HWrC)1GZG4=u$L$h9Oj?qhN;uYDwE<$?JcL9!ZENUWL@lnEY8A|#S|qD zN*CTr2qDf8fJJ{FDy62>24#>Ub6yo!;T9W?`|F4#K&k;ltFO0Mv2E%Q8IB01Q6{_X zgo9*p2Ia`Ri{@!Gg(4-chX?^dC zr2{;lPRwCCDSWsC#Edg8OgW-e7w28m*`;Mo#jDB_CW*au1%qv4%c zyP-R!qG-=F8lWAZr#T6|@$?e$6eyRSBnN`NW|krZ8PCEY?~hi#`E@k)!mr zLfIW$2dY)KN;6a5FKJMa=?E|p#XaUI&Lapw6enRnJTvDX(s$s2NTXrNa<=|+uQb+c z=(XldSi)U!6Y)Cl2Iy9>eaETK{tI3x7a5DPL({U2>aJPQxt*htQ0y{La@~yy@VhMQ z9KHL+p7BQ_#-j_32O+M6zvT;QrOzAjpGJ(q>BeZh{$rs>JG${-b~`$IuL9L=d};KQ zDZBi_cn`%_j0C)j=|2B1Oiq}qixX|-RZF+wF;mLeu?K_2N0vt)W&HR|j}BYu(9qBC zB%xwJs{*F;d^(Mn?3A3~T%kB)c;cYTj(#8t+!QRcV4Q3uaAH2_$6|qO6!&HcDsd?G z{!J$>&?ZwJ3dHeif)7p_51&FvQw7lECgPGzcWl;*lF zc_`D^zgbzKWL_UaG6??$^b#6N89sZFb=Yc$7WOi?*ge#vj*>}*xj2-xF^SkVvqsC$ z3I*{&)ATh$e(Kd{?#0s}>k!%3#;zLFe&2TJE^3kmv(WkvbQ6D%+Y&Sn8`7oa=_gr@ z+HBKV0uTrgp#jPQh%i8~Fqs+>d5E5G;e4osYwk;F`d*9Sw{C~nwtayfXY|<=&974d zzPVn#r-E6sW%HB!3?TbKt!9$c?;AxTV8IuNZ*LS1GW;* zu@l={CH8yq3&qyUK~=AjUGuud{6K)GJM{kLt<<34n*2^_rLnZqMkBsAV!ZW`u_eQp zo{?7i&$QCFdu5bOE;jz08kvL!(~K|;NJOm<$gZ2?4qtMYtF5GcX~_nOq9k= z7bI8F%!gwmE@D)ad2`?9-={Ml^-`~U^D9zr6)nX7#(dMtjwvsQ*RNh%x($b-4?lK% zOzCUamiovnA0Bdo@X&wkg2FX^2<{U#n8Kq3E+|aoGw$;SZU99F+u;T9E4FcolHI26|)o+>_%5e0>{QR4iR< zS(dRW?#t3GSQX+laW4~lG|_8bAV7| zJ2#ye7sZBsIFO_};a?p`LbvLyU&Jy@Lae&8`0AB)^|CsSE4chgItcuNs^joRiPdp0 zi~((zoiX7~`7JloSIbI8A@?>o*;5WlfnTTR-aN zci`eBgJFGe9Q&WuCt(Jr*&D4eF34A}oB8qj^=-*IRAy~Fd+!1u z@Z91=*PL0q-aHkgQ4?=v1syzgc1#!_atEYi9Vm3v^dq2{TlFKrd1obtP9kU(SQw9$ z*xXe-yxGS$<2~rvSVl|NCq51Y! zZHd>NlC>rFi^u{nD@bG0Qr`qSFqbSAlg}f~DaSNhBrbx0jHkuInFh7i4EN;R%KZgK2wh;_cUuoT5B-{NNc)hq1tf>7n^!>&7>Wb&V|oVuu2zgpJlmDlqYI#Q2V9QEEpAx<)TT4 zBcUVvUCMcDSD)Hvt2O_QeNf(;_C|0AI#fn^2m3`l+Hmma^~@=l%Yx|9VeMq<6YFie;0k-q z>6t5SKfEgBULSLeZ8gw&RkeqM3*_{Oi)8CBFjo6~6UZm7Tcd!%si5T0uHB3i&(5)03D|5^+HLv? z#lrYU5qJXW_PuqjF0T_d$4?;J#1Ihw20N76q5$N!lq$0O;jOxCmvcwaOKdoCK$4@=x0G3(PkN`)EVfsw}0lB&d5e@DX_{YE4o`Z2B=F;A;Wg zxH!9bW66sT$F}cf2wL&}@VN~?5lsRVZnFp|?HA+ZdSk#QXKn36Ms;W?(%N!Ge?6wD zvFMuj0&TrcuKJi=#JqrX#SR3(4UhF}zG{0ns>tvIa99r-KO8E}krF_N2u8NiN$C^F z9LWxSZFh*RMNySeQQPs@kPH0C3_2-)IiI8fi;VZEB|+O?-r6d<1v>KuHWLGIXtS@A zUPe6nA{3(_Od(=Qh?$J3J_0elyloQ0Rb&%rCpi%Ri6eHbE*}yxtp)q#n{$WV5u!WH zyy7y3x|(l~tALmC&$&yDSVH#0iUR|9(OJkSr_rNRRD-sz*)S8J0OT4qeh4_PrGx8iuWDco2er; zpK8rBY_DF{On4ls6YRD<=_BU6HQ`Ox+41JKlen}Hq7?P@tS43-B7CQ}ttHk>u#~5v zI1#6)B${oKo=0hR{G&ohL<0OXk0fu#vcZp9Bvu735@NU4j~P;$SU8nnWy(Z07X&^5F@2X=1y&y6f?;O^8Xq_O1056~HJ0-~}_5P|@1h^%E8blHij%wi zvNU6&kDwA*;1z#8C02)jP3&uITJS4Eb7{GE%Z74!_b=;!W^>rfrc4xtbDxgqVy?_o zb5p&QU)Rdl1fTso%K=19*J8Rg1#OCyLn+0nk`(c!y-X|rAala^kb`C4vK!4odRaA@ zD~Vy-y0EOv$hkvyETrz;u*vq~Jj+lz#CYJ8X?y^*n zgfz40o6ixXY*~(kxb9OnICNf}bhq6FkCcKcX z{Th}RqoOn^^1~O;)gnZ7Lr@DvQM)#16ws7<6Cea&GX_8N6JVYIVx!PXifdE!BqRgh zMoL{z)v9QeK|Nf_y08E66thY7)B&u;76i3rGH4zt#ZAd3YQsRt}SnuVz7STE6!) z<3SGquqb#iWJx6DqoE$a8wt``6S%5F1MRZ3yh4V3?qOWU52R+5v6d4&N}_xoldG+Mgi#o^-#MlG7*E+HEo) zP%@R_pRKvkI7^&j&ntA|_ItKfM94e!e6?haY`HKE8HPjRk;nLbZ+C@P zX2-radfb51G6by{_9_7jnc}pkE+w`=(N_?cjJ%y^wQ@Tyr=C`YbCg!wV|0&jY@vJH zWy%s&cW6kPs0B{iZ{6A%2kbgi&DWwQ|4WgXh>hBwC*u*bWh?ML0eQ3IH7ibX|AF`! zYzn@4b-24Re33D@qcOTeTIt5L(x?%yju`Ky8@27zO6$`~|GYP&?7PLrD_0nsQzG}E zob(?-UzPriiBCy^a3ejD49|(I0k>{xB5#1m^zTU}l-R^2_c&V0W!7$TA`? zofECy*0=8RXF$a*1WmU))rjr7)o7k_byDxttk|RhM$7M>4%LR!;tznG+xB{-AK1AI z@XKU;)=NJhyun0U9(u~BYu_cK!n(Uf@b4|I&Ut=I>op{wEBOs~Ng(*fH>Jx%$qiTV7jNl}AFrezC*eeIWX5@% zZKNl0>MeP{r9)i1NR{^`M9aOZ`^` zSm?8Hp!B57dhjvXUjP^{a|y9=SH`PMDlVU>I6m|{d6oS)qNu~ixpFmv?4+cw&c5I$ z+_i8d@eFc4!L}|kX*@v_VtagPb=6whB zqD?xw<;BzRycV0OJ7BY=?t$Fk?M3Nx4Rm7}newfqXg$Go%Q4p23EATxX%;W+X1|Ds ze`stQ!*Gx2i|09DIJuhGnJ4RZGV;#9j|_`iK*&*&+mDcwo3nPP1u7;*r2^XSC=QdI zDmciNjDl|;o(L$gAy7PpGc?wq$dAcyi-n{}3vAW*7iY-oe8oyA+)LOC@=tfgmc1B@ z_&i+2A07{;GPbzz-}jMJ(nT;r!bw~+42n zQNC$^v^6xY(1dWpToc?%_>3``ZmuyBG`KIdC;=JE6BV3Mv9^7F_nUv*g3qVha=oa{ zkMY6UG0+}Kr|H$ax3YN6Vl32>V_`R3=8#YqYCxnIT>%a}#AHCjh=Gw0;D#GMD~LUi z)~p~qHywPn5;D@DjwGVi$~&=R8h&bKTNd z={dL4vILbGG)-2Nl4@wf*EXW*(9x2}&vr)lzZ42}>=ioF)2Qy560KZ#o=WaXH=a@*6rRexhc2@>(=ydvTluAP>01d&+zn=8x}Ghw)Wc6b$n=L zdP>plhcEn$Q5yu6#8`QwH`G|MGf06~NAUMw`HZg6KZh%|RxXeJj*-Gv@w#q#^iP+5 z{@~>VVth$AlYx>NoCMR#4Xeosg~ECbCB}5z&hDQAUeTu@(J%uo&S+YfimaIa5iwYa zf=h+;5JU-#X8%(#&;<0@5runwn+7DJ@PdVtyu#w6aTPMDM(_b>f7ha^SkolprUjkZ z(qUveb^LD8f;QL-HR*a6SYk>Ld}M;6`z*})36*HpZ{|1F93+Jp=e|d%BDd~}5NfFC z6x=SZDgVf4PtNnxQw|k+%BI&-g(p~Hw%9na2$Nw+R(+1zNE>GtTlx|$eESsyWUFtf z`az|pDsF~pU3mz^+0omtkcMZY%6BTo7q+zPt2*psReVGhh^CzsYU#Y`I?R zVL)^crz_38*?FZ^^Id8o`gVt&fK{T%8o3HY_ye-XNn^VxIv&%ny{0s{vc`S~*>tgz zI!Nnzxd7M|1IR`KnZd z$P?sqs!ai0H!=p`1+gu>B0R7)b$ZZ;+3HSUsbFr?k#7?3$!}PC06@w_=gz`T(Zs4SW$avKFb<# zV9%)OZ$gv=GBWe-tVU4lI}&OKw5zze6te+kiOSS?LOar|Xj5sjYsL|4UpCV?0xMhd zaJTuy_i_1qS+B9BDx@w%0ATU#O(A&d;=vZv*gj!dd%z1Irj*F4sY#~RXnB%2LtMk7 zl7r-wT2k7Y>Y~d-Y&UdMsnSBqwJ1iL$1Em6FOH%qw{^bMGE(IXin$*Rg|`bQ(FV z@;W|$nW&kYIIQxPS?^}RZ44mih2_z80{HdpaHJ>Wo=uW6s$Xe2lxSEp96I)g%SU^n zyBQ3=`sE+7`Wh(T5(myhZ}4&psM#1g%gT~8-A>L9?>=swCFf*Gr3s#Okcx_FvC0!* z^W^R}9^Fn^bgQ8pcZPUua^L0wIsDO^@Z%Nq<51JYgw>I8B6}&!O9752z-@j50OGI-)7i0hC??#e`befm>P3cUTb@??ZNDkxQ znu5P{n?VB4ix>;HOET*Y2J-7M@-|Mya)hnvg+(mfJo8ZYX>dqcZXh!gns~N$IK;oBO zV*Zf_j%y6eTVl&bcH*~nWah73BC!Sfb)u!sA5NLUt>or>g(DixRv-i4Z`${L{r-ki zKJ=}~!rVp^Yc3o0+6RFWW*{ASYPx4&AZq1~eXwB^F;`>m=-WJS2+Qhn*FPM~e@C>p z`U0#RhvDWvu1$+ufjXwE-W4H+N{QeQ%?pi-{WSkRZZB=jBL@?oW(2~)M8l<7ejVV_ ztmY9amvZo3G#>B3T>LNq|`06{Ry*^u#;UCQ)P*a~rx-9u#_TqDOr-5g>oThWhPUTr8( zoTB+#6^oH3^HhUrFSuCeS9hB6Z&U53UeH4V=>Xb@Vl0w>dV^l(J0bKE?1w$XHhXhW zgowH~4*T6-#+sDKWx~Ql>cupZQXCp+r|mt^bE&P{j+AT5xvefs!b~sZ&G{wi=x=zY z$OL-B5^bM$R(LFe;eb3^RW}$G0k9TnucG2oCuLAN7?5je6#wk6#%DPG^MqYUa+-k$)9H82Hed$C^_Q#J5VYNzI+!3 zoN_ipW#>(%vT2#|U>R6q<)74^i|EO;0Q0dqAQSiYA(=hHj|s!6P?1J4515LMp<@oqy{|@Ry>+o-ytuW zInHRl^y;MQ=WHB(Gs!E%7ryuY?9XJk>QXO<${mj#Cmze6l>-A6z{)`l z_zNwSCR{eEHmn8}=E2I)kjhp zqmyxiWNcABmZX-IajOM^<%5!aS}h1Dj`X*&T!e5{TQXO?&a9Xt*MocgZ7dO*1=(2Q zjS_7v@2Kr~ZM+~S`6&Jy^h7eNyFU88!jL^p47rgGwXc@RVacdy_jPRb_|l9Bs@PUv zyU?i{M;fxG5D96vIX3%EKX7DYe*}2N3eH@{i zJ#=*ulu1f9JfJ_W?yP52f9DwgQtEs|q=w_Dz}A9wBUuJ6@R?dPt*K6XwSc0w$KEl}#6F zy{3G)$KuW>p$j;(Wxh!tlonH(RXetlJx0?PY_h&%D}8wV<}NaBiK{{<+4fQlCqe>M zV&N7~(Op+CH*onhPDE#B=Pv6h=j#*s!N0Xi%J4Jvs1lRVW(K^hpKDEf1VA zxn=wMwnxq$802ztTY$^Sok{*KCync-M4tYb4lYr5TNzW2%0`s}>ul07h^ z9Gp>hQE{aE4x=pV+0gWqw0N&^NfSC`#V#IX?ECiA=)h38eqL&H*yM~B@~^Ccf90-$ z87;C{u4Y@`8Lb5Da%GqB3Umo?cwta%q7iC+HYHMV+mz^PxK<8JH`-*i+!%BVPs@wE zbmzObx17B8(N(u|VVAy9c*#x2zS}zIzE|P7)cTjF=MyYwed&t16$0M1?!b?CG17Zx z*@~kF(Xovwx33Dm?dx543(u2FhE-llheniagYU=rb(2{J_~e7(6~n20TftO*z4^iL zEAPGE^K;f9_Uy1o&~f$4qZ^no>d&%7M72}t41q&9;!J>8hyTpi5EW}OIF=_>b-RO+ z---Dq(WTX90!Kx6to<|?--$DL?N!xsR>!&0167n*7GP8hrvSH}^YSO*WD%~~2wD#Q zB&=s`KI@5Yk7CH0`{cB5{dp>lUh_Bc<1gq(aBhTaX`yi{E7fb_0ENU;;XVrriXo*H zBF~uj(G@0e?VVyK4sSN$Gt5PTdW1J+_6cSA;kGj9EfMH^5@dC9>XSg+R-|n|FlB%1 z@n1@1>)+7q^{?IA&O4X<9Xynllrx__XfZi?`H*u+Wj79-*8+Xk$U$k<*@VktH?HZ{ z$&KIW9qZ2JcVsAYyU7~16DAXPB1VGhqw)IRw>-(3InkmRz2%I z(?HNehC&~ycql$n;)bY#JNigVkJH!l$rS@+$%tDk!9%=HXighwu}*{~XT*eJ^A;hn z6@NKGVw5d(FS<*}h!1l3R`+53giX!yGeDIPK7=*${QhQz#%s=$=O*czi{5mzMwO zN8cg-XZ%>lKXP!oH`oObd1ZyV-8cj{DG9Pp;uRTOFV=BH<{Qyo?q4B;yOS zQfM$JoHZLxtC{8q#c+~1gKmE!n2@oD5c^^^vZ9Et@6|ZbAttb_FHVO@Ld={PwaMEdu^c7$Z21-iS}nkaHhq{g zdYCM03j+WL-4EbbJ=j)v4RT=J z3e_!8*Rw9B%E&agPvKpWv?&&rFH|>iw&#XO1nuj2T2bjnn2 z7@%8C_cLcn-oZh}C*rLfmlo*UT7+>*2|du}LL#DvcAG5ui%O1rDR)Fti}V#<6&P4a z5$!2`g;2K8YHdwj%;p%*v7T@HL!xpU>I`p^?$GnQhaCq5(@Yn%H;W>Gc zSHsb>!r`&~hZx<^8@wI-{qfI;mqDtz(CBt`(krqT!IqIp)xdDw@5qTyju;KolcL-A z42r#Egv$4(MjBt35}gIO_S4kp1Cuk#E-a25dMzjZy@;`7c_`j(TvB!Wtk}7OjNad# z8lwi_t@o!!8mgzn7E~IrTM4!P zga^tu1xZ(OV@^B|AMuqPYd?{ERn!Z}eg0ioniOwrSS$LMRiKYtjc`7x_{qbu5*tg0 z!Z=n!{(^DWH9a~^6TQHnSRSpyp~#B-LrVZW{9xSQdVbzYG}j9mDO-e(<-Ti68ySam zljU9SUUKT}MDolIrIz0FuTzH@xFsqlQO3bQ?%ny;Rg#otiYknwLnHS66Wb@pTg=y~ zx3JEq?f$V%bdKY+0acJNN&=91F;=GI6M11dP-on59O-oiV!V#nu~@umAgwbM#r)na z?MJ3lm-Cj+XoM_{Pria5h4G118qTUs2?r_VX1*3an&u)w0!(&OqI(0JLyA$_uOBe7 zF3Jy^sh0Bvdja}U9zs}qEP}d-TR9D^24QaBQnyChes+l%oy# zhT^$Cu9a-oT^e!L%Z;7T<2SL1gVOflzbS*-^QSo&$ikgwLd`M()QEgk{LH&iB1Qpe zh*NM6-CLk~t>`FCe8MLQQ<*>(&a(yhsPyd=YuC**Jwf?}KJH~tQ0LwygED#k+{Sf? zN1yCo{_b!M6UQzliita&=(mER^u*CoV}XXXRtSy9ng~=kUAa#93wrEfd_F$&tV9EM z5%Te-X)$r!c_Q~;kHp4Y_EiEkjref^|Hx12A;falX0D|hfbeqt0<|>2XF#G7Uxdu{ zkCY_$yc@hoNF>f!&D#ATF_$P8N=>12(qIp9NSqNw!?`B0DdSXE#Z>|<4jyNzrVL-T zaW2^^ioz=CXP{ACA?fA^jVZnpr)d6GaUbCFgoVC0!)e9ZUilrb2OCpY!spXK#sF@_ zS|vUJ{>#It_9suRQXdSxs_(ihsL|vz>(^JTepT*&9$5j$Wn^WRdx+{FA#1O_jADv# z8QIfP6OK#tcJn*%Ju_vZnkN@dI_trVWBw;5y%cE#ky(vODyX2=0%qMHfxLm=iK7E2 z-DY8M)AQE7uJGnYn2f-gB(_}H6nd1!6MDBvAqzyH0SRQ9k9b@RlLgS!u;CCjeoGJrS+W!m1z; zh~^H9qyq>vla$gDix>(|hhGeQ9?@7#=9%Bv3qZrWNj&qkMBu7wjIGyrqqXMkRNLd3 zt&Ss6nU1HVis~T09_c@p7;~7*$>42Z;!d_1XRWH-e8&0>9E8gCM1Ukxxt87;bi7s0o(mVc}~d)A4+w8#Q3KaC}^R({oSCo+PX zfPel3`q9ZYa0*X3u+y9|Aij9%lhhlcUg>EfjY(GJA5s12EGXuLZjJw>rgpS;JL})n zYpgG19qD$gBx?~Rv7}8nkb|Q!@h**H4sN!^!Sx2bi;o!8rRD4lLVEFg@Z(qMM-YA$ zTCJu7wdLy3p=9n@T9z4JGo#g~-#|aug4OsNR^k*YK8(xaIIH16i`4{t|ML-mR#Ro- zh1=@a*DP3|D(Ai_a{BG&9->2eWyJuV$^|^pv2v7I9=8aW2|{k9_@vw5Pr=p3t{T*S z-*+B}{fttb9jdRGI?>%nZuo?Ewg>bZK-x+rzZ6n%5E#IbOh}k=F-?Bb%65CXLqI`E zr6R7hn&UOjAWdRI?#(<2po_O_->?^1b+PZLO-Eub+TgC?4Q{+Wc(z9jJdC{O!Ip*e zo-SpN?RCqRkxWpXui@NimoEP>Rm5A?maJ7LVJy9J;wpj?IdK{5rRys#P%cSdJ2Adf znRu)b`afmyHV_U~#z7QnY@&kn@;9!ZD}7GNcU?a>_fw{4DOB~fk`aSWuuj&!M|B$2 zDHB`noy88Do_I{aL*%r1OScL8R20lXoKOCjdSwZG3%nZa2$~DdhdzDc1T}fL`t?#| zW<_Ld^KDBk(>s}9(H-aUYxYA#3u;X#Wh*EJrM84(L3`Pedo~M_N*uK&xZ^CzV^N04 zRyK#nK)s#>=>eOFYXWRJyp$IUq>Q!tF_94CImr(;Yep02r@$XjPzsG%C&5&!g~)o% z#Iu=^W0*(aP%zMPZ1L>NFx!igf6nWf0Wrl5PBhe6saazrR2>p?&BDNFz)9qeWc~36+Lc4Tyz~$gHNfLsF_DZ$Z0FqD)q; z*bq^+UH;?!jlWIyUms2m`nl*#CllLF=!$p^_~7V|zS%nRyHt7Xh2aoKG#+|V=~NO& zn}&5aeWbMa@R1UTQfnq-cR%^pDmGZtY^Aiiz-N&+U0Q|keXaK)hN5Q z*jPQ>SdkhT0uPia$#s*BveIH>_It_Es!;5%P-syPV_&zd=p&t@zYN7DhC)ZOLtB@n zzAz>wHK6^#_@MR!cX+fP=$+5ZT)gLyjIzPSkrD_iS3p>~A;Tz|zBAH2DYRtayxe)kk;h&$%F>>#WCP3NJ&e}-R#xVw zhK;(h$;~?-4}aXiEx|Vf-BJI6?YA00V5x;2^P&x?l^cz+xwDHS6_Z1;M(}Taf$#qD zdZb+_xo+&@`iCkqcYU%ZgD9G8eG7uAS5NqR!>X*$hFm~g9Q*fsadCf}{aMqx^V;D3 z2X1GW&`RENe%&n$5T`yEo<)Zi*H=|+e;x`k-&O?m%PTUC<2Y&0gW&_>O4HA;o6c8C zXWI_cnLN>Y*#(e#Dxl`@DUlJypFeN9@=_@xb7C?q+mMp_s#7jVVEvK$Q(oH&#R-kx}iuPK6iq6PS@c^_g9zM6|xA-xFl<;ZTTGF4(%%-k( z4yfa!Zy9Mj7n-QGumjcx#?F71hw!LZr!sb)X>w%yRIBix;Sl2D?YYClc`bEa^02P# zw;F^ODc_Si7Ff=?H^}#7LqLCU@+{CZaxd5zxwXNJ-0WaRZnm3|D|0(o;?pcyl`3+j zeam#JN<{ncd!$HDi>K^jwC^peFe7Dmh=6!j^`dJh@QjGptg8ZHn!K`eU$x?2$g+f> z%Kf(zt_Xk63vGd^6-TL9l`KF{R-Am>$~9Uc8%4^ZaAUk(zoJhbM`23}ZHmGXR_tq@ z-!Kya!Mzj*B$mfz(@?Dy6>}tArPPAU!_Xc@VTxw*ZsgvI)W$}Sw|S$y05-r~zV<5W zsV&+s}=(v@wO{W`M6voPigrz%0dcfNvc{$hj_JlGG~3!)g^! z0khT1_|KeFY`VxI(@tzXV@sM_LyM(VKDBZX67hp1xe~)U>y5UqKKVH%!u_LSmYwM2 zyJ8*2BOqJnsZmf9-EK6X3;t*IeQD~{2UOAo^rZw^RxO4X* z+J(){QL{7Bp=wrP>$=F%uFb0mPB~hH5AWRiI?WShh+>}Mp;ta<)&^qVQy?K>w<=rG zOFhXFirIq(uWF6arxP_alJ--Z8=<5oq6YMl`@AUn*D0lEu#enhXAsEU@m~RQ$?s); zXc#T~uSqJJhJJFB1{y6>MIX6F7@e#Vo+m|XjQCNQopb@DyJbrB0Rt+~5sh0`$p+Oc z=7(1^4_9~7`32({FYTN^w}M&`P@z{FS`kq9?lv@%D|(TkOdCv38rJiVH*DR}bWB#O zMSbLy1Ze^YIqEg%gWW!V!faUQq{1v>ZO$05U|0e)pl-^3?$|x5WH*Q@7HJCZOmf0D zc-CvqRHC;im%%7ZaevlpL4e8C-#qDp!V2rnPghELv~AcwMAKM3qG~NakHXN*$;<{8 zEuXTVkYvSo`fM!O!NXrMqR+z4$dl#!zqVcb#uI8)A^=k7336Z zUaKY~o2%<}3ZKa;IV=IQ?#tm^`pc?WL9PiJ`i1dS%Qm(}&m`0G(mb(Wu>dNYGOw5J zl><7~rc}|!FBW>AKHz2keZYq!#MXOOj-Sv)Xg_)Y9v7a+@OYN#Mp=Kp8TPKAL%>{* zO-kwO*MI%;B%iFc;() zkMQU(R;Oi3$256Fi{$cJtFuohnTFjK*$Mj=Ah$%`I9X1qbU4dZu%rSa93qoWjiAZf3 zcUiC<@8lE zTD`WUo9cJglSQ+gBUniyYM)I!t|b2UU^7?EXC0ZHR(6NEGWf%170;B`v~ADr z*zq&Kn?+J-rH-JKJ)dng0FH~K>DzcK*An-{R z%~XH>N7$H{FJ-5rS?O>HjFe}>EQti6*xi6P73Htyq8t=))hKYfP;09+gL8L_ZV~P% zW=O*F04&j2kMd0xiC%lMi&N>&j4Te}ZTQ)j`;ca9!e>>4|C&0jn1M`KSg2#IuQu_& zgSpmx3ZA{Yhqm>4r>QM;-l(J{XRO@-JVEaLL8%eYZ{W?Kegmocegm=iom>{zFmq>Q z5;(W{5KiXgMUTSF$~Pfe>B;s zpZ`|$Sl1jQA9ZnA3Ceuzgh#bpc-hvy-|E7@xdq#A}6P zWyRD`tY%zJ{A-}zUJFMXo*NXq3ew77JsrvnqinC&y*(>Z&J+HWddte%0r#F;G#UUP zessWqj`V;2MO|8!rn8oB34UD7KbF4KYfgTy)DCtOMR1QSk3PzEanJPVu$2xC{rpbu z6L*DB_zmisE&2@H$gi^pcO5%;MB{60%(3U%QUquK6z^rEd@9P~_6xW*s${3hK#8D$ znJ|t$aXLo~jc?L566rFW1k-S0#~`Q#Jw*`Jk95o^3^ECLOx$Z)47)qcm% zx4edzF(|(GXbu8ZI3~kA^%}viZ;Enm$q8B{Wb-P2Nu_>l6bK`_Ogg!lsbEL8muffm za-hx2jWfno@mv*nIidqR(H%z!(q7b=I%ku05fPi2Z#f)dI*RpK9^x|+672I1QZioL z17Zj%inoEfe)F3e*OTl`q3(W=Tvgn#6}DU28!fmYxRkWD4!i5r$nSC`p@nrtWF=ON zZQ#V34kxjoERTett~gdiU396-SMCqcrqQMD?dH^`N%*JO>Phg7?y@Kl5+TS0H*fCSJ!{E_Ah9RouH?EiV724c011H1q_%t*WN0}_FP$jzB)bNvu-;LO=a)*4+Lr|XCHewKvM}&KKnc0$4Of1S8V3T>7+*=B5CA{Q@)jq1*h9-anWY2q2`-jG$w(TnfqlkB5yw+w}r%qGOqOd&gHR@ATc zT#G|HRox{@75DReb>BIHp|lbCgJyAtCgm(bisNWS@EOr^u}{yOzH;Rmq~#ucED0*c zZ9rv^m z>OCweHegNaqpKjUY@ZQX@EQu?`k)Z5CKRisqPRT6Sa?OG=+{@3w*8xql_ze$Ulw5% zco)h|m$;7Gmz9j~+uW}^D;jfe?%O}_ zsY%Ev-$I>_&9brWu!=+!wi{*P%E8+NQ>JJ!X1*i_Atnq#2A#5Yr`+91SjpQPKBhVZ z%Sy@uwuD4+T)#mAV{7Z>Sl_~-3qFPIjjT6@fD;(pj9Z*=lHnz*PBO!Y$i3SU(Br+6 zMl!3glaTo&Z9J(In4N{{;{;^4P%;ai)Ixv?Ga^JvA!{ZriK#mSjszkL_+t;ltY zRGa4_EEY!(M_^2K&Yk(!!kkWejU2p9SnUu=$Y=uKrAXOkX!FBlJt^|&nwVqU*a{Ak zUo@dJMa&9Dz=PI`perv-Obd(Oo9CAw07L!ZcI+!H|9B|5XcliwEw4S4-0&01gWHPb zBq{8b^#ydmM)(cv-r7)1Zg@-?%gDaO;X6&iN0y0pwcJ#e+APqq7I;4()K6m9u9Z_M}l?s@9{M|x+M z-g>-d_k!C_egDeY_g?a+aeo|l(JdceJ;Mv&^QSR6HxFMUkfw6^2&h7`FCS{M@)OT- z%9?`uOAClB^Gr$b8-Ye)PTCz|V?p6-ajS5o)f5hpBZ`Mt=F|{R5W>~g8CrJ?oF^W6 z_kdoxi~+S+a>gbCa@hx4y*crs0uyRhDO${#an|OAEG7wGnXYk%EmPzkKYd=vA-t3f zZQ}G-GGx4u@50PDd9*nw9{C_2MXOY$l247~7(KT>af-*7`6yb`L=ukhrFX{7F|t}Q zsx`UnQN{Om)85Uq(2@T{ZKA2A;$QnM`KZTXj-;W-8CyZw{tWSid_~8ZS8JMI4dh!);W6ji#hQ}BF3W&jR)aX znU+@iLRu;6(ta8-2B#aN@%oQ@jj{cW`dl(SiM$k!9tc6xh<_O!rR;JD>?@~5j0$Ol zLLCVtb60dnQZ&fMaw)7-u7*p>ytks6Jwryz=N*lP%V1fV6vEqvTRylZyo+_0|0{Ku zQ^D-50XLVF7JsgrQU1p3kuh+z{8qYAB#lrWF^Ri5JyEK~wjv(Ib8DWieU}_Tn(w_j zol3EMdd-=Bm_D*gF6Q!YMgT9&4%2*_5lFX8IqM{-!VH#XCul}M*Py3b9!`e&gqh4co3hIR{q4~1Z8raBYfleZ<)v^mBRq?% z*zueN@mg9@D~B7vBxA|)DHDl!EP3kT*jgdNY+Uhhtosz`F=G@q5MZ=$oWPXUYv4L! z8fA9Q1J_N@3_miXWlq}xa=_=9VuX@ab{9CHA8F0}+l2pY^-1bRIWPByFa;D`TH~GTb?Jfe z9Ao=7-9h2lbgzW5><&fLoiwOUjDIoKh!IG7N!_wNbMYG+*I`&Ff^7xBtB@TCh7IY` z^7ND3b+7p*(L9{=Cy=>_h{AJxd3U!qrNkh@LOg>7%-Q$&h|A7xUR}+3RI;$NHTHl$Z56 zh}vXP{${KEvZ#QM>(}QBV7@#tCgHceqY5Fp*29E*X(9E4ye-38G9EG#huwb)Lu_$VFo*tCJPuUW5N#Q^~HgXi&QEpP3Q>SYXRO! zFECUT3T#4!qmo4fs*CUqkUi`pb^PGEXPem5v3z_1`I_zqK+y&wl*|kL7Y8x;UW|84v8tN~ccM^yW}UeZbw zJf``ydQEU}swH^PJeE^}(F6pAIzvrZ%wfBgD0g}Iay#p&o-w>knyE@^*2hJM?%mOP zZHhCzE*LVZ^sYjPo8cx@8!dIJCXWST@k8zbG$qaZ+>-dI)?yJof8GVgJ0ySNcSH z?2I1!pQsW2Y3JyJq1aWSQ2*>u$82LCfZP3{*d_2x=^mQ)*y_;e_GzUrrj@QU;)^53 ztqYAC(v2kmZqFIM7mO@FvWzUpOzMzb!OF{pQh8Ym<>gM{f3h8QbJty+G$AD`)(&k5s-6yw4yVPh z$1lx}a%V5=&-~YpZ&$u~4x@+xU9&IZLx;b;aWwkLT~5ucH*UXa+)(;|Gzhwfh46A% z_x9xDzkh(r^nCje%z@3nZ@=*R)k`2%WNtVSVv&3lYs~yP_(7-*J1J^U zQEzkzwR6N{C05{BfDn2!z8Xg(lEp)~4WLWfAW}V&_R=gU^agRT6J=Ec{?MkF0^bABg=r z)S-I#(&&~RK*`lPCQ5$Ztl;P^7Vf`O*3mLx@D9ubvGKj;*8W~fk$dlIXqA1N5=9U~ z@=$<1@YF5$wg$G4`>=ZpS&FHk+FFu1G8w!LyY|ufy%+y-m@Cv2uuH&TKI!a>@oswE zUlng&#MAxdCw~i2uF?Bla)b6#GVGqq-X}+2nv9d0F9d*lye9|qR7Z#A;Ou^4Ye~>B z=A+O28~6Rc#s8#Y2SQYWb9*VOUUJ`eI-c!4tmJU8J`Tfz;0Wkf;Wja zNb7lci`gwwjhFS1;t=xTC$G%owL%j(ONUW_+@`m$YK1?U&lm&4R^)yi@ILV_SP>o` z;yF2wCCFW;a8oUlH`+>9y6G)kiKnT4rrt`5(0h*42w20Su$VkF5bC>j`{e_SdNEPU zrN&7DHJs(H`Vw>4LLb=ugf*ik7^ib(hb?2Cr*aG{t6-@TQJIAV`3z2@XqV;zc(lQe zx_M`D%GnC@4C}f|$;Kz`9tvRumH@dT+S4(4g}m6}8HdOf)*hNkXrWsnudqGO zwv3K^@sy7? zJI7>`f>zlroT};$p_J~2pzF|LE=YV?cvfBjH@E%Y=H}r15yZ_=UpV6C2rjk!^@_Q< z!V;wj9p+Y9JqqrZ-K-0K%zi4@KN!A=Eh+16J-=WQ4#nPoh0RsEi5j_`*OtD)fa6y2 zdiS-ZZ}*xr3<%~o(HE9S*D>7N+OxwE(J_3JDGFSPsxX1bcxy{UR*i33&`PpP`23u5 zfQ1r4wZrLPu_6M`Il!Em`s6<08+O-f=QVcJ_+r| ziK%=0`vk4P5U{9XZ?)+-(y}1#YB-QkJDZdNVjHTgDm{QD*RvquDm_|?jaCkmtlG+UQg<|Sgmsproxcf&u|#G^2g8t3C8BT7#{G8|jfY`4Ay}kt@B3fJwH>y>pzVwRjF?=@!;xFltBjbftQH z9bTdCSto{zi7uz)*j}{75d(}%8}+bSEet?S0M7y@YxqVS!jncz9R>-{r2e~~hUA1x zv~d;p=v%LYB_|-u8dZA6HQz)|mdG+o+~~@`WT71eh96fkA6FgBqs?%)7rySJib`bx zu@hBG&(g(0Qi?c7>c_Pc2GSkT4#Yp{P-8AFRo%#JMO>;E12nPZ;=`6`7Z>}A3m(#p zmgSt6121#I6^lPUz`*U(GX~tki4#^mEkUO*MONwR4qiS74aKeQ;N^FnS9yHF zU4~IU6TJ#v2pP@K_b|$3ckmmbrMzuQbhQy~`AtfsM);wWtF8p=AF2p170u-IWph)m zzWMJZ1VmcupPs)TXr>%?mP3+Q|*8=^C`HG5F&7C{LQQpWznlEJ0og0O(E0HLz-u zQJ7rE6)ICnvqaL5YYWADPpd+H9k|y6lyUS$tQ{*!@q}v;Rjg?erMzW>xwD^OWoYxc zIql0ArjqYT^JVz)Vft|!Qr5wg++iX_!QdynAH6@Xt0(e|c?YjB?JIjr`^w7L&EgOO zi~P|^3WYLA^F%S59+Iw3;vm{@$jZ6pJ=?dwJ~EwXD!Z@}+pb99>Z3Ho8MHBLQ|BU- z6K9BP$buv*3n@aAN|DvZCa6MQTlB@*BZ0H^VG92dw<3U^G;1Y&ffdzVT7EE^NyejK zwy&L=B3V2+~nL9o1tcIZ~k z=O@}BeRa=IF<7y~30|S%=2$n2l4A)h$vEn#CBsCPRboo1FX$K$@ikRvJ)S>KkRT5M z!Xavu;Ml`aRM?5C$!N__2UgU=Ka)b?Z4PI!XQp%Vwo|6IOf-8>OH4mI6|EWCam$!{ zL#WKbh!=rj63rU6RffS{*)y|MfG`WyFPaLFfSt33*1fcfv&Nd81}to6E6QTrg6ca# zG+UgWf`<%4C>_&if&=$TDJUES5aNOUZ2@E@BF*y!p)t)&09J^)t$c%e@~vB_OvSv% z7RfI3!SvnNkjUa~ShwbOnkO(boue|GY9cx&I`!`(Z?Z)8*}`w)GD75H{c$l#Ok|#i zwCuE3(;d_%1Z~)uM#+rJoI5{1&r(DoA7l3hSqC=*W=5^y6m#5q&6~L+-Qc_jbu(LM z{iLBaW!cb-y=PEXdEf_?p-AUGk&LA1(r|ceK{&g>sPC5-`79ir8V+aYhj(8zCaJnh zR`kx!(HWuGS5Pq5E%Zz`qj%@5=&hkxdq^=K?P)Abi8QgrN!?hZtgsk!(6X%NfY2} zdGR1)-?yhm2ZqA+^HQV3Cd03?vk|S`Zp8OML0OPxQ19{DZGG#wKX`Z9AG`$p!B+`C z%NkVgtq|P^G9j?M^y;MQ$*06{LyYN5L)=z z!sMbmD`zx6j{jZMo9Un8L%k7t@}u7F9ahoSf9HdK?Fi6H?$n!;NEt#^Yy9})bICE34|nwJ zud{ZIC$Rh!V6T9CDK;eSB1o@OvoCnK;^JQPesXtDAltSP(s=Qh(3miN$IGK#{D?w1 z2vNGPB8>^*X{an2SO0AI+{ULEvz9LzKDUJ1%8?pnbu$!mB!{P(Io|D@AdhsGbOC=* z7YC?&#fOZ80;Dj5ck=_L_ep)Gxi+6ga13XZ2bo-c#3q+(19g^b1M1``>!vfgM4Kqn z;}ZTQO9DH~o%-Y%P_$GscV+xf=I|{s%jM!5qVEHJSB5$`8;0MPNXD%S>@279(lJ{GFM2&^;zCh9^eu53*TVHT(;!0wS9{0wRUIV%-b}wIFYExm5EeRC?eWs zJI&8jT(@feNir?jJrulE!|0!UBK-9dtKdXbFawF_lOOcd8i&cZL!BJL*ru<3ORuiv zCe_O25>Hp-%6IX|d&T;Aen~L)@gpo4Q#DsA=_n~WddHNTO?jG^`h>pXlCL9ekeCa# z1{{*P4aZ+4BQ^~S%R6wq`(>zOoAZro@N*OCo|7$#>?7Waegze~&r&QKz)mLTgY_{4 zrtmf-ljZ)ONc}92ZmFDsHsSzp`fyMMeo$Hp8wCv%8X<<)!s^ZtdDVP@nT0-0TQy`o z1FS6bozyc>zrj_VR=6SwYkQ;-h(2Ma+JRo!0T>Cd6K@6i6crsULWA=zHLxq}J{K!wu z5K*XU7G7%9xyVc?*_Z~ni9^UKSSq)~m87cMgQvyLF*~gf)#ZZxrtN{YPgt{H3$sW# zn%GC&LI+}{Ah!}s7%}?pp%+&r`%qFV+7@ej!4?nbGor*&DtkB;q8=u`D)Gv}_zq{_ zE2DXo6Lu&VhO`evCE{*bkG&&HgilgrO+u~oFBmyv9aN}HKYHNZMtd!|#bfaSqFde$ zM$VG$vaL39(yKcmom0#xO-}AgY9=57;TJ*9j*+rMJbON^-+WGH*@jN$D}Q}D3E{we z8dQ}hnVj4IQYh51SLjGjqq=8SbbaUOx=`%dQ0SVTq5e;<2xX#+_%0*9HDcWGkTE>N zn2tiZ*V9UO81ZPt_;hRg$fZfq4*}CqI5!|KazUTSxk=H@;qdN@!^V#yu<&+)|D$+A(;|IwNPbjP(^nc+2c zt;))B{|utscul=X5OLV}cGdjnlFI$hN2tIdhIa;$>1B69EouYG<&J%@p_AUsa_C?B zitQhL2sVsbLOv%~#;Z((mQRR{5B-iJUYPoy9F6RoACSX27nXGyId=#|ls}|=^V~^0 z#${`0xl0N!F$0gO?0bpruy80Z!b^z>va8ATp@UhzB^zpKGLVQgWiDKsny0uHvdVnU z&bgOz^%_gLNRz5v(*hWL68D)4hH@NS>vNaFzyyXU=4PxrjUo-GGxen%X6mx#p(o{j zO^xOziC*B;sGwxMkdigrB_Wd)VJB=%Dm_d)a-!f=jRQsoe5j11Mc*!WrkOao;K~dU zVQcO(hkN<=QG=0LhR}ujXev#`nA+^+=C6XeD(G_5jWUPK7KZKIG-T$>jX!3gw^w4F zCE53oPwyi?qGy924`KqXh}<6iS^!l^(uA)ljXIG+FS6aQ8yG8|>2!D_B(%W^bJEBx zB$7sh>aw^DtIBM-3JdShC#!sj%*FNMG?kFRDRXK#7W3#(O_#B9+KIHL0AeQ5l`zY# zU>X2W7>at`2F`=2Vpuz3vqoH8BQa@!Z%!Smg)-O*q7$(NWhvmyLL$(_1;mbu#Tkwj zXk3|fOa9O7BmsX=`>r!_CuS~e@W~euvkjwq}KlcGTS(-$>eg&!g0W86s zPrSWFwIJqjnGXDjs8vUZd~D1S`f4UlC&%QG7a{28Whd!qVX4*@Yx3hz9epM3>aM2x z@4>VW;fZQ32kK$si|rnYMH4bnwFtNe4o@iBw9^8V$f)6WN$lo7n3_KX;bP@w;(jmn zn)<=B>8K-H#&y#Y^L-IqtX4-&!GKy8OxGf{>&e|8fHfEwi!;<>ZtadT<$y+WH5Jro zZs)?JaQ4vf?jc6uki5uUeIhfHqBkXlmkkc1H(63|7@zzh6nh*-C^?};ImW*1tmtn$ zN1qDCDnp?okDq1CN{#GFNiMk6D7&fHxPG{Cb!ucmN^;$gk_GxiK(x8468>=t^t z@!u?o3+ywu^U?ZW=2np8?$7|#OymE4XfF4+6qfWY2RU~aQ$f!x>vQyAZFCW10M9;i z$L?(H&@rb&e_?X6a(VQ3eEwGPx^6kFYmPlw4lfhms<`@;EU*` zl@poJ>z99ovt|TYFukSj(jmnh`uSVb7LWEv=GXYScfO_|-2JC$y2EvPTx>U(STFQO zXZM)IHUqpQ&Vr2>HDD*?;DB8wi55ROkhW@}T5ah7^%%apm$XvAGagqP)23dN)RX>P zOAL|#l#{+|(Q2;Hr$;!HI0yXwDsjf~%~u+}ew8bCg-Hv#J%xj4?~=Gxfh^pd`}8Go z$9^dS&u!q(jsd6e9`<+?k?OU>SmLv+$*Uk4o?POw45enj|9{`-Ihv#?z z^vgt<5}>K9j$}L3YozR6PLy6JxSLR`Y88!YBH@9@Ke&7pKCo$!zuKa*0o zEU;IaF+#c7%OpAXfyi4;idtRSdNY}U;BM9|`L?fk_C*5Qm>0b%gJN+8t!5N%FcoUQ z04h1&i=uXd;j}xBwqD~Z;tc!W%FBr*7rIEeqsUQla%1cPiZ5q%NcSW+aTO%U?Q6Isq6LE|--y-S)`+S><{X7=VXv`2vS$T|pB7E{%=!$T%4&H;{wydx3Br*m zY@5VdzJa1kk>gY{05=R2i5-9@Xhk(6Yt3Jg7qgGL^}7M}6^~M6N6p+@Z!{;Q8Y?Sj zgF~dPY#LMl*^`e*IA-IJ6{fBN?{)O)5LMKc0JMOn5AUHSEY0BwPvDE#Y?E4Z4 z#$Gy)1!HPFtQ(-{oqxW~FMsEs*3m3cP*I%Y)HzLpce z25rS{ox?vwe}PY}&f%*EwlDlQ{VPx0-tvjC`zZBRS{4y#0iL^t6qjegv+~_Am(IO* z#oU_pj9YHlx})hBO0c-t%cuVUDG$dEVstZdSmkwn=rYSh_wb3sDsP!}B7<9B?|CqM zKvd67Kfi7|*O{erH*AR0pMYOK*W=lW*Eps!LD!ryB@D70;0&uRVpQSYPi@2D;3&Kr z-C#{#YDD8d3HH!J;nZ`g*R)WWZ)r&`H=R19AYydOr6iE*yXZ8;l{LRcL``d`9cyiB zyZG*QfA6yAhHl?kd*Z5J7=LVg=<%l?KKti?+x-3?x4pmTqx;XDpm5pT2C_mq`qg37 zV;1PJ;@ilJ{vUbEFBl*oNw$ggBu$7gmv+rLj=XQk@&lTqB8** zTmtr?i}tl~TeAWLRAOW-Fl&#l@U>lXJ{9cB;1^8IY}3N=d&?>~#B*P3ohWRyU)Vjq zcv@P>&CGLvZQbZ7#P;bmj=GDZ6<2fCUWElClqgdi`hvbFGkC>OafWp@^ebS1E!PsK z430v_)AdEU)FOJc^qQT?*5i{e#}|b%aRKPWOrLu+wx;w2uHr|L(yXTx8>1HYJyNel zcRb&!I(9eUhebPR2=jy!QDQ*G3oECMu%*dwtR?v2wT*7pK$F?Cel!1HqyyEm zKTJH=dg5xaOayh=AcIkEpXd45&mc_<7~`*K7ZwvvPOBmSp9mMQ`c}>`D>JD~^Ba~# zPy5J@X4+n^%6jM*7ZkU;B}7k=65NgjQCC_^tCS77SQ~*@Z?Uq*--~z_b>Ul^b(*W@ zi^*pPEi@}d8Hb{;)m)Un6Q@Hp{GT`C0ZnBlW3}H%xLOi|wCNz^AmR|$L396isr0Z9 zm3rK1QAIlgoTAoQi=b0yO={BFh05lbJi)(67hC<95#Cm<@jR8UQIL_`)6zF+5NFs| z1>4RS-SXKkv=XOQbEUIq78^+;jFYL6L#fGicO)0wVU$fRHg*m-UU_eFbXZb&_a#Q* zz`V#0`b4fviuMGpHz+)IkWpBW7rAU!xOAHlUmr2HbTp=?r-ofn-o zIiu`?;>gRd8D;6uhPEfC#k=GfnaeMU+k1)2qPm)GeP?h{T_%d^>N1RI4O$h{ilVxT zsi9cSxSaUcB1YS5;b?;}MQQz&*=sJ4ysI5>@3|G!8#{hcM$L-?>@hi*s#hg1fUqyHCw>E8~Jti;I1HI8(*UuzS2F{mLfOFZz~rsEYPru$2Ha zcpn?z#!Dihwepp)CX?8-=Cw6-L@4IM!*YzUvuwnV3)s$*G+b1s=R}P5*Wk){cy*t} z9Nu>x`%;)Y>Qmqc+hz~o?s(&IhdL!^l7%HF?QZ4QwemHE&w)+E*Y->9TQ z%W=CaR2Om7QMA<*3CMa#xn5pp{>g|Gtf|B0-u7o|c+jQ3Kcp7eyZUTAU^YOg&nM^q z4%OdT8XGw%tvVaGYldy0&axz#fD^bu#Jh6cbS<0Vq-^5JH4q+?eXWAF% zio)B3F%)NL(!T00*soz(?mPA~(q0mSnCPma_D+GX^-&_WkLD@ZITSR+Ymfm#)` z-WEh8pe*l1aO}M8rxILRNLpf$S;cbRln@@A-oN>R^Ft)naLbyU%2C; z2aj?O$&jF)=m?y0E{CZqMIOvi)LuKTrA|lDc79~#ZMStm;uid4qmx~A+iiu_B903m z;78so4aHlSZHQj=*Y>9z@bpSg(QEK|v4737E*@f@XV?8_PRdi zJ9@@M1PSVKb>ZCQv;~@V+Sr@DHe5c({*)w&{CJmzZjE<04X({=b%tkI5W60qw0C&N zmp2{ih+k2gee>XMITfj#iYBM|wS=?#F6UX)#-Ud3e{x`9pI14x%=trhVhV8G_cDvW z=TzQM<}4fKEXq!F!l5Z+ob9JMlX~VSSH@B|q15iQ*uc}Aeg*l-B|Vahr&ZkXtqP~& zzjG?KI?ewDrfsp4-PJjSzH(1H&Ho9+x5#;7+bfB6nVIzyGm9rWm7~j?M@KpjW+g`B z(6tV1&|xi-o%~KLb&TCl2Er?)J9QvPerH<6Y{AntZJd^TG!}1@y$GmJ`0RVK8+sKb zm!d7f@f)K2Xm5^_+KE1M@2tum(KXWn8`!)KG+p6t_*ibmu^Aao^Ayl{(=X5Hmz|$# zf(yzzVTe-tnA3a+PADgpC8}?nmi)1UKIWsFo-LF>XjE?2w->SUZ^Km;8#$0)o14|~ zKQE2@;5kq^ZFgRtOT1LuqD7eT%hg zBkTekfMpt~joJ6g&eCx3cHFwl4|cb!kjsnEv`!O?%+d0fIeDj{9i=_@Q>Q#Y62@n> zvXEPWHSz8FnO28zO6gdBVSYL98H)Jvk1<_))mc^ZKpu%^(0nCYpQ;JRd^5HH$w*W-Q;?lHS*{e04UK#+uYE%KUO)C$&`m zRFvSA7mkV-{Yd&+P99JumXIx`SNP576^>OP($})_lD;Z@cvYESmK5Sp#^wwHSFks+ zL%5L4CuQ$~JGEn%^*_Z1b+E=$VU8vn{eoxRlv* zr0d_kMDWXW*qB@jq^;qX$yA5ozzW{9c`kaxTN7_EyJSZ=yhqgE@(6F)Iy$oKPCJlR zC*6>$%ivinp+RlIvqE3?2yb7b1x=*)Klxfjv8g|qx$8CiljqN6_mftlu^PYMf3s{z zK!tOIUl%$vm-h+Fv%|&?K`Nl{&g*{liBC+jje=COAwju1<%Nk#-x6YyP;>73_c|6o z`c`CXuAPlj9|`p4k~d!hIZ*pMU+jxFB*j_l)nhwXy~2Jct!Pb9^?$SVNp;7ewJgXNI;1kkgI12X-@)N*%t*8_m72so_+4s$mtwtzO8!E#!q75FF-N9B`Ww^bYo zV?}7b#KiQ>VwDnt--U8E@lSXCTKCSjgpT~6abMu~qi76!M_*vrF}WgAcDoGP$<39@ z3Q3Qw%u?vrS`MyrJM;!!4wb$`V4Y8!L0zPq8c%r$lN`6n>WhKJ0hab1$ELs8=F^`o zqHGx_%s7O_L)F;5XRESI%D3B?z}K*2#QgoxmMhJhj=jxF#V0-TS2`(tpusw|JlU_~ ztn0{j(7gt@C(<%}zfSk!lgn>xWA30p>lks)o2oBh2)M4t`g8Q_zp{1)bZ35oa#wjr z=sq&KLs6GyGRulCd|yI$?|2L1%+QE+olep4yFUf&UI;;lbZ2|L6?B{DW26WqQnn?{ zDDxQ+teh}78j~CP9J{Nc?b8|cphDMa(1r_GFlHC_o!0$VPUhD&M-?`ch1rQE6^2<( zO+2v{fKQ7A&XVW~?w|T8!W%&`s}joU{^XOjkciyKR(V#}X((~#Lzsrudo^Efa zKH|e0h8HDj`X}aOByY`#j~^EAHOy(apeQk|f8xelho;t`ey+E(`n3GyZ+j$v9ZTH> z5Z604u(#6>C3dSXiWN_BDz7hdZpcn-Kz-efnf3qSRE{rmE*|9!SzEK~!FYz#f0WZF zJ23#&b`_b$6;9>YGUsqsVh_}qlQQeCe)l)oK5lnBS^D_U)VG{i`mpT!;qN zWhDM9w)ZR81H(&}H8x#0VIvv%Hv;;Na9nfYLorjD*nhwVQtr} zCl)ip`SVp3J8&rZ;)=tB5F>P$cL0>6)nSf!DoTe*To*0BUQUgG*JN;4cn}7^D{I{d zAK)X&6dl1welYHIpu=3DIW{>(y{{2Q;3B?1g{3*aDIq|}Fsc2n7+$TH^obflmX-RM zIkU~-`fbOqfS-iK;3Tdj&(P|x!3p9;2DH_q`)@9M2|p&G_@VZAv_{aIbByhMjnDHm z?xEa6adEkZ(;>#t?3vUrbL!;w**#RqKRTwsMNe^P!20(N@7%>so+aJXh#gfUQPKm4 zma!vJS!?}CMHu1zreQ(%YhqYR@DCRMxx*34N&wxjD{hzw61dQDQl5AX09{g&9@(MK z0LxDDl-6T)5Ap3GJ5f!8+v#->8Jy#jLA<@ zBMrR+i8I7zc>)_1lpjH^1qu~525rwR)P~p$-Ed2^?Up zS|Y2Xi)Nu^t?EYU4Hh(~dh5Yg<{S$a=euwWBV#udYa=~bI<&XiR8k9z(fwP(v+RxHU*%k$A31g z?2eHclu2p07hKd9?0fA$NL+(|C8RpJy_Zm(ge4+ATZ;|FQR|sI8+eq+F7Kz2^Pi_V z8zDtlqR5MI65I<7bk-nBvWiEa6`(R|^I~!Lx-f-V=Wuudt>Z$Kp)o>7y`tg}X&u9v zIpS4kkP~GTGKSg#sUY`-WkSg0c>n@gdErSvlx;L(SzZ?+Tc`$}=m0MVk0aHoU?peD zRBlFPD!|Lt@h|)VphQC)X>QuYY1 z+P^lo6x|7qUg)$nT$wScYkrCv72I@Bc1h2o$3GW`iLwLLXvJ4FRXx64CFdtDOMn4m6-m0Qk^ zD1##%QC37dqRh5Cq8xBF34boKNqBo@bd&J+1FYOrL00b7NLEg6$qKFdOEUmaSWPyj zbED9ZSrTp62+5Qv!Vd<~6ess0iI-ZCH8iF-)93$&xaGmt6&!yc6O<1^;}sG4j!WmM zCi~e=6pFL7p|5x=f7r&E1rc{62PWUD?N4`Qsz!4!+?7ea!bu&fJuStT*WXm{JLbz{ zm|0IMBPI$R#v7WQ1UY<$Oe2Rli&A%UA&v6Cm41z@h%!s|>)YI0G=7A`c&Cnn3>CJhAb$ znOk-ZuTQ=UR#-Qyvj~K6h=i5Ci+qMY1YpRtnW`r=(i{VIvZfLbGzk}EgN9RPE`qI- zneWhtL7ZawTMZWB2~ySe8G*5xxz9g%^nFp$r6-x5-wHlcd6QD~We{o&M!LYO%1{{J z-L+Dfi#2R2M0Bf?ck*t#;WNA-5RF?&d4p;}#g#@mp$Jx9#Il|EQ4a7s$tdO?u{PSI zaI9?!#d&^nE9HiZ9CjkKm8%im7-nx|&;_5t(B9=WihtG6uW<%F$iX>} zLxk?wr}Ce2La8NHC>jySjIUW6gVPpNmJ&-@1t*xcNU3q%wD*r?M-G|Np?9*)HnYIu znJ=LnW#lvepsB>nx6oDl z?E0I|F20#HfQJ{vkFrLyewHX-Tetpg*>_t`Ih-!jvI;sK`a4~NWH$1Mfo3S!AU^Kx z+@q2q4B|3goxu=p_=qU1pwp#RRAPnUmg880Sw1<7D!QogC2@BQsrC+sfe5F2!>|&T zV4U(?9erom7I{UG@{q7KIkz>VH^24RsS}2Wmc*fxoyw_u$u+*QBv?VSyE$Y^>SpCM zQO7re`%rELb{jJ|u6C8P)kAtn9zssNHbPR_uxTm#S$02{h_?I@?`C3rXmnePPX!l; z=O3~5gygDmE$5d+^$IVH?iIe@)+-#1c2>v&II`uB)>OW6E%prJ;Obo9mzza3@w#4nRSW8fT@)D zt}S6pq&s@(eLR;2vI&8ZDHcq}Hts{4m5^Hs8*=n*u zxHmQWk~3Rnqww4D<7Dm`&JXRA;Hr=IU1;sRr$Ga#vtunaJv-2h(;I&2G%rOTjB9{qGcbeC#QKk15 zi#VKvaOTiZp`{^!nMwMPa4zxS0zEP1-~sp;xIGV#B6cYsK%};3;uo43lDI^_zE*Yc zSVK#?_agk7yYI~G_Smcr{@!-HF#wNAr@yq?+C`dSfKc?3_n&LLOn_-W9wEukeynk{ zV7&PC;}E3%%(~#H^#Xtm>qWmOUct%}KvSx9HC6*ym{gBF6HAX9g87G427qr}~CybUmX=U{Z>{Kwk;q&0- zGIwyFg8ks?o!0=x}s;7_Q%g$|F^?b{za@D6GoY!(Zm3;Kvi;GfHRoo5ZZZU+lIW|y1#(GnjG7YTuiz!; zh$Vv|s76=3e(-B~D0OHGNxy}Lw5{`%xy1Rkg-~*&0S#hAjTxy>j$0ylSV8ge)bL_& zN1+_dTxFKFU)H^nJi1;S0-IhMf$DHZ6eafrQ*tvSDY*^Nl-!**N^TvD$P_0B!;hBI zQ%{hba=5=(4g$}2nqOAb>-s!|YaFqqaA-;9SW3dW6Xqs1(5M23u#euxjE72qyMHOy zWp{qfpVJjPWs(G-LBQvRdWc%42dNbiFmgP{z)kJN-khNZa#Sy8*O6fo=LjhY;9t46 zGSH!yC*0b|EvGF3pN#g-MuLp5a&KxJS`w`4!%-ARt zK{U40*9gJa1V*D>K*OLpXD?w=$@-N&yA^xCFG=nX z5>=nw(vZ}%~ zkU(P)c__MlHL-`j7HHqBgecS#&EX*shb+$#-}b%3iF*nL_U=H9T3l~f^%pW32&u}# zh;5Iq_pETG)t8-F|0Ty2asy`HTEY1+4g`3t zbf9e>2o+_APh8PR-v5|mgD)Z^&3PVpp}^hKvSgYZ1-P`*h=mu$W&W!ak?pSw=21bN z6<-tN77QBYx}yp^#RENH{;@EZ(Bn~AgOiTl)n@+ME08w`hgKrNgGKb6MC*a4Y5Kc{ z!K|xgR8=Y36UY$=TVra37mDRFdTl046)NxZ$*g6U4w?bJ2il_V_}}WdBgLQT)4^f| z(6ib2)!~wV7D^DP98!tP|4ZFE_^NfVkIBQp8bVg?DoIXy~zO}Kz zkX9amwDS1d5fok+cUlhvm)!yBWJb&xKQP|*_7(A+pHWWv;)eHOV#0jQi(jpI<#aZu zA9PBu^Z3x)U)|jJ#F1a3V}XEe#6YEiZ2&P*uq~<%?&Z8=RUKEa&C5yy++M!m*nlv! zC0E|}{=Qxm!D;?X7sWwq!V1Zk=bNh-%Fe^$E!p&VN2b@}QUE~1z`gv+vD+WLoroq5 zlrcUmlE^h$PPmFZBpuT5TM_pv8KuT^JFZ3H7(0(xkR2atZW%)Au=YeY6)m=&{GSC2 z=>Pt`JGcKIap+?F_%r%($EHmkvYgxdM&{bE7AcIN<*;FQkC$!KUBWltuPWUdauGOT zv%x~XEqys0&UlBK%%vmc{_%JM5~K|W;~WSSiyMf|ErVZQAa5wnh{SKlKAMNoAc8%w4c)TLL_HXf`ttnlYr1$xQAK9+8)PlA~~YUUJtK7JhMOITJsJAU7e*(i@=; z?36GoHSA`1kC7a0ZuygUr(}ol!cBLlR@@A#8Zo`5<-mrArfa3>`nFP0w`E7Yc$hsU z8L}z0zi#08S`6G07AbNd$BbO#+`1z(IOtCAR7u*bfTx3ra^X~oXk5CzTZWl%G$0;L zu3)60adfS&-QYw}Pz1!nC|pVe6=vhcUXF@s0#{;<_!Q{aA0Nwz7FbivP<@+}1A zt@Rj>76hl;!N?LE;gE>5t?+K$d830eY&enB zRa>W|UIWNO#aIf?6J(Hopka=JMMnRPVe!n&#$^ukGi06_-ZAuJr4nl)4?I>=%r1b3 z+5Ai!B<4>|&|;>P>w3L4l2`)R6>mfe*D$v!C+5<+`sX%SA@jt$*Up+*oa(_mBeM-d z-pDPIzOWgCccT+)dx3hf#@67`R!>Mi<;z^ah8kr@<2$*X@IYEd1_)40Vd-IHc?8(T zMDCw@8SpG1_3UQkN%Dvj%~{b-CjjyKfj5t`d(y$$B{ZSdX$zaaNX#qNVm%Mlmd1-P z=|gFyBztTxeABP#jo38QvusER&sU{KzEm##*6G5~4rgEnTSi?Ra$lqN6!s_!W{-}` zB6LJlLJiI4n%Km$+le!=Oda`c@`N=YX#PI?om9&P1k&ZC8hXppnY$7_0rUvn9e=f4 z6}edMfs5roR%KW9%5mV8nHo3cq>bExpeH&Ikj+UsY);mR4g__NJI(hC7t8Ko__UAp#UnD&D;xXhe0sgP^)PU&(ocH$YWcZ>c5 zm!j{S%fu2zS`1gx=q($`V9lbOF{*W~Q>zNYZpWs4%>#X&O|Ix-04vNVNwU8WU6+LY6u+aM+*KjD~G zRF>OR_l8!7*<4PV$``&#Cv=+ooUW%URKRQi0X1xF$MWuMKkJKa(&=R3uc(I{C6h|8 z$P#E6wu9U}uH~g{VdYiQOT>g&e#xmazoEByy$oG2FDc?Kn7qzktlsLNz<_0xWCPQ6 z=i`eX{kQyvFRJ9$5ZOEc(oyAPi4K=D&qaqz+MsULy#>xze{nw@0c_m4mrxii z_HU8eG&{FD@rLb=>n+B}gt&FPQ~I%BK24lXmX^M{BM~plkPdf6nvkVYz)$3~w#78f zZ203uH168+UAauEtVW}8vpEO%GqJ^?lU(|Wu2$+KS4UeFb&``fj|?r#H^I#a@K0CG zxilo9?b-)3!qxb}42d}8E&`F!v;R`mjld3Nbo7B6%~6a0Pj%7U;5(pryAwroRJYX* z)01jxVflyEC$+gnmkJ;-8t`gx4=VyjME2+iTQbe*ZJ+EdcqXF3&|; z03O|5WEWia<#<9w#4Y>35#7#q-BlmoGcR2g=tgi`b0)4}Lr@`b!`0ortU29=CE@7A zA-uL&NNu_+aI~Z=ITO>nF0P_sI%g-3z`tBj+|jXcRgf@T?t%Zj3yM!eNL>YvDB`b- zYB7WOZM=xo)8#9`{IM^EL3{=Tip3YpEEY&scZNsDc0A14U7{_qQUDK&oQDPYt(y-* z8!JV0saL3Zv1FCp=7?^5H757)PKCW7sFp7@NHFWPI0Wj+R62%0k07noO@eEESzKmB zcLsXy&mtFokz(+q$}A5g4$Z!;PD(99okS4~u!}j3wSSeC8PIo`*7hWv1n@TZQezU(cxR===IX=Iq{hCKl`3 zH}-BHXLXA;xr5@N}SU_ab`wxD>{ImA76QX-i-BKx;l${#b)$!R-cleoZBOr zh@~!x#kTi!CY_w0tn88e;gh|af01yOFLD;Zp)x0@;)$FJz@?ugoC|ZEv3UKv*T9}R z4Ta=)l6WSbJiv~X;|J!INX|}rptQ^01c%U&K=zfZhI!?uD5LA%C+4($($#6aWLHv= zc7MMj_Rb+&Gu? zxT`U~Y-Aeu%4wHRdcLXSTkjtz=v*aDP^uueTso2YtwQ)$UelAU8voBlA53cRvb|SY z-md@PgX@#`u1Qx)*MH_)?~u4N4Q=w(Rx~Z!@At7c9dRUN@hbYol7M zo+T}Skzo`d2`*X7*^PWeI8yMlc1WaHs`WN9;8V49sK$sET#|&+1;M;9Z}Xv@6p{;# zY4f>52faq+fGeh+(K<*J*R|lsk@Vwms_UR~ywP+|U6z^V^$zm)PG3leGx}vFil53+ zFw4f+y*-V=b3fgXV2%)D_RU$HgRBtNvo>ljCmFk{y`4oT@S5hW|DIqj6y0~G$6(sB zB9gJ&wpBGGX!wHqNQW8ul@w8E28dnq#iGr96izGYFThjA;Bg7bp`LM6sp#64l4~C26^tSD`5ehWOREC$j~{#&XQED z#3Hw#axfewgN{R}En+Pmgdm(oaL{Jn&C@yB+PeAaYp(X%?KKp;IVC~ z{>SU&18g}>lwuOEq}x}T+31<$J*}(nPJy=LQY)_?QxcJ!H1imndPaCK*ltc@r?nVEo+OO-B?>9 zfH8q3>9rC}6Nt<~IxXvf`5(7m&7*S7xE9^F(z52#p%O^0!W(07))=R7hfFV{>c-ET z@1l1K76y7AOlc5iT^Gbs;l7A^j{Q-{>P?%f!#6ArsQ-?1K*ZDqeA~nDZU2dbZxc_T zAw))cP$;)H$k_Fkx z>(Gthtg^%kbO_IRxP~>C9|E1-38W@HEMHlb`6k!O{q>5BQtB-?9UhmDiCVcw3!JK# zUrLOFo8?yk4~XO<=tfmmrMKLj8ZBB6bie+L*58<> z;BYvJY6?y?Rs<;r0xE)(gKU#{C%NYU^hmXbJOonCUgAb8YClF{N9Kc^lGS#)iq3=3 z$OMHbE_w#MZ>3-FLcK`f&6h!5rWrx>9#9Fd5U_;g4NifA%!aoFVt}sl*eXggmEX&< z(W?1t=Tp65+ez^3lm35T`?>)b@-F$Yz>fJ?W=}(MY<2$6Qh7=l zX}S;0Ht6lx2P>Il+bIdUuQ%T(BHAUHOR3$!Vr<9JddtR=;xaD}_u8b>?oCY|nzIl8 zO&i*IMlGbB(H53+L$*dv$Z_LK3xY=)eY+RvZWPUP$@3iw0_r`>6QqHKqZm9ruy`8W zg!NN!{I+*EC^m%8p>u9tv7MSJN?ZSSp0WKzIO*EPPs7YYu|$_ZW?b;WCCK zIHiul^B_Kv_a{x|hAh>>C=Xcw-a+P)<&3pYVx7mZ-wU#^^j$<+iJG|{1&`}*5r!r@ z!ne#+val4KO-M^>d~;j{luRfzrmF%dQaBz=)a!djEewRT(}zr0=gsjD!b(X($&_Ir z%nf_8IXlUy1h@c1Gk?}F@&ca~r(h}ZbJ6;>;Gz*EyQ4cde1@FTbgi?dT|{RAdTedn zu+*J-F2bz=f-_IpqG+^W0EE7)gxiSCwtfyxyB+v*jp%mi{Z?i+fv^!`F3o>`*o>lZDi|3Qh5fjRCnm7cP#X_8) zHuGS)OieKp*Ji=S@^S+{u}o*G58%2iQ9mZ(WzQ`LDvL;T+BHJpTh!crSBS<#cpsuv zj{_KE8^D&6c5lO@7&>>kvpk$;w3lx@;0Y{O*-(#BGk3AK1y=>xhl6hb8Kaqefm^bp zVZV6hEf2I_d5=e2f>9B%hd(c*U(0>iN?Z$t`^P(9oYO%)<0?;>d*xX*ElYp@oor{+ zV&}rH&e$$F6`OM^l1}sLg!5*uvmUL%8*(cCFfp(48)eS6kfmF=)LlEakS5f;lK1DvK}F(5u5}rA=57Onl~pL zYCbxIgku626Lf9^INTqNMZe$x=VA2$TAjx?{Vj%XW)7Tik*WS&uZ zsrpt^q-EuVjU5eVh_vqzqTCWIAF8i9rppW$YY@=QEMivxZ3#`_x^i7sn3Wp%OEVM?j$W|z+ z2?DJssRAh2ZI++|iImTBfW+ z&{nR{${P_=HAfPzCidV&2ZB%Hje;fk0(a}E0)R!b5bmhLlzL)#iIWQA1b0{wRs0F& zk$U60X}WW^Ll^44EM!{;sSmw1G8Yqgif|+GWDZm^OV+B8&a!F@1?*_ad5IlEe~m8-N&Su4fi+*3jvMA1qPA~LKBg4ND6`AQM52(PH1GU8)%1)t zD=|Z9DW%XuC5Mh~07Rft@tGcM3=Vq(28fdRvWWa-iVS)?>SAVpo;7yMARDSPyrs938x{D5g#3AL;EpZ-XRgFcUDR09D$QDGSsI!+V|ug&H$Nv zrf(x-Zqb5ID?{wt8g=QLzRcCR-p$Wv>?c2~3wHpK>!MEi6#K76*9jjuo~$^fT%&V6 z=&~sXU-ozidVnlZYNvJM+UwvPP6wGvPk-2;&u}mPu9K3_0?Zxz zm|79cy7QJYCu6j8G&^xHJG1`g%;KA!%IRg!u2IgmH>V~q&4};4z$qD0l=xQv#MK$e zJ{j?qL*wIzIwi$LiA!#cSL|?_Hzk~ByE-#-b1MFnQ}MjhT%T}$xY+sb6&WY80r)oA z09>{rDBn0Od4DY4Hm8r%E*r@$g`RT5O}(2ZCY-_d#hQyIWbEsbpStL=?1oc{k~d6s z8s_g#OhKQ5&UuN2FQD|TKT6-~VyO*O1}8dzm;dUriVo^UFc=-cw_fNtjTdGl$BGW% zrPE`n(WsI;wJcHinA3c#FsvM_`pCifL=|s^m&TFeE_HAa`hMX50q336_D9i*U=0x6 zZ44$JT6zDWH)M-|_@&%Cp!mkK>u+SZGJQe(Rywrg>Dp4#?xkr(0R80AokS5`cuKEf z3_oUHS~G_!r14zx1t-5fM_3shKv0@a8?*0~mO6%T9vc&}wF@8ryO#C*NMFT|)c9FP z=Y-kQ(X>OpK&ZeEKE10s9}2Z&E4EOH9@hpiEJS>UdTDBSz#}uGc8JW21AfWupr4YX zQzF3}J^PJiEX!=`a%Jv6MZ}5bcXH?R;ySuxOj1xYg7eZeBOp!ZP#%q?8ESM!C{#N( z6NhldCs88G4QyN4v^tl^YqRFss2_Mu>J{;t(rMAX2wpuoev+jsZmp{q!Mf)NfESD8EI#R=mrG~64%QORH{hQrDyP2SR^ z*{Qq>ieK87ZTT(?iv||F83iu|Vv*vQVEC*v$2!2)`So9g- zWJDlp8k4(|;%c7~Q*TWs=E5Oz_D|bx2Sgn9$zg-r?gnlPQgI~p(p204w_%@UgF%|M zVN>ZsE9}yogZm=21!Jsn)|Ja z`9SMKkGh+0*cV_p;)`dVA8X0eAD&3DfPPCb7-*!a1K zjM>4CrSG9tPvWP$k|$6^S$hISl)XNm8o2nL&n%*ZZwV7pf`}`4bbkB~^W$4ZRowMw zw@yPVIaF|6MKs*a^#955Gx=MbD6F;@M$WBswHuJ?#+olO-dg#kQ}%v|A=m|%)?7q~ zMwjn^`^i-mFABfLysS~8rNC}sh?1AJ{Le2AJHqDseYz~}!?kn`Ym!ehSvL^*Ap%8j zPyghI5pN#UM4`NN&|e5E0?%tAtnqDJ zxDKCUOfETd3T_YWv&g2Pdzr5ojiZ7V$$_CBjmPAG_OSHvagC)|$Rxy0jhi~5l0=<@ zF@)C*Wj!bQ(ljrkA7?4w65j<}1Ya%XvAwn@ytvaLzGuYKSC{0x0N1fJ{RYTkETZ86 z$-=O1rz?M32;|16@zT+WJW6=~8p3 z#SIk@knrF$6oW}Anrx4y<6c8-0s^K3#xywM6wBXg;7Hx)l%1f#93r0P*X%(MIe+OI zzMu=Qv6w<@!p}TEl(I%qKkuITUd(;)ELSn1oPyt;_)eN0@bkxw<7QhYoQ4S0nVSnL z{a@GI01#%N9mHSA_E`S30reZOYPHQO@{mx4TGuH_X|g1o&GfCH3X>UY4A?#E(8_UP zliu)=rT*5S-TaiXxCiD2s58uhtJh#YY430#C0%oAh6?y?LS@H5If1Et^wD#lf>6rp zBqf$wva@el@!!%eD7e51j&3iMD3dNudE3qvku8x6n%DjcIDBLQq#7>b(|kAq+6KTI z?7^$a8%*N0Zu8!t(g#nCAgsX#zxqD4=i7y@mLVxhoV0%C2k8|i^;pxM%HIy(p@3T0 zEIG)a-T91Ap=kJRK8|3JQShb&pk53`54M0+5KPhvXPxgw6M3j`73DEzJq<&Y5_`9% zM%U;8)~m3%ii#an!lQ`PCGupc61B)5Od>}WSn4BwE{#&-N@AisqUzz>H zxU3VVsqFW;Xe!qxj(&DClj_1p4ovGqXrp+-+4U3782H`XtRLoPEm*_~zfD&)?^@pQ z^Xj^<{tc#tbsHe`22)eF@$1d~*~28Y5X9Wesxf%Oit55Wf17y{U+q7bhP!4#{5odl z>c4w-@e~|N{c+pJEl+-%mV&8##ytz-_Y3ZBI=|k!AinL*Klb^gC00Qvh4%*^u>S3) z0T0mf6xhH?^k!1eM*+k{J z|K)6gxpU&2O$HMCYZA5`Td;U#*qAOMWm&*#%m&Gg%N{!)UmH30=Jvk5xEx;_sEySi zCU~--GCHN>)txpQaBNvMvpA?bC`dOEyM;t}ut;h2#oONVVN(fLGulL2&qZ0BY$ezI z^{+Rh9|7ygw4^dxl%}dQ((QWPjihpgb?XASE5~LVL?-&G*gM4;CXUdd1)tP@9}Ip5 z&+Nu2@XgeWVEr8&5>@ZHW^U=bopn?J!6ebF=^{AfswJUAxf&IYXiz~gn`kc90{QQ$ zdHtI30<~8BGx$*oBTU+EpkU^#JAzHaU`R{%B!fTP4m;1p`}B3Bl1%S#&cp8BmRZB zbI!2{SO+Dvg>2BHhG~SX`6q`dX69$qi%=A{c3(x$MNeE;vx>{0-N0O4Pu~n+?>$1I z<$im&Hz>Cfz(X7IM2m6$HZCJde{|q#q>JPL_yM0s4*+O~M;4EW z*;a;R+#zuzSLBdB5f;fXddGNu<{h0=nBGvaUcojG2878{m52G4TyAl$d zYr`Ct`7)YO%7!TX4!{TmOQv@v_03UnihJg|yHE3Qq8<&4RK(C?k)-$GV9QR(iDh)c zULHHkl8B+z?dbG8^vv@l7BIF&@f`I$4a_}x*eE^A`Hz^81M;?vta-En;_OC67=y=} z-#m+;^+7y}kdmYwTXOGDyQ^`-_d}!<_s(0duL%?>+N;yErmq*kL_t z(N{j&@2KfGklbNLa?K|W$(?rsZ7E;-oG7`^S{?WFU1OzV;;7uLZ!enn%#1ZjcMI_U z2YpRi3vamnXXjqL(-^f>T_&N>jK8uB?>)-t|k) zy73`G+sf7ed-&;Ufm|9_{N#D!5EhcUUFnPHY~7_YGoH;VhNW7 z?nt1t@8Qknwnkc4jtH@?yu+BE%()}dx^i8#b>+H%b>%jH?HuLpqO2=bW=~`vX4n88 z3r1k?>5W)Tme)lrnaIQw$o1k=ED?x43sE`&hFPJypbN+A-R+1G<$ZR27HkqM14numd`EAQ&=?^WcQ+P%MW``Ms$V<$bbl-qB;D z*(Ffp26-1Ah1|>&8#(CGqH#`bn7dx96MQQC;h*qs1QXSNm7D&;|Mv#8oBSlDdQ;To~AnHWxP|6 z5XJ}%>Z)cGPC1pqUIqcp5O%asrHw--F|0w6O!pMcNkbqL)~Kq*aN!y=M;76lnSz2j z!n!Ea??_ApAc(s%%zh83ri;;=xv)R&2JybP*bZw8v9VZy;#>;OM1+&WdD? zAeZp;;ImR;q`Mq3$SonOj|IdXhO;T7p+pgD1EYk(HVh*QwxSGIg_ci@R0zT*l^x)9 zIz|-2LNfdf52c9_#Toa`KM-23g$WGZ#s7x+2?#Z5>*c$VVvwcuLkAJJ~}g!?SU<97I&*{x{xVBJa|Ce6A)O9d`d?&IoReX4m-4}$+YAK%7zns^j= zxgq6`d7?EO_RJsEd;Xu&Tw=zEI%GYXa*@%m@r;D-ope&oUFhJZbmP|b5aSKCB#J>< z*Ti#Ak0?WIe$vSSQ)a1L_ebqod?+cO*C1o6pBwOWPX|X<$~yJ3AhT3ymW41awp?A^ z;Q6sxoG9~9@?hDBRJ32^Q{*Mc{G{zfo1Yxs`CvT5=|9Tplbsllm04erSzO^%jxBQz zXC?N)=wwo6{q^6?Sbb7{^7}oKvtp?)pdUd&?4bguUyuCccVnr}vDm=nxz3`jL`w#2 zQ1H*GEJ35TkPl541Y5#%r zK53sNdk#E`PU4RX*UFiLoVIH&&zO{zpDIKzx!Xm-T;uPDroIAymgW6t9$26J50*6A zZ7bU!hxN%pQbD#q^#Hr|w?8v-ZnbbeX@3zvx=Q9y#l;fKG?GgO4QcqfxB5U(nYQD= zkqaQGWEG>Ab2FscCy$IU?jtf@P@a}sl;je6N;cV~ycsDed2Mm=eZxQ5ZTC-cy$?3& z0HcR0XD!T}QHxA1%t*BJ2+JBJnJ<-(+Kj}2D`qQ&90+)-_2??rcCsY1`m3<&#rRaC zn_Ri*CRdGaa)qj!92s`f^5Ct>)30^wI!aHIXx8;vwB(Zd4a`dKe6nF3wU8^S>?<>H zb6a#vxzlk7kw;7S{;~Y3@ziJlT?WR8E(6`>6|N;O6#Rp7w173D6FH&QJzJrN@AB*Q z@--#cW-0tQvy(dMN8*o2S-BW3v#get(l>429tM4m(^ce=Is*R&C)$?c*zCo@24vTe93TL|Cdn+3llLUhsApAUpU2(LbD<^ z^)OdpjK5HtiF7k>^g#f&xM34->cJ3OZ*T}7Xfw8M2D};W9WGR4_E1%PParc2uW43U zHaI5ci2IN>n<|mXDFas!OSMK;WtIwqTkpDu38AZ+iI$U2CuCgt(v{p~ul3U5-r8gcbZ1la%|kq zu>E3GfxuFb*!jx2F57=>iq~l$)AvyMxo?u(Dm?y08|vzg`!+%L5f<1J!vW{~qep7E z?PEx_7SXgYY}3a}BF$aJZaMSLm$W20Y_4gZ|x=d?Q#Ad;Jl5|j6NNaZ@8 z(-G;8H2V2Aa!1k`12fU3iztcUro7jdzC|B&f(W~@8~U%KUnA8+#FuxaQ9eK-9tM6= z?I^-@YclK(&O#j+doGfHp2*$!0%I^mI0JUd>Jbf7gO+OPaZI$a95T606Y(47VnlB$pnGS}5RX#EPDVZdJrPU?5VLGESgPHjp8n{Oi3 ze{&bDv@_YC1k+)FoZK$leWC@J9}iCc_q5ar6kslo7GQ4AE;+L(@y6+i{qf|*@%YL? z@$rM4lGBS4e~u?lj>pFj?CBJs&m0?AR{YLsuH??(1(#>M`uX6xqvUD%8F1bGGY`lH z)bac0#~}7X@whOU50*i5xxSQ(+&sl`Vlxr~(mn$QziX}%g6uxnO~Mz> zDtVEgJ|Da8(MFT2*Wl4KDL>wp1nocUt&yCCECaZsr?ynIBxs+vRzU2ZjGEhc9}h+Q zl2LO@+1GLi8?W&C%pgzWJVx)l_EVhh1$74J5eDVmZvR)DoL!4lvxkx{D)Rq>JOoWv zL^<8ln|2nCio?+Iq#|T}_glM!&mNQl<@3gqRIQlPkNi(W7|yRJ<%!o6VMjJ3Xbj=x zG`K^BL=?6|gdNSvtq5>(|M1t+AqHRFTSaM)tJ1xGLInRjiUtG|CLoz59Hkgp#Q_sG zrl}Rz8M=^|1Ad>;(S#1hb%-HDZ+eY*mr69#J6M8~E44;gn1<6PcW~m;<0IJLzH7SX zu8dP;xS_(2ssm@iG>FpaZSX-Uy_LUq8;a=~f3S<){HxyKE(P4AJ{@SoEW^VawaXzL z-tj8jY4LvSZ{#N@D0*H=IrC6V!2z&pf9D}KWaP~ebJdIV%~Jy|xsFAnqc1tLRTk9U zjvw8o%Q)sRNwk8Y zmIj03jI^lzxd|)pF>sGZJT5C-|qZK?&k!iQn9* z_n-~RW&7KW+Oy&Sb!4_qH3KN%bl&E}Q`FdXl({-N@GVcZ|9 zOax=rZ63jJ?nVGrOWIX|cs;&Za*ftm2J;%~PSVDBXSeEXNx2{n%B14k3QX}mLHrSE z9n9kDZ!XRd*ITyb@IhMzV-0dZ2#ziJP?O%lGvBdhFX7|x;qAC+Q+Y@Hids1Pcgv|r z8|C;xB#kDyNn?f5=WuK@HvaGK;_GRNheLEF0x4%1(5` zp($gW?WZ}Ddgdor#!@$-efVjyfu}kBP)4_;M{@DB)REsNo#an?Bp1X|mw{mG73O;K*e3p*o+l_NvuQPfr**L6P?P@WzM4`od>fLqjBh3r}C>H;NFNQzY|Lx z6MYCk*U8od(<&-2B6=>NTFJ>y;M9;{_C47Ry^4}#V)-h16P#C;==q{kng4KXCYs9i zoscmJUY6$%b@qLAdU8lC-Y_padFj-=w)!4UvTmo-yk|&WTXDXF&g03t9cL6*kMHE9 zx|6YGA16_IPb^g~Oe{Ng$CIUx4^4f`iM6lEN>s~Ma_3&2(JwPU`JIbCxe)x7%LRo@ zGG6~`&HmGgu~~WEDQ(l?f&xSRvs+u17o0~fD1%Wfw~QOeH7~`F51L+b+D}E8pPX}P z%^)Uv#t%7VHE_^q6AXJh_vlB;O2vZjaEN-z5o8Tgb{e?R zV~uYkmR2N)CB2?OAb4a8~kbiOEMZe)3rT)MBpkAF~aKQ%V zGlkQ>%9>fGx6GP}x>Q@-DI#F|GwqImafVnK{8O4Pum$bvr1&!!N4tksMB4hgCJv^P z;6TNe|CaUxV{*CW20x$F_OUuFb0!9;_F{hK{iCHHcp7GlKT-`k;$eF=3pu?N5<>`0 z+yF=ZK4M+aO9i+iz6gXla3)eIVTK%fAdX{QV4bOVT@VnOXQj-U9qSvhEZ{NHzD9`3o{2T`c@5IgG>bS)pf}>>$HVYTZ8u+) zCU3blUuFhZRc7lWW@Q$K+}8+$&+t9LPHjoL=_EyTx{Q2JJ|x<$;so{|*l+@Ax1}ps zVhJ5kc1fS2)BBn=+{az~`-etL0X4c+NF7hQ-v4^4hD`Vyl+z&+A@5%MXCMQgm`yKb1{d z8t=b5)qNU%WD<+%Htz*o8cS#@Nd_ttllY)=A*MsD=@bTP`OcOP(p17B(;ca)d}ra^ zPEP4VnG}t+P99Ju!jLW9uaPxzgJy6yx#;!-GrMag z{a&@0yHitcW*}C$>F$&=K2hay1E42c_mS3Gs1vLlJey}L8EsxZry2)9JL z@(^oAr6rr@emqeK>f-#>TcT&6kpLn%X^M#0gsN0WP5px`@S_;FDCd(1q)b=ExJ5ai zKzey+LP#&&A*1|EVsPn6zigO%UTo8bh(K8kBZN zg5d#`Fny#y{sL;yG{A#C(zJ;zU~{8wCI6-?BdLZ0tUG61iz}%B)i%^%r!HywGwVwI zpiEC~^s(;fH}9ZW>naX}IHBz2x-kO?HRd*$5g3sYVqShj{#xT^JmW=IaPaE>Ze}qM zQFf5@F(b*ebvScG=z4FEP!_Px+&*=N!F2kT@JXnGV`|%yMJOB3aBdAd+lm@1%h=09 zxC#<)VwqNJtBY=b=V{E>r^-W-DAeyLhGPW+ti1!*E-#g4H8CI0It$*T{jO2A_A$MVX{Dq@4`t{CF zJ`zi&l#KMmd*fCl;bE z?uN|b8+xyd?oRNfKFz;KILj9~3!t3L$*FiEr{Zy^`6mhI!dz!8UjJ?)V3yNRSOlId z@k~5HiP0IEt#kS~Rr}V)2509qAGpwIdjgeox%sJ)L!EwK zo1VNb7B5-2JFxRW2^pm`GM(mwL!7n`ugECvn(0*hE~k0j zkj@Vk?v6RFTUTXQ?ag*lJHP9+?m<;t*P>)2KIT}_T&{X*EY*ara%Uu*jz7JW=oCX$ z-;$^AQY;r+p|FqThN~*zSqTKPHaDx|e_k5*0o9k0rjoUl;iYoN(Vc)PsPv_B^d>XX zb|+9Il^}+gJmXm=G1l9E_=K(zWYh?C^pZwxav#5D?(5N71A1Ts~`?I;-S7b=J{1osyAp zO2bdHH=oSGZpotETiytYDjOrpCG7EoqzZz~_Xnay&jK zvu|q@fvF{!6Nh&El4WP&g^6hO`D~6b&MXpMz-FR9oYK_sJlT@spB3|k*Je;tEkW}F zN;E||Hsh~v35S0ynUqYIRc=SMa>_&BwCp0j zLZ{Y&nQH})C6J04EDv(Qr?nH*Ih_vQX zkCz$fL7;>fYud25fr-?25??BvHci-%m@lcPn`}%?#3$6I2d&Pu2Iw-8r396FLGY{* zRC4AMt#+G753_54O zXlJdna){G#UQy!wjO42*XZuoo{Fj`PfklZg^iQ02a`NN$AD=+@*7x&)Z?&ZlWjM_X zA+@Z}gY5FpPV;(Twq9k4YV-!Lo6x)Y-+)@2qg~G+4q({9gR_5 zCy)HCjtbEJ<#i%@75vNV1T{IL}~le_Xd~1)|wd?$#O&x)Oia6)sU9dz;W36 zfqVV=Tu&&0&l0U}FBDs;B~GMkF@8hwSlTTPs!Fj2tpSvshtL*kT> z&ixRf&Dw*+L<4>EF-UEE^CszH`pz?ssx80!qh{WYVrF)3Ft3 zI0LgSF$Cdg&~WJy^4{T)Z&G&L8$|fSGn^qpH4mVDtXa)yd~J%Og9D>sl{kZLe=tap zA;BXmf39&|?I}CRs@qj$2S%!?BhY+UMl(D=F5CgVq^`EIKqi82V58co&I^SuVjf$j z^10F4wDHm05WG^jiAHEDK8fww;^AeUXZbKw~mtBXM;9vFRGlFB(zl1gl`uoJV0 zR^?)cC9p)8lIbt%Sv`tsImSjPPQy;VG@(JaI;Re|x zURId|sC7{@t@8@j$y=zl#uM#ka*&%4yzxvd*0pc!-9FChKKaQ_J(3$^sfSVA)+aXb zp8I3nQ5(0%Y2Kc2zI~T73bk^;Y%k?hyy!G16V6B5J0~v6NWK$~_X36+RFpWUf8xxH zhS=jRQKZbGmsukssZ&bE=x)~v+saIU;Avwnh8`L#0Vo~*%!S{oWn*j(n10?gw zp@DgoCjrpiI4${O2h8B;rf1QXnrXH)%p3K(CQ;>5Xz*IYM3qmkD3v0`uBWl2?-y&5 z%ULG)&}m(^)1l#?e7~K_z7~n@a?{Q({vMZsJu0M_GtRD`v7}*5W5>W<3*xUXh#wG# zXP#X@lTXW)n!gYJb3{jjsN$2kM2o1hqy5G^q-T+(YE9Va4}Q1l;Ay0@3gPJx%Vi9Q zrOBM;t*kfGwQ$l(MZd;F*V6g4jUI!JN6)^y<2%CzDaTKD zP+&>YwOij1rbUXJ69D0XOs%6W~= z#35v!+KW$e_AmUULquumyCk=ylw0fMe=Fl0=(_parpKt2*3K-B&MMU3&m_~zvgYrJ z*GTLQj@DVe(HYE3zWP?DNv&DVHkZzF+YQ96om4VFt(S{5*YQ>Z>)79es-=K4Gp6{m1$N#gHVQm{;p%d;A9liv?d z!@jkDQ!-Rbx)+D2f$V#BNtxaUNm7<|y;8X;C7DJg5_!-!9KFh_b2H;jAL-+?CX3Km z;-u9}4lY~G*8Su`RRHFNcZ{)#@&I^~xVM&>ouEj^jh_9;TG&kWh0N05Asa@)9!!p_Q081LHmY3D z3-0iZwMDo;>=KXbLrPUvwm0mHUu#66E~dgmTLLRJS#OX&i^x5}Qu)DS#|`gJ`u>^E zJu6a2)|Ig4EYoV&J1L2Go~YqRneCYZ+K27i1xr$cN; zq12A*NUcA*i}CG%JCmhES#n8JcI67JDq20v!jjuOA}+ z>Lb%8%qZJk^pzb4CK^_S4FXxzGJ6{Q>T+I&E*v!O<|@b+Fu@)0w!||c*i%dS8Q(fm zoZt-369%bbQ8T0Vs9jrpPqSXp-Q1scwf+_un97GWj%Cpciskecr=&xE)J2dV5c5Fbbm`2b2~pf+kt_>c z6zojA9svj?w9(RKliMj>8ep$0j;piA$6?r=7y9J0A9svyIo85Xmdz)w0o|DOT_!rc z&X4acc1q4EN@Vp<#4?h1g5VqMG@MnGc>naoKfXUKbq;(|dd9l;boQY!!Hgcs@5NF* zVzC(o&gy*Fip+MXF+r(lOpuj$PWCG3y|JM$DKV4E4P`z`>v|U@n>NH!vPD5VYU-+A zT^rj850uk;JKgWQAoZ{lTYc&DH#)MOOz!3|}ppzTVbx;!YbU!Ceu5b0!D z_r#H3wvp*a>C9>S->3g?xjQxbJT5Hje*GD(zcFSgaH$lwDrwA6=nCo0P#A@zHACqx z!KzR*0QJaC9hasY2Fh5S3#Nvcbc8H%x^lz7Jr z5ee)B{uGUK35c>3a_?~R9&d3%Y)*tw`dtJaxC$^p2+%t^gmQ$U@~>+CT2{VR{dCma zru!IyRxO1S1UpzFBqxkcQ8YnKUR%AdsTl&iTBPa-V`QtPd-{@`7q)zII88Af+Q8PO zXYG&=d*{&BuRHU)pG(Da`6CR>D?YV|1t#zvd^|D4a7vq%Tyr0}S=OXp5w9WJS|8C^ z?p9>nxou7wE0l?|Z!2GyzMzgeTOVzrnC(=KUvHFPv2(wk3O~+aBZ<^yLpys*>OSz5 z)X}@GSeLDa_a-&sokZI1`&cf;jvcWKV?)3+{->NQB?aT6IovTXKQ6*!Lr#1WQ<+Fz zvL2$Y-@*gwL!xTm4)&yMrxu%FI?zEIKMFf&JNDIbnAdaJ3xxu+cKBFUzsaVRt<<65 zfqM#w8*IH!betngFIk=*; zz1Dk3;TcL9iA4#ytfT9{9})zb_!oYLob&e19o4!Tu_&V3H(A>j5`JNJM+Cr#i7mcr zipkcgd~kQNiZO^EWhil)WaR_N2m@#rH*7j2;KFls}6I^Su|v)uuf zoLhIAr`^2(2;^)ZS}`uTJwVR%j9Et(ZG_%T;*%J0s_PjsD~>|WmOz}J7}g5QX&`rP zULG-K-yFWL^iNV0eeG@V(zOnY{^~n^``}UZlN7^bX1pFYI`q({O&yux1zWs?JopGy zAR%cbxiI_&?-bl`Ji~iE^s2mG4kF^qxKf4-(fU9m#-g?_*!RfpM`EI6K%|aKAhkpGICkl=UaE!kScMDf zR!a>sXNM@IR;oxhnf>w&pg02uE%^j-$bF64f(5*YxF2qsdjOl!$e08UV$<^}nCT&b zmjwJ!4lr?d`apS==i&qnES#l>Fq}9?3*3bqSit^>ik|(6X{e@`q1GmCTid zl+4W-%&iHUWyxIi{cX6sjSol@y#`3ohwoZYA6 z*ah8JKk-GP)aYn)9FWXwyKh<{SH<}MGFX%=r(aAkKSIK4sFN~a9w}~;(x3}Nj z^1--@XPIbPu$b}4pRTId#;S`0V)($X;mW?sFj0zH5MZr;qx z`bZdF?&sGi{@%rcb!NsmtzcpA_##$(CJaT6s~N#(Z8Nvgw76p!rwCF_z{$Odpc`qb z$P~{Dw)%S-z->OX69RDQz}bB6P&7kEjo^x@XS5FL%@kq_ejG_Z4yXR8+TFyD(D*ex zHbJvWR8q{%hBfTtx8?jqEebv_-96h?`+!@szTt(Wh zpxtrD($Y)h>C@%TF^Ph_j}BWU zC6Qz&8a_fe3?w5=Ut~XC4J=GDe3Z+%+&~i#i!DyE{H;D~ z3drG4gu*j;8~L_+_0;C8@J=TStnk#zo?QAtHaP<-*utm-hEvgUI$`UmNw2RLQJDWS z`Zc{%aD8waiq_u(zH2>@uMR}j!|IxVLk|DF>Q{{RG)>6?zpE^hjFR3tqI7U&^f3E0 zXm|s#`Bm3jucNP9i``V*O4`=WqIz_54>deqF|1S3YdlVt)Cvw^dW(8N!LC_O5@(3A zT1&8ud^dwQ(aOPp75y4l5od&^9t7WIQN3qORvZ}>V)D-+(-IR>a8+U&hMXl%aZg@p z*JxH7P}m*q_Xk>?xqzVr$Pwzx5$u-nPZfmY)8ZalP_}zr5LA@=$&rtVSQmhp)JDws z)G~vSceKMUOs+u<)ypYU{wO3S?boVBSudhy*l;L`9vO$$S01*tr8CdV#p~Z|T#?;+ z&o5lh1%b&Nll@@zW#gW!T2;45%fx{}M8l?zNi+UxGeR@z0%Xtb{MM4j%fw`y#+~s3 zkp_S8H!eqePPLer3+SO9fX}^W_Q%xalAe@e)1FJGIQ9;Q1xg*aCOGc|wkG)Kx5=zT zkN%0gjN~%(h%1iwDs~zM6eT{6C#OTbIXJ%eyxXJtg?9?+7yg{=D<=&u%QsF--VayH zIenb=hAT5Bb`*%A zB^JKWyOnwqtVeHxSBK>Ns8^1Y{7#S5xG5*~Vh@y059j9e0sF-skM?%qRb+ueh8b1+ zUfM$BQ#ct{7p=pjHr{EyEt_7Tt^#SA%B@@fw(P$xi-rS$ZrAM}a!YU|VS~PtPvGi^ z;kG=j?JNlI@5)5kj9RcwjZ5dfKX}2V z1WLn@9QLrLDF^uh4)eFHszy7l#_p$m0NolsW?Pplh1KYpy*pPe%BIiQdI^5Kn||DZ z$9TkD&)c+|hVrY0wrkFeU$VlD68y`wjlRZ|9OpM3yz}EoE0pTOd2fnU<2jA_3KjYq z5XXpP)j0a!!>gC&kC1Mc^(PBbE@S`Je;2*P$K zkS=rkKT>6NBzQ_M#N@`w5r&s^m4GA40iPoZfMk1R6uN7V6vyKX464e52bXXp*$|?q zjf^ocNSj;2YaNW>A_%Y3rZ16IluVaz$I%&07@g5>tz}W1E0CdC#$?uuP=_L|eGA$C z@J4p`My0`=`5N-sBnF|9I@ze89Zlw{-v(< z0^L8BC2l0{|0VJfgtVKf8q*rh=;-D!oY6$h$eLifStg3Z&@%+qDs78e5_<%uM+(Rc zXNZ{FjGB%b=n$}Ixpx-V7Ld=5ivs|kGBaWtlUJ?s6SsUP96C)E9A+RcSL-*f>SBq= zT34Fw1cyajwFb>W$b#O5Y-I4J2NzfqMXDC)q8=ei?%=_VFb2fa?;eA0V(x>akiAU+ z&^1M^A9#BLHjnSwJWtgKE0>;Q{JO3e(ZhZn-A(-V}Yo|&3n z)?lL_H;ZczeCSwT4Eec(TsdtcS}5dBK6|bEdDd(jeX)6hNybD&wdbMI5I-6Tp*@$e zEDJc4gg(W`+bee`j=z#nsa4o_B7##(oTLTGfo(gyDZi77oVIt=|t%&hks7G%EO;c zR~gx^?db>U8k7xYC8YZ#p$$dJ!CuwQ9FQR(&7?zp=S-(|=29amsSWRI zN`MEz9M`u=I*|D`;9!5(02W6{VhoJ~s2c)By$quGxpfO8+uE-H$1UWUbIaFEn@l&O z99;@Fh%N<9=u*&AbtxdQ9nfgH0Gw)xKtVsW**1hM8B_oVE^Whr%rh!4#rnoO{S@Nq@NdH2YE8)jNVeCNir*iUzdxh$ zYV{|hvvB?+h(l?Kyq#X0QF*QU<#Y6!50$5AYm0V6AtS)Q-AH|Gd87Rw6$5q`f}p=a z%;#a$>}&As+nR*1A-r@08FNGiA`A$LF(%T!AIP+M(x6im{_wn2xxxeWy0yYQK0%?d+z2};_?lyC9Ra?hVlZaRSfi3RDK|Fby*s#QB>#4ys7l>ockqx7uDjSp=G0pY&0A+ zSXxGCFTZkZKv1shp25_q@>CAQGPn-Y74qwNdB?A#)oJKv=~t(T8)1S$oKjaU?xgz@ z$T==R?7BzZE451YGW7UJHJw@kj9`q`S5dg=xr%*=i^I5S_XHMIi)K_Y(2}I)kF$3; za2Fo((0PfeN_Dm@K{MmvpP*1pHc7Y&B7%Z$*9stKJD{1p)uO)G4(J(aC9onHTAi43 z!x=e778ZQ8!%|*)k&MWjF*%Jo?%$^tq|baXBH|^wnT1BOWPWR*Yj*8Rw(R= z=GqEQA^_%SaBZG*56TdycwvJNCxZzSOL_RTWfF_7>suGx-ud8|P#C0O*HR;riPVsj#>nA&v-z;--M>!eUi59q?lslDI zmN^GTIYC*Dm<4*bpi9@DeCP*jQtJlIITM)d-qgWCgJxLul*5nh z9T`9O&A0CSSW5mX{>aM5wOzBG5X#CwUsbUKhmtR@I7}@92KMHQuU(V;HNW3>@w$Eu zY)L;@P}NQ?(9+tWv>$zUAQ+t>ZqkRA-TqKZpoWG-U-xs~A9jvhuN0jJ#kamPQ*s?Gw~zb;r`bUoKjvr?u#;R= z=wWpuIJpMkyByj{OIur9IG;dXIg#GQDz607ueLBHp(Hpb}a#+7`oKZL`Ui72!oqvXbY&I?D7XF6JQMgA0x16gk>B+$C#|o%5 zoW`#ol&^u8OCr%+pgydmK7v%->F7f(6HM;hZAj z+_8WOpxl@lFjxr3Nzs5o}k znM`X~@(YCHU=7x?XFC*F){t0aeMn*1v?#ajO6lerVlg@d@|1CtrG~`|*>v`ZFSYGk zC<}2BsAgr7GkDVL)$8ARf7oBP2p-b?CCy|brJD0bv{LPXl}f3m|JLxG%If6MnYX1^ zs$nXNH9H)Bd`N46#g5Txb`o|^Ajt%8d(`f?J<_h>ns)6TyQJN`b#Jws z5E_#6r;waK26pYE%iJzx&wYXHxu0beT(UnXH}aKs^HKSv$Mp6yAhf({NYJ5C z_tp$<=cp>|P&0aNmq!kU1T}m%I4 z#zev&>wgxDyt|68XluCU#j@1p%152)zhPLDcPz1X!W~pqL_oVE5+tr+#5V5Iuz8~9 zPmOM7TS#a@by+%8>X?W|?5`0Xgj~hqLY*ZtrcnL`fxMC74iku6J*} z(0s~&RZHnh`;$`^v7Azqqjj|5~(}jWF z8;XU8*Pe3i8zrocsE~(qC*0w1qfq zbpXK>S#xX|C3bW&`O7N428Ua$lQ#Amib`_ZeFOYD{?qD)SzsQ%h&QR#)?>Rh4hxvI zh>V+lfC{fg5kdYp0*~!ii$LBi&VMw)LQeuR81q@Zc8LgGu19+ zt`HvU<9F+(~gKALH>aP)LRWnfZH#KaXW&_J%nHxVi4 z&p~@VrLF3xiYcjxnLoT?EK_1!y*G3VXU7bF68PRYGYZq4Tj6K{=IRNJ3Yu}VXvmlM z*3}L)8|%e&tetxHY&>z#7ITC<0#!EToYCt#zW+I`M+w zB7Amg)AJXaFUdI!aw|2Vz}358Y5SV{0ndgyx(^Mix+bCEm%R(_zJEx=W^4j}SbdfyOqd5402*fKoFnUWKfJ0Tj{duD2lc%bls6KII*TCqOs{dKe+fwhm-W~rZMlk6Qls0VP zs+F#;_kZcI6^#~u_#E3X;tx{gkbnep*VSLSHH>2Zhlouv|yes&1o zeeZd;4iYVSr>~K9XgV&cNLr{Rk%Ps6>ttQ4+Cp6MYp@Q)qwz8Ys5$>1;#%*Tm+F|J zW)|>`@Da%+NUcJNL<~M@asXYx&ukquL}#YtSX8MMOQ{|O?HF3fu3ROfV`u`+ zNKhRgCBX?(b4GSf!aZW^gy1AIC}hW@nwByx49-8h_HW5k@Ua8(a3vAF+xbm~`-Hb7 z*m9Jz=Mr*LBt&!%B^Z~GdyRC|;9~Dk6kSj4MNZMF$g~7%VcOSs6#OZrmApZCogM2= z8ZzO&_P@UzS3>c>CBXvP*EgrYJ#>FU?w#}q_niIH56-x>tYOxWkVE+ImqSA5Q5pCz zCR040@$t2IK^C?wV_8HR!Kj}eKpP+W`Gqm&q+uOlEn3v3ygHlp4TZ&^%hJE1V2O_0 z7J=DP-nPEL6@H?=2JCjnm-)u%s|9%YLCV@Jy)(HJ&!4_K-WsC`SaizAOKIl4#h+5HgynlL9Xq#Ai4&) zH87vym+!2dc^cCaL2uTPH9x;Gx@F1)4j?x@e50pmL@zCzPUuX?#cbUT)T`9;EUh~O zoBO5(!pQSBz_bZkiifc|aZ9kj7{)3%ZfxESTr^Zf(KtuaH*lbjNL?>x*9lO7in0j$ zm>oEPJu)>~5k(6`UNC}|n9IVq#VIgobQ4wCo1;7ClsF(AEmmU3go;ZnzN@$nvDxLn z!2#YS@-rfV5iljKP><{{kP5uyaYpl2ArA~ zzP74!BBC311Dv+U_^B~7Ig1q;rUzd1$?BWwCPruuEyW7t8bUkzxP{D=gO<9jF|U5S zfdH(sY5Bo$QX>k3OcX+~9<-m-ah?5n8=jM~V?X+c?*mCpuvG!%Sm_)&GPbR@-KDU& z#tE&rmazems1y@A-cBqf( zXgNO4apYgihQlrcz+;RQ_K$KMNMy0!z}W6T<5S;`s-1BA`5T$V+Izbw(ccdlaPM;l zT=QlSaEloCVDpW&29&}{I_YLP@@NQFrY(^Gk6Bn;ZG4eqsMO3gkqN3Q=M&oj9K7 zo4&?k55m(#4}F@uXnr~$FtQfJhxSK4m7g9{@xxS4$04jJ-L!A^X8K|2NSLb-33Js* zn2VrA?F5gUl6+=psGr=q*_#&+q*vc`*BYrn zu>pE$@@I<_gEw45gGU7#%m4(~^7YmwrwXARk!38iJ&y_UExhoDJ|g6{;i-xrzVu)6 zl|JHi^Uamx9 z3o8)V{PvtpE*3>GMfrj)NBy%$@Pae&ATN23A@~d732i$k5~)aSIp;q9sRZ!W-G?a- z0Im4BcmAxn37h4RuP`=*w;zm#QTG@hdnkPm??X*DJo!!dWQa%TsKuN#m-_7Z^3J!NO3ZLX1h@pJ2<~#85Ab(`u{svsQcVV+tT{PYM z?TzHkjx%VZO~w-zofmT8+4gi*zhy068n`D#Ttaik>7p?&@*6i*k;9_{gFrm84}nwPo@{1Bd$>DvikKH0wz0N-YD+gfq~fs*UG-ZLk*^soa9z!*ANT<9V3&F8HI`U=_`$ zALBvR33&2B2nq8Q+f(wGsWgTw^k~pZP{0Y7^JcRq^9n~n;o^RnCNcSP##x0dHVlYG z5q(UQk{eE1i8plU0W~Awd^Ww{NK0X zkwxx3;Y0xf$*;zS|q@q%{dgvJn!`qOyK7F^ z?^ST~?t-EY&dPg(a_`9q+ImCK^BoF?LX-JRXXV_W+?%X?ec`0L4Tjk zwJo8%OrO8D-5~5+aOB#cmN)R%OA=}xog0)^{FGE)>a{JOYPZ)J;)%vCber#Ppv?;M zv01^)y|Wv3WCT$bU;eNd*LljSOW&}YcbaRR*P$T)*Ju`OQ1E8Apw^fAw4V_i*$|H% z#~*sCv)E>QcR|N?*d6`Gs>f%#b~E;~(O%wrQ`HlDj{^y`zTKzON5DfZk3Q;blN%L0 zkB`S4gp#IR)u(>XvjTlNM&Cf%$dtR2EAM9F=-w5M`{~q;?`#-K@MXV%GhKpj5CY53 z_;^K^;HPfhJ@oVs1UoJdd+c&f{;Jws^A01A0h|nm3I91=A*d!c#Ehxf<&xyv?uX9M zJp|B_Ggn@;VF2yO4P8*D3m&0oBrlirm^>HwZ#*31GIqcDBUSbEy^A9GFlEIRriTl8 zCsT(&aF3wQwt&6{H6+hx+?&_+XRs!t0qyhX>i2#H^|n-B_8WqA8o#pyTN7L|n4C7KP$@7=8zF3t}Y z`V_mGDASSeM9bD&y!Ck10F8M&qHR51HE18!+^~&VYa4DJ60)&7J*MHd*9XK_n0FN) z|LD}SQ_`-{zJISq=B1^w6S9^(C&K;)PgL1N-8lsu=+6t!?onsRZMCq~0MLQ9D&RL( zSue&_Oyw8AZ8^a4ehZhey>OXE09$JbM5dKRtzrOf@W_F6S6g`+Kai)fVIzCW-oBSw zyT~2GgWfVb3)0b$4|3Q45tEK?=d+>sZr*ZMk$2}*eKA?hh?rB;FXSt5l7FM+MaFyF z#?nqf?c?!UlODr00ko~!6{d%a@G3tVppNl3l`AQ0puOfKcSpMv%wigk(ByeBHhe97=D(-vzPq{M)-ES@k<(-`EqVAVPfB-` z#VL4JxI)U0)o3(b-d(uv#y*;KkfPki&;<5}v#vDC0XiZm=7`XjX{4dGj2_xxgubpF zOa`LkC=xDOX^cO9Z0gxS$)NYJ5z##}^$giVI37IX%ZF%2TRnnj&w?YE`>Il+#jwGt zn2I0XvQ(atd_`huqm8jJFgR87p$|xw?XUU+(@nr=V@$({q;rmJ3k;H<)IhRrAG}6k zHBHr;0FGHiKo#vtErSOI0?Bw#0NPo2>zpj<>$D0#RyfhdIy~NZj;~-k*pgw|UWxQ0 zToZs|(yovmv(brWm0YZk~-;6dcA8vtNv-}h-7ysg_R*BiFOt1-?sd0-{?ebr-l{OAs!{|n4j10hA zjx{m-5V1;smo1=on_?nIzRXJ%t-=cKT3m)x3&?M<6|Ay#r`E{q(5?jS zEOX|baLJsxyd!-%XYP{wu5Q?moxAI?iT9hk3oZel2l6iO3;b0h@AeeD3Dz!rpq?|g zN8gp;YR|0RKxmuVHB`*OXR*y(`g1F5S}VhYYT`cznnOL5eAGj!eq(mcjvzv#!fxkuerSdYZK;f-$%b99vFD4VD0Mhv4}oS{N%vJ zP)nGo%AmY-?)V{wcx;3jHnNMMP#+oLL)c|*^Ah`A<^b{D3b0rB%r7q#l?H{x-%8ud zB|LJFs-Y0r_RQ*^!42E5(~f)F0ZkV9mtW#hMFCm43&>uzV&%RyuyUUSWaXAV9FUc} zaN38JTkOlq4GPH0feZUd0kqk~YT2=JZwf?Z>t=wJ3miZL`2=U!Hh@N~9F}nlVn>|< zD`y=Z#|T@y{3NeG<4Xd2z7GfTd|TeTx$4jB$W?EC^77C;xlj0JeEha=Ct74%v??e+ z$^HEw1XOL?4z3g}xmtkT20~}h&$4s^mA{90{Y`f8o3Xm*(T(Irq<{PLa`Kb&KAr5I zd4~D4JQTEy|4~wFfWM_;ZT5}cDvU4~KCr*cu7n4mxDyfU8q7vK&frt1V|tKaq|}Xd zk4-N83WAFnc)2OIocBd)P&H;&THd3WoQf!yUBAGH|X#2`a&+OHE>X+aiS z__-&J&)AASGPHzj86ydYa*75lV2q1FoH}3#Q*Z&5tu$M$&ASfYhHB}?dKDM6TgEZ% zcLG%%3ZQERd_zzy5PSpGr8+L%tSBaB>I{!I+%*l4VA783p$j%-ym5(irBGT(W&r4j)4(% zrzD)*217MOOun;uSn8%D z;>icvh0ecZ{erG5JrBD=r$&WNEe(n2j^k1~wIZ|j()`6cap&y9Ii0m9?dIb* zo+KR=BcqPu3JxcT8-IaA)P7DLY)2pSA=K&8pkcvG26&@h_=rDIb3) zkGpofc%+?EzRIBt?Nu(NQ_*pwsBJUba>Lr~yij!R^Racf)0CvT`$O<^lJJYFiqUX5 z_1lQ^P4$Pqnc0t8Y;U60VpqFmBbM~Wsjru{Ubwm-|GC7G-}hdF25qiflOb++ct^H_ zuZ$`_{9NHPxY^d2)l$LC-X+H`*F1-zaSZ-YBW=lTiHD zf$^v7KmSdSr=uFrXumLSqWFc;liuEpc9`v+^6AOUwxug~SC5aTAN+Hz0R{%kHrY5C zr+#&=tnHnXU(ehUGN?a#q}20*TXx4@I_4hQK59?Li)VwcYa2E;H+_8JuCC)MSB=XZ z6`pnK9d~HS`zt+l?$EvOm*#Z+!z26hzjL;Hmn6IWIrXUNcvNEJtHBsHJx)LSdAID` zU)O5mxSKZMKKn{lVp?NNpxz`uFQPg z--#KChoMy}SJ5wXZhhamHGAi2x{5A&_}4D+clAfp^_!2BrN8|K4_^E_Ihc*dE#E|j zpZ^4ZYdIbnnm(Kxp~Vv!TAk*+b1_=2J}z4Plpd$o1#H`|)h}clA-|~_p)NCjv3T7- zuF9Ba*|rb#IQ?e8w*7~{wrw1G8;r$_+aniNp=0-Mf5P+DwnX|k=(Q(2pYTT>!b2A9 zOXsdgq~9m33Lmpop;P6qWvwUv<_P7kDWwr%?kZRJ^ooIxVO*AMJ5jOLx1;oqxB(OI z>;+6besUHKH1DqX#r2pi8&`aV+2YeMnBejkm*Z&_t4~&}Wh?DHx?#BK6z1lLxqGU| z(?}!RZPi-KOJOfwnJw*w9NbaG$N5_EMHu-HFgH67#oUw&gn44oaCdH<`^~b}C5?CF zKbM3?w)}mt7eh{EwzbZ@f!eRMm}a15d#_xUcgjzDqYLVGW>#ZV`fKaH;pb^?8`>Ao zmrYKuh&5|j-dXV&xkkyKuTS^JvU*G!GKsj8G}?3wqwTztC%u^A(f(R^<+4^wD`I?1 zI=>Jv589IwUQpv7joV1mH^Xbd=syTQvs1^5-*SV}%ShvBZs6&u$0jISQ%rNzr}dXF zU)Fk(26&Vh?QEOP#{<~_co`x&#D&R4o8abAUueICXPsF4v{P7td33MjmXY?Y(UJJXNw(w+@d#J?8{9H#;kAZ_GWwen21c z9po_7f2%qLcj7fMHnYz7jA+r|Z~$hiQLVmcx);B$boG1}E;j413V*eG55CtlIJ&gX zsvm4&Vuz(38_GY_!mzG9#zPsGXiKrrxRedmLoCIHrO_}Vd8s_sz;oHFUlLdFnySVv zY^0dWFnSgS6AKr&tTeFfESqiNf_?EbZch-VHjY3%bG^zHxu-NXIu?I7v(_YDN1>e#qr2uusU5-;@+*7+Ir{pD9E=|h)xy!Mq z$gu-jl8vL*)_lLY=6GqtRl#BN;=;zoh0geSSxrM}L(kx_4||4fdoH}TcUbJfVX^Cn zMOT$&xHgY=J=eSTRA}t9>!Q;(0tyXA8cb5|buOtwn-c?khkpq@=Jy4jo9KCVbY$B4 zjM_ornVw%bf~$un)$Z+*>ABSroPKRm?RLgk|gwedlj&6TSXYB#lI zdj<`QOk0^zTYWU!v-i5ls-leA^l!2~^RJ6cTYv?z-|9_-aP z^FjLPd~3^BlUw?9ADL8DnB)F-blpuMeY>akotDnW7w>Yn%fhKX@m~+BbzWTy4AKX# zc0$aD3DGk~rPN09@wi?w^EN-R_te&hald=tcivx2?~_tIuCkcl=UG0+YQMRa9V|i} zJbH02Ie+D;5Bgo0@^&FFXLt`eFzJ|j>i{v(SAETIQvXD#dsFMbN<>$*sOfAVAbCrH;8zmiV9co_2)jfR)Qz2quR z$Gqs_R7+1dtY6*~LQPq>$>HiwZ(HKZZoW5xU%Uw)m(j-;;bP+j33%Xzg=sY(PkI;g zT0YnNRb=a8UT+)~dDYv?MRQr8xWZ+vr;`~h*g-g~;prdtE^e+Y$$;@ayknF@K#Agr z-l4#Q_?Ph1L(%=i&+i#cfGjcXN$p|x@^RHVaf~OOoUJu|MPF^%T)+>X&=0@#8m!T~ zVzP5bh_);)>=-K+?5uh7lLfxOC2!85v)Q@X+Es7LtFm*){r*RQ{S}SV}KFiV8`92 zxg<7ttt23(I6^kFeBAe&*ey~9QN$^vyQ>g}h3&?vD)1j=1eAj6g3KsY3e-oZTft=-)e^3DUgyOVw0 z-2|G==+{_8g>Kysm-r@MCsa*aLK6t&y~J**pEN1TCi_E0wntxlv;QS+47nYOm}R+kJE_I2w1qZpoE}Nx3;L z$M9mu;B5Ejz+>*@N>@_ubeE&ki#O&3jrIKA5&TVwv!egRoR*0>7qUISa|AyX5xgRz z@P`r0`?~%*+I66J?OniQX|e0mqN{$F;i~OjJ3K6Q{qWdn!+W~N#3olRN={jnl$+^t zj4pBvALBj>JocNU+%({`IoZVj{|4}fNzT2D&%k4MKQxZhV1^c<0-wu0vxmFkPtLHs z`PuGGm%AHY!N+;o?)uBz4V#@|GfLJroJb9sF}AGX+tiRl*<}rlsiB7+U)JEl-yUDv zus<~<{JmOcD2=WOhT!|Nm*`GnzKO5V82JccJ1b~)P< z1_)byat~0)!HlGCz2A3U`Mz^7BZHJezH0LO&f8yXNUJ8qa?Nb_bqrd%XZ4+y#YpdF zKK{k^sh`dt)N2jF5WykfJ)}D-PeqOY@e77diG*rI%#4o-9IOWJgUzl=pD`*S078rB zJFoP5k~)SXw@bUNdeU!Ko;*dbQA*8|)AazBUQ1-ABcxxAJhpBVfn$IOOZZe+-JVv& z=FtL@O>kwzQq%9sNNTxe4fn!Xe9WYepYr3_vhy9)P0f!xObA(c2|&ogao|(bAvL=q|)5>2&NYTIwV~5#KEuZU%W&k8~fK zz<}7XziRD7-DB8zqxSVb&>f==-6W1_0lr$}pr*#4^Vly-B^+G1B?dq7GI-QSLX)aC zYT&B91XqcF9xA~YF8OjWL$I{pX;v9>tAIgJa@KV;YwS$80R^4W14_xnzKVhu};#tUQ4jsZ!5D zYDiwlMO4<#UyuuVv50XgM5YJSc$F3rH@RGKs~FG(ezFLiB|b%to!N)wW8p1p@eQkW z4aC|Y|LYpDW|kZR24VKjAIbTmOO|NiVYTS#%N0%m@W7!)FJyVGTiP)bYqZGwA~XcC zO~l$y1i?PIR5dcO%^ITY)7UEv(;f8XFUo0?XHC9)C4t`pZNKTdZ5Rj17TYFXYZ{=G6= zFcFeEI>0svqzX_)cGHaCxJ9wxYX$xq2Zw@*V05RItoXTEH5klrkcLL)$x6&m6 zT6Vz#9{}tX=qX=@P&XRGj2BlYOH6UBlyS2;BVGAD+Wx=={zZ?Q`eA<)5>b1pJ;Fe zhkc(A_F+QkjHzWcA>}n4o@n@GaM*yd{;qaOwQm7zrAJr2oZ&h#+Vzj#wQIv-r;Uv5 zKJvCdwhee|%h;SBC+3{V_N)iUiU{so;v96v#2inyXEk75c<^WAiyERpv-J!c*E4j+ z#Il-wn`=HUZ8+x)+twpAZUR8qr4Um_S8dSpwqFYc0>Zz@RYRFpYyPNLNZh!xhD(6d ze&%x5U;aeH(G>}~SGwHs`1mG1CcE699|7D8cTbH@&K(x+9*X}`$Gaaol8~F=a!O9^qc1lc1@|clmBE~Jt-giDKP{v9+##}Et|AWAlzVw&$?03H-=q%#mq+3tlq@{hQKwYcP?vUgZ z7Cj%ZMW=%2v{=J9MZ6K;EN`2wo|G{Tzn{!Ku~|f2Jsj$yKbrY79X5eb`UKA>F2cy4 zDN3!bz8$U3g&FjgEmuSqe@)Hay|}Qt0~3L*am9t{A^2DbY2`_JXcV`l5bl|e!RrA? zTfd5ITP*WeIHSI}U1K2AiZr16s60jTOaf~wJp(k0JBTEZyr8Ahsqpg`cBHwBBQAtc zbY|p$Bq$>3ugxh>WUzUKs4}XA~ zcLzM`G&oQM*6=SoH$!-(Av>EFj}{+aL_~MVipMd0(ZseG4PolHJ-bMZMffbS@n+3u z*f>SQYj+br)pQ~oVk|RIJM0bAAQ3y%=W7T?g$Zs_pzTxcof2{f=&epZ1;MDVL!?fT z^7NX!S^VXC(Go3SxRdy6BE(~TcXuy*=A14Kdok|0MkJTXf+w#cEaHKI9P<8iB@B5 z;{A917_=I#%>GfukJhxBifIcE+75`3DOXIDrJ#pMVODoy?&~Px?mJs~(&&wQt}NXe zO=EsATeq$JX(MUl3&kjq{$a<97mD7cpp{g9nFM^o(Itf>iK%sG%IrxKv=EOXEI5t- zNMRrI_4GEcvBzhcJHl1uK49X^0(}}!Vdme)+>qW7%5*?5LIX89{xl0l*l)IW<<{Hr z%0?#*yt1Z9k-W4dn6t$I1BIlfv_-4o)azotl5MrZV=%Yn%a!0Wr|Zh@++yC~Zkm{_ z8k3<9dc|abRwEIZJ&{4QT0CgAtGMZOU*AY1kXVG~s6OUYW5`nsnElXdPcf}V%#<$| zqB@C`B4$&dj1E7BX*D&MO+zJ%blI##2dzd@Y(KOnXf+*E+u$|UV)M$U^^T;M)%Bp& z(k=T$`xfc?e`1KNeB8GGjZWk_r4yOPoUob{ikS!Oo=FN1OQTQi@1F7imV7mny~mkY z^M@f8cjc2p(&S?clZciAWczAaYadNWlI4r+=u|(C)JMb z6nk)V?E2BsRogRMS7o^d-hyqIT#lQv-5o)$ElW;WmXv#k%h9{oae20TOV{Mef|s2g z);OEMr?qB#UUme(7ZLnMiE{#u&rQr}&RBnF^4#e4BVwnGh)#PU!}ZZ<*Y4i6H$g!O za_#vH*Plka{($_oESKZ#1 z__grDuRn5f{@VQ^?v63Z&F}UK*?3b~!`T%{&420@a_GjghF$ph3IH8Go=FXjD_Pd? zq%&;e>SYa&JHz5uuWk4;H6#uM+X4LTapbS{%yg%>f1;tHAmNeqH*o&igo5pr>E9=2 zdpc%dlP&yzG(EcI(QxO4MC7ks7u|9{%4qbO=qbU++rynD_&5kM$mJq`tzI%rB7Y6c zip*aFFwpYX{y`uC`D-Tt0JQuy`Y7|)@R9P@T5|<9q5L(VGLgSV072xhg>G(0&L>pT zCF{}NTFbtk@ zi?+s_DJ)*b<*Ap?l~r$LB*pFFE0Xl3)`sBs{j1a(Fd|7d6(-bWmI5|%vc#!ackvGc zHO_3~XaR+hsV7fC(>W#zXgDjko94oi*9^L zd)N$O)S?^TR!=JGr|7h6w1*YZsjS>#Fq+UVrbB^T_Z?-j;a^f(87wlq1jNBK*oI3+ zJWqk|y3rb)R`oyPs;s*E=~NdewKEZt_Ji0~U?G3q9on>OrRR^pUZtTE-giwqG&Q9$ z>bj^I(^48Qy)JrNIW9W+KK~Wa*%GWlY8enkfX;$Rl#(xLR|{1VAk+4;kp@fhLH z*+Mjt3?&DzIT$Ma+|`C;C?g%Ai#`@Az8vR}I^an1IP*ZlIy6XBo!`D)`xr<>IF59@In0AG2f79Lk@ z>M_VlF^IE%=L3G_NOyF+C>5c;NJ31SQ4M9A_<4z?b{I~>&?dj!1mPgRR@o*NV-Bb6 z6k@Q+89msba2Az4(10%+sijeQV-$4^uaC5%W@q@8=!I>sC9+A0VWw}3 z1Da%fAj2@eXzqiPWo;o^8qi9bA-<09`+SX#-ISTqZPeY1!C%|;9<_1d))(=EeL z2{DIMB=dtoB-D?*;-Rz5SIJ=xq6)eVuV8|nutvd$$0hlrx22K!VXT2mID}-^0}HzT z6YHIC8mvOdK-n_YO3i=0#c=ANn5e$Xp-~^nI9!iIDkLKTGH5rD-~P|?TD`q9*jDo>7DLt;?Xnn7IK>JP3|!zgrlc7T2-%%Fx)fo*ip1$A3Ng)xNbbGHp#p?N99gz3d zF0(o1@dWG(Tw4)zpst08w?Q(AtpZ7GF+D>zW|cJ@1#(M8xx>ih=2@$q>3duaS3!-b zNh7-il1BC$mRd7`5X0b}ktA$vxuvKu9yLe++|rT1)^9jNl`lV8uxh3RgX7{^R@r=2 z?Ri#OejgvxnG)6JrhYI8H_B-*@~hT_-XplS5I%lkP3RjFcPt)BD9^7lvPcLpW$fDr zeuebAz`Po(6TL)}#i#MsQ=?XEYEZJb8t_tF8WC742InD*jz+)v{`xWIGQ&h3t9)17 zxR}5$S#JVo5SD2oPBGW%x9eOOY#yt|NDo;|kFlWTnnxsHw#(l;GuW{ZjZ-%W?e|3n zt1+AKu~Q~K?xTbggDTu$yd-qY$;n5H9Jo1eBksJyMpK6mYw_|q5mK2P*p zV%npcTf0XwbgOX+mk`E2F)QAM_S;#soV1bTbHv{u<&hafnvzcIhG7Dg9N}i1&VkM* zc?`pFDD7$OA(o{_o|manzUZxdi9~O6KgDeVz&GtSeiGPHjsqB^#gD1GurM+0_heh^ zVt$jb;%wJ(xmn}0POa)8ijF*fzO24z47q&xmz_IDi~epAVbOUQQZT*nTxjLAT*=w= zQrwsbV!I^=Zj@YWe1l^$j@M`Vo$W@ZXoo1kd2cf0}aaO)-qrO(Mzir*Y?0B^Hy^>k2L|T-aU)z?{B9R-_ z9NBUkF{sQNpqxWK$A~S^g**ij8dxq{Pt0*u-i+I)o#6ti(%Q=@7|BN#z=6 zj0s6u|CSxiZM?1g#fbCID^)H}NFO?;98V&FC6d^Pt+n)ZIe*Z;stdd`U^MvD77nxC zZMX@|X0z8P96NUO$oyhxN_^$rAObOI%&kS+x=2(-UYjn3QogG`EODwAYG?7KCI=W_ z)sM9lL`BS#SVi28Z&kS}y5mKUR(C~VCX*B2$8JiiF?gc^J|`vm`l~QIYj`eNbewW^ zlNU;t+Tty^aY+IKlkgv}G15zLQpRQ!P{Y`)B;vGmrKxclc44y1q%#afWq6&3TkoQQ zgMa&f4<$xP$vG-(Wnpv0sq#BelB@m?r#`|Lu$MEdNRlRxOdDOI`59CTo_!$n8rQtTTV5X~{nj^)L8Te+Sa4VM7b z1%hnW`G#s73%R8aWCM3AM?d=wO7?6A*)aJnmL(%(!`#=3j?j|p6M#C*{s{IhGVV?$35l#iYQC z;lU5CcJ2km_I0)=7yR1CPf+dTubE%N_M{tcru;Qjx?E88G)X6G>yg4%>kMl>9O>K( zj7FT>{i7qRp3JBvo#a1RbmX!%&T75rNVVgZu!x+K#*GswNKrM1Laja;?~4lbmR!?RE{odC<1gF(9yjM&}}RjxnAG~UqAnS>pO$rfBsLLeD%%xoTIF7d}?** z3*oG``+mJA=DG7hZ6jyxhVCoIsnXaV;ff4=2{=VzF@NNOW^)^)b~QchIsK8la8nC%|; z>G4x77hV`l;qR62I z#Nw}(pU*{ZiiCJ7Tg?RdM(wy%Xd-pY=1tjAAlaJ$OpGKAB*p1S1NkW>`HCbu4!$z5 zH6)XPF9)t7ROzF2jfImEe%8$gye-oAU$iXp4 zVgr!tqT;#XU6|P>E_i!>BhAq+Bsy2|1&&p#!WdEYc{WY2CBdp0c1>X?M z3V_~#bp8>5e1r4#qc{1M>P^0pyekm-rmfGG$N?*x763nxMZUBz?W5ykVSR;<;BC&k z3rE|+*H+iR9C4mjYRf0c)$KK7-4LTYp?d+iq`vJMa~VlJ>?o^hJtM$_EyAE!?`?r} zC9U5~T)K}v?N*eH%zu!Vm*yR#lUg*L zVPhJ(H8wR1iWTVthRfNDBZkX<3qmq1| zpJYOY-lG;sG2p1BlqdT*kk)l|gEmO;ea6$!uw0S*~T^ ztN9A@HDn*sP03dX9B>d5*{1d`tS3)ozoc-b5On^pkTKwqw(?p=Xt27WSR#qNHG#7p z;lioNp_6YTanBaxifCbGacTj@c`cuzu73cRv;t^(346^7NuHD3V`R(^A?n z1(hY^!{)_@ZoH+e=H%uYk1OTvEh$MUOOh)ePRbqUay(n)SUbkO4p1#ODR+#^aetBH z-Z`T@#g5=N!h>H}?HmN?c5Gq}pwJ_Z;9FNaOGp|y0A)ZzXN(s(t+e5<61s)#d6rY$ zT4Z9ItP+`z+9T;howc`vYkL)3+h>u^2;jJfH#kdnWqNLO1W&+!H#mYLP&e|v9?l39 zjQkb;_8zK0u1=_}XgAh#I~0-QdZXaUM9-BOe~65K5H(4H6R;N+{y!5RnpfVwc5O&( zOWGKByr?~r{$Xju98n~4!fz6C!%#0}OXK%(enpab<;&VXLO@{Cw*}dR7bXLrZDV}a zl2cT;2{rx!FCLt*abaByrKdPb2xi(2R8d zH)uv~4yYLc!A3B}QXIBYXhuM=%?}WO92Y<{QdoT*AQM$G$zOyVPFd8yu~&5K$w;nR z(5h##X$x^62yF6OHHMd?wX6Y?j&t;4A^kEfjd0m!J=O8z7~5Dmn;l(KW#f+3)z2P? zVq=~$w`CG1F1BunB4o?#f#ukG)Be12bThkJXTxR-MPvB%*Ox6OCW!uuUdJ zC#l-A*5kJBJ*kQI#x44>MjqR@ElPfz^dGl@P|`LWmL=e8IJ3v1Nv+#o@>I$mt1&O_ zGmgbgyqYz-RPR(W2TDd`YYsRvrni9iz+_FQD832#mkp)HORG-~r*3rvhr{xl&Rrmc z*7zSoXfG{qJ@uc{Ko0po(Lj!B`FqSy3=R^n9Q z+Ka4pG*M!teI%74aYz{aP3trzr8h;!c6VLmWgj<_;?ozzvG|bE+hA0v(1%x=98r@L zSSMNx8pm`A2F=J!hfH9#_)_m`AqId3Ut3o}mJ)JYhpPR*7S$rp;^T7qcnpDO7E*er z!l~mIk06=CCYg@Afo;O%ki<9MRJ>WZ2D>hiNDSgG|8*f@SI!nKc1Fej7)5T=Bh#`) zHArx2NiOzXLw24RKWZ03&w&`U^o`>LB8+Z4pn=4=@w}9M3y{e>PUXTIp<@2*PZmgx zQSpgWE5yhBkU~>zeOyhX`WUx&lD0wb5`cO$s<3e7pHm+&Ezp5gquJ>;e zt15Hq2LA?yO`xSL`KDA`%?_z@{x6B=Q``~Y_IXtq&%&?7DS%rM6PL0t3+s%S`xvz= zlg)xA1YiuKnUGKq)9pE))IET^!22>u{g;K$j#I#KMRp92EIW-$5K9%XL0_XxFhZ5j zBd^M+-j~O9^*vq1P8T5{w`jorOv~GMdW}vt-=1$gOZy# zD0pp2q0z7Qy@^^QiEEaCMPKYhU-7YrwqA><7+9Pg0H1NFnyGL*A_nxBzeFm+%HX8{ zl>H$HD%k)h+$*wc=rnS)G?&_}oq;x6(y{CfvyvGPlgeguBLvQ*s?^de6=W!X=8tU$ z$>cIW#^4)s_G0J|R#e#_VE2?*x@ghcd44xrql@1DV=b2nvfpeCld0*sgjO!^kISkt1DZ)6F-u4jXGp+EH^O)Q20bn@v&?E1yb{YHf$Azp>fePJ(4p zf>rz|wQzQvF~B8;Aun3yu2P0Yk9Ew2ikZ0A$;w zLpD)Q9vH!75oC2s9|BKjh5^~EzbbDf7H)8AcC(Y*1fk%x)cXp1fJrGayJfQ`{N>(G z*r3nSf0lL%_E_XZSxysfd)j$Tqp8fiOcLQ`p^~;Mt*c9!k*}uUv>J#$`D$FtS8v9Z zsXAC7bXJahwE(-W8K<^!r0BC&Nkn#@x0Bz~*x!5TgphAqwk`ArB;UMHp2*Xm

7j zppLOf*6$hywWzQ4W*>s#@MK){P|g`#WQRiW)9IpqE9`Wq5;zkzcH!Y`L@hBlUdYl~ zgNCb)cplnJ-z)TQ;GzmaK-+<5KYDbvUcsK*g>%$WiI|aGD*u!FupK(Jx6Sc8a}E1_@f6&i;UlHuhwou6n}1RgCRY+gd> z##_s39LJ-(XGB+R&2XLTUAqe4Hpf_xzO_^B$Z-@PZHx#>P@MpIJ z8^GI<0O0M+;*og-F)q8hwueA=i7^9UX>h8qfEeZHC+gxwWB!;9)(-T$&`df(p9D0H zLvr=~NlUAE#~UWe6hmf0a+?p#X+fnD_Ik}+ow^>TroQdZstljJ3N>sA)$#k}bUh>G zBE&74^4E00Mwc8}ZNtGa(rbx>0Ia%jrjr@IyslWC@rdfqS#|TBWKz_$A-{$ASL9U;i06;+)Uss z%D%pTet`P^C~~*Qy1swi<+A7qbdH|KSZEq?h1L;INwgQbmriv-czan`Qq_y>DL<b|cPY zreo`J3-F~duIVO*l#3x81QI3XK4giBUF)$|hP;jME#aT$K(zL~mu7bQ2VuO^ar&Mc z>*PjktyfnG7q$7c<=9fm6NPv!kRIG@qgP;vm2)D;bfN8( z#tII-fpQWI8=PC~$ZrCrT(HU1Lfs01`Rt~ks#{uebu@tSgN0yB4Opl6WON4whF0CC z&nKD-&aL3r08RYuyQFw88QiwO9;j6!f3L64C7om!yXwBcu2^*+)Rdy!y-nM*`dn2O zviLzbr6opMw5mkVJf#EiYcyevP}sLsu~(JR8eoV|x>;@sDce;OkC8)CfP632^(d~v< z7#rJ0uJz*&tRX+^VQr%h>p%i{FKhdV!NqxyES3y}hQZ@`p~E_gh_KO0{od#Apa z1y>YM5d!OFOhdye&f*;>grx@iObas7hhmQE8(e%ucvr^5NZ6?k;GU>w=^cTo+}idU z$<=fNt4tJ67sw$0e(H-RV%at?{*UP(_x%TTkURcQbdV`rEwYoW7Wo~jMP_w$_lrty zUX1D@AG;bxpt=b5VA~4;WM^cvX`5y5`0G+xe?H~@ah3NoC3Dj*=VY;s+JoboSB8gs zDEF#mT*sY6Dy3ti>|Mi{YKnL;yLq?Vf^A6PE!ZwoTd)yEu*_ClurY$*j9-ckG@VGQ zVz!$(GY+OpL{Yb!YL(C`#2guqIZ4Pl6S2P=f^0{$pG+}wGXmqW7rMFeG2)~A>mWr! zn(6*`>YEC16?5zb$GFu7fgz0zs!qs87=T(x6Z0Yvss}{o-#gXCI!@0XY`J!=*nRB+ zK8~V~4+vQ%MKJB58PuN|da!|J@e_J|y7Qg3&(s{gU#lTfRg8wiv_?6sL5-6Hs-jKU zxDW1D6xbM?f@rIRYWe%wCi|I(q90~46;FD*M70mF-UX^OB5RWQt2FXkWH`UuT`c-` z5bbeUm26Od<{@6teAB&j%;eJcx?~cqJ<`KmE3&(Hy4oaaApb-gN5A9_RLa(w%-_F7 zHX`I*zD}_2glN9By=;VI+JQK>K(Y~0N5rpe1OaLGs&&uH!0v#KXnuCI&~;iswBHeO z`(m_M(jgR+>#)5fXM!cApsHJ8X;V2Vj#s%XiCoXr5|;8)k|$^twtNZp{kWVzmU5xv z8~{YNjuBb&9#kDM6o~|ABA?k;U#BQfuw!lhHVMmk9oY6(V8H;sE-qm$m>+xw(#Co_ zXTuI`xQi$|V)C(6(~-Zx;%5mpR7}_eJ-9N87n3I|jZpAf@@&IAYP;o(Rgj4Mcn$fa zJGr^}TNIO-sVFetMx)woLb2&9^iWO(yvIOlk<_1(FA*T9fI^UOav{jm|1lPA){+Uc zvi)zs_Hki=4cKn8*??{JP_Y5qvnchrKBHFdk==R-)Y*L@?l+^8o9`MO-SW>>^Ph=o z{%`8l{HGS&IICqOmGl3n0ovBRKdj67ipx~ovpV!aQBCCA^*K;F0*$@5Iy8O%OA($U zM+K2qPW03!YZc`5rK_r)I|Q(^tLJ}U%sDDdf`a}8pQdH@U7yJarWYTtAJzBZs8>er zE%^aY+tw@QL!pjLxVkoxAEpirNRY9_p33BIJ1(^tFW<6`T+pXywGGD{=b zp`_5~rn~zzHf`rV&VTyJ!t{^&i&jG`fes@#+mbF5P#DN=kj5Zcz=~3+!0K4b#r&?> zSP(zJk;_Vc;^2t{YCcWowI!KFj|?wO5Zi|P376n8UoJL;IVkez;s z+3g3gk%Te6{z%1STULkGIY_U5-`i3S20t}{m??45>||{M=q;v(C7puMyZ-aJEE+Rm zWsUHq-i0gOHu8ZFsY;h=1j{l7{Jm}E+h9^I|J!+NQ)wn)RY(DYX%bnB#SgL?)D~$Z z8_BvP;+cJ83`|>@C4zGFD%3n1>}@a#4vpOMC(~CqAX(DW8dsPJ-DT$-o7z0 ztfHjV9(9=V9RXatyFfn{>5hH|F@%v?u}`y(Hd|n4vcz0T#A5s@iWrhc1jPIf;#s~o ziSi9NF+9k+LiU7r;@Eh{?h^$gdD zEZ22OwO4kE?LIp8;HWO{eyBfEn3S92attqa49<3c-ZeSJom}Zk%AM|Vbb9fooS?Cu z-(y4U5@!YWVQZO~b0ORFJJctM2wo9U`0a@0yW?(c56|@EID+#RhPaOcxGe#2lRL7NbQ$Y0?xOue{9O;NX}|lWtVs{t z6x`1Feg0j2zj*%En-=a{m)89AO_O&2`1k|w6#Ct@%}|ddPFG7#i+C#}y2TM$STw~M zns;%Gd(*KDPm&`zeQ#;QuInPJiZW^=?oVpIO>Ek>wnxauQF!xTroJ^u79+GndA05G z7~K809*L2cv#v*SlHf4;BE+s82g%GEInn`}SVFIQF@Bt)OImvu*zk0WeSMUYS)})r zN=X(?Mr73uauO)5z=V{311=HF6RR}E4hyup;*!4xpyTMRvA@ihesX_8iCCIfvlpEB0-hYQ`%m)c#3W->Ue2bw^lAnbxc0 zoe$5fEgbVzR_?6Y!Z}}MSFS>k``dsdTtK1-em&&(_~-4-1aL2;lo8ksu=*kP^8B^3#%$TeC{UwDMAa1* zAuXCIcT;1l?R_^cXyVs*8>ppf>N+$N!3G0@OE6PJYe*;7yLA9UMO0lu3vZ|aCF@M? zL)B+u9Z=mWqw1oO^{J>r>Vy>R7&J210t72vr!SO{pqdjSEl4MQ=xUy~saYd+<+XbX zaMOG(f?$FFX`KO};o2Gc*z#@~Q2eT*r_8MKZkud7$65_d1a1Jm!KS&UhVe_-P_1*peqx{V4jm~-4j2Xl%;vjF`9%-QB-W({B_T~sXb zYY-s16d9M^qh1|@6SOc01|e-47wR zEy3lUirls#;dO>R*>dH^+EWXDwONx&#{E+zBcGgewj=-(cSVG!{)ac>zgd}Pi9rS7xHsiE5UJMPoNfaazVn|hh zBuR)b)iKNp*F9k~u008x<0y?K7wkVN$UaNx1e1V$iBTGb*L<_cQJyNSZ%@({!8=mH zNP7RgOOe9XL1eeZ7Z=V9!ACBVPq(0U)-0;|co@RjJ}6PjG==&ULGuyI6YgB%x1|3+ z{7WVAy6GSGF3#o^uUOxcEuwcQ<~aT(tbQmOud-{jz*$c!qs_-v>%=iy_rz?`X>cF< zY6%t#xL+qw4H6))U6y%l34hzGC9)yD{Keh?@Jw40{nR5t6@hg=GRe8#GD%$Z%Hr7? zRD4LSYro=N*(HPG0rq@`gL}J=w!|nv2Urs*T8MIFz)LMgVtrapZ)~|@8nq?!Z4(TH zw{RKjD3n8F;h+M?(dui218Q}b;f{uq2e5dmAbn@ARvF1d)Gt1+l2`P2b;vYO0$Nz{)6$xwM$HZ0>T67vVp{1oLB zt+OARGZ0LPTy|Qy0J?9S_y+HKN2{zaN2;zz@z45z7y{}~3k)ImZ05MSb+VL^bA#|x zlrYmlff|x@3fN5SU8Vev7yGhO%s_Brj$EM7w;^nyKSFp7uT%1o@9#A>0WQgFV{uY8 z{A(GES@|Bh#mZ|fpJJ@vXFYmZ*J+zfC9(w1WL7`KBizC<_3+L8rBGjk;)j(FMqZd48d z3Kt_#-w@hlWU$rQCQI|8%m);gqTr{6W{_`1nls!Z(l+3Qk;5@BURd}yiZGXH-a{-2 z9zO97J>5qiaI~cTxBdurG4g?5TE>-eiyF15qL|+%M#bK8aKJD=9|qmUzIlW2@dNV7kX$?S&y zowg~%^ z8k#_^9bJ)-d!@^rer-~1{blZk&CakHC2Jc_q=w8GTh{PxYRIANvWCXg&_j9-(;~+Se{Y zdH)nJYhP!0x`SEU?P|CJn+|MP@!EwSC4{iqfvsnchBx8zH$NR-Oa((EIvbMyd9OyV zdXZZOKf-X#4?IIvlD+}9Uc0#Cv0ed$lJ)s7e}pX>HBe-+YZ|JKGz|=51(-#;TE-Cm z>VaJsn&9)H445CFHht-PZHMkSEGl!Ep_~rw;0>Ie-h&k@7|;sBIm?a7Z4WQq*LLWS zU*&)JGuDYxEX*R)4ORE^2c0@8N(z~C&N%*!Q%tM19f~;$84dLeU>oCRE7*pr`PHEg zDM>LyK$Z7~5+w1=`mtYPhH#?J3nGy_P)c&eoBm2k#s@4Vxobz7yZ8Z?r043DBvYQq zV5p4}Wc{#nT>6OL0-`0=O;*&|Q+&Kc9wRBl|6Xj6MA_}dwpebZjYd$j zX-wnt;Fyi2y=(hZxq{wGwEmi=&6>>KavCjZKRx}*CJF|=4Y)*AD^QLtuS^L+-n2V#_-fW=j|Dhli*8z z@mS^FUeR)0V5$YX^|2>{4SMif z@m7|)Ga8k?x>W2(NqwOY`#>OhTRBL!042V`pov_~tlw#VoOi%N$T&p{Dd1oxE0zbW zNP?RP2(7$1o$VwHW7dpWKnTXiL|j=8Ugu%v9}wnHEVFgv(?$wZ_rQnS)bX^~$la@R zuXz=lvuoys<&uzdXz>-R)F2W|o?%>UA>@^(uyq?Uu8IYRBXh01#~szt_f=4GNKET6 zR3xeRw^StAYN^Q6U0w=Gz(T<6)xN?=u`!!=Jy+kb z;D$?!1p@{ni~5dsNm;USjb%+_6DwOG(eAZ8m5n1(0%`qvT;iLQC4q<(#AePtG`*}Qsk|ofiH6O=Vc+)(`>R>YUG{{snh#1FP7oZr!u5FX+C97n+riZ6v}bAiY}ex`Ncr#G zNV{kebdFuJ-SrN)C*|=3+M~N7=s;cWK|r@l3KG(`A?xj*DMykk2pm7AGwA^k3hy8gT=fvA@ORKiMJ zCa9o{Xu^9CdZN`1ZzMytvSN2N==`(MZPCq8Gh$^Y5?vMH-9IT|eAEMYu0)Z>D z_E8ldp*^#IB*CIyKvD@7`4sP&Jzw85o2z!(?3pc!jj(c3hCS&e^kY+hcYIp@G7KtL;GwCP#DMo^F1Fb&<0BfM5&Q_4u`p=SVAzlR|sr*6DI92NZ zISm|coX%kJRYBlb`?m^5XaR10k1O4IlCg4wkE{p6)D7qqvUWIy*!!#cF_O1<+0?NK z#~zdiIPH3%F(Ha1->PpjPD#K|;7$E2nX69G48z6jL^?T4&8k_I8bR zq$XqF7MrqBErNW#+&^AN?m^?bmJ`G7lQZS*4{&Cd+{ZNmhDy2I z+yR`^20$IjwetPsv;k4s0%8FffT)PH$~HH??m8FGk6ly!F&D125UlAv@)0g+TV6tX zY#ll%-;G@|GqtOuEqJ4>J&*c@YSvabs7TUmt{`Hos)p5}A@P-2s1Dqoq?>wHU)T$RLFUvgkKKU5Y8s_5Lgw zS*8%?>p#v?3(4?0$rhOb9)k=_*CDJS?MF8-fg@x`Y<73PC3<1oi`wXUaM8Xt0~KR6 z@%R5e{Cs|hhMRR`V9e#J6W8edK{lKAnlZ>F%J1lTI^DLH1wgx{=jpacLlUa~d)0t? zIsAOQ5OYM5j?=Om4rI1PsPLltrs-w}#Wg0wBd%1Jl^-w5H}T1uGDl%MPFhpb*?Yi= z04o%V5w^=3ZIV;le;5a{L_HpTlABt(hc@GF2k$Y#Zh2W;&bbtkI5rA$4fbOcEO(aB zaWKcrw95XX46j=1B zTZ#(fzvDf#(+~D)>^GcvCHI$~9G~3Mr~62zj_iv2zcTWJ`nICb*#T;jycxJA2~=Jd z^qM*~Y3cU)Un;~T7kmwmB?x48=0qzm@JhHSZL5yG3Reuyv*h4(h7A+ihwh6iCS;~k z)-a(Z&N8Hr@v1@qjg6`ei;4qdS!45v1Wb7ePG<&V2TB0SLG4RDUt}<+*MyIq zGVyUA2}i7|P?~@t;ZLhqVPHsU!mNh$gViYP6;KU10&2+IgJZBSTl9&_H5sB04y*0i zvY1rc6n7&BXYF^+{v=?k+7|0nwRwhAwJ!o_A)(g}oT~PPcdFXqT;Ei+BY{%YwEBLQ zK`EeFEqnUqm93U>FN6|JnknVB>ihyu^%AbIw>9Pn0s%2tXRu1LnwKx2DJ=$`&guF% zd-pc0kN{)2q@JqeTM|69Uo$Q<-Nv+%iF@Ew9SJw>>oZ`q=HCy${4Dw97Gg*yp|lOO zjaeo7TBTyP6N}mGSI*;$}rmDPF?ZCA2Mc9VBB|Xnp^#2hG7< zi4+S3&vZ9N1d$s|0f; zP=Nk3jK2&CWW+t!_KzmC?By(pFX9q7-!frSVLKC>HqEZ8ryA+|5pX)8q?SfknFDl; z5HM&jr2CKr`hY}ZLsH6ebirL|L6`v|;Sg9db}F!eV#mN?;9tW#A#fLCok5c!{4Ri*l1L%+ zUaHq9o{J8Fk+otP%7&mE0lx^UBRlZni4-tejg_|Eq`uIAl>{1X6IZ6i6m$^IplYUz zrC0)L@%^7?@u#6iIGMDZJU%#Oh7(Weyn`5G;@K2~tR|vk{U9fgko7=v$qH7$+@zY{ zM+t*Wh8hiW4U-a0tV)5xTF?DZ@I`Q0vl^+YC$|6a>jxp zGdpssX#{kJhuscS^{{$ocVkW;>zQlaH}v9fp*u*vV^Od>$Pk70eA!T#=j5mZM&QI*sS2G!b$<09ac` zB55`6#RsCgm^I$~x|>@X&vceXGOrdp3q9D9Cl@762)^x!`6e;u)8*bx!zUFTJ-MjY zS+T3ohAnwTlMjqrdi)#danr9@T^GByuE&a|U!YCf>Yw}Hw-OB7 z{z*mq3W9!5%;MfLb6_+wY*NuD1;Jt%kG%Cj-S(=cPr$JC$G^B$7nAWw-F9Ermj4K^ zcGnN()ns8*1YV80O2g(q|1L-fT|6YwNgH)hM9>7j8c2xc-n2d&)A)7AX@R2v8Ojzv zc{eWLJy8@_`6p&AWMGoVud^0StXXuzTZf35nwmFsWgbJnG1vLWF*UR+U+`dMc}Jv@ zQ}XOUA7UA>fK&5zwQS~3^q93WO2!QO5Yj-#HXH`SB)BB5|1v?fj)Qb%&7%o0s9%#l z#74x4cRK5|QYPE@svFmmEOEr24(8l&TqJUh&Z_iwK@0hE&Z|NX8r+bZ{wQf8uTM`d zIg#OG+kE^ukbWcq=IC2qqvo)obe(>U{%@W0&k{a;oM(1Apop3yLT6+IfzWLr%eKmW@2zmoD;)m_Cl~+NW&^>8=fa>#J8ReB3hJLMZQYs8&gP zJM&wT$gu}VdhfEL08~Y~Gt0yT3wR3Hg>x$*bMVaekH+NQcIE)@yXN6go14eIyLG?VkUZGnl(5 z=X(@eO}=;|8Eji)YR&Kuj=g}(j@*D5pN}WNwS6q#DZn>um&qNOr*KE-4k3cfKU5B{ zkr3zqM{#1u6Ze}4Fbz$FnTQ-%!$6a(CoDZ(RAu=r4Fc%HDKnI*LFi@Zb%<-fJywwA zR0s>OlAIDLBEz&(Rg!jXqhm4yPliBb*a6cAuskUym7vmHYRPQFkzhcv1bv1Eb)Yjg{TCCWfbLF&8V4Mn~|;bW7*VYfZN z4V~dYZ=x-R0Y)-(p!pfu5dv4ke8{y7GYOCJBKT=WFCgCW z@39-Xu@&Tz#ibd({1^80n+u2|Gkxcvli94}vbapOl}gfpMlQ_)&a$*K%9ch5`;SXy zJ)}MNBO$Kcvsa-%{h;Ge&1`Bt$@yNUgw)ETr#Yvw4C^%x(aHk+1!ha_Muu61L!g|9 zhYo*`aVSaTw*)IC9&Brma-P2k9+jlq47)uVLkrK6koBe@ADo#&Z+ibV0^jeOEpw5L z0NH9QAv+FNw*jk8MA=AD-y`#PGMxL;+!ZWYJb6bJ`ieodg7H1;Z_Z9-W z#Z3s-d1B@y#@qlXcgmz9@8qKH6N1lsVve2kaNSWWntmP=|3+He^vhS*wXLc9rmATI zpj=w~(bZSAjlHb))#Y^u*3|8*YC0_@er9U?#MHRyKV4ndRMj*9e>EU};}hKigW)GR zDrM!U-a~ach-|Medh zcxiTG_VV1o2K?@Xp3PSe0E;G^|Fb6!E-2bFDR@fmAKte5XD4hxvDKo;y6G-?@yB9=X>4L>>X1siTIj z9L0y8xwgLW_I=&k|4B&X#!*a9wY!jPw=T8Fq~9jQ0YDYCuPi#k?E}84jO&qzLudB? zWGgNoSqqs)s4t=x(xRJ$;>M(f)FhW=keT$tL!BrbL`Lo|l)P6MDC4{)KI-EKq3q;m zn@}EyW6sye)C=blUMnOs2NyrWaSRuyj-%et62`hYLSo;hp@Qz;%zV}Mq-|N08xm6c z3&yYg^u)N@WB5JVN8`st`VsHNk5 zHA=_9Nw{QHY#|{sUbC;_*DdljAaRtgY=A=tiy`SJ=Mmnb7GO;zX-CDOLyN_aq)j~B z(nH8UQ&qLvsY;`GDKOqrxtB%g$~G3)Idmwrb+*ZFKkYn{1DaV=^@C$cCTd@B6&%$29wL7VRKp>N+R^~9uq_%AzG`P$3sXqm z{dCN$aF)YPZ|2{Ekf-3(NLhu>HUrO6_$gp%Q$iUIVL#33+jInGdQ z?TVXPcd(zIf^h4zF}sc2CqizzF1m@owy^ri>l;K#Gq<~I^Tg6%vx;kz^? z4^x0kJpdH8PW@y|=iqRGq&)`2rzts1jpUpWu?*P+mY6&jBDcsyoBY5=SQ{%w2SZb; zey;|sAfF#|K{a2^9CDOsH=J6wTXNPcMJqDNIk_pNf#U@Cr3O57kpPa<>kGAaw{|BR z+vdhF>u49EO7ow}8^H4tByfdzWbISe9^O;YVq}gjQbCocYWg%dSkecv8Mnxpt!BDK zIk}~GYL=opOfFy{!z#Ns<2y;p$%zWAEsbIE-)xqhdYsg=PKRb%4V=}@v3GCgS&DKu z;dF1vZO~t_T0s>@Dlkhu!&FYdvfy1m^yP!;Y%X9|JIK9@GP;XXEGEgZ8dyQb&AniP zS-Ru8=#43kT)KcsBKxP1x0yZ_WQ>#=i!gX#6md%JS^Tg%#!NiCOFj#~jAJ>#dk9u? z(WUHFk}%6Kg7%gL{l$gQ&?n!q$3W%$9QdxUBlX z;T`zIHUkjNv@t7%tL&LSYx-EOC=Ak7rU?ehw0xgC_ym?SHnMO%Cgi~5s0*Za#Npnf&tGRUnma`%4gGBFaKsLIZ z5weN5b6lQ5x9t-E)wuve!N+mmrB!1SPupyfYV~2wv=NjAGZ|a$7-YeT)A5>vvdY0Y zg#1|Qr%Cw{__Y7}j2MD;lf=>zTpXRMF_YVA_{n(>E3)1F)Vy2wZR;W8kp3;~`BkwHE~H)cH;_;NPr zZB?-CUswWAUO~Ha6wJmLBqC;$R$}3p49ScosJH-TBLXV?_f>6`&&+%E!NC_ioB7Az ze7xg{e{^oo@15rTf#gv zrJ!hFWICOKihg=2rL=i;(ioTn%?&q(|blb z$%Z~OnouAShZb=wIzUPZ3DtJ1Uo&H6UzGKXmVA7`_GGJZa?BmDYi@G&stS4#0L z;LHQ>qv4}R7Vn`ZwTCd84=t)939+|1T!)V2P$u6(J_w{^=1~SM^)Uhf=KGauOWN* z8XTKY_Hp9JBG{DwNd2p$H5De8Kub9}h(TA`p84`CQ`XJNvhm{|x2E}C;WrX0tkkas zrxkMjYwRA9Ft!IM7Qo7YwQxvymS7uHrD!4Gt|Wo>YWaPy@a6W$-~?aK&1gScfV{)_ zaV-6KJ04lEgp_r5H;xpZC5Df}z>_!q!hs7as$X<>j-&uPdnpG~&spr-=6kI@XAd2s zBp2{<+BZowmUnCKNL}W3=o0QBC&OG9jU<`gei~UMd4iQtE3`6S-DEa0=|ju*|4HgK zb(ot?6%-d(G{cKhLs?LK2@c%^mhB=My9h!_R>$q#Q&609Nm&CcDT~MU-WGh>A6NOx zl3=|*ZpSO4IxKB#G!IFbg~%2(I!k8X*R6dVCYvtueWHLPlwHuaMw(4i_@=`LD=rRA z$62BB)N$Z0%cda+#XOHuvHfH@2Vgia!JDqYCcyw@1(sje1Xr|gJA1i>0% zSj}z>H<&|)();^z5S~OabOg_TQESC31j}~}sDe!ZrNNxxF8E%7+H;DlL{bR`aS9uQ z*`f-K+(gnHI2FvU{*o&47yg~JgX5GNxkOV>GycA|=%5P}GJV{TfIw2x>S91u@5ny7hFIPor!4;`=D5)fnh8mZIwD zV7N;PYQaaTKp3}GGJG5cG6ClR!~Iz1V4s>G;Dq|r71oLJG(iDLeVCqZ0@T6egK}Kf zVxVaXHOt|QLQk`CabjXb)?!iL3$xk)Us3>SBl40Fu?axVY~EcsZ68wWjEi!=bME{iKF|at_=b2XSxpch$IMlk`o>_BS zM)7E$=M>b!ks|UT@7c?|BT(qJ2mcC!Gewi_-er5<V|WvWP6{3evE)x6mOLjT zP<^7m>1l6#$FAkx(wk%b8J4nG@+&Qtd0!%}+TuGGMlHhPPxWu<{I69IJ_5SYQ)v05)mCyGwSW~`{clVnB zcM{T8RlzqB>FDxg9^R92Tg5UYUXKv3F}zCNK^eu9r4?yeu*-TaH>o$i zT-_P#8o|F}z+18!V@1*X8V5#ims^VUL8&*pq46Wd@#Rz^RY)_8~YVt-hepfx{4Ur#gD)pTWnKfU*2`+9Y1x!KWCOvDK+MYY+}ta?!Ly{I z^+koyD}u@wtvwo}sC>DvJHDILh&kdVfdkM&1hIF-T1?PtqMAw}2YHv&Ytn0fqwGjR zI=^v&5SRb)e&oW`BAy2*orP*DUU$`f6k53Y9oc!p?85BQaeG5oZu?5 z7J^G_-FD_ORKSbqocp62Q4;npB*xf9*JVmFZ;59$LN(hbe^?|ef5I0QJ-Aktv14q- ziouhc9s8A&2*&`p+YHe}of*bw=k_t?oPcJ9?41G3!w)->(2H=BNZU;X87Dw8eT)zw zuErIMW8@u8lurH!?MU)(qgaH)5tXE5fCk;Cnlpw)66lM7wo#PsUE9gOx}9WD)({yc zJspA@N$sp#q+0>wUIl&iU(iW+%wpPE(aczOrDU5Q6m}qbaag_OA)>yKM)x=b5ZpRq zrBOku<`|ig{LgRO?_m8syrYcCEz)(ibpDAQzG*)V2?zK!;51u;DVE(HWY8=Qoc!Iq z9W9-`P}sIr6}DwRpX)n6-*;|C;FMk|{qj;foMsbnPb6$n#i~|1$5Jvhl zYkV2SSNS}>)?ZX~)P&&gJu#nFdN%>LwNENKToC*b&3WC| zbQ+4?R&)~P&<)~px6k&R;Iv0m5^js z(m=w_aD*if)wgt`40PMLbY=`es!@YEBdPWk*Jtr&=RkWx0m({rIFRHJ}XBd5+XNc{xep>mqeI@JL=#;xI;Fz;qQE3;6vntB=(jlO_ zcNWHHY+lc+^<(iEf4T+P8${@lU;lbMdESSbk36$E#C(J*o}w8@!n8{ohp6aX10~M! z&fsWCdpYPRRc!3`YHxE65{4d#s|~T8%6bA+3$$U!G4hSX!V9`=UE~e!CM%__B7TlO z8Sf@P!?q-6vIl{zjWg?SqE=mT7j2TA7>pyClG0tYt)#Ruj_0w>;js-JQMI!nUm!$U_JyKnn_Q($3Xx+O}$^>OR$oH-Bx~HK#`7oj6fxg_z zx%8n2%W7XgnG}ER*H20AICF!ja6r8DG%h#&6ts)SDfDbDY`|5Q5L#I1r~$S{YM#H^%g2m=L02!Z%I|4vJCbGW_m$F zH@Ct{g-57wfVwq<$WkbLf}@ts-10)LE1>eaREw3kUptY6Ayb;wFF)R3k}v8&nWE&$ z7f_2=iflC;6WA9wj0`;qYPN1ENu2j)ox1J zj}DT~_#e|JgE~d>j&~FG)7EY?BTG;Dq+9L~pWJmVnG?>z=TjqfTb7OJy_=(EchaJ& z_o(<0wj@-aYy(+(q!~v?tc8UwNwslRS8Fk4p~IFa+l^pbl3JCRQIDeI>6A|RHd(0N z_L5jmO?}>?)NrG1HHO5%b*gKLi&)W#^&r@sJ(*1V4=6|QQSnI93VV&MD%LUEEjn6n zpDLQ*vER7179F;0(P8`6&HrMDZCASO-LdVPe>^$loBt6x=tRTL16ZpQPV;L)-J$jjkKtN z*ZIe+Ym-XF2v(Bq#D1Mk79}f-Vjd~#!EDlE@jCty*nvL2m)t;V7EjD7XK7y;*A`p; zon)^h&Nm1mOUUsOD=MxeHyz2A#j>LFilk+&V5Ni=hon4w{zzw@W5Gg!I6tR|1x5WkvL9_`{ zk2g#wuV-mZzxy1(?e7tQ8#dv{xDA-#a1_|~5nvm7(L_aUX;D#IXR!&j#pZWxUL%5r zGm#oQGAmY?Ke_=oQaaKdh|o@(dL~~~JbGDK>z@4LpV7ivHtgfvGea7*QB~Jpv!Au_ z-g#D$efA%4D8K$%Dl{RjN;o}ZKZJz&UlG@#<*2!TGiG)d6eQ?TT{0&!d?QJjh==WO zFE>1e-z;v$M1)b^7wKi*!3~H4@R{<&bjn=@ZGz!se56M$Yj)Z6|5ju-QI>uAuED;5yj=Ddy{0yVm(peFf4R9AmIOUQHmqx367Zaw5sFAI5Q(M` zto{2)8(yR&VaHDqb@)oDafWa$5r9@f0-aUEA=H3{E(c3SIw1u$4ak z=cvmV_B=XtNjUNqD}g>Ar{shi^k$G~L;`BI-)2__XVT9;(quTG=tMW`{e1E_=$t!hYyNNMKA5twRQbhG(Cvovw`t5 z*YpVV>6J2dY;tx@u5UoT@A!*b!UJ;xRoUR_Z1&7`8Q3xX>lWlV?}TIn5uUfRKe;4+YMy<^^gtE?#r zj*A&X(#^WKZ4L| zei%aA`q%mYPB)UHAF7OtU>nI*RX&DZr@Z1lgK<@J!A7GIg|IGLbzp>$DJnRnE6upU+|R~+aN4q2?z5_MxZl%>7Cumi4Qp~#6BVChRjrwjr3rFEBKmHn4|s6BPk+_y{1 zW_bC%OZp;v4UAQLbbkG;?yTXgO~|hw;l+<*N3izH4TjN*tGdDQLgj>pd`n2ipGI~h z%H!W?NAguvJCYo9Xls)el3T}mgfg`vVQ?JQ8T?Cs=+0!)aOTAUy9~;H&?53~=GU7a z7O%;+AgU_zVRMVNGMg$gioHM7jM41RMc0ZaNV`9$}+Cye6_1IhoV%prlyeFyknK& zvytRo#KMsf8bWRn{4;Zl2!okg6{`Y%b!Ld~%JG;zB#X^I%#CJlvWuSdE~VmEI&?#{ zU9Ii0Jjkn#5OxihLsCN7$p^2juWydC*6KYF0MeoCC&A+$9fs z3M8GkSw_Ts+i7~5HBjjf@q}Z1I>VUo*2qrDH^X7B3?1-=MaNeSL&Q%S+KH}QuD1Xv zfNADP6Bv}6$ag1Kxk0FSrQdW7jOn3Q!IcfI>Ddm?6E-H${jD@u6Q~!d18H?{{6ajc zjh~C1h9Ru|v6beXSSt@&<*0(k$7Cst)8nz-;#%&l_9FpqI$n%!IjPc=- zD6NSq9A!>G%_IBNB5rG_%XU`4{pF_YWZMwgF2u2iiFF!>Brz4fTIeSMTwVBbAiG5J zSq|4A!*mj10;x-^UCJ<>?90J>+eB~2(9v%u%;`F494KHHt2B4Hk$WA`ZOi$kM@D2V zg-iaA){gDZttpGWHzdSNsq|I?njM-{^kqSCg(s#*rMCr5*#0@G=!=Q5sgqaNy}zdJ z-KwUTnE0ay#2-M@HK4QCs+ta>FzxXnzWW9TUYF1;Ir}kz*1BI^d;XQ@9!VGYAk@|FGjuuZiOxQZ&(&L zr#JIkpRX+1$A`8ri_6&==*Fy6_2&zwlEiP_sEqchebVg9H1#F_6s@(@zf!Vin;`_S--Xd<{H9 znDle2R~z})@zMx7oa6d6^J=$lw!X4W)fif>Mx0bQ6jV5nTzbQ=KNBREERW|9s;YSRg}$$#gA;3PsEtE z@@Hbq_8Qjy!gqwDXy-=}wSW||pX5z!NjLmac5oXDLoa2ml;xK%04yk?dlaZ$nuHR2D zX+X1Ai(mlBuRH;m$U3BT=upwtGW*w7tT0W<9oOLy(QflzN1wW+=~TKlsQF@R+go+H zRT6RiqRAMA6F-Kk%UQ$L99qK$u<&%BpmHZO(rTR37GTSDB^Ehtmn;~TNv>j+w2#R8 z>R4C=a;Uiz8QjnfP`yUrEh1tAAf)&KW5a!0BkpiWdS5)pO@fmz;D%_c6*0*xI4YS= zv_%J2#0$J8qH}}Q|F5x4xBXohkHBmmp>q_ysd-#WdT+8(38n@FaX@Xt-@der zJ{2y@Akmjx>tCqB<9T?>=`1T%XGpeLVrvezI}RUxP9*KA|MC<(oAaB04jXVcWQNd< zD74PkUZD!sWNQ6phZnqR)1ego1c#B7GOqy#MzhI=!YsJR4+X|*91xHlPB=1ZLU>jN zW0Ff;Gu35TLNB+-4kX(MK2@y@65^@X#+AY=J_g^dspI$omSHUKH&Uc$J;2hL!~q6W z+DFbn&|?4{;`X@UXKAJ%j{VfQ!n{O)5W}ZX2!nk8DPT6%;b8r=9$tP@DY>PmnmA%PmG}$xTjMRDHgXZ@Nay%%A~Y%lb(*|A2VCgG#Jf3`>z5@j^jZ2Z4s| zY^|^qk-s<3|6P>lvf({?0)$Veh8d7$!c#XQ(S`>f;lJy+VJPv~^F`e1sBl-JA!AI; zdTJzRq>m4o@AGdWB`?#BUu-27~W*kWcx3_U2n(6R2fEm zr1iLmPI~p4kRWG$G$O_n&ZnnX%1|j)_QTsX%7@y%9!Rg<(ET51(n!_m%#SFAmOO>? zIpXnUA!OSOSpFCMcmVDN1>nlIf3~LX(=~M;S2Ya*OiPX14cPXNHFbY`Br$t!uJ7qQ z-|q$odV#2d-lm} zy9aI99?lIkp17iE$1!7qQ#>)HGYkB+XL$p8aVeWqhXwY`PTagAH}DSrtI7>*ot?b- z#GZlf_}vM^0#D9P9KCUHLGb3>#P6*=%G!D?FzZHiLtPJ*f;=7|tJ5S#;9>@(%wGRO(xcL4*|6NLH++u;++)&(x$}b--@;p-1 zT_B9*aW@DNWGuh_?ee%6CT-Yp=&$f^^q+N||6Hb+`sELu$_=&s7xQbV$<}o6bLjNq zw;y_{@SDHBBVY(aF`k!}J>)ggy=1p`FHz7egLp0r9lm|*cIp-vPQ)pq4a>W5CgD&- z$t|f&A=9Buk2iej4qiY;lk8r^eM3+sH01<(PisG_(-j{K~le6BYl zhP#X8kkzRd^6>sny~RcRdW(FG1o!RIGFXNbRt{Spyvy235;6rt zuw>WnNdk0%Kby6Kj#dSiTW8HWNt{&`d>n_U%x&HzUqmS+B;DsfT~vQ~_Mq(F<9)BI z@AIqlWvzWD7Uy45cHFTOi!WG)Jr*hKC_+fy$)g}Pkvluiw>hn$vp# z`~G+Qlg4BdTw`w(UXVH2`SmkWMaP-ORIwvF_I!=SR7n#KS($GBo)$o`rWa!dy`3Yv zFxLA)4H?~glk}b8pbv-mo!c*-!ijF)nj;zh3-x9=sMQ<1Af?3MI!2ZztN-#0gxEEr zw9!_HVrS22?1BGs+@K0y)V;VWm#{O(pO%ldik|e8lZR}mSVyob;>96Do9En_PGErm z#+8F6IKkLzR%DJC6q`_$N#|@mnkwQ4PnV-I^X70j)+(I}&AF@Bq=P(CZMpktkSqc= ze$DOy(Zk)wL}Q?e8U1he9qIT*_~H}hI})YK68>bC+}@NW%Ut?EGl6cZQ*Q0p@2_Aiq!)2Q-2z3v!4A zMz$Vr+rpw38Ve#D7AAf?$Wug@N!kamfUpqn+>k@2G3wAuZOGh&kTF_8)z#q_kxebs zwZPPM1p!B~Ij-iZ)XK3sR&O(Zy>*B<9$g%+cH*q zo?z;rt#zTakPgc^{PLqCSMm~h66%h3Jj>#6*Y)@kXrg3@#q(d?MV-UIDYd2VW2bBCq?$rLRGu<(6v^+sb9B==aL=)xLf@(h(;^ccli_&2XeEE@h?Ey3)A*$cfuH zucU*hwxivU+L3Dy^#18B)2cF;&iVYN^=po_?7Ho^5ee5!y|AF|$=mzSWm{I(5pJ#A|=fnd4y1tn;3;yiwvD~$&Ew|t9OJn2SNR8i^ z8kc(MY9p3AJ{p$0)CJ3ZC9s@D#(k9N-GT1f^YW70WuxtD@#Fh2n8eY$9`{x@j0s-o zi7CW?7kFY48YTq)r@uELV^Z*!_}%s~#bhD5V?jpSDe18ml#82m>PF(?I*y91CnQ%2 zBv&iAxK1?M&e@wAO!dTcuEfz#a)bTwqs@`TY6ktrX@R!}5*IdnK%a>NmM?t6jQ zlHw8+f_umUbvx(~F0wU>9|>I~wAc2^P_-l=VN|swHa1C|8Pb9{Bt$GKe{R_E?I|>o z1P=vcT}r7N@rkA63!DrDjBQ+nqk0okhk#sZmnPB@n~4+7S$-sT7e5stIL;??PMu@4 zfjKKPS$i(;^$-Y;%CFzio3{-@a4`6%LX2jc!9TbxdJ|(}kW&yKstf^vPO$61{r77h z{Y?1vh2DX#q2vxO!!i)83~YjrZKQm+Wkf^IU^4pRf_-?Pri%GznX{O>*qxsWW?Pfoe_re z1m2z~JmPIpjJ?gX1dJ5S8t=3W-B8Epu%f4&vU3Db$5+l@n4JZU0Ul0KPk|m$#~=el zo2w2L(m>VW41Q4T;w(v$@Uy_az2ae$i!MHv6LNs@fiBHn>Zc>tMDZvqVQ09kOfC@N z#+(9N>)T-2e`@@KB}odm1wsaFiiVIy4-868hJ~u~5V9bupOq}93LP6zZNgl{G1w|7 z#?(YnTu#(s#Wixb?XWSrA{NllO43&mAe08vGhy@T7+Vp6%P9+< z>`7@?WfLLbt4b?g%C>-b+R3bs`{)u+)+&Uv;dvYJVxSrF0BkCq>k6Q8h08opCp?x~pp#Mi++&$r;y_0{VDzIiRg5(sIDI`m$GtTKz(i zoB*z&GY(&0@?Zda(s|z%#-T=xWGxXWV$gMl<-i7`Iaw}?8*S+T$1ZRA1i4b3STK}p ziAyI~kb<=JDx9a1o$5A3Pndnxx~jJGxPIdjm%VswX4aC-nu?6#n|z*m6`tD){8!-6 z;>?;wCwL3dc`a^2u+9@RCo$%R<=#zbyXKu-)O|wmc~8u-lOC=+YDLq}W8&XPi<^!% zY;9}mzNu=O5EDNA*J8r30H~IU+6oCDle+$=mW0H$Rvg zsKk%cK)^ka8`wWPIXksy;FP$O>;XLkC*r^U!vZhOPRw4O8`yx~ozS!S>H%Z?Io($@ zZCa3?EgP^6%l*UKX>%Z<><-AbAvU@F+e&Ze+U|k-V^iAaE%SCh-aRl6KVFL;AL<^s z7C&C3nYNDk32!|4Cvp%eGIMPsMt$9_(^A4s6Lia@n(oy#^HH@mmkzI8naFxea1 z&aY{(_4olnfGI2f0#bMKgPc2&kz8{z*e$8LbmiHqqw;JZ(X4$`@odzn&c?G@o#{;E z7FJ&HY_*Y#*#>}Tvn*1orR|6bpZ&kdc7*)>w~qE;94g8ek*v&8Y)nK%JuiC&o`cP4!|M0&FR};rTswD0pltld`NdZU%5MC)pk^to3`$#8d`VI? zj&JWYRkb1c-um&Woziej0|cgQ2}9rzqT$j_Cmo^&m9cD2x~E}RPCB&7J3q3HUzYW?Mod3W=rFk$$`NH=P4CuU`KI^R9(7a zHiYoZjyi*C2WF#yI~=pEq4D_QmTx~!URF_c? z7nmr$T=71{WI6M|G9sh^&Y2*$hh|Bv_~a+v(tkyYArz$6a6=6m4iTS8-UnDvtvv60 z+rA5_M#RG;4!MnP%~I$?qgjy9DWy|6&EQg8h2emY<_rZqR3z&Nl}XZxYA;?#oC-^I z{_{9xUu0|~STE%S zC8*v)JS>3)I}b&ZM~W7t4m^ zG}(bL{TiM_h6#g1k+85BM1Gv53|ozpMhnC{!`?D#6U=Cs0(_ZrYqDb`SzJYMO^{Ho zXaRq9w$0B|kCU25%dk6kFHA{sth+%=$hv<|CUlZMa!oGU4K(}2q{KuN1>o^Uw7IyzP zax)yqjF(+ZdG=C+4>YFKgK3y{ZZ6S!`A`aEG+843cfF9K+0qwiAV(XxS~NJocD@!_ zE#pn555?teo$bWAO?2bjCc|DZcGc9?b;&E5d@=C{(&OJqkDGqQ>blsqbv;%z{Q}5r z^-p}=G6FBfr>x9L-uzUq?_i$qi@|~W<5Q-NPU$x~@uAj~v7YD3JsZ$rEhV#NZf4fp zjN%DC&%*KkJJEsd&di!QJ-n4DblX3vXkS6l?}=I5J7&%@?+!F!`=lUP4E^I<57ce1 zYWjpbu+2o_+r_9JTT{2)w_JMspM!2}Sw8=a{{E&v!DW9|fnPRdD}gw&>(y-u`vJ+0 zM;o^oZ?m-c&*}Emq9bJSpLP@ow{alcN;BGy6BhrCC~4a(w2^7}@x}hJsgOy|_*Hsw zJk*=(nh(4zfs2br45MFn=-W`t0HjDgC z9=l;jnBxfZwkv9vbtpz_WWO<}bnC@&TpgzGqY|$7#W6_)oW1lo9pm)>~ZB$9jvAI8;!!j(q)5-T%3nQ+wTl zZJ@)xjLNCK9-UMBGL%y*2r-I;+^RbDN&!#?H9ukPaI3Dup+U{hP@H7TdplBOYJ1|M zEE++sc{ZUmk>?VDvtu*UD=@w>*#g=EU4i9vyvHVA`VTBcuu}F+9S0mD<7qc5I3PH( zD=?;QeItpGD(ZV5b2&+(6O{qS;Sd!Jq8&zPPMyb0Y?URqVJmO!fAGn?jP`wraXIJl z*KQ9c#?@wvp3UDw8Hv<3nSr_T4j2l0iY%W(5ADt2TN`{h$)zN;8~}f;?j#=CLfVH- z_mFRdjaZ|MarN~>n(w@lUBhSGFr;}M|0vW@AGrfzF{W$nHAM+pjdxu`;Y6Gl@<2t(ziNhA2^)T`+10m~cpOT`m<= zn1WYDM!+Ot+0O39Z8(wZYWv> zfLORGGoA4 zTc3JuZJY)e&Nr)ODcCNVz%%YP*nma}rpl=2%_n^}ODJLPRdA*$DIjNWf6H<=bn2_u zX#Xa3*^YHd_O_t2ID%5?EIm;6v15$Pfe>bean3muA~%^8B9GGst)6Y0_)#w**fF`M zr0ZEleUJQUZ$k_9&R#S3W}oN&3ePITEo zVwHo5_SQ1gk`-=L#0DpSU4>>-SlV@uf^_4@hp$T4Cp=408r7r0JHJuVe=-^6L`^6M zUWQN_7|iAsqDsx!P9%6|E_Qcm5>py4belOn1~qRIaILoIpymWWkQ;g98_78yVP4;X zhX`dGKZ?+R(la05Ex&VaM8#ukb!2oV6JwFk*_-FZZD9cwMiw||G1WhCBZ+k ze5&vjUu-Bu!{#QvZ;Q7qSWdru$ zTfl@0HJ-Cv01WsEhfc6IaJ9$lux-gyrVVh)2&r*%i|LNGtjN?3CWATox~#FkB&En9 zQdX{|r4DL-xSY$T+H@N>PO1Eq<9jpkLe>Sfm+&%(*i{L&j3vUp zM4=0-(W|;FUB-x<_3Q=zrKMknZ-ql}SxFNmmQwJg;#)ny5DF^oq+yohP`y?4mmebX z)30%naAB~#T^1z)U6-V36yNFyHr=w_*neb0Iu3dG9R8AKWZYeVPD13#wbcGlBuhZ|xjKqYt1U-cSUfH{dA_23iH4sB@di0W_2 zmn`&N>DTn_)~||eLB`25~Xny?VDkEKqG~*_28{j3- zGH2fjDC&2zPh}w?QJ2+gg_z3doIQ&>XMg^q8T;3NvubQKWB&zFjQt;RG4@|}4%?8l zy*V&8b>ix#<3O(c#OEi=5$PG3h96&qePpK3|G}P&;_Tl3E0QydNA>ofga1Ze;=glG zdU3kXe+7PbX7Bo~qfm#|BO@@Q+nC~&7c2*J*HpOR!mB${f3Og8%Negk7edafCjl67 zYo8+OcjMNkRDV?a!Q^j0IEjoqT(ywYTX{t0GLD7_K#+HI;JF@)< z*$|w+J~yHu5*H~vWu?=_-RJ*^5#x=wLOed86*m%{|GET zXDA_)d52@dO3q57lB|AKZzTFy1V;8F&k2*rgoLW#>v?Q`)&9+Lp_{{LQH&g{9JeF1 zbueLEY9VV26DgG)K1=|&v>AtQmMYZ1X302QZ-HvF<#3sWkl?(yWGeXF#*Fr#aM$eM z>-h0Z-9Y;5NC-qkm5{XNTI`3o zC;BzimOIEL+gXPse=p)OhZal$2M5)L;;K@&pw>h7e)X4kMFJ@n5BriMmk>2=0gXi` zNXOethkOd+P%b7cK@2lZ9fHe$jwpc?!(m{*99JQO9BX6bN1SExUc>ewjx|Hz0j$NU zy`$j*IApbr`Pz7|9r0}9X}>e1L8aQ=<<4f`3SH_d>VMajP@$2sT;Y2l4vEJWrE85` zx>i4}SIWx#l&Sg2+0WLqn-MsrS4zLUlwD(w@ec#%R+dp*pQqRQi;9k#5d1y5s#SV7ojj?ieNxfkg5a;g$tA=rPAF?Wf6?-~18eH` zRW+T4eru`m6I0`+|8#X-Q&rOdl)DXx-}pqgz~K0lU87P~j!NFVI@h-*&-cXOz@fO5 zsj!M%1xNoghSbc?C_YzMLyq^~DBU6<-W&=u#!11)P@A?gH!!02nBdPmF}3Gp1f)kK znf%X!UF0NB%qG}~6!-UTf{n=S{k^mB->r{(Tego0=HhqTCj@WmUsiqx3`CBGtE9&t z%(^e#cVAwh`l$En@7e|VV*-lVHV!HX?#)SVzX!ExgC+%A;0t+MZ*R-MNx@3|I8B)R zk2p6mdl5eKJGagOqs28~Z}jhTpg&9H{&HI2UEvKfpijrRo;}-$ua;^}G6}+RAdl2cySvDL3$wmu9M;v;*@Ga+!1&-Y|#kseV^@CBNOHkiBs0i zOJOv1qtZSekJr2gH_#!W5gU~HO%lbW*~2rI3&>F1Zvw1FfFwLgyC-}`-%RLj+*Ob{ z*^eOn;CR94@sBP@#-7;)+k%T_#urJj>?fJ2aI0J++mO|sxDhAw*7sX${CCi6j>Vou z3!|LuP?N~thL}XcQ7*a)va5+Qmm(>wi#o?0wr@!^24nGmSu@fjs%8ZA83iRq@E|oK z$FLwIG-`Fk4R{|>BcJy@o_%yeFNP`*Oby|<*%9c9Etm7a?L%)SDOt{I@!Ur)f&x;B zymZv>7(a~v4R%$UA#_{v+4N~EKN-`BmXP)+%sI{g<&bdEr*^NhKo3h?#isP+@4_R* zK$(BSDIyJFj=6;D2_Z4v-|#HC958T)AsjI^x&le3FB-Lg;wH8NmDz8@7snw*ekHk* zH8@2v%`R~npd9yw52ZcyYZ#PJ<~jmQL$Hd>b>#dFyaRdFWh9_x9ai1ge6GI4HI$h^ zXuKwJ>BI+e)U0ouZBp{>vuJ3?^#x1CpLd=t{3IGq zr4^+7D|976?H4IBZK6Cm{vt?5U=4gG+($zHRKB*-Vu@eu<$Q_(8zqyEd?Q)o+vk=- ztYW%3$_Nc3+jVb|_am+fQ^&y+1e>^gGVMc}v6gF1CDDfFjl7|SMf&nD9hWiRDJ}qW zE(j7R>vzXx8ao$LN?DW6>O=8q?yj+RPDY4DDnwGta1;1Zrd2JH3et?C6^v10%W#TM z&Rf`m$j#Si;E16mzRYc8GpN>)PXW4(Mybufe2PM;anCyknJT$ven5!(?IHlqAZkB+ zm*PM{1sg6!74_U$&M@o+O6AMRE>V-B*Iw~_@N4bjKXNxZWM{0c8?>Tn4XV)w#=kKz zZu*t0>-w#&JNB1}+0W$qhGYb~_e$9{CS~QA!nTgKD(gpL@C&`0oU0 zE6pgr+~?U-?)m$8|1fw$-kq6s_b=CUZvOEkke|NE#cL(f#$WbfYh9et_NRfdyDwbb z)PNtK2H}PuzaANvTDh?4L2vx3Wec0`^TwwxyT9qfk+G@h$o3w7ci+Nb&9d}Bb+-w@ zD{>Q4f18w1Gdr`b&+4l_OLz$dY~2#es0~|3`*?rpPG8dmZ+yr3`0*`Y)A{)EH2nCw zujw@WIHae4U1Da}l}s=gZWTPRv?>Lu;Bd!3AXnpy4BsklyDFLT*`sM1DPS zV4s<579VOS?~?bI$9*d7{SzN4N@88$*(?f#wc{l4R#7^!k$VYwD1;MeD;bg1A4C^5 z{-`8dke(onfTY?NAN5``)~EQv3B>ek=)d@F>_i@v*mj79NLuO_72tAWFlt-Pp8A^R z;-OPEByH!VV`~@M(lQjNRng}+7N}ugCHxC_S)$Q6XX$(m4-K#dOj&-hyXWJcSJhiwY`sM@bDFCz!Pg+p%Ge}TV>|a}f=QPVLB%4~V{a^v)O`*PzV6JvX@$g4x`O(Ni2Bg!w(3pm#TM^@1eDNov z*dC)dtXhQCW-8ZZkt-KAArU*T5j-|g2$+9B$P}DWhXje6j7SdK^3^OOF$5e|q|p8; z+;wPw4lh!>h9Xm|NbNtQ6Op3L$dadsI+5mcuH>$EiB%LB-gxDNwx3VoJ?~-s_zV7# zxvWPcKwyE=z@JqsvPcE48b~A|lm^%ljafr~uU_MQW<^by%b;9lORS(Eoj|hW&vM=j zt5mm=D}1_^dAVg4P)bJ;w{2+G5B+D(0ZQqp=zx?`M^1UWeSd~AcQ2!c%b}DieQ7!w z0l7d<#GGXD(c0XD)E!InNqA|jh=NwKbAF#~LYfsGc8R3?B6`CeZ%Pj%d|g}YYt3VJ zCMlh9y0#s1f;NWWoZV`z8XnztsBs3{4(V7*J;t&tF&2~3>TeEhx>P*q>er6HxY{!3 zQv-~=r4z1LMx6Fch24vj`>G-OR`W^EKXnLLs+_-VXm&wiD>yl(sj2J&m@VTF4SfaP z)31RZU9q-UGDRdW%N`AA0f*d#U<$O_;dJP@QTh?jH8?f;AmP~z8>FBP>+Y!@b_G$% z0SEU(kF}U#u>APYHq{J33T#-T8dr5c9JdGsYvTD=*&T~nR_FiCBAEnv?10@0QlS5g z7-K_T3WqTAY!fv3;x}A-FVhowV1PCG9WzxA6-hIvU&e~cP$h(Z4GO8{nM+tpC9)F^ zodm<|DLcU-HJ(@V9o7;~1sJjcC{+RLRXc1kRF{#U^Jiq=_4O~M6x2z22|?MWz`2hAU}NLS?Oe5k3DjazKY=zbm-~Pd*D;eE0G(914GoY zj(8=emiWSiP;S@wamwUPV65CNLGb1Wa6hr*HwEc5M1&Gjs4jW6azZ+*nk^@LQTjE-?!|0a9GWA$_d^4g{l?&m2=4vc zzPe;A={BP&A*)wSwD8+|7NkqPgi++@$OEh;!~AS8E95%DUpH4E*n(|eYL2ZoI>+{9 zRF17SjAQ%D;rOB1gND`j>NPZb@OW=^Zovz0;UST+4i%4%<1D_lfEbK{Qqp&wDLk+e?D=ad6)c2IUY zoMrl6Oa zdEXz-y!)mFRek2Yd)mo$qu+SzF^-QB=i$S?jTaqZ&5VL}wMd2Bd>b}z`n;7e6! zIMbm+Y0Kfy0jeK;MTF!s^@D0Ft}b>?5c_wdf~;wSjSszwqR|#wR2LInAv-yTA`}Br zByWPc1J9!@^6ap9Sk7qyKCmdjddx*Brb;w-{L?w(LI>Ku_=HVqs^02S&u4HeB4### zOM=!+EJ}ttg!U4c@KlRy3i3>` zzUHMAc*gu7W6lcmebl3AKTpSKV)ly%pxLeGjk-nRB96rsf12N)C-L`%%Z8aK53$*p zyftUd7A&*D&T69=MdmO3mqRC+!#c@Fe=LjtwAc^l#SirT zHP83n;J~#|LuRL}%ue3?>s()8aNs;xOrDoA^}O!>@i3CSBQxucjN&mqPhPp_yz&0M zP(%I-#b{7R-gsfro`T@7xN-K`Fp^|-WYM17SgDhgc9FFsy@A`}Qrfo+jO&mM*ocMe zy2C$Jw$-AtZC1B2ZBi(?2|~#>@{1gC)R?xKrRjl9odv-@!Y^_!)R7~5`diUJ`|4%h z9Kphs*cz}Qpl2@34C0238d*oAR`84giBtsDepz{0n(+3|flhMRdF;CJ;XAi31;rCh zCrPwb6mS115boCXGnB@WpHn5~WA+P&9zOumil56cb}e@%qpRj(s4=5?`!jGw=GVHc z5LD{dSP?AHrg%JvRa9YI%~D_`t0i4{c^c=~;oE3t&iGl!r5d}FszrR(_LCFYue0R% zin7KY{6R|6D#~WW;zv}4al<&IerSY+vX8`;`ZYRw17ut4NH|d^1SKbxJ-VxP$DfHT zNhN`QOA`voyQdZiU1ah>-FxlOC1Xc$gGUaoFH_mEAe7LV%JFqJ>B87)B#N33(Kd!c3d2H|RqRw;B_*$6&aDr0b1txNY7kLi zEL}G>uy7EKM3k8cw-Oa~IWxtgbPA_P{g*3&L2jPJ*lDa_3j;8n(MDVJtTA-aeuh+E zHX4qcyjX9ew`ltpMD~Gp;qVGHwd-SS=UHmY++Q0n%yni&b2LQY#54JD&50E~Z*@LPq3$=16&O*9>| zGEr4U;woqHK+E}kz^z$n%Q!_ZPGl~@;~ zA-r3#3(k#%VBzvq``Z-q9kgdYETcaGBEZOuu<`+h=pBbdH5Ha<7kGg6Q`r9rq?JLX zPWF5N?|`>Z!Ie?9CZ|5`sj{XIzj0h*#B1feTgC_-sq30xwBpwlz?@f;rLF22*DGlv zGFj39KYU?P=UZjUh7Z9{19O??f~&+MoDds?tqKhp@3eoq7$v9v)uwTpKw)kzjEYPpTEvzBG^q^)X&V-K{$96;}>qW^MT&9Q&*`F2g6X zp+P-uvGP!QDZ+jR5u~Ph4f6$~3j`KTRRUpQ6jcd88q7(oWz#l%dfG~I%bCNlBV?KB z5^XB=M07lpf^QC|dDZH{e6!5ixFF<_R;DX-QBkjvQxGK=>au!Mqve(XT5SjQV0+=m z18ATBc!g~J{}TZ1!LjY551@ptS8n2HDrMV`Qnvh_{$WX(ZF2$3gif*qEwXpO$^U4d zzxw0cU<$D0B^!49^Wmb8my@FKhGlVcdUJ&T`O2bwd}#Z!xEwIa!Gq+2pYu+)on%K* zct=6oGr}u!;eh0lX9ts#aM$@ykN)2o{$}pC7Qu7m?lYr+v<*8B?Tk{*_Vt?4f0oKY zC%>hj()!VV9p6eKNp0<@M3cyM(QjbsUm3KTZL428&L6w=cwr?H%ERf`!Wv&L3K9{< zZsRxLOBMA6GAft!qfNZX8<2!a&@a;4P!NWJG_Y;$siYFSD3%GgO&)B&hu2ef(r21~ zVF&k}GoDC)djY=QrhQ+UL@Cw|*Z$@jyx(rmUUa9U9&FVEI9lp}zklgmp%JS+U+&$& zCdV8d{NN;4wLga6tqGNa2rN0VwmMNzWe-Cr9SI9bbNp=I>O?Ju{f5g=$;hJdy8or! zf6O+D7b8<@gst6VrY1I=(wU`=pC&?zR9(uW5(NhnAlN7l-;fP$M zL@q}N%G}GWFeZXrc7we@5~Mq^>JUx$;mU?zW3mRfO`K(+bWFs={X$V9j7DTy zMoo5=rlFJpvsO-E7JB|Ds3Z-!vOy##5oDO zdEW1O z$K{BgY$a%*U8v1SEau7;z}@-)F;|fb-R>rU5kV(;>v|A%3@_cyNQ9-o!Db{;_=k+# z4W|)t4F_biP{@dZXFh(A7%5t@olKp2jg^JG^JTG@Wcr?3a)dj+N%Z!$%yl_=FOGbz zLv%`KpE?&3jtkIA64*>^goklvZ9F)70zAXt3SEEMQjd%8uI3zRa+hoOC z+Oe|EE&uNSnvr0ajzR+o3D}QqMj|>q84`FByV-q24520tfrT?C!7;ls261{EEwhob z=Kwv7Lz5s7m1CkfB{Z6luj|R~pP2!&``^)Z48)U>A`FhI z#RWAAa}aJrExDKq_hi>s9iv#7*$yW{QJc<(8@>?^LFFg}rh|U#7LMW+OiwWv^-wwH zs<10Dgm)^8gAfw5lsN;UluyWI(`AkG65~p#GhlmDm0%Lb*Mu|_$Lun$Le^<`0u)rl z+i*8W#}niuOMWRHCI$_UW5j?Ll#|h&Mdy8PQN1a#R?uT?pKIb~fnzf>%=$J+YRfey zCXNkq?{3?U6h1}yULCvJ!%a-;#?JSCpCqlte#B zox)I{oTv8GagK-CKB;8mW-P97YN{ucq)t_57{x*f^Y1AxK0qbb0G^t+|S{}l^{@#fjvip*YEI$SlSWIUkO(l1M3qsrkmRQk z5vHX@)k1>VG9WB0%iL8Ci$<}fFEp0CLEQQFt=p>!nnu(@l8iBwWBXFkd(5#xl^JRf z!W)q0*lMG5Y!XJ-gJatzIX2-tQla|WBek&fmtgLlUe(}#JnkXCgaqgLhhHURg z^&`>!nDt6(&ut@@{WROcxtCUSK}t3dRo&5>I#0s}Zj4(vz~f>ynLC!MQQmj#Th9qr z4m))F#w4ds&4?lZ^^oR{7%MyF{4#FajLImJS4 zWa#fSB$*XC7~8o`DL0M8TmG-v(4TOBgqY#cA6!jk^=xJ`BE(B16ev(?%SycKu~IM$ zhp3>AEemvv{2rP_(v8s3!0xQ=>oSmmx=%rJw2K_Get{=*x+aI-FrtOGuP*D(X2gQ_xP~RGeCA z@HRq(tuCv#0D1{`rfM|7$T*zfE8bTGFYUYk$h=+)q^LtOT)N1NE%8AX9qiarRByF!bCE)B*wNPV;gF6|6Cx05 z*f?uH_^3_Ums#Xm;)fuq1OP+>ZS<*`w`WD^BR{7PE>k#8stZS97E(Imkko)VDK6f- zaHu_X5>9GaatP8cdWixy(p}abolom69T}Y%S8F-^*T#A9;RTwLYd*id0*z2%0`FQG zc_)5TE|Oc}BDwfQ1xCZ_*H9G|)kU(Qe6BZA`P%;%%tu}bH~4Ri+Cp2Uy*$(}pQR z#~bm&;T7ja>gihFc-_-%RkVaYc92XZU)spg$BY|>G#5U%l#M}f1Ax$?#lmcl&v|Zh z^Z{w-O2J8m|6a+d2nia=4l7GfBI-1$(vtE5DS?t*oiQ7cEvNX>0uhF!yWO53G}w~( zI9yO>>*~xLltnn{G`e~xj$MlSo~jzig~Y-tUP3>BTE-YIbyWD;bc&*r6!GB5tk(!b z?C@Aw!96O4!_#T}7&2JXOmrCF5?le;VX-{e!7deKV!5C?D(pixhFODOCKs%04i7Wy zl5 zj6jm_1p_qI#N9r+RjlLWtfcjv^5ikHMZ!lUV%j16BHSF(wmv{kNz#T87EU|`{0MxX zYO8c2sEk!+oD6fX62yduGpERgsd;Lcw^5++D6PgeWl?CE72fy8tWN>yIN7fRl1!;m z4Mk@nJ%-9KN)U$cd}`T&CKDI2A~I|0CTUwQUPR?qY=mW4>exucz&Pquqis2ueDMJ@ zfCkir36GxRYNwOqi`SU;9d4-)Ib%Eiaj2oC)Hm6?X70^C&;1pidkXyJXunpTnN^-q ze7(1n=pJ*QP0UmJthSI?1{M`G3M;$-ojHR6?MBK?&$Qm-IuSf zYp-g$98QvH@iWunR$aEb?#nfGLEqb3emuYb;0^3PQY!ua??Gp_hjRneLtD0FE1ii3 zYiD@_d2uP5Q-=li%ud|AA~*03{;SFjY@JQ5xwDt&1~$x2&OV`M^M9R2*wDnOKh)^# z{|8PZw1fbKwargYE*VIu6{aSn@1*>Z({56|ewXrP(O_ETy2RX%cSICF=9ESTYBIxH zk#IPcVy)4t6I)x>id^2IAmSXhhUA)Zv3BwMN2q+`p~y9CjbwN?}La2cO=v9;H)y4+erujeMz!Mpp%wKOaV{*m+Kg|GNx4FucKQ485r zwS!Ks+98>h{+4#yhP4xEOLYwwl7cPv9h5{Ud|)EV9Io&+8Lr6!0=L>Vz%}z$gxx8B zxJ9mEXBAtMOEdv;O*oQ-hQIQ)AlGP%2(MV>qNSijnj_JDyz;yz|O&%pSgRVSV)teTgdJI81U)ZsJjp zmlvxfw=t}U-F9Nk$D6vSe(Y?%mMn_wF`*Pz36-9M)>R{h2^yQv;XA%Ao$|;&hdIB@ z0jSB|6$%Ju3L!1u?WIy2VGoPWBN+VLRUPkL6o$k5C2A>P~ISvb(>$ekgY$XS5I7F``a4S*9_onrOLtRKZ zE8HGPOxj4$SD;~C0cmt~ppF;#AOa;Tz$P)AdX1aI#If6|zu+Ie^y^F zpUvxGcy-Z4BN|ow`D_|8iu&|RtV#%j*KIe9wrV5uVD43RD)J3Sjbf3aXM9o#h+-3Q-``kUUW$%lfF&8A2eerZm{KP@=!w1EsqB~pa+PeNLnjQq) z8W=xwO^-mIUMW+@CTG{=`Ud3tj?W0}hjIUylwG;W*-z*CV%{B+wIHMT3ZLiDc>fiB zGqXxEYvyMZ|J3Jsz1*|4=h(Y`JWKzEf0bej{!f2z0vc%l67<`4xILmL+a~m6n+9Kz ziGvHMA@?b{zj?bPZB9}~TTFUv+7$f%by{4>+U|k-A$`mq?@!yC8%%}t@r|mc8__R& zac-axo+I(Fuw3`vjjb1D{fa4VNlsLK7#P9_Y^`&(VYW49ALqLqi4Wv-Z15tn`4KQl=P4cgXX{I`rPK!|xkgsv*aK zY!@|KyDY5*)VjoWhN?CkLi231n$ljc#>OI!H;0LzQcrGN3!_HqX(Hnvd|0s^ksOp( zUSUVVX_qk4jqU=QLYB&YcWsxa3s=t5!KBHy*@{$y%#o_}XyI_;G4&b-lx)cP!5gDmLAMrsnO6?o=^Pd;Z&%ZIcpZ`3gpMU;cc_M9%PS%AxGBMOu z!R6M_&pJsAeO2&r9GZ-*R_bNx&Eq&^+D%xE_o6E;xHf%CEO@MU;)iZd8&4sxD9r2<#Ou%J@GBkZbp(97UU;fQdlVTN0AvXG6IM^i~ z132YcDLwP;g;KQCtH9bd8bfKA8WItubuegB{t@ zO1~3;@*q8GVJHNL_~fp1Si*3Q!34ExhF3r_#VuOI(uTIJFc!?rEbp=>$o6-$TW!Z?#t%DfjJh-yjH zc=PLFmPB%a;mV`68aJ6rPPqNEAIw>>f6B*uKY+`9=h+tHJYdYu)! z3JuzlS2X$1f;&C_jr6$bSFEm!U0c^LxSi+jh+fsg;NNkyL&1dG9}z4bud_Nt~&xTp5a{&A}=UR}3+P2F~1)|N+d zefQ-B8vFa30%&n9+q6-i?c4GaH$RvgNCO2o>zFaYNuHQZ4HJUJ{k@w2;coBmorV8y zecao!eT+>vX*VCSIsD%x3)`~4mw)8ezs{d4n`4{XY%l5EJBYffq3siUvVVOwoF%Vi zv1D`4^D^468_zPpc_&KPt01# zurZHcXDymov*?7k4*i|zBX`wgS>BOvEYn98!i1r@=If}wiv(o2lGG^BYXWm0v6V<{ z!(sS_&_`2{jspX&uPj@91{>yw*-vV(65^%hW%WCqby}&u>c+Lx(VcO`pAM!ycMst{ zCyvvOj_A?WINk&G7l}lixTm|)!&YOQ+qC&I{&C6c^i z5LgrgPb(ATnVl~``Z|0AO`l3OG{@H7vha=eJN%#RRzq{H@>|Ildp^RI#us0f4rVT` zw`sYUf59sF9ZEJXRauc_-`#ZTBm@00L&u)N#Nt(Rn!c zx^2i;nJCg4?7SNphz8%Y9^fuN|6i3m6qoB;y~BuY>eD2k}9ggT(D{eY?%L=x{M zkYJS%t%8aK(25F{RI7Zg*l>dt>x32uY(GgNDh_CZ6A?o4d)L~-8SXyk#8&$MKabBN z=Wsap-hKAjd%f#@*SkQn1$DH-=a9rDHlZm)xaOI?q$S~h+lu6c{YHTZi98jF?36rd zyp4E#gISrP_`CYQUryy4)lWq@ob~~DDD!A(T;ZR1e;}Y7i(CwbLdcCu4-fdp2QElniQ#eL;C9X|h!X=e1R=Q5McY-V93Hh?3Ba*sJI3OdD~DkF3~ zrGp@}avCuQVq%p5hRft1>``b%aK@^gjGd!m+5+a03xZ!n$AwaAU11%S4(u3XlLU2I zAd75Piy^HeJ5$)?mKJ2wBS@zpl@!JTZlI`-(U63hI)?&)3c1w7U^_xB9yQD-oRT12 zaY1(MClN5JvGmH|;YET@T$JykebUK@=;Jjnkkd%Ou;Lzv)lfmBm*xx`Upv3)oHsIR z`GQ1}Yv%qso8B*fktBrQPaBsMMU2+nfrip%@>p8(Qpy0}`MqH9JWcb$<`=-w?k-mV zcngyQgU<*YU+_%Nbxk&@fdNd53obODOm{+6Av@`6=OEpUk3y(p{nq;MqvtlVhh zv5}_Lp`nq;=fm%xsj9Uja@(geOlyn6IP$$zLK%BjE}DFsq=|;kXCSs~ zYv^-9>6MQkdOJ?7ZXV1fRq#0^T$0uM)hq^dDSSLWyY~&N7k%AM*yf$3i62vG-13r? z`%6+8+ngm)MDs+L$ACFum49Nx-m+T-NR6#tV-5pf^-egI z7+{|T&IFm|BmogrgD5cYT|=ZRr3n^csU`b;YC&g%Qap6JGeFzOEJWk7xxuvf zxEjKP9#;QUjYwF+$>Zr`6tvay*HS1@hJ5r*`2p~n0QT)!U-qprkbT>?B`;DXO(K(f zB;p~_bILeR+C>)hYstL{@~+A>UT!^Pl2~qc{au5@3spk6nMZf!MflE1xX0Gs#ib+F z_S{{%`4_CH)!o?01nexC*YzUcDBPxTo5auIY>mmLs2eZqvxfS?d_6nYLBXw=>+Z&e zPn&wJ)U(vInhgIP^qE->g;3ynEYuJ`Q3l&6kib6A^s$iw5=0Dzej;XDl1ZMlfaIRBab3@@6jlg?tcTPZ zW&Y%uPklh!cotb|Hq>5od~?fFT)0+#|GgFYubrqBcc@Sc=HKL2t|Kyg&`d>I)A3ox zh|UTb@Hqk|+d~HVxaiq=dzU4=p!8oL#>`?dUZ^F9rwcS3xIqsxs6q<4?Wu?+qUAO-~-~i3J^VCNv+rs#sIk%Z$@1xP;G( z`Hi*+PSLB*cin}YmAGyjj9yIIZQD4n!6o#1`_RYs-pu+%cZBnTr3Rx^bifOqBBN#b z*XtQ5XQa4m7perYFQMfh;ihq+q_UL*qSZHMl_rNH znI9LoKHV=Nqi|KzAV?+`0!VJ3HK$S9FM1h9F6h?&oq%bxiAwTl%XZ}2Hpx~+zsWvz z$EbmQ^9N4N=X$DxS49pdr;H=VeosdDHwbi=bXv40YX77d)rP9%{F`4~1Lt2;RM(OeHBBm1c*CVY2rx5i7&Kp{#JiK{F#68%rC zS~$j`^7@J?wR@FMk@F+;^;DbDxql83K%7#RxDdq``qP2dRF2!JUgU5=p28*dx3F?j z(`{DEYd%%qon!N(g^fP|8-G(HU=tr_zrBFPmVOfWktCo-PZWZ6LRyjzX~`)HDXlD2 zNa=z78G&1M-#ZfE1jPlJpj%Wn-d zj{MYU9GUux)tjy3t2=o^O4cgL`_%5UjZVcwC$=Cdchh4?ofkDcC>wC+=B*P6_o9YZ zlBmD)pCZnhwqkcp8_6!UO7bBkoLqT1Z+?FyK+xq3Spt`N*p^I-x6n!GSQ5=&O$iUu}}+GzeFr7dCzX9Pc9MW0^HnXTZBi6`I8`FK$*X$w9Q zIFoI|FdQ_SGZ}IMql^q_S*%A^6%*>J{_<^l(}@;uQVD8gXzD&<#Gv2&S#P z18xG}A+_c@0fiV##8Lpt0umc$Bu*iLz$JVJx<_GUTi-_>f>>S$UsX7Y=t&w&u?wK= zc(}j(pkumTiAwsQsd8t>!Kca|(i4Y<0qFsIhJgIw53Jp5)QstIChz%{Cwr(~jZ!LR zjlQ^$8a=lx%yjkgxoIR#&g~Mh!or-FbkU!xpsO1H-S_=x;cxakff_@D}ZC-Q&)9< zqV9=q9jnS?jaRHGP5hh7KSZx{_N9+nX*2$JP-P+?Aj>d?PYEaXDBbIPx5bu5)LKBE z4Afd|o=l3kc>+5Og;;!~mRmjbepsID?H2@lYz>MSr)F%%Ef3yo7hgK3qZCVYj;egg zzoYBHdE08ssmSf)Ls#Us@6b8@H~hFoZlzPl!pZ;k+_qD)5|Xzkz@;R;J!#@8cUJx! zP1+7mXxMUMBvjuqAUffutkN{J&fZ!Etmh3uX z#rufinS}8wacepZj8;j*wwu@_-`nM1aaKO`8gW*UBN>2A9X)XQXohv!e7tQzbe37)@jikh8;IB?7-f&x=iXfB}5>Y!7~p?gip+4jXISA1zL9>bb&KpE0RNvo!#ynE`MO zDL*LyZCzK8wFE!1?Hag`97paWhsKCxeeFhX!$^=n`{$-($ctZKtjashX{y4IdufBV zfjX=xZ_fZI9T%5QEr#De#UXotxDm;QkNqh&WzoN35CSjW;BOaLtfuNqo74Zk`uUd$ zZR8^ggk7O=*;<@RSJ3*$a3bjw2-{>f*+`vXCaScywAMB)(K;jT2iuG}cq?_Rf@>?7 z)_W}Pd<@&PVYGcOv$!$9HrjxVXkzhH-tch=0%M^%vBFqKMpCj>v74R++GOUt{N}2P z{B!v0wKr#{b$pe=1_L%H5_}y22Ta3eD~)nTts1e}JqNN##2KrH5~5Cah;`=bbzYd> zS|p`)pli#e^8;Wfl160vskkbDU85-5OL&KX>H*6S%$6p%#7Rn6}EktQ=i4*gpMVw2`5fD7-p?raz7=-IbD9K^rzi$puMLvc6_n+qUwzSxGP{-o8!zmKPZcvpQiL*TCFWXHb?pBMB z6X58eEIDxMiuAUgW&9p6GRvY)awMSygT!AoHkTkqM=CXA^BVejVPg{lNxd52QHO*} zK0hUPG7Lb&XthjI&Dasqr{E($O%H1-Dc52Yz6LjXff9Md7@7$#%WuxI2CQm_*HrOG z!*W)#5vcdDl1-;{46-E^#DH28RvG&_UL5f#`8iHKSM-%LBpFi9p3M6fFfjy)jh$23 zFUWeu6P+0c(no^V5r6NoNZO~BoWQjEstKzp zbccV@LNXQge%Hx*1TeZjfc}AfhV&r+gl zJ(hCE)W<-B0v|iTuN@*&dhEZU6zTULD@CFw?%%hvz9Z?*oPi^plI?{TI8a1z)wJDh&4T*+M^J|zrrFQoOl3V&{Am6Us zPY{49AO|+nryd*iUaduL$v0c!sT7i5;)O1?oJ=(CvMSPWYRr~n7@Ecuzvb#;I6+Z4 zu_6L%V%TKMY~e7XWr$YMm_7w%B7F>HZGjPGPLPgS%Rq%`87YX!FGwIbOU>Ap`=}rZ zSJL8)Z(tK$>SIz|Qa^I7u(Yvj{MC4uUpWRkk%cN{eWNb?s}iRuS`O;QR-b`xY(5|@ z=lOfyfLmTN6{0O}y;ujc;`mB&OeokNqCo|@eOF@crj=^Ax=G}f^Q;miyu6?Tu4J}o-)T`+Meg|gQH~AE8Dc;LiQ61c zh$gmDjYmQI`okwJRqgnc8xRE&ItbvXj5cCDh=!0tW<|T!-QH9Gns` z`>=&r$w|{yRXEA;){xbrlxz<&G(33;kw`N<9gUhHO4}%c zYx}Dsb=mxtd^$VpIk*P4pF-qCuQ|rfJ$yJ8U7O|P_VDbaeq7#_lLYTjt`-b~0j0S^ zq%CLSc(IYU3S1dfm~VatpFRU<`ES{AimZ{$FM#7R76{i=k+Pb9FE053cAOr=4~t8e zQEx82%61Z2wLZ7r3jI3CxyyDw62saji?#Hs7uqA~#pwmS(B3#0gR_Ip_&H*5roZ&s zp{XUq+e3Q<+O64bg^ZamLQkyuG=bYHwgzM6Vh{*k!BR`FG2@IfNiEBHNp@_mvaJOz zA58MXEX-j1WGw)Am3IbAT002#sH`CGl;8?WZYb%Qc`WGzGHqLZ+`|cDN)MT-|Ikf` z&eZ?Y*Vu5R<72d48#uA`=*q0ppM)dzXXi$1kBc-t3BQpY=%D?pgh*xD;L?#7MYitB zDm^|NS&RP-2uCV+<(8h^E7Bc*cjiTrvAeSJ)(tD*2%#r)Aiab=vaACy2tWZCw<58@d;N)PI+tAN+>e@ydrb@ zeYR1SZaFD54bu_^9iZ0G6spC@BvZ)w9%S-P)jv)EDyM{zFxD9>_EP`2SU_lf474N) z{OM16VsRTv&nTe4KV}OLR)6sU?@%!)2`(Siwfy%Qx;fDt%pytE)%x24ru~4_uJc4{ zXz2KW@m~7gkHJiz{=MAu@B?Bh*09lMzJ`*uzijlQ)XHwE&fzulWQDSZ48SpSYr}Ag zbY>j^63FI{=yN+xqiY$l8Q?wlxoDZp6e3@T+SE-h9#b)zh8D=xiA+$1yDGO}`QpYf&n z7gcQKg0|8zy0%QdMV&L~)7@M;MAa zqfcy5HQWxGdjsfzStl(WyC7j6W;EQvggS0}uuyE$t(R67w>@3Rz$`N(r+G+nm})DU zXC}+d+NJ?BU|hx#!dcte=kzu&Pub!0nR^dT=PqsTh!$)&x2I%z+^owPZ=y>|W^+!;0&66j|` zW+3}sJXZLN*@G<&8gkd5a*YIK*A)8;%7|QR8F>X?#4nDMXA>Y6!F>6KSw?h0J*Y6- zG2}E1Hy_5ss8N753&kmWF@^hq2xS;**xWI67F3jz>1Kb%6#xCwZ7{n@oxmj=j2=yxhw^`;`-+>oL{Fmf7`@rv%!N8&r>3r)3tiq?Y_K!b%L0S3u?hwDS>2MfODv) z9A5(9E0$a9W!K9uozg}k?kA~_M<`*obM!5lHD%0dHLmr?#^%?a0}-e@RZX_=3UZ5Z z!L*TbJ2u^;AA?YCBic5639;r9)7D8$+tw?3q%SW@pIVfb_ossJ$fEG@+~`R?()$*s zLqr!D1e1}9+|rV8+&NWorx!=w?K602WN=+LxAgLGT#v^tC^>9g>mTCcKdugKfWt`p z#F7KWt&gCITXOuO38$>DJ#^NPZ*2N+SVN8}|F^6m*KPZ9S;>ctx~^T4I7bj{pDr(f z0VK$p7ndaFqa3nz-8_Jj+D~r>uvzY{g&ZY4K$eeUJhu0$$hV9JcjOQ3OOWl2S-(pr zBB+}newHR)E9^uP`1sMKiO){lGPjJZGJf?^=tf2{L#xKuRWbN@P!U0oWy+E}Ib~un z+KN38ck+~rkx8mUJ*Yf*0{b~ZD< zK(H(9!le4kkp>o#Z5PyJU9oBjby9*?*(xd^ELU^}}ofMXO|gEu_@U7(+IyvjFt*$H#Sd;B>+scseP32IV zUveg;44b_MAK2^>T#GQ;O2X4NuXIieV?||ezaVy4D?y;75j{IJa+lFqswoJNpQ&5h;tg;&CZAd96$R@9xB7o zp{AJz(O$o4%1~4>qBCaYtu=s2W&sebFio({d{1s3rr&8RV7Ctpz!*j4Uf=+Wtiz^+ zspix=o8BaC(N8+8vl!&TxG8s;g+Dt1PSHC8fKwue(A z@v(tULse6fPu&kFcHuc%*k8 zq706OuJxM3?~m~5H!hIYf&eT3e9^kyFt|B?;b}&8|N1uW2`A^ z4sK^U)A9C5j#-Q;0s+fU%m9vX%F8TBZ#s7HEIop(IW5R~AT*HA0rNH>D+YvC&RWRr zhYF!3S9~;b;nK}}R&V~Srs>r9q#H7l#%Cm6`O{UKn`)Z+$0xnoKWY6#hern_rSBM# zzI;U5rd0*u)rH}Q21LJ3OrJVD4Q}D#yG}p8Zgy_zaPobOJ8o=bMsKiJ_%G`bqZSgR zSjYdRgk~N;v2`n4{I5VG?cWtdKSx{ajPB8s64UegcaI*2|MeRbeQtJ&t!4KA{~bn1 zYG|_VZ{G`8aFzgN@%NV;!MJRZI8M61q{lN~BWD0L`~vR!fvBC zAEPhgW1R_Jte7AR?j3aC0+vZK%jJ?O7Exn4++~w>7Y`leN!!{oT_Q@sqiKw`29m3T z^LJ4t77MNv-Inq`vv|xqUh>u}QU(@7ty!ekGK&A{@14||C0J}5xr*dnWvTp=$h)7$ zxa^p!ipC@O>y%|yRZL63A1khePR>$M+Ju#s&=N&-E~%u(Y|3yG7WrdRPW6{Vwg$-| z^8@6NzqobKY9z)+2JveXCJad^WvH-eLW7Dxe$BI%D&-5Z1X;gk5^I>;k(HK8QQEsNDK#Y18%s3^hz7mK zcT_szs#ohgbxuq1tUbfh<2Sz5e)xwSXAk`8$}y=g{pQ8Vmyg_g)5IsPpn^6Mbp3#S zZI%YijvX9^k@iSfHFS3Eq62dma`a}5V zN`JfAOsM*>7mKLE3QKg9L5z~@sjNc~9c5V8L-;3>%=UO(ViyrwcrnPf_7h$e5;Pab zQwWRjPv)sT^{SGgvT0r=O3}lsBN!dASYaN;gs4*0idVVVaulY2HG+L&?-^-$n~)$? zlp1TxWn4CGjZ8fe`xlc@>=Ln(M@M?XjnzT|up>i`yx z{C{KNcx>Q)t65y+60#S;3$5dQ$J;IvWiNv;`s>kKNgiq?eB4tOUMA%|o@u|CZBjwm z(~BIlPAr7jvgM&tf4-OuNDf?D%7!PNpaZF|^7lcASAB`kF>(f66xH~gF{ITCDb^2P zV?!c_N&OiBJCr-0JErEH4=dpmSdS>?$q5K7a+ogh^2m_yG%z1tX!m>+#}Yq43+V#N zX^>Vs<})r)63v*$PSb!ba=OSD%lDXrD(8a;Zt_tV)8s=YD)dW%r6raxmTQKfgurhB zZ0m*5TKn`VRIGvD!bCPq>@=dDS#uc#l!pn`hz%&)b2hXT2>%jF3EP)sDct^{K-rYB zAwMfxTGU4WvvbcpwTvH^d-SOl8+JI?Cg*VK!luYJV4hc@u5I_q`m)X*pxa(MbYSkw zkqd+TB6~;1qD9-ygNNRnTRJ)%S6CT$&e+IqK(^oHmgWJ%%@|j*tGM+yXvx+q{w%a; z8wg(y{8!L-=Xh$#)-|$} z6pgo5l8Gm1CGP{h)b=>c{NG_S|FXKmh6!^>J{Wp#ZF4$!BWAFC$hK*u)ARN9zb%k# z17XKZpvYQ*eR4`2Q!o0C>((SY=0sV-E2LkG{K*t2Fz({bo_zdyvxl|ijP`FdZDzZ_ zRrW??bqvp!LT%)zl@JkLV7D42FK81avLZ%m6shkvS#e@BBN9?jJ9Ny#1>)7_72tK3 zzqdX~G?JT!@!KejBOANtG#?$o0{|u}H?npP1V`dd<5wYb6k2RMOOpRKVMAZ8>Yz*$ZwzFr`hRH8TcmGV?ckQ;m~+)yw58{qsEoz z;~R+egN3d;FA&FAA6o7j$Vw~TD@|(2>yW$J_iVqh*A~UP{1@FcQ%Wb%;<PLH`a24R7#)hUfhK4#|$F8UVmeaE_%3z%l-IJ^^YmBVnH-)n(PjnYRguzW9Q_)Y0F2ZPaT<-_h>=*wZib< z2ShJOO7EMOw&~%5@Sh9A&zzoJS`?1EqB3ss*vQdntbNnqx*Ky#M}^}~LgQ^>-R=tk zvJF93ZGvpYtv3i1*Q;XJk6Y(wqA zIy_}lWkEC@gbS@dky@^r6z>pBEU%F(F3MbDF-2=Hcq|7G~x6@^EFP>>XJ# zLp%H*qgLOobZXWc)yZEc*!z}ZmoZv8|b3i zj0suI-zs35A*xs<1IACTVZhib;_))IZcQ zFiuD;qbQw3w{~|cDo2YD3Iby{H(>;Vkjwo^`WTRyw2+<>p0)?q^JZQ(mG(e?o-M$C z06ZI<`o!QmMpjHm@;;RuX8L|pB7#I~E#1O_u@aOMmbb5a-aDpz4(m<_`& zPiaY*CvFA^y?ZtF|PRuz!LNaZb-0ZdGF<&q<5y>{D7>jiXB?z>Pms&Tw zum(@l5Ok^tFSYlSWL6m9R2Mz)m;g_-%C$IK*x>{A`@%07-72FQ`_ zk{<~;DYfu2e$lHHzYH{QG51qWrRBq?B6Ye@zHj zb%jre&WSM?VoiwK$u-oNI1rCu z(%d)7b!xHTCbdfI=R-@v3JARtoEaJM_tp|}FUiF`)+XLX>+a}ye)qmka`$gOboPcn@F`M7g=$zjktj@dE^uKGdl{^Ta&@9sZOHNNHzI;L!G)>GiNI7+~sWS}bPusCij zcS*&rgL}nSrP)nn8kfJ=;1+BnI|&lSpgar3QM~PD+15O8T|VE1?>c4D25GbF0gJFO z&?R3`2RX{n5}vY?aD1wH-|Zy(ZqE@lY_Rwz(-EV^Kio}JatxP94TV#PiA!d_#-(q( z7sgh&1xMH8bUQz}fx9u1URxNGs^z5*R3|_E$vTZph@p;&jH3v3j zEA?;6Mm1B$+P0iWm4PHK*?u~+W_gb&*34<}4=h(3;7Ec(rU?^HTHac=BBA;u;zV03 z`c-}8QcNC;Om?;QjWjjT#w|cWo|L+`0-Yhrf5SI%eXipeR{GbsU z;^l*+81;tAGoJ!9y`b&k^=yvs$F1RIB6Bc5at(D=)AZjvf-$`4_>U?wM46Wt0^{`p zm$Ybm1NUv{Sgj8hT++Wd?o^dl!1{2&n4Ps|7<=@|&P|%Zkx`onigMKTu`>U3d~|`Z zED#F=c(nR}g@NCC){i(bsRdJoM~N7U+z*T(3r(~0@&4(USkoFhCSIdFO1x$@4I1uo zn1>8ZByI4nN~M>1%y3lz{3Y8?5^e$_do8CFWPnTVJP@99aKt!IjWmSn;KAx^E21z& zi2)16MXLKc(Ol(YKl$eK+}PpKbC?|ZJOBFKqrv7QptwJOcbr~9XvGi}$r@qv*Xq3f zT~ev|jW%9d=>Tb19ytvuN~j8S6GsPP2u8!0sE|e08aF>{sw+*l;Y|`*9Ectj%M@Ft zD&A91?+{GHRA$B37csmK>8# zypA^^*UcFisvbPCvzXdN4A~-B;GD0Fc1mzt>!cE+qE_HIRn4C^6^7& z$EnrLgNdjL=*UK_ls{&>H3+lSdSW)_qzJQp6^Pl&EjvhCp`0DF5sB3-#C!@Jie3V& z$z)CqekeQ^Vv%zKDN5Ju!t3l{zFf$yESc=p#CN5RZ0zEBT1sO5kxcHjkHQ~Q=#MB| zBPC~y#L|*nqVa(6DZz4NvMLcq-APc19HpY5ZP`K%xwth)oFbLT!z}&G`>o1QzM|Rz z4%O6Qi5Pll>SeBv>9X;vrfRc^~)VvbD zHC0?DL6>M7Yi|T+tjpjWIgkY-IEu3ko{~&a*@Tf@M8{SJQICi`#od}Ew#YKeM}S+6 z0L8kgptR&#;e6&CWUw$2lYu)^b({?I$YpFE+4jo%-1h6IP$0?KoXdsPfmd{ZT_63! zuyuPYb5fDs25lvJwrV`##uxaUnO>nbKQvP2;7$U>IT8<9r0wJdlVfGioB~aJCXq!< zVr3E>xpP$1k(IuU$hf%oUMh98rQiW*+XA?XHFiMD>mLDN2ZqD@-IDxU7D1Amr!g!g zIc1P1i74r`zZH~fw?}Mi@IF!(XV(`H;Dt$$u4>AdCUW3hu}+4ly)rX#zO`0}eZdMQ z<3Y!wK>ZrvLC3BOA@=@Jf@$Fu;+{LQg4z1G& z@_iRez{1JFZ7OhyXbb5;!rYZY2_dmPvkXkFt&T-fEw_XWy;8Vhxs^4}o<$Z7(I25v z1(c9q6K0z}_mDaMryN2hwsJtU`o^r%_AR~^dT$@rfex`JFwU36;LC)!PsA@pjn+qdG502jm^P>Ra9OyX+N|6oiEla&dt6n8R{qB1i1?9(7ZMoNd9fd(V@> zGX{lXa$iLbsXn|VQCzr&VA;(F$k2!95+Gc@b8ELYtGL-z+>8#->J4y&0M>Qb=h(hO z(#!bLHVKK^1Uo*Sh_`L{2i2%ti#7gd5tssLUD(>R(96omx`HzRVP0oAmJQSH91FI( z#*!PBdF-mx?=M0 zG@~?2>}88-9k~Y7e{nX#F{^DVUN$Qt`)sFogaj=Du?qWaLOrolO`O80@?EKh#u3Vj$twRD#uVRX8cj_foQ7_c5S4j!rV>9E zmkbF&Y9}V|uU6RB8ZZmn;w4^qGGD_x8I7|A3a!gb5Lm9`M07?j`IA>_Ox8;w(|}6= zaP6qdJ^9wyJG{9^UW{@>tYt}&tzL>k5pp&0%3tskAQIDhm%V5;aMXC~3+PE3*+rYc z+Ay$m3Svl+IM7_MZOC{pHFCoO+3+jF#cHy$*|^&vB7$5Jek58(>zH;$L^GH4>CdW6 z_Z-v))46lr?+4>KuhB@h7}H|77;= zBqL1kFo2)dXA8;>1f;|ugWSL}$d3=1406oALnnjm`CpMiHni9MHY+;hu+eRGcVtC3 zbQZVv?w_!-XjRkh;c4x|x<~e<3~rmgB$WS3xam}2x4XMVCnu!u7L0G21i=_r{Ax+HI#s_cPYpli9mx7Czy+VJ{CM?BVO|kM;m1ku$H3 z3}>5+zC{D47BK^KAs-|5e@pUIa$13>vQb_jcF~xPrHw|2#ttH>OVm%M41}V?bayqf zUtDy08Cp&I%OGh`-0;k53CEP$%-}LLD$K(!6+^f2skV_QQ`=~jsWD9UC{xQfl5@Ij zjmp%>xYGau1rf*gL3s-gThr-Nr$z#&ZT_s9OpNWFo*BmYytI?8}w@Rd{(SJ&MMv@W9&XEsu;;~Tsh5TMzR@87K zol>_6@E18gFLP|ef!R5Ef1rQe(EB%8i`)uKP!TBrfvL^c2Ak}g3Nd8HG&eT)8ctlZmQshw zdW+&{Uxiddh1XXs{7``f@>_zoa+yYwCnOb^l+WWm_=l8Em`4qP+rXt?&8G!HiE@AP2h?%nCj6%N)KU5A$X&ABaJO z1b_@o?K^p=B5OtF1}YrJhbePT{oL?n9`=R5b$5VCtA+!hAK8Pp9{Hl6N~W;g2mxaR zUq}RpV(L2Pt$NLW)dRrC$R%5B4{j3Zd!fi?xv4lKd(6brquiH9B$2QjJNZy5WN*iA8Jj!;)bvmnusbKrFlO9j2fwrMGz!k7;{ z-1+o702O3#LIE*1N7vrNO2m|LSS1SZuco5b7P4iWKZ%wfJFH?l^P^f%m`Voau6MQS zK&7&2fz5q=`E4Y%Fuv_nxm41Jnz(iR{2ZVDj5LqwR+kQJbLuE<)l!^=`x6<09akA_ zIdtw2)uYZWt`m7clv>u`lE5xy-bfj-L*wU;FqiSFm-0BZVrs0+m`b*~=QF$9+)$zxyY?dF^m zHw+0Z3ltl=dt1^QJR6Y6<}6{rUKPIDRvH-n2P5Ht+%DkS)=p}ped{43)#{#KJ2ZFb zyuo$1=9Z2R$30jXchA_!G6*0`b4$mD<8G{s`_+swtyOXH&-RLc6ec1NM1C@{WOs4v zZE^8amV~OwRODoIv`)P8BB6e)X?jlzA``Ygmd7>Hn=OLe*PGx1JvtKLP(cFD^cL>$ukc=@&}Qo!I(I{N0Oy zV@m+X4j9WKO~Vzs*c6ThlFi4_apDjg4gP zE@;hw)5xneO*25JEh>mk_)S)65)3WZzCB~#zK2FIQbdHcM-ZXyb&8*(L zYHLL;8b`G4OS}MnTW$CEVp^)~iNc2Jo7s~ibI;P&d-^kj^yT&6p3pJB|Ggw91Bc|r zu1(4OsIl@Eyt;likyC1SH=g|vUF6HKb&fikIzq-kCkw5CW7~K9t%SJl`bPQ*^ zJtdLYos&;o+PVf2L~7zp-H&a(iYSJh)Ds{&rls~?S4-_9q3{~37kLj~NZpspQkUDa zl4RG(YNml7c$Gjo*dLAg99tcUAQeAsTbk|RFo>vXGST+g-5lJQ9kf}wT)L#=?c1H= zG9AMKLlbL39h!sO$%Gw=A<9$xCS;K48c&gS+q%Ns_V=jN5`;qm9`o z?TgPrIHOhvXF$z2B?<>&ynzNvq2Pn(5;Mql4Wl(4BCOyqw0*W>ES`j3n#czj`jU}@n5gkAWnaSuwgIH@l7N>v%gG#H=h}wM< zmSd}=5Z;AW1HnQw9Gwp2d+5QO;SxX1&b5nfc5u@6$~LjZ z>ztQ6rQ^7Xx$uHDokr4M-wY8+EtNj|lv8{|&9#s2du~d@p>z9hK6I_y#{b9!oJa4r z3<>Cds7zAvWC?OI%oV%2@Tzd&@%5XhwuXm5_vH%|v#84&8u z+7YB&l|_Q`SS#@ylN=XA41Y{>&t_mTsX8M+ProWF>_;FA&B7F!&#b5)F7+?QaDEq|4b30iY76adY*Zm;WQeJ@= zTMo>H^u+7=@k2mUHgs8R$h;<=9j|?8xTm!UOvkNmL~_O}vOy%pc?$O0!e=rtXn=kT zMrQ`3qJzOTkhZZ>%!h|N5qB}hx>my_T(Znq=V$SDT<7OY!!1`Wo~+j5?&u%7I zrVXJXhs{&Z1%oLur}2klEWlt21h9BH2gZis!6oXQRq>mSyJRTVCG{x<_(nX*PxC*_ zu}ikt^2m%8NQ+kj>eHD3ls0x#rhjzo2cRw(W_tpJ`bR5xGWA>(OX=EF_gIK^7=DF< za`c?D(D~_MRlS|<1O#R!Ra|j7l@;EK$k{r^=ZMhT5PfT;fmd2Ik~E@M(T+_ttkQ9* z+XOZ!b4Dz5*GjIQ4A7fVQ#X>8#K=BnN(_rDY&Oe255RF`i=rrYYq&J0VAn)>r3-%#Q#ZCy`;X0NRe^u+_M!WAP>U zp6g*7qF8Ph$7x4$=FJGxOA1H<_JVwgppq2TmF~%}kD-#B7Q5b1&9O|fu}V_6kuYpE zIE4xl>!maTC~FIchJoEsttD986NPL<5BjGfX-JIO4_cc;>pp22na==i$;P7w4_!RC zt}3_mx^UdQs<<1ABbVaTqQP|wj}1*Yd16W8xYo^a@pDq*uU#72aMHw*(4>-X<68d| z7k|{mWt$JXyXoiXlbxA(B{PRn)VJ$xN_X8rcZ|@>?~f@)I2P4=Uoe$!uY$p?rC~+ zSc2Vq#MY0kLrNq|e~?q;l-l=)L|A1^bS01(k0@wsxYe-dURXv1q!Ta(_uG(?4e^zxj#qF5$j!a!L|%;w@T% zsbTLWGk7v#G(mth?1ci7EmC|iT0Y@XUyLT8AWw9BXNt?vK9(hyOgWw?D6Jqbf7qJ@ zh$NR;B1s0x9wN!cV38zuf8(ISNwul5T~$94Uj9BNkrXNidHHXvoEu6_K9E5Bdxmbs zmiJ@(jvz;#Sv_cJ>#d3lOO}n-(4aKJkb(bLL6u#w@ICfI=Mh;S=`JYxY$;GUSkFxL1o3+x$|mn3hBit|L+~pckR3TWT9{~!cV;9P?Oc~+rW}D zLqsdVGIHpR>kbA;&Hyg2@{GPXi-ET))~6%fm!n}B`T|Mm{%M|wt&&GY2mjWnh*MyX zunb6}4W`7LJQH0n8bSLq#j3u`0JV9#%ieYzZX2^KBJ}lJU0Lalw^t!lSlj!ZclkM3if{rYX7Rbu}7dje#V*!>Um zTJPVO@g{m97f;7}W6OD2v$u#`@=N-F8S9+?`~%{f$1b(vaS&xypZqXv{9-Y<^+e!# zvW?oiJM&~|Gz9X-uwt+kS_`b@D4OI+jNRKIRj`ktFe0%@Nf`dR_bZZj8sCImT2(m$ zSBe$Gm*AYwls6mab$maqBz^SqotLlLoV#Xo&fQI``H8*9h8}lCE1GTb2Q0@ z>$q7XVOzt{iOBaBm-C=|lo0ys=2)^{b4+5gOFGm>U+qXkZFAHG5P#^Hr`i6*q+6fFE;i0qCjM7{#_8rT;* zj6-V737LZ+a@`#;odo_iN34SC((+UEZ}ecla7m+n4Ycr4178xTNzO{ma!dcDp*AdH zM3sx*xFU^BK!zbxacpV@D9Wp07*$z`cuYz)DVzje2P? zMXQCOdWKm*OPs<@EHG?-1pwNT8F}cNNN-rO@I5k4aruB$e!z!EAfnJj~VzB^BzTEg6 zF4?sHt+<_aEN-GAFZoPmy9>sJlr78uF~uTJEdE+#g|$pyOuvT5@eUfZ81m6@dy4wb zPR^iti%|3TwSRknFxqv84x`<4=rCILru&I(i#GO)G(|&69Ur8GwxNJ+-=f^M8w*o5 z-CGdNgm2_b_(o2Qi{F4Mw$gr~4X9$fsb6R&{x@qysO81ctp#j2iL$m*Ti0xxSLB_l zY4#6t_djFB`^tUZtE8>zFbe6(*nd2D86RVR$yyEdG4>B`#>RzM*}Y%QW<6!rY0(@u z(zpT31`8Nhm(5k=7%#Ok8pc^qLDk8Lg&2PksHurHgR%d2!N&f^mfLn?e`2&aB(a=K zQe4flQJ)rI>|aZWG^WKhFCh5(Z0s+~;K5uI3`Cg5tI03b@}4}1Be$Cn|4$+@8bwAi zG~41&6h6Cy+DdLC_N}IMsY0zYPrxw)*%kKN_*AkL&3&UYVcXUkXD_x^7eqUkq;G+> zY%&>bTp1aoiPV_Y-Aaa0T9ZWelcH)VR>~xn-Z5QCjfo&w`~x+*R+Pfzm@cq3?9n(y zUGB7O!=>-aAq^*AqT^&Mf^!57lCiSYP{{a8M#N~BjSZMGeTHt?RN|n!W!p6$uFXjN zK&|i6A*o?(we-Yl1g$qLiV1pp)oSURO3|kF*0WI<9IG+IhLF6dAI7_N2Fli+c`tYRc^bwywC3QZXNLw?1Dyd`6}f>02fF%4S|i8p zw+#U z@)%k#ikYV+f3hm+a>QejYo=$fzKhjZ{L~t%*$2AKfhQMMTQW{rCFVh<(LV zVxUpdN3dA#`G;SYwSOD0viKt!h#wFKhU>&BM>}(kWG&ou;1tl6(M||66Ezh?dXbJYXgTomF0rO9M3SPW*#0$gsb7Yi?6R^_su5ykHe{zO zTz~xq{aSDm_ea`4W3raBgcdVa)NZ{3{GJ9#AnOkf8G1+74Kq)fSduoWBz0Wt8*%a9 zrNn;>p(BKkJtvj)I5T18)K#0)?rsW$T+2#&H7oJTOIK}9ShM+vyPK|u&~epI!iVQZ zpG!(#o}aeqiGuLn!tfUZqW6GY8N(vaW6_uG=D|a6&Mh4sjw`H;J7;WUckjVMZnS8we*7{8un+=Xi1=X_^VUkrLRA%ua~xP8-}duQ09s&)Eq(&tKJ44Kd_Kh#?1u zBk%6Q&3Z*HO&eS~qF3Z>{BPJrkz00Um1cz_m*Vfv=+$8Bm0i00yrp#OOK(3vyG(ic zS6&iIEIT+hQu|3kYc8xtw$(Ik9iFnOvLKp#V{Y4Jy+YO56I<`;m#}gSzSi~MzIQO~ zPGRBy>Z#G!Sqph`|K8*KGd|jJYV)8G-yS}Sh^^)sBSfR-$DwedLRCfC0ruKDc+H6sP4L9y8vFfy4nc?b!i)kLt;K0;t>7OY&0TUry1Qn zuoJ;(gg%BC9%DJ1FdC8aQZgE$LcVqnqY-Xg$6!__9`So9@RiUEs!S4Y8o!D*8rcq` zk^44k`^{b|t9!I`6Cu9oTprxZG_*Po$2@+s=~TaO-rbTHsj8eGqNT;^zm}n$B5z=$ zk&TsUJiI5Wk9b!c4-v;0-Y@>HKjK355zTZeb>&4swqVCLZtY!MI#LCe-6b_v^oWu{ zqWVoR#&357D4U9)$m&><&-c(Vf()tkF$t;O9Sf&0`S3UquXh7p zqt~y2oU;DxdxmxV#7|2Gr_J&q&+j!!JCIpajw5i>RTLKF&_g6o zamAF(8jD&f2giWuqJg!Zn7CG`>sFnr2;1-_c&xbto3cdkXVe5EV*vnnw1#$Ya&M;e z4@DM^WHC9A!JQLfwt<(^*J~G0cjU~k6OS$fXO>qvr>fPDm0n_=S9-LqywHuM$?A66`!8X(~TM7t&NM zJcBh=2H+^B8G_&P%*I_CCdaaA)(oQ?{d_EoLG>D9{cVBp+9WaPJ*xxOi}j@7y6Zkx zjJcnbJ|<0NjIp%PRpo-~X0KOp`uPGC8gBH##4z-k>=bGY@HuNn_weL+SQQC?;G>HClvR;N z8r04NiIO1${m(1W)N{cm8S4XUm}92b9GXYrzF$RCCss#_Q|Oi_{X~t+13|dhn@B5r zJ+7i@x*+x3D-j`&;8+wy%J*64Agux5mr}eF*3@>)cKn*aVJ$b}6ZJqD zn#t;9BV#KFHd1FB=WBSXa1t^*kyzvzWpjwNGOjIpVBf?gpOZ+`wP;f{v&;f@u{+~B zE{OCPHiTYSKMrnGJ5~W{Ib-KP8>4tS9`p5BVczCSH|weC2Pvm58nPXvhHTlJo+${A zD+-^R8!d#|u`qpkVcMpr3c{xrg->}atMnJ)xC@FS-H#evw_xzl1-Yd^562CtiaV}2 zvKDH`+y4?eVri)5#EB&x#jQ`o#lM&w|7>+=0*=3*SkhL|cjx4U#8)$t)@LMUOkTD5 zi!W$MCeg+Jf-C1ERaA zcx`graqyc2s+*1fa!b#m;irS&3ym5*lnRv_GGI^ytX6!y>Ie z)af6z7RUHr7;f0+Aw;gh>Hl8@9xkq|$o!mzc=+~;)E17a(jAdur;PlUa{uZFqh6G?C+wNFv#X=R~8Pv?qLxTO9$f4@kXe)~Wg z{%;u*CpyzqRn`!$!+$Bw4^>69!wkH;FdrgWThF#^_(dS+W^wG1QfGD5)$AS ziIz)SwkHkTlsTwj7yD4=Wq1F@t<$0vV?Qh^ogS^2@nLb@9RQ4<)~L02=Vd}&k<sKKlV~pUv?Xd^FkfE0%-g>wM%d)Mm!K z6y~4xB`1`jJ6Png5z`QIYRU3 z3aV|wJ|d1BE2(3CT_d9p9$E7{U~uBh6*#;|=j-T`yj~!=B-mQ1S;J+QZU`$9Bwpn! ztmAttbi7X(RN`7H?G>X?PLat;Xi7Sn4i{}s*Lje*MOACMj^cmz`+_6-2vXQr5S=0<3^ zy*Iq6=K`SZt=~L2GV725wBI%SwfuMQ_fA_rGJWdEw7f?P!mkyE|2`mk0e~3jwucMC ze=ZC^b9z>3Q8@03%DBm6BS*vd|E9rpH|Ca(3dfx^HnNp~*`p!QYeUHBAO97%-XIva zUKJm|#zy}gAH%?Z;KbIWE3-;}5{}fLog1w^F4FWQl#x4@hRT1H5UB*MHu9p#R?uq4 zha+q8zX9P$CD^sIdqukA@6Nm^GIm#1o~>tg<@F)-6i6zIf9$CJTYeG=)gKmWp56yg zt+S?S(1?`0#RbuAAlGL13blY-+tM$w?7nW%t6=$`+dZ;21>gDdxpO)b&g+#(*y-ct zCA--dQ0ZGUUHD{_+544 z5o8^5M!%29B7a8#gN8m=M~>m@sJ>(P_=Kw?r@S?5WhP+=?`pN2exHGs5|kU06)`9a zFwK)bW0DH_pPita_^Uzvj2|x-RvYR%W5r&eL1uxxNj9N-WMYD-HRK)X^6?Elml z5JVT0j#^T&Z&y+2&uBHT81x~!%AK^lwQ5Dekdv3U-q{bbOkDL&pxFrdY3t?Bia_-# zQh0H8%%eyOLmX5zfv`6lVTbhDbR`(|mIjjT3hch#`Vtl~N{=qD&c8-9_dRa!hynL^hf^RC{cCj{? zC#HMN2CTW#fSTzLuEv_V2&d>inj0prVADwzndFUs-ALi*WJ;v^J{Lv*6__caq$CFlyO+pfiwHPFGe4`Zf8w>ko@*_q947T4PJD$dTEQ6%$O3cjFPUPn{fo=zzu39+ z=O2jXjQIApE0`+dW4!zOF7IdQ)+F&3T4Pna)!hNUtaIo8emL#gqX|9AZQc!+)__*3 zUpAN8&H<#DuPaL?jZDkpuirg^;c|etNMq=5d^3OC24Ib4Nnt`8Ow{7Y<`Y7xTYS5) zB?scRUDGp0zaYD?S#7kF2P6u;U)H0yq@m(2unSwqH&-iYxIx#k&BCdggb8HX56W!& zuEg9;D;1B}P4-^Pxz8q!{Ky}*k^e~dKE1kyxD7Hb}{= z-AZM(J?f_T9<{piSg)&k`yxCDD-@!k{8NyzCISvl0!T~hI9<^LHj(DctM)CqV)P$w zm^ADBzfBuBymRs5<0qsnowTj=%F5$^_RY)NpShk%HuT^Y`@SJ^i2Gv(Z;!7*ll9j% zF9x`3VcTvzdViZqda1&W_Kzpdf1`X_`G;sPOUQ0pFaACGd9dC|7EPXQZQAOG{PwFr z+1Y_#p3*sDoKJNC-p%?UIbW+Ad5BA1rJwlhYk{(}mJ7aY_mioX-SZ3)deu)w*qV<@ z-hMuXX8<*AyPp5Z*cuu?s^c)m9unvAJ}fy?5r8--r6Ra)!Z%Q$IQQS{V?jBoHxDz; z7Mk;8A#v0Mqc|!&6(Nw!&5)2QR#Iw(fees+O(kLQF$6!@#xkdJ19zJp0Y@)7J47*x zGxS@(2K25@mvidNi7;?VnW4(&6pj%Sg93fD`n_I3HTd}e_)LOt6`|{~mhmASGscik z2|R?3mtT*nv2hgvlo!OS&(&6x6!8@NVLfxfXWJ6NY0lqbz14GrY{r`Mo$!Eo_k+}Fpz2fJfpbd=MN5$v| z6@S+2cW-{Nrs*TuS37a#g`(AVc&KzDk)>|+!$Q#;;rYLFX-GBL1_M`rW++;in7%1v zP;}SqludURM7QIAH3iWvv(q*m*FD+|e|PMl=)<#9MiB!yqadxlPfCSyB1xQh%Bx)I zb~w(hOGs<~zB<$?`)kiz66##hEjkap+g13Zq~I>-6`GI@rVv&nfNu=3%vL1jZ$=M? zHGZ_B9R1jcU8?_VY3sHTEEmj2Z|xzLOmbSVOp@6qf0-olN4_%2VE&D(&kW2Ov>wc{ zE!ihQt+5mi&LS0yPUqhoqoTAx9k%Tl<}S4gE$KLyLBI`FWiut_wLvC3ih^-tIWQnW z2tvNGO3Jo#valdXzA-;OZZV1^fUS@yG6?@y9om(TE!r>SOBdO}b9gSJ7Vo z<#`Ja`$hgv$tI>X$2gH-lRB{1--$$Kl5dkvB-69QBJcTvrH_2=oW$oXq1og_Ap*+T zbRP#6^W^cjqs!0scOubFgg#Cr$Kp1(_&br%ZT59OjY>lwIEVVH`ikAVic9k^s@TeS zuXIeW7u;4PcR@c|8E8c!^Ugkzn26b!Trz5R*a5^h5`(=*p!h+A(N@^3SKwmWOWEZ@ zyNH<8P{3=`ydni@P7G>CC+j`7H1I=|1oY&Gn$NBj7Hh54OH$*&Rxuip(rd(byVT$=4>M?ZPR%(F-S%r`PAU%U;^A z;E!i;TW!v_oT7S786pO83V;gnw6|m@Yv-hbj+=wSTr#vjFyuCl^*kPg(=)&tF(iHk zSSH>a3&h!zf6g5iUCgf@MKHfes7nN*hs=RUrV{UE!&@Y6fY8%{J8+4)GDCqnX2f+X zcLVgoslmw2eaaKqa^h3z0VU3Z9WAgiE4X|RnlFPKxr97zZNNjV!!{Iw$X!Ygjd~>D zC8Yv01hL9WQ`>uZqGL)+1t#2M$toFLi+~L5j2;mK^1>-$c<75DKmzHBnsZlIhHs(8r+_jkW2W7CP21R2utl{ z#J_S@L)eGZ8X_(1jy1zoXG}d~t=&~awET`9cjhE~ z`ec04_?)DZa}qP)?4Pk_bHBTr?gi17opi(MBci=~q)#1_mRDC0?q3u>CO7&yh_=z` zI||bBo-7E*zjgf3+jC1V4aa>wHgaj7!9&Xi*Uir@{b@MvmCCp+-5b7J*|2o;p4FQ_ zt7$qlKIw*xr12SvSN?R>=BApa{%GRXKWY6#iO~V@DH)Nzd_>x&RR!VIh2e(=M88f< zpNev}m9T?+I+x1VhKusGv5^_FqAjbR?3n#vVM^Zeg6I&KlKd<#oc1hwKA&&fXc?p0R0{R!1BzCSrL$-er z{nl||_abu2?|h=LAz#$8b;6FMeh-(m&6%)n!ro;iZN5C*pY{(fPaL?^uWj}-`|}!u z?MP5a!a_tT`j@Pib?V!{|Mg~uP`ZrGf^5<)1a>66x)5r!{?c(hl#&31oIAAfa`t1y zd_s`P$qA*DJe6H3Ev2NQIlYyVg!jCalI7iLdzMPcwFO!C;Y+n$m6g_UEMvyDdH7=% z{Sh42D}hQ$Kxu5AyfQdhKE`t-epm*jWW`3!IQBx0mtRWDjwG0I+e?g_v?IyXZP{+? zJ5Kb@j;mom|Fwa3Bms;Yh#?Qg?ODaRQMr|W9ord5Y6bZD_XXgd08wPwiiGNuh)Qd% z=vTFCethqYn%>O|<9n~H84x}3Sn3g{iMH2OAT~jeA3KOTw57QnS!%tsa`p^YPP*_L zPse=`!HV7A5!>qRX(22`f|Vug1{SMr*U9qh>}hUukV}{BblhzsV(!08Z=U?*8#7N0<0%OUWn{^ zsDdlWxI1fMfCvTB@CYp^Xs{~K^lu^Pi>_~Ck@#UCSU(ZRECzpuQIWHdm!1T+6Lj!r z^(=4*QvSk#{#F-EnH+W)DA4HE-+gKv&3W zkstRQR|jl$<+%kp2pw2Hky18N(Tdb0L=Iqi=z4e*NjaHbaAYK=kMUh(OGvxlsx$Q6yBJ)~kf1pq%nOL0~#`ZqeB5Xc;2(S(n^HJil*w~nE!4L}lHub4${ zd~zq_>E)QgUMVg8G>~_Z-nm8!zTjjowxST*U+LgwYtN69^{05*4LKRv%o?*2Sk9c% z(j)uN;JjGuP10(d**5TSWlXXeR(_t{-c1^H;U4xokty^@cVVnpV&YrF)+PC=kt0&# zKlbW}m54~RxaXMIr{9cvyvg|a2`?P1yzE@OzG#)!SiW5KGdHZXI5i7-p97h{|Ju(^ ztL_d+Da-3`X@We&$s9|;Hm^3+cCEAQ{4M@gT6pbSF1?(!8jfXD2Ll)dzR_VXsT;|J zmpcOCZ~p5Oj4Bf?^(B=_?J4+m##f0qHkM6(O{Hx^4_Rqj!SbDx`vw)ZWd{_t-RDu* z_K_%TE4wiv(!KZKHrYa({QaeYZT#I`3tFf5A05f>c6ZZ;+p}(y29gtQ|7G^5J?OpGy#TBm{=XYN*nSJ* zEN5bC%kZ@J8%1GTHU4;oFpwMqNT--LJ zKRYgF&M9tN#(#`{dHt}zDSFP!KJwTlUje)LxAxClKNM=wR-m>AyignDndP--eMC!v7&%K0X&rw{vaJVRfox0W=bRvp zWVD2;(9veO z{%t=Eloib$ydPe~|bScMe0CJ=V#Z5QY#5E3`!qz zF+lt1X)eDQcPxxc#{RqISOq=RYz1M*qE5x6=^)s`)4EK`lMOmb$;C9Z2#E-sV+39g zlxdH3Tm4BAX~0b<m!4ts62Mm?W0Psv~$^kd_;e;PcfE$e?h|uotGK* zf>0;VLyZAF>^tE;VJXAxGgn8>XiX+8{tSc(KPs6r%lgVoA&%7#3dn6DS*E9&M726; z<3QMk1vdpnR$W>YY6GgV$ct0+V-_`oS($_CDxf#~HwWL>Tt=lAhJR8PONkcD&%o5| zF4aX9TTb)QfWOrxIa$n#yHCAB&sl^f8g+uEac zro7$w;i00~&iHRoY&{NAm0LM{NT?rMdB=cgz3}_*Tsa_`zq_C{BQCxZJ+tdSDroJC zKi=IfIynLUmQRj)Zd#;=TX{lQ*UL??AzQdvt=Zco zvN{l$l^cOs=z~1fjSU*e&W`5}7r(Spx5>Uh;I^uY-E@isjv;V}N0Yd*xOB4m%Ne@3 z?Xu}zG9n>4<=OXfDvskkQ_pP2R&QAJ(kL5N{s6C<-m(!XHDy_i6ViP#3K%&HLxA$?UN4b+p3T)Y}K zAd~a7Fo#S|u4#o|BsG3(`yyM!O-O`m!vOu09UPaJhlo91kQ+I1M@H$A!OBp4yp`ir zg%x_gwDOr82hMzo?GRIHNVnr#pGI_(ad^duLI?>tjVx<1VKeYZpx0=dRUw-Oc9uKA z<&2|UF@pruI@HCT451hsc1pyQ&nhHq6vU|5hgMcfdWK>EX0B-Y;ZAv@!j z-U;q>OqPyI9=iD&LUe}rNQ^K9W94>QOA{3qnFiI_EqDGC-v-$+CUfl5EA$JVtdHSW z>ESic-)z--!u_PIWAR!$`S%2&%a&vi{4Vg$>R;#>5s~}?ej>2vQ(Q@{oh{2q(ZbO^ z*`Y8b;ik{zMQdlgvErHPXj?vcI_9kn_ed20*SPFrYa@6POONRhw)4w|VI?mM^lpU@xt%byPiubQSGcUcVZEM!u|i>bhqqY+zh#LfUMXRsxq$hRUe9 zq?Javg(-g)ujN*(2~RBk&bXA~;efPZ$&}CV*AZ|vCNwpu#?E?$u`jfhadg=68(u@4 z!X=Hg`x+PABGO&+HSApGGqtQ6l--C9)2bICWx={V*!zL|k*E;eZS8kh?!AcUlf zMZIX~PvWJti^;J1eeGXU*X*f>u4{J1zVt<*Ym?)rREMfhm{{`l#FDRyTknpGKcYI+ zf(o@SCzgCMJ|P3%{_m{b{8mjB&g&s{%&k5I4d@IBoIz9PD2@84RrojDkd+m2Wo?lmqRV^9chqtkJ;&01waT z{2LfBvZiwCxvRMg2mc0Jp%_?sI^~ssqP1n@X;_-XpWS|Rs=&*AE6QpQ;;!8t11ie$ zlBmp%T=M;8j>jB@8^>W+ z(tFwh2Tb4vytJfxLh3GQ8)@uEGIejZyb}^Xw#L>VGKq z;6O2Ckg5M6xJaz2e|D_>2>7@?K|C5Z>Mmu-vR=Y|;SQAKy2<6p@*1&`un0m(xrz1O zE1c4?pAcoLneEPliku->GHkID09e8#%q3~=XH6+q?83eb67zpe2gy<}OTpS_2qjA9 z$e{uDw0q7w_j4ijGSROd6nO|R7(H;LM@$xFu)Vhd+RaSUj>?%uc6}1jQZ{0QKPmdi zDil*<^RM{-o0~{&Sj%z@G;mJkxpUc0(+OK7udHttX0UJEO_$=fjWWycq+y%^E2_vu zmcbbE#RmGI+5);#fJehKZK=)D6b6hLHM?vmYht0-)hJL@CPFiyrpJXx-dg0>pgeGhrCZFsjr%{*-`eJL}SO0*usipo0 zq$sIe@!f|<3&5-bgUwG5Z$iX zIoegsPVv5uQ^-2kx)zJu^qb&(nIx&*4 z88vYz02E^cXKeMf{SXU?Sx^+$r0_7_`$e;VTPo|R z%%JLV=4Sw2|6hIgUHiX$M*ih9LI`Ta-%wU22q6X)3NM*1*F*OH%V&hQD`MJyxX(!T zfLQ~T0H&XcI){D#6iilVjjT{CjR3zWR{WM={d4RJ<47rWMCYu=(pJuFS49^I zu|Zn$EZtoDQ5@2KCCho^o24!OSE=)d9eX!n=Z1|{?kcwC*eW&{0nhuF&j{a*fBB5~ z(my|=&j_?%^;F#CPwY@aX%kNQj)As6wD85mtl{Jc6z0^YSn~xse(OS$iW!1F)>&RG z^U}l5Ky93|(;9UkXCz}?u-*94cFCf6tf?~xU8U}rf0>M^!DND3CG1qj;$J2sGKr(( zRtGxU-zFnjzkTurVuSG?w2!r&FKmb}hW;hzZY7+x5bij%S?rq{(_-sPcXVrJp84g` z+gq$4T2Y;GY=?Ai1y6gU(0ux&zmO4gE-PkOT_cU}qO$02R@t171Q?WwDsFLS%lPX| zMU5INmo3bt!K5$;iP3--Xi0PSlT5n_{}2Js%pYBF4c7TWR2@6``u*vOcy?oOLnCw3 z)}KKr(%jSq@GE#{4ay8o6X17G>ypO*FI$Y9(lha=nm``?=(hc-f2@7wpG$8jY&*4o z>f0oQT-h_ZbJZaBm|u^VFYwFFqn;i8m+6sap`{$n-N1HT9{GmfZ_XRsmzuA=`^Dq! zM^)#??Zo!Q?)J|~!{^_MZTa_8SSP~#6>t0=4dnYn_P*yYf@HPdQ=_*SqY@XVrpSh} zcdy%!$)#J&UP%cKm2#Q)kcoH!;;W`zcTw*fkBUh& zLKn()dP@^vh>b5%87^$QCW{LPTk!Hudbu4O6Y-Irwr-l(Bj-tIKqcwU;z#{eScXb< z!_$|HO-v8~IN2C5+6i{C06?jV!xtlDNR zf&|AD-~&l%?c+Mm<|pzjtsqgi>tc9(;Clwc*Q+8ONNyTR@E1>c)4CshH7)Iq8X$R^Hih7DxAHuhaZQVFA}XZCZuzqcuDFudaVE zV`}$()2$qc+6rOaeGrda^MlUsoL;Nv~SNw(aCu3fv3JdC*-SPsn47G2aSwHzjNjLyP ziO^XOAwRcn{S3r|$a(t)4^(7<#?BPSc!7%sGWZ;_1@pfRWDB4rHLm$Z8Zhxg+1#G9 zE$&cB<1}GDAcocqRSdNjTTCVEEcArOMaU(@R~W_iiy)ZeWzNs#v^ail&L@8O6juFt znqk&KmL_yaMT$NeRFLSEP3ega$H|g80pm(xv$X(Cj|*V*msX>FS~{4NVga@0BlS?Z z)&_1Y=|3T9(SRsz4Ta~98I5PhX~xMdo1x-xjf2XQN$!idE+O)AJ?Cow;SmG%glAqp z;Mw4!Fp9AQDbBGhK5AWK|k@^6F*1A0n1)X&@ym z%FBL=RAjSPDpHZT;j5!A`rZFjEN^-+J*oY&&Lgj>nR#Wyas7_CYU=1W9{fIZ*J56} z&H56^v_Z0q#achKmRXD9OK@?P1r%5-yj?VQ1f@(Z9>ucbQR{)7 z9%M@kS;;1HS-v}%c#>PildM+}Ga%v+o$dEomYszwpi^2&bD-upD zjC|O8=#o>g5pE|4PfQixP5)wbF46Qz9N>TfiqP>ODClV4LOi7%VQr0|b zZcr`TfdO`rCmv?cw2wa`b;?U~kBbUHT^8Wm~9kbJO z2XwDHHaR1=fA_j0@xOjU>Rz6mn!6~!ZYBP9ME7R$9+?jBku`T@N~Ca9iay*`X3&^H(zuoW zjkg3%9%^pAgHloL*^79{(NvpFN3P6DFFBHbV%zO_Ignn$bVM2MSs)rO2RNnjGyXWs zCyG(UQ@sfS$>nqW1(Mk1C$t9I`fpF!os>O5@NK!r<-l@;!nk5>7nugTNQ_YZI5P0h z&V4`^EA#uci^Y4AN(6S+MnjF{)zqzxGgo(rJbG0{vudmjq*i@(DvrthBMueRUrs?MC)s$7w<+fYmXrG)Rd!h8EoYk> z`|^y)l8QUxTa5%15kq?#5Cv~i$}h`~SIAwUrs5PBS|l5N(eVuhq6 z;t$6ut%vPF??t&#xuas0od57PlUEux@!ug8+Tk~k)szs>U9Ml10`6QJ0~PHnG6)CqO{69~dq?<8!S4)X+o7Q@rxh0he{~WHXVbGx&u3Yg9ens$vyx^&{Qrq4$t zZGl^4^N8dv_uSbO#^3I_uW8eWB=-700u9{Y^*`f?F?8wa2R<{mgRK6C9WbVN+W19H zAC~5{%@@9ryZa@V$Zl-ICq$MCACkP+!%at1Ro>L4{}@*H8lx>g@CK*<3tI(xW722E z={fDwCJ;1gtLvGZXKYAdFBzu8!*6HcwRK}?M>)N(DV;R@4jg){c_^`3bq5LQURJNS zvKZW@^7}~xdtLk3yxs6JiCf^-JO#!b`NUT_YR-5KR)1oIEh0OLw-hmvqLqrUoG z=(WaQm4aJ|yj)=KvF5vj(c5GX4~V8XPVhEj`|1rwr-=RCgWXVTM!;AUmPzbE@7au& z@bR2{X4cr(rM5@tGO9J4jsaW~<}cpZJ-hj^2wsBHsn37yAX%4Ox{9uZ8eqo2*f@y&De}VVBM^Q}5S{4d6Jw4GFcewGwdQ&3US>Z9K_J%48O+P$hrYXg`v3aEKZ*u{GCI46q(i zMv}QJ6$X2=2m9uKHzM8DT6@#lnAX}j>*n3@i`%-CG-GwjbJ_j{7|v|HiCuNHg+p^m zBjz6r&{IJQ7T*n%uEm`TB;68IE3H8o95|^m!BSBPlN zhgpmul60771d}3k!ab0__c3rnx)dUDtJ2)QH^6CLrXx6sumtFW#S60f&X8gG!y_oW zt@Fe@ck{#{|EPgd%Z^af{7^kGBn>@v4ci7~3+n;PPbZsB9f|3AZ_ok`v8>abDwluP z=To!0_f(9nauTYDh~KlhZcOSsHk!rvUBNiYHDDs_l7=~5v2+kbtYbKxIDwW1tn%KK zA%P6{CAOY69u5_IYybTHjraO_?345sb{PzsU8AK|VI(=&v|bk=!q?YLDvP^;GA4V0 zkr_sOnLxy*d2audnf-~@!LhSyD|X4iGz@Qs{Ln9E*5?Nk$1o9W)a#{w#tEbemk$j%ul0@y$))JpsgxyJ7B*tnt*d7KK~%D9Q>1H)`b1kq^Ac4%mS(Kk%5M zPYwQhn-4rA<1Kg0@3}{8>k+u0)!w*`ghM7Lq@O3=C07wC9Iz%(x#hLKI~sL_j9hTP zD$elP_$bXFKFO(qlnanvedv>fM%$GfW2+po*mp&TsvlM7?(bSRSX7~QGa|XYc})OH z@w~@gM88L5XvKF5P{Z5*m2H)^36Ce$@A)7had28xHcx zj~`)Y5~PWf225JkB>S?kz9W5;n8~u>6x4Sv|0EufB@0f7F+tLx1@C?#?tfc_$^33{`0FENkqB&&q5*OhB{g zi3$827DFTX8zGl$#ml)YmxL+VAdBlHMFi4?Qg~P@KFWy+rO1+Z>7zqw>o3vxs~r0? zufli9Lx`&@slIJ~^f(3Sh7D3__7QZb=l5I5HFEvv1gyXn#)TwhP%<_?{Fn9Q&FS#C zd@qP%>j-GchQ3NZE!iGaE!jYzeR?=&SqJ>P+H{VWWpC+A&cmS|NO^bf9k**XUF6q^ zydcoA5UQoZZKmVAHD3IGNsE`D^Q|d4y*;g5-;TjMD#B0}B zhe+@8uJA#2q|$$6!yO<@0hx6?VE(SRI0V-u81v>Z)gISIW+a{I9Yc?;prE((!3WA} z-(;ND4r~7oM-y#8Dc@=fp^=1j3=prcOb5I(LpBbs)xat+W8UL)fX~&6Xl z5-L;BJ=x^^V?|szCCGoTOP-vW?#XQ)ERNr#SUP^(ivS!etR>Bm#x3OO8W=QjZ_M+q z=OYe*1dJ7;igm=`KSf46hZRjCNxUV9dpy}6cHbA!l$=iKpeHkqImVYv=g7$?`%$?f zSNsx7`ITHq=w~vHfQ5bSsqqIFnJMVUWo4v=ZV|~Vsrd5EG=N?+GHmCl%IG*1<4YA~ z)}`Cr5Ai6=T(ivu3Sdk>%4V8i{hd$bGXG{&Md1`D@nxQ2fd^bATpE8FWQIeiXcFpZ zgL`G=mOdYNO)T^-K3c3eGRg4sNJmC+4`^q`#*$(P5Ku!9IZG{(u5mR8>J#!6;jArk zfNl?2si}r$5*uvX-?pi>Qc1?TeCM7SAs;!V+Xzc4{1Ad8d3Pt$Pfz zYtlvuW&{Q8)(qyNE^TAG0^B(F&1qO#O3wbD?JsBl?qge*j4dqsc1qDVh0*GS#8Xld z2UmuM!8$Tp7+nbcWY5G8-npYG4NbNC_n6c_`O+!N*1h}Kx_7FYz6N06Cnd@9-0KQBND2{w)FG2^poFxgaWS~fwzVZgc{=W`ui3tk>P1}|Zg&oDS{ zIUA4u1-#UPdUT*Uz5)EiuSm_mt+X25*#tygGEM=7@j`M+4H)-*`5UM$XtVAkgU?rQ zAekfY=!DTo{j}wfrq$=gLun7+oDN}T=a$nE%+olpNLP8D<9Xa$<$IEMoY9M|i+G)S zSV6}1LPGg_ETT;ga$_Xf@SQV8yZ0@Ly(G;Tw3 zHLiOkb*A$P%Bm{3rh0jB6RUa%ETJ=W=6S`=rlO0sN5+ZmI(lJwVVku0r#5WEQYfcA zym?lN{8>7-mT3m16wxslQsm^(Wea-s*ka38*+>`+s9EX&Uf|YZjOcTOCdK(Netn<# zW0+gRzArrf8(z3KdhdflzyQD2mxb2L!)g>%3CLkmSk9S)EY?a^mHWeI0k7pi>3L`d4u|;rdU+z;8ht*N zh1EiK>0O3+m{|`glTX$M5y=DekMK>D$%0J5O^L~GnfW+07cB>|dU{uJwZYXakOY_8 zHZ`5lp+mKl^#UWOfaL*pF!P7Wt8GDleHiK&_XlJ*|SZVP_W46{MwH-a~ z&QAJx|CO?B-G8NQPn_CoXwB@Qh&obptMluo_lq?Bcc@3lmPMh`-zG)Yqz`Sod|Y~a z*V|~qcam*kU$A{+n?W;eeBVdGxi zI=}!k{#in}zc}@XciO0?k5*+jSkvz*{`P!$);xxN%tEdt(-EoQbYLdlP!x=UQjtC; zUO^je@ME*+tdid7j)NhrJ*1$1RySq`Yf}p9hlTJGI>S=b>(SD{%pEVl?)SuPsn6Bq zn-op5?FlIw_n+ zGuI~M#MNZmZO3i2nR_v_-Z1eb#SIx4w`BsivExa6O(X@)@imc?Rrc5dBM3f|CX$R# zJe-F)9|Rk1*Y3zo-Z2r9A$*syxq4m|y`a{)=@e?2F|Rb=J3hWz1!NqXKJS;MJKggM?9S zL5fH`hd(V-28PX^DN84knZYPdna;tnS{*%011TZ{+PzaZt{AK*k+Qx^#MAK_Zs4B= zb-XSRcx{wvJ!=e)-Ci|kmIaVz9nfoYaGweo=I1fOh~4kZ`#|N$PqQ3(zo54CK}rPN zFT_kD`C^`Nos6MF7Y*2x=bts;=LM$>Q;9iK3hQ+5{EzYUN#6ZM&qpkl=gX@&R!TD> zVs~Sama6&nDYBsNYfW z8T2>BJBj^@5f?xCfG)gWo%5|>yJEmw)?a~8kfPWk{T#hS8uFC{*?`Ze8FhSo#=1ur zI6Qe1JVpR8SUSmuBJ8fk5X_3g$aCc528KPIHOVqAOu%h$>S%hyt@7y6P&Sf?4m5N?p(oD#ZT*Ed)46Bg;C_H3uOAODt(9zoy7Dy)GyJEu`cMxxpAB# z8zy`6igNFlJe4o!Q{UK6rn)UMNV2}H9VK@YZT;nB2t+j$X13rWhzN|s_c*N3mOsp9 zVBUz98$H;id|n^`$T9q`i;i#4!_4jw5rdv9k$!-^MEOvW9DusEF1Ghd6}u@9S#% zjU%z$%2BZymDt#}nTj5nx%CS(szuv0N4`8sgxcn!%rdJq9ph#l$MCe6HXcwbAb)54$bYWnzXH^=#r*#PlstTk&7i7V2{l`wc zT|yp1-jzk+yQyU8VXv^7zg^AH)8W|lCw@%z{9@?jNJ@~w2+ohI+Z*A%Od+vTAuO0` z5dbw;Nzbxl9nd=huN$;&nP=Haq?rv$DL{SFUd&gTYuM*41d%B`a(bT1g+RX#7K{L{ z=_lE&Yx?)#kZ9^|$-H1n%Qkh;GENT|SjkbvSq{WSV%M8AdhWz-$Tk3S6SmG4ju(hZ zc4n~jzA}- zEHs?I{^p~7b26(ZTu%g1-CHNry~9=`$Mx?usXr?cH=ocvYEG(X`EOTIXzeu`7etRso#H)*PA&c zbwsfB6D&rmofadVqt|Z`6;XbC8hl6~kJ<>AyJ6IUtt$GY+LIb|N46Kp8@pv|6@qVf zC$`UGIkg7aw{GK++lwsli1rKCQ+{p^Qup#Zayy;DwAo*Hs#)}gB8 zCrIZxYh8?zOswbTU!aqXojH=a5-?+ICla8EEQ#6Vbrey^AvW1XLJtyT7b$BgVi-IX zICKw$iNBE!^tr-w(_z`pwFfUWYclVYO>RIOP(YcW-ZV^G6rW81(l)=BxY{W32F_?T(ttx2} z>9HY+>+VR@?Bv$44hlXm_M`Yc@7Tg4-wIS6iL!Ig-YFol<%iP)rbwh5P3O44#|o3+ zwPa>Ec)(^xWsFvI!=R3vz-yU@g)hXRp%8^Xd%g{wVqQ|(3lFf%w8SL4^sv#Pu37rG zcoziO$&}XxcmEE*Brl#q4t)K>3P#J~F$i78kG+|4;%2}T(tZ)=v=ucPmb)h){`ST> zB^_kv|K$Gigv?BOb54(e;g80J-yc+Wd5?_Exf#{D>1&_J57!N<8`&eHdSu45k=-H_ zGKSXNICS`pImKhb3FFEWMox%qgTC>1ImNj!eVln_(bmG~?@$2OGx3y#p@zY5Ai;n6 zo3~6N2a={)J;y|g5)w;iCq=eF_&5i`$NwIfwB^iYO_gwmTmyH=q2b7fTXC_Tkqgs@ z7N6cTaw`5e;+)9MTeFI@!jTK{x08C-uRk4jA_wKv%{p*QarGIC=+YP7admU<&g{bI z!O%rcg{}WLnaL%~yVc#7l+m6$A(FW*KRN*D?X9Y&neg$SmtQyecUi?f@SRsQ?<|q} zyz0{iZ$6C)p8vc&@-#P8>pOCAdqMZ^ZE(=%Sr z;URgoh=s^P!f~1mz1dD!J6FZI*DKeNwH4wJi9zBU`Xl2#;Ip)X+?|79~W-B)@(~GKP|N>D*;vTR{?R z91?`I@=~+8b2-C1P(?{wwCl)ewC)lY6~8wd>^r6}nL50A--xsU7g9Q`Ir+5IkN5T% zoSQwQ{@{ZL=MI`68bMVa2h_&_#ye`1CyRRU&Ghvh`Y(y61VGh$62_M6s?x~Hue0yV zH|MMOCIO!OZraMW+O5%}}%wqcszG41+UJdMGmLw1UM9u*MlbKFKaQG2jdh zvdweu!f(<$)WbeIxlBR`F&U7f(pnq;v_OV%SQ#GZzHfq;RM7G{EXs6>Z#PFdFjH@N zmz}zp;wd0H%FCXu`g&Psk}aw>Pr|cb!*kI3kJZ$^@LYUSkPQiWf@w5m1tm}AZikXg zt`(NGgJASZKF6~3n!;Zg$pLn1 zPe~YN6Wf4$rGAkd;=2E?M;2S^mwtwyY;uM|wrxvlOaEVBL^9FGGqU+@c8@H%J}J_@ z*U&cU>)(n-Y!bv#PqwZXvQbaAJoIE+eMi=962wiuj4PbAL&^^^;4awV$0i6K{%ij zrjfPAiDdLE@n6hH5II%e1zXaHvJU1E-vVD^J%8z z^`ko>f8@lKhmXrYj`fty#kKM`Q+~${(FAHkV{Pzd7Us@;3`#!uV|b;NpM+jG7#1u+ zjE&G(8-h^&_FflAMi;1Yv%B&03POY4KNY*XX{}N05oS~?M|}y zw0#vdw(sUG8GFE{(k+@y^mNw7oAHVi zE+k4u;_pFHGHb~s^&#);4>^bL%jYQkTTbO#v;*C$esjuDvCq%nuGEpB!5Uu*e)!JR3af?z+DZ}z&dr77EUxCMee&zQ(4%lbT0jE6E4L_zzHMd z5aQ6UaH-CX+54#F-F$m76_%B=bTxBRibKjbufuc#d zY+HsIeMjXrv`L&EFjr(;|0-UQMVhfh9X1ADujJ~GWq+o%d|LFW)Qe;+ z(hW$CQKc4CvLg4T$4uTP{a!$6xn;)OX8Pp1W8l-0DH9=%2p?;3_`f=ZRxDZgWKGflD#hU-6iwS){}d7Qp-2zEtZdwn7mMs z)~3AD_TkZX!c62VfeLOP_gb)?+Cj9VB=jpEmZc5RTa%T1oudUh2tq+$@;RtwymY2Tv znf&msap5lq)!o-4W7_D9zN1rLYRwp%@M3ww$_bH;VA*aNI{cQL;)&seJ10bLhUx#! zLu+n2C{ziP{~c3`z9@`F5)$Y2OuPv!8+`shEkr-x#8b}k;on9`{!O;imL4P}aQk0+ zWTfd?biBT{Fw*soY?&W`Upp}?xuZu?efh6LfQvmkLQiFU-8$niK)M4W(HXt*|IVtW z>B8>66q?DUqMvpt2)IUIwnM^^Qb$j=#*ugCK71Epv0p7rzDbymd|qAjg-|^%OzvEc zu569H9;7Fv@di;_wHkfdUKCxmQ%2^rzsWC)TL18UM(3pyJ|*oLaD*Qdx3co7+v+DndpbU+l5w#+y$Df`Jax;1%EvC9>f9hN3SEj-`@2*jl~U!=g*rDv$9V>TV?w`W`V zEupkb(OUa9L&eR=33D1V>D8~POIztJLve9`J4qtI;v+d?(Q-jM^}?^_AD%CuD!Nh5 z6W|VPUG1pH5z`ga``JiRN!K!1jK&d9(nX#vs^2?1JNG#v`|5lBE^A>c3f~IODLbss zq~bFcGU=BW=tm;UzO2byp)f&U-s`#L4_Est!l?35mHt>_IXVRQ%wQs@^pv=0W9lXa zlYL4Y5Hry!t(`|$cXr&40*bwhdf(&p_SOkUg9j<^D?ZDD7uET-rvWFhzX8>|HT2~8 zG%pucJSky{y`!qdwu>lpA`ZQ^V}2U(SG|q$VfzBfX8JgJ9wN46bt8-n#Tl^qLU9fi zsYhF;@{y#@+P~!Dw&uln>8NvqjhQc87R-&a)LzC_{4h|t3UC66(Z+&C<7BZbLgvRp z1%aA*^6@O8SQyt}_SKD)O;M)Mm0<~p7Yj?LK=05A`T+_a3fr9#M5-vd(gcTQti6eQ z8JPiftr21_nY!|zr>m5|x6z`-jvY<`^Cao!SdvUcdBa5uS}*HiFa{cxfB0ijvYxXD zM{7m4Tpu!HEU~m}=G$)onTg_=usjy}d?ez8$Df(fxGzo^J(XJ9+3T>>)5@o;WG{=@ zK{deD)%>o&j#3&Fdi(UnkTPo<#PyG2cie%=Wa5s_i7j zWc~hgjcoquu95fonEwA&n*M)=4s3&`MDK!mY;u8xYV@|RlY4xQNw*9{ix*(=

6x9<{Bc>|q1JzD?^b>GTj1Au7&e_9DCe@Vi`@I`O-3WdhvG8lE|Jt0-vdrFB zL=I;aU*EC({^%ep*^&qvRA$!R) zg@Oc48S#58w$hY;#1?I6;?tY$q(^sE@|9f{l1>7T5@zbM`jeiSzndH}71;*sk5USE zm;`g8GINMd>0ElOjR50q316aZaSx!MOj`FUY5by z1ky8J9W3yR_AqiknM~i&h8Ov3!Az4OK>iK0$XS!YXFnxl7)Bl0a0sWoH);Xa#GLvU zwV3j5wMK5GLn0Q42h$3Qhdf68ePmMp7_nIvmVHQm{{_Xv^yV48LVf;ft)0A%hk5c4 zEU9pOY_{7*u6Ni*Mgwgl*9Y51Mt@G*$e*Mi0qnHeZkXj<_##*l3g_FIY?B6oa#DE$ zp(Em$chBF_eu} z8A1kWY+&7UIAjoNFT*J4uwNcBY3>Wpow&Ai2U(elifu!D_Jx6#YTf9jM07BA46th> z^B%59SKv3yaVR*?{(!Vzd2jTft+|XWBZbvJFolb8I}UqjfEe;hr?SiyieXgoA7SMy z_#891db~|g_ChX@ef+2lCS{pU@T7lmf_{ac&fnK7d(wc)>0F@yohv{?C4UTzmL(ZH zcD!T=qkQk(QQ&SBX&IyO?R0?T1ob6_V0uu@I0dv(ps38R7lRpDTtvWZXZl{EWtBk3 z11k*)dC~P~#~2{w=Y3nZ=XxE;aQ`d0lnCMiLnw^6BG`yUWkhmJ{vWxqL zR-;|^4gEs1@W1OG4Yh0-6V1opHcX6O)vqjNM^bwG;phqAMriwf9720`^vlyDRb0U4 zr-nS|_B&esA?1*)f%KO{DqW?&9FjPvctjfHkVF7o6HgAA87PMY)RR~FoM;=CzAIC9@#I_pW zClV!Fd)MbiDpK|)(b78lxEz$UQN+=vYuP>lGTXYWH1~dvHvD}eGXrbcN^@=`{Z{VxmOHTf@>{0Yl}-3( zLGg^bvYG!ZtXY7wChU&iJR8UsTe&pIv#fX=Eo2I_ec5ixQrCda&xp81)sTs-bk2(o z6~Nz%ikiJ_h7O342-mM&!Lf;O!{krEiDPRBgt-K)^r~t&hbQ0OolXy*6F|FWtBxE%^~ECSlV^86FS@L7Ac4$$zx-O2swr1;AjugdX(R|TvDkoEzrTu3I%-g`psezW?>-C^l$E~Cf-=iPj1JC3#FGr{%WcrNaBOBDaZ*Jx z&0zv+5h9!SD!_K%c2;cj>cgr?;9UJP&AifwNEOL9!WGJr9D&pgyiPbxJ=3c=^owO8 zU~)>I>+p7EdeHrBMO22gDi#0!B^K$(B1qhd`GZi$cC9l2|t9;}p>V zBVLN3i|MiRdwu8Eh#7oMX~P!zta)LBn1Tc?TbF0!G4h_Ffd%O@H=Z7_O2E#f_U58* zu$1r=wEQ+4qLYTq^OK#*)-`0z*?Q~Aou^YfO<(8?+s)FH$9J#3&wGX$YMj*Ye}^H7 z?5rMNGEEv$FY?17UT~&vs%>rGbzxD)n8N)hhL`-%?MujsqZ)gok8D z`-Dj8rf}0lG=Cd~m+yp|M&acNc==|y=>)tys(WNbDtg5o@y=JWR4Ve&<&obrjGK1) z;Of&EPCS2keJz@5Q@vGv?dNxJCDrsB3)^OL8@A@mn+n?&tB!0V{-Q{$IA;GuTE#Q_ ze}CI|-x2%s!J_2Ps6Rk;QR+iQX{>%cMX^8S-|JRtF65-Cu%||U)Ih!_^k{p_*-pY$ zyVQ{F;@dZ#eiI?Em_&@7^o);26H>%=|3csPY z8;MS$<)WAN(gbcR7u^V(|C5E|e=A;&>7wI5VBC5jQ<)kit4rQO`4HW9^3^LT?tun2 zeH5?NpsBWOaR{N>`o|gF=H8`fwKo;hmb;Lz(a}b2+18Ivz~EG*9Gr|Ojse}>63~SU zTXJrIylb3jkNtqU&Ncjg?PBqsP;y{fwmrqjOcU3}d)yrYr$IGxsNCbAl)Uu}UR2A; z^q+_IETIppDBDJdQ2h1z7+?%)qMhA*wu)qzT%aO>E6G6r){E=PTDKMy{|EisvLXM> zKhE2htv0ZjO@-SwZ9QpdMaT{txR=};ze%&1nrAPB5TFpf34-%xthV&iL)9GvYRq0z zVT4ZW*d*Ay7lEEgdR26HrM7L_tIO&(d@K_$sn07~BrK|JpZ?;g`QSHaYKYe#y@%7@T-f0mh3RN&M{vsN4dal^~N` zW3R^=!w>gqXfoB5cjGx7{K`PR%#Eo?115ecXR+1pZJpz~@wb(!Bh{0)(kkmozP^sm zkX;sfvzYs$?rYsK4+olhZ-P?@I}#Qm-d4#S1S1aH3pzzNca9x5F?2JarL-awgH&XX zxgUX!3w7}C=OBdXC>z!VqhnyTPRt`O=5T&Y9%g_WQ!IaYvk+v^HvykrcAQfG6}tq?pvT#oI@0&zDxC5UKZwq^K#4tkjDrLv1Jl8qFM^K+;Q64hB;$sAV|4SKDS7tz>UrG8Hh^B8j*jE%r${s{B0h+gYr zwS-YZ$L3&7Q_BBZaP=qc_+{RF*L6Z_sWWVrk%iQn7vS!W^=j>8q zO24Mr^B%vA%tVe9W}{`@>)9ITkY`OQ~=k-}tT#h@g_rKJd}DVRvwxEDkf{bmXgJVHhi6V5x4`6>_p zW+aoOAd{ChpOp2IhK3<(m<@2tRxVaJgvE{;N3&Sb&n>d$4tm~&ze-02vf22#rD&Kv zT{O%tMZ@e&*)Us}iHNZ^aL>F4nMaN4tY1YTvyNREW6%EQzBRm%QtY#2oZ79&i;A@6 zK&BY^w!Rayc<)D5#eMI-d1nWDLekoh>x0g?C*4-_&kN)W^(5>I&{&% zH}HLPGbgw!)#_tSBdNP&V5u6ICyG+M3#LY;97oC4QWyS&)1S`LEk)0RWr)`9rhtRS2%TW;n8joVR_$VeZt3(nrg5 zgA=bDOrYO2KLGujx#(hl=UyKqX$4vveY zpLQ&xgB1oqPm@O#gqOS{1bw5OW!4V`E<$-|2lO++8Rt7FFQcDm6DBz$C&ZwJ)N)5l zX-qTIHSjRfhx$Tauhpg(LC}L1dwqe1iOZ`y=Hg)JD+uP0qobEbRB{aaIBbYyeUt~D z36ZvD;Pjm{cM_x=J^%5c?=5)DgbS~h;5tK1j9N?$Wb72j$ZNY!*F+V=6h>$b>W@mC zbXrG$l0F|FBlu_MJ(Zs%6ndh0oy7?G}mOZ88jTK5zmGm zQGy_j8Y!OQP(QX%^2>^|#rEYYi`1Kkb3Y+<=_8z5kx#I5lq?}vSb7J(3gv1T0i7!pLvuQq@Lk1=ffU`)?gy7^qj9%ft!sxZa+`nhp)}O%*^2~uHnK!{}q+3=}CVBer&P?tY-#s!U zZD?C-zvPb7x<^`5hqi6R%j3I8Hsa-CM<2HIyZgn(XJ2%?jZN}FqFqIp#*ak%=ID;} zLo)eNeh6)9=BJMXBE^Gf7vH|5@+-nD{(abV4lg;n( zs+a84Jcp)A;f(&p)pW|W3gf>0WL3{|IE z9X^@~r777bP+9>gFY^w{ZeAT`Qm?jqc5_MuFQF@3%9r4JKDlerCH4E|3t6`*b4d7_ zGTI>Kj-edMKPFEeSW&%27x;NRA?cCL;5_V1X)IDue7K5sH_{0A%UA zW(*9Ad+=&`6=62~m-^<&WT4Ye9}`&XOohGv$nUqDCEk;YB#=M*)Lx(Vv7wbeB ztu^y9^3vh0Pi$2N0f^mgAPL20wX*~yHsUbvQC4yL=z3OCSvFV^kR3rMJ@o3Xo5fp8 z5Q{^6(w+^tlB%+HotH+@^tAEtxvy}}0>d5h@T+&DnMCDMT%bWngpnG$)~hZ1ueuuX2u*YysAj?(j$C>t?y zeMJx+KSrZ%*$bvwX$>ei>jn1A&!U~WzTt&?qc5*wIohxn?~T64FPZNat|6RCkU4d; z^@~6lTo}S^-@-uhBmhkn)X@sGoIu~aHo2)a0RwpVw(25@lVWv&B7}vm&VoHS`7MjH zbU5pZI-rZB)V32GI%=kPX~F#oD5L;Jnx%%spd+WQjOa4XlNT~08m05E#MR>il|~A1^u4%B0hN|Uf;OMoW$TUBF{9D4Z|8M-R-T* zu*ewv-GQ*pIJwZqwM=U{ml#PioAaTp1SO>+h~Lz`T6*_CM!t}9YWn!P<9nyJ{iWm7 z4+?i(^vK+CeHZ@i-yW#St$OPb2awi`yi{k(AdNRP*W^+UcB(JDHA)Z1T-CRZ?j{Pc z#I72G z8NjCt(?Z;O4t=(7m&IpRB_)BS=DP_jc_}(W*T-XZ>l7zgZ+s`6|9!BqD37+pdsAoKZ;*I73o$`DHh!-)eGe ze#ZY$l>`>MWe%8})%*>iWLn_{OaZC|z(5n5$Aktn1JQRtsT+!mbDzquzl%!Pa@Ibr zw4*O_NgEwnm5Pf@KDOH3XZ!P*KHEL8HBQj`-b^*lE9g*R@wEah$gRSe@^a!UmGz=p z3A^qC5r^zA;|zOU3pFS>{@1bbih0jo0v3izh0_S+i=Vg@PduBs$*TI3XWLuZa2q)V?ObS2r!Em#b6Yz9vu3?Odt)0UlH68~D&3@0 zBq6#L7l~>K5EE1y^f)7#4l&-Ktotzep!iQ-DtYv3#X##w%**Vs%pvKyXsBTv z0$T*A8^9I`Qp(8|$z&_!R@ zfz5a<@j19TmVb&vV!~pv#3P^Dn)x(8i_Vo+4-D;Aofo`m!)_)iLA$wHIP||Xc$@4x zncMPD;n4LFHNU7RYaUI9h+2*=s+M4Ajp%=d zIf~27RPxM<7!sP_Q|2-yaOQH|Wwz|k?wnZ?R`7Eew{Tn9<;N*(15gvIyS`5*H6rqp zfMIx0WA_;+ltp%IDX#>v%K$|b&-2&>6c*FO&GD;F))QG=Bxd5O>P1?fxV4A1u-Hr+ zKQV~;*}B3J5KY%3nY#~q?N>AO?IH%*g>HD>k?LTY2%Dl3* zEcC0C#Hp2`%A=P5zx3Qk^27Ix3*Xc2v0F|gcali$X`#A1 zlQY_7UpDfJ+=hkSoINGFyI*q2quuJ}pwIU336Z73D>6@e`!D%NUVJB#f)%;zrgJmrN!^+D zSqj(x%SAC;cli2WVT6)EYjJ5f2qnp8B)(8`J&F0^g0yH5NE6(c4WwPh<{?QN_exEt z7lrZkW}zfCp+jPaOW$~Qy5F-yh>8#tT+4dUO~KU$Q*h)h;tS-OTS381mqJP4Gw%Cl z#b; zIGFoctjV~30)yZ1^4;y%ZLsPa`F#us=d^T%@Q^_jYt|x0@`XT z&-&#o>YqiH{v4pL(kS;>*m~5KZQEK{oOe#yMwU(%kL_8o`_GZ&%Gc&c>m$hxuT>TG z9uMu-+k>>QAJ36mA0?H2Jh} z)!P>-DXo$aAb~>8Yfs&T3cw)d0suKCFc4dr49EtzzdcY?VYQJFiEPY9eY4>eNgb7= zYlM9yv1MHZz-FJ*&_W(!8v+^Ta@#j=%mwKu$+J|zW_Cm!788N&&Pi%G!ynW3$SMxW zD>l<^OpKebi7X?bq_Z;{WRL+y183`Pm>zzGwsaUmoRX|y^8VeN53S#iG#XGMy&npI zT`2{X?0RXqh`AkH#<^$jApxf&G(%RmRUFcTYDj1~b?F)L8df-iF)D%TAVUO5phW&! zrm!6Z$MOq;)8_0D=pYyj7BoN*>NcvA`Z3T~&Px1JD8X7dcC!HlG4qfa&?T6zK+NQN z04D_(96SE{J+o43aaSvZwE2@E8TVY4Y@~Bc&3r=bks!ZxP^ung+H42~i0;o)7HC)` z*ZIDbk{;T>Gk}gavcPD-dD#!DUlRMgV`P;jh-F~qAzcNfYuTJr58E>Lp(Zz5RqLkz zZf>B8sLgtzu>^k%i}0i((>+y;mnP}-C|l*1Ex<^iWJ^{-c^`P0p@B|MJnee4a~vW) zju4=r^O_loR2=7Q+%i1uiaq1Foq^jM-!IFAqC^Z^?2MM2I;ghC1B{$!Xx>niK!qlw4O{ZW=(>c&n^F^hg|=+R zPALkVThwi0^bZM%eWom4cfh?(zfA1$R%Y_07c5)X_Sm{_tC}W4T$$P9kj$hv=kypD z{%BnI{Xun?_sH0sn^B#czV?ayaNVG~k!aX9GGp4vZjlLS%XZ_?;Wy?Kj|nG?D^D0X zA+inh*za}WzY*s|Zr++zoE45-h`*iGv!1;CYY&3CA+A**l&#c=KsoJocZL zN1kTWk-j4bw-FHHGuvDKaa4!!Byvb7 zIbM+O5igXk{>v`yT&BUAk!nIO33sEGd(P_ExRWS7r@e<0F8vJx^VVMslM*rOdG##^ zVo_@uHM;|*MEC~N6XBu_zHf23uv+jl1ps4#=<6+;o?Z_dKuR9B`-gkTk za=G)*9j`%cWg*un=I&4(wdsj@b2+UVezYXlMs@!A;x}zlF{7TL3*GgazL38SN$u+2 z?4}h#4ztc0oA-D&_ue}akGdBCDxO4IkxaN~3<0mUI6T`Y1f}Cr7VK8yukb54WcBfr z0=-Dke@sAWe|?`L8E%FR(L|U?D&~z9No?Y)*qGUgku~Nu5d#i^Vk7Rvf#cFDVlZ7{{^p?%ams43 zAm~&A|4_#dr$)^>rY3g`JVd6`SZuBx0Sy)FNP4hkuR)~ ztcD~~H{$EdNnIg#vbMzaU@(Ai+z%CY>+2dbC=KK62EH;SM!(swmda4*qLwNy(f~$_ zDbP%(7RyeCO%_7=<|mj5lEu&tw$Oz57+HSWu{0zQMJ1wwnA+@MDK!<3aYih5Q8Dj> z;W6u^Nx8bpM8^WnS3xu2TV;@h>mLm`F+MO>iS$gq5+!B>NpLv z(|Ue`j&5{W`KT+nq@|8!_w6Iodk;5;RWq$Z^l9tYqC2`-5FQ;ujT^C1Bsg(@l73I+ z$sP=eqGYX|T!pH#4yo9bTMW*3NE7i)!ncwAGd&%yIc*7e*4T}kAVYa;H^%_IvIdCl`jN6`yx0)wiWLzark#e)#Z^UVzFBzfV@^ zv78PS6rYYAc_;}sDPy}tLE>&wwV3^7~;xTd9F9nJ#q0sk)Kk=f11tLpV7=i$%- z(!jg{q%fXO+d_tsoF~6Xop$Ne3fg%aHdeVskWnwmme*XR{?(R?)PBeh}80jv-38-Ggg8j@_f16-MK7vTau3!*)gcgdFM0X_{Tj<#;lXzeFkzS+{f zRUA{!|APVh8QQao8#U{el zqX3(Fpkqk|>4DWFN19+jGrIxzn{}X)y~xu=UfOKGsi&ulK#k zi!c`5^9Jmx?RAPx&TPiuoLk=QOj66T9xV5=dvwb8nK9XhG+{j=@C0V+4R2tB#lTq; z9CF&idkcPyg}6}j6>1DG~^-Hrg+2vm+rvS}-21^R{k z6P|Z$-L6MmY~`q|pwWPYQD4M0s4Bo?9i*LN)`Kf95=2Ek;a5bbof14bm`RxJTP|es2@eMy+5<{tl__7 z+r|?&hDMe%8FRy=;WzT3%Qu!SMW>0jZ>c&e@4W*z5oGZp+MXT#m+6sa#%I-j4cWeU zy#1)^{J2fCzu0}-4j`TQOdEHv^ssA2viSaxy&w2`MhcvDer5~O2eR}`+l!b|G#t8W z#8!J}H%-ZbUGlPha4T>lH7VPSgVg9_Xot7fkTFs#bs#mWk!$}&YI5zNtn_RjjhCtP z68;_nhcW7Pw9eM=zrYqvC|Kkr-Jp@{**JJVFDd)MHJVB5so>FJQtyb=bNo6rh$TOy zrfhNQ)UIVW|25^JJGOtf>@VZnMWQrp&Bk7Z#iLXbwY$V-1^s4G8#Vx8c#Uj4kt@)0 zpHREaC8`1~cLfe*K#HAm%%W%@AVk$YRH!T%Ic!}LBGphTT@hj;_q^%jVE`- zAU6^ONJ5Wq$+QaS#A5s94{!e|%8{B6 zzw*@9(XWZ4QWZOh)=FDxTzC#My}twAyG1LPxRwXZK=2S9eZ?tq(r>Ea8R zO{1o4zn)6JiespUmD0-|Y~fA_XZko48rr^rLKEfz8aGG`tlxt|s|^Uq{Hm!0Rz_5V z+%Cn;UNM--s~c9z4@Xpcs;SbrrLyCE!AkjuY)qj6N$ZG+BT9Es4*0P=9%G;GUl@H< zsp-lVl`@DyVxa2iiS;sU8qQt(&6qqYd5j5&#osv)q}~svA+zOy#35kPHY<;Sk%e%C zG7qk)Z{gYq{o*phq{Pfq!DH(8_%5;|q%(&UT5fJ@%>vl>QM*U|6D{E^j1BXsokh%W z;k`aYG#b=pN(mXmrVGF}_m`P_2Jx(ny@TdG;1RLszUzk+jHTM2WbtqVrv+A5n(Uej zJP2-rP%^ZifgB}wO19M^@*>0{Mm^;W=)#zB??&Fygf+b$M%P*5X%^P>LB8|Tc~Y8t zmlo5M?FKeGThF=PNJ0MQRiiG3Yf*&fr9?Kj?Q{Z<0I(P}aJH9VRXU-?(E z(er!zjz>f<*CpfCl6<8angJ7V2=x?=GyjR@A|BTkJ!dSLqsY|F`D)l#AH}!fQt=+c z-W-VG)^AO=F>Uh{(}sR)&GCsg0KxL^z2kP(ri=VKvDcQ$Ud^+aM_aO}+X-F_*Rdyl zPi-oH3}V?t8azDBf5%g8q8!ydoB&tJxPgA98D{egu%zcQHbpqOBVQ9bp`c5&c6IJw zl4O*!5uAd~Z{qJh`yIGCYiWks?10`xtljJdZ&fr#3cUTVYe(1w%u7VV6%)?)79P*U z7wGULlw*EpwV(X~MeU{c_vYxk^uY(pR6DjSr>1wD#0|FjVSO9tb-weu$j2~N@WKTm z$>REyk)Nk9B=Mtp_#WdFHvY2Gs&o_M_>_?h43Dk=8@>5lGus48aOQrIgBET6|I;Kp4n$y@oR+hX6!A|?-d9)pc}hv zem}$7PX)JSY&Gpj0$?VlRLU$H0#>{H?b8Gbj+xnW?_q;QFUxGuG(N}rPZg$?da(m8 z>$E0f1bjny+bU#Yt|=&>nT~)t;mhe7Jyd2crh=d%d1&yu5R^BAjfhToxsb+uQA-76 zd%A2O=RhB3rinbyCg){h|0pl}s5fustfeAEOqUrMnAQQsEatFl%4je~d2XJa?3r)1 zU*bnuT*ETE>5%lu_Y!*gmH~tZdqM(X$II$~`l8*}I0rj?36Q_Rz?X zK0|BDa*B(>2_q{KPArUk2s$k?v?iQWd~rD8;HSm-#8R(rer?K-rbFACt;LP9Gs8>oW0pyTPG(578y$ zpK4KXe;nN=juE~*I+V8Y5+#4kEm!QqUGfmvsQCTZ5F>1xF<{KAvRQWdTxs#2mNH1T z&CdEOoBi*Zf{3CjX)*ya&d^-U?_1=1h(-b{*78Oz$**~2iP+mASykXh~6B^CDxrIV&NmfoOT5m~DEAZL6hqX`{d;DZk5wb}AL_*`zu zrd>juTZ!42pb3mAb}%q3O$b4N(Qt~`o6H~eUPS?}Y#-+v;4((bq%UhQ8baK%+T74( zWRYDvO8$7>UjsNydlp>t3B1os19~5j+*ip$Vz);99#4TV?$ehwNR4k;;}vMhrg}xi zfoH-;#6I>SC?IA`Vt!@f-~wUU5OYE-X%ClmuyT^vsN<*8rGY^Vvn8vfC|AbLe2(fe z3wJ58jyEC@vv97rXV{sa{7>Ev%D3ronA<`sIH<>5ei*->y6G5JdD-Wi$RRAh0D`jD zZ>Bgjp4Z5LJ$dx>LilPOUW{N3^dDU}qMrKH1Q8Ypx?>0nryFNcZ}`2- zxZH^~ofGv3c6OFUQm(m6gX`zcr*AC@Dg#Y1kQU3jj44XXdWu5=9$U6P371zST=LYI z=&Jx+i$W>z9O-^;(LocVe@aL^BQ^1qMWM;C9XasEqky{hQ~G;GW4acySu zlJl3X`}(nU(QwV$6{+dfqcWzAO3!^VKm7K%@au!>&O-M#c#b@tAO7!g;Xj_3Ra_8G zxTHMcya|!R(6H@>p*7d%6ps!k96KShku3S2^l8iXrD)66@fldN!Be7#m1h4kMatnVGU}YjMtF-H6^^XH{|1F4<**nzwP&O|{&w;?kqKL~ za#sy1j9!zU-rgs*to+v@5P3aH5})ex4t340Jpcxgy~jkRcUCnGIXyLZetuoU^*L>` zdxlyDPKmCEFXV&W>Mn-`Nly33id1~(Bi{K67WrIJ_R!^#-?NBu+UbL6h z%zK32gYq4>?S`Jm7TwI=iUZf*m_ph!$2vx%2sm~7n`Q$;aN$~j^3&=Vtg_E z5$1e6@*Q7(Oxbh}z|?73u_hy2G3~}gya>oc#BgmPwh@=XH6&v}?|Hw%A+Kpa+l-(u z+ZxfAtr30M4&%OTqCneuc^)_i!G>nunE;Y98Y2o*6b8^Ct{r?1X_NI(Le5(gPYIbB zsD#`YpoE;nO2`#~O32RxS+o^)CFF4qCFGldO2}a@C1l&i&9CHKv~b788IRuE*6p?L zCLj0oRaZ`{u6zHMSH7P5A9MbA-I=n*%#sV#P+keBQBr(yU0Lhag5v)mU1r&kf95|C z<39q6#ifD%BQn(|Czg9r#9ULxAS!iksl5u$rb`CQ8x(2~7pyfCvA+)Vfgk zM0z7%NW$T+24Z@^L?=cyll?HRR?`TKOB2g0eUfMwKRCu-MCf4+l%sV_ePV1TgWj_M zIsj-46`EOH>hBDwBsT$_wfwyjZFwljx$DLNBk%%`@8h7P%>#oh1-pK_!gBE}@W=eb zq`K6g1q;qVzyFw|`cs3y%G;@N^s42U`YolktDv<|z{2wUBOvbVSh1_!W+|jYXui~i zhqVg<7syUgPUT!TXmjqc2#iiOF%r;}#=CAb^jzC)@?e;;H$vUl#W+@H>} zeuzWJq}66lj@LVrs*%wmzO?pD>QASDwSEYGNQ?|As7{3Gv7?RMki)*#nwq-&yGe1` zRtTDi*Lr$W1a@n4VtBb=c9}004K{@doI=B5bQzt*?d4d%p$n-9TFaF51S_=x@boF* zhp<~~p{^b$=K#Z=)3=;hvSMWYBCYqW*iRs4B--Mx5b$*oiWTSANT1E?7*BEOtRIk} zv{zYcT+3yfa#-Jg^;TYh<$|lx9rbzG`>X!e*{`1ln*9v}zb3~&GpKsISKIZRr8; z*SC|+f93&W+G-YL)h+ETj1CjXt`=&^hSO5lmgm>yeO4HqC43}n|Ct}nKu7R%)_ zk`2FpQqA>@qR!eFI#2LtH%)3gtbe~IP*s!KPVJux|Bu1#D|;q)O1b2EymS~z!m*w! zsrKuy2{V3EN36IeRFW7qPG({YIr20sNcd z&Ae^cHoYR-u$_ceXy47#Me#$*(Mq)XpsI&xphNdaXfcdT{;Q%->OX4J{};&EaB=ju ze#tEx=STmGPni5#xNmI7ZG5B416yu`M&hClt55{lUM2ft4|{9Q(J8WwjakJajRGue zNddA45^TlU)`;yCR>Amp%qzpy_1>E zjkiAL-sb#f40p%Ld-KftmKWIJl3e~LzsN6HoFuSyNLUG4!4&GNbRCxoVKvqcWg}1D zg`UJ{ZM;iKL!yz>0;HzM&3Z6I?F5o@$!W`*%83=LJu|Zub_#$1QhXTBbd}prYhy&;$jB%G# z)80ivhSxG0gZ4}Mk4ve&r?w28iG3M)f%Kpb!-$JnDjnuyS+_-gmb3&87%Yg2CAFfv zzlD<=;<_y_=O6;rA0#o$p*)KFN~Ajb-;kJrszeEwWvvZ;@~dMgGQfXS5V&{PHJ-$Q zD}RH+LPIgB1=LMBQPCjuCQf{!0JB~={s8{yw|=OcFWbec%zrRA_(@8Vu64L;6%tGz zfXtPH0a+ZtIAWzK!|qPNU~cA+%7c*}ay0Tb>A`ZHf1TKQy;+|B!`{0;~Gv% z5|umvr3!fw9=-~`BSsQPkWeXsq6m5`!53C-(d#RS=5RtF0hJJZfQkgL6(4wsrLFfC zY)G)8MMbJ8)?1QD1w?HQpd>&@e&3vH?X_QP?;ZV@jywLNV>pKA>}>WqXYIAun&14s z`Au~*s8@q~J&@{4F}8>S@Ei&pZ0;S_?F2ibBeCVo3|iNNJjD{*mRe{|=}Gm??h2yb ziSCSy+vHJoEr<7Cb6nf7CH=m&*SvrxY-59(uno{RVQVfCP1vOUNU1dWe?23o_5L*4w#XEQ-z;#=KSAHT_aLri`Y- zwYCbpyqRCt&bs!Y1$CiK;#yt>ZQDm5zH(?}Y;A-A$=Pp(E)WD;IlrIyR_MAHw(mLo z0ntzw-Bfm|pymn&PA*_}?kD^n3bU0RtXa?@E0M&0sp!yZbS2YjA0hIf5;fTTqq|2T z{R}q@;|GB{X6rk;ndBRi*$nv@#L#8I((pH=4(4FsBB_52+9R5}RMTn}}OjCc;DUwb7b1#3K zAM}aw#-FV~_UtiDVv2+bH{0h0st&9ngx6(%$~2{^Vph%So#BZ9(ho%4Kmw z&RH6Lq%XAo8=*-Fav;eK1ZP4l4oz72BgN|4;{w=j1*s#4Lb2uW*^}Y0TA*LZ*;{vX zCWAyJ0Wz5q;@|4edK&Is?88ahmNSGBhc4VQr(^w5ukxytZJbb~layt5C2baNv~kPK z_XAu=p2^48`CG%G=g^I9UsAl$H9HCe9>^q#1#3(dyA&zSG(XCE2-_F)W^!1pd!IbS z9wQP-CkaiZ%23Mv3iy<&XM8ufFzGiv(~oUGnZ(I#_7{kflmCynbS+9YB$g_+bG-~i z6re(}R-Fp*G_gY_q7Kki6vnvn>SlopKF1TR&_$W|m6BOIwvQ_?ekdic@F^}T!7Ny3 z!#p{(is`il7;rNbS#dRACC_YPzu<1~%x6h-rN?NX&4ld&iAXh*5{FQ@8dQNv4?cx& z&Y5k#hb7Q!LZGSvfnr>xy{-tDtTR=O3v^_SL{?jnsyP}FK^8;v9Mp=9+e&&)NWQ6X z`?;NS5axn_ZzOFC1Z&wvtyzEx2^l)51tyE0VJlH1=5|-KX3Q!FU^UF^OxT3zQ&M>T zy;54{7+4(lG|L8=@cmO9_uzav^B2-jNIl*FXod7F2&hK#8C^e`!|1;RE^>c%?BNa40BA6bQiBEpq=G4oCzIqO|WQs zVAa^W^wZv5+0Njai+SygXN!jx6aLw|pe*QGTAizCu=>>1`sz z+v-n*+kbIG-50Zogo>^mmesyav|~#|v+OPWvbgoaWuHO*$Y{#P%b%Dhe-@2trFmnA zRxogz^H%6#0bb_Qdn^hSN8ra#+>q21xtu>5-iG@L&XfR504@Rg_oX$V7Z5Cdp)5`}~T+M~9cDN@-kP2FH zO4_@`Q+l_|7QsTyw%2_f{YSSB=BIdTra`owkN3PfSM|psJk{HB(S5B=TKbz^v&~wz zdH1K#_t}(Bm@G5r2|Vy3_~rK-mWcOc;B}L6?X?4qkGqpUzGx73=2JjB=@pV=4TU65 zSv{Q`BnwLljX2;n=_pxPG7g7;_i`_PA+OOJ3C`y5XRq!wWr9x?d7&0h_qsIK!wB0&GEYd!!HEslP`FBu{K zmSibYk|9g+QrXZn#)H@T@I@V>@0<&bu@3SdzncpS`^Kt_!yk`vr;N>%@XL_6G=2h! zi#u?}C-EF$kFuljmvo{tqzj9h%_#7t?(a9--^h%f^MFvz4Xh>6tYQ?9pn5n>D+~uk z@Pq{&rKu58!rTd12{^ToMMY2A-wxH}R3)3T&SaC&i1R|#Dq>dBOHIV6SRgC_bbYOW z7Bk7K8V&1ci25H(Ewy*)6&?d{YkGptI;tWUVV%S>=*|+113$sOco^K8!-mkDX(D)q z!vmgid29H&b`eH%-w4FRVEITT?fR#5NDL)&(Op(Vc^W#biZq2Z+hw%^9A0=0{kLu=?2<%xdlm$!SG4dWzy8xt8K#K~L_2I$S2tE+hrO?_A)azxBi#^J zl2>+a)0m$|nIb4Z#4A?BSV(CwweVUxE9yqf5QJzli0Uck0z=8&r~h7?zra_er&g?< zMto*_k9AuZ9J#Y{oI)z@ee92WTIyn+i)+|_ygVZ(p0vNDXj+g5NqJLpY{DJ=62|t6 zzkd9Rjqk7A_+E9>;ZVYwzVWF=z(P~t-N z$cG;3*Rx~P>fswlw*vTdq8l4wmhkqB<{Du_5=M>N)cZ&q7+Sx&xfe(&He>Yk`M>jV zuB}3476F*_p6`ghY)SlnY(~#JR?a)vP9X22Me)0ZIx^|0;$)WTU2M=m<8zhgBj+A0 zxC;{vhq6BbvBkMahHEsj_&u@<(j=r|f9XsiF7%<;*elb8@Jz&qDh=bZ7MBy#|Ac$3 zLDk8iRu!w+q0@(G4T6i1#0KA0Lw<$36RBkiA<0Cgqfcddo#@8aky=@v`_JvhmVU#m zlMG<0Yn%ObVB75Q($Pu#ePob*xw5S)u(EBJe`VY0WQa0oQS?f8?#rXGquu zL%+t(xpE!!f^HwmuB{Vzb-bjvv2IQ>qPHczuOMmGo?V3{x#P+=^9dy*dlVe}V_AIF zTl1q0W$|0zs%AN5&Aa_o3ClVI1Wx4I?i`Azx|B<=5!KY*B^CtfjWyL{L+g^2VJ__w zcCtj`8IjNx-IF!q95eO;JR_9p-i&Za7FP%^O2oO|2#$pz24XWbXU2O*a1>^05FDvN zEC|ksti461Lg?j(;BZM1>&coa6Lg8q4^H%A5j z-SFVY$QXWNY82#T(t=|}9+{2>I7B6O+{e!3kG8*2%JiN&(P5I8>xF7$>cQGvLQ^z` zLlf}WKl3Ta`5r}67z#sZ?Q;vbO`}-3?SZv1cUf=Q2%AnLZDvode+Q1OZaqJ)n&FjL z89-UN7C6UVs;S8B7Qe<{Q=(MLSEvVHmqrH)k)FPa1ripXl%({K^T~&uzN}1Q zyLJ+h3G4Y^tDeplIWy~+$=TV3jjT)Fo1Gs$OR01@$E{l$e^Bj$aaQLLD=IQa8K>lm z>Pa;W<~Elp=wjPtGmfOAA9rp7aqDWI?zQ=8G4o?{c#vX*mK zxL2x;kxXE>F4qXbi-xjQ?mUIL+H%Ir$eWXT!o-XGF_X5n%<4FNHhN{BGvnOx#VHet zlgC8gkBL2$6uWy-Xe|s!PM&}Yr?};lS8PmK+!R5Jw)BK|(&MkcX2r(1RU3b>xasGy z3FoZ%QRKv|`t=FDmgc5xcquQkKR@zy|N6(!qHTDuKEsn5W+mjutgDE58EzuI2G`CU zJalGO$(TsYoY7_X!)@gL!L|1d3{?#tU;O3x;(dkDvY6O;J!0=$7}`2$eDUXn(GpN- z?=9B%%3dUUWnTj4k)e^Ysp!Y{KcR$DnEFctlF~cSRqgb$rj?#H7X+MgYF{g@R^ zUu4JaEL_ploE^XO(K$^K{O!@lnzm)frF{ef?Sl09js%Dvg#k%N=!H`ceLo>297xa= znjwVVfuwFPT>YU-B!Z1wK~KH6ux&aULpG<~SJ<}1^oT?mRFDM;w}$o+L0-M*@PBu{ zK=hC|$M(Xo4?C0G)E7ril6QM)+@T=^0kvl(5ySK!qqgtwsGzw`7|sN63Te@y-|l;) zGhc?Ub}~(stGzI6Z=H}*^6Decf+c~vEPVU-t?10BCDq4-I?>y%pJEA;UV42UrsGHylD2$g(s@2S z8zQ!E{Ao6nQw6bXPu`z040)|UlI=hs$+kpKvXvWV{^V!qZ{`og7i8v77jc^TS38T< z{)r9QGLuApOS3|o>1Z##*~x}gzcZp!?~Of_{2`nnj0hT@W=AqVLIPdD^dc}&y>SJz znAW)P%Q>ZZr7xH$C6AG*4OPD>>qF;FpE9QHXWWIYc?n+r z+-%4OB$mj5n&N^?2p|q^V+u+#qADCPLy!VQ+2^p9%P)K&^LuynAZ`LjEL7@_XjD_` zZn|pXn{YbY90)b(=+vkO(gH;z&P_04Ymc!}mj{G)3WBf4S+UiBoDEFiVYx0I!%xv80$2QQ{22-N5`rjGACH=JLjA zAdK*sI>t%1W*2aDn4$dCt)BuYLoNAQfRVo>rrf@=>PgUdxw8Oe*~DFw5;MoA*`q(5 zIX33@ea(cQJVH;Sd)7hOStb7BIkMPM+1*Q<^%Rl9k99;JPhTr8y#uP+G zW!2}SEp~pdrTHlvev=nDuOM>nhv_9ZL}IQiEb9&r$vK0E&dDnIStO=^Wy~3cWvkH^ zd-fkfKUfrMfscPjVf3Y#*f$ene_s_U!u!MHi`(+{?wr^s$SE=-z$x-c52wgx;S`CY z__FRj2e+D%bgpH3NkN*cHy%9^qYyR zuwP%jrzOFT-4K&;e>w^N7z;@&5V_^Ma2gN_P%=jkR%1xTwKX)JNHrPa_qLx2)t5Jo z)7Iqum-`?AFgr zF`7;Z2|JL^(sMUg|1kObA|v;AB9RaHFZpj4POWb04Xo>`EU!>1$SV!DY!4j*VHU1C z@cGo$aRJ^UEvNk*4rg{z=4ZeA3OCRgaAv*8U{%(*$j9Eh^?puKZRs zO>V7*SQ5P#&A$kFm3qI8RGdd2Vi!NpWIFX;9Dd5%b>lQ~E%l!gF#!q2?tR=}21&}Z z?!P$vv24BTybEHc*OC-;()o5#nY8INV;5lg)*t?omlbDShKVwe$!?u{|FL@)v$D%% zcdVjBV%ml=bkP?Q>fL>(Wqw^A%IzMU?1E&0H8@vOww+cOdu8~ar=A-SKLA>CBS%yjYRL;YY|fUiVyC#?rR%GszGdSp7o=3Xdl#`uI! zE@+tAjhVr4VnIV%2roy1KqHI{!kgGi6Rero~u2pjTOC)Y}&lk#Td7 z9x``$Kq1>dsVN(=S~=hrdKy1BParZVz~%{wBz!=^v##Jl@xpE3&pw@#khS3jt4afVwf zCb*oKcn{1dX10}+l5$C!+%iu#!n;bX_Q z&`i~MoD4@C6q<8cxY)uvnOCr0RMW8hd-HJBW`t!HWzan!=Wc2kLIBxcNOoMVaHH2hpo-S-1LG=+!oKbXm>kdC@*GvAIT3+q`wN z9uOr{9^U@*!Nn2IC`CydP(pbPaNLbTGua)cBsWRt$kWH|2SV_Lai@$vOk|#aU+o#h zCHeN%mKsmybj%7flO%iW*dYKN1EcZmruV$v38#_1EnuYt|77xX3I)=k)8A_&8+~6c zj>#_}@HJ}Nw!d-Rs%Gj_!IxzZkZ{X@gH@l5@)9UwgQBKwtsh_2E>h$8s^Y=5acXtl zk+hoHxF5M{d@bs*y_;^y4=c+XMR8kc8roBFahq^L;n0QAp#q4akdXW%amsZE^&ce2 zJT?^pSkAj=1{DYL81^+H(jn4^1Uc`IE?>ZM=yl52GKE?{RcQkYF0BHmN{=&zYX#MQhXW;JduZj-^qQPTVojw`$@ z{&A5_0f3G!^4l`I!T(aHW|DZXAPY&zT>?ZVs~JAz{Yr}>SY%@8xG>kXJD}8Yx#L4c zofyhC?|2v>H=t1aLcc<7a>r+<7F1F2@AazALbK`_9LTASOh&@S`!Mf72Vr@paGU;0HukkxH?N1xL)#4(nrU( zf0?KR)?S@%^_d7o_^kGyd_-yQ@Z-G;$Q0kF#rC^O7Uqn|RA_rEbZmYGodm4OPQ}0d z9Ru&z0h7GLXZ9jdRh70ZY`WUZ@!R!wKQ{{&xRK`Up ziuV!ofY-qn@fU84E3S{|Ka9H^Mw&3->ayjgde zrhwaNil}zN5)tvjS-%u2aLwS(rzVe_2!n}#Zh$c-5pOu(cj4XPzKpb}G?6m=P^!F0 zYhI?wAQDedz>X#qkMaRltAM#QTfKVq_h5%t9hX`@kVp!!C4yMd*nEXV#u!kAW|jB8 znT#=Zp-I)8vYLcOY-U-f(m|Rt5{KaNj}2OrepA;H7csKMSl~FXpZO#&5o$e91=e12 zlJ6o;_b9=T9pU^Vh1HZ*Ht&|YfxH#hF6elD&t~et)?9MDI_D+OnyboWuwdX0F3=- zR!I)P+4LV5?<$P`CmTgxjH)*VXT`hnZk4rdGkT1O7RSVfrOQ8=M@r2k^Hk4%;3BVF#U4U_+HOSgl!{)pm#57lG=+(+uJdG|0RVra3=K*6)cHk>qq z>f1Kqll)q2-?^L(N=`q&zArrZ7sJjveu!_k;FNr00^JB{V|7imO zZW3OT&*GQD3C_cQWNEJ98;M3s;hJf?$O6w>DhUVfx*_cZgMG{+lSurPvltS{omD-X z?_y?c`J!&yoV*(h3mWeqgU3bkO3xY`-12K#bn>c5jdu>8xdqxxEy5DunABev-P|#3 z3?XK(^XR0NzwD6ieI8jg`h7Dt9@LqkF8+&bWMxl_o z$*TZaT8|L}s)}ONA8qGPLq#+qD(}#&om7*A0@}z;@)Lp35)-SVsB%O7*3#%}6(SKx zkEmfJ;n&6vbj4;V2!CF;PKE4A@w8PSj zgslE81#VA^dM`injr|yy)it;YGcHD`Oil)4jXEpm_^bESe=o4W&4|j#Fr7X=(o@r(c4&l+K zi%3ngeO5}>oB(7&pcwE{0A1(!tM^)2lm#zC&}S0Va^tZJOR*h&x9;!Nm7-G6L81K< zyT3CxWgjyWmi7H6kbT1gFO?IDaxkCvymPpSRPS=6&d1{9%f8j0^;dq{b#QHmF?(=| zKEb6V3sjsI(A*hr|Ke#kk%sJfVJ-;VB+dyc1VrI~^B z%X(rPMZUoRj(PVm)5h|}NfXw}E#8C4!UIaDtVfFif)oahvs;(6UA}?>j=DHz5{XzN z>1G{Nk~4%#vJNWAA*_-VAIGb;{S%eB37kks*Xht{GAAm)9xc5UtCI>&_Mz3VL+ALj zYT0`O8?$}!Y&n|5eKVeUP_O3M_@v99GSJ$~27t?1#_9GC;t>mg?fRw!MZ0AE&p&*& z2$PTmlQKy-c-I6pW`k)#fZKl((&DRokF^;_zHz`|7zuK2i@#r_HSett#JIT@eqQJX zFBKTJdMW>Mud8AcoJ+z`U9fCM6$Zbg*)@4>UbBkTU$U5;2i6{c;t<*>cve;9Evw-a zAvtZ*)K>XIH)Asm^+ceAG?{l9FqGF_il}_X!Zzy1wml=ILt6e?UQhVK1ubJ(>${{gj2)=Z^otywTeU3>`_V5G+}WB5%wBB z$0k#A;)!4n9cxQ9>oYI2UWgh9hcOc=S61s;@(@Ij@1>xkQVekygwtIyjL<(3K#BF+M{vlFtxb~uk7ZIWcs-|b;G8WW?EIhj>ZL4c-=ca789=~F?ds@yc z59pqKa!~i|GxgoGGd4U$KL7Qydv=Fx%0^x!cjqTE$>?ybzn7oqn75Nk~T0$Tv@K&R4`jbRS_t)lf9NlGnsA3&`qEq;t(K9eWFc8 z7{PiG=o_)c`tEFmlRbAf{tL*oAic=YAiW4HAn9u1d(p|_G7i?;n*rI-nvL10L)@*6 zyT1pHE#hwGbamSn?PN;!>4}y>{!uR4ZOvyz5i`8*9p3;Z(}1^y2ML`0J7nK%a*O=H z?8kQDU9?~GLgB^tz$Zrag;fy5j5H#ZfXufrUbc~J0^gaj15T+D;93P1=J|B8qmEKc zGXdAA2bEY7ump_efB@a1D6%)fnc}^W;DSS-d;k;V+Z&NaNpRynfx{MY= z8~0`MC=^sZr)UMuufideWsMd|?Mg417kuo~)Kj1!t;&m29vx*&r10{GYVceIgMvJi7P0bU;rcY)rP1y#@aJfQhwQir^U;k2HRo%{3aDd>s!p@jl@p50E&;3o2o? zpij$xi#K%nEUWQVs{~mbU0?M1+KB>ksIjx%HS!;6-F<=9;z&I#$k;Vr`7OS-Hwa-V zqg;u!xd1%SkO%U*V6LZXiLHi9C}k}G?qp9%Kjfx`)#mC8mrgxZu}DO5ppvg zI&V%GPrqLV5>rqc!S=+N`3C ztf8|8*FKO{GBy(POhwFNXp0Sx|B|ed(UF+DD`IY+o)@i*iT!<#*yk69hQRZG@AzT> zq6cGQCoK$B!SH|A*!Vm8#$P`UP1#nG=l>_@bk-OD#ch6uo3cT{6PG%6MbnTTBldPQ zEeeI<^`Co5sQx-su)T^3w(r(Vdo7x2UuQPcK0EK%L&MxnJ63kn{x&VXbXB+d$55=6 zGrBBoPhKMV7?)uX z0MpdM(INEL4s;_)V@{9PA44aZMDkBj|HeQKr?8QTOjg-7iFp1&H~D-1&&oqAE>))_ zk0d8c4|ycRWPf?2kYaM4$^KQ8Jo15r5d{zUc4rGT{I5ydom}wXj?t)EOW8jyBT~+6 zFKNqV@g&t;$<*cY9~*4x@A&=BE5&y-Ex%bn(L(sCC(3i*J&S%7#}%Troi;TrRX8BX zihSrGD!rhBFy9fLJPMz~klPkJI)Y(+3?b{MkVL?brhwY3uo94rsbqinCOUFid1P6A zVI5noy{GZX#k~h`uQ^4>81u-2u@wC^Kbym6Ysy`YD+=K7JUXGScrVedYUS*NcIrKZ znRO%vbDQuOxV(!}$ZrKQb_=SLS-tnr{zgn%oU)c|^bICWGrHFzq*uZ+*_qgR+GKH#<}zx0ukD(PP8rAB z-m$L`e5qkDmr8L6VJqTw`cUhA6;?qu zoY&fGWI%Ve9|U!0dtBe0ZRqR&5~l6&_mx8;XlFq6 z&Uy9kF~&NlU(d1q_|T5?ngR8aBS5|aD*G-qqhfyI4s z*+n;%UCL!weF_FnE?}Z+DJ?hqO23ydVEo9CJi1P4 zG81@hSEqA#x%Tsu;_FUfaM?Z_FO%ry>%4PnSUny*4)<_M4Vjc^0i!@^VsVJ<-Kj*N zm#Qkg0s)?I%IaCY7%ELSShkWM8fa=W9THlx%;8`4hvWbJw%SE`;%0odsW65yH5x6A z8%gR-`Q&?-u{Ha5(T4Nerd{z_8$Q{sQvb(8!xfCm?(XKL5h-F;6E}A)Ruf%YiR}F?* z&eiVej;A@QW3eMjFl+%S_B89ZhKja2mLX}jwJb_&Bj#W}ibHZscx*c#6k8*Qu(k}a z5x-)lcI-|PyR-(LEkssPNg?r@B;BMPHso)TuPfFs;<4MnzfU9wDTwmF3)Ea_{{ z5=Zv_XcQ$?>iKmNZgw|Lo|ra35P>N^<4G+DR8S5(=PL#uUFguR@%!2Zu=OzY#6Ixd0-JF2`4ABTc zWt!vh7o%$4vOoX~7-np~KBStcom43RbKVL=PLOIw^`e(J9X+YV!*2dtaXIx~T*SG~ zS`9gEVh`J+s2*oy3pXEKXJW>h50g)@Q?M#^neG+c`Y{3dFpz4E;n*vZ%XhsJn=m#r z;hfC))N5C4OkK6H@8YH>ViO+8NVsF=59)iK)NAs{l$_eUNWX%}53}mOOz72TM6Vrr zDLJp?MPff3Fm!fS$u*IfgQLr?={EPOjvPym$iP=&SvuX9IN4CyNI9?TN|JO}M z>O%c1h+6V8gBM&h{GKZjg}lF|zYw)8;+*AZhdb8?zA!un#P*5+&Z9Xb-DP_bA`pc?|Btw6aWx6aSZHE5TR~$(f%O|D( zg>f5!Hm1;uB%V}&OD0ZmX*xvJV}wav#FqiH+EWpn8s8Oa$a8$ukc@v#^`k6*v#KHM z0@aX1iZUv9_+T^AgZM);gxf(B+GeKEb_G#roh5-3+Ab}HmbCw%QL2WlSKCZ~OMm!p za7H-5T_k0BReyUMws#$E*m}WL>*>+T>TSBQ{;n2vMt1=+EbD^t}CzW1H=h)Q#Pq;+CW@d`QAAA*uJWPEtjAVuYyS^XtES(=IP_ zLJPXmjPnnCJVsk3$52No$FjQMMAUJSRug8zX8J~d-D+4GIRiOSh60C3R}B1xeN=_o z)1F!i-9VESx&}bF+653dO6aZqUD(%tD?ALS0>I;LkhW0?%jPfD0^7-fn3x(}u!_u! z(2z?+QZ)zuCaq&l$8aGPHhGWr6)81-{^^m^l0bx<8%PbJ`@3AIxT0&S*O!n}_)%r> zX`h#}ii##MPN7KJXMOmEVyxWHzZ@mWSXrjFwTg<@3!FmcyZ-A#7c*s7_4G6cB1Poz zS)2XmYy;^%A6~#}N9U|o?@6SY_T}IGH#jg`;l{{zoW%23gL!d?PO9m%TNswP&NG_W zpaQJr%kAy{t?!o6&wJ!O|3%_iw834k`5a6)c?VT4+&{D9H9yr}nJAMbSi|YDv@f!> z7_-kpH1R^>%r$&b9)zy2hE#9_U5z|FrVFY-kH`duV|oEUQk zB$d+2KdaEj;QqGdR`bJc;ZAB&9B=|?(QCL=YQK&vYzSoA- zLG`=tO4_hEul@u4uR5=O(_JYW&gfp>4Szd*Q2ldvB@JKGzc6}xUP}9^N#%(j#HH-W z?$w?!{@kw;*Q3}i2FG5;|BqCKa<@gA#)J||m-nx)x;wojF;bScyD+*=G}s;ljpSU^ z+I?{6eW7*4aQ^%IE>Ok-cD$x8 z2C}&&yq@gntzRUrfV#^G1X$erTVYaW=}F}xx`=Okv!6(26BlWARpZ4KhyH(PJu<(u z&-yuiKbh2j!K#Trj*Bbq=sD|uZ~0=(t9y66Qg`iuPcA#qI0yOTZ^lu_xd#UGF1ebF z{&}R&<@c@fJwzzW#|jQGrFpwS3#N`^UCI}r6<3kVzP5*3!2?Fk8!D={4o2`+9bIHF zWi!q+g1KclMD``ASEV=mRJ;+t^+CeG4QYcKb}`+TlhOT#nN#b_NB_N`WLkas^uHGh z1!qvbo6%`YOCn$dj9%C=uR)qrvJHrPGEpeFmF8bKNt?mtbF;8F$=1g#> z7^7`$knc@;X2K~H7*14GIPZdbkNP@__O)o9BXh!V60JLa7Tb6-@?*L=AUp=NWs^KF zVI}-79SAzEz+u=KWY9TD@LvH;9Fp_Mv}N7EJqR_R0CXOraCs3;f*yxt*DiNvUgbP+ zk;cGbRkAVLWVL1th~3h%@7(u-P=`@3x`5SRHD^J)=J4)5JNvjoBZ3>wR=nd14Xgk1p;FK&D5_ z^bCtJ0@~6)$%+|;suEAK@4Ve%*bphk*f8-_^BkHag@{vg{=1M$_K~Ewk;2uh&*T(MitzBxBUcg%<;XQ{;mxv6g$zD%%mYtjbMM zy@yANWYgEi3x6R!=Vypg*KM3n47G6I-3-l_RJj3Hy}>*F*dh; z6%Clr0p|44&YGW*3zvp#rJW0%+4d}NEbTn$k<7Svk6&xHe;mKoY~`0QvTKt?Zl_hG zm;5MFMksF2y!4W@fv)hf>z4nUhLwfF=*%|t#1SP+uULeVx~8Jpw@u6WDzh-!JufMB zQgT+?!L;~}QQgZ1B@b?E?Hk{5Y4@_$q`_^Q@$y_@O|qnK-13nq$NPEqdZS+ImWA>6 z^HUt}fqn8Idwl=S$7`tc&8M(!-oXd)8KK+X3f;|Y($LDWwUtaLJ;U!2z`kHG zT;7UFs@tNyOey{h!YxVIiDd5_VyG@nc1>v^>ClDC_7k~99EKevs`tnylYUeNY4Vd0 zRV(y3c@(O)q^*dIdEEF7nh^r6zz!H;4P_VZPT{y z9LW6Dk_+gwNR{L6$@Vv+tPSe6Apv?xbs4fF$!z|K;YG6iY9n5~dG6rEBWcZtv+aKW z$5@CSFX0i;AQ%~RMrS|DwR_Dc{g!VH@cOzUs~ zqB8RDH!G*@`&A)D)hwB;35XUXr{&%oWt3Y^92`Z{4QqQ;N7v;uq9KQWm`740n>Mp8 zTUPw-lZsB| zAt|Ni^sxuav5tYZEtpQHE>spy8Cd)^s+sbcyElvl zKG$f_cnO08k5m_i)Vi#g$iTMDbj{>H)pyAy`w;U<*8CkNixTaF(*`}+97v%a!(_A_ADmgd#NgCzPNF|l=*WYtTO zE%#!y!S1qiWW~Lq_>Pl7%b_FN&_~nBl(Oc8Cq}lX-UnRPEj^A&w!>ASPU%20YhkEU zcFw*LFQ4pIe_E-gL*7-ZzjRR_pm z-_JhsmEl`Q6HWEeqWIl{e@l9*IGF*=#mRJWxQbL9*{e#@c+GVpF?jr^k-}3;p>bC^ zNKy6=2YMv@qWCrf3;dp^@&6s$k^3#SBVlPfk_(u-Ho|zZu6t&4Vi{hp+y%k5PJL-s zv;i~{eDx`4BbCWN!FD9^U|eRZ=$V~!j%>v?zogF?(LEb31&_Ut^mHoujifZcsE{7{ zjifZcF5e?o9;D1hE;87N3OZC!G7Nj#U=U8JMd{7o8b2v@yg*bbufDz#xgBh2=-W0U zkU*;ocK4rAboNraW8|)fVtb}m_iUaU+jDt!|N65~6<6gim$Yicy!1KXt!V2T_{Z>4 z)fw*2Nf*NvA=_nGC;kO>kUM&R`{AHIY@J6REcf~#ks!MeA$9A` zDVrNf#pkSbG7jAeWopZFmvo2@@b+{o^6avK&FS=|mM;#o%G1iY*`i|5ZsD4J2Xv|w zBP?kn6Ozt{V&6lDyi!<6O5tQ_30laf#&B?Iq1D3Oy0r@jHEld^fyDZ;Slmb z3(ldd{I_jJCn>LO;a#8wb}|YGF2h1~o9JGpV25Ow%}7wRrgCSAv0?DX5T{lbP=5&7MN)kU*Hz+?~xgd?_tVf^J$}ZJIL076+$V-YX(zBpv;0z?)TTc1K zv3nO2S>%FU;?Ook<&6iw;vrDuBCTGO@yM#HxtEpvd$I_f9dmuII7j^(2nXe5)MW`! z8q1Wiei*qReowjupyNtWFln2{YVf5ihtu{nG`Bv0MX)ZA}v+<>K(-7hu7y!t^YJQr>)sGuKC-a*(D>qS3!_WX8apv| zU{z?ynd6J2h0%pEvAcW3e)Qh#rsUX!JNhMz?H7Oj_!S%9U%BzU>ZZe?gf)HRQ;R@Y zJq6Wb%7$fR_D`Jv*0F!&j_jo8{}m)#2HQw}XAlSff7s9Nxoxbn@kzD>gwISRb z>SG;=p4zX8kCE$~OEMWZ=qRWy4vL)6E;C6uiDd7D=Lo}0S4kT`+Dx)l1(Ix#h8npC z`AIL$|Mnp8r$Z*muvQ30TvQqn8 zqI`E~X9g8Q^q~z$WYU*CM|`~px{27@2%GFXjwqbMDKbN?_BO}%g|5{4Vhtk(i9VAr zBJ?PA(UuL`HsK|V|A~SsHOt%(J+t`78>@*-x7fAxL#s-$UYdVQ-_`~`mxh^+eb-Z| zNpkhlD^CneQTO(>v(6~?tIcAZl=g`zW3nAJRa92GR zvJI>mrwJXGKWYQDInn|pqnFv76yG&vv@Gjj}W ztDVwJOB~{do;Mc`dacoDGMoO{_+gJ>h0mn}49f*U4KqS!7CqX&L>{TY3@}W^WC0Bo zKz@m*U!?<_NCA~^jf#4g5->F+a+Cmh^bE5Kc0tvVnp-Q!R6;!j-s%77{M8?5FH=jX z?dM@TJ63nC*P0=QCV&be^G?Q-&GZPEtE~9P>OD!UMT1DqH<8OOjP2FKY@2G*i5Dpt z5MW_qoxNrxtqIo4FjTd#33N!%arVy;NKq{{beW!RI3ex;&u^M`Qn;yA?E~$XGl<3% ze#1)xDi~z3#F$UGN*>T5*-%bqyDXz@ow_WRw%uE?((~iMB$~pmnnw|bpi$I6%h=me zs(B!WS@>a(fd$;tlgw)7ii;T0m&}bywpr0^b*e*(CXh>-6qG`i!7Z!^+Z<_=vFR z=XK7mRK%e>f0M>%Y8*%4!Arba`L1K%7~Uo}0i=NiQbun2VVGo}qT!lc+wJe^VRoZH zQ-oQ6);>JU61RF3tD$omU4(XbJ^R(sgSKW<2}xb%+B}qy1c3dOkW=}yIFh!1oKiwA zK?m(SW~j~yeGN=BBvT9xtcWfT>Y&|HF*D?+g3N{r5_@an0N(Yz9V0hCX7tV8@HB23 zXLdKSQZnaRaA+XV(y9h6iauau5EH2!4I5bG*4!ELC7Pwp$PGYehGFjnMyZ$dHrCBa zc+cC{lHO!p(s1yuEL)-4Drcb@&U)=5Gs0abc)o&$iJTnEp>@zR!nJPVjbaneBX4|d z1S%>bESrxz_TX>q;@)g4F(fK;SzP8FEdc$hILZO6MBWh1wnYkCvscNcuOyWW5R#$W z8DeP+motzv-nWx=By|MUg?FH?evawOKU7oQ!h?dgAXEnnC zp9Q0=`2Lc7527V z9%h6#_P8Omq`%BcI9|4qlXs4#25n83pg?VCq-<*6vZnum3kZ6#)tph*^a@_WZKUz^ zvZj@xgzKvwYucY3cm0?ZOczIoq zP!(Pd>8jlb>fO4x5h>O2lkRVjTSzCY{d2$M&e@%{`3-eu6YUO(n|zvRhZo1+6rh4^ z3@<(JCO>okZ&!94z2tLlLXBX^#AUO2x+Z{&}W=mCv>#ylgGeL$9Y^{;_PMG-F zk(a7QqY|)dA9}O%k3aj2T>P9>ZRUHDeyq!e2u#M)M5Iw|LFGS~XEUAtkAAke^ZfN> z)*|zMI##Xop?9`G-$hZGwz90bWj0CEp3RnDJvSB!Y(T}TNMYOU>HM+Qwzq zdXumxvVSbgO8INti(Ok!;0e4ZvS0}#-iV%R0Jg0P!)40fiY@s$mY(K3Y@n$vvbrrZ zXCV%u=B{Mv=%oF%Re6vsd}cOfGtK_Lc(xqP=)M_`OIF7f5rE@h9xD68QEYNJj5`X$(-1{tF~TVObJp)w35!7+)B+RYh;L}BMojVSF>%AKR@ zbFeyc_3GYZOLE7RZ)R9mGP1`T2ORyhW3eyySJ8(Kv|(Hh{8euM`q<3y+V)6;A&p=S zpKSpdmR=FO&IL|lXNyAG1}w|1`JRi9)ju$+Ta4v<4d9qzov%$>100`XIBjcoue^6e zd*&-1dcX7f3A=w(=^W2x5;_f_0vpm;5fP#Hy#?qDNZ@)r1sM zPXnn{fz*=i0ToxmNkq;?4=>L(q}9-gEzPlou?g$JdZLQpB54-HPIe`RFE#G`DcQLK+7XT<1CVHaeMr*%09(2M|cij0_Z+|Bm} zg4S%YGVf4N^s%pb7Pv72*^b9sJhqwC`+{jY(8s>Z0czcl>#`znxuIDZwi&MV#!zRA zv1Bh1nMYIuWg9zVJJ&zyw7v6QEpF2b1KIn_K4WR{y4IZSm=^tUM8@M4ZaI7~49Cco zGmo2N7ld|EtyvYCLM}BufdPzYf=DLcQ ziKEL-L#yq32G`!5RWdvhbN1-6&EzKXeCT4d+NRdp_^&W}2h@~3@LzfKUqP`w^8F~b zQ>M2Xe*N!go3l;5=g!&8SV0Ck77)Ozk_|FQgHH01K|*pFAcKUHh@W9(a*uC8VhOU1 z7X*e#ydX%rtKse8rs0R2krKm&ZeY7e82`g2L|j8d!{HDa{~sV2W`Z=q#F_6Ys&+%e z-!n8M2pD=9^ch;svHObLJgnkIJ|x^o_*wSCYNQqsj-C)eGKSU)AU8vZt(1&1<7u8J{;eyRGz|gSOv{BqFY4_`>qmT?HloLDwlC^!L2-Rsgty zapk9-I=1ABh2ly*s#Ht1_lye_iTvdb8Zou~8raGPl`yBd*9{gwX{l{nmBmhu09U2X z^y@R>HQqo?d}av(7{3X*hxSYGH5sp|YA8+UD)5@vz||ga#{LDr;vG%UQDSd0ke5BM zfC`d&PPuV~aV?lMiJ~hF!Da@5RF5(>Py~r?b28ABGHQOWwm^(5C@rx^df0KwR&s7= zVcrSDyg*&nyoY=j8gKd&!RZlikn>*dmLE6B#@Ti7`Y&3?J+vt(g~em17EqbGB>*aj zygcKD&}V1F{(b6nw(m3l5l?*kfZu{sIxDJ3jcrRiTdvFi*BW`5zc)z~OyJQ52EOGb zyZfe?cjgEL#JLnTbj@m{CZKr2#*gXeFYNbo|7|s5dm;pQHst-Xi0v-0sBd1!*Z(TxL zo{I<)f9JKC8#RvhDlh{m!&Co)Y%GawU$_Nn7R5i$=}=d3Zdh{RxNqkBopJuQg-;+S z5TF@Bb2gLL)j%jDZrP7Svl+=G7X}_hZzjX6M$6c&SeOcFp}%Ituq?rMuzrmcbYVA? zCelkJzHnvUxaC)_Xlk98)%HIbaXWv!qG>Z;zLF7_ikIJJ$EQ}!X?h}*uzca1 zrbj~wsS6)#`dfBfD(JTD_}im%qO}Xt>uXLN6TJq;kIyG()!sF@t>=oL-dgwl5ZWul z*H100=K8dymk!)(B8i1@JHmBaglcq zZP091Or}9uFBKhnWtInX24`LROIK|D?|d^6Kb+u*WX&W(Q|~F>c40*D&4k^!V*q7PwYi!fLl39v?BuCk(9;nSl-3KG!yAOa9a_ zgQPO3M|ew1R3`mT8rW$o9Rd&b>70bD4KMKO_>Annot5*lA!FQiQI*%i&Y4Mv*SYT`&k9%HiV=S^Y%lFsI&`K zM2IW=g9ZelTrXjch2mx8p=Py_|5s*`5REz0;;6rK6jI6HO+{`j6%%2KGwk5^Z9kdB zN$i5L_})H+Y7i)T!e9{~-1?cNE35!jHk?HgFsKW)1_Wll9G|1tqRBhYO#1^O32otM z5keWgn>qg=lwN9kHAc>bJ#`to0vW>l`==oS|`Qg1yKj(&IrqBrR-{uW5`l- zc%x%r5bFx>5j^3t`IH91!e~ENiGWoC+?Hp6sE{fj-33D`bR!8QiNGB3aM!S}noQ7n z?RRKF)#y=;6w*94A*K4GZIY8>llxovR#K%Yz^(3L2;J8)mHCdaV}BLkv3^8^P~lqG zB?MJc;Yo#BOq$AjTZ#1N!7HGL+(MI??@`VsY7O0}g(r90<{hv1-STL+3BB%lasMl2 zoeSe9|8ac5fK$f**8>CZS@7AfNuYp7nT43z8HGpxI~nger#z`t(GeoU=)~egdFK1T z+F@J&2wQBgbdqA`b2LFf4l~_+#078)lK=k!qXClVI!g+3i05geho{3 zM%josD(0@vp|t1kMHcsH#S|EFE0|@OuyREPXH^$|Jh&%Y=g#|^$ZgBEEJ>o7ZwjTt z!)yN%h1iTynsbA~QYcbpumAc_I<1Ep9N;-@jY$^2U^a4aeDQ(8=;D~zA5?`}29GcP$N1u} z$Ht|iSKCJ`H-1>%6pP+%{Sv!Y>{RoDLKyw|B-IfW?n$O z+RBWQHVNliP8k!G-u`4B*>%li+D9;nyeK`s^rX0kil2w#2`R?DaOxp~bEPNZ*wy&| zvC(BUqNXi(YjxA+OOrNKVA>;V{^NZqtKYr6p;>!GhT=s{O)sqh= zaE**C2%%AAG3cnaMK&fc8o@I1>e~WK5rUhvJ-IGGhfaU5O&FBeoFq9^kQ^IvQ+9-h z>AU@nBTVS=RcjZ(QzJa}KX>Kca+*iI8m)w@u21`>adZrN#gYaZ-%l=&R{pIt5z4T- zmZAefCfa^hQsr)(S^C4u^2Q%9mMu+#WnNs$wke}t$C*XXa0ge@Fp@Dk>T3*HB%QQ& zx7p^^IBq$QbLS5&da7<*(%-IA%14*gKN9Q@efbRZohU+CPLBUfg5GTSv80a1Aw(iA z7aTz6$u=pL+@#Q*$-&L*5(ftXb^rwj*cH{QCBHaD>^-FW`^F#MLHD=1d+)*$pDH)- zNWtCMaMJ$`xg>#6?Cre$8RV$_Jp&ly{!wsfqD_uA*1(L=Fi+H0@F1cE`y)1ZO`Mm^ z#VK?V2?*Q)u!xDLea40<^fcIm1>=1O)wj{rI#AyhzE2<{G9mdqgUhShO{0J8;{Qxh z$suYe=Lq=F=uI$4n%z_d!@%FIntv;{VLq$zrdKGDZkUivpE9ORc4C`)%YbGp35Tbt zwR4#^h*IGlVrgx){cT7fHRFkz z$;F1+;lU}AlXcHgF;3_f6(o^_)a?7#5GsU+bb*Pn9QaxcHRmn7Yq0?NNNUD89VY}a z-)HX?ch5Sk)@FFzbTWXLN4`u9t7c^ohv2)b)T>&2eGmCdp^6I8wTa}Mfiu=Q;;?lX zhB3{Zy_@M8CuElL+!1rjw%Fm2A99wE!kRKNlMZTr!x`+atkAIE6?vvQ4ZAuQQ5nF* zwK5m?1|wWYxk1htSd#{_tuv)10MYd|RO+cEuTmDK(AYS^oy2=Lxmo#wlaXh#XoTsL z`uML0P~l;=P%1-0C|4ipbd6w>={uoStb?OD@+Tn3zv9w4&LX%DQOA z81!77cESOfW!->h4+xG;JEXIb9uA^vtdR%Jxe%-OAo-+at@ZBA7sgWpx^-QqV?MZQ zEaV}6kj1Uhs#EVFF%UhbmV#n;iqgMUd-5oVBM;YY99(wihTr8y#uP+GW!2}O)N4n6 zuci4Z8-9})IjfqWrgNM$^D*0I?rhjG38HHu5dk?Oi{fE#G z7KK{Q8eiN|7=0-w_RYlD-&ci-@c!`l;Y-wLDiayql>$m>#^XxwXjWO_hUikaBFD*XC z?+;bQ|A3&`g?&G-z=difOwFDBR_KCCf=ffo`TfMVLf5^}oWoxAKFnM8hed|PBj*p8 z9pgYj!l&eMhlqi4xb};MA)OpNz=3Rl7YBC{rj#!)m@4n^On_IB447=s4jeo)Hr%{ z%HC#(ELZVfHbZG5*BJZ$kh32lGHdJRYP;3{ZihuA`PjdCKIO|eWu~Rytm-hcyuljl zBepLR=7e1KY{5+`y68floeIQ)oeXpJ%o0C56d1wkJxEVt`0Qz-0NH&t$aNkY5)m%` z5=k

    _i&^qPvoHrb4gD%sF38wLPFn`b0SzCqO+X^Be=xv`OZBy+K`b3viZ8mCj; zf;;L0G9?>4Ur~ub$YzpLp&9<pIG zjy@u*-7=@6zZTo6^MAVq9Gsw|tWmeTgCdB6lq{D-8(v{7Rtfa4^hCANWB?5;(-i7e z*v@2la7Z>vcKc~4&JGK`dub4MKL!OC!^qYXe_zG740v@s5v8L^**?K4L+W-K-6ADjOuI&L`55TJrN`z0Qymw zc?mTI<$M-{0(#AHZwgthZ9#LO*MzgZ^Nv`D6+NBfS@fV*H`{{8=om_*t9o*u$M3nB zvCWTrrNH80g}$#b2W_Sq)pkkNI~Xqb=bK3^%_@gOIHm3)J4QA4HQBAkb{Pe^WNu7c zcCIk|*L;RLpLzmZN6z~8)WUoLhsAHcfF-hgzkL^fRD0f?= zX$(A0U=vw&cX~-8+H&tMjIQe&mzuw#X^`ncVn7~mN{ws6w%(K17pNubNvULtDP@!#0$eeoGQhKGP&(4B>VR5KX#fDMf+KyRZMzy zMIE{Cf{p?9f6S6c3-&`a-2#UaBBFW98bEE86{wL}Jtf8HXX)S(j^dOSY98knHp)W{ zEhkbEuWgjD7y(v0nQq0FysY+%7ctR!6fZ~8%Wur8Y@8yOdD30lJV%1TTJxEq^=Y)9m4@C0pCj#d#d$K{YZ64dm)_9%nh}M8i*H!;}1emVA%uVSg)s z@*$j=1MZ}!F~WHQwiUi{NScn&DYLG?(aQ1)Rlatm0q*wDA<$nzDoHY>|9NcBomD-X z?_zpx`J!&Fe%a)k7^IR+G(;b^{mdF5QG^b0o_0;8zgWU$kC+Z$HgMsk47DTP7SV)U zc!g)5Ul<0HLSdz?lr-j^7U?5g_+Uj3qX2naa~@$dk@M9Q8X4z@$#A5KtPa}9&i|7P zjy|bnw&6Eo>AuD*gioaV?T9d0l(=J5wGl#nLOL;uZ#*?4(~w~XH^r0H{$b`V!6$i| zee(SdKlVHPF38Q;L7*KrBOTJve+g){57yBAU~L_&&jFWxBQvI*IApTq!Oi0i z>7(?LLxlw>JWIgIxZZ^QbS!pD9Hl3;LT?|vBorZ&QusSP;~CLCr}YG zE+R(W`A?0TArsmHf|>E9HK39`<8?4D)McL$@O?x(`O@o}h2f&RI%^e9u3Z2d+PZMv z8c45rI%$3s4#A{1+8M4MR};3P$hdRoADY%q6ugm#+eiT}HRPM<$79^E+q_*9 zAWvC;0|Z)jcw=39gsv>(z{l}=A()_oNWyZ-e?aG=z^?T+2)v63y7T(wyT$jmpanCL zK;D}4y<0v%=-(5Q<7g}EW9|#QNRXNrvACuuMHac_a!wH1Nj#Ol>jNjkSYu$B|F%a$ z)00AS%W57&fH}nLmh&P)^B9m>ES;L59*2zOA=w)>GvzE&7v_K}L`H`IX35LLpZ_ry zB-_ejtk_&grUk_2q6y+n|6t=;m;LWxDxw%=wTzlqz@dhE_DJ!Yx_l8AV#_w~sLeCP zmakHyf*iZUV%L_LfnLFCU2@wEdh3uokw!jxOTS<#4=mK z|CJX>E{L3xRsTUkugQ5oi|JNac7N}|L&I4mS4U!YRm6Ncx@-`Lw+9CgeK7r=mNQQd zu=jr|DQ8t)4Szd*Q2ldvB@JKGzc6}xUP}9^0BO?SKVkg2?|fh2hDs;qncnH!^DDu! z{pGy+4>O3Wd8A*@(p7W1wUfFq_k$B(B{)=cZ(-Z3qKoz%$RaJ~{?gE+{u}Xc(|WE+ zV`iu0y!sO$^DCeCTss+hTr4Dzu}>BMfK`z5#QTJ&ich*|*&Z}l^TTa%6-NxJNyzT- zEoJneyhm8NsgpU%nl^Zk$TBf^%^F8;WbU7q5rC;RX_F%N zD;Iqx?Eg86AfFJ9l&tu!ys;Z?)k0!Q3P{p&x)qQZT6iv!G}D64gy7VM)}R12kd6K0 z6t%YN%uv$NJ?9!jx;)jrzmL!8=RICQ>+CHbm{6ExX?~ zzRLAKF}_MO9ZFt44qFtFSuFV~GD3nArPMz8Z$@1k%BmXcj{ZhKW)JfbY5~n|XFl3T zqUp-_>?*W-MMfQ7k&lw@E3gHdWJ08Hls6x1Tws($#l6AE%iP2jtiaZ&Xb$I=B)3Z7 zwm?~frq`0#1bC}tZKK?QwPiWrDL}qWHTM>~Ku?-d$vrP`?d z+Rr925*@N`adb74Y`h+T@uC>oz2lo4k?2z;23DnR%~WqpGSG9^JoeHA&ep`>A|R8l zLT*Zoq+fyu$0=1J#m&D)EYbD}4y74LUy-CRv$rZ~L5KM3+67_}QhQmVXskqOR5vM8 zX7>DpKw!W95(JzZNm;=aPAMf^4U2z~)YjmN?`5+mqhTvLqzR#Yj2L(Pe{>p1f08Gv z)qO{Q13;@$*lpYZb8S&KK!n|nft9|sa$Nwern)L>Mhu5Qs|l&68UZ;X3NnLMGnWOA zvNjm_YA3&m)Qs*Ten|#k?#e>`&F1=Pg{^{Aw#+T&Vyug^f0m3OGpg2d?Gsd9#Jcc^ z=8slMDUYYXd@!!E)=?^1GX^u}t_z+ZaSr}_96Mge&5sJHjy=0zt_V|tTRFdFrgsex zCYxNSC4}^opYZUTS60uCoU^q5x`C7R5T16rh-dN(dp$t1nP{U&*PTkF9`CrS^}=#s z)hx#cU8?=JHk76`2>NL|Odx1?S_^9;rM(bl;1uLvM9HiToSC2tZXOQa(zzT!L8(~| zGU*`}keY=~$^}aUjc^EIf};RfAvig&tjmGSd8utqh6&Qi(tQ)vz#58c4OZljsa*i) zkxx5go{JkHwr}0h0eytwh`vX5 z(LMXRwSPSz_b(aE)1~#ui#_6VKT2jHBnCO=Y9F&T2;iy{9Ht`?$lq z3!5nql2u1;3jYCcgw{R@o(hTDVo1QO6e`O$A7zUI|C+T;S{QRsY#)x7N%RtClR(M5 zxLb?es7yt^2PPy{r-gB2tvVlKOvib#iE){~e@&GEx;kWIZ}mrMAbwh@30vBsWZI7& zoLrbLHoX1QvLdWv2UaFlEiLuOJW?+zAa*6ce_g&uIPKqr?LssmsT!nd!ZyFq=)ne+ zbQ!JMnI2mq!%-u)O1w>+W&z6H=)?ODr9 zy!|7%&_J%ex8ReAj^v#~1y{Ckh)${bRGZ4CZB3L>JPp-WvQaIiVgM6H;LzMDP;F^b z@5cEZyT{K2N+w9-CfBI0BC5!>U>TW_%vOP^3oNkyOHymxfaJD69S!2y{w0j%DO{>MhP$bQ>- z8W68$MAr6@DaNv=Q(*0<G8(2T^=x+<5;HX)}uG0Xl|i-iUfgR3sjwfl=zv@W{GUL zIl)v!3(W{s2-&Wp{%AYDmXNC_vgPm?nvv@a&}}{#R+PMEl_45;DY(>Rnt;UDM~q9& zflv&MJOnaqh${Ls{poF|iE}hDwXrf#GZq;31x(vCNR7#T&kX>?A#CPaErN++Sp+6! z(VDR(m<;fWz}h-*olwFEti4xV(a2nr%t{uG$FY}rwN#XqXV~DOT@3P>cVEO>AVsn~ zFnSm@BmIfrWy!(u;1k~i=Eqr6p5G^Fh`;g*YTENE8gUA=C6DRBr6(QaHsOwb31j=kUq61u#`jlle6PCca42C- z-}qEGi@Z4{Wk+_(hGoJ@q<{T7X)coVSx9EY99)t5z!_)_w7j2ndE~qZ{5*D5=87?TD*sA%AQ;p|9lU|6<;ll_UzZU ziMTB)Q6of>yFH_M^DxFKiT4$@y}hvVAEPhJ+Hv_QtK0r5O|pry`)E=8Zb7akJyo2{ z=;2~!git>+-k|;L?3qGnnajTNE>T;^Q0>kQxxpoBX+SP+n$2nrDO^W*Kb6YIt9tI* z!`P9__RiWzv)=-dG?pPaCEO0|Nr`e-1sfk~PaRJT&eV|b4(W}YHAblJj7y9Qcb@U7Ye96rdW;j)HL>9V{`oE+3h+?e)3Fes;HEDYBtb z9Kt3(x^*yXXtt7{Kj$aU7n$_xxT^j*1ij|Apkg&)Ba*f$uv#q|Sglr}tyU`-Yk_L3 z{f+!@MQSUc>l)nKYnD$M+I%cKIrSRax|-uJP1?LZ_)7~c*Gs?g0u^6Z;8H-)CuRWWUAkkqnZ02iA z$Gl;(-qK@6DY`0 z;jcX_Z~2MLKka^_Fik~=Gq8PanY9CdqVsDNQ8?$2d7ZZ;47S|G9CD}hMLhkGDZuY^ zRTuc3`NbS8?_{??cTQL2&!OF;d2aGOR4D>WG%QJ4y#AXKd8u}dL2 zr*YonmcjsPSn^eFz^pI~;Gbw#>_~bD92fwIbtvn##xdx5BD~{Xy1xDlQoHF}Zz%$8 z$8IG#X4*}_0rB$m1H>={5yPsWl_N=meUkcKD#sTlnBCq*06`@2vJyTTxv5zT22WUa z8Uk^#=L(rv(&u}f{H{HAC2ixM9^YS_%d!GM#7%V?aw<%v9pp792+&U+y7NqlV@d9e;Z6 zP5&;KhW78&sFAfg)&>v5lR}{m^gtvmbu{*5)SD!rFXwb*1*Bt+iJ`<77L4M=!jJwI z^m|K9ZU36>Tqn8r>ogY!QLg-ha3L{FEL_T;!=r**aO&Dtv3m{!3ktNYwj6mXs`s#? zY~U?O&u@Hrl57@emK*)ZLn``g5-AaGQn%j^v>)HG)5@BM)CWzt_z?0 z*PB)Wo@LkdKEJLHqn)l9-OmvmQGb4qtj=ekr08RqD$wMa;?rOX3Hc%Cxm@lcOi&^C z-3_XVp(=u3#%YW|0D6S@26iZq?_0rWMm<)3l6sHXPmGomMWSsJK{;Z)u%n3%Yl`7A z!8e5An&Gk`E(?U!cA**#0k9+M^U~?TUIZJr2hhStgAkHNUuJ&@GIIRFmjr!D3b2Hc z`FEaVjqP5=v?Ys6_@lq}$88XmqujT%T{A_b+Uy&M2RZX(bRiJ7LLR=-Lyqx3R%hW$!f%m+D zUcZp?yiJFOq&P~~-rl^isJ`FNGd9;0)nD*)1l5pSj@|#Ng`AsJE4IJ#p|`Aa_VYN) zrBB$gw-{MQp@5SsvE_v8&PH__B4)p+3~jm?$#*utF+paSOFzj>@UqqCCd9puP;;jk<)anP(s8I1a6?{+9x|MbB#y zxPcNhbD;YBuyC?nHfV>+mV1c!{SELFmxOzWIfEd^g_tq5FL*R54_WRZYM#nvpx*cB zWH}Z0IChh{LU<#>Wq&24jLPlPPz^9NBcz0k$}UKVq80r43}I6ecZu z>S7`p_FH)g99=*;)hM^?ZnAI_QNxNVHBpyr05WPjP)pg7wo&sHkqc+aCSUfrXtA!4 zgWeR0kFsXHo`C8Rbcs}*O(sFL%3=_c^!c0EA?d&9r||nIat!0B{?dGvQH1w?HEFnw z$X7P2p`OxeTQNKXmpaF>wSQ^=Xu494v}E*WtH05u%|%X2G&y(Y1=Kj>G?c>6rn6(r z1TzdPDXrrEB>lqWv;TL;&a=Po=0}Kgt83^JZmbI> zw11cs+Ii-r2(@6ld34g|hw|#uP_j1nOEFig3%Etoop}mI&|KR0b=lcM(wO>s_vqTSACAgl41{@i@_+yVU zwwSktlrpuMowB@;9Qie)dY?`nz)B#I|- zV&BS@yA=u^s^DtA7#>P@_u$BCR!3PDj$y>hlI9YtcC_*V64m- zBea%uUEt~LMMe6|bR2=yIyowE5&v%8-}W)M=($vF6(ODE7u=IALU#UNq00r1-jewe zgW2N({a4!)l}ZSvGr=m-l9!NRSH0fi^2yLkYy+!#Z~>> z7vN-aY3t;v+zhVmoPqJRJIF@wspCOHR80+lb`5VjC1hkYvUT8{#%pS&(Zd zx#QE!a#?BU`_HusF$}3@F=sJ5RZ84}lat7;PlCBYJWy&NTmxBX=|kGVi@=vwcegP1->HrDWA zGvQ7u87Si)g!wBfK5ZPN9@R2tq8%Os3Sr1)AvTzAS#0 zo+Hg0)D^7i^+!@%MaQV(f?SOe_3-);bE3wbKvl%$tg2-4mE z0V97}#8NkL3@k^eRiqo{GUPPwIO^NNak<^MD^2){uL0wZ_Utt#>CqQY$;w`wRa2Q+ zJi9Drer3$f1>tG9v?!})Veinyb0!tVkBe-LiM=f;_NU838wXA*3QaCLVO-=lF|nsi zTCw%G`x}1}oA7#C{MA>i-rBlm>%pqVagbJ~C3H=T`(S>;=$K6vG3&>Mw}VZ)BP;uk z%;IrnG53rO-wwCP+p}tJ>mOPMcgW8t6@69^3CF}P>K1z&*fbbKepG;FPq9CI^L}!N z{7AY(#;>|e7)F+5OBg44w%Q?~x|`#Bv_U4BJ2+H#6<)$7@*BF1%!1oUPwh4mZ@G;e zFKr?ZEemz5J)!OaFlzIchB~B7@>;xL_qV4r9v?sD*?8oO zwC-Kg!8Xkv@)xe8^6$MyPrJywiT04cN;c1jX5#`g1JX;bBoXGc9z*#g*Yw z`JxN>Z+3Z>Xj%`;-9l|zJ6Q=U6iM)Le{67QjBE`A8eN*a9ZDGzEmnHCNVZ;_4%%JP zin*~d^9X{bVf^0DW1(sTUUnIYmv2$2vZ*J4hKiOWiB?Z?ID!EYCMDdJCTHPsA0FM- zM*dhFeJN!2fy`oIxi*w;c_X;j+K%Je{d=uF%DvV~3!jJx{nR=#8{s(OTt3lS5eP)~J1@Huc0J>?$(ta0YxLLQ1KiZ7_OAD|0irQ0p+eN}wq_ptRS8)G2 zwI@fZQ}F$8DtQ*Y=Sx@4q^^>&Lpz3z3v?gpcrU({%*~k7^gwI~5`~b-0G234VEJmN zKV=EeqR%nAbDt88=~xz*dRcW<(P6WCZ}b_Ehbe8>tcfgC@-SmMl5G2p#@m>6vK1?6 zH{|)H<(UJfAyyFKLbjLo($bjK4watRg=b_m&8pzhAY*@6M*fn>INf~B5?&b(Ar5@IWi7;!ip2Hp9E!}Wns1Wj5A5ay8PFn_hnmH6(2aFxf-nV`2!zu zTA10JJo64hH!XTh?-x#qydfm8v7Uzr-3BHB{&$Cc05!?Yd7;6&L|r=C4f8x zkFp)K`Dw6t2U7IFyu7<8R?6^;FvOvdO-9^C1RxSS6u}!+rV>3fCndHT4e!(}rAw07 z@=xs03U_8K%7`tN;8B)9$q_Mc6TAmtN6A+tNAyl$Z}gZ?i`vQI6X;+<$v5ACiY^P( zM2j-{JvV#CvZFRy(Zee0ta$e|r@GWyCMAsJXzv(Ikmy=fB#>k@yfEe;%G1 zb!Y#bsV@V|Lzk?1RZw+{_T~Ktu;&2>#C0*7nzoqG-7A_MY%wOp8yd}H|b zo5Np}#Q#Xtw3*A?2B8&Od;5-iOJPUSa7*5W04-yH5y z8ox;NVLQyY<&|ae?}_&vD~pbUaBtN5WnZl*ig~2S2g9x3x&Lng4YUueY3FTDF+eU5 z_Uy!i^bjdoloL}{qlK0N*9L=n!WaJ^moACJ~oX7~WsHZR+ zQ3RI(s}bi#e=&|Tvyidv40H=d^&ZO^t7Fi83V->svVztdh4FtgUfx45Ke5b4gh*0j zHT9k_V?d!BbYiG;OZMg0a&wmO;BHbSaQ9lcY7bo+^2zofJCZxej-)!!j^v#{JCccJ zJCa?yCkdfrZ^YL-vv}+CYR5)x04A0BLN@fs9R6HU{gHVYIlsnNbzOb;pQkTv={deQ ze?s}Gr;IQD(Ngs0MRV7jAbt({lK`|vnw1}qx#;;)BV54Ahhs0ua*w;mIfE<_Tj;G9 zVf6>bv>Si3_#LK?5zLo&lqGIy4pNsm9upm@Nz4Ww?r631Oq9ua85B1Su0Ui?t(=>v zADeXgufR+6%qFg-%`%O&^J9kdTEx0Svg8?DfI?P%xAVqxbgBqQ3A4vr0umL3aU@_1 z(gYiYklG_CYodN^IPa6FqN<+yvB`ef@D?iwxNb7agFomX@cNnG%E};r4mq;z6C{27XRF1&c1|R?WQn9(jfdex+nWD^Pd+VF zf7Ka^oUK%)6E@O5!VTU*Ny!A4eeiT63e;JOb!sKFTxl4x*-;NMuYnrZLu9)*ox%Z4 zMWL=)NTI*>Onzaku+`50FKN&ly_L%SK=k5p54)bSmP9*x zleWIl?`lSJW>rht0a7Qt{YYP z9odeZb7c0Gb;qnD+m6TBo1{&4WRu4J%_w7&f=Q}qy9#zC59ZZryI_ob5-v--GqN8Lq+B#F|H zWG4(sPCX-TRsQP622`}|?Gv{fZP?QA@e=H4tIsThL}PPeO9whDigCXueAlMi!7u5+));c3a|3 zFXbf)+DP0id~hFLG6W;xCMCo*FpJ)Jt!UU0a0}$ZMA8P(W>!e;PIHUQx3su7*8ot0 zdioeq{N<}9bNiP$S)T~C%%+&ypo66Jw9-K`4<%hK45Nli!mhyQ@Dmwt`gH+_7q`33 z^DvB^lR-9QC0Vt;RNuU2u7u?wjtIRR)D@U7@3|Z6`2i%@#4Wl z*v>&iU-kP)((*esrAYtZSZofRwFgea?_o4Yw2vxs2bleieZdBlp@rntXyAMlaDd++ zh-1z@dPVxY=w%(I2U3>&lr^z6%2hOyFvGV_OQ{r;q?%$Vq36!K*R!SyhY}J%tx+|T z(TD-J_*Gt!X=h)3nDOM-SXfPd=u~92vd6|;)brnmSLY&FSASt8RVVTc$`>stFm31_ z(SY4>o47=L8f;2@1r0<{30q%{H>7i?!gkahqZdPSLmESkFcvC(Y&2PEUBaF!PO>CV zq^#}oz4n7Soym&{`Lto{7wLsznraqdn6<~Mi3@W{^Ge3#D-cK;uKa1iWSYnvM1Mc( zy?99U#7!2wusD6V{bGRU$U5f^8NKE)@If*W@P2f%x+T^+Lt|A{C~Da`vxOA0gQSp4 zkCha%ddBZN&UtCpvGR<3RC-3f(Kjx2{OU&8Gn-nk?S;|EK=iW4%Wvj6(%u!#+O+3L zuH`wBD?LYY;W<)xOL}oiS-5m(Rb#d2nO*vNRbvC7Syx=OJR_%8f6&mfHRsV^5DGK5 z&F(-C_Qr;FACp85bQ-s7_Mp@L$9M$i8pUpl7 z*{v){hLVH~t|AR5m9~A}qiGD?k+Pc{NcdE`()x+3r}hGR9iI9c6jXucBel1WJLIo{ zq-Wi+Lzw$T#Y2kWLCD;E(e>@O-j7_QXg=kLo5X!|)0+Q=^wbE+WWWnIp$}C|A}nz0AxSh6`NwDx7(uD@m8DcNBwBHaH6c8uILcLju|A0O!9Fdyl1EvN zsx@u@UN6*LZosTTisj)Y(!-8$hU3A8>;6pn@AM2ZANS?eFm{$G#uRA4o!^px8fgr& zUx0DG0MIHH%NMdV^&Y%hLttp8j2Ilb*}Oh{=JU{&20}H(E$exxq64}Fs3w|Lsi^ts z{vQgzKl?h_X9Ux{o$zpt5i&6qBk9mrUCKfnestS4{w^d$j427JW6BCZjZ32-l?Gro zfIa(1$);}#sA=rRPj&i-7$yxv3x0?@yu#3zQDEIE+8z2wQTC^hgq*Y)|C@GR{D(E< zp7m&8qgbO2YwgBDhCdvaAYIe}sB!ZEC5Dhwj8VbMtf|(6BE|`J86cVr&`(ZZ90`^j zJNid0)udeT>Cu0!ecel`waExlM#0QF3r&P!-lFQ_@6y@BpI7sQEcDxjuK;PLHcH3e zvv`-aHveuB!sfY1><^ohGFj!`PQ`bHYOrWTEloSsj74vGTFZ7i4TCMV3m$w59n@Nm z-XF0JaS3ds)QMDirmra3WTj=I5r^DmAk{4GI3vy|uGX8xh)-q(Yza@;m$G3U;u765 zkQ?9W38uElOlrFh%MG#I#Yh>p6}mLRtuBqeAGhrM(si0jf?1BR$eych&mqYdNfp+B z;BfI6I%K@Gb{lzk^ZbJZQb;4BKS{lpXRl{rCK|GB*;`#pM*m+OE33#8`ffgUwA!a{ za8X-{?3exLzHzHAUEN4NB+@8yJ6?V>pZ{F={BP7g{}U~r|8GO9{Yf@tUOBY?F436n zSJ#H08BJQeUKjUo`$e~Ik$>QI+hg6hUvv8v73HO(8(aHKy!;zCX8U5*qA6|rWXAw6 z_2ah5?O>@w{>|7LatU{w{76wZh8ibU#h;SA_|44o>!efU+QXAdC2=KD1rb@iN#FZ)kx{bCsdSd`fxJ0*n`OU~R7IA)Ca_xUiMtF;U*djT1o`rx5ZQK6G zWCZcwCrYo!0L(gvP4ED51nSYy+$kYe#!AaV3t|Fs6r>eEZ=-D$-m&-VUC;yYXrX?^L@@$Q*o9yk z*dpeYU7LX>ELUwqR3R_*`5HPfid!7<#V;uekf9m=U}0LKqHAeVkg19-0&piz`17xSXxCVS}@81Po`Du9^1e+jT_dULraK$EyH&#Ny~@0J!KwI zQ}RrA4KHrFG-@am1Sp^-`THe*foRQ=lIdxvCn(As`{K{WJ$48v?^L%LF|bYbP}CF3 zLrGOdQQY~Y#b$C`c!-$svG@)EvwV{`n9&eHR`ayibDkSRT4)U4lxS*UWiXvnSn+CV zEjyAoYV5zz7Jd;sifGrP3{RCue1CTGWS-8Yzqg?5x zl-2ql6_le(vJJIcZlbo?VHVc5Z?jdEx{Cz~L$Ok&Fw~s1u~w>isf(UM(j*Lx8vE}q zPGX=N{V>4A(uU@JrbaAs-gAPe{m7sbQH(Eh=*|SE9N&(A?%&?Lacfritbjs!z^2*N3FFW_I>BZNS#avnt?s7_2%{^J!_hc6Tq%5XiWz6XX z;dMQ;YVQ7H=)~or=CdaiwHHL5MIX1s*xxM+O~m^{lZsmNJ~@WHvzz`SP_5+KDj-J$ zLp2gE5>z7-dmpIwRUlL=p)x5SsMh<<{Y*9W?f!aSW}DK_ughR<;2h{0h_$L;Gh%aj z9^pu50UBE=@}(hv6V+o%tf&n_@_xy(#+x=YyK^4oxMPY%oNa!X-_Mutp|c!3vA=b)-e3Xp!~wXp zIC2YR4z2F3cUs4EBGsf}_gnmls#GJNuYgiD>oGp9v$^(b=k}^d$>VX$2GWN_%KKC{ zEr{)&TGhR2VQlwRRsFa>KyBhOW#<2;^=nNOgvc)X8*Ck!9f7XwS%Tzoa6)>o93~rF-bT`g;Sz2)+mqY|{X=W56 zkH7l4-DF9#8>A8Nuqhy|Y_ze~ga_Z8I8>q;%}oJ%fQT}nEXY2Io3ah1VQ3G!B0<~G zMl3`BA%r~1(Ele>86mE~%b)Q}CLjQtfRr)W6takBBf+Iv=edQ2wNMOk9815Z2EpK( zCAcMxQr^Z0Mj$kY8=n2NN0&(X$KsYWinrZEqas%!0+aS}4j#=~=&aRzGD?vqV{G*U z^huuPel+Fb1&MZ-h|GP3ddQ56c2@u6C4w=txHR{+@I82m+)rsr4buJ}_Yz@HhRqVE zp1D1TZAMHxN(E&oMkRd++A`vvj2#JQ5@xS+iD3|=#vAFbiT&9KOW$EDmEt|0l$0$Obwh;j$rp3C_S(cPt(H%aTI-JE z4b;$r(XX;enYS$`-L&d2mgdBaK&XdC!-}13h!%Z^dXFfiW5ly8C-EL}z%nx+R_7ap zin(6dhRka?J43ewNKK`0KFNCwP?xed)DMpBJsr7*#nUy6Ch>k}0kLAi)Nsg-0AR)L zu)v>e*=XO$374T^oPg8O4^i($lNlKl^JcHUa6Sj8kLlRG{~uJb)_2P>!)v>aAzs__ zAHZvu%se{ci;RLu7g&yL!2kE=rhpXV9@@=AQ`&A8P1u&<uo0vkThkG+{he064<|H4~7JE3iU{r#}pBGg9Qlox87TK!?e=TDQY3^#e2L>`#j zK72EzU@3FRGcvm_;rzO;jLNzSl-A??x>IgAa7X)=PhZDP8@$omvxx_q|8NsY!JOom zoe!J5;DypbZ|@h019yQN_mzYAWHWb_H@@l%KLN*G_uR z!}CHtd6CShD{tA8U;JZy?yKbk{*~9qZRFokd>#EOeV4Z@ce5M#yWLJmpz{@0H)JrW zBp6(5`Zg42CxHyo9JGU`sNo{;#&LqCG`_G@k`&!RWg`;(1*rEVtK?Wk z2icriQff33K*@MVE=3F8vO_1+iI%f~0$Q#a%JzMtMk6}B5Zn%W4iIl;&rIaa!8fdk zPln+FL956?hl&!saVFaryTLq&i>xw^T{wFijS6XN-?mJ+Oubu+R+Rirf!b zEJ|wCc7Y;A!BKk+J|PaI<>*@#!68pr z2&_FeVkjKp5vH73qNf<%<3EsRng)v+`=r5Bm@@y~8n&4>3>&iMRSLf#hD%0P7lV9Y zWG>Y`TT)ZQ*s?Q%^^kP4B7}6E-t&>S66|lc@t$MQ2k=)m_3hf0wsx|M?6rHw>aCe; zw`SbmxF$B?i@pi3_l>{$%GKY8UF7&qY7DB`3wz z+j_Pgv3j=EEJ?51*ijJaE-Xm)0>lmJ5^hP#YMr?>l)Iy>@w`w%$^9qPO^NHVYsks7 zT4310sLBlH1`HdrbuLg!WK`85a1yieo0 z42KmqR4QoD5F^R*48C% zGc(J0LqHeU38QF)`M-YfBltG~ll-XrTBGZPQSB9;IeUU{|-q8XAh-|Muu{vlL)`7P(;h zc5R0+?gvsfL}8g!sKpHeEK{1bw!{%KDuTf6p(RA5q0-72X~bSihoMSppx^w}D!WnGl05 z^F_&1-#)ujs*$Ol=oC<|$wSu;0 zgyz+wPNxT0U@Ggb@~I}$JvUK%7uD@q&TaKA!0=OW)b@Qz($ow#WK~uA-#uGOakN`Y zy(9f%VmruojHe7PU|OFf?orI0-qqDNfphV>0F3s6y>GJ}O~`sGOwg>)?h=G8g%j46_mblOc2eVPIOv(j$Az9z+G@cj;CGKN^t_^P9HE)`H|^ zUccm0BF!z`c^k2_G+D`^=$%-Pew_B6Ail{OdkVEc`3iuK}Oh$}p+sQpH$W zFkCNM(79*6ebdp2NRHh!7^Lq4__twxd+R_n)$Zf=^ABxA54Nt8i%z;IZq@YFTT||D zECX~)Pk22&{_1I~x5lmAdgA?!KSR&8)judZKC|wngdQt%Q#L=FSGI3-*}wbMJ&@32 z`iLIAM*Q}z(ql-ueJm)qc4-nREl6qyXVz7p9&UUNjkeK`t@OsYa79X1@$d=Z?R(OT z&nyeC!~gn~g)8=C77y+g?t*{2XhL}Gp7fjz{R$#Cq95BSN#zwk3&qdrlTZ@-^eKn8 z{U98wJucKVv**ZgAuLD+3`@#coL9FK&XDs26&e2Hwwmf0n|E-9;K-i!se)rR^B zOm*}4nwuFIUAjFqMO?aVeCy%~3+`^F>a4c$U6`KAeYbDZcF~V5@wS52*QA;M|G1A3 z2z*}m5STvZ45%+8{8gThP8&!ta0GwdrXd#dRoewo$Uy;0BkS|hiS27UKPkTUWG4FB zM&M-oZ(dDnNsLGBYkKHJ3!BlGX{#%7f9%ECIYirXpuNES!V zx7Pi#ejIsK6TOytFqt-t93_w}R_2iNU=E4yanU~hQgm+VBXRHs$RnvNBL@e3ckGJ1 zG3JRxdECpbcZ?PlH-@rt_ZM=lib(&3BoRN4Gi!c0of#V?$HO9 z*r%ys9Hp@Rc3ylEoyQU)JGXC{8-uZc6x4gsXwVCq<&BtS8PqIy2nvwDmj3vPu%NKu37cRIUl!`z+#(&3P?rTk>?&%o;6&z2 zX3-hm^7i00uV5;q=Dz}3jxPYfEY}Il);{|C(p?0U^?eA1j|_aPjjVYTCetuYPF^&O zYH0ZN`_Njr#zYSj>ZbSpEYR9TFyH&I}A7af;xIZ=v7(zzyOz_xw7*S#^xK_f7vz0>&q`!Ufc^ zk-yBpquwKwRf~p9_CFNCnwVWGKcE-1xWs-XGLG=*S<|O8z*v+WcuwNOEvs@n67J22 z+i}%NJyz!Tn4X`K^P9Y~;rV4lGwTMP)T7tv9=k@J93Ft;w(`v4qOzEaD`PGw2*2Mm zD?6N3Q76H7`rI3{NKYD_AUF{ z=(2bF)m@v=V^>a(l{qQHeo^s#^wUn-J)SyiH_Cq6rSsy#dsDJnsUw@TBO#an&U;9b z%YQ9g{^uS)vUuf>meZrB-F-vfwA;YE#iqw4u8&J;JG?BE`*vC5IFM|MZdlSB_+dj=uTyu$80w z?;7>oh@H*V{v6xWBmT?(MsxHF<#Vrj4ki9G2Tr3;g zcz#Qk4LA;DsCB-1)VejD6-s``$L1ervA@edj;8%HQ%AnMZA1(eyj3hDvwvFQiT&cm zfTcftdkWk$cZ?QovAs?Hzq-ysh#OVG+f5uLP8@)604^dcws$Wm9xlSw%JMEf#2*JA zOj+=TTV9++R||>@th-*6cNG**z$J(+YllCbSO1Fy{u-VU`pW$JP4u2GZ6MNW)zs{! zqeGHYry&n^epCF=Bv*|a&g`)&c$j5<*gCpYP&`Kj50M~CLlI8JRB;@u^R#M%wAW#o zV>0}*6fC)O@iO{gnVGQj;<@WQbLp8(jv#6|TDsp4>Q2*nJsJuV?S7V6OG<4ldT;~^g2ohv()P8S&3 zfjz_*#u0SeW$#h;p!C6>d^DFn{7`xAj5?1#{z9&ia3aKpag4|X4cW>$?4GW zsfA9)kraj7c^^)2+ekYmq#Tb`Q<*<``WP4(pnS=bmc-wv4EUr1%)rz9mr{BenPfsL z(kQ{T40Mc)hlr8qy@Xbm{&Gabi{zPVQfH6C^Zp3M|@=q-4Ds9HEbu?tP*DLp|ve# zok6j0yodxu!px&$7;Bv6ATl##1{>hlUNn9?DxYDbg~hkTW6wq-W-P3-4!3ye$gwRf zCWhH^D-l1zlS_sNwk6G$Cx$~&?*_{9M9ou-uas)@H~LMgDbxr=&sU5HRGl;fs9()~ zs}5PuM2t>7ps6KtGGuURCbQ*U#}}kgQwOB(FUn`=hPCLK{rUg*T}X~G#rE;HqS$)B z`6cW&7%Dw-ZTJbciJU&H|H@%}>G^BxC*E??-7G7dH7xyMME0%Ebc@dwevu_Lqw6Q$ zM)JVWOY53?)Jk`dl_U6P_gx!4z>nLN+rL*HF1>ZrgIr{V4r0(JZbh%HhOCOgQnSA%7#%ieghdNo6F5EU9PSov&^q!-O#m5TOKkO87q zJBbQeh;08D5d$0IL6LqG79-3Dy3Zj@U;GDZjC3dNgf}k9x{B|k58L1hsp)j%Gd|Jh zCd!D3Q`X{8G8`wkWR+0sh%_4FD!vM-cyn>Z%nwNc=1}K}xWYcR6p+g;$_;-ZfP|y} zmfxkb&7VKN7M1Ca4hw*(G>$fak)(g`{q4dlxE{z5ODiqqTHNvvio%WGv;YyF^??E; zQ<4j}T;)}4B|`%MWR)6@4k~1-m|Rn$@t<(|Q-Lw!hPeSQB;?q@M!mM^DeBL|EmBP8 z(XAO!3Wj;sf;CQaj5%b5kb&7lTN#ZhK7)+b*|WePu~ab`CRM<`U{--)0ZVwy*w|Mb zEybF4WvP4ay7z@R8$4z@O;te|Vs~5uoi^7vA_@Y7PODYs4H<4ij^;U3k}R^~5wdao z`aK9YFKR?G5iAN#s^azPxg{922s*EVL)c($%4{8GY~0KH{yvJ!puREkl}mzW^;DFF zl^T-(p|u?r0!4@MTJa`&AJE@}qxjgcBjD1jEGuzPyo)JoWlPJDh;j?(*u1AKI3?Ab zPLGz#lZgFbzJLa1&s~&fJJMlleLHZ(D9ZAA@?^YIfJ<%(U@KwKh*j!W*h=z7&Y^)nJSJP}{l57gSQ9xI2XY+jvL zwq|tM6aDJ;$M={%G$m&h>^nAP*38Q+9*W-66)~re4bSQhfg%2baI^oUqCJx$gDcXD zv&+J@Lqc^o$M^yqwkaq`;wFNT9)kLt8&_--@DE-$vQLIh@teQL*c5P{}?ZZEE%2j_Q(k zAxgRN)zHmMK4n*quc>5W?lFFk)|R7>b_{y#b7(s(;!F`+8mb+5t-Yi6)2Bq|Dg5dZ zYe-y`T*-B8u=ejL{f&B(;V}e&Zh8fnjzcmVjB~sf#FX4cB&vf-gg%V>1QhiIcO}7= zREE2Vtf$F>mtY*TiUwHMmYRetxB%4k8)M3*(=WB6tf2M3L=9UDUM{4UpJ09CFLrAy zRm~jmV<~xd3R4W{uObH-w%n^mB|X2z&vB&vV2~uTHtpl&{2bAct$2EY=#rhXPkBaW zup}}uQ52_n7ppbBW|hHReHlic58EbDDK`NpOYb3{cwn_(4wh^goD)AijnPD^YQc7{ zWwO0$ig?JnE8JN<+&G2u&c^eehaKU#0etD; zj`6b;Q}L-Sk{O6cE#3h=*|GRRU*(Plbif92BS5r>)D$3-W(>e=aO5Emk_Kmd<}zzo*qIuuZ7ymCL7Md8 z$E$BtRpAKHtW+B}Wh2e4Eir+%lulBJ<7x93EHe*#0&A@9lNS92h`z0q86ysbFo3yy z4LUJHBw^+La&)K$ufEd4f;}X`G%P2H-AG)uZ@483lPEUo52RJlaH~-jMj%2sc>>mU zj5zHt0s93BKHr}W#g`9^Ls;Fh;FFc?`ln_tCUcBUc+Ey3wR2~Gj-}}Z{ehUllGX0q zg!StlCq0RsbQ{e8cP60|bld-JmLpdkHLCY?lQwj=Bksw#6$_~(dW~c4OQnPv&q)Hy z_7K@KS|epJcAY`W0SO4PvSrhEJJ8D%RH#2vrrnk0o*Z-8_G8C;R$=>6n8H!SJQU(6 z2iEhn?=9OU>~9B`8N2@2k>nTgc`uNu@3U`it@D2SqGWBt8deQiwOXQKX~V+C5h4<@u@_nkJ?ri z9c6TPa9R9`#GYN)=c8=^U%QBVD+>3#E_@oJmR_Uy{aM$A&;9F7tDw(}?#{+f>&`d? zLh=LfWEB4YJaL`q3YY3%V*w}P3K!6WLjmEexa~*wSkF*~A#Cz!5pU&-E@&rmN&qn% z3e*kUXj5BW>M{jJ#&OD~1`lA{6xJmY*NZdpbqpvF%W!-|y;j=b!X-O5vCsKQbC0!QJNw*&fW3k^!L zwEj;$B?XZ(!IqqRWfOq1Uc1MABJQGaVAUlSMEfm6wB47A_awSL-y-CuDGSLVxKvQS zj;h&!uUcB(s5&j#-B~;NoztbF5zV)Qk=(?@vzJGFjQ_g=kFV=FzBqS6`F2Kn#iP2t z{MDbr@ylLa9H|e-?|ijN2s8sVkutoN^9`G|(|<(R1X3o~%R4@xpn~>I{K;lFYqjlL z+>`gl3$Dzl&C#IiGGSd}7U7Ai3p_*`f4M=K2#%>GwQq*9?Tx-dQr7lv-<0+*mJ@o7 zR8$nS?iiiE6wyknt%bI-46(~{G&ZnKa~U8QV+I`6oR*!Ylwi@fZGtgCy--mzwow1B z9}9_9saKX?BDNceSIYfM!dg0MeONc}Pdh*MqBTX>F@u(p#Rr){8^Y`w+DP0{Fal$x z3Hl8k>H`Q__<{>_mYrM7@iK>zST_!*%n1epXSUuKddwW$Ctzq)MMlhdF5ouN8MSFq zmf$c1&n1m5%t7rBEt#lC4HbS0w?Ne8D}#b4(o8(7s8ES76(H1n2S*Rm8qiW$WFf(!Z0g2LEjes6P^XUkeG1;VW9pt!-NxE z?d)?aKMK=yPFzR)2`mJR!pn6zZ{q54@m)}hRWVFq2l;H_NZ zn{8`XkBOM6;O2a{1bk@=h5x60MpjkQqO*1qqdu#|$in4Ep-ps}9*Y7PYw&6?WXR`UD%553+2e<^N2BynEo@10G zOvWv9E#^)ig$~GuN-#5w78n{M2&YOA&cI|kXMt~1GS6ZhIQ=Cn&eSw?WOJ;QsaG7=v-R(fB;~BlD|>Eq*)#p>PD<#}Ygmt6pxdhR z${y-hw$_j@)r~ErWqax-m=T zZ4KE8l*R>9WlWz9d^L0dlRMev{C>)-p{t&5%3;#!qU*v#1%k`(KRus8+-3Y8Wbt2t z;~4yjRJWT;n{W`U6end}D2ipBpfffX2CE_m@sp#amFhjIvE4B68hgnain1&KELI}! zcM@rTrF<840B`~`DJ%z=DoMlwk=`XuaE0VA!UWLpSu8bWV5fhcHd}f=HXm4=@aUV0 z=jaOnCO}BuBV76QDdJd$w}|2?r6v-8^r!xgkO=$(IJOUq2dqKyL_k|MB(=V+?MVI+ zyY0xq|D^56Rl&9+N)YDQY}I~_^$Eg#$GyK;Fj|9Lz)X+joKN}BB6}Ta3-au!s zk$a+F2khC0%DvAR+G8>UfJL3@I$oou&Eggn&*^`($9D_m>FZ2LT2U5qk`OFNj>;{H zBqfO|TKnGH*UT|u09aYl32a4g`kPc=BduB&_LgJM5%b&^P!R%EBnNxd*Pq!WZT(9h zd8j=1jk8re3#6ip&5h|I+eHs{Lz3y$=EqTWCyFh{QX8& zSi|NKj=^oD)Z0UZzcP4@G*C^HPn_#O01EQ!HIGZwVv>j25Rpk!PLW`X4qYAh&#@%e zYJep;Q|~c%-SRR!KAt`X$VlZ~>=C{mB4I;<pz(~6-svt*7?3Z%jmC{e3FQ6dSFnbO3lGbvl!AS^lSCvq zCuVO~XU`~*bC6JdfFLukWc4jTT3H=Tp+!}%+DoQ*l7hMtM)p3#YBA2n69gNB7LDvP zmP{eappq!cwi-860ZaPQO7_%&So^_PuTS7@rl_Zb08?Y7vn05LTXF%j?c=VenMB8Q z5Bxv`(9-{^fEQhYu?RPfbWeS?6v(0E6V^C5aH8y9LASBIe0yMhdoJG^JzMpzp=F>* zvKY-l7RsJ*uDp-lpQYiKDZL?)FIutfq*iP*@3^rn=7GwXUlfEZ;2~0xm0gipe0^EW zwUsefJv}n=GCV?-hZ5lw*=2IkiQ^)FjEVhGQtS_xhb9i3RCN5U@maAa`*%~RU*|IJ>Gkbnk&7GOW(~c5!C$*s_?L zD`IY(H7ZgW6Z^Yvu}>`x4a%BS^vR^6y#svA>RU#x`xf1gIeJLC#U z#FDowBpNd_(DHHa-;M|Qk!YXoZ#%l}-oD-ao3Aw#zWXtRO(2To*5E8G!@BA4YmN}GSM@BKq**WSMT#!Huso_xzCr*=K@ zN%5u9 zPI5S2vYq4>^v{fs!EZ)U2!1V9FlT#D<#)t;nn0$7nm?&nmz3Xit^#y@EQ~iu`bT1w z=PcoI-lQC1?zN-|d+5@TPXZgZefk9T2>*H#XKyBwijtFXGx&Y8e2-5W{)qMD^;TQ% zoX7dY--$=eK;cC(e6q-ugg5KE#UEZ(`Sl|4ikztfy#C=#9biNPU$9`(4S~##eoEo9 z$r|M;oFC1hof^S`vMJO#M-k3ifidagpgy~XT9b)NLO$}u5htLxG+1!!oS2N}ScrdG zo|fuI^?U2U6+hr${x}xdu$D90(tyl-3ZCrMstOZ6?{zW@t6B0RO!19R?-SZZn-Irw zAd>NA%cwN&>FPKi^#NPn(1#Y@fw7q0k@cf9+a&w84KKe+;(k@C={8Qx`By$Dn$7xT z+8fsY*4VOD4GC$*uFn71Qk{8tlJvq--}1XRSW#vtIA~1Od-~nayj^b^hO+P4*Jcn9 zAA99U3UOMmoSIUy;tGBVTapr+xg@MPJ#n;v%8#mIJ=$3Kd(b0EEC(d2>=MdyVduZ= z`|&wuy_$^Fc{Urv4qN=-`8ME?uQ&nX`Q zmB-QZ5traZfv6n&iQW_htkdsJo5b2jUllfhbTjfkRJZ80t_2RETiT4!OcOX%+NkDg zJ`q#y{KbY~@ci82)zxN86E$a%5J0v|$~|@TYlqReWs`CH)VU0zuq_N|2&qXr%rv(@ z7_oH4`nhUZ@Rz`pFV?y}0VWl7x_ccc-+=7^ly71HPb87P?TeI!T^gmsO-HN``&J@% z3)-|fC#;sT+C;T$1ug+!b6+-(3S#jLnJsYJXo1R@cTFEt`sJZLEwzwIzIE(mlAr9J z5*I+hofbsFP4S@MiZ38CF5J*3+*k)EM`;|1y0?=5jtjpC56S28veU-@cTFF=ym9B< znXLoUD29o>J;@ih`2@golYHjHg-#%3IX4{UJ=XMEi zN1L`4R^8i>mn`DU>>Sy#_MQ{kDE9Ao>V#|11iHKw#{bo~vOh;DbdrVIm&9`6@bM+7 zCO*igyPe0<*u zMv%&fge8eKpAsC0A6bu8NbY+#{x7u~{{yKl#V^%*ZF)+3Z~lE+=i_BMy#xbC@R>Fj zCl*uh!S>}^Y+N+q$M1PbuS?4NqWVRJox5QJ<+J2eX+45ts{IcC*4pwU8zBnaT6J5t zZ3Qt>ar5yL`<~xK=n1tg?{N2~roC@kguH4a$|~ zFa3+%$mOhK1yW2tzLDH_XVtQP{>G7bQth@Nzy_64r?d~c*H!c`OyW`N87gzGqbywN z>uzi%po4yt6-cWHK?e;o60;j|o6jwhpE^}Fnq*7ne*x;V9Pw z&r(@m>sci?W3`b`qjp$+z3k0Ku91ThQTGXL&Wjad5NJ(WR#~3m0!b2G5vu;vQR=DO zA(BL5-X6)(k<=LhD!TVrO9v;rElJJaQtM??nZ)CcR1vCa`@wpae(u1_3;88Ss}P}@ zTQ{qS2`ni@VA49q07lYxXQg)x6MU56D^x#01&v2f9YF~mM_yWRbR4s><))cH&QroK zMCOk@;}res87F37CupMADKK{y6WZH5)AO-Al#)5NZD{qEDCRFZSOxi#kd%xJPoRLJ z@pnT6TmrmO5wkTTZ89=qiXihviY1I4#EHZ7d!V?$svtbWw%Y-nxDrbfTeWQ~vdgAIn5^yjQfpPQ%74@2G zJJ586igoch6qp;oH>Ewr!#Tb-P8AG`v^>Je!)nKCwfD%iPOvTAP5EFMu7GjMEp zJ4qv-3jGl59Ios zH!k&()r|*+#HTL1r}3ds!m6eBG~O3VNL~6s;|D|HQVUi$zKegm@16)bljOpgWaHiG zcS&piiFaSJ=IDqoG72JH@{&eu!2kE=rnLQ{TWDg&q)0Q`XWuNEvn|8RtAsVlAj{nU zvrR|N>@}1Fd9!t7?I(Zj(t2>e0BLLCW#I|Z-d)#vY4%;dMBI!857vgdzZyDS5J=(i z+4u1MqxpUI!tpf=XDnDXm#ywc_g^`hf3K0(g~u@2vy1hTL?C~Lq?ox1%?j_YYxtuP zV_{2XsUeeaoD}n+;7*>!=+w8OEzM%u^zW0j4QPqB970QgMz|$iNErFJk*Y<7lh%S* zVgk(5LGpGuNLom`G$F)LITuKGV;Mup+AbMQiDA5i`q1ob!o$ZoT)0&dg7qn7qgDMq zv}hx(^?MR@9{3h&d&o(5wJsb6l3-XQ3 zRS3Xd(66^v5taXb$qnomX51G=Qds_o+DP>yaH$g}(}<>bGk52m9D5QeHh19ZP(0z5 z7SAfHWKnG4c#J{?I)25aYyq~TgEalVer`AVg^5QXrR1j3Od%vDRz*iVoDoFTw8_s8zgwO8$66nf^dc zq_VwZGRRq#4WD6vsgn~I6#7y<0E9w3rAJ+T`SehB7BL#o$GuD}5*Iv)JhT;W8K#z< zQ;tq|%al3Dz9=9qJISm2!JwJpvy{4puK+(b*J2S2pc9uMCxc8CD_ly%G`*v61T<;Z za&>|-4M||yU`EmE%c5S#ZH(X?3rBoSya+mJEg-aJTD_!4g9+u%dhnSWTd3Uw*f)(x zDQBRIjx{Q^QL_a6*Rn5-M2`kfsN^IF?Uk32WE`|&(i0;&Dwq|3j7(4a1;AfRNNo6; zWo|*(N1o@k+hLYsRt9bfR$OkJ+|(Ib_RetsyP`YsF#$C}g11tWM+2;U3a}ZZp|Fy( zU(!5CBUc4xL7~au-V>s^A7XbdCxMu% z?Z|op8ZbA4Q=2nA>B#OfzRAAdW>Q(R{MHbo;-&`|1#3g5%?Q?pj5zLJ0osruBm=wZ zFTvW7P1pNsL#DpnFu$auu&Bq#xMS8&J84l%k5Mt3Dq_|He)Y(zxg#t4j?Ch5W%h1t zCx(_mEcyAQqR$E<;h5M(-C}QB8rlhj_EABk81U@P2T3gXkrYeDuewa=C(E+8JOZo9 z(RH#Ln`I`sdwD2#aH#Go_)4OicI`!>y3z4HHm44#+cPg|^Zj{s@8Ew`d3D?7rEEUE zOWg_hx84Klo}33nhHl!k&`rB%QhDM#aVfio^k|Epbnd?@eiDjrKQ6AGyd>M(#)eBp zH@5cSc=={o<8XBCJ`XQ%68y zc>ROMS{a!uz5b`2ky<@TSR%kPg~8$O;O z!^vY05f3#-So|lPU)PmEOV_^qSv}6LJLQHUJLlrFicWr{2vHe8;)zx9rwAOQ%|~RZ zm|XshjX0W;MQr5{9Ho8%E|kUf(=Z(&KDb#*JSn@*xQ;W9HuYuiH(QCL5~$1mNlJx@ zy_fAtILgEC%nH7DUv48vE1t@I)=CHcVPEb~PxZrVpqCo@m0`tEGQ|w0+Pk6bp|)kf zq4(K4v+XR1zHn!Iugdb;6B#MCr&gBd#^EKgRSoYvWqh1^{93wc{U7)y-i4hX%lA z(>8zqe^g6Un47b8we0_a&$eAas4O$*m)1Uel|pIja0w4d3@{==8N;@B68djW8&JQ8 ze>SIYmuv1UtScY;ul(Yfb>*}ERZz18aPre2OG(Q**gNH0EzZ0u{-1#?;+&VS!#4>y zIiaRNi#+_)d%;IxZ5UuAK@j%pME=@0AL&(vf>^kK>q@4+jay_RaIa7f6q^X7qt8R@ zecQ+IQ>A#l?X#r#+>0|Au0_yRK1ayEtj271*+(?DAaE4^Xp)-v<|MWH zrRyRfAcjo2+*`wDiykr&e$NyJ2Zo4RMT=w#`UQXYI#N@?8)u81Ai*xzv#c<0-x`j&=wKK!MooQyLIlTE;DH#)#snEo&?b~`_89#+6j;p?x4wNv|o48P^D0x$*i zyaV+G^bG1t@ijV57MD;~srLn@nw_)~-G7c(M6D;RD6EXKb8tXo_5i$lI=THhy;U?2 z#%Xnc7puOcu(7;^TNJw-4I{ACka1_d;EwHULjJIN7>c?I-Po8%ReY^}<(U}*4r7Mi zhQ|chkT|M54I2Vof~hI8bB?5f%?N}4g&va-$YBC`3p{5ys#pdo)~RgxZrGWzEvsiZ zb^$B)ApaaQz}8<`a50uJ+*c`vX!;9#-Y}mA@W;M8+ckFRW?##OlZN*L^HLUMIu1N> z#EY-S13-fdY~V94df4@QtD;*yR)2xCiGXr!9-iBNYOgU#OJ6)CD|>NPO=V{B?6R2o zl`%ILgs0)sqO6*Qy+adG-4;JCvNb05wxrmfE)Q))XC)R@itsDJFK~_HmKoKB0}7lOnhD3C+d-ZhAb_ z{L08k9{%l>ago`5$`e0_$Ny=|L#45yrfnC7L!j|WV=h1GC|O089*1jZS~4e ze+Rl}=kCpmq{hT{ti;ui@*=(P^8OR*ro_QJ<+%|r6^5%AV67b5f0v-zesyj58G*Sj z?%(!{2@CG#?#^wGbz_Ujj%P>L=L)yT4s>U$)eW>q1b1fxPqeZKEyaMSrY?{Fsp!BK z$M2t79{;;Z+wLf#UT?lCNQzy>dQ1nIC?;%!n2g-@rMu?IhVECdn9N zsVpq`vxy?O|4#jEPzDy<&!(CqY0WID@PZFMcD_^|7mlM`WeP9!>o|&jvI6NPT1YQC zP?K5luEHF_jp1Z0hKe=GSCPHH?4F&Ln$fhPf;-qnY-W+F6<`i4gkyiJw1(+dYQeOxG9BokP$s#F=Q@;`Zjm8LKq|o@T zlyR3H$}_84s&5qqt&+#%7LkleJcc*4K5GYrRW^)j7{3S6Z&8G6(n;Ge!|9%6i(ONr z<={3Jo|AZZnHyf%;mPEOQMGlef251QiF#U+WX&se^B3$LQDdWa4cTDf^3<=>RPCQ9 zNE4i~`dK;D>bc313MWwpYy$=q7=E)m9Px75s*kViv(>CU*e_&1{m2 zvjGMt`j0H=SUj9q<|x`wPcdt?n&+elcGbS30GCDwB*vaD1etD;s=6P)2&p}t+@)>s zxe;KY;v7|O1c3srVsWIQS0KgIZhf6fs*spcl2%?2S&~+E{!OA}1*t`A3+tf7th9a6 zoOzH&BLQyy!c^#nwVstuosf&{>QqMTIcDWGd{A2huc?|P@CVUlksNXnyx^1C`s;AZ zTv769n6wfhJ07D_1w10w2sP2!YrS!NhPofJwWIG@U#VUgHiXPhBrGKJlIAi;XH6=K6hxNB#D3f@_PsanZcIk!wlfmOpAmocq}5yBTC?@ds>Va1gbjV- zQzwExdj#xO%I3$3J}bMgU)?5hkHmjT4-aPd$ksQYa~!|A@l>>F`$1W_;q?0(_by5= z9#|Gm!^;=(GOH~7{+`U@oNnQ1DOttCx`hYhe?umOZ{L$%oL&~5hJU-TTYbf?pv<}! zgll8Mk(s?mQUzS@@y{+~r^wvnaIGgqk{wlzg}0;^r<8?D(VngP=Jeu3ynMZ?vEk?G z#a;38wX(*uK>j_xXw#8{DKTt2Vs;`KG;GLLqPd7t<0>g{N)04b5T!z`yYuY)d;6i4qTI|`02aoQ>2T`irbyw0GOqCtjr=Oxxm;L_osnz=|w>^HC z`c!p4)k<_U|AVM~X7??6FJ0>|L+hV@U)Eh4F&OU=bjxo4wrpoiV(ae^e|CG1I2Jz| zb&%rYcnflVqC$aa0!PYCyPqVV26RcVX~bMjiOX2@qIba>j@Q&N$Z@4KU1{-cSO3V$ zdCj9seZJmaaYraQ?O-x(mUrpKY%?>khkpt^=UVVIcjiEgLGJ$ihv$iX5e{3U#j7lM<@xu}Chw8d%hUZ@ zXvZOo?PAgHaI3ahhXTj65EK{x=ejc+^+GFol$021ZUjmL9`jftAENbX;^@u%$_DOLE=Co;j^P{Gn+V*)5p@B>`n}V!|q9pOoR#OKzJ;-#^+&}Y82C&Wd zvD|dQqE~AtcpLFzsPUUGXflhz)EvgzZr<~=_FI|lwo$tpkX0g=LwCJ&7fZ&}KSWb& zKyLKM8@W^ZcIwWCs1Z@LitjZZ$S2&)=>do*DbRK&xzwfEsF3h0X$2~&ac>aP?^i(< zAjE2~FP*oMZ&F^BB=!k4rq&FTgfV@n`v8-BDI}Wz_nZ4uuNv`{S_v>(aV*o&D^?vh zUbT`Mwz61MfaeIuG&*9~qf^b9caO0X#!qv+jXxuj> z?&@)?8$TTqw;LsEO+(^$-*-=A8UF3Q2O8fV5|{QK{6j8Ck8e+a?D1zIxK<{#hn_y= zm2WMD+~(Vct@e|~!Ym$V#?J~R9op}0 z@sE&qXZ2X}8p$s{1p$)y>!tTdesLdMl3vvxZW|E;d1PhcN%1%cDZm?{I|R0lR3 zd4J3Cw}P|~!2zopFLjcPQs3U>duD0NOe;X zY_m>JV~UjnG?D7^NQD7uvFs~I{0KS=xB5j3l+8B&5)>Pwi8kBn%hb0$BGX|$d2tdU z6CLA&F@qf^{R_m82*CMw^&&d5gAC|vhay8rF}j;qBj-LluhvPgO&tv@7$7fke%#{K zj4u5Psg9929({_5LDuhS(R$Qs@W-@K&upkUGq{JGukoWNDi)E{g#qPNNuTfi?ZPYJ z?`kps7fS}}mQqA&%wO#5ysGD2WKW#4ZdYyd&XF0X1b zlZwo<1BRHrjxJPYWPuJ-wL(#rPmGo^Hv#OYIuR<`!7<$d+k()rWxc6UHa<**;T(e# zOMzzuuoUJ5IE@ZfKk}}ixX|rltQ&GeG3ODW71(e37+7fF7S#!jai&Z$N^%PVNS1`UO!nC^8lTKIkP*Rul41W- z7-BCQNNUI627C$t!s~<~ut|$)L11))PRjq~i+-fq#6SV$p_!>8U#?g{ULwh9s!jG? zy~d?;S}5lE%@4da@2$dA>W`h$MGP=F((l2wAOSc@Im1)YFgAvNG z3`XGJMfy_40Oq(Y8`3VQ)p+(g4=;H?3`QQpK~0gD8>!JLrO36j`HxrOyhcvyl1gT) ztYq{!a>pc#S~EAR1FLA3o~ZEHGA%H~CHzaFMvV{rN;3Xxnsw#4)}Kt}-)IG1{)}HT zvU=1Kc$qN*0bzguNmjWd104HgP5|yr&`8mIh{S!d!!A|@b>(Bj68+Vc?c$WOzCW;P0^T4EMj@HTr5H50W>6`mSF>e! zkU)#+n_vL!A%@z`yQwdWIv*b-^ip8gP^F`B+ffe#5Wh;u zv+ZdLyGQ4SwNJx6&^;vfCJ2+2{!8u)hNW+%;Fy=w9`%kLMBHQ8o%Ht%=>_Nd0}yi3 zd0u%HusN19h@O&aH<6cER_&cPqY^^dGS1q(^%-H3Ma$l%v1ZS znu#P7Arp@j<9Vs>ABLseVSsQNM5|1=SjvVV0hjQU!T#9?2RfE%F#EPnGaF1>v>mUQ z`FbT_^J3#KL7F(fW8RF|BXS?P_-7wG7yp&Jr}PT)i0m8S5&5u(N90GsBl7#|)5bRb z@59FbTy5jubM=f}Y~vp*ZTt_U#h0u-q3(gW9&I^e!)c;|Ef@00S@4Ejlvg(qD#-*i z7g*Qy#h)aL<59qt-=#|{IyrYQdFlj)IupUNJ-?J2fZcOOO34fTh^N{${N#155@r+~ zX0Gj(W%2JNvh1>BWzlgcQ5&^wGAi7P!Y2#A!jb(~j$~}nm*1}()qmHhllLC_8ldI9 z?%VKGRe@)v(dTPo`!3xzJS zq$i4*F+#On^XuoHz*w?25q9Pwyd1TWk4+2tiFg5j1)Ze$5qduvw>$}QQ0lgwq#L_* za)s6GRil#L-m=ib4L8$p^50phlIdexvY2P^@fdkK_w|02tu@Q?fM-A4HVT{@g|d~2 z{Uq&{zUcN8be&7vHkx{=W#l|$ecp}ye)9_Pp5ox^RDP~MAT@6?H}S_$q*iMuLl|QA z7#VMQjNq1av#40Dpt!*L>t9s$Y!h&4HWFU}Kn`qB^{T1aP9OgrQAKQ^rV3N96kK0M z(-W2^Xw?iBr9Dp~7!Gnp7FeGo_QOR4xwzBAMjpeZ4e(rKj|2>;2@7ua9FUj1OdY{d z2|*P^f#o4K&ls)Anru7-be?ax5M*D-jkJ-LUm2NJQP3*wA*p^1KK?8Wo0}*DDi&+f zXL-jAunOb3!>3#HfyHEnb(sHn%tia!v$@@4^dY$kO03#3Zv;z7PW}4O=b1)13kMkO zB#$g@E{Qz3A)Q5@Wa%$~+@MYON9?t1q)3^OY;+z|LC0@UAUXfq`)Vf$5bg*b6amfs zpyr=;Kh$kjZI+J0R3Jg>wk@-E;7|029xVxjjUHy{3i;x9x~fa##s~cBv+OjJ3TSl< zhD#_AqN!%1h^*dj>V5>k0fI?5}>3>zK=dKG1I~Fcm@4T&bzBQu@*p zTtvX6KDW3BgHt0NJy4$`&L;i6mr&Bq$20%zrq6(Z%we?p7ATPxn1;YJh<^4^Byx+0 zjz2xV-L$%OI);|xi1T~fn)iI zje~a?w$)cMXnU?f|Gl8;1gop)vPxQHJWccG@G*#6aO#SG*Z=6aU$kmdJGS0m)S(Kp z4-4OlA*`=kdZ>zVu%{TK&n6MAxh$IWScr*-VO1S7WfRNj(-3^2P~xcc4Npc}`bWxN zgxt#78pQMY^JBO|0=kXJeXJa`Mi{z)Gl=c<9ppw&^VVk)$lq>uqOFHjAlD}@m*5)q-e!GS&F{(dX z9=TSiB)eRk*>;^ajNBGn-A0U+Uv=A%ose1v7)A!FB##mXL|DrRjHe{M$Ht39F*9a; znpcRP;I!ZYVw@x$U4mhxXxJ8w0@ZuK3L&|)e-aYKR8GewXdB46->MuG?VKfwa(IHg z$RJ8po-v4$=N1OL0sbSF)Ux%8^duU;lG7^78-z_{Nm^xjA=^YQMfD5eGUw0R-$Ri} zKkelIBqc}?lMH)va~7s&_y9Lrt^O_8ZhC8H;{Lb{zZPtJxLNkTpk~>n0SZa z_D|ZkUv|rW8;RhSh9nVwCY}Y9!Ta*VRx*4iUAm7_Or$P7n^DM9>L2Mn-~U2AC6ps~ z^>G^f&yF_uM>6=yJNzB~5y=MIk37jq*zYDQp%UmoS;_Wb_y%Q`J+GV@!3P3s7`;$3dtIdDpKcSP9 z>1h{qwnrraI!UV!8>qVV;*`R$CyW$1ut1@7^s9L;$LV?VLy!d2v|Q7Q@sam(EQ-bCfF9!;9D z{{JmE2ae4Y^wJR>F4aT12DZr%@u~Ys-1avs#ExDm{JxWdGkS=z+()|aNrk#E#U=`v z+>x-x%9bI4Bc=4RE+s1J3CrC86b#E<%C~r-18lMYbXq>0L95sZ*FZNJzCJr#`+tE@ z65%a8dCe+TwHP>F7J-P~Cv0qnv)`Y6s%D6WmQ>}6et$fXqGypt0jPuZBjd4kqr@@l zyRr9ZW^(D0PNHD3lnu)emz12O|C{C>&A=H%M_GiVo`5PJgG&qHkh>y4O#+$at$V5X z((sE&aOU8~3=;}Ni(A2m>3`@#coL9H==FHZ4q66EuKJg_FpHO!#IJeBs`uPul`3RMs zwLa3Vl1ivb;10P=^vF*8VtM2NUBd=KDyW8ytEhbRkq_-Xw7K-*^RDv~NPe`Yo%LZZ zTM@;Sojvj33KlgQ5gHlc|9b6L#T{`Lf>Vk|$Ni%Q?rgPGW)j}Xe^>XDrB~^W_lOwg zidd%gusEFNBP(RbN)J{G+100UOUt-S8zIKa){f>P^YciqR+7wkl8_p}aR0XRVPk$W z^M`FC@G^;BZeckZ@7K6RroM==(+r-_ZTdsC(&B`XFMsV_&xYOMohO;c%=v{yX6&$t zxSXZ9gx#Snds(_)IoouUvmp_kcqV7fFIR_`cz?A-)v(p_H}Pjq%U>ypTN+kP<~X;n z#d+=iUseihC@!I$bNj*7FyisSY0%otpHWxdvM0az$3)YW5BOJJc}vgn#rYG;Pd#OP z@sE}=QI{4-u>A*wBb_oa46+}2InaIt!aL7GtMtE%!YGhYg*5*-6|1s{ju0bTc+mT0 z9br>na)h8O^~$b3gAd+z82OjpL5XZgiRVi=&?yisL|)N>k|TSrJi9%ha($;lhq=Dh zR`m_nJxAbw?3E+QNFYKw$(J^8I+Ls-?NKd5uq~$@I7~8!^d!@LZiHI-iZxVn8i^UWl31~Y^O(A8MsAQ z$r*+gDtH1FhAf1yP_o90XI3E|JtP35F-E2@HHylzhxi&MIfK^ZN&%BOuolnIjKbdl&Gis&jo@!$gvZUrgwZvORjU+&jsMJ7FL_Ia3UaRy(D`+T^ zI71-8Dj`@EZv#l;$4haax;boH$cD29mmq|UViDB}w(7ER6Q8cV;Ke z$WB}~b@}?ZPtyh$rLA366#QmLcojNe7p89=omTL8QScw*g6;iQ%qgAj=Mh=s;}LnO zi$~-F=@BVy{H51O58AGwvG(`HqP28?Cl>9nYdbBYd2`n_LTN;7MNU07g?Odzi)w4K zgol6O)|~cXXA`XydVkA}ca%wG;lwPK{dJuj-pftgc4iG4K#kU}8}-UaV#vHb{84}A z?cv|51dKwk2=<4kFo7>>ee}>y;-$>PaV-kR7*JX`&CawD6g%yqT?B$+m6SFP zjYunbm|8qs(Qniz1A1+T>cg);b`@Mv>`a%FOzS!)8!WV-CR?70gWe0!^uAz8*Uz&V zl6K;c^XZR3Iwab($sHw8r{#|u5pKKSL?+&NK`$G7kX`jvP;0@jUrG3d^K$>^KbMjY zcdwD=Gx~LXmcDCxRuFsYi?#D*Z|u~pbJLKSC56su9eH1p713GkLL24g4@sJ?M;3DD zMtP&dE~~@v&N$a93&d2axJU&8jt~NX&0Lmj|Cll#UV@8@ zJPr3Cck~m9;4OY45x_pO28cB#QK24`mn0Htr-s?m(Z8^uxTP#Jj?=F;MI_Ra0#~@5 zazp_dX$cv)DsMK_a1|Q`b}%z+VfSM3s-RMEN_8!|O-{O_|~@fVbhx!glgMiUj?nX6mR|Hvt(G?IkLk~%@V=SMfqCe6@ zGU4o{hr}fFu_B>xJ638(yDb^=kQGU@gqpDcO-xNdO|c8;xogvjNurdNcr-ZI);r9w zsTwKG?bz2d7`6)Ih$vVLhOJoz!qVE@eg}$cgD$SAb?Tu)GUcceMW9F7<^wR5=TX)eJBDy?X@N+B#_nOg^v5xrYSE8 z+aXUH=w)V(9a`3f)LdP$N62g-KSdTIQ7-rbFY*KEW51ePMM300)vt!+n$8Ky%iOjU z0KYqkp)zWTX@+x$HLOwtEY_MTE3{xAVducTFWZo=tB|=8?Dn=E>A@n_!w6kAH}e>GQ-w# zApQLhm;^GzRRTF9BXQgL%iHP?J1_F>{QavAgR`Xe_b)qm#Qd(gTgNYMJM#AI+os|F z?^Ne?EgTpq&7Bn4177W+e&IRb)$%8V8bu4XLd)NOYUBGO!!I+)dM#tn&J4y~e?2+; zCSSVq*g;ibhTA%NU+ZcZm{_>w;LTJQ=2FD= zN5~urVSD?K0A*X$627`k5~+Mh?t>4>j0ze)4#+aph~$b(2m{W!gHN<)EmCdmofgpB zPM1b~+~KE$jF7p1VZpEYK2xY_jW?Ei`Sl$68o{%-4+$#Ivdlgt4cnw?wIk0cOTAP`0Os<4Zp~qAFQIM8X*^wEljj=VG1xA1=Y6b z$?AJxU20-iJqfKEx9h?&PRJOCRa zmSL7%Y)6*>Q$?^PumkQICn9W(Dtr(zm!e_xJ;O1J^vWaHYz#zUijKJ3k6bG>J{*mN zA-3$O{xSDCx5wQwUK6v-)rM33WLoy))(bU9%+P+{HY2hNq?)go!r#SaWQ&hp57{y; zDf_q02uUP+PLtA`&(tu-=c4j0lLkqT78rpVIQ?ui!|Jtjv|5{qX zgQ6u{zv;72rTVo{^O=G0yu|eG|H(;McER$tj>FC?-(Et!va4a}-+X!?JT5VPZRW7> z_M1}HE-4Cc!T%bH!W(W%TYF6Za6kOr(Zj-z+>|nU^^lUt4Ml0)N2JswZ%F{Nm)@N; z>7<>>FQA8Z9Imax|M%7h3O5Ja&O^5V^vbRmjo7k2Dv3Osk&roVdD}1;j?BmR+j8SA z7Hj2()8_T-{u)>YAdQ)wm1pI3Pvi2mt}vk6yQzdzb}m~m<+e&;UhnD^**-jr5W*p% zJSuz8Yec=}hg10V#GFAhRxH?a8?jC|yd1chnVV;ne6#+OYWu$7GmYX8XK}Kz}RGS|7j8vPkOr0H{kucPW&z@NLHQ9iOyOtmR zl?+b_bh?zP;VrFI44Y)l80S^&Fsdq@p-`*yGtc*;em6DctJmr0le%Wv#n&s^mDP4%X{31_=U!*oB`?t@C zQY!)aQ+dX}eMT&l@xSLYvbh=oczcYze<{IN*Mu#pX09>KrYoJhi6O<`o~1lHx3Wp z((Dh_%s5&@HKn(->)HfBR)mkmD;d2S8TOlV-h`n<(uPSRiJ}$6)gy>Yp!{l?B~$wi z;bJ>Ehy912o+DV6Rs@_~m#!g_S+p|&`sa#X^08EcX5GM%sfsGurde@plHdurk;-cN zgRT=!ozef);xvhE05P5Yl54RP#+B{9yqPqy;>D0U##3w$)Y5_#o9^qKb`k6fq72!c z|9xik$J8X!AY7EjyI7nc$|jIIzD=*FQY~`Rgb7F%RF0FOdN8PiQkQS1Qq+GR-z+mju=%Iy#2GLIo=?l{ueYKEOU>A$9 zFu{vXWK`E)*}S%G^UN=$|neZavSqr7j zg6vE9me?T^2S-wr2)QJBsxdFl92hTIJBaGegptB~S`5iHoUaz;b^mZN|8XDwIF|l6 zGvmugxHeuPGKRIJ*4LlKAlby|DH@G>WdGFzpMg^H?zx7Y!>>qf6c?#w`4*{d^es{= zG}35sPgf}(4TtO81#risnuttQo_3N|v9@dq7z$ES&TX$J3%!e&u=tVs>x(Vc@#^^` zlH53iy#4l%O3l24{-Qnc%#_!o?I5Au-1mdy*|d+J=Cc+gp2F(B0tyBY#4F`j(wYmwT@#VSi&Bf2Eb~z_A&;D;_!d~(c1Hk ztQQV2s3qOS{_JQ$Z&5v)qC*Q2YPAS+FO{*3K@(jMjuTbT26xD51gmUXNSxauGwC4H zROXTOAiQGTSTDPNEc>~blafMQXg6QrG^1DGYi!^~Sezrjif7Pq8DdEOVvScH*WU3A zcj2u@)9k|6M~F&4%qP^6xec%k&rY&gcp9R!yxeuq0RR!n=cUa9BlaC@tBNwLk#y@s zrJn?c*q&a6CkkH!yQ*pE*COCn&k(Dr(=`ScQ`gamN&^LZghX%IKmF-{mA%Ld9L2do z=DzC4xRI4(u?x^jbyJvIePP|9!sI}ZCNd696$R?o7F@Px@$HBiRcE{$y?9Gu9%2xH z>4(7$93>sRkd|>2&j8$#knv^3qKJ3PY}|%>wKbJ?zyQvgsn#Kdwn16_m# zP_WMt9zM3IsfUcbIHeBL5bxUvrJ4+9C}T z3RP>0RJQV7QArl_HTZY!aar-TP$IXiMd z=Fw-Nrbz&(2Nq79B;AZnuj!HPo4-iKZ<#<-Ih{IMB%BDBI+2I_pc8o<^@~1!LwJj_ z^wgVnE*a`N>n)?lOYfyTzRA4zn!^Cv8T=(8P-<599<$ygT-W0(YM$NMH+Jx66Ugba z50nXHaLu}v`FZ(shOhZ$UggAK+=I1ocTWi2mp(kdGOuz%Fz)8sxErn=8>x$ne{Nv> zql*F~LA8B6seDICL0ocbC`JH14dU zH{MdZ;rJeZUaguq9ajhco|A9s7MsB=l_-6RW0{?072Qbi+^&-8;@iwOiiQvN&w31MZ*Q$lK@Rpt5j(+2B zG7^P??-lVMJMhOt$Ka1lX4&X8J~N$~$v06h??wi)9+TF7`|*L)4W8gG*_(UAJ+^Ns z2-PL;OQ4Y(ePZ5XA=vB6UsA*a22`jO&= zbCwmge3pn~k50aV<>I_UGJTv#ro)1|%*TNQ1z@u$1#6BZ^U))Y*~gr6?`iy&;+|zxev<2C5oFk#4R_D=a9|TVAIBozv4z6x?+OjDpLa*>QZ0k3w>Q zpF;9@vqCaQ_SA082(^U+Nj)E=1UA9KzY`YzH;+qM`+d1ZZr|_&xcna!Xx}gb|L++R zE}fZOnHCH+0&i~5NZ1BlWG#&SPwXGsm@>TUJk+`60DhkK(EBLXVi)5qzGfSaf`vY|lM?zt zwolTpsiY&uurfufGr2$wETAVsW2+Lx6Yly*Y+kk!wng|hZY69a|MV_lD`YUp+XxM^ zh|)-oskpcLl_^DEHa@R@@<3}xJu+2{!l(19dS`pv`Ae%lY3EdD-HAsnKr!V7bmIc*goyXIm#* z!5{d>R}wPlv+hCW&&gRQItj@KE1ko!##!>PN8&fC5=hE=)U7yBww|tY`{l@F%^kpC zK8D>e;6!A}^GBiy+i~+WXK(SAGM~0ps1Zag?Lh@q-z)=i145%BXob-18F9m5f=i;w zjRIiCv6VI{?!F_Ut5obwv~tRdr|p5RJU;n2?W^wyke83|2nf+aukl4=i@&j_rBC)3 zN+@;%GvO2oBaqQn?wLBK#Gn?PW%djrf-Zp| z6dkFA9!=jwy~a(lc%e9|R&0oxAeg@%%Dkj8iKTW_Q(!TSj|}qgRaf$2kH*I8HI|xy z*!Hy%M~hkJe5Q>*#;u{V!U2DCThb~nCg+veRtE~%$4dRBgj#44hqY)3mlf(G@v6l_ z6)S>T3_HK3{kGz%lj=&W8LHx2J-6CjY=m@nftR!ElntN^ayl10T0>+@U+5`r!D9o8 z;V(7S-Du>4p6RqM@gH?&@2szvW?C_uuE@5Ktg)l@4}0D@UJQwvhK?axj2zOsQ^H_b zY(h(u-AKpJGvGTG+h{>P81C%ceB?U#j*0ncpEznCz=G}Fmh&N52)P4g ztmskmK*KmvEPj;>lc%J5-2~_un`4l+p!_b{AFy!6neW(qV;64Jj;jL*du~iR6Gcx( zG+z@gL+Z9NO`%@Pqu8hzqE8Nw*xR|RnT7vG+t@nLQ-(9psv*{iQz=^uIzL}70 zFp_oZlhBw9KU;zj3*{9JV09*!qfMQ>5yBJ!HjR~M*a2Z3LLU>0yX2*oj)SQhL?}JG zN2JsdKy>NG2Cov}m?#qy+iPe!NmXk6D)cr!8X#DVEfFEV4Fh(TeC00 zR^dyqHJb^x+;uAihyv~9V?sh*V3S8Ij(FHcqR`@#pT!SyK)0LLK2_c_(Oard7r>xI1{gZTUHCEY=A=wjZrhxuU7SW>2+k4;a;k5GAlx&+L;JAl>_d2k)v z5`0irVs5@;I!<8UG1pj_!d?8}DCOFBa3S&S?F{lv$U{9`NLc;p?Lxw~orVmCn(5@n z$g(z04G{@V-do-1-=BBY5h)zkozo&g+4 zz_0ZXzm{;|97q;)rjPY$#&(EbGqzuwo3Z7uyPv#B!UvdN`|P{&YpBs;xa7{Sp%jey zwHe<0+8kegjVrml`85I=9y-YONsYd>ZE(itEJ_TD%O-vbS@yMHnz#(lXjB3WcipJ% zx-=kUTOa4JxI2YUe)x8Y}(mJ50c)Q-Y~5X9wezKa?=6Bvhyt}?GHSH+b$5Vv4i_` z%_hI+0-T+L#8FmKS%f_UXL`wt5!p1Gq>=@XaH4FS#h$JGtu+WY(>-_gHq8T5}?MAI#0$L6+g z8pvO=y0WgOxqy<};hN@idt=plhV-`O0BL#WVN9QLHZ>BzWHiZqb8`RIl+dFr#9O*WfFbUn2=hbpd<3I2+&KY1to~x;y*>Zh9#(o>9GQ6JDtN3JIq#x@7Tyo1bYxdQeN5~ zrldrHVm}aEemF|~YDHe!bqZ7di>96I?QQ@A*Rh7n52!VAK0*Prm?S!Td0H79=MeMw zJAL@oe3Yc4n0kV@img!v{*_kUc8*y+4-F)6NoqE|CDI_JqSU)mmCeZo_IB|h-+v#| zky+0&F|kB$)AqLkEHytVjG=%0NHg;M^!^-@2RpCjuR8A2ZA))^FA)Uuh`3` z?9`*|+f3SM5NS_F!77C+{jhD7a6 zl5ggB(TUZ^x(Q7$ic?$*Ct0N}QE>KOZReOiy3kv|*yd5+)Z(EHD4T)g>Za zKRk5f+CLNp&npg&&kK)(-^jT1rQ_1pK2sDtxj1;zo7t5=4aQwi66$}%@HO*>=g-Tl z{7EowNL}19C83q@8@c^YfkPGtIz~(??+yFljpAGww(jl$fZNVn|_g9IRNU& z4~rt{aq&OAaMh+S?koS03)tLh+8$4itRq!m<0rSX^Mz|s+h|GnEOj&Vzwy_7a3LX5 z&eJS1OQ9^@LSxj35AB4s6L?IOuxa%oOF3~ORzi{K=RyKN$Wa0#FZD@w7ZReSY$8eI z7tl5qIMe1rVu4^`(H28vkqZgKCdNoI*J_7R>_P&Ue(HEYE!noxmvU*zRsm>9`AwqJ zIF;jkO|%|$yEAZD~c6t@-_3{Q9C=3n)P)mort$hE6L^r zZQ8Esc|lHOqC*Xt`S(^1M`E+2#!Ux_6t#~GlIp2U04)JU&K8jd7x*flQZ;IFO2g(s zr4=LRTeD+(iNnypcu$m}|M)J+W3CtdEwcD?JzeH7>;pSL@h`sfw^SB zpw4(H8pG$%=^INHQO1_+UMdEH)aZTgCCGNFXVujdp3Og|yB+0gyFo7F-}ZdH*mBDQ zK_U+&w`dp0k&`ekOc%~Zd61Kze=My)(Jm0GFsC zBMZVBxQU!IWEHV0x2HRM3^i(`sy02ej?rEXwbvDAY=;^orjSd9&A=h_r7BL@O;#FO zGN~2>FA&a^t;>7WG8fW=r?voGBC%J`=`8kAYg*5~Yyp(%$Ng=SbtCgrXI44p425=IVt9J%}B`N zsciGg7Hpf+d#O1X&1wJ-NF}E+(?I>*rIIuD?-gLf$9TFEaZQ>zStcu|1m(rQ*JV~> zMT2~k3zv!>Otd3fQVU*LKO9>JXO0m{fqo1T?STJ=>=@ zBk*VQ$PrjYvAc+#wV3>KSKrOxTY+s2GXxeri{8|Ks`nnjZ~Ey{yeamG!l@-qJV8n| zKXr+Z{`5v<1oq{h#V%zoc}ZwA@ur=zdseDGB-wVHZa2KyGQ$dKx(;0xRqRaEDH+zpPkoz6}$egnRE1oyFff~W3^AX z3)|qrq04`u14PMM`YGWfuK#Lw)qcV=r(YF1 zlOfIE;-S-u`7S@QXrL863QX~0bmA*qdZP5}&otj9Lk;U)DX%j<_SG(Z7Y)gTOj6vn zW@_WEOSE&p^bAnwj9+7`+1Jd%68l$*+sx&1dE`<#ZTvc!>EaiBPTTPrA7gk+>mRRl z=>{Nsva!tNNa5~Z7B2*yMjVIz$jX{I`8_TSB>o*8;0WM)3nC-wp}O{yEcmu;oPcj> z?c?PoB(Qz$BzH~E9OLQszwYiga2#h6#g^9oxbiBn;dPI%%SvP+pJ~8rs53wejqq3w*6G!WB_oKvsM5O?=u%% zsi+BQ?hcXWRzt2wMO<3xOaqXqg3cNrn95$50tkeNk^zjeeqtLbkSBQB|J_Sqtdpo11pC7(~aAFWb zS)MGFJ+Q|`y5l>JbYs@mp8C}&TSWpo^2+@y+3rN$BghV+wh3)jUARwRZ2@Y;3z*aJ z0U^^81YrcDjqk!+5M+yq=o0;cL}R;>=nz`J2IGgEfcXOinuV}ldfywT3x1?9|1{HV zqj<5mD2t$2&>%($G#l2m4_Fh9BzD+1`9~jW?@q8=)TG6YxDi_?z2(VN6Nu6>vCr*f zk}RT9SqaJfZ1vk3!Xy5s+CuWC>m)7&oDc%?Koe*wH*A5oTlt@FipJ_#4awuFz^<3ZNpZno7P#rKzfEnFL~Sqk+;|+TsDo^o7k(TD3Q@* zIr@5S2RLYCGSR*rs9jJLVPO+=sh&bAO|MfAJ>*DZ)=eFMf{{{M2+bj}=!$Xy;Ko`N zH)cGusHDuekTGW)*#C>G+f=Lf=q*?rk|844RBImrw6z%K0)^a2eX;QU0$SJ%3=1pbIz47O! z)_nA2eA2|+q!V)!GvWN7*|a`mN!vZ~Nq6QX%~)|rc+jEg)5fM1tSJf}TO2$pFZ@|j z`rt9?TZ_^Po-7K+zj<8#?Rk}#1mpHh2wigc@cgRbYv$%v{x}%-YHi$x{$o3rjV&qv zdQ$mUC6T2dpOWK;)(1wSF&&tMZDJ|b4$@arVHOZf$-`dfO1I_U;7C9{x z_~F3#=D3Rv{kPeOh=u-bHey#gGQOp3?wgb%bD}ohrlQS8u5W5B{EW&L7^=pQY$L!( zMTRqvxJfqAUKlyWs>-P!f(`c`NW6$%+d2XC8o0Hbpg0y^tZfKj6UTokub-W`PIqc_ z>zR#CjcGnS+wkPQ^@O~BX(2Cz-3aNsd;ps0b6HY38OWrr<_zToIn}~g_Z{9u-*?YV zd26$VwQOhLR*=&_H@aqRGD+aPc{a{y`;b6LSJRnB-Jp$zf-K;Y_b`RHXbvQz@z{XQ zex^~>Xyp3no?u;A65`puhV8jA_J&Y_ilhjlvGullLuZ^@E#$~akrjgg32*JM+1j6u zXoI!?RyxIO*8cwrqBZP{axm zM~4t1T#nAv+VB~?D+vNPC#DRq_08ZV@|jxz!{Om-BF1!2X7yX_Ab{R-Id`^dsEZvs zF`87?rZN@gpbV`Ok}*4D$6q}UGM4;~(AOU8ihaQ_w6fEcR&Bb)y7;OHseN)cD`WBs z5RYJH3FS3QA>m07t-G@WB+-Sse*j>|LU&?BU(yFcNq3jd^ zUa}hzFZ9w?vrJw>FShEkQ$Kuail_iIKc^h5vG+>J9>BKkN>@&GL;AGmx03I>32gJn z39`4M*uQVh?|#h9W~6ZthZi^?Y_*d}jCB@T-dcLv+Y@Q?p~?icWZ!(`0IR2gGK`L> zyrljtx>tAvir0JyVVHw#9+!|4axypEkVf%5iDkPeLkr{_?i3MkQv} z&uhCUkhE;kytcanNtui8ZhL=JLT1VGwzu$icg>4TKXy#0tlyHh=WoxxZRxp-(N3W4 zlH1SEsmi(~HLoi^J0WW-{{K~0VpUVW@ZAaN-31duSv!g%nQ`%juQjw?i;md~io&H3 zMJA!0_sX}f?YtoW7oKHnZ}#u%qHfJlLtaN>JeStlQyV`R8II&oe>c~rY+p?t{Qwa; zLH@uKUJjfpoJDH*^^}(bmp}H_j&D9DL~-;f;S9mHG?sKM{?ERa!H%QaYg%`P+sz>W|47%Z`1r6#9%SSYk{Bg z-B^QZ8HXsq#GIf^+Dcjt7>YiIqA`4ov#gCTRV9Og!Qas!v|kbw?wl@-*@STj52sD` zs9r;U$<~}6RckOQD9x#xw#`ctS?yyD2}yU>N2#7*?Y*-Tk42h#1#xDn?lo)Gb560I z%q4>?T4EF*t1GEa4F-sx@50EGV*EG;JD5`A_%HR7!^EGO;ADS}U(H(8FMYlHOJkU{ z?XgT+y)Tp2#}wIPL$_tQiM0nk?VGu`XwPvkVk23&~x;iMjs~%KU|3+*y1-| zGRtUvW<_Z1Gg4g<+ygmGY$+RK*Vat&E9oa5a)`HS8S=em$%m3tl82CGE8qg4QG90( zq*Y)|tJwJ%Ns0r$%CG)aVj^Fp*t0_z@7H zw+48Y`+%CgdP38!F!>lgd*&VH6QPB-mb@6#r<3s$l4+lfN1M72v=QPxb{tt31!17a5OKL3#oOx+rEMoi z&_u8y*v2%5l^Ft9644I8Zh- zGTY%-$qxzGv_M+F60bs>4b` zt#P49{$1H*d-BvlZ?#yG$%1cH^0wjKxt67fyJQlbpNDo&?9WihT{0#Z?3m&|B#-{f9#Us|TK}V^oGe!32rR~bWYoK&63u|yxD6c@ zyjA`J_6K*#_E>@<|xRW(Wb4^)*zf0BO3B{!2 zY#vbDQqhl{Qks*CTSf+`VZl>eyu~M=TiF}lvibY4CMo=rw_p-DWJ2}Q2`RhQ)E7aO z{(Lhdel1R5S9uS@Vs?k{42wNFhn0}vlW)@KleL79kIrB#SkuRwOeEJdpD8ou z&1V(2G#<;rOiu)@+GeAE5`kUy9QoUA8WGcRjq$T|qUKS;~P^dexIeZBMv#40-2!sU;Dpzsnh5_n2&zj|D;QpWZnF#a1xu)1#{fTn8KU zmfovKyshGB&EawUeF@I%-rkNkKCv9S6%Ix)i}aS1q|wi(klPf^>J+hB2WCnc)R74A zR8qonxmWMEgq7|o;-4$mY~ydoaDX}z`Xw}Vq#0`%A;B(jAn7I)1a(wqOyFCIiPe3j zj(n*43e1*RmL1`6P()VT&Y2gj#cV`xAOM;E=?H#$j$sX!=uakzxn9<0s#COybj{@e zNx)S~szle^QA`cnW9J<_mdx!hRDJ{gz=*yL{}h+Rlm(%Idt`sn{9~@0w9i<*8$y}c zv+`b3u1jYNP>JN(o(bJ+qZfxB%D@unzSpNou)*fs6ilEq$LG#suT3Qbz93S*@Xj5;b^&pqn{8g z|AjI~%~5%O_1a>r^;)3&|3)pBHFxx^+fm;f_ees^uD9djho;6K(GW;JX;OLe% zMP7l)NJ{*3ljgN8j88f-Coyx{^7R9o)(=?HHVb0P?4%jliOZ%gUmy2r+TfzJwX2GP z-wX+_f|#-}ee3A7g2#)3{}>l+pOUn_XYwtZ{7nB7eN6wK%`=<+pY}ahw%N$I0-H05 z#l*Te-ju5{GeC&oT-Bso0D+JobE~4dE)Ho2X>fK$8)vxGG`e zK6HX(YAXhb@tFopd1{X~8L@p*1DUhPBoRAnI@!SN=NP;ZMhvV1@deLJ29-|sO%rlN zM==4%fL5?cc0#Vb^LFk9$A*`L(da>x9g^GryeM*O?VsCzWC&N`nQtca6v5ZDhuZLBo|)$X~cHlA*>{DkPE5&K%G9^-@)cb}nA?sNF=DGQQkH1#KHf z1C2G*)y|hf$<*W_i8#CPU-qI~)6m&qya zeIqV4B$N)r5e!p|`i+B1D$lVZtN!WYCl&p^@eKJ#FhmuF-ipc1eQ+kZlSF5mHa0km z+CGY+;*S1mO!Mj9eY6qdqBC9e`Q+I9pa#1^Bp<1Jlr=UEbt6|fS`|L_V@zvMxKzw0 zNYRm+dOw2ypIcFN^f%(yYwiFAqKYFN1#Rl20p~-Jv{BV3y#@SeA;b#6X;39uVUq9HaYhLAby*GRARZ z;A=5Zdf*kt7ATWslQ_mXeeEHe{JonXPvIy(RbMVV!>m5eI`k4Ft6zWj$}54Cx$Z%P z)HcRUM-$?}r&VpWbv1-?xCEOFN@&{`$@NXW9N_7jVWEjjn&(h)F~4it_!sQBRTP|n zqI67Ryrq+x!y@(~pd1SkaLG}7Wz9yXMz}>d-mB*kg==XxljHq5W}^)du#SkKKi*-7 z0EVB1OIY@E+)A%xLq`&=0)dIi8$CJYGJq?inRvFvWQO(>!KJ(_@#I|%eM7YZSQtP@ zd@6Qfh$Lvi8f#t!Vn7ivvNPhJmar<*w(`YrqCU6Tn4l;YtPevy(Y!(ryLu8*)yHL1 z*<@jFR2*Pf5XnY$m5AOQxY*dk;vdftKiHU2JvBRbBQTB-2C&;5r5L~?0jMN~84{z!1oi`p7v620iJ!{CNq<3NQe)P3+ z;oay%_+G!AxSx94MK2EZnuxjH-lOZP9!8HK`yM6xzwx)_ouRk4{c+1)C>mo~(I{pP z2feIWFgy@ea0=ybDw}!5j48T*o8_1GDNGqHV^ktH-rj9{c9aDt2d2-yAsBaeUEHrq zLbY&tFz(8_xXT|K6L~Q%{@%rbzIsz)*ku-RT#GCT!0xSUsd9azjyC_Yo;IwLc5sLzAlFA3I|A^dljt zJSY@da|c`|drKmNj!jrryu57(dT5X8AL>L8?ddR-d^Om1G7#L7e&Hzz>03t~c5SC@ z*IfS5(()Y)vhH1!_~<|eSv!~FE*WhEpFpX-xC^+b@U5Kojc0LFvgBJzy8gU~z)N}O zX%Bq>j*0boZhhH~UbLGW^0yW-2s`|$&{4&??~WNdcr3p@{;JSPZ+^O`d;f+zbG%5k zCz8KvEqm)D$hTuCEmkUWr1a_C`#X;~NIGoAux{L94r^fh{!XOhWp_6NvZG5+j^19~ z@dN0=VzHn87g)Y?PFkexilpQ)#*U=A;y-qeeD+sSK9791NDgc=IIq@M*GjdBlNht! z*W4ecc5~;8q$vlk1A=>z?&!v{wFVBBo<&ak6D8$*)6#9SYQ&BdrIeFZBP&yiCwhXo zLSK8yO=K^byf1+^lFTF05KB`yU2tdKVg|q9r_TErbW-whoHD*hAvN*(3h^5G(O2&# zSjXGZ1J=H=K_I_}Ni&9UJk#t-Vzp_*TUve0CBcn-?d|_hg;{eh3)ggRFRuI#0*p1o zJ}jz<^8W{$m*wmKF9DHp3&iQiqCG+m|AZ?EzO2|~9p|89bGMFG!(qU2H927*j^>EP z?YslJGiShY6lK3n9MiP%XqQ}FHhH^u0mgK&Rvm}VHOi)}n7R699QD;@)zqammTW6# z%My!hLnz#N^<>MN%z@q%yT*5W5{)D6NJ0HAANqW9c8uLetEn)M zb#H%uSZ^Cn^&5lgif4$uqm}^F7q08s)p^}NnaY1N^4TSkm6wWufp{S|v0 z=qDgBIJwqGMJH0Lsot+ba0QcfluTm?{|fFO=P8Cl-)BZyC3IUiz6~+ByG9iwE=eqp zxDBKoD5~i6y6s`nChS}3Lb6ZeQHzl6l5x?BPXnSZJ}w$BD(&k zz51@x#ygycjDW2PIWl2WBn65F&Eg$>a4CUbZnb;;6$9|`QXl3Fh~=sfgCWviIIvv; zQ2~{2b%V+>n6(=q$?d|PFv29r>*Wq}QD$dwx ztTys@htSLSxm3E;D*&ofh($+pw6#TDo!9SduQ&}90ySz4T z%7oC7>BIA94qtO~UghXu+zAsx8>zbO-oOtR1x5~=1ZV#8?In>JusRut|7!j<2<{`R zVcYb>f#iA{+J>E#Qn0Wnyy@n=uA2r1I&vmOHe@7L-P z%CBKtt)k6jj?VcS+pOXKCHj-8oiEH+$u7s2#Z|+;~Ka}-bCF1M)qrn zyjR2!x&wbabPWCoJH|p9+L+=wJJ;4j09+(Aac`Mirc2y|n^%0n{DX3Cu_@)RrBIRvwqWnjq&a6;s>IqjaBxBP~zD+=klIgD=D~=$_uPeYzJKeB}Ymy&rSelzO6l$Z!4_z zt$mYWv~j~jh$-9*i0NS;41;pUUbU!eGW8mZ6x%HO>t_ZCS&oQSgws?9c3YD*Mq4uL`Uto^zBOA ze#1OS^ob9Y>N~dHImzxHP33T$jl;ZQ88X^_uuRrc#= zt_^xlDc+1_n$@<T5Cay>c~| zW`0?y{@!puTv`kRupv%Jn@6iS0OgGZPM(o{7oBgDUyE)F1{j|di<&e>zz~wt7{~YS z_}Fn5HDJ7oEiE8_U+aZq)vNd8E;}lCqjKaH#1_LddCeWd*4{iMVgf>`nB{t=A<2bz z;VpdtX@JKbXBzoYg0Qv%5o^e0<8(E3&1J>vCu2@&?;xoOBPqHL8te|8x*FNB?Mn5g z_td4uY`bmr)P*}qYc;t@Mh=uDvR}>t099PU${wu;|j}=Jx;O=vT@@iC2X@+G;;Jdeie?5QNJa^qOb_kbO!xm!ZZC zoQnh>z_W3eiWmr{P}$_U&s4y_gsa(OFkJQmMqA+6ScFc0ulX{DK9qag*|O?t$`bI$nnhT$3Uoz-Fn&D?axd$1m1PO*At?jQYEh)k zjt`{1cR39ool*t0)|U7yc?mJKcSmiUi~D?Jku7Xa5}AaR4}~M~XuRw%qf%mQmU~w-O7W%v7ulcfjZ7C zWD4wd_9-HOQvsU{K;Z4+i`3{4c7F_A53c4!cO84!Z+d1)qW~5TvpTqf|u- zia>?@idDq6DYEX+?s%FBNLxl*Lfzicnb(A$PbXE0>7pn@l#ff02GSD*JQKvQ;a3aF zNz7Syc*a&H(GN$zhN`mgyE=3_FzGHXchZ-Xrd0ZPg0`(zrGn}&9q%XkrVx)sPk#Lx zg!FZ>J41?a7ohMkbaDDuW2(`m-HVP2B-8Ns3z62t7l zQ2yIoL^|VW!SDaJxrj1^=d>Auw*Rl2i|pUPTk#a6va+s`e!Y5$TiHZTNx(h7cmHo^ z!p6sA&XO9kV{U(blYMTNusB4dHrS%GjyWjKjQk7f~Sju$K{3lq4D;Z^rd6c*8Z+2m|7e>EHAtzDScYePvZKOgl;{2 zcz$(W<;B6c?X_|5O$ZG`r4ZGT(+@a~-@e_@l56OSD<@mRjJ_2l+pXN_7_}Qrc0S)CRi=-QPHD?>O2^7-AH47%@*Yd6Ay&U+JfD;QPoT%L^Rh%-GjX`bg zS1|=>Doqol)WtR1_wN$5c1GrnJUccV)ywEb3oZEvqxguy9YC% z`qd{|iI*OpCLb!x*>W?_mFmc%0Udc?a=^g}A$JH@ zSa81uAXi!H$byF$b#F<;uLGUbZuXbla*S-WMvT)@TwRg*V=F{BiCz;ZSvCcijz%P% zJYsRgqv1B(16Qs*e4?Y@_KUm!JCs=e^1?_l`*N{m84;HY%y`cyJThRxXxA z6E{VD7*SHOfu5CQ?tVQ-R7Q8dh348jxv6%fwzj0})p6O25ZZOUH%=Tc-@9^TCsjPf zYMQR;3q0;2KQT}AMo6<~_z7&H-*X}iB=2Jb$*xzSh5Q+7Az3DK8Q#}ZILu^-!FeDK zZiTCe#9|7#X>3-Ea2vg#{RCxC%I#mf%o_6!gg~gH%#5$IMj`*Jj8dIoIt{RD~%|Z&7}IhOxz^=_p<{vZ2-- zo;Ds>kXEGFbfTb&*bW$B%Q3O@iyB&4b&}DeiJy)3oqa^dH!&?`L&p-~Da0pesi8!TS4YyU$VCJul9XunCUjngffbkhuyGKOmj#=yLB z6>?v%>|LHkaN5lT2hZZyw7FR2gf6KE@k#cby`hehFV;VkpX5qDtDKH9wy@`Pbl$*P z$p7{nQRE(II!I5fSj5^E9L==b959?UpUBFK;a zC1q@T42{)nv-UH4GJ$P<@F?#w_ShsumI z?&U5-e0Dx-1uawfEG=Nw2zz0=#sQ+&Rki4KB|WnZr{4F)vGp9qQJoHJ6*VQJ1tep@W(X&`3@F} zF*j9x?>#lmua2M+wC<~?ruCe*ic-`(<{-~Rh@?(;9rTM!zh@x%xfDk+~yx7@6A#IQ^B=G^TS(X6=lxpE56O%S=KXE z@K8Q3A$poT`;FcN4=+t@1owz9>{6=6mKTn@DA1+DWn5|j;?xnAEdildf@R9-*cv3) zEJ-$78I)n%a0yyT^Q6;Aj@)(?!$w4NROOBUX=a~3wSfSnVa*%~=@_q6 z!&;Y@P{bw$wQAW$D>Ek}CGlTlJoHO)$dE2rZ&E{}jaR6BE-;xqL7 zABH=Su;jGwNV0_mzm54d)x6!zuHmTi)idZ!m~7`59Z7gJ3WkS-&rV8TdUpD>vj>GH zqz_*+Yk2;wyvi}bxN)^{XHN+20EPC8yvhRbXV;!bw9;kG6FEXOZ$-c zX(Zu70!t&qM`%qYL4?-lY1Z?^w+OkSnNidmZyf-9Vp$VNbznMT=hmU+oE>pFL0#85 z%PB+NoWP$=q??!f2tJNeXUl7;7Z^K29eRKi<0wxI3H z1!B<2jM_&kiM;VSj}^82iio+qwU1l)>sd+E<*i+fx(a`q4fPqX4k66u3gZa;i@3!j zp5UlB;OcdTy&D;Ok(<{6f}PT!Za}`)h9X#Y#A%L%F9udMa<(U9^-*+~TN)fElSph& z8;_oBWi+!q(!%-BXB_8RFoFC%Kpd(lSteFJtFET-Y(Bla+rhf+1_KQsrOMgtF@Zjh z?5qWJB@^5PXENBuO^^-eigGlEH@Ba`?}En509kavjN5$7QJvcu_YA#Z?i=-}rQ!su z2i1mlYOYvWgc*)z3#{)_FOpK3v&TI2^~N#&&}IY z1jRNWuB%}X^u6`s{tXJbher_cyMN!+u*G8ttjS(`Kzbc!e`o1-V>3pD3#f#$_3|UT zXWhZ2fL0r-XvjhDI`kY*XQHW_u|stHH=hN3Y)4v;`JGgf zI1z^mfT1L*9$C8Qo#O?JP{XHTMK=BhZ8w@bcYAvhsq$I_a1?m?_e_i$G^tuy3RfMm zZWSQWo3yw@T(NSJQWH}Sj(Be1ilJb}(I+=+oME{{Y}+O_JaYsv*Fc<4I`yw;5?aKz ziE$)NwaC~Z7$YHu5zw+*hj?Ooy(>&Xm!!ryb~UKgfvlgW&VlRNd}_79hGFqrsMX1k z%UW$gnE9EHRB-+#`r)Y8RM!@3EGt`H@VjW#WSwKwzN@C*w@aa&esZ<(Xc~pJJ@;AR ztf$6O1{vx#Ew!8Lc!j7Hp*IW(nj7cz?A<>y|Bmd_Dq!@VHn}|YyvQ4I@n5IJf3!I8 z6f7eLOfEn4^n_*b`cGTZ7KDs4JL$FT#49dYzCNL8{UJ-*euiEE%YP6&I4}GHKw4qi z+9!&FyT%1S9}>PBGRo2EgGZ#=nVXC)Po}^*^B4_T%Da(bxaAfadYv<9Qq>}=mV+} z9dBzdQuaZ_J~iw9XovNEOU!%ZHAZIF>vdmH{s8A+Y=iX45SbckHo_5E9r zz#L3;uc0p7e7Y0~<_vi<3$cMV+jnc3K+MUGW#3J? z1-~Oy*D~lA*)358Zm9Yw+~5b>bQkC7SQQhFqh30J(PoLtJ=#Sp1t-IU6Uj$fmSZK) zqfwoHZDrgHUxE%u*b;K?+*1N2jZSQG8XeDa(f{fud+Q~aXu&ef(*IV3$JT3@?}|bs zg9FL`hTDH%ATdSh#%IS5pvQ9iXK|;8Gvo*C17;7TqA55)F;5GUry=|DEglSycmVZE z`#ENF56Ez!D{@vtFkDIz9b6H2;w~Zu?f?sRY}FS7lGv_y37q)0ksiPVZ!uZomXV(} z-ceN<1q6gju?hOHRXz@pKwMakuFNJy$G}29X!v~5mtAgBPTv6$or)|v&3ogl!Ep1U zXWc_AqLXGt91jabt>J?T>*x~5boMohMs@l%6;j7SauHxl6mg?AlsB_()PlmOch+-@ zJ5X>RukynenpwALR_AvW-AX8;UgL8pG?$8h1{exsFd!i<2;%M(S93yvQ`ufY4?*; zHlkPO)F^3Zjg-s0rS=_0xT4>Wt%ovO#!nCj+CS+LDZ14W!t#N??Kn82#!~Fs#Ugi! zFs;9pK(p^lvOx~N>)XIzHOKT|h)bArlG;WzA(|?xNhjywckRfgxqGN1x9xeampZcT zK4A7E)8~}t_*wiH`B?li|%fF ze^f$d$?~?h@OO93i`d$+-L~}H#f1DWx&8e1?$JAQOCpCBrHuYoZL?Qx`r^Lw4;k=W zvncTvVeS9P((+G5OSMIbg&>0?&)?TwQd`sb$?ew>=y7StHns7Ck>LoJ!%Y+ov%5|X z@8#D!vjz$_MG55gM17|+F>!}!0I$5uO3qGlYBq%=NfM)ZwE>J8wf#t?5I5Q+i_!KY^f9sE zN?sE7Bhr79NHqeWZuTQa%SjFUpvL5SLEVc@e-)t?MWpZv>_QAqEW#ydD66Px_VlbE66Y`Ku54MVc=r zpPpoR^4@wMJqg}&pV1xA_YSr5bp3vg#~)Fsa9bKBjDOfzAY5DA|E#!~@ef z4J;LBuf41yKk${A0jGKCq0$<-?H!3=Y*}&+Vr{R%%{5~7#VXWri32lA;y}e}V%6@o zf+Nea+c6)lrOSy1rlCBu=3vzbd0SB%9*ZQI?PKeWfvLL{a@D-pjKBy30hyVoD04R; z;l@-FNGM?7;4EM-UhNZ(0V=*H8eH&}?kx2ZgDG6*-Ny{I(vVZR)>y?+bj>~6M*8dw z)F|2A?2UH@fZwS-@#0$Nnl6+#;CeW$uO35O#auG*D2$X=b-c`^UIB|7x6u zWop1_#Y!HjjC0I;DOM|ZHgyxjv!P3|H%WUsY96EF^8bW`KdE{DX_W}NjrpiV&}GgKxYSOi6viz_y`RiSdf2whk>OZKrH$5B>nNCv##JKag%!4q zr?mSyW>2blxQ+lIyU@BEfnawMy&$6xK}%Lre)#@%J34z0tV!hb$^(bewtT-PkzH#R zq0`#I=S40DBGX;77h7Gkv!=q4w+qbQOIrQ~Z3ZP%h*_JX2t z=`XS?lOX9_*}ij`^t691W6;hF=6wEoa`;UKS9cyesH$mRKVr5BqD}LFXy}=}+rMWv zDI1Sr8RMGJ#Qb?I@f*jl^XE@oGymwn?0Jviif7MkNh(RvOH2`2`&BUd=lF~96;C}0 zu>v13qC(H??fyNp$@CssTPI7WUsEZD6S?7&PEU=t9O;AZan~MFDQX<$WYl2gPwT8I zl@TZvvAvGp)qP4zV)J1_3OO2oOrbx*M+7P}KFw$lh5n8>wPL%~$OQg4-W!;}V^y2_ z1}vMtZrRwB%^r1ZShA1I@h)UT@8>W5w2zSs$3pg*nt47VxU}}i6&SmKU$0Y$aD#U# z8z5G#8K(`f;j|6DI4#*9r)`MFX+QmCEKd8x52v9L%Lk`Xr2Nm>YLQZtqh^LLE~<3l z=WPcsGDsvG!M-aaE&jl9mZ)HLGjNKl;GY{#Po=l0P+1*P`tZXg7;sYc2+dOj=C&x! zdd(gqR1em*YFu9TyEzPVl9L-E_th4`LVIcC`C9Qa0qP~A9S{mkUcg% zP+7gy)F^E}%)F?Tuctz!`tCnK9(8MB*W{_pM0vE&KJBx4 zJ#(I{!`Cr#mLgmO7uLJ(>6b)v)Yx2qZbvx z@B?I5qB-*)w5-P*STx0|?L)B9`}Vc(UN7>z(!D2Hn^;x zs(=umi}yVmul3c|A0X7|1g^1))$z`W*q&3n3wHs;0`{ca0iy&Qc7+v}=?XC)JZ#I{fHKds#-Ot>C;YSSQkD2 zt-YM6pjux^uSkKhyNMFm+V^i>*5ctAhM6t_Z;HY|wKgPbQLmX#LO|yZGJwz&tkrYI zK8--axpKp@tvQsCrPh(@GO&`iinWJPTZy=d$S|wHZm1J*nbyy@>29ZYN0A)Vd$cW8 z=Hh17A|#oy0(JcuEYl->TeYJ2lUFx2cXoG`!7g&t0key|KBs!ylms8Y$fNxHBBz-B zBFFCee*Gdx+uCSri~p0pw2Ir3SE0ylZ+)O}bFl3^bj)5hBwT-Uc4cxfl=V?bP9XI$)R?4|)AV2MQ+bsz6|3!|G$%H99{YSt@(f5_?LkvcP z{|NU-(@0Cn8`0~Xb&DAt;F$!Yq-i+^QwRSMa^0sKuk(}CYuXQDpkwd=w_$slzKu&8 zwlY>YsoYU|h`RSZPEz)oY$w2uWNfnoH^nE^A5M1N2 ziSU}e+Jaq~6U4q!?Zd8RFuPXo%dTzrW!LJBsI6#q;~BEyTGQx{TYJX}K;TNS5jv)b z_#%#+eq@CW%PzKptE=b3S>(RRZ5fFj8y80Y$VXi1%faimU6S9PdS>bs)AHMoIx}rG zb%tFv`zlM4b1Nn5c=x{^m+ERZQc!P~dpN=&b7 z^`TuWsyLn!2WM=MBD1}yOp|3wQTA`@q&dy%f~1ZaeQ3$Tile!z!c#;B^GzJ`_LH(s z1St~k)ij?eGvJ28HF9ZJg=?(fl4)>5&JdSGU>)m^r<0gAC0QUhRu6_JmCuJX(wxzQ zuU@0jfWDXM>50;#if5S4N};TFOXJnYwRb$j)jQSq-&0fg`Ur(%0b2C2Z4CAwV5ykv z_%_si{)hR9(TzQ^^uzj*@Jn&c40G74>LneAwXB=su|?^WVOZ>AtLiUY(!xU1$r^U4 z2Yh1T{-{brM9$)pdoXsjC689CFh(l}SA*8SUqi|=nc6lWDR}Y0R8tOP3!TpFHQXF_jx)RMlK-s4+l{f9j=ME3UzkAkuLQLj;f$5 z8la!-RMH+Q9hhO+;u6Rv`k2@tRbBeMZli%F3&MU#{iVbM0DZKKV)GRW1OoISUXBWN z?axbAO69tiN%6cjr@9n`!nvo!WNMA;4L1l8r&Mfu?+Jo8Hf&7v_~24wFO#(YYFQ?Q zQNde?TBd`Y2gd#`aS6RGtftd$7*0}FbSha`&|W4*1p}ybqdb?lytGi2_o|rPu`>uL z{l`15|DcTU zyn%tz9N43r_R#yVC}f`N)Qy3&xf$DbM*0lSo>O_21+AL!+6ze!gjHvIdxBlJ$<>=Dyci(C0$oOV_ zY|{Po`|m-Htzwzp86=nsG{Gu_vJ#MKuDRovv)`zuy^vqp+Cb~fK2mx^;(+z8S3`z)W9{rfYS!1ObkyCZTV_ws3vvK}R?ruYr6zU5 zdAL@RB9c$VllXO~e2uo0KCRe5;{37zq*ZLSn$?O8He#ilNEX=;S!`tyD^$_iL%2i^ zB$*dww~yzDU>o5zj;AiQAlnsFVJW>OhGLr?OR@dKmtvEa;6^S0BVY;1Tnq}4od9C_ z+VUtd#0M8B>}9u1nnT=HW9rVBDge&z~F5Oipdb+srNSl&Jl7Kg`LjmmqTBYlxaDbJtWd5 z-3Vy~LN&t#k;+j#v*ILl;ue>t3+bnQ0!upr_0YJ1v9&QF*tD%Dz7^{nl?S1s7#wV| zu+u}JluGfljo#I)0h3UEO2}povi0N_v7ml!p=8I03(%*A)uO44k|c%_6&ffx$%5c0 zJXWX7+B~c|!ucpXQux za-{m!dZP6~3v5KM3PH^vA%(Syqd5kAs$s%gIz?8@b=KOq7LCx<224JY#HnHXaobbr zsTrFj$l(`3`ri~W6@`(C2`MuHj-obmT8mG9+9=2gM9Mae+cRe9n%M0?f@K9|fC(FU z$8gOq;Gmb(6rbvjMJyJl+741&^Hw%>b@Wy&fgL27z7>|V2xaQt!WUw*;Cpg)7!qCz z->XfJ8b*i%|17uKNXq1F<#*vNSca}SFBD{>?Z#UWuV)k4;;w@1=C0UV4nCRWg{8Afx}IJ{>cGf60Jdisux&l( zu$5ils8~e#rBG024O%TK)_P739|St5X2B!f#60~lg&@ytA44Gg+F(kqALB z{Tgf8iQQ=PtwRqx>0%NdA&f$}rUF(a8&9Qr53h2SYoTx_pNqoq+EM&sZ@x#AvH+IJMSG?{c!Hu$5_hopsHzGWo@Xh zCefXa`4D%CR6UeXe*!(#NKHmvB&zEg97?J`3-AYWeE1h@^h@cSZHR0oRfdYI-8W*K zwD{33X)tWjI09vC9vgXyv?dusauQ8%yCO8vuYrW!P6H`tuzPNs<6Y2hl#h-*ns7o7Ip3}a#|eq2K@)i*LKwRyQl7}}V=G+3^f0K1Ovi;|{iFR2^-9N4JII)k2{c*$Yvj83)FO0(w8Jfhvt&eu6+HKido4Mo z#tZgvc|Iz>TAyDMWvFNC;LPC}67#Ls{?VtCD?CIx4(-^B73=~>X?Mm5?7QS#+Atfq z*@DKV+d}?CunQj0Jr?v@MK&3IWAluMR zy^Le#=tx_=Zn8Se>(NyEmd3?vpM!G59l2q1CHbaDzs7jCt9#K zdV5i?@i8R)bY+XU_{{S6bWX(J%c6)1>l{+aP2zxdW8QA%B+G^DK2%E zIF1+VJU4HpG}CO`fpph(+yGM(6q^VNqHm&KGqx6NTlB-B*P)BQ>|EB^d*}f-_+Pw! z=Zf{8Hng1_pEM&gX<}yL6+d3SzOA9{*!ZN^j!jzq@WJ6Bu>L52JL7vc_}_3AD6~WJ!W9RP>B_$=o7lUj z4!Wmh@7%597q=aGd-iS9@c(zJ^STy-M$4TP*^`l2^-#a?98|OA!}tFmMUla%X8Uun z?f5{_jEAN+zCSYjGQ%x5^fo1{t)uLkf2Gq%)hq%!?n=nMIgNb!Z>JGHtO&B{3b}tf zjqon|f8S}u(ta6)UVSjrW8+F=re1Bb2y<^0a!b3^N~DXW3i2+y224dhj$jCHX%^XS zL%=}InE_NCw^0EA2T(Gd@wXNzNIR>!gPjU$i-iCwS2K03}sO z7xiK9n2wAYx<47BLUl8ImuqA9j-cpP!H;0I(~r=u_0m5ZmW9t^K^qed^0x~lAzu_n zK@oSfA)bVO45)H)Qw=h`LIE+dWZ7n=m?Dbk9v*;1i$l#oZsJoX%@7%`*uPg}tN17b z^brIc!uwK-<8f@p+EUYtDH1y{IU4#je@FCAoRuAj2t)1OPPsV0i zG_mtdEpK$REL4d#7rRdSN$${>;|%;F@ueI(SdsRTI|5j^eNU|)9!fcITFC(&$JYd| zNsgadAE-ZmQu&@q<-1EFOXA`WsSk7vpH%+Er1H-PBxFurzW(hM>)&i>i;qtlaBR}f zjKpQ7%h$is(6);((ec5%hJ;^DOy4>R|D_c?ND9f|UHzv2@>BwMA*q$Dl7{~4auTcl zGa>Xmy#1fXA8$wq)xy62oC`x6Vc&mTFtif?8xjoF!o~lLfua8RyVEZWP1p{cHeqqw zrrYznPRLHkdNeg}%}v9*1}(pA8e5U{!!@!Z*^`ynGroUlSnBYu&WyyKGy8`+A&}gN zKb|DClJ`-oHuM?z*{B!5CV3b}_QHzth|QtmHLWv=dr z7F9F}f=m7w8Kp{nqKzMWH_MfV6UAojm5q{lq&+!WYDt%_8?~K$6>0W6X+HHDyKQVS z+gru^;I^VwlCq-0OC>q?jSBKFAy&rBkyZC+!FRGyL>9)w8Vo-LSlt)yj*)mv>z^i&WU{#O z2yw1|C1RIWyRr;c;aw^IwaTB%09U={-199w!}K|Zc! zBNGAwaGM75$y``a+%htmZHgoBjT3{%$~g|HjZrAZ$gK)ZAAKtQj}nQ6?CBK~eGumK z5knzb?kL0|>@JyV{_|zjSbf5!V@MztAyfY!a0~6otN5d>n>IW_qzufA5(tohFY?N) zlUYFvh$pK`ek9QxXlhRY*zz}_hn-flri3mT+x z)?CAuY|3-nZi<_TsM<9wyNt8hLek_2cl9skz#Ajz?mR+;TmO%}cY%(g%G!oCbR>yN zE(EWT8xas>2wq0OKpF@V2P7~kpuQtPNAbc8Fy4Ylnr;FKj1uAvyb*}=3M#%t9mnrg zoX~+0@920#XNDxAfTGffkN_e1pS{niQ&ruks?gDIt?ys6mTTEvDXP0qojPYf`+4@W zy>Xh}qreQ^fCuSujB(dsSvH=af74w+N~*o})99-1?Yu{ibQEtb3l>FU9pX9jk=6q= z6jW(VL(JO%rJtgUiwc>Pi=K7p3s$3DUDWDU;(COZ$_2AT`qkr-In^p&iXG@km+g9*wr}ju#S-7gH5&g4wiCVTyynIo(jZ8JX zB0rqo(lRu!YA1#IA-&&7J?uwYTb>-;G31ig3wxvxxN5(=qC6nluC-6Y%TKri+t(`> zO}xvyq7AZ4FOWu9#TBF#c!D%?EitOqn5*FT2*iRb7?MFNsq~f#b=kZ}pp`@uIpi`; z>?gcN%=bum8JAUcbBL?$T1f){>ja9}ut+OO9-5BjZmkzVD~Sehtd#^n7SFzd5YSIdzdUX~*rcX?Qj{OOJ~bcy?NwXiNByGW6r;rH9g zE)wsN`}Ay*U8XH!BRQCX&YHAg_6HMNzA?kF3DUFS1j}xS;LeGIBQKvMXt0 zr)3?n>-;eLk!!R87TH}!;RzAqnc;>gk#GZSOW3v#<{Vt4%zl! zm0D?7mMhPd=qaZn4m{!N_J|RS76GF84x|Dv<(#zuRdne(QInyl2|NPp^<^3o(E-B} zd1N67%pa;?FP(Cv;%}6?N<+XcQV@lL>{avAR>KwjM=69l1`B$evYyJOTo9YJY)V76 z;}Eg{Q$-VZ&Cp4}?13z>1i*n&1klm(o%vo^3Ps;#lXPMYACz6LSeP3%8v&@6PZXAa zBKFO_#tE}|VPTX!5uf5RvK-0}-z@AjV?_oSSxnrhh7#t3EXEpG_&b+c;vlM?wlTwG zow>65%^LjbmDj3d@K!TUxBnoS{=$sIDS*qo>pT4HTC10R^lBl{9c%%k7X+zmSC0oM zH363u$86Hh3afeM9)-Q>+Fl1mVWcMNKZY4Qky2YSV6aI`6?R4nndKkaL>;Xg$l8i@ zwp;%3etefneeHTtH6GHUlf=|&VbQEAbO^piQ?=Bcxqv_cen*QvafUl%12y0k$Z<0bR&nrsS{ihX zM-sS?1mHeWzuO{fg0KrqNn~m)G&jy18eJ+vi=_#vQO- z?ESBtx3YithULqElXC-c1>V*xgFX>pRf~9;px{T1{M#3k$dSbK5Rh z#xGm&ayY$YmP!W>ZR$vPY#PCsB-;u_3jpD~32ve^Y53;IDU|P`U$T|pFY^vL;t*9@ zc=cgxZ_TU9awFANptubIbMjfdK3rye*n08)YV}F)d;0qqSM^>Nxzi{dOP1Z&7Ejqe zUZex5RoiqjjVvlH!fMSK*cYXVWKrojTEI~!;cY&$d`-8pr3K^4w{#;BQ|X8= zK&_43F@*rF*alq(ASzKrg^|5Mw4|{xGqLg7#j&Ln_VZicUX6$3md#aNsw3p_|1~4B z$R@n}E~S!Mxt7h9I~VyP2$KY@m;;ai9QE^zyv)5$MgN+MIH2We^t&8Gf^t%*VS$)V z`)~{^heN!XMjly55lY$p-S`WTQRE$0Fo>Ue{aMXx_$5zQh!!(F;xcw8an|UOZ^2k6 z#{w0H;B$z}a*i(u;u^uZ{bwN+CaD-ByX`!@Ty`N7#(FrnA_hqt6tg-XAEU+3#-wML zLY4oPaLQQr3pDin;;6cL;|lfq=FC_-X4@&}Q(M-p9# zJ)$vX zA>OSNse;?3H5b@Y)V}K7N4}&#V5ly^7q|+ z$?%K334eQaI5%OdK@+ygW)rsF{ulDNRd#3DDtlK!TH7B*1>0U&L*CNGw>NuyqzW&u z>f)=y%K?UuznA|A1eBdynM}L1^Jdcebv!k)p#U_%NDmRz4UNxcu3sqBss%Q!95qR(`da9CPTuM*7M}dUqA-l)};r za5s3Zs-Q!Kr9%i?QAke0&Xq=B*-83^1 zOZHSg%I(*AM-!s2|JMyigi-`V+j67vtK8b7cY6?R!k;fX$tmAA%Y=Tu%9gAYf)7wo8i<7!+1>yc5w{|XV{^Y{ z_jO~O&2>?(ESi^51Nbjx)Sv`mT$96UrMOgV%y04ZHbrMVqbZ4Nh76EbX3`dJ6cidy z03YI@*g3CD-KNAaf$1UHmRpSvvP~eZWTOQYwc6s{JgBoNV7I6ujg}S)q0wcu9TpI8 z^Tno&0(S*{n6!3OKiM`4&j@`OB3_3s&G-v6i;7BiASFaOWNg63u@>hya1o-R?A(Zj zDrVuMbYs;5f6Qh{BxwhWyuwNAoEwhxh*Lp>zsd&9Rt=p5wKO)+UE z7qfXa|Hw(6Xe-MEiyk@%A3vUkwU*AJP7|8JS;&6$nyZ-87FQ7wb@Y-V-!p)~#kmu@ zmIz>GT`&GJvsWBX3$SOKsRG#{l39_;<_OcY4)+D7l0mu5BQkoz zFY*5~u!44NV$rbS9a&xltwro~GjS<2jrD=`kCBTOWh2aaXj@?Etm0m+_DkhFY{zWW z)4An>E)Z3@pfu8UB)`&XXgi|Om7rjdj+!^}HI{Zbc<$mnI%(w_9yUba)f$&?ZPT@N zRO>o|hmK)S?ILe86SAl~$`1vjEj<*rLN+g%2ymaeh{7m z|M>Z6wH-ent@fu2*?gq+ZD<|Gu4+29BCGTl!B8Wd{C6$ND(#1MXn6S|UiJxwK1N+y zewWZ>)TIsS61ouo8#FF-2Wr-`f}zRy+xcA@DsJ;7wx3iKs*ejrrkydIYTycv@4N8O zk-9%-g)@%>o?DU?uIVU>3>cDE7?E$p`EntYR{t>Er;nk>R8M`i^+*AwC0oydU}IfpRj{b_<}9wd z+l!YA=p`{$o)9j8hUjv8uzwXEXNnSP&_es!4GYEIl-qC%;ZYvwN^yuJA$^!8*qUQ- zZ|53eZdFelLa;sTt)7*wbya>uQ=e@b5hoVW%H*DjxDx)$`U~qK|1nZ{Kt5o6*vL>+uvYQQ&?aP4qMZ=m3+I(I87oq&OnZn`d*Rv2t0b>gX{rOdY4psfp| zilJ!o$gG*tep&x0V=OC+q=_cz5d1BVK{!G?wKiB3!Vw!grXdG#h)y{YIN6DU%Xx`o z0YRb3y-M*p=H7)v%vd=F2n3cJyCYSuhoSBZRmU_cR)VsP}kIT)W(XVPnFolG&;0S6tHin@NmM zmgD7({E{Kq!-8*$rEnA`c@Ob{TX7ZDT!3JseOaF;=LG=-8;LvJ3AO=RF9DK)8B%6_ zVU(F9HaRP&8vufBW;A>g-g*kY8RpjJW!3%Rr-TVMl?XQjTPp_x_KZL_RFKvDKaJiC zrEe;tw#`>#W%&3MVRWc?4f7%@1ibRV@IZ&hz!KH!G#p}JF|Ch?zRzk9RIHlA4cIyO zRaY_pXgz-+oKB8mWgtf(`^u1j+DBe?8PI_yT)?p$NhYWT<6-_Y#do4z&8O7d z?8ks(afA3{){%x$+mym+qKtm3IK=}TgBQcYE=I^<-h#!8%9&XUr+6vGDJ0v}uI>WN zVu@3oW)wEE!KI%>Fv7Y{WA<5yV>2}&EPk{JQ8})tg!Sn24wV9HaWw;$u=(xFi0A$3 zM37rW_Z#baluL%Y663j42&?@CT9IN-oSDO*k_GY*iNTG^Hu(hG?hel9CX(8xLd|WU zb8Z>u;%!QZjfIfS&%`H<%}F{ZCo$v7RU0#EH+H|b>4Es9yR(yKtUe*!^~CfkBhvEg z{J|cD!P9fYdy>*m8=k(+pO*iOKN$bPS$XqwOD6~8_Kyxt?$#%-tWVv%+|pkMS;hhOXQV6*u}4Q54ti zTYJh!6~6(HJ1(K2;&S}IZFDFg`ee5c#mjesO+(QDZXjO18EhJemuGbjtw}*sf-~Ov z_DvD|JJzkC4I2TK`Y-2iBjRc*4F0G0U~G_idr|9hei{Du#zB9ZDT$yblG(w(ZAHmZ z_AEJAm3RUfg`D60ix&wMxS+J=`~abq3w|3K%z)*z!d_Df*%;$eg9Q4(;G0C|1dv2N zvxJK<|5C>tG+C|&;2ulmS=QefYYp1IMizpJR%vpATGWo{lMPiT2XS_Y$G=-vRdHFH z?fMr~lZun#LwuP@6V`Jze>COAI~x6+9(kP7((%9S4dwvjVl|N5NKOmUXrtG1c}!MP zO~zLTh8H|yhi1I#@gV~R%YHS^#cQnZaz|-$aky*S=q&IV`=(_F1uEHGQSCz*f&WsP zkA-HV(q&H>WQDe2cHv1gr63YOZqs>0y*ZbzAy`iCyDU=63Ei2BSz9OW69~|LK3oJn z*ytkvLvd_F*FR)E5zVjx;AUK5(0Gp-Y2%ew;!sZWvjht)zPg(DQlK&+*Xge)J<>>sR@g7CL%tv)CyREXkrU&o2Wu=wnQ#0ZyxNd$?;5C77CBxtp zFx1dL(6JPJRk}T@`r@+GLvuTtbvfFGwOtN(;9jbD*q$usNa3k2hpW60+K0nC0u5Eq zg>IlPWyfe1D0Ram^i|}+)Z7Aj==$5OI}VTgO^$vbx{yT9(pn$C{-ru zh}1-cFB96DvjFBWZ&*#_B84Q1#kd6~T2Z%vOiDNgGf{>D9()%mJvP--3aQV31EBh6 z@L+LF?8;!m8oyEd6v#&avsw^W_P|Z@&2n)7x23E>G(QN3fSmB;b9%}t&PSD*lyf`c zWYm8FB&&fkKT54j_#E6EW6iZwbqb;3p%y(M?K9S#Z!p9;if`+v!4vF{df?5WQULha z{Y9=a7-@%v7seoFcXlfPrl(l!DYamkuHsTB&BHDa*)F4|q3!$zc!xw84#{Q~=4rT0 zV^sa7`G^=u4aJWho=)OX4xYv{&s;Cni1MQ>uc5#}5hdS@X?RR6o#`R8fH)J7FnXDU zh8*5Vjjg?O?68E%waAEhxND-6;MiVFY69& zjR%ZPJbEITKij$-I9PCk-HS?q$=R)@wp&81@TDb1bxYb`*|mi7#x)w z9(iK=wvp*8My73e&L13D80`N+R_QgtxG_bc&ZqRLThJ$OL2l`9f^j`7$HSMs&<6&d$x7+o$f%+|seZxJN4D z?i(Fij#g`>xuv6nako^&-8|hNsf>&NOPBa3&}t2qkh{m1fUmkME`HKdUlp}p8wymtKJrJ@tS$G4_nv%7M7GMV!pp00^nQPM z&+N4PjLzYHiRt-0I)~51|GM`MzdSo7f0;l0H2!u*=VtQp55UKNs_EljbKHj`I({^% zw!GwXSN})&_|K*4D;Pksy+-|4%OYEc@XLa)mqi9##G1cXNNfbZKl`_#{vXU*nMn>AXPg`E&f>?Da=MibwB_L55wHcF{S{K_Be4)$U<1QIoo>( zTBH7Eej}072EfWxv8jeBz)pjfX-@E=Xftb^Q1OO9+PR1dg8_Gr0&O^FU_Y##9BcP# z`zj@|;35vK+9D~5_4%SZb7O{rSl;S>fq>Np6di7{P)r63eo}JAZuuwu+ftF{3$Nf^ zJmp!^uTkH%m)9{;8SwnQk@xwfz)bYa>h^j?*HC9`naYj@2ng^N54!p)@KPl7iv7Ea z+(5+r*jg%qla0KUdzh)TK>WjIXGZ!^PL`=O zgy`jb4!yDEW33Vl0dSm{-Jss~zF*dU z+Y(L|I0SO2UK3^k(MVanr_yCwO3dYfAmK^;Xo%4FJ@yWf&$t>l+bGCdu>u=lSp!*kRbs|WJ>1qGR9#77HOKmrW71&?`LjW)DLidXMFn!{5EwI2IHm56 zwQ7p~qn-)xJcV zvz(sVDZL@fGC-bf3X2_%JVxero`j{uX;vbTBA1b&1K)yxQ=6fY#+=k5a?gitd(tV7 zq&*RH^fi3(jEhN}8FRl(RD9jMcWac{r}|>K^gL2j$BP^VE$+O#&}KUZ2oweHt}tQ| zXuo70%p5=B*hJ}a>_yDv1}i}{d!&(DoAh3NOWVH5CF#Rg{Cw{H=ll%r{gKRTTkh>Q zcxmB;S!0F|3H-I=o^F>vzq!*tx1D@z)!#2Z{<0pEzghxGW89gY8xH=oZMHK$f`k7B zS&3yQCNyk1Kjiymm-u?JBbj#U@X*xbLJ^Wpu7_;0t1yt8LHf>guj0#;N5*D2`8B?-}@D;nJgPySH{HBkuAPXB*Hsm z>34L28N$^+>D=&10>Wjdy`FY%IQ`u4DK{OMQ+CvYg*&ji-HNC*wQNrnAKbIc5#hbf z2=Bl?^D3H4;86Lbntg_%gfFH(Rh^GOJJ8Wcp@of#%OYbDfed|5uQj?_I;az|3DPNN z-k~MWc%cAxY?)8uxV9YOJo2(#Bh&yNV^IXu+sre zV$op|kxcbRqzrV|N=DpSxaL&_89P%8^F^^+>69XnakP=`%e&Xl1BTGN_iAWFJ#k)f zu~JChWZ=!@-n@X^3a~W+=wPVc;9prtV#D?}_@Cox@IN)i;Qt@72LCfX%iNAJvzE%6 zxDMeVNiKJ6ZFo_H9h-!m*l1$+00nH=y3ZCFs{oRhZ|OARS|cYK@U6lpWr>hnosW<4sJDR3ae#-bp}~w5AtW~lcgDghdL{Bq7EfB3Q90-W%%Brn zy;+fBnaFb1;{j0}-!!ow-7_k51H_hz1tt27f*l2XW^_t|5M8yrhCwknO>Qu7f~Vm5 zXDIOzXVWM$_2aIJz$BCi*$6 z(B7OyvY9i+2#CzF{!0AATUq+)${Y_8(YYn@V{PWlncyMEh?nUzwW}dk2BfHkqwK%a zu}11TW9PLc4}naV5g^!?T{*7s|B!pHRt!#9&dF8?%K@dzuy(?%#0xGZK9M0Q6k@hj5PEi>x?Nq*GQaAR+c1#m`PgG6{}lq{_X&$ zyy*?Ec7CBeC@O4OBqFW}_tRxAW{r4-vWOgG2o!eFA>%TqQWaZK=|tm?(^Xa571?hr zXdmtEM1r%<@}S`?nJ-%a7Q2N@RhSBF%~Y*>Sp~%efM`*gG8x!eAeyMV(0mCFD!2z5 z0GK9HP>0SJz6B%#l__Df1Uv0n&KcsAsfx!_nR46trwRLeOQc}UnL|$;AfY6M;~IQM z#TE*mnbEX1Naza&c8~TUNHxEBQ-gcz_if~dj}E=5-OF?!u>7iaU1tivun&Qn z0io=XOiTE$5v_VeyEFr@kF_!L-^yWzf)ha7Glj{$6)Z}OEOOR1Pku9Uw*m&7QnrV~ zuQYl(q)5R&p@H^f!a9~>lUx%dnfmm*n)SFyGT%*g_e>Io2x~wd~Uz zeLmPbddOq*J8HIry=!#3alFz<5^)Z>eYsxA&Ovuk(tUMtxnmk0hvD@dW?46#?G`Hs4Mg7LO?&WqQQ4H<_0W zl1gT(4aqvbYEOcM5m_jM@3Idr8oSa>>SN`L)$^^cK)WD3J+NX#PiE_TagpftMzYT| zxh1;wkpAmu9J2;^Z0r^$dXP+3DeMLrywY$=q<_%% zi3y2sWhAZ3NX(eHYU9q;8$YjZIt%TvGZMF>J@!9WZ~XN1l>AzM@R^apCwqoZM4{Y} z^lgLF@@xFT2YQBgkQQdOt$*Vg zp{CWoq^qm$YuYy`;p$6PHGMfKVSCZ4rshG3+wWP>6vW@|xv%NHK?%uw64KhvSmvvT zt-tginF{}r)}deL6h%(-r({s4>;svJWwo8c_a&sa<&O?!3S0kzx2l__3;U5`OCz>1 zd4FKSOK(0>1IM7&R&vR&`EouNQmq-1+crh``iHwD78p&~Ce?iQ_?*NblW6n!f;+bO z_Ased7Z{s2ABR@Mc!VzD)+>8b&92Ac)abkUHQz1lfst1^wyu(|_Xxj7E#}C1w;Do6 zP6sgi!w7PT-CRZTpEKaIF4lE1=V?xBErB~R&{0hqVvkw=7f}pRyn5m9xZBO zJ{&a{>Z4f;X$VM#H8IYYu53~pNuAip`~}MoK1QMqEAQ4i6rI=%dFa%5xPzg(&;hH} zuojXTN((io-jhKknSScNiek|U(24DuInKpw6f%2tVhap~<46op`)pcnke69VNyd%9 z@1EO7K&~`nc>DAF7Wp1AwP%~`z}$SKpJQO!=6wcCLp@dgWbZwV0uw6e6+cNCWyw+Lnz1Z!?&%;%@<+9mcwO=5s7f$)YY)9;` zlP3pGKL{9`ae*2&I)oogk-8;~E?Zuh=Zl(1!PZBOvWz1ZlGC;;tmw~4atyHSLS@c4 zh0OqPfQ3Y0W{b*Hv_+lGw9W;U>i$76+40w9uM%oDahe)qDLey^-MeFMo76d)6s&dL z(d7Q?p4W7eG5iA<^Ae#+-g)!Rz2C!y4^|}g>)-p1)*RRM;>f!LV>GI=@de+Ex+NgC zI{v!)8GI=xNQ$2dE)Ld!k2%TD!~lOvf-qx#IzZ@c;v7x9t)k~o8DisRYVIekFuX7W z`#Pt^c!|(I-HP!N>VS0yc-vPFeQHHGT-`h#+RHlq2$C15>`%G zwK479reJ*1*I7w#WhGucdDX^*+KngN+jL`m(mAVs5j;LO{BlzIih{HaPx*uUMh3s> z8NM$meaf)((}txiYDpgv_k2a%)1yOM()-lS>614nxAc-=+=9`eJ5cp@N1wXePw-Wt z%5Bg1lCO#)p}6=(UE*(F>f4G!w=aq!rLZx1=YFbk`$AT^C9b?w6ut%XUU`UYO2UoZ zLrr1Wfk@MTYNsvh-N;4q*qgJFi)7i!MWIHxNao#>MVDTG+=mT!QakNa=V#@w1DM+b z)BnQGq28%|TIWClDLhHa(3@>5(A!Di-wY4QO`ps&kk>6^>uzDNG-iu$qEIy6KDKpn zmo5>wmh@^H+j%oJVlMbFyLpQ!c1wmO$?Hq0HCRc@`Hy}EzY+fDE^0_B$Z{`sgG;_g zv0Ig4C&Kg9j;?SB#cpUe=TPh>>eIUKMbR5ofzft_ZzQX&=shnlV-ilD-bPR-rWVd@ z3_#NE=wiCshqwJ0OOnjvEpUs=Sn&PTp;#p(n|NNAl~#5pgUQx;c$rNv@5Ey*frA|l zn&=RxG?E@vF8?HxWGusn#E>ZAh_i1bPPwDSoo|z=bQdvX&W8Km#ae|nTC}+5I~qrp zdC+YjmGJl_t|t>oj)Ue4j9OjUe6p+vpn@xRvW=Lj;n;v(T4~cot_<5v|Htm_w<1#c zXu^PVRz&XZE|rH(`MWVoMMQ)oHFhxcB36wrG-^eUB(-|+TP9l2Gljen9>?dOZ=?xGdLnFIj&0zpnrD3+kI7)*7Gp!qF~$zW>uw|+0FK01hOEy#J#db492rS zjwCP))Y3=_i`3$jy&Y_jWWAL#iV)#0YR#~to7;EJ-)Ygre-=X$=*l=W=Ab?{C_tRR z{h1wE3mQ$hTt-!4$tD=)MGdDvGwYzhF^4GYY607nW52>kHrNE%CKBR>W1SEOPFZ7E z^OymZExjPAt5{0IWP?Pm+Ijpbk)SH5Y@wKnTPb4GP}arHQdGCS z=$UGWT^ZtL(<~MDeaa}SmxyLM;t+6?zQR)729(H|D}w`Q2Fph5VeHJe;xpVF4}6Bi zS>=(c%N+Dvc)Tef?s)18mg`3)0Mz87cvta`>OnUXXF(VXF`#r@QW>E@t^ElPr47kg zl50~&fh6nb_*+ifkV}}>u~iKg5>>?;9jroRQIwiw#)4_gLuwJgJ#loatE8O;BWL5F zWd{s(SzM-Uk>?=G2Q7BS3Em487K|qxdN>PONnJxTzBq)Q-vRByVqFsiN>;bc^o^2h zg4JPHv&!KR#Ekk`hlxnZFKkc{jmu{0PAUjgis^y0d6W1eMnc`VtyZ8c8Dg>UI<5)g zVb!b&;*eE-;-}Vfv3Dp}6gyf{WO98M(-Jck%)7P$j;Y{<9I}Arkl%k47vC#2{*-E8 zGSFJ`gpw|oMBaeGNJ{))#xH1E3@DbJm;tEPrFP><_cq-IN6DO$P=gW>hU`dcc>oa_NGmdwfY5OYdv?*Pw(9&~6{% zZ}%*S)Gb9(8hW!`1zzom)ZDt+eOkM%x^j2&%i!SRQp(9Xvh3*SP>pDzU9h#fX^XJ> zPrfC$^$O9N?Y`~_D@OoE-uT5QM|++&n4K#gx;FIKNP;a>hV)u7gbzJ?ZA0;`Z**!q zi!co&kwgZyJ_&;Dqg3LP%Fvsw_;&K?@V&gIIXzQ0A{(2@PKB)l!-u&g_t%+SPohe- z8wR}r#>m4ul8x>!aI=mys4F44iKV*69o)p&i+xiJ;3cqF))NftW^Dk*$}$--nc-n{ z$2&c~>JIppa7aaPjRQ6Nd#Y{bkKqiGgm}Jdh?Eq5lyysB6k^zNTEGxTp%Nd#C}gUZ z95c{bSu*X-k`kO8^v}STh`fn**}oL^Ye(>M1iic!yDwuiM#`2x0_y!*h)+P$S#g>A zDr{gv-1*Gn?8-sg75HN{lDA?uC&-lavv{1Sx(9~y9|7Wc>0@N?#}){x-Z-L z(GU$3(MKL?NJ2SVva3e~ZdBct;wpB=tk(zH;m!SPg{ymnsp=S}-X>acE#8C7iKnPem749fWhI zx@^=fX#DNpNQE+-Yo!7YK2To&W?uqkZP!5^dH&PHqH-{HsfenbATA&>dGagUY86_U z!9dPZrmCV$2jt(?`QYXXI9aTTx*5h%$e=1EZF7d9vSihM00`FSe-{diZEG>+O|=~+ zMBqs+^W!}jS( z8iQUkFKfqtlimX!QH7FLEFqI$$TZ*-pj>^5Os=!SYD7HZ=6awWc{&8jP1u{LIVw9{ zn5mluk~y}aMaz2r1InvvAFiz7*`rB4iXBU=%H~MfR%t?%&d4YD{(TML#klo=}K0v6AdI|*9`M!ATLJdVKTXHK;c5MXwe{3KdasF z4faSr;sG-9e_AEFDriQ1>p)|++JVkJ34Pqa zosI2TM3?U8e_>X}?GQu8qnhn$0Jg(bz5?O#U$(MmxayXy(quG$Lr=EnyC-CfT-DUu z?4eyCG0R)syMEoB5!T-ahCg70b$5@hWwi@BwNanvg7=TFCI-nt61iyNU3YpYBAfdF z9(f5O_hw#y(nBRDKU4yx6F&8X>cmrop<-8lk0^C*NBdL1q;1o}e??JZm(_e-|Kf42 zqRXNMeu-EZUcB^AFU#Cpf9hBgOYs?2HKeN78gavc(>vzgLHjSo*9ySc{{@2U#+zri z-FC0cR7hxt&adh`HrMZ%yW`%#L%(s5KJpd=zN_QvonHtKG6r6}Oxq=rtDq-&rT04! z#Sv`t@EQqp_2HXQ)(2n9qJ>oNskt@=#2vs@z*X|3nQpF<1)i>wRTXo5smVtY=sC|g zB@K%;L%F*plnbt409TxR7B4xM8Tfd;cz?BcPx#o=@?Y;Nxp>O<@dDJL?b`C`RI^r8 zT4XGeFRHS(aX8cm>FPbXdtw9g%f^p?rxWuh=M3>D6 z{~AiHdTnu}A(XiFwQA8NuBWPdf8H47iCl@1Pun?g!T*g$HpkNVSwz{+(F3~7DMX-; z$~sAlF*ANLbs?39$e&=+djQXklRuCmMW^`fG=#KAMr1~5ECt0RH&o#`()e+vUKq(N zm0>H%valdL)Hwya68a>Lm zP{b8hX_#%CQv^%Q;jox#Dg-cbhCIZYPrA?AaF_xN2s$a}jaB(eVnOg9oO|u(8&S9O zga-UqR^6H(ghN0xB7yEuRwYgWmn9~p8G)&&j}F0Fgp%H|lU;?!R~qz_qMJMK3xifaUG}trwh|dHS5p@hwISuDS#Z>_ii<=~_f(VRBQqjn zGj`585YrsH11XeD$#5YON44?hcFqsmQgD>=H6>3`_)a|rnD#ByQ%?Im=qw<(twRd} z|1lpKAGzsKU)8I_|6@MFwR2KwL(~>eJ0{UEJ7+yd?3K1uE$&%vtX96`=s{trPf0b2 zw@k4k4vpA!dH}U`k~Fh?5^E6UM%EaG=n(Jy|G<32|7gvNs)nzNF}SCuiu4si(?fII zxi*k87KY;e?rhsAmVhFFj45g}y+=TRy+72wstsoOqyuwpq>WAjQyccr z&fSpNyI}`c$K_{t&e^cOq@m{@ayHbJG@SQ`+$p1QrQ*`x8ldnFrPR3zhbqgr(0Xd_ zzS^7^qAeo@p5mC;GIP^Tx+m%=$BU|3XDRH(BCFmrqoh!>O((%2Ad_TD38u-j;t(1H zi2Yj=x#nKOvV5EZVF=p^8akcm{3e>TP36c#rFUhufbqVa?Efi{Zu>Aht^Mm|#3n>4 zDvDa)9LdE~7DeQ-qG352n$6szDRVIBTu4=rWJDGnDx2>OBLg!RsXmE6x$V+Pyo<9C zto8YIxoxtAHu?XrxndN@!8E`wg%o0R;Qw!^K((1+|6PyIZ@cXy81Oif5S-_|0jC@p za2Jyt4p4(JuD(S%slgOgeBnfP8B#MXdaS(yUBtdC1Sf4p&=M@07ehXr`ZUYMnUd0G z^dW%+5l-4sQK1rf`yZJ~i7nZj$`%oe@D~P=CZD7NwnC9k5b-Kp7ZFo~9E}c`)q8T8 zH14WIyefp__p~x(ditjmeAdQfeiC$&gN7N`>2CnOY7O5k6IlqbVZPT@RwZnv*3Dwd zmvjTKCDofzB!M}w>?)#+~3goD5{?RCnHrBNe{Sf&$YV2!Rz16!aROJ!9S^^CMm z5f#T`)*zJCo9U_v=uufkY_n8v#wsZkmh*PxWs`kP;pK!&Hy?doKAW`@GtdfMn-EYB z?vO8rcl0#YTiZ-E2YC$}ci9$pYuOY%6Yrf(R%4WR(>xZ8yk~WLKv<2jAx^Y(Tx>5B z=AizH=dH8zi1jMwGOExQ9eVV;RUN7BYSX^ShxeVx&w&*o%IopP{$o87TLbY^S&w}4 zAL|hh>yd+Ve^k}l(_6oFZ_-xq-jm6ZZ^#qB?1@gbY(au1ZP$uc+MO@TZTl^|kkrk+ za^8}9-zHJ*=3+fECNJnJi2QKZp^r$l=c0%_CL8}}v>1yPWYK1A?Xc}HMm&)kTVux5 zwwbkVkb@c6(Cpr}nQE&*y!9cvZQ)+=bt(={V+~^S3wo#~10|WnO-3|P|bFW#0 zr?r=X|6DTgulF?YpXq7fU#~at|EzSRz)B#@3Oo(`J4!tb{0m|X{J)$R-!-GUYxBbR zt}Cm1hWmoNd*6#vGXiRDXh|6cX|erbVTU&Bg*d}jH>3~5QetD==;SXrrB|Fjfm7;r zOD;Abu)>-#S#z8s?ZY&|tLZm_VoDj9svyo5sFK@=D~J$YBXh}PJAiK5MxVB3V7Bj4 z1O+vhzctlclSP=Qh|&iQ3I(g^$&63MAu}q6ovF}&Sz|i-B&p%Fpk4_d)Exl|c8Xj$ zY=}^!HiYFV@FZ|8Ta%1ZEJ>qcvWHC>XiW1T#vflWg+^gE|GjO?wpV(NS^D|-^nd>P z++oLE({bjWlAEvFdeXRQoBwe3nrFI9*>Jv<&`@L$lxyB3n<_oA21*9c!#N1aj~Td5 zGy2M|K#isxU4eBGyBBjTrlPpiN)tn{*r9pVRHy6n=dG*VINZ!0C~Cl#KtKmVPBfSz zv}j5Q4MYMSDTow_^xChJ4|f#2@qyu4IY+@DM zdUf|1&PZaYa*EUZ0j10QEY@Okgrf)Dqp~;!j1Nd{Hl2^Y6Xtu-8$wH~^7Ald6zVb^ zy0{tGYJ#X2a~oZAxifH`?J>a8KHhln)%tBUoaZ1SNh&Oy=kT#Kack`OlFV;>wNado z%5kAGSu*2JkHhL&?|Kt%qz-qXg+OE$D6UK`vsS4%*{*P1cJz`!zrXTG1)FZIDcIdu z@Rl*J)iP?E=Y#M793B0oltS47v05VD`x3qDj&$LZ6j916yD4*Txcme3iJN?*yUSKB z9?HM|_um}MisaP8YSR+$H*A0%I=D5^cY681SzYF~QUs=6wbM;Y!fm*#>f;_yXmF`` zd$&`#>xt=8Mx^D}`GY+QgQw?)_n@wAc=|SfTK+TsVEp;HdGm8iCkNy9j}A@l)+ev5 zPu;xS(q9MT-mHk*)Vblil@0fA{Cf4qud15{#wX3lNE(}wc=fMWZEUJ;>JgvxR*$50 zkHv?3CZ%s1lD=X{+J;sB;OddV$9jhMC#Fw<#pFsDk34@CSxOETmXf1G)1}K~)=w>? zq&D33Xv00zY{R|&xC0~Zk^t|nDX=H`IFJiN5npl6_{je5iDi#=3eSbjWFD&EzVJs* zi;FLKJ=kIau5KGB8Rb(MU*0M3pZ)IbJofM^s&w=%G53gjOs6v z&S*R#8(#k_t0YYjrUaL;BO_uhVG9|TxSC3Kp5CVSHc?Va%F4y?e|KKfVSYB+Qx)v4 zpZbR#1=pWSN=;=nB10#6sP;WPl~q1?dJ#EK)rMlfu?id8GR6nq4b&IBZYn1;FizUS zx4ZEGmwgJw7<+PIU@c9zStLvE&!9!+reCm1jdwTer_mx$I#XIMlF!}$apEhuco-630npcHnN8bp$`+RKnUSlScJk0MkVmn$d?})& zP+>qIYDRe)SkE8Bs%HEfO)f$mN;ljzr9FT(rjrv8c^nvfh)CCA3d7~dLpOYr zenxOxyD&&Hd9wv>Q;8;4h3JLhNA|vw_lRy7Hh<>ZQW#G=s33q|osqIwI&70ZdgQR; zW@W7J547oJSATdx*)Z*jf<;DVAqbMkqE(w@@~QS!H69x3%fzGZTAd|6$&MaO-5!S# z^hq>ZERZ;M)}k;`RtsC9zxn9%qmKCvZ-AI-B}Nzbh}c;k8QWx4YwtoQfNcV9!_n5< z#ni?ohn9^oHH@dZN}^dFS@>vqd|8u*nnL6>Tn82fgG|3iuNCHdaSynVNytD~fR9#h{Ghri9$f-@Bz@gIab@wUjc-&p?IY;ZKX^~i@SD^X0RN@s zKSJJ-!FxJ2@B3+cY7bl2vnX=2KdtSQlyb9S+km?bf07?m_O}1~=`CKp0KV_sdInKI z(}!e{y@!K8$*%C(WgdlZ&EW|+CG0y>*cyh$<}G0NjFJ34Z{gUwg&B|TT8_`CTQoMW zg8#PQHQyh_>5J(-7Tk)%o`|i?bP16`u!L5ypl6Nb(iRw63RI%Ob#y5ComTR^r1;lf zUZ>t8IuIA52n^_n)4)OTilvxjp$DeT?*PJh(LYCMM*(ah2h$nO}itm6wM<%0qZu zM-u`oEIq~8!j~(M)?Ms}hx4kkKHD@b4&zdpd}1QDFTxDNRC(K=?U&%&W_p&lZE-Je zBS(dd;;iNa1{BVaMQw4j%Z&QAhp|t!|7*T*hjHoz@H`?;!peUnRF=EpNduBSZ=kdd z>+&d;&d(fe?*)(%Cs z&@m*|-=c_J7m=7dC(>@`5f@~!FA?a@SfRNWF!pSHCQ;xppg7`2^f3kHv>E8Bc+T7m zZ72ffcC-(~?u;@43_?yu<57GZjJ>Fs;b-gF7QN{SUJhN9GqvG_d1DP1L^Y!3Hp>AX z8I%BJXXSXXagdg2of%AQ1_hhk%V`$Jgj2b-8_0v4a@4UmN!Hbc~eShDSF&)P0}NUFiDJ-8~+vCchi z==14zE;yoR@-Vt{uAD@bBeGtmd(KK2BNRXmBLa4*|cQ z)$N7ONA;RiH9!;Q8u~NMyp+00>s1`a7?NsFkS6;eThj~nhHc#CiScACvcdr1Kj15U z;zi|Req|9~IJ3bavMo(??y(G4D84NkHOl)OofR#GL=otH#TezES@cf!#CBhkB z74X==S{8kunz$nDc4LArb4rR85<)hZFU73mk+vc%IkCEN#>*8%G|0!@97PF|)x=)Q zoRnDPqw{b4-r2E8O->^pR=cuPpXxfcz3j4Mfxhkgon*0mPwSS9!&Q_pL_MN5)hJHk zy*GBD#3;+6!Xd|iec$(X?8PZHLi(zn1`-dGI72Su|22v}DOQAQJp|m!$rQI~Yo=^u zf-8DU5`7`qdEFjws1%O@k8SI|xXW^I+h5tWo;kIH&MuPwCZ{%{w0PRDN)C=MIZzZ? z0o(rM_+C}M0YGb!qR7&?_?=ziKYAzBlnS=3N7C3HiC2$bwej868{esJI^;`Q*F7-< zQ0>)E(BC?3!=nP4^$b5R(OSyif7+(ntric-Val51mGPOb zivfX?)qh6!FJRN}rkLFQ<<|W>NT0bbyX#xo%m8Jb8_r<_F#{P*s#p6e`2cTZx_UGz zqz*inX|dcFajF7QrIIh&%b_wpE7o&_AA%93htE?!d=4-T0r%d(UyWwb3cbOTnlNFJ z@F2NZE)9&JcxkYMC>+8mnZ#QPHcMj?(y;e<1k__%M+ucA%p|$Nxz0=yoEoy}9KjJi zX~I4e2A+({1*dN#fiMdH&-SIh>5A3{ROK*9bB-8x0M(uXs4k8SqC;H{@7a_es+49V z!<3t3Yu>$n#szTP>J6Xwe_J_89GSjpTcM$xM1@eMhjJ1XLLSP=X>^fEdCf-$rDjZ~ z&8s_80W*-3CwI8rD3I!ByDi*5qL??N!UzfDeVWj;%!zp zdQvPQzGnMEN_?yM=_*Exh81|Ucj2lcvFl*FRfliX-90D+;4TowNYSE8qduwr^-u9p zh~C*ful*b}W?RmU*;?O3z1xkGXrJVSHUx{sTAZcE3OEZpwhWt9tW~L)%ffTk_Xh-W z1;gwM=#ADewKTHS;5K8vGE?jk@-}+VJ=+YcJq1xxSsrUCC^Vq5<_Y1rn3RzK!m_SX zLto(#l#z_jqzcRyvxz~3>>494I3r^|Mn!QzJyHCF3~N<-10s*G}=T?fq@$?W()QqzMNht7;su2LiX4q<|Ur ze*#rT;Eb)*md;_^uw-0^xVZ79CZ&fYd8ZVDFtaW~*w;ww_3kF!rOZq7rMm4PV=v8Z z~LR$XwtwO zdR~~f#J>?sr{>ATA!B(!0gE%Qro`9k`L|)?5yr!B4x_jQ=6i!Ljs_*Nz&ms~w2|t` zHj`dzLB%0BxH#8t*YeooA{aB*<)ORr@GI_KJ|ybBxQO*`;lrydO@mJwG?<2Zkn9E8 z?Z~_d79ZfXy$|XQMWFPoZ(N6`C^lnZ?xpmA3@&8zEzMdbCN0rx?L@ETZ@+TY#@yPC zIrlcLMkBWDq_?sYufAf{#?xvyo^oea={3Q)F-4)yr}U{?&?j#}Zs~7=aXl;J&MXS8 z0f{z$k?(|MzP)|Nm$Vl}oIxHC14{hB@`1lv! zFPHiTpl90-{8tp20sPkm|CN9FHn(jp5`osbW4N>m$dm9udQxiv z{w!86Fi2E5kw2a^{+Y$u^DEs9Nub_n^YD(838743{(sk$B4PQTw$JRbz2mWRC_n+6 zc|mznIe3oYyyPq5Jwo2hs@`3V_~8O<2%OrfphJbFL$I6Ofarnq=R-wZTV;7?X$%Sf z%RXWed>e?Mw&ZB7>o1~ju}hk3scjP^H78Rd(Q$`*PvkHl zmLvaG(yhNm^`!}faOeh6j-e-Y=#xU1|Jk%)af;r+tm(x(a$95bf5hJ<4O4sMG;i&~ z8@crUXKu=(e)JsMpFlDq64U>NWTd?Q^(9oaF`WC@xmZPlH`BY1si5hG9q#2;S>Nq? zVd1~LbGa{*>luXsfB&?tEoFtkKAX*FK3pONY=r;vDrlP`~nU~m!U zLo~r5e;cwAF=A?M4GuOJC(}GaZ2~ZJI4mPO+hSlWqJ-K2mp$m87h_PHjIa52F=r-Y zYAv&au=gxssuU|{Ry+b&H9Yd-`Oc+hIF{9g8#%PsYNDG9kV;9wc@JT?vDg~vBH>5L zcbk%jz|*OY>nJ9V2&tXYUzpY&nh0s_T1hT;OeZ5;f3nC3NZJ_+5UA|;>Dsc&7oH9vG(Hvb6>BzwW#%>E|psl*tcC) zQJ%S{k~Dh-+p?Q`Uqpmb=wJK4U(-6qgIZhJ-Y1aQYgyti1u%<$sN@93W{br8q=!mQ z9C~z4`$>e_oD0?_1lDo~cmNW+w2_%CK6LoLqo*?sr>SO(8_2-wgG_H~ZPu$K7o;2{ z6L5lXv%MD-mHlhLT#Hyg=|%z;3vx~+K@>txv7AbRjZO7DVVZ37)&Mt>Z7vE(M!?Pn z|J?OBhS(F|j=vf|)^H<9-A+H85rL*do43LL7BcvE0cnMI5~o&^@hcuq2aHBq5l;oA zgTa3v1EF1t9r!%fHRk`xSQD=@bdC9Ia0oKW0m*%rMQWKKn%URb@z)vfYCylSxXAlS zy=;DwpOsDxmyiCuuyk6ueER1_bxVMWws^Ubpn};$${*kj?Sr%!)m5VWt zdD@CvM;m=(OJG?+w3?UZge3b~2tJ93id|O(>4WvvAeA&&MFL>>dPrg_ieSVeEFR5T zeh4u;HA}8Mgysz%mi}<n{4Pi;60W=D}SOIJEE{LQJ<*9Y6fjKGyWKiprtNEBZtp@3$e%W6NK*TJzaPXi8lV*4Xa7tIL>gG*6Iy@jUWGVzuu3k4=0lBWYbmV#dT(8+WeW_<42HStvxyNZgLXw12MN_~{iX`L+JwGb4jf z_6(m0er-tlHh78D_=6Ah4DTSR<5h8Ijt(uCMk2YTzZBjgof{7RUnx=BT=5&U&OSb& z;Rm(Qo-^xqgO$2|Y2xhyuYI|q1im3a8m}%*EO;+`{4i1sqJj3GfQy_9)P}txs<#e#zoJBpb)Wv|l#A zR-lPPO9Mwe`Ze&3rnC%GFKa0SyIC6w)HM_iSvtGq1HaI3uG|v*Wl^vd~?Y(oW z`fLRMMm~!Wn1;Nn11W)_UMZYzpv9lDB=RrLv6O4rsmMIhyKQxfY>7Keu6 zWeU9{Lk{f+F-ykl0Z!RUNLwxr_AjQ4M0V|f^l_7m^byZ&7kBL=vfMR0yu^d$+oQro3jMNu#T>6mxz zJ)(7-Hr_}Zh>|tL&qvP9b!&VF*heR=>~H!G*>Ma(5NT8Pf0PEy`ZJLZlOZw4wZ_|y z(+-vj+5~JCp`Xem*zS_iN7)j}S&OEaZZ=@Ex~v%sQ^+P-E_#^3ZKQr}IMtzM3aJ}_ zJCsei>d4>}Iny2j%jAR#iE)uk4*bl(SKUQHQy@ttsU|HuAn-89rW}aM%}B(g7r34L=Lfqi()85VH#3@X=P#JY^KL;~qWk0AD-`tH8ZUCkd45=utD6 zH9%%$(bOo$jOu5^DjOTFJq~94!fKKA{pW8h&?a8Quz>w(d2^1|4Nt8E*x~{OZ@dPV zFTRjwaEV=BhUf9<_TE|{rCPY@`6zxJu43Bz%(3#6QJ}sW*QaX*r!p6Y3;=OxM~yLakPx?E%YQtm*_9Ege(eS(cNT0C7`csQU_W{SJ8-Z5hGbF>0v8` zCe|Gft)$4Ti*D>gt7e&rxx^t(rb*J>0YU@W3I;lQw^Z*9MG01f=K7>SHbUokRm!*l-Nz`|hv@BNX@aq$l>^Ch1%zNGVnk`pe8yc8FI zaZ3Czm-&kOjW0R=$OYRN0-_Y+_9U9~$rhwMSzWsu$oj$BG8^&R8c=$`=s50k&N zUV=?ZYsQEfaVXMsH%>u(!Wc;kQ|#}tLG#6UK_^;`C=n(fvB(fl*bM|5iqaU(I^j1Y zy7*Kf4CPcXzh?zCE?VE2YM({R^480y!5<5!uY#zcZ`Tbt`9&QEs4;iY4rxRsWcK)M$OWsce0)^Y+j(u?UEH(sN z$4hbmeqV<7BRoG^IMnLN_{u@8X?GFB|6+ z!^q5;$u!wFGIMW(f1?nPDMq1C6}03$$SNkBqI~$wKibYm>tWg_Vr5el&r_Wsxj>R~ zM5BXbeyV<`+&!#q)vWdiYPW|lN;Ft?+1CskZRpEHa29BFZ zqwU$5Z3jo!S|lb&hFVZ(u0?CeZi72>9p7+Y6vqkej*9Y3K1d03zSA+pmhN>zvl&Y| zDl{d~vlmE<(or5LJjzf}-eG`YR9Sb;sSy#K;$x2T}{^Yd($Uk^(kPPp_5s zLma`{rmT?z`%8*gKb$9TxGw~mQ$@&ze-(Wdde1o6dtu;<#7;U&O;X2WF3nO*ojF`; zQqmz-b@EbcfA&po$?)v6g=Dhpu>`@hsgt54bTvO*&|VCJ&6OfLFctfh!uoAy4HJh<>-%7ZP_U5t65|4dJH^FJi zCC~cAvy78wD%dqz1wGg`g`}g0TugAVYXflaEg;xhCdI@aO4NiZQu9ct)j(DEn&V{M z(KrtV-fL~|cl2da*wg9JF{m>Wc_?szLkorWI(h-djZ-T+fOP$ByGRkF`66;T5S|~3 zl{MUjErVQ(Q?{DbV$uAjczlVEQiC7;Ad6>vmdM&WJm{QXzZIvP9t(RV$9$~o0kS)! z5{{j5*40+(KH(>==a*-Ro~Z7{tR7+ps!}^dI>;Hv#;>c$s-|H;P(FYB(aV|&T9j;(0#+2tN4ktcbXM4qiTiJUg)JmTU) zjom{{VSuOE387tSeOh11PAvO^!SQQLi>vyw?L{@#x znr>rD3&xdi>4y6^wsb_7&ENeslvwrJ;z&a%aqDZeIOF}Uq?ec^Sxn3;Jon>(iTy>Q^^Xl6j2orX3XaQGpV|wy1(h8 z3h%aUjoqJ0KKu=pT9u(~TTKuC)n%o~U}VI-M?vt+WU;4DK(9G1`DXrbG>j$cxb?f3a5 z@A}a4BC~4^yQ1Z$u;D3ivlqyLJ3o}X9?4&PjyEqXjdNM ziV@CJJLS}&OpXdrB&THFMx(=_Q+P*hbd5ZnZ`~_qqcoVj&*cH`MCS1{k z*Em$<>FR=uG{(kk;%bU+dr>>ARtTV%;9)wZ2)UH0>2w;_A6ar`G?C=NuUj(7yPqRY z*-Ips1t`SKSUk~6JzZV7Q;Gm>;q}7HV{5J~!^W@%r_7~}87?l7A9BvixwA8-5Kh_d zF>A#GF93#S)s4enrg$$3LgOey28}Rn$|xAaYAXe+E0s|49?=rgqG?pA2|3KaL%Ucp zp(w22g;>pLIn9ySiCPYFuVJ~Vc|J~YEb$ar1NG2~e0=y6Lho%%bF2{gtw|Tvf=W}y6U^iG(5?j95r{&gvj&WtrGfd6@QouKiw{H zlN~K>N9r#M#9y3RzVn&*q_H_k=j0@2z;`60c4PN@n;rmbmYp94?m5+ozc0Oc(wpKuT3>Ouhks)q2SpvCC^p^ zo^60AXRF^;6xljBWkZEOT(Gk!GIMZB{vv<4{&RmM9d;<=HhnTr(l~i~p6yh$UaKIS zaAA)WG*|1@_H>uTj&*Df*|rHU4WG!9*N{O6!>;eMiI=-Ov+KG{e&0SYd>oKi`J%_$ zz6Z8&sb#z1(OvWzQ%3e$F_QVI;lB-yV&Jf?P{{boN6wQC1MC__6FJR#u9$I{0&OOa zgRF+sj}7}k<^>&b9AM|_=DW!U#GPgvz#qfYTO!8P%5PY%NutVd->;a7V+8DAHv+g6 zKnrj;aFzwf2SHz!7sSW0QD-LXT0ZtOKiG(uormM)yYRK=4;2U)DLHo4q&$-3)@lKV z_ThW#zKS721-G^EIFg3#$(2c~m`XNmtb6OX?-Z>Ww@SpeIC8E5;;P6h5*uF2b;otM zw_1DqqQqB*?XF)xu9fs5xeZRu*0Al(dvCKTHoMfQvTG(=lGK&sFw{#9!$j=C9k1P; zph0Tio-C-d+O_!ZW5%|N&a2Cep_qBLf$bi}p$X{n-15KxeD~4`d}A(TJVq#c65b2c z+^rU0e86jm@WqeJ&dL8D0=x}f|B%%XWj>OQ@BU2Jh_WraLz*!cmEOeH-a=_I(~Uz{ zxkluzC@?}OMq8@Zj9}1iL!99houT4J{A*j%9amHDdABugqb3K>yfFf2J$3z{@YPS!>f~7)DyF zPpld*>qbW@XZMXxl7!SONQ=10=~NruSwCi`eMC*pl=nh%AEAw%ZNbVOe4t#`w3R*d zKzYseT;9g#v9FdaJt@OLCrutr4@-=c#-wuaqzt{`O(E}$4aEdN zUU&thVpqM+tCJZdr~Heb3N?=}!%64;(_F$3`&%4)_}{IoU5BM$;T=bByVuEGpGw^@ z0!P;0&G=5$5!oOF))j(zZrB*p9~2D2Wrs23B?TBR(-XtVkXxIy5ppvM;SJWpmQoia8JYe#*)ECI_XqIuDzY^LW-{=c;uKJ;^N+!^m$rBvxfxe2 z&@i+)-yB|5;{}cX#{!K_*kHqWUsLADj(Hc~z!Ox#lG@*7e*e5jedbr5^_h59=^~J? z7q(ccghd`$jEybn=78WnW(@8CA&fO>x~uM?Zi%&tnnt>eL$DgrKgf8t3#r(vq#Mwe zd`>B-(?Vl>P82kj6ISh%sH)Q}2R0h+*rBm}fUQfjJY-o8s1b!`>qjd#+RkIwZS$f= zt)j!=1HU!SEjvOi+&L~R+y!24B)@QVBl$~9`Unp0TjKb$*ELd=Tl;aM%B{U^bSUs%u;~&QQ4Yn+cY;kr@p2$u z3Yld4S<;Q<%6VQg$x|OYd|aU8x>pOnTo#!*B9a+7*iiL zZ%MsxlMqLeIE8_%t_vF01pqzz7|KWL0agHN$#&V{ zJhH~0#U1gk%}Q%OgMVJ@T)fPpmqW1sGHVts&Q$L|pboTlk)$2#k$=`2Fl>K%QH@&{ z5r-9(pDMeqAq00XbHlJ4nwBwqbHzMrjGdZ!N*bPTdc^NK?GoW03EuubfX=C&b|lN) z9V6j5Fjd=-4bq#{f=Gm)z|5^=9$6i60&R%Ey&1Nd#r;-9${tOq>PLVtQr^9C$GrHi z8P#2z7shv8S=}?-7gpX?qZLM*Zs5H&B6yoY`xJL>6|sXY#1g4n98&U*7$4b=Vlwyj?W5@;;esH+^3ztF z?t}?Gfc8X4ths{}70W*4!#5LJcg~XO6I^PfXb(1o!s;2f;ItKt(~z9JRS3B$jR;ltQbxc4QS%yYMnKJTn8=zJ5l?V5CfOEqk)q8}m53nfyv5T%$nN_Y zRgGeXI)h|Zu8aF*F41Z>(jm}}dV)D-+r+bAE)rrtC@Mtk!T1c1xdKUY;k3>eH#d@} z7T?x!1wq>P&Sn);^c1u$4v&%|%8q}d&wuPYK;)SDZcxo)K>Om8+)|5)lPAn^Y0W5i z!~ER^3V>z3#M@pp*cvzIT|-Z1|GPPqZ(!BnC&`?yY6kO7n=%RoZAYdCC-|~m3feyK zGWB;CL&DTQ!NZQ^bT2!Si4Jxo-wQjE`oX^N0>HA*QhZy_8XuWEGOg{ODZZI_c`aUk z1jm28yz^0C)t2FrU&X~2<9`>&#V2pMB=Y<2zU18Tk(=$vhi{(+1zgtE8m7Bn3tY?ZEK*qby8hL@M)= zR-s~BkMX4LD)4drT6;P+WMDb1401gu(2Pnr3!kRo-vpfU4BMX=Doa_Wq`1ESvqU7v zvR-n5X%Lg`$r!~E(G|*)|6+swnvFwJOy<^#EeANw3>oqtz-B6z0IFTf*|1!^r?Q}P zjR0=4^7tQwLq+9lx?*37gK|W#glsnWHRu9MFJ@w^Lc0DJm5wn$)Gj)Ni!=|%cH>gg zMi;p#T7PK(KC zoKFG672K>j@{f^XPe&YejFz1I z3mpi{O!;*uPq_4Zy#(Z9mRsrq0%8A$T@X)5he0!$dz}m$=qytO$#j$pMg!TiU5f+E z3ZCZyOcmXr2}E+RJ73TGHFI#)cuTZ^_c&!QaaLM(M1)a@;Pq8kBm@%+0EqaZyrH`d zvdl%t##C}tq1uchjbukW^P&bywmv&@dh=W-DWu7OS;MhH#Q_6_37(Bx;pqgE!j6!O z_G1CA7{hGJe9>Hw1}*Bnl3bd%Ono2zGDLwP@D`7>W@6zG`-%W$m`|{R3aPUDhd2He zC1e^dPTf5`X|eOxKf}ZxM5B4(Ulg!k~?SS?;E+&Zr zV)LxeNJMQRTdm+8TU91eQ!W9q7y9*|FVUn90-84r>JuWXq=Lo%jU@U7dG)xeMpKU{ zp4uYjp%^ECCll!}a~?-O^DhtEJ-WqEvD*-Y!odxi%jAd$LtX99dlr=Gxgz=1cv>p# zs(BJ|$S4+cXlcn9+7w$nq6@@n2L5Uev3z{DAC9Pb#py?i8QpVEz zn*KE?A){zj(}(!mJqse!dJGR0bh@``{rs%ER$RPnqT${KA*LaN>3?x zHQpC3q#Lx5qOqEJS;4lM@4+N%w2-9r#6HT**lo~rxLb~Z*rJQjj^a$B!ApdPwRK`v z3fW<{4Mdw@=@&T^FWD~=gdIAxDOSuxgo?uhVK;z3%L&BMhW6&fgq**)pfhZ(?%)hW zT#JXN|K@$;Mr=X%$36mQr5Q|Te}3O0-=msp$W_)268$*^1?8tEwa6+uU7XVZ2@ zv1ugA+JPjX92GU{V85|Dj-KMTlwwOR=t{wbYQ z@sg6F+0u_{xlW6WW1@kpvSYC>ppZg5HMUgx%5|Y^Ml9tYS!T0OAB~D}iW>Hh78R z@OO(HDsUSW6=@t$Tat*1BHbw*AVNs~cdfn8K11zuP8{&N-~D_ZpNFcGwVX|EJ{c)P#Oz*b2DQ#h*xwb05g3(C?S<|y?FtQc#cswrS>%|QlO{}7oja#8zb@! z89IquNL5jDpXO<2v{u?WPdcf1PmB*e99a*7aaqY}&$5~H17 zCw&}5!I>?Dkd2!W3mBBJGpaIYJJ(A}*E8q56ucPNKOko7{TX-RN5a2AO?99~?ilAf zy5T{uSP2__bapimY{f*OdL({X3sXUfaeqb*ubh)psO?_z=Y8-|TZs_1M`)HQ-aH5o zBY1G+S#` zQ%|%8gW}LLhO|q8R+Hovhid05nyR$dLU2+*yu5{3#%ZE>Sfdj{3WE@O88&cwVt5<)Sp{WHO41yFI_pF1OV;yoiYwVH*2d1K$1d*qxf@{a#Ed0m=`RtP z;w};4Y}bvHhxV3;&?!$|tfRxQtyfp^16(`ae?iC1j znr=c#TYAFu^!ViyR&0pblhV5&W&P@c;5YrltKrX|pSojsO5PI%!H)}r&0RBIT$OaB z5ZlK;24b7~9T8je-oF{f{AUf=aTcSrKV2Dmf`MD_vj?=@*`r5w}^@z~cC~MZ@flWu->}j-0!3TswJaoKQt##JbR!oVm=~74rL>`D5zlUp9Mr zV?IF&k0xxheVefDx%Y58Xd_D3S^$`civ=Nd>A2bJ*@J-p?gH4}n%0IVgr5kD=?KfE zYZc73JFQ#cJE$nkVmfS)~mSOqg;(> zs%Jv+Et?lc{y&k7tFHRkf!6RrOMSJEKZaHW&LCWl;at7V_gug1htH%7Sak3t#uc0z z#naIt6oTSlE+wiOn(YnWPZ!gmR zYi*ROVkcYR@V)FsR=FW<-rP)Ljp4^n_99A`_LIE`DRUk1m|p!w0F9z);^pu(db+lJ zM|+X`H~p<+jmj&*Uj$MqYdmj@|HaNir`)+ii%C`NdfCS#I+6|5WC}gTQjA~$%yqBK zZSSu|&nn0B3{DZ9AT@B2b3*?)l0FIJB&r-W^Z*Xw6bhlE!K=9}kfO~XHJ#BgWrLyJ zT*K$sav0)F>&h!YvbwG`EHXZtaG2RdjWdx1pNLpv?Isgzf@Jc#2>@9NN4sRkpeS@? z#!{#)hoa|^w!kUT^p7pMz#v&V#akz=6`K<044Xn%oNQJ*T573nS+P3N6g@hNq3#1^ zoB1)&++JE<**SlTR#d9;2({qd9P&?N3j#=^=Wh+2vfqcU470aa^v$VK1SMEC0g@HB zj=5{a$&^~imtR(!io#WFK1 z!c-)yc0=YpO)Fy)_GTo!nh`&F(uxheYd0MGo20y_3W6tQg}cJye?;oC5h?5ct00(M z6g(y?ygeaxV!^Lsx)z6S0ohiORenh@W>;m*`=dewdu7+%mYs83`i)1<1TsUm%RZt7 zYNy_n_yW3UcR|tGYW(}n;y_1j*YLgQi#=;mpyQ#g;aPZjC0@#I*%$N(lw^!8-F5my zA0S|7u;i|LgpYi9Ud8q<+-15R9on`IV}dDv?~=%1*(-bdj5|>F#0}EM2vd>PA>o7U zGqN|WXLquDyf*h0CXir>K?Ks0_-jP3wK#tN*pm3?#%{coG?w1FN2$}CC-#hJ7&h>b z$!j^$6VXnX+bz&v3OJ&{adhbT*IV_?+*BDGk;)R3S?o)=jkV~ly@vce_DkFfq@9WjKHGplO#b(MZ+`pCN|t3#U?Fsyk-X6x?{5v zM6Vr_Kiz6bLcMPtfilG%_|ba50Y&rRW8C}-_+uhu;SV)vRW5}to*bGh1Qufn4h=nG z-Bo!7o4`N`RirL{&E()0cL}Fxdbi3OAV$<(@yHmN^s_hxqAet7%|t4ygM~jK4w4$j}ajt%Mvl1W{B=%M**j9$%aXw zB{Z|&mP~^=$SP==)<1tX4gi zBp6}hJ;Q^bHIxcXxnVI?Ze1U)72ZG5I#9)ufkHDoPZXWPxg00gd5TdQWgBeA=hSO) zodGfkwL(2sMG9=K*bpd*_E+yIYbrGL1SasYRA@&3J()k2KgO?@v*4}7%4eI1>Hp4u zl?X=s_a64U!<#q5I>&2pp`C23c;|JGPK&vzG$|%vkkn?o^$xlRuT_*baq5MSJd-Wcf^uY7SX6G!-uB*x_zcv^%t19M( z;?Siyv>>}~{_%klG+c`x9oY~Qdvj9kHA@0((QPd-uC(ju$TKmq$BtdP;i!9>eifVW zYFhkcbYE*-xnX~G(`fK;X$jra;@+Q?kP&>hF!)yg@RbRvJMvPO<)y5Dv>+JnpRwcY z)MaOQ#bf1d>A^5-}19tPd|<%jj#6Uxwj7o`2QFZep7fl_U+j|a@Ft+^#e~KITni5t|b+rTd~^L zC$+!0=oMNiKOTLTXrNI3@@PRcz(fZ0`rm;Fx z=o>8`W_}WJ%uxm>NgKJEjb)~nK=O!F7}Uyaqn-Cc#s(md(JVVOYQRsiTEwec>gkQRH7d!~4;H`+np)yvmzlUzpw-H;#%!Cm|Gmkciv)#&L zBv~Ay@`-E?f!(|zv3sX|l3aA#j!`fYN!d3wBPbTpO_kN;HG=<=wwxVL8qSpjrjwVS zuQpeHzw-j|9y{x;z7=a9Ya6j8zi*&7;+nj{8%+1#pPP`i{&60mA4x2c^}=9 zKo_yGYb7rQb9Y{v)0})(^5ls*%|AaYMJP0{GLYXbB>&@B!{!6o*`(QTGAp8XK)ESe zppALnPB^yoJwd$T{!>lORdKX&{s=c}26X0)`H&H26Q2V(-gL`0yZ67vRnB^(61 z-YybLDlo3O3lv0y*Aiv|&cCboB$n(<#H#oI;=y%1>Z*)eBpLtZ)c%Ip@_q{6|FSF7!EBwSuuF09U0QW>KF21AtrdZv1MJVkBm7^X%n>bjxf~W^qP9?G2(%j!g z53*+;c#v8kq;RXuS)DA{HYMhJkD)v+s;+B_QN^sKXu)t~(Ig6S$XE?xo}$%VS4GX) zFn0LS&TN8oARM(ITt4p@egA$npw4{y)u5`)yX>~u3NWOqORI6>{djbwA~xAb%KKffA+uI zdt^7c=o@uyU&{yDjrzi4NvCw&d$P2S*QyuMgN?3Z>B061-HzT>^Xf%E?e>0E^YXc$ z_GGhZ4HI&TlQWI}aY_l9mk7nS{hLXre-Q^4+*qc7a#R;KoU;9#JDAg1rwmx`G8TYc zbV#laUQ}H9z?Bwr-a!U#e1|IAipmEo&=|Q&rHcj_3)T!C<`QbX1{Bt;hjJS&P%h#N z|5xrh_WEsKV76*K*yT#c{(fhwS}?NfGIyFgp1BDIprl~s^VXc~v~vnHTZaL=#W zihV=$;ev_{VCjz^W83J}4lmo|!qUG{aUtrDW?+Cry+s9dwA|9z)auWYv{wDFqK2(K z_$TD&6*UY_6m?XFIV3^`r3*13fr_#$xo~rLcZORQc}nd)xd$3QM-W5*ndktqWN(P7 z{PwkeUo}4aOUWnmMrOH_2129Dv}K>nJ*YgK{U_mCcaL-_vXD^VH3+nrk9NBq*r9W_ z%x%BGHRNJW<=O|j9lqtpWt6_KF##gZ<{+CTWvXMd84qCMF`(xC9>472^KUURsYW@5 z35c+mri5d3RZeIALfMeqyj`-tymb~6zvUWZ7*$oCWE@vj(VFOPL4s5KxMT$JKAr=| z3MrR1>p=rdR)$6~lV!Hp#?J5zvOVo$<&}k(o#<+1M9#KJZMzMOqaDL)_31PWHuX=m zKxyZ@RZyy!U-YMw_r}Gj)2*$A34wvpI#qU6wci7zq3YpzAve`8s2E}!4Xv^@<`^b{ z!J^v7==H)nv|ySVAM{Bt(6*?xx+x;+nE?RY{2N7c~=N~29x|vgKCicRu zDKc{hVoAVdj^$O}guV(5^FmINJ*mlz#sFe&l7|>G#qKtpO4sR{x{4y(LTel-i>ox@CXb_JPX>6 zfBj3MsysWf9hR2%avvjD#QT^*zcpV)&7kQ~XVQzKR(dZ)C0wqx`H*ZHNAUpwjpXf3 z-fZ52)7rIyODoYP{7E=WF+M5%0hVqO#}`=rIOVg1Q;LQ8;^4<1!(2^6B*1EJxt$gS z!Pb#IBmrd?q75jS<7GF1wg~bR`dYjsvp|z~OFF~?P0B}=lw?%x*r~=Lst+3?ky)vL zPsQ`5sp7St8}Mo~9YRpHw0;b> zaRoPQ${_yaIrsKr15VCfdjK@x6m95l##IAYMk=CZ8~7EWGJ)kBqgk`_@9KRmVqrEi zu%&hdO6G8wa^eOtWrsuo@+x35U4Ud_&)t4HTavi55ty1&g)hD}P&?!Lj@RtMD$M_u z{Fv9w=8X1{rjLiD4|$@)t3Ll4f)O{FA|~8w`1hp!j(a5*vl+e)3vZ(j96CaZ!HoJV zXUs~Sk`Mp$E@5@#AS6{(_zd#ahfeJ8-`_6nnAD!Bc55TBiaYtjQ-6yT|3{QE`=~vZ)9Qu zY(Z6N?=amVro@04G{Y)$VEB#CxINrzehQZo&vq1&tg_ymqCR$zvF-1q$~NK$Uh~y0 ztP-=$lHoEd^lTX}qX6qSR}6PsBj{OMcZ(8UsyS#|uy^2Dn?0@?&suc(o7xab!nt`E z8+h4rN#X>)3lkwL`?S7g!ybq#?LWy}VDT$2g7UJaH?^PDIlWlSNE+W+Exo`Dw%9of ziMTqHGmIAiC%#ti;rVx%ge8ttnZ?1=ku$JnBF%v^tb^2um@`itV(cu}W10VQ5~6*0 z_x6e-6bY^Q5@IPnID2b8chdgsh;`C#_~OWQ(*DPf(n-6$?#}e^cj}}qOQ}3MX;VEG zxkqxPm3t>`6k9RubH{J+Eb;85{p`WK`hLXyd|yM7nXVA7maR%$+OTDEY$71v8+%Su z6tTpA5)!I?mX^?#+RaZct)VJ5mletWKM*CLVDHjtfaoyN+?XrF&lfog%ANUibG zYDa5a&q(MExH65g9eIz&g`AB|6K9UtP{d^>Ae0?M< zX6k$-(OG7JKw+@nug`mgckn_(47vVK#`<^z9fBV6ixn``Apln$dCWj>TNR*t8uerK zc~6$wtKI%_C}BWe=D>!dj~um*hH6RZoHaX)kwT>7ZcKM-~nmn$R7)VNJ$A6 zSzVabws#3hL?RpU@{1%vQq7XlDfUM_1*CQ{Jhh^It4lnK&B}Ta7xPZmce;jN3}}Fm zr+F$!Q1eD?^ql*8aX{_i^DlhT1Y+ER_8+tNif+F zH;8-7HuL3$F#x^hO z%%}@iOEWXoAs8DH&JGVUyId(uEA@`n(k^p2)rip{e2zmzCNHAy5iU~3;jpuy%Sgjh zoiPe1>5I7&6?%su+=XQBz%Km>=&RWVpXZ_+C45yqm!zH!5PkI6+LIPhKPmPsra_*K&|TO^^H#D}9`jbT zX_k#BG-1AfZwmZiy`jBj-cqKE=el!U=l6DZe(PK{h0VhZ?WgVW#(3jDa^A&2KrgBD zXt?294u|mww(p+i zB^W8~b@;ElpWnT9Lw4wmBiDxQ(<9f0?cDOyQ2kaJYD8gM7}ado$AxyMWVgPU5kKd5 zaiQn&@+rK$E-qA=l3jkzMWM~R(#uZ_hF0O9{=rb?uB`H*Jwn~^w=*sZjoJkp|532< zzdft<S7S zUAeBSO64wv1n0!o`^$ziz}Rt+^cL5H)G2=ewfjo-Z;P6i-AD#~X z25%)(y>|G7DVX~%IoZ?hPDqKDLW6z_7W1p-a(oXiGNTn)F-jJ)4Q<#+wHaLzrz+&I z)G`TYzyALMJCa^c)zox+$aTY{>D;RHkeOmM$6fzMM~06h$$yXN$OnD7Z)#c)(ZZ(6 z`Ld2JIqf{zHoM^TmaMNnT+vY@nNo)(No2D1!~S5zEvuV58+T&!}fS=_~ZI~ z)xmLsmnTV|uPWkvI%cPKJKRA@MlVy6JLa}mG<jvH{5z2@i^xG7BrPa-_Xpc!H!%tfiiHhVF?We9%GLS*e zh|Tfd;GAr!P!R`nvTjW+z^f90X+lgK0*eM(3j3IDYa=-)pMp{;@pqR<7jTcZ;ppU~ z+}fCELlK6qq>ZdkR7o4_yf`myr)I3lM{-e#PS2_h%&I1nJ*b>Ww!6@Id61a6Kf>fs z6P54?le0he>ZI%GM|t{1=JSqji?9TRCJqrrrfM-1+hF41;I*j3xMubnW%_aqb=$5* zh)HrWhdRe{+wAbwwza1#eHUor>pW&97)P5Yr;;_=0^bHi`G#SU#0-X!fr4qs$3N-m z$q9EuYc-{QisF8Qf}@g(NdfM!_0Oy|{2u$4esLf;M>cxP}k` z@^h^jR?J)3DGJ3pFIDOUxM0=vi;J4I;P?E21VD7Ma1l}uxo!5k^`kN$dkcupq8}vV z4+*M?YDPakdQhWRW9!O8kD&JEDpsvR6`EN`EKn;J0fhB%C8_sY5#ToB*N%dSDhK|R zzMbX`LqE3zpLow%JcJ6mYJevWv9TDBm`k^2M^;QTymScP#UD=8Qx?i&d_QfXt`-{B zQZ26Vge27_g#uB?UwU?uh*2dk)jYJF4|$>Aav=Tm?aP7Hvy&2HnAw`*VD(%Ji0r+U znae?NB2Q1}UN2RfmV}sGiLD>wW5@Z`q2-O-jO_^WX#0;GkG5zij7or{UPHIFF)Nym zgBQs!f}zF}?rGY+AiezLU?>eQpU2DWVCWq{vb-LlO99D-^#~2cKe-o$ZUJ0N4~8zq z-%jt*PKs6HkXnR@&Pf`9p42F)-3j`eDL)*U*rw=neg=)_h|yi_L)NK(2B zzeqPK4fZ3CP^)HdbN|KvCUz&I=c^ga%ccJ^oXG`O)7gQ9gkWE9lYr0-seS$O@V{zN z7WO&iJ2D#vB!1SBLMQ2ja_T+p%uz5geS!LDrmb)aqgM|(Bz;V@;iEjna!7Drj4L6B z+(htD+bx@DDWz*(_Hp&E-Z}bH?7x+w zIL*CS?JM76wa>J2$p7noM?O3})Xnq?S>MITi;0w*%a#VO%*Kkp*Z6BiwvK9rSj;t- zHgV)5(27hMaDCgaqBkvbUpfz3k^Gsve6%9;rCLt%(Te1dR^*9acxgq@KAQwHmo~wq zNYjmtLg6_UX6@{p4Z=aq4-vbJB{o6j9Z2#>Z6=)Lkq!hbvgg*&7XgBZsLi;|f{0lj zi7%f_7_#k@ljG~R45E0j?UyIV=fB1;c@2T(hMu|M1%rs82hHtWuig3hSrmimJ*-V| zccI}zqCqzln_w(h&M6WFw!L5a3<94Tl}{?=?Hpt)B368ZIcPWpd1Q?v`b~29c4F1? z^yw_^b*d!=Zj(8wC`-7I098nXZOvIY#f+g@ zvq{ai%<2GLx18|aK|Gmm(Y+wrTE5)tuq;#rj?6s}PFFln6+3e}$#$l$v^2lEtpIU} zoGH?zm>6)&lv%QC;UlGfi9fayyS9viNV}<$E#1AiVkUt&=QOrF{G3q|=`IhD0k;sy zE2UeA6~tv-VbneeLlIAKjKVM3T_$>(3UX&Yw}C@?(OeA{l5LQ>9q|(&jU=U~*BoYm zEmjImmJx?2*Uc%Is+x{))Vz`)!5bdaVB`1bn_j4Vj?orU^EG&>VV+IysoGxzN)!B7 zK9(Wvki|UJk4LAIrX!#GxDeqo_2PooNn6dD5uwEHWVc4ylFoGzkyZV4t((aCzF9(}W3M5Q_Xu-NeUr2lUPnMPLW`^y7Me&ab z(rxdu$lzhQ8;HA#lnqP&2jc%WcRA!&zWv$Q*Z&ydZ8&1UvJuS5W$^oD=MF$)^xfb5 z%NxNptehj|n{ug>8bs3^a>mn*OrL2A)ncv}IAzyi*-B1J$bq9fcjM1H*mlyqtRUIU z_w+Un0dQliPUnx~wVI)$LXU<$CoE3Lp4T14AyHoQ!6W=T^X@cguq}rBNZukG z!s3A9YfrLm-YqFJkq4IohE=XjwG&H})yQuI0aqQDIFL8g8?nnk>6oT(&NgKIN-^$` zC?@*wY)FjdJzmgoC(&nF>z^?0*E*%0Tmu`A#FCSj*(=&Ui|)?mFNQ<_t_gcB`wf{F zxYkmMfB}gwPYg%mPFj$iv4;8HFdneQ-j<+(I5yLsTNB3P5ZEzSG{K2&cK*lyD#9}r z#+m{Y9GXwYocOMmctMaEM*ILNq)3Eol#nnPG4WdG%PI7ctUx8ox8%fXV<)F&u z1B?1Q3)Bl2KQmRYZU)CD^)FvWR~a>F1f1}fMb@p!YWu}%)_88g%hUNKtB_t(YA+R* zn%G$!B8E(v6gd9~C{m2gnVBnhSrf4(+YmG#s8~nZAI7rB>C*aY)MT=(5 z0BQz>)t>hMP;QEtT3y!K!kL$TgLHDAz}6Pc)?mn(Sd8q1K(uHK$6%VX0RBMrh=JCO zEggc!kYKkfNzBeCQ#k&!>7Oi51%X}3MQasZBH%^jt*;i8nAu`5pt(hK2yCCja#KXg zKI?)7Gc(CnCU2hy$;R=0krmz=iz*Kl*H6k4tA)ARu_<QfH_@R40z zmhREH2Z@Xd5<&;^fAbexGrVF-Sy|AOZQ`p*X^t`x%};lR3+kZfU27@BcVfae>KARn zO6Fu;h)x!QV5bfM!hS?g`(iP*UBceR%OsYM51 zR|I1&C=PWyHoI_nrGf$Ap_pic9y8E&aMUvJ8g(iLnC~2L_)swlq>4SrildNsrjKU%$O6 zIW}Q>--I!J<0p?@vEhxC8(yz&`X-RDrcZpo5*YgbA!SEy%KC?3w^$gwyMK6HeCm!| z{Fn6Lj{?!!zZ*oGu>6B5p=wqwE_3OjebKGA*OMnc)vOUA)P70jyo$Gz`Q=o+e3xJD zUA|yE3`K%;X3lIbkJ`04@OdS@ud5i7b2|>LY|aK>l;dUc4?QHQ7(Mln265%avgN)D zg{O-iav9^QhJA3~;IA4jc;v+eU-d5GC?c!WSVKWPl=&XHZ2lOI!ld#Nbrl#PA+ih) z+5^Ri)S8OT4Cl)*3Fp(HaiGSCc@(A}7HW)F5E}He)3JF=LJ+O9tLgh|TW8~Oq@ji+ zjjm_>|CaZtRIBlZXOt92rP_1PdsV76`c|qPuvDr|H;Y$N%oNB|Oq-yjD@aezzci zRChMBTFU6oR?#u3p#7#hBHqR$6>AfT+Buw?-0xD_Et})dN_x#%vj(lo8Qxk*$V0e+ zyFCi9?Qcp^=3=>vL6R_7Y}YD)W|3EpuCSWRTr z9jS6M;O6`_V>O&&iAwD%5Q)_|neL1zjQ5PyDBrEYY6KM(x5ip7A3Y@YYJ^~3SWRt^ zG28<#ii*q;8?m?e330bI(lhTde{$nh|4rToXr}>C?pd%_!{M3Q;%y+8+Pbnei7YNG zo=!`Pee>VJRbU9MtpS)ESQWQX3jnAKy~L2F&8I!oLiS7Q!sx+uWCx6XixLQ!WL0#) z#p^72q)_Z_8QR(hKNUJMb;l4iAJA7}NO2I55mg>WBPT1E!YOk=onZ;dB3erO)33!d z;st>TrU})hChjv~9ZqQxs!Qt^kg=dhxY!`Q5qnv4+LZ_HIr-&d3dha3 z;KXrhzv(z(-A9vlb-C-I^0$W!>OFhe!$U8$q#ex7$dWTl*5l~;Xnx?kV$j3xCE4lN z`Q>JE_@6rSx?s$`RWWxKhbjTsDzkGcv&ydy##~tybNS;VBAa4jA6ODdMEACC<4S)v zI`Y?;*z=NN&s-8HK?AogH^p~PkKc)gZEe*{)}$rem6k9)Eq?jMD>i(wazi9ow|-Sp z%Cd7(C!Uj%_gF#jwZh=b{lga|r1nPlwnqwre<=(;ds=#VQ84E6%9!z^LdT_M=iHcG zcSBbB@LX>xX8A z(YGzB{kj89$PZ(w-W^-1gRDF|=JKh*`{jn}8FKcQQ=EBZb}GwQph8b+|-D7?doH-Yo-y6^kycR{+poQX*_d^fEEA8J+GWDH&41VcPMe1RHs` zPY+m$U(=U|tUZzxUvX<3`v6v0RFj9E&h}TI zU3%R&;z@VO3DcAQx2ji-KCmwCp1tWO{uWD=CKOlWO^VM8{@q5+V%-9dihG}>?0wqR zvHX8g;oB{fMp6R0{?+pvYOd*J#3RCYgwsJB@oIwJ0jiD?BU;Uij*?d*{7V_Kp3oD; z?(QQ_aFKU&UEI!53WdYi&84fU6+NK6BoWA%vk*xU>W-d@YP7iON;;Y;+-3%BI0Ol$ zGtHd-06!h-e4*vkXZC40xnHyqnCpHJfbce9ZvwWG6V?;3k^<9xfErgo6`$OMYprJbA|4wn7p>-@xz6CwM8l3qz&f)^NxcUH5RINM z?)mv{bxTDvXw_^&VGG$+O>BK&iz*H?el`cexw23hlNz*4Dw>QYfhoS%f^YK<;1rgP zX?;+v_}M*WCbX%SvYgBbea$hl!I2`76gRJy2gw+TrgQEo zGt4jJIhD{sLm}fLQsAkESj|fbv!gE}k+!h{uc;Mn+ud&fm0|kNTai=yKK*P^`-n%G zbN6=1pLO24Ry7hs#d^p_--0*2DumKzRm5SFYd;nyKOzphwP!w!FA|0C=ph!_3|>T) zCPX7Pz#(Hlx2>CE1(t^_%R%jfJMy)0!$Y0Q1Rws=KI+aHdwAB?8upI-k~-K8f)#*wzkXY$mQtAHl}|ux_d`X zE_gPFYu=8QJ=#7;kG7(2p@GTSt+y@;4VP3TH-Ps0x zcEgyu8yLx5usJZE8MK_6$Fwf&(Ic{v0K%AVTxym7Rz~yYVGME-Z!T_qc@g!zEInb3*!|{v0plYifUCA<=S74(dY{YhPr52}>YFo` zPa$;U-f26t&mM!8;`pC z*8iaUSi)MLfZAvL$SLr3d0zC`ZbXmm<7AI*z-NvT@Cz-abv(lS9&HBT?P*@y3L)v^ z#Ur24rKo-Vd?TH5gCT%iUnLG9Q~c%baZEKXA8D{RR~y67Zxjy6GUe8~#RZ@(Gk-H*vPX#Pw)BA zML3j=bp7GMm`z@WB$U$6OB-ePjckbK;PN+moBvzgcCC66bD8C&7`CEdZ$z1L{@URL zLPlj?p!R+90yvam2+3^G8Lno&_phyB*@&jL!i>1kY??xGtr>c<@EE}?n^8njL6Cwj9s_LDzL)ijCRfJ5#iu!Fub!ipVaIF8a-Jz$m!(P#jo zVq(<6vWUfId?0izS)?9oyKn-_+A?WNZY7=MuO{&SJb2wbkyQ-QI3|LVqacaQA$JD3 zD1MV0xfYkj>^rZ^H=m2jAx#vgC==F64!$i4*U3)dI6EE=+WjS91=Q;%RRuEsD>W{t~J*?~H;Gm186O z2T}62N1O05zK8&}&WW$y8w|CH_?&UBXf@}2#PQ?K*w#|RU7Ya@UwtdKP8G^)u~RlW zG4?kYBA2xTT_4jNq-QcZ%vnL)eQ2|&fr%C|#edbJFJxSlBH#84?+0WR)v@||DySRB z-TFp_nt?7m8vP_K6{W`a;(lKp8tg*_!3sq2zMAzAhu{rKY_l({V)cd|**kI~3RG>TK>uc8v`ih##mAB*t1y{5LJ<(tpRIkN7> zXA6#uTC$+{$hBu{md=s&X9dD@0muG3BX0QxE1Fu486A87%g&U6#HLVOJOkR9% z)4tre$)i^^eU=-yvv@^Qb8h_3yXQ6q@wdD0ZQ7O_m-cpYR_ht*@pFy_oA&EKJlMNA zv5%*|*Rt^!p+Nmnf#!{a@NawnaEYjAtJz%~+0`d*XW@#bN_5darCVroQg-WTwAju7 z8~Dt@y>sHv6UA%#dbXYqA3UmY4#n^w*&g&D+1^^!dO^-@49eU<+GVrvs}J;i$;XtW z>;r-sg#(rqGIkqrRcIt5jU7ewoRgM0pB!^j1;=!*SGmV9t zQ8>=pqb-fm4W~H_#Fj*emSjL&(?*GF;U}VymlYjB=d=ikYtl|KgJ#!K8l)PYXA`Kg zgQp8nBY`I+%{+jb0EHY_+qpgis8!5N7_sS#V-02aR;PI+wzMtY)oj>}4(2RQ|8V2* z7-R~o5|5TLNefQf!KNf-i6?Sm%*8nppY;B{hF#?P2My%7QK&2*#xF&wTG`@(OCmFc z_B1&@F`J~^Hf7Cv3xuZB_Lq-CDi83yevHLz^z9gjld!AS-vAzlGzxiuzjqc#7=+4whFgp zk=U6-pSMBTNtrE|oG)6+RZO|9nRs7wtrgp!`5wUY^0g*6jE&!t|K-!_?jXC7ve!G| zNwReh_$dGdgZMG^d@0*1Xztg7IGc6gJ?}n2qFbJI-;yDn5#m+cKq@&3*xUDjO}m-D zW-A%}f1Hs-pWVKnz5i$QvgW)^K2mK#0%H+ShFXU1{*M5DWZ0FIZE&mLB`q{$SUp!4KLnnKw_L_fmJHdb)FAG zP9SVes3KSiJmJ#w&rT60S1GqlgBsSyT%vYRmO5H<2UgTZ#wlEdVNUWOStD3X&QyG@ zP_~(Jrb{uEriMf#X2c;_jnwd|DWv$+dJ>=yK)ymHD$cUab;(b8iKFo3?xS?H2bbnQ zX)*mbZ$%VIJ4}=|N&0tEM#=a)X9c38TQsH{#i?;~Yy}`T-{ZSn8@&i63HJf|aL7?n zkB!PIF9%5~i4u}97qoy$20Cq&S{Yh8oEY} zov`opk+S=QZt;Vl}SnSvDU)>odt*G~EFfhKu~VeZ#@H`goq6 z8|_XcmJ2h(V>EzMcgtLKX(8f@HRQcKQ;z^xR3~j20P{glI*`Z}zv$RjU$M-f%{8zZ zhqx8nsL!CIunewaJ7eKx%j|BfgR%~<>=wzV^Impf&@uiZk;^Y*)rLxKnGdr#gxq+= z;G@2H=OjuMqHaaBaCHtAwWDi!;0rP3&D(z_sK{py0?H~qt_A$!bE-7!2R?}>uo$A!V>u2aiPGO}`JW!KHjDjyS! z`D10wy`w@)p@A&VDjyY$xuG)Vx@iTGs+id4dc-~k4P^G%(oe^h0uVT z#{fmePre9}!<8HUUfuLAYa*xjiQE38kZX0{nOwV7)~30WYg+&Wh+NwbfZ>l0(kAHoQ!BC)RQ_rFInL)atm+hhz&mH~dF^ zwW)=iCuSoyOJ>;^sUe~}0}3$u8Vn)V)J}%PYw15*7r#4FRh|`bN|2vWS&a5L1QQ3U z3$zMGdEUInt{ko3OVK*M7HBrMBhjKay4>&d5cFXZs08*y5^gz=H;Kw zTS;4UMoK$cw9$sEkejudir^MMthTJUn-)xEy@3k=qy<<}5-@8#f{>w;#w2s1V_-|v z(LUfh;+ykdNnj7a{4M7$98y0Ry=r#LRr_~ypG*h^-{r^t(;BZTjI);n}G}! ztmjiJ*!iNnmPWj0-cX#vlCbZBGzvCtl`I74v`aZ`W|(Tp4VXcdrOd(^YsgIfPu3*% zFc)Ko$H3j1L`cEd!N=3`lh0yYbk)liueya41d z8-3^qJ#&f!8-EZDzXc-ah0UW0UW%%aLWjGGTR4|vq_?7!3gdjtifBa1NX3!1uvA>} zU+9xMgN->mXV2-FKiVjuj2Ytg(~%dF9GE3XV_pDv1sG1$RGCP^GHRTs5IyCzUxIHd zDH-AUB};>?pvjzG;L-FRLqxTa9Gh(@r7aj`%$b*S=DUCd=|PP~f-2hqrowYDG!O-D^RmiIgE41U#hg|gdZ$-* zPAI!Bm{opRFy`pTFDN}~bmYI_FtRwX7Iq|UV@nSeM;=81Tw?5kl2g|wy;b(hBL&!A z`%wVf8~J-PizDA!4Y_*jzn7MN#Bw`*o3?xub4K1J47Bw|w$692XEQjZF`^4V>yYrl z3?jt#ruFQe4v=+i?kknx>gs}w98P{Ia2l6Z<;>&v<6jC~{&;g9Oh-yXXIvFJi}|A7 zMFS=>!xp-b--E4q(_o#%S_ptAA4)=4SZrKnM+888aafQ=hDj?@i6j-DpbZ}YcE@?XCjUnetdPOXp)$qvXYFd;xaX)eZ-K0 zTg()rjHI-$Ey}xu<~d24M5RmAh6{G#6(UU@_^?fxMgl9ge{j1PeYYOkAXONP*PHB z$BSEH^EJ_g3pkV^vh6ObD&GhM(j^>7+m?NCISxGxzY(>ss8HUd(T+E=+s@QB=0A0B z#wK)(@&yY3_G?q&P#EepC^CoH(AbAHpeW>uDqP#t_RV$>ZU4#;BZ%{U3=Ij~{`n^H zKv_VD`QC6X(PW-^_iG<;GBaA4&tXWsHIel`K77zb%0xHKV~!21;Q_UYUdHjTCM;3+ z4cbfEs~r+4aqsz}(#v1KR~zt^|U=yI#QWIB=mTe+Wy}#SxQBq(XO+*M~ zD^6*>DJm$lCf@C;Fbc||pHB6&oq;OuM?i$EP=%$bs7Kb7boe={eGL z3HY}EA)X^mPX-d&cffY!+x60Z2fegUHG66IT5;J+i7&*Zw8g-NWHtUhxHyo%E!Z>~ zb!yA|hZhTf|FlnvBkTIa^@B8W;8{s|^I>zjea6iIBc2#7+=Xc>K(syt3&!_NnX{%p zyIho3&a2s5MLf^VB^ys_H%i~!og=xd%ER9O$>CcM)DYBIFeayx(Z<}D0(T0uv5?*q z*YhU&DQ3~<+wqZk=* z_?s%qGFY}i?JNsYBM*Ce=omq9kYuN@a0sWgum+U-q{$H9ilQ1?y3j(_Ix?NfLx459 zTK-fAPFEi++_&CmJ+CS0!2EdL4owf2zLlc>wH$VM~a(6ZjTUxVs=^BFN z+m?+=@@l6&rX>s5(!WvLXD=H?ag#3;)M-j{v#6W92)i2OjH@>HEG|Dsga%dfy2+BX zqEjrI$StR_cYofahT!RP!@ECk6%IjIIXH39l8A?SBz!xzY)=?qYomR|tSRAnqdqDs zpBkPw?W5wlJ7N$oR~zVV=OiQMeS!gCKgw6JFhn{kXx3C5GSXu*b!`T?3f^&16l*#R z)srn~#u5%94M}RBIE=T?Om$WBgG9Ih00xtKPp#7impnHnU|dlfM`))LTr^4m%!ok3 zFA+tf5u7ujEBY`v)ckpy?owtTNSEr!oy;C2jteLqW!clndv|Qz z*Lu?(?D9Wt(&*L~*6^>Ka?Od&CF@xIO4~jh9{oaJfHqZ`p-HqaL^UKpn<$xZ{$0H% zCrJ(%Z1ji-9vPz0L`{=KtDO^;O`SQ0;PBZ^wTqAnCMB{+-)$=AM2pLkKHm*AYOBEX zbm?TxW4(_I3dSEI`7K*hP~wMYuvxc)Q@qeHV8eVg70^=4W;Ayy*DDH?===bB&ykmJ zqJgMqE$0x-WthyK19F*B*vp)gI0P@03jVk(A&ycB9rL|GJDW5!GDH{tGk>od{WY~{ z+T2(_*-8Eu%~!!~{KT+TYCy8a@m@KnFwwwBVTHNQf#FB`36WH0$HU~@k3 z#p#Ax%^W@zN)vvrL${wVY>$4~Hae zWleHqKE^j#!ABw3)Z8H)Qdu2rLOvKy7!b)e?V5YQgp(&ToF(Mj(d&mAf6#PVcYEAL zzN=iS_Plq>fqebll4V(${1>!w!(%BnOxtLh66ik0!>@btIrI}3iDKAHg{CkUFwzD3 zEl!w!SyxDrXfPK|OS0_o0iOtgwR7}Wk!I@ceZIQL**)+SVP|vM+`@wn?q1$l@k6?f zgf0aK_mk_$Pp%`;%14&r{6FnFvgzQh?#K-~$h9@zJ+^TQp<}Odwrf$&Cio_L=9iyu zz_Mot-j$W-x1CZ{Ny|09*G?(-;rpJE#|2TVdH_=*~-01l0s3a4?^

    V_>e5ZeV?kc!K~T~fMUaL#5Fji~tpgiY>4 zJaHt7wvZR#@2(Zf;pgF$7OIX!mx}126GK!x-x4#c@YMzifQ0~Mjlk71&T^1Z&B@*3 z%s|c>pmD6{?zd@1@&a`0VH+1iQg%o|KbN6p#LR*Q=<+S(9Vk||O< z&@c`*B2^c<;_ohHlV&;q5=JE`%eq3_bzbFU@V4F%u0o>`bj1 zAfzL;ZR?~Ym(vV2DpjGI{-A~NMMOf(`FHi+@bNRVL@N4bWL$M8f+Fq zngBlKzo?z7SRGu-r;ud?l=dsYBo%kF${Sd~g5r)yv-Sm%F{1kAqqnB>Y9;S{dWBfBi77kUD%e^T-vhZT>NIkbjynC+%jK zk;H;`n}C16PK%#Y+ckV|Txwh1s8HJOf=IuZ*nITRo(6JlK|#0#?vM$=m=jjLHSNF! zIlpH<%gxvS&2FvlUohvmEyt$BCtF%SsRP`~8dKf0l$p z&m7FIh##;d{!DtatTgryr9Z1Cj&6Z?pYVs$qn|yPS3!Ou-Z{aH)yM8Vw5i{#H%h~v_x03%BQ{meCe!}pv}04S zFwoj?Tkp}LXbp~j_r7uc8;$)4o~1NDp;$D6XDQ7u%J&eK4D5-qnJxW z)hsoNzv0VnCdSLOF3@fy7%Zb``uqQ=izkc}tX5oOb+AvT^s^mO zP|yWQpWbWT`Z2u8y6t`uvLivjTs4uRO(Nc;gXB`Y{56|N@`7=hQTm|In3H@5(Yw@f zS+q_j&?v(9J=U$yo(e?pIf!wL=ObG52PmcZJ^QHrz-}ax%CZn2o^B)?Wfh&g(=`6f zt4j(%%f#Tx$Ld081$++?SJ8YA4q8( z3N*wJ!;(K=an5R-imatqFe4Gqx=KsP_7*061ihu{%K0kN?OiF8(81B#e{Y+3qY% z%3E3xChN!>3X|5~Ul3l5mzU${eFfpquTRPA*DZWY*NMzrSKosOp7;hLk~I`_66BbPKK-kyHjMEtw8BCGYdzH!To zRx~xDBirsiaU1I)lUxkz$je1{w!x-U(kKtpm$P-$&h2^|i4-fgcX^7c+UoZ#iEI__ z{`q^CLZ#Q>(R}t#c~Lw0s(rUSd!%;9_c!_j>wkRxt7 zQ{{qXAf>Wo)?is5jWxjQgTnzQMls*BJ%KUk2mn;Eip=GcD$nda#oyl^xZ@CzirC0C zp+M$8xE>qAJ!uIR?Vu%U3E*akgdhTITNq(^-}Xt8a4?ty(-GN+ZBBXBJQ)06Pe0Po zNm>m!9{3q*(2z{)YeN9Dntme7ZV9!#k);q`Grv znw@WM<5f?>3H1HM;nI43?-)j1+$fzBg}JCGqk;U4 zt}Mp4kG>-c2X3R2&c2*a61U056Nzjv{luCXU#=ZbA7uZPahr^RrIS8CiW<~g#Epu7 zkePB-9Tn|$Du)aplZqZKDPh_3&3rht&h+Aw_+!@8q7Ko=bO4}Bcx6bUSOX8ZA%&*} z>bcNWF;sM$(Gy2<_j+S_u-_)WZIts#lysKm$+X5;nC^mFa1JJMpPjWJfdX8ds-6+K zLa6szp_vsmPuTW%i?NVqz;Fme6xW=Y7(mT5c?bn^l355rrx8sNH~JvVvgzoL62FeI~Ut8(XyyNVeQt@vu{dW}U~_;Ty+4 z%@2sl4Jq16&Rj5KE%#q>_@;XdtY(hn!cUD90JyP^ej7f~=}1jR^lChhWvk^w>$aF= zSfUA9ngO1(u5c_Ws4`pY#!oYZ;jK)yqS35@IAnC9G4<8v4ke}_C6G&p zzVVaCuGsL#$_=kqH+>UGSkouIUkO|zC!x=E%KC>;KUNsLyMK6HeCm!|{Fn6LQ0l#% z-71^2$?n>XC*0E{t^d>T^7#en<=Mf|JG-*V^Lm6XP021F)+00&|Kwg2x@A{-d3rE( zDgJhPkA}*d0`cwL(T+PN6q$Pbh?o;9L-}3)JimSFaU(+cN8wm6{M%98G(|Z7SD?9e zji_|101MYBnzL2B9BeuX2A2;_sQF-U_$77?d9_c^y?vMn`p1y)n|$c5zCGJVt{T2! z)BymAj>o!Q!9Wh}*=la$Hf0q%GMcBD{k09c$)mqE$RyAH+6%sHBlPw5lK4*qxJ&v& zX);5bGuZ|odXllQw`W#MTZe{aGWT7*l?Zme;6!{r7L|Q-q-&-i%c8O1%I3T3nP0P& zzcFNBTt^@C>0kr5Sr8(TiUK>9_PPn3K79R10G~b48x@=ihtJmW>D*qfZ3rySWh3o# z@RA#8gUX|Bag_P7Q+ewB9)#;QWGKpI&5tK){{8WBng6)lC?m|PG;q_M@(>EBdnTMr>@et|;^sSUGb+66Y~gdfS&P9@V@ ze3T^*vf{(D*v%-2(m{ny9I+7!R57!F8D_FpY?QKHvJZeGN~;Zi|AuQM8fn+(!2$?z zn59D-YMw-z(nLzuz(JsApM6dF4irS=657{*TJ?dmbzdW3N$igqLs(*pHgY4uHMp#* z?Ua+_>$eP|glZf7BJ*G4m&`1P;wnwXRviK}R%WQy%#cpS4%GLxLF}^b!#i>0cW>}4 z5H$Tz%4ty=z@!uul$D39*jT{>+xXN)6=~Ey_)-lA7b=;7=z6!lHbVx*VAw>aTcx#a zX>h@(k$cb$f8@B7OgpU%f{NZqW$jm{DeR2g_#MI#wdiN@F6Sql43d$eV3S?zBB117 zEKTI_3+n=1Bzit!TL{4M+7FO(wFd=XkTVE9klp&L{cLrbS@*8(eouC=V>wOz&>E!(C>O#!rv ziFrOwkxe13xehBlwu?tKn}}F$IYZ{P+yW`eig*{Lh_uMGB*o-YphlO`vn{!+Qb_#C zF!+CQ_tuWs^OEO%eB^YJ;jcBdABRyQxQ5+dOEGchcLAV}W z2)3S;w7#++oDYtzOvohb&FgLsc zVYf}hD0Td~up$3*HX!M!D{QF$VZBK5wyr(2^_L4gMUr`M=F=u+;3P$PnTI}pxt(AM zCB29;5@HV706_3lmt5Shgl%k3oXYReEdk*f+=irIQ)dP@88 z!;A#L5o-f457As4hguP6B|r1jO8P^!Z~U~9>q#p)#aAo2)LScgEwb5K>&UE3!#NUY zE^#Y0YpaedF)~+WC*eIOy&l4$Z6Kub9xw2OX}CxU`mxP#H&Z)24P2h0I3+AKSWj#2|K1 z_lN zCazY|AinaGo=u)xFMONyZ*cbK5Zi-ZV_BvV&}&s;z*s{vdioDD3>Z zdQT-+46Dt2ykgh*uG`&1>_NOiS$sPXtbmsp+u;zZ!-%4?;*oSA39^I^uw}pu*C27e z6gvYcQx^3zY`+Yh8wqNolFbx+ag=5P4{luHOQ(4f#7w6dLADvFiMh@12}mIH79u-M zP6mgtZpdu+XH?Q-duo$}#inT<H31pJY}=Q3tEhu-t{hibi?G2KKhc_p z7Kd<33wZ}cQ@H9zP}er)yl2!=IS|2oSe-z3X2JvYcXP_(lv$l>dFP@76B+|h(?aUy zj?=8!HOvCWYqrq(j$z@4Paamhns>Rbb!T^;Hc3m@X ze?Y&HOrvs8-G%?#{YL(-&uPJL1W6qY1T^m-s*}l9%>688(LN^@ZRM>;Oa%GH5fnjg z{4qq34c{4~&7LVJnjC!-w&VKtX@cxXFP~=X$Q|b#v#M2k z&xgMf1YBCrHNxD#eMtByK$Lk49%=gmD9@vg?Tm+ZGk9?CojtE`wxpDzm5VY}^)%5^ zV77~+vC|HNLYpOivloR%g3l-@XyQP@NXZ3w0M4^!|(~UwSeJ3cixs(;=_<~WG`uI1mM*~~UKx;FghPDN{ewwF z=26&o9#kY#mV60|G+RUM?v94q+mJjjd-!$?r;MrB2{B1g!+2`0os~!H9`D#30cv+I zB&Vv_HtXq@S_KVf*8o328Pv@76k>NCmx+#53EaH8%g0Ao?QAvLrJ2fme>PNke zXUdCJqmvFC;*(6cqH?}`=a~9A=a`ob9he<8SPV!yUL5v{)+bNlTY%Y?5fEH@_Jm(U zCV=OY!pnj{B!E`a9_YGbc{876Lm7f3l0ooDJivq3f9~yZ2n(Xdjx8^7%VI>|#6DuExQL)e6nrtC9G?2$#(JRYMv{_SC#hi-f)ZqdU}mfp zO*%jTYPyP8VW0|}4Slvb%aQYtRI}hKVCA2tSR93@tRRcJwmIZ<&dd#HI+broc}hz_ z!tB3Mv942k4N07Xa4Zc&_H0n=pv-{|33X|NUrG(i}68M`lNHfCaHZ%v*6(nj2tnsDPF2( za?v(Eq9FxNK@QB5!Gsfuc-EaXsCqBD!x{03kp?L`MQp4B-YcgbEpQTPUbYA^_jkpT z%R{{18w{JseyLqg#sE@`d1-YOuLb7%ZdndM)A^ydef1l%Lv=4cTM!&w6daiqE<8GQ zM`7x+!j$z-7X*hC1y6l5z5I$`%mu}vZs^Q5H#=u;R{5`jG5xDzPACqo>Xlu0`(Fb; zTM}qNGq(2P$e+=6J2CdT#eovM|7L7yYe8|#N!{aL?U%5oUwpsuD>i(xa>GZ}O((`C zbnh3xbKHsz|6IA@y+0)7)fNPwEDS!;KYTQLa1Tq}aaKxRO+oPf{^4CDcf26x1hm~| zxnte+S>>kBKe8Kj+27CT<=b`CXqtV%FHmZ}7LjgF#ya2Cd8J$fX5I!^iY5 zF5NXYGO*xJbrozx(p4Ife3=&CKC)YAU~+crfj;r=XLSo5NXl;AjF+c&3vI^BrG4U- zp9|gQufKlxaLQdm`S|LP@aqh`PU+ioOkbAH?HJNL<;Fc-+YaZS);_wD%GkJo%B_rT z&-k_zdk<&C@R{J~@&`=J=l6%N40S02E%xN_SEhvGuL^m4{i$d9Y|<98zT@TeZdLi*dHWFM3^KLqWu1qKZrtGG*(8GfbZAX z1u?wjLfcMCssd8(+1|TqUj5IQ^=t2kcEWLZId2iFl<7lRFe+&g`+Ep8m%4ow(NTdC zM^bD1?vxG(Pc z&G(RwbWQ^~&FvFpnEM*>KT$Bsj4d(DI7K>&W!@? zHFh%roYm;Tws#2woQ-&i?%70>D5DV)u0@MFtb3D@dUM0RWNdCf8I1_}5aO)= zkwzm929$ye_6YG2>H#$YGhc)##5KVj5+mI5*+?!JoRaw-V6ndA z-d6|*T2YKNDqyO}?h#_eUgi`92XsP=pma4Hn?TDxBoILEeB_j{d1^jFJpv%9^A-43 zgl2$bV3M(C2CWk$n^bz~Q38hW8Br)D{w_6OPBr7ycUZorNm?k(r<%WM;%>fcRIvb* zUA4?HQ&jo@ktud^tt7z&s%C7iRk4#}Jpki(!M`HW! zfEp(osLQyXN+himsI_Y|PgSsVsFX6o7Q0TEF3Rh{|3;^gmz~dQ?SoZ^wmwaIkc<3X zwVLnqan*t*6Q^9jDGb|mFGBBi88;?})?8JGWoijfp_-QHM{Pn?3|mj%0~oa3M*sI9u?XRT=sjEmjRSb8(q4q zIP!Z2WoIr5Y#lHb7^-wv!Ni?o0EkvL4XsQs&k2U6^a(Zn19S`f*KiS0mBwtQh2*|7{qKQ4&$ z#?gDahR4SNTply+KqMuG?HfN?R=S(*9UoW}|5y(`w09WZmL(x>UYsJ)i0>IwuE-14ygFvt$GczB^wH3$*+V+2uAC%=4q9L1F$4Y&@$`@FQPV_C+z_33*evI*PZb@xQJb3Znw%bx)@Eo0oFq+c?= zS0R7xB|a`?$saJaHbwI928ec|4-jhygoB?ofh<2(aJXW1PyxM(A%eY)2pFf#=1Ip< zXEc+8)xp}$O(kq?_he#JMOAW@qv+iTtIaT4$zEl=D@;RNrD7OA*gZ<1CUV$FJE+Mm zVHSXzvYzLV-$JoTR6q|s$la)Ph6&V^-qLkoxm?OePgaSLIr@pFv>>`Ri>1Ys9GrJ$f1<2M6E4V!cWsNHe{~o zgMaZkf^~Mma10}G@uXd(pw@<%Y&xG$Zk=wD@=o|{QeQ}F7U!>v7NnB_E3fR&vK953 zk29MHVd55y;i;!d5Ks=<$cd@6fVm;!5Zo&E}AwMMDGl?2r|%?MQWn%lDG)Ik}a)}`5Z&yZ;OAumf1%6d*M7J zR!_$rnm@M_CtDzEuXkzL&`l(!Ptbo=?Sl<$4|a=#fG@r4L`f5>DRFdr>yuzuuE3>1wgI* zl=Xiu2<|Hk{;PlZ-h|YN!&7??e|AR4q?@-c*|2xzhCS6yL*N+FFJVl-_{smVVnb7P zQ{UKxSNkTcc{o1YAJA=B>at-e>sJ&6R~7~z?jQa#K6N7awB>LRS(iop*;#@=8x@)+ zDYf*|%NVLbyOWf+tROtN$B4*(#>CbS%?fWlW<+EHoc7Pm3YX#K*qGS0;LXbW1lEE# zyRlE84FAk{DA4lv5s?D??eC)_*Y=s0_&K^_{~Ud`V*$oa4+X%{Rm5C;^x^LY(4HLr zLVZi{*3*w8y2!&#q`u^}Z{Unn|Jm)mDU6?TQ*rCGzDy>i-CW$dlwXGb{p;NSh6d5o z)c^E89~0iI3o>pw`K7>VY&V}XkKd1fDRBAY&3T}Vd<-E0!uVsv9uozXk}uj%NRN`~ z#FE)rv0gIvhZy*78(U@g{D1Ri!Xl7`9@g#0A5$W+YS#F6m$ zFc7dPiy}0};Vv%CsaxSV!q2w7YD^wt5)B?K>2!%32K1f#=)t>0X0bI}JDpVd*KgiAN#xD+8yuOT%<8(_*OKfVpCXD2m1UoH(?-q# zh>VmGAv|ED~~xQ{;y1w#ntVc@Y>WWzG#in6&9c7p^@;!VHT$q>Qh%UPpJl zW2NOH$Z;3$74AURy-rRys|LaxPc>w82*}b%>)K(r!i11Q*Z(nUHTyRIs$ z{Mum5tg4tBibI#;(1Pr``Ns!JfbZf*M>fR7-kcPB4e;K{V@m_$O1qAZJQEXp?AWCn zj=HDmSHNd!@slrEv7vS4hW*t|qk;F*61u0wZJCweN4jl4?na7EKV)85}eM|ykz{B7YbWVuf*+WG*6xV328 zc#=V6<;-Suwrg|X^Ge1R6=QO4$Dx(Y*+gXds3Lc7CCK4Ux?P;M2hX$3Cd4S;k!Y;; z;8iu`n6IbZ$lM;<%Ebqiwo)Ukq(jdfybE1%I4j8)Ft7(cy$XRYU~pmm`>jw|ch1|_lki?&bOT61x*gGWW> zL8=KKPb1-D2YAfR@q-V8|L$c9&Gj)hp23-7<3W}z&)eTW_tV9dw+50E55>`Q-B0yt z%eyNJe*YNF;4-#}Eb+C8-0p1?No666wc5yuhD#(;>r1B*!YeL!sc1txrYZd{PH!Gb z$+wi9M7=SkG09-xCQpdNMWW38_dC!WlI!pLY9m#tl-j|~=hT z8xIy~gJiKdXNdMv^bEKs*M$v%>#_^Is1a|=nM)u*F+Jpa=OzoIP^^kg!6s07pRyRR zZDdAnW9_MT`31odl0~+^mRxZo!XG$>dr-YMBGG2AS#Ygh}@PA7C|LPlu~7;vB?N3;h8~)$vdLr1v4SxqcTI_ z04COvWj6}6RGf{E;kp>}vg(uglk4s{)~l!|s8Y*Dm!_SSN8xBXDQ3%R@k5 z7`U^H<2Mz8$(gR+qe-6VV4?XMYFs6vam(-^+~NXB&9qHs-b(g(q?;M-IwVI4Ln5Ag zTbRt*Pg$ZunKb*Nfko;5O@FI+TjWML)_)=I(+UL|G!qJkZoc)6j^*17e<6K#;M%3# z4lhi1Q?gP*WWNCt2CI7jbVHqmsj!7Rg0?- zvak-Be2=9x9e<&*(cVMzc@VUW1;d#;053(d&zc4#o2}}AvJ$nEPC)}$Mfwp$AhR47 zsb+GUe5U1cFKkS3Lp8TqOf`{RqyIshva;w-;~1)Ik+gVJ-QXaNuyjK4%$td=Q?u^# zEt)tI#adF!Y2V7spF7O$S;nv~Ean04%bc=v4%(16K1Z9uFn2(45l8O9>eER{Gh+jG z;fqyIS0$w}rp$6`PNFUgef7el-Z zNXVeWTN=sLRSU-CR5I3^`%>Ue!6PlC_mDbN9+W;OazT*T%p}hmUWP+I?@~;#@y2a5 zXk*gq(&R-4KtBnhiE)peIio|UTq95XXiiRXbO;fEtmI;Bd{V)q{7QAvu91A18$Me^ z_9EE7oK%?!&(g6V$nUx~Mo2rC28&yNo6cY40A9|gm!BdphC4$0ojW+|zzE7rI=-fp znd3!a`MF1JtMC>@9^UOuyGij>`HoTOnw_%G?7lWNE6BrzsQRLF2~EmJ{$c|HuBEnX z3nP=odn^nsulWEp$}7`;Qewb|E)rEGe_*s+t$Y-wtfjwNAU>F-W6uym=5gjwwRY7U?65 zQrB?83$gB^2NbbiR)akel$qg#5d@3>LEQpHagb@I>AkPnpm7NGdX#>*9kLOU3%?QSkS$e_Q?5C&xnv!ia1dvU_6NjV(cI+0)Z7KM6Oifd zkdEpMWJ2`CW|mRGuWiI1{CWh-!?d9b-wyW@E7_(QW|>HMr-{lZH5BHA6NiN5fRwN5 zV+-;Nh13j>fSK=25@1(dmM+8fRBl6DcgF<5OQ~SfplmJk>@xgn(d=C&ur&h=R09%5 zTt#yU9GXAh#ZJS#Yq*H8AkjQ46x@9q8zSe3A|jQ}MAl&096AMRaR^`4--d|Lq;*;o z_MnGZh%&*O?;`h3R&i?SE&N7V$7LC6=g_^!`fu4DgvsWp2};sPvJ>)9oI)DesS2xP ztu9Q8CEXrpmA>M~*?r0uz%^*P&}c^MR@D?2;j*FXvEv!KMd3j}L#ivgja@1DXa1{{ z9m9XrWbN>-E89IgF=nGI@IcFZ-+Z1*^r72LcnOz!@t0wHzqW<+;8Ae^|BuMSK^xil zer_AJYgFLRq(eCgK_Ps{PLiFOrTgj@(iAk{4Nm>&)$=`CsfniOQC1s=lbvVV-fT)kqhI-`2 zJ{9}5BNdS;96`S=5;n(NO+IOdkylZoR>dyI~}0{f7p*yqO1ZCVJ1EF-=j6q7w_H*~+J>89BKpS^d1j-txe$EE2= z5|M-u6cO`)2>80-8!(cF1QSOza2XNxjs(&1c`q{f2qI}a1QHx21XS?F%ejJr8sB$D zoX~+06%`poab_fmsQ5tBfSL#)`G0$#Q|DB5pQ_F{@F(R-KS2Sv%md) z``gL)W+mU8m9%W)iVeQ6()t9_)~^kOejE^93&)VW^qnKqa-R%@KFbd^bCCLsf{^dZO5cRS@?+Bn550A8?Tnn# zks;qHh2@(`75Ri81#CkGi2}Bw$jzdrtw+V~chKGp^=7hbw(Pn_N=aGD*1i+E$D0z% zE74@_{7cF=qsiKdq4L%E-+)kgC3>x$+oQZ2{_X5b$_saA<*pl06uBvo)^<#4g)C(& zOIVZsc{8X2Aw%L2oDtnl)9GSAfpSLB{csjT?tCS5&$za2ApKkjv zBEVccBj;Z-`|ggBQ{Kev2WVr@5f{+*^vFL2Cl`K(Czr0~Mz8HC{IPlCtF#7q^v_G&}?Ttx^E+^BBRBZ>OJ>Iv%)k6ZUBpc?o7h5xRT9Bq~E0tE?MY|4?#aj zA44&)k@~7{Ci9W?qVF6LY}9d~?6|6m24Ox@mWf`%miY+5V{h}3)lTH5zlwy%P@}?Jw8H?d~v$wjiVdDll zZr~yGAxpxjMuXHKj{LCN*C{k+$|4#jOz8|jrmf$I(ggw~r6l5{o&*NlE}xjzE=?ns z0MBjLrEO=qX-Mt-8lrM{7kX-O1=4DWF(t2gAe1SxD9sT&3i^$a2qkzXE19E`eIkeM zBZof@`7JG&9_{D^19Ql-giFCON2CVPELbfke2b}SGk^?R&?lS9d6r*6Un-Ih{TH}_ zW5GyyX4BDFLDXeM<3#lyATij%Y9V)m&8}~B$P?AfL{{vOD9=)d^!!cjwEGCvK*nDh zVnt`t;7YKlRUTI{gA9}EdaLrAmYX86>O`N`IrkAtczI+Vz(pWV z??2CN!Naa?X+!mxMh~-6j2rpOMD&{^RVSxyq629Co&#vdQllOt{^WJA7BMHyw2~Cm z8Q?Q^yaJUgbMQOZ0hx|+Uk~ueLn;}45nEvuCwbBDjUGeqaRqu7@FIGF6k!oSt0dhPY+t_O<7Wn6BAlX~Y zxpCT|vwN~kFRa)QZ;o0fqvot*FLY^CGfXT!YP|I7qb`P}M|6L5=PuDG`!HGjXRrU+ zOd|X2nvfXBB=Wd;CXo|dOd`J%CXvAziRImT4Q~A{O5BzN!qg{Q*0u#z@SJ;5}j)^P4Rq2f=%U3cMm`o*;TO|FW_oQ*>d=LzC=5=Mi2h|9JZ?_cTa0X{_+| zZ#_MHkR2!YX7=iy1)O#Lu(zfW3FLv&HqIo;W%=3Hln-Y`UY~-2lL{EQTx==y-Uq}G zV1m~K3m0+MZpvsU4mzZ5_GNek*g|<0Xhj&X^=q!Su2DqDQs#PWbXnhbTggm6wldBJ z&f2Lbok-s>Bm+~iW7ChC+pE3jPD$LjVIrh+>6G9Me)^-SG2Zy_5p~^=O&-#8Y>0p2 zG-6>ZYhZIv|=ucbCURsjL16)(7Zk8F&?vO%qFtZsvjQ<_Br z$Zb56uv<>8#dx142@;K=@JXziRTIqA@Uo4;irdR;M_vg{xx?zORC&9 zEWT54=AszgJsNHwcoTGDiNX7O3mGSh*{n|4*8;7brG5S1*}tM=_V&A-h3e5G{(@bX zW(bO@&f;xc$gJj5aN=S4y*=WxRdf||IThvQX~1wwkXrY-O;3@=!P#Pn%wh3d)^#G% z(~(#JQcn9tqxXR+lm^6kC{wyW7V}@t_nN#xgsBs(YVvASMgo_=lp@Fse zvDP>hJe*n^<`2Wu5ZDcGdd5U1Hg?@(h>X=W%M~T-=hW>M zpbr~qxV=eCS03;V#i1tjg*BpRYfocArqGqx z*BdFu=Cf!o!WL~1oa;H~Ry}DWhvPP=0sr?83Px)00jIW`5L#2|&((zO$+_(ZmW?hd z`C(ki_eGJVV8c=p23GrroHVW^QWRO>OZdD;!Uyl(-PjYxBqt<~Jt66;aVs{ww{pX~ zHH|;|lh^f2$|!~dAirj&JyKhc}jQpR4 zk^hx})Ql&3!t-QsYws0Te39}hO4NKH+Aj4cl^rfDUoQM1^S0JBZWf)gQ)c9}Uf#oB zojoqHxYIWNgGPMc<&SqrkVXa=WoVUMeH#OY%)JXEi?#Der9eLZ?OdWZv}|l`87mnN zZT5F1nP1-p`{@EV6JQ$}PCg^tU2tW>t3UmW@G&it)E;%RJ6U&R{RgMXcn*#FO-4BB-P(H^7NL*)?J4~GyU zuzkOD9+{F2h(=&~)5ap71(YA7W&``i@jWUg;UnXUEBJl0e2-u{_aQ5hX6(F1Wm2UG zXvl?xcbvN=o~mk{DSaUa!&*g9~hwRUzI%LD&AoCsXGBt++ zOqJC*@{jn&MoD{3?RDD$PZaTF>4%U7#$lsiEyf~sAi6p zcOZ(O(kR@TqgD1fHhz^+6@-#PA{LE1{+9_}z!cKq*PFOROA9#dr7|h%#~4@L%*r z=siE!GM;wg2dhX5!|f&nvyZK+o6ut=#!iIJz-6LNN(aF^HXYvR4E+!+H{^vrM+-Gx zT4>yCk}5`Ytby!kzwNcM$D$_{r?3cSAo_*(W_!ygM{=9Q{uDVI2LN))j$KJoC@$lUzp{PiAn)oBKbMc0wxLasVdezGrnj?AF;XmBBJA z!Tz#!@@ndL;RcrhF7x7Mb3|;c?jxl_H)*U=(i>t#oT7fF4)E0>Wwft@)aV-n#HN-T z>gq0Tk>mVYH$qwA6Et`eE++=y$mos`1+Zh*)oxXe^y9i6?(Lj%(X*9yWy^^+T55)2 zBEL@PB1xbbyVR)j1)S!jMhuTHRsiTCB~2A&14K`?n5^tA!kI;0Zq>S=e?KftPCs<1Q9=XQw*Gz%(A+Fpwx({GHwJs_>MA3$ChUQy+k-av+Cicu4Pa3fV^-=n|C+H zYw$q+am^bfan$RC051AHU1{oLkZw8GD|rH+J&`;;8ty3h9)e1#`gDwoNVcw;0M%u2 z_XDP;l~_a2=9O|YvSKs(n*C28$Y}|r}ea~#FXucz>Mi_$>UD@l-wihVDCSrH>8mrHI0Kt zq~^{Kgtr2-P4D4v24>sXFRARouHmVP>1{dP%2%gC$JzJYZ>jQ(LVkCK4OnMv>q9-5 zpmGOqsG*8y8wqC1(>8L)o*%y_ulo-cCVigD{RL7VD(T7C;v5EXV6iF>^0c+_VbwRMarT)Qk3AY!of+N`v)AOotQ-8X_j2Bi@FE)-3PUH72@;w6bWn(R(Vu$7InQ;jYr8RwtpRtNct#ZLx z^%6g?s5F4ZROG|v2|za<`K~x1jtsiv3L)!qpDw9CG`)ZBGZb0W_xerN%PqafmKI!6 zaqKZ;OD|a9Ukk&Mh73ilJ!Wxj%dWt_*T3mGq51e%=tfBTNkKQqveB_Ezvn7uYhWca zA{S|nRiurGM4HINaImbYsIMNBF%Bw$F%ASxR@>>5CYDk!MZ#X9i9p?cPWIVN!6R3`|*$C%47 zSQ)`Ikk)L*6iCU)3~ZBdDKwW+;Vh!pq&1Lo#E9Y=h+g9m&M~VkhKDCM=rsYLT`)8+ zx+*6V9Yo<5;gCSsv2cLk0ySTk1hS{TwAOm;qw$D-Z-H<-SX(o2zZPJIPyUP0EGzJ@ zz>^Ms>68x=MM4tKhw& z4v%gy7jql6T6pB;^KDo>sFgVVrZ~&OzgHBM8`5-jWzH9K%yRvR=KB>b0>y(6v)&6P_YX`=a>@SKe z@+BNu?QaGi`*vK(H{BC6&@}sll^fo#X-q&9?Guvs_Dfm@Ztbm_#(ltJkDf%t+7?Ny zrR6>%xVEm@>mQ<5;Rdu`3;UDXWvlEKSc}ZaPhI~&Ae;$7Wal0JD{ljZb~Ku{`N|_h z@5`b~KYPTE`qG_a7B(IW=aF^&iy~j=rL`4^PT4b2tM+cF@qE;(p#$62-(;0`50x+f zG!RJ#lX%I-k7mm{Dz}=o9z7x(L1VRz?Y4<1rMw;4O`AtB(Ubyf|2Nriq@?AnM?aCQ zl^4NAtkuo{Yw~(uQ+}LKI*#V|CtXv1+WWUGn@niYy-=-xzpp?^#Zf_07ipr>V@lbW zpu%3a3sjS64`86-=nzWOUSY_k#FY}5srQr}V53bij2K}X+|%zt!I1{k z6Pb0CCa)a$$LZM|Xi@fRtFnWX=KkX$+`3@35ahY_$d-OtUINH>&P#etJ7&4Tv$y87 zzf%7Nunql})?@@|gHsM>68!nz`e_;CL>e${+iNRDjy4^_uD>pjO4EgwtZ|z19C;Ln z#@{bZ$vk9`1teOFA=)BQc93Im*6Z(IPVMhFm;Zv5D4xK69JwS-r20lHKWmSKbaZcR z<1QAvfL8WU*}O=@5E9}P#!Xn`0d<;j_Rp*9j7M^5nE3o z`w~VX;@=J791c-?JI^Lou!EGm7Ik5}u9AOpUM{*^WQZaut4<9(PiHZ3EPRs~l|ya` zLjsh`jzwr-1?gAEDt6RGh5H|q{y75dsmPn(w_QAumpfvxnl|>CM+}gP3S>}(%piHu5u)LRO!L^+6<+(fNj5#o zF^!u0ryeGd%#ak;AM%Z1$rXowssfTca&7F;wPf{$M3vmjoTMi)r0lHA?%cc>8dVJ- zdK*OHV}Km*je)+MFw(oAWBx*oiMk5M(_kx1<&Xp`Pb?QKFGtq`57wvzo5uM9z>l6} ze5zsE7BQ_nkyMa!|8`prpn@dcI(F&N>y9L^oL@sS%X}6kWh}{N47B_pdO5 z{NL2N%3lGxbwTkOx@6a(OLpML1{%t1muQn4|M3N>BEdPIFCURGBDQlc!w>=U!9nQbGeFV9= ztF*2?nx$=N6|__xuP-<1qzFgstA1|Qn>8mEK0o;XPTX_Z1BV(WEgC*6G4QAN?)z%| zTZ7-bwD7gPf2JJ~yi~)U0c~WwWvKCxj2qm!8kStxFL-P$K+rKNom4mtSBec}LF58c z)1Gz~sx%HQTO{1TVz)R69?+(`uJ|@Ir;9zq+63D^ABVnrlS2m}mLvNS3F|U5`Zq19 zbd)r0ickbmZ z1WV;AMvvCo0?-;qTHy|>+|fu6&EMHcA9}$cMaR|_i9}^W4JZ|>;hcc|SNYUPy6NzA zJjP;M;#6IyJ9w(7`-6m;GeKb$WEjxT-(7trjkOKG8gf5CYH5@hE%XzJ7g|1{C{xdd z1cmB7F+&YU-N4U^5UM#a2%krG105L+yDgEUA;pbD2!30h8GyEtwn?Ug;DAtE(@ED1 zwjKce&uUuIArM5tXSeMq?P>HG`aS5XSzjz=!$12l%MOx!UhO?tn(EGd@q4tEbPNA6 zPEe;e9N6o&TF5zUZw69RWn6sbigOG)09CvNnS}_b-BS~cjjNU#V_cy{U~KF@Kk<;; zCE9B4>s5rj`DCWbL)#}!ab2;Bu2oYT@?2}fe zEX3Myt2WPfqAE2y0Cz0lwn1Dq!+KU7X7o#{%Xr;uAe2Obq1H#JdniW^?_p_D7sOnQ zqL*do$iD+S%T9HU=iHt+fS#nG<7-QmBK>%y$7pqDz;o~pH+arBVdA#BmbTVY&}zH; z&(>-?^@*1Bd2x8SvGI7g6T!n}e?--7_)qh2Q)Zq?&9}=X4_79evcc`YOjf!$y2ftGi9Y)uN7D9xsAArk!fYg(r_=Ar z*FauYf701KZSCMDk_eKYXpEkRPO_!JXJgKIpCOPdXdHY*Ll??YwdM<4b^im8@eYOSQ#5?A!y zi{(kpo99RVkMC&l8=*b^nN;y^}u`-PEHaEH#j^SQ z+D3{773$~e5wP<`Rt_Pm$AzoGy&76DSMV0FR%PNlbR5TblOkT(Rhf4y8DrW*Di1f# zp0aVYEoxSi(LoHsMW2WUFQjIv0PL`_XO$+ZN_xZJ3AqUM+f<;dVE`IoieWY5T&&P= zfWlKR9`T}0)CbNsA|kSdQ7-_qC!K*s+4!fpnyPHWHIDeC{mQ3kdooa9IK;v1n$JvW zEVVM=k=9PMhKU`5RX!I*n@AHq|FM~GX9mmh?AR!!Nn>bv!7fz8Xu|%u%0w7qg!}QJ z602p*&xplgys#P(18i8I=@5}yG6JyhHrs02_$Bg2IW87ZhyC$*gCHquECmSdQ6+%(u?@Gzci)8$a|sVU6YEa?qb$R*aK(+{})nFm!@XPk2w zbx9V}DN3DUp5M1{YKvnDNuS2#aexWAn}uEmFzM%0RjRzHe+L9MtQBfl)vZZ1GtuA- zu|dbt>j=LR!)+4~_`Bxw^E#66?Vq^$%A?Yk7Nk!qNXvaO5IVmgG(0DK%2DZk^3!*Y zKDvC+F@tL>a!N}=zVoVlXBL%z2w7zL;M!15=@lX0QEM(LIpV^|GcXpZ_CI^_xRSPU zC5MV4zw;$Ln36EB_{0|v72jDrC7vHiMI1kp%gK*q5t)&g{YYd%n>6_+%g82CX*0LPU$;}@Lyh+r{=bkWn>;a{;8lX4+U+L1rry9AIWmjIy?Aw zO=E*lO9tNzHJ(HY&xwzH+%c%na3;(inp*xCb7+%B3|u;b4?Q=vzV3@(c55vp!YPQh zY!t`4<20+kn}=;Apj;f=NbXa1(;NO7T!b~WYHV#4+jTs`?~&13WA!?dHN!Eg$Vc~5 z4Iag>u{7DK%*ZvB)=8iISe}iiV#Ry6yiUr^SRAKu9Ugf87$!mFCnbu$Z;a+jdfx`= zmNaP*VUJZ+ru;|NkQ#)_*odjr&`e{ewi8mZc}okH z@A2qs^u`MW13KkM(gaWcFWv1T$%OZDW+&aml8^$HEiFvlC*2~or+?-JX&5Q|7wFfV zgKt}1NkwY~r6*g+>qGH=h02@TWF<;m9Wb$!2;NqeDnt`jNEDM?lBvWPYB)g7Ap#<{7a95X5WJkTlECrWs_jkk%RBPoaifa2D zj{$8JZC|A(<(i3PYY2>Z0e(OEAW)snFI zAh*oesx5oOK%1~@$QGf_Y>3;dXkU&{(`;$#PnIM$$H>!!-&zzZu$Br6Vd0CYZjw&d zsykGJ_?lcNJjYks;9F0U6Hcgb*Mi$lvj*REE9zv^PW)!Cp+(3nhgJ=hBgTlWxmxw` z7Xt9Am<{v4*&72--1dwu>Fo%F)WFawQg$Yj*cjF5pzBq+IB*af{g2}OTH*0$_+xec zi(k(vYxWLx9UwpjmyU-Y0sgKioT|o9cAs$6Vyb_14)F9&sHhRzS(!0HbcnrW#bQm> z=O_0Mp9~PDAeK4WMR6|$e_mv;z3pQG)hou5HgM_{ifc8L1%kg*?*aCP82{8B?BW`1 zTJO}~$^{Wyl8C8kvv+1)6hyO&u=Ss*r8E(*piakQ7>Dq2y}np)VL&327tz>>3_Ce< zD?ILvc`>=9vPFrf!zn!dqwh^mqM~fmAC0*);|^=5o7CLv)ug7?%i4tomYy$}Z>KDc zUe?tFN7}fM@>s5`ZImOwNCHH!c3`;jge#GNwvLHSrn{I@bPh_KJ+bd4v){|4E|Rw4 zjGnc(BA_70v1l?)8*_2Ge6#AlYF-hj-;*$o}IZ&5%gW5T+y5czLDpj2tpNHFqxR?6%)>JE$wx47w9(ccHm60krAf z04*=RuRk|;_VD#b_qCSHUw41V!!&aI>@~|lR|%P;!?00VLyn*Qd~kA?h|qP!qNoBA zY@&jAeK2!rMD&a^!a1rG+i_i6%QtCKY~lob3IJkTtlRoYE5sjS^#D-f1tseb3Ml&5 zu0fZXmOq`?v1KpOk#-2hPqI8jw)=;cE@2Nd#>fI(*wLFH)>gLEOGQ)YR$V?)K#U*{y}tY5?E>JbaA<*v2v-Iio%4ZdvLbr{jJ zV+V|0hwX(^1`QVu!Fo_^>?#h{m?1QYqUw^~??B{r)p#lLa!Ul)qiUmh?#Om(*cyE; z>peS>G<;o6O3usR^8b;b)5!4uvur_5{jgBj<&hlhc_jxkCwu8JzU(F_Fzr-L^o|FF z{aV%A)cKK8k6@ckBQ_LI@vQ%Tgf(N)7f|m>g?iDq^xaMAdyW;$7QaM7lCEeB#?xD%w^25PK<$lhe9F+BDg9R{A(#GSC1iX6*G5((P(of!UB`m3 zBv~#z|AT1EHu-h}GX7UrH>F#&)lyOOwr^_r_e|OC%p2H;Y)Ia_<$j{A!m!jJZ;Y&7 zZwvSTRDM4;yVuPt=N(|xrH5-|(>QQafmoHJmLm@PXd3fq3eRczqO5dlD$SHwlLCYo ziVlaaAGVu7O)Rccs13Uzw-#_f$WW2~Wa^roC%>waHAVq9*o4r~Vw-oOD~tPz+!26W zL|U)_sWE~!A~h-6qG=mWiB~!SPcT-6M2B12qHq^D3Ofgr2Q@-S| z&GGCAXf2@xs>VoDM)?o7A{Crc+=3Y~(WP7pe@RC2GTH5hk*RC4@kZ(B7TA-Yyr~^@&%Eag?s0NJ=J)iqUUeuZ(`(CKi25$ko#h}gcWdzjVrQ^TG z!0*f4rBThflWF6m35RB{co0pS9`$-mv?ipY(skMZhep#+GIFFeBbf$Xllw)3@~(CK zu{GOS3MqF8quLv}0O%a^_&4jdQDG|-SI}xqr^J^+F2~{&yZETZh|1$wgW`aT^C#)Q zqCi0EHu?8OucO3#i|yAf(S@ju^JTI+fO=6FHYIdsLne!(T{)8SlhnxKW61T#bn94V z9X|PB%d&=c(n2;posc}XfAXpQlQQ5RnXzg^zeSA?BqZOPoqY4kBg4IpN}n`3Ew?rh zI-wwRTu%7w3d=8pO0sNl?d+V={|x!IRQfh{ z8~x7`LK+&gEjYIqJoE>*{+vR{ljokhr+(V0Ijw(xz1sI59X@==#ieH*OUUW%e!ceg zX94jIx3J|PHVro8?#g$=+BlXnrL^_=GWKlts=~Fd-2-h zeM?GOnFD(wg|SE5(vrjc{)g(MBgwG;#e;)`Ndp(gaVBXXj;eOv*rAoI8l3Zn|F?qr znJ=iHH6!1r_RL;z?fWXs$s!iDdSHNyHw2fl-cWPHFD zTlMBU9Rf%B1t*OG9J5*|h!}`qs>V{-o6tB7a2cD8TN{92H!SksrE#aX$m&4WgBYLI zsaa|5eT&%Xe->V5(Mx#uZ;303WVmelDh%^@d03rVaPu@}9NCXnSso;b&QNxB{yqE) zUr^<2_gXvqZaRdnQYi-#JxkUiWbdCBmspEmv6h)poMkYNtl1@PB%zvIeH{U~qS7J@ z(7hr7T~X;Jgh(LU&iG|klShlTKDfwgnEB^jQn9%=#muFndu-YlWgJO<_k}8=zRvov z)pV$+^k(cQSn^c5+Yn&lkciDuMltXp`PmzjgLX-!OE^Yd=bBte99nV_c1uq$6+&xp zBaxcD?7@)G#-ZzR$}ZQ9O?0lIa~My1V2)qMOPaU7vl3tURqQCOXrx_sJ8vefO?Wvo zkv+W>TCgWmt<=v)0ZJr?SaBDh?-H)A0F!ql;6>KsIxxyOZycdv)a zVjtHWI$|8`7LfZaz^)fCV?Gr@6H$7%A6j+R7=24oxYik4&Mb)*DDl*Shja_6Mk_K$j;AU!jwZwk(6pLrfZxXJ2HgS5A~mvF38IDb7CPjW`;STkz`+ zy?lRsjmULdJT)TgRt1}XJpAm2!R15OzZ3{vSP&YM6V8XJe}4MX{IvDY2STS8gid=u ztMuxS@1mmeZYX7&GkEBnoYG%~d;_X{#}}2ahN=JEfAAl<(BF*ewf3UOQ@(`PQxaaT z_7~&*kK;;O150;J=##ee{PaoZr{z8o2)&aZdUHVdA|SuqwDpe%Lch-sy>w=FX+g+$ zWux&F z+5M+LF*ynU-x|zmJ@$meWd$o58_<$%kMNSr#LHLuC1$`@axx0plHq=`x@qqpB?Gi{ z_`sc_lI?$|mOm-nAkP~Jk!6nv3aAFQJ<@}rnHyl+c+yB&6Xrp#r4TalP)BZ@Ngg5P zXJ1o3oGZHe6bzhHz%s#$`8}|}`xe3<7ZV2^;!f8F6o%%nY%0>P8NJ~I z`eVo=Q#Lm7W8ZgMH?zo5HPDs-QN5=`8G$_kWhA=UvFRZ3kMP7)8$rQ zY`h!~q590rX7EXzHgkTg9BMq}=f450p8BzeED|TZ_We7;v2fWKLBXXpU2Cy&8HEIn zp>#WLX8UdTw8}o)8%O)Z$W^5bNJ0V=|7Fd)BaQmW$sN_L1pq?nI33z(`!+S&uBrO+ zXpv%HMSdaoGN+lMO#dIWc*m`D2-s_Nx86mi~2@joE)ww{IsBSO1R>>Pm8!G^%n0Y+q{MIPGV$Y5a?R&VSX~iHU=Cb z_2rpR$4_Rhm@(zZpDg^!?OUmeRb-gPmLi^(4JhjpJcfyBqkw4LD&BqNMb!4?2r5v_ z=AD*=%v=QQ)L(mR56;Ds^Z$^P&@kG!UlGj!{tkHT65b*d(+*%y{0^`daP$)VSFTBNr94{G}cjf(eq+ zPwG%vN&B|p6w-tl)6jg+l${VFJJuxXn89v3(w@Qblo_B=&erBr;BqejI7wn?3x@q7 z0N3YRY_b8S%NfkwmOaBZ06k!645Ujz71jovA~W8!HW;H}@Mv+$t^gN*Dmu1D!y@CY zas+ySMN4grs}$0?M#Jz&F_O)Cp)f8CQm9Rz%++k*rW{c#_#iunC@e+P{4^vTz>z6k znWY4a!X$GyxG>H7E`Y8RBC@DIrG;OtkDL+fQqUGVVeOE0w{AdZToLKnv6?REky5Ns zPqdd~4@={ku1-T8W;F>nJva-8Z@>Lntzn{yQopMpBo{usE(iiQLK>I2F3uR6H)s`e z8okUXQFd5!)AUIKJ-c&l_JXNc{uYa*Fc`vvoJC?zBU={7bQ6=VNIa$_esb=K>OHmy z5`B0K?FiOodKPEH*B?S|<6~a5iPa!v{jX9N=KTyd9xx?ws7l47qHh=G#VN58@Hc+1M1a zpOQje-@X4owK?03O>@_@53KxW*g_h_kSqTswvb=^L@^|2wzPW%wPx<2J^9wJy54m= za8_j6h^%?U?Ob+3TKnaN%c2?g@_QCOj!Ihi$|Ivj%5+QZ75J4uN5$?{c zPWNo4_R`M?A9Lfdt;{wZ{ZI)U-~fn@tVudXXa##&rWoI&%Nu=+3=kMs@ywNnDi*WE z(8Qae$Baa5@Cu2C*ods6_`vx|>ODo;F^CPCY~@n2js%E52x83U(a3=jGiq3Wg(V4Z zWRR4`x{_4Xbwy@Rb~+%OM&sAsr>dgvNOnGH�=%OC%@D1#tVKOH>K0s4l7BceZzS z|JU3NAxRYC?FV_wdt2j?Yr7xAw{71yj=9a^6D@lF3Vz=#-;*@+A?qHmwMyG^AH|`Z z`t3>BgixThdmrpQi!D;1i(K+j_w2Et8JMQ#;5l;iU3#-^v#KvVdb8b2yID+cw$EoL z^vbB|)igJu*Rq-cus}h~{rVYn7daxiLuEE?#cM+4rGf*!5~@Q;aTkl~X?O7mLj)Nq zsQup&Ik0#$ZoH5uf?)f?FTW9e9a!>P+TDS>3MxUEo%~5;MJJ6mezRE*rpAEMTz`@22{lyCd24aZ8i@U3RYULtp_{fy zXGpL_{Ou0TewFbZcW#UcgfxYKQ-+8b*28gxHDk7ch!T(?_Ns+uY=G;)@%b-qnN!wr zo405Wy)T1av@gc|Eg?=bqA$yG1m9QvyINBmQyjeqGK~OQ<2_4`?Yz$%~-~HE;2oc zFqNCKOZ^4(l9b2tp4x(fzj^pe6ma?8)<9hp+`&G~b)Lm%abb_@tlO-|pL zo4yo;+wTIQ@PP1n$>~eaOP_RJukyn5!L_#y9(r3&>8Ow|ztVSJVfh{qZokPX%>{*Z z-G#)q{YJ2D3;bIL3IZ;W_)qTPws8b{v>llfp4Melt8TMBx=gm&X5Q^X@NV;kE6IU= zNo9|A4bMtUZyQ=zzWlR5q>nEl&oYjj_rfgM3)-X2HuZuod!j3Md0q~8|EVW%DOk7j za@wxZAPz)TJP=2;mlufpe&!!L0GkBww(7acv9Y$1U*#V@G;b)icyhukjuhmfUoPL< zaM@9M!WLUuHHpT#{y8NYZNdtI(mRNA9jW_e|rJ)RiGQ>sWTizWAmj zyW*6%fiY|O79(U8q;ZJ=zn6#5cAHQ@TYq8ezcwIn^|e`PWydqs*E$<7v*{&KSO!>V zhk1Z?z}<-R@E_ken*)k~$q2e4>E%`4K9Z{o$<4o@^cag1TA~d9`-v^|$0-@3HdW5{ z(*vG)jC7AIxU0{FS)9}KJ-2@Gd(+>Wobd}#bZJdbT9n;Y7xDYo=-1NT8k#UeGS-&>CNdf8Pgv7{?dawdasBTs{L#KqR0Z%91` z*fp&79wCO5xXf@K4smm3=dR^Ezk;1eHj{?54T8ybS>8rQB(VCYZMxOgm@R()SM~{u z-JH}mNcuEn#sy^X@E{=%u-IFm_Dd`gBnfkmAu8DHzYxV$+=!|jR{y~zmlCPdNv7jK8-_xfcyI~AlB zo!06me*gn8%vIx)i=))DE!cOfFvoJkvM?bDPWx5l!;?gOEbFVHM~Nm-4_Y}+QDvFHQAmSNDhXN@xU{i^tz{g0819e zT%@Kd3QPm#bqYL-rPHvti04)~Br}I)OuMnXn}Cu^O{kQR9AjAzZB@i6+^z4EkvJ9z zMs9H2TH8}m(!=e+#_AvtHudM0X|7m-q?mTnNZ)Jf@sW+8n-+T6RC{xB_5`Bw?Atycig$1rXVEHe=KR3^);s-UMV z60U7SGw#xJs3)86_-^%)Uo)f&N>j<-=BMT^353b`f9uhsA`^WH!JA>@f3`oIpOn5n zV^Dba^wjl>0^#lWUrivqaeCVNUCYJLT2Hv^F1(t;PQjR{Qg|g&HsPCzmZ75Uv)+|Cyf`MP3lH$wAswa=G;9FqgWO zQFk`dA?9uG@*Lrw;@gT^pIg9f!RDNhR`&b=k_7HN|LE1NV7Qo0`%aX}W%gQ^$?w}w z4<7-#tzzEeZNyu7^xr=F(LM0V_bPHrtUUOQ)me=v9N>+l#-)=+{hMX8lyuQ4h=e;Y z{R_IYMFcr!6}s_^12ZP)dfMoX$K^4`i$-<&J%aC8#byDyZ(5>Bz@zvTZqR`8qiyEk z8oz9wxar?Tp^?6|6m2EoCVWmZ*8mK>Zg9EmnJ(eJ_cK`V;SGJKriBOx@{eUd-* z&>nZ!e}tiR!3+{drZruz2L1x;;h&B}IE8xHc$r78f_Ws0Kaf45DC;cF-*Z}$2)Ox? zs|c6yw%G8_VvBxT|FGClbLuzJq5ip#5l2K|maykC5rP%es|MVEI9HakT>u;#Y`Ip* z!#sGnS7(h2RUAwD$fD8-*7|v!(o0@Gg;I2-BNtS(>@Fz%FFK)O(5Hd!(WniP!>oT> zl3-7xZS9|DNpfzbbmpe^Y_gd)DJ3Uoh(^xSQe<$ilGw5Pk(i}SS-A%Tm|Hej?b6Ri z3Oi%tXL}?jOb()h4U7G2xaLokOwVGwKN3kcT-%!h|8_KH3qH)ijUQF01&P=)(rY69 zi}JKQCyjw+KLD|(O7a^ts}v61vMCim|IREtAemGw8BJd(9S`kc zeAFhQYUDZ`*?4z%8@Cc2VQgp_`GMS^7iq2S^VgH!eS}e&%%W#+N zMm)n^VK<_XuidOfqAieTsurtA&>+!jO6jEUGGfH*fyX04(YrPicaNB8Gb4@%)KY?p zyq1B>V3oEzcT8gKM3KLW%>=27()Q}|NJv`;!Fw4JTYrJFA)X zQ`@pp_REGms4d&!Or`i1pI=)Xpm})o!#-Q3jmC22~M=cj-N=x z!1}-Jb_OHSD~5Ux7`>GqLT{lY6bx%bK{@Yz{b;)lwXS8Y{8D+ea58L|9t$~cgyM*ZrHnW z!&fzprza%eoRK^>Bk8LDT(P0CrtyS?C3K^Tp>DHn|P^vFL=E^p$3IM>p)uO4qZacRJ_h4xnm+P);i!@g&P`w1Ck|6ZqJ zRmY)$!YpIe!ExNk(gvxlN8dWMW6+`P+7CpEBP9I>K?&igH6wRvLlZMyhwf~!m*jB6 z2?Q0nMme!qptea@*0B^)mf{%+FYHR_&IT9;kWt`L|JIg4ZK^jRMqwDF~ugo;}K zTlCa!!OOYy@(VoGEWwO6HBs+jWf?^6o$;%%q>^fRCNCSE`kbd)vi%2d>qr!1<-I+i z%BzsA{=6i(v_#^mCF9qzm4V)__^=l?7v5h2;>V$Tvu4$nFB{JS{cKA-zUx1&U41wX zq0+7F&Nu)KZY9em4sAL-tY^k$^tVk(!&5gON*&lN= zAGYiY{_y8S1rNk0w8>S7x=ieQPUVKRk*`|Rr)4Xf?{G2_2W*wHkt#{=S{DyX7u`3W2@wxc~;P`Z%>XUQ*bA0NXI;1F@g6fi_b3pzW< zO>d+xr%Me6>i7f>Q}Ku#T=gCwW2_*sJ~?vk<3*Dt^J#3tz;8GPu03VRD} z2pyv`+>XJ+1iM1-=xjER&|k%Dh-GVFi$8a69COZ+-jGWoZnmB!CsYyB786uj{Y8Ob zSxW_W*0R`E!=UG!I0U)1Qs$@-R^B@?-KSVtaSeKHwN(7cqS>4g&hY0=8Ux@YsVe7K zXz3D))aVXEOuE4BR|C>0Bfo-JaTFcG=ZJEuXn?HV|KLvC6)}sC>=en5Oq46XJ+jxqEm);j;=7Tc zoJ!shSx6NP=eQ}gNoT3=Bl>xh;@efMYCZ7g-c&NIchiEY1sGEwVKv><*yIe722hHS zRuXZwoeq{Oj6L8LakWeGhB-1THh2E&`8zE%3NR-gQv;q8hrq=(cqMM|D^ML4brFe_ z^&Ny?2`!V(uOTuFQmDtzsli58&pm53$Z|!=43Wj^;#dUqCW-H1Oit+Kn>iyZi|d5r zay0-blCn-CpvSC3shfM#8tx7m)!;2U*CH0&`X}R;4C9o;OL(`lXED;YX$_|HQC0~q zJe>^a`2j|n>a{p6R49pUg$mMlSH}@zW;apt*)<5@Km|gR7B|OFZ5x>xOt_$D#m{FO zS^qP_v{U~TFl|SDd+l$t!b6T2)mnQ`R`}VDqDU`cKe7iM*@ks1Z%G~8I%R=BZ%e50 zbdYO{x`rntg8MuAx)#Yxh0WD$gGLP7Fb$D^-DbucU&ip=D!h%lQ4F&Hk=@oaZ=q~w18b-GtK?e1*!-_dd5h0?@?hNQrfcflzoMk+n~xut4yQ{ zD&%U6pG<_7wNz`=qJn4+nyry_!ZzMDYwrObL5@-dozVyZJKV!eKbfPog5PlnjEl%1 zCye`vlxGYkn=7orJc`=L|K*~!?1`4-w@7GN7!}LDR17V&n|W!^V)kRlK0U^`#-$Us3FuKSD-P+9{*|jt{y>Rk>Tv``=b1w&#MM6T-ccbMi>;{EMte^tNZ+KpTAif6
    sPI-s~WB1osZhRV_C~QofFH-5O=Ed=4WfPwFmX#4r*7CsB z*m^lP=E$ANCePU0LAV7oH+*h-u=QU&k6$-!t(0)OarF!F#nWt45{rG>=^9NiGDbDo zhjHPYRdZ`xYnjgk%Z2bMw3KnM0hk~JPzAN-><-HGkn}R3Xnj|Ne$#a!X6ODoRUWKz zDUS3C5T9$(Ov*=urHI%vU3NQ|SbBD};Z6j1u+x-%W3^-Hzo)mWnvQxb#35Y{4O191 zmd3A%3zNmb8*Uqa9=k~XNOH(wKbsu##wi`^f4(N$?>s_k$@Dhq|39S1sK|do1Uap! zJnvk8_{ya8w&${w+CND3r<^!0a_6J|>dm7fzwjj#<9`?U5>hr_82O)m{*;_?ksI-E zuLGzp08s04VdP45)E+ya2(FcpQvxr&$6dCMl3ljHMwjh^Zsmh|4sM+Z^!9G3@qD1S zr31oS;UC#ORKEPvKqMV9%}ds9{dP&orygCl^R_*i5_y#*hReUan?3VakH~48By35- z;NU*$N$_m%vUwBQq|C77AYbqG>ZA`u2Y`;HB}V`xjNUk&tuu~iUZ=Kv?9e%k^z!-r z(79u4=k|T~0BEIwlkx{H&F31cQP-4@VHRp9+espAmJibr_P}?CaztK{{+WG6IIE?Y zHnw?$B2tPsDQ)It(s&+1ym_WD9f@|M)bGJ4{h5UQ92i@ll=U)`Z``+zXtcUGC2Sy^ zfTKC%LM_!e(a7)RA>cStf{o6iZ&}t`C^{#j%iSv>7I!vu>)&*2IbNbYH8Fz}A>04n z_8ulF!nlOjaePgiHj-}uuL;H3RYlX3GSuMtzua37iI5{+Gwn}GshM^#uIGPd+-BMx zMxredCSfauLXw-W1rvo!NTqW7zsRB+_s}7dOi~#8cdH@KGN>W9#VKN=P0^Mwi(XPO z`$*|U+{>jd$uR&fL>J5y8{wJA;tmyQHkXtx&ss=Trn>Qd8owzmm}4UoT09&Qz9l^G zVvw0EvcM@=;-$5T84JSYqZoU*yYdNQazrng2wqyLOPlPcosv=$SyD-s+e;%h^GIy5 z^f8S74Fim5AWn+X-sL$}fZ*tX6piO7&k*%V>V3Q-tJX)>u@Ad^c0L3ufLn!AUihhwoC-IPxQq+;45Wl; ze?eKec;d1tPih;Hpch=qm$)ifk7=AhqLGma{_Y6q{ z$q8x4KIf z*$P!8k$(SB%8UsvgNV0=o&PB7I^NA^SKWnG>|7GBVlC?46tTOIZJ6PZBDC1icf>xd z%4EnpS@z3bdyH+IL3DOPAUzt;H9jBI@P`@HG}iD%nChH}K$eVi%@mnf?&y@&SZZRk zS_qoRd>5xVE~6)*MFNwvG`J$jd^3?!Sv=L~$rxB;7xjxjd zk&e@b(KKvR^=nNh`Fn2n@c2hmT}{o}=0{Fdd?)qM<|T5NppxRDwX0FR1{@m~tqqBb z)`ByQvdB$8%#({Rm04WCYp)wrBTi~ga?biS91cIi_WcN9bJq&ES5Utt8D^E=+>p>~ zS9PzZ=}Z+ZTiCT{bSG`dE8|wJA*m=h9Ie2!Z!HL9h5TNou??s$eoGsC*|_PkV4OG? zNFY#SAFzEIUebAXs~h zsbf$G)Xv7Acwh-ftQ!C;Wle3;QOdj4IgkGj*gV+{OSE150gX=Z{@%Ph2OoT(qHfE{ z1T5OFotV~s)>=Yv%#1^<2*(S1(zPne=K($15PZ9bX8mEVjUu@CSdFr#M!=$|AH;V; z!Xyb@!F)4X<|V-DPoalg0S$Rb3pm|msi#_5ag|n26y5}O9aNp?=&~wO3OJ2F2x0AC zMAZgxni}fpISS#VN>vn?L7%kDw}rUkB07ju>Ps2o+KEenS?$f5qZxBCzyd}ar!7@Z zAuYkin_N!j1~&lF#A6?QZ}fW$Y1`MN@fUxmo5R8Vf^tAMgl`y@bUbApak2~4M3VA| z3~gQfJH|K zJucKE07O{FH_?!z1)QZG~Y|<*{e(Ngp`8Ahv!fUv&4s#G0&5! zO=BG%M#ZgAQjIV23UdEO?sq=dB*)B57I3;lTRI<`VJ2NTmMubv@bvO&4BRYLZ$60f>&MdO#l z5_c7?XlxpmwClb(jUoKoeTy5n4NJ`Ypl43&*;z?tMc{FL6FJ}z!hnRP)MZ~t!bPd=m z7amQNr&a8pU78dk*`^!XTiJAP{DL)dRg{@Vi>K?aKbd=HOUlnEKg8M#w8XSwj4z|c zW=&{LrD(KG)B>)qdqMrQuH5RoE(Kkv{CJ6G)Fk1g=i7VSuPEN~tX}0{idb3*N4Z+$k_w&!m*WBD9_Vh3ceDUu*uGLPqA+ zi0wX-k?sEsGO{?dTFa5c@x`6p*h-_iv4Nore(lG1m-pXA6)Cq34^^b_gm$mF;DogZ znhhy-Is^ntvAJi8H(qboipb(y<%WSW)}rIkv)_(}8G2 zY{B=7IW^!OHURVH)H?JwE|VYBXiyF8dhij_;tF-DcF+RLv^}C&Qn?m5yqLGTcqZg@AY)PHK7fUe<#3B!a|P z6(vkXJb5;>f#Z+S;hq7&30gxz3r?FUOwKfp47fe`VWR^qCSBqtp{uqjuyg1^aq?tz z2){;HlqeEY0v7e&qM;pgIdC&S8G^8Y+XtYt%oP}>4i3rokTF3li}N`WMj;xWa4{Qm z2;kc>t+=SP)3Bj31;9Ef4Fs3d$`?*KL=9N<|D0XbE0E1qK z190_2Yi+?bN2e;(d&|tv9bgDvOFf@@iOu?fd~z*eVvBfVI8Pk9zZ1l8Y{smHMcNjh z41|Y9adnS4(Qx#X?NqWXcrMQz*`Qsm2f8{7&%9UbJ6a&FVe;59A#IB*OCPbAiJp*# zPv9B84!(|NT&$m304vutS;1JIXgEdYuZ&zAO;A`&#z~U_C>qjal4qv7D4AjDbz@JZ z==JjNN|RYuyY>Yf!!lP&He=Fgj*p3xChORB;IETI3*q^njlKgf9^6}& zbU}|KV4srDmzL~d9QWXYq$k9oy-Op#9y@qMu;Yx~g-}0^?N)FA03vTkcGKn&424o| zFKT_0T_Q_bUObo{OqwT@k8TKVZT0>oCBBEeT57M|dVn}8ub$f9uWaXu7DJ#ZPVK4U zA8bTd6v+Cw?Fiy%O{;Opps*k1cK&TUB8JEA3)v~#k@l&3do0SiGIUDv|D2S4_uYr5 zt(&>=`E!oCZq?<7)?D-F`EAYRaL9kd3L~0Tx!e0V)HA7?z=7RzS49X8hQ`W(@7UDN z>M+B;<=8n@h~Nm>sJ;RZQPhG>1^6mb!_N7t+b3{%qBW#$8#(3{Ky1T1D&&uf!unIr zJL)nj`jXL@2x-Zf^HAR9jxN12gpr|2{j?siCH$Cn6S9wdA)kLZX(eI#&#>-?h1l#d z#yBm0PaDR}dY@8wK_0S~+ow`+9~%pU}>;lFw81Z;*^nFs%d=2$2=6ne7g*iXHOq;+y!{%whwfBPNow5$aty`_;T zl7p-YAC2w~6jzACM|%yY9YIh&4}rN^!76hY<(ESE*`V~f_TqmlO|Wj2Wz%Byu8KX2 zPEkEC40(ARsuvCCV&geIzfmz**D>e;%Cj3%|2)k*=|XU>KXx99smWb0EUrnc362Fp zrO%F9%Q9=nniG9K77yaJow*D>Xs%iO5OtURhVWJ&pu|-5zJPh637px#>{rtDMf{(L9}sxS_kk2pThC znbbYtLFnQ!6f38-**n+h>mhFOBII#L(8fW!Xvb6{AOSWP){hlc+vp8M@IQ1x-CoIsJXjx}iXK%cES8xkevW$R# z7P>WxDIGnZ<>S3N_lux0398qGOJ+jR?7U~V*+m7^xmDynZs`r-Rkhbb3VKXl#Un$x z2H!=kq%LxjN1@z;CQZ$Bje^zQb~13b$TP>9dGVfE();jhds;fknI!UoFJWNMgkx&_ zDW{DqNf}?#WzJ%9P6J8!Sr*S^mw(O(~uyH+BZRozJ@wSBId$W>n&PrM~am5DT zS806$Y3tVpLO%`&uZ2HHFx^$!HX)p&U&Z2#{MgukAi zmYdNnd`eP!?g`z($K!wf28Capo|?Nb5MGOagI03N38TvMx-M#b_U^1)Sjo70tNOj+^X&JOwDn495sFI0g=f{o*}yVP3J zp}z075@|)LbxENxeD@@6p<;EsYrTrkI1t-VJyNz+sqkB$>T;go6QWHVQA_gCj5WE4&Ai8zrw9OuZ=y=KC

    zF%@M*??3jnLU{sN^Ubl)GKGocl%zq;;F+8*jgk|KWJ6gp5WFlDay~BntT4y6jX1U` zcij|>SzHx!Uty^7`2Mw({RdU%lwBW;xwb0is;5STH^;<2v@nneZDjWeCC7{n|0O2& zf~44U7Y2$Tk39O0_#T<@yC8>buW7n0HsSt^gqt(smtVSa(t!JPmYkasGKVH={P>ml40>#ZilkXk3OmCDb zJ+pNrsJiZL)GoT?;_jRG5r}(wR70NVz}8tis-f;HZpJpfXwzvOsvGw`Oq%W968KU$ zmz0kibT1A)-rS!%J$pA`+q(_a)#Op9E*-@Zz7f}jMsxYvPOfF6WY`y+>9R3m3jE7M zs}5eb;veTUQpZV2-bJM#GG;q*l4w6qUt}~n*hdQ!Ifmfhi&Gl^<62X3886uw^z!bU z;7b&`$Gq4WmZMe&3)^nWTw1Hl7i{F~|){1np7)fdVlX|mj$YLa=`DOi1t9||bSELtZ8X9QV*!=x>_<%TU z$lZGvmQBELR8+b(*4aBL#OEyZ7kTr8)r zWvP{h3^KA1OL)@HpdA4{rue3)8VKp&0(q#rcH}$m&gM{4T6>fu0As&=lUz-mAEp*; z<(wpsRzZ?{r0wf05p~){qE_}&*#LbZ!}7==&Uy{>B#J1u8(V zA>)pv(4~+H!7n&EO*G4ah#VVyRv$NSTttM&k$);*%U;e>9crJf*7p~bRkUXtuylY7 z3V7MpwgQp-nScnWVnfX%PK}J|tk(*I+cCweM|KLI0m%W;GZqa}?f7ClPI>VHW2~=- z;CTu`6=HC)%|SE86>gA9c!&u8(hQmB0Qp5g4T+ld`9hUxF-LYrh?iz= z+3M>;Ph8S1yI*r(U^?D(+nvjJ*d$gq4j;hngRnarIKNK7LV zT{$!vY*`ivQ}%rz*OYfQ|6o+qG*kV7lT4RufgV|;lN!s*IfLdtXY*Ry+YeLa+9OA< za_ys;N33!!u9R!o5N0IhF3qnWh=$q!5))fDIH!KwaU;T$V7Cv0|5E%oJ|=b@WRGQO zfpw5Q-kugH#ecUf3$(m4BAk!ky)rg@V_HSxm#};M#lk>&EQ-|54h2vrR~~cevAcg- z;o5OiTj34DZWOQos{Tzj208QOUgJ*YLpy)fTzva2_i{Z}|>><+`-Un(5!K_vly)%+7jqSlE)Jle7gg<#M|AUtw@f0xCHem z04K-P#2+t8xOyowjPFouC90};8;Qbd4X?6pQ(z=HW2H_}FjQ;ewj7eK`-BCeEqBwn^e(eeH8j= zB1e`wa_olRXL(2LwYq?7wKD?R@CX zd@$NOBRb1z&-3!{r*j(iPs?$%%g&Cn??3rKMzdePY_$v9mQ)9EAu62V+b?{w4T~e^ zw)}7!a%c(s71?uha;Ycm7rG2MZG~oV>^q9Jj{~O}i=dEItl@HIa_mQS z1%Fl*lR1bEgwcbPD#U@^5EH1^q=jVKs`%vW=BYyXn30~{yreS!Onk|(YLjg*g))p9 z4$+fKNx{f1%^H7eGe=dtaNy%4jJ5O6q1r`ae5@E+fCuZfim9#3>OtmrX2*U+?%709 zd9!RHx9tixz_M9bTFdVA4a!dXdhZK!Kqj%!EBXfL>r$_`LmZ<-h8>)^;26m@wJXb| z@DPK#y@~k?GfohO0w|A=L#08NHFP-C0LBV_z4kk(IA;yWfCHZK^xCYQk7iZHUXd&o z5Dd&!oFuQQ;)vDa7B%QVRa=IB2n+*^L(&G{V|2CLQeW13C!XZQVgex3?J#10Oe9Qh zW5yfF2!)ZgVlx_uNd{diHlxE~J+P|g%A(wHTcs+SLN^vUdMyj9I{Umd`?^;roH9Rj z)rE1HC7mao(P#J>>uGtfXys7t~}OCB%Fnu^WOfP zP@$UTEQI8O{+4kCo$ArO5!D6U|1pZju}1c57C_1rm|A0<=W|Or=cv_iJ9b;92+k*U zR4?T`&3;)YwAE=E$JD}#QOcu>^tQ|7_Yz4m8|VHWv)9fXn#kDR6d%`K-LsV;5J zWj}A1Hntx*Co_K5vD$uQR_s%$AGc09VMM6-s8D!HZ~Wg`)6`E0CAZy{(>6^oZJW@s zO?vw0bPuggLebro_e`$~^cAS!@ad zYOB73tu8WP>c80LJd)Bw7)5&cjywUSU*IHxEkff4&E_{8CCb|7jH{h<FN(Ii_wu2q*-X{yH|(*1s_@9DG4z6HSI>qaO58RAef~>afp`~Vw!k~0qT6SO<+7G zvxJnFF`s@7MTG&%&DC$sP!6Z~K{zVatFY&?eS#UmiV%4-}$g?HQa`6b)~3WbFXAO;9JS& zV(0(A-EidN!&5q_{hDFlLnPg%Rqlp}1BbG3&Y__l?Y^E`>#D=ln#~PrOdbtFsc!J% zAsxTeJC$McqLY^FYNNh$7I@Ixk!)CVHa^qL6^nfIX4JxN&0p}rEdZy23l(gX-$q)p z7dKMlNaEcjrla~c82s~=Z8P{U@7Aqds8?(_)`1RC4ar-a-s~gc2R-HjgrXwW>|21! zKib-(wJVwkRS~#>{^K|;JopOs(@!E(?S>WCpVHj&9E)zsAAYbRq7j?9Zc2k}ux^N3 zIZdm^&JG=@WP%#8`Orf!AR)%rC?BHWk*OVXdkX0<$gG=LQmPRxV{>T;Zl231YKx3S zl8CpmjWx!WF}2&+mh}D+)mQEYBq-2A819*ksP(VDx(qY?g#YG;g>~4>pz3?a&kwun z4gc#Fr5CeLR}1WdVrsp546hP~k;{iICjv1#>DBQa-c1@o5S14{v4=!uR%9lwX}zW; z<*i@AInbGTvRipFY_2Q4@k@Vs>fZP9_!^fBv4T*iI*ZOT1MHE|oqF071=awhb4}pV|Pp#YV69*TuILsr-xNS++wkMIO8ks+SAL5E+-Ixrh zI16O|ZXm}*?U^}KM)=Dc8q?gA8c*9|Aa6kS0*d4C4{sy9JYDcz{pr9J?MJ!vr^n(J*8Td)?y$$9b0D-YombbzH71uk@5d-`~iUla$JFa&-pAU6>sej^Kg znY6(vHp3AfL3%v_r@YT^XHjtaa-&7Qh8@`QaZ0E8j#5*mU&5KC&jEX>-&*eS{Dv=ylcVpFyAq>`uLoht zD6F_(6#4`IE}y`4Y*}hu-1YycI=1t02p1_Y^KHqNv3$~?=EFmh(=VsLZH_-LX~&@m zzkfu|)n}^A*_-Mn6p0mkr`YNd(bAr3L`@>qF1IyT=?6#FB8GNZo+0ZLe;v8T>({E9 z!x5>7kVCGz@I=(UWA^ypN=P2(R&$N}z)9`e`2aQuwnYrVn@_ZS3h(1tht*u#Q&L*Q zz)3ffa2mQil3J8ykW&8F<9r@w>n5)3nvTM<|~v0UcSy>D2bq1^nCK*N7Z>nU%SjA7)O}{p@BW6`Kw`&c$q}&6(cW|D}FM+KuFmthn@X zE1RTLvK%J=s&n?US@9h|&NFfyJR`5NJR{G@f8i~zXFFEbvwewrwgTZ9Ib%T}@6BM- zFrYPf{BOH8v#dujv_iR&oU!ejYEL(kyzQqYhL@0R5g?7(C3leKU(W`v$Pc8^TX}*s zMOxXQ6K`dVj12Fn!WydS8;ukEAhD@R ze-;~H;=Ng7s0mx}xSMD(UJwZe56eVAF3G^+STvpl$XtTjwcm>}wpRQ&hs)To_cF7_ ztfkF@Qon|BlOeLLUmI@#^hvp?HZKYjb~rTD>;Oct7a6}SB&|jplnKg#nev< zYv<#%7eDnDN+SQoUE8A~EuF+gKu{_wt}@9Q+lU9S+v|E{iBTd$I}ld>9^Q=+B6uq- z)M81s1&dkCIYJ#_Vyh!1q;W?QNSV{a+#}ADo`6^~!`h*2dzT6;IzCMjOji3lSt%V~ zv59}Uva+xZzWz^Ql-g`+$)C4XaGOZ!icZly;JNf#9$SU|F zeSGrum%Z7jYxZhJYz!OV+Hb}HRzx1&$5|-D z(FVm*rN($ncz2Qz;YYc0S}G=sYd{)dsgjIYPsGIs1m@Kd83OB8PZ$gY28K!d#v5Wx z0%43K%BW(>seO{bTZkncK)}vNSWy^cWi5D2 zNaWAVu$?&&1T*E=_4;B>Mm>~t2o2B7Fj^Bs%FMvZ$>Z}#;Wl4t5(K1p?Bz2pz^cfO zS%E1aR7Am4WOl*s=;?x!B|YVZfJLu4BgE#|nQ#Q2pd@bLXi^m531yA7Ay&1BTYj7+ ziq_*~ExnoOoxzVG?`zIw+q!?s^i#*e*7e1(3i`@0tU14dE?BGwRD?m91#6@ulLuhZ z0FuRARST$1;)p}Uq!BZ%@;q44=aB)gZJeo^tt${%j)`8}9P6EWl;0_6cgsxvY&uA4 zy>85X@?DyYW>vn*X`I4tV76h=bGD}H8-i72?hmUk_th2)DwT;LF=ii4X}_cIhP+H< zW~e~Wcp?RhMQ)7jpl%ET@Jy=mNXPT7;!+MVm_??kfc}~0KCicky*GMMu@)JHkPWL6 zEy-xb)-tTTec+(^nd3?aj4w%)>Uo6}sVQ4arerl4Pv&*N&h4!ZO zZzE^_|ICWpb>Ye;GL3w3Zf05kVCcQw=AdMm%uQ z_bmgCk|Gf$ZrnVLQe~7D&bI3_Q)ZpS-=ggv{Fp^Q-UW$EuYlB*Nn|Knyjkqp7S?mx>)UwC)$|^P zW|gax3VM`kP#Wd*k8HC~KkXr(RVVI`%O3t!--YOx&{n%JUo58|Zz_zD2>0{HD$s2F zzZ0ali(ib`oNSy*pDMYy?N3XNunse}B`LC^TL4>>3Ym?i6GsTO6l|{_PKoesP zYR%k<3L~2TI>pqr^DiLT)r8&aCR)2uba+K%RqLGLc!Hyol@e4`)SZXjz45&avm%l= z?8bDl&5#q>U(bBaUs?9fm2S#1Fjq};e^2`PxnOAiiXVRJuPj?MG)h?}qdtxT z8?So?0e4t=l6b;6Gs1PiIJwbJY*G7+KN%EfI+}Gq{Qv45)x#LT5MFYsKRi-4BMnP0tqzImoJ$ zw?T?^T^v0Z`h(-wWHUJrNiUvi=su*T(=cCf2&b&05Z!G;Vru(<<7CJ=S-WWrxAh}{ zwVZ1fE%2uC93yV_fCR#hgk>aNAeAzA$;~8U&FFx5;v7n-WlOpZqLv6`I_MrB^f|GV4Ldvk%E6 z(a^R9ksfa`NxLt<-504405WTeXw$(Ik5w8DlY7-h5^;_?Q- zZ|>hppvK)avSQTXZ^hd<2`&pZHD6a}R54i|h_GVv}jntjqI4z?n>)xGt9zYFA>BIZIJ7-gB8|>xXJIBBr zGG*VCte^lLn=5O`KNA0v-WVEBW}0$gn5c|KHax`Z%}n_kK3;>1bSt*ssh}aTAIELC zBg!EX?IVW{vL_LY_y;H6xgreWbA~;^ImuDV?SI3+se!nS4uOtp$-4rj0UqtMnX>v} zNXoA2Ud_{ZWnaFqTXLjnBz*se`uP59cRwQ*L#&{;aI+DwkoZf9myDUBV*&tBk%rd+ ztz296YZXTu0v5TWLB=RIUpZ@d+p+IF!mq@TOT+|Dp`12bnji-Dj1>9^jhDN=A0l!g z9O?KXDZYF+UoN5`nq~c? z%!@4GDlh@$kSiEMJFta)#ep!B5?sdn#&1XoNPJ(8u$9Sz+DteJfB;yjy{xcQBUX9H z#xGklQV(l#)$SL5q9vy@NyfyMVOS?zeBjX3D@f~)e=p@?^W?jrWB$mYeSRyyr$4F4;ZMukm4U7@M|%#uO!9(dHn3Ac_7oW;?qI4 z^<1^F$D*b?;NYK`aC2t-@<}T<#(a^|CqHGwn*87oee2ht{dHdI&fzJ!f65Q;85L~q zHf83KYt6Prwq`q`V#t4cYZiCVj+Gs>zsrc9wW?eF;<(iI+%chyz4_sE7*xJq(==7| zW-EdyG65aFSHCm0^`b$yvSjY8ErFrzG*Wx}xIuUDp=-BPtUx7Y+xJu@1vTV$0zgG~ z6t+DpoFiwSoHFaVzQoP#9QMrc|0)hOGqdK(w0-eJ`zcHFJ*^|Z*njtz1YJ)2b$uG6 zmmb-@&P3V=hfWouy;TRtv+bCovvk3j1=Zf^cj#TCn31s>qqs{mTC64ktn!TF?9fJ= zT=1h6lkB%b85@eIr4Z73^I#tf;jLW-<|r)TtBZaA9oU?W5Q`H!zE}Y1?I) zNrVsEf1MOxcN|lF?Zfe768#7tO1Nr9fjt{#2Z1JOINj^osCs$_B#{q!N+OGWC6Vv> zNFodF>i0P0nZCyUI~HpY_XuJH+`pWLO>`(b_c1J&sn~(H^eQYHrbx2xk~b^3xm0#) ziv?QtF^$9)oOt@e@G1bitrvg z2QZ6B-P;!IZgGuavFY!&R{!IA`&3onXSr{ODAtFq<}*W)CJGT;;Hl5Z={t{ z-V(EI-ghSykrK4g7Og>>Oc+F9DV<)G&o7Y%UVEI{ju8h70hv7jp%@MocjnMp#~AeZ zIsu*j5CU>Ox|a9!D(0d*iq{YuyLoiVVZ2!TfEj5YD!L$#fA*NG(n(^G2ew|r}1d_wPPGkIwRj?whCosxH8|@!=R%n8JTj?q`SHd6K;_3`~xwsxCd2tp)ZQD zb)%noY?)5eMU&Rokl%#EQ~;<(4&hw7IAfu~DS19|8H&MRT!6(xO;v^H^lP%b-^ay= zk!XZh9>xs*lNOZE7S~?S(!a9{NEp4IMH9%#$#1>JXW`!N&d^wf+G0Hb8I}jA>>v^Z z@t_Z(OWv&8jRT|E8R}UF3@*doWO|&O3F|h1k4_8SL0ie|j{CIjdV(y|)t{4}LY@O|Z01hEN88o| zIa#gsK@r7~tF+|c7-3mYGRH~;-t4K9RVdYtc~#%p~G z!~gUeaW7*lWlU!PbVuH6;>Jaer;UY$ex=I{52F`{kM7HhEV*rztxoV25Q{VS4Y6>RTg+6>mzb0T@Are7i+H2 zufiqz7wi*l9?=V}7()_B`kM9%$(ccJ!P}zKx8EN3S{ppOlc*rniN9nY*i{(W?|s0V z5OYQc8oY;LpxYiDB{uNz<$!GKM==m3_y2*3qMAxMMIs3Sc8YS)MvVuu07~v<1*xn;+2J6=nMD+0S^GjXhQ4q%Dt&Q+LHo+1s&&uQe^XU#1;yq12uk%J=}#zjMC zy_NZ=$^h3|J7b=RYuI6F-n9_jJB zCam1}KaX$x;1@}`tMY@-j0*m#Z~d`g>&{Qzd0tBHiu~Y%ed~9}r|ujwG3KN(p(VXw znvZ`uW#@v*s*E|Q`^f#Gk@gWbhdl7p%pt#6f9RAx=Xut&ebKG`8)A=8)JCjY`=yl? z8KRSR-p;J%enYuN>!TgF%vajVCxsHSeQC*I7Ds+x9e)g&J)WJmXA{wKRNz(;>U~}4 z7X@^|J|p?{Y1f4YynV~^3}V&XiMH&tFF;HY5hy1vBP32;%R_Pf4w+5)X^$5&#v!{v zIPpirnrx)(&R({Uj67^K+h-Zqwl3&4IAzp?+b_Dh{MOEsZhP>oO-EHPdiCyA%j^H; zwYy}uH_h5%V#8A;;zQ=&Y=Oy1WIHlb7;5rrs69d)Zv0SHa!h4ZGd8pwO=1pFOW1yT zQq^wKQU0Q;qVX7ECYe!HQ5;8I=Ro;U8+4?9om89k8kC<rm%0BKS?$evt!I@Vk5>vT!G=;Mb$P z&8U~$9@)JOCisn$Zy?5%cR7=JWY??*&gIZmD#Rl317I5>4nOMwZe49+@~rh9K}nih zSi@Lj?J9N_FbH_8Bqg{X#K=5_WZAs6ZAtNm4?RTZgZm-9mn|JEVWPwq4UK7d-YC0W zcYZ;`ij(={b%ddxoFN*|l+P?AVu@mbV&8azOzqt`Oz{{X*`4w^We{SX#ikmK2LUe{ zdFXUz(k^`BNJhyjHS1<2EIC%Ldv!iL13+s9`tVFBC6`=AU!iR)3`%}GNw5yUQ7DHp zGdwzy+HNtt(UBbP43)a0mdTtn)=6Z%kGPSBdnw`w5*~=qPr zsaHD!GfRVI55#Q%x&HJINQ0#c(+qeNhv*dX{aq@mh=coS(laq{ty~AxY1PG|Rh-=l zLNF&X6n+rxF+HXw=Fd}7DrXd^$@thVV9OEQQDoF6dN3Qxx9m|{bm*8*S}sY0u-r>q z;TWb24k1c4Kh*j;zx|fC$~)gu)vqqi&eNDpGQ5SxC6Kt)`BHEL*A1u=7MO+w6^B%y zD%TOiW9vf|B_Vr+cvw^y>XEd~Ntu!)8+l_;&hO0C*0_d#F0FKorYOlUnZK_!zjO{p z9-_Raw{=FAaI~;7$Fk$5wYDQ4DQesdXh-A^?QDAFEE7?Is1o~+`M+Cr*e$cx1z(aX zP8(U!L}!j1yV<*L@4(Liw`;Tvh78{co+IFUpE2qbL>UHQS>lbg-4ShGn3|EZQ}m8B z5dYR4vwBC`7le6zIDh7oEVI6KSBT@r@(C9ACS=u-_eyF!qB0ncH!|*o*c|2Pc1qf9 z%8Z+WF^j8W?kfyc9^b#Vvj3pUoU-eKG1pebT=mq5@aCA!+(j1 zy&x&}+=YRnGsc%3eMfwc%=lfGt=!mN({vfclo<&(XT&eRbR|1T25VnlJ;={La%8wF zCic%gW1n0Q7zm4!&&HSREezis6FX@^pc?)pyU&cjIW7L0ON1XuP1F0*q9pDgBAI+4 z>8B|so4-o;wfMhV*0mJ^0E8z(B)OJFlIJSR3HLnLl4pS>5vlTPJlmEjiN1;p zj(h&#{mOq{{jqop>wJW{hgOubI*3wq>a-|B5?&gj_%>b4MxG@zTTKv)1q!%qcxFC9 ziCM|Qcm$$JLc>PUBo&NFu{y&)jQ|-ib6y9!N|!o1A+6*ct1Mx}<^#to9&MZUB*_+s zD5>QK*E-qdaZ37d@&VOF`QuSu*#OrJHJ8kt=O`eU&JiREh`8(2z(&C#w zUPvHPo_+aoL3QDjMjw}dB80rDwgw`EBCSZ$TFzJ+u38p1@XV#*`_tGZ=G^f>%>pfahPU4e;bstNn;at#!mCN!v2IOT zb>IQSrKX2c?P*NM8GC5DU^;KiC|g0=08#%G9fQc*!f72jKNLwW$;c#8CV@fm@THe9 zjK!H0Wx9&g@P#8>M;~v z3e=6{fjM_1c_EqEhalG zNP*)citgB9Sd2X{yS`eT71o?F(w}Ju&Qd$prM=9O5RxdjJbH)JR7nj*i-6m%NZC0gWy7-k z-~*$A5A>~HPuv{-CCweo+#CcyNx4h&>x7yZW>4hts-ofu4m>r&SGWuwDTh#R8y_t6} zyAYXpeTEMct4tJFY8u66XkD%;8(IIKimi9 zZ?6WMP6NbTHh29Yh%q*L<}!-0S4DCZeRM%k~~(d0e_|5BaZw zlH2m2;g%4q6J)~ajK2lWV!~-q1;3v7x4>0THRpmZ%PiRH$3K0m0U8CI?Bm(y$f{1+R*rh4|b!{ z4jR4XN|V@4nvZ1vWw0!MgU?@GXUkS0YIkeWMm$=1f#;N%$6eD-dB$mf5 zukPz?)3zz1O&b+|rKew^{{Aub_sg%uq3q^oBYZ^+iyjS-Dx#8R0|s>9?>_(p2c?C9 zPqHhb#O`T;IfCW^K0|t1v}@*U0`F7ykZ8MG{S@UTH?zSso#JDcj7G|J5#fT9sAzv9 zOYl~0YhXz7eio;(6BBE~ODglz@S(ylR*J7hRI%7YBQS+de0Ry9UMo;_>oLiriO32) zs%e+?E@=UVQu1NL?pMCJH=D?DiCFnV4_4G|J)Ji3_Uk95bez3LLLyw6%ZkgY_V53oftXpRl+_;0a-BR**&E&6RR$s=t#0Rc9l;P1W70Shr>r(=bgijQ z53PzG9FRgS*Eb!DdMssJ#Gx6j-NJ-K&t8PqD}>pf>-7CVG3YK~CQ)rTMbu--47Y)i z1LDHH6i24#2~4Z!s~Nn=|Xk_mgfNOFfaGHDV53aydDdvhdO@*XShcwLEdh>FSu0&g2k2ly;=YqlTDO6U`?Qlc)54tX~qLD z2TI=+Yv6DwNxYHE)6UC1-iXs*QL<~59hyAaUpWF8=IYCtO`T?H=c5Oq&TQ(jibQJW zW^YxRm=hPlt*~c%q@R}@T5J1tUlGifB06LMc1=(dR<>G<%nlSPNr+xs<&@X2v8#GI zZPlNCuyuK3#|eE#CoOpK`2K_D^{=hUDZ4QkGqWn@w!+ZmI5f9^?VJ+>MWEN>$A&k? z#7<9&y#b8@z^w%)lyn;#em*Al`0-0N9<`|H>ez(WGvcoS*Vgv<#_wyI#=^@aBcVq| zT=TFwS;6;51>fmge{DkQ&fL_cxhWg|m>;a~TR#*n1BRwf9@;B328{vk=s)O=oU#$Y zm{FB6L&t>n0)E{J$A0J{r;aVzT^PO<0Iz54xo8YVyF)3ru!1})Tx}J@xJUUlYBWb3=1o2dgK4mI z>58xDA{0ci;*r-xrKQ}=Rw5Jb*-)o{uZ>0==cAuCI>Pbu?Sx|t!dL)^?K>JJR*^Y~ zfPdVo4dfzYU$GwId>Jhfu|`=jXeP;46ZnONiJZxz*NAiZ5VvY;%sYVeBMpCV$iMKX z#%9`XUEi!rLx#u>o!oUg0vZxj6qW<2p{lP3D9pNilvrz1nn%k+c&fcSY2&FrJSBaE zS2^3X4U6BJ_Ew&+Gf8#7UO*ar2hUL=l1e&MPBgp&CHq}kb7zyBODlOQ9n-8t$oI&)i1edJNIJNRd##(|WYUrPMu-d6vi;Tr4 z%m8|QIoixyT#(O(t029ViK?-pRh&|eIF-jYs_=n&IAhm;#I8(p+19RrJD9UT+@7+< zUe@vHczk5q*IZ~1z^;(U#Y`8n`b(k|l`*z;g{-ZK@NX4C%ljwocQ zaKQGils5}dIFE$TrSZD|BAw?5?$hcG*GR=_%X3>BzQ6(~&4MUQHh7AR+Ay=JyE#;m zKJL>1eImRz;1#}gRG)%=Z-(qS17@CeKgxYt&P>g4O8f_#L%}v%!Gicn#t{6l06uv@ zVPe;i);9FK;xLFQ2=HQpAY#1i3|a-$bu`X0j4KXF%E~4!ivq#_o_O%m;xfRS6>Ij*GO^eA zmZte7Ow}9S!+Hh)97S+#$BObc(@o=@4R;#8Ay#Fo+S$}0x6BsrKa`fi&Ze1V$4Vfs zL>`;KmBkEMB7bka7Wz)(7C=*+!5Xe1jar;hJdY+u51w45Zp*Hix8U25`*L3oFe*t= z7pY-&wv8oD5Rajeb{1Mvq=e@9X__BU%2X!?5(J>x>4#HV028c`fW}-pFv3!EOtJ7)Y!c z9O~lXgmQ;cQqh4ymzu!K{*aL%4>O-<_S#kDEk7J489atQvINxZ2wo5nZ$;Fp;e!2TdObPOfK|=&AN`-8WNRRNgyT&0EfAOq4ji zu+0zMvn+LC*I2ed+H zP-e>+6LBmn4>i0x+=#9arX6XY|4V!ACN-zVXeyaYR{k9I;FmIaA`ih2q`aNs5|f2e zyI^P5+SA9=b5RcgIKMY*9gBB-1gFsdmVPI-d+cPkM1|FZWk;89fB+OUL*NSW@AE?O((TDB1~@yh9H5kBZdwTiNE3IRdqi?!E1R^nd?N1rG~$wJf|G&HR1 zL@&1P-?N@15`pL4MwJdB7!9vY@ zwn7>Vxe!EGVa<8vxmN|eG)5Z)ou|@O$GmPb{L}#aS<-L4uy4~t#+Zhmw=gza$Zazv zo2G+N?YD_fd$wy>G`qBba%9F_GgejxB67%p+h~&5>beqG0%p6OEv?luLS}y|xZ(LR zbqr9M%7Yt#C{BSp1F3~&W${$bF&M7GB_O1{=ruE{RuLiSHNhZj#G()e!wZKJ%)wo% zKo-20SqMhU>rx&pZG1`FQq)7jDbQ+@FT|+SDzM;tF8h<{fj5 zN-#AoNePyD!$+}Hkmm>$NDu^cDs6L8Bx#vyWt>HtiXuCnY8+x@_R-SwPJ8 z0QD+wRb8lsR;NS7&h?GeX8Wll#j{mkwL7o*R4-GKN4-o*F8;sYlw=X9B3(OegO&0s zXk*Kf7@f8k9YD9YVB-7B5xKl@*n+cK z2+@c;a-%3Zoe9jDsfeU_vpK<{zqZq<*>8I`)c$6|=@cff2A>WLIZrEXOj*Z=1P51P&iM{+Cj*Q$Jzji`8GDJwu zTvk;E=0M?W31(uM_|GZsbqVjSYF{y6G}U@p2NNp3$hj8K3REb%Gkc_ahs=2!Ar;SF zYqxS11+;&hA}kCH#cWh%p^Djn_NA#;bTScZ0)hi3qj>5?g`{#j*U>pdPDT|7)R1t) zxV>#nzC3;RwLG~g}D6l{|r*y9hHvJ12pl36PBiVWo`^?*yx-z*B) zs36kFB>XV@J^5hNi*pkk`UK-_K!bw3?A*#gH!+S}oKS%|YB^ou8c>o)s4FUA)r=*$ zMlZ9J|5}D3>P) zoC~pqgoELD#ySH zhMFbquY9tTTeR(aHa=;5cG95i#Eh%f)Mc!z>$AM^VR(o1OS*OK(V?E*)2EJ3%dN=| z^eqgWkQ4d>){djncju?&KARtifA{1e^K(kB2*e#28@!^|z#*jrYwpV_`Hw){j`Fyz z-Ri$vUB9gE%e8f%S2doAnzW3h@fnHJ{$ovDV^w2c$S3A8Kog-*i% z`kWGac}`01vi#77Icd2kc5B)?98$-lb3(JajA|Y7U>1twLYt0yZ}h!Cuv%^2px8=C zRI6)u@Ch`onPVIZWT`+Lc$;K@X}G!bY3B0ewM3WscW8h(<U)VMPj=(|6ztp}Yw<uJ5=b%y+^UEslIhfWB06{VHHE*VXjEYQ01l&RT%}!s@f%_61c)|j zd?6l@QL{8_ber5W&=}JQ^A!Gt&^206c3IEEB;yo;9NH0k$Gf;hD!mN&O`dX!Ak8qj z6Cy}l(rk7ok?;b|oKcVrGZyr>5SUCx@AiNmm!N}W%!n7+vx}{PHB8%tZD=wHZxyQ$ zn{K8di%MJIK}REC`3F(>%_>LDH9BkqNs!HQ9U!RnoK{e2fXJx{Ll8Xan97EPsGCp^ zM?@+b&jB;iYiP^YL?gA%CdK3nYzXzCdq65o6vgU^s6YZvQLh>{hKU?^eQdQi)vJuk z{>bHk=y*|DttLbAHbOU(q$fk5hz_vtAX8@Q}V#F*@5~Vj<06D zrwXz~PHJFiwHn-VZ5!ii$BK>LWISo(&h#g!(FfhUSG@d&zk{yga$e04tFYtVBLc@$E#7 zS_^8_ZZAmL@^F4A^Q7gCvyL4Vo)j0qY5TZvNgv;)oJrw3`uJwyf3sKlnqM6i&d0yK zIxc)upR(jH64KgET;{8dM`_wQK_9B)YU3{Je&7d{tDUpz1Nen79CGWT-FOsetK#~d zcjONO_HZ|aw1}x8_wnuaZufn`YRDS@_#yLgYHia%BBp}J3bZh4z{*kVq|uL$SB@UA zd-SoN93&Phh9_kG+ES_6lmS%~1vdNn24>1k0F61Q%ggp2-Y*(k6KNjBwBgj~ttWIa zA}oricGJhS9yp;J=5GoySV{wl7gY|eDm=4k4w3YGW-mBq6Z6GM9^9%lnMB;;Zw+tA z&-xRAWtaZhL_T6^HLznG7Qncl+8WqE8!xqd%BoT;y(g*5wnJOBYO9a6#|+LE3|Sy#&1wbW@B3h78%(;S>4i>y7zG{$bdwhOgt5OmM`qKq|WuFcm-jq>0CEe|mk ztWh7zm->vovCz;>`wmM7`5^sS+u&YAlx4XRKwgY|BAdZxk2zxK{J{O$6o&$K>jy{I|56)6WiU(dKqv|yHsFTLe! z0)hD@5i>`w(sD&cn*$2XW2eg85}mvW)+`aNmCcDmLB5mZO;NMeMw`;uZC?q?5rni5Xm+Y;xj8JL0aM{Qo|OYa zsDvc^Pyme#+=jlXhf;(K)=*TH#c_7^4JS7>KhG?h|Ivrb^4>g6CD4Jej7E-HvvUUT z+u|V=s^3!&&37#fv};IC_1cNtw^jX%yjUH-NEel6Mm2Y}UB`iM!}@;87uvpqBN-;6 zVT8U7SB&+x)f;)!4y!a!MY9ePj(ky?uYTPrO-rp1z9RO3qWSE<>ov>}Co_N6+`ZYu z$Pgr3i~j_{vMu6p7%N0%L=13-c-wWrX}F|QzKx-X#KlT@X@HRmzX3PV$4F&5=DAJc ziPszqjYHkrx^?#}IpZGr+obfr|3|p{?2eO8|DPFGJsKLeYvX;tI_uleU?1UiN965-D&XSo`GYH1^HtE1=qTQzS-riB(GWsLl>NpZ_Y z2UM_LehWPSI;h_0IRe;d>>!XClyRh*jech883M%+q3DU5nPD=>zT zJ}P`sT>M1*@4~qFWWc>&_VFbH=iP#Tdv#RF)UnGNtGkQ~Z#X zlpD8o%}Pk#0Q7etGqHVL*U&?7i5xLDnDVm9~{FFa5tMr zQdIFd(rE_#rcSN<{(8O%G2~Gq$}Ez4f`s0yJxy}R<1t!vdF;$K4cKH}Q~@|X#%2~R zvdkZE=iu0DB;aB+!jV4=LIm{UuPdCO>`<(}wkr|ULe|;3K0oUb(yDIH&uRNb7W07( zkV+;8N!y9S31i8efYjtb0s@YVKb^I9Qb{S8OWL5}`2Sr@$N#^1I{ueZgEm>>Hltra zkQLju7R7N|5~^yJO6Gb=CCfdfl9@)SWLlL{NRTkaamSO!${m@eXsE)c@xUE z_rh|`t+n_6kF&M*u>b>OmD>u2>q za^XwbgQ>-~bd4;>kNf&@IaS+t%gASOb}xIL+>q_1aMrTqR6(hRV&W9N*A_B#DZUsl z($`i-UVoAb;qWm6RI5lq2M_!)=U>U<&=fcrAbWxntF|G3jN<3&*LEj6oAg$Y9f2>L6DQ8l3|BMxJ%H zY=ktj*N@2xs_h(#yuS$cL3vG76;o>V+2b#oWCrn%#(=c#{_3)tvntlP1O1s{e{qu$PlXC1x+jv>|A2e#2o1y7<_}_+7C{@^<|G4ij=MGT5Lr_j zIAP;>39=*nGs#OxOd;?@q}`5A7^qVhDZe1fW?H}JSfZnEl1AUtw#o>y2_x+``0l!C zH9irI_IghgFLv@Nzlsm8Q($-tQ5IM z)99FC=RVh*$F9@Ep@r&iU7}~8V{KTU`>Ef35#^7cAZ@jre@hX+#bT$N-Rt3Qhv6q? z-rvPglAjno+~_?>&20E``IWfQ#ffAU9LFrR8RaP>b}2eO4dH}%j%r-hP&1Y*?sfQ; zPquT1?eG!v^`CLik@NL`ufjhJ7E0#{`Z-A8&(vsolna$S+Sv;~GJ;vVkkj2cx7 zI!(@6=P~8?qxe7S5lNVk?>cpiXp*M!kpY9@lFijLDBQtXzS(>%PnG$*vBV=Yj_JaV zbm2tO(YvC|e=L7a+q|r__EF2w7w#JlhTRI)IMev;_iGv+v)|wTk%ZN)Z~P;}KC8!AZ}klpC6_T)r^ZTlkAU$#NfecAkTz523Emn@VK9&uKe zObp=eBr?}(HPrkYXSSq5-K>&;!zIWrJc!6pLmzuELrj>N)jA4j!G{FFk0W4G+f@!R zN_-O)EJsFAk~;$25k(W~cjfLUkActFJir9cm~^w?8T%Bd;X$^K()M+9Z&jeo%zM&MsLXB9TFP`2Z}$@&vhasvOtt56XM24ssb_3>n@R6!hpCY(CswGj zEuz=G+d~*_xQ)cg3d{|JPDZ$oxUCz+L4tUq-;mJ)Z(Rx`Ab?OQu^-c=CPx%9N0$+* zrNtZmf&h~iU`7aqd8-<+8{n6;8lg)(>at-ON_(#j?nZbYb4v>%ubd05Z12YJ%#Tw| z%azx*93-$|mx$+VPpFAsLys^aS~V^2FP4LET2@zgbblZ_Vf)qH(^nRzPc2N#eK9{U zqA)NlCvcb#qi?nosG3+9@0#wDei}Xfm>!tgKm~lWH zPR(S(DTlPLz8zp8lu98>d+djM;1t4?A4)Zu_2Na_rkOlv)&3QH=SVjab&REl7%3TO zOU)#@y4pY*NGs?*xx&xQEs_mMcrzoNqXA!$>A#eQ5{E{NU)yy+W#!-K56 z*x>$FI&dhR>hb+SYMMPGGXS8xO}j*vSNZS&=oxIkAQ2<8cdY<<7g-?hcYM6(VsT6$ z_$=$wuTakZLvgC8Y<*8a*rO3n?@cuHUCVlZ=Sk9!|H+Y``ghjs_f$uAOl#c>mDb?X zcmSWa9>q>N=Gfr*dgO&0_Exohk<)Z|Xllk4B>!tl9F}6T9BKL4sU0!pkpQ{=iF>}b zjK=Icn$D6#sn+t?S-6p_7ca&gF>~|UU$)=Lx;nSI;|HJ-eWlAzAxVuu4k|W7oKMbQ z#+{b`D3(epQJ5jQic`=*W*WCA?D1)5TA)owXNr(hK!LV1NF~i7o)z+uc_hoi$UJgq z50NXaO03QGT}vXZh71{as!iGWtwj7_)_p11MYOV_)Xbn67}MLA)96~F$ReWDu&<}^ z(};oB+@xQe(P3+srr6e;kbFr$4_{P!a`@RWnQ>SZP~m*RiV1yPp3zH6yw^z`4))c?PSS0S?!SQ*Lm0BT-r0T9MfY!*>D| zNo-f1>%muoeJ{s2mn18Yx(_lEDNU<~w6A@$dTjukaVYj6lhmMVF&5qFqNV zr8(UIpW(Y178{U7fCB4IX~=H1y!s*^fI3zowt6Kn&T;fE%c44GFJ^_L9ODNtWlqB= za0TF}ayeBM^ioQ#hpgSg0zu_Wc)epyTo`Q_f7N20s2&jafK0@V*5Mt@)q+zX0xUkV zLefZd1VlsUAA*I&{W%1tHeqP+XsEm@qIjEO21=sL3#EDrFgWyDZQ))R@oXfstk(lo z;bnMtto6X+sf;8ob~vlj?6-?sqlL6$wmD|K0$&mkh_|3~rB_O%?f6|iVI4J;Y6?2G zG+fef(G6nrk496%I>=Gc>Da+UaRAL(v~kDMqnA%Lg6ewe^!{YDfT56*UJ_tC8FF*D zVD(*B3uBZytPM@OR9(P~l{&^zh-CwEj$>gW3-EUrAvg?&|Wm$zy}Zqtxw=fi<`1l#C3-osLpC;>;fLq50Zi z)VWdfwW9E?q90q2vT)~xkHve-ewgs_mVTLcrRKDD%TLLeg8zS#m)16>Tkw?BfvxkF z`uuMN8b_d@Ze{;a0|bx9pcULl`Qdc@eG|6+WrVa^TsdsO?qN*&{QA1!Zwd$&^uAyK z?$WK5DyIDJbm5|~>30^jzQ6!2Tn_o;^Zn_<-6M`&-wFrcm zkmJ}#i+eE2IJzqFI8mzAlaDcNpV7V3c-ltsTAx33U%Mu@oMJ}bIyl=CYqW=ZTi{PO zDkI5k@$<8(pDV8sPUWOhQiAGg_K)Ig?3aOzqR-$8xSAz!udk*3#f`?oo{dX({sX{| zWDL^tUpYGRsiuZ5l*w)_7lx2|@o)me-;@WF&FNhQ)0P2-Erl31LF4hlV*q3f(EerC zBlmhJANM^`2E)p)CjsQarvGFM^j*cr&GH!RQSPHyXgR~Xfo%V2uya@O+WSr-)>_nT z?c0yJH_=)nyU49R^=}qx!zEvxz*}~PG8{R>3J6wD#xI@-+?;&cvhX@YWvPj?6inM_ zA-BP&AA_hoQu`GGURso;pU%;n9KZ5>?aXmV$)uBXnWrhjP!v67Ql0_&4xqgVYvEtG5sU+9xzOUnNF(j~y~g>VXW zT0%l5d<7h4aU!5v_n<9mR%C-7F`R;(N2W1Jp+hp<#EP*UziNRGX{;&UX*2aB1Y&kI zs2++GQ5B_oak#AufM*e%?c}pj&q~2;`&)F{;{Wd0s1ouuOV+WpEApFQn(0$?NyAOi z@4Efln(|%{P!9$i+zX%ma`ABe3+En2%QGqqI#Jf10u_#0LM_q7J;%N z7uR3e5U|U-A(<*m$cN#}p@f(HIRdJ61_67N4^QX(0kYY0q#|*Y2LeV)0!H)TvUuBf zR$S^^iDZ`R*fV8@7gf5kDBxtrQgAES!^ zh{Z}y97+HFOD|z7u;|etZ=W*tZrlD{t_*lHV4-!$4P*0FFo^v7fdegs(%wFDDDBfD zh|-$gWJ8kD+Y^G_dJb&;-+l?JFJ99~<|DuFm$2ueHI39O`&o!0@#D8c6Ei9oG(PN0 zTD^2Zt46);|Zow8b$i5DMZAYN-OaQj!T|<)-(svI%c4muZ z?D)o+p|@D5IH+&W@qHOE>^`$;@bIB^6N$Y!zb_L~ZI_pq<$X~>o*n+({h9_}K%gb~ z!8@~;O!wA3g2`H0j7lovkCr9gC`!^2`1t8%i7!psIv3U=G23L%ex^ZwjFZe#b!pkF zzJ!nyA+937v|$JfH?bXV0sG{|s1f-Wjo=hQBP;tTe3<`~`eYwDi+~#S^V>o z#F$XQLdJybv#>rT|C{O`+m z>oy*Y+KTYKeG;3uFAo1V|MtY!0=@TKF{CMVSn9N?Lz+$)mbQ^T>`?eM>*mdatUv2r z{`^XcYwf1GB26gvcl%b8%yDUeyuggfD~`40uBdi?y` zerdV_$Sjt1VZt`#uV+w($M^(Q$Z!)bVQH8v85E=Dk3?Sru;DHuo2@6a$_dC*V1jUY z%>2T}@7Lr-*mowd=*tE?=#$*yJ^rMrV_qKlX(bSHJkzU8xWjJTRI$?SlC!ojJLPQI-yxXZUX6h;L0juFSdRXahM>}%kr zPW^V`cMTVRL+Uk?1Bqa$;84=69ZJ-%I_`GU(7hxxq|FY1Q!E3{V{CQ;=kdZK$vE_? zv9WBzIYhxI79YyYqeNOjR;&ZX%k6|!7Sd|(OYNs*sU0Y`=Ugc^jyOz8)Rra1{36B4wP40@(7L|#c5KPb#95WzSy5KqS0JI1 zin8e$qW#pYWHAJeA`{9XSSmfa58>2Lq8|ag{gL_+#&HlIvh7#fpwkm;cbrCnOWRqe zCq{TiiafU~7=bH4(6}qgs6rz-NFmwiD@GX*(4YvrRhX>U?tVU1p%ueN$`bWq0 zGdkrhGB&!*OUB&%M7VXbDwNdO1p6>w3P)d|9J5~FC@&0Zz^w1WG;_2LQe0!dvqaH~ zWJU>`2(jH6Hpbu;uK6%VapsrVE^I&u?hcMqfO=7ANACDh&rv^DNJc!?T=aHSHbM4p zO|P&_Nn+vgFl`aWArTP4x!y+Kq>vD=4ec2{=6_5Xhr4B;iD1S%2-|8MT!b3^q z5jBljV`D7=kW%LPtpokbsk-}{)sI52e`?`Vr?6yjzc?ZpT$oFCJg_R)Q>f*elM|+G zp5&oY!sPsSjsf_3{JA9Iw$X3;t)NE+^0IkEZO7mtOR|ocb;hLPv`dRq$A#aHi~lYq z{^Mo7O@k&CA9HDO_j40gPhC@&w!ASApY&x`(i>Tc)2>)km$0tx=;e(!$0rS1^Rqyg zoY2cj=_~Wnw){3fu)iSiRsYaKN$FEZruQD1@^nl3BHxY4@lz^&m8VWBJ}{~H>!R@T zxcH+hea$fS|Hq`_uf`{2T)L+2{k3)PRyD@QCmqu_>B~Net0%6h`(st(esE~3P7XZS zKeQteycPaS%YB?o{R0nnoiX=p(UxsyLAW?BzILcDbUSjURWztN&bI zs30+YOU5aoy>n8wEYA{HHDHogmnQmmg}M;wRaX1ie6Esmv3}fo>PYVf z*#viDLA0%NN22B{c<%MVVGLw?7Y>-pj*r1htg5PaExt+6wm^k)-bl6!6>B8g#2U#^ z%3h0>S|Q9r`ZQNsRd1SmbI@@ozST-}RZM=3+fCa*8$Rj1Z3kIo$#?Cn0J65@<3GAo zv@u|7?@RixGJgVT90#;b&PsWc{%@>X6M8dFw7Jo2xa4ND){?S6RD>sD1(;@^VO8X< zF0jagG2#?m@`tfh##UHz9Ktq!UstG}*?m}OO7DG6tlG7$7bKEqQ?3r~t+JCR4iF*A z;_xjNPEJ0Baq}I(U;tqLl3dVDRLI5~Og@$jM3&HfX^iH-=R%e?xY_!n0`8p!NjY1d zVKDW$h2(NKSa@qk5;^4G9N$wr`lBVwj>sp`0)av zDv1_ABDoz*R07D(!(Z1ujkTdt9!ZqnPmcosu;K$zvaU-A0@*RAOyNo;t0 z{OPlve+2ek;U_m{asRk0r=*pxxSV37)*JBSWBeme=qYpG?$~pV8wM65zw5N*?=w~Y z)(|?RVtd};?MjBvVm~n>S`iq{;U{Ltztnz8yq}$LCBy6kq@zDH+WSW2BnVLQ4Rv@c zDQ6X<;9VlIk?WIS9i&cvD+a<5$pc`C0zK)DZMX`TNYSPm;)oz`;e#SJcb&twPH*-g zSmHnr^sAyXFzc+ybzm;B0L#`B{8r2Ps{&X(f;d(GD)Uyc$skDZB8F8u6YwW9$L4{4 zQKlw_)i4>wDX?&AWpFXz>?bOnqDNhYwd?9o+)Oc|P9>=E<-wKJQb+9)ZPpF>E}TNj z>nq(~u8CwIdXkCJ_sEK+!rgu$fk~${SdP4U5=b7+qBuM{Q08cZu!O^1Uu3TYFw3TQ z7_$@q!8hb;_8HO3ZR84kgDk(mgS3GzIp4%7qS#J$%2e%Lnt2UCOX$rJq>%ob#OsSm zsS+;s^*~QjD<39H9V3@hPTE-A(xY*hV8tMU{PB_zZl})56ypMiVSxoC@yj+@sNS@s zWCWLZNk+PPNJh}X(DLGSf^e;LCn*lHR5BxAkjvuKKa~s1H`i3)+eGBP?s=hs|Bsmbw`?x5) zxlcky!J5Xwu<2h2{P@o2FC03#wy87vNeBSlmX2kSO zvxc*=y}R`zJS*}EMXVgjzr6pt;J19g-FX9g=i}7dvwxLI-~i=SL=|QAe1i-jhlEo2 zcznN}x2|1uppEd=`^y&lANb(nrGtK-Iwo|+$)B|iZK-{3z^^ZD>hjAj34ba&@3j2k ztNX5aqaXiz)>)x!zQe7esOvPeXe$w3_D=Ao;n?iw=?`%s*w_~YE!w{EY|#eGeC&?; zwgTnpW2(T6ZqX);O;|X3IoPx5o-)byuQ4Rp`pwK&m}HV*#TZgUCV4HvX)0mc6yU$7 zwp(_S^6n@6i+B?c$;RU8V_}1W;Dh52XU!aqH5X;iOBz*pkGm}-LWFMyGDO}fd$uhO zpJf57N^&59jbG-+sfG7SS4i#M|C1Y!K&!LYSizRE|9}K0t~Bjj57|>13gjZr~I)`AXrQ?UEG3 zUXP^LA1psEy-_M~MyK$lU=GF)%2Ke2I6{YWAlZcDz5V0uF$bYzWUmdf9lfmq*XnJ@ z2{Woh)||SNtHw5wfri+?>IsFLst2ryqxH>x!XLo^;z+M(193k>X(n(TK%9(DA?BIN z7$Y*G>;1!&X1w9al<2JIQI|oU;uMloOSZeO978PD^czpM1+43kac2)P_fQQG8CYG& ze5~QJ#cWi3fevz@2VyF8U~%vR>#PDpts!C?5neH);(s|wb)ws?(h!a_nLJ;0(0j?> zJDzuzSwMw-XzZGUe;yYNt}U6`)G-@Iep>7Q!!(GyILYonZR!+vp_{#c6=2!s7S0Ik zrl4zMgHKApY-8&fZc?JQ{Qd;?8xl`Lg9>#_TGW=4`)gAoC-@LZ4pee&<@HJ@W=H|x z6c(iW{BIhagb>>a#uJ&gj*(w|;?ebv3u~=$vq1|vTk%<@`XGtddAVSCYOcG|-yAAY zeh_-;P)|Xg0#KHyK4f_mMLx9}Zg>%MCId^;iV#EW3zhmSc=6l{bC8e3CP$pM+awhm zrE(*3th)mElB+M2v))NG53QNqhhI-8Ovyru#~FrzPUG+EzCxhl4MDnq%?kQP5^fZw zQ|w@YL`gfV=Lf_?+Z^v+645REXB)0=x zS$ljhUI6qg6|1*C#v3x?{nz;oF#sR}D54eoD_u9>BDe1byb4OJW5k(n;G!s&m?9|& ztie^){&|CZYmEpH^L;j!u6ezmFiKq2>u?7yqDD0ll)MxY3)pN=DCohnZ{*+|h3r8= zvB;!^y`nyg8c;^3!hr8YxA(}Dr#i-NX(jG-WMPXm*x0|)AAezL*=I*gFIoTf>d{5T z-%Tq1wkW(3{v65i11f!kLBxfN!b@Q{*(3h_x8^sd#wXp{H)(v|#A%b()V;m7?yaiE zgTADVeG)Uk!oB(^sIasxs|4%TKlFkm;!wpbO?Y{hQ@3{C5TY;N|F)pr=RNw~AYkVsT+=!HFgRy^y?7g-X zKmOD{B(xf^HZp^NT)ICm+cfrfw6-lm`3Q1h@-~}efNK3p32uJXtJ%yoT zD@ZtQtN?d^tJDoPBzXytca}}lCPjpyHBb|n8pl*uhe|n_vc#j6xhPd?M=8tXiTx^i z?@`!vn8n3SE5z1?Z%awH*oAILx_N=Pci27hGcS>(|DM?HkpOX@89`j?4$Fpwq${zj z+>kHc;X%OBOAEIy0SBly~2+`-Bo`O#Un!S>5bjY>M+c_ltYx#=BhvK*k|4) zd(-TWr7A&<8@9A|3b>~C&2E}m&Jj~)Mt0MRaxSn6|E)l5C=OAGeN@%ng_xH6z8a;I zA`)}Bmt_Gc8#XpHV)riIZvO{%VdHb=KS#U!6O0bOura6YXB+s(9UF6O5kM@SQxb67 zY&cuR0k&WayEAGHX)CtfSN^-Y9-o7pZ)DhDi~HPtymgN{4vr0IK<96@J3^F}OhzK( zEoMzc#k;r7KmTu7q{! z_$JYqA|f1%7*|O~84)SQj+JC)Y)my)v_mO1XEeDc2OmRXcJfpqo&~}6Vu1DGmTW)I zW1h5~_28WKpc}~{#^ol`YFb#>#i_F%9nK^AI7^hG(m>?o`4=~U&J&!WEMBfOQi6?J z%`itILEHpyvh>?9FS;;`B%0NB6LKoz{SAu|C*qXTiOZtl?nz{SPX&D={Cu5h7%|oY zE7KntO?z@+^}B^!Yi*eP`WeX*Xc37yw^TW6xf3)~)6OKLhP|kF69RF=uu8Zr=qlTr z8?|dKgpUa)!Xh7uK9-!9^H+d`bxjV(qAS7H{-|W@Z&EMg5_wmHey0>I~uK>90;z*|M~}l<>@k17HW9|Yz9Y~PU7sODlQ{(YD2>IL(*;ENWl@#S#);$cP5OdVIXzC zlDE~Ll=UY&HiPt9@?tLNi;VV-lo+QKY0#;7OO$_As0YVH@V{B>a-+~{x>)IX2 zZ86s35fyR~=P?{n{1_Km1y6%4`qd~#j#q_={odVn(($+T!Z`Bi_EDoN$+U;N9X@~f zKCuqE)UT~WyTybnsw$EXCD7oHR3;>?Lzd@r`~kbpQ`WBNcGyyxou<}~G=##1C4{4x|yvcV=jg12)U{_Q3P}sv^(XbSvFiRBYwgm#nSaJZkhL6k&y>!rZY24lmWlfI{EY^6BLD$5~>)8z}K> zxJL@@%Mrb?ZGuzSzz6}D9@NO_0k|txm6Z?eAYU31t%ZsLXI}^}uvl6%fMjwPqQBUw z8fB3uXJ^45|H3#jYN87Twv6Bui>4zwoT6^&R&tD?2UY-dk8=BFRRq@ZXMj6?+47O|IsOQjBWI@t?L; zp@?nvqbM|?cSc5k*!%IrV>1OzN(3ZY5JoCQWQJN zA+o<7>Uxftt^E`WqfJZ6v!AUW)vN4a7PG$JX?;nEBM*s#$a$(zltW{w9*L7sqc_Y6 zR7~z84UfCMJG2T}ZCZ0&rQlfnWaCH=-x05p?CU#|4l5Unea5J(DrxLEWI>mY&zNvy*7 z?YZb>8&eZ);_xaPps`+LEZ1Mdc1oYj1#JGqr=x)F)+1NI_NRGC{Q`d}2)x%nbX`*V z?%ed1aPWUBKM?94x&TFQ7o<j%@nvh8aY-eXBwkIXn z7_{^cTc|ZQwO*sv*wlJWw#KFsHmQ*GzfsjFo&5c;2O3YMs=F!G9}Et?#@3K;^y&F! zAJ!cH>CDi(eCmO|J=@2uA6d73C!qyEG~%D!Alj7b4+n?B{ClqLvA?}^B)4V`FpxO? zHQ(6+ot5$N>(K5kO5i2EELWV_lD0W%(mrpVnW6sGbl!VxyU>e-~ z$0{MEqk_%KmnW`mdcdrt)K(-=5l4ME!fK_QtXTpl4O|kFLZier$q^OH#Zph1aHSFN zLGid~lt4Ibd<>jMbSFj2PL50+1E&d0XkkDTPMZk+Y-()&%)=-Wue}$4_BZCwzVhVH z=6UjGUzz!{&n%lrZ<7&tjomXTP1sAZ+FpBb>n|0skKwlu8fAiy{sUa-6Ei}Y9$hYHA_+E4Ja zD_3N+rjEe^`>QL%n;wczui+gj0(Qdp2V3NyK44DUHR6y*gjQyUB!@1_d_}gni`v(V zL~GYwsPbr@caiK9=N7(EpSY3l?SsQn)D>2~6J>1}g!|%QH0O&M5LW#EQ>aCNGJm96B>ebBn;n9C|Fy^58q;x7fP)csK#|oNk(dq-kgzj& zDgbB0(^S`^Q3Ijk*xrVNCBr1sC^ANpw)>dm79-4)DOzL0u}Fg|)=)2kju3DPx=0XN zR;a}}8xeAAU?hC=*Jm8X}BieU|O$Dn2T!fjBW+i z+%4|)EGGE0qP;Z1HDXcEj;~`=&X(2;OA2O)zi^7?_T7DrN-(SzNQB8Sy7rU;C8$)0 z!4X*30DaGU@DW~dBCk-#;HMi3wzGSeWA&2c6kP}ShHSvRf~#>Sx2c1@hc8-fR)Q&{ zK^LLMO57($+}#0O@Gmvqxd-4$3f-M}L_Q2=;-pzds(5=1DHWXJ_tHKA^~{rtDZl;n zw;BmdIFC`o2SFz}M&5C~c(G%k3|L88V9_~0r2{|KWN>g4bi2(@%11IK<-R�f4hd z+t+;g0Qu<=cta*E z=HdWQB~}GB0CGvdd_}8QUN9UD-3-^^6lowM3J2wV)b*r*{2s4v#130XX1p?c=kyaY zU?-fyU2ZXgFZJ$HvaI7Ba2fdHUAtU#%ILrSbN(cCXC2eGq&NB{PP=kVUGH^u$K8^W`&@qDc^4W5ENZTAiwa&Oi#&8L=mxR4y<HDh@~U!cAgmIz%I#iq zzW>E*zS{-3s~BHX!2slOK1S~08H>3RF+-FbGIhdtLkjgsMCX7h( ze2g61(_spR$3b@gvK5;@xv)Od2XW@;O*(akOKjAhXsv*F;Zj`kGR#us^1y(Pqc}Px zhnN-_dZRZhkCz_S7+5W|amhboTSdxTdMv6fd&@&_(+(_Piksg**~)psI2W(TCJp2K zC9x~nprvdY1$1>8?|qk8`S8`3Tk5DAn6D_%z&?jIe{Hy<2M(gS`j9*ZW|V9p)vidT?;k%N!Px972Rd*m7{2Dj*vjM zox^HKULxLx?CQuu1qC_iC&$g>jKVYx+0b9^P%{%j2I@p)9$c#@ zsAI55aib1}`o6j;bR%eH^OIP}jf1T;0zb|Ch@0XyaSgg9nqdC}sRc89BLuV3>tsq* zg#d5SSJ5SkXjbAojmL1Rb4~69i>v^OevI^(m<*BzfyM|P(WfQ+lp&Wr6qZK(;*}N; z%4J;fZ8&I-gdqBuYq5wljF+HeFmYP8lGM7I*F)?Qj<~6Pn4{wVO>mxkkKO1 z-J+t=MmpP(7j19&96>@m(n(@M^O7Xr_@m}@obnzK!#WRUA3k8y=8pGRTcTEBJs{!~ z)jTMpf3=9}#006axpBHPmmz*x-wZFuJNe+fTWHW|^nar=Z!quv&rP$3@e4js{( zG69!Tvq)=6H7w6`3dLa4BN8-{Y*tT;J}G)_m-nNAh7v)s(C#xglS6!A=xcfiz;cqh za$7dYhr|ew_3PjXfPvINF?%63+I)?RZT2I)O99N`T^1uGCv)WWAnVM|zG?#;F6yA2 z0N64E6>sprMljO!qXi?u%fD$^UEPuNKz73RtGlPKEKHwTn3nrueqcmlU|3G*^zP}s z3(|LwJ~ns?yh+M(N{Rz<7gWTZT@?H?&{%L_O(3V_sz6-#XD%*2YFzlgP;*x4+k`IJ zZIg-*6@{OIcS&;mqKW;s1lCozq>qk!p*(H_+FyfDn>%pG+?T*|${%o$Lk-eI>Q7RoY7|TxST?&~Bv>{`J3Uyo>E0~cKFzY-8iQq{HgvIB zHj;0~RzuDL%T_dtzPMQniG+4+cm^fe0dn$>p@jUf5yU^&W&y6Pcb*8CC3mnOWQ*@jd8@47F3%y1+0}n zap(3bC)4(E1k<+U`!x-Z#irUE6%Y16LGLyAi_hgCH)hisFY!oWr^F-Ot;jkuIN{@e zTa5&!<}s}k;m}RiC`@ALAWg(=`q+|~xIsB~^2bij>FXFy{;sX+RQLwk4GHfUPX6eX zjZ!va(5!S4nv;;1}Q z=WsMBZ8icC&Iu7O=un6n2mzczfWL?FlU^$bH8J$?iJu&$nr6~+s%0D!i8RzPN8Q$* zXDtwc-W+E$_S;BGN2Ld|kEsT)P&!7GLdnQpC1a>XovMY+emW}4L)SxKXpWiHqX3PD zB#^aaJDE>rLP^?;91zR0-BAP)%0`Avjp{dmqn$;in!jB zAo<7|ZJr89yB|?Ek?McDi3G1e+1fch>ZO}Vur?05#}h}zom3vo>+;)$ZPa`X?Xp{I zmSlyfUH0JNDY=XDL$#k2g{7T8xkb)`P2?h+{U|@24p=#1>t9A#?3a778W*Wu(rw#* zqSJm`P%ryschnTr*M7;Zv+tU?_0)E$EOhO<)-;gIrH(CRx57~aR*quq(vOcDqCY3YMhvFNlcfhMn%w>Uu?d#Pw)e-bzkf!?sBN+XTjp`H?^@x#s*U!UuRQcN zw%d%0SUUY1WngkKAHO1xvE821(^`tpwfMbqI#pOQ97{1eu~uhH%xe19GE>QzB=+yL z=M-$ie!FuZ#=a`N%%U4-sb<=%a0=kBHustQ`dI#uxJi^c^559o;jRA{M&NN8OA28Dr6w(+3{6MRpxFq;@|Y22S{J45nCjJLs&oWb<%w+9j%~J*ON0PhY+z)oE3CC zzZNOM#Jh@xz_T*Q%5}RLzQEj)0H-~omXxy{We7XJ2ny6Hcnh6&l@7X1Gz397B(hnN zRJ5)AIa(eSxX=+E)RSJ`-l$U(t+M3ARU@p9@uvtL+3*31s*t3UJIJ8G-3Dy6yjntN z#?P^x{w*Q4`?R%qXnW#e-D6X25PAyvdw79kT=Ing$7&GNG?P_T=u$@(L*n1lb^#Pb zFg^%(J0pYy#Y~E@XqL{J>bijqA!RM%a(qQt5D8ZyHNzsu$dP+f$XSDkf4hws88XX! z=-+N5imm+r!fk||x`bZLhIBjUHqs1;ZAD~qm6tDv1jjo^bhNei>OCgqvCYQ~9I|*| zO+`-0O@X+16>+y01+T!VMFU|};hPAf|HN_Oy14keQsQq!qc*tw`z|f+IxhTTT>Np9 zR@5D}yz!d&q&G4Xr@{5Vb#2{uRgL4&s4X+;n9PI^j$Acc!cXh=pZozUsKA*hqn8=lE&`DrC$>#wVSN} zi-Wzc51vp+P_FlAK0fvO;2H1EUd@D)yB%b9pU+oV-PE)^ks~arSVTuQFj6i;$iFcs zHgwM{i=@-`A&NSqQ|GMOPkdEuAtWqItOeh~Y*j42WW^%;F5V-Mrri6T=m@(tC^!Pg zv`snTl|0qdJRe0=@^6Ke|A($n&GHiA5sfTR<6~I)lX6qmyE{M5vhwdJN$OJAzGWd| zBi3o$PQI2*1XYMrz>Wt#xs*qDHMPvOTIMTX3qPY`Z8UMrlQ-GG{5SLN6|@5chE5GklgzEHai*#|Y@i z{>zlU<9inO5>6sYCnv!Ge;=u|hsx2+-9Y9gt1EB`^4BplzC#2br6OC!9$7(M>FtR7 zDIkxoTYHWod8=18a*mxWo^Tm8Fw3k1v$B#x{uN1uQiF9GxyYO7W3S38CfIO~3c|ZF zp_#cZN<*kilyK7s;1aeSK7PYZByb#Tf+?C3f!xyHk(^1dhlVHWQS+nc@(B2fv5?4_ z&6XXTk@h9M*rT7z*V!BpJJ=+}o9V&BsZRQhtp7>;M)G3%jdYxO*WlmZ)_+`j{N`Wn z|2BMh*2e6z#}eZHp770O=UtTk{m{(8m((l)6@+(HWY8UChDII2t6oiQEe2K)=C4+X zqJgwhMOU7#k>aIxjHY33(^|%czxa_kr%93RGK5XM2_*l4)DV=oW7ehYn zVI-=Arjl!(O z@tNlwhHCLj9t}8lA}1qeu3_J#Q)t5=#MhDIr;f!filgOj6}e4` z*p2g49;$Sy>qPl0HsC`KE0Q<4!~$ZVlsV z<1R~@_141XsYT)L`6(keW*3FO$V+Q0>=ryFbztkexq%;SmS&>FBD&<3!uC zo_vgG;Tcw!GuE=BxFY(Scs70h(0yNMR3+!0g}2URLzBxgn^Bh{9Y$OglMrTlC?cuM z7|2`OdUA~9M|lPxx5#7aF`)b+T^eiW85NUg)>rSUeGJK6Rxk##`Tl{sqGQM8q3HbjRx z+1HH8X}e^KR&s^OhUDdq{1nlht!d&5ODy6`kwS(smZ!i8#j3N&!4^G36e|TS?Vk!x zH1<%mk&1P7_)7tXG#N2sIS$1TEnx!F-yMZ>b<}U~8QWP!pLDOckFB2{p=oS|2PCkQ zauCs9!~V^HoiZ5)$Tsk5(&>2!(VqlQ(c>oLJTA_%f-76(Yt7u^USEB`CXYHhd-6g0 z>lxx*ki|lDX1IzvI|Da&1|)+9d(uPci}i(6V^)4m0&<7>2vq2Pr5e-2WgkK6i7AS( z^aN9KF?kB9IJA$H@<44}Z>4aUii1);kAz;)Qa6=kg8srCtyLj=DO*w$j*q=4L=e1= z6_s~J04>)98bM{Md-fPyFB$d5b+9Is)s42z!1+Wt zjY#ho{k$wAMSWlOgn4??Qi_o`XsGQ$zQ{;sQTeek*7xq$I5etXqkIX}dx9)pN5{!#!&xXujANcdQ=ygN? z23>Th2ma>B2TgCY;GsFA-F*3p?Xp#!%(g@fB#A^ezu5Xkj**PFDN>I5XwzgO{r-DN zKGj4iGuuf{m3ES{$jvY9{^u2>wf(+d!k&xPG*%uvF1&eBR>{CX@XwG* z=Jp6)0eAo5J%Z=se?un(?}qb#R=_TyEIk$ulv+YL9TLjJWG6XY+DT4_o#ZJ(Lb($r z|8siyn)^)(Z|##a!DaDw=+;4j#2)G6UKr@GD5na}HS$SFJ z7c861gAvIo7jWa~58j!*1OSJ9{+|{m|7|Oa5A*SNm5E2E!1w>0KKlsa#OB}f2)@MR z-{24;gnc}JEtsYpL%_cYhY$l0OCh!k3wX1HQPHwUjsY%CICdtt{YK=$pp7M0z>l_L z%CIX~!*_-I0F)#e#vNrQ!F2U=D92QBKaBiKQHa|)6No$#ez55 z?hJmqM}5&K{05G!`RWE{pbf<-q~!h8RiA8{98*5oNmr8g@b%WG zY*KK}X8#Y)sC;|RqLL972i8qeZH41!k!BK`4$(@n2|1#bM#R=nf-|aKJ!MGK;i0J+ zSJ1<4N*tE5<51Fo-0V~8yLTUu+kY$!BCtbeTxmV}C#;gW)mPzEcGGhd4cW5c=Ji-I zPjJzp9oDH!lgp^CadHclkFnBY3LHwjsE(~zTgTS?{8ac|+zU*PM_v-`yxjAW!fW6$ z-hDanv68D@cOZguBy4K|h)TsnpU@swJP?$jH zieraw3MEkS79~v(Aro6=8XnW!HU39&J`!EuR*Sz%|L>!;{X+SGfL z5Y@WD3|aC0F-s$O+Hf;a_(L&7`J!CrLKt)3P(Nk~BylT@<=OqvHi29sufsnvT9 z;FYJIO{&or29U_i_UkZMP+VpXx$yZV2v&Jfm|x+V5yQkncBsuGsUu!_@qX)(Qtx{l z1fMCLS{6tdY9#%5ZJ!D&Z(EI}eXfwXkl5p>Gz;%D?xiN!@xwV+mp*oRhsUnJ%<+ZBndm^>z5&Uu!%iOM4A_Z+7rd*v=KVRKLG zY3g4s6p~81smz6WtOn*6S>zZ+tRXjW_5MbN9Qaf55_+r#?rmHxqOHA^h_<35MzsAd zJo>WQ*N+rlyXjwp*V=y&Uc2hPyJ~&?g~IXn@in(I9dq$^-(+#>uJNslC)_tbk6J>v zjqkRNSg$--l58K&g<;8e6}7&u6_co(;@^*C~2 zLoD;1Xh;{{kS`$fSt#l`mMiJnM&id5`VpVSw-9`!P&v>o_3@peQ-MuQ$sqy;1a=& z1&MeP*)a?wJ^E(DY~9>9JLTX3*_n;Z=3D$C>dH|4*v_y5vDK5Y$(<=bx# zXxg62o=tQ3w^uLgx@*g(;`;u#WpAk|u0Q)W^g9N#)iCu&8hUv z4iPP!#~aO$vLu0lKBy))^u=fzfqx&j9g|~_jiSU!hNgy)9jAce%8lp>ff(ns`4TVf z2nw=dS`XBdbtJcysvTZfU)`7gymr*K`R9tp$~KHP-81P+miKOffY=%ls|!*I@3SeM z8uFtBBxSX)FJTV@E<^V}@^D%0j?*ZM*>=N}wDxnfZ{!&alg;6TNbA1Hk}+2dTm?~P zCk*NFIU=G~5ww=3sgOZN2d$oTF@Tw6!0+||zy(%2Lxy$BKW%CFgW%xh z+XCIwbd~ar{4j~lb}VS`Z}fjt$?z9($;lj%=8#RkG0MM*x=32N1072_BD1@=MWh}s zHG=wjT+-}sfBDz3{r;gFgW$^ytS~VVUFk0#`K=KMOVfcr;){E5@KU^KYdRu9he8|{ zJa$BMc7)6xxkl-p4tKe^V@xh6o?m?NM6`Xq9ybbiz0}mK&2)Qyd@Dj}SQb*uFSFRk zXq#2psc7;6QNCSCZ5n+{NWevnV{L>|ksvm(x(j)y@&1Aca_u1o*@y+uARCDQQff}i z!3R605VT50$m&7CIHS*Vq!wFp%_;{$e+rhr=t4sa!x8rCv}oi-SBQ*cw3ldvV5>W< z=4;%{H|U~LuSBwPxTCoTt2jWE^@x+gw~JKi_k(mb&S8^z$dkFPbSoV z{eE2hfYkWos(i_3Oe#*kw7AE(@E_yiUrmXBY0`qm#i&s0mzXhiOKu_yLu^!KY3t4i|-&!f>)j{QGZ}HKxWV-P$*4eBZ=rlh)L|y|(VHs>Xx9 zq>X(NGbX|%^2)T`L({gbB9}<=`G0}hYU96@`p5aFwm17FWQyxbg>S*Xy$auv zrCFg(UB-p4hVpTI|Dxi(lftLx|8DlkFQJ0`r`3>K|8m%J6LG<4J$i)66dw?+kQ9E^ zpDznv*O&26Hx#v9&yvWRc_&WTFSUB1Z$x*u%$^%FnZ{{9GjtRLk7bLVWb6JIEwrz? zZ*?=#Qn6)_{)Uu~!RJPSGrJJ-@S$6}xZ0BZmDTHzLN-iD>x3=I`DaUtS#oExZ-8pk zMy3a=&)QGqvE<}LVmIp#3gtn92#1h8(g4N?lQ7TWT+;FnX5AG1e6@(V+r0sHUWL z3}OOHM4oUpFaUG@a5npgDh%#F*tpvVou24laXE!{tvkT4-OT)&Ai82yREY43gp;b< zkYSt&I}c(Brr~+1>u(Y|&e#!}Qikbx2q98ns)haWml%a%dcJe?ER|H|U<5{oB0H|x zGuZSo8)^F(kMcIbVMT_ryzt2=6b6J19G#`npApSD;1raT@{RQYF!7v@96@}JXxkR~ zvZ!MvzceI0`1z3?9Nt*^TchWi0unX2FPgab;1@Z@*jHS~7d_iuD876fiV6=fw<*haEJ9;62&9(L z$=KSX@wa-qkZE?ja{8;W0WnfkEhlFCpK!4jLKcS;X6^H~J`C|mG@MnZNLRA&W=p9S zxqx7%6TQrT_q35RBym1j1^Ad+w(_G+LWqTb#l4_!R5Q@A#>#vc<`gb*U5UlZ{V5Yt zG0Tk6S(!2=_Q#b}!D4QY7t?b)@N{Y3m^I}^{&VHUHoK}q*&nhYEixaF{ z{r-_#L?1Z{YNK2yaq8{auX>lTQTbm)joci*8?#?i#|Re?7bM2<0T?#4iSuIE_G^ai zPlTgg7`AVmjG7?r91I&d!o&n?rS~Lt+15U{yU6Djm3<=X0t-u0Wr}y?r_I%|;hHqp z`bjKFUY;AfE!%|_An0iUfP9fDFDmS?Y~bjcj)oPY{arzlH2}Z1Qir$GDXRMl0({3X zksLyqz}(ES^@rwU=l&P{c74y=vKF=U8edX4q3rnM#+O{U)VJX#gcl<`VOz3Q58;S< zb$Dv|iDZ!(wk=g_Uw^T^#*BJZD2rl`~ zDppbXTZf_sw)Ey6Z`QWf`gj2gKUapY08o2D0JO}E?54rV+!QQqb<<|Vd&H$9a5WTV zRC*s1F_=EKDlgm;zW#@L@ovxRuh?kKBN_wFvgU%QeTaY_DJXxcM~n%hKR3j{Mh3Wq z3#CkIOR0B4$a}Go1ES7ULI#aXL=luBA1qFFrrL`BTqh5aT0!|}TNI5;%xD@2habb# zF)xq&RF_E^zB^8##!OmtXNzK~NIZsPKr$r;E))sn$T05Th9i*-<^SIdM_j9@nD@#< zc9Z&!=DbYaMvR8TO4G29 z;1nW!u{!)H%?P~o{bpNFm3qV9QcXz&W1drtFhH0Y{;EMjck)b)l_L2sW*lKjVu2qw ziY>=OGZMU9+ytSYeOs!NinZ4J~a)x_x$^=2MSJd?O=iV@6`eD?_guQB z?(b{s{`zQ2?z;TIvju_Q^bd6hLpMBq_pr3w>iodN{X=_+Lc2Kbq_M#jJy8mV|8h#s z3&4^;?xb#`W&7;fVZP7;uxbC>FJbk?YZ{wTsrCsf)lLrt^Uli&HJli1T~pgcOS>o3m*4D5ytPkK zY5X(kU$-1V6=dxzAB}~<2sk!s-RvTReDV8qf8ruhvi3dKtG!y8_`W#qSXq1&FvI8# zm0zzYj(fDYs{jzo5^tn>ww1*ReEjsX#Fr***m>YDK!m|_uMZAm6U5$y1Ev;oUEL*o zjF*0GyB&Bbs!O-b{_Vu?=qARpG~c!CX8+8{Js{Q@O~n9enOvI|`3d?MuRPOeoH)0P zE4L9^5+!cMy&yicFtAuxErN?Nu=M{R3``KgjG@{xQt--);=+2=tWBjq%2>08WV8|T ziQ%R?+3m=AJAFtvD4|o#y|H3s^=K$Y+|46NF*1<9;m&Lkm@N*UWn~5{sbmd`k-Rth zSG3kt=3|UM+&Vf=Nb)L@`zK=6#(!D$7bKE$a}wWY;>p&1uHxfnd5lWt)#I@Sanj~J z3lV2b;AQDEVEJ_Ciz|?YUw|Q|hqPX;X(ZoiHqo|CoEcJR_~u-L{L{!+2{WU$?xs)P5N_nNA3u zp3MC;&zPXsEAPHh{x;fyjTn$NJ|NCVY$7ln0@&kNx`aHxR7ha|N6T$w5F3yQe5OMH z2VQ%Feazg7+Od_vZ3N$&WNY=)`(`)o?7`cGzoeoJKL6LYwS5M{?K>vi*oVwRPX;U} zw)4nnNhJz3Po*lP><^0eC{)-Heu_pdDr~^V#Ei>OuEr+zS=SO1mAB)f`i2{N(I3`+ z7C5%Ym}BDz={RBKXUy;lY5{6UY}gV{D?&%n9NE+ZPML{B{<}K9Nwk1ye-|S7vM6sj2mC~sq}_-%<~P$4)T)x=YD-3HSV73L7k%x-6wbNU zQu;|8Abzu-xF!-8+mF*N-q{}UMowD*ZSDYPT<{pG;X!*uym;S5#A_H^Qe=;2mt4oZ zjn3W-_K0}V=2^P4I?+)PN7{x+(A=^$LvLx%6BUY}K z^7N7Og`6?>Y(i|oBa73H`>y;NUt(z&V7JTh|HEU0)t}~vbK>Ijc2+fRAC|JEJU^5S zpOGtj_$vEN3P02*VfAQ;H*X&LGODE*RNc5V@va^Wnm$`u47DSm#;Z#c^LB;0fV+YK z^0WEuKj=!NS?tuxk&LPKUl;tA8*}f@8_=5!{okJbD`KZ&LA0%N(c>)?t2x{3XYy3% zYco_^^(E{_`1O^ji4}spGUq?V|1x{aMDJ2HVx6Mlo8sL_JQj8TTdr-hGilH@GCgC) zUQB7>lFDkvZ3?t$p(33sYv@W@c#)67JXdE_dxb7mZ5D>B>8s!LTp z$CM@cignCfIkk2|$_H1gI<#_ZJp4}IFHg~(P79V&d#P-0(?*! zjM0Z$gYhvW@`z!}Abdk&#=G#+a;u<_brv@<{}fYzgDS8*{H=BYj@SsTU9kOQWGra6 zH^?B#ioflvegbzP%9o`y#;=8GD?;mNh-JqTswdWa+YW6v+Ks`>vZH;;V@9o%LV$RM zo0GqR5X^lqY#;CoGUF-@cg|m=<*HPp7)*t(=-J#p@45$TCx{fXHGVd(jcPBb`SZLJ zmgBFt{e!$#MIi7P>3C)44sTsq()lDScdEQF(7JOgKJDsmji$bJc8)mHgCmAAGTBL5 zq*fffUAP&MJP_dvk{#x{3>nb;+h{~6BOF-c{~@Ch!9@t85!qWVG6f^@h5)@S`*Pcd zkBhNR06f=Lp7r&gPZenj(;I+n82Y}{Hs9y%wRhh|kHprM4ae(v->W2wkWK z!r3ozVgiarKw#(YSJdg(`4TPbif$7oWYZHLknB1oFwn3P4V#STM`984tA)z!{d zq?&6k@TYbFtw;07`U#mDH_8@~9B6D@SKEB>lj(s=ef?hfU4CF(VPH&7sGxiL?t=7{ z1!-HJ&kvkg7&znItdeU3aTgZ_yB#;MX2HNA3vx=X3B>iUh&!n$xW3oGn)!e59lgxg zeA=Yq_M-4_Q3ID8|5BxIB90GEDsFwZzJ2bID`Pg96nID?oC)oIVUsC%X zbXxmmLU421z}D+v82K#9-C!8`?a;)G$_0%N`;t~KUC{WTFDYZ`LyaE{O~?TC_8$K2 z!3E(Nec=Y#b$R3F`C0d_yl`3Ft7{vtn19P!|L!r%8jr`hsrdh&{W-0RA$rW76g~iJ z$W>iK^AghAulsV^Z8=`cAfP^utSX z`d##+@8$JP>3LF&7z6GR#}ztS)N*C@d_(jBNa5r0{d(TIcF_U$aD0DR;>UumOL??7 zl^Lb;SOEySXA%gkd}6O;6c;6X9Qqf4Ijq%==4@i^7IfBW_4_S}C#~U8pbe+iHa$QR zPes~s7Q=Zg_%A8$Arg*)47S*`(Q$BMVdyJVdr`3OU6}6~Ni!t_UZg)=hoa{g>1U58@Xj%DBEx6KXrE+^{`t@SEsu5 zaYL{^;!z7>ZXgpAhLx-Mk-U?LJ%t)-m0Uj)r+N7RWw z{tKt$i0X2twe;!R77s}I@zn-^l4!=<`=*kx`MS2Uq|~mN1TD_FHUKQ zs1Q0wAKX%r6K!}8DHQiuEWmDP#=pNzofh***5yo&%eZb-bp2U-2qT2&uDcZ4AnH_HA zK-xDvDJ~)F|BJ2ZXm}Z&67S@8&Pe4Zbfaeh+vyxf{Q<4*V^{gj$ ztNFZMY?R-6d0A=kj$|t7YMq83t?q2|Cc<(gOpX@;Rc)0~(AlKrd#a1y>2YR&*Z-=0 z~ z$>b{Nyna26p@RZ&NPmP@5qpV_noP6Zeb)!v)RKXa0UP1ecl*GxYoKHUPS)#-J6@K0 zAg4AhhCwH~ZZlNQ0n%L;W`tSYF{2@|gpeh?&a4KR_@=gNIfRphsZAPAC7euyaB^_4 zaPmMj;baaqY0DNwT;b&WbTOKHvli!a<|uT{UHw*KU(0@k7l?@#3fu4)AJ3M@)G$uD z`uj9#E56GpoXoQ|X-iu^fp?(Z3P33@$@6iF)tBSLr6*Dw?fk-t*6>W{w%T-R%Cve& zINkRCAA9Zn>!RBG%LcN>4F!7)LL}k4@KaPKo&E_b6Bt7(Xl2-Oa0)b?Zy*-FF^6{? zYU8Zj1N+>MS0MFhjgktf&EI~6_B4b}yaqeC>=_!2i} ztA6*FwXPm*Q35l_DrFRD49qgGxtn^l(RS?py$<96kSMBh7IfJ7IquP>jJOppqFKvL zU?mqO3MSP+xCa`kd71B~<<~SqM{r;U=;C^M)CNYpLj|wpjRAdI#hho_&HXi-)EhLJih^2& zm&|UL9@VEOmEzrY8ADYTp>Q(6z9=|QFNPjT2tvtK(HDjPADfSOfxjF^8>Qr4D&+9X zR&gFprCwH9mx9Pa-!HW^^EH`eQw7!>SxMCEVA(_m4pX}Ef4BJvdxLr9!UQM%$Bef( zG<*@QFZ@Tgt$NRv#wIfOzj4=h}W+LW=ler0UJH@y?y?HzyhlogwLtlHH5w@KMA z=7vwrtmyy}Zg|(F!&5f=Avc_y7d|1gW_v={$+^Fd>5yM?XSe=Y#hHbdhhuh^#e6)v zq#sDQd-`YHlYaZrt8A-{+yF1i&aswaQg+g&c5MHX6q*6MNFv(W?HpM+dGwo2_{>Q0dRx!NHLs8s)b;(ZF!%XoISab)R@>d^wF;VEH}~v zKQ8W2GcgX;cqdG6vK$}Z9bEH1vuOi+bs5`>xtE=T8>XTi8)2)4I%z$!4i#=Mv!#C| z5!R9KXm|f5(cS&YT|c-V8*@tj4YZRxe+PU>AI=D$uu|5(1?eUS`5vAP$u0ssm zNFc)Dmxn|{(yOU$<9k9Nww&lR&X8YkF>0x!aSBXP-I8NlzV5nL7Pd|69Ts@xbuEn?2;4IBL+8aBp5tLC%k7O%NaSj+Hc(H`I_gq&y7$9lEr=o|73eb{cd`mj9+ zaoe2vZwYx=klX*Ex0jIgE1=8`6jkxmMf;)sYTBJw_-{0)a{-sU{w-S3gR+~yorN5go5N`9v>m*#X>lM2Ah&nyu&hOaIFv$#DmGkJvFdzI9AdZT9@$XS5d z4r*jNl%wOi>;9WGDZ{C}sA+i)@|X(#phOtM}7MuJnG)?i+WOtXBAMj`ijxt zn;Ns~p~Q1O`-8_U$)!)Ewg%7Lj8o~kLki5Qvyh>;!g?U2+Lkwbj&Elw9+mJ7^jO=~ zg&*NXoJTN+Y{a3pJz7Bu2%nlOJkO$ovwuM$rjBH;tn9n4?@~7i7pY_#!w-#+G)fg! z8Kx;NS3HdPBk`zEE) z*yhygkf%5u&~VHO>*?l(a<{0sod(}aIbi4{aTSzu!25V5XYQ24>|4;&#*`=!-x70G_<|lYbC*d?#hV9U=HMUnl(J#F{}OS8 z&DrRAwvRQ~t|c@{gE&i@-EZuF)Mn`V^$>Rx0>8o6&89IDY-(^ z9?15Pl5{c_^5L=l|FAQOx+9L8ga&Q2&w8h|8rNuYk(vQNG%c@dO?V(9?&G7^PJ8#! zYp0z*qwjdK=a)iB>HJ@PUS`eK6NX17p(YK+Bs1{i_?Xyr@cA$78CnOQ|J!?pX5hbD zmxUVN7#_*R@7@>_xv^(y*&V=Xo$^bn(M>z+!F2Q)sHr^W(c0Dv$@#xyZc^$bkZ}9c z;#)^`Ea{irzqzSreCzogOPZ4UH*dj@X9(-aC051TcYi&c@<2~UYkmOks}3!P2c6YA z29(uXR00YSvhHY*92WYJT2g&CXzvGfk!dR@_a>6W}nlR_3I%+CM8<7`hkz+m> z_}zzDOXdUj$g~8FwS`5bo7Gdt6kHQ3C_uwdq0)?@o zk7Ba6?pR8X3-M>mX-VbO8LqVlJQNYJi%!36hqf1A?CS`Vi z3|@Ze>J;1%j&9H3<0g5Ga;>P=B&`QcVo3C*Izf;H;Llq3KEQ*Lc$y_eddwmqx6-K- z5MX!gmR~pl0an|b z-s~^@<_l>d)J}i?Wo}}pKWAoiyDSZO5_BCkRZpGN{kc5vXk9s+}b_6e}ggd?= z#K$RFg5kJ~A)Lc0qO1;AdXnt6>jJd_r?U{uD}^McL`fm-<_x~EuGq{KdevA$ZMA`iRclj?-@2)n|&wxAQ0o^&9K;)0lXxK z+zh<9Rh=Tk7Nsi7XQJ6NM74s8GeB&G`G6Rq*0$DaeoOThoVy^+UK*Y=q2<)5RM9Xv z1r+Ae->M*0uCYu}ONq6@U3x~QKlV{~7>%0wW%YU^{uS@UZ^Q2W((%uQBV>nVd^JEh zTE7Cuf2ci8X4}zhqvX9v%8M=vURjqI-byS2SSO_h+e_exE#0cP_Q`%>5r~3+F*vm~ z?I^h-wTJQX;L)Z2+4@m7d>;sHP`2$F0QC%C*5&Kgt7vXcfD;!~Ozw1-nv3)}U&q2K zzh|ArB-h^scOd6X1wfP?FXWQ&DBdY(9VM}txQbWQtTKgY=~%4NT|P83e_r2g7mq1E z3;B;3bDRwMsur|v+|$G@+9F4!pKa8h%_4 zGs*tpl8<(07G`%Yxgw>1;n2<_3eUf^qzq@y>s-ft}S^^z^n`Uw%p&jbA)N0OFF(z+*8~0BWrWM z=Ax{&s*$zTHjjU4z8|25ByP$8(K4ot=(}`8-<>1i@xMD5q1CR0c?pXy<+9+xNVmQO zW(u5zfkK01gnm=IcbKYoGoUz}g2%tP;7t|4AzlGNX}On4+VA=K2U%*#bR0B}QUt~_ zM@!Zywjt{jUQ$iqFYF|M$9B*z;l4y-<&;)>^Wp~ou zI!2011my-=jUub+3*`W@(2KYW<8C6Th^V2MwU_^4)e~Qkn#;TM!;PdJJ<3}t2kpD} zX%!?pYX_(x5qWO}|B3v8cUzaIz?*4pn&NKs>+?`5wKlq`8JIP836m)tTSO;=Ui5Kh z>5O|mBFKe0TMj#3()1%Ru+RKONGx!is%euAFVyQ5t>n$jGnZU5076GeO}XM-HAITu zN+F`!b^gxgGr4& zjdow-AFCOQCxiMuT0w(TSRuw1G+8l9Ck-A@^QjA@MWq#s+DA1I)=Fu?hi$|=SqJW4 z%2^B7d~O?QL(vi_>py_Ye~xeO3pH`gi5ryx`=`i602qB_B43i!2mkOkQDh=83;8Tq z8Vv(UAY{Wsic=CK<0A4wwZ_*C$4~%9PpVf8p_os?gWmN<%CXh3VrMnv!|cP#eJS&m z;6P0@d5Qrtrgw!|!v$HGOI>Na(usg07P<=u$GeYyYY{`e)sdSvWi#GqNn^ zg3%>=;3o1rU@DkHP9IaSJ3sO}7CoN(P-rXDX9c@+Z`?H&9*ryO&jT>a3YSR7ks`Fv zt~{lr{)JFN(arfK?RWT3{5meBwg|+#_1BrQyf*J>yc*jcqDSMH29tSARIYsd08e>U*Q8ddb{{tr$;x zq;B=+EpR@A8nP3xMZ{kX3B!AKLEO&m+_H+sRu%E_p)H{fR0(zRL;E(H)sTx%y{_c6 z>q^e}%dN{<)OYfkH9c9!*D0gRKw*;6myZ!!ekD!wWguqpSIQYAuHPX6pA_Gv;aU=@ zF|l&*8Wi0?k{Bni;fQ`4Jn9GFN!;L81`=rkt3+8WQ?YHuI!9 zHBxi>s392u<5@BwFGgZeUJB0|rr47VVNho=gg>c@GMsz3ksxQP@?= z#(yhHrEm{lllA`pz*Be#waM z0t>NazsixrwPe#sA8Vfyei@cOoFWC;M~|?A3>A<*xC7gMapE2O(N!K>i(f)*8k)m3 zo>16&$6{~nHb0+W1e4Fg0L-@ni;L5+`AA-YCy^VNt-_k`r*Wz%=RzoYi8N*O+vKP% z8&|WtLpeM~*LCzqm z$Q|FCh*z(_Qea~L3PKt<8DqP)?{qTpzXyZuqN zf9sT)R)CL}QezRowXf{&k!BmsD+d^(eVS=Efbx(zSy>N9o$K zBQ9M7hZU%Ny!BqDr~uY}5B$=6MX!|BDWl0cqqz(}Ue7;P%|7|ky;cp}0WN@hAg#;V zG`?`_;F@CqwMyqb4GWT*&?&;szGQ6HeT?Nsmgl`xIyP(W*s8f#&RL$5Lu`|O?b?Jn z&w_khg3csRAM9zOg%(N3mUS@t4xHt4G;0Oq*r$`7ZvSSd2+Td3D#Q>|;emp=wC z4Dw}hd1RM~XN!5W_VH@P|Ahgi7x#3HKmwaA7#V`KELRh>iI1 zKKk(sTJfZS%n1M$AVjlnE)WzQ8%Zb0Ng@8UtdpczY%`1?pIIMd1X&r?2y#ypBglb` znYSZ{RYoeuyutx0oE67|#4+hG-%{M=C02b`1D#4l1Axa3(n?iv@OaIf$DQYfy?9o!$ldb6S zM@uVj;P!0vG?TK?4+qN(_!+jAu;pI%EgyA{`Tp+Z89Cehr=!p29U0qpX%sJz_Wo@uQn$7k5ApzXg#E zj}Kpy#E)Q<14SZ8CK?}Qgh;5Rpkx8~Fv|$Y#zs0;ToinnQMB3|#YNm&T^mZ~N>r81 zGJGT325ttnv<%uX^krSX-D9oKDi?#loJ-qt)NO2rSzKamc9XWj1M<6n*^5MeyDYTu z4bW!)vKJ8q5XY7tQtN-Gy$CPGmeNr{tp6MKB46apP&`^si&`5*8b_=O2whwE2RlE? z=r+LlaTS~&Z5k0DY=%I(>GLSyOyJR!tH%j0Eu-OBO1R=*^50L9tQm|zn*MPT=V5~! zMYz!7GL|Y_BCnCM{BrBE*Utq}HmKqIAt{c&*@HK@iqq)L8iGywSpyQq=`9VpG;$Q< zxXZASgXK}Hx6g(KQ@W_Bv8ntvK?qG3Jh5okll8mA+P@89m{f@frx3x_s2$3I=;i_m zGV$Tq=4>)QCeXP2%wzsYwgbxiQ3yFj++YX;a0-zCsWcJ7b5hpu)rnNLjQDclZZD?2 zV~|0hMepEBySXoTV1s^a;z<>B2LcWHT@jRfh+wT>8APBZ?cVe1+{>f8=;0+>L|0f} z3;~80scb_y;II)9RsoLS3Tw!=;s^eI@$;GgnKEkIX>*Hi_;X9Y$9}bcaltzmtg9dX zhe^2~o;xn*Qss+d6MlK(Tp9X8!kHsFzN?^50%`ruSckUxJwP%N#zQ^2NoWm?=nRt{ z1|w;r$yJ80{7cYMfm0lhW8f68T7Zj)WQPi|6)VC|3(LBU)|>fC_GD$dO*e__A03)& z$D~&C{0RF#qd5)bF?jg7L9rQPD9#r8GGV=$$@t5ycm(`?#8zJA)6^WH@Hj@+Zuy@& z78Oa>GENb;aNbg)G#SP+mF*=Qo#j=AVU?#?K`s^^eK+*fgm(#~aAtsQM;H*3s{3H= zynIR^3?U|!&Ovs>6!-e!&HTjxAmE2tH+9(O>mCk*(D3IhE?)OkK(|Il$GFt#UC@To z#>H7kTHQ#hQr!69W^yWW%m1vT1&0kyt4_46gA6;GP>|GJP|8g&!=&&u(q@o=H-2|n zE5t7*-bXI_^zCL0LoYhDq$rTBN1HzOWm|(ShyYI^RyOQy>J+F(0*=Py4SDWj7o(Rb zTxd=g-D-IYxi?64OvEVZ^_^ope=^2Nsw-(C%bG@`azpHXB8wKZ`^kRoa9B{wikiqY zw`EDA#e3>4RN+3TApD3`x$c5<+glCJZPC%Qi@f}MTuH|+{hRMcf3_vL zHB`nX{YHu*lWf1K|1c_Jy9s4%-IGc!zsQ8~r>(6!vJTtu8<{hG(dgTVx~VZcj`SP) z-Lb86YN-!v(K8*mjH(ibBiHs~x~C&dBd?Rp+m&+`6i*%f85CQfkbKM8J{e&zXeBX4*bb8Ueu~}t&{=PRu_Y1;qJ{?n0#b2to z&ZP#CEPUk0*5fVm-CM15<}Viphs0C~a@WFvNu=3?W^HdPbMy3jEvV%0Ctpt%lGJn@ zdluL!nWS(YQ!dT(_UX@HC@2L=Y}&KCm1&5revEfBOV|h@BMChHMUzdikc5nsEhLM& zK@)}#QQax8c18!Tv8ztZs~r@=kL2Y6>XFuU=JR+5a2`XTNs>|HN1=j7QFPU{q&yNR znrxUL2rwV_NIc-f;5^bSy(2e|h!NF2<+`Np&lkkwdmk9j!#nO&Yk05Z<3@Rm*fpy2 zNYUL!Z~xa1T3y%>S~P{zBn76k$_gbd^oY3{cQ-I8SgdknbmJC;BA}9tJZ<1 z#}`YX!)> zV_HIR_N$}}wq$^gab6@|6Etf96{F8u%=~1cuZ)W#2Ha{irDmK?6v$J-2EAxka|__+ zRO21ecfca2)k32ecv=xqO3Xdw3@lM5EpCj3AudfcqFR-MqiR3pyJH@HlnlBqQXw38 z^t8Q27+%U?!CCZ47j;s;hHr#Z)`LtcQY#X3u9o(Mngd9vb+cXF?V>?rR)rd%aXFzF zelP)C1WRrhu=Ia_<|SGb(N{dMd>D&l1IiQNr-K(O5Gdb69e;tB9QOKd&L{sM3$3+7 zB8ddn-}yFV`s#n_;jtD?pGGy@PTlxa*h2+Ybovs@1>FvB+_R@G=IOZF|J}wTN4J%B z?_uN+d3ztI9JmC;|*F&M4_rvuUqJRz#|64_y>`L8+EsF6-UdN8jgj902 zA6r3yXEhA|CkvHiP3QQWe`*&t^vEV}lfWL?DBX(EBYOrTDrFI(5mtBzBevST+Pg$4 z9-Ruwh6muyk2Q!qBCS>3d5o$LCvc;>pi7oj+Tj_|OuouTWdw3)>K_#4CW8q^Y!b9J zG~G)j87&!$2q{L^1kiv>sS}o!_BopA^31#GAH%hQ?KJX?Xm}16f5d!9=Lf&Fa z^vXEFLL^9b3GS>F99*$1Vk1L;G|8rIwdGsUHyg$0?Zv=4YH|-U6Kn#IJe#?%bqeDS zm*}Z>VB!z3?y`dv8PciY)z)2x;6(zoD^m}oH7tCoguVU99#Y!+QwKJ?MDp^~19cN; zIn%G!*jR$57tj$h&N^`a62Ti87fk+acw4fKgwViR!ImrW5_RB4-F*B9>ODyvEO~m_ zhaj1yo2*L?6;~$Vj*G!jl{#hVd znZO(AQcU}~_#8}4>QNNIm_D}35$!V_i&LJDy3Z3g)afueF9n zP%AW$WHA>};-gAg3J<=T@IuB{{c?LT|R?fT`4`h_QqiI~hGOU{EMw><0ydFncdjKD|DqFM zR{|G*X+AQQ%ty!^aw=M7PwR&N+bZh&iB{QL;W#q0bEvWR_{ip-@kNhysJS+-YfEOw zlGRE5t7cv~XI6EnPZ^V7w~ej3joGVkQUD?A@?M zIQS>d%5Q#4cVR0#V&4(+EBu%5h#(_e01dIW^c|^AK*0i#;-C%{k&i`9TO)ovemJ>V zLNEddM`12Nu88KG#}nZ7LDrn)Rc16N;Wc4(Qr^uo6lv&DNutK;!Uc?7Usky70Sn6Q zrc*<{XiVH6moZGV$8Of<{kr|y?x(2Lz}{#ST5f&VJEG2}&1RJ|8m#RhN-9a-RZM(Qr! zX#C4AkaJ&%0;#|aO(?(&>$l#~@J^JvmO$K@V&-^_ogF2c zWSr%9rhn;x5l|iy;pCfoEUiADrJ*J9#dn}x#RdHFT3|KOinePc{b)V7NMHucP?p3`VVZ#+mz+EfHT&T~fwobfL_5 zrmps)V8_GT)Ors0rc~)}ekyp{Q%MzQ2n^HgS1W}W&NuWG3QqgziW`$p1Oum$*#F*5 zf+Pu2ir$<7nR>(x69}gO$Osu4^<@0DxsS1OB$5I!okS3KFW*m=t)27^?4%AnTDi z2Oo+uWl0KpVz=+{WP4=WD)$CV-8O8;TfsU=m|wR2slyt)h{2cQAsus=UNmg*yQT*l zudCqJ7He0*5T)*;HQ$SNlwPTVn7>2tXwT^Zd8(oPg}Q-c5PFB_^HgvGOk9|oUvd{K zLTHAnNt_}#0^ZZreACN=IE5WmLF+JW)eS;GW}`WWSUE7YugMii9pi0$*=eyh9B5iz z+d(Ka386tYHO#|8XPPl$B$TnLKZP`!SR}eX56LD6Nvw9_MArN<2S!tTE_Rrbb zyP==ee(lx;uq$Dq-dJHj(mc542X-X+Cap`Sbd;stIOJ_&r3!#J2w^OWzac4}j^p_F znMLt`9KUAk{=H<5AHcDFSoQ<#LCnmUNZvd;5Ywu^bk`2#^zsswA*>afNu$D-xai=eh6fKKecsgmOBBedc z5B#Yle&L(*Bef;*Ti>kUUT)ii5u7j!$m7AVr_1co5<($BgP)`vX^)9_H06lB;@SYM za0(@C{>qVkRQSAh)fBF=l1zizqZIQ%x~Zjjk}UTe0vS3~k*{x<(HUT~g;bl>L-T4Y zd+{#a8Uf*U7rceAi4|LIvk8s+@s0wj0z<#d`cFONZX{v9h!2dOwxc~>-qVd9CKjrW zNFS03lagqwZGmBA>o3+&)E-$fWlZySM-#R*BmPoDC9Fy~GvMVaiDJVBJz4C=-ruWW zj?|P~o1yzWt2@tVk%Uc^k}`Ex@!@%vxd67>w7#R02$H+M9b}us3jj_E!|m}|l8bMP zUQH3E^2xBd0TsR${-L&vQ{hqq<_O!wp!{>1uvvVQAwcA}p=e8TR`yWej*>B|n&fhh zs^z_JV~2g_HomVJU@O4Q0NYyAEdcl1Yufncas)Vel;e>$!(!MQtiQ;~A4rr&OSHgz z;;9#pcnamLyoMv~5J5t2$u@z$$}=jSfX7G_8uRct7X4WM`kE*Aj1Z|9N;Bb zDvb&YwH)9%O;6^vz>d}Q=vm57pRy@L4J-!Fk zPAl38WB(KHO}~c>{y&1j|NPFOaT((y`$4QN>rgWr#9G$qlFH9=BR$ZY`>k;OX`zIh zm(6?m(1Dbg`30XXE%o9LPxFvC@a*C_U!xICCu0zPb^uS}njQsjVyB1yr=oJ30*vH2vTqk?6S zG$<03ST2#=piWe_6?8*C>h+Ry6lGEL-Gzc#Izpn1&Fsf-v0yE!NqFAn_gxnwWSmRj z3wevMh-|`-bJ-$Ns4{~Ywl=d9C1vmVtH39C`G{xd&uLpr`D{-7?NMJbSqvdNY6YA7 z!%-^@q1E1LX<_7TU)QbI(k_xXE%)rk%mb{Y;_Q%Lq1vMef(y?S@0(wGMHg$dhu{<% zYm@5nqPsp2rez)pWL}dcf$WbY`u&5&KIZF1BJ^P?R|k2K(D#b&MM9Qi(NLOBZRw<~ zU1;}DQH@A-*H2Q7pm|yp)d={rLC$?$r*Wlf1a>9SR3mtwK-Ea(yfWU$mqy4b@?}wy zmX?~)FeovcPmpGPnb>)d8c-fBeq&3~jXm~v2W2*gE%eeb$#$l(=@_pMoW)e(mfRe1 zEP`@pC~Z$q7C+pLvU?6!CCEPbT$W25gEZZD}fIR`Hx8$q;81ODT>*#&7Cab$U~v+F*%+0FnN{7&Bo# z7E&?0EK~==NuoP5KaGLWu51?PO=GA#VL@z1>fY|uNqm_e5j98b-%NFw%JY@!vHy^) z9g;42Skcvh(EvgO$pTfqqzLq&k53g^ZWBsT{w<*P(#_AY7-9zf~W#Ml`7B}3a6k66okCtIp<18eu;v5jzvJNoO3PP5Is+A zqsA4nfO-NA+NZqTsFS>$e-5$8tzuvO31X3&0YA*%I-~9Q2QuQmd?7YrY(~PsjQG?m zS8PgMwW;Ug`bT3E9_XEL^UC9Dx*Xqi@`#k|s@!m|yzohxHD4!m?J>OT&fJvj7jna~ ze>pYl-ps-)!ZG_tmt4`Ue^ycdsyUg3zY52^Qx>ziW9`A^wTm`=vvSi{74?H-6K+mT z7@Hb@^{-ZJs;{W;1>&t&!rG_eYx;ksDq)GW=AZn*H%zC!e2!9Dex0 zmouwoW)_~0u4ZL1r;IL{j?Qc|@h`pS4DQ5+7=!z-9UT(w(c0!YM+DF&RsMZY&6~`2 zz1y?PH$B<&{m+AI{vs^-dvz%SyIaq!6hx4DL~-SqMUb!j(6u;zo&aY*Ffw~%Vf+qp z+_tpf7;t4H)-2rjG`U@P9sOB_DINXONOC!OWZ$JD35k^qzpi8ytM+#031oIz+E=5S zEQlo_m9_9$DC@(1hqf6Z7H##{NL!L!3(@#YxK94dO@wGQF}$=v|I1B;h9i58mDhbI z;s1i0NK+ZhJPZLSIc5#IhAedgWDaB~a_E9qR*EUjf*mAKtN_FL;&lrnWUIvq4+G$s zJPJM^)#p7tr1clR(H2IFUbJLa^VgYXCEhkp0F$f8>`HhP(e`V6A@;nvNYdA>`7ftO zncVEGFr)Z3Z2i^iH_-^K>s%3;E)ZHyc3$n&)EL%w)}@Nwp42eM9oH<>DNf=?hXemI z7qJ5~6`oxAFLMz=vMFKXKg(RCcxwJ`^7AtwLI!uNU_8{E8r@&-&R#|}80i#jI{mfqIXC$Cf5j+;LyilvYS?33&p6+Ov*6WZqr%N?RqRJbsy|l z_powlFkp_C@bqqd8FqdpSa-1K82L5_S(i*KiuYyWHFj_)?Mzhg46rDCI{k;5A$XD~ zf;23HIE76yYDth)5a~r}KqcvqG2oP5eXh)c0waKeQT(0I$4)A=U6dm?l;iP%F4CCp z(m4u1!Du-%??vWQD$JGxBEu4k)pOrNKK>DvR+d$e0&04CD%Q#?)0LPILn2DBRE|L+ zKXr_|7X=&xF+t;Ln5gO$EFX)R=yiWrAiPk9NNqvBNNNzVe5wg3kHsA2?s4|DRwX#p z>q$=ui(iMYDgIdU%yMY)POUk94>J;w7(he`VgZoDVaCSLfyv*eQ%gwalTG=scJXXQ z)Uw?_A314T-Rz>asRdn!_c(g?kYkUcJ>#2|`I)(ecI6lK#}MvmnqNsP>TjNoHg##R@VWRuL5%tU&bu6~Cg z?c_|P*9wxPoLE_QL6ONHRS;rw>e=YFvzP5%$X9kNF4ZwIb7*CUx8YSszSHybTXZY9 zss+OPgDEU0yQWXkOWG2o6B#~$tpbfTGdNYIyqk2NNMhYSM=x2Sad&M}SW3yGyMugJ?IGY@wsPvDIS);v`PMv_PSI2Jog%em9 za$Yx)-OMkXWT`-wbc1c=c27q(cQG4Y^iEWcZC`LH8>iP0dxg{<q?1-89%wxx7+A>v*IiFFLCdv+M=Dd|BY?zd}h!7GN=#) zh@~oX&q$FQlQBKub)!qQsHdHXR}rj6o52><2>nEo0FIB6xiX?9 zMD>dON_o*J#lt#^wrwDi11o_fb!BP%bmCskcwB@y0IzOJ4m5LUs;)%gCkNBj;PXrq zT&v)xJ-XFql4T^KMp4dLb&OGExZ-$gsmkWtNKL7%rWnTWgRcaBjF>Aco;6@Bn1m=v znJk-fFEY|2-wX5f@_`Oo6rDTyA{(VaWCcR+V?Jp;{XSBj&EdK3(iv7pQ z&nU5aG;TN@+%k}{MJ@i9z$&bywcDR=R_?Y7L|cM^-|cUSb-h9rIEEZ9f{-C*>N(7qa~fkkj%Y=;;HAy-7q3L z525VBu)YYlAje~k+%8((;bG>O%i6odQlR%TQ3Y-M5OP64LpC<>zxAj!&3>fz$^O?g zb&7vCHDPUPeCos%n?7H;>C=k(Q)3f4rN-}?uwv8ytlads$CI*G<%VAv8Gf!$&G89c zTL;e?^w#3tL%PPU@0!`Xx%IqB*S+`W&9C{!IAYS?^sN$A|pvDGn`9e;T9FHq-p49*R}|E+y$#@(J?m=Z3j z+>;;K-7{|2$QAWvC~X_qv1Ch9|K>5!QT85Ru=|yL-?YYEB!rPZTFIyO9n&^z{o(ww z(#kLIW!u7uy;6$S_8~XLmI9pprkn(av&L;cZP*53=eTqjkG1$o}A6r$%`w(J`qQv_2A zbL9&$AXenuu}s!A6haFrohOyb(%}#J!L4spvcaUjtg!so$A}QTAd)@zZE{JdL;2GO z$R!zaxgA0ra!EMx(}31OF1eFFl#g7J>JDjLcfvt+3`3IQLA2lT!r{bn$wE2`0CC~m z^iK+p7b=qcQjb`Y;A=2|LqWtnOGhVpwUS>_dWa*AU1Pq$@)EV7_JQjxhIL(EE6sB& zPF2K>BglrN_?Kosl5dqC3ET$0-4ooiZ2`+hFaXp{yCU+86%%Y#&9z?|K~BAe!yYPa z+MQSUZv^a0`+W+{oFrul8E9l%~KvYX0m6 z4*z9-+r8@I`wO7$j=uD&i&eO{S8OIXJB3;hx?JTa%;IZtzXL?$ikDGayUs11Yz3z( zV_B{e@Ohm$&HzA0{3?F0f9ZHVu&>CotDrYer03;vZCy_e`%Tqs=of9PX7eky(p#Vq zsnFu!6RrttKwp|`E@h3D4izwfC~cg|zt@));6xwyJn+%)WCL>}4a!~39!FqxU)zf+Y5H{tc;ze)JXEB?T0LaNJzzBBuu2I!Vj=7LcAc!9u zRq!11CVo*OiP*(12z-8itE&w<7$I@W@Nh`o*$eWi=5MXSPDjXum(&&W*!mPx;jw*% zo+B%wlataN;jLkdKaJef@Ad2Pk|wH_3efb)ycY$`COI9xDIWVqZLMGHq|Va?DQ{n2 z^kfv16E_H_=EC2Gr@njq^|2+NvSUfh4wNocGvIG*LKK+bK_wK=C!Rb;um=W3@j`xhrQjdkP~vi z8LU)wJTxwZTxnbp^vtt7SS2pf)>} z>yIQiUB$ajVs1$cjq*wAE#sbsj-3ni@Y9^bbPv8h2H;1-NGsN`Rw-FsAf_#z_k~BE z*OB)?ajq;uGi`b3MUfVDGi#hW2qK#xAcszDyVs!;Th_)`bHiiu!lN>4MjqdF=g6*0 zN2Y9eDK|VgFMP&d(hIK%$6TCW(y@F0s{8t9-IrPT>u^k;@|aWdOICO5Uv=+)g^pVk zY8)`Wpfx}8d`#>ciLrlNSbE(L;|snYU(lR8dDq0aC@}6MCyeVcwCheNB`b5okM^nA z9p825kclLdTq0$XnT6*H zUYHmzN&7rM@^a6(RN%FK=O<;)1$Nwi>#Q6JdHm#(_jPF5i?ILa@b#Zmxu-GhhE9(c zobY%7G*Ecg<0|62Gk_@Q!pGRDo-J;>>`4+_d8jmX>iKa88_zg~WgRAz!k=eQQqJHngQw*;lqko3WXC}p0pTOUu zWf*=;q90+KUmYLAhG=f2sE$!p2heN&L>bVN^8U1WVqC_&^{Tq9v$e;aD2+xyh^0~= zwv>h`ihsNa$GpeQ6xY~Nc2_7l?O-w`FYlS0FPukGDy(Ydr0v{|?aP()>g46*L?U)b zKHf!6BsivwTyTvWwWu{hx8!%MZrfnq_60L|BOMc+9ud^cGJ5{Frw>VW^x`ko}PGfO_xxbQ+`a>bVU~cq` zosYB0(#%g`uuKuP35^62>=dL2%hp9~Y`~BrFasWoLR;Ke@9**-Gg!pKCu8!?8TDWk z!6Y0n>rVW^sJ*n1V(aJQq05?D{3)Khb{LTydZ~YO<=GWC`6#SC+_pv#Jjh$* z??O@>2u#6v(q>(Q$IJ;Q)G@)qBh0#MLvO{8*^z^ASC{4v?Z~Y_ z-bP5%d~Sl(&n^&X;FGZBLOy4yO?(cSw8&QMe(~QlWWHEdrh$b!!b1ZC;a+dxrbM>D z?zO4dA4drhLug)t8RFW%cDs2vIK2qJBV%5Hu%k%-V}-U~|M~gfX%t=1?c3=17%!<7 z3NtmiJh=y|VN#PZ`8}pT6#kWvSa{b5;$ysI9dX%F@v+a-EcP8du4{|{Oy2{M78C2Q zZSnoHWcWxp!9My%FI%;2vx>snUM&b|<35-RQjfBO!qkG&pt3BHavN$x**;*er>960 zjdBQ9=+_&|vLp}@c-Efr(ty`KdH89^7e-52zx;$!h}s``HDWZ@_`=bJsC@tQ{Zfxn zzDO$yb~q>_DcE5y^?8{!m8X={zX0BC=b}*Y zO>reHoaWB^~j*b1p3zy&HxjqhUC5Z)Wou>2Yb# zB%{i%e{;7LSN2Gom7Liei*u9k|95HeMXNg0JPa3-?9nA@dvYVGa3OiOqJBDn+q~SG zacIw$0D!r=;Ty$_o!@uo`Am-e$F(KTF*(-bg1+eP*0Gtwc=XV|hWtEydT5_~>0SsS z`Om);f1A^HatM;evhQqyk^pm*~7&F}%imE8=SI)*BX zTF8lfDC0DwB!y#$XvMvD6&c%$syw4I5qTSUmXIWqc7Ewm!bGCX{9v%_I?%=)A10Ep z`SSE68o#2G%1i5xV>DTmR$e+a4nM*~zX&&nPyt_0T~Z&zD^hgm<`}^V*hI}Ai1!QU z9`i}dhQNkwTen8>BjM6vgCB|F*tWqel2n~yBg=l&Ld!Rivwy~hCE^$y(764^`#w4?$T5-(y;GJi?9*^~NOI~GgmxR^ z&rfQY6Wb-VqD#Zv*e=T}`qT^{zAma8iB>tP9mKJzSSju>iB>q)*EvfO^J2@3qBBF6 zsOX6^pfPjOnZgB#9HV!%LJc5+KBFKiSqM(>66~yES2V|Ja|sDM;{xY~#%?52X)wHn zZSb=%sI>~)kO-IRE1600d_ja0Kf-&!2QhaN_SasXPVshl8)4074^-q`I#jm~>tqU=)~x}3!%l*Yf%ClAK# z`=}i?L3l~y)MgGD>`{Zn929`Cq71G8?%9a2Ox9Hi(bE(gn7#BAA zk<9YT+jFQg$)O+BBZhTX4I;hduMDnqXJR+ZdW9dzK&9kBVQFWPaT-skzzr7>;Mfsy z+a3#IaU>TVFnI}C8##LEz^}V&VE;3Y4nC@Q?$u=F@YOb%Mab=fXwngsamnOSAhy&) zZTLbs1s2ZNr$~%Gs-<+VJL$-0zcm!z!YYt*S6*bSD0=EN&G%1_xthLwBPrCKvF?m= zFwN%0l#_zQFFVLHJH^9Uwj9MK>AZu^J%0J>+y}7(n=T@U7i+x*j_ur~yi0PzLSVP? zkA}9{m!*8$I>91hp85)H5&1w(V>cAo_FU&5+SIb_v+Pdnb)c=E3nQ%JbcIGGm)`L~ zu@X(FL6^T52<_RSU$l+0O&|Sz)eOT+azr-Kr=kiY{QP~bZR!rFzjR!z;j)g0+l&mf zp6i!`NKb&e%)ArTGmQ#BO5u_hbdtunyQ%mBnD>;EsTi?mU`z{^MK(03*FCEc$=ZF`EZC(El zfRVMmySyv#Tl$$b8I0#{MkbD^wdWJR6|Qwc|X-4sJm-_2y8GFc=hc+E;jBPnK8lh<0<5K9b621o z^0R1i$l^IhIi$9Zyty&+c5K2?^^o7brFgqbtbMjYFmadSo1$c8{gy8Ig%@J|Wm2JI zSM2Kem)CO!nfYS@1ZB1*xH}uFP0|tvEQ+k+9O4Y6fn2Kw>U5N>Y;l*fWo%qmP}}Es z85^n!YR~#zCiUShx;q-FHcl??K6)oBjlJrLFU*NAP-X29kF07>X+<@aNg3rSGHQU3 zRFz5jgt>UhM%UJ^cJur^7JF>7U&b9!$jd&!nR&x|Sg!|EgHK?i8LJ88lQ{pz*5)La zQq*s2qwUd4I_dyQu~NVwprBMHf%L3`xr82)0&g#D(fg&yQ+wrpSW7s9sD5h!^H#hu z2b8ou;b+OOws$BQVaYE|_|y&1;TCyzkB!>aimDIsU0rquwwy7=yn~DrSXMl7%xe~e z=~RTBS=KSHKp+Cw%6{s$8)UiHtJfPX=tW${v0c>ybgWYuZ$l7|Q@~D$zOjm=RPcWW z#BrsKTh_zr7-77R4y>epLIqlUeC2n;bB@%JQ%9h9?3-C6?TB3kBP04-v_~X84VOWq z=b(8DGC7S{Eo-KW$#2KWRSRHWXmy9Qe`N&?2&v&)lh1({WT{is1sYXa(#Ag7MemB+ zL6(7icvdp(vf&DSY*}ddEg+;sz{8qkev-xaL*A8jsCvM?yo4^?a*#Y?q^RB!^3JIJ z*o3`_zlv9ISB@zrw=`kAkB1&FSrR<5@X)}cED@hd7U&VfDL4jB(VG@Dw5uH$KTs>2PXrsNal8OPOco=7Img*{QIw@ zJk6<_R0B`l^=alEJ0$MvF)Ql791^!He?@)6koa8> z-d7*S?;d=(e%p|^v>hPM&Pk6iIzFzp>^C8>c?m_a&vxH&q#h)ove=O0hKJ&3i88h? zmlk{_oFE^H&)HUUY!NwYpmdEZr&bTmY?(ZonVy=?@i~7@CUo%4@_7?m4oViu=ps3J z;EH)q<0U2W=ZYq5v5yxV$6CjE;yB^)g5%FwwugzPd1JH67`5E@X6SwaWaiT`G6Q8l z!23fWH-W2EKreNYwQsc=VHP5WqwVznY~Rmy8@Q6oVffK{Xz?j+gI^;BqaV5U9mH0{ zQr?WNg`88&{(JbESA=s(+ZlN$H?!s3MJ$gzgda!Hj|b>-x4dn-oR(VBd;aMO)?e}_ z=lpmQy3s-~{D#({!*8EKJrQ`d@O0Ov7rih;KI1-(+2|SH`X*ctKiNDi21`wO;_>l_ zdMWT7_CV$$zH~Sa=KmD7izV03{2v={6h{*3s%C7>c1__(f$GS(d-(*$wY@F9%&uSI zm?MPqaf`Lx)0yrStK&+u$P25oi-L=WG^dicJ zCm1ImDyU|_$+n)XhR6t~(9w>9bz*lpoKS%optN)uz)O(B_D2@j&)}X?92lUryw_Xo z-_2`AX14sbHwW>Fi4~D0Wx2n{8`~U7km`;ywgAvkkel4)ktmY#FL$g}ONi7T_|k+% z9l}&!0M55eiy*fVJ`CX6;MZB#_B-|oQM zO#eh8+ZYp4*un@6al4_-m5NzKOFK%`$n>^7mJ+X`m#Ci4YN)Kt0!X{bYsE`A{o~A= z>sOs&#oYFXMBXX7t=Oy-d1WNm1vd6YaMEvpt$Qc{w-1~HZ9tQ0rkZWoSU_+MxpChl z=gc>25;!vbR2#k!P65jJwrvyu0X@ia)Xhm27}z!XlDzURFnWsPeT;v~D=x32?Avh6 zHEld7jv1@Q1>tRz==Q8mxz}9z65pW>fufp5oml@4Tn_qm0al(Bt9(kz`aaKDoJYjWnBnYy>_R z$SwF&ePo3j;jAGaWS zjrc6Nn=+|}MK=jLP2`bioPgNS99otM+W0QU1(E_I%8PWHL3L|b5G16oPC%Ga61Tfr@jbUz<0;Y)+34m66{PBIW`;3OW9QM{X&ZIZPzw**M z)xKt{T=NZZRcwG*Ky-Kh8T>}h@(WvdeV#lDI!3v18bfpd{xGUoBtF3d#4 z=9H&IermI#Q+OYdUssWVj49+X+Gk4-!KuA68n$-kouRm}ytF!7)JRodD0dW_7R-&T z)C@Lf!R-K8wRa0o<|vPHL)#b%h*!1CZ-lbo#?4jpYfymlt{lv-&ET-#5d+xwcH-B9 zs;Q{R0{j}~sJ(QD3Rl3d37~ACqxh~7zlL&1R{uCRdhU+Lkd><=4D=1>);M}n!cA{X zAfsog{ahX_4-%dzOU%e9ZvY>g2-1)K%Yq^22)_xfs^n?r~1J! z=`kxiV+zL79kB^mfG+CMgI%xhiIir`;YEAqC`-dFo>C|cti-f~ju9|i5k9(ru8s9JDxR)S0i+LW+DWX`>zjhPk1o|aRuTt?Jlpdp9HsA`mj^44S)OFzFecm6`0?p_Ch6l8a0%sSLl3Aaw-+SFjFRaCzW zr+Sm1J9u5dhpF~13l-&DFwFTKr=-TQ8YtR}6JFTfe1;AwNyf;sU_Oa9Z14S&Wy+q5 z=!;r5zWSL^gDfX++philkBa9=XdFC9{^dC$Y!?6D;W=`M$f@A|Y+N^CK_lV+VQ1M^ z{cs)pzWksXo8!aJw{iA<0V_#yW0F;lShKa+%9uNUuU=0EeQn>*QvPr9WuTh*(@8a? zwJkYCVHX}vh9h}}K7<-pkG<1UhreDnCq(f3k~p#=vPk5ulH+G6r~hR*H3908>}PWW;TkS74qlK& z%i zQx%><5DBatSf7GQuxQ0qqOnA<(9{fsRlq8qdQJp-2mj9;ektvy@3?jsPpr4{O`$r= zG~5x8oinqoNXF1OeR5O(~ekS2vl-;rYM`@ zRF@685lz)JA`vlo5M(2L*%c3X^koikL1sVau{t(he%`BVm+YsyN zS7{-OV^*&R2pku6fxwe%SRhm~64_j#Tr3w(A!^^zPqNvE)4r%nAAU-UJD&!1jJG)$ zRUnDBF@|wfr$Bo7Cd$Zi7m)@SI@R9Ifz+(cpPTcUv9%gjaS_}{^F)Hba z_1*ht&F^1To>_QfIA(Tv%x(E4SK!pV{#A2N4vhob7C$DkDJFJSQtS<2-%cN25SmcX zVNB$enAq;)muxy_as98+C_63w>dRMbYF@eNU`72Huy1Jzozmj|K0Bdz_^%_wANHxa z7A>^1yDrU6+3@?^a7~|@3&6`=&~@?!9ZE)b?O%0A|ExPQ3x|heMwZ1~FuG(9ydi&= zS(pvgpl-IVY9sW4K3OJVYOs~ zV6Ca~Pa;d0s+5msSb5mx6S4AI*ZI+&{-sUb#unyXT6$vlv4s~s1PBQNEn(q6Yn0!& zMtPht3En9n7eZ%Ruf0pAIEBTPcOvwF_rV_6=+!mcOS9Pe=8~$D{@M*&^f7zin?D3! zbL}cXKPCP$NaqB#l?zilk&g;DfH>#RNE&r0Sw%mpNOBnhuyn3_=MpHFZ@USx;S@qGBwU-b^ym z`eJy=-^t9yNzRF|PNsz6Go%d(6;jjR9r|c2Ag{45H2-zZduw7$JrhX!Xr@Sod^)u9L~bsy%iojJ>C#P;!LCe5nB%)RA=mUiBkE4z8Sw_( zG2&`+?SS=$(mtEl2T*6`9D=$He`6+4EEPU>ozg^KQ61{7^iqTOp_Y!mOb2Vq5#N_< zd!YAd#yDWBr2|tjvYut~RB$FO1XrOdZgXi+V5^JL4jioF18Ty8>sUSUmGLk4f5u)< zTRZ+eg&nxyJ3zti9Nb2{snObI&oBSPoxVIx| z0U~d-7}jl&Ngh1|A?QZtgY+5qv=Faz46o0C}$hnF>gPj|C|qd zH*6Woipj)T`OR;!VzQv=l^?nm$InY-pL%0y8+4Lu7VcY85c7Btj$5;J|6gIXQF6|8 zCFe7?>5(QAlEh>~F9?L1%jC-BTK5S5Kb=1ym=n;p|h`*lPZ0t~sp`5hiBq%qTK9rGp zV@V}*Oa_!jsEuq-2qp_5AM=Z@3ikJh z)^B0-&f3wfNTBvR@TLXa8WooN3r;1+FQm=G=CbSX%(jRBA`ClqTVv%Nir01K5FhP$ zc(fH`IYL_urbG)DvF1uc2Zbi|>!I9gJI8Nrmuv69#2*&g4E7BB);7FkiB@$uNe=4i z47j{%ie0j14jGn0`IhXf$N{K;R1)9&VuGe#gMW^%6_z8}nl~vc5SW;!*xXnLW(X5av4zW@3>?&0$*~i6smfcVA)O4= z>+v?sF8fm!T1e6@Vn56*IeWs-*$xu{ts|@g@(j2};jS$5qT2tz4H6J0E_1zkq&scq2m@Vbdc3 zJ6UAm4#ArmwvJZHN|xj3Q?T`~Yljz!YL(@Kl8L~IR)J~tfOC^}_@oTyXj=pGlr-26 z)Qfq`533#rjXcVw3RtTG)jZVmmS8d$a(NWhp4%u^Hu z47dljUR;ESz{%^-Depy6=pnYR4^U)^;wr<^L(jyIskib;iD5zJN16Ld`ek;|0orWp zN$aqPzwCaTH*$twEl)DBfZAB2AE$m!(K*E>6;*d3F0Tz~MC56O_R;->i(@H}hWl>a z`g7sd5rolDG(6+))E36M>Dh5IcX9o8egtOLMsy{DFQT}QhXDbIFvs>}g{73sC2xTz zPqSLyNLL|ubU?Ds!#Pd~ym$okBY~x1m)|qQPr~F;&`KVf8lDiER(Ml5=Hc>~`}0f6 z(1oq6e^yy$;q~E|Ys+JDEzyF9R{{91f{Qci~lHDXx zJ|CAlc18W5&ch@B786@MBfli)yimdmwg0_mC=va%e~aI}F}%6zq4b({$Bv0ym76qdY@hss-Qy#t=f0YBr2hV?QTwyq zL-^(=+K_NfTM=rqLZ=8{k&>}l_c0$el8>|Ijs{S0%8uJmcbCNboZRQt-3fxmpQ7}RyI5U0+IJcCla(%Zp3j(tRjKnWnFgf z`OzxL;+80^8`@A+mrf1wTNibMMHs$vb*H@A868-(S)G_yJ1B%7N1y|owX6AoHN52M zC}<7Its4Q?wrvb_`5)cXpAqTS3@d?krgbA0_Q5GU)!TC3?Ni=dwJ;a=;*-rIV#Iwa zPwXQ-NYdxsnR54>57IVAt8QzJ1l#-rY?YbKBgb0ik$B1K=~Tg%Uzl%U)%B{#?NSOq zknz4SSStytrP5Q`_KvHv?QEdkRAo5wl%n>o32JBhYrAFY|x z&d@|RF6CU1#`nlCD2rZU0coAN!a^agC!$M9@DfV{<`>;XtGOzzK3@js#dnmK=3KyE zxuv0ZO6$HP!CGn3EjlalWAr3@TeFNc5-bU{htfk2Nf)=CyP7Db+3n+}EN z*yfklX14r#5(nCYUS1qoJ!KR}#|T4j#52`&WwKE&rjo<^H3#(8Qw%6((?E{~0Hd4@P_9I%x`(OnyBS=M5UyLJsRpkW01?lK zoVjQuT|D))9e9xi*%XqH4!y|6ZhH>W%nm#8cHX>$n#la3kH3;CPro%8Rb<6dmt&3R zeCjx3Lm))leH1mj6MYhk$2z=};h&aEDzs(y>T<6eA`RIf_-gNFs5eK>W?8|CGyBOm zhS;`hRbpwMQh6%51-8xjRAc02A)06nAf1lXQtC+a)K+I)J}g7n`XEx}~vmF1O9UFQko z|I#wb^@o4R}{phy+KTCV<14q_g`>U5{qYHGP zjpTEQk!^%~w%(rGi6nay;Y6~Qok&^=%1SH0DW_7gS>raJX1RrA&0$=&drRo+GHw@E zJT~iIoLbq?KZmeNfbBm_Nw|qvpecz0U;ONPgps$tWX4{UlYEh(9ZX5~MJaUKd=jxz zv@AQ>G<{6k6U9hbC@+aDRVbjKvNBOq7DY2jd>$={_&b?)kriotswfe$y!3jZPTGIO zr)j=EJ*DUr{vgeB@MCZK5#86QpoYP88u?vBQ9glk@-4BDJ_CZ461$V$3)OB# zrvMZQRe9MYxdjSW68b*gN;j^6Qk8C+bTXw45q$}r32ueTf9nwur#K%jlfPk7FM1_5 z0G#x#y%$jzjn*f(`N_4sOLK`;Wcrjb&9ANHKc0HSsSV>^W`kzp$3+8Yi#713%(2Tu z5vd9`4C9-LNEZXi$Z1 zq`O>wq<)-gO5t0NqgEQ8+De+$<(%q~f^{+0ZoD8K^AQMjnM%ua<$d(A1(faUf*mV! z`!%2>QT~TXsu?y#R#N*lKjxxbKoBrfvL*_5fSzKz%$IX*w@icAR+M=wdc|3rr&sOE zgwS2L3^xbnp_DK zpo8s6r7{6kSIMLz3B?c7%H!Q&?B5R?Rn%#VXWbG+lIxGxB@1oE%?V94xY4s>Rs-$Y zeLG@e`zFVBuLvc=K{9bdLFX}%w_{@8NQ(XA`1|VTqd={9d@AfCJFnW*X>t7>pwZG3 zZcdM1K550Kn6FZL?$XVF zJuF=qvu+z(bsNKpi?@U(GPKH?HMV(v=gtvgy+{oi+&~-9imTZUa4FjZXbBOw zHOBw~l+Jtl9%}L&K(JkT*YdQct!K^~TjlTc|7`YOaww7Fp||v@&)#aK4??;T3ppvz z66i*V4!bP1F%7>LR&YLQ$hXfTRTV!Gl0lPip6 zjy)lb^hB~C0W>BDkWoCwQZ-IGjf2JB3G2< zH@`fRH~6&FjD|spVeU0GBT;-aM@f`Oz*>{n8clS0yNT1e281k5IYQmA*->PvfZGPd ztu{AA4le`-WdLfpO54J|JE|`wyO6#uQ&86S3odJmECFuY&bW>1$X;O<%AVqOj3Eh6 z;l82F{8aFzyz7+RMjFM!Cu|%NKSlf=`#B`<3A3@HXctX(mqt5*x zIFE4=(sX%Yhz`o^Pqlqm^)$7rB{km2<9_9s=iKFjRt5_@nUEZfJ=#o0jmb#iDN<_+ z$DOmSSl^7btI#n`mqmGp3^?0{5Vskmlf8E6*sYY=2SINDA?!b>}N)cxm zGjWPi(Sl#Kap;cWhbS_7ZefNoqh|5)Oru2=MY4y9eZcbBcb{l9_7p+U8lL39b~Tcl zveA)ptyVWuT*SQ`=Kw)31YjmsnL4sr zzIz7hk%AOV`!w2ny}3#?vQ)Rm9kg#edhOV5Z#?a2)veV=eikC_-6JE?Zk+|zi7i6{ zJR_+LD`)$$%tHBl`+pV{rnQiheQ-Or&BO+c?7MU%7pe`vu4EJ!bM559H3*zX&M|xY z=W#Gg@ZlwLN}(qm@5Nzo`R%g$2VGtXyw&c79#i%Q@YO-WorHNECmPgH5TUlEyj1Um zF0W1+EiG#6>^4YD{V^Q3&PIbe0&mnhyTTWnA)IE+=Z1Oe7&a%k(FpN^XteDU;{r)pFdT4HN@Rl^^+2j%jC_9TX!noysO1A zNknFxt!;tVShQRAGQK_Axbg`l%CSIw42|HN`N_3bNU zWZA#B*siM*tiM=|Q)eT2HT#_ue8UZoXeqtM*D|EX|9(M$G3os5|&hQ`=X zuL>(Ys{#RNSkU^9cB1hIOV4S(|e-gs8H`Jq+jkV_%vaQPpKoO*3>)cV+%0r%PKO2Z$RFdyM;MsUtEUA9a=^66RxFtZ`hWJj6ZVZ>c$AlQ^fsN z>KN}Mkh305HNav53o9aOe}Oq@cse-{E;bH{&I@Rrw`qbyH}W?)qzKrnmjscQ(8m^g zY)C87%!Qr|x)I~@8ASn@M!yF`meA-BT z>YPJ#A_ycJT?BXp{UKEV;J6pjU@-=#0Xw_PhDa_C;+e;SjkskC&rt`)5f1@B0o-33 z`>dKfd3t5w?Hkh;@Z^xP;c`NVgtPr`l2&TJz#@n$o zp))+Ek{?t1jh>Obvx(=_{GiiV+k8u8V-Uv$FhLjj>8#bb%8l5RZI;7&c=pzX)iF=U z)gC-*6k9{#xM}$X2getDmmgUQ>M1d{@50ca)5aG>@*@w$#D3m6cE|gr^~tdbH}^^y z+bjO+@hdicuyWJ;74_eT64v&NPaU^n(;HJ#b`D9|uq-$H;K=ZUeQI9j%C((ClK%X2 zP;CEEReWScY#&ZkdA3?UvHYFaFZ8`OuP+a{k=AVfxVY zCx8)}pFGxLE{&La(j3tYUqyHQuYL_5m9#XLOj0K!Nz93WVap*r^WyicDU@V}^?H2V zlizP9G)Y{v56wnIw1pDMFOv5F=&=>Wx0c$k$wqKS*G8r9hUP8GiPY=_+aP>S<3~{jlJ9_EtG&u9(#tNx ze$G$!{qhu^>Zo@A%~2Y(?c6oy3k1)J0<8IfH!Lw9KWqWE4Nq8~V>6w~$bO121#;U) zF=Wb2%kCd9$%)cEn|w*45^QaSSJsp^?anLwH_D8a_WShzVeehQtE#T9VF@RaM9qz$ zsE`{G;;rDV0tON!NKiEJqJa8Vf@rbU7V#QH63-!!;Dr#a8mkgOD=Jv<((<-q!wFWj zs7MjT`jsT2f=bQdW&(udA9Jp?*S@X2Pt?-?|DOMS9-qf^b~gK*v-VzV%`wNAV-`H? zCfMqtmas|Xk0XOEc5{p}YfG@9DeaeLbr$4~0BZX58X-)~5oSE?c{O!D(MVJ{rRDHM z#9(X6tYKWJwhDclw=lZ1>~{~7v<&|NJ5%6cyNt)C_H*i5f?FoW))!Ba2lKz1@EDTQ9iQjJvD9YC_Y1VZM35io)KK@7e)1iV0T8GW4fBV z+WGvX0S5D~MlrGNX%*==Iz+uFwX2Ugw&(sj4Kgikf4Vtsn*XHO_{4k3X0(g+|OP*lvXTA`7R2md#Wf4EyZUf5H^dLHihJ-avQSmQ2V`DY;lkTDh z4U^rz%o>W37ZNMf^vAz9u&~PcAREAzaoH<*yBhPqvFF5)Ww3jWkYLLF zC|T)uXs_7vOP4s1nVCl|7G)kV`Qpd;9FNpN1u5L0&%>%=jJ0j8=J=TMyZc*)m3NZE z|NNs>u$B?GWp={Im{%)fR*Vj9fRF#I+(EN)O2-6a=8g{Cot8W3?%dUP^$%3%jw|_Q zT*=o(kx)$R{KVM1;Nw4FT*((jky4<$_a9#OR!!3vfrMN7;J^6Amjk^mUH4XS_1Y&0 zy@eb5gqp&EgpSXW1DoJ0(mFqw$YacBLXPo+Q(~J{FM8?I$u7bHpX`jF8sYv=JF17uXMS=R*K=b37i5aRI>^xN%(nyQu|jZA1;3y0cHr9QoAcQ5{sM1f$iW{P)Rlnt z1~xLOt}DJH?WP-BA>PgWJO5?ycEBR~P+-*!cTNth z>`2UP^LBumL()AE860BxcroEaVnjrc3q9v?#?G$e+SYtKo`+{K=^=S|ZZss3>7#K7 zA~4YFs8)++HUcMa9&>Md$c*SpNKC(Yjm!U67JJD3QSBj@N4JOEvL&Ja+Kd4WJD8fw z%j$9ctSRA&(VrKVP7PN~|GbDhm2K>&H2$v|k;4Rzj`N(kZm+y9QfPpr9=R|4g-fAW zi{Z7eO>J6|!Mmb@WwIX5d~bN&do0d?S17-Zq8kx*Br0`n%Vx`-uQ&w)5)t9r!dq;I z;{16a=Qvt+1|7!x$c2@x5*yUKDzPRqU7)f2yuyYl=^&o5_Zu=ufCzvd!Aoc$D#v1a zXk~&p-tL9O=lTAJ^fgiN8i^k1)_3wUDQG_$hRoXip>Gej2Gu+e8Gg8?C=0Mu$&Wu&X#BlYO+XC-57 z{=k(sUl0wV$01N{`kw9TE}(z5iK4Tqubg}E0eToe8w-Ik{^ompj?xC&Fo=X{GVj&x z%{DJ=L?P(#UDb6AOBH=!K4hBIj~XfyB#>3mUEG&~cQZT2H7^F@iFMna3AB|&U!iTJ zeB7*qD~)zZWJ>1{E#(7~$>v}ET}^Ov;Bcq`gxfPb+pE-yQAi0dW0>)nlQYS*s1Gh? zyG9soKAobOI^?=zy|PvihbpwE8D1Q^)tK$(Wu1viqcer)js~PQMaxvGjfuSZnLIRy zL~&$AB6*367!%cN{ByjlJ5{0wOqxiUsdNIRHLo}C28o&*qYf*zgY*gku#|!XlWt*f zYS4G90h2~`i*3~}UsUvmvv2(4pnv@CrKL;1tDpAqReAqiHTA1-(vVAcjwpNi_RD`} zuEbqhGHe{?kvDd>bjDtiQnBNu*o3j!31?-;r(d&VUHYARcj;CwiTr2y;Km4{qVFw_vMsM3dZak9h%fDcTidG>Nz>3 z{}qhcSQ)c^`HA1=Z~Qhs{@wJ1mFe;66P9Q_M^29BIWob+;$M1>)DH=S=fWRSVz}8O zQ``Sfa$pAF+%&xGvSVcF(2-U z$~*dn>vt7Ia$;gT7xxS2?<|O<<0T5(>c1$6^v27FyN4&lrEMGh((u>I8@unj=_gbn zRy(BswjnH0{N1$B3nS@J?~D4kKXApI`#OhTuoGgJ&gZ%_?^OTwqR1v;@}CbRH}E1l z8v5s!TknTXF&dI1{%TtRe@?F(LMO8nviHUO{`4C{XMcF>V)o%U<=k)|_QE(myXRTR zErUJkZ+K&T4Ol5KheTZlWgwZdxB-VnF`E;FAy2{0=8$~bUM#;V@)Ry8NQcgUY7f93 zPZA3OXMK-JBCsB(KP21}f<{qIM^syxWXZI^340pcLg*rviZlL|%7CP~m|Msgqob^Lbn+gF;HG8;1yXy{v>n;;bOlJrJ>zi6 znz&*j+!)YY!|z+F5%D6XZv5rna3ON=c9)ol-3}@E`pQ2a z{8+jG>$kvUr2A)gU#x8ytNsP9G@T0+yaEvhsY)4UgKTPhI>qq2Qk$Q-AIcB`2A-8h zot*hc+xh6+=F1B;%t(|)A%yE>d0SFaO=ROpmXx;rE6d2^8ezPHrBGTsui(InIHdlC zG(CFWjVT#SAHN8LqD@))Dq)FrGo0}XXL^O#|E5T#LnvqS?dgbRzkS;g zaw7`gsfXJt5!^M$f%4APyl!R_3vI{2@<1;Ko8D5k8zaUA#55GnqWUT0fow1#59A!vB>@w?nO*JP7G zu&T8sm8|%s#N2}KP~up}PrHh`U3EgJ>GUE_`5C8Y4uByOebbDs#WgPt!9$T+B%J8Z zzicoCZKLOLYj@ty?8wEO3@mule+eG^$)f~yr`A^USEmpX=SNjKow%iUDz)|x6WV*B z4p?zI+QS#U>By@oSgv5Wd80)(2GNjA*J&9wJi1J(0bD%a|0B=o+Jz^UYALlJp=Jwq z*ZL?z5{SsmyqEb?tiO&P}Hu@!K6Zv8ct#aYoz{tKd*{Ev-d^8Y1l{QHlKJTNjj@9~0g%L!v5l69-b%WHA; zHwEEuW~Sz)_XwXEpO)9RNBC6yug`$+>ob${78Qh7;BTk&Xr9`4L@2-e!%eI1%gmD= z{{ssCu$twO-GSm(fRsB}9q3%vJ^Tpxwb=^;olkWS&&JDXcqv)8%Mt^{S>sA}od486 zL4`5zrI$yRZw_}OZ~lt%E!}34NYERsa4p{G;vs*A6|14$-WJwgWozd?dRMcV?RXKMc&q zrkdCS0Zk{5dP7cWe#5hrP!w-e#vz>2@@*8YY~CV$nt_rx_0|H=q)T2lEd(l z6_VoPY?o&+Y{7BBZ8UsB^FVjRY_`n|9c^kEF#11dt+&xX&02q-e{78DU#1l13A@Nr z(}F}b*uMUr0YI72?IMed`x}t%X1P#^g2e@qJMc58Bki!o1)9pX>^GK*p+@heC?@~8 z*n^YKSmX!TsC54qu|2m}_iUcYHE)X-b$@y7s*;9&w`H$gUD9ygZ84D}Tv5}StUP(7;iQtbRC^N1FG9tkUA5JqfZrxuBMQDjbFmB$Kcy|5P3i9P5ZQVYn0JS^M#d|0Pm&bqqDXfqtWK9Fq&k1z4jDQ zP%Rs|s!hlYPsJfyyzF6HK<=Uq#0NS5vH2c9OVnOz2v$x@*1vNNg&(7@9znrj8~jJg z9>0oTqQiFCZ}}x72RLo|4Im)gZ#l3QxL1RNso_-vn||68I^}XR|O!KndQj z84%j*feaf#WUqv+f!Cuc^1+H9RbiHmP9iSWVK^Bq;?P$qS_R1pZG=@WB(Fwrooh{` z7}*{Ci-iFNE5d=n+tU9uRRy8~Guzl;E@+B69pW`yM8(+7uucu@cZLOn=4rtpHuly) zD`F%?-K8~V_83-nPrw~;f)Vq*WVmh2(6WbXTSbRR9{q`%&qz9yq2{#01}bYn+|X6& zc(8ruJnqjmkbm_v)Vf{cdze9~th0}UJo_Qy(*$zf3= zMLwNToHAg->BP~}*;nPyK#!7AK|6tXA0XwaAv@Fs@d8PRbX6wNtJwDu*&{Rp3N==M$3jh!fCyLjJGD`Q(a`-h>lqMs+m zPFxtMhD36Q7fNe>kM)vex5tHg^vrGhYgXLi%a$~eHu6tdaoaCl(zF3DUqYWYygV>C zKD~Nw)1!fe#S7;)JrqbtU-(GVKL^L97cFVpioZQHH$t7*>d=X8QbBV1b16BiXXdu` zTyjlsb|C4NTv2%y__g-Yq4LearZJFW4#P`f6xneOUTzFFor9yNnc7IuTpY$fF)j2o zb7_-@_OBhvhhCo6P`|6`7&72*sQ>CduFaZqXHna9Rzx;upl|l$ra02zetg7{#8df; zBiSdzmr^rYNQ?LgV8w3JSn&uNt?@R*oEYZ<={>Rrac`0Z%A}NqtsBK}rU1)z0XlTT z^mZx+^A$%5uYVh_S8$674!KFSv^w#j)bs}Y90p*}8@tigTv5Q7?DS-|IO*tJRZ)K| zn_zaNqcK$+UNT8Vz{}J@az&cuJu2m@Sp&(LBb_Xg_W08S$wzjcZRi{qj}-e?YV#;@ z2>GhUa({zJs*M6!R#iDCK+#juakBGTcKRi$xEc7b;QW>x7IKO;-HTNN1l}HnRx{%N z14bW&bm`%bep3cMBlE9KxxAbCkZP__G>PAmUSPjeIki6T zNbT!*1Lxh^o;b{*yfxPt8audGPWO02-=ua*c{%x>DKyHaH9#RoGY4N}-N$IWy%|doskDum;&BPXM58&h{@yFD^b3Y5v*mkV! zu}3Q!Z|p_!Y}pf!R@D8RUlL*^g3OWJR-3$x$S!9MniFZM2rI6zeeE9y2`@e;var5C z|C_mGIMX*2v*x?4>`rzvxnJ`djM68O%(g{W);s_RFhlue1iQ`$umQGZo0(ZM!LFre zGv5%pjF)Et4|GYd8|Qpbjjrt)N&Sgk zTt}Ypa2;8AVNVo|<+jPr*(X(Imi{CdYDE3o&iR?8X9h!5%C-hCbAzFecjT1jC59%Y z=9Ugk3|)x-4Zb3D_m0fc%wT8|{&s$1!}_7nIUbu6p3-eZY3(J8P%qb1eBY&0JJRn; z$!YuEtlDdbc6YpS-rLXoiP)NwFKbJ7vWoGsh4IfNqW-M5WOr?3;LrozI%YRKjLFvi zWMb8q05Tm{RaTS>-;s`~c=-uiknCPOvF@|_kH?cH3oxj5ID@`D(?SOr2X4#n-<#x* zAAJ2xdu!w;okPd?LAR|t54K!Boz{~Ro+yFV325Zln)nk1An7UmF~-k%zH{+$J7HPI zIEj;yjL1k`XB^r(;Bd=nV(QvhGieYa&g+^3iC$Z&w3b9N17z8OWafOOr3|cqaU)J) z_$BA2i(v!hMIS?prrkG09%5M}graQo*^w6aCi!`mqm}j)(Mr1bkl2bNds^r z6}?uUr={1fH;f}eo;5}>AX$fW*SKKCNd()9N+)0dd9RzUVBFQjSh;#=Z*cU`29cLh zeU`ykzArY!l=xzi2;D|`gDz*y#$W#VkI{{w@l$l;-c5$ZpN-rwvu0#~8KsvMozk(Q zn;0+q)`TG}Vq36@o9Wva0z79TSVnCRnq`@sn^cKn3*kQ-u%) z7Pn{9gOiO|z{qp+{n7HboW%A;8PIGFNYXStijftw&Sl3qqZnb;89*8 zq+l|$t)4k^TtSV=G`AnWriIw%%>s~|qCEao^e6w$@5x3RHNM_sh5!ta?IAAD(ws#t zo3XsuuA&f`?ffi)@>-HZ(ai|PC-LQMxLUGPJUzglS!r`EPNP#MMJc~R&8As?MIJfE z+9|;V4bjSI3j<3s3EA31UmL#QQs5Cn$7%aj<^$#>1yfkWN-e8@X2Jz(11&pfxbqU~ zd~5)|+Gu9iFim-S*fj^K6mGLR0qmOKh1LA75;#r40`OwyMO3yzO>`1OY;#@2YEj6X zCUlUG+gAY-0@6n#9S+B{Xr;u$ZzYb~K-(7$IV#LPQ`W@2luG@A6n^@$Yt%u*ooX|e z1JS?EnVhjAQc!34+65AXX3Wg<9wk{6i5H{wWfU?=uS^X;r8uR9gx&W1M-k6rk>q4c zUF9XDsZzip=Z5wo#{qJntB}nCA>n|2^H3+*1&t}~yxQfQ)*0BFwhXK9 z9Ef(=6-Uo~Wc}=ftl%djgIoKBrzNCq%S)@xOI`bHK``7ed{IJL?L}#mFX|o|4PTNw za|hj-Q#v9TGqN(~qS2w9Kx4N7Wd&nSoqlP_j-tqI@F7Wz{V9A&`h#@Ce+9MM$C3|8 z(}jR)gMy(cFd_L9`l8J!3e}w&YI+GTZz&2jo)T(W8c4XV`jMtRgX69nv!v;(!ExI` zt2Gaf-~Q0trXc?I&?8No2gkAb$jOTW^uMMOucWw%;zyN2>Ai+QdM0*0{*PA<233;EA&3*1HT|?B@&P?-C&GtJ)G2 zrU3`MwJ^SO+1&2bL5*aP?D6j{gPamw204&0V0Xd&EQM9m#KFRMvP(0r#KSn-mQhI21U3+Rbd7My+K=Nuu@Ou=GS3w^} zam}UW5;bW(N>GIN!L4uQvs}`=tbk?ad%;)9r&fLvIr}kT$8~&HeQ30qaiKCsvNY%E zlp_~T(kT)-P{8+%++9Giaz`@%j$rh6W zpGf*;;zlmM)FAWzK%bVf{W9?$a4!nYMuq)(@Z#ucEO`YY%!KgEC|*N29i)pq`b>>Tf?AmiIputGj=R~YK=Bm zHnv18_A(H&+Ae6~R~!NfCw6g7w7^}U`4j}g;_uE-LT>q}QWF|6=MF00GsVWNmRkWy zz3P+#(&{#*(u%hI%_|jd4uYbrum|Ss?Rb@+PUp5ZK7D__R-2NeGRjbRiBzU}X>}DI z-?B&kkhiP6BICwb=97GC)iz=(A%yWn2?#V^#ie8}R+~~Ugc|eRy-PYpI|gUerr!Tn zervZ-)xdVYUq)APM@#r^vP&8&s`euEwqq2*I@-N)aHt_ zGqP#|0k0b+tioQwI;;ZBnG1z>VUYGFQl)1Vg6R<&%C^YE#EVtO zuRfa*vKSy)CS_t{vD|=VqtIJD*y@&;3kg^@(K^~1Z9DK8XzG}%@7%CcAtzv4Zt~~` zSDXW;yAuum7Qm#F;iL|Rm`^{5pN|-gNc3T-gA_oSJnD68C6x77NzYw8q11TpBQI%# z{PDKSI2TrN+0`OGv3-WP!SWjT3of&UCB4pjW87P=nTTqWA?B%RP~=P?aFa>jybUm= zNO-BcCP!7hkB)zV!pxbhe{R8-d8Naol`Qdc%9Kk#uy9HdlU*${)`ScNd*UKaMML&u zjN52Vd>{O4PF?q~PEohUwDDEh7}~ACA^zwV?;YFs?I;h94>XtlG8pqnRm=lLp~@3; zS6Ai^s>~_9F&HzgD(2efM?~I?iG6HQAnB}eB|XNM96KiR=a|?_l4F0mC{TRnxRP#n z#vh*)LCYu7V#)M#3!_@r$oqvTpCvb&(h5b?iHOoZ4^y1Dx8;OY^B$b`QuR zCxTkrml5AFO7zNZ?GxWIq(`VVIk#;CUY^|}v;i+4?-RH9V&KMK9Qfo&>VtilzR~+b zPPyaj?)SD3TxqcaPLn?EvdQw_r(O0b@9$er zB#&PoHdI-mr?GOpaB*nOHwb1+BX5Mo7=fv{4Ts_8A@GQaxfC^qLlyJSgZn;KW|Nv* zP$U*#sBD=`N-g0L8K z6KHSJep)|__8N>#*GndByOcFfLb80nQ7|M+qnraIMVoF*2X#@yo4|$+Es=2B z$y>62d74mp=_3I=p|;+53Tc`7a`GVXbn#T@qsVPWRQ!M|(bsSwQ?sA@_d>rOQ3dMc zDB!jX?Ug-SGw!l=$ifuq?C5!uzrJZa`b)Ib|NdhrrM!L^9zO6}ox}f66HZd{FL&U! z45G}&3rDnnljc`<^)|eM=KT7t`YSZNvl?fpE<5ME^ERL4{vfXQ6ox{hUy&c@N_vYt zIH=;zIpy!+LOx)bPx{9u!@kH9vbo6MzuU_%|JPasTaKNh`Cn@hsWh=eiY{A=eCa0< znf={D^l;^+My~6M%9OwsxUqcT8GET+taBkT-(zEMojd@_EkE*yJjMD>K7x*_w2f(C zD8VV1WH$C(&t75X%Bm(c>MbaJMg&{yJr@W8r#q=2Q$JUbJ=rB>(`QXyaIaF6c89qq z>qVR-o4HA<47htiwr)cF)S4sWeu@ElWi-NyPS{_WQ!OTkiv>#psRr zsmQ9KIqk15Bq3oW>7tzWjr_8t_57zkLxmKB2d^eOe# z2bOm5o~7s?&_(jGlZRd`{_fyD2lg_2NzmMt&R=$@loDy<1Cp@g9$#)p8%9jQ{l+ zF?>cGLdgI18F2!I|1bEApkedB*Jp%9U{m^LH%p8EjP&g0$15qfT^spyq}VcS4k6m$ z7E!IV4Rda1ij;%A1QsGPcJQdUxTH&N-emXX?K%LGjXtURl@KB6$g3-J+JCZwUv9+9 z^Z6y`As})TIxjyQXY9q+z`%$~2uzcAhZby`fdf9Lk|Q!+pX`E2mG@QDn2kG3R_ZPW z(N-qR-0NjFoTXp_3wi`-fSe9j0c#-@ulJ(z(NoaJU2i3qZH+apzf93y3rNJNr4^x+ z1cxkmpel#fOon8TTS5xv0mt~FN^WWWaCtQ~tjdYoHmpkDWXL`>(O4YY_}L`Z9>Xe& zB3%+;^A53BrxT-J@Nzjn$BY2GDuJBwkUji#j^0g3AgP`dQUKqB%^XPlswvyl`-TsEW_rC zHR%WBmZHmSq+ucAeBjOZP-MDohjPd)Na8*!b!xe<%o!DGZLv36DH3ZQIDS z+L5ViUnvNlQy4t^!_3m_gE5yCg?fN}o0~gmZcgdXf-(K7VoogzE$@}P`o2F0j$IUJ zIb&Q&M^WU(nAo?HV*gkjD8~DP<4W2Jb{##4c5&%>P)JsWWCymgnQ@_=skv=$WW|^L zIxe&ZFJH#XU&e(hQKUBPiqHlWsht)KEyw@*1w)mnRl6`T)B}IJ;EK@b9S}M8D~e1n zNNw*0N0DCy;6Itr5qSQ@x7Pmz)ojN=7l{UJ`Dnm4<&MnK)L^K5Q%zId?U|)Xc=>Kk zQ{!!!rN`stJHe*Y;eGPd{8tZ4!pgH`ZScAz!ecXrWrWOQBBp*) zNGi?uG~Zn)F_}C=gs>9&#+k4T62)a6@Qb==e}^A{m84Yva$qWvm@Y$U}=!qWa99huIqRFu|ocgZG|R1}*Lsgg=M z97mT(KT(>C6y`lBg^=*OAG3>Wz{G-@_oxa9xCQpw$Hn_A$xQ4W1;8&iV%=ssl(P5= z#8mgwXHkJ)u@PW(lLmt+|2$QaWe}{@cV2P7aNithFlcLkXE12D(;-r|5z+Yoy~$bo zg7L>!)%h!v=^N)|jBYqQGdu71bV5VV+cF!R#cDi2R~uhPXWTEWz%*=yD>)YDU0R`b zUcxn+6?0t;S4(FFA> z*FqUHvcM^<6_YEKj2Er?9=leILbhj#8pnUKlnQqMTmB4TOyV3?^#t2+wgM?Oim29< zPa6@hY;Ym(5iSOUp*uOCwmtyX7<(f1oKq!~PJAH4VVv?3_c4$yn9ML(#m}&&N8Bxo zi}bSvHB18h3bbclM7>x3_Fg3}#2_0m+!9M!f1HwqYKFGUW+Xqb+)C;&iv*4ZL_o;w znNh%Re2x2IVWISXZXcqMS@xljU7W3Uld|4w2Cp(s_FlawJkymv6v_G?Rbnstey)4E z(T58n01vHl2Fz18&blVhRQ8L7=Q>PYSc-tBDXy^7z^P<9-$4PnE>|OGYZp7TIa)Tl zTu6puie1RVFW`w0yO?*xy&^E}V*_yzp3LB%?c1UT#Z+%4Iv6GuE?PaPK zLbHeC5bH%WPn;>H;Z_!goj++X-F&1>LAkknbsvo4WZPX8&`~Z2kV$?_Mv)EQ9W|rK z=F%T#6dAcU#=|Ib5%Ngk>FD$2L@@bs59mOC8ugMhCXD zGvhL(Q)I6t|5epUJX=|}xCXL{EITqfR3}=nP=ZAkdy43(-yb!v89Nb2Bc_KL>hlHa)Ua)@$J)JH*hC??E-chL-Gmb6^JRN~U&TUnAt!<3(B+cpR}XOo+>VY>h1bD=bYkG>Rw5 znl<28e;5ap@^R&?KuU(FV!PM$@}HU$6d~Q4-(7JrtWjp?;^IXW-c7U{qcqVbpAc&m zo3W>%G{UT6E012qwr2^Xx72p6Ci0xI6Kzs$*j_;uTe+)>?e9+f2AnZkRS-`{&WsoT z<-)J)7D_?TF0S!HckkjaE@-VhNUhg2sVzj6Ed#IL_i@yoYe)K)caTcNnTb>H!6(&! zJi5)}b7Tq)BGJYt$Mh*bNhD=88CPN2-+0-KcG2g~9^GSmU$BceEV3qfqxc;-1(`dM zOPbOEYm%0$eugjZ1{^Zrt*9S@Fu(EICVXCm{3B%xHsaj|CQ-h-HnK)^+op=_`e3&X z={}~}We8x1ZP^uES+_bPNJqhzzIqBclm^Iut8)i1maVSs$(0_W&!8BKk!SFwvbN#fx09 z86~RWG)D}FrZ#(EYBM~84@cukf5)`bGJAX;u{xc@|FS{M6|G+lwmvXbtv?&Bo#Pk> zCZNSh?~F5MfOGWajF>p*gYqfQd=XW6cKCyH*zQl^ZA6NJW$dZSS6+QToczigKllh& z@*-@p)zL|H(|4|VBmjW#bskPaT0#E3P=A^@#};IZ+hSf8aME+-BEZc&&)Pp`>OFvd-Twx>V=mV+6abE^99T6fuX-7-}lPdF5Pi z14R|I#44s3ry!AKMEb7v6}uam$=RFP$7OkNDNw_HE3MO178Z}8298#@6)Bu=| zSiwqR9rADM&sa~^`;OHig)(2F`$ZQUG`#&6;Y9+GcyuCe3R&*@aGd*F z7uR>5)O%EN!?nky)fT2rE=!SInrfQ*#wNVmH(}+|-NOCgQZh8Hc4+F_B?ZBy zBZE)(3-60hn>-{HmUY30&O2@O%$(98#JQoJ_QkKvB+tmzGc(7|U{{ie^dhMr90=cz zX4>H4I%F4Za{KRkPd2iTtcQK%jBX>^20fG+d}w5N)iGNe4y}G5Gd%E^5pApQ&kV2X zEQ<8(8@IS{Nz+a^kPH^3Zc`Tq@`c-fnbAcXQc2j_bBM36bQ>^q@VbFS37yxMS*G?Y zD=RX-siJT{e_K}b0JDF#K{}1i;LEk06F!thT-ff6p2rjQcGKX$fl`X%Mbbeq#@CA^ zUjZ*(=Umw9sojul@;WIULQL+g;9365t!n{e*lYzs8@`d7(L?`ia#a+JnpFr%C2ahj zYFS*@zxn!3{(Fw7TNB;UOeRb@keG*XN@HidhJiB=)%W16^hu&+_LSI|&GZ=q;pp;d z0e_tqygY6MUVdPyEw-1msZS>FlV1a2%GJTn%ZV1{LJ@)q?d$np|N3YGbcLlSiaid4 ztV~Jz;E)uXeYSph4Dyaursw}~MaIQ-$@8DO!;Lk9xvd5%mwp8fr8a*BUU3=fv22!+ z4-?~t|8lM`9VuWGisCY&TSk^fwT#SgSw?PgqqyRzmXTe|olz@R4Z}l+lzhvr^p|^I zjF4~n0}773%*|3ozeOTk5*{Ibsr2*Nlt7bBM7AK6(%NkIpC5&y=qBWXi=ME>e@B*v+4j1-_jB zmOW|^>6Se=EIuD|c9Ul{2Y*uI4KHDHjG!AEOFgGwJ*G|e&7Sh})0&H4z26Y&Y{n+K zTno0$$W@SNGN#bP80v)9-K0l&_pZ8M%=oMJpx79Vs4 z$cPM*%$4^OJaMx~ReKYiB#xRA_2DJwA9qTI;My&S+GX1|f6 zb*epv^UE6pkw|z3w<0F$~#vE2poe5D@A9l|@txN5lndmfgZ+5P; zq!$ceqeHb~F9a-c^1n+yEquHLW)`=#h;20me`=r2)|`o{G%)(gsHf2NU|QTbW#dNY zh7U>*J2b~r_ISg(B1Gv`0lUb~cCw3n_GsBf*8SK2^!@tifwc4gBOvYK&L0HQUdnqt zKUBl&#@ZqMw+-QN{&&+tF9^ZoMg7|gm*=gkpHX(O2K3sNZp=t6M@idcVMr3jQN!PV z{)Bra8#HJRb4EKh1iq;xc4&3^*g^N<(9-7I41x&9G9Fl&+4Ef?oJ{8TW3zhRvUL8w zb^?GOEsFnA_(>)|QIf(G&`*t~&f{P=!Y=tTPI8%o4E`jEv6n^(3GZEkbq#xfv6#4` zEzgCvaA;}sgTiiv5trqY%=dz`vO_#X)<8ajxno)j0GIKg5COdcp&h{oDXUS2iWixy&~&9u>c!oJfO+a>kG=4F0XdYrM` z?o@PJia12!ax_Q(&U5yX1?X5~C;%|@$ohU&@o?A@v8F7VY-p?dVTiX8TY z5hAZ386_=Lc~9X`PQ#XXq@2hV(MqzFN~_4Y!o#x+dlFP)BSXHr; z4pF~BJB7cy%Q>x9A7wDDC$c)oS$XYiO87tusketY3YodS@Vx+jaHck~eH zdXF0|K(<;A@ZmA%FqoE_GEobq9v;oK{)ce*a|2iPeAya1OBI;e)m@@e?ZS>DBDlJW#0ILfGxA z-Id&wD5;xwA^)Y@}o(LXLRSYKZ_>#70!v^dZ{<0v6oxYT{zSQVRYAI zkej_eK|ZoqTQRF%Eh4)?QZT^Ff_0FkUuQ15SKcAXLoYGv4Wb>S<}=iUG221fK0}~% z{&cn-!`Gl^!Li<|d5s&u$k-fkwg1RN)()^9jw3>p>3;Uji^{E6r}>d_0?OIirLue4!Sq-IFhei;;3lt8m9F4cQP z-Q}zPy659Daw0JShuzTSJkt8sX`Sc3PRyMhrkT}j2cUns?n?+#{2<#BoV2~?PBg^? zsB42XlE})kldCEkMJw&HjH-$$+)Dd8NNxh%Fb+hZdogRsS9^UUC5~)9J&>~gTBAR? zyT}c@adkGE44frW>EI{(8T{m6I+S>5$NIcbm8@|at}OoB20dKB;|4QN>co~mELtbF z;;5b2JPb&#SFjD+ZFE{~R#aMTii=kJ)QQ=U=kkYaLJQ)*3=0`p3invnrg~LY+r7@p zq#0;6iqL&&H4zwVbn$YF^|nNE25*zvpAtaSCCF zTTDYl)!~aXfPBt7PqXEpH4v6?QGz(w9$wb= z%*sp(7}~F%nA#x?NgfC3wq>Q*&oMHvjijL45^tG-mf#{c1NSH$pyuvm&;R&D2pdsMOs(#d?v zRp3@A&=%w8D55fl*QPnZ5<)Y6rVsU8Ccr!D(-#Vq7<~n7%ftmQlX6Q^bKKk6JA?AqTkW^$_M#NBhppuO=V7u3a|IXkx}<|_q^6Qu zwKmU;iJR+YceEWLmhJo*XOAmM9bb|%Ch|c{?03nrUoHx)g2(^y<4caaAa3#GCF@cj zZVCe1WhT6v8GqfRCF|mrtvmMNreDM+oVDa9!EQO>*U?}tKXvVk1;ITdga7ImegxQV zcv|n_$$xH5yEx|6%9s_SLmSX~ZC37}SvjR+f-!SPhwe_x9dvi@>bs5&R0GR>Gp^+8 zqDUwvc79^)U1+^FU|h);MUhgpVtfBL>)xtq`T|(45B`f^d^xpZTlZElSvF;xkvJk! z5))f44M}E?Ol|*DR^0YWmo!xa%yqqwcIBNYRyzTG++soz*_CYyy0Yz=deVr{lw)uV zjwDmyNHSn(a^8Z1@FrBS%@keP*7u1od#rnST3lLtPLI&?WXLp6dH>t>m&I{-|HQP= z)6A?*9@@WlC?9%xT0`+2x86s*STjo5*pS4dl&v`ploT^xJ_xQZI>EL#8o>tN`X~e& zXf0mu?0^e1H@M9p*k(s1*y`EO--LYv37YS*An?cVf*?N$F}9sxGr+Mo!PZH_Pk(|< zT5%G=Rzu(a@nBKguQC~Tw&LYHdbtamGjnv#dN}nSj{iLRLgJ)h;aDU#AujX%O+H8s zTU~qJ{TzP1YT&uG4;ne39dt;hihb(X3in-c@hVdxeWii<^!40po{=cSaq^4Yl*rpcd1+NeeO`bWDvKS7 z5lwp=Hi+5@CumSQ6O66&1jBkE3Z~3GPe3gx+%!Z{CXXp1IlLEalw^e+Ia2=Q#vA@h zJN*bWw1&fwFTq9;wU+=7VF0KxNh;DZV(j(`8^pbNYV7B$US$^97L16%*t{b+rLlAU zhVLhWG^CFt0u%c`$*9lUiX62MK$9zk8S}dX2js(B^6iH;b|dVItp(&1K%;z3kY8#r zG~+|32tkGqRcX$;1*mFD7%m3pgH3yIzj769!; zZZzMk_Wz^dEf+WJUUG4Bgpk5^OQg0H?T4Ez4X>@U{>24r%O8^h|2zIRx8vIDuNRf^ z1nJBNN!^Ogrzah%HWE6{Y>qttDVTZ1+5hnZ%6nN6+JyA|jpt`kO|~hzR`-af)BoI$ zuLEZ>_4JrKj5-8ovBo7YVtImX3iVRgEzaX-eNe6v3(nCI>ec4o^f)rwbGkDyem-wk z{k)lUecQt;rU~CT)B6CThM|aa)iK|T zi{L(6Z?)C8^WWdZMo%=OspOB8jD*iPG<(yG&f^};j{EfJm8;7=ln)@>Et-WLsb>*IP-=#~B45!v?#}yw;pm}zo zXw5bYMQZN{n}(rBcWuA$CgB@d_jy4i4bbw66`Q_&yySBh>v<7vFC<0&wUelv=ejS0 zYOkUWjkTxsWth+d-u^ep&TQArxxYTJo|TWDO31CF4_-DXICgcA(ZiW<2hL*!;GhbA zKjH1bwa;(edGI43q#teW{}P&T)3gV8ku#g#^Gd@*Hc?8f-vtit_vr8c;DWW!V^PKm^kzM2WnJ+L*8U9<)CTla3milWUiG>;gt=^TqMO|(bJuJQo?|C7=aUNadns7zyj>6LarVLoc zfX@q_bvMqAUaW?wn?g5lO^%?uuIzb4oB2X8ED8!`u%O~JTkvo(su?bevY`#%)oR4n zu-vhKBYGb`gB(IMeWt8?3Qm!}VJhQj1_c(Q5p@}lEsc8LY1t0r{Ejcj&H12(c&+xW zS*abn7qPEMCG18vjubT1zp})yu$Gl*_G;R}4W*a!XDh-BY=R6RL6_6M>>f+NPuj}e z@repfjG$WcCHQ7VF1(s~t7O)4*4=nobIU7Sq*ngKqZRq@ouSk?(&pYDs}=jmb81le zNVXkX+iLTE!KXw-)+!J)V>3a|K*%VmDaZ3`HfA;Q+4={QmJ&+^g(jqD9FOfxp* zQcsx(c*XEt8w9DO`2@ln4n3H946XSA%=!$G&(@4=kZEO-O8Qk)(NhQS`3${jr+RKd zCO-pKi#JOX>J>o$#akhBJ-<&ngnm>KVrhOA4xxJ%H;2@*67re*7rqRpw18YW^ab2( z7#cj6IcP~4=PLXMyceDR#bo6O+dY%Tk3U+Yv1C~BN9jri7gF{X%3^| zb8o%1a`}}An7J#BoPj2>+|hg-=;g>l#)zpbM#zK(YQ*Re6qUkHNFm-F%6VBc5ki-V zd#Sm@;|tsa$#G?cweJ%P z%ShZk>j0<$i+l6Pdafn`@zk31D%{~bPNb2A@ko9BZ{BxwUvo(P(G;@0<)X;PV;qmX z6lK!~aMgePq+hfdx(@Y|tWOUXhR*GO|4e5VBt#F0-EEQ4qtT-V_x#hVn0%NB>| zq{c&Eiq&GevKTqp*zPybA*>dMAIw8{Og{h@=?d3Ip|-#Ce*#isJK}TJE;jc5?$W70 zd;W#8bPe#V`DErQ(*MgqWM?aoHUWolk;)#uib}`x>V07^1a}263#y;OwvSWf=4{E{+S8HJA>S3}8NdQY zO|ogTyNs=dex)IY#KY1>bnJl}fW{PQh0U}7M|#H)=pd2$F*hx>kC~T+tm+5{=3ZyD zU;Yu*8kYa2S0Oocmp}GsMg2yGZ0$Ef4|%@Xm5n_>o_xfj6xBpif=82(K?P(cgvdrK z_MdZ6mI=ues4)XPoWkZ`J`tO(^28cY#3(cj7;&ZpZYgjehe#1u+E|#LL#Q^1xUyn6 zODz>o9zk6BV>rXd&xN7OdRfO&d0)XK6A1OSo(dBQ*@ z5L7Iil)gtzjli7+f=n#I5o@iFLh+}eN12DKLt3@X@4?{ey z!5GGrH5^IQBc^%kfV#kK$@RqpK}(~2&!vGsM-cs*Nnt~KR;MdrX9tA#xO zykvMHk~W()s-cokGZuhM)IxSa1&N*U*HLhW%_YRTw#p99Aj_$VGxd@4T2iqj{~OIl z?l$C?az3b=%J0c12mEmq39p#S9%j3|?5b;+P(eN2bf^z2ur;?x-9_L)nuK<8UT%1_ zHZgrBNoXx?h=&xt8VLt#2R8L(PB6ea5;K{9uj{##b2`Tbjj6ZHWqTz@g=8wpmI>4BGh)SX zSnb-z6!{p2*D@aQu?|{3Xv4!!^i9@?_o#mWz4ea__sIr`QU0>Jc)dtQiq(ggw zH7uv(McDpThy&$p*uPNKw!ud$iQIS8s{lD+ZHW) z%iO!u;NM^&-&%iy2ubsvHDKPgMnkr|HFzJ0G6pErTkgn3;~wwHI#Bt|kwX8H0dw{K2e zNVXS7FROs>iJo7QJ0-McO_rWi)_B6)YS^$upSYCmU`cWrMgKq5t2o1mncLsX5_@>t z3Rp#cBIvfHq?!o2Yf}}Ka`_hM)= zM23hrcwb|We*^=iFyM+bKXEtqFo7#%Rk7Cxl~ajC_5;ZD zmEE+iN)jL}yRz^83yTCdJ`HSCSQav9Zt8LYgSxUek_@Fa=W3lm>v&9|HcwT`aU zw9_uQ@nShM%t@(5u&n$P8I6FxxrJ;crxp#BoHN%J14BELLMm9;)rgX|Gf~kt_UKi# zJ$Cde+E%~uAAxB*e-KPN|EYhW%Wa~d>w-0aN9+Mlbg{B@EY#3}iBMsBS<1QD3t=~1Bl=ZAmNTE_}W%~=KKGmhH2 z;jp?&QQ_Sb6!RQl5Jw!-z^vB}JyW_qmW$%4pH@-gv46 zWT12Q`j@rE0x-`_$XWY50=UA4Cl!!eVPLI|33QRyLUyRO%?Z`0UEB@E;NDGpAa8>= z^X9jKb}RPQj^aO$up4IU{L;M$~LP^L9|dnAy9}GYolUR)K9dd4BR^tl8MD zNLU}0)fyF-=JU#0F$?Z6to}@Ztv*QxW#E-mIi;qa4~O8}KZVCN^<^(pN1W0a#e)x^ zXNH*rcd}^oUMM$>bpgX-;rzJylgi3lrsVQ<1|f_DB_oonAC+ub{eY zdFeROJT9=Jh?!^02#OOM=&X6^ek~gTsNp`+`upB&WTc6}MVyK$+4#$<_B%tKIWVk( zNQ1SUFv+Xyofv*<`}p{BWlyAeiUFF*>%VdKK@5R^{i^`pn$=1y#DH9ww`!8|1H%tc zAttq0MInyVPi@|Q2z#Mi&IH8z`*y1kj9OQj6i3N294YH~!bAbY^kp5-ka}ztDdv#` zWA4ol5YvMUj7h?7H$7_o7~$gSIj$&OQwc8HVyT82+#4ppe%+PLYrwt1ZZ{hDCQ@L| zVOQ_PMbw5UsCcW9C$VA}_$YCMerl_2&1&;`^YtB^lyGJXNH96))rD-9<$S2sdzv7( zV;14Dc5aKNo*Gt?2PeTw@}{F_C3(luvy#mE&#-Gdekg1Moy8c<8@9n*pJCfrfC_Ib zNy9$hv%al4m+bRHCls=ta76#w5$tx7#qVn`?vJ*5I}egq{>rSL?`CmPSmwFmY{nb6 zAdN&IqG4&-EW<>EH)$8ab?YlrOw=Tk4HA9`r~Iobb{zT6Xv(epAWbkvvcvT!muoBJ zOdgH)a)oN}o%xfJH>iTS&{ZE&rwf5(z3>_+u{}}X8rB49^;w?~$R>9b52ycPMtn*X zDG{K0kFEkd`a)^Nj2>@oWH8A$fYo8klhGe?jVcOs_0xtlZsIYV>HlkemEQ%A{^2k3M@<=%OH+Ft^ zBbROO;`>0EtnErFIfUdS0yWe`mN9gmp_*y06w4>Z@=Ru{cJ3%D&A*~z14}4NFHS5h zof57X{dr;O)NsZ0&x^Qkz^X_zjbujD)mPL`dDK=h@-wfYmQphjhC^{iGE8k^GQ?Bw z)!ym3sR6YW8~c%|=TZR|1*7a$!idx!iDD0=#?2T$IweX2S8!|f3>BQApYfi#HQJ*! z+}bS;ZcV18<+EicC5d_ms-}3t40zRm$U|n34sMM$TP?TtBQ+z;tzomTDx$CN(~V6y z-_sPeLnsr3pd8Gv2rIY^;Q3uf&YF8@W`-fetS|SE$1o$2$kM<8n-59-liFZPAu8Fc zQ5YoZQx1d3M9@C$*2c|@Q!w>OB&NO3lKknK-T&QWjQ}S&u0YWY5dc@TXjj_j8|ilQ zB1Y1I8?r?enwfK2GbM4zYf58irBSEVw@*K`_^Ne0Y_`>c*Ii?-7%rXajsX^1jXDjJ z5f$2+C_)_)CL_GHcs;jJKPn?n56ryUD`qyyeMx4ya2_w@m|IcIx#2$IkJ&xXVtf|v&+m~KAIFh`L1bxA*46=s zcYESAAz^&6x|IvIiA%G=Ga==nLuj21Nhd#uKt{COaIqS#Nb)`P!*F55g5&v{@z-Jz zq-;_rbEz&)3nEDR7`JtZJj5bMpk#K)BoQPjGqvXaKztA z(ZbiuKcYv@nfStk2~fG!`%N2QlVnAFTb;@ zBLAWsx?FoR97*;jb9`MsTMi>#3X}BDWI3c^T`1y|JzUMva0Ui!*eUl%kJ<}}AoWT# zbxVK;y&^;YstnYU0MwY1@)tpp#?YWAZQ)xy$5jwPV!IY0GDlAq=0@UWh#*B-g!Av_ zdx#)uH}<>_Y*)BflrLExDk=Vw@fT;SH5qU9z+?0kkhVRw3UHBQibFLG3jUR55mtx@tKg3rFf~lmd#T=r;M1_8Lv*|l(#1NZcY+_#l|laqfOphLA(ydTiL5qsj=ESy8p5ts!CEQBtT0Yt zdlKi9Z{+6wQ)&?qfB8}K3sJ@u7)&jcn%{#%Ak&bUv|?5B$qnujY7tn7K;0-8PL-H2 z7g8L`63r?+o<+7E`3)?*I@N=kZIn(peMmYxBbOWgA)37!hd^|_!OG7f&&5N`ckxuq ze3x@?4xV|nZ)DHV~AVq*KJ#GY6aNP@+G()g0ZF_FK; z#J-gr`^Rx}n-;*oKPx^R{{D%})*b(F)19#i4`wFZk{Q2v;*xbSU#IpiNL{<4Ab7A} zctt{5@BFlF!&CEKC0g>tIyxA0 zdu7Zorx!%3Vq*W882jA9z(831?;2NvLa=*dVka&PRKwqY$JluI`CoU1@bj-}`b1j% z$8C9%+qEGp9G5S@e>p3@zj;0~E;jELKA;6#w%r;@_5y zU0ufS4{r!`Cw<*13-^AC3vDZ48m-q2p_7@&>3uQ3KmCT#*&p7z7!=eXAEcJu=W8k3 zlz|w__BalZs3~s1Vbfk;rdt)?B`VPRv~;0ZjqBQ+FwT5WJG8;4)o}CxebsNPO#NkG z#q*+X43B{{uCQqiu`tjnZBd$MqZ3}PMZ#)ddSNwiRL^Ly#o|Ve-tIwLNBZ|# zf=@7(nE0FX=4CFM?cG2dDe=**xr}l*Nf*flk#h}{TTMOLpb*^^O$FJ^v|7KsXua4f zqp)faZB`Fh6uF1e;uO#(<%-RILGSJ3#0E!>BGbu#q^PtAI}9(-?AT6)YpnCQT2%l{B#LTzudy~l?alpv3B^~xy&%#Mir@5hO-V1PDc79V_Dzob z=T95|eDO6L&Am(kt%|FCu!P9X{ur4^Km^{F=*zkCYOMuCJRuj}6QasQpo1hwFa^b| zdlLXN2hB|uYXN5vCC(otc42qeXM`agUnYxUDk*}j+J869BG;pG?Tf5r8mo@)*hBK) zFBbXi$W)d?DX1a0xJzKYQ~tnjqpL**?&2P8W-zAWGXXcrdZcsvk*3jCkDz?EAGwAa ziIzWPYqel1mwpqvxwXD^3haJ33Mdo^*>+o&&HmL?I_umVh7ZnLqTZj6aAWqS$M6&F z;i9}Lq9l+6F};2*RBVptnR-!)6a*rLYkM^Lo0?gS3p_A1M4XO5TGn2eAqr;kKRr?$ zJlc%U0I}U!2Q$vS#<1X*ji@wwFPx&7W;2qrMZ=IAQnK5P@Kk6AFezwc6H=(##J7zU z4g)2jtxLQRKJ%L7c*ruzE8iYRt2BZQBTEsd8+%x>g|I21XZP=NfTaf@5B}==!6$)y zR?m*+xkE)6h1%{(t_IlVWU_#$W_K0bEneLp)kouVM6j*NY49zgQNx;%ywA&%H=Cu> zoqu_1g9j{@4=6Ud*727Z&%XH-DwGYQuz#?Eb9SA-5|WI(>K;Z*R>u?)SCT^v3aos} z=^x=Fez2_1RuLQKGqM8l)Y5%h6zwUU^y$~_cd|pRR}SbYH$Ij~T0QARxt*urhGVT! z4{@bPBWZt|C3Tjf@9r$NOmz*-Fw-HF)j1Xip756K58ItlRqI?};KRnZBRrs#(@DT2w&Sglw7Z<@Drkk8Y0So>sdebahwtI!BEycFW zd@u7}4@Xawu|>znLY(c~A!WV}`dgH-9i%e0XO9-$R(H+h{G%s`TsrGN(p&q`59_Uc z;FBY%5B6d8;k)OA-)FY$tiCOSwKst2F`)%Z&PxIfCX=7^3k&B#c zl6)&sFxIhGC@(d&X?zIm*W>tn+UD)lmpoL_*qy31ganl8*9ZZ(pPL+Ce}VvX!|^hi zUZNn3tVoFTu-D0{_h>CJgGH0Pu@NiBaEA{WV|(b+331sAeH4+1w9`kie&%YWW4y%3 zlFy_=efDpvoE1p%>&(VYwCmC@UlQRy%PlJ$^~tYCMYO%7Qp znFlZIMu{lO&-#?GktKce!NI9E$H>iV9gdMW>+XI2a!Bm?|5R@B`gIp}gBK!p`A^u5 zbR_Zzc}vmPS;Sx}y0itH;GDGD4?J5nydke#hWbC{N}#rWTin~gjwD9%)dJ!(n?@Z)e6kV9+Y ziO+s;f9D{@+RVNQ;>kTMW|G*_&{alQ&RM_;PtFb*SKa#NK}RkbQ&t>uW{eH3>+1U_ zS`*^DoKiM{b2mo(OQ-zlJ*F@=dqttI8LNXwmV%-@c>327(dA~GRYZ0Tszb;$-;GF& zwEiRn@n&yk%~OLz_%2^*A3Kb%wCn_+j%NE0Y#BAyEmJgc;h#wbfd2}pW=f*GiJReW zuRMhGqYFq!ALC}e)FD8v45|5AnJo*Z)F0(41Kp%xLe)@O@&`OY{$T{wYC<>^sRY;% zm~*R0Ejx=J=QW?L53Y>MwQA}(1lQ#FuT@(Uzz~@XR$Cj}C7u|IJW%qN;K!tizWq^= zy_?}^Llhbr;CtaNJWwcV!)*5j5db`n(yE+nM<$N-nMRmke|CBG{yy2V3u=g_vNQB1 z3}#OOADud>H?{E399c|_moB1U4F7rHGaE&we`a9-GW|bhH8NdDv2i08+&~2z_|FQE zyRsl~*2|cbx}T0mU*@Z@LY#v|&aK~pBr?gI*_Mk~^EYC_a4ywWNo0O!0@|`|IeKl` zzBqbq*>3qyNFuj399;cCW_aK+py2KY1=m>=>Df1Kap97tov?u%+#}Qq8_20>(6%wy zbPgJ{J={G!Aues(;1i~|f|24d-V31(UWckF2579r^8{!T^#ptX8ePB-pnWmxFOB&d z{pFB*$3g0)fnVmKM+Y3f%nN=Ad;Ce`XRYN{jSSB;jwCp=^wH_Rg8bQNqs^*L zZ5!<`;Fab{q4AyutOhWRHHw!hJ2Vuyl>}pHp{nMy4H&j~q_MAzl7~PzjV1^G!qO9s zYO=?b$$uXl!Zr3xO&{@QJx1)W-_y{XOdmlZUQPZNC9zj76D`-Ak(EK8 zaa`M*oOC>y96YsXpbd2U9XsH=^^g=FE0zuCsV!d<#BuCR8uATfb-asPXF(m2%j;Z0lM0eao)f&+R{)`&UY5UE6aX|-H9?5~VX&5!w z2#3gyGmWCa`Y1A>!Enh{4xB9z3J4n(6+p=LL?3R(MM`Oc|RgXA; zxb4|RB>zoW$Iua?$DDdxY)=ptknQ{=d*l!hn9_1(nBF*NKg|T(mTlr3b(*nO|YmjrBHIWjr7g$s#g7%Vw3`M67l$^xgY^;3L4P9d$K`{qbc z&0=X;a;Jr27;K*P_;rEny^I?XHONXv_p&NRkXp29n_HNRO?1yq7F_$br2yYXw6bMH z%&DBckdr*Z?p~x0oR6j8jnP0GQ`#r>$X&RFCfm9!q|#)T&pIW&CG+QoSK{H}ItwE? zrCmEL7DVGOe^cMA22PlUjevdjs|Ot0nd}UynK%sSCT`A$zsO-C)XqP8zL0-DdcKg! z+sBedvS|i{kR=d8&WsD~OwDbZjmFu3%8J{L#@W>;jESt7pIMq441K%<#c+wCNvXM| zLlZ+6;(vp$2;IFSvotdpnuNcdpV&}&XCS`pgrZPmOeiw-lo2tfR)(f@dvV^LsVAYg z_Axlt3;*w|X&NvzId4Hhc+>4UZ8H-CEm`9t>-)r)J=Q%uEiSD+r$=acGMap!^8UAy zo(erNE%fwA_Wl^!zji2#6knd!P<+R&_hk@CG<|61a|rRheIemr!39>e!};ItnHD<0 zm)n-#zc*2HWevjuXxQ*GahCOAo4U-ZDyllMM(&h*2$;-niSK~Z_{kN*UT8IX2 ziSd{H-&=UP=XbDzlhK;E0cIa>Q58sw!i2cY?HSx)jlp%d{D zL$?0>m?$0Cgvm(bwF9J5#KXm(lp-_E-}CVpxlpZ$$5u(VBkw7`JzJ=34N-IBr?>Ta zB1MDOHosxuHEf=5EO}4!ecZvlAD`|E+UQCns70pD%|EUMK(6Yk zxPfQYMjq^gyNfdKHCIrniMah9E!p&euUa5+S41xB|1!Igj%mBM*PQwGL%9jReqqmw znD6S(S@zXAy~lsDaP61}65e}Z`CkNEWNvaPTm!c(in}2N#-GWKx#qvMZpNQcg+4WjNm02_c0(pqBDt$a6G;$Vn@23DAjoJv?dW+Zi zBD%&K{z{qp5scmQH(uHxdu`Xhduc=Qt3uR-xW%; z?PNUOb8dLZUo1&9)8Vp=&Y@%6k)Qlr2F3NoBlqeGvPdc>mS>vbQ~G`(mJ~-yLZ!wL zKevv!wnGTGT=@qXYNI`L=U*Ou*RD~O?PATin||H=C-Lh%>oi11k2%dh!%1E=8`vId z`_XY`KOt4(c-iD}RA7x9P&9iIL1!Jpk}1z^js6i7>wCKE=Cbj5A~YISPZ1L8mAc3 z)GL7jyUQp-YV!AVuLjH(3ps@`3&pl%CJC&015#=E5SU0WT2}+n76dwBTXqw$SIJ@ldz@sUa%!vDRc0#n)Z2be(C~^B{T*=o(kx)$R{KVM1;6Q>Z zw=aq!rEny9|KW9S)iiw}OWfiYUoNWNg11WZNQv$02M5Bpqrh$ZqJUBBwhevOE(nB2 z#;2`K9}wO#GkNX91>r6DUrj-H{mj(0r}hYU$KOsF5PoK6^6-`YiXzhsQrmkaS0rtT zOWig&tv!C+>7VZ-Y?m*wT|Thg6w#Wkd{a$R-R+sBNx@M0yERRXw`G{2W{KpxeR}Q|s>#2f6aJ77J=nKr$EfAQ*ENFRb@r_ zM=2zH?3jv|pYY4wi~nuzB*R}Pq~hJ6?fKy!a<@yDP?8mdylWqLhc7}@k)20l*$2Zt zoAbY?W)j$!S5P~OR2oWTgEc6!mQkd%{b$FJ7!N;x=E#*}h!dM}oP_URPnyYlq-JtL zW-^x_o|7D3E;WfcuxVizvVx1L3+#oPYug| z2>Slz-(dR`VD+Rk7DasQBag>6?kaZM<0y7Rd02H+9uAo||86d`Syu&so+WN^_tKUhv*dQ8S74xBO57u!rCt5j3?w}(IUFB zlSLzn6>_jd^+T|gv)P{;wSi?C)V!urY%Vr~LQc%MP?y+<-E@Sx3$fIT0%%GcyIs)n zNkEyj%euKxGi(+*F7MmGib>R9u%CSynrt`T*o#1J*%Oae)cu@aQd9(k49-4#MFg15 zR?bNS?uzL9vfqm4Mk$m`MjTOTI3N)Xium^mCP7MeXZTlGw)AeH6lMqe4= zydVXl&KYeEFRCuL$ktwH%|@*v-N^#cPfN4-1B5?^^DhNnD zMnL+8_F#i5bv|ef%pg`j|U|@c2N8%DFJtv!wVONJwRYSirE* zkVzLsRJt!>Wq^3yd5EMKm>?#sRu|z#hA@BuS9FgQ(k`O35O6=)ESPjWTPz3nZB>C9 zN0I+>4!e4fyQkSLH~)+?!T3COasiL+yB_$+P{&{X;NeZLka+2eSs$22I{GKO;q;_K zM(ZUFu;rnhpgg_QJ>w0Nzbil4|2+z-s%YMv@+x-NWuwv0z;{vjH<`@B+#CLKXv(6f zQU8WWz;WNy*2){+*81m7DOe5YUd zGMJaZOY)h5;GaeY|8QPbX<;zt+RB&-qeCabujG#0)wk!A4iCnhIXbifUXn8(3&1;a zAo-KvzoN)3!cH==;*0kfydgIn>d`Z|?XOvJi!WQ!L_M@63AX_+UxH-`__zau zHa!|hSiEp<(?fxT^o5T!{c~_!deM@mt@ztRb0bsx0+4loxM|gWnfKOSvWWP-;`?sS zVpEbHFe+J@T@=}wpW6Na+GJ;si?l#7dAlIts`2vL#6UIpxPe2G^X8$o_m*4lLKPWz z&i?tr_`5{s?5}D|zUD)3EsXD6Hn;n|1a-={bYl)^%AG}R)47nXIRh=UA2(~-Km#$( z_z{xH!JD}5%D1KsbzMXb2p07uS)Uez*z&1qGY$J?V%?6{fX~RF-_G)FEu76Y za10kGk9vc%RfSA!3OiTw)aMNc7h~-KPFX{rM9jSFkNh(myJ6S7tZif_izJJK++HrS zp&+OIf<^GTtc|qdf3G12Ky>6B4f9`l}^HkPDX~UssYf-;O8gWXfWyaN|7B zjfT21eKZcqsH1Ilbphho&(>cIxJJ41s-%7rv1QJ`I~67=GuB4~Y8{aqjB#GoADK!H zU2f$&Ts#2b5wKCEwz#cu>W0JyOO|NBY&8`U=r3DbuJspjfWHZ=VcFQIPSRq zdHqHMwIFwuezlQ&^e`n+)e&ZBX^l2jB3qyPqD7dQIRPR}P@qBqIwXtDEWww+XJ+JU z6d-A`bTVCzWOx+8Bs>T8nQrVoDME1q9iJ44YLOgbRl_jQg+e4wdm=(5Z8H!HN2r7L zZuTFEkR=JsBWZIkGt47@RDUb6<%|7ZhRzE*90)+%Ci3)vDARh+WFAt54VXo(zG?G@ z8L!Y)+Qv^zCam0!s#`?+Vv=fB4i}07@0eg9s6|#QFjZBtVc*QQ)G8dX=E4RP9CT zD-=mQ$3OxqAyNfj1fn%o@e-xVZN-KYxT5ue))&?*Nu&xYHU|L{Ate8A&b9Vhd!N1b z*^!E4{Oo}D6veD#7t`^Z@u3_-ZFk~m#3!LjU6MjMh zG!e?GL!q0}2Gci;i}6Rj+(0NN}%*F_YdZ_3sBhnh>;He(|vqGK+@=l*S*} z@vD!Nq{Nxdu!CXQGgJ+pV>Vm)G2O{g&pd&TeZVkw*qLa%?wcQ@1?in1oAxd{XG#^c zd7?rhD*SXElTtjA4DoQl%qcF|1F?lXgzk*Q`0?_aL}rO3v&?TQp>Fitu6acC*emBu zY!fC{8I2T#Q>eRUQ*Su_HJND0ws!^^vJE?G4cTsNYG1!+mCNye#Jy!P_lytT31D_- zR_z@}%BsEdQ?P1h?C<$!yd)kY8zWKzTDDE%(VrAhjMrjW%O?XNGw>>MvL*YTve)#>zARr-Hjkz=r zT5ZwPwy^ZvI4Iy=MZ$+YnNR9iS$GUcn=zZGw6|X#h6w#+p%V;F%)W;)-dKK}y>Mdf z!jsmd+ORN6;}7Z^e*=(n69n@%ZRbQU~B7#Co8Y{*a7(vc6ClW)T=l}7lCHh;oj@6Ub5 z!o%LmN$Pq>o23QfoY1YUKOFS%7xLc{Ht|cjNMR;7{c1LsAjFwhbt&B**0V5A~PjAbApE z*#(QF1}x`E3ou`4Rj%dK;L_Qo0jq}wY+TV4oQUP6=W*jTI^(8UX~h+N$T={z;vbP} z$=Q(&wMoQz=Y;R+1Y~f2zIE{CyvWb_T1suvpY`oS!tKv3wZq)STt%GF@X_z^MhKbc ztTqO}h&Sqb-oDS0rYXpvw^=&lklJrKzNkHFS62{iGkmIz&$jypD7;{g+8}#0x1>QV zTv^WOj&4!yhdjg)M@W5xjIKfuJ5A-8&C9jVY~3yGtGwECOar|=TwyE!BxN{u!4HI=DGroO5PtpohXFcSb$!DTT?|CfyGido5^9*+2=2?Lj0Rw6VQoXdy53L&l$V&YQE` zypF}v76h~#)#}t%^n@AOr;-wq)Nf|imp5``EqrVZ!LX^7?BEqM$RX5X*gPw-#Po!m zQuE7VbR*|KdK6GP%!d6e&l7b&csjuzD>$e(9EB>d*y%I9jx%2Dm}yfk`iFr7BciZV ziV?t2f5pf)QNnCkI0OqrLq^zBT#@4;5-W7ev>_%=5p!1hc9M-kezx*i(wwWc{$1Q8@3DRvk{=cmxv-Lz_ zNz#BSw_W|>cZ^-#n2H}a^ot*WA7?-inTXbMHBEc|VpXRNAGCcqV};*e8~g(osP#R6 z5O}&CVfHEnT%#f`FBK%<+FpOSMnzpwz}giHRyBb9`jc5M?X0*jbvb`JYCt#iu?# zz4>lBdfU+q*#n{pL;mP8JyBd?S6j(2jd%KK@Tag`5^R`Bhgg_Ppx8u?+Yr2_y-nxx z+G@%@E8<6m07{QW#L!B=Ah+qfGOU2Nl5cN*Qf!1qf|p9u zJV#tC1_gdb^7~RM$BL8{_du6oMC|QC(K;kUKM)aEdm=Q%LDDbKoexl>Y9tmg1Ep)X zogR*4F2pXfa1*;E*zvD>RC8H$YTtIMA1La$TuI#u`&?&NgBfCYT*57N9c!ac*{nD2 zX;%%O3%K%8Nfz-gkt3#d;CS?Z}`Urir8l**GzY@ z#|zkDkobh5!Blsvj>I&+TIp#VDasfzQKH=4E8;MNWh zw>JBzacgO}HJ{o&;mrYwYX>9@n7Vq?7i%{Cqq6aoxWw)Q5_U{kz3G3}Z2J4dNjcSd zf#=2s{xC3f90ZahdT$?|l2erzcyM57C$(R@Fy>@5bR$6|{>v;n7a*uC=Hzbm;p-O-EcTC!RYO&SjZ2>c7JGaJu z&28FhOD1ODjZ15qvWSHW9xn)_(SufwW>A>UuUC#4w0+F+yS}G4j%r2%XB7T>jQ|Q^ z>|=X2D_%>+Q2ZKmGwpBu)36;Q_$X~?g^8LegUecO8@UyZ>j>7HLdqbpmjl=RH6PmRToW4v5V z9MDYYEu%rRy}e_~+l!(KAX~OR!K`7J-AKE7{#bFY z%m$ydW8@P|XN|R}w2i+P2VH*st#rwHn<%D{*#wevQpeXHnwydHpY*Tmd;KPDQ45;2 z zkpihaj3G2(7jTVp$-q9Uz90M;nRvOPQ>M_^p8-HGDs;ek1h(YU_I8?f*ovh3gWvJ{tjI$R zm;bF33jRS6dM0fnag_#N{w?|({Gk~{p#J$vw2^;LXwe()E$FiG{ny3;e>-VPjD!zZSe}l%dtJ}Wr z&CFP&WrgB^JBr7?#zsLNNiS@eNNOFL!K4u$pZNj;#yZAS9agtknJ=~18UhE;=S(XmH00p? zZ570-jrr|OftcmxG4~b(%TCCuEz8O-%PhJs5OZyL%#}}%4sVHxduUl~&(kLtcAHXo z?1b>2W8yAIiaU2%?4;8s7k0Tlp?g}wj!Rc>YO8F#6m7Os6K_sUSar$jP2aBB6b{tB za`afWt+JW+7igC~Xma@8u}L{A@D4 z>JRbTaXhxUOt=add1@0dksygPu`rjU5 z-P%{{4;^|tt>O!o?V0@}>vq52t+gfLLWrosb4H|5-Basj{Zra6A5UnfwG2OA$3NE2 zzhc3Xk{RPapHaigN-s6!vlZVqZ+^KA#=TLDB&SvFYB=w45|f=XD>$4nUElmc(^(1` zyog_8fv;&_@N;N1mEA|C5Fzq0F7ebQReKs%>}D~U;kRhWNKZfO@QpwAz7J#RRzjTScIt z^(Il;){GzTp&xf+0doOcWd8v58t#?o^k!wu@TbdTlP)pTV-KNKW0I10DBL<+Nk;QU zmIa4m&Js@TthbQc#``RswUaIl+r9Ot>CXo8Y*NKwQ)BJzN*0N5+m~M}nLjyDdLl_7 z3yP*%!*hd@MqWM>l-0=R3Z3X}X(W0hk)|JbQ#migNiI;8hy>1I3ozs1Lr~IEEqEMp zgh(a2b^-}~|AEoD#7!ALK^$+?r&*=I9U+A}@9CC9>y9;Q=jenJQV(LI6bPZyL&`-N zfrxW6Teue(4pDi3Jv;yJrl^eCvwbWWwKDvO_S%$uZL`{IQ`M(ijHD2nh}pymBm%TZ zfW~oOiSl62o5*NQAA-V|0YXSRaowNfj<^*!sy~T8`Mk?HQJ(ZX}H&?EKWqk#9$@=`f<@ zs@JCap867&tDwXq<0{O{fNs1`>WTTM%s{T0Q)Ve*t|N9zEz~mTY=%qs$fK zJpGi(eJ?o)`;2@~1Y6&uN3cyiiUixYKLx>-lzK;UW^329_|$3m|Gw0O_Ho^UgOjsb zTlyum4;P);w&KS#x&^nQN!yBk@vFwbvGP~LUIxTq{BYLR*z*M;cH6}4+xgP9TT4sc z4Rsks%E7uX?=C>}A6^Gv$n{GJkA-`lpV{^X|5(^^_G6z(3=#ZNNGNj$P0wWl?eMk1 zE(AV~AOAf;mvfW2lr8Sz!ebdYEE2C1A1*xZoX2);Ak^UP1Q}m;5Iv$rxfzEDOoMR* zBoRgZ>#*|2p=Y4Z1VMFK^3oPiO+xQyKyY;F_#4`^Z@*;porD`zQ`i_eu|3@QS&l|@ zYpbu8@-O^L8NxdIT0m-z1#`b!M;tE?PZ8dSo*o2{>waD_fGL_(02LIY_Qa5 zGCM-PXLE%7($^7klD{Kljmr`8TelIjgI$H71zgTdQcA15$tW$s{V-Mp z-NY#h8)mb;Zu`KwlMm}=Lq>%!vGmHseoEi<*suZ11Cx*)L=)3 zTlW4~zr5ztk`IYRjy#*PNxtP75%7njZQ~58MmJ_dITo4o0tvjrPplOx&daBz6wChE z*Wt%UxRo|1P^LjpCg6b@+(@`E8xpQ)^%Irl|9o-br33aOz%U5?@|CB-v+}R@j4{BV z<6Yd-3L}KnmsCdT?`{sd7^hhs)jel0mI1$^Op0TgO029x5=6XiQw%x5>61~eX0zAU zaMO-Jm6LuZwTm)}y}*y@u-P2(N*jFl3|CJKZ*rvYSIt$V#*Ay42A^6VcLx5ZA6aKM z*~)?kcltOG>&6z(8ngj7nRQ?aNBxcgvLry+FUXbWInvHn5{w8u#S6L%9KdLNA{Vl_ z1X0K*KHGv1`b{qX;>Cq)J5x|KM^Mn7#4wR)W((d7^5f(U0j~XXMD9z4{dx(DOOCgR z{H5Fiv}^xo{I+=W{O(M%E67b=CBJ4i%8;SsO^8yuslFmef`e-PM&`OeH{z6+wxC|Q z9RM{PMvk^p}3>4E>r?j5(`l9U5G+t2&(Z{amvF1UEll&w!6b13`# zIou6irP1W>B2GNm-}s$)QR*)xAqEDzPQgL*13J?BOIfc-n~KE^i%ISoNN9r~<8j4W zOg`a}WiyK;vDUdkY$X2EuPrgAVR6a*0#1>;N&-`2(_cASPLc1A>6kYu-LH}@&!>`Y zqDLj$Tat5o3vQ4TS2qqt-?o28-?ljgLCe(t`SgVLA10Dq2a)8Jc}ag8+GPTqDZ{7b zz4#7`B#)C`kzd0rGGAc0d1$HqcA#-2T57Kx7}_R0{j2_w7w!$|=Ecu$`{#l;X7mi4bZlEGwWE9=*Qzucv&k{-{928HU(;qK%&)DG_}H6YW7()Tzcv8DD-8S@3q#67&tV3L$_=8; zc0;#}rW1qsqH9=?$x*M9sMExq5@d3g(+%-NQ|ClK+Q}sv@ zcm=YHl9mH>I2c1| z+BWg6o-pp-Iv&Q@5H2w8>=MHH*8!pZHXrbOnoGh~2EqlX>cA60g`mY(xdU90Dh7i| zp{Yt=swmG2J=(};gra~}@`{xF8WpiYB}ojJth*o%NS5k*1#6g|TsE!GAB6|xC3`R- z8(G<(4~-wXNId&;RHsNFMZk0@ue|6rR!Ne_I6Wk4!6v=@+u;*gVnc5z6!eLNp=51P zINN|Ps5Sz2o#q6xy8n?QcNRz^uM`HVR!$%?JW}I_Ve#=YtcSdgu}EeHyKy7ZJ!fu5 ztEP%ROsaO&CT=k=ZN+7d;H52~ zvVI_HPSPbb2@@=EK_&$%_|HYTi4rg?7476h1&ZjrLjA^T+If5pAwR%$!wt=?mfX@j zXNMsVy4zede33_j*Y+oA^CxDfm<8#Z!3=;|8{!wsim>k_l%oX5nFNU93WId#ShCse$_k zhTfo3H~g29^C)ce#|G~2I&=P6RO%MA3f!d6KM5vxp~LoRfne^rnW2W0f{klp6R)mV z-ne&I{M8e{ja7R4{Lg^T|AxEMo}3YbE^uoaFT4B3rn|_731=_1(l3FyRFr7C!#mwql=NBx}ON>jyuSAoQ0O+43*FYSjS(N9_UdW-u*uk*YMMzH<%lgt4DTw!OY# z)^o(I!-RxzbSs&VTs4j34-!e*Yu&s$j4N56Uz>8utV7gk4xtrGG#fqUyg&?N9@-ti zBN-|*wRJa$Gj;>eXS~7{DsZsU++LMMC?gcpb8!KeV^TIl;9(V>Sv0j_f$+!FE`jL^ zia5+&ov&3R^k_6jt}l%tgkX`QNzCMv zjBIV)u{bh3i7jATaDYgVu#z3&*-8$WB)cxL)I!Q5yNY#3tP(s0sDlY5AQcSP=} zt)UVOmbLBj6A?3e;Z>YmQ`1}*vnj=2HEx;2dyRo%%T@DXDsDGMBydivcO(y4mxzeM@k#p+u>1?!8Z{}&GJZT#%o`t z$n?UsZ3pn`+-4=&)KaJE)~Ws!B^@0fn{+7FYJo=bpWlDoLe1FTIeN|5cJG)P@8d%9 zGd~xSsb&|FF-1oN(i*=U7QX{6*qVkV?707)#sC1^{mUEQ9TuPZAykm(q$Ly|7eD6U zDQTef5{u)WKH=4}t6~$1yTsR*U5fu71|Hj;7tRD8L)YxB!;?0a<%N2}EAn#DjBU9V zNm}Of=4NcU5J?j0Mf@7Kg06u`a=IvI3-w6IwHju_DpDHm7kbsQZJtv6Td-$QtRms- zPwk+i`}%+D=j%_L6;Tg(C#g`992$Dc;ERy2zY9*sSmzrAcrv6gGg`~I-dhU1=Y(xkWz;G%u)$8h|s2=gF zUz;>3{rxR}w&}udRYO=V16*p-7`+eMke&xC7?*kXVS5C9*kCp$_v@Mzi{?3lx11LF z`mYGLee{u22JBCzFMYt0p$?cVP!-xwtr0O-Z*-naSv7^nCf$kxcKn+KzJ}vcbr$|) z!Vyt=X6Mu2yK84bQSQb0zs-Z=$ZMs8|B+W3QOK6Xf3sgDn>;&>#}#&1oLC~I%}25^ z$F9y_&n+6L(_E=#k8ydA*L%9!=4FfvCP|(X#^@Dr4ON7Qws~w_5tj7l#w`egrP4Ao zt;qBI%D$p}+Cp+V)UwjRCAW%GyjMF!k|ol~pSrQ3f{nG4mQ5t$cB9|CKF2uXwzvV0 zv6V((coN)9*(LIa0JVcAoQ+#HR2f4#M0bk&@sLMWThSL-oejcP zTg|h`Xcnn&WWNL~))AHiCcjCb-8t+5E?`6-7D-%!V`>mhoYkz=6Pca{Jb@@O_hNNL ziiU3akkJ`zWxYX~g-R`-2Q~O|s&=YEFx3V! zf$*o+a7-78^weuCHDPz6@Cef9RHmgPvXhxWHZNwev7**$sL#A*3qsirOPcUSj_b`{ zP2F@-fs`d4502+HB^3am%c)Y)NCe=^x!Xsm!H_H5-sRCSu~EbcM$H^D>pj9!r+Tx( z=-g$@ip5rUShEu~P{kdr<}Eh^FRa#!*)CfCUJaGw{w5gkU~R~yElD*7x*GMtalYZv zBis=fZF2N=V07JimRY4Z^LzkOh6(rsx)DS=?s zu7dDRVdh_kZrP`cDz*u_0b6mx1)|rQ2aWdl!DCW$!5@Xs>Ywx$Ku_BxWu>XYCo*?? zdeh+ZiBb!G`rcPl+fL~_inV%Pb!^X#`s;WixS|xYfk{jKG&)GTB=xH&<|hQq29F}v z|7!Vy+nkh_hE@U6aU!p_^$>nw?Wc)vqx;W(Z0}}9d66v>BL~akpOu$39LxB!IJLZVMm&CGQi{kk z-5y(rP4#QxdSWHioPP=_O&K11d1`F(8&_J`ET>Fhv&{N?}W-Jrh(KolC9R59BeM2CO`oWSb*@$H)HE1Pe<)#INVe z*En1!sSepUS|8SPFn{=M_@>m$cBHBQTIOd(mJSl9s}DeoaU9GaeToBi5 zKxMC{g>k)BRSpadfh)m}=HvgtmP4Dp^c#pOTTXo5N81G5W=H$dTqrl(oRUhTk>cd* zfnJ{@$u!VK&2MEct-F`gwi-U!!ztp*nty*Di@Vg3rbB{!^3X2qb3ZA!YX0HVz7?@U z%QHVWN>1c~Sl`f(6fS4ng3mLkA}Mld{Y1F^UrJwb(=nFwnBkPcCFw#LH6LjO>0om# zkl@sd9AOtL@hmu?X`=>gh|=>())~G5P)+)dkbUH&^-HYrr^3L9WOwA4LprIjwFY4- zo}h{?svYF8FY@>LHKdcoT)C=?=#mH>n91TO+bN~h9a*s#5QcV7LNzbhGeoB09XQR} zQvMEMB!D+(Zx~zyZTvz=)WEHU`!iRTGdFCQ>R`ik z46TM!j~Fi1ia^|rsa1H!^33CY#d1meB9gH&-$%v7C8-TNt)_|w7>Y}vj1T?iK;s(6 z@N#vexM^0FwE{AUsBxE8k{EQ|yWdiy$H+VMYf^CSUdhrk2YMr~P5JMbVXK=n+edp! zR)5>p?{l_jjvP-1S0@`3N5<9So~Wq@z)5T6X!Ic-2EQiFBhYiwnO<=y?+`zvv6b3i z_0W;$uT*flmx8+AQfNC=6}DUkb7KJ5k;@$iy7b`3d;)z&&iPSN6R_RD()Uj4;6cDH z5P3C6>Mc1!jD4F(c14$)s$`c=pV+4|V2E((^eCaiOw|fb7mAR>Bo&;|WxT9>_a0Ya zIT|hJfEMtS1#$Z{;ZoX%L{Uv2*NL_@op8>9N6(C0J}~XOgGDoojn@up4c`ivuz}3* zH|=e%yGmL@dGvQpr>aW#eqrCxal~F`RGP3pioXc@v9)x>U65QFK5G5gW)>a2Dz^8I zUKQJ@i31B{6M~F~2^vHMT1W`c7Kn$lV<%d%;g$x%;YQYvo`Yr8kDhW1@q$BMH`A87|IU^y zUZC5gG;(qH7Z$>*AUH?)ZS1LcZ6BY8Vu!sm(*pwNZYisj*8fROkC&2Fi_CofK^anB z$P~yc@-+#kOB}Q@b#VPoZW)l1-tC&-&Ipx`|3`k&%uwkKq>co& z^<||M^zOLaV)FjLLGJ)c^Ri0J5cC!(IfLM>R9@SFI&ji|8{IB_3?I@~TxHvVbx4?@Kr!-e$$P=|`P7hh@ zOjiHfdhnlDQdC|FBNFysuo;m^;No4yX3m7`0K*|3d5l;mCpe+!bU|kf_u9X=h$sDM zMed$ow?4X-bXDzHr?CR8{dAPCNgopG#P%!JLvncZ2<|(Z4?+b3B16%4$pgUx4UxaZ zKEpmFBEEi9hetOMdXeEUf%%gA$kuYwF28U^M(A8Yn>P5zY z>h%>C>12<}P5~zNnN=}c|CE(n?(Fm*c2!oDmlC~_)4Pzk#Rs7rlYtdAhJMxF>{SM z3NV6Z4_f(PI3w?q9^2otDz)Rd`!eFcd@e3=Vn*WW83_ZfSiNaL^`?FgG(HHQ{`AC~ z*Bl$_bzJZ1V^VTz^8)?z13$|QeT{}`qkC`9OUZdIFA(?sDcN^t7F`yI**89TS)Z)z z;;h;QnMMB=h(Uff!D$~w7g_^Jp7fsm42XByl z+0xFQGiWjasq@Oxin0U2x?!=QThW#671)c&c5GW0Ww!n>d;dDv`_Iw#{_BqUU`$8t zy=kGL$Bb^RU6K}B*HIAeC7NdM0=YI!xJAx{>Bt*_#xr3$@<7+n)OZxtJ(u%xZm^Q= z9aoy0X6M)BY^tkU_|J}LhR3yK`_oB~ zcVB9+wW<8N{mjrY(Bzdadg3mSIX+rQ;rBtg{^zVcQNd|d#^Jy!?<&VV=J$+^PJ~)t2d#4Y5 z%|>*=gIV8z5cfF_?>)+-IuS2mgH9)8Zv$ug@KgsM(qmV$p3e}t$TJNH3+JMWG!5eX zlY~`)`AiemW5+9r7fnt+03**pe2y%eFaQInFhQ(wylpC0+d^TSDps?1sQ1w(!6N6I*dK9Av&d&JCPRE9VQ>W@X{$=jF#mCh zTt?tlWQh6H{nfZ-KBb~ixg0zMfeXzjw&S}Ie>*elO2o1 zu_3@Ogx|%5tz@1Hsn%+w!vK||a+Gis$ra9?k%O;Z!@s0hXiw~;TUpM8u-*^=m<@Ho zYFz`_@~YRFGWWF2m5C70?u(2E5^$^eP1GhMk6u%PLvFV1JN9j*25j~Fj~kbwvik~p8g0buwM+7 zDlk`rZH7_JT}TPHYMAUgw=RaIo#8j~GTY7-RNEo^IEH>iTk4KM@$7Y`YSx53i>_&- z8aA)Li2p)L=k%PEUfy?y!moxNY$Zj#Ifl9+*@{YkW&y9QbP29Osb3^VrPba$KLL34 z%QeKGC9nFWNXcW%;nzEeT$B3jUs&&aSug9uhVh5j^ukU{OqLP1CBI?W`$M$$?R_}Z zlM{NLYN4=)tV4bqT>|)fgm}{KO3I1mYhuH&$1r(ovNn$Mn=+GDT42Ou1t!*43!Q{GV!6|R@n$h zk<^)7nY%(oZuqoUhlT)*nMFA2+?n0Tt+L$l6r$IdGfTv~keq7V{XW)uFtcnp@t~Z! z=Gd(Li^&j+%_#W?7H20+Mh?B8#9)zlOJrF{GI`le--}IyISDtyBNM*Al)s7ok{kki zRA2O^ulVan{msfQ{AW}VYRU2!sS{V~b#Z)HQcE`3S-bAdU)EP$-^Yqtg_j7=DI);J zst~M{z}hOh)m88*ps2Fvk}*t3 zaeSCDwBu~JWSUiUD`PT&qoLvyp`S(-(zsLIuYzv(Q{+z$4Y`7uEkiV{*h=Ugsk27x zv$8jZ;cL?^ifdL2!GT!NK(3DI4+r*eM8}}v1d%Kyrgpq1a^Uth6DpVBNUgBY(~va4 zrWsSoL#$cLNyf>_ET}QnZY;ogmbTl3M`$%eU z9#=pWv-PGLYEW!MTiHfLFLcU-A#eaY=BFCLCdzti-Z5=7w0So({6< zE!ai&OKg`Nv`+-p_H$5e=VgYfP7XFc7n|6=eOYYDP4U69l&qqW7YDcQOe;Di5Uj!f z1_pvk;gR-<@-DaQx1+yQYs{*7(ugnXPA_``S~acq8YZOTxklywY)RwL;TYgwO>f0oYLM*~;TEAu~*`%zk6P5AuhGg^YRwAS5oO6NSy31pN4nWAUR9izw1=Es#;WWwt4p8YzBs6qb?MCT$2tP*>W}l@4xE3D^_-Ixm z2&Qj~-Z7h)v6e}c5}3j<@Oi3J_B6a^HeyAeY6_+I&CPVkATu_uE36;*n~aUMh4p9s zCUg2Y;EqW}et^w7IGtt+YI{|AGj0U!41@PqJU{>Sz{#(!$@=r}|F+_@_VZ3Sb$!|~ z74uh}HM{qpZ~e0Mav`R)^Gxalb7bFO??L&~=#d)49)dSkL5Pe?=rxQ+yxO}efItC1 zoyxW&Uj@gN%q^{@TU2i+^-F!0;gT$5a;HbF)3f!_v4V#RQ#0-LqJ}MXKt|Kho&iEy z;SFUX0zy!)`q)JtRTv9g;-VIF6fDQjrup%3>;$?XvdSH7=%~(u(}q8S=RwKxJ{-~~ zQ?ML`e**@=Ic|<9MA$W3AJfKgE@@8hWp!_J#BG2l`Z+v@TRQ5u5|LyG2l|c%ARSCZpy5+17guL2ob`hy%!8^yKRdA0fr=~v{VGduz?5Ox;xtjkL~fD zd#!u5sx8_aDLXy`6TvCEuNAcb{ zb5Hxg!nZX!yX!)QCMs92@5jGg-4SmMy~n0|D*JD#{0#oFy_Sl1fe~cm#VFi+OQ28k zDnS!l?t`RWVXS9%-o^M%VPmq$0dq|Arr&itJl;9yBkxC>$f^b}9i|*&Kqo23^s?7N zA3+@8W`=X|N*)a%r4;*bxmi;p^q3y$8XjDw$Fy$Ju#Mkaq zUd=4bTqVASy@1TIIYu_UGcC)79Mh0lqUjy`?Cys7?d|00A3x`e$%QFX3X>;<--(I) zE-CJdWwGl{pIq2|3feKpubRGkQ_2I4fw;syX^C&9C0u>k>P_+0n~r^;@mFz)r?38b zpi5@x<;31Ab5k}xlNZ=KHt@~B(DKCI(?|90J1S{uOYcQKp8ga4JpE5Gd-|up@-Vsk zhZ_0?8$-ZrpC!d^J7sdX1=7e{$0lujFfWt}XOcN^CYc-)w{Gi%a8bY5b(xdHxAlvi zga6&~SZwpFqr-Xl-K!JAv-_3y{5n3R?WAR~b#ZVWIXf78ZjZRSm`f7BX_w!tKFZjadJ^vU7P@b`bHYv@`4yUcFEnxw4SxmPT>qb_!! zC_eKb;Vxcq_naTkj$}qKVOH>XE@GQLdeF+z#482U`Sr>%gSL-(Zq&97|0JxD-s?>P z?$Uk{%3!c}GxJAHL!Rk*fDOl75Xxq^|H2&dG;w1vD$-uWu#ozkckpTYkQRhXwkWi!xu1U~z7Yky@qmZqiRmcXFUd0Vm-d0dlU}3Ti%G>{9TmpAg zlJj(4{k>f7w($=Z(YD^=-ZrjlVqJf zCmJ+YR%GqsLkReVhENfel870OXxK8F_!focK;F7N1B6wY1}BX%kFqeok3UxGS3YL<4hz!WyZ(`J3v`g;p@3S zNq@2>$W(8u6Tlp~u~HIo8@tshnQ;W*ilDw2 zDw{k?put1!qxZOC=w|NpAfDAsCpKxTuq!b99W=aADg@PR0{YD*+R8oQh-G`%{VwL@ z4NT`ts4&tU*C~Vo;aaM@^Rsj-;P9??2~t`WP;7ce@YoulKKBrdsi+hkGuyF8NHbXv zl#IOCRv!QixTpB1NW3V<7b#b~N4c9nRO5?+EHlVbs~4LB{ylwV^D}sUBOx_fsZj=3 z1kKS>V~{68f-T3y9_8!46Uqog%`r#HfYf~O!1qgyg@``J@uy)%z5pC*Lx63Gpydsl zKY#QODr4i;%m#$&gMev60LMlv)ynMX`T{k{nP4_-(^e{ttsvD>Rx?oR#c>J1icqu# zZteIrx%^NW*Z5MkZ|e2%miD?4>?N41l8!9@#%mor?rdDQT&9UR6^B0}-&VHFoWe^p`_c2NwX}8qBxJt-|#zs@yLtU!3QkL-V4Z9 z+%`0ONm}=#2icxFdXR1AyrbsuUtjjCSj*x6yCZe@pCfG)qd5HkbRD$YiGI2%S_f^| zlcX6-(PU)**d8LXC>o;*Hp!w1bzR<*{F_@gPV#q-Bn;&Sd^XaJSgwfozww&Gx}Z)7 zC5QfLZ|eGc*^GpfUQQ&1OXNS&JTSm!VqQKab=^qP)iRhY0|~6b>Yr>ljpc3hcJDo9 zd^`2_{F%@GNgJ3s=|I76;@mDto%O}KUimHFJ+&ezx zE0VIoN-BAdHU4FEDZglh0AIN%Gp+wR0l^W)M>(qn{z88lX($PNmOC6SwCeLfW`=wxU6+)gFV0^2!M(1(>`CbV$N3xl9Z zHHG5tl(3GNO|>jC`6d!)D15@{Z3rY()DTU)SHNER(@=xuS8^daD|bOmzO3`+hT7DR z%3vM&BweeQSvP^4#{hq2u#e(feDG4$K14nmJSX zbv7gETshXAM_P`RM^%kHEQl6H8d*r4Al4c#uM9o#CJ`*Q86M9XF5CTFzZ5lS14NUL z)5sknF8#Y;qYL{io{r^aybHG=Di>-?!`RX#lsHjUqg!63VqxIPBL&B5Rw2?g`-o6l zQiOQCs_8@RRe_4wu3vhCrczZ!k&AJ z{G}M?g~Id1G3F9OtKlZk6H%<}Kr@)}77IlLq@=CfNiH#o1bD_i>5ZQmJV%|jvOApn z5vj}%KE_LisTnq9B4Wr<;Sv}&&s5oBI*-|`1bHwqnKkrvOl~2^G&~-a?@)3~`hRc=JNG#R*!`F4u45v4}s_7a5D@HjZ;tEFBIPYL~RXva5={B9B^;+tlNR z<{x;ppP%D@x{u@kLmrO*U;HF0$=ye$lKkL?mJ73g!z#VAw#J^%TwCpJ6SHsUOV@5K zt=j#5x7PQlU~AiLY-Jyw+CQau?LcaH)>c?nTD6CpINvd8^QkD8VjKSK1^kb9ZjJp~ zykE)0?7MMkO;Z**ghV6Qj^Fh?Yx%tSwfZ$cN>SbZZ+Sr&IT{>B;4453TIo(GDPxF+ zqFk7XsT;HTXC?CpkVVQU^=nFmslqS7C{Fw)X>n!n()Wp97U#M@CTTVGy3_r#`^}+o;h5FX zFT2FIUv`yWzwG@NkJ$=Erk~e;jW0B-8Zar%-Hk1#GS_0);4oj(@gJLnu*<6-8y&1v znKh1&ydBgzn@_FF={0Y&)2MJMWmUz%ro+RM2V6!sG$jmAdh1Z)pqz}s^~W7IC}-e! z$Odt^-hUmTGjD)~;FVl7IfOFodANjd;RZpjl?&Gawc#qP=8&VQ_)YT>xYW6>&fCNm z@BRc?3G|2#q&;*uonZ$x9qi)4O^i2-q14B18@4ki{ECmq`7#&}V%!`cF$Rq-s?4YNFKz zifsjSQ*Ov)4~giO4cD;rGlg)nQ5p5eH|}X4XobxA3OB1DmM*=Vvpd)LkM;eyrS9^} z)aGD499ydXmOc=SLFpBBbHKZtd7~{Spi>JJ0rPUMwky*-b_cXWGd=^oC{UN|P7uk) zZvsmI98I!-t7F2cIfZsCxwVGA8b!o^gwMmVPsGVS3=>v4IZ;hOUJvWKQpuIVl^T%nO7DhR%n0 z^8DV@&+io+-#e@J_N?sNGmAzCV#b!moIgIe3#O32$t=o2S=bE|3U?NSe-jf2W#YL@ zW4EzMWZ}-d_@Bfsvi|!cvWqPlzCrc@U9LJzh$*jsy{H7a+O+k^crD9n(r0UlF<8=%$enG0yRE_3sWPSxujbXq{cUmFwM##`MT9ig5KOhebbE zf1TY@d#R2EaLJ;${uTQXQf~5=94%lVJ8Yx8LY!)%Yjae)2JYI-Qx|XLJeFdwY2(QE zY%wLp@ub66bzIRU-^R{uZ>_O}VCzuldV{D=%gxEJpD{qFz${~b$C+;rV)YpZs4RqI z@(edT1tAw(RzLx9vKcNfgeGP@x1-VTEDlutnm)-WqvWN7A#1znL!lSTr*w6+UTmbf zcFL9_0|ek3Rfu62XAsN3hB1aYZr1lJe|nERuyKd5;KJC9^W} zOeiDi<5p}lsnql?!45vw;pyCxd55yxMHH4$vnnH5bu@rhkg^-bidFFsN7!Js9yEoi z=@JY`1TCqixxIXft1wm9H+#NA=IH2OS>~GpXDBFU%#`Wwje0O7CQ+4)p12ub*f&#V zAcV#KBgd8dWJ{g1;rrkch$b)1V`VwZsL_ra`K_p8Y1D(!VH|CH;5Ug(lI4?3L}Z$H zSUXun5qae6hajpU|3)$ECB42j!sdlgi8McYJKJPM;zRYCyexL#c%#e zTqLt|_hb}=kE9u!WUHdMNRl>?nz2!Jly@^WR7Nqa7Nr>*b#;s0j17F0Pcya+zRlRa z@q}$cN7=7=^M$QY#=o6%d+-i(dFk1T|@kX;hlZ3+=3lWV#DNVgRRG^k3k3N+{6uu+`kchU>NjvG#E^n8A1jThF z1IDp}|8mJFCSQGlH4bxeNyg35dTCeB8@vW!)vB=a@c9SL)cl1{`5I0|uCSV9!O$*} z5=;Knf^1Kq&raQ;7nbn;=T*T*^m6T9&hv26+kIiiRJe~c` z+rQ(2HfbWlak>gJPNsR&!P$;LpsA|O`zeIil<{UQ1&!kEEGdjryd9l?YTh8C;=24 zg(oC-n2bn;mxzxX`3P~zPfCXQ%73o&0m72ksppkm+R^wm+GBk9HQXYqEXhXGgI;5E zO^N-~aOuz)mw<(Oa-!UtuNJsauYvtU3v@@tjEIgo=q2*9%1B=7D9kGfP7{?lW~aK^ z`~Ygjf^`E8`^ONQ03!fO>-|MY{Vtx5V*F=g&maa%*UlS?Qb_7hd-5V zQsqYA+(~&XE$y`-m&Eb#w0A?cUhN8QVlZPryM73nVc|b$oem#bagJ#~^$!?YjZWc4}9#6nf4PUPgnDMVV6FKKvU=rndqRF{c`>dplzjE1n# zEHROXXz`xwRm887?i#s$F?D|tBuofz ziix`;DeiihkDNBSFm_5|*9qZ2#>Aa4dBvt<9%#G@P1RBpuD*2jrq(r^zN>7U0P~U5 z#O|r_TjwW^jagq7^X&NGR`6%@va;u87EK7m+%rCSCt%v0S+#czimd=F`+9QWR|Vl< zOx&U#ad&_}12X%(AY24K?d=B$mVGW^S;DG|1X2qu+cA-TYMj$!bht1ku4Ha}a92uJ z>-@1PZGTFS-|@@UjTOgF2ya-FR+JS8egsd*oF2i;;0ZaRNANuSZ`j4bJK-0Z76@L3 z-<{o~zU=ncgyQ21f(+_J_K?}m#c{{K-)Ci+~*PRf`Z z-UnZi$GV2*!&fAGe6Z^Cyl`I_M!p_sJQaWHW7DcW9U6L#8MQb2_1e>q8MVKi8G2u| zQtRKVxO!@8xM&Qe@e{a{g1-I|NlHjn*M*`F;c#Fen*o>?&|3g zyF1t*#~HOcg0d&XJszwB@70&|-#=7vOonq)CLh<++gSQ9b2xVPvyg~!fNMChGOM|c zefG3&hqWi4O5EyL3zue!d%S{Jgk&&;{9tqkV@@~0Mlp81BG|rK-NEbISwm(Ik8t#m zF~Ps49Raw63e3QE*iX2q3}$`9tpBF)eKh-Ampt@fY26z`NN3k}-L#bUv!5k_BObZC;=k~bm4CdEsDBnz07$TA{p zWdvG!4D3KUGWi7zk%W{M)hW}q6AAhBYhI^@c$vcnEPdwEq>wuTOe9md%o+wsEk)y;@);QdMWx!KiAAbAc!rU75kD~t?%qf-QWzGr zW25YG#5{I92ohVC@Y)cQ+{!MCmKI`Y9q3gJmVNw?H*0v5w_WqFf$K!FdxjdeG-?v( z%)bz_Ll-*S;}r++=;B-7$*W&);%bu1BtuRcqSw7wI$T)ifK9 z{=`nf(VwfMJb>EqHLbFhb>YEfL`1z>k?>(pb|vgsS$GWSnlYQFpm~5_r5ZGu(VQUb z*T8+mr&0|R7PCDT#ihiJsHvde)bAZ7zp~)H^#4DI`b4lk5GT zS-h7#Kd?nJ@;ZS2O?NKp2Xh^KtWk4FI1O^EHr6NlOCURAEP^x7LU=aNrwi8LJ@N#K z@MK8}fM{fUZR$wdK6}-eq<3G5>g+$hC5yl6whW9DuianZH4{G-196G;U%77%EN`u? z$ip`A+2%1Z;@9Op2PR;c@n71aJ5zoO>#@ya6DVNLc~Bi*{CeXG@tVZe(Yt5E@X^B) za-vnWhFfasP=z=;hS5tkc-|KAJ5g-tLuYTTD_gol1xIe#egK zJJ(I6(8-(?M!3BecU`>*G8VVrgND!ODqYh2@wfh}vii5IE_L=(T-HuO|AcU&D*8Y%6Uu_w1N^%J2uW693~5pEdubQ(1!K{<3ZH@J4*iU^HBkK-<5K&6v-m`B3DO(XaVUEK!s-*s`EerPN@;C1$)%HrR2j z5`9E&#tIeMA+*lKIZ&Y#ibhRMfYBDo5KY=Uh7}lZFJZi8JFZy0DYJT0#siIOVCO>lFXt@12H?xVm=uk91Qe#S6232H_q(X=&6|O{#}o_FM7m%_;zVya$Mrg{Sznl zPq=#W>P_#g+4Od05=nL4~O1r zkB&{sS(z7_0q_4m!TWy)dbRyWd~gHExobhrJ(nIo0OZ_(VF?2&?rD55HgVO`dm8VL zO&qXvdE=+U;s+G0Zu|hhyZ@eW?b5W+x-JvKSLP)R&~4jx_k0<>0Ai9#%Ps|BcNo;$ z?!0g&sJCsEja!E&Z7jknm58r0%0xn zosdipqd#gzprG~FX^dT4@Z&=IaW@_!odLAbMUB-Y5Qn0(=zq}{!Oraife*Lv`M?P$UzGn&a(FvLJ7%pYj zzeo0vkOb~8!oe_D93_?EyXA&gzHi?m@#DM5;sdWoF+rZ*NKZ!~ByfV&% zXE(WNxaSFHSu`B2O5bCRj$mu}B9R*=p2Wg95-g}ae9h6 z4E3P&C-LbO);C&{ung&v8eNwp2d5jTkGjs4JC~jxo!|bbxWoJKMgr+A)~pis@EPT@CHi zgFX{A8R{QE9JSZ0AqJBko$dWxa_gg16qSic+}ap9gjL{pwq13p(XRtJi0*G1MyP+R z@5e3DtxxFmGi(KQcv%!1hSK*uP;13_t}5p0@N9amUF zZ96!Wu$cB}>;}yvOWH{>dHT_F_Is7*j{kRVQ_NHt~G5O5GA}N^6S&PcHuX9t{ z^1B5GCugo9JQYp*gqQKOp>ThS2c$z^5W;5&>QqiK{g^>=aepAlm4V+g) z={tWttKDkU=mdI7E{R5sCF@>(S^1V(C@sImhSCnezblzU4N;gl?oUa-7h@w>cKiLs zJ?XQ^j6^bVTj-e>E&hl0JeX>v{M37sO(i2_&fGJqQ^XykI?j3JEKIxRx5pu2$fPNAZ<R*~C)@gmM01s1Mo`}i~r?cUmDd1C+kY-VOa#IRew zCF{tNd0BYNg3_843oD$#uUq76B*s=vv_9ek^${2Irt&2?+iDW1n4lV3Hy1A!q_F*W6c^`7c zDoE9NLU9ShuhN$4K-7X9T&mwRT7+4dN=_m&zWTKl3VZIn0BG2~qsmKA%#FmRo614F z8oy=39c&)(E>6CN|A_U;_`;5)q~rHPjLBmeN7Og;Lu}sg>-)fL8;aXVXwvpgx`^X( zV@ruKmbJ(ELJ}4^x*AZ};|GsUHZwoe)tQvw*Z){OJsh?hyxc#9rQDns=v`W$SS$L= ztzVnjcGWcgLx-+^AY60#I6et6sL({xV>Jd5@~DT-5Kt>3CeTfrQqFwkrCI@Z-GEug z5DR5lhNky;lK?{|ie-3{kj;~wac?YDW{4Fo^3hSA0%rQL8Bs_KnymOf5s+GzXV_;l z6;K&7_)^J(lCt@hUx11<9iZx5QMxgGTn4$<8&mAK1z=fBMmgA#$XWGyi$u!mH97={ z2;1;yD2hleZbK~3oP@|~>nKW1YH8oe-kpq~+{}*%_fXvCFZC5wvGyc`)htp1VMqCt znjwx!gIlC7OwBGm8^tA}m)NzJ*^6YM7>gWQ10@vsC~ylmnsgyOixjbdaDWOKy|!O$ z9j|bU;m+*~dKxN9E%q7!Hvhhu*u^O+Vhf9%S)9<2L@u#%iy}Sf|XeUS0!B$pmfo3&x@y7_ocC?v`rAq(6YizALYFQ&`B0!^K zVH#~Z%}rFjJ^;B0F*3(sJO8Lf8wwev&8~UgqU_8wH708??N%m+hlEQ2KoIHLht*ya z$U^1&8!gA2B~bnfNy6V${8 zS1h|<7%(H$k8is`}F7)xMgqraeHXn3*55rPowtS>yG)L{;ng0 zYYQxx<o@cb#or>{^klfMXr&%KNEK> z{%5ZeJ`EL@N7*|UMLMBZjkR{&apJ3z_UZ6PpGa4s)75~HNRs1NS?A0AIZfKg*Lz;2V=%3NFtq1>kB}L_>bvcyR4wcr8lwXrw?R_%7bRv62v=}h9 zCy;U;-6xtqQtTE?u2FK@{U+*=c;T>9ih7WLi7Kpy#m$Lt(m7&kGRb0^{q5Wsn=C|WJ(d0#T3MbMr`T{^5?CsMgmSo?aoaz_L>e3lySDh ztg}SC&~qyi#t}cIwh;6DFK3O3W4ss7cJ4e>mH4t34b2aL%P;;Tx{C%PiaJr8C6}?` z__1qH^fYuFbm-{rUz}A!xG73X2eoh3S?nBNJ(R$BybtJYQh5wja(_7%RYu!@1%fm0 z;wqa#%1lmzI`T>R{S1En-QZ_!D7x;c)=-I^HeRF$n zAC;2xhrGb&V*^cH)4k=8|FVJPmp^XJTh7{n1>u|WQrb>{^ilQG?)`T2%o9fkXB-m@ z*DirvvZEl}tAG5e{MC)ShNrX*>lSQD%4(en=IxC@RL@*%c{Ow7QU7}|C_pBx z*|*yHS1edk7rU8TvU#>>`(XU{@A*`|JvVAw)jtW=csFT#rso0WEg~V8wJ;?Lo3h}A z+)ufTjVtv8Pi~d9r7dD1<5xQmGZnAQL}aFcUUOh^Vf^tov+x%YQppB`6>vtv+>c~m&mhsBE!uo-~$=Fz1Sbx@UGKEGn zx**bS&8^O7$B`|%#{T=bbCNxKFn8iaW5Y`u_2b-Bd83AN02zcuInjtG)6N0Pl+2!; zq+7`1#GT}2otf|^Au7o#ztXXlh_&Pqbv=4plw4wNW;b_TixjcME?-dt}gAYZCHk^lWSMf7T@SCTrSk(@J&o12SfBhwZNN zS4Bpc+$W0=+n!Mul#^reh!9)GyZ0;G1S5HdYy$W*3e8n^+l`*!*dG-TV-f?N`Nd;b zrS5Oe{WmsC^aK)}av~@hyUC>a!f{2_RS2UTGAfloa6F1y(sUW0i@G0w%X*75)giMT zDNlP>co15v(=se{>r?}0M42@wpt9Y7~yowh)4m4jN}M{37RPHC!J1i&;!M=GQS+Hk&SRKG3UpgSzO$4I!(f?H|}Xy zjUPQuvlk$a@|m%?WrjO{DV5p$w*R%{ulOY2hK>9#?&P6L246B@f36xgFLM$Sb48C9 z@VA`eg%=(ad?CQocK(^Dn1-AmSO@PT{ zI=T~_{n)2gX_Z$W?Z#uMw2Inj_XGOyZli6vhInGMrmPYlh=w-WM1Os_Ea3}L(U$aZ zVKQ??=URHb$yRZklj2gM$c)GKP;nj2TN~aIWlw2;!IU{^X)O0^ode6kIH6fqM-dAV3HAT)|P zTgQnqwaWr|XVXJ4nf>qAxiN#(k#}0yr8GQZv3%T>L7aYkiC*(Lzu}8hW0OC<((1gH zQ)Ye7d*vnS09iqvB#Xn|TpL+^MuTa>m;`t6cIvH(O!e19!pGHJzJ|P)O%u7_R}H2b+z> zyMlNB=Sc(8`O+$nF3y;3d`t@lNk^`Z)H!K5Idv8pb*R_G6}#vWc}&t785;74CaP>1 za7QR+b(S$4SOwyEgtig^wamJk)4D)B=2{FHUzFAjmsC~Vi2VAh{`^O_hhh3ZFFv2j z&27wDjc?hSG2gAD2OE;Eeq^^9E5Au=&6CDqScqlYeBpSeq;))^^6Y{eT>c`6qtou~ zq%Fz(b<3IcGLIo17tH!Q4h@~qDrhe^Pf2*>)0>V1Th&4$P4^Azo$*d=85Siz1D(Zg zzOM9aJX2DV@oF|RkSd5kGWc*EwdcZoGseDn)F`^+?6!@gkesuQ>8uc&c5*U|vppLg z>S@AiKWZ~3fHt8o%Xo-`+eo;8*SJNszVN^vV9ZFw8D(&xi=5*d`${r8N)p{D45M$( zjiYFrQN{!X6s@~~flzt!S?NsIE4;=mb0h&J>1H{uu%|axTIt~d#pFd!NN@p8P1e^0 ztpX}O32ihy@K1zlPySm46opZ~b%@&jsg*-wDal7CXA{7lO!Xt z0%#M5o*SL86w6&geIw)&$31;H*#1Y3o0x-WsBPY_Z5smh-Y~wu99FPuI|;0%zjCyk zB;T(;ynfHBqt#J+_K)oQzj*}U*s+gn!UJG9@@8e@4a1Xi7UhK|p_nZZUBGKTxS^$aLE(V6#*qPq7hrq8 zTWj;Bx55yu_2C}nL@Rl>)wV=@PYD~zwv~m4S%COmMZ&R!z|QXXIXUUq1{f@y_FC*& z<%F2BOZoNG*J7`Hx+#YReCK#~)gJafM1Vrq$Waz^SR4~cQZ4!7?}(-{W44Fzd6@qj zVI#t(C{$yid^PapEF3s{*v_H+&dNYib_Yr5ovC!b=Np)~dz-tVeO?VB>*+|Su{Dsp^M7@Tq2_t4VEF@Nh zzX}5MN$KVBbI3`PhqRw{S`U?`kxi>pqqk;Q+l`m8){%sCI zuwMAm{JrypEHZw}!FglFkF!xr(0ta-2%AYzu*_24%sJIc6+VVbWJl6)VmrmT(cl_I z!$q0UXgavH>Uj}8wZUk0{N=*GMwdfUG{1a<<(#L%f<3e+f!AN2dXoiSp8a1*s=DCm zM|~n*QgY7Ow9h0>(}_u(#>e9jipH}+uc0(xJmPeuI1=8_Jw}}JI&jM%s{)or`n9$Z z)+F9RFH0j?>FJh6auJmczztn@hL1E7A-ExEwPOmUCJSum^DdIh0pO$%92tzhynm<= z%#J7od{$wDjwCn~se6417iVy^9nDbE%57j&j}Wxq=FL} zuc+5oSQy&yoJA~!eaTpDm0Zq|%}FH^+`aps{uD88X({{fD1Ds~`TwyKMguRqE%-#ttCs?9X%?8- zW0G6-R_Z$%(lYe5DONWo)wv$h7hKTl2f{S#{?t<=@rWx*%bO@>JYz9{B`CR~!eK}h zeafw%r^KU{nVn=>mCg{Bh7aJ+2M&)A3hyYX20FNJI*x_fZV)Rbo?x78lTK8digIn9 z(n_{z1QqdH_WoVUHr>CM^Bp?@hvAspk^6}?-AN0n+cn!)qD+Pn&m%nbZS`l78v-b{ z)a5wgqrf^8u|;Ie3MdgK@d6*Oz)0L~%%PSPI}U#!)wNoWPXh3Q0|!6rkQ=9i0r6r z96TZ^XK`L=8|)zG_K0nU9pq*-ZF{I|=vn}|%x=M&B=}XH^!7i2!PphU%NTOlCIVT6 z*7gGtdAHIANMf4kOCh4n9W*_cUmw0U*oE-G@#DXzH#k>tXK@b~9?RU>BJn!$;lkry zJeZT3OYG3U+NF*B^ojg+<0)2hLN2m{J6CwYIH_|vD4dxcML#zXS>aWQGQWjtAxKukhk-fiC>o}UM_kW!q?II#87G~v+si=>3Q zQ-`jQ;{)C4${@BXW{{3=63;swi>vmi;GKXl|HCzC+(Hy|1;AAVuf$nToA+g@8t5&A1{Rk%C~;FtuDvc zA@cLRn8NOU+Mihcv|%Fojjt4vd)h=pY>3KT1lvwG!Ir8Bw(#2)LYyQd=-X-Zej4S7 zXhR!4TA-A5bbMy6oX~+MkQJZe`smR(qP&gk9AYD#xK~j2Ys-z#9+TO^XxcM7+we+_fR4=q1-zQ=QL_rM zblw1$wp)_1$Y-Kn1BzRiTfhHt zcSpR9<5g)mJ(|Kk$jCzRdrcn)a}AjfabizdF-7_u;bw0~RcaZ_=K_>UuMxKzUZJ%*--_&Yw^0_@Bk9HI8_OKXOa3Rdn{Fhn zXE+nfD&OP46*R$F5NWYhuBk@MI0n{VHUd>8wqv2HG#!4}n=^frVqow@cGJEi*eyCG zCrnS(h>w7=thK7AR^;Ab+YzcCnQcyN&FTyWE*g*^Ph5xVEK*n63M zt6fa5F)_;yT0E0TNZHshk)x`Ay;!qbGIb_pj!1an^Dj$Y+5+fUG7)_od4JFZ;HE=yp ziNR>Z+r$6t=1kUYinDCeDHd9|0?(#0zFEEo`?SLcqdg%}@YiJNPrF^tY9WPz;jWEa z+cEM9K6b_?cJzJuo`0Y4Z<$M~uT0F|^#8HKHvBK|Ig#|m~%Ke_hz4c_FnIL-}SE2uH#Q$JAL=LH{SBM z5wD(p^_X?LA6ip*whYHk6^pj?B7rKUsi|T&Ot$WzdbQk*PlyjHZg__4*XY#xR0_J6 z*W|R=oJd+6SdMPBIY*_9pD}JUne{9M1R%N-vDJ6Y*hM{rOyybJTFopMg*2{^grfl- zbvtG=ZzbY(uZRLDSk`g41+vG*nU9%!+hU6Dg@V*)}p3s!EQQ%oI zpEndrXzP)N+Ts8J^w~v)0IddmF4fv7loTk!8P2I=Br{mgreRk30mROp&Nw6WI(mjH zt~{Zo^=a;6R`KBdm4$Ddq~a*xWfHYoLM>Zr*CA>(2j2vJ)zE*j)umC+f-WWNp{+fI?SmEIhf$m)Cf3-^**M|L~LBk>}{Mp8_lgi?qIq* zr@uNz4DO$YKrW-0ly2H27N7>K8a6A)s?8F3E7eJ?x=?@t?@Ss1g(vSO5(tij4Q=by z2da$vNRYVo^|z2fLswNOAHWxQkvcCu$H&8$ZZ&lfx-D71D_=S70*Dm?y)DFRt7e>OhMImF!(R}{xWuzGV5qPeQwa?9=+uHuRCfGA zC2d3bDhlVMfeGA_d=m^fuknMDHecq$rmaDL;Q ziiDr0Rqi`vrX(}xPZ*e+KR>VT=G?OJ;lRUHfqTbBmS*JTm*px`l2EKVknmjZgvaj= z4#}HX`pLvnl#$&MNSJbWux2po#>OYj?w54w`RIJMYST-#&F=@3XZOQ@i9awC*KX<2 zHgbGUN$Jjsv6Dl;%dhx;owy;NY0BwF{MbW2`SppN3;N$rN-lsIH=s8q z^WRNXH{sRw><6%7SJT;#(nZJpB-)Rag-2%hIYlT72k|lbyH4yHdCZ&0=mMw6<(Ls1 z0R`jKJA)6jop3Yv!c~@#ageNg{v=u-O7oK-D~y+6GHWJeoQ$&kF`w|~_n_tQ`kO|R zm)#7QIRlLqo)7b%*VHS!<(LS5Tm>gHwvlwwfaWnsIjZT?!cRrY-3(IhD^F7HHD6M0 ze|}BYhtguwY}+>S&fF#Z)WRjLxdc)D{7K4h{)v&2n%c)$Y=+_#Oesk~N%9VT-J9wK zfb8Y`pE9Jbl8z-MW#<|J>n=Kln~Wg5ww6KR!v=mUSZ5%y*OJMC_>U2L$F`x)j%}oR zxx%>fO-v|JzWN&e@U65~c1OV8cpYUg-+>y`B^%cb69NXFuKN}%cR)_9bNd7t0z(rD zX8?}kwI~2XN0)S6v&qGxPqvCB3(ym2fbGd%6(LYyCy+Z6%0@ESetpLqtKdfVXKXp` z>xyza-WR_ADJiwFrB&Q6wEcr3vBMw)7-9|oHS~mtwzUd0aStot4jrit{Qy6)_v4|e zxR-~`a*Uj*2)C_qu*@Lym!ht1s2f7(4LoX(u5FN~`iRiZ;Kn=dF$C$-01ACXXb*K$ zgho%s5kruRo*>BFXdk86!Lal*h*LA4ZhcDV;OA`p_B|*$R>R35(L!O^adwX6rBE6( zSo7l&ZJ-;0?A~FGajl9eD$FAXhPY_j#4>Q}TtNwOE!Gji8St&XkWfx4j2=L;z6<SoQZ6Mew0@XN zgjzoU1nEuGu2BLGW6D8d#@x4dRm2PG_BLCmE#78&^4i5EfIb^bL|!Ythn7lfh#1_5 zNd{!qOj}8oQ7YiNw`OQjtx63Lll;M9KE-Zu)&ZBsd!Di6X46}2tkmL@kQwuuw<0qp zMn$7q#3^LPYDjeVAGH90+Sk|V5Ry|PR^WA6Jvn}b>TVZqqiWVT(+T)G#v*qkiUon_2f-SF}S8Ms>f#itaN8d>^JC!);r;>yMv8`faCCAXxR^3 z@n&rVVCmcTwP7UrUFDM~+G6U)ot2xhwdVSg|Av0@|LdgOH6L1po86U8HdV}(26}d? z%wW&zdvV-(jKE|w65y~6I#JsXKon!}I%^kcn%o>N`2`o)@se&wYz7i$^M&d!J;}L> zKaWl0H}q&@XQ=Y(&s^h$O={6Pj_2uok;(_}s>tAECt>elm>78e+!v~2N z>5Z@OQu2W@B5|l-+bJpgl@iSv5F=Gzi#>Yb^a1;fu8vIODfptY$g6S55JHzSl|$1& zg@2r4ZaL{Be3D~4M9$1?m4H9~Tn?Kw!G$~p31!#h7>I5Lc=}14FWMeiJ}gZJ+bS+$ z)u^!u*&_)VpP{-B9GF*prZ_NaYG=H=M(iqn5to?Zvwjv&b7P;?Gb4$s52?vUgdC;2 z0}5iNweef%xBM>_+Sw)J4z5kma?W8_$4FGfSG8Ed@)X$a)oYFE%>dZVoWU%bvN}9S zPX;IIL+dMj@NCXmL+|GNRZft}A;CQkgM*lTnX@KP5YT?%=-}>LBdjs6W@OW#|6EAs zyYcG<-Ve2Y0Fuw4)^x65*-=*sa9|E;Hf5?fxJS44IX>PbpvcPScRe9=%JC$RkX|>4 zT2OHy)H=27+G<&1K-)Db5rFeeqT}vKNx)WPAi~GGJq3Q3xolE9y#Y^TQ;m3?iMjU~ zSlPK`8;@Yhe^{L<{fh)0CC0*hJhrjbLKVtyOa^Cch?*Xaf;t%Ijr&(b`DOz z?)aGrq-=d>IGnri5`D8-!`0V6UvXcf}v~tsc)tma=(|kXC{Rbw`Uez<& z=ZK7HqtgrOLgD_!;UDEjKZCFTsEqBQ^nxct;e@}Rkbi4#+0<}g@7T!HzIplOd36hN z%YGgXyjB(1yyocrjfW1Y{m|YdE=aBOdm*)p7Tj1L%n>&IEB=W2^^72V?00(LtMMTO6I%az&w=G7@dzS%bOt09Yl#ADJGMN*Re$m5WaA zq^dj%OD7yjtX!JLOfpLtpU^n5ezIP+FP8~HJA4A70>1^(DThl=Z6))F+_Y_mG*Kz< z3*S6A)o zXDM9vBO^-MN~aK&XMUVke2eU>Z5Fsy)yl?gsax5YWalcndD`-yDTN3h?>N`^q^>ew zE^Vi_z%6yp-bG@=nFM2!_IciH8g+Z2o$x#BBHwUyXoJyK;X~fNwEmrFpy>TEw+U6GI1GqmXy9K-LXD|jUcXYCH!Zd^nHNw-2W;ySm>KpNKXkpPAZKWQZp>>4hC*CBG$NZ7t?UO zQK(`jjR$?1d7}*}W9n8^Bs}D;bs}HQgJ0Zy&{q)I_F+%%>Yh#VPJ4b^jy1HE6|KQN z0#jP(23veqM*^&WQSWz;VWuzj;wvvJ+amjr%g_uNk^^Th$%sQ1sJ1iS{ zV~c6A7jio++46YtGG05(41t)tu)5Q~4eISc0TS@EExDkkJp(&p&TtQIf2GA!SOmG1 zmFD#}AB8F9j9WV?(FS#=lyb&%ZzlLNz_O;!mI7=h&ahZX#i*6AUA*1+8~RB~hI{B0h?C=rQ`Fm{ zwa;ehZWedlVqw5JoW9eGSd#>3QAq?-K8aJrTs2i~Bh8qH#1|LMYK#9Ik+lx1*4a)x+xR)?)0RG| zG;Lh$tw6#zsRla4^o_p{h4&PNzsQN+o18IiWX7>0Qy*x{7#(<~DzI*BWGmRT zd3pKsa?8er1GkNh+ysf_O?h=U_6*iQ68YK0(%mJoNFZTR?}Qt{ra=|??-DqXC!F=> zeI$wex0FOCEk94FB*RPoJ3QOL?}grAQ(`zDy7O#JB>9^)JnL?@QhY z)G*;jMk+6?X~}?RVTg=DmW^cGwdeB4*IZ1uy>QU6A)I>a+WQI#U_i%5fF-NX>sf4% zlFG;92lko0YRZaT@3AkwJ9@hyb2n?9a0{7-R9HivTD|{@dW#{y{VG9HVe@M>?_z~* z=3TG;`bT7Uz)1!BN2u$Ob!)4?WDJ%bUVR8$VnIS7G(kkxU)J4zvy_zC;S%QqjZ<{V zbz4O;o1t7Bnk!FX+GoJ<|Gc^n_$6_v#l5~-H0C^HTa6< zwoU-PvUHK-{*M$Sq~N5u&J4o@Vlj`6&RUvXltVpkK&QlD&QMDHiN|^P9EM*Of)AUl zQUzLOsDM*5fwLj6(!dmXJrEpr_Pvw`*HCSgwN_M{H5snkwYVp9;}4)Bq0Y6wDv}k# z`j2O}eTU>6Xwwx>A}=2sJAD+PGOuRa+)n|6t`7__&sIZ3sBAwq!(0qmC%{n^gB0h# zt7D)+C^Yq46T=lMnKEK^2WX&}n+F8>awcg?=#SD8p55^&5p8F!1wtXgJ1gjoRYsIv zxC}2>k$eukwH!JySj61!k$!N zTXTeE$2@+i-J3sS(&4KX@`EyCu=tov#M#!EZ0(?L8m)*8;i?9xDy)R%;#Cwv|FY5| zU=%}Tn_D(tK3PfSyi8o9S%1uD2y&|%-Jc-?b?I0oCMf{;m)en0_(9foi)@#ghUSt= zxVami4@PKhhSF-C=9^OA3cQ8<80}KfT;Vl{IdrvK8+VhIgNV@Ir3CFD9F>^4B+s_$ z;BkqNV>_RWJSLK@bDwuw(kP>L$%r9hf|3GqZAvJ`V_xw9$P9=JsmuY7-_uqYi;#AD zuU~zer30g8zriUmXp|F-&%)poCByJngF!<+OeDY5EVLsvx1RDQycRv>9T_9pTwZ2! ztZ%bsjl(Ii|9Yb|)lu()KDz{y?BL3}Cy1=*I`}?CxbE>n*N(lrW}pkes(FX3W!qaX zu1QT_HX>u%i1dQTL*X}y!ms2+&jnRikiPM;Q20+p;b%|JDk}~LF0Kkp9ve9Z=8wU)I?z}E_<2|A1+xTB?D7yK&^o_^&iuS<29XB}o z*mbERExolZE!*x)c_A^qBam8|vJU^>UlT0c7H%GgF51g;qBV1~%2L9S%#TZA&-6ZX`I*l_4&LXW2e8J7X7=h8tLDseD!TTI;eaLHYKN#Q=+>~ ziR7+atJh#t;^Qa@MdVV#FP=6f(hWt1NO6b_HYL>lTOcJFB+xNQr!7K7*=L ze*PwbhEt`9^VbrG0Mz$GkLE`R@yIQN*H>?t&$M zy|&Pd*IJ@B2c)VV0u37i;h_YXmxfE)e+19kNjK=GwT1g443;0PA8rfRc!dr%vQXaWHzypO#|ev%&BX^k!iLQoHBG?W`OIAc`a(Pv}&ycWMOwO zFtlQLP?|5n7`WrSDXc)@bargk^l|MktY_dhrv>D%f!bv{Pgl=9NjOk_~OKpUr{fmmZ-ON-F$f zUCh0Pcu|>Aa-q`dfa6bkdUe*_4tmV5?h=Eu4b&r2x0i z@i&ig6CahSRSaeYNajp+OL`-UrK%~hE~t1}zb=qVrxrt7KSmSlrkY*^Osfsi@;_>L zNG!>4-y8uo)8r|ub;;rqJK-p~I#}}QBW6Vp!;;SzLLliiYKxm65V2xzenKiTW%LEeOmJqI+F=cI<4CT0BxPa+UNEnO4f?nbQU9)n%7!P1 zsRGQlf|2xXjt2%;k~T_b*)Nl~5qPRwIk8qD{&a`P)@3DE$fMQJ%LNqQ1rFeDt0`xA z(PWZxP};N*Jp{u;L{k!|hK5iaU#eMdOSlsyt0}$GVaRy3h`l#{IW}Jg0KNF-K&r9% zDw~6lxiMRjJ?1UVeeHtGdx;G`n{K~001HPHyvm&Q)`S2Q68I;>_O*_vIN0~tF{yvM z_=t>U#TnCz(+mC>3Xdob56g|7d_=~vMH$;i9~BvVbY5L$ZdqwKa87mL^pePXkUK{5 z>cY8Y7li{ytUb5%uyL_J1QI^332r!PVrj?3(gP*2-v$yMNJ&_fQu)P*g-bSlzG~C% z+UC;|l4lP{9zP)I(x0!~)Lh%#KOy<`{>kefJv^F|oUwg)#PRi z2PSp?Gc`Epgo&}Oi*nm925P%66#eYF^nwAsq9-S16!h;EJs$t-H#qvjb*TkQLeX{j zx8r)XSbDUz=iikTzN;v@;jsHhcZ~czyCil*2>1>E|Ew^*qqtXOa9Upbyt{*iZ-$#k z0IDs^i8i7;+mUFr{ZS~E0oZu{y2dY;mVU$y&#t;V>BipdIsfUh(%pROrMr^~!2`$k z>=g)&i=|u;+LI&yIwR7;iESIurAlNM~OQHn};n5LCfT45S>LvX@0 z(~g8;8ZPmHX{6kw_n9=H0zg+ zr#`iDp%GusRS>mWoFZeOJmzQ`W#6{Nv7ZYVXsLu_!8lQd5pbr{(?%E&*4%oC>gPtzxd#mJvRx7hbE(qQ+(0B zmt5@1s9K7+9{NrP?vu7(hB${u*rNNm6Mx>N@BDkiz1C!|V0w&wQ#1g0G2u;oo<^+JF{h$V#$o*tGk_aU|`iejX9C5H^OW)Ij^vv3L}hA-Gc8CMo9m8{1n$>O;aH_ z*AVoYGgAe&bP6C;SQRRK7I=ygi5brcL6|n@!HoR`!170V1oRLB5QLaGCI4G2bYRZKf@mxEoB}^W^qg{&xhcK^8L?Eo7EEn< zbdNY$g85k3!rz_2q<&aGMDr^_YH5bn>;sQW2u-^QcCmYrGs$g6Hb-+WoT`gAFJaBL zg6d3I8*y83tXDbgo-<|C(# z0mrs?19}0Z?m!J&?&?k1_cX79=5b*1>jRT6oxXC@v8y*7eNAe?Q=#w)xzQd+WNaUm zv20ZO#y^C@X~p5Aa-(lU^BB4$(4!=B6L4KcZrO$5z|N|`2V*0H`{vc%l9zu=){(6z zRtB$1NtjX-tT}OF>E4N@UzNn}2_*Ea3ARE5`Q^mYFOEzcFlptccUNutdu?+>|D_i^3|vjNS(D z_Sb=l%gq#T61sno}r1Cj$7IM>CnU-B`cd- zh9>Q}>$c`F{_U=No3{;3%zPJ(w9m{+DnBBzq3Rbw_){jAC#*gC-KtB1N#%#*+=cl6 z!LgAQpM+w$=&s#Z+q`vH>c*;2GzIiM6ryltn%oesLuDcUVk%6+fMKywoGDEmY`3b3EG+`Na zY`gX)5^fSHZ|CsLW4zGDZPh2O9rV>MzMk}w(UW%L4Pm{AwoHvTbGoE6=T;#YaRYD@ zu`pE52=ZrBJsaWa`Xh@Q=JXKa$duxSAwgmWH*hCe-sNzKmTAK!?=xOBXgmvKWC&^W zt4jTshZYi5TQB@y(~(1IIwG=XF7~^a-Bxb`71x?z0jCm0gl+`KCSp_(8Q@}kZzi!# z^772ZZ1^JjOexn27*lk^sVHV6kyrfoD7Q6H%tm=~4Pc|p zwWXM?!nb!264{!hqS~5}RJ5o^4(Qb*-4a69Q#M>`0q`X)3O9aIor8W!Z+79U$=r3$ z!yNJ|^i!rxwMSS6j?B}u4BauhYxaL&1*g@$z=djrYunmQ&k=0u{k^F+loYjgWf%y> z3&z9kt!B%(-}~ArX~c+D@E+5DJ$dUKP`^+EGm? z)r~-Rg70y@FX|Y^B{Mh-)1*Nd=24*)U^RV=%ueF1u;(4{(h?u7E%QkgIM+fnn-Et7 z#4>sV%G8>Gn~%YWt!=}aN&r{n^qFW=ueB}Iv=MEH?BclwGb6yRBrF>`0V7DgQp9~R>{YcUk+Zpwxxi%DGxSo=-*M8#cZKicR@JGEGz&F zGrZ@21M-Sr%Q&Ex^*AH2Yd5Y?kWdw>nLrJvt{%FRkS8q&^0rb@sg7wu;E&-2L4Fc+ z+e)licl&T;#-u-|3tU6DwHXZjlz#~Ok;p9#z_2`pc5msTGBq5UjW6Za*^NLft;Tj) zq&imeOm`fIHyX|)LwBfBK8-U;SDBwPNui%J$*0tMyZq)nj>^`g8{x{SeB36Fff9U6 z+Exshkk#_F(YRp1M6u)rww3RYKT;F5avwHsEkoei*lK`_w4^zTgj;WbjUgu&Q_#I@ zXGvM%`ITGyVvxs|jqY8%*Qu5yXE5)MZK%%Tl1h7MZ;hBhwl#?cST6Z7|B!;B46<9{ z=+_y}na5C=ER2g4WrQM9>sND^8c&6ZX!IS=oFTw96ei@m`70^AR5?d82EZv8eRvDb zI^Gy4;%$w||9FHvvFEEqq=c6R0j*H;SVz_5fZPOQCfGHt0@40TUpOTdp79a2iZ1Xm zo^2G+dYh#x)nqZy()jc}v74q-o7~F!*MHWq;;O!eoCf7;qCd0=JybX$Po2j|*cE3` zsYHXD?$Ue=LU(D%0qjhQ3K?X|+Rl2B4TD^H<7hKmdI7i=%EBrm|Eoe^Ip;*+HdFNG zvJ|(heu(vAme>poQ>(s{EHBu^?QSD2mj_`9_(7K(B&d#h6@ipBV0oRiE%HFpaA?>RkW@;(1f6$JcsuG9Zn=)JWFL)zNN(8 zGHQFplnvFIttgVN@-bM5UXt=t5Co{iQI(C=u7cA zFYH?EiuYM$`hbCpJ9v7UOproR-I%dMDJgErgqaJwn&EhrN$=^t<4qf5tD)2m-)iWz zovKmw3R5ynu+8N7<>|8pj<|;PC$>^I+am+joGIKtt)nY@MKU_1qY3P58?KqS2lbH;jO2)FIBA9}^3KxhnhT%K*_jrj}-mT6V}FTGlCtkZqg) zUt!zc{Jv}(N})UjlhMOF5%X96H`q3)CEHR?U*uKNwsz#->+662hJP=%qYr~0n3hh< zdWcaPda=z%K*T-d;(0Y*1x~1(vSLru3+GCQE*Gdqij)n1-Sgf!xjq=MEeoM_^6R3oDitIU@(yZmGgoc16v9o?6J!cWCKIjun98~twO%GtQK?Q| ztGRqlqM!JjxPZnW)`W&e6P1NtS4JciGbZiy&hC;GN!!Aa&~G^+HZy_~W+YATWm0d} z4ZUk)vjj*hEGTZ6F#sMz*sqOBHBT%@o=0h#^u_QJI;kS29aY>?jfr|55jf zKrQ1d(?i9r5Jq~aSm3aHL#$MVVD+rS`uoK!@e<2VX~gb zO|7=oX1SnfDt59qXw!@>F4@qn+N?0S0BY%RgaZHp>$rx}3|-Y!lcxx(8v|=*!qgH= zD7ubVaralAsI_4S$m+TtqMis%A=al{f_)*Hm$`T+7^e{A)#YuJ9gZ zGaQ`q^_b{!_7RcWF*;cbaE6l>CM9zW~~5yFjCU})>>V};RDi|#{S(!(;Q zq!}=%9fx;)39_%btV;`Wmvjpt>t!kC$nKxp*w|Dbcr0=B-b2=2d(|Oquif$nyGnL` zoEl6SG%fe5h`nR)lqbrV&G(Qbmy6D=#|s`*qiD`Fy0| z{5WjTcEPv(?()bJT)%egIfGC=*Q=dj!s9*IJ>T8_AFLB&gHOfp77m(L$j1jSj~q^V zyraf`OU;~T3**S-pG1#ja_Pu{Y=)BYljzac3~d}jh~lUROA(d<F*rZ^ZMj?8Wt{W?I0rsr;bCYt=U(xKq*!C4k1dH{lKgB*Ofj{aEeWFP zw+kLLQfKQ7gUNzNa4NUq?IbLLq+Fv7CXQ)CC+Af{)CX7ouIn4~IHS14AkOBTXrR_d zaEiJFbRC@R)1{5rxY#ZI>^-#;o%WHqRUcp2hOM?c*|S8hgw~39RO_jD4|=&&!#hJ; zs9~ZO!_rH$wsQo+{*nH$4bl=GTp~AY+Ns6Uc(VhtGqzv1LN`bzJ%uAeGD+598p$MW z%NEHb64|o&MD0|zKhP+TQbDOXAo3K~K8fcO{7{RoK?_2e$P*u4 z#s%6*t2AFe$}d;v$_rGi8ijS(#$(N3beMNM39wdwy|pp zoKXL7U%{jR$o41vfQpcrBCFRL9LhfkYct#n^=3l~rvAgSGSspKP#lyqxU0C8!<@I^ z^ALAi=c+k!aEO@*(Ko|J@;{~p!#Xx(%jv8!Og^{?L6nPXrC^P$)#-WoRbG6k&oW!VEm zeSuHzBX2Now<7u^51fEW1gA3Y{Rr8DtDP?XrZhvg;4*P1v*DRZvZ*YBx_CCpoZS`YkL;$K6eFSBIP%{_8*X7dMRzE54+*rep0fh}8yGh9izv84T#yO}&{`{VwM ziljv;>~6AOSew)&y(^BpmX#g`lr?%?%~wlH0}uL`lRW8D-L|{vAZN7f_=$rudFp}{ zpI_YO3e^mA5`+N~q^a68c`QKNs+Kz;?_>oCgF9Xo?8kWH4Q5T@Hxysk1(CQ!FM`h>3@U)V;z`U9({c?u#c=J^RC zXZQ&sKj2nvU--3ZtC;U?9~r<7tZmhnw)J@Og~WfrD|_zb{aR%U9yAo zi<-Sk>PKw1G50%l4>|>Q%G+ar?MUix!WJ2uT2<2iOc4i2nFF$0hNN&omC{3UM-0Gt zJASeSoNjV3>&~X!IbE8+9?NF;K06zQ<=ViIzkSmcehfSe1UcGP+#idH3cRB9#_XpwAn zSsau$iY?UjOguTig+;}~&`QSDZqXPD%?dd-POs(vY5wesEK#$|)CKw?njl`BrbrF~ zf`V1je1c#jx+jHfXMJsn0L*H#0?L{;B)q42gl%L(BDz4@Mw~Y}iFol{GKJxvHO~xHQp*6!HFI<_l>8b>%1FK7 zw&^LgoBt6220H8N8uHiOkVIiswnHOxDxp=1qsM`oy@^{6Zl>u;n#s15XXL)#)2@UjqxGEFd*d7{lm#nSt|NKJ)L*E+bqbnV(JE<4cKq!_7n35nU?6Om zfA|v7M$WYfEb(eDUi>P)cO0W>&R0^U{NpEHG7*8=x(Ky396C$MmP2PLIsU(a+TLl{ zUw21VbjV?&+Usu5if-sCiS_B9xV(5}^R8j(9YcFX+EVk{XTn_awQ%!kFqgciM|5%` zs_u@O)h6qsqOYG8eUs7KDgFD5@6UAC_S0Gh4bRh)GV?R`ASv8F zux0Q$1Xd#ddi&bjn`BX#NB!Cy;>|qmNJxw6ZAVh5YR?pi5@JT#SM8C{iLKJxk^Gkb z<=TxCj2bu%|3;_Icw~>LPE$xsx|OJ7B9<*woQSTeWhB8yzIfW07fDU^r#(ZSxr1P$ z#T|qxZ+U7G3_@CS!m@Fr{<`{JptPFW27mim29umVmL#ojKd!wUlaur{_%ql2i7&E# z7eQk76(lCfJ~OZQctfA!wxb0zS)15MHGRA!XNyyGQPX}Y1#|1Rjn(47odqyJ9Qk?9 z5-hy!bxT5m82xZ_NkCK`R>#g~Hx+sN3ye?Pa0180mx^QT04=Ym{(0va>EaeTmA3qR z0N-T5=$%Qq8BBEIR&@_GB6g)61 zgk`E|+N7SN`4F+s2?d#XXZuyLv!|1Es$FM$y3w}=qwGMc|~D%O06BXh88j=K9wu0 zH1kuPx>^ueD$8v$HEN>LC0-yIaV>;~$ckg#aN-$tY&L~c4@wO(=i}K$z}H*{ytO8E zfXNl+;z$$?8b+FK`FAPbjA!ZPTg24U90Qdr_Q?vQqbGqUQ=%{pg^N?FJG0AckI@C_ zQ9wJFnt3>)SsGVCU|DLfBzK`EIrTNu{;B~`2xPEF~o&@z} zY2+wDIE@;#Y!q`)1NnH_=t1b`u+=Q{adw>VZI{cI2^Gb)3&S(It(r@6>?3fql{cATt(ROQl=2? zIz*&xAetokCh{=^Xoc&%6}rXdhrm`#kcnQ}mk|xQ?&bIjHr$8=nsGNWZ!8+e3FM-< zHz}B;V^-6AfmYgEJ?tZ~H~Y5IKDz{oAumDXZ*NK`d8QxDRye|24hheTn+qVx?qmE- zAwi8L7I$v6$RRr`dWZN8hjHA}dW!Kx|IQDlhQceW7|aX+tkEuF+*#xTj7niQy_YRBufix3BjD6%;>om3 zoeGKRgN(r179N`^bpR+O1=H{ZICY%ht+WYUsU87PtMW16!{^W*9v@0b9*BV&*GI<# zHNjLFw*UVf2}!=N15}ie3b;%_VboYyAFUi^dIE;ae%56~VR#rwf+7><^l~WTYokDs zHQixUOlhtnS9{#Wr;Gx35yX?`ISW2WSn(0Mae&~^Z<*Y|UuqZ)=;tKO>E8Q?8BBx; z3KSDTr~1Sp>HD|*sp4HK_%VM2`YGahypjwT%I=Fg0!JiYR^yVUkIo@Fy@WmUbx3&I zNh$kl43OYdH%WZ5p1L^de%ZYK$*)#cl&cuq`IP9Gg=5s>(pAxw{s$<&D#)2Uo^&9X zCUOS%_c}P%0BFwFTCO;t2gQdtnGJO;TF=R%1A{P@0k)jJ+|gOn^to738KA%fedbEZ z0qdn0E)KQa{vw`hgP#XwRy-(+Um099uitk0o|vve6_(~I>SAE(iht#7?!Et)e}Xr) zVjS(OYa;3%yJbo5mmK3(%$np_EqnVh*~6~&njw+X5!((3yGEZ$iz+aOv3&{UysXJI z)mvI2KXY*qub;I}(#p)5Zz>~XWVNrMj**z;w%1s^_&=EU$djI4Pj$D00GkoNe%gR5 z>`5+W4}SC2Rrvs;oelMo^cmc4^ryg=d0yab>*%Hnpe|Ni+di+dd}gi-r&{7NS?V2@ z3yBhdl4?4B@y3K6NMbKqOr1-Qc`RIn>@(iBn|*?AZ5=1)-<~!8kfF5H2ih`52cD@4 ztV3t)jJ&#edHM5l%f^KRw~dY5gwEMF<<;HTGgt%9|Ia3t?ksb^XeJtK&p~7DiGhR-TgOo+?wFc_di# z5~#DgL7g2wE_N{*vyIOwLGSL^$)RVj{eI2bHot#RLZ1A&3e{s887O0vANa=ikPkR>a5o3&+RKXh(9wCLihGjpkik#E|dI>M(*_0x;%BsOu5rHg$b_ zBXLdq0s$kjIc4>i4B5Vm-GBW}jL{VNWkM@H#oQX#XAxEgvqqR|M$hb)=fnJCQ?Kln zVvftk7b$k}Y2?!bXOTYZ7!FKK-;o8y!cICL+Vy7;OR<`XdF8d$Cag~EV3e9#_t>KeKJZ)rz&oOit1$l z>K5@?kIfT)mT3L1?_mQvjqSY*#9_uB>b7(n7Ll2w&nuRS^576DL}Oeq)&9)-+>T48 z@E$wlnR{Yu*e#MpQP4ZmE*j6%sF(>Rk|>+vB63%}4Bvc!WtTwK_?$x4NZH~RF#|aE zrs-=$)%MCXw2+jTsay7&n+?FFM#~KAa4KI|{#nBYOFw6h^uVG~9i#nQ*yb--G^yU? zKh8GgL?Q+hnb3sg5>(b+MVI)qXcTzIKOGy3w#Uv(# zE`?Shbd1D&)_@OwEoJesW@vE=o+vYT!Ldzm4hp2K7ZC_mGkV|(;UXf{xK^NKzPE5r z%W&Y{ z>cAZ(kt+1luFA`=$}PJx9JstXaPitvu`TGAy(E}&%EZ!MlS+Gzi~Tu}@YB?Uvz7!W zoIJ7g@Eekj%u3pE;mS=NwaphMB;T2tJUcUK`2{OCeYI*+Ed17sYf{sfjmVfbBE8`8 zQ233a@GCjdbK%=xkiPM;Q20+p;b%|JDk}~LF0Kkp9ve9ZK<@gyy1BV!Bg27{0rben z|MB2icL#?6#qGp@C9&B6a=r0iW$e2-`csp)@7!Yhord=BMXW&&Da)fg^pGh)6r_hF zji9?85rp;UgJQRxdlrQN*Pyg+qI` zCT7iSF8{nF7!(?o1Op|iPfFafhkzT4EV;Kfl$lNri}w>|$6zA46E8fKlnJI$l$=^i z%(%BUAT#lg4d|`S{qbH4<#wF4genJP2k_%)`tf?i*LXOzY*>HaatxM_M?aptqs4^r zCf?~2gQ=hGI@N$i%Zm)OG)A04rl-D~(c1o+r{=%%RPNZP^ZxgQofGc=XV*)Sj~}~s z=*r9Pedu>b9=Bs{!Twi&Jf`eu1MDr8V6V8WA5H=7!frnI!_6ZD7_I7*BV^m{?2&io zLNJL{Jbd+zai5S}ON0u`e`b7Qk;mc|ye0D(= zyCf{FQcL|x-avp?VF=47$xx0zMr@8&+#@4G-Y?ga=loaR%}LL5vl79}APpYC{3r@f z?U^=5kVUk3JgwknkpR zCOXwI3)PGl#2#Mo@VB2gh2stulEw(pNqyvN`0)&uN-`A>Y~sj}8zW-CHvkXMz#{7_ znEvBkLb_N5d1lsc`4G`vq9>)F=7wi8OeAZB#nxL;WDLVr zPwF^wA#O4OL9F{DplpszqG=aN&!Hu2*Up#;1@%Px1D=WVwkAtF@8egl#le37F;q8)xH^rbLvm)WAX_fm9nOWq_`4eWAlzua@^y`w? zGC;4Cgh4gIA!xN7D~a75NcgyS!n?8 zGyr|JUz(ndzSs6hcQ;*Xost#+vpYY8Z=m_J*jl?)?`3_Db)FX~mIjni)IL&^H-5X%a$-I#-BY7VY zzqK3hqEt7Ev>D-88U1K(MSETrWM5j6Rvc+Y(^`WH%rL{b43e;}^~Cqu7G<%_t}Xb)tRvE| zyfg3Xbr74UCabB!x=Hiis%YEG-@}POSl~+~C1<>LTc@8H$pgI(-qd!lXeePM4b_Ha zxRKx@KSjyw(3rv8Pm=yB@hD2bR) z=zxe=B)nCqW^VT46hb%CjzsZFwoET9Gbr$m4^q@1i@@7T>zQ#Pf6Sbx*+nFay0TWT zxblP+*`962RVTDqxi^C&atR6$LBKa2!M7MI4Gt#V4Tu*%?oG&Tv~3;urp(K0{Z6we zQOwiHjowj#23X%me9Yls!01RXgoN9Pb!dghf$QuAE}Emnck%i;A*K?G0nV6>*#v9Zix9+HOiJZIife43#X<&iYR)j{ z^MYzXgjvE=Q?sN}U?AdZk4M0qvb0FoEW5-c!OdN0FrCYaI>xf%!x;Zc1=fxUX*!ys z`p;i@y7daAXDK~lR zJ+-`te*SNd^Cps0T1{~*w50#mZmTr*Okfr&+Z)X;18nOhJFV{I6@prls~yqI6{t6z zPNCblx35;+?YN{>q~5GhQHr!>cA;C96m8-?)ejqxmkmVg>n#>eW{YnV*r_!~ZFX2z zny6R1_VLlH5M1|3+la!hZ$c235$bNWQaLOFRuUj7ZaWuwA@qN>0-V(!xcSGC zV8vmhoo>A+R2)0;$w~cK=jDADtIgRaU&4MwMorGx6j5{Nv}#7fpuxk=^I>-7s29 z9@x12C}e&3<@H&8UKjTMseC+sV4vBm7VYhzO00L6Bz>I9Um*3t(ljQ5&N48>M3W{{ zsEGp~14s>he+~mD)#ci>7m=p>4SQg4!e5nweY^OP1BoO!MG^<*jnB~2&VkP+op&cm zNd&fk(ts4k%W^96fz>@tk%`bN2hLk(bL<1Y+D{j3ys;v82kjsoO`Uv z!6~dT+X`RLscx^U2{EfSIw1bJIweOEYgvnKO22u*TbY}S2$N+O+>h}VAuJznTq=$! zSiZuz_l7~n$6d=GKVdK+?@`cwZS7N1Hp2M3dP$R!;_-9H&L6o|T(V{soh>OVF|K-E zUR6?dK28DeRdikapbwRXuF0P|Z6lxBxS_Nm=Qr6K>q;9={|yv-)LPk>Nb6>fEa1BQ za+!&zh6I&XBD1KrGR@W|TH>g6*<7mWgQF)s#1Xe`4>IPAjB26XTfvv=kO3vIf%UkZ zjTEBj6T84W2&nO}R4!hI7RGV?Fq!bCDz?Vn{3lTxHrPcDr9_~meKs=M^dT(}y2i

    Ca|6%n*WnD8nNh;+8ysyKft8>HJZ%HM>fvY7zFpo(|miftEa`Mu| zhWr)U&wCfQDnAe%JAKp{7y}Ph?Mb?D%9i@`y{zglqR+hSE}F?S_-4pugU-ZNwOw9< zKh1s2xxaMlP0w|Z#!YG3OdKs>8+P!n@`g7IBQo2dOKT9U)c8`D^lH}mkXZ?fimnM! z<3=h(upt*RrFYnvNNJ{dy{KvOh$w)(FlXh2)CX*WghqAS7K*9ux->B-3E?R!NR6#+ zxKo8)w#NwLCYk*9xDGF=)wQVD68BR_0WEbVtayYLTKXF~7XvY`M~}0==w%!3EPJT3_)x>;{UhYlYb|+BUwxWUNhdq7U;JhUuba z)5*=rV`e4M3cmVl9y!gciBnL7)DPLU$Jd4LKweJQaPE9>^o3XH2m*g7k7U)^ixYSq zagM(_CduWBd;U$Yi1|vWRYz*p(b2B!FF#+86(D}rm_8(=%S=F$tdOcfgqlQpV*{P{6bg@QGU#>!jv^$$j zgsDj<=OMnTkeWJE6Y3Zl4k0XY4J7t2Xl!D#ZRe8-$>Xz=PsvUiaM8+516FV9cTe;E zkUkDfp1rDPw9gS4(?+Kk)P=(Ri^D(4jeZ8Tb5%1>%}xx*YArO3WbGaS6xv<{pai^dEJeLW>l^;ZAZrki4AWsmO}tE73KL(&VfblsZR) zHXFV9M-|#qIa7W~WON+e$K9?iJR1P8k7P2FDLKW4vEho0nP|{^;{}m8JGLg2^*5lz zj?+-+a}*PR9V75#D*f0WSuy+}$g61>G)6M8A`zlW-pS&1l|o6KJZj4~?uL$>@umO_b114m}H)ade));1y zkS5l5Wj2TqbfA(bD&xpuq#iLkym`1q9!rVvv7Jsi?lk*g8=zFK`_7bNRDZi1$hdPPtDtO>KPSp7K!qp zA)cunsxcA)wHp&h;sxBJT2Szs`GQm5$<4tWbM+X4u)4%og(oFzKnP`+u+SRlvRAD; zLIcO9bauACN4oK2_5!B(_!9jQDEM9@b)x`*ZC$Nnpo~pSz5dw}F01Qt7kcSDayM9M z!4(NiW&X9+h0<6KRzEaU6kJ>XkycTRRyQVWw}g;g?>L2PT#OIx+G*etZf)ez(rqjZQV>MccV^b`K~dr+|dp zo0-%(rdMQeT3&lwzogD#!ry-@emu2TWGk%xm-b6sJ{sKLFNVGVPKvp;tL{#^QE+RY zE-T$F+N#~1)V2Dy9?X1oJ>KI(Vx%h2lWoNfOioq6;D5%A^keXaH7yz7r9vbcTsD%c zs`gwS`I;Rlw-*jNHiT1eU3*`l56d>?(Opdyf5$zh6%ATe#J3rBMPv-qQ`?K_7=Bv) zcUL5eqA>$Z861p3dtEsAKwIY51iMUzO~J6V>W>Mwt=?VMRb;SiCg`%UZWrC~QrAHF zcF;$gY@VuvyP*HR84jf+B_CItO3b?3Z@%HJu7N&m+d62@Xdtn~fYP*ZUmRQz%X!Yq z-7C9_+KaMS@i`$JIud_wY%5CI&J@*cZTN8%{kV@V)ZBtNCdsg&l#Y?IP_WT}o7HqY z+H5y~?rh6;CD@8rzWz*VMVg-2kg$4#yw@V5Q-HLYSh}KhTEX<8hoPrs_5Jl6uG<|)hJ5?0*WRdTVmFr)* zh&~A%+mX1=gJV0@%|%4T0%SbzMf^~~1@Ihclj5TgGdMH6>AjdAM5AKv*>-%>SU_ku zgT|j6VV1_b4CECFxs{Ty|U)QobLva?EF;|sWFY^i6!v5^89tNlm~Ets8asD zSfICDLA<>+NkPaB6>B~fD(F2zx$1bWpL~3rJCH1um0VEhmKkDlMbNSgsHO(MCO$)Y zpLpp7 zXCw=zY(Xn#C3TAOTo|G!y$Y=s^_cV+ToLeMXSU-f>zav6+Jrl=%CndxWBkfuj+e?c zq=A82NKS(8uRBp_f^^ac-n95cE(AczRO_V`)u&VaGDBHh0+einSq|!LHM?v|*GXGa zK~XnkZt3ucl-=Ld^=Ozt^^4l&pOFd`SI>}NR=ZPS1{-g(OTr89xM{T#H!y4!{MrDL2--2hfR6p|6rju$rLR58i1*@qKvve*E51mdS1K1UCv z)hY`PttlP?TY$2`1Pg0e%>sv0#*6CQyR56M(RX7iiNK7PSq&NeCK@)>h<~x2X1SOw zmd9qeRV3f6%aMFL=+%gxw7+ZlipIU<>3{VO@MpQJH)Y?`yec92^MT2)4@|mr`pQkm zuHJO?HK_$pg~BJ~MtdBQv3*p=vQg<9{}2kN6^D> zh66jR0w0Wx4DOp(cS~OWEm_yMo_J)^>jRS44@ep?dF7^$S8e*Jw)upFMI*Mky@Ra-C&p^<6vN-axwPQEB|8s0c!qAm$MGWPp z1qJaYWB?nCxZn zPws|oNBjojmRR{Ci8UwFXT~b~Rktii=rf?UPs_rDKFe!!V0VuI(<*KyUY;Gu0BqlhWBZ~$xtlYcsvn#a~#I24E=RSp3TdOH3*j% zNG6CbYIp&rXiNgz(1jWTV%iQ){i&eZ=zHms2xL~P;EMWH+t3db9zj;#LIjG6$Do^V zw5<6ee5>zb!lzjPBz$J<>?xp@^3C|%yop4NqAc4p5b+&P_-SLT#-;jr7aW>bF%{+) z`7uT9o>y(;KdPC)hB`Q7)w!H})LioBSc$B@c#RjX=7w zaG1koT_DIz6d5&}3|suWx*ls#rYKRiPFwl3=uvS=Tc#Enn{O#f+T5nC>_dd1M{$~vB=v}UNNagGlv)J<^{ zrsdrey$AoY>-qY?V~L}0K4h+ubuX^T&&{8oS9f!6+4yka;i|yB0AlclEXyq$8xG8^ z3jA``=vZ|i;kn)kkHZ@hrvINzEZtQSyCsk? zU0KvcV8vngj~@9wX|}K5AAzDQPo8aZW^-%(>+OSKFLGY5;=LXSZry}EOH14NW{;<^ z9Ispe0FXiDI>c2TOZ$snsHJocv(v5zp=~Sn)3c8Xi`TaiKX)aw0L@J;k6ZV{ub-8&Dm$C<8bO)_dn&FTf4A4q%`^fJvzF*Vf)VWpAm>nmkR-^ZMpUz{e zE9-lT8n*$}m4%7;vC^%>o2ib38j_4PwO@s3IH43nW40D||48g-v@71@YY!WY!XRB6hQ*bPZ&c4K1y`4PB<_P(r z(-9IEZT0P&jUtNfq|PqrbHI%PL)h%=IRc1zkV6z1x2jOqJVto>YhJzGcE0ti(V&K# zOt@{-kc#1woBVp%*lwdwa#C04&cR}|?Xbhx=i1*`1pypYK06FY%ICpH2*iQ1hO}<^ zn!gRF=(VNz%^VE1Bgs=7b2B}I;6fD5JGXuyQBe!%-2Bur<7#7dz!$fKXZ$hj4T-ES zr?$NrHpn|QQK&j4eCuU&r-{l4Jj0Yk8-ADqKcGKRVPOp%WN4Rr&xKd8R9b6jD$AyT zZLy=X-b=$m$W!0Pl%5)1*{C6wWjd)K8;qdW-rLX=ZSF-foSN1dOQM}ofQzCmj*N$2 zf`&vFkS+#+#ymAZv3U#~WK-AAuGGlpta?GZ8o* z$RiJ(9P;tDj77n#QWB=r1Z&_4xp!jeS0%A~0tr29f~`PhUrsFjVtnEN5N_|T+VuC@ z<^&LM{gXdO>owRyzFNEF&VI?WA31@X`CpTskm&^v3!BIuGv}R7gj+;5Wh=if@lfa? zt5aS;8QbBhl_~4+|8Hx8U8{RU?@i3;n16S$>yaMO`S|g2{P;kR=;iqFTySs$fiBN@ z4P)Kix?)PK@^7>KR>mZs}NzeRlS<_vk<`FlXdF);Z(!QsnW zD%rkE~gG{yy0JIh$aV&#VM&v^S2&B zh{7?!8Y0;^N;wrDQc~F_j}bNd^k$}*8&TtEQYuFK=twd*K+os%NT0E;3_6Aj^% zp#n7rThR#el-0x2(1AHODA9&z*m0yHsqocgj(s5RmbkO@AAY!J{K;0&zSAe#)cev+fm z&N+W2W>(kZS^AU#0gdU)fK-sBy!97Pa|=r$W|W1HuFbW}ae}DJV^T`LAmWAdG8MHe6f%QwL96$W8 zEx)s4Z4)L7tnI}JagpTVTiWBnsY>5qx2vHxlSasE8yRtx*fX6&DUX51NV5&d-VGai ziq^IJ0zY-uqQYxm!@1=W>C8AJ4=DMQ?|_nB9uitWbMr|Ho0&`fm#uFBWUn;_gmfZt zNx5-eOx6Iq&1>J3`7N_$Jmm^zmJ#%G?=@XgbXkl6sn0`>`;ry zWCASyX*2U!E}+buV=$=d&8e+6b8WKJQl<{9>#%iy7yP1nwau{*r>sj(h=95!nJ6c@ z5Rv+nXS38qv)}UPO&im-WA9dK($-RTsOYvQ79N=M{s`Zyw#rwwuhPS~<~<m;}}9-dc9=%Y7lPq zQ{SK#k+X(p5mnXxMDL_RVIWx!@bauIQZ1i z_hGBgl?`z)FFcKK&`Aw@&=JnhK$66rv?kaAp?Qoo!9b6esyPxs3Y{ez{2OJ1I%Gwu zLKys`29B0RJ+NvY0fYbNrMz=S0*GyV|E1xQ_Fsttw>JE^kbZ=f2TV>l$73h-ti^0R z`oe_+UsGxwNUF&IlKs2USNm@l8xXjlN>kf325ZpRKU5Qgq@cQrk3S=~P$4wwdV8N)jGcki-VP|425%#1z zJ#FGFf!;Zmv`^sgXhh^S7qDHf3#f>YDORGsn8Ga8bfkh&cf>1xn2l9J&W9cru2n>>R|xp zs_H=((@vON*5gG~e1`&%a_Df3gr1w5aSg*0&Hd%9wf3r0e=AK;Se=M%MQK89QxJ=RS`Xk7K9{15W4l+i(dr&~RAq$etJU=?*I0Z7ZX_ zH(u;)W+4AdMkMuZ9p6%c+q6Sh;5Pfv6}XKq6U^HohHSTN2UI)e)~o_5aQh4eZpFfh zWZvDuLeZA39CrVWzsV{)G8|d)Q7D#y#sueYet(1%-Ypw8X!|hc+kSU>*Y?hBUvXI!K zqaG}U(*Q2+S(|jU5cl=rW3*vh%Q_N36PX~0lPB)JcR{BXY9UU8j)bmXJ8}oLb{0@f znTnWDEcveU?rxdArxT7Ee0dw_!`lqkDvgW>`UX37(l@xFNqmk2>8I>b-yO%mJge$llpi3#rnLq7KvS|ACv*Aq;)Z*X#c)U$6hAdawW47O(&3 zeZBtcN30Ds+#!q~pDWy0M`Nm6Vmrb8Ki8bCJ_->bHlQv8^l0?Ra1KjgSS&(y9#gt1x6kZ10R*?Y2qq_Ra&iZ&`1VLs%K3QTJxQDk5M9%}_- zX6L`FV=|1SnPW%VA`sz+LgB6pPNM=0&dng))wgC=!+>yCwYyNmWy2FQ4c^(DowBKu zx?V-q>b7QhMxz@&=^vcA7N*;pKoh6PWPz!Q61uHcIR%xNC;kO`Ma!rv03|~iY2rK; z=>3pVGut@H5%0DM38ZFvGfx#98%~|J3IDUzO%f5zs9k0c!Mytj&*D5N${&p$e(tZZd-Kl6m*_AjoP2ZGC6!ffC5-|65 zf$t($3(ZyGV=uj!zpvJg!!p7`ooyRoF|GO8a^D2?F5A}LG$KqH8ezqJbSnNPl6p251UP=;z1F@N)gn9a9b8BRb8F%UCI7?Qu982tolIpj;lw#d zi$>IQP&=pVdH^O1@2Pqz`TJXtE#6ybB=NFi8ma*>nZ)#3{ukUtr4=-Hpi`2licgoR zyr#H`6;|WJOZY7o?}PLh=@$M~BjDV}Fe0Q0<&7Q+&s`i>I7X+N9ofq9x3{PTz%svO zW}%|>+s`&^%8R_U@!3##Tyc0zZnOwaBSjg@iqbbe9SWaT96t5$S!I`n1Lu}RdL5lt zcUxZmZMkKagabL%f#XXeYv43;>z{)?mjqi+npoOd68n81;iZ&>=W2ozaQyAW()Q3N zJ0>5SzHCIsv=QkAkB7o<6op^OiJl9m5j19dEEN7zQTW-@2bL9w0~c2XCXbCAg95kf z^Xlg2mW>PtP97WCO3ot>1kXZ&8x^$SzmnK&(VMMzWy>2}(AFu-*ktQ$GV_-$**dqw zn!np_kp=5>N@BkZrFR^iS}9wyl_#t{`eQQluSZL^nSDn^W^~my4<4Rcus9TLoSWNz zo$w&p41!IHBXecv?Bm}266I4K;>ZctUwi8c;&DKzJpe%BPN;!`@*~t{_!DY}v?CiK zOHBSUP)pLrL=~gS;xwMXqpu#ilW+`$TXqCk%rOYoR>TMKJpjTbe1BEDW^jKpb85HG z$8zcBzi(iy*!&RI1Cvi}w5wc-+Aq}md#++0cqX#0ffE{(z9>Yam z6ABg{wz_NMLL+paYfx(&tHmjzf`DmO~oZ5sjYY~TU#{*#UO+Vp%oMP zV&~w?;1w#(OoyST^!hV@0iQ$?Ft6|dT-n$*?`k%fHxvrD?qcGiw88$b*s4<%v$DgO}Z*B(SHAYUVV#v*Ve3c^etCp}Mk*D!U*ZVge^j3;M zlUXrmq>M&*{8Ubk=O31pOKDhG;5hayxVJ)>m}dy7a~ zS^cAJp^(P;@e6cuHqTc~Stuq2yQ!FFEEKZUg__7iDpoJ??3K9z5FkoD#sq*vX;a)s=HuTL570 z!eCPQK`0+ZFYUs{+UBjpQa4tGqA9>|(}j8Dy@t#G*Y6)Z@Mc!c$HXv2^o4B`uHW6G zqb=!NytVBQ_GXX#ax`MA5mx`1pD&5s+n)eX^vhp-ePZW<;U`fSSKRGWLC_TrnpVij z@ZjZObr?G*zq$ZGLw$%dP(={vrx@chcHY*bPXnS~)!wC#=kgJ8e94(i6`IK$r*Yi*F*?k% zo=gMcsF_Xs2Ss9&vdL@INYN{L8r*Am+xa27*~mU}GN!fcX6B~p*Ug6Ykl41otd@jajJx%P)@&B$QJ%EdXcrpVt z6|-?bLa}a>@?)@$t!YNa7}^N+EyH3aD^$YUwS*p^RPfD;o)9LXmRd~%L#xouyuSz{ zfS3@Ldj@AO$=Xbp@+!Q+HUmHX)J2D@LeH7YqRX9>=hnU_0v2?uOND!p74yyAQbLk& zQwtES(6~Il$jYFpMC5zBl|k2#RMOBjwAue15nhEgb*PjUV>6Z3?x95P*I>Lf}ToHQQCSAQc$BPafd*H4+2^RUp!(f zTChaX(pG4)0|!u4N_|mVNg^tWwAnnA2qC%u81u0nd#=6kf%o2X`+a;L*()n+?>*OC zbB=%f|M*9Rr5>>%JKiqUYZNj1mJX7*(>ouAnJm=Uk+4=*WMT1+3gTX55Qh-kC-xN~ zw&%13h*M#@6U5OYEXL=@s(FMNEnwP04yhhfui=Li=Q_>_>5E9V*+L%5@zV{f)!b0G5B8xtR=W2k@@tuVuz(+p95r8>w;m98zf1N5y2GQQDM$|D)^~`1J zDv~j_FWw}&JQ%qi$c`5ht=V^ZOG02@a6hM2o1(8q3Gwp(w(dx<7GMlDf#GjhDpJ2+2YBMrnzo zdnpY5a|O#2d-Fax6PpEkpp_KoBEv<#{~S=}3vh%C*gX86hzc^(z6?hz%F4T_vo05B zRg~?aL&Qk=H)Q+#XR)1kRd#Ni&e7J=h3#Hkzow|J=PxsL<7~(fKUt(=v0W48K=+@> ziQimWBJEi&dw$GPGAJEbz_Y*t(SmN@q(PU1%b3eaa157qS&<^MUi;Th_}g~(5}~x( zv>fySH-uhYg^T$2%w85X#$|?$5|FKX(Ioy3-W8fRLKXQ*k0k0QV`JFh8OF&@Bl6^I zI)&4bwk*~rsO|Ehna^mbt;E8F6^5n#|?RLDNsMnuhGv~LmF0HS^_pG(o zFGyTf@~f@|ZS1z>FegEi)SUeIbU4W`^#^BsNc@-)HRIPV6GMONj0 zR^m;e@zXQ@E(Gu#x$&dqq})Ut^&GBZjHZNVCuO>zJ>#oH4|t{C%j;&K#LnH|*JOVQ`z~Z7Q_l>b$gQ*iS zvFFIzA8W&`!65;m&7&X55bLD4_0Z!G8gUUT6+N{ecxd~uNs zR>NczhfuAi)p{Nv8I2O?c@0H~)AU^c$3{MZ-qV`03b@8eJ3RGV7*|BQQY)r&0`7Iy zj-egnDfT4lZZVgn8Zuo>-lmq^UEc}ltYtSykC<4HXc`7YJi}tn!hZL@?*N<6V`#T`XH!GA7l7Y>yu9Jdf$?dTa~d8EBrIJpr{SSMLfV2y z8a^BtmsYsE;a&Xhp*i8|1sS0=XvlV5ep1@A$ywFYvzxmvzxK0lO{t|kn93;VjsBsx z7*qA_-g#7aKD4ucWA8yt`3n|9;HIyJiDm1%9zLFhk(Cic!@S4B?*S*H~|# zQ>2s>%|?(|Pwj(#koV{a43*!PNwdacCEPR`{^JY%<+ja}m{W3SBssKW=*C&M#ZGRG zWGb6T%c+XhP#UJ-udtBC`4UA)3y|lyXPbfcQaG+9jpSbvP_Bsya-E>5Qf($Y$Qbu& zG$xrd{Tg&0kG|+vMmKwzf_bhg8d&d|%P^3zCo*7&@5e*y;B=G#g( z{ehTcUWg?S}vNmI#ixum^)t@w;Myx_S5<8UY{(p`rUY1Sq zQbO3-D47(QUSukCXA6(ns~BZ#D1ZJ@>y5U+7a0yxOOIYo?pINYDWHa|srlXeu<-Q) zpIDc{F120%57vYgGG=yi7-GrJ%2O_C#eLL@)*K|{kE5ecKIt;}a=Px#BJL7t8)k5LqUoE!c@A_ek~sJ_$1N6lFPd27 z=)Ufx&obo`c&TGOx30a2czvSaSQz!raWLT`5J()*gV)zR7L#|AR#aQ6wvqPA%K zjOs2rZ-mKMq!aXP)!_Bb`D0!mvr`pSF%GAYuu z*N&tAaKB)|#CNv7mHmrit>xL}2o4-61Qa1_bC`^0@z6M*a9V+7m$=9!UB5Hlh8F_w zNqlV9_PmnV%aY4Jc_B7oRAxfo%=k3)*iNgeO?|lG(b$9sdnDYt;*?P54qYY;OUbRy z4|XpIo{<&$0!3*gg=SH#cmq=%6XGY6|Umt=RvYzZqjAsR>GCuol2vb%O-hj=V4w7R5SCi`sX_if4o z{#;xxT4{T<*2bpr$!q#8u265$$V}+cD+!80KT_FyVfb!^O%t=lmz%%SCW)N3osKRF zFH~nuJ4X${Q#eF)+^jJ{cR%d{jso~LOhj|)n!zs=)g7Ilnfp5|{WsNh{$)np(S%;P znZ4^ebm*1ab42OY1K=#;Th69Q1|BU$Bep_b98!9+0NH^V(G?qVOHN4#16I6?L-?@E z7%W;bm;jH~YyV!ZA9?h0wYDbVPGX3ra42a%H*DKs@4jTku?{rC=J}rSEM_W{<$LGM zW`D6s-7_1vcjDN;1l_c2az#_BZ_t(v&(e-)!2$5camYwWz=`56`-aJ&*S6CEB0m#V zMuMs~dZ*Uxb1NvH9o-<(AH}JZx3SQrVAPuP`U-9;^7lFy51UDFQ#89HBM`pr6xt%x zW0j#3Q^AB+W+&KJg!44UkN_))-K@Y$Y9!luO=vg!IhAF}vH`L_UlC6Re8bN|_5G+a z)5hC|FN8zTOxAPsl|q*_s+c~|>P;f+jmQA+L-)@X?Mg04g9=;(wQ4?~;@h`@r-CX* z*uLm^GgDLcM1xpBtkpx%TTWnH^~6@o?jCA83LcHQ0t>BSV!s#X(F02kBiY~UX_ko^ zN5lAta|}1nT7;neUVB`HnBIQ7*wL`Q+~J&X?zq>lAtsq~7Wj)zRTm4x;?IjSAu{Vp z=DkO9iL!nQxSx8_@ndR@f*yH!h)qSf2dEu%%;#-iq<*A9zPJgw>F9(oSPV#y# zc#6|L540=*oZ&&lA=<%tk4y3d%#4=BlHwk8i3;FlDgIi+hAhHIw#Am6>s6TITzBMZ z04O4NUgL9)&}fiA+>z@e?t*MvCrm}ebFB$q%bf&==Fai8;nmJ0_9eq^&Pl7Rs%&(w zvO4Ldt0UbbqIV>FuI$}SUn^fI>q*Cx-w0^x@1IrQR&obb9O>+Wh1K};jNg%WxaF&( z)Rt|)iEGRDr|;C3EvldYNrZLW-ePxR1kTT*lq=$v62$HGEC z6s>CQM7L!lUJ$wB-S-yzTKR)HWC)_THKByGxHmy=$rw#aDG^lDPg$a1jD1C7xQn<6 z4q=@H8%Nb`Xj#$!j`n|icy(cy*mb{ZXsB(yXnf8+eO^1S*M~p3e9ALdwUy-d`7a(* z8$ZU&EFfb|dpwG=Q(bsfe#QbYdd*WZQf8mUg>}vM;>RBJBb*osW7E#+C6=Vx3pauk za#Ub19tgzsE0ccy(XojdnQqqwgP9_ILt3bk_o7pBSEU_FCoWE^B`xpBbPeDJG-Vxv zFBlxXh`jW2CV3PqO!4_@Wxht2w+-Y7aV1G4t8zcfqiyJmc*!5+E*BvvI@k;snZoSX z5pv`{gTn=N!?hpeZ&ai@gAR?wM-E{aqSUpKFIZfJD@oW@*akiagH0HE1{?hxz7|{{?gRx~V%ZPA- zT8t~XNFAI7j4_FdQCi;YAvhvKJsZUxx}FVBi!QqM8rvEl1ZNeMKsPMfx@xJTh`f$V zj#}dd!TDacTrr*nbiS@6o1`F22EO*7P9B14HL5E~)iE^1i_}KRvDLxffG_6EK6+fi zupw-)yOPZQ1*17DU7%r?I3y(5ai?;I%OV zpK9ru6e}5+F+|KDsl59qAfiKokf^HCqv;-9lqv}nE&^zkO5%rr6B;CjI8L)7$8#>0 z{XzBsxj1euSc;HX4yFaXjkRTzFDpDgX zK(atH!nxD6ny8z>cBd+Icc!$0pXI6L5-e_g2?HxV2o4N^Tmm(&u)OSbBEfO~PFuED zW-shDRm{b|I^?s%pK@}-4mRXv=(4&os~44s-5t=kN%#wSV`w{}#H(XuatxnE)mAWg z(>tMc{+B~*B6YrU28($s%pv8IZ1_$XIv_Bf;&N@ZFSmu8=xdIr#%@v|YD9YgwB$j; z9G_eT^wML!Mva}w``AAQS^RRC!e7xVTc(#xtQcl*EIl|(knz^5yE?je#Zi?+Jxn`ovLa(KHP9S__vINTQlO9j$d9Ivp1z%e#-h)`N6MyhE{=p%j>dp zNJ{SW`N2o;A8hR?e*K>btXwM+IY^ERT6)MCoEUL>6Zij|j%hD3AHYE&1V! znAqQUihULik!ZI4*_a{}i`^3wJAOf+k{lyP#owA5fBltEPOhlkyr|)C>==1#YTTCZ zx1z1trk5OId5|R4CuBaTX5!cJx;}EI8!`S`6f_N^j~Z?Jj9{N)*QMOPvqZBoR`Pe4VEB zNMacA=7_7ENDilHl8dpBlW^0#m(Y<~GDZhX3k!fuFzwkWFil{>XfSP=FHBpxGO6I# zQ8_k$d;hP=-d}QT>8D9%t>7MQANk_?NB)8wUfPctC_k@>w9EPR2KgE}yy&(g+1SI) zwj(cYSRpVN_|6uVlDUIr{Tjh?+x6*<9`7U9-^s0zoL(cY5l0bM`bItAe-~^eS0fKtX79=Ipf}|3Q&m^=byQk19g^9i1ECcbyRz{^aC|)NC8WZf$2kKKAc8nivZ4cd6HfysoV(G3Xnm=7ScqbR6?y0USxA>wU!Fk-#N z`zSS*I|}62ObKgaZar8e6Z?MM{)IWIU6A(8-TtBTLP?os5Ug*h4+*aZkZm&K?FI~3 zPXM%Z4DG=ggApLDXk^Yp-z;+eySK+faI`bx8y)*L?Vcr-nI5TjM875Vw|`8JubgWR z`6|mkDMT(=*6|axn=}Fl@fTFxbo9*gSW~7$Gk2&4+PJ=aL8LfUngG(@fw)AteRSUM zq0#1-hB3q;J}LJS?po=29-XAgyex5b_vMbq4^{LcSJ@y3n$y@APg$~90_?L`4gd~) zBby@Y&-73R@JOR)1?8davsnDzI^WNxh#OKEb7qAFSTne=Pxq2gi)94+TlJ8CnelFx zY1mg_S@EfF7Z_>*pB?f|fd~1me<%tLKB!bH`kk;b2sZs(8az3Vl-yD%J`Q2kjCH*7z3Xn1@~ zY{{*Kr8Peagoej=S)bNBw0nBe`iJvFZ{xp3`Js)|Q`Vo=KGY7sJF|D_ndwPGZ2tc? zSCWsFQu3SGNDh1z1!7!O=Iu&CLI3{<|NntZt<~*s^=IKqEEuK`og&!yH6D3JcO1xZ z!5H#MSFBzJbVX|kDIT9w{a20hM_z8}l%tg=8Q3*_2)2>*5Y0{&tlHHXNd$sh&QFT3 zIh6rt%MkpSL_e~}$y~8KFyY2=gwsh9=dAe@sMCoeC9ou*sS;iu(Q1a6c*wq21<@Dl>~CyxH~>;{Zawh3hLTPCM|Q`Mbd->^%0HHOW$yj^7dT|i^pLdY%8!g{MW9RV?~G|< z6@)}H{$GII+s?7+LMk`J!m!~`WQ%4UX2~01ZRU8O_X&ZRJu;^44}jI*eXq6EC{9tE zs!b+Ind~Ss$tS)XT5^#^$2jxvLUQV-M_q*ORC7TnA@}YTanXuUg;SbJxR0zoz06^Q z$S8C56w=++Y}MmO)je1v0pJil+{u+zGE{I6AC?DOuR;T7M2AL2;A&aEWnJ{!i_YXX z!j}Z;B=Q7S`zsx#$|%e%IJq8venonu5xCm~l z*aSuJl0~U6nN@QN$_>mc4vDJHNc5^_xzMSKVU^80+=FwQD#~^byY*UuWPMrd21@#} zX>eZa>9G1=ubQ76i6~V*8{pTnrrc)~|?P`$Y0<@k_4|lw0sG>mMhxl2APyNkVWW`6wx{{p>N} z!*KYYF+6Ggqxqq90JdoWY-3_#*K8RXE=~=s$r=-$nHrde|89FK(6o7II3K^;JTg2r zwJhRfl7$OWQZ{j(qvRzSg+QI3itYHi7XI6XZZihwsDQ z{r{M=e``)<#z(S+wpGVApo8fzK| zv~*(~TX1Tg>dgj=>1lbvvvN?BwR=7bQZJY32frhThRo<5ss^+-MAm3(&n5fX5s% zYmSNf?6C~TD+4So8ccO;166`H(n-Y4IdVuX$JQQzSrq6J2FVyXE3CcZkX1{?Hwr}a zVKaJ`*>+T1c?P;VNCn+h)5cM;Oko@FljAamIZlU;1N%bQxdEz8W3EF&G?~;=I5#Wu z(FF9JMSg)QF>YTqlDWMZacW2&$XV$gdl6y5c>>S0eSGcHl3)snM%$(X_1L+=sIQM(>cQYp;m(cQ$lVh zEx3pk*_=kfDPNHw&x7R!xDXzt;>mUoiJao1&eMSoK5~&rI7Q%N`&se{pht*JkdHGX zc2#FCZ8l-x0Q+IY5#r{IE9-++7PPz!WaXIkKq(T zJu7TPi4%}6b$k*Dp#(ws5B!bw@Ga*(=}|34t;HlMZ{jFHgKr=BcEza zJ5=8t2lVw?mgIT>8f`bxXp`^$MKI=(ikJrqOUuzLyF5FmJgaz0Fy_XJnCqS!8r~EW z``E%jB9L19u|=nh4F4%6_Ohhd3l|1P1FW5Vdwj=?_+0>JEsGkihDb6!;nwu{rB^Ml zJ+PuS99(z;X|w}B2#vPy+bJP$yQ5wjg}jm4d7to*{A>TvJM1C(VE4|mtLC(8AwR^t zw@>Ca&ZBQHY<_V8TNuuPB=RMeMD84X>dI#L;WP3&B>pzN^J-!9-`YQP5|CKgyk}ad zlkWjjhp$bwoQd@N z%%b;U2v~jp-J~b`j|2VIKES@2oO8qlG^8Gtn7(H zNN|y*WV39^X1V<*_LNp68PDFCf{MGhhYJk$sKR8w;MeOHOZW|3e3QEJ8g2W38^8JJ z-WX4^E;F-ge?3d`YZPEdH~S~^>gA|Jq*mIC!q2MI;db3x`z6$ZEn7N1r}5ap{0+j9I(Te6PM-1#niA5nFsqY@dY z=+`Q`wvdXx2FDm&mZoc>3~@XT0nz1IdyI4Iis@g%zap#7doP6y%(fdj&jfi!W}Wq? zA8GbiLry9oAz00zg1Tkh`75=C(E#8s%4x$ef0hi75dI#0DOs2tV% zBPmD!T=9m|pn8Z$y3tqs&nLNes~%tXcq8H7boswvLKY3hy+WCR^eM4?mF+J*9KLhn zrF5HX-uQ9dvYS~Y$zdC}VK9UQ7UqDcQEvBkb(2mfr}PI-7RG^*zgKJ~b!y<<+(hK* z${+>Dojd_a!7x2ByO7xRB!k3ganK#zPe&Lo*x5YU;K@b?MQ>rw1e%XQ{n<(h>h1fF z5sB9K4oCtNo0)31qCZ@OqizQgB|wNCRz*2Eht-PBqF5XT1%&1&JnFClz0vbaK2in% z!Ez{7k);<0+|lQH9iVPfg3%B4xt3%f5%N2UxrC_;PRSF?fu37#scqHO_JXTqVxF9g z0}bD*NES!{aBFOF$}Q}C4A#MO_Ynzhq$~g$Bm4MMn#o1-2V$ zFQwvpq5ERk&ysgkFgNx`**<0Luyq-KVcB5O@Vyy7bB2%d<&vqlCO!_q@>Jx*bXLcZ zb@1tXH`6jfir9{56u6F!g(oI(EFm;X-7J1tXAvA=x(=moKJX2GB;>B*obIyy1w(EE z`*=`grb|x{$JnU{RijvI$i*S7!`k2TxAVU=0^GXikB1hXM1_>z+t~OW^|M-ACKYuV zI&~L(BD1P$GaqhP5u32DN5UID;;)~$ytZ3a?P)(x%6&0Ecy?B(U573^hjv*qG-dto z@`K3*!BewBZzpt_kpI(|c7>&PcFoQy$tu1&7_+-P=Im5j={4~JGXX6K>2tn^fYC5HA|GL$Q>dhqKd!+Px; zcIuw5iM5K-K)Z1aM41@^a`D8~T0KT#IP}y$Gc|^oIO8#9({5WoNngVQwhi3fkI|a0 zS(9i@zeXBzTO=~P!H(DfMx<%K6dE`Y6yjaD@dG_`P6eM{E*o|`klBfN%Kj;<_O1!$ zr_v4ZaQ?5vccT=$6ZPrW=rTakPUA=68^9(-qkG)9GjAGI?{WS%RUxH*5|&{fJ9aQS zQWLvj!9CLae{zqYpf5IAec9eTuD)y}{)*n1O_O7TRr0#M)IAo=v~g_pA6`JfEi3oe z>g!)8fm~MZN}}oza2$g(aOV2w@T#EhN&VYZ>JPRhFlFVo|4-g^O-^I-#mU!C$Z0&| z;*`}@@Y=s5_a*gLm#9TJ_Z7bIjii3j{&QkjM16^UZ^JDYwcF;Dm;h|3s`dkHA_nsW z5F#~g{1Cj30n+6$aq>O zHaEy=T+?Y$_?B|sfAVq*>L#VdplfYWovKQS)eZ&9=6UpiqlDN#XgBh{=SBF_+c?X6yI*{%?~u#q_vDReMnCDHiHj9u9oB~Ocp z+9S$h*5e7B(%K=N9R4iOr&Vk+YVc%%>{cVKZ(e0ebRV~zj3(dBkup;mUh4>rlnA0Y8M^_gS^UI9Ib&WM=5HRC+Z1es$pKs1WFBOe2ePJ{`dTBV%tBnOvS zW}lGEY#@)tBOi94B1@(V3(cYKLRBoK2htGg3I_vw5=olZa@5QOn-B8U?IS!_kC!dg z=lDg;Ge?BY)Sl0$;~m}yN^SFf6YBt0Q#|n}*4-aZG0N2N93sc)02YqdiaNG4qxQik z+=XH($(8G8504`Na_dA&nh-eqUkIt$^uhrPRE81Lps4FV$rek&4D*z&!(}SnG@dXP z(W~$WvcaT{7PU|~Pwn4Tv0<16Ki#tm-}O;445QplTmkdL)!uw1*BaUDnp&!OXaA*LCQ! zq@c@$f|T5s^Miv6f){6n&h5~p+wd+3?n`@vbt}s%E(*q6QW0}OVd?uoW2M>E!K~tI zgE1YRyS(V6k>TH=(OP9-O`kDEEn|v~7KVQfRxUAiUSiol2hUwtyKhD9-bD@l(Sj{4 zVN_cD^*>o&+pwsiJCu{%6IMTca;RrQmz{&UEE$xtetCXy#qi+MJwpfMyG%g6+ESpg z3(ltcw2MW3+KAFyWXW1a>aWt!?U}??3KxhVfW^X{x?0+N$ra=z51#-yi zP`~!8{LmNEQ*zVVht7@flH0v~=q&t~+B@{>^rYN{`Jq+#-I?thHx5F7>{GHr(@q}R zobyly$t%~K^ysjo_wKxOVZ-V7W!y6X|G!_7)jS_PvNOko52nV?eyUw)R$P~soDrqV zKFJSvi;2y9E!c1l7U-n_Y&{ihJyg0UT=?#D14tUWrs(ryh0I0lodo*qt#A3k2 zI|`d$W4^8E@XJTKl*G>yHX~k^k*g{XEG~+9JX(J?`1-#9pyR``Q4v%W9pqKZ_FZ=v zA64{8lX!z(lg5M1QJ@!D&w3FCZMw&{LlU{7@xgSi-Ey#QoPkt$5YMv#sBvp;Nv#2> zNztYe9SNXDZ$le=6rhIRaUmIi8Y5*7payX#FYewzjbXB9IU7NU(W84*6h^^mh?jj0 z{ST6%f8x6Vb@EK7vr&5=p3~`x0uBrQ%U=Z|C5C zK&b5}G%!{vzwF?RqF1X3&=?U&`snyHkWcdE4DUeOBx*^NJk&$Jh2Vis*@X{cOw`&e zMp$JUCF@J2U*6yVku!Q*LbG-!Hns3I<%P{_hjRp)o|f6zFOjm>OTy~}bfacf=1MAS zw7r@o>5*WRtZcNpaSGVl^#q#h=H)-2qZDLuXtVdk5Owu8{Ws;phjMePDE+em)Py0t zKxGC91)v5Qm$%TAj5*PN!KZct?;|`+3;~KDf$hyC^k5P4g$hiru3m*udV6QeSdOh8 zD@Tx5$FfcbUo640J|AQ2=@2gQnPn7iVgwQF!sZbu8B^Dzc{82>*pH}GwvB4gz>8=h zI6;bDRqmUkUqeXA?I>IoK{hVP*+jxU({ypLP?9-@N^NNrT$5}nD$9e>8}?rNV^euP zTLGBJR_NCxzKSfR3RHm?kPLqH{p0OYFt>6MODzr&izdLY9ta6}HTal{Y#tmA7?!Ti zOcsQWD~~W2Oj}Rm0#l%wAxJVAv*4b(VT)!c0ihuhG4V6lfW*3Mtb(!t1w|gz7s1IsUMO1A-@ftQ}LkaiJWPh z*yQu5`dv6tku_rg&dg6ThCKftd2i6H9z>a}$!eN;pkMG(P5piYBdo|>(HdMhiqstL z&LR@W10>00vEGvGW7PnE$DwIIgUwfDdce|>;A3w%X{(PkgnUIdT=%-rlVMA>v1Zo& zz%zUJgYzvtne&knYcCL64cD#JojAnSc;XP-s2>8cy*q67w*qYQ*3ME?*o;xtGnj(8 zd`n=Qs5`r36kuPcFqL}Eezx5Wq$OOocVYO(?o36sza*>WCieQTo^|Gx`@u0WNA~$} z(%*H<>op;dUmv@%^kiafP95>}C%91bnzd}`rB6}?*@MhJqoS<-6h@P?(<{m* z#o@;?y4=bujF;1ZkovyWUo(SG;?=_WwtOCK|7}CIzA>qH#lO7!YqLn~%QwX{?xlyM zXZ(!2py7# zfN5F?xmsl|`<)xWMcz*6&*Z zQ99*&%AM1WBxOh^5)go2h}$JTpeg(S<*o3~3Zt~>ll|}`SRTsm08MItKEu%B5Tu36 z@Y$kMYyjAbdUxud_`^`%3?QAZZ)Cj%JAZ%beuxJCM2!_JLyb}(-aQI^pJ6+3yZ+fAD4KNX3u6yo4&U}sg zDm-ftMrCfD2oqpx{K4b_+3U z3J%#C;Nqyu1)gK$!(lxv+}0uL*n$_sBo|Z8fvNN%xRCTS&Nxiq`8#)BRW=V8g1t&9>kXZW!}9{gl=;!Jd92=xcyYB>Xfd4!@~R|D zgY65GUQ}>@eKHcqHtgBMX8-$7l-d8d4wEYOUUada-TyEjyZ^o(cK>gFzjpt{6Gnhw zyDzKxyo@-TRpcRM75U9fNRI5A9d9=wiJP||-ep2^;JZ;%ZQ6Q^ZZaMql%81o%wE^8rD<+1H@Y#T;_AfS4bDF9n?FFMlp6ODli8K?zN`{#eE=Bd2)Hlj3 znG-f)jeyZV16b5iVPwS?M1WRzcysq-V>oBBiBy$pT$X12J$6WSJ+z0T(U;0;r&>~& zy3m@CFtp(A4l0J&qq}M+i$nCZMpsCJoJ%|ymznleGToD0O}!Ospa1FX^PlJI^IsX= z=O5-4B}VEk@KMWpY4ksU9Kc>ZS;W5OQa)K&e7Vvp?V&@0`>AmE)3(!ZlZlBc9-};5fF{`gS3DKhukVuZgQ-xjKp=V1V-v~xi5ZNwDsULbl8ap0nUPU5 zhrfE)rz|@4mxb=hv*-fwAew0z#cuSi>_o~elGX)5urYWi=97r1y zZW&d4)W;cfS@1`+ook5UKHt;6Uc_2A)zzm;NoQm=8*@Lpn$1F;##jjt>-XWVg>tZZ z7n-nTdu1_@&{67v=hcdDEN^}_uiej;O>7?#g?&3u#LKp*$wD7ubf7lFPR;fprhs@+ z#UZjt5ct9Ux7KU07#vbtVX%&>HH!h1Vi57kj?Rc?Dal1doE0@i!^fJH;Mi30l<2{g z=N#IqhbxZW@>7Si9y&hu6B1w%XE;*tbSKc;3Iea5QAQ1+(aYQBv>r}+m=K+}56M)B zvmhoD5tv6rwhv0NWE1$a3b*ia@@D&C@fc%K%@}zV zy$aze5qeDf^o?Xxeb zM9^a4wQ!XfU*sHI_hwNAb&InNGumSaLdtzd%cXz$x5KGBOo|U`--q$7QVb zQ|?qEIRtiXlJTjq&c`Y1U}u5>w~sm18Y67r(~gIhSK!y$)%8@9hwRepq)72IjSzq< z&8MUGw9eDu}W2wuywjebjbr z15!WXQmY{IdgpEpTu{MPZ)N;?+}6N#&%L|n>sFFsMyX?axBLi)r#6Djh0&fH^0&j5 ztG#B28~8IiPK8jo1zY?ht>Cr_6x08KN7qdKp?*~pFMKCagE zRmwUt_bGh77y!GQ0$`FP?Oc@N3&UC8^eBrLu#kb|FxHh(W(LQiih!G{q;H=PA(#wyx*V~vu~T4wSIeA%bP2XEYF)w!Fyya8z-h!EgNCn zXea@hTR>cq%cc#=D{d$mAaM^=)wGvJKt6KEA7OOvvmgq+6MN8E>3Dt1`ORCdcn{wT zVzPpch-SmLOdQ$#>S_+GC*6E@~R}=6u2fgu$|nsnHM> z8GLZF1Uf^rtl)yZ|bh$2C(!L|+lcV*c`J7)dFMk>l=jsW5~asZ69bkJ~4 zA#KOP@b0E#8}%a)>T^E?8OM%JIW}k{Pf9le5UWErHps)HcT4?8L_8`c6_x(hUw66` zf`ObD#Z>md6(`Q6vpNihZBLXjrn2UypY~@LFHc+n^-9T_9tYl!7phMybE2b89HNt^ z{o#PG@NCvyPY6S$NVp4hLGGGFtBM?~)i^?~#sJicEmubs8EjLf5tFaix1wvYTfOPigsMkGNewUfxi7>d5fAc^Spo!P574<6@mk zuSv-+9@MGyBK$Y-%F;V`XB1}yORvH226U>kmA4fyxoja_`kMQGK6%K#%))So{G_z; z$yv=`rpM2&Y8QG07%g{1Y5Jc0a2iV5-dNOd3wmhJ%MXqIWkzuVKJ&`PeM2^sjzML@4qQ;0?Q?@S6=)gt-P9t0c z79&(mnK8S6iS0FLJZcu}SbMFuNP*R-9dl3Gj00qvbh zpN7WjDi%0hp?`U`f`U`X(Xd~m+kd40dRG@Vbb)U2#cc?{Iq&HVdu1*R;e+Zf71Q4< z=0|9yjqJ^fbR8sz?AT{rR%G3**Q)M7jJ*SyaE_g+HEoWk!&oaQ#UVgDk=8csePk_< z`G!-#GKt@NY-^y_gkhE=L1`sD1+_-?4UyCuf2*3oaz{CCbC-x6O=G=A+NdRkx33w0c+XrLSGD#wE)f5 zz-lcO-&?pa5Q2{q#$pI@O1dDtGCeh306Nj&FpW^8)t(G z!{FvbRBeai@(uV6rIeG3JIx6iPLm<*#_po$mh~8mL!>AXP3Iy(0wx{VbE4}d@lcT$ z(H9jwR5TDFxGd8n{EtynNNt0EQy`frnRuhn-Z&MZdS;D90i$0r@k^13vP@ER*fbJU zlX)Sp;QGq$IqR_Jx8q)G1VL8OMKDU{M$OpkMe91Tjd?7~Hyw{sWOk>hOD^@oRy(f~ z9$CARa2ByP9y`9xMD-%Z2dDUfU&&QPPJX={0Hx9XAcHY^AC(Iao%K!X1}#Edb7Arj zZ_YHKK!a|g>{Ns?lytLMr9(7Rm{4=L<*S<7?9!wYXAk-7!(Hb2*h7x;vxhv} zY!BI3d;uAfl-67v2+e`($ba{UTYCBOh9*F=J%D8A221lU%nH??S=z87kZ^tFBMtip z#$7*hdBf)e<8~D;Z)hACzw4nn4MF_wp+_2a42-ikV4HmAP`dP_ll#m)I7`;D<*o)y z`ywv|z^b%&a(44w3j%p>1setj5@s*y8QKn9)-hPR?Bo1!7XZmCS8e~-;-Zh45W9In z{2ik2+UHA(P_YImvw1;$-j3%J!|N8e6qc7Q`~1GCL>a9dl+`kUYuK7Yo#ONUmP|Cw zzNO=rebg@$W@>JfaQ1KRA3DOX_oa94NNv}qemS-^WmHDgdbTg`YMn_y!n+CE*n6L| zd_#Cyh+x_FK631Hi`{bKFQmIjpm*@^YFz?tp|J#6a0-2#MZ1ZJ>_B#O&f zP6sb*T_;z4KY7;s!ZcQV0+pEAH@1nC{AVNFNBUxXylf&#vKiebvPRoPW^yK-mm|_b zGWACJ-F|Q%*||%PNJN)x!;&ngqnxrDp-TY65 zG|b1%AEer$2RS{Jc#c{U9~0;;Iet4G!bRSmlaLk79+EEda%p_imigg7@ZXN!icWGa zcSuxf?f2`Ejbe)9%8fl*uq*Ei!l?%M)F<&sy3XXf7iEzq>qgQqqtryldk{$-e4hhv zLp;cl^Mp&+=6+VJ(R$)#Qp6${MGMZ?C_q0S6%3brnasfv(+cfUE^ng1}Z@ zBsH7SNeZp)uVB39gc0G3CUWId(u?%9mQ6XkvFRly(@GwHv@Gw zT)(Iq$bY^dpU|0AtvdcRDF}@4X~G9=n&S5 z>Tn2#dlY)>X^v?v(XyWKs&SV0^aXJ8F*5v5Ww_$es2QUmm?#?s=H!nhKywMlA^;k! zbB^H_dn=-JX3HxY5;?1XT2}h5wINAZhRFPwYwYx(A#|JjQ>$p$$r$TU5A1ZkP+;4` zZt3tR@h`etS_PAa*@8pVlTrx0^n&2VX<+Cq6g{8n6C;g|2O2O~)0PU9)uWQd zgKVkWM0P*sk{Ue@2AG*iKYyMZZ zC)x3>Wq}mXD8L-xt$EbNs*1`@1lE#MHV;3Hb6gjZNZDGUQS7LweQU67LSUkM@SGYX z(bo8=)j+o3FmOVenTiAolK8~u@GVZGIz?AnM=U^cGXrkIQ)(UxG3$Vv%;k`naly~1 zl-bqFP4niC!C_l`C-{=T*XJXuK5~p6WtaAmGF=iC9_NCv;Tz<`q=k^1Om3vw4r8`h zug%DZ9k%2p{M=Z92`w(6K1XJ4Z(v>c zZFsj~3#XuO74VgpLFC4ZY~hPgj#z zP}eVjABUlZ+hT8)jN)JF^7af&QDnxF0UfXObcn^MR8ZYI6;0Q6Y?!N-ycrsXR)|B$ zm5(?ay;6;IU+JGF7v%2ixe&uxJfSc{Y>E3Vm&oGYE6@V7zpqVXVsx9x9Z2ee#V(8F z(l#AZTpDpo(fh7Jws;&!?em|-cHULlxp6u#uS*xUd(q9KkrvZW{s{Sq4~;g7gRJl5 ztHr5qW*nUW!-|DPlG>Z<8NUOy5ZWT%oLiJm5GJyMN}~X1=_0xc#Z8lmi=yC5Bo*~0 z=GxRsW^g7OebsKSD9gKqcioo89x1H{lDMv-^{a(s6L|s@BK7H?qRU%^G=DNO)lHMe z!Uimo3bG&}p8uomkN9_5D`}b?er7dW;Ww{coz?Qw@f=k5Tl;W$CAVGU=an4wnqJA2 zEgGHpNr!8E(nCL@7L6VifnbM`o>S! zj1$5q9nT^Vh})!!apz) zSMV*~iK9ktMeGmUh*qh-zo2hKrd!TSV7tXW%&?@ZBK4B`5fIW$11P~EO#w-!DRKtt zG?ux$B7s@~nmI-mK*gGw4)_c#YfM;Jl!ijgR$%4?&7nRZ#WcBK;UoK+fqFGDtd7(S zhFQ@EYPNazY4`%_3f*D6sL63x_7@u2iho9=hjGUEK#&Z+VpO}FfyR( z7D-Kr4~i@2ElRec&}l4dDA;z(PYX#~NF>dv#~2Zt4>+D_$u^zX$@;!9n)EZ(I)RPY z=&98Z?uQYK%z=uF(vt3{zfT{ImekG|M2oghy5 zXA-H^rL-b?)!OG{nU-97sba>bC|WXY1he;fKDg~8;5M{`^M!5czOb#$2;1TcoE9Vf z%7h4YxN5Gp{x>SwJ%bKGxj2bd#Rup`*81nwFW2hxSLcv)D>r>a-O=fpxxb?m>N@{2 z9rh@y@)9F_-d{FAr0@puFRTf`uF*Ce$(LGTFTjy+)tbWLPWCoH zt>hSBVEv%bIkHq;PeV(x>BVIj+F2>9P92IN+}-Z!=|@pEo<(Gav=q!RXgEZtJo$Hy z)I6FrV?B6PE)LA#Xy{d-$xj@r7Uh|CtRjD}&wm+;vO6N8ZhMk~fG|r(IFhx)U@KCB zqKqyQ8KYv3N&ied(~H&bE%>Dejtf2dp(pS`!JxeW z=nR9R4+#Sju`VbDSR`B%rBc>B6)S@ES{snK{?rcFaE+m66_^l)A+#3_BOI_+X=jMK z!Riyuhn|{@z(ZJ!njSLAGwIi0g?-jr$g8!Y+5X5E7Vy)HNek}Goa)jU7Idyy=FhIV z?v1@iGoxUzTE~NeebSJ5oQC}E{Da0G%@};gtk)CJn({c*gB~yEiX-z7`fg!Aoc7@# z$!E^+;LueAt3D$lDas;>p`}yawh)q&OPB3x9G9^sp(7=6C9}_lHOD$1r*I2N*L&Q+ zAC*tz?0{B_8*~IX^t}j0gsKyBU5}=Fbmh()C9?n%$n9;hmnD~d@XgMM9TuL%ZzEPsx2DKN$PY z**W)R6<-sKIXI&9ny%S7v$L!3%_{y$FlJkM%*OV0UoEZkYr+=q(}ZnpmbnRA*+txr z4OY}ixl8gxlTu3?{uoGT-L)`K@?UYK>rk+EBMR1D=n~DoiOgQvsQIVcg$w-G@$lYBc6PrLq^5NI| zMt%)d*5Mn8*PPSyxh`X zbHq#IKP#W4T`idliRlZk%FmeR1=a4hU8Abe2$}6IEWSm(*-8;*--=k4fUv52Fa+>8 zq%ztb*17@+`_y0OsMMDuKC{Pn#1bQj^4$GOuO47;LS+bngiWMX1nts(RD4A{^Y|TP z;iWK~3`=!DAu*k=!#qTgh)|DGR@_jmM(?A2APG)K?l^Y&4Yu^_IB4*QVlhgfv)F=& zz=za-Ss7n(*Xrndc@vbhr6B^D7PXr*Iu#{bk@H$we8{n_gIq&t?@>|usG+Ka zt`Ao^jkmbzA#gO+ikeuO34M}Czac5c7Bg1lyJ(IZ6$2O* z<*?~Mv|=vHLu{xk=*_Apx|w^@JVi{ZZK!leVr`Dt3`GY>_ZEmT=Cg{r0;R2Cl=kTf zLutbfpE#7(@q2E(Hn}@o9@{7N#c*QUm$4PV~S8d>NE=-zV6nMNS_VJ=q z7@y4(U@hVCq7FZLYR@`gkLr1&a?05na?aMk1FX)go-c`>@*~pwUQS$ma3>zCdu=uv zZPN~=FjH&)JDUFR@W=V5#jg8R%O4*(T0ddYZ@MPj)?rH7$shdsq9sF?c3-Ufsi9*G z6|$A>zE`%R^8$qEqD=*6r0a2h1nV_>iKq~mu8PbDRw@1AxQ

    k&`tHK#OuQlt%in z8>z?Jc!RvwPPDwqF507m`Kg5Aw&Z8E{7AsL!}xJ1{Wuf5H;pY&N+g5EsO33uTdc_B zUDEkc6ovU&`sg0_6}RQI4H_w7YbUiRoSrO@EJEa?Yu0@&1Y%JfNA5kN76w}B!qNX8 zOC~uvD=2dE1qZJrWoOf>&aJmUOknUTz3iUNSmS!}8i8=hqyhW2TFuLMq-4^z={H?w zP)ju({zp7Qv^9+PttVmq}JQ-4@YUthHhMKf6mwg&5EPGt(&{P z4S`Jkpp!A2GQRKtb&|~TUae6kjT<4Yv<*us;sPmCisXJ|k`4k+B2s{&KX(?%>5xT5 z&2CnSA2*JXP`!m6-7_Y`%mVHsjVH}$yqPohN}+cO^#Sh&=E$8dccd!uKH|@HoF*?P zUpHP3Pv zvs%qQv_`^C4Q=xk*%qg@XHDy965;9c9@3~Uj_ z+mTVOPVRs$O4n?WHnEpRB2qKy{g|UdbV2k zSI9aKH4V<=zS@61aYb(bJaI*C17(rhMdcaAIl)p%!p)vmSi0=2(uNm+z-}!pJ$^5e z^!EVcF35;)O^B;2|5*Ud-x68_&viL)c=G8(OY=@D4U@@#9!&lx&CDoH36_>@U(~Qn z;JXrdMb-;5l9Jbg4d((_eiMxV zC1s~y6ITt?Dv}s{SwsrKB^LHuT?HrmL$U%;xIEeEHIB;QE4k7LzIbT_BCKhL*jhr) z=5ZO^P_CtaQhdp&{CQdioCH&~-e4&;P;eMKp3BS$7g zkRn#FmQs7{AlxkpsJVZyUn9<~q11I0b;(q_@tCze+jnd_!~N3(&ZSnWOm1Q>aZQqSBI`tc;#NAiO}ba4nQ0<>Ad=Zm;fKFT7q#Di*^xIA38*Q6yipY!WSinyHY7`EVha5=$}Y{hFKlouDUZ4y zK-){UDUrjjZv*b0kZ8LWX{Fj`M}IhtNLjAYMHbAMxrh>9|GBL9j*Vd2Z02w0v|9YL zc|pESoa2@D;+1CQpwIH`a%)kZAzMQ76Z41r=`ztaDS)PThqgtF1MvN;_AYPJ;%Tpz zHRHM;@@@3q=}09f*9Z?!ZND8}mnRgHZPml;YWA^s@{Z9P&(Zw>hOpXl|Ba zenryDq9}^Vjdu}`7ELdC+jF3%m^ySaa25mh)IOr~%n*%;Dy-VYpq#js0j;2tB$E(~ z69)LGoAWHNWqT*|ookI#Eept$>7?E~*Ws4oBG{L@xDgWA#vu}1^ci(J;hj-ni66JSl1<%hZUfFa**T*Sls5#D>E2@)F z_rSTiTQy8x)kEEQr}o*~m}Bx@cdzhu833<_joO?smmX)})uyQp(vi+pL(KJTF8eYW z+DSU8j5}r%Q$==>x0LhtoSv51xVSu@nW?`I7jk?=K{Bj1W>hhARX$tBPlog$JBNv| zVeA!$%ymcr+U zBH3kx0#+a(eEVhOY6u=O(eZ7(f1w3VY4Ypq?A7$0noo(R6Jpe((daT-fhlf|>M(%T z*YyNl?D(oHFL1YgCa|cgGK_~lB&w<~fo0n%iv-g#FGw&y_YqxHrFjS&x>Zb? zWoyVc)4F-|VMuhQikXa=6HFduAY=ew!9rds?4+q!yBb3J;OH_)}R<<%R=j>zrpK?JXp+Nru+$KUL_E z1`XFZsXupeH07<3bOFb^+tZ9w45P11^3zw2udj2uW}o#mw%q`%ql@LKC_dvzoW6wD= zS##*o`3@h|9|8*%iQz-g!vVj@!%J)0PVaVU(t>rTW#`P#uCB-`o*ImqRS`3zu=E-n znwMQY_sqcPzGI5wM}})-V(&q`|2$xs$upMvZC!x)U1tK-f%iNwf_|?tv~DG z25Ib{jvtS|?OOTmf%w*rg{3ty;LU`wf8NQzo!k2D02;Wo`op!kPdq?`klEV5t!i9) zL;cE6TVVNBKa+W=@EQe>AI=$1e` z0wHHEI6!!BCqcHN(ylj^p24`K+c188&P}D~y>r`Ak|Bo9JU^7m(4u2z=e~j)>cy`y zH}`^iqRT2C$5C;Hy(O;qQEC=P;j8wiSDHp9qL9GZ1xiU+mqTq@C*pU2b~T4tch&> zhX}jK=IVv{f=&3yYL>m5n`Q6!YnEN&-zv@q$}1duhk=0%$m? zt@Vo_`G1Y&?foL|iA~bd@d|G(?s^#nRFxwRKtOePcR}%g&@ak*f1E$S4bwt$N*nlM(Wxu`)(SUE=1XIOuM{rFT4_4rY6f5Wti&+p2yvIPd$3UN{0Y1|@P-<-iluJaODz9NT0~!pT4zh=$i?{aXKW(L zvHrEb)_YJxs?FW(L4wd2eX(iZjr_eHZoChWwJV5l$7V9fc9De{3lwhU_}BrnM8s=R zy#57P;P+fDp0DaG$}21(8B#{gTyAAO75YKpr-^lq)LG6jDfOW)pr~6XzeriK0`yBa ze|Lx|EkApiLU6G~J@7=H?F{eaEVsWE^-0G&*?DRC5Q$Lr_>^RHkw}UGUp`+*zwCvX z*1F;dex*_~GS{Q;6@(lH>Sz=%6=GcgYO|_*JW(3~YFn+8n^Xj9)5}8E4Y;3?B1WLL zNk(vDM{QIG8TqF;#N$h-4cAE0-%g7QEQ@~S4q`2h?#EtJRmQ@>ICzBMqf++Dup}r# zLeos1!7mswNMsGOOm?-z?gHk`oZiwCYwB(rqSiCevF=uL7XSz5lk~Da|7C@j4M_|h zd`B^*;W?zROcGvIPi!@(zA%;P*-;4ukKu<8&>kT(ao-jF8d;*XJ_~}d29b}P9xMlP zqf;Jz-KJf(FPIWC&lC^47$?V|3Or4RRz~)@bDxh>>f25w1+?Q66bO8L+;n?nUduhr zwQ?J`W0Nt%McvqpY6xrn@VDkIY5ICkNpNgn@?F0O#ynCH^FU!~`DxkJ<=HvqS;bR= zF*jDkT=(42@TQp9#})<>0p8k=Ejndn_)jshmnFqsxG*pp`0eD|<2z==?*d+HS=4Yf ztV_}pZcUG0de!pU11oC7!K7W|x}_``+-1Vxl-y_YgKrKGzSc8zIebjeGW(hQ;2(zv zU%8-1aX~QVy7HKDBT7&2lASX%yLv`e@sME5xg$!qkkQCv0kp{O*Lw^q&Zuklt)gpo zr?Nvg^G~W9+eb-(?Pre(Q?KlQBn777M{*%~FDa0SAMbuDP`PDj_{TA^qw(KmF|moD z+5RgvkO+S5=lI>`q0Qt%vJ5UH*W@P+k@ao8^IxfEm;d%SM$Opvpc&f(odTmX$Ap_M zPHCCZDbUnoOt=z1Ue_s5i68q}1#XI^^3p$|SvJ}Hw_eeH(|#ZtZ&s~U(R24QO5439 z@C5_6>XK18_u#OqMYbC3QNg!mJ7$78M-t zvJ{ja%-Y@yv_FxPdDG`(WoT*(au$?B1PLNxT0qcGHqHK$?Q$&(5RIU$LpX(ogdSsAI!pn?_cVDUA~~YQ0S9k*3q3)I)p9?+S=&sg(Vbdjti--c-Iw+L0t} zyCj|jn=2Sydv(pGsw!UztK8rW%sA)*(LWST&?_+WY)UTO?QYi|K zE;Lk=at6?9($c`SmFOs9>PH^pFOUX-k6JhnIY5!i9_dQj)zYI3^Qp-wN!09Y_#b3@9=Wy!F zK{A^up7a96BR5~jV5&kNTpfp9gPcOVO#!i^rIXrl3zk})B<{?~x?h*(}I z&hbEDjYRAj5wKYNc?skGe^?GQT%rXMLyLbh=Y^<1vP@C#0YFyN)D3RVPSvzoqi5yU zsZ%#Vjs3iy3{9))XW6{oVxtj~&kBWBt!#_VieYsV4}xJ}acC~2n4CATfP-zwI0O^U zAI^y@-$*1Dg!2cJ`NoV21{fYB#sw17Y!UCqtE*0+k3Sji+A<4yzOvz8aau0BemIgNg<^-QmY|K zEecXQbzfndsUJL}5iAtu%ol5Xu1 z$56Qh&DbW1W^5&B#+J^_*tS3jLcARjm{`Xpz_Y&wiAOEuj30%VO}2g7-zBD^_26kL zeBZ7mkeT8TD50jj=q`~sg(@yKmETKvEIIwO6f7LHcHGl#q-e?(vt#`ue}V3ETH22p z={>KJ-sSvygLqA?Tc7OR83{LDd48m*)N(phP&^3VcB-=J9gG*eO}Cd8kOec9QRG9o z%&R_5k?JzhiM5)tJ-Z8w^R6u0!pvLouuhx8u1Ym}Gk>#4y(w8FgMIZ>JGn>li|&L@ zLq~>Oa%-bqJygn%QeEo{4A_5C2h&zLw(>ONCcKnDsas1{C?R$^+HLOLh4Q97y<1O0Hzqzix`5J zyO~ecv9^|3dLa(SBkdXQa{eYF%B39(`fhIGj#q$4AvP7l@NcwoR zvJxT*Pgw$1Xs}N;ZNGb3=q)@?q_al%a=8sREZz2!_i;mhrXDe zlAG2(bZ&f?-0tl|XW_rp-l12gC*>~853R!Q&TQYfaZqWX=RA}Vd}w%R z*-4Mq9l3Ysr3)Jp@5{Jn0{-7#l7&jMxTOWl8|r6dHSbA{+XchObo{tZct=ix#Yh4W z<;r(&Iehr7jLJ`mE-E#f{MVrGMDyXYv+>k6U(}r)@>^DQif>!plM1j}HsZ&Z_{W`t zUl{V=HSdr88jmi1v&5@#ZSONJvx*l(DU42#h-ufAhE$9RpAxhMhWmQLRYP-IQA60L zj5%xCXEr95;zzVOV|)z3nG>j4ueA}00W%VYZtnSLN;B;d@(2IvLBv&uf=cY3_H?RnB9P1W(jY=fuo$ayRUQK!kQs=nm5m zM~fKQ5G=}NSg>CdN$H))9pd7Y`jQ_R2@1q>^F?g@1brH%!_!-qAx36Z1&bBHvycjmX|wflM3=HPqwV~>{APw7f~=j_KHEn9Xo|43^_ z73eXqF0PUc0zDS3#Hx@$k!T^|Ad8$a1>Q#(k$CeJR_TT_Dw5?>7(W(fau$Yz0K*K_ zHIa6Bro&tsO>k^3pb`{LepWc5sxEX+*}{MnjR0d&ohT^3dY2$FI$+UT-k zEUABV4Ck70k^s`M&Znhw#$)fF1~6$3pYQz{_KljQdVAlZ<)I6QX z@JC)mGR=D0?$~VdqZc=aFZx3%MldAbSup~j=77djNbcOUqM3}uZF=)6N)haMp8Ye$ z5Lwy;J!Eu{Jh`D6*f{hWIl~?uMkzv zuDD77y3F5mvKBO*ta9?q^-3#z1qgtsb z{{1h=OGUx}&1&rEV^ftGf`>&F|9WNa)p$FZa{zp@0T+dWS6AVbel%@tzjWS>#y<2; zku^COKiRbzk0(x9%n`qV43dZbKeHIwU;g5(Esy>qt}6Qvzxmy?oBvsIW1rX0%FTEt z^|dFe&s=!rFNll5$gNpo2g9lXg$8k4a!+&$AY|^rt}K^YT)E(q(vvAuwN(@}M$;C8JLB~lmxR$q;DNNt~qk^;XNQNO0 zK)9tFSy*`*`!G|#Hs7MFNj^veJeO=O=frQ;iYiP>>_cMly|Shdr=XzFro?r!`gA^M zEJB@;lV8ky+*MeR(1zz)u3_k_ZmPE{FK^ybk`4lqgDK}k(s?aPwhNWepg&y8Z9P#P&*#J#A4S@w_obiDQd8jSRmY6T3Mn z_V;7vG|Y#cNRRlm3Cn9cRn>NUxZ!qGtYswJni0Qr{PNnEy(!)DQ`WD_4}RSp zye>P3q~t!IAN*u^uyI_6V|%`loqylQI>pZG6nn>lz;+a`eNq@MM!niw57%y9)bI&B z@>B6Ie(4p2ldJFJ8t-0A4xM%8Rzpqj0qeQEFH?B=@`1oGYrHVj7FfF(Uc+l8C|vXAq_ zUGSN&T($jQi;F&HIAiu7$wOu#yn%X*E&KdFE}>cp_mK%BNQ>7T>J*=+to-MVYY9uw z`XhY3&6V-*2{6~Tr067oh+!MY!i^&oI78S8RlxTmyJ&%j!V&T+IS&%u6w+IYBlVx zt_1+VUq|-*bL}D;Vk3l58k+(xa-O)I_g7YD5J1-D#3gAJ4fBjNaqGJoqSC5`I**U4_ZiGOPZnhxsCf!A*)!j5N5e_9+WL^)x7IZaa?+ifrR2o9ZG~lpGL)5x0!jynRog-#mbe_#zc|WJ)CtC59~oqcVQ#ED#n zxOVNNL*ko9ue1WQ-A-@hYkR&Eo@h=}7`K{4k`l`bg2h5Mh2%$KZ`~0Iy<{-V8c#<# z1MwM30`AzgMhZBsGUA8?9aGR=y2*V-$JUYLSn4Jl-(_6aCUuk2Mv{E~Wly%YU2yHc zH8xn0+`r_)%Ys|>q?hyw1XtpJy#v9DJy|8^cMW#N@6Nj{IA#w(TJM7Jt$E39#Ll zUIuOm0n(pjT=vBt^DYdxq;Ab}*B6HG0n1Ep{&jy+T^fm#Dcw)(G?R{sMMS?gJPYGrx7 zu=+1et1K_-sMSAonf%o_YS6$v%1h?v%t7}WS0wlO1VMz6D+x!+HyT`7>KL3tm^}QQ z-jyX%D``4Os*c*54_-EM3-($stz^-XamGjNR3C8}W0TKOBS;SY2TOc$Yx^1L!&zeC zLj|#6B#BGD_4Ffot#kK+5|>IhviR5$-*)0Y&&Rsi@ z;{b!p+z^?o#GrwtEL#)G+;Z*Pys3WlC6Rqdarbw<)4g?Q-DJ?m_Fc<}@XAOfNhW}O zTX$w8x5*aTWE#13R63=tTMVmzTQ*$5Rx6YPK`h<2)m9$~C2TiwsLCJ}sdHq&j}7f& zDKV}EiJYdaed}sU%vZe>>_KRI+e-_>((AwWor~)tw2~3|Z)0eTkOEpsRkWdr(g;k~ z;icr-jL6@c$FOV^@u|0SJ}GCS{*q> zJ$3A6t|!zfPn1x@5p2lD5m8UaoaRCY?o4u$m59+vL;LsOuarezTJlnKb*G!$Cyaxf zl9A{_z0_kh;G2j9lDYKeQ_C8e9HL~I+#O3+<0e)=2EW4R!fr71T26OK@-C^9|*4H#`NUU(-|hV zJb)iF=tpHU!Ucd9*`+#0NnI{~7Ka}8k2AU_9IjfKc*}Kv&;61h7caL%4M;R4U!WwG zK)JJQk9IyvLMj(*mVfB7k%JPxH~{BB6{?y7E$GgwKsWI?C1BM}uFm4U+js5J{4+iV$o_0b^*ulZK^OBw(P7Ak`o7B@lqQ zS~-k!@~W1ETzTzKDFG|H3vCB4;IG%(2&{H6k@a6?hGC>rt(A(lu=VNb1cm0TU7;O7 zGR8$)`C^S1XqR@57uu*kzN8x1);2UQ{IrSA%OrE?YLCioOgG&3IuQ$+b6RIh7f2iJ5C2D~~@7 zQltRd8Twi$mA-fq@tU7(@mWu%2xb$%SoGGOP;5GHWn9R1WbMZO!Ni~H6dAmy zyMt5Y@0(NR`fo~znONnoI%9nCq4C8B3&IP1F(*{{n*hqb9$);`xY$&9Mt-n-z;r0FLbGK7mJihqkiI$gUyDIZu)HG4M#I#*)G085uM@{r{2? zyX2B(4dgr`8?jNR?E2G!4a@!UQ>qp<92gQiW$dzs&xgeBE?CykI3#ZOBeNR<_}wFm z8nzFK2$d;)VxbH#D|6UAL#`97z;{qX8i zFVB(ovX6dArb1CLNu7GW@Z0wGOV5W0hTeaBNg5_N_mo55=PgItpS{+&lX5JvRl=3e z9n8EF!d>Raczw4cI4aGsFmO_d@*ERu^x=%hU@yrny0uC~yyuWj62_xf+X~bac57@# zQ#OD)_Vz689?RyPweXIdF1#aa;T<_hdPkOV!CE9OXC4F85nk4Q6@gw66dz0pR(^Gs z0n?U@GGflr@)RQLF-g2}pN#lGP>FQROG-}>G91mWO8>{yv;Z=&n>UU03Ds()Y3<+0 zwj(88R9b=uSc99sRuq|)sjwPkUY_-Axfia=t9ytzvaI!gF!0qn*&85h{hDktKpD6= zT>5lu)j(qH!sR_H_dF2OEw#E^frV+-7)TFSAvxc>@8bzD} z5hZ)kVGR-8sI2iG1qDEyD^);ir72a|fw)mbg=&b+QRk1LzHErc(>e0a-Fk7IG$Lt( z+ke@USE}&>=|yFa1{Gl{Bh@K@ydtHeQZa%V{w1NP0>Z;%*qBYonK1iAIBi5@CU>AB z5DurohwBU*i6D`1@^c_tg#&NV^H_!xxtp`7=6uHs8gsbqbCSqq{wVr zBPoe$Vd%>SC_oXZ+ad|D)m5g-GYwk)lFkGCfB+A+A1 zc6jrsS`wQb-nvE0Tkm3tZB+6`0+{_+RUUZcFy`TtFsccB2uooQQu)7b(9Y2V>7fagAGKjQTlmK!*%YlBnL% zESNltiNDfTbIL@J3gWe#5TWS|r|{au)q=i?Ud~ON%w6epZM8|>$q62CZOD!p%#2-9 zCvs}w+6*zL)9{09i?~TOr^PI24n6clk(I;X|B$+?yx-g3S#z0SnYJrM49`wI*jDq_ zTRH*=<{=$HnVDnS%=cm$CmV)YC#<>Hss#q8$W9*(%~h@~+F*-?mCYm_vBP9ZIP&s-SQDQxcdc(<`RGTm4a0-}Jz+>*pD)iJ{Dbtk4;F21O1+ro;-|F?}vWMl0? zT}Om}>5D0wUJ%SZ-ygarE~WLwjJUQAH)$)GJZ3?|hqJO;&PtC>*b$q&3nrEEyfeKdIS^dBuOPff7)e$@9C>Ev;1-A@$D)B-23pCT`}8N$ z=)*sFi~j=AQ|-=i*>^E>cKw#}rJpq%2Lt=!x@~vnaV6UyP|5Z|5)oIWHKXbZ?`Ej- z`pU)>@K1~~#-%Y9X*oOeeFj~&Kdsw|={R-skhd#H@fYA*4}8-IUisCrvI8% zjXBIexpveB_b0#xM`MQ4QRyMe7lA*-R!%IG0V^bs6UO0T+&qL(;_O40Wj`O$72h=q-C^d z#WpFBcP;>Bcxzr(>(3W(WAh{UaU}hC*i5QxgpORRf@W}O@490+kwf!s-!R8Wx!yr$ zF1)c-ic`4c*_kOk$kP;dJJpDvw$Le&Fn23$JDctQjkJ2sqsFJ*8i!BIT>rQ@CUWWq zqi%6+Um?|=7rvdj&_WVHo`em0OSgiOX!HMZhWS4(S?S^ZPo5*5FmCCRiP?=uha{y= zqTFy}+|b0WN8j1bB3k6kab+UaF56iLxl;4?>bk9+Xn5->{J?9eCDIdlXsdgi!X%ufEbe|OXS>}3 zF(@J_$El?uq!k%%;+3A1?3E=r5P^ol1%=PA5am{3vL#tJiZ`jW)XYYhMpE5H}X`V4Z2~@Pve&5!AAor$fLwB~a#~Gxx zY+aMp`YTH#Z3ruc;wv)NSKR#v~MxJ)9$9FC+tO>S-(l#yVpjqXtgE<;Z_3 zkEL+TS9xUV74*XD2bP-Ew6FeDm&Q^7EfQyJHO&ixTyCuL$pC7VD?79&j z@1bnu3;_H)EFE_`ExdAkg{g>`fSM(NP7#Y0(d{CWDj|)PgdQZa4X0|kP~OpapW-)_ zKFFH4irCS_oJ({B$;>~wE(ic>;$KJ3esxUQXVSE49rN%3)G!~gAz*<0mb zJ79cq>-gd$1>xVL?{-4W+`=>NFZGA7`= zqr!LVw2|RTU(BmrW1d5=Y_M>jjV}iHx!)HvalXHbdS{;*2fL9emkGO(>V}V`14-<= z(mL|kBDBLl3ZcDxaDQ9U!#x>Oy>oWxJ;qdL_UbmS7oXa7c4NO`pB~>v2aVH)rOzcv zo+CyBIsmd`!ZdKg5OYjZY(wUWVW z6w5W_BbZonK8z?aeqRliEHIfNRNVoikDQi?1L$1d_%Qk8Q@+d&0?lJ=+{SIWK8<&P zO4GKIHIeWX;W|L^Cfb1$Nbm_L+VUt@OGm6N%`%H(&`1^!;_*NMEuuh(_3-sSMf&>l z&|;U7=udKX>KHu)*!4y9Fe9iO(ZP)<_JqQW%7W=McpP>1RO6Q^PQkj{-7yjXpMwTIOgrzkE!B3CZEqyW22I9!?kt<^TSsJxHjj#J z4MMM5c~J&NCY_WmP7({Sz|fI`8*@d)3f?hu|6e=pss)11f$1aoD*YxTf=xiQH0@S| zNLN-^;F6_obPu+m#fCm4GM+Pz%kGf;l!j3wwn;H+Y$$T`H3Q}m9%6tDeEr>ZJ+tf64p3r5fY3-M-ju`9px{Ak$= zKQb(o!L4S&3pZiONo&yCrm79C3KH;iT&56+{Cbg9g_id1oF)%3X1pZ zXmb!X*We^rhz#o*JRkoXavAmH z{{O{O z(>_u7&GZ#%hDH7}f_yV^mm=fn6r4rMntldxmm5+;K5OP#^G(f~lxQNtRtRtM=nW9H zO#kJr)`8&~pj%1N^N3AT9Zs$K|sNvr=<)lxg> zVQKmwn7n)JX8^qkeXwxP3twSi!?oggia5q#9jLP*?`Js5S=Sc$P=gF!Zx-0rDPllVe*j>8@bi z&CUG@h)yp6L)f4Gfe?0(Ub1L{RChV;a#Kk!+2sLOgFQAHDJv?2CuKP`WLw&cF=|^F zs>plM00Xc=RkDH6ZN71WAsp4vS~x|rv!WNEzKjlD&3`~Q8hwLtqgYT=YMz>88=0ik zfA}h^#ca6Q8o!a4^?|pZF_#T{s+9Aj(_f-Qc+0bE*yFEd9k{h?nOkcdv~FQ|Ya?0Wy&?XjJhg?E96boF>Zn^hOq+K9}Yh(T7(fMQL@Bk zozQxl(}+tDt)yPoakGVYART{9<<%|MSwJ*9ia_b5nPuwK3gpF(h&n9&m|-l?{oF*y z^FUE)#)nxe9Y{23w$Ts#wZ228SI#Lh0gvx-g2!^1^gtWfr#~J^2E+CQM}bJ!scDXB z%V1mQt0<-|oIV_I9LcjezX+5@LA5qxBNA=^g9>AtV`W2olC-kiB#Y$PRs?Ql49H2_JcSq&u5ES3Oks!OwD;}lzpXhy`u zaTW`r372jbtt+f6HjN#iq<5`guHLZ3*zHi$_Vss%nkN;fjJWyO6}8>jbjFEs@1(}B zNsUXreA&jm%QybBx}gtB*HYtlqm=FM%Qt@fL}Jd0yub^i0)OZo>Vob7!%}t)P0m@G z7kIRHXb+XLT|!Xn@oxQVX5hc9lAlv4oA30_b>W*}qTe+JdE#K_Zv9(iJMDV-`ENUE zM0g@V-E9TI+VlLOQE@5jQ~QPX%t&0nFfX(N|EtamZJLq1{`AhFPWatv{X);qNKE}8 zDXZnY^tjS45Imk6^yAjGzAL&M`mwf4IrIWIf}rzFkfM#4RLPx5_cTSuqn+6bn9-xWVyRW!Zqxkk{gUr@E(~rEhxE#w*~h8@3?IS*K27r zwidfy8;~=e$l9~~PU$%n&tZ>XV+~DukMGIyQ*1?Q50m?l72BbAd1NwI5q2^fz$FO9 zW$ys>OHMNyGJaGM7IFBdwo;ifHjZJqD+;?bo{?m&X<^{-bu{S0y3f<@YGi#)0Q>NIZVus z<=t;Gmj`}acszNe3bTwk)|uQ-+$oKh@nw`Rp^0^^!$6mO&Xdo<7Xzr`Y6~->c?swg z3J(Qnj)W`f7&j4Is-}33<7yH`dFFbMQO7&38QP>uQR>6Bmh{RxrYZ1MMMhn%1oG`r z$J&(v;uS?OaZl@e4f8zx>{PM88Uf>NHoO{yC81beWPBVs^D<^QXe|{ILB*CP7#5$& zA1I+QcaqeG_lz;>%|A|?5n#!=*@(+F;*yG1gS-7_V3*mg);Xc`sIVa8 zd8ye-aB9N6=snACv3!L!f4}uEn%PaK_8(lZ58oDp1~XqCfPIcLkPa|$@!x-3SG9>4Aa7Me-6 z0`Wk^t_PhYwrG#3JIG_|*3T?GQUnZFFdc2#k|z`=jSc_R7xQgm%-#k5wP%hmK5;^E zm-Awmpfg+Y!iGRh{QmU#chcjgOj@=vcE!dM7B>6}o!gfEEO2~Q==J!N#ktArpU(>% z7!~-ccW6<3%H-iGJ%%T~+MF`d!#DB-FW<=D>D#m2{_pUOe01cI3-_ZhTbI1V)QP}& z-=xKruILn66r0kTGbWg}FE5cNMg}$d>-$*}alWU+PV0YS)D?4F5|}uyGU}IO-(7c(%3A z;yhRH$S)4v|0QWBovYbad_SHwq1$C- zftt;h(oB(F}!_d!Vq5plAdjh*zt=VXZdm~!+tP5#j}jVt_E z`6xpqE7B^j??hGzaRI_#Emx)|vX7@jhc>;f34?SZnc!wgh+IkmF>V%~kG~p)3Og>3 z&3x=NRR`Er{H+D$r^u=`s@5V$|K!H249Aiw7vY#!#u;>~=iv`GiSlMO?pUvE=8=8Tc-_*!b74@bT_T#4&*Ig->AdU?9BOAm4JH=JjMp99B!T zzqv05yIMD6t;-VtsbKTBbHyJA!A|Xk?!J-Ulp91awNcHr=SNU(knH4oQf?jAmVEY8 zv?YuvbrMT+b%)mMe2;b4fWU|gb%}1$mf(^N5t&&9>Zp7Q|B|j@Au1i1Ta=0f=oAZ4 zb)R8ek770mNSkN_UfZ=gJgtJW*10+Pbw#Pr4&$^@&DH@as?iZipV6hY#8N}xJ6p?q z09dAJ13QyZ5__;VFywDv`__ZJk#l(MQpxtDP_h_hJeqjPtNUF(+SkD<#Y1uixJP?+ zL}`maaUAH8uL$;PQEy@k$jt7n5K?Q0#f+LPbdot66+oa<(Y~BN!qy)&ohmo6qB8zV zI6a`eSRCr8@UL)*vhs_>7(zidhkZLWGuxQC#ob}anMNvXY+8ae*N{Wh~*yPbDwt7~{#q=Nz^0qPj{+k0~d*v}4 zuqGt6n1C7%82L>Mo(KE|*b||%<}r)ll!Xi$L{15lwA z{A4#GW`+@UYnWkENh&HUF@BNbso+16#|lHM_1KnQn7T*4?B@GGf@?-%mwg95UJ90Zs|BoaR8Z#8fOAv4BdU{YhF^{ehPGxb;qi*Eup;e!Z`~z?q9y~ zi|U56W8!a1jUSg9H{}=0Ha1i@^oogpr&s)%XO0i`j!)S&EM@Vqu4ioOsAUcPfZOKaJMXyt?jlLrywkJWeo^4|zh{Tu=K{Bfdvz;aF}qVMJN)lB zzM6oDV>|8pjgA3MHj<6~ml6_luWI`iAi}k(%^<4mD9n^S4*3YRWSnM*f^qOEAS6|H zpiQEmXeh>M2C8Z+^Q>x1GXR?6^Cn#S&Qb;NU>r$(S}*xc>z4fl5oH_%IBPc=d7qrb zs*(6cH&@2Z(S>I~Y^dh1Z|-MGCG!bEJ4z*2UMib4YE+PvnPpXHxsEB!3Ax z=#linWzVGl*#*4O`}@Llxm-AyNz!t$w0hs#%iWl^x(nhUEj2j$%MEn4{xL>pGvzVz z+wM)aftnJk`Yj0GXMhaFv-L1|HZl|QpxF?Or(TiXIGPsqodY!A(B0mu&7ZoeJ zk1NT&tb9v%LW?CMyKX-GKf$=FH|K@xf^pm4tS;`3!uTDYw*N};s7W@m{K44rI4INe zKwkFVqQ7i(qTO7Fg;judQ{dfk;&xVEN_Nb;ubVqQ8qqm*%SB^65`(cERVZMz8zbTX zC_IHy%@$uUDJfEraE=tL*1z3|;dUy?}7IVo%^B)PSmi$W4g$|Gf$*|+(~ z(9Np>-aHtEEhgs^N5K$ zp+iZAW@DIybW|(6l2MygI;xM_6!$fc%0Q%$nr5zP@`g)PNNomSwkH7F6c1$-bgn-Z z94oYdaG8oa$)|~8Vl!$&ASInbltoDNt1lN4aBPB(Q`CgaIH+SI=Ccj25I; zj3pwGL$^Xyf?5@KkvIY@9F_rJX?)aFtU z0RS7f@sRSg(aIX=-7%)BnW9l+=G!hBKI0m7nVGUsmtj}`DEirn$0?^QsxOOzx&lpU z$O~4C`oW_~9A>K?LD0I3juxhnRZqt4ie3*Sh*r0y+YI&`UrtSy*jh~ScdJd{|<(2=g6{S==@K;oy(VgK8vjhF9)l(hFP`N;)?R6`&lu0cj2Zp zhI{*vBoMpPNw7={;%*Xl|FL}h+=94Q$8Wlq%`$FvZk~PnDpX~$C8b;oWmTC;jXk|z zPm2++Gmb%4V~OcyqR~p?)L(CZgTSR?;CR)wALDIja3&z0B;44EU$h4(?hfY6=vEgE z!k9@zZ#GtaVnL1qa(t>amGocI*mA8eP=N2RwTZkFd27Cfs$Y}0RLAclUq+)ZY&IOY zwa?G+C;jajqf{-Y!iWPNlBWGDU2 z5}WEz5&${p_r`_S7&iVnD{)F$N)s&0Y`oM6&X!zh#A`RsA&w|~f6ut4E%U;Et80S$? zK^t0!D7JR17Q&YQq|FhL02kN024Y}1`O-*qg*c*s18M?#hEcwgm~SffMGgb^xzMCQFs+QI8^Z!Wr_$MS(haazC>;3R+f^nl@z;j!p{?U%{E5@CRy(D-@6cKS` zDiO$3xOuA3D>}**9b+{y3x^WXga(^$BUjoE4Y26yHd zGRV{f}`Y5_)yuig5HVmb}g#u=Xr=?6) zg;ur*BU@M?ShDRa7EW&7_Gi&cL}uw%Q3FJd*CL;c4B>)TzWny=wlPiLh=}s1!GffghX-w(ceD4K)@BM3gtWM!*5()APYiF+n z$QYV3^?O^4tMCbG1ii^&5e{32*MFTUuB>?w_(}=E^w?LFV_D%!?x<_0xokZOB8J|UqqQr;xkul-Ebpj)q z!i%qM2{nQ?ULci%o2p#}P>zsMQ+GD=&9+Q=^A#}EB$_Y5PhiaofV2a(=+2hC;g!6= z*!;lgtk9?~DZ56cEFP7-{-wOY+4+I9-cK*NF5tVQAlMn6k+b_}&(12jHsI@B={vn3 zxUzfynpuDLpRmB+1e?gVg7EXcm^Tt)Uaj&M;`qDq#VvW0cV8asQOI_xS0UTw`a-sm z_x&5XvpqlOXw5_Ep+U!uD49GOX8&bbE%RX5pE*8!sApX1)15*yA&1N!6I}XfUbqM3 zkZ%PV&Y+HRbJrdDHd#pY_AV~o#}#0Y&5wJoE35YQFAjI>)w2O!h2vVD=vqk*_odH| zs;jz-Re)*x7lapam$uN?zZ&uva7_HqUFy`Ta3eQ2|DF53Ya_DAg;d*F@jbf>MOo=9 zVdARcA2plQmL%Q{G{s){+>lM(#k4R{x1dv3Kk!Cv#6K3lH?IeubhQ386o=7&=IOjU z83;1f#_l340keZ5))mN8WI(dP!!dFhm1Wn)W)YNY9e|d>__t1!n&Dh#|3sy}gg<4j z6UWE6j(aG|N4ZL9^#rqvc4HAnpmukGA0Gm_tQvVt;jJc|?afW2e_duLK}Ava_EyRUVs9 znk`tv;4M+rLXj9u#$nHL;#eM7cP-=k)_0dnC7FguBa?z3qJc!pT(1LDXjtmNC8}h^BHQGZ`C3fl3=+&t87^JX?98W1Q5arRCZw*=#Da0YIocFF?WA^;vRfYE;y5l7Og-TK^k~BM^6_neyn*Az1;9#TD0PJ)J5|X3QUjImr!8h z^*0jel3uH>BwHY{>d>NTR54m$lx@z4mT{KsR@4ft+DXEBtM6VtYKHqrS^b(lI!i8d zmSahenp54?YnCz6EXjFeee%lgVy7M zzODJ4JGXGlwdXorNA1lVT56wm*#X%eE%>?kx4C^M=ko22ULQQ3@W4r9zN5;j+k~%w z{5hc$*&OmjVHTNkPUz%YhHM)|?9WM06r(@|u;Ya4xRXVZS2sQeBm9j)GzskoV2oW9 zS-$8fDK`bZWv^{Zs-*S-)P)V-gUY$0nKRBE^g#>lQf(_Y6PrOg#^8)YJ-#b$e5?)r zXOKQ~5SYooyEB> zZNK*6wr6R`Gjkp?GB{g}N02igrzClFpr$GhLDxT~W&{MP-Mp#5$4hil<=$t?sX?90 zmHk20zfr|na^nVZj9gjmlOD210}({AYngvGT>e2Z)z=<*NwhO45GnP_^4DY!fP#|C zaEc5@7+I|1tkzSCd0UNR4z#w$gQOeO_}!4?-BsNhXE6J=WI?AS4Y@Wj=OyD`U0@{6 za$eyZUr*7HCDmoFbJnE~;pk77iU2;+E*>4dsZg3jXV)z6392+%D=osnz^!ws76%}< zNPbbEHr9d2?72Rq=)oKlCZsL*YdL0zf7+up4*#U%YWsEpb+7p$kk}Rh7%K>cRostR zB8t4*_^t}g$@t>LjzD5p^FqKiwmeJ!2c#JmsU=3LH>Y#w90OL)>!A(hD0M*_qEvgyr_gP{!Ac@QS;G z$^zjmrk@6e;uP!&2Wdxyvz0W(13Fk+7-fP9vkEd=d56SBIv_K?2u0Yg%dOy;Y%h*TS#U7S*i5#Nj96~d zfAbvgUzjp?gR&{R>UyztJG`yiMb8N_`5f>Wx~9XN(>83@qA4zy0G-8snC+KYZfkew z*Zg5P1sQacD1?-~)~%Ty22XYZ$()DkRmZ80QERo)Y_R+UTJ2GIW?Fmc{w9pZ_7F2w z4$?<~MOX#mC4I!Z?3x`j+(+zBK&Y=pegVl_sne9C0KfYYaRP0A(Q{5EgRR-IJd`hE zhm73IdOA_cVrt?EdS}PJpSi85y$gDb;ZV>3mYDdiGMgF z{SSKaWPKYmTmxYR;;_&&P+^ z|0T{ppKI1oVUd`#I4@MxGuZGae|+2S1^%+%#0Fo3+5h!0`+p%LHWg<7-wcUMt(x8N zs6T$m{MikU_~TROFKYN?NNj4svWA`b-6ONZHS^O$YmXlrz6LfT&n0ESjk2ZtvR@3T z{c$bWzTaOOcab2`9C)>7zCW&P=37;F77(+x*M(X8=8U%F6GeyY!=lnHt=k%L+;(>8 zI6#N;iD{n>3i;1*D_5JibYEdJfW%Lw)|P&auS%0QN*b9+7zPQp64RkXRhw0lxYIy& zBR8E&1Qg|V?J&?hMzn1(c8Zh(!%TpUZ^imYT;foKX6PRU&&8ycOv+em+gSg&$<#kC zroRb&*jj!qI%hZI$2s&P9P)*_QE_O}!9*QX&o>}X87UCUscFQvbjirX7hO;qwzPx& z`7hiiTeHmzpF_Zx{^%-lgT$7Wx;H65XMgVn$l#VtE=U((>_GNHpf>D^{>%-F04lvX zwa?#D@a6^x%7`N1x^3`-!&&UR?^kD0F;2n|r>t5Z2Um8x9)&r)gGd z#Kr!RZt|;`ZZ>o_WBuCVy57IdTwhaMH~6<%WOUMYz>8Bem0y~j3a5;;_oMl{bHAiN zSEDu{HI@potoxYB;8b?VE$chc?M?F$yicxKF(LJEu6DOFW87yUwQ7I3T;NzE@Kte^ zl#AT_JJ^I70L&p~c$mJjr6kDAA4^!*IK8VO{n9x46Vj=@P;SViMbidT3YQtVYz~g> z(WpFS?&e6m2(YVmJ~3i*@W3O=r9@U(QWe+?y~?LrFP+Gm&U`ZGc2_`>VpB_C%6Ju6 z4jEuE^5)|)p9D%PKvp3t=3IgONtxF}M@dN7@i{OjCVdO+!)A7%z?M#7vkNC-QdLgj z1;m<^zPG>;SzTcrTcWCXIgyWDy;6M4)*!0<$(&BsVk*>~iFEe7AskX^uEHf?qFi-k zC{5GbbSip++L(X)WAU$wl@RT|CTgtIDRcfxlx#)O<}Yp6PmFfg6u&pG2WPBV8)S?v z!d>drj8aE^&IPL~hVpIb(f*95=%P2HF3gH`m%^_|(6=DEh)NQ{>cjuzFA}+`{>NWL zbOE9jLM{hC(O-n!zKXlU6~t^jp;9om;jcqGH?M>kE;%Y%Tv*qm>z zIl0=O05iyh3B_Hn|y7W)?bm{*aV=7JMS^9j2RJX4O?VJ^7PK;ByBrTT@6p%8lA^8;dT^rPRN`O%ydS$Z< zran__UYU*C zx-u3jE2%86&G9o4rQn$vQEN3d$RrrrF2@JJRaX2*BTc-yxq{oNMedPgB64KX`)CyB zq9enr)ZP{l(kBHV@O&kv;BQcW!wb-ey`I+Ee_$o0o86os@tHGUs6^*Bm)IB{Anh}+ zf28&@v2!X}qZv&nqm64eiL%bgk{ucex0#W#!gXZa-@N23z9^F6kjG4#*G~E)IUR9E z*NS6>Gbg1P5KGh#Ga6$$%zK8-UV=!E`Hob_?$-ULbPK@wqX69trY zndx||gfP?kC6I};gvb!9c?iv*t8)~ZiIEeYgwh^SD#NTXE@nH~1ybjBgUM9S#vJC3 z7G5QOX|#YhGXqka$IWQUir(AfDUfWQa8cV`TyCmMT0l|46j6Qi@b5&kSG(?1t`rzm zyw(p~{FuA(9^vK({%bNW;!xjwrSZ0A3bYKs3?XT4Lh&I_O=|7*)rZjU)&*pfv=a#1 zx;A7{5u-Jigil-RIf+3r%Qs(xGH}YMkMf~eq&_W!7hf53sb^$+Y-pn(w2*@YMLVmVI0#l5DNjoqVy6q@|LzO6~GTAV|cG!vuS*j zNYMk;A|vEbJRIsDM2w)PIFU^gVh??$mxC2lEgPDW-$xO76nfwf){}YardmfL%h*pI zUDov7cLSl4?0L*olCLCIHK)w=-;@wDvC3Zs`t8v8;)4a@g}#^*s{Bp;#}|J+zWA$g zv8m|T_QCRv?^id(po3el`29WOmVkkKySm{3m5TKt`faPE-;#5l5{z7@<@bv6wt5h7 zAy_!dPTO#dER{Umi}>-@*kA?vYF~I+a0^6{eFDLi_+Rfpumal1^SfF)ZQnn6%z}n( zv$9$SrpKm99fh4bGp=oD=U{VU|CTNI@vP3l zE%@>Ap0P_tf;{}yHy<5Mepom~>KnBc-T6hQ`>9|nJbhUDT!MI$dL_4AHHHo6EAZou z{A0~bC?zFVo=c(SyO3(34Xu@Py9{Ya0y!uRAc)ki#Dd2S54 zQ37kHK_2;`P)gS0$IenIS?{fsBtuSDrDW}ebJ8<6I1437MwY^9pJ~$YKfmNuLv6Q( zPTl_3Z3%~BGlvTbuBB#yAl|b6Vd^CJJX21p&GC$d3#pSf)Av{Nag#iTZEuddzdvxn zo&(Uk78*F}co_{P9yuTl{>fjyIX6SZNKQ1|Y1kM!Xm;@+(aI#ZM-p);(`wr54)74{ zJ{}z^S>+4B1>6}H3LM&%?PP~0w`T58D$u1~f%gM2&?mtay&&0}qBMAKw=A5A74t(8b{3&aaRXuSpdJEz*Oe0zZD5!7d@Krc@G4 zlSj-;8ZNhIprUi)i?v8Rdj?D6qHm(23&gz9m`AFBc<(#^D!xZ0ZOV;gK_Yu-KaL-N z#qG2iU?`$UpYg@(3?-kj4Qu(1eC*897|?NexfD#G7ERzHq6B8wB=WV+*CYIi5BonNz1?HSr^ctNavmYx$R-ZxhT7-63J=S^nyI8a3W)i>(m>@u>n`l$+(p&&b z2THQ#pxg32YOFGhXgPBD&0}HqV;|gD?hhp%N^y5l<768JOdIRWLF;ger`S`QOKE%Bpt{nKgQ~uBn$(Rr1$R!2ysMB8)|=v8S5Gj?egs-rDsr^LH#wnc zp%-0SjrP~p<-q#)mhqKWblc|yl{+$F2}atBEopnBQVnBi#+xzylRr`Qr3N61^}yyE zqERd36kW0dx75zrf4Th#_mH#?uN57WZ>dS2=@$eeon?-=+I{3HR)NUPFrD_|=n(=y z^R&(JYS?;o!@i|OfeHSp_ud-tEvocAR1mCy$w)>2?24?C8w0-UD}C3j8WG;?i+OB; zKLJhII!`D*VQl!%zL<*=V}8ECUpR1l@$q-XotPfC`^se-TdNzc1T~iye_LAIk}H;N zJh*&gIFPve@*c^HFHD(yVRFuMd4YFF1>Wi%x&%C3PV)L^^8$Yw6?kQEMoE6acTI)w z@-e|vQu=4#*}vwFtdijY-@q}!EhL3}%>VQG{z3i7zQ1@+LHIWAp8fN#!^Zzl8L_(|jwDC_j_V}J(LWcC{%dEY-!IG93TNGtkxON4o%0fh zufhNK@A+`;U-aSlF|mh z1OaCPKj^la%WgFFYz+P=GyI7n0fSe*zjbySTN63x6R|d|dK@_TqN~ZPNW^DN-ql7G!{0^5)-~?5Mk=EX=l#@eo3WoEgCF29-r;tZ~ zLCNLf-&K}RW6yp%brsAE)0RxkZag|9DRmOz!p69viGMp1-!~_-UtO0jeRFz`;qtC} z&t_~L&5k9C{8mI5ONvNNSc}b4?pW4vj!rRy#;Dg0!^{=kyLkcciDA;2CahEXI$ z9l)jzgCeTd(1PwfcbI029UNM`zC#?lj_Wf=7`n9&obtx}4O{!(N?~={jWKWtfzSaS zB+@N%(7J`;9c&!Q@&C&rw3ZoG9WS#24Z+-!yW~?31BycY;n7b^0YCzLtjW~GG5kdA zE}GVv>7(LB&I(ZW%1bM0B54(B*`67yu8Qo)EHhDnUTb#O_(a=3<<*1(O!hG4B(ipH zVY-2Qz&-V5`rGnwR3MF;2*?tNLX3!zMAC?x>HaRY zD^3bj114J>1_~&N5*?D^@BJ}FSGTOlH6KQVNxUYpB7EeJ&?iNw1=OjJjocV92T~*! zIX*SfL{GkezA7d3GL$9LdHB^cGasWMBt!wFQ4ndJvTOHbP}0E^?I8U2b@BiY_A$j$y`}h3QXPP874rK zH51G?>$VsKrax3pQleceZx>K7GI)qUz+1$cp?Pw&7(*p_$F`zvYhe=k*Y(HDB=WOk zh|(JW9Vl&TDf##(0J)J#j%RWdVHmq$be*B1k+`nY*<>=2Q>?Xe#5U(ok zg9Ijr+7}le2NB-LO%qD*1t2)Rk{Gy};JEDBT;w&1kF)2DtC@4!dxzLABct0p!hoyD9UEuEW$e*uRh$2`DWWULmejV!*xY%M1&o^dVcT0nbE%frt`=}`0}-z|4H2`lj0K2 zFceOY8O-7~I`s&B`A@f!(9E4xs~_|xXZ_#6(L&CS!27)ORVOwHFw0M@cI zBf0c$9*vep{CKaZd9vsxr^`wt=Z((MdQU`Qcsz1rzrDNJ5!#lp948cBmc*_cKNZIj z`|J>Wi!V=AZAfQklG|U)RTGm-AHR}f?3NqxH~80k;MbeFG=1;xb4Vn7 z?K$lAkbfdiUZuM;wN0eLb}1vIfs$p9F5*4hM2G|x31@;85xZg@Dk+!~Z9!qXHa+2| zHc^Ik0KLDC%LJvnbX=Xe;uzT3HntY{%trEfIrVT>rp)$mFM;Ukr%=|4N5~gdBIA6L zblRCES{kC==qu@^u|(Kl^Jt_&(B zHjj-$X3fJDGCk2&P7-A5&%j^i&rCJOhq{k@tPr*`MD@DcQAV^sQ8mlBEr1)PifQlK z*93Er#gy$(+Zw*fPn3+HbGu<)X5ahNUWhOv&D)J6O|6_NkDw+bHLNvULL&DiEC$t; zECZoSlpa`*jLA4(TbV2;ufT!Pt3j4%^~P6Gq72yC!=4D(nFC>mod-b!u(MbVOi{ZU zGP0N)BDUG+^;qRGa1*UG-AE1>cL&M#8k1}@j~U6f;26TUJ^u#ycJb8QW2u5{JvwM` z+gO0L9mjK(l@rtU#pSJidJN~PDksax7lyxG6pXtec#^Q*7}0m}2);gpj~9>ZyKCeN z!?!hkO<2bn#XXKbPE%(%!o)KiE~m8$ z1Q%Hkr;f=8Ou{$DZ5*)3hHpv?2?vk>gH|1S!utxSr*EUit2Dk}{qbZ4CV(~pITCX^ z0XfoUM#(B|Kn@a6d?@d84g}k$AlMquGK@f$j51PJqva`t;O?f82w3l@I=885z-tAY zE9Uqm{z|(j8ITVD<=xhlh4yi;RV4EoGXUX*kYkJELE;$8!&mNxB&T5muIi_2m0e)e zxt)aA*uBmTzK4Y#b#5>T+!0yl_K%VXlYbO-ZSicCeW0W2k?))m?q11W;0Bn`cR)R| z&Z!$Whif$t+SoCkFr+cF(J#1U8&zmowstcpND{qf)>)nV^w5J!ua4H*laBH}5-&EM zUjdVZyAi2`sU6#XwulUG!jA_N*)F7XCV%UHn}S5BJFVGB8H`zAo4bM}Ozqe}dR0!Q zklIxX>4=I&kU-R!GzqbUuVzG$kH8k-Pn4WQ0&sN<%_X5?EgvP=AKtDt$p$#(8j+Ke znI>ntB#MwGA&w{lbYX<1OhRXcn~`C5}mz;_WTxCX<*0>k_K|pKa&&}^?#J`at z7jTvG@Z*5ru3Vy10B0VRRinYXv=4feiU~Degm2qH>w)wAT0t1lfJDny@sym63Mxo3 z6)U=RE=MYIL$E8GL8@m)zH#~FtZ67Azmi4VP6G#^OP6?i|; z!v0W$oYli3>kU@F+Eyk3AXU7J;5^Z>Pzqzi0NMBhwOEe!U_)Z9-f?r3sSO1>I*Zx6|;=7a0B}k<@^_Ae_6_@O`}QAVdG%)$3WzbI0JVW%tK z4(7`d`<^%B>)IQsr1crNW~oYJbLi9>ic@ngPd!}Z{I2s7vNGqh+C9^Gzq4+gb~_T% zRYhk7tuOQysfX*|x>!4^$_@%zFthSP-`#@0vGxg?BaAQe$?bQOO3E?Au)Tg}X?yn` zqZ8-8c5?shdHriDvr29b_-0o6?kEUO!l}9aLDKjOp@fVZ8{X)PxjQlDrUm}BkU{z< z6n7dMe%Tju^7zL$9=EXJ+L-uv(&DB-7TL0V25MybYynm!ckQ`NlT_HL_*) z^sXbq#lD!@A^y-EaVf1Zj?6vZAG!uV?pollJI;wO?q@jKuW|^FllDzv{fu zrWwiWPwyP+gx{UkFZApT2p)SEgs0^tw{}k~PuLL)x5|{(xbbIgCPR|8<6`S7uEhUa z#{|o^2O7r0pz=ce_+FslLi~6(e%u;pI2%9q=^R{{2odLL?|ls@LM&Fusb1~&SqUJh zs9fz;MEEY{YH$K^MQJeh{|X4hyk4s^6u24G&0NAl6uPRe+4eNhD=j* z%1Z`whFNdkozoS|7S)W6G+r0FcVi=+3q0XmsJ6|*kpyc9zxY?sV$O>T6|Bt*PZ7u1 z2|OH00J73j3)36FF?h_>@q))pK6}G*qQHdzyr&eh?eH=X#RGT`dd?W3ZFI~r!b^3? zLEm?EsJwYke#tKhu9WxtXWp4j0~d!YpN<`L=Hl?fJ*6)51_R98qk`UBaH_I=3njif zo+8Vmv~jeaiSLnM5J)Cbrh-^H1zaf-iz-5pORhRrJE&UG)dDOd22f&BwHX14)UG^Y0iwp@c_HYN4-!+P8kTIDgsw@#gzD@5`M}{OzY2%JAR->KOQmlq-wnwp= zz8jVL(}l!DCW^Zq)fQT4jOXQ3P>%qNbdzgIRXJOcJRW{_4Tsn+~!EE*!2 z<46n1N+)Na#u!Z$bk3@nyPF3<8WWo1w z*X2Y{STzD5X7})Wt{GW*iryWAZIIa@JBCH`48oJnlacl|EDDlVvc}qSLesj8jtHnB zMz2U|xgd8UJ5oXE_7UYFi<9DZZ zZltzrWoWxrbo_{x>_^hkMj*8GxSb>0sacyio4mg79<{s?EjOf98c#fGICqz3uC!=8ZW|K18+_P9RN0A`o&}CPgCtx_y6jt` zrYH+Rh)nWgBFB^d7-Oj0*B5#FMh@aTM}jzYOp8L4p9wq&6gM|{bT4GIq(W2#`cdcz zwPP@1;rPLB^e9g;jg}_AYg-DmUbscsiTQO!o%oBCCFIwo`3Y6Rv|rnH+r)mL?8r}3 zM$-0=BWx}D(4+(_51(w{rELnQNi`(?=+7qzYyHtiuIomFbW0tBQ;=`wzSFz1MfP6X zG}0%qS7k!)I4qy|FMa+!$@nf?M#a%H%6ZgyMz`|u`p3mFAjE+ijPJ2l6|CKkQ`NDB z{Q#Z7kyUB!BJb^|Lb|{gRx-G~=^Y3Z2ev2!KfXRk&M8zpVScxjNE~o;-&&WDX@$ z*s~BlG%x>+i!m8R84p9@`9H^RV@0W2<7M7b9V1k^)rr01FJ<_L*minjI#GZ@XxOPM zGYRr3mCcu!A)Yk{X%rGpxdz7Wv@t|WDvk7c?NCe)(so7{uQ4xW6t7{YQ+f}_olYzM zGUH@(XGblnUTE5pY6<~coOZg}w;m+%*NDx;AyG57rkB`ij#_6IuHzpmts?wG81EzO z(UM-E%w&2yh@~$|twhKfwP5bUOP@o+I_HG2bA44F5mOwi8ugGe2-U<+7LQMd`1s5y zo03(1{}B;T;}y^sV_+fS&z?M*2qY_O5IyZyknl0#A)3QkD=;IpSqasok%mSyxO7T< zlvP6|b!H}l!AHTl({*$mp|JIJrJHC$uAjX7ybG)l`@#@XOiUH)$_Js2vEoOo$fPV= zE(^e09<(Lw9`Vj=P^=m!h#yTA**toQql=_=IVr7;WnrQ)wAVkpN|Ev6*IBfv*D;D% zB{26{6z(=}rPwn%xFyyRFL52{J*@$Zxx9-a$41?k;H8=C)}&{j{Y^LfyQ7{=|0XNy zQJ9c8i=*)-6fo<|0f(HFS!BgaNkwCkk-g=#(1zY6Hba(Fl{Tq}9nNUz{QSJ5PTNce zL?YbhLDof>Y@031fQ{^R50S`aptr4mpGrEKe==S2=-X7YqF!86FiOt5cz^Ho$c@o@ zfDY6}&g-Zk)?h_OzWMgyO{NlNxy*EgYbRNkPR4BA$)Mjuf;Z+qOe5a2IKF)pDYG!1W z{5;@0y~1}o+PvMwW+Syj{h`_5&;F6<-_~b*cqaO<{XNk?9Y0blwhw^k@Z)_?`>VE$ z2w&`rDa8LS^2H=<85{mhPk%zz`0y?G-5Y>pXvMY^t=J~PPGnr~0vMWv`{li|;75ud zhrA9Bi`gr^$A8)O(D5j5D<-fu#2u``Kg0{Harsn$KW^Zg{=p0-vdj7S@;Ck0tlGKn zyZZqg++6;@S>DEFj*)a0$Z~n=`ND6%)AC`ol;*Pw|AYtDvsaelzohVvk2!dbTfTEz ztI?jqE^ZX$8g7QwrM`U@FHhp%h)g6zA_N?nqbu&-F{l5fB-Es&xE)2wQvhw0)N2_f zTCWub@=gSl3~$NHYCTUljx^)P5%l9>)}(NX5tm3hh)|+U{m7s>n!dA?vHizhL`eMp zV=uz$_UPD2gj25eB18Vw_9EwxY5gow1JQoCzeqds)|v#ZLR0@be~}#=Q%j=Ax6Zh?DfSP_GaBti7L|uBSNIiZ5PX4Hqwl7$#}Q$`yq={`lbYv)jtx z>8fu=C3fyr!#=AOpib51nD9!R!X@qfj5*7%us!A_q&#&}=cDa1qF^#frQ^)I{l{d) zrfw9LSzbJQN77>Tj2<<)If;S)zcv}kpUH`c$la)8CL@3IG8sY3>{{Qmu_F&1vsP@& zXBJNNYMEW`(K7oghnCrM&lfGT-$hgGamyM`sYoyRSs++{`oe~NbJI%(27+n$@iqL| zKM?#7#{4;5gOgy|Kdfu;eEe_7Wx;!3+MgZ>PQvfb?OIoHmp`t($Lq6ydC4z&pZM%G ze@;x?*mnL0%Tga*lG5+jZ>8=$_1djRM^9|~{6o0;pJlrFcVBjO4~B9v(#`*yw7Ak0 zokELZQ(AMz1k?8Ag;RYox$jgrOcUf=p~Z$|y{ipLe&^0DRLNEPT&L^EWMe7JBCi*; zTjvY1T5m9|{H4dfo0UIoIsR{IBVQEs(w?KzFoi^T1iDz1$26ND@Su$>gJBA#)hyV~ zc1-!ZC@`65Hj0Bs>)Q5zZc{bSQ`AJA-NLKjK5We$H>T15tK-aYm+ky$28zJ%Jf ziItZbz1^3HW8}{^+zw}(+n6q)^$$K}Mbv57}?EA7oBWhpbN*Q^wo)7buy z%Ip`lAS~O+n9G!;`Xuw%DCANRoSFRO2S79%m1%MO?be{p#fDQnh3%UywHc8Eh&Lo} z^R`>Nh$2Fp5J!-wG9bpAgG;vHO8FXb#@D=_F6PE;DD^l0NHz1&K$3os@Bp?0&w9|9 zcpVM#mneve)|0`3Q)Iu#Lh_5lMpJBQ#Q2dIyS0NA~Uhw8UI|nNL53M zLI}yxi&vOD&epCx_~99Z(+#v7i8{zSbmpNaJVhi;|8*a}ww48?VTXA?e4jnm2UtGx zrIm_-RC7<3Tv7u!RU4;%X}i7=5^ZxzTGjw_^#$D}QXbYfMe0=4L^d{KSwdnPIZEaT zA0ww>!#c`Z6y%ti`Ul` zqWfCwk@7xRBwELHX67n)XGY@|Vep>-spMPp$#1^6`Q`6Z+W%+%+ax%{p_6&!>Z*fH zn_fp_S~r*f)!Pm=J@p=4|J67rc$k6jy|tC*C{sj zsIs6q3?Q<;>~*KUG@Kv?g*@#L<<&J`go9vq(?r7vGX%Bm{bImjxLMY|eHLlW%&ka8 z6@bvROOlbM@)@x0Yxt#1n?tw?8H-a69k{UNsZr?gPCnc%WI!T&u_Yu_haa!t3c!}X zjS@SIw{e-m$C>|2kRviY;C;^UuR}ownFmIP)F>Sk1sfs)Gh;no z8(pH<8I+`w^*45>f6;c%z_>DtaisJ*G2&9qU_-3xijk8(e`OAG*jl7*J`sEl0eZW5 zM{0%)ZIn?e7D+xBG9~=c!qZz5=8990-N;HYqgKcqJArnmqFxf=j#Utt{wilOtYfP@ zPa;2wjza@3R%Kq=d4;J4$yq&LD*d%J;|l18N=T8waCi~2oO}z^Ye3aXM>%}ce1eS4 zI;4{Pa-X&+4cUW4BCzbzDit@|=#DvL4j4U|t&B86=pu1DtIpL5pKBgOwWXKp43`uM z10ppLkxPWFAkzUq*9c*n0%hh9s@uXfHGwe)?*LfS#P(*w5DIMZUL{4VKeGf(z#r<4wTIZz-Y6(D3~#bVU{)^+3e)kGit^Yyhd1;n8 zEC^4VLMZ!ZnAbn^{j%D!=%JM~w3uVw+ZX_G?Njz~kmI6~hBg_hLt|Hxhk9_vcn816 z*EMqNw1L@EmTOj^UG|qGl}y}y^|FmwD>h~>Y*-!>zds}Xos76CS1sGvW5vdkZ%NE~ zF)z?3E7Yk=%B~S9i$^4{|L?p&Qhwm1tk915l*xJ5`Z^T^@9Ew@yDY2Z%7AZAh3{`; zg8jPpuerZ}_WkK6Hl0!K(JlK#uWs3A=(}Z4y-#$@Hl&iW8yi)(Ne(Xe=UJip(}E4l z{qa+(7Bw6g5<6w=vWCxx#O^Lw*3dX4ZucXz8v^*pR!tB2^t+Kp`&Hg*) zK2rEEaUa?7lif%1OZqJ*dls1kbM}c0yBD~@$*ob_G&^aNQQt;3+Yj@q>g4W^n{1jL zpS6A!1FENFU0hbqY6Ej^ji;Nu9E@w)GB5l`jsptc3@qL~DZ4RgXwsC)*^Q?TO+EA$bp5bQwUDZ-9bSsx!=r+ZLy zCZrxtbKbG&pR72DK5eeySlOaiaz4wgLl-tCsFvQ?r%{%(l|AujdG5Odgd7af>AD;= zmt%s)@nM|1b#_~C1H@^b6v5@Gk99se#!iDue5S&HAIDxX7(FlvaP$KCT)OIMshnc62o5|%TzOg5&6n$fdoax=+LEY zlV`@q++G?;!}cXCAQ@Q~Zu~4qL<6623X~JQv0C$3u?)bI6tBG`@R@3kN)8yU)1$&+ z^AVjTN*^7kHHL=c_r&Qrm1eJ)$as_apAn73|7ZqRoI zW3xWm^sp!Y^MaAoi2OW0u6dAJx@jrh0(R?UzyS8^1mxwH%UOfT_T^>K+RHN+>7L9UN)!LfaMrvGkh@y0Lva zE(&1Qj7*b-?lQqYX0I&p(#~GwyD+ce=kEoB!K^Li4NomLR)9;BUEIXXRN@)nzkESB z^U`5^qirvw8Xxdx+85ePzVZvrZz~pQk?e{b95`eO5Up-2hg}`xW7xUZysb)K$eixc zncM2iVlGN5-}6FD{J6~cGc)5-0oYPkZ0xzP;ZZce&WOKl`3a$JT~a2GOwOsv3-rnl zoSGH-5@ly2Qg-Df=e&>?h^;BMZmYU!nbMV$=`3A z8J`jOXjEWl@6h$}DZ6q~7NcMG@ACqo-k}TPQx;#4GWmi|!7(ZQYwqfweOFe=h=6ZY zh3|qf!F|v{{ucU1)UQpun3`q(Ry50oe`H_LKRfT{-Q&nFvO&5;md%I_?o009LN#r& zZ8q7E{4lX(=}85_dS5U+^|TQvoC_8m|NNY`si%wx79EFk-SPkS>V|&8hGZMMu+0!{ zvp4mOD}Ag}=z7Q_v!qMp)eqcV>+fAb+*I}*<7)0;RCLJ}|K;M;-Q!y3UG~7NCaSn1 zT8+phXDOuNH+eXN0NcZ9-PWYB|76?Qq2oaCl+S(kexi-s8?tR8W^YvA#iN+n8*xK$ zG^_b`<&2*#@H=KWT!OR_3>4sKp%)SW}QS~ zn&p#BGf7IEMKg8(YXxek5B4O|*l-1VBK_5-fRR@X8uQddMBKv0vvE=&?@WTsTT#FE zb5Y861V4_XAK{EIT>AL|3RVKoR1~RkHhC3V(-3_sh!6Y6@yD}lq%fm$*KP%VhpM5; z7nu}9_cK#4f(7N%k_?sJsYbNFg-$^>lD0P~|NdQL(lNLPre*|0qGoeNwY2n4+;Tx2 z$ugI-?$JpHNiny(43fIPY5`i}fkSf{Nllyp8mHx6cPh$et(1kaUNK-NAWe%prF-e^6yyp*6rq8m)+ENvxWrHUiE#7^S#a@o%OTD1Mqwyx|67LE} zP-E`u^iBdR3;bovD==1OZcXSmNnY;2(qdSo*vk<|;17<_ls!N1NX4~eU8G_Lya4q@ zfxQ$~o5)Os9{eKIJopk^erm5>i|&h@1keyIIED6AWrKbbrk-;y#v`P?>W46&LPC-z zFaU{oCd5)T-8GmsW03)|IwtqNnTx6s#YbTh1Jg7PfC9X!&sA;1MgwYU|3LiBNRrn! zul-K6#l+W<8<@l&J`uapjdqqAH?pk{=8dI{(*lRZ)3f3>5gOYcx*(?Lk-(+YlxA!H zE<~i<&z)0cPEP}fOG!>0Ky#h2KZwRJCZd1=Pjn}%d^5`-^GzGC{uDE1q2JrW9%Z=mL*93)omz zbOp;QBWJaF3|M_+azB<-S=SB={ty--N4l;leMUbnA`~E>kt3V4oz$n9!m$&SR9Vs5 zy`rCF^iC!oM^qfBEQurq{_ zKT$A(cG}HL%G%-|&Wzo0O_!9#`6-k0lXG6q3tX5V7@8Ft*d?XMsFYnJPYU)sxqnT0 zR!MQdcR{6Za6#}xfV5!$nm|^`)d63ZRhJYWH#Yp=zL>pL{GpG7l*s`>e)cNRm&4yxtVk6^P}pj?qV31 zwtqo*k?9`!uQ4QPrH0!sJlhwe%ob`qB^fU9`zf*16-)OY+Q*R0Dr}TEB=KFoz5_zY ziAxXE-{X@Tq#fZY#E0x$S^M88A8Gq4>3?Phjt0r!QK$dY?~xl{WU(8|85^5tVGZfi znkhir>b!Jy!89z=Kr}{(>}x6TO=Ja~5=5pg8JT$Y`Z-4I zdIzzGJj#{g6!xBQe<Fbdo&mNwn?Q6PKmw zr2W4p(N=n&JBkAs^l82)5*?lfRh?-lGv6~txloh`?4l8}q{20e zDo&|=-E46c8AvFdmt$VIBX-_sPh-Y$1{$YRya_VVCZlZ^_$GFBO-$>1u~gtU0>4Ja zQga+A7-h!{qe8PUs%fQZpL|B~W@hyK#Dgh&yd&A}Xf01r&IfZlQmo ztI87jT6(i-a^v%mK1V&2$X=MqTs)8-^mp0gX8C*dv=BhAW*<4*#|s4yk8#UWYv?X* z{RdVuI(_WX^4hHfDB5gABki_xSJO9OnJ@4aMa-6n>i@9!Cg4?7SNpic8%ZLP0Z^-u zgb*Q4B{(q{NPtj6jRcA^XqDhltkMrC4pAiWUWNn*Lc}U65)_2dk!b(-n;KUd#`uB?|PREO4TXo zA^mW21zOe-j-&VS$8oe{`|}n6FghHkOv%0xMmED<71`1$;5bJR9aZ7@(Tu7K&+rUV z#xNq}IG%n(-{3uR`6Vg2^D-Z5K5 zJBrBg7&E|O!0Co-CGLpt;^#}u!Cz0U)+0G`;*yyUXs3~hEJ)UEgt~dTB+a(4>;iC8 z>vvQcYtXmK7`=l_$TCp-8<&Z!7Hyv(87S2DqTq3f5PaF5a_fU{TNN}piXfiW^}KR; zK{8GRH*gy?Z<>C+t(CYMEzC8uVV!C~$KOvo%qWag$v5X(A%Ve}Mgf(Yr`GBz4i<0; z^GaZaBuiwU-Q*pK;z+JZ^cq>CI8r{q-y9EPVF1L8oH7%D*>gGgPi_90IHrc#UyBs1 zn^u=|7S(N9{?q)iB(G)X`K-pBQW7G|)x88cqLGY}gz}zZ`4t3U7EATBiw|mRA8JY7 z8BrZw-O;+Aid!|m9X~D*Lw$Aou8MSquu_^}j^JwN*b3c1o{#8H8DO(#B_V zf(J$h|J|o%8FZ4vx}QEQ;V(_yM?}3?8ui@h&{j}scV=YXnO-y|7&U)%=#K6gnRjGV z-F{M}R%{RHTd^g4KP20MZ$+}96&qMB=Ci!jkSU}LoIWJ=X@YCdbdAmaAd#r4;;NCg z1-CO8d3kL^ccZEEa?v*X!1bZSTno1|yYJ~ac!wRc9;&2*D!66fkJqPmdoz_OxCB1V z@7-<2>O}|d1`Fng(q^qQcq^Ajh`pBh)&*}16anRa8`#3ujfJjY*AxW&@97K@#PVxO~@2%<2% z$%x#Oz#dk2CJtJ_=rFL^68ZsdOie02lO>+b^YF4ay(HUw24(z9T6i>d{2pb=v=ETe z#{5;FD_NUx^P0NsFVyqsV%g5YBOL*r{RWyVjGQ0M_SU7buaFWbh;wE z)VOTs+2XQ_@Cuw7{}9W>@`}b|r!QMDknKb$dYEYJ{EzflL!bBH?DUN({c3kJ|CZIe zbK1rYg|&Ttowl*6u=c`Vr^8P?`B49c3Zv*}*AxVKys16IIrL--_Sr>RCTKPL2~LU5 z#h#^7$D|rLCjV{~RolX2R`W_BV`T&>nbFG1q}EjIRKa=diSM8bRfHU`Z9W)!9ZGk&Yz+k#Y+ltkj8jJl@NT8K;P9^**ZGjgyMQ8*`_cp#{O zI$~<;(_t-`;xGg(@)uh0%!>ojU}Pr3E<(ECUn~S8PO+E^zh!lRVJ2PJ1NEtA>?GZzb zs$V}?xoLqR=DVru@m5V}Y>LJA=D1q}!iqXm{IVvL$K`9-| z8CZ@0)RIPRYr0H7$wPRC5mRAx&0Ha3zn!Omc!)LcAbrJGMFFEAI zls!)pU*@BSylURc@-GR}^zQa%Z$>$(7u2LNl$ar?CCEh%zw?SwcgDN9OO<%|^+Q0;RT5YX^Jxp{RC-zBr9;^2_LMyUM#Y%;vvcx%9-@PM8MRav+i`BvRE8kOaI8fW_Y5(y|%b zs5aI|7P{%Q=Z_dhT#CK1<$=q{LTpL4bl9u*V=#iOGc)VWaQZ&Yt<)GwNhKx6Djn@^Jn4x zj=^c9Zqo@RPpI!~!YOr%g&_uwsEq@7;}(U2@D6h6&oJ@}sjLyZxWfS&(YTvhR{W25~ZX}j~%8|H*%rnzq zlGKkVe^+%3(@B7A0U6CO!g5neDoDOcF0u^LXpyDT3rQXTJR9H8j=tnCVahpOV<%~e zd}g#baV801YI6RU2N@)KXtst6RM@s5;e1JRIzrGo!Vpu2Q?JB45wVM*! zz^}N-tlY}g-wrqOwH)OBGG*%+Q~s0b{xuEKN%?Z5VBcsB6yoyO&w-;yhkqF^eIP6oOE>ScZ@e0QDjR@6pXk!|U3Q3QY7?xxViDH>; z`y0kw;Fg7G<>y!o#KT}+c(LvJE)J+uQ_r}1&tWRt4CA!aJ>!nAUF21-eLJ_vS+^VXmAmp&YC^B1;`M!4lUG=}pmIOkklbFd`Rw95V}k@9GUstw)$XlM!^@@;p z$WwGlQ%95*Q}LP$Sk)+6w2cxi+ECYakyl;YF|=qy)41x#>b8W;I&y_46op4}UN)RY{OX)&c(UzFrD=RYE3AQ*jYq+3bHGQt*q@S@}n83WqgMT5=d&tIDN zaL+5(Wc>GU7u<8g*VWH2TKSuyFV4EWef`}JeNa7i-K{+gh79Y&r98B9+%!X}XfDr0 zGN7se)GD@)8sNAo=fnk^bUrA2JdvD5B202aNm{}6ldkED8BGo*5e)fyvxO&8M$TtI z9pfh77Bj0Dju!?Sx>esgu}m0uDv77(uu6&NH#IKGnPJ9^vj4lZ@7IK^I0bYiRtRsj zVe(v9PDGWJf+BK)N?&~oM)-R~g~A2D)>?f^_P*dFq!Pzh!n!vmoLWkwMQr zETlJHv9nkf#2TQP*a7rwpGHDZZK6p1h}eNE4gMO@i(QaS#{hFArFMU3T6KjfCx!{k zT%~B%tcF6(HK}j3-NyNk0?aJDCgo5;lYMnwETr0Iwg)aK{qW%Vts0i6*>hzUESWw@ zMnG0oJ$g=EAp|v6SfS?OMOdxN&@XwFGn!;v1SPd_FTvN%b--iH$#g1%Mzx&j59Zp| zr0hXD>APj_>Uj`1F=9sg4e+NG{2Ytlo~btFwyU4F!XM110Kn?UA%R;WDP z!!F|mytCY(C9<|v+42s{7Lo^uj2m>^$he>O{xapprf+2;`TCoyH@#6&e}F2*JWM24 z-}!yeai< zV^)rUWy&x2eAJeBe@~{p-W*W#HoJ3)EDZIyA@n20aHo&pmJ@@;y>Z@3<|`o(t%)M;VHOx%RM4CENygq}*Rq>+g@pKEk5SV`N>96TlLuA2y;v0DhHv@$R0Az-?`?!YY2p-iynj5&Hkn!X z8QHCEM&Vo55|M#x&n!>%Ytl9dP1>-x#-GJd>|kPNii% zfu$1QETz2PQnzFhO0k(zJvj>ISJ@E|oJf!Pa&hEe;?hj6xpQjGg3`Z2Rg0VR~g&S={oBwJUSS!p89b@bKSsO~ zk})~|)skWx=@FmE(kf^E!aNpbl=gLj+DLPHjVU%jFjY6D0sJ`i2d_t3%`ahBBH7W! z%e9klvZ#`DGax2}FziZfWS>Z6a8`pNOk}hYD)0u{1(9`Uk=F!8I_pLsPciwXurH+Z z-_2tqq822QdsU{QHuwR`A+e%PG8ht%BweD-d7108uq&|)ft&jQ+;OZ2*19Ub9+&8W zvjo!2wH+{$iC$crSP!n~c0gNKg`M8UDyivgju}oJn^Y2>Un-EWbyDEjU)TfQ`jNWU z;F6Ax&i5VqBHf*B!L&^h4IRmqe$oJ7h7=BOXQDby31KxKTABcZp>$08=xUK0&e?OK))V( z-_^}+Z>Acz{Nq>S_P4ol$ID1^^7m8Y*7mKWlc;e+8)ybg-ZgGaV*#phiIr!KTZvQ% z`qa1)QRP?THnDPVW6CeOJQ|@dTgMfTlcdB+dYSavvjtx_o__*#RqiS^%xk8tAWqxb z(uA@ZXsSptm`*rUZcr96$9DDXFZ*GJ88B#Saj*d@>4gjz8ds}qEqtZ8q&>prV)v-}h z&qf+b6Dcw?O2^Q4GCtPT;Tib&(B=s-X^UP`u(+!cu$?CDB=M9dk7ATM#TcV&e`0Ko zv*}dNLt9Jl3?!y}ok+Q%yC>(tXOecU1Jlxj9Q{ENI(7s))ZLZAL4h|1R~e!1jZF8m zknW|P+Z@v7h%b(|Hb$FmWW*vMIB-_USEM$KqAfCM7xio7C}WC>Vke?&^}FxJ^|jT! zUFXDaP+acuhi|;cN2ueT%7F$dC_UsKR0wHon;s$Bv6N*A&8BAG<$Dq^gLR2-9vwXb z#15}OfClBuWe@!#&~ zEpI+wD>YS=5J^o>t5Bj<548z@#a|u#aEu!hns5$J`n=bl@G>FiT>H!y5anOi{N(yn z0)hUeaI{ssgIX2wkqT0AOPXIN4gK2z>GWnN>Uu!B6m3*(kV&@0xGlL(+A)#&@oHUV zh7K0EYy%o*GzLhA=ovwSiti*tZH9S<*{A1x79RY*Ew>FCOv=VIfB>D{@O1jNjL#G7 zwxzPA%u(s0+QTU%WAf4wkh1yxncK!f>ttzw_oUS)lTW@#S$3GS;9@ynt(=AMJh4iI z-_LC~LHguT&#@Vn`Q)qVW1hk#d&P2~`A2ce-m(pJm9c#>TokzqX0Y5?L?trtgWUv@ zYli=v6}6j*WyDZfS{7@(IQ4`G)&K+9VuIW z1h#b4XV7Hr73L81m64W%S0i?UM%y;57&d0fQhUXr1UxDu)5HZvcE-u%sHChYwNCw; z*Y~!_v(3j_O?)~9WGN)4&T&`A29&m~N#Y)S&#HUj4I-OmXt_fqAi#mqPOL|AqQ4C; zsf(~242sm>W+JueW+M~ZdXq~zF}cD^nwTCGt*TMjnQ0w|%wK_1YqE^vVs;I4Pt8Fh zhh)te$*HPF7C9u@Ez;vE_L9u*ATPEe`N!qw7C%AfK@QFS(2V=WW9-Nf5GAH=Jg=baG zPA?jao&cp$XO0d{M~{Hn_%C(A_v0mb_qX+u3`g{m1ii(TZ$u1vRlW`SAA=O~;o6A6 zstPi8IQO-{gBP za-@O5Mu}6{$Er*Gk($^IE6UMptbVZoM3#tv0x{>JEzTPa}7dyrKz4s4!i*l}v%P ze*0NjDWhv!W~XKS9^2XtwcUQ5TI)2DL<8N@h@#1`0k$-;B~BSmV;(6n%vu08bq`O# zM!T;cfDnmnZEc+bY^D^<8WYL*6Q=;!92F8~jFv_lGV3a(z|#v}Nbbt5K{4*t`c`jI ziJ{@z;JIz%+GMg?`?Yv%tF^;o1rPmI#G$wHG(?`RG9nXfL@LDMA~$eXyea$XCB5pX z%&i@$3z%8NyvBf8U~%A+2co|Ea0)yBdBFqX-`C)h#;MJ$j+OazYaAm&1zU`2-L9jE zuHuIqm&|;cwP&^n%bK4RvTRFGDwlFg{B~?ECGDSaV#69}OZdV;#<9ATyvsb;4=yD} z7TqxZvyH2?HlVPLb%+skW&dYfN(gHI@4J*-e4zb2O12UpeWY_+%XFTyn#Wz9dKr#d zYCRQegOe1k5#lQ46mXlsG5;=Rk zt=GfzBa&w?oKlFmvZo(p3k;^^zOTME;Mf4NHAI~XEugWAP?8C||4Q!Gw&!?tYwKM5 z_3^X&U-ErGw4bhS8@4|!FMM)N!m#!D{}lDT~+bEme@<$Mh1qSfsjAo!AOze9g+ zQsM%aI0Tadv#lGp3uTv*T2t-ZY*ZVXYeVD7lb5j_5(M#~?b@_2`jl$-}pl&ZBa+#FSHIEnDv0r;icbTGaC!m+e4) zI{Bx}%sr!^*kyeDiabVox?eZr_q*6&_DbngZqX2I85ziX>8skFf5oh7n;DT&s~#G$ zs%@sLs%^Kss;xXCK$BI_?tGTOpj=mlFREl6hC@Cs;WBLxG7$#tgxnqA9!&ZN<9MEv|%N6I5+8JSerLItGazQ9z|4 zCv(`vDJ!e2$8pnG<7rDUSeiNuM&6*5G(RxC4V&~!0jJgl9?`C%wvwniQak%#B55c% zz3NjWFXetrRA>xUW%jqV=rmzJxT0pQ;tsSy3_Il?i@L1$7xRm$j?%51QUMt>YUS>_ zDj8p=6Whx)+UO>cY0_#z!R`L;=&BS^WTghec$ST^0*zLTf=l6(QNjH#TxO@__Ah== zXE3aeA+1E*ZNm>Lqr{V)FcwIsGSQUX20hg*#jG?XpIX%!tHq@Iy81J7awp&sMU`D- zuE(odIZr(A;EQl4&6vrY7e*fr{3O)&nvq~)ghh$CiF4E?SaaV*HoP#$mGzmToXWJ~ z5L#3PB9QWmV7G;o7p7Zqrx$p_$VX^B3c9bF^(8#}Ec~N+%+IM2g4m^zGT~SgQ6U0w z=B*IFj!a_>6$_}DFB8*_Te6k^4V|LgbeuW36Cl@U=?Z~}{QlC)VqxbM!v9O?h|`p`4%*?;<_eF6JAH) zCm>RATVC_?O9;4x=fjoc1$HIbIrNmZ&7h$emw35JE_$++NUck#m~DAs^rM9*F+y1+ zj^iFh_l6aFUn1gXQGRAAGi~!<3*5)dTh(Go{FEM*Zu&Cf1(1}P775>KUx!(V2UOca z#1oNfaLJEEsu6!h5)(TrRL2xpyGXU0uWPqrHMB~r?*HclDKFj^#{2rXku>%R@{{Y0 z*hr1l)Q{9ygq(X|hsGVXo{}&(X?*u-?d{a<$pNMDxj&n$h5h64b+rALq}=f7hHkk{ z-X(2xQQc8)bnD%5o8)HqXF+~CY=0jxY>S(#mga~=`X`(7qVR*@zq0Vy5oFyWVV`$h zTZ2E(25-pCdV|SD7K6!#hz66TkqssroCcEvUUBpRAYc4Q29uDLMA-2}#w$pMVOp-J zflNodBMAf-B;aJq!iiASF=)E4?E9P|{A3BhDayjYD*zVGB{mt2iaXNDLtmJiL~a8B zU5Q3|V(fKX&PD|o8j6j40IlV0$5>)vZv1LM?pEUmRf9cDn2yD7}AsgO6$X?0MFA{&F=47q_ zA1XJ1NEQGMkv$?_kCG|@1Qns=#J)0ggO!H>zScB5%d2@}gWE*6HUYt{Z!SL%eLmvN z0k0#7rNzRakd_JVf|GfyB12>NqfB)StH2`WFbhei9l}`MJPUiAuWB1TR|Y{cE1F01 zt$}rvVUY)bt|I#{zmG!KlmfLY?6$IX@$FetOY0!KglEQD^3b*7nG#y5}!} zla>Y=(MY>JFZ^s&^sDjFe<}|Y;P}Y6!shpCPafaacjR&E)V5>eqmW3Btf@mM?HY8_ z{v;u={j71}rbX$^b4Dg?{7p_x%9#(=&pc&#cwAKUhOJ}5MLh!>(#M5w?HQPf|IJzv zXnb{eI0ygs>X`6NJr~4(5tGz<#?nA_bf97LMWMhYU8Ad`t~j}E^N&M;>JxD89Q?n% zPffwCsYOZP()Q+sclV6hHF8yb>ENW+^E!vNCS){^LAUMR;|h2G;pqP2*q?~@*&ZFW zpE!C#^~-=!)Bu2hNA~-@8@3K%PAdNPyyn*#ycITGwBi%^i7?>%OrX8&zLT^0xb6B- zCnCpA8GVFq;E&iyap{lPBw^))*tEWnv{tZpJ|Q+GrL<9M*eE&Zz(<|lN|H}s#8$#; zOJ0@mnR!dgd5T{VhsX6{V6##QVrhLoAU$GJ63%+lw{{B@n}?=iYnSM9Llib*;KxUT zdCk8PPLfS{xsYDM!=t~Jr87YqM48}6<(_JPJI2J6hn$Kw+RwkT&=?0@8||+nwb8E5 zj?hMX348x;N~rDiO`DJn~>bZNh%cBEq-FQ3ug+4)<-PXjevimEl-wbbTcO{R=Nuu|evtmMQ zW-^BD%Q{HXjA-aJDvt`kB>W^P8bGW}fwEj4Tq#FHv7F=Pgtqp(vkFWCV2{2w_}1E+ zm^U?)B?|XCLNV^uhCDeNfwi9pO6?J*)S9=gPjCI@L<#~q8hecG@c|*cgE9CPO8YSF z_~ydW4^>~Fn-uS0=7d#_n4C*kTIEzr833F@v6)IAYAs?44l;(@poh(2aKbN+xqXuh z%Z+5Y0wUwbyxB7S8e8MGE2tx)>!+0{LyX-UU#;qh@gX6AZf{f~FKMo=5U*+AfNM$= zE2&1yHrso<^<;A1s)+2NJ;O`dshP7~^CWL0!rO16Co}PsLJLvi6_jA*27P8CI5n1Z z3Y?%AJN37gf#b)99r2A#fHOTEH56IvGg@ldv;t`7q^8RF^Vs135j8~2 z1Iy3t0A#+Xp4!V68Oxk3`)&jYPKy&EA5Ja`6fv#D}2`3RT%~dCXYGlDfArug8}7ffoqpEpPHD zva+{+VfAzxy9{6Q60ytLNh@7BZMV_f1E*#QX^Q5#P}kooTTSv$YUCo}{AoTi%&)1D zF}DxvjzS7Ap4HArSxqo&V_5kFlZ^ng$r4b5S#;#Sc)%>RtdVnJZF9G0vkLOFL#Hv}t08#I?~_-!dtCV3oIh55TTX4BtftDGlV3H5nXJpU z1||qG5jMqREvbw%a=mhE^Wnq#i3QpwyN zso>q_T|7`4JVuP17~hOnYr3C#J%Zf!z5k#co}@K!Ryx0bH8CXb_(`_usYO%kU@C{$|?Dc+8wF*_hrpWp{;5sttd4Yhx; z7P0JsGNJn)T8n(Lmv*}~ah$UbpzV?u`j~ey?qzWNqfgsU{hH}J9V(x+eLgXzXS2+;YJJ2bRXut84j`B@$6M>PPe=3ie zYEaFnJau~7>BhN7VgI3u7CncRd#{6ebp9gdvEl70e+Ti4UOc;_3R{%hdLO7FJl+{m6o+urqQeFvGy-$}yR_{!bV^33g`X9#))$>i~? zu%=N@29rM!W)S{^b~3{H@CSAYlKD3M@3zyRlr5`=pK#;?QQ=&o>u?cs4#{wifT8Od4sC~B*kvFNaoI6R0Msp=~ z>)AKY+{gr`Gml9o)iaHs5YN^3(DtDvZLEd-J`F}X2Hf75_u#*NdTR7b-J2in)KWLO z=Jkh?21H-@^K*Yw@O1g8mS{J!8xaB7o#%n2n@>!QiGL2<+>!D?$C?vsmc?{$ox3E^vEsy)xr7Y04KZxSI#m9Od=_`%}7gNrjH`ra^BoL3E-ZgqFtsLg*x8 z`1t9iv40x3`OaZ_Xhnaf8XqA(ReNegVs8{>^vMVQ7M+qRcq3}pF?D-VpGT9`GiRcFT~8# z@!E`!j>IN^6>RBQ3r|QZ-lYa&u#%$Yio_WYXc|q`+c?S3RlI+Z#S2Px$wxa>*AQ=8a(Fy>6oPA%D(QoO$kdRO+5jr3s6_5(PJDv*q0q)N!5)5m zB)9#^Z-(7}@$J07G&kTSKpk;lmbPsXV+wsZV8pCMKqGK55FNQ}yQiq-NBy-_Ha|& zq|yrkal(U~HzfqMk~Y>Xxp?Gxb8(Rv3DgCd$kG1B)HR=S8@LXLhRN={hc)PStEPjN zpNksZnIobfSGFrPNL#rS+Uo~VYN?DMa-TBLi5f78mbEopX4DPwtCp}cxD#28nMy)! zpJ5dbxtdze^)y2DRSul}zNPb~WkF`}hCUaz5fG3q=b!mE(zA>I{!O)2H$EWB);Z1N zTuE_^c5Nobq>zJc<*oXH%a*Nw>~D#4U3o*XR6h!kg|T5Jzg(gGABG zT#PNEgzwf&OPf1DKQ3 z%F>H&3P#N>i<*-cnuJq}GO8Ay5hy@E?btElO;OReCq(~ZXiR+RQ^N+c4k))A*1La`bz8F{dd0sda z6}_ly^zBOm+d;VP$qN_3$K&k>iEY~>*|yk~mkR<+XM%4ljEb%v6sVa4u5H)S zfI++MgwFBeK+VY5?i-W))$E>~u<=2NDf@f_aQ53-#oSf9Q^Epj#@60CI#lvbuzn1b zm_zYWc#pIXz{_pH`T;n5R_D;#1h`jTHShN7Kp#Pr>3O%)AdM`U`+D|gOT#mVFl*I3 zsaI0_l+grKnoIHWMt)f}_q%f+dH1_;A6e^N(#G&@{Jf3aUY_xtGLba#7f!8ixIf!0 zPLoxycIr(Shd5E1DXV0Z1|Y}wETLTS}m3sciJ`Y9xfi@I~Y;B@KmpIh`JV+Y(yr)KRaCqIAb5}B;rN3!mh zev!ev#`If}IFBTdEWYakpgxlxu{TY|{J8FghN7|jaFV$g2V zf28=)-z=#71zV92i9oHZC~9^gCo|~{g_s{bL=AqX3F}eyJE+)NNpp_X$3enZEBH5S zExESR526Vnng~P8U&Qt!L09KUE=czRbG8d6%O`FCD5_J)Ree{hOXeD=cDYtBZ;Mw6 z)cAT5h;g;4JKho@Sb`99y49-5&)44;{3;m}j5S{psd9D73{NFQmbo!MAB|%Y zO;@{XtPJ{@R-Kx2(914TcdHxGSTchBh^^aQ617Bkd%!G>#L07Mpa96$bxE#96ge}i z39>j*F~hdW_T7-hWWe(+YXjxaD8gMx606mWvj+L5g()bscm@ki$4zgAh5%pwDZTN$z z=)Q^3r&a{w&mUJ9KfbW*nD85@MoWnP)42Kdi-EX$$0ko+wW;fxOKdAqlu;cJ3)p#3@+)YsA zOvW8mq5RDG%+rnJAs617liqsqQnr0*!pq_G@{n0Etf9yl zcN{*tDx1`yc5$ZoL0G>DYr$3`%%upj?EypD4y zynYRv3i|e4Q>KC%dLdB=LO_ryD*=TD*m2O1f*7QfB|H$Jh7o!&hc=U7*mEP(TV>1a z&3O408e4$oDmH|XCZXhFz9bpu>d0(vc_Q@1;42f)XBRi7e2SKc7TKt%9;k(*M_tLN zx7aAlnSEEsAbRRbR&Dol(`pzv1^Mize|Ju~F`Aw5x4-aU_~rE+L-v31!SK8Mk|S5r zel<-=?4bUtww$f;G;_rv7K_IFE^ha3=vskpQ`8Qsy1e(1566%m+ny*lkKve5QuonU z8O9b7{IGm1At!lP-6L~5Qb~z;@8AQM`g5E@|xx*;&-N~0$!9UH+$0-nNdR3MB z>2MPfX+1%=z9%sihi4xkRXAAyspb$+9UV zzw_i#Oe<@eJsVGOVGj^qdV!e2A#i)aGj5c#(R1w*IWF|-YQeLScQpuAwSQ_Q*3eC| z)0U&GkG9QN1%9y!mh|6yfEueh*9Y?t=y}ZOV@!R`b)?!JfkujruN1W3_MkfI3y^ zg-MBZc9Ou_7)M^b$?N14*AwKN#8N_`O&Nj%=O>@iVr8Bch8BidGtP8M7;ECm*FE<9 z&df^D+gMXe;wRt<&-|Fmb7|;woZ5$Oro4=ptE2*I5<`O@!(%OGu{rz2ekN0DJ%vND zJL#kDC2nHmisiy&#n9xO0~$_N0=v_=w3Qp$ap#CAHKTQH$RVYtJDG@&3gsu(dG%<> z@mI&#FSSY&+HXZZ=jhz+DIF)H3ES4=)r76!{4Lv$AIoMKkbE#^jZ525+n`b9& zJhOAniD2N)=vVXP?1bbGpmn@3HMTu2rdIlol#u@jb;B6nAu(33!Ci~O8@XWypS&MM?m>vE*)=WsLf076*h#=X!-dh9-pvQI_J|f6? zf)>6aND?3VRkzJrS1I2{#%M~hXQ&xh><1ymbIzrd%sEe}{feq(&u?u79El2Ny-IYI zTO?_UCb2{F+i<$U5MT#=LUPbaLej~=!d{9dPc5iEk@jfWhwbcy zG7@o?bjhuqdg5ZD3QOX1Yf}Pvi3Zf!+WI)NUal7s*OBP8%J0H}B<0gYKMCX~Mg3Nz zwmIt|M%fc$jd1H>KAtO&k-0ADYpf_Y8|!X-Kb%?|z7>1}b#+dsP0HCWI|fZ%rMv${ z*kjwK#)zWAQl)#m%s_g3=@h9OX}9y~!8VOVl10OoENI%DTl91K>jnM(m2*+!xy!?4 zD`NVew>*4*PkbWMRAs$jB#JLJ1SDC1#Hrlc>q)0X8DoPU(*P%&B6%0<5&~QkBA4cMUX48o+TdE-|(m zZCy8_U6@W3XhR8O->V*s?U!g5Ds9`|mABpOtlZlE@xmO>;y)r)o9<+y0Kn{`$0RN9 zODb1r#${%3jt0b91P#DRp5pG{PHt|vsR+*98zN6%`W|4{aF5}v;jFyztcE+LFa({pNGf>0qrV@C=Ph|U%##`~|;j`OApzqdxWU||SQ5z<>klur<^Wh1v3Nb zT#;?iJnfceRWrHeDGQL+3@cS#;-d-%Pc5&EY%O?#qh$=dxzMOjBUf8skVmyEq|Ah2 z_QIE+qGjz>B9VaVGlhaZ@WTw(av(`&^Lo&0P-J5H;5fw22GOFyNk*0D(r;vj8_~MH zZ|j|Qk8Q$U6S4I;>8ihYn+yibT~2L${4+SnG3zp5PEZ(&RDN&fBiupKNMOoLh3xSr z)CZ9TrT}O}ExWOmnpeP9DghKlo3VR@(1Hz{Gsn67w5Z_3oaYRl@iywV<_xCAiB*s# z?WBS{ZzgE9r16D`W5PS4qQ6dv{uEkAXdkB0J6f_Yo@<6x)snwR6cFV9Ze_-sz_z{udg`_wFp>ppo{_tS?Z)E+;4wscoM z=^^U}f>_H8hNi+N^3Q082J?~1Geh;i!^;_Yp}I3d^{WGM*Oo7p0wqgHY{SE!^>nhDh z9`DLcn%l1|T~P8tB9(TvPs7WP_~rhUizc)lknOgQvf}U6^4JgJ*}K1EdEp5tLmRPq zd^;H@`Veea-MMmm%k~Qv`8huUyo-kf z@;B34;k9Kga&vZI8-M4Hj`qjRwa{cyfUY+dVJhvFtbHXN zccyr_lQ03+eCaCEzO^Ut_iUb!ngC|BZ|eZ~FPxH2_ii1EmkIQe?k1Ds4E}hLCH%)X z;zbt?y6Mjqk0&DxK1EcYNPBe;m3`*jGaC!U+R-F=%q=?A*mxdMs>q&V5kHbyp8CnO zlpubxEt^L~iQulRv_kT23EM7>1#sN6nrr)3QbRVe+Vk-)YRHCT3aTTDA-8GWBGYOu zV#x6QEF_CR+elqKY{(q9(<#t!o9D-+Z(PUG)#J)W@;L)?ZHpt?>JK5iQDmMIr)?UZ zJ=7nj4caG_?%Mqt3m}$OR)HnzJ2eZQ4yiA(S(>;&g>!TY3Q@{wTY?04(140&XT&Xn z5}9Yc&rH8s1!XZT=ClJ73={Eej&dp|If2|cOToOc z^2`^IgTJhK{6r42Tj3R1z3m)gy;{+UE&DBg$kmYy@P!@i+@uSlHFyxn#QCRk(leokD8_I8 zDxzy~o)=XRt;`T=upDuK?1+(pd-Qxt0u4ICDF`L&N*F!q#|&ZV8f)tbqx+O| zXvH-O(V8GvP3-Y_f_Cvjqsb!iFJNHT1d?THpGC}|!^5lZ@ETsRDB6^1n&Bl*`G&|9 zRb2S!e0Btk=Bd4Y4Z5Fc3##Z57U4WyLR*AOm0*q!P?p?gJhKaum$IZXB&}>N^teP+ zPibI{czl_j?{j}3ZB|LWgYSlhIekd*TTh*$4Qx@&uw`0*Ze>Cr| z*dRe34~xjtdiAU)QmT1iuY_%c1)l2~+p(UTt+j5(%kR4H8e<1P8RWbo@pwoVsbyZaopF@Ek?a?)&ewf-_imNX4eCd~#7( zw;`RyFpA}%ZivPlT8XsqBwR)on_653phJD)D7$9wf~}6h*QMawYd;D*xlq`&bwW() zUrUxN>mQu}vZH0gOVCe|{O(0VD7o=|BSEv9PGJLYxd5uCwB)Z7Dfg0fo-9yXT%;Oh zzk^2EBmjFq7P>B4n9nhQ-iJvH{Q8Sy5~NJMhy^K}z0nq|fe7pP>%+TWyFX?_KjSOj zBLX6*zD|+b8j{)7{Rc?x*GL0{ZTzi)!RF8@G|?)VdgQAO6C$x{jVJ#a+LmzfikR|q z={>^>dX_afI%l7Ok+`~FJ>)GSHfuc3_;l~__F7I6muZNtT`4)@jpyJYxSV$>P0|uD zR&{C81P?exPWhZiqwv=nrE=Q z08w$44O7&5SrHgJXX~d4!oRrqwz37%FS59(Y{ArI;Uj1v)nuk#%V*(>8d6X*?zeId z0t{p47tu&S?@2&?og)sdh_1bcmP^*KX$w)OZ^;@LRoRBi7WtpAwsvh~7DMVUa@=dI zfL0PsTAjauIwo_Gx|VsZ)b3Rig>$$;#vmO-PkFt1taH?mjZIKxka!B_-u95c3KDTZ zb?09l`S5HpJym>QPEVYw0{m{8c!69oD49PE&g`$mHczbm%Ym*r>Hc^F7jZP+18ls(y<SB&|8lFF3KE|(MixwF ztK9UN$ipkQ5@{A>&S|mrOTngZo>vG;I5@MXc6j!35!>#P;P^oASB_U3ww#^2CY+wM zd}#N{LzA+e&I!IXGWdF*n#)k@mX);e$(-PyM+X0RVeg{cVAR#6Q4>aoPJ{RVtr=Bw z(u;-#qs|>2+Dc_=4+k!R_dj|4p9>;^psP|Bsi42gN3r&#(VYAa3Q7`SlM3;*ysvtN(aVOfu-U_wjEJ z%ny^neq0Kb@FfH9MoZ+p3?vR!2?PO3;LDS`UaJJ1Xj@iY9C&+bilT=cE?J z2SX)qR@B#lUF(9EuLtYT#>ZMQvGU{oHLtN>8XP}Jia|CKegKoa)!*TiMybt3%g)+>CMO8 zFQ_^7=EHXmqpoj0t=QHLdk^mWOfAVp*Hut<_;tPs0RtDp%jv=}?)|MTWLe1nZZdSX z>(6+*nee1rDyb^tNP_LgamyVz?vUgkg@c6-#}Qnl2i{u?iKcIivkAvBkk;{QL|8?{ zas4CWxNQ?+(rSIm*r45vNUs$wADwWZsyqij)F;z=gJq*9e2dv9`_NcYZVrv4g>CWyUFhM?fg^{`n=H{C>%3L20R@y+_y&U}H@VyRRwwZK;k z=~+A>H+?}!iD*m{x+P&`ATT7=j61AQzX-IDl*jhLYe@XcsM-+`r2fsw3W%VZo==Id zWDu%EG)y4NoV*$;6Mf+n>eZaNhSWcCY@lw6n?xv{h-0mn`fj$_CLU4HYa$iy(We>V zDvI8WMpIsF2HX{?)0Bm$kUxo<$E3GxufBq?naO}CL?IbAE2pc8g>VX)thxkkl!~Y; zcoV||>(sXmsgcUuaDHxN5(p^ymwY=FL{n9aB&bAA-DVP^umN0g%=%PvBN5NP=Z}zK zCEq>lID3n0{011ROiFOc+@u|kE0~6PoAxwMA~h3j|vSAQRY zOb##F>Hk5J3_a9TS4Hd7N^U`7;sVKu=T!5gi&LQB4ARQ|yo;Mk(n=vY{B+!?&hiUF zQ3;j~ms>xEe30ljm12+x{8}YXC0X_8WO&3<`}N=b*8^@VhqrSMyLoId$L6o4kf7u( z!D>m1eC<5J^9UnIA^p_M*PB0?_$&{_SO}|BcSn`%vgOli&t_vS%~PR%uQd~7A4z+m zrWGXrd=zt=Rk>FRaSZpoFGp?{Tg~@m!Ma6^}d{NdQ^1B%04yO zdvn6cc)1*B_vD06$IAy#teFsl{&1&EZwgBYq3@O5f5@OsR8B=K8~Gf1L9|5^;3OjG zx_JyXc5e-QA?*K4@-y$jsnrb`V7D@BP7>~fsomZbP%VLv^Luxjv3k+L)~|tKK3E$2 zsj%`-c(gE)IiO1nx59C-63HRypBFwG^wCTPVLFy=5TmEs1JGphM>RkiFW2~z26G1| zNMR-7M1tnAAg%0D$?=!}p_1=HQ16&~uzh`yU>kow4F?v!BZpF67765AUR2vFrkD(u zSY(d5N`{F`ULv3^7Y8>ZXLM)`?%Snm0CleYznkmt2r1QQJ~Z0^L9@w%s;u8&y%tM2 z6|1Ffd_)`*3jV=r#JL7rVr6GMAh*FEtVXl~5wQ>ZLkLxZ@$BMumPRD>~l%5ZYK55I^O**|`cYX8I3E&C#V$;-j-peQ5HYj?EOKH&DwfBw@y zhnyO{_ST<0lzVe~w-^8RO6kd&FAS;PbJNFB8;e$4tzz3h9JLg0Nf@F25y+ZC&KGzmMQ`v9N{PzlJ$*t z&7c5=3}M(MU8%+N_5z5A2{FXM z`_H73B=odf6gXEE5!deYg4tGtEOgq6geRDo!T0CUB~O2RUD-v#irmbvyJdK!!~+UK z6UrQaF_8V zWGOGBS~XH;l$(=v?bWZf!$i|_GV)S^$EzqpF#LYXq1{@zVK{*x>-c&6QSp@8CLt!V znM1>EB5`f&@UTIxBjS@00q6GFoF&l|< zHExu6Ma^=fN?G$E_;;$h7f$<-i0GeS@fD$^)IZ07WXKECK;V)(HTU%-<{+*oG-jYB zb)!%pk#%fUYg?3LXNzwNke$HIM6z69N#B=hgtYSxqR#&2gKe3->PbK{? zV_$R5(WaGCxsmp6pGMl9L;X(8s9KO-R2Yo9v@GhvywHa|GBQIMRl)S4tAbG{-;`H) z!kF;y;rmz~*l^Cc!q#zxEqURmqM{#;k6u*J|DPoff86bO5p7$)+lFfvx5(zq2U6ip zSyOq!;o4?d&DQ*l+(zDhxANn<#^9Y7okrzpo>W`*rs~SaU_nHVBdjc(dRt!e^j`cj z<@UVhNBCvUmv_#(7a~P2W_wyxvb8QRY~$mv%VSR>RF*a94FZH!L56wfz81KUORO>% z@bQG#0$1;=x%2-Ujw9jv`(a9g*=^!B!LOOekd`rAWPVxlZ74v(SW*B<0Ez74{T=;b zn*l8(Q!z9!BB^EN+t|?`f=U`8a`dkt99kaCYraKTNH*f-e0q60HebF4kEKO-sg8-3 z;*Z5iAUl*}Ak8L2VAlvGaJ zO=88wSg%4hx@cEloe*oCE3+UGC)9Z-ZD-(_L)(G+9K)Hn(TemGdIyQY{VHV-E95^ z?i9!CwNo!)sIN#WnXSB{)n07|1~}!_KaFSxQHuc^O4DgT?OOI+B_x?>rskHUlu;^> z9}-A1DPnK^JRvswQXaBawAQZ9B5!S57dB}Drp+W`J4XQFsF`wZdA#?_3Ly_$t$b0z z?}i)dM~K7>!8md@?fnTAl$V}9WlZxm6FK^L1TTNdFPQ+l%1C}ISY-~0I3+Afyt}X^ z?xC7CgvY+ad#c!(?TKI+fm2{YsyhVX8s#1d!vrOc@B9s_E!0q%6B4I@)JizGFrSK{ z&zJBAP`?vG>}qk!j2=`#!B?$c^RYE#)uRer(wK58t($pU=yU77yZt`;2yG{USlu6+ zB-G#AhL=R(hHRifJ*>9)6VI#YGo?mC_W8gt$ihDa^NMu#VBX+9p=X^H3(ciQhROlY zA1^~zt4_jfhKy6Ld9JXYA9~JJdO<3vC#5FY6EI8Dd@DQTAeYy$m>W~u7W|4QgP#-< zJ50rMMgg~xHMA+mkpu(wCphg>>uQD5YMjg|O1v|h?M7gS3jCMb@un%DeLf|P=inII z_&Gffc`DNd7`bB5KIMYwTr1=t0slB!hFI&W=&jcX^AP`G&5mebD72t=rFlNB$1q>7 zV6<#|5X919t{X?^7Z2kl!h1br#XJP)J3MS#}fq04G!L8FZ&G7 zpqAWI*S~{H2D_24tSk^13jsraYrcv%2U+_si;2A+WEEUkYX;@>>)h%gjU;l_e<$VZWg>A>uNL|=^2R|Aae7{f4^>N*IW_4cFfXvX$)G-Df>PF(=5Mn5)P6E@Gnwqfi00p`({npl2?2Tl^uJ_d;9ydQ6l{PdjgzPf?^Hys9<$z>KM-CA3qaP zpBcE$R9nJ+5bFyCo_aLKu_rFsK#190m?lp(G~Pow^VZ(N(tmAEDjSaIw2@C{E4wl; z;*IziqWM^+V^>KU%;u^T7)1_v#jg)r`yu~c*(lh`ezQ+mvlK^~-Xo^N|HF4A!6IW@ zM1%jY{8=}+19*6k{Ar-{gp7ia%D)jC38^~w$SWGpVeCNef=x8XL}TxNqzB#Pm(ksJ zm3M2H&Eee2r6*SGnv~g)I5_dz$(ap58l1EqeVx8T?-6&qTAAwNXj;V0nqUp5;D9_nBjyaylB4s1A3?IH{$r$@p1Pz+~CzFZv(h^n%`lw zcUkx%hb?5OL?KxiP#A$+_s^iN|AV%Dt0gF=l#ZmS91u9?JR|xF?{1&Le^fzu|KH|2O8MsG}qt!|a zZVopQ`DOK|6V7Nrwz^}F1p>Dp)Cy3H zv_@f%wmI83)J;VRTf+hGK5RQqtm$@g_sJuYvZ``|y>f#;O0W3>6==h|@61Wc`dv;i z`l9sAd(w+01)~m*4ow2AQ3EiovB(NP*mDYhs z=&U4b;R!Gk@`h)~?rh!uB-iw1-Dm7~+YF&<))JhO37`Yg8`A8+mhzJ%1RIQ%L93ZT zu9rC^g52E){1F=XFvh?xFaNwP;Y9+8jt3hyAnQpi(Ydu=t|RZrB-J5FuPz z!xWWVpIAoW^!BY4cE^#hqgysUPfA0D@mIT4KelK1#>s+?$kx&k)-4TM63kzZ%ohus zj4Za7z#R>Iu^kzFm?iSCMhdi%NQJRIb`e_|{P&T7_7ylGS?);4T6P>ZoF)=Y)+e~# zE<_a%{6>Th#2HB#$LL@wc&q03dnXm&o=4nTa|2%98N)XCb;)W^k^!|>AZB=J^DVS{ zpXnopqIfM##>ds85bG}}AO_~TGg{{3x*Pt7?vct2lyBi$3C9JRh(Vr0;hL0ky152K z!%>Nlbgk#yuUC*I^p{fn<(>Yui@&U2EQU!a!mA`96Pa`8w^r8!L~m8vQ+;r;P>8>b z{h#v)5-F=cpm!2vkY_;lhnD%p$bI`pWM3K*GP4h{bErjMVsRIbkW3O&e&CloSs1H| zowDQ5+8g|;kpxL8(%H^H45T5Ak}cCZUa!@d3XoC-%o{WP7i4N74%@;Jfk{S!dQZ z+c~tT*&Wkl@#A>ps+zKgQENzp%Ta}p=I~t!kx0kOxBK*Bq=^iAswDeSHpepO^qu?^*Hs3CBDh2o=`q)<>_Rjd0G#?p*)b3Q>4nn0(nwLNcQQ%&$l0-uT?Z?x(Kma< zUAF|ImX$@_mlrC9?y)o@voyWv#$eR-Wl>k}3r6UheTHw}?A06BCLAv%PnrZ+V`F+BOLQB;0);nu*VqWJy4jJf=#>IF(AdZR#{YW-?yhV$<_U@>31`N~nVk4If zG%;|KdM3$7w3Y!Lot?Xtz5iFR_y4!Az5h!Qm66;;yL2A8`6s5FDuG+>-O50+f|ZeL zONCpchkGPqmP7skjtsaDpeK5P5gc`r;3;vB;Ts7Q9Ra=tp%NPx^mB}auUh{CG+ndV zMecUkMS`g-A1zgTc}3F=er_$D$}5@yiHs1633^OXI)ce1XvY%@nKfLwuHV<5wz>lO ziZD(^w5B2orxxyu?5I_;qfW%yny`j;IZWNJElsF2X>OdNOIkLZWDhu0e3d3$_r5!# zZ74+uZvPRAs#?EZN|j-=dLYJOVk?%FnWin^#NxvEO!nouMS_8p+u^nHxg^`O}5 zIAFE~5e|y&C2g5X&QnH&?A4T61p-hzW8B0u;4yol+Jq?9s^HBL5x5m)XtA{c`nYHf z(U1CW=2)mxKxQJ(Ej3`8Zw(R-g;9~fn_!1*Mo+KW7BYuwNx@ z%u_d(?AT6Bk`D=FyUSvFc-`rP8#qCyS$uflm~CRaN<%ebzX^XF3T#s#gw*$;5~FJr zL7Gvj%5%w6P>m@6P7O{;efYyu@j>(oYwLxFx0AcfhA4XhW} zK$>Ii9R424(b5XV)WoM}!F?0SGpl-Dv<6f0EX9CEQ_jA)=T zNez``E*?g~vKmYivWcAx#{5WxX}mXSD4;sPfADM+T4>GKCk&jx)>vyJ3r1$qGbjT+ zyEIr0|E*AMism;qM4Xn)bC<{8J6XolgcFar%TVpOsKDY_v?7DdG&pZ^jq%q;YL`DB3A^MV~fr4|#6?Qr$ zwo7X4t}9n{3Hcgc+ykl8(22 zBa&Bo;MCe{9MRO-0S1SJ&*+WYPcK_geUdnpT(%%P1}_%~o`?et27C(hDb{TbF4-=6 z0jPg8uQ!yYRO=8teV0;yy39vd-{o&UB7=ot`}-_~hhB5l(j1X``^5B!43Dj~015Bv zwT9OTB=bODE?`VjLj&XHm1D7i02gi6w_tnPB7^)aq6~6vWEtdVP8sC$8#i!1^5m2I zX7w4(m2~~fm5u+~M%Yk5u}4LCC*}s?QyDmVr6Lh5h~V=-X~9OCO>H=52^J_ht^E*k z*#!hqooqFET%5XG=uX_!0(QK#H+b*EP2>*j!><8d*wVUUGc$Hi_VcIE$L;4on|6+w z@hTGzQ6~MZ_FEw;#n5uwIlnl}*vWrBs#isIQwMU(?z?5aJiooeJI>+_^e<3~c_os0 zn}r7Q%T*oOWIw{HOrK)4kB4`VKUc^iYgGHa9Hr+j-PciIU^p{+Rj>h!^WSSf?|QTY z&%GvUR$<9i>b-2?gXI_b`6ygcqLrGOZI<5ab!aLDKR&3#T1BCdml=TI)DQL|?zt>< zn{HcDM6_rs3kmyB(yHo)Udl3&67`Ayt_Smn4(93&mux5U2^He}CBAOTOKbFi` z3B}@e04(2?7qW@Oqv=(qvRn#FFR^Hg{L?n;1X!>Ai+Tn& zq>l^V+A}Z{|C_ZU(D>@`a1Q?M)iL3ldM>aw%ierZD1c@GB~kC4yxZ{c|7He}1tqsv z2l@!r;+*`dIgHmX+Zvc4PTij0eA*N3CuA=IWFd=4?vd@`;@^2~v*9)A(Js4o{*!kT z+2+$OJ9))kMskbtGfVli%zrI#pCEx2(=ir=(xc#<1fF2b@4`y8uai9qC1G6%OfXQB z5|=c|8am_cs9;E+K(xiCP>D)cNRa=MrrHBM)vUj4*sV=-i1#|(G}5O zoG#>JdCJv^jZHdr2n|#azI6l!O|W&I_za}r(1odMd{mH#tnTzjXR~%}h(8#U=7-L9 zKe2!$nd2DG7LcO|FREAZaictjt3cMH#=`KNVKS1nf)|E&W3gh=J&LfbJ3Fr>mNnlw zQVeDWa!sufIJ9pqfz-H)WGDMWG*||dM*tE?m%{*z=9NrbyH{ta0i#Vq8!OYoiEnj1 zZ=lX_!aT;?8VMV{eV1gQPYrH0@mbxY3!0okC>In{$&F`y?8f(e~>GA-OWk z$vBO+qaVyfoCE!T-%JE5FpG>AO9gAef}{nrpBGr2#x5x<&zF+wjg6>0nD{cI{E;x`lKs$*y~ z+gr}+LsAiMb99z)y{^Py&Y()6ik7yEwOTOzL5_H`5vI5NGMh9mc2?2`iqQ?3zL%iL zq*%m$%yp2)TFgNq*tb++iZ6HyG(4+}uu||cpkFEEoCXag30AV(oq}r zYT+1(1z~g|az4&KHI^6I!m4bigEp|Pvmn6X&u8UvZ6k(Xz_bB$j~nikfVT8Pqv>9u zt(XD=I`MNl?=OZG(nyHeHA>#a@2#v+l7(m?Xe5VydCfQ{y=6R8xbG{BPrncgROgJG4kM!n(5Kuj^ieD$9L^ z%0kkeu784*U?fj5agDfukJqn(gGk*Sg@=)e!?bo|SaAyMS#?1dNGd(HIqu|04k3+S zydPnaAl7Q$=}-7C65!d|&%gjV`l(pZGaDtSqDfQzVFjlcZT(}d2F)yQR!_5Yt&svw zHwsitGJ|U)47Mj9Z$@Kv=qMcR4C3XLOy+@BW?r%`inQO!1gUs^5B9e8OT8l_55Qa6G+rOVz zgfk$`wqispY?K%oGBGG2Kl4>Wu8pXKJTfhL_!b`}k<1@x@ z^rC9Fy&jceGhI^1LykH&l4V9LV^ew>B|@_4k_?M&kE9q!4~SxTTX)ILZS3SIbj#+A#umwnjVG!C6;v4Fx2%#W6%W`nJhhaYc1xi6 z_X3nfSXMpQkb$|GwgVKL8kKA|>4$YYGn3~C|L^vLz6dbrAu>V(4}StSNU>yk&0kP$ zZTQG@IB@nF;vc+5yr%CDULyn_Z1cva-0HY{jTF2mG}$z@Lsvc|_83JMXdZr2INq{A z7wnty|3_AXJ};*c-aXB~;lA`Kj@Iu~F1}W&V927Mgxk65mDFQ}ps#+Yorp=a>G;_MFyj@K**xJIP5pjCqzi6EQmDz6!ZH&1WBet*|r@5`&mxc9$4 z8+k&fmb%HoE4yv`*^2$c55G43(EHa6{M(Vg>Fv$XrLFGZUM_7r=xs$ZZhnN)w$J@b z+vXPZPS2d1QFT{(QGPJ$@zSVeqeG9NXLeC~(db~*oYJUUrsssqqN4xQHTr4v%+45B zxNltH-n{VLQPC5Z1j?z0c7E)Pp0U@CUA5^|P;wRZA8`-u89iefM3=VqPZI+1eaD3* zii2fjP34)P`rifO+Tj^la!X986w1hk?pj5th(y-l-m>2o1<}VBL06kCAN6Yi8ae&y0>PH4yA}T zZQ1CiJsl-&i*jlT&{R7Pcyn#T{$U$MrPcDmeRmFKS>kW64?Q!IxT(`G?c18$xic4Z z6+eCAb;NB|e!et(y|DN149mzHMibRlHTR6M2i}$5lb?yd&F(unn{U^4eW()&9Zwm3 zgh(JC5O?#S?fpqac*>)NXqO9y>!ga-T!mM{QmC#$3J`V>R}3_`hk_n0@FJnJDy+UREUMAhM6pqHWZjJ4!e+)(y)b z?=(4Lam>qYGOS1*g6yQ|C!GB)l?IY|MVA?C=w3Rt1|OLiAP0g=V*xwI4eJx=^R6uK z)6h03F?kZfl!n;B2@Q^FwtK&VVTnHvCvs!{+g>(f4DXU{=yZ`^6Zddt7e&wk$H)?G z;@uieAZ_lu?SbWtAe97%Olf38p=T>O;R>z79=HU7fY|Wl{_nh=>MJ=1s<;Z7E0)K# zbvY*~R&G1)*7q$RcrMNM2++oVx{hi^M>8g!XIu(Dp zff@{iBFa!j5?}Kn_y=fyd_C)A%Gl|Tj3V!t!bC~+n?J3s{6!Bz_8@40kqZZ?Hq2|J zAvlhMS`~#%p5q&{qrpjgDca0)qOa|JM zN3jhNOWU;R7f_h+?AJfn3W_~sx~CW-F((Q79XgWC@Bwj#!Gc^!tnsp1TW0zs&QOLy zw42C_$$iDimIW*TdZT27gk5D3j+iuI}l%o%wj1-74!g2ur5zTUtXu6iz z<)iwc^-_F8Xe*IvPX{gmd?UDq|MJ2!gu#E; z1=4@y_=2>bmDQ26e{rW6u4pT68y%_?&9t+(SJZDEoUpMprzU<*dh-b@x$mAK7H8My}$R! zJ^~zPTu{@K>#(|{bvsXBj=p@1NMT&c#{v;biLDE&T~WFJNvgur$|Xfn+PY!8Mn`U` zokCw6JiA?wX&z%|cD=dX*MA6}RtaD21REn*I`!UcJgp9h^*Z?>^ZH4{98Y_*)Q3(A z=Hvq(hPUOUw_YNA{#)>J1iic!yDyVyPT{3GUPrBAU0k2|C`QTPzJh|@Ws`S#m$OmG z97!)ZHRaR9+`D&9R=OjrZjlG4+k6QOfhdq$;`g9Td#4eA6;Xk5tWe|uLRgpwjud1i^K zD#g9wE1q;xvyi#DHYm>)HfbWcXlO*N#aA9F`z{<|V;=j3zCyfW4SnTr!2+I{FF-R2 zBgyN?ZzFr#PiyOLWWd&b!MU*|k6oz-TQ;HdN}?kl4-OO<+@~8G!e^8kI#TO$ctxCt z)#ouon5I;NW(=)Lg`^BiF2x3{rB+k|;2OlSb68Xi%gEWck_8@}&PtY4sw}aLHEtr1 zv&Y!hoO2c9VvX!f(Z6VoAe>9{ws>(jxCDW?zwXx%l{udS^Y}T;@cvKn!{Wsv{d^R{ z?0I*WSC3WNFZI-#W&i-YjV~{0z3oALft$e1bkVZa2C_CIFWb7Y9*@aif3l6_r_L+P zV?^6ctFH&xx5MI?+$N%~_`1R#H{!{x+wlR`6wco`Q!+Tle=lGJw!etS*K-vBlAB`V z;*89N9ciqOMADOXQWFeYQxCw#2&mJjJk0*ZOtd8eYWGSia$>ZKSn##^XKK#JY&rCI z;7#W&7H}k;7EFd8g8lO0;ofXwWus+%TXTjec0brA)>iAs(cWg8at0XuFc;j2y+`-m zAM{Qi8&lYcumwARsA*_+M~~A-CDi)#W;-Q9Kkapw`SjC{U(gU)F4-x9T=K%R^m563 zS?)FvI?2pnsCrPKW)3>Cz0y0j{htYe|Ht07Ku1+wZ5<|(L?sV`qCyhF2obTwH);zA$Rylqn-AXhrQjz zddXpjjpTOPPj(kad-qRVUAU%YCyXP9_YAkE=61|n;xE`7YB|-PT(+!7WO5>k@s6I= z{%@&rbc1li8;Sjjp58x|c(Ss`MmJU8%rGcx_tNNc*+=`j;r|Qtz)-=>Ir8`=Uw9L3 z;HEr?wtvP)Zy?&fR=yx|F_cAV2k%L8{OcW?7@FBC;P`2fl1yT&<+we+ddnc@1cs-{N<)%?Eano!?v-LC3)36+|P~eo^nPV8;UZzxz7Z@4Dxi}>A zZ&fM#6DcP%VrT%~B(xB8?n^DSAE)z>cxZ)ZWG} zJ6vEtV%Xrn5!IEw)l{{b#*yV7WE(EhIuhg+8UE+AUQAg`N@T9S)DWFt&E(bPk&5;m zg{42Ct*c`2hrw}eu#&7=l{jqZ%IH1)Sas>$o6WYdDebLV62ZlZLSu3wGrq4~J;nWC zrKzDl#!D1Rm-rhSlTOY>+l83AX6F7TH@}L8bMU;8Q(F1W@ZXWGv*G&R^+hT-v*|JnBRPEbXcC0Ino|IB zt1}~1c>~zMKqyr%>AhbV^*Q$bMU&`}bgiNwq=@a-dH`Kr@2To}&=LWJaw*!8r)||M?w<1Ci*cl7n zrC70U)$+V%Lc(OO+(NT)g1NGUAd-5P5gIU{5m}p&w7OA}w}8$n`j5DD&?y3XD-&;P z*d*sGP606`CZic?*&?%p>UuXLK4k$IAUSDm<_2R2ph|(YA>_4vZ4_7uC?OwNSLqr0 zL}LNYL92JPe$@>12Awu051e%5v%F2`RtOBNXdux4z( zE;E;GOBkDK9(9S7ZO$gPbpQh+#s;cMQFZBDsm<50EfFx;Hu@H%Oo;*YG;1Ij4k0>n z`0J{%kYi)T(R!D&AH?V%YRGmX+M41tX8V3>$jQkEx+dK+!K*df9W807ck7=#u7A?> ziEA3)Sljq|P0Kg_R?p zr*}Sqnzswrw0t!@DYJTE%YFXj)k_w(-0M%yT(Z37{o#q3#cNvL#_#T37$vX&8Swgl z&h+}9ddKx^%eReL+Hx$8O~e21mIpc(3#U!MZdzNW z2r^FA5~0SsKR70#Ie1#nkCCF|RM1SA+dB@(esq+fj6qvCG-~5<{%SYT7?HC;#_USkTn_SJ@A= z_Ze4OIKJZ8W5$(UumnA0KV4Y%^Gwx_?IDHqF6T|}DFjgmryY6c`)fy1Y4_Cz9SYPT z=s1PnNVEQP&@gQb^;(Y~;{wR(RUg1|oVu>R3u;H4BEW>XMcJ?)-bcg)isP||7Ok^; zLH9u)HkMXnR>~rXH1=Tk+^&GY&WxannYhYY51YTpWZ3QjOvcL)l~#FNx6t9TpEm6r-2~OjZ2mcU-_Iod2yL zT;47LSs5tHak1uINeM>y0MpIz*mgz=R#MHYF`HC?TCm&Oh2887F2fF73=f((WTHOx z7JOFyT~omK%=qx{@%bJ)>4Y^Ps!IASCRR!-1ic)O(hE1gDtK*eJ_0F+CZe<2&e1Z> z=Aed%+6QFc=w7%628wZ4oWeB^_mS(rn%C7e^QEOp?<`IFD3!>alGOW4(im%; zWsp6U``V}e;TuSt3~;16qA8_f-M*{3xBFU1>16^dp@hWbO)O+1{z(fqln+}OefK27 znY1-ZrYs;{^CYZz)Wsm%k|cXQf9Pka(wr#D%44;t)2l)=9*xrL88Cjo0g|4W2Et zh_6w8`^uu!c%8M!wFOAi>DiX;6Hm+kb(?w@m!5B^HG0Z&wZfsZ$$$j;a#)0HKAmkw z@_#GpxeJ@FFl;Ig-dkiWK4Z zk&WQPiXto(F31Q_y(WU#g<;~gnoH1BiIllpTQ~4)xz`PbkM|gR(B%JF7myYhI7FiR z?>hb<&p425>^vn&+x~G9AYIyRUFUZng zj*Z%f%>5DD{DQRm4V{{gvfEY+CjKIizT!X5f<;(t#I42~;db+if3ad!6W(yNY!Yt# zSJZ5~?P)FN3)V2~u9M#cC*=!)=AiE@3jP29+HN33_L) zBLl$|*BcdVoKTtx=WUHx3K~f-n}>SEITup`&b)}aM@oySd912PSxC!7v;3^uD3nn$ zanf3p5$wlfV`vfy1STRSv zz%dzmLJ5lUEOR+g3xxRyh_21{hsT1rp)23vV(FWHs_`|^UJ%$f?%<+6PUYa#JhUwz zO^{YPG=Q>gv^rl1G>z>Iv(F%tczcucw$^jrBF}_*D<5EM4oD8M*d1^FpXC{~vg=^cNYkp_-U2^k(LcJqfkl?x-o$F_oOW}JR+oyf zlrvwwhCPXBvu93cd<@Ar^I0+{CXljzJ3~k&5@-r9o6oQ~WU9OWbl3Gc!x!VG^E^8-0?T&+)CQdSoOg{C?erbGH6?nabBOR!pAYi)^`-YKIy9PDR0eL z4Q?u4L0eA0PrbXeLE=eZLPLa+A``Q}O}eB|tj6$-A8QMOJ9<_pyOfw>M`<0Y{7gtR zVh41z2SBYU!5|ZD&E7;CsW;;jZH9jdpc+Hy8As%_J}2an%{_Bkj|~HwuH6B5UaD2& z5w(aS&DZ@!TRAP~X8^VC(&WQm?3(9})DY#i)p|IO2)&e-^N6`JxH*qtWq5vGd?{od zE4&Juc2PS3+LrUzU<|NiqKL}q0Dx%v^VO4hHydD(U)z6dWYy0h;mD&v;I#yZ-0B;^CDK*g*mBHU)_wV@UXc3Td5}PYNv=I=3P7zF70)1y3hQMEDxyfol575LG z96!tz(X=(pwS-|64S&0fH)$c~9Ey^)6 z*T;jc>M>%5UnDUO&UU2yhp`=blcO@3-*(1Z0_ho2t(dI2jIoBgG0tjy|7$xUXPiP$ z|Nm@9irwgYJjRFAc4X4UFCKV?cW%-rW>tV18MIH||(~9|zEnAj}8_lOciKTtvNw$q8Xg?MD%|F3>dX zoZBb#U*nO&CNF>gLu)*;Pb982SwVqs#lEQ76me(~V5hfqq|Kvvf(S8$_A1VFZqtZo zaf-AL(}V)D#KLwN%{Ax{%tmI!t*2s6t@(LEWU)YvhzJGLNG+x%dL&3^A4D=jT(!%NNs=5vWIf0-Brgkz-K>YeAYVbnPt&+Y#7PPIrPeN zP4CJ91^=ZKRJvOMXTLm=ijE~V)cNE_Y4{h|GVYvW;32bSysc9cLTyPn8fq_dxSgQR zOlKQ$nKQMHf;ESx%uiWtE30MzWoKU4porGju`#$;!5U#{Ww5OW+*$>%w)082 zSw154gr~(wLDu_k#5fWKNew1b98-wEDmZ;c$IZ)_;xz}&I*xqOuwy>)^cd;Rf90(w zCyOC2)Q>bss_t*HCLD-sa|(GSJ>Bly8&y!Xm;b0^m!D`UE$IaYhPGVd4s3%bcw%{o z2-Mc{<`05?{wzs=>Q~ zhLVi368@;UbEYohyr)yuwICCbuw*n*1Rg9=p3wzCvq(1(w%0w>yt?&15&S zzw$Dv-y7@;v5B3w31o_FGvMW#iimDFr4IbtH8zfAa674#Tam z3#HAkkJaRb|M_3tGj0|YlJvxvE#xlHIqRJ}TgfZzmnV5EMt;7T{g2$oyScryi+XLR zZohm@W1y}vXIabIgyh`=l3yK=H2tzQjeYAHkGVcI|EXZ;#6YCSkr~@YW~>~UzTx-5 zP+DQ==s@JnHB9*+cbVCsBj#&9cFEdj5l==XQ^ox1CPi+rka!_#+EJtNrhQ#MKwAX=yupY;@g6qah9^!RyQbxn;fdQ($<{hNY5TnkTSEBVy~|s+4o}Q_ z2PXb!W+#;$39HEK{OBx@T$b?YG4Fgg`^e$X8zocMEJ?bh7wZ#0Sy=)nf53;AmLwHy zjT~MkFCW(gT0d3+!(fN)$ZpQ-f6I1 zupeWx)>5!i!MF;^IWHAxL7)i=vAM^f$VUf#WOj%H)rREt{$(e0pY}ZQb8=Uqug=?@ zafmdOBhpFm6_0SkQ@Nmg8ELfeh2bE12L4>@Ao(R-?ia7FDLDK%jY?g@@mL6khDOaRH`@josKO7(ku!W%^p*vdV1i(R z_CxU2zy)sJj{jnhw{2fy&(9&ZKQ3^CWK;a!WxJCO`bM7Etm2Zf2N$?OkD@DZ`+)MS zmI60y!*Jf6Q3Z5jLhI1;?=%MM)~gJ-zj`bvaU|pN|7hE1$cfv zhmHjJ-sB${1p_d29Vj2~qV|yi!xQwR8PS9a+RSKTG0gO*5v-8uCdvEBpic#XDEBHL zvMUzY$fziuPYJ}CVu-LjZ2ErjGR6TSzbuXlO$;q9?^S%w3QSuVuI2)V>D#W-XtH+V#(Av?FkE81pj|{;@M6IYMGN%v2;qrG+RfLKei&^_w zsFlDRKpcXB5k1nePl#nyh4+CCg#?A2KEp_|Z#3m&n(u={Y(ipW3bgPGHEU|&2+(65 zTNgOMFd?yF6Wk>Ax=xtz$Ns5R+E=jY(o1@~iodXsAd22ni~Fior3%!6DIM3bJ{vVt z2O*op&x#YQHa`P7F z)>j2euMPR;SNU!%4o}6QJ9Fz79q*qobYe-;*l43K;g-~dYnJ+-Jb7Y?e^N=0vC(IJ z3CB!a(RkRhmY<@Bc2?5#OV>1ZtZn?dre$nG^3bg0qiQej7 zeqL_gyg=#LkZ)m8_*QuT-kln6{C7%{Y!@h(&dnMem#J>gh|NkzAlU>4D zuP>ukZU2_7+LBgZEV{Ra-q>(IA-l+IC?-pM35JyR?Dqa#MOx>mF|cxizp#T|Da%8pWXXa z>Jg9^N#)n$2K2sR?VWpBvKXiKt#u--HY6J3Nn;T_iX_GG_Xsz}a%tGUVLCtnuxSq< zYKW@AXi=7>Y9^;gQpDDK8mOWyMp|h~E!jK-Z^Kam#)o7Ts1c{1WVzeT*p)py0mLWd z%0b8(rYs$kv)RqFt8bO8W$T!qo!*r#`8ZOA(&KIC+|+Auf1*_Y$68eW#_K=EfL0P3 z13b|Ar2f3Zuph~a}k__1oS!!iQm~s$3-9 z|I<Qu zvfAweMAF^AY!rPJ+;o#<3rL&(7+xK>lb0leh|3I}<@70iDYlL2*!G$Oz7hL$@6}3s zrqxX%69$`3*EJPU5n_Q_QE2U6i$Nd9$YI*zF9YiS;o!^;%U!jZVmBf-XoI2T*9Qcu z8cMTSNyh;X!U&t&A}Cy6+sM( zZ9r26CdQ*JWS)|<(W=ZWHl1b$Sz!Y@1fxSo$;ObyZMAFysDcLS9NE1-ex#fG%EGRq z*ci$B_2`rXvxIDt+&rz{S)FIQjtf3R&quV5w1v|m7BkPsDO+Z+V}T|#l$0$G5?aa61TQ%2fsORO^yYV# zlvFHil~HJ92pmihc8jqx{p`(IiBtnT^t~+k4w6dFc79uq9%K^hTlqX_h86u_>7&+~ zBV^GjDoLB~p;i~XR@hRdoU;To z%kspa#;%+raFb?MTR@>;giH^8?bFWTfLI;Q+^TQ=j5YQkgw(suOfSO;dm@HIFl?;C z+siCzQ-jPSJZ%h!u^HActg@ZD}LK7JYKM%Q+`T2*r5b^H#uP zby+igBO-;5+s-fV8kTog_R!g(qpgfW?5#j+#_*$vB zU)B-=8<(B@YIf4}scRY&>l%+()^Z)N*qR@O4i7}0PtI6bkiOxuU}(?i&}Rc9%fZHt z$mlyFwP}9xp~7fOw~r&V)*|ixrH^D8irZvg?Xnw+!v{~frR;(*@;I1KCiq)t9zT-U zyn@3YTXdlQ?(9g`VIxb)Msf>mB##wt|IKLAwzFU2cBmz@@Z)nrI61@YtLU@OL&E5NNQ6|C)sC~{wbi|KEhI}e=*9t2Md}^~N!6^;5G6yF}FB>PvhsuTL ze~Kt-D@YLH%q-BC?}`@N&G@k=x7ZfxGhR*I)sqaBRZBRFkW;BAE8!?PZZ<((UJzpp z{~;Ukbd=muInSRa{r)qLNykvqXy)G|o&TZE9Q!f(fSkyv?-B*c#OvweHBjH?<%H*W zL3zMHjT;5Tdk9$^c+Stzw{1(?K2abp6lzt@BH&hBT5QB(&q*LxTsj_yV6Bmrc}aF$ zq=t*EgLkB@zU?+geH&~OKdCX$-FD?F`8!_sGb!#j4Gr964@Q$KnYvaxk{K~Uzot@b z`p_fKP?kBDc=NTe2-DVqiG>#s4ZE}~wpRPIZOaD!qL5mbntdTeZvm$6Y;9Gr$Hn;gj`P;+r1 z!IS1_^em-6YKZBb;#Oc)G+*^Z%D#u^|6=5e1nJ07QaX@OhwTZ^uo(%3VdP-JNnA4o zRw-F|2m(HA@FHk1qYf2B1mL&{^FX3jV*?fQd|)Jz%hlXdF&wP*(ukuHxoH$Fgf#I^+(c-zBcxmUKxQkpJ=G zLehrD0FP17XtpiP>>|rgHK-EHsn?{@Y`fxAK1rn^OrXQ(5o6XpV)TbI2D_G8iC!Ju z3tC^1u4FC(KVXaIO>cG@y&yH?P1g2U{2JQ6_(tb?WCqIYbZ>06B-b=&07LdEgt}Cx zKs9#A3KX;L53dk7#jVmyqZc9XmTt#1ip&^jD@*1 ziv-iD`Z;_lj7^$7RH>4ziZgkMzhj3J+s7y*V2Wi8lFWZ*iKkeyVP`7OYj;#!-uBI|^3Wvz+Ic?@`IcAt?k)~j9+O*NnVVM` zD7`x5yRyo6#rJHs{l*LH@&bADbL(#ll#UDe9<1~&FAA>!tyUT+EeiQ=tn~eS)~INe zFX4~95*|TIZP08VPb>j(b-OQN$`XGywb&jC-D1-8@q$yUX?ai5Y>6ACEy+hP9vL(- zdN(?u1AqcF0C@6_?0o4;G8nET>?_$LJCQ5izNz+i z*Y$`jhwK0RCH`*N0N_ge_?sS)D{=H9bO#uK&wR$J_YF6Z(>M9gt7J@d)405w8B<-k zsiK_3lH`S7{^sFN5#7`P9px5bOHzPFZNttZhm!F7Z_c@EhF7QU;jciL>7kb776E5o z6+VXzHTo6~npVi#%ZvFn_P4hxML#&HMT$`@CJx?j5?oH|i@iqDmb8P2A|yqb3&vQy zNtwFKiaSr|?;HdDOacyjHNK1U`sBbV3{imCz(h&wez6fTG~*ow2#*fIlfRtAp)szH z{P5zNb)1)tq__OOfNOK<258|L2}KAdI$@!DO}z%Fvz+duNOdG0>ik8A<&~XKfApx$~4eXfs-5 zPU3rUHoJ}x$eUt-ycK%Xw>!k-E9q8cw@@Hn%ql9q`vKb8NNHF6p~aT2{SZ8~or6whQ3>a> zJ1(1&UM5{huEvj+p4z}-l$(wxb24LfJ$Q!fi;A39XeDSW=G}@@SRiJgE9z;n{(Npb^bOPqx8LDpnB$~Dtul&ykeeBAfqj_|ZG24~J`1x<`8?1<qxi__O8F#dD=JWHNx1+Zy9A^}HjMEgpH^Tw;Jk zs%SRB&!P|qn@_olV(Ach;d*Hwx#rP=9Dp4|^dw7!P1?sS6{hOP6`j!?!M+U~U_LK~ zV#EN$g}>#|a}FS=Id9W{&m05t6xpI!(xd|%!zx;|tEuH&Pdy5-$g?vb9@AKap&&LE zeGk`ORFkRFN?Yt6{MWdvX>-Fr$R5kPufA7@{>_Ad5H-@W>bh>!Tavpp{qe^E!7j_0 zi+gd4>J6#6i~tV@l_F~pa!*+*X-!1PIci;2%r!;j5$F&;i?`amE0b3q6Pl^S1K-j0 zLnCh!-6J5|w)2Jf@$ZMO zi}q*dFB%YfZ*=JGfsredGq&Yttjtf}@Y`T0GB9!;sJQbork&S2Tm(_%&AE9u2TDhV ze4{IU=M{x__Rg*URiHE<#cH$0mh32w{t6<KY3U?ajlv zg|iz#`*d>WiG79a(ark*i4m{N2q#??uSMGvBev8MFBPXp+hZwTvF(`Tv7FY9n1quS zJYDd*N{1Wo6|Y>%0Lw-JPHR)WbT zI0Pw4{;LD4O1BjOoTcxnUm6tm;Decg5C=EfyyXXkVe!~WXJ>baSWItV+ZHH|BRP} ztmCz{@~t!yNGgC(5=auh#i7kaxGPE~^uvc)NRPG%A?b|A1= zi#)Y!YPa?5%g7_>id4#8$1712nP|$txZn*k;E+4IHufO2d{E|wl&p3KbFv|>NDbJg zrtIU)@=|>Pl!aC^s4BOc1O|{I7&XapRZAadUQ7`3QfGBLk`QQwBR{6qQ~{J3zgnjq zYSU)(*L7**jF}7OO~uVF0U2u3^yIS2Y}~bPO|!7McSUG6;Od-cA`gg2_UrF&NivF( zQI=efiCa`m3|g5bVD@P%yIgSSHf@ZJE!V>|823NHT!G8DVGE*)E?OKJOvu_O-kB_L zS)8M}f#y1bj0q=rZ0nLOxxhKEY}bUB2LKj}1Xcveuxh6*8P@};eHQ?1o3=n|D-1=b z4XK$vEnvvRI-q8lb6tQ!5=jGr8(fAz_9E#uyjZJi)!oZonze}wzNoa}^(Yt?il$C_I(?McB!TyHK&Nr=sC3f zv3W64erdNrmA2X1Jim<9k4GIk_2VP$8FzX(j*Rni968a&ab%uw9BD@T>xW{?mo*v4USue%XJ;qpkT3XcHZd%b^*>l-QkP ziLCKon+E%G>M3}(gyGqo4IrGd_3%ruY1a}2)ja9YS*?%IXkP7?)k7bC~a^*)AHqGe4pQb#euqn%rA4ffD z=F07CnbK5F!YL0!7%G<#J6hwrK`m;#u3bGUb?Xa@3;=ec`kqztduDxE)b2^8ZO!^9 ztuPLk_Onvv(oW6)HLey4-^WiT%fH%W|zJW0j7y|KOX z4j&X>5XunQO`BlYZ5QzJTd6dU7a7B_lMX?(_~ydoz=lWpUOcE;wLNZtuFc8t!uaI6 zgx=e$d$-PIedFq-J)YX&Xvc<={_UlHG|cX`p%PI351|c#`tf^fLqL(KjG--DP)S4R zBSkt;ZR|1HkTd3bYC}-)C1Pb~V$*otaE=Vg8_q3k+R}?-nu7eoreP_e;S|e1C&ZA% zG+?XNAP5D;jE=5ylh>pMO;+mnbq{oO&4idrQsov%0Pp$k15@AX+!e6m2=Oi|3y-+F z`#{fqe3A{%Sk5b3{s(Bpn#-b-rlgXKXD59A8#WK2-Lmtvlas6+*wBNyytdCC(P}X! zijV;V8bf#gb=r=k~0 z_W7L1f=*gINpbeh;T<{!n5mp$1z21P2pAKArxg>+mTx9yB-EYm5eKzPL__AdW|C;# zD3ai&FQ1z4a+HN-h=`CiGarW*Va4I1H)kMczQ#p7kwumgGb&2JHj{)gNi2Iqxk!?8 zQd&qp!dA~sTda%E!JbCuC|WQBPkjS1+uqs&n!}gEryozSD3jD6B^(mIBfPlcb7^I! zOis&NF@_#alBDa7<&;dIA@m-xYq>eX5Fn~HG@3bvLm`8e z&Jn4Gp7dDDS#0cI_AIBQt|;cSKMLk+zKD@hHozQLGh3vPN=Ks^F&yG53(bg_V{cg- z5?|D2yzo&;lNq~q6bq2d!Adi2!MbF!WG@@qI}bF#Fmld@KLkT#3qxZ9kRj28(8H#p*Xw_wvc!H#ec+7f7_6W zC0)hQ$9xGdr6l~Z+CKrWznNIl@z$u$4Tnwy`PzR%1o_5y6G1+^Z+BVJ1;Qxuqm?B) zx$x{aOOhTDhjy=oQ^{9{kN68h{f@Kyr@}B}Q0L>llDeg5!h0Rb;~bB}Xu{xKq4Nz8^)1t9CNTyK`J# zCF8n|ly_CK_~- zL1NTN2-;NZ3$bbEreL7+%%x;Q8EwapBk4!9fRwN zu`yF6n3nA@&xhQSoW8ViwU9h9g4T$4)40DJ^D!d0juY})@wI}vcJVH_LPr6{+vpZ) z4z!QW(J;HXIs2 z52Ne2ag3LN*36Wnj>BC7;&M5){ma;%5L6Wx!OW?qLr7*Dk*|zoD2hH zq|X}QTZ0$_`5YfVumK4N9maBDt&$?hhHANPH(fB#etV^)b*7lJ%~%eu|A2a4Xg8yS zn6TR@3vQ=-*Rc&Zik2-_>_ChgBV78Hfg3ne)n2aJw3C0x1d-R9)6N zx(uH2Hs)oWi*48Ty<56{4<|N#@s2NHP+GzFXG41F^&@;JI`7i4T3k)Hp% zVCdhYL#;gq9725d$!lD$R(4}zxM%O&4(aIMerKTL&jS*-U$~}a6MlSRKw>6-{Azeo zX7$3B`~1nPmn>|#*PooZWO>W`!xJ;%?f*7@ckjX|wa%_Z>+Gq))XYcH0`+rqJ9@9V zd>glAI~<(brT(O@&Z2O+Xv@}hK7JIfv%5~kkDEg+r}~q-PV5<8mzrBY_xRU82OY&_ zVEVRf4Ij50Mx<9$!zXtPr@F2gHy3xz>dz=83%#>f@Q;ztuN(fqprjbyJt9m+I#-q) z;MZSQCmlfu?2LZ@CKvp81lxT(zxh-5&?{e#XX^jT=mV^9iV+{0t{?gD&2TmB&whZh z)GIs8tC*!{>JoJ5@_8?zqa&}+7axepx`t6g`2) zTl)+7h^G1Xd@FsVBo8$0j**I$I8wABgnzy2b!sKMN8;`)nx_5=HilZi-SPv^W~m?WP)u*Q}xqf%3Zh4x@>HR?9H|UKVE0H z(FP79!c?np6nR|XKq5O0X}|>4S%9OWC@~H}ajoE}9LZx5#mLaXjI717XUJpVuPZaH z*wY=D${8Rtha9*4`!pjVF6bUwW2-2aZz&GsxH)ca7Y#l<{Bd9&!jWh()jMjnOQVJ(g!+7{bh#3z7CrUrd#!-!m*f(djy(?y}*m? z-W3DHs5P6nY`WT&uNK*dkGtDDI}dav-;ldDl*ve-gf&Pyl%*+a53}4(h$|rDcRb`Dd_D_B7yrdVPGnxHRRH&!i zRFY&^^zdT!iKbs{L~X6CM_oN0o6zxe(iKYvo}RN|wV`V4A&It}zgS%TySKmip^%xm zsU=}4(!KqQe?$9nEIE!8mrgc9yIIPCN$6-L2k7^!m}>3*U&zMzbWUCP@K zoqJRR2-|Jy*Q!s`a_w^5n7FU-!H?C(TbF!MhM|^tQ%@mMl@L5)N0+_}nt%y8Uc*yc z6S>uJ3_fKm8xh0}F7k~2xXviicwK%Y9t+yj%Wp;%=Xo4Zb;5h33%35`9MJVOJNfszqeZJZV=%PTeM(GMM8EBCPm+rUF=l39tTMJFI?H;N9=tg4 zdM4?~uJ{}GaWw~5V?@$C>rZHk>kgjR*P;J>^Re5%(uvN@A;3xPPu|1HaOtqEa^_|l z9k|Ft7d5Jd`eA4!f@VxDt%2;SG%?@#F49~!f^scL;JK`yhM3C|U^BXw{+>=;E$#&T zE-7k{BmPyLFx0K1KwVsob0PJ2i7F|DVAXhRk(7IzTaycQzOVq) z;BJWkS&OP3ZJQP~R7D{)BNHInOkwEFzL9^CbvGdd` zu+<_th%3W*ExwPYgbFL&A9mvC=#9iKs@Vr56K9fAcKS=rUtw2bgJZ^) zG^M7B;UAr%7-;oN+d@pzL#>-9^Ot-Vm0gK-#s_|8>aJxeh1~Kry9y5d5FKlV8&)IE zxvyT!o!GMLsjZWrfRa83=5Lu@o^U~0#b-|>B#+BU9-5Psi3Z!5b&dU&wcH1)Z9wu3 zYmbQZJ~Ct4sPz2$V5om#=(s>+7pS(88QX&C`A-Bx32&X4cSoRfYRI>@C_J@KZeCe# z{enR0k3+uAmA;KVN8L7IW^u{a6HC4-j;@5ANJ_$>YX2}a*p3!Qm-rGs>Xq=$>lH0& z3CTC~PafAlY5K%9jc=@Ne7&aS8-Mcpeo2|=uKm(w>Dz{9 zC7{Y4ghk}aU_^TSccBkkxvX-#5=GYvE_PLzw(m!i;2S&+w+y+>Wu*D$-je+54e=hn07qwxj6WKy0d8tP3gKv>D@lf~hQQ)+qYf^A6({T7Ov36$ZJ75! zn>zt8juVB4OYuS4Cx%-jnRI)WxP3Ab)lt-gEvNNjL%y*3Qsbgm!+Npow#y?G?K=uf ze?o|^V(^E-er+eOj8?5m95!@i^qzj~Y0_PKm4Wwel{;#2>Ge2NRk4YJSxXHSHWiyd zOqvlZZAnhKTgup4q-;hBDiEMU_Z(E`8sY25Qx{>7MT#P`mIvWCfK#3^w-tMGv$&8u zPi?iP>}@+4ro4vY#yCW{p30h-sK-# z8?xNE9cmYHl5m*e$QV;dsJ-$fZ!5GTNqs>CYx zauGbw#td?^4N!xCv6X4;Sd<{}rnkgKF%oya=fWgWtLSE}{Ii!YN}*6OS*_WZ%w=Q;n1DcA*b> zihWU`o{hHk0e$jfxgvQHx+t#)>RGl8hV}dMyb-Rx90jNfDY5)wJ&1>XF)a_O!yT1B z+U2kEPSU%uSwG0-xD9K-ujxI7FyaqvYv00#)8nn?T>AC6=_GiCxj*X^deu$N1mfh; zwl2u&Tg^1jPsRv4F~!~nVrg~QiSbop=bL)!hgf~f+{{Se0ww}9$E}egh#C3{$Yshc zVik;Wr{oVdZA|y}z58dt(SK&?598>+bbVIxJz2>&WF@V>WKH82Ya64XyhG^dzgZTu zy#_txxHT=q&^G%gzJzl0Vl(<>!;Pft9r*p9I5B$LDu4B+kc*=_#e*}1T82YpI85y3@UDGmH z^vm9Y25fVMXXM6yNoCT}KVXU_jS93^PmR3JMQB6&_a4`unXhf9whlgbcq1`i9ryRD zBER~w$3{0*i)PtbF!Wz;Y9$SqkT_b&=ZHd!YevGX)A6T8`UwIqr}t1nJPqR4NF96% z1Vq%4oG&r>tYY~B@=S^0$0!x}B3ar`g`qzfDRB)dTXO(BI)q9#YCA`h-_E3|*JNry zZ^LW2!Ngr(cYr?`Uv=6Tut+mv0Tb^ecvm5kgf~|$4GOj(tw%^ja}Ns2qjw9Dl!`${>#5NKG0`dKKg`&5v|%!`;eW(gg-tioA@UjV z#A#h$J{wMI+q5|PCx#ajUPdpusd=qw=cG-ame+dRIqB=`5VC#ovH?D{{j%AJ*P}L@ zLUQG!t&&z|$mUU7fgKX0njVKJTq0s+D>O5ASrO(w;At=b#!O!%1%x2AXTq&SGIN^9 z1BX@`!Ll=&kgqGqiiC~$$%>@5KS!Tk(bS~!c|syqZuZJ9@HB4_o4FbWb01P6gR(Pu zY%zW##qbcj^1lpP7F>5d(ky0@I_yLhz?vXervx0h3 zdby^=g)j=z&NhZP-yTgM_a#vi07tWa@~L>#3W8n01d7@m+9+$cW(F&UU#GgP8BmMJ zOgQPsrG?3*h=MqmX3cn%sf(Hl@ARNARmeJH#e*t}!$rUu8A%^|bjC@1N{Y)ydfG?` z!BL!!&W4qMg%Q@obaV&{6uS;MvpeQ%?(%4CufV08{*=xpf{uWGZh|U9i&cb)S|}>* zqRmf>Y_Z}3&nPy+#d$q4!E)2cII*rrVk`|-e~Vzu;Z60( z9+2)+7Fx?|6}xDU;CV!`vwYM&>;cq7ZKN6!MX}|iF{&cL zrD{gu+_6L|q1AZU&A%0wQ?JEEM06riFC~j^FmI}4f9WH-uLt@^*%~J_D3OVLf~|QM zS`YArzl#%*0O7oJudhSHwH;3$TCThZjEmA>U+-ZVX>dv+cmm|Giz3s;e{C{Z za)KL3Ec-W%r{-*!;cXc?jIZqE*wky}av}Y606agjXOwGT5C+yt$~LtwAIRLoWPc-{ zsuYWb;!k-A!KK<@=nt-riMX;p*o%Rq8sHW=Tew9wz%6nZyG6P+aI4}zdQ?`A<^g1- zDZ?~b+9uylAXf4e$Ou<`=?>2*fjvn`5XXoOF1__?dQF*|%p&ID3OtM=KY&r>4*1`D z`9x-U`b2hXeIgTI+&9laXCzrX{RSwfXrp&C^=DC3R|oo@j^RY&c4E{xO10LqAq?JM z3e8twL)PG?9gOwz2lT9)=V+HrQp&$x^a>2Z#J;8lY9#1d{#>Up?25qv3JG`r^^sS} zY0PLkrP5;xP9YxdUhwFH4dX3~W=yhu5v&y&W;gED8TOf>95()=85?RqAt%{M#Aq1Z zYsnVOu|eFjS-=0~kdTv4Q4EJm2!|RueN}Fy0z3z*ess_WR3&K%s1ZQ#VYhoTI#G9s zu}4WCn{fyxp(ePhgEvvuF+@PRv}`84W_S}3tTTfv+tqL#Waiw)jaOehkLZx@4AI0@ z_^Rw!s7j|a$!i=!8jLPN*~pmCRC;yVTuk(72`C>4JjQaGCUiDvXEkGSyRA;0Tz@6! zf`h!Q3vtTb;;75Ae97={feK^GE$*nz)F zvJmU$QP#FJCP`+ktm4oG{xPEcqh@IrIKo;PrJg?918FP=m-OB*9$@WuX7b6lb-;-6 z{y~k?y{oI*sJ<~~BQ#bGSo~X)T57C7ahYOWI%UcbyC6Z$XG0|NJ4;=s6DLcN$SUH< zT`?|??nC?5G(69nqjhO2X2V^hybU1(zPA1$7O88I__8jH<})n91F*PWjcWQCi7)3n zsn>WL4-Lb<_SFU5RJgYJ&=sylS2rzf+`YE()0&o3A#lu09+#Ok{l{w>TWVVRCnUex zKY9H_Ns)o*&~|Rd%5&2hy3r`fo%)XR5ZC`Sg+t2 z48q>tw4W!x3;D+GBMV+KIx`xSnYf;*f<%(~K+2261@n$-fVXUN(-iNH;xwV#aAC}N zd!ti!)5u`ar7tg54J#{Fmcn)j!+Z=DN%~0NOe=k~$VBWMo%#eQmn&d(B=3fH0LSQ! z9d{=Ts9L#Q@rg0rvPmYA!Lh=uG$W05oWXc)qC=30ShSD%Lj&lUP3w4R-?5@?w(qG8 z%iq8T?AeHIg(s=jpeNOGHmnuI*%{bhJsYtV|keBDS!P7uZs+X#tS^4|JD zj8HQ|4xlv7cG>nkxredri$a^&I7>h|R%+^kIWq=f5vGB;?Yo=Im7xqx7AMPeLrfra zv52O%@o#XID5to_YBHcUyebMZ4b(SNWPq)kmHfRr|5}{#;J~hr!KB4jtI_7Hs@1$V zHatwVVm}w#m_om%7}c6XcR|aGNEhl5KRHk{$=O^BGI#@Q_}k@l6~lgH%hSuEw_Y}e zL`4m+Uf5K7O&=o+qb6&d+;)iyFy&6MxE9!kR{#UFyI`kCkD>{{J>l#ZFEUd~C2M4s z#sW^sc8T=1+T_h?Ck|oLm;KjWXE1k_vjx1?X3do3z@udFcrhAcx)ZE^lth&lL16`i z`jPlQmC&|^#d4^jepJoeGys?mNv0}}`VkiqDF|ywD1&VAATvVtAlre8cxRy}TL+Rk zB$n6EBNEVmfXIk`ozbv*jZvl0++N0YN&39*_3AAle6;GnG-HNCpM9~W`;?!t0jGM3 zozwE#6)@FOoSE!F0O_Ccn(kC}4tS+Q*K2WmXc9gv?xnkd=?1g#(VHV&3u z%vtNOfUH&8T%$17NPW(qSLwAzm3rNma#J5X42W$XeIgzz3|-MG+7)mP)9UX|g znVc~#_)}kx;_$6~a`VaqrI&_$J1TwuED8^X!T;^KdADcZ)OONQNv~!mug^@%oV=#- zqqU76*0h|MkbG2T()LMf8vnJn@gMi6=GO&7PmB)zZeZj{)V`gYu?_I7HW<2ZU}Ogs zqg_PUY=y*Tfzq=CT4|yZf<1?(e5Y-V!eS{dZMHAo=`4rcqMHlu@FS9uKqv(nMJdT4uIq1iC@Mnf;^q`fanPeJy zRd@{JhHZtC^%=dORJ5IzLgW|}Hs@(Dd$b1?^ER&P&k#+7wh`DI(zK^+zHto=jKiU| zt@p4mW{kmqK*=QWB*&6@RWwk@qTx48mr04Hj1TN^2zw6d)<6&PdWgRhX%~LYi!u&1 z^d!NT_^ch5W~Y*YNaty(NexHy2k9JvA5-Z^kawihq>PXi+gV zlM}P|ZMniA3iB(C_3mzY2(roU{JS_!{)}&cF-Y8Rw3V=vOkd);>=^xk&2~k zZ2mnGrG>8DKK5gA7M-}7dS#0!nYp^{)GHgW5hcGc?=&M8nX0sp!+8;3M-A6NUx^Bd znC@$Z?MC-C@MSN|k6)W+T*kvrPa&ig;vt0p>2v|FPNY%?<9 z7KAOiIaB5ixt2m~^ZFnRqc4i6)G%nWj`LM}?B8-kY^4%t3&iL399!a}v~m>=6*@1T z!n=IV61+jjGpIz9wvfY~Sr%Q#5RH>Juxl*m=!ioh`))1|3AYh3a+)a2M|!VmB7bbX zJN6M~-6VQJ#t+2ggUyKn+tihXQFz=z%cAX+1^_51hs~bc1Vq@(gJH)4bO<`nY6pb2 zHfiQRkjgLj$b#uFs6835Xon855xA`)E}*Po(WcCf+2o0J5Jq~)oFpx!=B*lNbwlVP z90{A@v&u)PcnsW#hgpC`Xul<_WSijBqZy-al~il+0}e@nW_*Me>oEkS2+e1{U#wWv z!Ur>!g@GYCIcSw-VnKm)5~WeHX`3vHQsX9qXM>fB6X)W64yt-8O+2u%91bOA_)*mG z;R;949&D@$nqZhXM_^*OjWo08AJ-QFB4@<~h1?ioI zJ;Q_3ay#ZN@fQe7|FV?>BU|7Bc@!K*J`6@P03pXezUA}gTaTD`#<4`^xB<4wSuG6{ zrhNeg)s~x>lZs9TP?KHPN-8UAcUMt`(=8JopRlL--AS}g;A-PUyS0u}Bl{Tp?au0b z6p?7xel@9UfAmLqa<>uFRub-WRrol@WPL~R>yxeupYqnHdpp}No<-}DyD{XWBi<^n zJ#r`32|MYHU%++&@*+T0p{>lL=*^ zKA=|c{8`a+T}!5lFb(5^t(|5>tIW5^a8!0v#&C=DbSyeB2SL$QuBitgQU}KN)$n_o z(TEXoYi=+B4Z}7>6AG(C!nYr5p-agW;wHHAt9aUF=A!*H1?90JvU+arg4(H^!!UKf zOyL#B)8t!cKGWt%smC0LKpP)m(Wgl#sOd{3Q@9bftEgYv&L4G9UfmisnH4=x8}q!Z z#(LW*{}mf_aD{;MwUz2u7R0ZPk#CKsJT!uY38)0ZZ$T*?Hpd=Bt-o! zu<`4M*XX=%KD|!#Yw2oX8&Z*ABXXJ0CW$fQYal%$U-s6O{nM;)NgC@44W#(K&hb>! zOE{0nw1pZcIc@@gKm!TjqA~~oD%7bF@WEU(pn?33JsHjMF*Khpp1Mc~cpzh}v?y0Y ziKHeYq^3Bfotfx!T@7wBExmLZ8lo$KpQ!PLWdKZut?I$w(Vx{#h8RLE!|DOI-G|{~ zQpZ~mptLN|EC^}BXOr&NL7HmT3jZJHOjV;`Yj5Y~Dngahu6`jhQC2`|%~+(zF(4z@ z=J{P+C+FRjebj6;UQ3@;k~TK_hA-jk)P#?g`kx#+vE-;p=&hW%8ok%jm$igIyJaW8 znw>O#>YB#Hy2c}xwOohZY-@fLIy?}0J~?A$LHdTrf}uU5L!S+dEC=m2BBSq!)JNJg z?(|=ik}##(Uk%U4y%S5mD2^`kB^*)hZ_Axn^7+J)&&DNYf@XVXZR1-tEeW9A`X}%1 zm$VwTkFV6U>_L;YRVRk-9T?dxJs;Ea9~3r_J!Z~3UC?ZU50wJ4WJj>-J6S)TvFd$b z5C%#525b{J;OI`};HH+=uiId?TnlFk*S4S$9B zsCcfA8|R^ofJe2O)N9I!$-rcaBugB;-(m2+WFC|KTDR+5@zD_af z&~*O&<$s9}(PWey&%1vkd-ofQgPU_dVrkF1wQ_CU8U$Yk!$!I7_ypS#AgmT92R1y) zx|#o&2|Z3n$IFXzaNKuOFB_?zh)0>iMTJDVIO|^rW*c-}LYYMele82$rJ!_B8qE& z)AK-)k(77-k|H!Xbv6x6*#||MH8Qj&(&-j&#VNCIT1lYX8BO!U!-GH{Tbt%ple!N2 zxJUl}(=jTVut8`s_*^*@*J&OwnSxcTiZYd8s71p@Ni>R@HMg4sNxO1XpJk`6bPW4s zN?mi7Xj~UJC`O%NEh7nAM8``5Lp(%)QuS$nltiG&_rcDf>Blanah(}2Lta9Ig?yi) zkQM#IGtLOYW*U*T6R?=C4J2-Rc!XwKjw`q?B^1+46&w=wB>DwJd~4W-nB)bs(S~Z; zkS@G?3_1mfYAPodQLphn1)dlVP(1joYa1zWMzGxB29UZEK{hfxB1unj zh4ykT6RoTnAhR+#*=ud;2u`G4#^ z@(n2@haWP9^udUwMAUwDZ&R9|?wD8y))Fz{o`qLguG$csLmP^XSkYPR}ka z4Ee68^i3`bADfYzcT;Ztje*h;A>YYG;Y}ore8hj&692Hl6H9jBUvcyXA&l%*@sVhx z-SrXN9|uj09*cUmAEBP@>_DXUgmB9fXluOwy zSEB9O*}cL&@w+p}hl_S(-)`}S%#xk7`>eVAV?!jF^{qOv!F%MQykBvZlUpSloJY1( z>oQPoL~nJtma=7eBe{d{9U&s_%&WrZ2qLa<&@|RZhA-yVSP$L;tmFpWVrMo$lxYYu z{%buwhFvfF5e9y9HW~9btuT~K0a;~^+=ET3Pg|#kf;ZqpMK=cnoo6j2_!!-f zA4k!TH$gALfKe5!RSCxt^%@0ir2Nz>HjT0Ucs2^yif0oM$=h>3BmNndBM!B$sm&zE zk@VKf)L>m;WZmb|p?+WP-IyP)k|N0ws(`H{Z&^@`(GO%i0>8C#Z2_AK~fu)e+ z&MJss!lt|gCLu7WsyJIUTAN}73M=}Mmq63rxgM4yIIC!Hd}l~x%MIr*tz1U5iV^9v zcs*&#;0@+l_fAn%;et2V_$xM+R-7JGsOo2vzz~P{HCbJnK3LYAtQi*o|GFx1WYa1@ zN=eSB6s02ziK;t&k(Lprpaxj_BgExk}0y? zu5rZ~bI}aaXpE4Z)mU-(WCXSVX62k|4yclsGv{{K)8)_i>%X=vddh>l*2CWKV0TAe%qb z#TT&1P4hsBiHj({VZ~Ka9?=*i{QVNoM08=}#LrtVi4(%+S7f^i|)ggv}f zuT_9;WspLBc~Z!Q0t}P4Zqh;Qm~&6eMd8CQfv_|wa*h!Yo2asMJ`L(M5Ss|G73-v7 z-LOV*xt2TYhDQ@Sx;Y0G5|f84rCTOSHi@7AI*-H@O5#B^ok#Y@aUSU&F@MvHbtE`& zW9!L#qa1?C26yOyASAiXt_`q`+)37vd52CjIc@voM2`Zu!F9X~ko@Z8i9>jxG`e;!QlJSMfm;_UyS!NOHu>QDOr zYeVwx7J@kLHEx$=eTv$DO>=B@sBYg?4P~ZK(ZFK@Vh9ckmRoJ+Rb24fSI1GU7C#Vn ze5cMwe9&IYr&!N|poFkoK-fPXoV|bZpRW8?OdNlN?`rRl?b?kP6x<|j*Km()b?4zo zcM_k5!!`EI%pCb*aS*#RzvQd~V zqcPc^nV)e}A4p!D-Yz8U24)}=Tke;)LDPkVqLp}dk=`wrK$@xuTEPVa|3V9ha2rh& zP*|iPJ=FQa8Pv#IXyOYhj?V&EtTER;IVL{F(G)6y_tM%$o56v#8MHGn7SvLnh{CNC zIMw0riML@lV6^ru&}$8WYQ zBtv{R$c9sPn}}5I{O0q15HeJ<$=S_2n<^adP7E*6_UY=&9D!A4V>6YcdVpHT?c-3>N zO-9fR6)G%4u;Ck)Kod#_H86p)PN}_GFof#@q~r)G8n-IAn!wAOAogZPe3qsUG$p0OXxcZ_sEp11SjqbcNyY%ExxZtcnr1|)8%UXZ(^y=j;dxj@YAG@aI zli`Wmi`TTY4o}*C@4}W4es}NkmaW4Rv))Mybex%;RCZ)yQ{{F3q#ODrmnA%U%uC-D zq#2CWHA|9i>BTiypR6qTlu4|YmLwHyjT~M^&V;p}+`%(DKGS`l+?5!gn)$F`BX zSKm1qZY82*%dJE9#!2whi1<$s+)sF1-a@8-M)T{uMdRuh9sl}Xs?&;t(JDqC_(Hsm zTx+Ea($p)qUPXPPB}kIBS6}oqXd%otde4-sNWn!PYMh!3Oy! zCnW9Va8`m1W}YG|gEkzJC#r4?aZx7Hm+1u8sCeg?%a>NCy@T@d?Uo_nIRUx!kcF2iUA*U^Bd(Av*>Vf zX|ZwIbMmy}((yQy``E&+Zpw_sF(Vm3=Tz+6QCwOuzG4$oaiybr6_z^8{qczkng*(@ z{ximyPv*k>d|ENB5-_!s*hmCXRRrrCk;&F1u&^h2uoVYo+l_2 z35T$C5ou)znl$?kqJL?rjT=y`jq9X#Gxj11a&V1CoS1>#{3+K@R`~JZ8I?D>W^FO6 zL60Fu7x=*pT@{}2qbuYe^L7cHSL}7F6zztBLUsg;6O!57;;mCRvCweWCttB|Q)8i` zEEN?{AhwclyCr^vaJK8_=MtbJC5MmFIl&>m{gpxn(S?E)_vv;d`UKe zG{>A*diBiEm>~2{r65}yj$&>}p>@LY=CSL9FkI*yp8;IDjb~=`XpcSoSin zbl8FUTV|IhT##1r*%JxL<8qRR<|JjpGBUHSvEQ9~;Z!Ajq9m_9=5SQ;oT3i)oV^!mrI?c+G0rKIWOVUoYL@uixU_fX%~5C0N3d3DI%h91{Xpndj*Yg)b< zo|IX=u;o5~^6DiETkiEIXD(UZ^8WC|On|kw@wKzvj zC_9uY$PTK~0!ZWb%q}2pO6>>3B2l&(aV=>3YSM@Y2uj51zxMXdZ$2hrpnIF_x_uL~ z-Zy$&bA)9;BxM6~N$u{38IEybnOrTU@dzX$^wXmwwihYaNn0(OvAU9DB!rzBAg4Gx z@JT{(X_rFs8}M7R6cVR!y`Z}klHP`*SGIfP=ru+PiQh4*1X3Fp2dU|eORPvuphAul z2o4MXBLx&GPM82GZS54PL46~VeceUI58yRwF8HT+Bc=Xpsm zxcqD_DJNb1A}Kc(AEKe!|Fb<9nF+@Bez;L-_Q}!WC-}5^HOPkA@3x7dxbO^F?idnn zhr`RC%n3Y6w1f5=*`CEBCmV7T+aC73YE)gwc)<}5$nzn|Y-rfQW?9bl+kB1Gar?eR z_U{tx1>mCzS!jhufgVY~PG7vEO}x~^mpH^HeR6NZ`1IK1At5y(>UbERsr@7m3p3)> zUU+P?-NGyeuF;1&auvT{@2=fTnT4vXEbiDmI-3ZN&d-D$2@ztmOYShb(`k^d2+ugx zXhyqgZ<2@9zK;A3_r$TpwIXZT#Eya<;TdiH+8B|+N!Os0+2n);VDStjh0h5kcHHEB zqJ5o+EHnW!6-Dp?Qwk#;#vwK!hT-7!w^zj)h%C;! zF|_7fm>pz$;oBqA>#mo{56^t0O_-%DNV-QH_ey+OSrZVQXsO#U4gv6fKLZAITV zdMK?kH+EbPHd*Gkyf$5S8jCpHlD*Obk*k0<{Va-@5^mDHb)@h3P3yRf&uVU zOyAy|Xt)cZY2^7;ECYX_wBd6vZ+MEY~uS5oj zf>-hgk-L-nHnuN_tDLD##3-7L(rw*0d8kDWie_tw+p`9+w|Rb9H@T5iJaaHe-;KrLsW^0JF8CS$1o)C9jg2Q#Pb%p#Hu|hD;h2dl z8V_66@>BFy%SxJl>6*rlwT)lbw2XyMNmlYvS&19xCwsNf?h!7^$gRIQH}B>^>Bx|8 zbfxdSqVP@>u>C3kKLpHXzDthW zwed$`f5TxoHU$574U9~y@JhZ5ws;&Tq&|G z??x*=!ztz6Ono_dFWztP4xO13@tjr6@H1#c2kpYW_(<9g+E4P1$Ll&y5sM7I1+5(N z^(36~Jg~S-?AYayOm!NKTeJW291Rtx5qlvn<<;_7X$px5HO|gy zy;x$v^wpOdu<+_d6ro=pG0pxf27eeF*M|OVRjU$*4P6<%ryq7TFjD6~Z7jX#89ZD5 zA8@F!=}OVM2KjA|G&+et2zgK$#;N$%RMar&?!&Haocm~svjxG~Z5!r0-RiCgHu zUghY*7I@fEaY>)oNbvF?VF~wO*P08rf{7f{W@O)*T7f!5#@u$6IK&ke68AZ=vbb@8 zmk5ZhG*Fr}DNq+>j@ek5?73sk^D|}xA*HBB4%h9N=~+>P0g)4lIJMs!r+7_Z$yv$F zxFxt#K^(e~i-0&D4Fo*x;q|XCox~a)ynzSHRBXm_zhYdaVw1Ebc3^0}7WVlTh6qc4 zB0bD9g#JJF-UYg<>RKO{a3V=m@*pT8k^liC>IHlvMiL-MP%3a$M7@>Zld8Q)QH&zQ za||R{u#Ku*7tyj0dI z*2VjXu-Xo0WW>yaPc#KCp8a|Iuj6Pj%?4pVJACCCaY-4ggqz0}xh~bdSpTKsk%(eu zIBCRyb`YE|_-DfcB{F0WTTw$C<0#W$7$&R<YouIn^5c1=Or@NxZ$@NO|Q%ziU}FKqpf z!Le^-?|)j|=at1DuvE`W1$k;}Yzc+SR$$mi%o8OU*$Y~# z7Z-?Ud)Jnt1UW{#pDe>?**5`WR+Z!>fJC^R_o($b)(~o=bs_Hw9LlMED;Y~6baVSm z&@8>;yeqdndm_jl7_hDmb75>1#wR^_`alcqtySZ78mv1? z-|d1cIW>^`UAF$CNdxAufpqxn8UPZD!|N%d=2u)15Hir}%l;^~&CI#7@`yr6HqW?n zp?3MnydSm2&_nxf(Ymg5wv3&0RPEH!=^YGBH27<)5X&8PVf&L|DCBpbq^zlE zkvs?t28)f@Kim~jwOueJ-$??sg}haBKPo>zNG~mz3QeiTOqu}3EAbE*7h;1}W+E!Y zl_7(A{g@S6t58};W+GU?xOlY7R`pF*4W!@LDxz865)~5Q5S&riUfaeUz0EQ(d!`-! zeIy1m1~5Fx)e9%t<5a_U!Xcc}BvRzE6_tHzO9Jj{d(^DLEX~g0&^-8zpzO-<$Ljo5 zIAykI{uZC`;8>e|`xOw79D#Y@A`kL%kP z#Nvz^s3){u9Cp(~E5lVc!oq9(1J0rk70)TCL%>;9j6X>?gm#Xu=5@I02oJN*c3VA>|+Yl~|(d%Yo;$car{unL>UaSZ(pQQ$sfH zT^86l`(D(Ukrr|&F;AXIt#w~u!CD(+f+3%RwX5eWt0`SNN-g?-JfyHY~@f zMe8*}{n=+Mf;3A8hLWF)vlpp3??1j?@*aI~Bb$Y&=$Ut+PdjrED`h+cl(kKrNW`{m zDD{^Fa9fuUW_L)FQRJ7<%M3_j$1?@lumw6G9UvSPBpN@F>uk)H(WBku`h*FG`pBe$I-J^QIT>H$V&si#_~>=6r<)zP6uY{>y$|8$WT zBFRl#7sUR^e>?t_@Q$5V4sJ|8KmFRtgBwpgKVvO*%$79*dQs4{Aj`|Z0%9-oDmUZp6k3ch*PAUC>9Jb?(^|J@i$i94}yEFZB%8Hc6_Saz=PfpWU(3w z_i%GooXeP$v5QOZR|{OK=^bACrF*BE^mv>22GH`I=$+v!uQ_R>6QU3zOJMglqR<*|9+pj( z`QD!YV6O^@5nDQ(vB|#cBL_y&>mB(6dQemCH*bG;oD4*DVk!NtuMy{dhPF3+!|1^^3UrwjfOFt{IH^Dqp>A?aE0(w5-~2+JzU=gIoE^wfUISRC*#`?0$+ zsB4|vOjl=EsO1cX&CT*PGFj*P_iZD;{`UN)bqJ+5E)$VMVQps#;`;viJn1?{Ut-Wm4pHW(ak+h7(5{7uCE)fK)Yp58)5mt;8+?IbzSnREQ@32wZ@|=84rqHAT zzA?v*Z~~~${)h1APX?m=LvUWR~|D&i2jf3$X~n>NQ0?&{=Pr z2w;taiP^GJ4H`PAR09spBb63wRrQ<>f9#Cm=$!(}ET+*TYosPsC`+fj#W9&oe;ZO4 zfP$%hLyYc%aAPE^ka+!+n&!Aj7QTSb=#m$2pgu}O|MszxWJQ{Q&Hg>0X*BUsiNx`r zyu{Erd2C5Y%YY0FoAMYkOcxvi!#0RxP(2kfPWG4-ec7NPKDjp&m z+oxY!YHj#Pi zpKbTIp_e00R=+P|TOk|nxbbdm(?<9w{|u2o-pS68t0hpctFNIbf;Td64DGmz;SxB+ zpVvq;sq=A_<|)ic8F*~|JnS8bXGp0JtaPDmg&oHAkM?oViCYk<^pSox8EE2p-0rM8bXNuJC|ppD_tH`-nlb#ov={WMm2T zXmi*`*6#ZOAX?_%o2H%sY<5&6R=p5{%J!mIx092W6|QL5gF3dMT_Vk>W1G4tl)o+9 zFeH>R_rcE5%aSs841IR^i&G+1jHQ;IKVZlC3@LtnedITMsK*5ZS_@YX-@IWtp|eN3 zR(yoOf9`Kb)mGlgE*@EX7snntnG3i+|H*+<+cJ*LVvz79JCi(+)orbK-8Lk86lkEb zNm(BbiiXbggK3kN@42SgAE&)tevm~azAW5^&y{ph+5qrt1P5v4{DF9*1nj%y-W;=xY;orq8`l-r_WN1ijn&1q zXZ;KY1Ek>eZ_M^hq$QqOFeHRa>!<8lcNy)T@u8Lg8-2h8HUIAkfz4}5Kf5!a!4)K@ z^^ec&ReFLa{{k)oq*=|)2fR!M7EpK{9SHud=B|KX_kkcZ0sBb6;3g67y8I(~d#dj( z;(mu}%xiRmS|O`rz_QY^bajQ~}am~X)3(Z?u- zXFLQDr~pnQBG2H_z!_g&IE}bTNd>?Pk-NwHnBlaF2O+!6pDQMAJp0Ty&>ID7tO`)s z!ZJ}E1gjubdUwh^ncw;_6_*jxGB-+Aukiq7#g|{nI9o=%V9KZgr22}6nfZ5oYr@ht zETCgv0S~4@dx9lG+H$5F7n*tK1JEykBnEOcT9;V-w9I~`d8WQFi#YogFmEl5sIhR6 z+&fY9XJ(pJ0L@IZwP!k?5WdyL-TU)398*l}aWeIV0Tt#VeicotoxkV zMz!+GVLe$fOFm2*fo=N$sJjr_)i#xef%KBisp}=Y^%z-nSZi;aI7~{a=!!Z4q^@`e zZPfqMnNOkshTj?I<)9Zk2W7m;(@6u5Bi)q;nT44PT}RD^7o_woF2|xa#r91{y<#~O z$-I|42RRg3wAmXTW?7m0PN>%)o$GJi77#qt&u(j@rrJr<&zMl0F|jy(Z0ya1#IMp4 zKZ4^3)RMtI)BX{QxzB{L6Y&hZm zF_F9ADS20J^{k^qmGF@KY(nv;MX^Xi;{2|Ov*0)a8_5rg;D(TR?i&x2hvbLSLo#{U zkA$scc<`o23&Ot`6|L_TX^5hM_O7&09g5nT=jXJ{9F?~5m*STl_WQSckxcD5BH^U+ z$dpdMop+%6=h@LgM~x_%JbH1%yQMiT3t%_WcS3AG7_}!lM`wdk8$2em{KJA+kA%eh zSHlgb;)6ahfBhlppI`jZ(&9Z_^7hE0Q zr~GVp{!J%%m_%|d*0p%vDnpQM(Ml1I+~^Sdk+lC${;Nd)ppW*Nf^4$nY`HNzW9~`( zMO)_J$3FBUXsh<&KYdMj@j2Nv&DT-yfq&w%gCLpLR2C)GqESE1~`*ktDRsF1s3s`Zhi% zlD5cn)5a>PMqZ|AjDAHr9{dv(SD1>jC6$Uxb*RdbxX`fc_|8GO0e1#F`qJKPpiWuzL$vB4MZDl+BTvsUj+hoaG_%XOA4j} zUk*oY30w@mD#`lof&<^dnq&$UQyKA##BRSQ6-w5PiQ8#w_EAc%AC3Z~na?|t;_kp< z8d}aOQpx>G#A{ToQA_sOMku2J98Ou|)vAmyYdIGg;)!g!X}mN;JR^!J{f60>DlQ^R z2^Xy-iRX^P<4I%)qN**;I0E=gWW+r>I^(Z!3Sx2*U&)E7=cWku0O4HaLlVy@d~>x0 zo()1*9)9}5l984RsWqBvs1f=bGvds33Y#JI#CozH_%`O=7xuS_Ul?TcEf^t?WcdLt zL4c@OW@lo51?7f1H8rd`psK{8?U=igy%J-LF;~N`Zv);(fM)}kp~07;vQ96xG4wT1 zFgS$7Sd%ak@|>_jZz<=V8Y(>(v!T}{^{0DL=XJhF0BS7q2*>}MKb1RjwvtI}SBa-o zJ(79=Wikvj=4;T0zUYsP(IKF%zqO+3;Zi7p^26Y>-IbaWLaS*@?9(?5GHBYuW77N}V|sDHP#d~&qZGlZn-U+xww=wqsMXy*0Fu~b>^AT<8K*Smr4-hxJQe-GB6OGM*J-!p&8jvowI$Ekc;lh z7>2T`QS1=u?HJj%?*{AL*A1{f(Cs(`FB5W!L^;~l-N^k5CHwKmLw3L(1zBseiX+?S-tx<4oU^*j38`!s@zUg zC>_5OiOtDMZ+pO6%rkp=hKG44^q;}tmqTf15j{``O<(MXEQOqcj!{fY0h~$RrR~hq zl1m@Eg6<3N-Eb12Xxa+2bZn#!fqSc0_ejN6I&esb>G%TKzS$A{B4E+4@jfChG2CS< z%(Y@0o5~9r15$yBSV%xyPH|COY=}VZ^%w3LC!(sUtOdiZ(vfff^yWHt{ZoGd#nTjp z1>LbSy+*y&YpnB9A}{?-)uCy>v=n7h0j4Vh6^7bAF!f-TyfrqllpbA-&EyY9wD+@N z2YruJFpN(#w^dM0ywp3fh{)AA#9)RH6Nm7fgrrOlj{5c?@zdqKzsy&Ir+(=;9|dix zff1R2#`Xouj*cy}5yNy-haiQdxACd~MBhs3 zm^i~V)FA4KhR0YW`&Oa7wu=Q*^NhbQ^#_=%1ng>@98Um>!ze1Z-5h(N6w7r%(4otP zScTjd5i{1F%5n^b?fatM3d8!$fWMmyr9>uIO*4(P)Fp7JV{O@FLr6n1Q>+L?XDPsH zm@7DhQ|8{vh(nLNx)m94A;H4pxQrZcHQ(fAsr<`Msj1)V^Kr@ss-1L}w`K^u!|&b^ z#@#BJFprx^1_ND7fM%A7G4=Ptzbgo*7lx0^iM|EStl-B9 zor@xObCH}*ayx_d_2#s>?cZ{feHg6NhR z85>XP677uN^*jY{&}eJTW`k#oQ(puuOGqm#zao@8_b_;~j|*Zs;LYl)8n&LFwz0e* znu?;etGb3N`%H*E)GKM(NRWL$8Tz6`Dn7>k#Bfoi8u{XTSE`lD5;C7kwS20h@~KpV z1oBWFA%{@|mX2b4IO2xLXa)~E3I+XjdDf?B$Zl{LQHNF?yrI5Sqa?RK2-97cXEo`o zxEV8EE=Eo}aH8to>cOai2yQN|=Qj|4BchTx4-!1AUOXA^>tNlQNwgVHYso{6&G*sj z(%N^iMW!*w=7ecAX@uBtEo*HsKrntgG(;o(DntzizKz8qP(lvzS3(A~&_O33knO2~Y(67sscM~S3tE_T;t6Pd#5W37oh|I=c*&c`7#4Y@n8Iop(FlLj{) z9-7|!O8VQztuiqGeJ*Yh@Pshf`fyNuwD32F=;SK_vww#tG z#O&J&AcV+YS8&T2y%_5Ck)vtaI^y$*Vy8qO0@}q@sHU!;qXOH&x(RqaVtbXPB!&

    Mt;PZg+pT?MSfi zUVHQJuQ)txte1q8B8uT3x1ZN*U4UGKK6lM;dKxdK(F88lT)F#zYDn)?sPNv}Mz$mE zO}#3otmYn6rH}lwtY+Ilx>v0?LKb;8%OZJJ30D!dE(fX26`Bfi#(lE26~56NTk$ z_oVrnYOP(B-`v`EhydKQBM0E_IC224&&D4@4!Q5!${}OY1kzg$3F3_9ki<@T(`zW9 z3X($tEC$FS;UrS*=OsDxZ-eC9h*b^%M6Plchj!^#+W=ii>l|l8lKls~Kh~9v^}(&# zAnxSI!^xqkf;Um=6j|rbf$d{-CIOA#6?7#ez!NEY8YV z%deAU!nOdOEW%;inC;cqP%Ejv01lxw8-?HT5Jd+SLh51s5OhtHo#`rNQw0O5`2l;m z0UK}Fa@tAupy(xTJwvCc(Ak}+6)`L-!D^TlaRMH6Q{ogjy69yd-qHHm|SWQeB*lXlC+;t8Wa z9UCK`^||j5uO*j#oj7nnDq@ca8PF+r2_t@U>tITRqC^YtT2Vc>FgE zWHDo*j%*%uYaFZ<^I0wq5G@9^cpfSA()JueywAv-H|r=jW_&Y|wn6Z|#4kI;&SQKL zsQLd1FyefwWyXfGcNQXW&r5+%I!3Y81z=x@4i#*5&ovQ!!Tp1%r1&Qpw_`yMULJ?p znpC*6Uvd`03sR)M+}oXcO%339$8&rPDZy4C4I6|ZJErz~tSN!>77x2D4D=|b5Nb|4 zAd`2%{x)9HVP1}_7q;!(|9(xvQ%SX7y_1kQAU*N;s!%Gb)>0=HcO4shEg|vcw8TG5 zxW8cmO49ly_eRND*HxR3eX!w982V?Y+?JiZY|@I&37=;4D9G5jrXc)vzv!Bj%pUof zJBDZE{iY!N;izz9=RQZ!2y*xLYV}_-c?^ot?#pR8BReVet)vWad95iEPJN>OTRA~q zJ!e);sGpE4&Ky@glhtyUYz!%kd|6#(Ky8WwX16kbqN@ym7jlmV;sw?<&Ju4Cg%S1$xC zl6p-$AhJU;7(&w9&^E**8;25>p(ABy=cXd>hToA&wDHPt!8m-Un9188B1i_nacWAt z2$K5E89#~=L5_zAa`H|^Dw>@A?+$YIpF+*qe2UYa+w{lZ!GY2Emfsu zG;WfwA?iG(8}1Ls3g=)C&e~tPjs1tZ^^e3y1sVJ2s|8bSNsbhm zRX}y|u0!iJbL)upSADUfJw0GQ%Y|;M(Ii1r`yZ}S+FnYv9exN+XuHf|shCA)OI4OJ zQ=3C!HNv%ryBw~xHa(9`?E(5$o}mEInbW%}#abN4Mmw`AQ^O}36Aw%s@utjU>*z_t zvH0umN-x&3jvV%b<5*xdAF~i)U{vtY=E$UY9G*H z$W)5}phC$fRX{*snH=(U>ADR}>+>Ol6eN=fkgausv%CWKB8T&ZMcml4;puFL&r^wO zq|7)%e0Qv7Qsm`H5913n$E!Af%g_Mry9Gah7J+EhIJiJAc$Hq1% zB+g1pya`?-Xu}qoSloGR?1hBH<0mZHeAI&tKTb?}JuCTIv}J2qx%sQAhOq#4St-Y6 zC2g6V(kJ}hsPNnUqSvQn?#RnrnwPQhR|Vl{zvu;UA-N!P@&%nEW8gJ%XYSxTb4o^p z6GoLMTreiG2iA{2%YlhO!bvw@T)ev|_A_{kbWJ=LULylU7wv+jJI9f?NW=85BVxq~ ziKR1=B6~7&TV{{SX#G>4q@5RwUmiCiHX$K#9a?3V^a`y*tL!^^g{I@b+n)$Ey*wgT zfZx45Ha4wSS^1q1IUZLOsZWT+2LB?Pyh_#`^>*z&JK!X8!hP9!->7%?miOk!3aG&z z7_Dp97;#cSXfILgWer-!tobk8Y zmw|^1s89os$6!TFDxew}V@ga>`|5DbW68kE0#CabJSA;C1iKJ%4UM+!O31SLXSK2U zh#TVR*OU>HmnG?;2;t62&u2);MBAi^>mhDc@N~Er#7Oya8gqa`o(262K8V2|AmD}?0K@kKcW^QRy2Bg0x4bD6d%PJ99fhtKs z!`aadV+$3i?Fy`6L%jqq;f4sgsF3NmX}icPTgp!;UUi9ka0u>&03h~+ue_X$w##bH zr?wWn)w&gughEwgwvJnII>K6urDu5(%RU8Qxx8CbDx2vzVp+)o9lGXk?33lbl6nn3 zy0mn}a^1(#fy9N%Wcu?+b~k_C1x?w=8mugT+r_o@H+Cn8;4WuF59rc$-MO*24qn~p zdJVeHo%c@5XPE+I7=}9?sct%JNgwE1C~t(fvO#C{n)s*s*?(YNnb93mK>Q16b? zb+_+-_f)04Rn(4l)obLUY;ahRHUl%%pU$|`cuLL3~Lht6o6L)Xwm)Gs3C!;cff z785<>oX}kGWUx!uBMR!9ONfaF3d2y6Bhb4h^h^7L7Z*cvj$Z` zT|E-H@2uo|YMI7GQN%gV*WbEJ<@ZTihx)^%ICH|O~B;1ZdMJlavuk!Pnda20`X z)3&>fPq)fj(A3Ym@dF>j-U9TPgEFJE{t#M9#(Tc#n+Cr4(8H&5e@8X?dc~n9Dssic zh!W=fda_ZAN{_U_zHkkx?^11iuvN~0Xu78NKee=|`WC5_10A}7yX85+xwNuZr%lAt zbq;?xFMLfRX-lqc${L7NjL49Y!=@P{*p-$hta2?`NYefhpSKea0X+`P=vtwq^Ul*X zqXWU64YHKwhExC!LA_|sff4!=_8@f1Y^$t5rHbTJE%rCmjM2WV5*KQH(Mu3`FCnz% zY`)wu9rbO*9R!IL!CYd!-CEyTdvmmWE>6|^gD9Rz6C>xh{}{=T|&=Vd)I zmR^`S`NE96rwhV=85MrDU-S|Xb%1zJ6@>pZD*XGi`jixg6Rs&wxNJ=11UN|Eky|}8 zr(}3I;WRi*lDq#SAwar8u#eo0e?_s|1j_4L_R;ssz_nB_f{%Zvv9W6k(uUjoNAmaf zEs7n3W8Vnet*QHbN%05Fm)*1|d6saA{A6kIr-DLTl$;Ms)7Xok#HihN52Ap)+ihD) zOE3Oy%S7($?9*<$=NtQ3G3Z_4RyN8~9XfT9Hr#asaZFU(V}UM3%!RJIo0vAhyAOlm z9d(6^WAJQm0}V(Vypsw{YJw12BP>6-ahyym#!+9;P?85H<#sK~&B4N>UFrQ_IEZQ*^(Ejkf?6BGKOqX7XQ0SsIaXx zFUaCQ%fGOV>#t14kV%ED8>Pqp+1M8kpI0W7r(WKWPst?k&zYs$_!}$ryWa9C*|?Va zC-=|G8sl__e9YwzISswxruo}LcEEfd*>90d_vqz}neDJKG4E$wGv-OHanyp`oTuS6 zM{<)#i|)#v@fHGt$9#MSWJv^0vGhf8mRWL}94B^Pb141Qcu$UtJBZ3+Cjos~5hm!@Trgpd z4jF8;4S+-vYEC8{d?b<}fIKBTB8&=B@b&#Gd(ayRJRP~6AuP1x=?@uPuo0ZJHE<3s zRNJy92b>s{JI32^tB~|t)m`1vx0C~I1=8>kEjO2Bu%aEGW)f9Pv2>bVc*J5j<-83{ z1N9}xA7?JSLXen_j?d<3j4GEvMR}(l1Y5P&n6_Zq1z}HG7}^&=5aJcWB0NIIY!(x4 zDnDQYvLxSf=G*lz9p}T-L*#5!WGmlPy%tE?73n4n(WxNz(+x&zWOeUjor7XvwJuau zKrLxcA_%O3I$d+TEN>469k|i>r*t)1xTL$t2=kLHWdEuFN~Ip$NEQ=U_Eb0O0mC3d zVaY492=4Vny=JLxfJ71@x1pq1=D~mxa^tK+^fN5h1N3tXN|s5#eA~T|Xju;{*tw1v zZ}Mk#t_OMw3#g4rxnRH1XcvPI2+T4lBlHr8pijJ9THYOXx3C{h|L315dF%}$ur~(~ z#vkL9w-}6S9Ho+DQc6PSsLSZla;yxn_pk9T`{R`Yr{pWdt>1$zLZmH}j}fo>+L)BC zbe7{m$6^S@v~2B%atkv$M$b5UPU9DC``Z|?9Y@EkJUF~*PhETBMd@W99w~>&F124R zt6jW#@5;@eRy7PsOu4Oh%DCRi*Z${<%?(uzCxfs%Ic4pW$vO7SIyq%o13Wk8abh|FZlbtDj7YMxmg4Chxr|0-7mYrlA`9;KOTc_u+n) z#!~;%asIP;yWzFPNiHYO)8|7zc3~>RM9#Spo}+wG#pg^3a(lXgpW-A4zdZ046Ag5V z_`L#CuJh-n!app(c2EM-dNnD?T;uy2aoUnDhpi5nPRmJ$NRn<^e7ziHeE7Q}=!Qsg zslQ0_9)JD~Ik7DCV!L<87&Kwa*f+ILSm+yHDzB35v(vU+kPLwOVG#XRdiR;)&K8wj zspOJFaj2+lHM#b~24kDiDzfh%tbW-z<$ZsZ9HjNtLHjp3QLFV$b&D|8Vr(y9# z7q`PAm+NYm%^hBqJd&Vv{BENl0v>2==hwIzF}pW5fMn%kk_7X6@nkz9UiQP6IU?up z)x{8fO!q9?;Qg=uhGzjJVq>lvaV&Pa*v)9X_98c4tDSOF|Hkp_r5OX)QNc~mJkT;u zfIo(+&oD(8*WL5esa)dZo7q)f^!CiIq>xm+>jBg(PfWzSW&=%i7Z6$~ouwKs=kPMr z=7-#ZECtk0*%lwGuv%T#O-a09ch59@AyfpZ5-9HyHm0BaOW4@+#aAPVi*QcekBGnn zOOc+`fm>hOj+;18a*UI;=ZHsHHJOShbEKl_TrUu9kH;*;t>5YG!gelW;pm$Sb+xe3 z48mYMbck#LZ79G!E0s4d5eOO7$U<&P{Ac1X(@1+_V}BJ%#OR9`Y1Am!Q`w|*`H@eB z7*J6*84_iXTH9q0=W$B2up&`_N@WBj&1+Yi>7h~cH=V3~s219PRlu}0E3Rdb3?@x; zQqJUp%y?l8;NM8gdwfPq9RiZ;08VpVL{=SZ%m=TMZz-A91H>L8z@?$cc?n*Nm9}~S6qopa-DRYQIi&@ckq+pk_jv(qzRShk zfzZ}5Li^~5A+(XrM-HJKJDr7+=<%9eGB_Nm85)YtOwMfG)F-*^A8DcK{fTmmZJ3{3 zk{gb^yBku=u8}Jt5f#W?&z4^ zUe$2Qu(Z4d1<^VfL(b?LYU(o~wgtV{9_bvt9=+Fcx2_%$j2jc3XwcM9+C6+w~_?Y2cR`hfDb*tjZ^xlVbjp}%;q30 z$z%0mV+qJ6cI}`-AXTbmQ(8_Nx|>{8h);JSLG>DI3Rl9hfgfL$u3uzVM{dQQEIz6a ze@0yZ{IrpXvt_e^DT%QE0%bP{5cmB829r(r@t6_#5!GM-$efO4K8}MC$7^aQK#EV3 zPE%rEXvgyA9kuK5R8}hS=GE1T7EVYP(gad!y#KxwhMaE2t%ImLHiOimI7Gsb=|{)j zJ$0A#@lQP_88H-vQNxuua+>I`w&~7VVZSxLzjcYLafzy=Cixnna`Q#qv^3-~D+9Sj zbO3k~n>z7iv#(jy@2tKXmsxv!XGwqc{j&wqE~1ZhoosC9!=}}2xc6(w$NZ2Qf`&~a z-oOWM+PWb2M;_GiuY`NJOWF{l)DP>VdtjR38nR&0%Z*3S3id@M6%KJ0jQ2Md92vq3 z%tcj=?6JS0gPbwAi{D0dkgMX}98p>VI!&1Z=s3w9_X5!k;*^@&*7Q!dfXqOcahq;| zKK)C_`JmH;FwBvcRFQ|I)1i8B3z-tw(R9JD;$mis zS5fvQT5D7sc!fMcUyFmqv|!fI=9cMspOOFx7dNz5V1DBwO9(N zoIIl}sghc#p_PfveHAw{ID zC%H_diwKi@8iWYkOkW`0IaWUPLf8rJOwC=no~YLrZgwm~cIZ3_ z{VbvH@|BokHS%)XlY;`aq%;ePT#-XhrvbVNRc1U!QOgGEn<-{DDl&P+k$wEw9^a@o zU`ef+U{H@R3$rDYq`))-Sx&$e^kQ<&Of9Q*hpA<2J9Xj=83O|<;TrDg2NawAmB7LH z5$eb008mUaN^V#~i?Lyaz#;bYa1L6P^y^g11GQf35GZyps5H^~{+NOR{6j0o^Z3hp zbZqVzB}__9V^Txk1Oy1ZnN2l3J3PbJu7U%k+=!iDT|cF8C@J>>#j*nhPQimLCZKIe zv9B_cv5L=Is9Ou46F_U+9>*pt)!5?kwI{e7QG+UJwNSBX{DoFs%RRvqDjCiJ?9le_s$DTNoal6CDK)|52GsM`dh$z92lL zFnq?}vP-TDCtOk#>2iE-_5Hbn@6RduaX6u0MZ!r%k=5|Bec6i5AFbT{K~+P4c=-2D-U(0tf2`d6 z_eaz6RuzPw9Tomfzvwa0MGni{0a|T&LHL*bqPxk{{}STVmPk%5r{r8gvvsN6cN6O% zOJ^oUx^&BJ`8jyBB?Zy?lOAlSLp|Fh@ylD2BIQ787hW3K3Z&LQ99fP3`h_Fq0Bq-V zjda29&bc%)W_R|zHvf^Vr_;gc<+gNR@t+m-^?|HU7E=Q@uB^Jeye#W8w(-c{(MMBkpI+A5 zzsGO}f}e07w;uTeCg=0(!`DYT5q0zI@Ykk9l5dC{$2`vD5d)Tv;NSP**GoqZ*fH|i z;dNukJH!XUy{0e33Z$BGX@_FofWwgze5W-V))5g~Y&Qjofl# zDWf-OQ=;BMzb0F+IQW-F5>CP1MXrFo$a0AlWtNao6+f5~Xc4dq263qRV@WEJYb4?{ zl56D4Ma-EPUSTThd`*0o6X5BOLeoIVMN%La@pg%%y!KQ44fB3(J-OQ~89QZ&-~p3pm{rdGw4lov2JwV_Rtu z$z-{rswgXLM9pXwMubUxAguRG%ii9wzw+CZKGdCYH*d`T8ptWUotvXAO!HN!?2>$y z<@4g;LkT*=Cj+?Imjgm*&W17kWH<>6jzx(1tv3SV1Y3A-u7%P!fTM0fd+7Me5%+>A zbVvR#uW1?l_Ho2owmxztMEz!H^KkF z#W3G^bJWcuyZNT}Uvr0ZPk7dWS&O)P{(ZuNDK3qW#2j-+oJqWuk zuZmyFpKr%0OX8>O0xEovp!#>@GnRw^*6Vv=SNSDWAV7wmTskuDuif9qLNeN^Jm}@d zhat%dSP_C-UVhoUEUW#P2l^&$x#pP6rG=T33p4UwCj!UYuxXB9=>?VdY0l3N|lDY-hFaLhB86dyG<_Pd0{k19j!0M1$`6dx*z z{VE~xk<`Tbsb!yBI4{V6q+5^y$yl=i$>i_XfF$)ZbX)7WI8>7uYTR;mBm}#YnuN=b zIb^v+ZW#!TWLv-J_&c&oGQyGNdx~PadnN5eXYBIxGg?pY64{!T+cLImXnY?~duKoK zPs=#utgWF7*c-C?j&XzUWXat1Tg#Sz+;9{~t>W6cI||5A(VJnjXe%EPB+*vYVZQWf zuY}I3pbHKzT-5T@Z2mOO_;DWn z__0WyDR#{Xz8DvN3LV)wK%sI4aWZZ!HI>^u64%}sjip`wAWc?L?VFkv#;tg%e4dm& zre|Fwtw;)vWdjn6fA?b8z%S)JiW|i@W(~1w6n5u5!T0X1WDG2M^_fxHTUM`Zobt(u zgxE5AtJZ60NGzAJ>@qBiJrPf*4qP1baq&ME*|XK%$2m;=U-}|m0l8$1iZpJvkl<>e zR=ExH5}TIfskl%~N1e7Pm+CS+3#2FytMbqB`<9GV@s&l^OkFlUS1y8S`3G`jb^e_z z)B=)@KjK1X5v*e4b_7yX8-?G1ms8a&9($Ts+gB?VW)FW+@G`)K&XB%d8ne(Dh89XN zid(#gmK2+Qdz$tH{N1X@J~qon*c0DLD7iC_;Cb47Wh3a4O|=)T6DOtp1>@w0&`_E* zUGLo{IhfP0-VnG+R}rj6j$0%GL2H;NkB0FG%fL+U!NH%i2#VDORtOOmMN)!Nn-e(7 zSqkd8C#Z;tfhdU`vEk_S3Kx-SaY}~Zkg;F&Lg@W4Z4BYAIs_!QoX0lfIj?9Gr%VYj z$l2k=EUO7m4WYj!{M9RjkNakUsBL3P$zIj^Sz!0+>R z@T+FKe$sJhG5D=J@QB%xI@F($w@H2^aIZk5<(rYSyjQ%RNs8Hf(IxdZ?Pp%4PT0@n zbLzRDNt@g_%%U-j#VL93XTq>vO&-JC9Om^P18_-+2u?w&%d^ELp*5KJ`$|OMsA##2 zIF@5+l1jJBsqL^!b#+?Hs+2IyM|p@-UG#k$2g)RuJL5aFBb5?fw@oG$pnu%cP9FaI zj+BT0WydtvJwJPXK#%NkK|Qkjn|oxhy!(6WksW<%20TLg6~%5X$Y?!2txUW4pYf&D zBs+Bt;M>8 zEgdtHLj9{jvK`&Y1grKY}?sE`B5~89uX+ zJlv*fxk(eZVqZ=U6AqQxL1*C@j>6@id^3p>Hx`a@qcuzR**Y+d$xDw$Yx#T$Uj~gv z-&Y2i1$)S>$PtQ^o}4aw7eCD0F^Vj<4^ZUke7ycnA4y zm{0?xPNxv5?ex=jQr0bxAiMMUyGfK%$@O$228 zECMu#QIH~X?2jgWZDFmgC<`6EF|q7)0@uHEoDUQm2-%pUa;7Bg_M2+~x6tv438r`! zOpG7UmY7hU0W#2*S{G!ol$#ZUO)s8ji^GvwPv>jw3GBV;{!=XTicrHfvONo?#Kbjc zx72-Ly4iRhDeILZOV0IC4-KeT_EzDP>4N$eWTazk3k9})l0HlTr}i5p*aJ88Yl+GCb;HeeuwSz@YNGYC|S%uby%hc};A8eFlKPY}?jVr|m$J^S~NB(G1m z*ZZ&%GyBYFU#8M-V_uCCx(B!qBVb3INKqV`IOq8~;;7>5%@6?;oXu$J?X8EDY|_BI z(V?s7yle@fEhr|KIek8a(1%ZWiIyE+c`Rx!`_Jm#Vf-j8!5DdpL?M3Z#BOxTUK$xJ zA7(uwJVlTo>yW32cspLdozPHy0$5QBT*t{AMfN=?a_#&j@ZkU3Q{>;CB9DDHPm!0t z=BW4k^Au_Ss3Lz@A;OgJZYnbVhcXpmZ!agHktG|o#< z4o!rXs`-KWCJI;62ormMnWe(!bfT7RAWWVn&QVmHZd|7r5&AR+4Ovx@f)^|hgR;vz z@Ab_!WEzO5p`$hxeT2@;kR4K|S#Euf5f&jeKLf0Ml##WI1DBnXvB5*M7yy8U@ko&J z(RC5YENj#Nq}CAt?19uocbT8{KzEse3l2^=RKN7sjrVuDXwf|%^yvQP@Ns*7Bxp{C z>Im5ipd2;lC7CaiZ=wd$aSIj;HY|QYkzl1G3kJ%?f))%}$7O*uySDNFy6?y|vw);y zz9U~51SCZ&#{=JyKkeDtM8#_>@4hvh@K8m<&x<1EkU*B_4ld6rxiOq@eMQ1G&y0w@ z1c~F~P%65!b(vUv^w`*+6A~{ zNX&n|s^MnvZSxDFgU(jM}8w}&n_6yTG*uvacSM= zKHd2`qO!C#WMh{La$0W~!<5qOo|o>k3{1T0){YzVHF-?lMm~{)qbX-bk7d+ytiWrT zXGV{|WoX?X673!LXmM9&n22d>0J*fw#)&Yl&fReLx+u?`?Tve@4#6tW#=t zv!_X3pDukjt}CwX_p`nmtBY&T`dJRwbIcCxo(;8^5H1<^aGIo6*-o0sg&U#)&aU~2 zpI{>$oJkK*JHX>oCBYpDN0oz|CNgvpBr)Q`ubk@ygyS^1%p1EYG^U#WFj$!Bt5{A2 z+g6a)_EDPhCv&MH$t%#VibSqls6Y(qRvhA7xi>e7C_-=I=!-}3O`Mc!9u+Z~2u_VQ zZl|Itb1b46HjI7+iN1UoHi4q;x+O=61kuIp%qiIpbzdrouH#Hm!AtNVSkDqE9XX3( zcTg8NWvj9;aNni?7AQd}y|6kwI`OaO4PO~VSr^Q$1=4H)=9${R@ zlZNIYp0pieawM|l%u`?4F$z$Djv?32NmbsqkZiZ^JAbP!dkdw3ly^zJMso!ch8bx* zzE?EwET%MW!pS-4Ww=kN$#Fd0*!I&m&c#Pv?j$VjR0O}X6;2Dz6VD2fP-wi445>#2wj6;mrJ2y}Qh zaa2Z<2O)%L=;*0e;57k{v*(me$5Tw+GVG@5$U|5|##FmS9_u`VTG$ziOdid=7F$rc zD4L2x<=kKblbHIdyt)2Oc9ICUFx0%)lJhGZ2vF9TwYfGm(IDbba^$kdPJ`^uA%biwKi$_uKIT;lF1%Wk!LO2fr)K%N|xZs z^}}rk?rxj&)}!Q7=F7h|&1@uPTO4j;!+Qe!VBkuptdi<5CwWHukf70?8NPFhDri%H z%u(R>=It-1YVo6u?o;nXVK#$77E(x&VFGJPMXlWA(Qx^|DGXK}RwJTLj}Co`NSyF$ zcLt{hF=I^^O4wy2B;%~unu6@7@ts<3%+8p55_5qqbMRvy`Vrbo@;iywUMel7ut33f z$O|z(y4(>J`^Rl(SEm)8KE00>d-hUPXJ2s$sj$2t$>h4!{Yia?@9l^7*$Y~#7Z-?S zV%L_U1b`C;(NC7aY~`y7yuW#uv(;SaU(QDEV|8!Tw7wxW1Z6%?KR zood+Czw}PTs0eIgHw%Z6{Jg^2DZPc`#q?CM*E{<77&aIj`evm7LyWr|04IWNOG;|n zPwl&0x!*2VR`cpYc1z$&Eq??@P|zy~69`W0)!dOBKwKfx+Hm5eP1na_553Jr#O=U0 zBCca_PFmQAP>B6%>2XT#DF~#qw2V7i@Uy~0D z5Vo&`PiaEah1q~1ehaYR1c;zc-mD~Z#A~=G`t`F?&*>ljvIUkG;uq@a!(Oa$I$Qd2 z=QdSmrN&MV8V=P8$0DzQR&1Yd)(hyYzyx1sHW30 z&+~AGtQO}O?v1)EN)A)Sx)A?}a5!yQLwe@I@I}9o*i1qN$M{gs3~tDdkl6iu{W{2q zXcBs+=}JHQx0bmufMqBkrx-DQdUNcMj}&89)3p_&9?hJ_Y~8pUpB zcqr8Q)Vi&eGnE6K$X}hy3x@eoM`Y#L3q82!jZL_wea;`-S!HH$iEJW z{!Nsmo!o8is{1>)5?h6Gw&%!t#Jijg%oQUMpK>Xol)L$F&>z(WjsBh^ zxJY@I?nNa&{2E>4CyCv5R(5Ng!BB76;?6@hx#%?@#xGx?bUlS=O{Wu_ps?X@9I{ms z<6=*N+qd}mT|7M5YguIV7;nH{$+SKlNt01wv?q=OvI@56md++#rBeidnTzrb7N&X` z6j)hwX4iz0r(4_J#RO+#<>f^!+eT$C!dGZ{f0Rh8A{^)-0M>)860-7;R)txT z;{}oP;=xjWfS%)uZ$qKDjk}ZssvZRuRx7xN0hT==dG1aUmYTlICFnC^?`=rcO+J97 zQ^V?E%?ZqdIw=PaY|9uh8-e!`#w3cAS3pJDoIGE@2?&&3?Y})+SYS=C-an-VD6i^V`yI@Hx^_1*?&iy(Sekh;Fk=5MiM`) z2JxM?AkO@-A(&BSVy(^5fd^;TRdUz0-AAhH z+UnyE)*U%UZQ2jPs2y|oTbV+>aB%P3~tJq<49k} z5y?r(5Gjtss(p2?(Qco09ORHswvftN(!rXEybm3MEhI@b{T)Zx*aVl==&s;&jSksy zaTu;nq=8wkiFQmzp|XX zMG8v>3W*K&F!6ejl-ywB4Mb*@Te}sNTxjWvx=3GsXud2vNa!PX@A1cG3uW2xpn;sf zdfI|lb@ZCy)$>eAl?-3x^zA>}<=amx%4xl?vIt5Ks}WLmQIO%4bW{>D_;Qlw$dGkJ zW~>q|Tz%7yuv(!iW_vi>_pi<>?*k+9Y3p?rScyXvfuNtAHklKUGcLkJuL5XD3=tG& zat;bQh(W`ysAMLNs20;+?DCS2ih_edlxYw;F+`IJ5tjgxlwWoMq*8bYFy+=SA`&oa z7x^jX5X~+$Nuw<}et>{WVeDzg4l>|Sln_H`HL)G>AO~wLkFVu`y@I)?y5}E^-No(I z%4%M}xOVwX-K}vTmzRJJRj_aY()NL7$nEd)@aR+HjrS2QB#xx6?V?0HVk0-`c9Hf{ z^cf;yZ^VVC-DbHdb$9N&1o}*5!&Q7R@kjs2->a&p82!Sqr!Dzx-HXxrG>N(U0#ZbW z-aT;e&IjWSvK3SyjbzjR9tT?=@QfmZt@Lv`HU z0I#`laQs}RN8GhuWs_H+NDpy;1O14LM;c3lgNX%>5_8%f$U2_&ziUx951P+n4U#7=I^zAos8{QJ;}0K9!ioe2Cd zn=UMEB{k%@Bd3OZq&ahB!us-rHDe-MGjpr&&K-PrPRZDC!u?|+cV*@dzALwS*3qHL z+zG{>O(_1fC>BXboZmHZ)}m0|DHDo6EQ*z&dDdD)LvY<_v= z=9k0On;wNXd+LH^VyH0|0D5IjIGwLx>gz!0chE@hMLg=|A14b|KmB4DR)9D&b)v>jC1u* zqC2v!AgA@*#r)$T{5X<+{F0Ff#~n1*9vBWF+X|eWCY8cLlNtr1?^?{;(;o?!EkPKr zttFuE+ETR5CL?{prg^h#Z#`rg`EN9J%+AC$DotZ4dhqx8p!tpHIQEgX&}X}?~x8eJP^>P z6~P(UwcwnY=;LZg+$rdmAK>0Vj-6g9^a_@1u^A`)vfSB46+ctl1uIV2Zs}-Q1<+Pd z7CWyTVQ-tzjJO>lNye!hBJoVJwU9?}fKV9mL6cB^DyZeo^hHgRI0h)5lf8Apz1%cd6+NZLkZ9N`>eUNlHrDk;)Zum?Bj#bvsNw-1#s;0XtX6!Q0PkZa=v3Z9T*Q@OD0)e6r9`cj zcF=xPEf9K&$wR8kdhHdLciUsEryw{Pyp5H%w{iMRJqpshlfzs}x2}Tz@oUmQzIo(2 zV}HE!vZR2{*e3>c#=gwl8T*H^hL0U$^^XLp$j;wi|0sR-{p|dS-165yE+GQQ zmt2F-z7Xp1d;#*2551J@fm~yseNA~OjbqYF!VhFoGn_*N81W|8gb#_b?`ObeBioQa zpB<;YEcg{ZJ?(x3v|z=!Jg^})wH^qeP}od+wOp7DI!k_5v|D5Srm@|ckYiFHOz2l5 zH1!(I1*l1!3__f5$8XQM?`0#OS#aKed_c);9^E+Qoj}z~Hv-GqF`0jd&9~W9}D&4OBXi;PJ#fE>_^a~yl+1^oX3_?{A zzqS&%mvX#<3nHn zDT8sol49F_!&mN232<##;GN$U#ZA(f+mL@nkgHa36|_2nTvXLjJRYx(P)oYlT@CaR z4$(E4Ut1mw(u^5z%)u(&lSu-JSX!3ND==VzF9vV5umorL#o?-L)3aWqKuzh#T=i*W zO;B6kTLxB&YclTD0BDFnzkywM!LOJ_a6>F=q#net+s)p%B=@{68Yj>Xgv5xgq#ulGUY0e7$~E@H=M9qQd2 zpm#Dq(Uf{ziVw{Ng0e0z9)yWWjE^d2rg2*GEi{!;S@E@C4?u+tl3c3^7g0_DP3DYG zv}U``Wy>HD9(y2U$x+4MLwy6m+tKsvIk#oeltNBNsm}!S8z@{&`3i3$#!ZlB|cP2!OK}Pdr(t+L-_H;!N_g1$F zR&Sj1W|P^qQ?3h${5in)1hu3Dhk){NUPQeCvOTTlSO@fAQ4={M2Gr9SE05SU-K({t zfzj#=k12a}80QN{_kMcTIV6iCX^UQBq8=T#k;U@GzT}I~E2*BQVw&sM5K&qZ3ccQB zP?k$hb}E_#IyUGT;`eWJ0KlnSB+Fz;yhk|&x~}0-4hR#>DiakZpeJ@s+X+2Jr~P5W z@wtN+siUy!UM8p<>s%d8pbB3oSv0(Y*rF_VlUpjcje|!s~UzR zrrg#$WnAy%YyWe_=7y?uRlQ?y@7=8j>RONV7_Tu~5SIV${Qzv%wt%*p3x z;4zWq9~Q)VzzFiyaKot}`<|Gz{GWrOuW&h--CIMSi7wlv;|AY{Lo44N^EGi;0Y3gu0s4e)eI+!Lk;CAMan%)k zy~ibz_;l2qI|lV(rKXc0QydsvL@GaL?B+&T6il$~$Et%RSxQJL;TH!VsxAv1;?6QF zDT@eDaWDav*-5Z*tpL9P1U}@1JK| z9P7=>=}=+GFf65Dzv_FB&u)yjfUL4nXm6cGI4mdcR~9b2#)4{jt8oZC-0I*0wJ%?Y zBsXnc5c?y8;PJ17hr0W%Z3Ya!!GgheDILx&yyR~s#gJVB#`8^Tmh>iPWw~$Jb$BW_ zTr+GqBiIq@wYvcvC`KZz-J-0C~{SlpjJ;DRNZRgghkL*16*Xl`SSm0%{c(5Jv#~ zQR>zssRS~7gTds(bafD+A3w8sCYQYF_hFsquiJcX~ zv!ZNYN=;Qnxu=|17G4m{Gw62`by~6|$M?Tl^HjE2){%FRJXO3>Qn7i{XztECn{}7= z7+KysA=D!+SY?yaf~nDR6t9u&>Rq=KOifRTo9U!89~RmyJ1gm!y;tBdd3TyoQmNJ^ zMBDAd?|2<;C!I7AWFs^e_`J15jn$v`!U2dgy`~N#VLEBgdqydkf+jAK&3L(af{0Up zqhX2$$G?zKw+4r5(0RmZRR;8}0L_eVVIT@{h)bw2weLMEuU8fH2~IT z6djoG9JN)C4kQ_*+O!Pk;SjS{#-&A_PZG#?UBas{}T_E4^HY z+s^&X)H-`)$q{q#AG~pOTE@}~Gbdk| zk@s{#_%EZvul9>xl9JgYFJt3V1>rx93jhAB?2^K8!ZqaymyL;>0BPhMxz#gsN`{9M zPJ?uk0Nc}{a~FjMK^M6j|B7O_K^oZ=|H?l33rizWh!IL1Fd=pVu-tzD%bk}KU4Bxe z;aTy^ckoMyBky@4RJnCT?BayP@%Znegv8XXV`Kl_E0mfuA$ALX_wtCA>P6Ynb)Ck> zt|>?xKCWL8!Nb!EeqTL&Z{MQWF*vps|9_UB0q_0DDe1W_cP|R%zY%V@5IwY)_KViR zy8qa4WcddLu}plvOSil?N0wQ6IYM^r8ly6+)^S~!qssTJU_*fvL#?+2j+;BKdM>{{ zv^CV3AVkkaUyv>SjskWi?0!S!#6s$~-D4!bKJ|vk8GpNd8A&ev9sC1(W)CMWP}nd^ z5`$i2PNSnM@nT6^5AC*yI#2M{wht)DCV3R_P)i_*rk0o0L>+Z>{l*f=ruGE6Rygq# zFCTb|rcRWDY8jo)aP-PBx03s|8ps2XWkw8}ZveDMAccTPd>AlC`kW`YjR96<@UhLX3GC^$n6{-D_5slzqcI^ep zWQ!tKpv#oj@e#gzv*l~_2@b??sb(2u4;zLn50pV>1z{Dzw&4Q ztXTE&!w()iH?n{Ej(-1oXLP6MLY+T<`+!cMRL-d~x2|Dfr!&qKQHPE~@Kk%2a_}ucog3|*ZMmnh%b%Z7 zB+3yfksXg3)S;kmctzwwBJ2$~zhsa+Q3`51PHC>O48`3M554wDyvx!m#0Xd&PSjDq zhcpT`i{gTAxGO;n3qKth3~6GZ=euQlZ^(kk?1Uz(odKH;Y~ddwz=PC?*~2<{marJ{ zoxpI5G;+~8%od_e+%R>|^s^#Bo&A}12U=nkwn``UBdmaig{hSm%u(KIj|A~;%rvwL z*+^j$x9-?5HCMG+tdtS|1H=JX1*9Rs!U6G!VU>-X-O#&whDThj))BmqkZQB?gaM42 z;5_(Ja(2|Nwkf}*egy{fghSZ!>T7QKlgYVUO)i|n*3w1`7uxYU>O<1@6E83+m)(x} zkd*#lJ|v|oRcBrjsIWrtQqDw=o~KsH)xUj6$eW5c)Bo3eNa{)xFG?>vay}#(J4aR> zxnj0AH~tXCYzMzxH|@9IJ6sy-Cvexyan&;!cU`hIbeT9bYh25MOYgj|l&Gg8;zUBy zzJEKB5GCP-;i%rCtbp+Dznw@j8l4`I9!?}LEVoP|(LZg&@&EuwDJ{AI&miGeSo^qQ z=+=k?Tw(3D6s}zQdT55jCvv-?4cj&Q{f#0Ckb8-k6hhu0YOQ=5)~LLg4p~)@g+ue; zL}Hn6NcqHMqe4^?8kCT zoFerpA|!LC&3gn@S(VO21WO;b?bEd6(pduJY#Ei)`e7eRa1UtxS0C{W5ML^(n2N3( zahX>uDjgQ9{&Pt9hz?X5!3j_;m7V?oc#3BvHUn7<=G$60fu60!;@h0Z45R$uk2EQdZjLZHN@}A#un+PYIDPd44FPc*A$XA)GR|o#lK| zB<&#rb4a5>$%0HmDS{xI4T|>RD+^oW$6N>y@F?$w3Y7K* zPmY){tE{T<-peWjTGKcSp^zFs7I%LVT_Vv)POPqm)s_ye zw)#8j6pdJC^=o+<7uP2nTJUA*aVMuRk;rne?@Cg_;22?p$CJ)1CKzTZx2!Pk_GD{m zl=JAdB}hLqd30&h*L$X*QTEh(ju_2WeFSN?e+AKOZ95i+N^ebyltUVM;iZwSkVf_o zM^@v%e&I+t6q4t4jda29&bc%)W;e8uW1xk+FQ?@Ubj5x;JqLo!mhLOAe&yR1vw_)S zCErbpl4l8n$WN9Qf69kmUX+}_J=%$htGYW1dXdS+WhZCMUE7ZVLveZ8^1W;yIcxlu zQ!R&lPoZREE>TmFJ{SJsF*tJoA}w(8&w?K=}1&??Sm;;1Xy3ilI(HgV3z(m_jO&};$v z(DD*Tn&Hr!pPtn=`>q92Y^Ft&%rG&dCUVA2zaOf_X2ew?CPeXtV(N;!gWqc3`Nt3P zZ#sc=lrr^5Qc7Mca8$fs13Sa0}9trz_LQwd$6fhF;X$-bxwZ=$w7s z>?zT*F&`9`OpTV^{6SImLiqpxtIASZ?KIc5A!E(Lc<%&cJ?~x#RbCN4ZjdtlRg1UY zCUJ~cMO$}kKEtA`?wa+eqZ$ka+Yn_pZChmtOWkk_ivbP+ua-`E_Cv8~@K_lD z9-{t3eb2O1m$Czu1OH(_r;=v$_dn$YL5;6_yh#T)=ui9K9c!ff_~y0#=p73EOCRpe zSIOUYac%vLY$Dlq=4r{Lk6odH2S7Bz1ex(KZr)=ysBMj)jg+G@4N@Z-@18y%?<4&> z<3+9_kN`<3Jx`nwRXThB@$9l%F}sIG4L7EuWLG>xHpsYK#YeJIL#Hd)!2ltOx~#rU z^v;-x>@?zQ>83o7YlkheljWSA?!+gvt~tc7b$wl z#S}wuN$D{oL9NzHKWq< z9xjM}HX|dicbDjC$(ea4cZr^a|9YJgeQ`!w-r|Dj8vL$jm&Pr_BB7e2bE4Bbjc6JC zi)?bDTzAylwJp^@&yEf{3OdS#&{4J*#k!rGw5)JN!=CdqT8DOtG^gdZOoe1}Tex8e zB$E$zj$W3Oxnt;YH#f_QujuPTqHl1~+vz8F8+S5;rX52XPZ>6JGu3mEUnF%(^Y)9h ztT%l8B3T9L;}^N*ZgLXwt8=^knJND0t?v_imjJBLN-L36k*Z89xQz?b8cS+63C_wB zxZVD8y;eNhaH0ZBoHJh5_l-0 zwh$D>R$rs-6hsp5C6Hh%A)*o|hS*OWutBl4Uke=)XmJJ_MbvJRfC4I-iwp^n;a_{7 zQ>UtKovK1@`v2bl_xJgI<`y?~b5EUNuf5h@8`~=Htl6R#3$ibxT{gn$-9G)YK}R)K zQzfC77YPN^{$3;)pollg#=Zr7bZXHOs55kzV@5&xX=^q{pRx)dE|^=2Kp;9!R;u`1Dg%3{VDd81T53-t%&k0Tr^cci01j7%sM?bc0V=w&dDmREeArXOnFEp=K`yq)1XbsWFoST1%+eeSEW5j?Qiy=-+hsJiMi~ zQz~sjyfJX3qJkAHF^VbyVB4$z_~3*kg3qyjJK3^X*bfw+`w1=d=$OI0B+4fQD{6u+ z@mf5%nbaxD4iwTQ=d6+xNM+@_HN@ZqF>_m{FM}F18>|5RB^714DuyE1WE}`WOL+!n zY+0hALV4(Tcfw76BK~z|yEQ`_M8{?XVMl;}rD)SZa@sz|Y4noz1Wy85Fsy{cqV>eT zsO%E>SB#qmKz(jCH7V1Yijcp*9{(yq8T|c4V6CRBIfE7c#m$qnt665P*c!+?;5GWM zTT$HFJLzo0f27CWVo*xa6sUTw;%W_tO4B-g6&xD%d9qJ(-zsKHEuh%PVEE-V8Id!r zI+I&3MpHX#>n65m5}dQFvmE1R?b%G5{%CZ0ygX_#s9~17!h18?dTALpbD^eSEpvX* zT{a>;%-pgNwC~bavL}Y*j8)?c1GDPcu8&z{A_rQMS>{AhUkaCmX$iov27^i62nyf) zbR)3&0MbU3VjFYoYvF5R+i2(!P@%K@tsh~>X)azh8uAiXRY;d3xrU=-3SdTIIHDez z++fse;5{FA-quw+-()G|cSoxwTgSoo=l9I$KdtZb`!fnhgaS{O2IdV5&qvK#VMgJw zP~fi8z-^PWBV~b@H#)~WkD4{u`|lc6urn|6Kpp)ELl)P}(g~6KZ;7;-?+(~|BIotn4cKo_|&dgx- zF~Np4rvt~e^{UCgJFPGorEB;zw|9@-I%H9ODInXK9mDGY*@nZgq$mDr^ki#mdnK-X z2XQoEH*5cIIcNp;EPP1F}M6G_Yt64{0(mlDTma2+n8 ze|D-bVq3#p9dErx>D&#B5>3R0x44KMx!(0-ve+fwjoCyE4_B0c^H(D31{Af8An2?! z_P-m6Y%TUvSP*WOzsp^{&8;|+U^v3X9W z*L>+3vdYgtqtb2~xqW&}mz0Vw4Kri9EU4(kExgNzQ8JEyXR>jdB_cm}yNIKtF_`gn zqxmf(-K`{GnnV2v0^K7LL$WIvXL!|)Y}uiqWAs;);Vnf|ILE6y)~GXF)NOM0)ep}h z3mw(T$u2l)`EapRlxgr9h1gab62$m=0HESB0fwW$wi;_|4Tg((=kZ4ZhPC<=`Xd;X zOnGe^8H}ZIjP1>(iJZ0=l4!<(hM1l3-E-$;S)eRNq9bSZ8jM^&4ihw(hT{#298^Cl z6L?|R(%#@O9wC5{)pmaX99fGP+61>Y#4Plk*1%b|xiArm!5&8k9h{_-TpHLXIV9Gs zwVJ5PZ(o{f+W?%1uh4kg69`kHflC~Dd$IEl=1hk<cYc74~^8+%g$2N6%~XwIRk9hn;Qr=M}TG3dE5VbjyQ8W({QSi|~`1pO86rY%1dE z)jQaElXP0dmV_D@xjNq4y~Y(NS&WHEu~-Kfk&7sh)Th0~xMFt`%fMXUYBtg((KOOK zH{kgc3y-9E+wyrVwV6~N)B>=d@ z-Pf`%+vz10S^m&SGU&QE^6BuXV=vv*V{#>B_0_*=Mi$vk2*b{t?OXr8%~vHd?bx}7 zGf{vJW+&-;2fp02H{o9tm77!t7KLOaT=~;YrnJn?(TO|`da+fvpMhR%B}b23TXXct zwMxG}+V$lp43GRwC&`y)?w|OJobZHW;4aw>|8K9T?=vtlb9Q#k`nxik@9Z3G>^Um3 z2A+~nbf~!zo{||I!%GrjoN~%X-@{^r@q`;IdS^XO(4@`RlXrTGU&T(0rlSoll zw0uZy{=Ectf^RKp=xQ)lm4oG2ln$2>KGyp6Jnxx3#6&q6I)&`8 z5fRg6;oWF5&pyb?OZF-eipaly`F8OKzoKEVKqKB*jY}}%s4frwKr2_sMumHk6;-vQ zjKcD+TlnR&Faygj{X*$#e_?Qapd zlHbR+-qYtkSv@lGpV!f-Q8b%bYOM3$%S!+c)4zRMtZ@%RlE!m?fH>{{H6Hg3<1*IN;q2?HZ!2DYE9oU8IayNW zWAv?0h?@Y~N)+qIb)3=7IXWY+UIV0~I#M10Mir73K#v;=7ENm4+^mSefBEDbUgru+ zDZXaT+2LM7Je%3C%JNa>W5DN#pxYLAn}g5XBX)(v!`jwVV!n|Sve+BF;VtO7Cdmsm zRPr@ZZ9vdlL}JY+`hClZjm660O=r`_mmZKsgh%;l;~x6N z&rBiZ_Qr4v-Tnv(je+5OrY$g>ej#Ie<(|m{A*@2g2+36*J@+JYrB{^mrVgrZ0zW_- zG?todfn`B%9bpew7t~9E@2)9kF&oRvqtBz-8m?k?>LSdW`liYN*Rog(Ip=7>wT0If z_I#;sQeMGFpEfXwJsQg-Zrzw2T|Z$e{+{?K1{% zyBEwF^FU!-h%S57voV!)oS>4@x_pqvn#)9fJ)&oq+ZWH;+d>W`o9D*u5UsTnpDak? z{$dxf&<|((s4Qu}A-MrZ8QOr)NWDodsU(Ioy&loPhKQruJ!px>f2tFbanM2J8Zl8c zae<61pJcwqx$Q)YWoR7733Ou2Aa?90k)epM@P3O@k#c1){U`ZnlnPP?y9AG^IUCfS ztXG^61GjwqQQFcq%7*D^Iz`)6&B2n#i0J+lV{8{yUotZ-ebr0@mffYkdzrXIZJYhb zwe<;mW77xi?gbI#?B?ZjvqdUr+nT(9WH`4!TY|r3&nTg4q~6&a17FLp8|7;xt}~xB z{*IRoOUcYfBt2EMjz5wxqJXdo5DP!}oO%Upe8I#^^s`jKX@lWJoH# z2sYxHGE;}u9=iZ zypLI@S{`i(pwCGnzO8HL)9YvTvVqfzU*!Y>il~#){EI}YOxTHd(iFf~ESAw6R#YT~ zZ%%qhFTEZPJLnixWtwuOwrn-1oOLd7TnN7 zikMh&Z<1H)AcS6?Uelt&!o015GaQP+6Osw7vMm%=tbgs%$bH<9t)%*c%W4!$DRQ-zP*}aj=ZG>3Tf}AwL^kS4@yYhb)c0yxX8HXpY@4!q`b^yWcQ zcjG1pQ!Ah4VD8m`A*^GlArTQR)ZYQ2Qd|W3|A|HLu>k5pQ|LfZUdcAog$CcMSt?IX%=wv`HHrc`S281nvj;RRzW_ zmYhD|OtXi_wPgqXJI&L&XW^G6G%rN+KLE&ga#tZn)CkZLVI38SH6ZC`SLrHeZ% z3wAQq^u(OF=Y{vk?n>0Xb+6}2H8}fIQIv}Y^1NrZfretWorkkOky>8P{@?65XGup#I4dtM|0vq{PuS{*M8<}j}$i!aiE_CUYLEEb@ zJXd=sb!Q~l+83O&$8ZnW#<2+kvTU@FY1sT6I7 zn~>L-2=rXEBWNX8M1El&M;zUT&UbBzB9k3cUPC)uFkOlK$`i4M8#(7XEWW9^uUf1~j*^q1B5{SAkulJ#mr@W}u8r*Q(49jUrCm33&;s+)AL;stxcLmHe|eKuziK`pp<@=QWRd4kQ=WGnZg=M+ zp#*NG)e@Dw!E5}NE~Ye4u}`Y9C%eg5ak4~y>Yx=fz^M2z5))jCDbi7tgEXF=w!i^j z&7j5fJFY0!P)0GItUaWqeS3mmZTfJaud!C~LaZ`OdoxXE&s8pesBP8GW!2QTZSUql zOz)(a6DxuVXN@XI7+uhLc;vl6%-e}EZ;YBzKO2p-d&Z@VTeP}!)#~FPt-mKG{^7Lv z+tcC}j9s)k@J;eb*~u$k&JG>yRr4|$vSoGMGAKFo#q7{mLqZK57EjBc=tseo_)u`y z*eJM~6 zi|Y3eh)XG-QU7=_e!-j>^^XMOQ|8R8|L1_%l)OdtpWxpfnGvxxZc8x@B`Xa>$G>GnolHjY* za{KEC#}R+!O2D=JAYr0gh1;6Z>^5?$02tvB{bw-X7{agn&m4gU>L2Z;)^6j5^sXGj z2q$M!cxWz>U|VwOHEeK(v>p&2W3OB@S+<5r&~EZ#$FEtqGwjDly>VNb;Oht=hOa{H z8mK_V%OYCP5pbXrnRsAO=fHI>DT(a&$j*+aZum;F650*RB~9z-C-lI#U+4 zC7BJ|p1LoYz^7dCOOg$>#97RXG|AVXvwpi^gpsuS#E_?^jKrnnhOhDEs>q0+Qfkh& z+Z$Ic#<1d0qB^bCxB3YfUD{RP+PuQc4P?EOF2TPLP`J#i$c%Yq!JXGejJ|DBB>4!`Hl@CGZb}d0dZ=*%j>;yCLAIYvIMn>=0>BaJLu(aki-6W^ z-vv;RB~4m>frj@^srAs|h*w348&^umkVKy|t`wPTOZGSFRAa|~s+G`KN7A9HtRu~u zBrX4ggT-*fiv}9zqhQsQ3_$b;ARq^!BPUuhZ)$kTs5rq42f2KsZqna!sJ;z$`~|fO zZ#u1^@ip$FUi{?aC0QSyF5&^~2L>9jC3p=2nFFk|ytD1WZE)LOj#U8vd){9t&2L6) zDtd$eecM3>IA+AHVr!1dJlefdM=G(3aTn2a${Y$c;lw3BwHe~$T9K4jff$56hTWgK z8Q!wQ**r`xfalW}s^Z$kDI?Vm>pb)=ywiFCwN^VOZ1z z9=lPLKRFpDY(#Q&t;DYztRn%JmZZxj0T*dctEEYZn?b~Kv0}aFE?Q9-o3d23$pD)} zVlD@gwN2VDgc$@&2R+a=pF>A3e-Z=g(w6NGv09z9iP~>CIP)lD<_(ff>8N$Rnxlj!6r6TD-6cDQP|a51saCjc2PqTf zka%#>4^q9xU2f!a(;Y}8D=nt>J7IXeyf~#%h_KO&PARC3l(r9Pcnfk=N;weRwK`u4vkqcA5F7*ZOzcvyHRu-ET03Nu52Qzs9ncG|xa?X>3v z*Y_43wzKp8#S+PhJy5gO;nDgxA56q?Z+CC6vbUC$Tk=L?b$=e6i<7% z{O&yJ$+iPwf6+@rYRm88@~+g~b0hP5kaDu-``_&UWn1#`sSFABvGe4^sa=+e*KKFl z90Rdl$=KAd`qc!_@v3VZyKv`J@YkA zW{ptNtg2(7qyd55aau8z^5LX5s4+p68RQ%bC1Ed;_lLS*I@yq%iO+PiA%UQh11l#6 zIe;dP}7fEoEPp+Gy1YW!rfBzc-@|f-B8usO=nr ztMUn+rB(&qKKx;kpU~_nB{WlElUbCVO!dwHa_{4>F<1+kgjM0cLpOFVR9skAQk^LZB&siwV}L)&$HNSYC*MLVAgJtg*5;)> z?+9551pvWPsg6g~fRAxQ*&M^5n2ilY5?`gS<&D&$65P?FjbbZBG8sc=L}|#fSHO_Z z-F;p~62NXlpk`d@((Qr~b0*(a*ajq6 zboi&@U$w#!gMJa!p3WRSiIM@4*`kZ0G4mhQE$Gq_7|)~q+xbyRAv;R0IsM3*Tvdyr zNHyTmBS=y)1tuz3h3U{blC1+Wah!IvV-mFn2&lR7N&JxnUh$-bq9Z|yVK%7NQ)bP^`h0WRjZGC zwEj0B*cSaNbZkb=Tk%~hvyxZ7lpWeLB=lXcntAbE#|`Rw(xAlWo4U>l-kcCKwmevl z0=B)Q3Vz6oJQ|2Ot~}TXcgXKY6?``$Hf8jp)teWu{lwS?@#Y^T`<~!zZrfW&kKLZmdb)wCXo)5 z#~nvyY3FtSik;;{j6u$LCwQ(P&`S9An0JELz0i=!rEczR+0wgzBb^sz_-cnwh-gdZ zO*gF?qwrZS+wRA%J1m8v+G;lxzztq=vVR>LU{e(Sd!hH+>-q1IJQ{Me+`Lt*c;saE z$9PM#Pbu3XTJqJgj0~xFHUs68%S!5wW1VJEYFWvISo~Omu5QLz#uOT_c3W{9HGXSv zAF-(D*$kz;lu32QYl5o8x^7SHy_p0gCmJ-)deVDxkd?0XhxX!bIli7F?d=`%TL%u! z6oqYtTe@GUB;4 zbSXXaS;7MF+D*w%)6i;XD_RZ7Y9Et-s4Dg?f^L|)u~kH#H*np0)t&7%YMgxkwKYVk z75rx zN*#jg1U163wjD$)<0R7%YD62h9hggg9Tbx-^i4wQ5td@UsA|*bnRn~9!)wql(fJ5I zfc&ZTj9dN1BJJ4=d9i315jMQM#kXU9e(rU6u z+1yNUqY3}ArWK~pI(nc4koKI!qScJI8WjW;wk*w)<$G0{K&*WI#57Q{VTp^a)LB4F zPXS+8o7S}M4XHlEiU!+zt}f`BlXui`+@F>gZyj?|a^)pm$6b<~`FwWh!y%z}d(~VH z-DGC+%IC5}|2rh~r*nH2=7s{-l?KKP3;zO50q*X*{H~0`L7~7I!@}!G^7usX0yG7n zCII*^FLJwJ-a41;_}6yW&Rp6nFLGOUa?6Q{C8fU!qTfJ#QOpY`9$ND&82-cf|3`U9 zUOWBX>R_)@B6|AYHDdW)45}_)7aSul-8-Ur_Q>fE)=?D}O#hjwaxZu*^)Gk>4D&2_ zyJNvZBJVt*95GMM?rWyKW1qt&-zAr;QS+7v!%Q6S1Ge__cabFGiqbohrjyK_dQDj~ zGmugQ|6=>EP;hq~z@aSG?Fo@}=o*e;No-$Bl3Q@#ZOVMj$l70Q^d8Op6E5Y}-blm@Vq#4w zZ@olHFGPNbyj&|Wr07)axN5()9eOO_bWM)&LxuOrLJ@gbKVAkOw83*@X1}P+tuHg5 z_8bRqwAxr}a>%$Oi<)SZ>|BTOnk`944W_XqAt{*El0->BGzT><89Gqga2eB_GrCrQ zo#^z>%fR0V*eblN#X)F(l6uX>QMTJEn9QHdmLyY_omIwu9v%g5=%%GL&NGj8P-OBb zmtvZzpbIkug;;dTx+CPJWP$ID3&rCq6En>W(d`BVAu>x~jLYzu`ErhEHSj9UkPmAbMnp72DZRxY&_u8TD=!zd zj^2t$L2alRiz7g)vpH^KMI=_Eeld&W6%>&uyWx$`zNF0&09;e_M>6rvR;8{O9w;Iy z==PeoAlj&Nl;g4B530oO+HIs&TToX8O+Psy)ZKyb{>G1A|DpLBdgacqE@%H zZycg;jCNej-hm*%m{>GHl$vJp5~w)r>mmGN-*rFg!+g zv5!JDi}>E5MGL7RhBG;UOX<{LJ<8HkmIbbRAt$mn5c9;`U;>!8j-v~X8y@*fAm-A|Y{>g$)fBGS_z4e;6=XzO2K3+;q6g75` z>eA?`uT>XsW*la-77=OJw5J$$y|-Q1bGc|H35;Dk_3idpT_*vvI1VOXjq%$FnDIty zz_mnzvvfXk-I^f3uTQO-txt{tai-&gYhoA5eaCzx|A!^yqajA1cix1 z6B*r_jm$gBm0RRTM&yt$?6|fN(zJ-d%wxoQvS9crCZ`61?j>vT40Z-u;?Soy&zxcB zP1FL_%-4qWQggbp)Nm}Il~LPlKT6C42}p%em@Zr>JHYl^8;`(X8iW-Z9h-5nigu0r zLJSGt50aFXR1eIpUD$);@U}=|T=CRcK}}7Q-;BVtTXZV;S#Zhh3}yiVs`^A`4pC=u zxgoxBLCUrCy`0>emGA+Qo!+ePk+PX=v;4YD2*W?^eICC-_{oi6o|Jn&R zpVpASg3H)wV8{-`JioPZljzVaq6YH`sVIU>`2f`PVYB&u03U* zA7pX?>=N@_Zj)PzH=QfQ68MpOsLagqU2*?gtJ)dw@gFm8C}^Vz&U}gqzHqwWElmsg z+q(j}U4p|A|DMOo*s%#5q5s}0%oVG6mk(I=M2vVhjlyipW@rmKvB5MlS0IyAPDtQa zZAqN+0Dn9omU7ep3mC_fotup+jAe4#m^?{>Uc~T{B7}gcu|qo9bO1HH+jT#nCGWh+ z>~VHLTm|OJXK1YnE$5g~4^~4X`g0T`P(&i1W!+unc46HPlq6eCX1RFiwMTn$wW^g- zCXJ2}BFWT&>>CKn7vwK2sVmEJm)d?wOF5p6`K+MLbnXVp73SDW+aY>-QdH(RM_3Ai zb!u*qDit6v@d+<>gV9$!OpDPu_o<77_7mjDVcXfCQ;f1_{oldw8aK54AA3J-T(-Jz zc*Dv+Wrv36hK6R;4C&N$%aE>>Ly}j%mK{1fH+0s=X@xh00+;87I|9DV=-YorM&b3L zK(DgEsd?cg-TE$n@GrsR<^~&2A63wn7kLRzBndHZln3+i`rxR7=ImWZkwk0ww~%N% zekw>az?FL=HsZCY5!(|4*4!JhNxyxMMr;#^#PPQEC+m4nTYsb9o82(PQbzU}(NbC* z7aBlc1h)QdvIBvWBjcEZZ2gOYuqYkLaEnh&`)h>a{4pHh;%_E2C!2E1K5X}(roy*p zc7iX3RwVycLS{q`7U`6bL`vC#m_UH6H|0G<`HpaM$4=|9X$hGPT{80+o9wH*L0>=iCrE0k-?!^ZmEPM2^Z5SlD->77kNlT)uSg=ktj4Mo&-U;M87lSvvOq|INXsdsNkv5@L=MQS88 zd7>XViN1h6J-7C0qiiknWdqy29}n8Cc*8%^W@mZwIie$IJUMa*oXr_@4Xi`JtJ0b3#Igsx{|b!=lo?9;r9hMo zS6yKkg=wNPS#N>2C_@4>fVI6JrC?E|6!O57aHF=aJ4TIi;!e1>mcVVX0oSbFd2Z2z zKJl#{g-Co0aN<&Q0aFw~Iz%6l>NRfShPL@Jap@Z9$}*y(yn&h?Vwu4)b5QOYoX|K_ zN-Tw(a{8o4cw(x=u1*7axG0s^3xX8|^(@bV{Q#=-hy3sAWJ3LEJ9)E5dDxbs4t&x73grKHVT*C z=5X@xL&j)=CA|F9n46DU#J?gT=AI4Xq#DK3PM$%{E}xpKG!Rr^3<*+M=B?PN&28GC zoJ?X(-Cq*JpuUuuqxJg)Zy@~46l7P#)8d6Wx1y)GVaCN7gXs0o;lXt<@rcfu%_eUU z3!6#SCDcXE9~ejDC61Fzelt>3C)sXkhwqJ7o%jYXZ!~}n{Sul!?nC@D8| zLPpIdl&oc6ALx)5zOP%~{>2%ESA_!GO9NjF3-Zy;`PdOncGAa ze|J0+Yl8t|RdwArQ?j@!7!1VEiyl`D09O?f^8r^!?F^|_CykvKJUZhiYV#?U0 zjOP8RaYa=fYUZJYEpu2nb!T=Y1tn}BRMbztC^2(Zc1`~8(hB2IW4Gj!$xWB{{~ZVW z=dKH0%+kl@caP|Q4_~@*UCBb^!I}@xKRN5~Jqf}QMdb$JK3URydH)Ca7u|gSubuu_ zb+F4jK@VrhYo{;BV&{8zYsf28zv)NZf;S!J@)y7$9 zHxqZ!aF*0{puD!|7(h6Vq?}rY4YEs8W)j)&glG&NM6)V8iR!!f2nb#-$|Fc?`v9<87?cGPNtAa37z7wX)~6)RARa8?u{9DLpD-M zCXl&>r>mTFDrs(-y^$}MXAJIUUk`|1_PRg8CpSh9yjX_Nydl3xk?AR&trxO6A zeF?UNgwy9nsyJskMHQ|sHE!DPjN@^WP4T@~ruM1b&X_KXcLUIu0@#=k`0y4coio;U>hB>&QDUGjFluR=ByEmJoz}NA#X_V zRia~*NBtUPj^zJkjG@frT7wJGnOW8wMh9$&_0r$?^h9uQp12s4l&E}k_jjO$Eu-qF z*+nuiF_c+eTUAX<|&StSTO| zT4zFpoQ{OcM<6O&PF2=+tZki>bbmzED-85)j=1a-DM#047NVtk$WP=mA0ecTt4p87 zlRSE!k8~OsM(-tnX-85^1=)xJI4NY0!l0sSvcasIsRe5;2fr_JIhv@LK(bVAI+YLR z`whZL-PZghn&as2X_8iUMmG6X9!>15!cYVUotk4NeIqWxxgU|Y`-Xr?Cs`}YO$?q{ zfjYRK&^mOoNb(sW@V696nt;eUe~o61as`{;Rt*5Bw>%422iG5g53lE5%gD+{xC?lu zU&#n3K3V&h;wPw@;xRI`GO(9i!O@HQ8kmAP6}_AgHkdoSrJsgnvz(>B9_nwMW%j4k z%x7`w^n^pHYC5`^jljdNf<*QKZ?v`nS+* z2YxEe2D^OLEV|Qdb&_V|LMd084a8J7u~Hs18`nyC(rjqKT@$?Ro#0&zRQi{VSYF1s z?`eJwqlOniOtF`VPNhgx*T@3`zB#d5I@45$G#Vwd=%mqjrb4zkxrJn#$cfphH1jou z+sZpp!o{kmZpq^Jvn-Rlv9arw9Sd2J8Z~VDZl@MkNvPG(^dL#QTGHpzpF-7Y43u@% zYDdtG4R4w9As&&33KAzYATW$}-Fs-0lB}sO56^l@zs-)EBLd)x*mCmnmq5bbYXm@< zcWZZ;*5HI`ds1(_Z*H%5|9Io*J1%di%DwDx-MDv4X9sQ>^v&qc`kvZcx~Jtmix&xw zw9<85@(M38GJ+kYJ~DTN0pv(D;tk|*lYAc_2cxY`(LnBosS|2ShJBe^II*T=GFkqk z(%`EKLlEW0yUBuQZQc;0CF(cukSd&^hS)c)?OCE@WC0m)L{lk(GbT}B4`-*LY%^2A z0)g=Bf^J~U{Js7AaoDSgCT4HpgymEI4rU_^yFweVGlRJjMZ1w@M|VAU{hTBQ4nKvzKgo#=3-xre2CnJ9E_?3iZoxbxeclAtoP!#5p$UJ z{$;h&-=F$xTQ!p3BpllAQ_c1Y-O9-N>YfE8T;p1~m_wx5k^e9iSSo0{wl$}i6q|NG zGb<>$dNhA^Gqg9Slltj}tX<_yIz~?`v(AMp>!clzn9wK{2LEr@ z5pndv9hW3Mt^G#Wsi9aB+Oza0+@e)sKOn|P2`tU*J1)&cu_p5o(J2b^HE!Yt6nS`m zri)J6d;I+e>;l-QMdW1BWbll?gBN}DqcLAMEb=4SuC%OZeQsy8TMIjr0z-wSv{^yI zZ2<2S=7bb;w66gnT($+&7#QCF(Z6aCC6sjl49Q+A3^z5|6kKXG3L{OV6*VWTXn7d~ zno81`V-_l0`Nk+(UOE=Z^nJlX-op<^gJgI`p&v!@hH}t`ntXu?ap^q*0H}ot;Wz7^Nmg&!O8j=1$ z#xqcxyZH>bt%c`q03uYDIa@bB>wY+p-FNliSavrDIqWAtm4vIY9dkTLNg@+yi<=lx znjyrjGP9v1idIWmGh1-UIKIshh@P0dlI14rQj(27^=Fo3qmXUu-tEOGWNRpNFJ#*l z^69*GidW~gO`V$RUYj;Ju%a~ZGHgR&;6Jr*|EU>;!$W}?!@~E$R^-0E%kS+SEQfpK zx1$Qa$%}*oF|#_y+zSJLIE#Fh7b%3T$VZQ^e!HUnD>yNB$A57Pu0U(E#jD>AMGA#y zq@>x3?u-rZM8CCFJ>!aG>$O+$<886w(&WB{my8Ur+n!cO$6(;lEow)LoFJ0mSNRkmL1vS|ECsrQ0i>ws&-vh7WcYa7}z+$X7Tb5r-Y zwu?H3n-cprufvaLbque=kMm{kwW)_z96o$W|MDGNccn*g)w}j}Y;GEJSKB?0My3qp z{$edJcaCdc+KbRy%NqRn8vnTEl7x|ajLJAScmIyV4iYcrQPFnEfc5!AK%MYpL1!j@ zj;n|}F^Ri>cQJ^d{R)9e#D$bO@4lpE#HEsXty%!aT1H_J(!CL8B1~fG8UPR}m-5UW zyoF=jALTbv!N^XMY^W%b{FuyDk(!b7s~3PqU}$se86n7q``_#h<7615h`2dhZH zncATu@@Y2L>eb=Ljyd=d_JI9p;E;Q>Lq_!)9`^pm{_|lsa{CnJ66ugiVkf-hAq;ay zt(#7Az^+Cz_8%miBAb`b%@$O~wl#SHsVCk3Yzbuadqyz|%qMq9-gB?z*NyTuqUp>h zjXn2e!!R=Q8UFB1WadxzC`3fw$2dbCl-csK{d;GLA4l&)``42%!0e@gaWVJjH_gmy zyXPSPmr!XnuA3eCGv7b|9h}ZlY}>}veRcN@KltfznQGIzj^GHL{=*X_ z$aD!ZF^xaQjXr%0H=^Us?Py0lT%B;w-eqHmz;XgIk<9D~z;O7u%=Ur9o$s(l*2_HQ ze2m$U&(_2b>e6{`{Xub1%_hQ%EliNk6vlGqsb#|zz-)vsoy3w19t38iiW?_pWAhOU zW>Ym&>R30C0%NvnegtMpG5V31ruk4pv)IXY100OCT=d8DQiBwDZ!!S0N-mq{E%w)qCHvxMVk7H&d$(3AqEz45Fr6#>C-9FbJ3FEI> z=)())JF%9;?Q4N{lQ(fzrQs+{6FMe=8U)_Z;}4)qu#rAfSIB^^Q`ZWY3%Ye;W$he4 zG|D6R*_0VjI74X|ohKPqGEERH1AS{(NRJ`>&}4Yb*=@+NqjPG2=#p@QF!EqE2)E@P zgxkJsAM=qDJk3YuZXFXF_d!bh(v-NAF^g93SiJhniu%*w<)0F_72f{;SiJi4ClfQP zvO}*73BA~>rW0%<2X@_ZQF7+O?9k)AYPOTN|K)*GhlS^JXYA#7W)xmP-u{78I}VmE z|3??GZTX3a?T&jvr=<}`^+ES8ySuZV?(b*Ud@Q=D_2>fN*RdHEhh5sP=(u(dfIvHJ z{a}@(&y)8jF?O%Kx??LuYROXab%gL_{#FJZa}e{E8A1HSD26N_CM4<@H^BZ zPY=NY*in`WPVMT_yynk(a%fYEAHV9ENbz;^cjD6NNL>5qA`up3(XTkctBOf+=9@Nq zX(ZTk@}EaWtexzZZRXRhXx^XblgeKUQ|dahVrfRpuh`H(@;-h%&-C_(Hd2K6DyWK% zzJTQtJaBLYL}Ap8jhP)YR)~%r@Hql1dquoT9%ehV6NTXjPhdioPRXbVnN=fp%KX?t zF6ZEG>x2qDGSaq`6G?{I1Rh3U&AB>z-h)DwAEkww-Pese0PChQ@zK za^S02#E<4{tzEZe?AgOm3fbJZg(+^Mijr;OG^~xWRQPI!TDa7cm$VUESrChLRK?m* zYXPZ+7f$=3?K)NVQqb+HX&X!mSC4=3mTDn_j)o9wX1(zFj5ej1&ReVP`Ze>`ghNL6 z8cYX^QcyjMD!gUB5V%@-3)|@asbecP1+(FV`E>^Q zVDTPvraP%P!9rnXyTXajE&diZ6}0Twj1m8DW1YH=?ky;%^pQqF)4P!K8EP!klV}ke zx<4?W=tW2rtCob~-y(eL&;E#|oam}cRYaBRegIU-$+Bn)$$YIPgKY?Sh6@6g*cKBL z3S0V9DShtv($cG5Z#$EBc-Ev0I8_ebj= zhudS%_}dqQM1t)|Nk(BoC~$FE;M~0Mrzlkm_gx;!D7-cl=yXe7!7;-le+TfFQ&yI<|JtclbO5FJWTC}>pqP|B={0BYa zmp&U;(<{E~mVsR>2PUsvlpR_;B=l^rn!Ryd#{sJ?K!Mtd48mp?32ZhjJXylEw2}+? zr{)iP#-@x|RR4?8w8CG7!gVNFldagOdhJbgW$POXe+m{Yvvc@ruxJB2hcCqc28;~f z2Zk*z6uuh&c3$UNs$Oe5J}+F2>NVMlE$i5CW*(gQi=1%QF}T(Z|8K9TpKy0tVR9&3 zyuPA-;azEk3HTA!Yjr4FJ03q8z1TWVZUU#n*zAL|Yd&IZc4m(*BYNdxPN|Rm`D*73s%?QdBEL#8wD|LIBvCQ8Q3pSpF95S=^7~8v)7{qTk(d}EN>5gnHXCu3)fPW z^~eLNTlSv9J`+K98M!eRNhHaq)_Hed>q^L#M$ZY))|@JRLjYRWMIOd{ONvWSeX^er zlv`5Vdb%_e`E(k(Uv%RwbBsZJPNvs<=^CP?oF@LYhvR!^ruV7s)Twu7uVE-EfNrd0 zm{R*sHg2>;5Jb0AHj4^oe3Fl;9C7^|w-KoQg(fUI0ybX@iHAMPkOv;`@Z=8(H5V5+ z+s_j7NP|yE)R}IErP33cEhPE-TB;<>!wQj(K+!kuUM#p>6J!K`*8?p$EvL~bj zM$<^WDTI(h3F0%f6mTdlza~ghN>3s(wWi-_H&D%TJ98*S^3r^5c$v9HYSr&P#O4d^ zNmzuYYjrnuBOz1KlaH4yyqSNbU=*(U?g&^kVFRQ>L@nYp^RcM7Etbshm($d>AO}-> zrtseDhh`L!2bg=sg-2GGAu>Z)+$Ga{e8B{z%0Jjunm1NRzpS-|l2bO}20n@OnI5xu z^ns485PP!h#}e6F6X;dQ9-PhJTIR3Kkx^mlW0r7PGr)A|i0t5mxKiEL{3Ln?y!9&o zAkD17C1LHU4Pk>fZ~%p!MS@AU2`=@S&XG3<){tD=(YJ`LDc-F)9lVtiewkmx`pv+g z=_zp)kV3Brj!p3D6=lReIgB9(i;&xC^q}xW71PI_bC$DRYS3xmAIZlG{!ZjN3H)m_|PWp^+^KWyqZHj6<{Y=yI{Ry_mOATNowsg*fCChh!!gWl~LKbRpj~=wUlMWCj!nfCssKf{Z9W^KW8> zw=}k&=XUzdM*NI`?X@7AHvArjw>q+y=sF*OW3p$qTRH6n7##3w(5t z1UlV@4sikDJZ0-Cs0Et_hbk^$=OG}3EwZpi-Qeh)F@yx#B2dG+#y8TGs3eLMI(9*Z zzdl2&iZ<0ynbQ)N(1d`#uN8w>tOD~DY-Gw4&;`M5lCz_!u3;pCavu*Y~JX^aJWrW6K5K9 z1y#UH00|~|kFKD!*csoQte~0i%?ZmO-4#GDHju9w>5Bc@eC=yH0$R+_U)b|48R>@Y z2skgXN;T_TT>43VITA#hc{Ol&A~R=QGZLQ|DF;Kc>69M`MS>7;1F&u3^{hF|*aTQw zAnK>=L=1^n!JZhM6$`KXChLWjB2)wwBJ7e8f9@u$}F#a0?G*~=41n9rM1aU z1&L8()Y{786Z;;G%148x_sErrDTR^5>N(8oGp-wTRpwx}?REj*@Dp&e;yu#JCCs5IV_20*SXwF7(zwQdS@1=T*BF z8Cko_+#s$rMspcet<8>%7q97jbvlnUSPUXzVYNozf$e#t0~=(LU!Tk=u9W{m&R8eOsmDEEV( zL8NgAyT=RtI9@P&53B7`ZpT)WNwsI2kpM2m8hJ1k$H~x6y+-MUkFWvsnatR0H82>( zr-WWpo9_J=+WAlEEQ|_jm!Xh#g)k9GNT`UsKO}7qj!UI~w2Hv28ZI;TH4@#nu`>1& zBtfMS0Uf6#{rW~ozsy5V1Oc_TLzyKvApRxA)T#Sc9e@=9KSj5=q59p=Q}(g`kkZUuA3u)dK9?HLWV4ESqNDzC5FU|fvjPXN6)3$T!Um&RhrS8 zhM0&j?2g92P9UM=tZKKAY%{N<2 z&OGZAllN1@2>iQ{{$B0YPw#^4kWjRCp4qHLiiMS?3jA$A0V7V_m{OQV|D#k^!l4Ah z68QoT2LI`-Mf&(`2#SM_$|@+x#i9~A&fVQPc_oW&m*<#sO`~v*QcTXBM-49W*jyQ_qHSe^ffmrKAFXDlJq4$(d;(>(dkFBbq;g=6}uUho-HsV}!Qux=vjybGweq zP0oBhJ9J5I=%S38GdguWX-L;CgA;=J=#CvXJhD0vb8lkI&2xjx&>A~9x}d}G$m@Za z6F1CVeaxfv*F(*i8aE!TvYQvLK2T9VJSP6k)cE5Ujz1}@@s}mRn-gNjmIup!IjUgq zsDdBzB98`Qjw=r~!cgS4 zZcYp);K%!)36`(RiCh+l$;baL4a6kC(*L*Jg9-5RzZL)Xc24v1IcYTuj~yPlIy-UD zh+cU_*Y(L>)qfBf_;TSYn;kVvGLI-;ToI==qK5O?F6h;?;dO* zwfqS*!<&Ta(-8N;?yF= zs$6UN2&XTDK?5S)zSA3=JtRE``jKycMdW!ykF#F+_`f!oQe%aK=9DyT&n^5lMI0r4 zzRb>yCeeU5_xP}ALM7(4?@E;zrf;^Z$_v>sL*8XX4YsWt8)q5BW_qSbooEObdJ#Af zg_-uafkTUNLm^<4f=w+20G9U(f0YQ3o=iYHWsVXE;B!1xA`KUo%6euLsR=+>BE28~ z5BrFer`{AZn$pYE8~ggP9PZ7>kH6s`85Upbfq_L>0qcI_B)3G!X3$L46#zSQ$U6UB z1=;wVDZW&MWTm1e45MU@sD2dwbqU1KWk7_m<}~fnl=_zNkCjFOU{kBxa7m?O1W&QTNt`%c@U_M+du z?cLT6j9PoqyYHx`j8ULX;94ZW8i&26q+puJNT7} z8_@gk+Wv0R(_A{I3%bD_r2Vhsu_DEt%^g^6vbZ#J6U3RN8%ZhVQPqmlv1PP??b_{R z%Z+Jin{K3j6dl;6Knz&`G30BO#CJ@WUNgV^s{S=##DgSs?6iYcsIHne>I}q;`DHUFj3j-l8 zAzYW8(Q>}9{BOdKIrQVhregyOD-9tfD!7c-K$UCsY@Se0QdY94Fg|DPKKEKS5MP^~ zV-7O&_XYxZpI*}_WzG>*87=+G)3&V{6abzrGkdTNQeCviu@JIff}qsgDYo~03fTap z%lqhhu;p2-`~4}lO};|NqWiC{v17Q$qMBy+h3=#LOe1slk%0yCX@jqbY@sxx;;pnC z4Z;Xfn&DbQ1t3i*+7P3u^U6kI%tXg*uE2^8lg-!0VQLFpEP>wADmD?3ZVCWq(X{&EtlyKpnF>tn|lZf00#ok=sDMTRWvQFzX>=5u8qtTsCFYvDpkl zRJdhhELwA_Q<(hH2M2gYvJ8^~e^(gI)+lzRn7E(G2;omAA(-|#^JDvu%gBFRMg$Mb z6ta)000C^RPVcC@jPyBRT5*Q2y3(p?))0oc^b$^?+e(CRgf{`r(*DFMv39ZIe2MoC zH@27gP28r*2Ur-I5x2F79T0Ds#AD})^F;bEmvWV0q zRRr%F#P?QUAcOrR%GvR_VDo106?3z;kw7?)a1$B6tL-`ZF*$D{&3zR*sWKR9TUtb< zmdeBkaOTY`WeMdp)jU#mou{}439vE0Tpud4W8*_5$?aSqYL~~!OARPJ*iGCpy&Ji00ZE9m7i! z`!2upl#jkgM9-B^dUyYP7?GrsE)hMnp9jC=);cZ(YyhKGm^u!;IND+M>jKyB)i>=YcDWBX8WA48SS z&`KHbn^21{RilPYQE+IVO$gMjeYQ|PG9b2;%&FICpPg#Jxag1L*90%tjfp+KqG|nK zDCfq6 zf)Pd#{E6cFM&eR(!`I*wtNOB)SbD>LgIr7*C3KL<8&@qBdtg`GruBy7g&d?*wr0yK zyxdUl?4(OonY()eafiU+)Py%K$jrv z6u&KinYqua*JSu9d5!3M@M~^8q4+aRD1}R4kr#40QyuGOg<3L|Xz^P-X(F6MC~EBI zd8!rPg)@uFr8F(H>10+dw5_EP(Z`XP!J<>NGka?ekfl>}tiE1*;?J(vt~KoZx6wIE z-PfcUN!vMR#1%hv6`i{*TmpqDgkXC7OXF+x^wb1)iQnz{yml!UA8DYt88_f9c&Cwf z;K!lf&!M1dyU3w|si?s?XP;-B+@Ex8-OSXkEyOYsJsd>s_m-Ri-_E>-&F#%cg1nvV z0INc{E2&GQqvXm0qzk&ea>&w_PJI_!0SVqx_ZJ>(sSwj-BtQ(RkXbX$DYrfoX)m42 zdYO8TxQPm|qaUTngIhbdYvNc=t7o|ltxTe0m8e=nm-LP_40X6_n8^pj^Td|=NTUXI zLQMq`o?FG0jkKZl>PA^&m;Rg=z2Qb8NT8_q&qUv?12iS0p@{kVzYRgA=D`Ww9mWJV zKe~SY=plc*NN9Nr&IoQgXi&N&2$swE9!j)l&YtXjN)3voaaf>M%~}_i-m}howZP3W z!==HSSw?9cw7guduQK0u)AS9E9~z)gKWDKB{hCldb~Vzxk922+?1m#c}WfkhU-~q zoHmIuKpef^;Mb{}mcvfZAbi!ny$kA!$a>@p51r%*K2DJ@T<+-<7JMBrh@t?vkBjHh=VBeG)t*d&H0E5jTF+qSYG~ul}f_{$MbEY4^Ak zh$Y{CG8y8?mCp!qWUrbPQZJb}M><7H>yZiF!}b3gjBf+YRxCTPkxcRx_(%e)?H>@A zQa+>p@nHOdIWy`X3C5?)nOFbM0kJ80i|RkYzdbS|GO-7MSO=I(KA84E<)w4c=B@tf z2QTZnAmOdp0dNrc@e)pye`;*oHqR|J_C|t@#sT5Rsu=T2pGIuNnHoT zorMBZHUp;t_qpOUa>=Aq(OII#vMb;;feATY!JjV5yG@wL%YIC3`V(&|g`@+ep{P2A zM3Tu3*BGd2{G|qvx|1$--}hHTfh3BO9~hShM}G>p_DsxxcoLg#?Aik(2d@Ja z)R)shOOlO^^(=nfBwr(%_JL?bCuO3FxQj7f*#ddx<=#I;`~>DJe|=xhX}PsuL%f}G zWm>~f4k)%>DKK7o!!>FL9ZQ8_(cQ7P6*{|PZ};ntjl`EMt#-o?S-g?HG`lHbFLhqi z97;GA>#EB>^p=tg{ut(5jm>-yxt*$w+kA~+dHVDcZz;(s^rmxXZ*LUau?tc|s_pRK zPlet@UxXo|;7g}(D6U)brDzuCcD_xdzx=A}+jNF#GI%MezK90l&D95?l%euA**2TJ zM_?ey7<5X4h}K8rGdmvFZ1|696f?La#GcM72{mNIJR48WMZXH^uZR$I;enX-258p& z5#n!tZ5y5KX*UpZ_J(cxN(q%G2W$g!<_xMfNG98Eh*R}Jvnh#^#(6{K1+OXMM zpn5O;QR_c~otXoEnGpIrE7X`xmO@Us_HKTU`8#T=not8M&KDhHdFSZ;7{vLGiB9+M z+#*11NvB%H?U=Zi$}qPGOF-{%m{zt27X*(SBXYNUv&^OtoKJZGJrrzN(U*z!#14uI zPrs~Lp5rF6@0&TUAnIsU@|4yiI6ws12;IUJ+A7Utjq+7;J=^vWvDkeisaOkWQh_1S zaM0>8Q1;n-SV6Jj&dC1LaxQl3B#Yg?MX_71=%YP#PB81EQ2iyqbCtbn)(gM?gD$ZX278n6OY#Y36G>s2!G-=f6r!7H!c4 zk47)!)OIwdWra`lmZtWA+y(6>j~tKTBN~sTEL8K*G)9_NhqBMZXGGqIIPwCalRS(c z2h)%H%;U^XsF~#5{zHqZpSB`Ixts7MdU`Nn;Di%-Gkf3FpTc1?~-fJdj!KePtB(f^1I zXZEh~*Hsd`rA03$X2V?E(QWjojI&7ZX~L{O3hG-x7BNW4lOh7{-m-DSj1`HxBrhR9&yYX z;N{k<&vIciBo6c8f(S6z{*g~T&9<)41ORmwQCD6lO)OOP9aWAYJ?WnYWCGWgPaLxc z00v3M$`Oyp(DPe%KCvzcYv*PTOX!mrpSWim%w~`-BM1m3)^4J}!74RoLbd_B5)4ks z#Jy_N=u(CoidGYyk5(h4&g(%v3BD)$_hVXWXiRkpOdJ(=IWsWW88WD^M(}0fSi-xP zHereV24YAoW*}z16Zl{v{P|Ck+0$_<^6QL`z&>$vRe10C|h_jT*rzc{1ts!(8iY2b@t;Xd8^E`OkJ z{|7E>c((5iO@4kP$NTw_{8H~nvRF3GHvCBb)-!g&<%{YYA%B!T*~sL-?v!x-;$Zyv z@_F@p2E>jZzNr4|0kK;_uQd#a+xp0i`Vju@k$LqS2gIgsPReLLKP|4PQ*3SNZ-M}Z z@kKE&oOr18`e0nqvAA{>{@*$*yl_``Bm*4;(35T5MTsj*vuhF{io8bnk<9BJyI?TT zcFUO*9?Dp53-`}PkUIqAR)DIr+-SCjNm-1aOx>dC z=zjxRzJcdicUVC^cbH~ci&#F%3ezC|G&DW{p8|P20k|WmIT#L;V#K>ETMs0=wcMDb z%Nm*$pZj9k={Ceel@fths9$!cQby}~;w9mI2 z4YOY$eV4bV|4@0Dd39vNA@1zR_EfYC@|zh&x20TT@N~}_Jl%q8dAEKQ8zJIXzcON_ zC3DGpDHttdFLgxOVzERKORK2ZL}F^|1<6;DL_IlY1|E2KxAs0Hui>wk8sm7H2?d zc#HV=T!lPr@5ko|VMugfB40OwK)HuvEQLhap0k#uUd8CkR{cK1vX-01CbwA{W?$g}gGFR< z3pa^#j~S>r9(Tsmgmh-o9G@e&J8Q%y9~V6zHMV;^$OsG{ zI`o0b!X!7d3Aa{d+0zkNMRnc?c5;;MWVqrkc&|6vsp;kuuXdQm7y2@I06+Jse zK7O2tsWr)Y_5N!i)4I!(r6zDnJ&lT7Uc5jGBp-uwW($$%~EuC&@)Er zp)si(9T-Y>nQt`~zVX1vtQe4b5C{GQj&t1@#M*~FeX621*mK{>*Z`$@8Wpjil1LVB zl!lp|E~K2+iEGX<5mLJ)yeNQGUW3+&sVEH`%b(X`Ongg->-hPZo3YxTWm*i?VbgTu z3f+htVNhh^`)X1#;*u~Yb?4YID?y@(#jbYY43cCAkpN{#2XKu}gFS!_(2DtPNpP4pxPE2UI=K_e~%)G^CWXD(+vgSLXt^#j)j!~F73JSaB zNZ}gDJ6)WPV!Or2z%EQCvJ@z?;Uuo9 z&?jwFGFNfGE=}Vhg#f_M3?bw{={*R==ma=BqF`o=B+A?%*{VyPo9EbmZTATs; ziyb5PqCoAL4mHzIpw=I-?W^p_Nq}ue)9hI*rrqyLqLI9=2<_L1WN}Al(kMUtwdmfK z)q7kPL#);t!^aZDI$_vBAHxyIAlup|zCG8gSnZS2Lru$W-TNuGd6TIq2Ta=kZrk)u z4@x&Jhc!v!hYY9o?`fM}0~MH%LAv0yu?zRq-8a8k=tJy;oB9pXhy3-#br5xkpwPgM zA&VMwY1Md0w=VqE!TFyJC5-A6Jo2R%P>$Sm(|D-@)4mVFNF2kB)3&wT^AM4B)$yW_ zTt76l`$9O9G~&ljIY6ZwkR`h~XSI-p8<0di4M^7b8jv*m z7?4zM9li?zwyy!nCSL=Rd_Ai+&cdqA_hr>4__J#B9jw{`C#&{_FRS+N5Q~T!XKzTM zr;G?i^FEBaG(^gtViE9e9P*lhzdDqX$_{c2a0>|?&$yLluSXX6VbWYJF2@pf3i(Xb zhJ(53+wzus9*w|AvRS(QlanM`vGIv*l|xpasl;F^xoSK1>25Fo33qdjlKKTwl~tF0 zV5rKl#C47$Cn08U{rm#&#;&C0?DxNJa#seNEmv%&=%o0uZhYW+Ya{g#SH&VwHA1K< zA`V>y0vQ;<5%E&iI4%19kR*z@+XKg$f-p5~gx*X;Z^K|f1-NFYtO5(ak}G&2?8E%A zlw8ST3^TYBQZ8rOScWvLjjWb2kP{n-L>9a@g<6!1H^n6t)Jpg&>^Q}wR7rBPViT88 z%H~&cb(Rq_dyFfnW*-$bh3?wPd&Hh3Xn9x6a9k!lR8VHV7QB^R**5x$rDF3CJH_jQ zorxYP>vyBK+|S#J97gLu%1e+`PEd44`cnpWP*F2YV<;;9bY*zUgd?iCl7dcK5UgM; zmh07A8)@PK8}P-zV*~N3x%!Xah{B^bh-z#H|M45)6OnR9Y+B;LZ%5T{WXsR-8!6&F zL$h#r+hCS$@BewXk?>LG*xvqW9NU1m5~NZ8&2!@J72Vjrt}FnH26XWDoVcuwHOCe) zbo=_jN~#ZA05%}4vF6_kK3HUK85X-1;;=PIC#xyw)6LB z%Lamt8$WvzY_j{Ch4$h&%)m_zlR#@BBdjIZxM3RzHr3iq!7yYiO{hmYNaC`DcfaM6 ztOT3zBjH)A2)4iTPsHejuM%D&JQkLUF`Ei<9SCN6D|3h%>Z>Is5hS&2Lk9(9cYH~$ z0`hK@DJv{C@JJZG=Pvxzt`8^+eXWezm`?1s@V16$z?a#?d_3oK1 zQ^j6Avp24sN6-?MBorUGQEWIdS4=&}$n;*Vq>uwR{l0{X*Q(?R|81O%tuhyhFPKp^ z&8TEUzUy9}QZ|AV3kgwFF$pn6dHpZNYc7)LcSbH{z;|xXv#B6+qpR8Q2Tk@d{r~r9 zMuPtW%?Q{tAI-=u{+bcI#SD`55nX>j%}Co0tO6={Zuz8Va@+2?RMQeE1!O~4HmaCv zHds*C7{P|}5VX~rOo0zYC>Z(er-+hRrpE*oUdGl+o zmYgDv1;2)f+dbu*wsmxN+dvUeYhq;eM9#xoT7NN!raRAg46%i^U_fqc8|lNFV?oE-lx1g?<>3SDsM8!EE^>60lnaIp<4X@y4uM_8i292EMt|p@ z0)6ElA(%^$W9DMEuGT9P_a1C7<^Tbg4>skrXPIn_RP<)wZ5mu3{+6bjr} z7P#((oXFZh%oB5i324;Tadg3P!y|tQ#9W#fbHUtTK6nW8o2{jy{;HVx zhg0KkPmNn}<)YO;EM6T6%{_Y7khQWw8|=-(9lP{x{ym7b`Pnsfr#@Q0{)C*!*g#D2 z?Rnwq^Mf@*;<~O(=~J`)&cv0EX4h=O|0=R;*4&xA^3;ws9q@0b^r?C7PIO@Fl^3}! zJGtdVxQuLyh2KfnmiSS>e0}GdUx9}^CfKm%bo{@qS55xiX@$w5@IsWjdFWs(0dcYag)L)2zZjyyE2C|@6BG|Lb2g3S4?VOr)ZUt~VJ8CpQ^|7A{qUaB>6I|-j=;^Ob`KJO- zQu4*Gfe+HCR1Vb_{vQ9c&Sjj66~HP-7E_R((tM5eY@kH-;i#@)VvgWn&FS#^FyOE| zW`nN?XSo@0@xr!JZOxaYVa1D*z>*0ufjir4z@WLQBBA)Cgt`4z=4&Vs$+<)W4Jp=; zs>pp%MSgXbQ4hFah+!)-R9-?Rdl;ET<`$l4z^VDtJTkYiJ1${&**`HQ#|yK)x@z8r zJ2%v&T*k-bi^hkIznovMQlIpuZ_Qd=N`9Kg9kRm0u5CIuo=(5aKNxWJI(p5Qmf%uX zgxCHZwD-Lpj%!>uJMw1+ary6r21K`LTMaSSB!f)7kG5!jn78s1Sw}IFNm)gOv=#gs z0g_=cX2`%aoAVLGyNoVHfgi_aN(f|98^Ezsv}}jXE3gNi)GMYK3RkD3DHJ zcg1PT6=V4*&mDnKGD2sAD9Y%|#&2UUmSZWB&Z<+C^tjhz0B8Cb@eFNOW(QBDq@`Gx6NUg## z!h|FV)5g{OcX<>8syT;Mk4DYqR3;uXzjfXiyc= zcv>bJc*Q54?Y10|bH%LGE@v}Y!W9TP+WoV8ZuA;o1_0-y~fA*%v<<*sHJR= z-$~QSM+(%6f(3RRe_=i%20nV!W%-%&Cdscd@HJNoNNu*}bWh+|%OuqTrwn%b^^}Xn zf1>r{9o<5%c-eFWa>4|WF^~dIr|pY%SV`!iQ^E6o%;*fu)O9LiP=J~Ua62a&oQeZlHa@59?o$0IR|O+|9if_ z&%^VOb2yxvyU#v*uXnxgdROuql)syF336m%98%f!vq451F1;SzM9xBg>Bbj1mg?n} zDDzKq>7o;ltb9y=QTJksu_och3(reUn1}xdYr+nBB|s-K5WDb<7w(rrHK&rqz7Dni z>UEZ-*!2{Syu;>A(>ps)88A0z<_kZ`A227st~#&$`f%Lz>bM(AA`@_Gc7ENgV*htDBD&Ny@4BOqsQ8&en3R{Q?zL7DDoa@ zmQC3XmYx4M2WsaJN{exm%PexiGgn8Ry}FU?{82;2cK&|^4)C_~2jRqRsuy|N`8N@Y z@wM~6gAm8B{5opGiG)cj544tg(YK`Nq?g|!%w+t}cKBFCw!#iX+`}TwG~%LDXFR%# z_$pD0Ci9)nV?+$3_mpwB{G{K_*~=~3_^xF=Vd{Yfs`pkt^m7-0;x}7is@usmEA17N zjFDGL0?X5jNORShN3SkW)#^A1!0?$1!zJy%7XBk`_;D8f_z7M4{0VZoa58HAm?>O& z7wPo9@BJTEVlt?(BScC`6EGzsvg~Uft^|k4rIvJU|YTbY9*hwPz=*g|&ZQmGK;iQ=~mx zQoCdII6s(|3K1p6_F~u<6WSAqh9CG=LVGwy6U}?1zuT$ z#pH(QY(c7xMbfL_abCWMDRj5?w<_qmkxo&_z7)ljKE1f% z;TIuW5ofS2146nd2n+COTH43`|OOeB%B`JiJ^4+}8yw&~QnqHb8LShK@O?FuQ)s z(kskYtPv;iwfs$;1cp4NHJ+YnWB^we1r;ZvbnB+7WWzh!z?)*O&nkY7GBEff{7Yzr zL=v2rP$z`OB|7ew!|i?@Agv6j&jMZ)EETIV@WgH8Cnx>Ay*hP>2Z%s^1}63+wVGZ; zho6CbnRS9k=F#zO%rM;kF$%L=DGM2;Gc2dNH`o@Ypz;`$!u#0uZHI3RP3O1iCDP} zL%aXpBu%jpT25(t0+CDZr(IFeX;zvA=-1&*a_bM~O>!%52kvLXsY@O+Pn}5##dY*Y zy~;m&lc+83f0s8&W$Sm}twxWwsRyk`+q;d2kKNaASlpVbxRoO!8v)*K%O7xCUirvy z++8Cgx57p8*8I9#4hz-5HFD>evd>DQk+}HTJ>qWxc!P1|_L68hz}#E+uY0++dAoFt zOk8}SFp>cc&WME0>JeWbck$u- z4w#ASdi9k)L>|47nf`euTZ{bd)P{Ha)K2c5eoyc83Y2U%uOY$Vl27mC7SH2)r&X-# zOMF&GS=G!XpED_U%joqd43<6|KERHMt1=X3fkhiXKk+)zi!FhVpPZlg$1&?~Bl621 z*kx<)ehykSoKP_9(a+7+{n)IEMlo^LQVzW$V^B)ziq)Z0^EUc$K0Ev&ST5o|Nm z(*zyG*BhwIoRC|V^+ZkCLg}gLRq4j&C5#@-^cX8ACGaLV$9?9Ea1I$S8e4R9rm0S*d9-~8Pqw^iMZ2&nK4B%1)sBFPS8pYJyrVW?)*2=@{Bbhl7 zw?su9bDK8$WeqTma^E(X2K`6?Ok2XFl6l-joU36pUT6x6t#rH?P-rKGe#9NDS;q<_ zTSKZxnLoL+-$|?n&HZ*$DZC>$J+~lw>!s}XUjN1gjZ3cUsdsag4|kIcJjr;}F4RtA z^eU@-y~q~h8IiF8csTmnvCm0AH?&p|(M)*1(Q&}-+S z?Gx5YLM&3XRhp(7JM#IxlQ^}nam2-0=nuuHL=USq(qi&nhW?CNI_8LV(_W=UmiR-L z2sR^aGtX5A^&#NH^RM02*;lce7B+AI*p&sJ6ngV)j9GBW3#A-ZThC3yHOUt}(0v~y zVeS#LgRG|t;5o{2Ks^#NzQ-z&BmRZHw&FC*lU#O{i+xPiV-uP&=K^AP@MKdEyt9I& zG<+qT5)LLRmN%N1fV$G^wY-n^4dYT*Hy4LqZAQ3fy%2zVcx5%Tb4+4^szm0CT3=xK zG_0sN<<+msT0?COFRW454h`Loc zw>^f*ltMoH3Eim0ye`0_q-*i3Ur}cqYc$2^TgTWL_pLsJYmN<1nXZmpLL}I6>WwzM z@i>J`Dt={LFR^N%1#HZ1tTA&e3@bRcFnoq}JuZ2Hl%!5;q%^UH@#?%kj%MFaT=G%p zFObuZ&CkM1QrM8_Ni9cKA1lJzy(6m3QCZJ@FYQ=TNpAn2J`ul#4>xYw)V)^~6G>frTk`{NJMv+b9C3O~;+i5`I4|9Of$ zJEz|k(Yx)pS4W;6miytLe!b4^hn@l5+sRU~@SQ`a5wID3b4mNNZ1Eqx3$X1uhHYEU zJ90(4w9s$xwvp^03;nYMM2ml@>@Y?av&C`JLuH5mVNXF*B|uWcF{d?TvR3kl-1L)0 z1G0X6jEOsjt@s#q@q$P2i06(vg???FyXmZfVoR;2mT|aCc@s?U1>02Ol11`rLjC1l zRvqD;sLZ+^Z~ea2Q#fa2|Fjr`lKvDV;dA8+>1?M43nO@#OmD+)jPxTTF5boIn0G;( z5;R;|MJj(bwQbwA`$d@ipO%tXe`Fp5=fU_fg?=OleHl485ugyOQnnNJl`c%DKT-8A zUz!cvcIw($ip;uEmAt{mOliPBNS|S&Z!zr``SQspx-2RLB^sP1NEz|h_ zE}q}**_T$AHTJzBcWqr+FYD{p|<BG%}@Y2_d!n$DQov0HY2JO)}VN_!hT|EZ<3_WZ}zKVrACD>TGJ zIOQoG(-FTEkI~jkZ9=~IpU+JF^9Ep3C)2Q4a|jD*2!)xQ5Y-s3E+MlAMY6Rs<*ih$lsWPe z5=@P&%>V@m6TFQh^tH8i=CBVRBkD4|F&hWrCQwl#^D5}VM$j9mmc^f@uT7U|hd*iN+QpF6OAdfiK%(!`c!1aQLN1Zz#y|uFHCI*YlV3GS{!u#@`wLa zxkbAHiq346m6rlw@r!DBcX0~p$*z4XcR_*SW9zvYF8Qs`94@-kQFdXkahRyfTmP@4 zAB^-HWa<}p>tCTF^DE0j+<(Qe|AD@Qokt9jCr*(pWXhk6;DAfpd2PkFd#S*$d1e?j zxo9fawe|HKroMXi5vP2W6COLXXaI3|aSb+6GARZ~01#(Rdg=EUH-VPhyDApz-TauF z)Uw5pEe!`%=6-==ni$SGl$-h~%_;oB8^vW=9{DheB>cfK;Rvi$MvB=Yg2h|esHd$B zQ!)^lI2@&2Afx@}FlyPR@G_`!hc=pc4#`1Tl^I03<8uOX4!uogMRto%pSla3xl(l( z<|P%0G%T6AB@Sd`**bi+O}|FSLt7uY+34A2-`Ogc;4Lx(aGu$9b36aJyREepRd35~ zJ7`sJZ@;u6C2i4p857S-D|oUf{N}Lmt9=`O3iJmd<>N)+KMxE4;q;vH;&9yMRdM4+ zM2-Uec2j=cjd|sR!*M5po`aBb+LIv&C;L;=HvB1xP8M3q9y6oY@lOZ0>g|+!4y@ny z)LRsD<;Aq!)vbdfeSo)9$@ShFyh$$&Kk!B<&a$7T=x390ki;D#?Mycqb%7urU-9XY83E+z=L^lc@@rjKbir;m|H zPtYYrPcY1*59A&IVC%S{w-;!ehvc?qx@0QSPKI&08*0?}cLmWlu(1Acxc-~$B&Wdu zkU*guB|VH)*o#0x0A#rB$wKCWp)_3+znN; zLI|uVw#q&~4OwF%JkMy&CYouLP9x3g0Ag6-yIINHU6hwe`p5Vlb1W<~A+LOj1-_3B+YZ zSoSZ7oC8a}tnJ-BKz3-I^OaGN3TBXkmsnOr@0i#jXfeJR^{S5vuF z0MF)W+bK*Rz_aOU4mDslZd`(^|)jomoFN46PZ`{~CI4a! z;h2pg-y=^ElnsqGhy`+Sz2u$sY{xRgf-uh^F6;RWy|T7I%Zk0`T+xBR^XUOb$fo!N5{5mN_6a+AQA)FE}CN9!UCCeD>{oRhLtJ{vb zyW%mVzT9`ieawPu9c|-2BDB^B#hSqU#0dpJ07~63d}ucF3$1BRexZGuX%8%b_MA+lU#~6>DG>3vGCEyn2k^+%&yn?*ue! z+j7vFwdFj~mNE1o$swoyAacm09c>3x=62bPTk1o7Me}TDKkee@ZXPw@7WC7;XvUrO zFa3{f)ke&fM@Q}F_heKi&KA~@URIJTYra}o7Wa^!B68)X-7848=TqzU*|2>uAv6ma zML2NrlFvsIWo0JW2y-HVww9~7HLP+0E+DOCE%yMy0uEXtNCDEUW8tFjM0ugBxJ-=6 zu)4Xh0$DE$2?bDa(q7oAVK_-26AQ37X9=uWk&G=uw!v$Go#fiePmu2Sz-j=7hFdwX z8bM`WtX6p5temTE^X{jOUuSzz!?wy9i$4o-4hky{XiV|7svRd zFJ9GUa~CRT-ogNS@kN3ddtqaGN%?t71k+t|YQ<03*tM_y+qIz$_NPM-hh5A=ccBU< z7GHv6ehSy3L3QDI?vk~6u_bGTfhB7)qINHqPssHX-lwJxs zxC08QzdQf7w=0c2p1K}TP0H6=R`;llPO4%^R9H~lSehm3r4&{hi(AEty&Rj9(tEif znzaq-&>W4hPOacPgT@vDKKy7-qGMPJ7=AP-k%k49pi~FZWKQz!fkI#n{lOi-;8|KF zf%1NYyW`0eHzroYf@_7xQV!-?Kx{x6t;p>-z`Qizl8T*H)ag`wiA>LO7>}?3g%x4m z<|J-7;lj+z??vTukeCMm87hHX;xFyGWBtD5hhWn)99vX(~oLDaFIT>zAZpoN}5Xl?hJ*Yo@VOs1>>*`88kezQQ`Di*BA` z*$v1UT!P;0oIB_A9^U8|j&D?^BGv|_WwbbjYucs^;I=Y(1j$|bUvUcAv44&FZfAyI z<2>)Q(KN7G;m#i(&fQRlA1Gnc}Y&4+ay_}QQ2Gv z*M4fl9=_e@+382*;MDa4U*jq(-wL+D@i;XndJ`5ln22PX?IcfKtGG3gOacD?WFqoZQKMfO8*VcC zq(x3%?b8|Y=~=buEwkd&7uWV>W#x(yx|U*+f{`oeR7v?{>?nX_ayxcM4*yZP2(ww7 zvoNqaEWHVqiwF}DL*$@Op~Ow(v@Oa)JA6yI2y~S}auIxvk(^T{D$%p6gzcY!R%2dD zK{b)trVZGbfJIJLWwt#61Ewr1HD!6`uFe6*Y6$p@)K}j9aIouE>D#P@@`}#dA6bQc z^rgKGHP1!Z;?GACi8OfAuZ9Px%T__ma_^Z+T~-~-6;YemM!M?S64@fs8R%Ld13X*X z;!|SG&SbnH1{sS*(@Fo7jrg1l7pcL-YfB+V59%Okvd54Qf-5k*V`0Q)C~OmKvks#Z z&~MH$!&$0ne)Js2OdJkbqNd24B0gQzrX*o9!|l9^!@0{WgMF`vSsLuCB!vmX;q~J@ zhXL&H5X+J(L&2a9Wij1FNtPp5BO)nois6i>b#SYbHYCcOs7k8zZRI1;$AXq}1!y5E zr;>b%^b}#%Nvbs{;;_tg`0S-{~ZyD)N%Sq(=mr>`uTtiptT?o379bB zK81;{L?pN*NJP>gCD`u8**XTar1#p%gyx|@65#}2<$SMQi9}fotX#zObS!bvm6Qz^ zc>&}kYx!w8@(-9D1OILP2v@XitdDy

    fyyaq<09<9|{cNv}9-cf^9`n^EhQlQcOeaq;-2>*793>s6Gtc4blcyS@!8 zp>{0H*fKb+;OV09_F>_cafk2Raq!UF^V!E%#T{F9usgTyIk3)c-+VaZW@)qNYTZ`y z$9bZ4+tbO>l|MR=NQI=GZ2QrHgjeKyRK&E60Pp_`4kVop2a>K75lJMUKw?JH<~^u_ zi%oP2Qbx3It)6*QPu}v!DaKng!7n>_x(PZuDj|*GgG&KP>!p zRfZu|oQAy_;0FpDkC0Y<4#F43Fgx?jd{a{CNFeS4$2rs#R(@$pSd&D7NMp;^)QStV zhSM1FQiy@NJeRuc;`Hw*18qZp2%S5;O#e7cshdl*l8km-dQfc;NtgWjO`_C&{72}` zSFx~!u5J*r(yAhMMqSqRxMWg*{Rn00VuiMLdIGOK-^4Z$`ejb#*qRElSVzJS#2mSy z_%7S{kgSIW9Z5{A3mu7WL{K-^?i^xGMEk%1jnT&#$qO+_!ID83CPAMETHeA?8H=<# zKctVz<|NQK2Qxe6S^un#u{O%8c4M^7N;su-a0v7WEn`d$#C}seLWYTDJwcP<9-0BN zCJ>7{CD$g5f((nXJO!73Ss58SZUT=DjrAi+rI$`kbUaej$B}c06TwO7rq!mBz;dkz zxql<|qAgiA_QFR6QEZ>h{GPJgJd_^6T7niX4m#uuyT2G}#r{gw;2Zd8p#hPIsP5H69pB?rE5&4N?9;4E%4Q(zZowL&o_|6Tv18^4Y8rAO||3AWqX!3k&%f(6SY4e{{%9J2BL9UB#3B|#oq?{slw3+4_)VBtaE%lMx)ZUzT+xtF&Ahh?Z?6VkCC zT}ND~?SG-`2q)+r%#(l`9Qu+`6R?Ej8TH(A1YUxtGNS|ml!Hw;!Kn$`K8^1ToSHz% zc2eZx^nE)RcvCzm{6w^u5U-|ASdUYG?>hCC{(oNZ&ytMz7w+%)?;E?0J?Vz_F?U{m z$b##xeSGrs^Qt~Nf5zLbg$x8VJmF8ovxgzU)rvD%BEN z0g-|b=`-)%&xi$ZNfQ-1`atQR^ce-p%Q?m7z<6T8uvQ~gcx;cz?T=}^!$8E22J)^_ zh!sYWm`}banHxc}r*148EQYgB>`kOJQ{xSwx&DD@P6{;Li>M9F2|HfdkisziwG*yu zov?(Ez@}8@=&PKhvNZQEVeJI$LZb5qc7D;Cmv9H642BKPsx1oLK&U*DZxR5K8&Mz^ z;8LV~N*uFYc9zshGmjDrB@b047R05S#lUPqBwdw%c~ysTbQp66?sFT>U2m|J(&Ua@ z0-!1!lAV0Y6A}+u^$p$n5`&NB+Fh#0&@_!R2isNWnVc8nme zrR`2A#3T_~pA`Ctisnku1Mh_}yQ(yG&uuv>LREO%&@hrzm=;nK!b|HI$c<#8MlTeB zdg+BuI&{O{^(^b){UcK)dHPF-NK%KCPLUewHQG_qNw$&p>*#f*E2@+CCeU~eJ~?l` zu*GU0u%L+7(9rVRj$ekJt8MQmvJx$~-~qk3Qq#5EwF|{DNwJQ-(E5!`QpwHmwQv3P zIrHAAj^3^~wB&q|B%Je!YQ9Y#hA+)|RrJd9V%_c~qpz>FuO8>eyIqN>q3y79@Qbg_ zKE81qw*)BY(;aJIpAiltNz&8Z9A`<_J%p3UmWkY&BVx1rZ1Jj`u))jILdZa9TQ!%0 zBfAJOVsObt0C@opL|02VGL(8lKV#b4rCx!9?CFY2nzXBlh@#v zDHLz_q1{@%8^g+20$M<64rU}d;-+<7$Icx1%Mhig5gxx8dEVB&Ggz1DbsZb*Et&m4 zcw1s}xUvs4GQ=bUEA7oPFqvF8=iB!#g(}lFCb3}y=&JfD$GHkPa;USjEv!ZuUB786 zU*r`h3**E7YmKV2;@nHg1p|NJL)cV#EnAh$p^_*g0)YKRvU0q)B0W8fbZRoUR`ZNKYFXRBJLq6Kh=vBg_;tkct93EOhv@`D`T6H>g!1^9q9uxK&CaEoTF9xlc{CRD>-B`$|Ne|+kcys2#%sU&_ zm=}4^N$|As+z;xw4;5L)t`-tLaJYdeqK#G62F6NqoB(G5q7vad!a-IHTPLwIU84IG z>Rr(>L19g@#JEN3M@FsMV)H$7CR+cCwylMM)+EAK*9X7BGgobFlZ|gcrkC1MQ*}aZ zX%Fd~w@qbQmc z7hm{BZS$maQVM1lHH^L?r#uN=!&kgFsqLo&ZeY0Oh2Gxn-oCg0=@m~5BLK%mRvrLu z<>*}t%i6iM^^?M(q+?OpK0f}oCh;)x_ zBEuPmY~hkOYUuhIH*}4GPnt$szULagi4mUV%;w^_&$d|xCwXR84>pnlqiN#JP25p6 za}0o#4dEE-ji(N2!6*(Uk(}0vDTL$Vv(`h&(@Qj$>o_r`8kLFZ6-QUkY&wj8pNj12 znWYK%aV9(>saKpZgp?Ww=doxXhXt88GP_wNZKm{9@J4PwA(Z-$%M~~)s8Y!4Zg~pf zbq(RUIZ=;(+Q1Wg*B3Dz z3Zb&RdS-or&~VkCCxa(DedY`sIE2Ef@8>~~Vy=*YJTVZo34d#Y)Rln@s@ym#ucn>Z z&qQ(-RZsA1%|Vm_PMxCnbE7zrZ0#HIt*ufCn|RQAzHmw;sh6|BLIxl79Jo&KTcK@p zhPzUyhv zLfJ-75N(QuNX#(%M6+faR}*8eu^+WpUqHsy)&BuzC!(2CsDP)6elip?L zuOVRu4gUA4V|&8uYjX{=sOPlaZwEvC|Gh8C@8+~k;*EYpBdkqkocL4947%EF<`3@7Yd-~;?8(%;x@r0CJ(pg#<-l-S z%jbUy?$JifSH+Xv*o(v4=0BPi$1TRZ>?7BY1-TVOvXPa@`|}e&7LD6d9x6-aMs8;* zC;BmPNM!4Ltw)ru4hpk9`Z+-!Gd`Qg_-yLhLVXW6{t}$r^#iwM2a;^UuY{A0Qm=V& z`WHL+*Py#X1F7~PBZI~{TU=7V&q$B)evU8Fk%iV&^)pLD0+97Y_8KEpe?)QPlx~az z>yu&pAHt7AS6cbfOP)a!1ZfC`(Xhd_YSF4rZ1g$*{X&Y`OL{0Z z!cnQ>6m%y$vxYN~d8u}HJRip#Gfl%OS+?Z|)2=FKnhhl!yl8kkNCxAy8~C+F znqR{X?C(ah86A>u9~1Va){sDpv5?7^{%yx#tFIIY8HE8-RTv4?|;*|=?xS&c(g zRQMO%1cg*W>SW;oT@FX5BuNzu^yDVkGNp%1b;lYRCr~_%;y}sI!#-?kY!TTB=ap@3>h@+D(9ohckD*pea*B#d~dd*zr0Zn|@ zOTg1c{3VCfQM~0CT=9k%!(V$92i6YdjXT#4(|gu}`NX@96ag=OQhOxpoiNC_q%9Ds z!K+VZc+CL@s;-2r$pRZWPdWn!= zv==QzeUzDUaNr9^K3?0^djxNsJCJSQY(bV|8uM$<%P!+3``3nk#2q#3xoC|bPSqhb z#t2L=1a=9YTz|yVbKIZaW?84WYB~5V%_SJHEl!1HcXp3g9`1(=(XLzhWm zE#jWh`WR$fAuluT|Eks`;v;a6qI*2+sj$3a%WLO^Rf!0B4J&Q{{Do-b*Ie@)xGJD~ zq|Qh>fUqQcuAT9AHe7uT$SRjYY<@H0Oj09yFq>flzC8H0LR>Ef%HtszZX=PhHH}e9 zwoxraZYapd+=^9~OxYKNJe%W)m~!DoJc+D*M$a%d)%0Btq-Ha=RMrB-oT*Yli_=Sz z)|k*raA0^%r9HoS-a??r&1oDBL?uv76)7xxPd5iT!?25KI14lx^^%D(E3!rqoSxvB z@kI4(E)l;bKQ2iC2e=UhF0)ueWf5kMxyyxAWX6=pT7X&6%!bD%6Jqsk9UY~x_#b%C zEdGDl(s}7Eo93_keA&9sYMW05UzU|LDl74dpDkV2T-)3`KIx6#Nvj@9Z0MVmv1L%k zqCshEmllPW4GTZkw_$f;#>8{d3KoMuTLY|=SAGuYDHx25h)km1Z1|IN#t-5bIkfA5 z{34$j{9=7u__i~SBErb472CM%=BAY&LlQ%#k?RRITwFCXduKKGXWP=J<@j?6yG8!> z&eXZt!KRTUF!VF|Uw<2SC-d?6pZZiVOcQP(tQHh*#thD|_Oh50>q(=Lz> zJ`GyPQcKoo1B8-H!}5YPLETn=60EL)hcM$A9D%obL6}n-Q4#XGPhoc_lgtgCFA%n^tcKS48Agu3;sPhd%xulq=S^3A(HI6>4|o8vO#)%G4%AS9?eP{juUVv!32#Ue1spvao0a||UT#UgZx4)6@22u=-%t)URXh}sg7?h+PY9q;u? z>-?N$kI|~ClJ?h!vFNM)AH&4=fca+ z7)#>RC$kh1yEZoj^Or(qb5N>=A_mzmpQWXnw(Z{v(~6PO24yBVBzKIbk1tamCH5FX z-h^TbhP%d=&+z3{2tuj5%qUFUL#j4oV=oaBpvjr2xwMAzJ?bro|6HO2YD{l&%K0Mk zugO={&Ao-iUDhb*xX2>b=#s*d>7`W(?N~VF^rY9L<#-rdf~g-gmu5pCA43O+0?Ys! z1hn|lLQZmvA|!>Oi#wHPHC}2KooEhB=F!v>0X% zC*4d>$l1Q#VjmF-X)`UM;)W6`eU}$dbZKQB~6 z4YN;9oSd0>#YK=oE~65*51C1uoSD$_gK%oE9BAuoRT<@F5!rAk+3u6>F%!|gci~!R zd0^ohS63Y=s;P$bTQr2BL?1q0G_>EAq0bE7w1W-r|4rHBl0y8Z#3vATVzDwGV@`*! zXAt(gnl`rI(b!6cbGGZB-*XS#{2c#T+>fysUyrDJZ($1OAtglP)8S7nptoxI6|(CL z5V2fN-4q*L(pt#Gy2e&wj6@?*41SEy{}J>kOt5V~eExwVtDJNii`n>;RzJ1?x4q-5 zejFXBe%u|Pe!L3RYrv}o_wyn%jw;qTDfb{BPnXAtm;F8di4eQ302#S-vCx*37YR59 z!Pjg-F485_&&OGB&<1VeKkiD(Tl*BFjYqf~4dAz6C60NqeLaaR@7?mp)U6W+w4|Ps zdd0*6Ek~b|whEo1E6T4?aPU^G$N43H_(l@FSlv6t^x_mZZE-eMO(e@wJUIjZY6)65 zrEk7(vaFKVcW*HR)P1@N%8<3jDNts@Nz75JBZ>ym>7BJ5Z`xH-hU;*OOy^yO(lY<3 zuCKfiR83Rnt`R_GD~IKEeAtJhqvYh;=)x*)NELlSv}q$w$8ykgWO)Us#VLf;Lfd87 z_QWYinGAL+Hvjpw9YeD z(hkjWJ=Bdt-(_J99b1S~*E0|0-O@D|q!geU>AzyD=XG5Cyeh3QJaG{_AX6oIQcf&z zaLGJS(aCBq3j=%q*rC6Q9jxJUJP!Ee3ze7-+>i|;M#8#2xJ`xNvP?)qnfilxo5Eu} zM-R1{bP5;znqzMTV`Y8aO(Qlpd?_i$d)>K<)iqq zzlo-_h~44bim_TuWLX~NR`wS2H4eeGnD6qxVbxz=fLGohrd`ZkDm|Qpu;E}8#3lU$ zSPHuu&6Vob`GHBrFlUx0_MDz z6j7R9fh#UAiMfd+m#ujVsJ1@{*wpmh2Gzve30>92U)r!L^II4sfDH8Dq}F_{MM=B` z{0h{~g&`uIuz4&*VquVICE}Uu!feA$bauj#sWHoaVY%tN!b&bt zd-LEmV*B#ES$)DE4hz56x8Z8gX9XFH3ewg-Q50_I+i-4D#-ejGCZ3xf8Ih4+cXR%L zoAb(tgyV))#hp7MvIF$l4SD4SpwlLeEZbHRy#WPoJ>t)r7uwVh$PIsrdTkv=$gO!w zk0H^rxcJIx36ULX`R&t(rFHzdPr}vn$YaBZX_`2w-=aZz3!b?;^6b@(qi>peCupnzEt3Z2Y%TnHXI{&SL3te$N03gg zy`e{9;fJZj$$h?f_PCC(0=19uAZZ94%ObnF$fyB#acSHzJ{~Y@RNbs&-r7yw-h8Ul zW)9o;t@xN&WF_{imTQQpq6h(Zk}ILqCWZusHgnqy2`Q;CJ*L2n3_A|6ZFa8p$rFde z(&KA6BBGYx$z)+S0$?JYW-JvZPXV40@hr5S6+II+Q1dsV>~SN^K}galhSn-SvyNeA zLSjvsZ;WN$L}Dx>uDWygPZYS_L|~Wqn_O`U!E0-wKRlE1?S`ruAyPJE|0E4>Hoe?; z^ctBXQofr}oMsgT`&f%i-+hhxRaWb6Y*RT<8+|;2S5R1$EHI!0-resMu&lQF%Ogc5 zZ28#iucx>9x6W3#>3?#Y`De5uYOSnE}&Pk82NrwX- z4S}*JEI{N$M)TIr;!I}C-Z6AZFpt(`Ob)9}ka;xPqYXS7Lz*uHYD4W=p%!}>VgZ~I zyRglX!aaUa)X7{nvN~q#fl(sjCVe&KF`Y3MA9KX*6EJXb8n8}rCczg zulsQTcww*h#1J9!{;EUzm(*RfKsfJX9Ya2uEpW9vPUg3Q@R0+%+Ba;6od`*{u<*La zMn4X4JrID&*g=P`j-a;@a!-*Nx1Nr~%KAUxm&p!j;Y$6TKhac;154C$&J0)r&6{OP zyZY8!-Xs3q00_tdM2n_0pe1;w7F5y{nh8%ehpA@HU>smWb1Wm1uZnANpikmcK6O-K zE_9^wOKlvW<1A|!aY`yRV?nePzv_4?Bm0Cvo^o@OS^)l&LS82D`E@3dg5yn7nH9`v{ zqMTc_ZYyT6b*w5Y?CXO)88Loy)AWkHq=!tHg1&2MW6M%UM&FK$|28H5$S86&Efc@&vTOA$Vt3n!qRmK%hw&Yp!pYQ&9?Nv!iVNHya)hRn6~zJ zMd4k;!e8`lxGyPV;^2&4gH!%?@Ju16{vf81No`Z=@ddwk`)ra!wjS}-_5Vc>XVE$R zww%KV?zdM*o*tI_;h=uK&h6JxykhXW`q2~7y`31g=a*!EHs6b4gSANO&ZlUl0KD)q z7wHA|F!6_-G(1$(T|^C=LH34t5@TqKie@` zo&=Pyp-y`7(3EFinx%r%8#NNH7N@WgeVfnUa1&XL@N0Y;tq%01Yx?38mYV%&rA-Y7 z#_)YcleGxn=Q2KSmB;Y-TRcjMGv8Bs&8&;Kkm6HtmrQIKS+?TahaqAml+nC&$>Omb zN%T=0{nnqUq}Vw)MK=kqU0bqv{D78y15>jm(En{oJSS!2-lTp7xyLsiet5rvz9Zmh zk6H@E)$)h8v?Y5v-R+0qsTb+9VuTOMB!HnbHeKP+{chJ5$m>^DB?}F^1hxdDq|&B$ED!b{)@S_=t5Mpy~EXP z98iG%Dm8*!O@~1dsZ^l!zEkmpMQ9*$o3lkXPLBO|ef?(TL}I`()gq>{mrmLPFj+u_ zGy~&k44bv%>ljv^(shw`fujR&0GPT+583e55~3*_ZHQ&)^CFj^As9xVQ`OF~uotgE zHik>{KI-87gCU6fPx*`-Fg~_m1g@#M0HDRB)0oxTvelh_y)C?eQ)tQNzDce~O6YsR%><){-@*QC@H!w69`9E=;pJH5 zY%Ck4HmG59p%x{%o`PnsJO$D$?=&na*0VCP!%k4qfHb$t7Z|<|PJ#Ur`cFLo#X3uq z^P4B1qhWApDs{R-j)}M}Q&*m2#zDy^V7JIb%#|>*TXA|P%&fVD7wjw^OFaqQh|Q68 z^H7XDx?hTs&-p2O&sx;5b^hov^8bLxNJYg{!y5mWn~ZezdU0Xqt2boMja+`ioIPXq zHch->eNo)=XJ7yIQB!qP8f?aKbd}nChRYaR5(jkdYKKFMoKT zO{+PcOh2Fl(e^&%8o-V7;03-4Jmvj8+vf+FTMRRG&2fZ@&^a zt`$Gt#W>DZT;bGZ>Sh9DjY5u}l4Fv;h8^z7^#k_1zLZ?yzx*rRB-Yq|zIWuGs}xuTi3rriz{!1$$P=880hhfm>wZxvu-5?^WFTz!_aVVe3JabpYP_4Il6h;N7XPofagCBru3eRan~XkbQMBUGgqo9R z0HZTAt3RC)pPp5l-ZCpbeQ|AH&QPxyp?5wHbAOq(laoi6n|w->34gdI8)J~LVb7n5 z#k7+iat-HTz}L1L6&5bNnf$LhTn1zm4Q_NC!E;$x!!(?Dj3r}!J4ikxYee}Pi6v3d#_sqQ6Q{OnE|Tlbz_Cc*@DZE*0%x+?!zbxj@+Nd#})Vef$mcC zT;zJ%BFTqDHr6H(_Y3Z^&4uj`%Ud;g3)GfhY8S8>MwxoTuCaPK+BhUfG>A*~PkQG5 zMwmJlkUNpqE2p;69cb9QFu0$9n2(EMNYez4^KBtC)7m1pWR1HGN>3b)&IcX}DJ7V^ zOdxsyUGhj!Mgl6~!>4eGir>8`4X8VPXE4%9LPY9H^NWCyqeUNsA|6X%4v|JT)11fB4e1I z{cz&M=LoA3LufAEVv!n4`9d_DGc0zZ6&p8$v>G+%x<*eLYl#vgJycfMj9vM1E^Z7= zo;|}r&z%(FJ&!RH+(Al>2rX|po4pc37F`}OT{Y3`_RTm@Q-48->V=p->2Sv4hR4kK zP};-Nn&Z8pf$rjbn3tdt&q%(fK2Q$MMQW66%e#eYgw)&pOYrZ-%hf93qA$U%;rrl} z=EL*|RBMX3m;~W`i593>|LC|LfKaGR-IHG*-=q?Aj?9BJI-#pr8f7%*wdR}>jJp>< zYhSW9@Cnjy*O9uwJeIxVVzX}6O~vkP8**#zQZNBNv-P|}ZpJq6U^Qb~a@i+^ttZS3 zU6&j`z9v)yAOGEB%DyUzE{KaitR~co0=6&5lznkTLe|)&>)v0s?%mqv`1qtFdMACJ znYb8M{;$azsSR)SA?f4Ez0*haPJ8wJQyaFN+ERMc zXWi~-B{b4>QxR(N){2H~rBW~1 znISw*Dq#~@Dr_PvVH26nHj&`arU&XJS&FJkSA(=#JshO6`blO;Z)8? zJCs%uBFA03{k4*?ovtSp*-_*xQc^xnrN1Vz&j_8mls&)-vPU%Tot9hhd-}M>^c!*> zcDY1SX8DYolSPD58+}WW9iv6Ec?(EHB))3k*z32Q0gi7L`x$M>627u-L?bn%(;@Kk zm77s|8agqa-x>lsj}dcZ3pT?LwNQM*5Dl4D$L51Evd2QeUuKOMb{^?E1_Bb7_?A`x zq9yb0=gtYM%ukO$rv1Y4455oB<9kP`|C*%OUdNBNN;c?Kp-Hok zxH~8a5~EU|R}?35e;-Gt9n;e~#^)Sh#T6vn1jYn__#Pz|hA5W;h7FVtcLZX;{HSxi zGwj7H(pnn+7@7>vio!B#IoH+s#qIlGo)tqoT2vd|YW_#NZ(L@ zk$5Z2QSq!x&^U6(bsanNV&i80KbF%)3-95J9I=@I)q)iwx!B+Z+e=pnIN%^K^eM72 z+IMCbDZ$(m80$olU`5G%==mi9P40kVjd0o?2@0~r%HUwdNULXPcZWu0(y-^{U&IzO6gt0pUI+ssv} z`ul?~L;66h;?_7hR&}_YxE2U|_hx>6;>C1PL$;gB?3YX%I<}NretmIDxf0=(-bfmA z3rgq}7Mb0qHMe%`%n)0X9#L!1aOxEjR9^f)v;bJMR4gdgF})!ZkLNeUb9{;MYU{Cz zoW){jFFinf(S`xIlTATt4J4WHV$Xk!ey7zRi?=pb%pd?^9Rod3Y|Q?C(*gCFXr?Xj+mi=%Q7`SvLqaX<4(^@y;Jm~qd(63J=c2OD7e)II8n_NhzftXH_fDZ=sg9LB61!IQ zC1ljG9zQEE*ZE|fKyJn2; zQ{8K;hO_4=gmy)-0Bs0Q%QHC2nk3Eti_4Ez;p#>@g`hqA{_quKLqY~-yD;@e ztKjQUnIBJm{a6X;$a~Bm!~qMw9$-mAH+eqdEO*=DE_Z`cW_pk%iQ18!L1~(8m!ps! zLD&;c*k*0+{*@QBg@uGoAO_a8F6(1aMrFi@ywKOpMZKK4@IvPaNgK*>gbq-xM6c@d zN?pu#%O%5gI5i7Jg8hw=v%0>5EJ<3)th*hykrlV_Z`j^~A8!+dR^Ou;8$x!W5Owco z*7%Y&m@3!uFz}?7>cN%Vt)hjMBbu=VCTCwt%R@*0N!1kE7QTK#W9xHl@L&JN1&wyE ze<+1m{8y}%JNg;OQ-EmNZ%#)7R;)~hWZs0$Hlp@)yM)sEdVCD^sPjQ^r+C(C=8=RNe>CkS3p061WVoym zGhGr;tBLM$6sY^bHW+&{-E8;(J+6wyGkqh0j**(6rPB3A)0QJ}v(3c73S%8O8BCz) z+U&-B`WQi-HUZV2qkON1885x?p^2?HOT$JXmzb+_L??tUKXn z${)RoTpx91r3PWpiR4|S-c7)|Y~zg6IRV@RZ!%lw8$k!a39R$oH<7hldTjyn!>m*Z zN#_{zB17~V(4)Y`4)l=wHwnl99DM;IJ$S!ou2ZMC4@jBeAwdaTB~HdX>StVE65$T9eAavNm!9i$zvex<_x{ z4EtmNRaw2^IcY&*B^-q98Jc*4-#@5jaWOO5|Y zZ73N6>NPHvrysIWBIxx7Bt@+pL9=7(&U`P#p9Q* zi~B6CS5ey9l||w2`Zlag%IH;?v1M>t!P7r2| zKdN`a;^L*vO*iJX@5oHpI&5ikHhz2|Ga(Bp)MY#^bS zU_*1cAcD>E>$iQ9g$-%vjwRUsW=ChJrRd*>QLhL7LTOn*8C!3Zu_-cebXXX7ZY;{{ zICDPx{kP%AA@n0m{%aU*xs{aq7$tM;+9_&nX^<_ZjjQO`rf77ZcQ^c{Yn6To-ET|i~t^2ak;^S=ISX;*@XM!ZVd9>B(0?hnu47gk7uk)4V1`xGn{ zd>aHKvY0LZHRPGwo)BNR;0Z;?U9QxP1uJkW13I%*;yEwoNb5o+f?9lu!hP4W9axMd zNkS`Rk=2hT^gnq~^qx$pTJNV{ItZj8Qa0H^#rj5iX0JHf_I23;NVAgQ_H3p-$mLkt zikCO8`1=?koW>GS`tZ_=Rj70S|Z2!jg6eJ=Bm-nC}hQ*#|#pGx)YHnb&Kc1XR zVk2qW=2t;gY?SDyc(?*)kxke3Bfl1linow=ouj$QYcFtxkCB_t5cbD$S>1Xl1!2UJ}hP zdkigV9*qFn!gLIvp|@~E+}u7xuz7f+x%Hg%CzW5og`=R7q?lU0$5x!GPyOBMpMk~- zB*%^&SI}}?zh$Kw#O$|@LCXmW+8*uN<~DYF)_SK3c&jI#V{WOkNIkjSBWySHG);fF z3QayR@GGAZtqct(f-TdpvIEQ)eNEXZet{v8Y!e6i)wO1sI0fNiVCgWuVXmZ@1j%}Z zz8`&J&-Mpsj#4R2-B7Bl_`jumAF`7vyb^jlcFKAQTk6XB>SJ)gsqKeDZlHA06Uo(rw@&gN>D)u!BU=tyFSeW~+A@a5t*MGz zIU=$Vwve~w54bI_d}KK8t`U)2;qHHHe%&pHg=!#%+&QM~vyx~eE`D~8_*-BL2}A$w zCDC%&M80)Dx%qFGZvKgjFBAsjsNr(_h*sJ4XNDSv zC1$M6I=*4sw3M|AiW=U*U$sRI>!+oyJ+^y8H~iZ%$2UAaEoJbkz9rE~MQI&9Q)VW= zlaRJ$U`9vcm=iX%9vF!R?~&3-EgMC8A+(ET+*z|k(r9lIRP&(Go*Det(nyQA)nIlo z*>!c~8$RAr*soU+PQCrv<9Be4R5ww&H9zq>;Wm=M$4|~r{NtFFn|5y|^NlmFiJZg8 zp;vLgiN$=s3nh~HX4&BX77iYg0>E!4XPk8WfZ@o9HcuXn{r<npkx**qD*ZO zi8YO7YRGv7+l@d5D&uX&bi7n)^(3)hOUQ0yt*_n4SAljTgWu>|-CkEy1TE@E>xagP zNUA!yZz95F{K@$v3nBjzD1`hrPzd=CqYyIpZx(}Aal5~%KQ(S$icas|j*&yjvn0rF z1jTUK{u)T693?H~3V`e>N7j^$y>M;iB5s{#hZl0$M2x%d?kxdAk!liw^{#~8B6bbe zpaffN<&v7{>ihi2wDYM^UB77^KX=8Sjv#_A%6@5=Xn7aA`4n@bna$9owobpG6Xrye zk+pU#2kf|qL}2f2hr}i33iUstj|{^g%!q#ro?)?|jggSjMS^{+4<9N&(iG1Ng2FF8 zK+4;0MVT|(w#8Yy9yrHfNKD_`^h@xT7ky+EBkyXenDVPzQ@L%&u-S9n#D?9jf)O18 zSfLzMT)_7dqVzvgpE&EBqnx zD(X{z2iLRF5|+=uJyz)8ITL# zl^C0@tfpylAl~b|)+QjrCVHJZeF8srK?>>Z~NqY>g{;EhDT}_=d}^&F$Yc zd=sPAWUAB?)AJ$l2*a3~fYAFwW6BP?Tfz51IVo1W^DWII3Q;{T= zL>|mN0rhf9V<+579FNX@JuWGYweD5jA;px15QIBZdF&?G(y+6a23z;GR=y~wBWwvw zHzU=i(vms<-5x`f23Yq(2U*{BRIlMFjR(*4zvaAHLEW?u4d|wQ`f=uN+M{GQ?MH^C z6f7!g=nwDz{}UHqe|BEOrXz<$$H&DNo|V@y1wW37i(kEQWVAdpv^sB0^rpM9ty?A;jZ zM#$xud0$Z()fNJCWs#oOM2;2^Z73g~a82Ztcc(692y)~@WpES#4mzwh@h8FsBb|>y zyMLoNC#U#O*Pu1N%B|##MQk6jdkBiyq-2w1HcGvyZ-7%H_M7$v2hVM5LI>|x{>!u=2ZNnOt zjFn z(DP!DpzETnfwGUVs*EggK`cTTqGuhJ1u69M99MiB;Vz^fmo1cKT2mozYcxTgNJX|B z*KIcIAOkMqnl*E4Dl2pau+qaSJ5ssx1gHZm#6*f6b`71WHjXK1hc~K?hm(4;Lq6&p z{W~ojaUxWFPb7Jox>jzFVRr0d#Xu3{ejAOfFp?b#E7#54KckV(PmM#Pm+AkU(MV?^ zuj2rR?t~E^|7bK4D_;2jq0tBfw1a0f^0Pnn{_NmEv@?DXh_>&*AlfqpFBS%|Y`5K6 zIi=#e<6$atVRvq0R{36^mL)ffAGh>rDZPb_HC|lNl94SvF-8f8k@iy?_OR*5=h^8; zKkzkTrRu_r-A;NXbUNda0WtqOO(jQ_|dUqNbiSRz;EG^^3mUO2*kb9Fbx&=~5#>pO1NPuLH0h~0NX81to zk#^JWFv2m3X80&+J_SW;B~$c}W-3ykzx3}%qqrY^B*Yy13La+;bC{AouKleVgd6FUD3r==8O{m9 ztt5ry(Z7_AIR+UI%BnJP)5cn-6XeG(C&;-_RXyx)1&MILqqno}%ZA_hOb$%ZhdtI{ zi%;2;Y!=a=^ImkS7-73HX>5gfkf@N)mwb5)IkfTcdi02LHP`~a@?y+t(M%RYh@J_M zh4}hQH=@Fi!)iASQOE6X_7Neu0i{uRY(r_#hL|*_ir%8P7BEB=qPgK0Krh!;NR%df zUAcp1Sfe0MZIGboM<7tNSqRAZaLRMs(k_Qv<1D%MxRzTlWgevTy5m|_@QT@5QR{pBe;hOvQ-h7&Kd7f9;BmEUG<1Bn-{3m80dNgf&E8w$t_4N8F$ny95kET2FiN&q)|jisdI-7Gy&8ovwf zb;CL~7E}G`Eb^nXh@C(pavTVH|C~i~h0~^c8AaHzA3GU($L+^XCMactfS8x$@nqk@ z#@v)kF(7&((r^ixd>K=T(uk9YH0;X0zF7mm8B!>;)p~W6D}g?iw{b7Gse%sHL>d-6 z)A$O)7fToZlV%draE7a2`4doH$nrf(|b&e`uAJQ8jD z!9!@1e-MPmjk606K{M^1C}iuZZ7#hjr#vkjsoYfCyyV85@?`w@Ms0HwD%6g^kFSQC zPayxw*=wfXo)~7*>&8)aH*zrl(~Y5VRmIQUJZiu#qfqab*-SWu3e`Y8`4y_|l5L{B z3)MOUoFFfo0lIX+)vttp^-AbQf!(S{)m8Iv^#~vTY3^sk_U)bUhwnZ(A=c^9k|%!i zYUhmjb+WHw4E{mqrU^LGtPH+; zoAxGCkP#8KzB5y5O-C*4Sok4zj57)mvB^L60aPoe_Lsgx#_I&D)_U$Vl6HaVt4tCN z!LuPQ{XkgyH{r+bLx38oLG>d!%|cOzU(06FZ}kp9|84@bGIj1ma4fx~zM|zxLpDTx zJtEip*RLV&Z68pR^U?ajaUdtFlMk1I$lSsA(#SsvN$p7)97>g_ zXYm3Z^Qnt)3TD|ekJqH-`EUc>Dn@xKoX(8JRm3GVkDR7G1MVbzu;cKkN{v!1sWPgX=kg+cQ z)cmzwx%eXtXuWo476)2NdT**|H9U{t>|i5Q`{%bXB_v zh{vByvPi2=XX;@uP< z(7tiYOMO2)GVjuif2MtPZ0+8rUg_6f{Mu_LXEr{6{=%&{?G9DD+|!u^CAJ6%J0PBO z4Vo3tNsw)ixiuViJg6{Uct#2|Zcmyyx+{3-Ki0XK5T{9=KpIl8SuREUb+I zsx$0=LT&B2Ohe|FtYnT3EMhgJKuMfAW5fF&Wh-D;qUMuYs-1a`D=sgIxrx2aGtXwN z$@G~1oDTa+i5492%D(%tl)sVLvT(Kku zYOa$KGALV#A~x@ACG+fblK^X?scJPaa+bV<@=a7Ip;QmK)VN0E;Fyz_a&6-h2&k;Q z#H1T$gZB=d0S`FJ-G>Q9z2Bi<$1G$ z>>|^H>>@{+?IL?gyGU8;Rw>>7rw>c(kX|G;M~;lXFgvF_KOFgBTV8oVkI00y{PICP zB4^{Tffq$?-Ii0H6OK&4zn#&evFhefV#Segn2ZCQJ7x$q5-2_Nce5Dcl}dc-JGO}!Iz73wYqv6FrK(>)Rkg%wFf9s0Lf{9C>9H1R5Pn|eR9 zO@|5N!kqLsh?N`Akiy5K`lL@@HhXsmIe_@|)ovLB7KyD>WQZRlRUG)?6eD#;N*fY) zVlOPu4(8v8Dx_qmES)PM5Nta}UV_Wn!1!6x$#9AF$>W932>+@^?1VKBUWwFE0P-3q z_~K<5r=rt~kUN((eWmen(@2=vAL#8L0?4)E$HN)Ok)XsxwUS@BLm2B5`7c%F_BgD>+Wd_o@0E@P zK6P%5wDHE+ko0m}MFN)G7-&PHB5l_$Wb=xw;wZHUwGo)Cj$IGb8qHr@k+N7$-774t zJ+M7v$lUv#bu4sSej+obh8k)UEJn6)}wjH4g0|ot&9yDaww^7BcBjZFY$HaY&h}n;%nIv)146){}YxOI$pLErH<$eB~0|r`Z1g+W@g78@6qMr#RDaerBevQ zT8Z+^Km!QvK*(bmUGr0Ae_l=cy2E}kh{&J`hY*2%4g4fBXia+NO76jh!_cIfUe{Pk z#DaQZ=>cQ-fNBBuIdHEg#kO%R#q!F~*sxYwQ`|;ATcB94ydk{C$e-aoG~C3#;0m8k z+7=l$T6rTnW=v(h%l4u8-JBb{P9}~QfAU|ytMkPT8%!0F`bjK5Ie8C!M&x`q&3|}% zzzGZs!D3V)eTGNJ{b})p7S#$K72LsrpHNT~+>os=V@R!*N$v$6fx^ zkm!cE_y^~Ql20B})_rW*VI!me9T$IoO8i;#L!(a`Q+DXhiAUrlZoOpbx{liBOW-7# zoisT+aq-1V*L}5YT{Jv@>$qNNi_XiKcwSn;lSSb-hlOA5+wfD^N*1K8eY`0A=V9SL zoZhFrI2?C*Rou7{k)vQya#McYjd|sR!*M6U!XzDd=fTif^FsaMPqGbvN}`itQPKl{ zW^Q|vElN5+P6;LV8xuVW*8cw$j!-R|^z`q9A4%o+YsvNjTC$yDwPfqL^k>&^s`@#= z+o3S}zXX5p8xdI|O#KVN!)-h#Wo=baLo)E%r9DD5z-#yIcT2Y82YqwsU(S;-?O{L_ zFPQe;iuRulxPvJ(uWGi-X53#NN`J+#CmWc!DddUo;~9Bz_GfSt(PG*V*Qa!Ja!Dmk16z{XM;CJ z!?d!0hE3WD(X3xZRSWKUeOyBBx7+k$XEOAk9%Sg>6lCb%^4LsjE|8Bn_}}Q5O-eqa zRx&U5CD}3i45YREJ7&{O&hzh>4cId}repRtmrfFuUH?HllCRB)FXE4b9rsB4>AhFc zI!&@G%0BzyG+{Gn*sSy^vKicJjHD2bN^uEY6C^gfE=X+FPW~hHw<6ezPN*trmz}e- zvvOO|mW?9p7KPOS@3EVEZ1v*nvM;?Mq_XyQ4!h}FBBROOp<;^T&^!!=A3S@VqiB7y zg38uyogqVKzms??&)wR!ZuL6>aLwAhjX(-;tyBlShQN$Kvn*taO{?qcFH^ONhWJHS zuIPFx$CjO=4I;l+FCeWfwITwXYGj896#{TRPFi>B*qE7T1Bg~nq;9UgTuMi<4yhyn zm|!D{;rORC3QnHw*ESD7k9Z89W5&{&Kzbf(h;n{Qyg{1%r8Z0$oC2A=%_6g$m;=3P zh<*sFxT2x`ee#~ixtz+HsP5<(K$`l~xfT$cWhb-_-wde)zUAo86vSg&84FR_W-l{Y zi3PAusqT_!if2a35UCfE0gw*}2IR|*4S2=0X0Sr!E`W6LKMUW~YMB_|_Wd-}hM7yU zY7%Ggf?$GTB26iBTQR}1DsyGyNZdm&vevw__+SuRW^z~K0-ktE{ z$3+u<(XP7G={>~0N{|xlFTedO&27g*s-;Ynq#$FW_@Yi}OyuJ~()!|WI|5zIN-zJ| z-^Hw=Y4_gJ@Yv9m1(S1=M&%}*oST?++0u1c%hzQtXnp_~tWVP9WrsDSAD%IBXj(yC zQMh+;_~^Wboq%0KGPV?@6+BZEj(_*K0e9w=PYB2D9ub+)Gk-uue%*|`@}Gs{UayK< z-@Wnc#f?E8B#8kYBx~}_9waAA`K0W{rhFvPO1tx)DWNIHjfs-c|K)Hfxxc7k=d`qf ztnLjbC1w=#?%r@L{>nVQ;l*hw1@nsE5z_P$X1;7;H-p9nwDF-kk8g!x~l} z@?K+S-Q772{SO(^UN<+VVRctYG+h||?>Hx|W1wiIedWAR;p^e%Q_%^|){9Mg-Zz*% zNRCG@He#!yr}yRtU=_a`)>tEo+p<5OAHAzI;hmdH|YK+7NgQ`ZiW*P&Xq zA*v>?2(dGu8sdYr0e81`ciHsLfW&A4RR7En0t(unn&+2i+@ey=m zKTi{MdkI%AIcvKx=wdXe0ihU4QV8A-saXBPuam0Mf^8a9O0tdK^1I&XxHe9wvy-4UI;i5ZRJO#QWz*iEEfbRr!Wd7 z^}e08setU?mXCO~2(r?#pwfcmBe+R`d<1Q{MspPHATSVBtK&*C{UB~1Sd&!t_x zUkBQ!qcH0uw;{OZhdC`tCtbf zu6ZtUeNFvq7c^4Io9xS03Ge^HH%}B%0-^$NPfcAPvsJwsk{J7o%~Uv|hRhJP)mDG` z2WtN8Hv6`Py0j2c>*rh90F>7WC%BRMb*vLQF>VnCoFG$!giFo~%mF8>9I!{=adJ#0 zhD6lX^+t<&f%JHmOdJmXd2M~+h?LSa;ifD)=O74IFD%t*T1*7so=RB947?mC@NE>m z#|D5p7$>v*K)y`{-nKbqXWJ1GQ`~JV)|5Ef#umGAntWJ*og%Du({cJ(foLaA7cDSE zoDao10I>dU7SPbAfUL}KwayTXg;TH2Jb3tcda1Lm84kI+0MQ!1SWsEWda+s}dawRd zudr~Gm&S2fEyZ6wW0B2(rmkR8N_9z^mZ-&f*7RF z2A^iP?YF)RzF#ge&b-UPw*>Ilpir=0+iN*9CZCLcERHADM|_ffoR=OMqcBs zJL@MX7X-Y8=4Drxa0?kW)J}`MkQODXg$u6Y+#E{_Mh2baFkP@OTKuIAq%kN~2wFcg z>mFigKe4xs{{31bxhk8^K=<-Hw$`#U{+3UHiZbhENr47;3y$iFAJZJ6`DkHCgIoiS^-&e zQ?o{hdlUw^{?B9A|BZpJ|GWHM|BpmsY-4&!`FR4ARnP1$i`I%B-Dnd;Hq{?1(VItg z+x5z^cPwsHJ@afXU!!Piyt?tip63&RvtwI{qrRL1-CBof18g|)VVfMx zuuJ2VZnpCKk-DZM(=y5${hC(rB?UsV6P{$qQNRBv9hoATUx-!8y zY`UuL3wfJ{6Fq9C@Zvtd@~+P02E3e%+%zxg2+y{>H>9h|n8CNk_xC;aEF_yDS6;dFkd+%r3WfQczoH)grKBqFUNCtD}EMmzdy@4X#?l*z0xFT3U;rP%sV7 z`&>lj9#3mk03E0E6OPHMXCNxU+^epwa7rjJm3gN#!@(n+`z%U5xTmX1w>YtcjcgOT zku#^I>d->17~<96j}46xrSX~Q8BPrwslTKt#la7`tmk90{DCK>Mw`~4;S?@uA{-q% z*>H+1c#%JJ+{Qe&*)%_3Nq`N?Gsva7ERBlEK13$moLjvCed5&qNJm<`9+&JO+Ait( zW5*IAHJq^-AU(cFcQBrFK4#0}sI2F{zwFplk3QOsUmdhQ+E-2=ePv16w`0n_DTywE zpfNeVUrnh031iBlCDD0t@gMhyfB&tS&8hK8lY1wP>YaGSn5FC9UbgP7+UD;k>wd-E~#lROqX>nA4`o_???9N|z)2IPAm*ic(F?99DnM*!t zK7<)76tB%A#O7JN2G4uZklVX>jX0?WpW?OgOLma`e)_6D>2CcPMLDzS zQCypoR&lIAaWn8^ANmp0nQ}A3B@(2N+|q&BNbYGYsZ(|&ozaX)^{1yO6t;Mn0%pVI zDFn_VQhAJjqJY9eI_6=@Br3G~zVEKg8nU5kMu=jl?4P9JpXguj=rvLlw8cHU_P)2_ zpP6+5Zl2ciwED0se#*x$=}%gCe7d^xnd-x;=u~m}Abc}w)NIy9UZt98(0T_>-uqqedf(@M?)$l) zwCEjdM(tK*^FBtNi-k0d6FTkyK|>FEr-&VF8X7Nq0y?$nzlZq`u2F)H+I1U_WybLG zi76c?uOOz7H+WNjp{whc`zlqr>GuBR$0 zeBTt2ov_Sv@M$`7Hes_Xl9nV@1j#8bvRuX4u0NHxlvm8J)Rc3wXwL6Twj0IMrJH42PGC^jNvrW`H$mX(z+4%N0amq z=|)Z|=omVJqHaIXMkjSUzhmfV$MiD;Btl#gy$l++Tj=A$BV0^H>gNcKt*+(c-Q+{$ zMS$kDBAhI@o$unr3CLwh(n{{zBwxfHlX(f`f(liM2iK0vz!29&>D`EFDOoF>nLuXO zQBKA02)*SX_rMyYQ#A>hVA7!C$)!o|$>|h4ER{MUp@uVkGGUAWk;f8pOY;D@rnFRb zhL5}hTTp*{?}sq6Up3CQ=U>DcI8?~It?RBt&Hy>lKrvc)^L_FzALg~Vh{~kP1z2{! z;59q~C{_*UwqjwXL~{zp-jk?)Z*a@n+N zf(NLg?addLXJ=+l&#JvOvv^D}=7I8V zP`8Fw+uO$%Z7U4j785&hexQ=-*p83Cwom-UV?|%>s;2j4tL?b8KM8Wn&fS?&7&;Va zZYBQxEH|aSplj*C)-BIb#31sclPZuvu0M8b~4Avee2+-$o68~O@(bwiSBH(`lif$x<6r( z&BMAaZ&PjFzGAokJJWis6b%462G<`1BvLl#q4qCvp;JXq?b!Q@4&fTAIpTG~eMN_! z^58ahua78=luAxX@!*3?X}ABj_;_U`hxk3=BaSQ#0xNx?`Y|H!N0A1Sydf^z|!R&a{CK?7QHA zqB)Q30*8q}6g7nw*rzcBdE9^RqAREGh!lG1E#4CEMpTp`!lPo@dWoz_?5u5^35xy53 zVL2je^I9k}J#m@*Xo4(sYQZ@D6&7-;0!gnFjBwji3b$#OK39;d5ejfyNtALnt0kw5 zuqpgw=Ux?6+eQq5Jg74v`!plqoxqd^6F3#rn$AzS;9o$s#`Z?m`ZerLh_7-CxseXi z8xoX7@IJ!sAS5;ypa`oJ%uoqV&rVIEo4b&NXjw`&e^8zb{!8|PtBMC6Qxg^j4Zn2n9JtL62Wm1=Zv6 zDKfvJgy#mh3f8Wno@<>sGLgzvvCO()nn<8I<(Qe5a03!Pi}JgZ?O}q}#e`+H%<3ps zU#H;2r>+7nmW|eAkeS{>vgt_im~0)rO<=|L)?p=`y$)>m>;qW2*5{|FT)U@X+VJlO z&~~iadvx#9S*qt}47sCA`{zgJ{<9xhL;fV5Be_3@=Sa!_4$qOAZ*Kyt<`6@65D3N! z1ENoI@b1?@YKx*GwU3VpB!6&`5_#s7D+1;Yc?e>V$_&ol9XQ79)7%Z_n+IAy)chuC1_Vb#9Crhpa144?vO}*S;deA51%Gwktjs_^cS?# zr1(NKK;^Ez3X3BMA?YICQnl+XrD|J!OVxN!bLO^OiByNvv0%K;P5Hdz0?pgH)4*i5 zrPbv=p;#8&!4&j$ZJ>JXt@(0k;}@MLzCetpB_z@&ULa@w4PVAdQy-?@^zdvQG#r?A^E0HO9vCe%WUE*RbR z)=Ci*BkmNYBFHD$AO{VJ+ss}^77FDe4&Hq6MUz;^g{8s!d;J=6NGfymNOK$^1_ck; z)h#abF*1 zyO3M4+YkpZ8rZ047>%ANJc+!21@z;rBOusmUP~et6ED~C_@|~3xaJ4 zR!;z|C_!Nsg8WwQnKeRO=*zOfeMH&wx>-2Aq~HTh)vli+hQ=pw!sXg|RG=09#SK!z zSRlQ1QIygu&~T;&ycB|YWop!K(tlp^$`PabSI~0LT%F-c#gmNb4ajEq5fYmjHZKaF z!#NvG+_E0bzgYNxnf1uJOM2Z${+iJfj{-vZ6ixA>0Q#Ah8s9}hsy}}a5=M1a6vYVm zG=Ig&cA*#<>zuG+dE}h8gFkN;1r|;mEJ2z+pSrYlFwSxPaly&VUh<+$t3nGNLXmOs zT@$9W0Mv(u#QsA`Iz<%w6j-ow(|@O;V1zzPM6pQed0*{al2#IXR&rVM6R|&Ajr>?v zBU8Q|F+XtX8@Vriadg-DN6oF-e(${M^jUiw7mvQ{__&I@XMMBu;6G1z|KF$m^V_GJ zcm6}k98VvxS%AC0Tw3~Ae#ZMl2lPH;0JyuZM5pzb`A8S8+p6BNAap64kTiEaBeVUo zQA{ySKXU9YRmSxxcMRyAJ77{SW5s=!mL5!U!EU3z1!49-+K;4nE^8)Py2(x>91Rnw z;>?U2XHjwzW+Yp;VNhrTaypR6tPqIAFr_5JY9FO!8rhVPG$p(z8%rJDM#1q__BWzf z5g&w{$a3#ro>Xw-RMM3)>{Onvp;&Z&26h)AfR04hieOC zSbL7{HGeLHWy|`|cx?Akjb0ey@5AsF8=?j@Dt6^^Nm_H$L00*V9yBC~HGZ}uaag}27A3H5RwveMC5} zZin+S;&dXUp_zD+1T^ zmY{s~LU@QpQue_Jr0&U{pFwxYt155vN;6rbk}%|5viG1M}_Wq~}A3U4iez5AN2+t|dg&0}m}ZHFbW*qmyKeF&e3SLsU{9-= zwNsI0dOCV%D3)B^1W#tmoX-HIL5B#9w}OJ^-sp=e%G$d@atOyO;Jz6fH!{TAdY~ZQ z^26WVI6WahW>tC23V8bW%&MK1l|3!9cyusk)~M23pq;!WtM=vrfl6p4KO0}PqcBt& z6FaAS?9KB78zGncxG+?V61lOcJSzdNfV``InqF-kQ;rx>`W*-3LSuYtQvBndX*F!o9=`rudnZ`08y` zdoi4Hm$w;}<;+B4zK;et!G}3Nn0GC{WN1TPX8Wn4@@+5vJCgo;BOX_+fRuSl_v=T= zzs!y6^he=pJknE79oMg-FamOeek7GVNm80G&}!AQl1M_q0z%F1=S?O!X#VH+J%+RG!zlccct z9QDg9wMWSXQz*GsJRGvf3e$Pp=1fxez$mSgxKdqcIaev+rf?|R0ws#b4!c)Nhcn3w ztZVW~oMF0Szc)_zdMS0S&-{rMb0pM3^7s0Dgw-vA^svM*1R(xDaV8<8;s9_EN!jc< zg`ERp0ufpJ@f5U}Fc&1LIH5-KCPK66xwRD@Ik!(ytfiw6&*oS6$~@ZYOSyke$MNXW zw&=Wq-R0onHmr132f^F{3N;IfrsGqgP~#ZgT2(|a8cnUDlv{wz5U?d&G9>GSoZ2zx zjFFXA7$Zi{YUj3wN{pT(!igjie>?8agtI@|$SDYoKfp@u_Km)i0ha5Jy-*3E!cj*tdzpA@WI#E?ipx2Z|1wGuWa5-C*I97ZJ9JUBroC^ z8UrpwD-4pF>sn9qOAQq05XuNec1SM--i1M6X^#$0lF|(QBH3~}@EUQ^eo-utG09P- zXCSf)BHzF(u!9iZuQEdVI*vHlkQ;;xxr`7Rfx{f6_BTAI!fkqG9}K>%yQPQ78q((E%qVbuGe=RpK3cL?##s!uIi&gwdjiPn(kdD@%Lm>@ z3sO9uJ#slHY23|@BqNItSY2D~>K|U$R_4>G?em|+J@Vzm7vfUdW0J}eSK!}0m4Vz% z!KTqDW?Ry~zVe3j;>2KS+9!pfRej=8^OrUaJUyu)@sDUxfOgud_{z^ba>yDgz9R2P zF01l&{7;MS9m$ndL55|=y&O16AhR-lea_2)iypma+qW}2kH7Tgz%?%iZVjWI zLt*0lvWZJ}rFs6?b3SBeogLFmOEdi*B>NPfvGlyRp#|$7&U$Hd1bGRaax}AIe#Ry3 zspBD(E1Mi(ggCQT+&XF^PB{>oRuA{9|QhjLj zU1Be!qa_{pnpWSmB_optzc?GuKd-AdXs}x$4v}yqi_n3W8VKr6oCQi)9>Su4l|BjVF!108yeYcWjNsv}u-K1$mV zq3$K@S~s+yp}H^c;vFHllT4#jHrza>%TUu6wd=7MycpYzYG0yT#>zk6~OQUHb@5{YucyKo(~ly!$?Vi1LnG_q<^c%y{hoC3{N8|G_<>!B&Ua` z^7}bIhr9(sbcQE?a_jE*;)#f=!n!+t7z43hYT3IC`cIUk!NR9S*460{0CZq zW#>RU@G?WTG%F*VLMm=$5E2CnlbcDDE-MSdOg<&Std`%RUhtw61e=ZD{QlArTHYME z1J99`pXiJX;{a9XijO)7D`n(;&F=WiF&PaTD%`NaM%>qAi)nj>8$ZaK>Z3NVxh}e8 zFJDC5NwZcsEIc{GAVr?m*@S4d!N+6BOm7F8uDQQ=E{~rV^*y*>NO%{EGhIMY>~_3w z!alNdNy*GQrqn(;P?TENk*zr)0HVnxE4C1neb$wY2H$A@Qk|vzqP+BrKUftLd&lLhAgxoBll{F12uJ(>wUx zU9&<{`hq#@vasp-+tY7beAa?BFD+|2|Msh=%-nqDf~F&IY!d!`uOzc=Zuh{rjPaq} zed1?6*rk3teE+l2ZTsWAQ16)7+*g80-`$3U(TXphcAF%rvS%>7+PXgQned}58IyfG z4lQfWB9bbSf#jUsY*g~@g7{B_H%ZccMac{^PL*B5n#b2k0jHbFGN^6H`*xRp%+BCu zhGz}CU{n$mxWt!FBl3wt1ve>YUL((~7(8uVnOs4BEznRfJVH4c$iXOl8X6Zkk(8%6 zt*$Gzj}@Ploc`D2)A)7NX5-B;U8UAp!tw-oTEuM{xXHKU*F+?&KPN7IOPcAIXzj;V zGW`MWY?Hji$L=3{SO-v+@v|c-nY!Wn z%g%~sNYZll80fNyHjCi<-+J#Lsxo<$BjX1Wu=CUApR+=(sT&d=f;Q!}mr3ZhZ%A_L z`AFBD*c^X)(%QWV19CD3HXM5BfSmrL0Cs>IE*jWGK?Wx>Z_#<*s|h%&w4~aPJ-IGN zT2k!kQvT&#Wd$(Sah`pT19q#98ly$2;s$n0RE=8Qp|3&|F!lFE6tc$pst@&k>=*1w zwyHj?noEI)j!%+=V#Fekd=9~!gI!Zj2UbvpkyH+Zu7E(LC^$>We3on*Tfvz}?^cBl z+alrUy@+|O!fc#QP}Nb2rWnyN8dyf_5@4SF!q?%0G{2D??XP5XU6 z%*(wmA8C!F^@Z8iudb5im$m?nXA?c?Z!KuQi7}-k=%Te&6k8#fpp#ZSWR-;JxlEF< zv}a`095@0>WQfxCoJmPeRl za$l>%WHgPOjSnJPqE~!E*<*7DH6TjRCmL5gECnR=_1RV6uhR~u)EiZ_H=o^oUX9&&Q!TSrF0%#vMPtD1LNkO-$^~NwHTf2t0Y*_@cmsqAsID z&&0$YK7P@fgBCXZHa6k4wD^n9Te_xg*_v;vnnuSa9G{kOSX$ht!)EsjzMmg_r+@vW z2|YLG^jw^ivijk?V157kGZK0(KBMQPGkTPcf>QFPtn8aIi$?@w^2=k+7*)Cr$n1}q zNcG1YHPENM_D^I|(lO)v!MFW0DcP5M(#(hR8#dm^{`sMEU>~`X?IYWZ%FC*ERuGYM z^SBk`b}cGuV=<&(yS8WcWR=7ZSP;od^4UE(b<`WK*56C=(cy2i6PX)Zab{f|`udfb zw=lla;EGWd$8ivk5X<&>Wh;wEG~|*&9=+xm4c+A)+(q5x!ex^h_K|o*v22>$gLp)P zze#6;P*RMhZkeqM);Q@3Y!w~Hbx0j(2=@PkD*bRAlWHCu=rAn-s$7e|mdmbNAeq^n zM0@y3LZoTXwbL5Hek z;(sS&gWZ_LFnNL6%%>ispzMPc`Y(v`V43)hmg;8h6~{(98$p zDvzVT4VCq&*g7M&M`~4%=Gn15mQ?kJA0?va%H!3Q-%xKhj)g&SNaZs~w$~f~lXy-1 zC{>x(fjNT-8Ap*SgUDiA)?CB}v)T625h#S8ZydfX??9_Q5NPzII!>8+Dt1Pz@=2;l z?%xj;2^obs83-M3sGzvxlkxC1qcnIlgA!*h+zYnEvyD_m)XhyXSAZg*sWb~@G$o70 zE*m-TE8Grj13VweepW>S6msoUWZWiF-M&$o@YC~+8wfr3Q{Al8h$@(&U)K;R?Wckg zRGOc;nQlQ#_Wyj3|0yZ0MTw3^ydl_ETs_z_C$+BRsR?1QJy0=t+W#n?QE} zl1N8t%P0R>FIAuwn#0w1;n4vv?OVkd`JB|R(vURRWJRtVKRkEct1Ph+FH|tWU{~L- z8*$3fPFRPN@snY?#G!m5XM3(wZs2;1g-qsSL;}|NrY9|F!s66w?9<9-2rfg5riv-_ zAIlg6uaAYa#0l+ydv*4$ssfU;`eO;Hv7O|c1BcY!I&hV1aX*Ef{~}(^(bi2i>WCm?!^yERn@ABls7qDogFOB;E%@)DBk2=#(ur4m;g8c0;Wz$0GEmzO*ZiboAvX%~Itf4-sU)k`MeXhw5Cwd-ur-v6iL5TunIUoAyJBsq>$g&t85 zYom4`dAb2n?5!8pm$hy!DElvmc;YfN#$*s>!IA0zz!(NQ(^0yVyWR&TcC zLVELLtn*abhk$|BYldYKc~K;9keX;OBh%M5sKGm|xQ62b&TGPIMM%bUCRaud`Ze@m zog6LLP=tVKT#FG=JpxrEXR=Mmq&b}`4z0<(B$_A(%$mq6>c5qjQ?E;I0Pv?kz2v5f zvfMNHE4MepgXBxJz#!!I4SSJL`msc-YAjz1hj_DA$yGNsBtZc;yAfl1ZjXJ|0;$>i zWk20m!aJzh^z=T+w}L%K>dJCoO{gKUXYPhyH8ft{iz17T6H&)@|9LuCfL&j!Fy8jX zX)qi?EbAFI^&06~Ld+G><6T(#u~(Qi29aaJYAGb<=8Ks3%sOsBnNnp?y}nVL;gDp3DEZ5KM?W=jM|hotePLcRvK z=o^NMJefv(yDRb*VIW_WaKdDJC6EsJ{nFmp|9&Iy; zUzlNZ)|F*rBO4L%rJg-)-_SF;49#cyf~e}8_U2!y!JS#ion=&t#4G7oPM=+-ncX#D zDGImQD~BDLe}QBj{=A-YC@Q&79Se1)#3anhAaH z`K(E2g7MNaawnx_#-OhN)mBMJ#=VMJ1KOpvu@rA*WghW9v=>KM9e^Q$ss#pDo8H7K zE@~EkMcPLnKr9kyoM&i^4A~HRJg>|;rE!p#)@^QDT5Y|ES@0t{`0^-x1^fT@Z^p!alN9?2_%rZnhfP3duec?Xmaa)z*c6OS*qNU2T6+A&=Pz9o zSF`4jg-yQ$f420O!GkmFUjR#%o3i@xyx^|<;OG78?@s7BX?V}x!=Jsb^Y|C8{GqM2 z<9!;my>1$`yW?wue1NcXvy zeYUu0=i*S0KkqrH^Wsw=B>$=WMp~7itNUrX7cd?Xd6 zW+2tw@Fd|3FEebM29OXdnW@0@ljpaB6=N5VFiR0$MA1|3Br-<{2X+SN!MVV4eKOY+ zb#nE`f;Sr^Yy3o+1Oqf)A@F(V5LK1|=~0;#QHr9Gk`vsu4NJ#nEm0;w+f&5cSwS1c z*aXPD-X;2V&3wez@_jwn5UVwR6=c4L&4yYeQWe|GTYK`bT&de>|3Vc8tfxcQy}2=( zQ{>h^m*KPQie?m965S|rtJNqnY1eL5#)d9!ExBxZR~7Oj*XW!>OGxDv!7cJm?2^M% zU%}Sq>aTs(mh-cllTS~+cv5!rFHT2RAG}uMEu*2Gl+f4)-AME~eA)(O(XI1TgwV8+ z7IY!4r(Di9dlAJWt;_xp!L+Opt2!K2N*ZscXE>~QgMN*BhD9H>Vs}rUUzn3+6h`>oDyDh0pacDnOBxC%-X=n1+zwU~*$*E7XZZw_p+mR3)Y(Yf6+dgs_ zc@S-KZ>B|s?;F+?ohB|s!zvJMkzq1**`nQBPGR@06{fQzlB~ zvMKs#;S*ENLVRD5HP=|}@ws1n+^+GUh(neHM^yzfcSXI3;BtnfIy6e2ds#26$M~UB zN~k?BItu&hzKELA(U}O<+-Oj?nDd3b{NTWTq+>@eE&r~FTzn{w8QqrMmM0R2kQsZ@ z5MIr0Dm_Wi$5F5bVlgG&R0(*!7MUHcJuSCmidzMwZR$lk7%9B9#W%VDz328o|u3C`DK4P-(*|MbuPz zR~G$R5z*$d?!o5@GJAJ;X~m17p6lZzWU>QnF2btV@CB7ak$(XMiKpGH!jf>-Qygra zzYXVi7LM$!R=bW~A>~H2AN25krOfI+*hqNIHD$%XOUSm|`6Sz?EKX_s)#bD{nB_uu zlmF5EM{fC_F|B-jL zecQgTcYY*?$aB+na4W|!6{7ZTtl4u}wZmy&i5mqKhB}M=6hl6<=T2oz+&NglcXdodPe*0@8iNzyP z9%&*!_2=3Uuq}^FwSgM>Hi{-P8B`mR*ySm_^&hKn-+SWqAn}^?nU^Vz>XQQ$5q}*c z@^J%^oa%Lp5g&_(JPGwyXXZrgAYs~vM5R%czL)Q&mJ1zk5e_1yI4GdUr) zg*t3=y8Hsgm0disH}H3uCi3%eP2|+5nn<|~dv_GWvKn@78$G#iM)St*oPsSWhO96rz*JS;uuPG$b;hX%9ys&b(3UMH zb}J0Qn&8+OJIJaj^Gjy-e?Nt~(^}dJ?w)Esnwmu_EW{l4d{Teg4v-q{Ki^1DfKnFT7xXyo-{Go`T)Wm~fqxIYAq ztemt->z@F7-SxEW@XGU@aIymikQbsgCphRcFZh78Qd7}tsr4)l;TMs8(Dk+8aDvR3+A=*8_otMX;GG-S<7!R*!dR1JbHYnbxvm64f#o{@5!rAJ8EIm)NUg}<6~lN1fE+bFKjNgLI$_xg;SztUef^!Q`?9IsvyEdhq-JgqK%hKl-fo~8HC zz%(GqUZSix@GO^w^qPKD*_F(0TS8D)dC4$Y$5uwufI|SC6pdTTu7>d4zztEPlQ)Xv zSx?c@&sHS=6Rb$!{56s~RoJV3I4A%hr#^5nEg>Orc{Qc#Ds{; zBN;WLoJ8+0pJ~D-;aW0-(LC5!U=3%eOF_94zG1OQq8cbFAfSce8SZo;*>fkmuBjXH z1fhsJ0)jv>8k%8Gd{?AJw(n7odaSEcM|`+89)`Tp))FGvZa5Kwn|oGQpdKE+o*qJgt5JxJT?Ei zAI;cW?unOHUGmYFT`vrnpi52N8ltX_wk(_2(~voPar=KoSCVw^$Bseie~-YtO&@lT zXd>sEqJ;fQ%j;LNCdG?R3i`-Rl&p>hy|X*2V$zO;^T=>ML zrB7YjFz&`3UE24S9gT;reN5Ma-H7phP}?(mdUpO}{;PO$2@&|UXj@VFWy{(ZNWX^S zECLJ*;;#@OE{tE4UTbYN!{c!Eb7=hF(68pykEZ`^oxm!~3y+6nob!+#in{*Ao zUcE@Xrkyt`1^4{=9)-ojRNUEBwqq-J*g?TXZ?e6-Yp=rM2`peeO*d$ps4lvw7n%fo z>g@n<{p#=z09&JV06_LydO5jJnWGv2c{RjNN*`v~>wmO9ZBL32nV_{NUZdCO(?)!j zN`PC!P8_0BI+h=7i}|rB6&A1>?fToFLn$#f?l)hrN?|XL@YI>l#gP-G25&-q^WKRb&RvH;52% z1(c46xn)4DgAW}|XCx!~MoFhNqBfx#d@F@h0k1snXhOvhxQ6s5)G{whIdE+TCk9kI zDsN<^=Q~h{%yat&Xy6msr`@EC^B(4iPhS)lVW$$y^C88OAhmQR+#Ps2YTYQsfoi}Z zJl`!-{6dGBff*-PEQii#l<1BSn_Y!O)IRQr@dJgDEkban2f;Bs)M3gkw=} zS$QypxWPIXIW=l7Qs_t1(JvDCUyr8A|FGwXNL!dW=$f;v{p5^1&vL!+I9FOygII4H zFX+s!Nw&#KbK}_uR1@M**W--=fgU0FpZC2=?D$aUmgqCE``4(ATTQwP2PMbM^V(ft zX^3#=W{vdJ`(`t-jxBv$wY|#>m>GC+FRl|=Im?aFlNK5IRepUl{^yJ;P1}|iN`>S9YgJ8Go}QF5C$E0oAJfay zK0^QRlOO!IES(CRv_5c#aO=NuO!iHMnHQ}OT)MujdVAAB@Y^pU=Sc45?Bg7{_88@; zAHn>e+c|d=kYU7t#Uq%J>&LGbj~uXhWVdbK66F-p{J;9*k?G-njan(0@!(EEJhaw^ z!?XDe&#qeyI!azc!!2_G9J*@AR&vTGf&w3hvNlp6sb6DB7p^5dx8*yX+hf}{XKkB$ zpso5xUMhA_jJxPK5#zeGjhu+Vp|)X5Dd?<=%Vf()V6;#09y5PYQPy~f=-if8Q8qaa z|1G=N81Uf~hxpPIxZ#v>1i2$HZHOo!0L!~^s%}``M~{JQ%Z5|l8a5AWVrIiC#jmZt zQ>~F(>5yC<(!NQi;Ow?Zh3O(`yK71&s5it2Xg4=@fzNz&6G9`(!@S>W{UR%yv~Ozu#n(+!UPRmK;MeibJ(oc4c8-LkeVc}yyRgfExP!DR#IfQ zGX?yxpaJk>z{nA^=s;>zPv%jXIbY)wGr~oDe>`76gz&JTpV4KYcx-?j~LyEPz6$ZAgNxG6!b3TuTGF z=6fKRq?+|9b~zE)qR+jNHK54av&dI3m`D1jbTYInTLVbrRVXN`816s zq61*q_eUqy@MZKob1jzAyb}^eO%D4EEZ#?CyYx;eGYb0kjn?Hrim~Y|u2}iTmJlU%|}&DM?>qF8oveF$Nyfo=3n4ExL z@jcuBt6$ubUoUNH={7pFZBBaealz8uQ#0!uk1TCk7D%|b^6sWxL*gzTy|n4mA#q#K zm92S5{Fb|BH3jjzyY6n z;op5|#kM^!lnG$Av8rkP=}D{0^Xe0C$ZWfyd!Vx4_|V;b;+ngjj*fCq41ZyAX%#bR zi%%c0`E*82f4Q{uvHXnphYsj{#(?&M<-^ysenGI{kuH}IYgD~sLFm%H3~sufk=cIP zDE7{ne&pC)`=Ns*baibyw+%};vHmay4~O;R*HQ<`R`);qk6>wIfcCTh2!P-U!os)@|KH6&l%rtlt==kd5+46kA2^$TqoQ_w|{p(*`zdWi4e+zpgdY zCfApZ`lz6IN`2XtAHk&#MjvUf_t)|5DwTmehhsPFg{%=BwcEPxI*muDW~1RepCT$O zktmX?M3Iz85t|6s8Mu+b|0rUy!gf5f!XsQ593@=n;b=c)i7jym5Df7vK!PP$pa#yw z$C>@tPr5TEicqAi?sOTCTdX5V)}@3K?KHqdmR@q0=s4FH{TFZMl!*@4-a3;DQs@;P z6UYQFzOU$IrSlT+tlM{~`(|S1wvJm$9TFY2RvJ)9ga7kj1yuLf_u#wZ4(}=Ka_S&U z2K|9mUYS%B=|yrJf)EkbIyQ;gHd2gi8DlyGcapmIUPr%4%?xQYqT}rv`cMzt$&N_6 zT+rM}G0-gGdgaUajv=sYO^;)!kpp|N_Lkcr4gqax=FYz zcjkID8K%ps1(KFxfTbxf{s*$Fnpasp$GlGbHPM}WrGLIcba+A}W;Qt?A<@LT;#^wf zlaR@Kn}EV((cu|Zy|uBeFTUKlJ$$<|)7-Cqackr$+Yc1&-kJ=ll%l=TjGzW=Ab%Ry z5A|Im=mA+-1J0}QmZGn*kZ_JrKd{cdj!zmmhdsqCgDipbjR+(5-g-)uH3Ft4b%_r_ zof*E0$g#xm;w0X>-5Cp~k0(9ZKZ#ZZUB#JuHgi_Yaq~KwR%FtdB`x1>D+x{r^jrOG zUT}0l@XXBm{6l+g&hNQ6KV|jPdBMR2!4uw2FTNxg^XtOWu7_vU&dSQ3m0A4TU`+pt zn4=0ym-ou5z5VZjLly*DjvZgrQ5bq0v|M8BbCrQ{c>V48qPDzATh8g7vUpg}NyAcd z9?1*7o*#UrfBmlk($J^vp}gSV@`KNw)UUW87;{m1%sHbfgdYReAMmuTNQhRM+}0_}!5M>mRy4 zY52VUZ)}e@U#=<~)Xoy)G;L1#@e?NAl}qE-O9Z?H#+@_mG|9 zH7^HlVEB?f&9M_^~>fMJm+7%R8A zNqpPf(D5q8J*dB0w};T7?t6Bx%_*%&+#5$XXZZ1%3&g6ftEiHi$wqQ!aTeaOuxvSb zkZ?e80>5sRuMv)q(;g&OtM^JA$nSk4HZ$t2mSb(ucF8nV$=0;AE+=v!8!=KN0Eozj zB%}Fkp!TW4PY0dap~x0Et-I*zdZwv03dE%2ZzH~ z?MJHj3TU&efwPMB+8)S9*-|Hg%^$^!Q& zvTh2=c<@75IxY%IMm5{_ueQq5qsb7OA5aQkEdUma1LCEK=?6U3oLVp-LLzA=5#(>m z30zly?1hdVRc=sG$T2V$4+zE<=++}L}- z{V~|;kV9lz7Z8TaXIdjs7fe=$+d4vZj1B6jFm0XY1kPj*6}X9OP7+$LaD;83hbqT% zE>7q6_|~{~7WMj^qDQ|}48Rqp8=Z0uyo18yRC7ou9OWR%&QxA<3^vbc7}%AohwLR6IbX(T*|OX) z*eL1+Y7E@6Wmxqg6qDH>WXxI0f+2SUYAo8YqG-*2Z=C!ZE+X!vPCTH?8DmH>W4YhD ztzyEl%I11!BEL3khC^`j=zurueJ{vJ(+Pj|-7$*`g}JS(be!wnGkFiKo!9Z-yFX|l zkhW~vHNlv>D`M^_EG-8VE6>U<&n&(?7;|Yw%teom2(62Wy>~$%5$6A0ClnnrI`sFL z*t3#iPhAighkn`z-xPmXdi)mj&u*`3Iu8oTw1jKZ;>$nH7dpsg(+`{ua;>rYPaU)L zKso(qe^00XAuoW`Vt4;5=EvXMoy^{gK3!b2V?p6_^W$HdAD_Fa{@|In61Dc}?TiO} zb&$hXi*l;P!x?q$y0r9b=I%D<4(Lt5?9J=`lt@>PP_DLi+RP{z?Yc)NYYiiBJLCY# z(-Pzz??eyogs_vNQ^Fe{;T`aagq+w8i=oG43lImiGtdiY@6GF8YGiLompYP{X7x6B z;Q4KMIhOB)h8Ofde#6>%sKtujJey2h1U8aEsuMnyHFlvQXH%s{z2&S5Nb<}lb)(h1 zUqLgVXuOx@?KCvodQ92+&UwoFj2iN0}Y}y!(&f2mU z=84U$X4=4Ii@NR`B~h7=StMFWPHu=IfrQwsKkYH$c9DNqwQ6TUL{@esm9P~S7pffk z^RlF^uy`yEUG*n?82WvLYBgM@q-@*P!s6VqW$U?|t$1Yjb)kQh##g>PH`GuXzwu?b ziQn}lqL-+ak-Oz%D>b@M&bcB-POQyxGqn+8our)@FbnPNA9aaeu znAj<$_RDO0;#QywL39M{R~N$(12b&crz@yRs(Itn3q!YDK<$&t>R$U*L-iHhqK(T^ z*7&1f8M_G|eEPiM+z!$UF`H)YZ(BB2g7IRZ0|%L7*W3@voOoAuD?{U0$Vo98>B-0< zKRlSyo4>Ro5^YfUI4$=v8`SWtjwNCFL4EK}ASh>*waFQ}XEOEzJ8Ofi%c5ZOI$Wv~kpJ$G%i3MLV0W zSX2cn-l+?aMh|S{7O{4qCkq?tllV$NWWWlKuZ^mpYQC||x*0|@R^;s9wtp-*cid%P0Lvn2kbIJx)>suj9)tso+J<~p=pbS5NzAth@ERO4z-4C@$3`7%{ zVsMC5i{AB4G=z)@(lGZmB9rT$OTV+~Axb>Y(cYPYYLIg=ks_<3atn$ix#C?D9{bZtIWlddc z`!{uKvPx@TJYXbR;epdYHW&XC8pxh6wR}G-$cfee9#sEwKhh??)~CnLJ}gK4=ivIc z`&C`rx5u4*d&~p}+_aupspz&MCGm3-8R11R6?upd*ilT0)s~LQp2ZYdKEKYMJ*IZ{ zg)^3<{TuH#DSyD?d`5aBE-O8g`L@m6N1K#nJ4Uq%|4K=>yQ|5tu)Z^J?R!C&U}+;A>V~}6tb~XLP++`y1kdO z-<5_ACaNI8Mfqmx8)sA5$SsQc#&z748(k{$S$~MKJi-+b@bL~hk<36^UeBCgcL!(v zEionz@kxI<&Fw5^_CTtW!a%={&$MOvM?&Lb51x^UMrhFb_%HFn!?!>K*|7DuoNyi9 zuWdzsX8Zg7IMz>0tO_kE&pQln7J6RzjgVxtE_kr&RA&a!Ij+PT^A2c74491M8>9-W z@+g@dK44PG89DeB(lHC>=69xUY>3m-nJ9^eZq8VXmDJ0mWxX^|^x`VCA`o?0`cDIt)OJ!H z3N$-!s|+(FU^6_GyNExdieEz$HCy(y@+q4VoEmsz&TjioU!nCX4HHYstOKGr(n}6{ zc$c5So`7VCerMcPseJJ)@sWssTV9yYkFTb|j6ew;Y$%SRC*|&CQC5qUcryQS)&`rZ z(p-?lpv%HnB^i86vI6{B=RcM`fh+MN54h&S24V%KlZJ16BeJC-F>#u#lOiyuJdI}J zW&#PG`$>K!yilcf*tMZkme(8B2$LuIWR#9QN!LZq@q%^+5MFvG@*gO}>q!y7BCGeVo zWdj}`q9vqru;nt5f|Oo_j?WJBtVSLYpHvtxye4s>pMKX`A|;%W*Im)f(P!ONP9hN% zBlZ5dvbc47n2X;oOn1flX$DHKr*vp+I5zg4k5eX7*`z!DQB@(RVv#$C(BwoUzjHdULOd(e}4XxO(Tqs!8f=3=faN)d&WNhr}iPA{&(j~^ZE_%*>u*F zvyMAwz~JAme`qV&?m7|`yh?+J+p>G_&weCX#ixv=-fcM!v81Ec?Og@)rt}(7I=Qo| zY2eVLoVj`R8&TkPefK~M3f$KAiJy6Im-eSgNZ_#nmi**A=- zy@5g3uh$38DKB{XrZL$!k7=9Ry?Y2I`z)3uk9&WyBzP(Q^9W)|veAzumb`9B^h!6$ zZOQC}qQ$hYNW&|`eevCgLJGKkFZirz@Q!FLF!xN{aYD}hnzr&Vcz4QEC5F9QStIO0 zv=<0AB`=7hQ(AIk{NSHmD0wnI)Atohe!XXD-2;*7I7G!@&!G@IRKcrlL~HWqmn;t9 zl$I|h2z*AI9B*}(D=?TkN`w})3KdJ7SGzz^aG4+eY)3-I1%y`oX*deuAz59=ZD=fc zMMNdog2gpl8=ko)i9fBlMdKLOfT>$-U3Wo`=^83aE*6LWhue`{ol)Wm2Ikw|S9hiaM796ZOgO|%E=^UD0eQJcCG=2iz918eP|#I1eiKg*e2Rhc?EeuMaYR+ ziVF(}2lHFweT2fyhBWxuWzpHe>-LR$oN|Shxq?I3&l>Bl{#%&qklOabCcYjl^B_>$ z3ler(--82L-b3*a{*FJFfrok-VX(K-MWeY=YO=vIfG*g$5@MnKG7Ugu@DYi)B+wWN z;}aKfmd2fWNr3T5Cne3?h(pa>)n+^Dj@BWlUAz0P=vA>?S(=!^67WObGk{7{UFqbe z918-GXx?p=5Al6s<4CI?rb|n9?xwZ1?=?AJS+nYafSGMw-;5X8(u(UCzG?aZ;IyHQI@}PG{ zmTW$ALDLbp1Hxq#hCa(pX)ownIxsn_ZQA@m?i<0TVS$91i~HAa{9}6YVQAI%QC?i` z%T;S$`rXIx@9W=NxRCf+{J+|zy>;rcJyqS~bKg36Fd>t1Hx;%$#qHQav-+mYe7Zlm z8EzidZF!sIw(7rPa_`Qx9xKxrnRE=UKL}u@Y|caN#BGIlW@82~;l83nPkC?~>j&qI z$u4KeH|yoV9fG-=ORo{TmMaaSKTczHr$Z&F+jS(fEB^V(^IN0SaKu^BX2%2-V-)vm za8_;z$Ghdn3>yhE)*rhAPoh+!+KSAu0#8^EvYMzd@fE|BjBUgIKXfE1pB6}#m29bp z%Vul^Id{sMw%~u3oYwRttS+`j5373aB}_frs{5u#kf#=#Km6b?}RD=?gyIFl;ZX*fCrDo%C2 zn?sU{<05%OMn&Dj*^2b43O?1pOhjgV?mZd3*I5 z1;JvM?$`m51=+!e#-{a0|4zxt>qF?CDgul7zVeEN8TfYv5#& zP&R>pGrE{96rxU`aW%R{x~VrjB;t?-(PUqO4I3Hn;}yoENC2& zK@bD#Gr*8;T4A(V8k9?kYjkC!NV>JqqLRi$s5>J-HHxdlV^w_!G#MUyNr6wX#AulbFlu^`9S)qF(!6+_teZ_N?Tx18D8PZ}YJHm{sL5D_|nhGplx5 zR`#^a;?cpFS))pCfyu}%S+zG02vnkv_GjaZb`*w6V`ArYkG*+*VB^5?MIRT2ieW7> z7Ebwvp+B<8$f@|vfbm6J@sqdtNAAY9b||`IACg%=_23c3i_b#K?qyBq-+pyJW$mB1 z0?oDeR0eW41)D|#oGs~JUwK1%abmDE?UTaLsy=b4=%YRG^rVKwKW5fmf8-lqsJg4y z2G_sA=<4{sJ;wA+dF9=~^_vGb4;(sV4dJG?`?^sW-o2$g3#UUA6fs! z?}q#r%<#E1?h&DyY+qcok6(XN8Gi`%T06PV$6QnuWb}31%Yl;^nq-&p>vLWXT=eKW z+rHgf{)_x5a>=@q^By1}iI((*@oA5b`=;@tgQSpzI53`E3R6KyLVx;rWosjH*pQvM zSu$@Iik77{SHUt0dNB^bE z4TzIgW*86hH<&#qAj9hqDMbwfSbB9UvYllR>8 z;nK8Jl24{IKc=|3i%CA2()^-)jUYQpp&RgX<-i4@Th(_;%n~`Pxs$ex7pw7+z|R}$ zXjN!|I%}$yQ$C18urn!{Ha=K(1eLE97N4V5(<^m%whN|^oyikL?uOdWc^hiuq+!we zX-}1ribwJ9sN5F$?;p~`2GY`~sYo&r;i>D&wsEe4cM*_l_V%tikqAJR%lv`N9qM=t zyJAeTkb%68={w`hyMz|39z-lha?UexXg2b^b%VZQRhfy$+zHI~*O$?y_Epu5O|gJC zrl>v_Qc9AIH7+{y+s}FmWu#>*Php$A8PDE}tNB#~36i++h$?*{*_BEoJQ>!m8os?` z@&}*5voNd%%#1Cz`_`jxG3$nV@Hqgfd*wp(>u9cgy0kYTyNJbakaS{}a>VbT*$gNE z+{2d_SkE%(DT7N^)?n6K&Hc<(aTO7Idv2JD#yJOks}h$w{!mr1*p69gOwjJuYz2o@NJQ&2S} zK(QGo4U*}QWU#_B^!mYp^S?+_Sl&Q{Y#`7v@B-etvJQHDxXCl%oXm4O=VXA`Dr`y7 zAu@)IORR^LI@ueeTZ7PX^Pm#){~t+ ztn*#pfvaE!@(v?C7vJYtOjIixpQl+!7!i4soxX$KHDe;iC|Y$^-ksIN+eI^aP&Bd* zgiUPB??YaZK(4Z`N{<%4oUFUTGmrW>WX7=DS)O!p2zHWq)+364`-NX5*+Uw`BSmme z&W|A%>`ZAV>C^VvA)8F(ag~LWmdVW(%`$JMSCK-^Dd7-4>SQwkeblIUOq~fMH`wDI z)^$-54;QH_qq?ZHj!0XDT==Bf) z&iW*q*~HpznRp-_7(Kw=(Ffq97H*a;QLj7LTle~5aEXxjOy5|^819oNViU$>Bpja+ zpL*faHK{dg`Ydd^2Nom!60Ti#NPUk(drlgel2e-(>{}50MP~hHuznoTb8}uw&J%gT z*td_)zCE+}{9w%PQKjei%F3RZRXZcI_&L!s$F==z zOQHyL(1lWM{=e9kB#`v&?!y)6-KZP1Wr#!Aqc&ELN@jxLa24$@qSqX&^}!*rMM3tM z7DV{E&a}@q^=_YUqlg-6N#3K*Rqy6*%SLbY$txPhovl`vW;&F-WGr^4BWZP^H10S7 z%rcq}ra%%u$^Ui`ZCAENE_u64X3_^8%b4yFh3PiZYmTU%Zg>=*H*{Mcw{c`|Z_yj< z_HQV@Os%}PXwu?p3aypXmWfSW%VHHTlQ_h_VHi=L(khWU(*_25!8Xwp4rTk*Xqe2q zih!Mgp1cNKlckQ>StHf~z*Cw}m0PxR7tf@48tHf@lOh&zXN%3(e?#l|BuSKUw7=6Y zrDG=(YN7J-!nO_h>11cn_CdZF1pK--CHu6q!MFAlT#2_gcOX&$7KPGv zugs%;egwCqD)Dd222X!-)sGP5f*jNKo-TeuPU4EGJr~ry!4ur2tl}`h!4{nfBDtjb zQi)r{Z_OWwpurWOy0IDi3L-S1Q|{Xl76zzims0KflUFC^+nrZ;)>ZMkfwb9Md>1U? z%XcE$s>$3FD~4|%3fIR-$QNPPY)M=>#?w2#fO*kL%%}*B;B|OTfU|`Pp-HuuAI@m- z^D=NwUZ_9SimouXP-e@VI+TpE(oq&qs~A&9tYG2l{8k3JSb!Y^5iJr5Db)VLBd7$x zg4IW(6^58lSK8~l?HwG0FKaH-MTw+kuHX=Ob-!_yD8bkT+OiJG6?<>#(lPWv`jR|8 z_oBymB#;Sv6+Ioh9kR-3p-qxTGq+hAW_7U7R)nO^(>dGG&(ZbAVa_D?;z7jB@|-4a zv)s^L#XE)T4QP3InDW^8VS6zwW4}2w-PcISc@-BlQ$faihtucf>v?}~UcWkPW{0Pu z$^z5u=-2FFfN36_s^?oqvM`0jDe_?r-?jvWH(MExcvoff8p_M?HKC9YDxw7ePkntk znOwtyE6htMBC_HyEWG@{OPJ=<$42&&3+MR7zQ&J8?e#yd zb!73RQ45;hyFIh*g!DM$?|;lQjo+_kZN(6%K zjCpl|wZhzg26Jay*9Si1*R>^MvTw(sW$%po_Sf0LF}0Dows%o;u`OfxqtZ#@*zo1JTx9qfY3ndSyCLCcWq1NjmS@zlZ}5tv5$Ks(!!rd z!D(XRD;MiIex0fW-Aq38+6=ZxX4=x16=*tJ}mb1 zX9JbBK5~0DPJP*6ghf8G&o6bgA7L(o%5d)tXO0;0bc-ll_J?WCG`A&1jYB7+!|GIP zAcYAgJJ;YGlPx!Up{5w<5O}G{;Zd4H1I3(LBNbHbhZBjtsw$HzU3eaw;}4p|2CA|-HSJS&RusL4-w=d7D2XcSee=WyNR^#vZxel zM*$&RLk8@o;Ty0TB5N5u(f;lx9tsg-j|AasjhoFKp$-;q;|5LoaA?61uhm#5z>-d0 zT}2`~V$@JFUD0qpIrHl{I45$QU>)UnlPHJnxTvC(EMd2VXzZvjty%QfAl&Js&_u|U zLGJkO5#u0UscgOzL6G^-$-jo zL7C+&E_-xe#f+Wv{MqOahX#?D;wE$9<jPm|(=!ImDKU*f3#a z1ZPgUPUs3qJC%u$7O+xKM&6Z0zt&MTM=)+;Lp<<;xpK`h@ul%OROs-9^1f&9aCr+NW<8gxJr(s&&a}Aq_g8MH(Fh~%)22I)iR?ipX9tX zh3vyC@;$KF!%62I8QE3~Y4MV&zk?#hIxXc$A)h_{;04O(D29Zc;l+K|k@DjY$3!(V zt9$R>D1Gl>C%KR8B(o1(PqxJet|!~bAK6Ya@iS;2k6aL_iw!hSIdTMb>z;h@y^>7$g9OH9j1TRGx5$HC>Zik7BpZFUKh6vFMxX6hf=M$M^bg+k z)^#^emM(>UW|FhIw0{YVgYvbf$iKtCeC>1L-4UsLjeJTXm# zS1CLJXu^lCdm0FdV^acV{0J*h6&%8v$c9}V#p)WqSWTFSM8ay2f~s&#zSD%W*O^{$ zrRTkkP<3CTFA(DmhO~>rQ(hLPTDYJja&SS7??Th_@VxEq$fRzKxB*Yb2yLeov(cH) z&@)H5cz1s{{hB^k>pQ7sUkyTM?M*qYNJ3pswy%qbzFM$*GdIiPSx-Dfo8>c>^%X;-E52 z#L@b1<>lmScK&BxE4()w%fIS|$xX^IP9DXnBPU6$Rrs)I4C_SL@33K*_dW^hOAiS| zW6qff$x7f^dXmbd+*FbCETe#A|Gx+dND#cOou^-5(3%AXz%4}Z%|6m7g3?Li93A6X zpvF8{5(zH@XGkH#6cGta5?y2AjQMKbu2Z9o)WBqqhEd)i=P*a1gMc@Y4R_&VM>Umt zQUt8C#llehaECTM`{Zns>#egx8!JO)cH1)1Vg9d(algVW>8S6Ckey?QiwO8^xp_R@ zsW0OoyIxw2D;7ihLyci&x|s|RK}$vK+=!1l)vES$LJT3x)^`g)>Yi=}fj1v4Lal zCJ9Jof2eiN;Tgxupn6kz<|ehMCaG5h_7d-dupA@p9BkKw7241wedqPg$z)pN1@rc@rA9k;E$>sfqx_(WsdSF{a}EabKdHFk27bX z5FY{wMX<^JR)WFU-N>= z1;K8a^=~EgoRs(5m@b8-xAe-&F3BuDFBr46Jm!N@r32Ah`?jp?+pcatbi%#?<$)^_ zV<%PyDvudow0nHfmxZB)F|mhK23kO4I%mv*$KmL*Kb1(aBana7jp&ozUb5J$;@lzLVVyFC#A;Ir3xf8({axE?t z*%-2#n5iRAtnb5qiic(NI9~WE4&c|IfOaUhiDCm6Cnm64+`zqt^&0!ia*%C2L=0@v7hl&tS2g zxH~Q*^_yhcdvbK^?Bd>|(Fg#KIhxtr{U$^sQ7hIe2Wq$fQxqyMr$YtBLqSMhg^ZZB zcXr^z_SMTU5(tJ)M$Ty83kM6bv0S%g`!)WYm*{PD=q!l00+>sYs@6!ERXlutS?kt< z;@{9;mksNG+|zYmWq8Fu(QO)^ z9g70Td3|XukeO{0P0@MG>}4%2CeXCU=D?Xy><6_58e z@oVmEi5fdShb(}BK5_47#m?uRRW!8)N}e$7&CWQ*q3HIM9Vq3 zziFzIW|7Enl%5BWh2TvZ33NaTWwiN&>%{s1@$AvM4&)jEaWMCaR(#gOe5@;fV6ef|9E zZKbHW63N;*G8JLaz3{>W#%=Q zCE>FT(UKI6xGv)W(@q|pdQ(RfjZ8Wdr#vr6Xg#CpJq8<|&pk5k@l`X~?@4w}TgQGT z9$@bX31MfHX=|05s5OM$`!|w%|H$GWyNmXcbC<69a@m?tuy*zG?9A-xS+%!j7LN(W zJWw8U_o&iE@ER%3EFKk%xuHDfnkz?!Dq>=v>mK{a{Jx{~|KX=Fbgl3n>0b8v>kQpcbrDD$Fg|odd3y0LgQaz+W!6_8Roe7KAfaRP zf!0i_4E5+6x1?Zc)3(!7+J|&4ZB5E*n*x)_4Z)_t@F7{)rC)pcsFGUuH3U7~MVkza?lwG%*RMQhO3l1q4{+kHjd zs|XSvQWbx=fG|DyH3W>08UZ73(Ij}}DXECH9e2;@B*mr`!WHbsRrgB<`OcN?DqZe= z3r)o#sodDP_h#afm`_=2&ZR6?p8H_0LmF5(AM9@+tuvs)OZ@psgOn&H^$ zx^j4bgy_izF(eyw4njZj5Av96Yr%gH9YOuSh3t$M9zR;``Zoz+T0f0~!e5myPbwHQ zRTGDO#8-m}izLrrLNaU~=ebwuD!)4wxxSSS4cWdiia}%?t0C9s$I~*JbC3G#Yxwo* zMJio1GrB!w^DcB_2rpXOpo-SW%l;QE%1QkX^~FyRkwiwyl5=o~JV>IeAr~$0HOAdW z`x}onWH-=Hk)0t)-f4FZFdG{M=GX%5e!KJ=he4m-rrdGZn*AuM65ZgmX2bv$zUIVD z>!>Y_tVy?Is&PnokT|zFvumZlBn$j~#M~=-C#x)lA1;I#?PtHc zXK60@Ew&5`T?uRd7uedrVe%D6H;-Gzjrv%nBR1>+Rzp8-0nQhoxWt&cZh%NAkYT|T zJlWPHy*VG^KVqPgBqWiUw*UIfz--H;*@D#nd9Msb)JXO)U>ok z`d9HWk!?a|YMYkI;a|v1DeK^CI)b`Fnf3OKoFOwM<~jl?go{KSR@@qy821KRSVM70 z6-s&m26oCk@+%-QE974C8m^SOxBw2Jae`PFyrZsP^2z{P>ZSSeqf3?40pgsj?_ap(isu0Pw_A33kX8qCT8VVw!W zNQely8H1{=Mn_I?r}9i*7-_&AG6@=CIl!Gr(BSJL=@tbvqrV%@_FW)o4G6yoHGITv z((g!KCS0t{sZjS?RI%=oUibM+s*O9cAfqrsV(cu-v>~_^!t>%7C@AfWzMlAsj51rF zej{F@i+Vvpsr`g3-v@`lzD3iM+-M3O!xjLU8S3LlJ?o`OTtq3aXf5#|Yg%{JhHaH_ zU=_H|r`fw1I;|KbWc{u-{M9p&_O){``KP{X62- zSk2gO_q#E%1CnD8uL>ldFuo{pLQ(h8p;u#KUrLI7Zv3pKxv>ey_lr-Rv~*4Pnl*KFZgkOu(`{!>Eosp z7JW0m=4KEaLsC{hn8(ap|N2$&JvR@*Ptu&zdSumJpVjt2ep1fjy!y#~ zN}K)$K(_^e?hkRL&!=RyU7DZL{zSjH)L$)a`g%xwYUQk^djbhd=Fe)nE0B;n|L&%L z4~a`HT-x*wes|Zb(3HLaaa|TRJ%4-pZN^S=%1miRa@&4tCTkVALnpI5vh81d`hd;C z|Nk$SmOhrB@&3>Oz0Vkc?g3rfh|)rl+a(c-+>RW(3-lA~@;=R{%k|D3Fe#T`@4K}0 zU@B1SHtJil#J@J38v!JoSbvyc+4}M8o+s8He)W)zT&5M#@;@Xnq#IC~9%{{(32H1w zux*^$kyKd>i$2dQ+q&-?bq$wyq~beLMy~xuiM2}3Fn***31{F8_s5q1k}Q;j55T8X zb4ZaQYR;+|W$`#zXu6q+6vd0rO16>n2~yi^BZp$QW$H}}BWb{S_mT(?d|+-=L|3^y zDR14@6O;kSl6;k~JX0J(RGe11e^w-GhsTiluc>ewu1){1+A?}OAzmRU@$0>fSbkH^ zeG2fcAizgUM9za8^u863F9$<^q`3+jKBE`(WpkFwAhl+?Qb>Y4+uQOFF}A+Ijy??q z7w7U8b+(FImz;;!jK?pZ4V#fc&EF15*;3h~`FcKK$$~C}S3gEtplTZJ3N z+q&1bKW0URMZC9Ii>OTSv>){G z#Blmp($RXdBj4U=dcJyKcgDTM!MWPix%2HBwgI5u!Zd_WduKfV0DqFy^+nCXr_-57 zcOYF}L2wZeO=|gfn>z^gDW)zan}Eh9(9*;N4q@tAXC0=lYT_8ZW23BOR0N$nlI8g- z855>szNqn$Knz3onUkeLoSJR0|;I3nT!J6c^<)fN*hS3w;{^ ztKznDE9n%u_l8eQ!P_@L0?Ep2o`Q~1Hm{}2zI(Bi_Fa6Io9}fl1Ns}oOMv!XCB-p2 zn2sv2rbxRV1QSkhMjWxY<`A*5hF`=?Z&kHnNskfkW*d^{xpVe62MnL}d+~>JZ^y?& z8F}$oAhl&AmV94|CF6c%8F_XU25|@QQ2s!s^fnbWj~z%fXZq7dX28*rW==xif!JQPy|} zfAE=U6=jp-=mYy{E{z8BCOp<*NKt+iQO-t>^)Z3uW`CSURwWOy7BuG$g`hSl4lZXt z4#9uxw#B28c1g#`^4a7^LQ6v0|6U`K!@r}ZBxBF+jI;GfvPIj#e~5 zA~djTg#Y!e&~crXbdue4XEb%>|?waP9Ovd%N?CN%sNAK02E(&GzMNV-p%4Rym|9rR-AC;8^R zFyfs0g3`pyFr5>ShnSr9XD`W*?^M5fLnrCOZO|=CJHJg)FVJAJmAZ##%+O?}UtY^h zFIbIeiMt*rH&R1?9nWfzOht^2Y5pgLlPx!xFB+lZ>YtRg5yL8qrtSW_B<*o@v+zF2 zV)dTAKPJ7=^d$9~o^k*uGhmpLz)m7jWqHoi_ps$q4aldG*2b?y^B81y|13-AiJa6a zE;99iAk#UN^7s16B*+xjCzQGP@c*#)F3?p~*ZR1I6G;M+2f?C3UPM6Dg7pDnAOV7e ziaf50s8?ODgd4ZX0FIc*bc1oC) zl3Bq@N%AUO@#*}nL?3x8B_%`Uqm--wLlrt!AcM%HVYhSj+c?U!Yp2RoM=p=LY$KQDzj`72Fi^<5pSFipRg-4 zQuCEJR>PBD_{Ts_t+sPk)|gGwHZogv(!Ny{ww;W^HV9Yx23EFJ2UfNf8Y|oW9j+rY z+~)q2uJ)qSl52%l3GM@Ox3`C3|5}wu--Ah$iP{T8u8so1=#U3^o0gVtDnD!AvmvJR zEVn8lp(lI$Bc7IJFCHMxDyDMBbMkdn;~CO@JHDJvnB&jmtwIVxWXXx)YvGV6(TL9; zD*S|81$JcmpfKuNxHokdnQ`$FgN@iSoFi%Ht<2%bguA+t`ZXyIlKqa$P)|f@2qY>?TR&cVt`I1yF+?aKi>pM` zU-{1AXb|ZWOyIVtIz)LZ?H8j%Lgk4&pdw!B=BdxO+F2N&xk7HpbDOlk#m|sy1^%ht zbLr&oVT`qyk&;A^M+kf{2A$s*5G(Y6_?9CvQw1CqS7EI51MCGqLM-yM538KPYM6~` zvTzCDtZW~^8J`px)UjN&T!q;+q}_LA|# zBu4Ha4N0yx-YDF=8nGnUvt5n_*{a_bPd);+AJWL?pbSZbUyuDh+h)4UOT;jHFuEYp z*i_d6cP+)DsD6!7uN&X2&b3@x2@=Y^bg0)bB*8`8V&FV#Vk1%-Tq@yFZo&JONJRc{ z*$DTU3*d-_6O7jmH~o#~d7bS;huxXeZw?fY8Pm$rC&pe+NZgZ_`0?V<+7qXi^_*7L zPK>1`0fGGm!$+Uqu@cIiZuC=yr11~NlV+26JtB> z$SFS|94S0KKiYguq-9koWoGR?Ejvdf&78Qh<*N5TxC+6c~gQEems z2ln^uXX4Xco3)NUj{mI%;FNp~|401Yvd|YK1Vb{OJyxOFdd_O>?^O2Wd<#ZW56_)t?;ySaBs`6z3n?=B5Tp#QQe+#pVD?ph$z zY9-c*>y$IeWE{GZ%tmNkoW{d9Ts6<0Zk@lh54(Hp*k0nOax32Ds&Yezx5fuO%iPMZ z!9p14xTm6syh}*ziGM)-!#Otc$cYiG;7qxQ<25$lgqZ!gqNYIEbA zZN_*INlMA6_J2oc1c(P^lwmhQ%)fA%g#xk02iZmnv6vr#<0_5H8)~Ww&*YES(TYB8 zUxRrjLE#?m*XC0~p}wG1LZQN7JtFduN^_G{o^gbR{a|nJ);Z5`nmg@jQoOFX{J7S( zXSimq^8S0P3SS+fvbHEIBb4nGW&veGF*(O)eCv?krhZ((4XuU4 zEh!DAA|eGi3{eD&$QBF8cj)G79@+*=HKsD|^$+}6)K#hffcSbBzmbN+Dt?gR?s!gS zaS-Xssw<>ckHz7iS5Pmyg+w?@-Lq5v1rtrpG^+s15xxaV7I&-fkGjy31H^znC zZX|pVDRsJtvF939OdZ_90K67l{bRXQ>PI6?&$1#g*=HokGt&G-t8qf}HqC-+KYKN3 zbnhft4%Gf|O#?ow?O3^`9h=|%auxd$=>!2|Qfhr5Di(LHQ z)jY-rLCut1DycsNY0tPJcC$)#*&liG6cQ5@79pZ=O^tvY5rt*JjbY@)Fq-B+h7}x~ ztKx0p2z~?2x3eDkgIlMi zpet8~f=ihtt~(fx&9V&AGtAAu+p=Owo9LFujgVAs`d3Y0-4{!}`j!oM&>Vh>e9&5mo`ng{@VFunIAi) ze;O4;b(~jSmHl}QJ4bHKZ5?zbL6XS(Z(e&RD5)F*aFc`{vi;=fzFd0!MRxC=Bze4Y z#9zr2zYL}sBobVle1*_DCh_Y>7bibI^@J_EzU;j4rNw?q$a||v#-VVME>ly7rrrEZ zBUz83<-;8tG2ZC^*Iir;h>}6e2p-GU-)>DWbH0(8k!(*8LT7es~N2$mv%p3 ztq;yOQrDgaG)JZ|o@5*m71_(jLN>C%;xDhW#E$E0R4^9k9T|w#?qjU>av)awIuNTZ zHDa~ms^Joh!R5X+(2S(Idmmq%Mi=>cVsA&q+S4z(D%N0o@@1V>bFuA0EAV@qgX}C? z=~yVnS$oQLh2FMdYh+nNA;5 zFwEVvtzFnl;E$3vqwh9N@S2;xyn|c@>QQab{C<)T??apZ6-Xg>JvzkCO91}<#QXf> zE?m`VNs{03Q7&u2I=;*m+Z0aRo9#s_2*3}B`Zj|L27DHIW=(l!D5#Lps&w~gWy3xy z#{X8-5`uyW={hO;4lnx#Pm=8J$0VlBxk>OW{58CVP)?2Io1kE&CWzm1Sxss{jD-V+Nu`IB*U+;HcOo;aAIXC4-u(k5&N&0~gr$SH>uU8&rO0+K`S zEi|q|ib#WJNh2k zS5)VeVs2uQoIw<;Y`zD% z@fVT&vciGDeM9|+mQvQj#>D-#ZJJ1-0j-J80LXeF5 zbh6Y}PqKJ&t!T)awa`SmM4|k2GU1;0^(>0oU{Vrf+aewFE`t)D~*CvG_upUwzN{B$rX?Y2&`5Fq?c<0B8J_2mp69yxGOK|)r)&%E-TKQ zQJhimTv2#barlh<=m|YC`;N=pI`+uOprZyiROOeKg%i%KNjRk>@=l+@!y$LSJrf+>ZhqIk@45!NYFIFCP<57+0Ne=7h)&*oOQnzq|m# z!r2qaE%H~wEpkz4%RpfuS=4viBr0cXnbT`btSlk1a&A&&2mD0l!B6DBbCb5gPox&a z+By(xgTs+`;1XHTD{=u`B1iU$49AZVlOs35K{6*Cxd4AVwO3R14PeWTEQvHHL}Ipf zY~NbGk+b^1dZeqaWzfj9f`vuVEuhfm_6oIuLfeSWZ1;7GUJ8k1e)q`gG&r6d^V(Oz zU~L5{180w9Dysc)m_@$LW|0*Q=2NPsBx!s^z{Mkz5|Hkz^D&PyZTF!tM8IIwRK z4!wTuZ@^&<^Q&6BW7n;8wX6Rex=u(WYbG_+Fe~&RzearUm_^x%yZ|FF7ndd!j0#a{ zmMm@mTrzlUEr$_)kQ%a?zOzmJ7_Ww0$9RyR2MGz`^GCmxh(_38j3Djw5IJ({_A}(Y za1qMh7@<>Db2o~~b;TO#CcP@wb`mnv6>0*R>DS=xNWLkf3g_F=+cUww{rFH?TlU9m z?~!4`4%J0FT|^FeS{p0pJt!=$mXEh^#O0P5N!A1$qHy}b!_j}uk5YT!rVL0kE1O5Q z(z}&Zt)^DmJeVi(5bl+)kto}LR&Q-S^newUZ4_>kza>aPCq3V9!WG5w)LQZNRwr$6 zU0r`#_!rUnF7G%)RK)+p5EEiZl9l*MJWpCCw2Ux{^&Sga#h|)qVhWI zJ~SoKRE9 zt7_0r!m-Z=^Fyqrn~tr-AyS~F{^RR+ju*72`Md^t(x18vl2Byg`pNvcW!kBNW?Fz| z|5redS9*qjwCx*cG@bkzk;Zk52_S#MNqb9==7t@4cUWC_` zOXUAh@u7aLM40g)k%Mn(En-UXh=>YnrzHl}s8$V0!p_d66~6^u;E-F?92&459Rl4Z zbYt#Ypk5nS3hD1BK!)%Zk)kkr7^qS%L@WP9tq4NmWX_6(H@^TfIPA>fF;KJ zc%g4Su=9R_rZ(RT9jly!&ReKK*Y_a}A+FSETA3gHsHpQTBYXoyrzIS-1~iydYP+7v zGz!3Zdkpzpq?hMUi!mtCCA3!y86ra)nSRjbBst z+sVdJ@S+u&+@}o~n5jsaEp%6|QJ=QmCIn`Ya%ifJ{J4~xQlB=m>5M1au4ktc{4HeD zK?-=t2$=C3kR=H@Wr8QZI{?d?zJ@L`=BtZr4b(-B4$wtD&s^Jkfn3|3K(1|_nQME* z#0EP`p_aZ3hD~O`_ z{ZLoGAkux8Gna;U-~fBZc!z7h+MXRdHcpvMhJbN7MNHnhnogc`VA8k;9)}_B2w23k zv(Oh7vQ!n^YKSP$M^PKT_Om3?H&nGTp0ldj5a%mr7Q=VLAq1g}ianxOtRe6ikcn)j>k^s(*?Bi%BQnQB(NRlw*5IaqMZDqGBzgh;n)zHpU5Gx z;Vvp54K6R$E1>28j}i_NuCzK0Yh(cxgM{w@iuG72fIx?&3e*t;#xIOf&-ur8x*<4+btEc&if0{)4@zM6b6$fq&Y)dQlTgA#KcLqY+3bHBFl{dZ zaiykpI#JC|=Ajsqfru+_ev;1zJTf40amXrT1Bf1D7KI6&7^b#4$%WAslM}0kpDAy6 z20K~~HKv3V@(`>P>NV75^%6yI6FFZnTZS^?g~Y!zmk4VT{pDO}PE0!pCl&ssK{SkO zym^Vei8H0!K=%oR>{#)|a={mMR7`5Jkk-2rE5TwFOy-!IICIsdRB(DGAf@*$R@)lRUnvzPa#avXdT(hK=c#gk3>i2M@a7+^pT``)F8l)cN%| zWoZ1gmYpx;_|rStmPx%ns>Hw$EC~Ud-M!rpOt4D4J*eoW<5Tz5#xExJYVfdygBxn{%dZS4%&SSbt|W2+4&5;rwicl&sB=r67~7DL zcw<`P6{veVVQN`uT3NS=vF8#JkD9u4!(mHWE=f#zB|CZM`71ZHuiCJuu4N+X-m+7A zW+!c&mlDLrbqiwSerjgpmi>F!xNp8M8)y0RdqKHt=M#xWIPQ7SIX-Nhq>X&oIHI@$ z*f{W6{%o9uXuR1t?Y1w&4RBb@VX*cStIsw_IJpSV3l5(8&`#kz;?`cO*Th9zmsPyq z6ASlnk2r>d5JXBjwIz^L(nZaY;gseI8We29xSRp%Ritt`l-5kvn_*me@+x7+#ldhq z+s3iWV>sn)lDbkkaA-$|_%($Tji*h?t@&h|0;p$LT-(fILBAnhcxqlIlV-Ei ztr+)cE8n)64#{YG^cMpbV_9!+SX?ByrVlojB(T6by(Tt!>{<>yIM6*$?59|pkK^TG zrrKXy|DxXi9E<*Ir93bo`$Y}|A3O|1<-pMo3L4G@ReSzoXf)6UtwA|AtGXW~jnldO$% z=Dd>;YxK8tO(M(UFZsUwahorDDs|tS%j>c)Po$i-Uws?ka^v2S0geMCPi@X`6+R)J z^%<{V#d*O9sTDrC5~z9`a~$3goLy1sjQ1(_5Sg--%@mqjaj*_pRSXNi8P)Sb$#e7= z;?OK%b7z-Pv9#27Q45tj14$jbl=V5uGskXWx+@vDV*$~MG#hTao=Id@lKSc<53c_m zRbt9@-dRtlAji!@o@tDTm~$goZyypBwNZ@~!dVQ-9ubgwtq;^|+vr@NM_-6Q-GE(N zrlPr;t%yozUhFFc%GW?r`KBPyC0oh*imA_Bd<=b!P=>`%X+2{t&l2wm%g^=v8uJ4! z_AnG6c5rITdv&l+dqdPr9WSVv91=-tlIwXd7Ad?&O6p;BIdk(_Xt?tk=BNzS zgH9ikH;h!I9tErk4=atJ8ZO)1F8@U?*gH|TrP#PSRyNvlbMQr}r6)=hd^m3svw5_# zW*cSH88#0Ru^i+Ot!id(j(&*HW zSzf|@C|P_dpVRnxDIS5=V1FVgE8~}g40?zu9OK1P*Dae^L@atp4G0?|HND)^*S;J` zOCk5BFI&jjmR0*sC1MWI+{T+J?tF2_j#4;CwjMGE$;TJ$pZ)HrLxt0>{P*Cr&hLfO zF1qc;%C>I1uKY0t?OrX}R*&A$c)_crLIGH#6TR!)EKQ@24GEcU_)%`xE+-PphjGj6 z_Zh5Agw&6xO}u<@(0*^}q@cq_Y*T>G!LnI!VQ4PE8fhBi3+3!!O<+PMHYBb=eTS{S z4K^e&=MRuh4h^VoqqtmmWn;AL(Unb@jIe6e2F28CbY)|xEHXV#V8}p`)Y9wFXZw+L zN-x>V+E0EYAsO=2wRw!CPE><60I&JkA3=wD?fW*sK5{PXBheu)&^@x&-#rpJ^TMnt zIj!GV5I1Wof&!$m5>Bq%CVxe~yKD5um;A+&a4e`SZQqU<_eW$wQc`}dCDEC!EJ-fR zrQo|fAHnzf>_JW284ecYcCWv=G+H&`!{YK;(W==WmNeW6Z-d6RC11ZYQNrOft$b&} zA9($}l!906__uw1pHXd=Zgt%T9&Wq(uk1hO>qpWy?tN%fb!ZxoQjlh^Y@WYE2vnWh zq>jw+^~f9$+@Wg__#Qu?ol$Q81It z5MvUzXd`wgb^-9(vjV4OXXUjHO%-Mdb5fP&QI;VHG#ro5@Njw63(178-Qk<;ybVDH z0PFWrnu*WIZ-W{)c2n^3B&py3-WqiL!yv;KCcu>B0lm4~f-Mz>uqlZ!&z9kU?PhX# zV7(6;6)^Ab5t|_jHB{s^E ziaEbi3B{lIM>^5H)o7a-9pcf^8PD%?l!CHMG4yCD+L8_(Tm!td)S=x6ve5_j0$O6_ zrLV1dFxL#v5pz8NS^#Y<5d9tE`3}7_SSB6z(M{*aS)}_JwBdR5jeadZe)mJIF&->O z6i;M?%s}q!Tw=ws#)%95jY4&wk!8$1MsIZmI&*!VY5B6Zr+m4+-NAF@w-!=0f+BD~Pd%+$8#WgN}*D0B@jH;_4AX5eX$Hz>DNLir(;t0O;k)(_*QP~=8EU-t(`>uoOW5{45oVe77v_J%#h<8i~3omJFD?zi)Xl| z?DylCjaWr;4 zK0P`p6^>e7Way>+vzuivZGgMd<BCwN zj7ZPAfc|f5@)>FG?N1q4kTah%bQBskVB6lGYRA(>R7&gCq*M+4Y{T$ zAQ+7c3|Qg~uA`%APNbMcG#R-ECf24nM?golVbXe1L7diME5<>OAQ6^c40FCM^ow5_ zrcK;*;8|5?3zTNpf*IsqI7Fw2NZ5@c)cN(7uv{!}F2k=p;oE2D=V<(Hy!rJ?7KiA; zbPq00VU3oj(i$Oe!6${NR;_1J(ZCVl5Kbu-gW$a|;PWbCQ?Z(wXoF?W0{Twgk35f| zkSu$0a17SWRaeqo5RyHBH42@Li?3KWp}15?+K@$~L-}zRk!V?uHAISriK}&*K2?}D z@jvoAn4a3QNY6Gg)|8p{XN1R!I%@wt4G7ma?f3&7NrYmA?MA#s>A<=0V;|6FZ$tn#^Moah7mFDTGf^B8>%k`WczCOLzmd~DI7gCop2Re*GtfY;m{-x! znR3W2NfvMTV%3Jv>RL`tOt~g2Wl~o1%>P)qp{1^+KZKM0Q`Y?U@aO=Tm5j_>HZo)V z%A)Y9apB(%i0(?xoN-15+Czo!KIOQEx%uU1z?`Hy;n)e0*)S`ai+?#)!}&MlgVPF@ z6-7(?MOywll+w9vaj5c_Ns)DMC%F{vBv0igWx<`~n-R%bwYRt28%kNe==PSoLn&E{ z?rC{{L{b*4{@=pi?!G-Xt3ND9x-Ds0H$UgrWoIoWx04IzUp4F2t>YKB{0PTp;O9G) z`Rxl~5t%nNwhKNa4|R*qgAd6txc+}s6zdDu|G$J=jz_!ihwgZGzvV&i4b(tovAsVr zN#6GU7Fp%Ps}Wh%;A`&>UlU)PHsbGKqJ)5v(>FjDPBfK1d5`ejXZ%75-h841MZ874 z6e_pZgJI(OGRh1pt)C6?!QOukUs-`NH%Ozbm~~n8PN6(qLdW+076vA_A%g@GHX($8 zZ5WEtsLU9GqN66-D{S`ugp@t({ku?iM^OTeu&4@hz9vh3elNq@5fWzU%+Ab&n$BI` zb!_CURp^h;2)!b-@@vfYhG@(tl_CZA;=Ki6A!L@feyMnkNO|5a|57$(;@#W&&P9R~ zD=FT4hrplI!FJoIhk3yqC)JVp>wkyWj47_8L(^oEZ9GA^jBV#x`0YXq(#@P+LweRN zo9kdfl2m&F!LV3Wzna*eBFVLXUKneNByah1U0EM|p02{Db}J8u-&z+~ei38Dr%4{B zVrlDJTWUNF)2MS3MZHXyIkS}Fkjg>F?^zQ1F@eSD0_VP^<2tJv%ZU}W2#EEI*q`H5 zjyel@j1X)ZgV%+m;fxN!DMPK~ut%uSlbuf7!I^%|?b+Ba<^LxRBnM;6Ns*0;YVAQA zZ=VP2KmyEUu9Z@`U=V3mLfb^7rPs8hOH@#;ARPQ~3eFUdj}Fmn;jj`m^AvH+8tyG8 zQ`!}$dI1TMs>S-y)iq$_K7TS8Qq8We*+=40QZcc!po$+JED>GV|M55FpycE|O9V|Y zo!;=RBRfV;lj~ASY8H&3K!R0~KUeypOC1VDv`g5$W zR&REVyW9b=qN$8aZwSVJvWuMK?|*bkrmY?pZd${VwyuHYqXoB}$TMlvfsLGS5XK^H zL$lF8G9C@+gc(btuioJ!jm8K<2-!LB@(5Y|>c!n&H^m_dUFy9N+F;9ed$1$&?IrKc z%AvRHd`B7!Ir1k;<`2I2#c_Y;estVg&a_UF*U`)=A6cy*)tuy~+Dc>a=iYU{a7beu zs7F-YIFNO1zq^H9jO+j4v&hJ}%wKqKv>GO1Pb(y^FA z7#Wl%1AMdI({?XT)}W7S2$8KZahO)X@ZHj6#%c@Z*sybRI^6l3D9$Z7O$Lsw@c?mb z&mTID?bA^U4i$9kH-~*CW5cU+%7=v`jUz(Q>yk4&Ud&DI{AXHd&T&&?oA1bPzZk&o zH$~CU=VlaSb&sBqoLSJnd-PcR=r<_(!rZii#YNF4@wa2Tw{9E>YyTtiqjL@)(?0C( zobcV_qH7Pkx2fZ{?}go_uXw-trXy}X^+zNme5GISFGQcVznvWYdv4t|{d?clzjp=b z;g;q>CrV*tfE&r)8_pw(ZvY1$Ka$GqZ6B?@$FI%yMYk=lY`z}GhJMv-pN%`P|361i zdrRpuqG6gkbl!?DnwNgU)?(UMQX#ek@!J!%dpRwy$T24#B;W>_dNxX)N3lo zkUX2VE-C7%F(5mIlqZK>_==+m3vvNs(cAdLxk72&*)b%ohVk^#HC4?=@Yk!zuBj?b z!mm|c{wBJ54$zUow1#^6qX^k4ZQdZ;UiKy1hWdj!Jm8-Pk!|ycY-b{{y*JS*T~}=| z)mI}<%@D;FWc?YWD-*1ojz9QuuqP}#UR{Mgcz;AN*Zw3}ws(cwe`;!7Y-x27h3u-P zO>$+l8=4{nSP}^^eXw;_MP}(J;Ae1Qh3AJ*a1Q%=4w-sa6~6q_rsm80(EsTC$qC7o z51g;`N&^Lb<&#-Im5sbvuxJa#G|fTICjf<_biy5WtJeU9v~|bV6R9{&Y~6w$a$-4Z z1IzK1RR!f23nM0{K*!FdIU+TDVONC)a%4nF=`1~0QW^BBI3!{-RHz}xFNZ|oUP4bc zs*Nf%LP$GtD+@>FTfmMtG=CM3Aq1?D{8fA~T};#sN5BwGwo$g)8I#MWNDPGIp>`|{ ziUmUl$1a^`adI`()cN3+26E>NF1EUftcIKzN9wfswyvpn7>YPX*NQ^vl$)seg)zS% z+C_gU>%Do>Y1<74JYjEXr5U;-+Dd3 zArywWE>g0IbRM?b1x8O4E34I5FVNPwT1>PH8rNJ;>Kgn*qYr*^W_L-A&3Iu~%$$`|&-vpz?_xIW^Pgss`+r93g~|@(EWGAi*IXQumc4$O z$h?E%+z1B$#99hW*f2#(4STw2agk`as zhmwY#xGZ*8KPk|B#!A&2OW5=l?2LizZd+m-_o$k}5NM^BG2adVfL2Vj1N)a|Q$dsw z#t?#Fo@|cAlNn+p91^A^Jg{*gXbBl=BM07M3zgL-<%kygig-Id2$l&;Cfm5)+29&U z(RHQe8maeTz9n|Ean5?s9D^)Hpf;{ZR$gfIXTfC8B)YH? zG5OHvu^15%5qv5ItVpM)P6{2IY#UBdtg~b0T8je}gK<5iEx;8(Ljwu;o`{%O1=73( zt5}<-o+Pg9q%|39l`Zg9uj|p++Y4dy;7gmK!|D&j zv&3q?A+PfuNw|&kTAbxb{ivbRbCJ4T97KOMGW7CY7N$=V)VP7Ik)>1E7JE=pz5l}%axoWsHe}h8{L>~Y3yz`p=$LDLrYkzBv4A6%6&sH@=|7y0{3`KD22g7c|GH6GOX1gFk! zq8S|3K0DVN+2w-IqFuh(SSYOLoYkJ)Z#TmxFK&}Z<5$$J1#|9P>+PVWMRJ3a9q{*` zx23i*;o+pFoo^>34opuxsxFjz($uolX=S}8#{Qa+_)=Qp^HXncSqOi}+~lkoD>wA2 z-_UbO%MFPscjcs9lasuB`pOLnpJntd%2@woQTW>d(I-WqBQ*J0}f12ynVz>9t zsCZ@o=?%AzI&yWprL5e?g2k6=liyC|F40}f$_|4ZaO}ouqe<>~q(CEM1}+=JHEFs0 zdfC{4XtcNETQ*0`?fptFJ9OmyB$~%I7}sENA?ae%Dlk_}OUW%Nj>fw788R!O@W7^U z6!0P8WELz|Z9gEDij)u|B8_LCOuqO?*Ob2wgl5}!!pCKb6(}7pH}SNcyKHL6_M82|8LUA{Y1n>E(#ZVvk6VGlvr0X= zhFPiQwF6omC2Oy`OV+3z%>i;du=PRkWj#rM`~PU8Nl%%qs8kk9{{R zfh*)EE{TVV)*$f!L*=`p;WHN!>Er{_=+iKn;RwaVa?e2^yFOxn7TdhuneAzF=pAYM zjmK9wBugxgRQj&?JxhkEsDCq;U`ULCx;4a&^kEGsjrI>4O0za$ZN$+$vyG7eRJoDAkLODan}Cwn*5HRPp8YV zZw|OE0A-8Fa%R>=^nj*!x;v~XvO{nUVu}=+@f@4Fw|E~K7f%Ss5JK6T+b3^&ji{Z# z^!!eZUG-SmBi;qFLAwo_y#*1N^KO#HX>AdtU` zMAH{uM(G^T)A!wkY9q{-%WJz$iA(IQJ>wYboiQVbWr5{L_f2h0wf%YtIEXW&Dq zg2a+07Ye&ghygF)&~tOq2`9)PQo9gQ(|2oOx1lphRK>@}Dv&01B`1e80#>V&ihZ$0 zq?w;=A;#w-DubL{N2vjQ;B!DLV@zCRu*4h+gWzpMg|e&RnDx$Fn{#bXP$#=shZC^yk>|e?)q~nZD9e|IRb$L9TybS)mtB z#Rj-P+j{sF%jqCKsUO?k@#96KM%NCe&?>Xf^3n!di66uaU+qVxMc6)yLCEV3vvi71G+s;i&ekChqO;&Q& zxhpq(ylTUTbuGswru588-ZpLJhJUWw@QV>$(dJ{9v}`$YOl&%O zZeLRpX*?|y9haQBK5I~P``onkONydz;zwOkbmQEN^~ZLPcEjI}85DhZZrbQI14?37 z7iDxDl~$GdW>Uu15t$t+Q&0M^`VutWK0K+Z`h5I6fWoy;iemXFT-#FDviXd(_0>hu zRP@@uuve%ycWUgOeo4#6qVM<5N4x-49NUGQvN?2SHLLZmpET@-lKhJ|hc4Y*wc?YO z!&Ypry}qRV{$4fQ;Q5JhO|>_$U1IhZi(~inCnhWU)z3%#k7bnaQ_jXco_)&M-rDi) zH^d*ky*T;fG)8`D_m`zJqB)%r4&-6g`_Qvp*)?i7r4Btd`rWlFxNc1y(q|Q&${_8n z&HL3=^dBAPUv%3`mLCycjQ2Xiiu!!)pa)6``Z>@aqHcvA*VNCHwNhTd3Cuc9ne)%5 zO~ORpckrIg-N-;G_G{rWvNj=1W-qaqi}E{8U(CPm$FF1Q*WFq(f>UgCn+AwbLT(Hy zY^x|wfo8{j~j$ zzDpXAgqLlb_zC!?RAJ)pZR-E#J<5^9)L$Di$0@ag7RPQ8#9DfC>R|E37tXN!B4OFT zG*Agynsr`It4~QAkyfkwOe!y&T(y}EB+JM4+O)@2(UyoUdw@!uJ!@gXb2%AWAiM>> zlW^pN!k4}}Zo-oP{7ZeyeRrK#@tgh4Gye9gqpmIbpZRC)eeAN#Pxdal<=x-eLf^K~JQTLz$~G7Eqz2ku6adL!{p5D=?{I>;I*(hu7p#h|vL@*ZA@f{uNru z!dLlM9(u`0#gROk9Q{9A5*yf~U42Qh3<#>LJ$ofyxO}>er2CLXgO;Y&~=+~xY;jw{V)3YoB@3KA)fujRC zD=YZzl+{rzW_yn(Q#4&?$zXgyV1)(|q7jIi=Ax5+wnq zy?}L)X>(TCxUK~dj^Tn9Sl^0{dxn?GFoh}v$`=FCKL{##kQ!L)Ek_ZF_2U{az1!GrDL@=Cs)2f zdg#X!r0QDYB31;42aE1$9Vf~}4$kQ?WZ0l_$m>bgMSuJxi(B*9z}Hk7PEA-m^9#sR zGjJEH?ArgVyqg-^Ia0x$+ji|B829@6Lk8nMIdm{?Z22h!=pxNQhJNe+J0^h3q3ndhI!tR{J}a#;ny_!l~^qX zitgqD>h)(xqVaq2mWq!l^7;DhNn^A5+@#Y^ZY%tp1dsfv$8}{JLU3|0f5|O*7-jzY zKd}b7maPBp?S6pb+xCXqA`IXM8^W$lG}2xj+WWKwU5hGw{!Coc{A2`iSSOxSB~XZ_~y$*xkn0( zc3}61;6r@PHDu4PI;)w3F>q{xy>aJ$vDX)q+h189oC1xQ_`6OvI?e|TCv+ofACudp zeBCss7w5gpn%4HJi%E}vDrG}^Hmaj)|58-9!9CAnq-@{VF@E2v+A4|Uo*Zg8Hm6w! zgMfWiJ53H_k}c0HiM_CfW5S`&E{VO#zjEFkWom>>qqh9GBF;q0yq;+*?<9NpN@zo` z2o7~|${f)^)*hwF`z3dJu6xj^K?Sj)4?4R_JBd2Ra}1RHwhlR+Raf%=21L6QJir6N z9X#0DJacLt+O>pk8MFiDsPp;dhIU7T%dMWiyH1mT)qKBNls!!hou8EW(vG2klRh@D1 zu%==rYOV++!#c$*!&cM~K8P_?wh~5MsD^8}=O4aV(e53yBy$Xz zWSe+W4d;nNgN2KZ!d!ixZRJ$F`ao^jINX^v(^(T3|0y0)RmkNgATBgZQdtlziT;``&4=#kJc>w3spBc;7We>Ud zMMXaXqRLrDY0edwiPGtBpndkf|GoAj4^`hijyyu<^coW@OGvcrN8lV;I6M@+7)~Tx z7l*zh)ixVcTOUoeB~Lwm(=KYuR(Mz>HmeVQcGb0%UY}E*5sp-XYg=($PI)SReWk9Y z`BypRJ@Kn$9SQr9q0v9*dYg}Qjz?Fv&4d6t&*{#ER9#@(3O}!5EVevM_}vD&XhvojfA;?qefjTA3FF=fviMwEr@mGH+$6U(*%3hx=f9uB>J z?Mr~L2EeEEl>jy-K_=UwFpgpeNt~jA{YPux4Mq(Pu=C=i$_9`uDsK`;Pj&eORhZ<+a`60 z>snOUHF9D+wWI0 zOBebNn2R(vpZm4Hnb|B-o}7CL;(k|_k{=Un6UZ?%o`J2sdF_U8p>&CdZcInn*tMI= z=95v8N05MZf$AoG#u6b2cc+q;U9u^aQm=~il_FpzQV7(x*XX(Qqk*S~iPU+Xc z;CokHY9VrUAJZk@Og97W{`8^7T}(YJO;u__gYX=X#QuUau{_zh8yzjnS^N%ib=KcY z1VVJvAhX0d9w1HBX>Mmhs4M8U#o5w_D9^LIGjkW&t{JGNo}83Q6(0X|6Q_h87)?UT z8UJ^^8gR{(Z|;n{Ul^c9$);O)HsR?;>gH=&C-w5I6BL5(grNwYeQ|?4rg8j57H;MH zt5$WGTv8`?Sr*`YV)$12RAt#(YlGp!0$cP5|1c0on!bvgnKFq1s?D_;#IL-t<1}R4 zSU<&|9QaH<*qV~GgIiEwo;kCGr?Yr-{H;AP1=wO#U+Ncr#k-)obq{mo#w(T z_4r$!0kl0B0TiGbp4PX5GiIHo*06ez&1|FMzp;Ude$uOb2G8w=J3+VYCAuwb+eIrk ztdnRQ;Kl%2m4S%m|NraxVf65pAl9x|ex#6#M zEj!6i|M>9T1EMdJeiHvO3LZpX06@5Itp`fmAAE$Y`lXem>;h0RrzEoC*htG$px_{z zY(6H^vMQ7^v-X~rog4>Clu>Nlyk-Y8h+grl;+uirHyfGpv`)v?vr{pAe zra&V3^HB0N{ZcwZj~(?k$tF7w6SB$9jtP;kyN5L_#eT70T22%H58u=}kzs(pqaM{TGB~M)%dHT|(M(8HVn!l;>)A>wSm7?fv zHj5=&v(c|@sn$(`fbs=$r}q139YBbG8vv%{j`>NnA7h@LdA(2Ma<}L}s|ECDFwUpX zE7LYkK8&JDNyvjqM51co+K&%Cg~%8!em|$GeFH8k){|BRQPX{JuYwlL@q)v$Dh3wKi2RkTq}V z*=J>7sm8yeDzI*xtm83UonhtcPU5w7uY8U8InY2d+t_D2P3WShS- zA$cfSN7lt2wfM*_%1ZK?u`MeL3!W-#+CMk1;P(`ZH1+;f&ZB!Fq%59X^`oOEm7ldJ z^dyWZS7cpim1}sk0r+=_NMLvQgGD6Z$pSs3aaRNN-qICi+mR2vmot@{roi;osH!TjSnOI1{={L&Ran$2)v8 z>#f5tq?CBa;1g;{&sA~Xy;agWvf}=Gt5#gWE|T;t*CQ?dtg*^n2eg7slR>Z{2TzU$ zFxCh2q_gEJjL@;(t8gQWFSn3U9^DHcOS6RZTB({rYGC%kW8%2+7bs$q)y_(R6MK~v zk9}~x1$ZST3sYa0M!=5SM9!Omv$3KF&`v(D@m!j`1AunE^Vd>mTFZh{zsFHA0SPl6 zR#OUG4H+XCmAq-9yh^uLw<0s0cNgP&qL-I}{cYk6i=?Rm2ghsR1!8rTGa_B=U8j2$ zS=y;KHLM=yod7))d&dvEoao|xXaVp6!$^Ye{6|8~{_Mv&5Ot+qM3oM?Xmve#pDDY- zTNC38U}2bUis{IC0Vk16_MMC<1J{KWLXU0uBmxEh)s6)0)3&arlgAt1`)W83sG7|2 z#L4_H%WJsdu^1_sUDBiKlao~9OV6Plsa?TLD0tDXp?$}*m2-6fwz8feQG%)~NMwT@R=rP}+rtWs9lhf9ozPbxUA zsm%c}CyCU`%S^Ks>hdj6J70+q7XYH|x}Yp`%!oq<(SADX{mS3G-#cU3sLUCoG726o z3coro{FedIvr)iSkg@*ZqVRu@3;*Giobuvu!o}4I=T3jyVnmtQ_QoN&T~ z$Yv^Ndo*)>$Y9Y`PxdPOdP1af4r$Z-4^F*$M*O8-?s`Bl0{&g09eV2dzV)-5C-YPvq zhTX=+a@#kDJ{QGum6L|e$DviNgR_HGk;FOqsv<43`*>(2(vKtFHIro6ui(NqumA*dpu@kCJK266f|o_FTrcpGHwho`3IBL(fEL%^dAmAKW0gm~GNyxO$Q za2Zm@jS81(sMdaF4mNL5eOOhPnD8`xsBHAlen&vIX8hWnTWM1=n(Rj0e4ct;5C8u? z@_luoo1pk)-5_$_qZZ`ky|}l zWir$^xkjef>?yZeX+w@NF94l=??!DOyVa2URyGG)la$754`^MqwXu=zVjhVvOkYuj zAuxqUj9aY1uVGc?I08e{!?H={y!({9a0K7t+pY#7brmz)8Vj;VN)w!WZo>kHP9KoHqz z%wXEzW~lGbI&4XN@EOwGMnRYkn83Oz9PNk`M81bCJZHt*4%lc0#(|c*frfo5aN~>8 zV{MgdI7dx3at7PTUiiE;IA@#=bM4-q=0I$j)!;&5=xOKdrYPJNLLmFI-xvNyhY*!z zZ#rd*U;jeQDDV%bG>XKz>_}xuuRlj|xy4lK@XC=%fYV4RQ~K_|>(@ACuL!8^@tFKf zbSIoov_k`_#{oJx!q2vW(9b(lx9l^jZK2@!+18 zE`*{gu-6Mh0Pdv;ZjeTc39TkL5O%ikx%V8|>+KJ+;X%hCoWe~eZKI&^>_YMgiw+qn z+`a@Z)1&9WX|NTkHKS*B%j9J+$<-o@sc%Qe~VJN zoF+iBdxT_LIBU@mnh{4HC`cW`9NL!u{mvteXN01+L;LtaT4)P8u+2mBY}t&B`e?tM z7D~m@TOJ~ew>E!j?D~G8IrwqyL!q{p#>9#e64$;oF?MCYs?^VuGCGb%6Yj)N>&8i zma;~MiA_MJ%H#hu>&>d^P;vQ4M5Rn)P04Bf#v0tLsbah{o?fq; zvJKrr1|@Z|#THvO=XkLN*2Nyep=pT03mz--^^*kpo&s-wM|15jU6>7~C+qy(Br&+k ze^;-M6lduIHWHlRHkj+D#Nhrfuo>BombkS+4)spZ7nb#9%r=u*VD}3tjv6*CY!q zq12!;8BQ0<`-!D01JP`X`4J_&b%1lh(h!D2t}c4@~Gsv z+Z{?GN)>D&5^DJ>U_M7S5VIH zT-FEdn53=TVy=9WGTTNq1|JmL_o=#iV&OL&N%C{ka3_pDf;uK3GfLX`yOljsQcM?J@r^7@Zf*OP}==Kt4EfYI3*)hw?z+h7^*eyCg%b) zt|;5`@l@9(w`Ch7p)T8a*bP~;eDd<-uL(O4JcXRBb@H`J(o8DGfUeJId*;4~?0=!w z7SSxu+Bcmyh!+i_c_dtvnkV%T+uS6Dbv#?=yI45dA{_Lx$I1+EOf$bM1`cD%waiwu z;JnXr)rk2hcUv`YN^btJd4n5n$uFN2 zPI$07;T{-4f=4UQFP{)jxUM?k>e)rHnuNsXdnG;!9u3mSPo|dbD2d&gkT`u&sFrxO zNf0C^&zuZP##I|$s%v={(#U@Jm(=I5q=6w^Zf>d?hU}UfI$N$H*Q1lD3Uo*^-T4*Y!)v!mp*M zYfFI{%IelHV8_oc9v;^JzwXs5wozg@s)O>uaMDlT2g8Ba^2Kn+yta$X^Jk13xNIB) zm@$_{#ute%2}BqJDtLvCS}wJWY5;?!~WT=-2D9^D+u!RBV${s@IQg$1_jw#R(?o+wt3T z(rilZ{6BB@>7I=gQ?`HmY*~ctht%FUsJ?7Sq(6JJzx%&s+y6Q7Z8UfVE5#RtAFqvK zyDNJi+hqEyxy`FCE`b+!JJ2mZ^o4-DLP}6p93Q&;SJ#YQmvZevQ%3I%1 z)^y6RP?rL(?v#R{=4_C1m=b4?NDY~lIJ1)4YWvnZx_{9G?MKvXj`u#bfyuZ-z2?NT z?}i2+5S6MmCnce8T;UZb04-DGi&IKfA7iycIkB3kX!gbpjc3SUoNWKM^y00fu%NhU zXey<)hqbP&778wJC`@Nja>d%kJ;DH{hp|OVF^;U3a@_!wtZ}EfIFzR^%4C$e!hk_TNm`9 zh(JBsH{&mWd5Q^g4~%>)P`pM_c$~&pXXtSeQgAs_>T>wSVt~nuCzhTNnFi#?*FN{G zRJWdITDXWhDI$>66Kf{%F|?x5oIy$Hzc3V$dysTWolf+bQh}+-XsV+=x_%&@X`6v&mmzPlex7hqu{BcaN^&O8#X_`{DN@8t_hI~P_I@oxZ$?^^8W}Yyj-2I zvHRFt4jn?<{2&*vb$kpSZQ#_HwD)g2a$@YdacS%CEsECS*Nb8L|C^%d=W{a(vbsl4 zNX{(i-#vOPe)Jm@ePM1|!Q!Imlla>)-CH+~jD#wW$d8sDKBj%x-8tlDvf{9N$Icu5 zMP5m)dlBsa@pDIEM#ryvg{I_9jkTST(Q#d`P#aABYw_#F!ry=B8EFLzP)hgaXV30G zuJ0KP47>`}{@%TP4{z6^J!{wm7zBB)fDJ!r_1DnT$x@fAh0*JC5r+nlJS! z7rgZ?95|zpUmv(Maya3Dr$+y^G?ILoUxT)%MsHd76_HQg9on8q{f3>j?BpHL7pS2X9O77Z_K-Pu=WA3^8HA7m_6agg7>Wb{wMwEN5!#r! zE*3(92V=$yLP+j0XXMl<^&BXKB=brWW>iARLI@#$cC&AfwvVO}<(0mix@a?YxZI+| zvL@gV!in3KO-S>rQoE)tAGmT|HI<_kmk&{BLA)L+UQ^Wl7wgJ1&#~UDRlNCR1Y$&D zZSGxCKFWeq-DQjH;zx$d-+)(k-^CGBs^Zb^vxZ;`9fB$Ur-ALW3rD$&)E31TsTBqm zsUaT~fY4arRsX;zo|!9BbXdzWg|s3)px6aEbWq0f%@6X{sX`IWJ&6tNN(;MsWT+99 z(J9*)jOoosjo!|cIDo9Mp3E!oIthhOeL)GRi zfkGXrZg-iEh!Ctawr9E=MME+yr?d{!2YPJ8a6t%3n{a$AtZ#_b@zwAl6jCGeuo_Ca z`U@eoS(q^zg%A=^rgOmT#V3$}H6f{Ptah^oxZzG^KceDjjm+tuvTvahgW11TIy>Jp zTw+OPK)kCsW%CW7{%mZ+t)n$rW1)fw1Py{vvSS!|H3`sc$!_Pg%9p_|*fEh{@Mzvl zI6|v7Vr}T@N)9R@=3GzBABGXF}O&K%j<7 z43QFhE*>9GqOCagjX52-o??yKw4FJ}JgqqCL_PWk1+oi{aiy_EU) zFLDnb-s_K7JTvXaRg2DD|IdKT8M@g(Ty00-vYb%gMr@meI^%De{2;twph}P&PMkBg zu7+P@4PmkfOACZTl7rU^sD(0M1?4^P7~5R-g0h1OIqB6y5WHU4wWE63QzME)7*vAt zwsloM%tik!e1(+NR2zd8 zb*lND`|#zAnI>MM*VH%e?dUvV*qu2&=b+7Y#^56BlA0EOd#md z9_^J}s7u=Z1+qy{ZiIz=uw2tBiz1X6v^WQn5$}=&kQP;fUL1$xa(i_vr(GI<34Q7L;ri{&}V<`;SEWsYT8;n zloJx<<4s>4i67y$LMsH9q*bJnpsRVshe9jbetu3GIdOLUBrUm7TK9`^C$u^}HjWhnIC1u^8`1#Ffy--1~ zvn=Q}7l%k5R&mzy#>wf~d(tT!zI8?kdT|pD+&L>h%r_}%Q*82B3LaI37p>(koSxAsy(5=iII;ar?fg%bRtJU;GJ#(m z0KyDV%hm*R(0|05Ku#oU*x1yGOiGxJ1zxR@WyzuQAm{ zKW|p^q1s1v?lmT5F!j*p%KT8PZ!O3U{=S%>k8ninj-J1#-pm$Xa|Hkhw3=PUG>wQD zh|m@Mn2J-~C+ zk=MMQF`oKslMPA9A%kmwh~4p6#XaK^o~=%J5|$#FgBxxhJnZKD@`>Su+b2YBf>iRR z!3{SK4Anv!`T5kc&q`vEgv2{~CEmCwv<2$Ok4j?YupD`92}vVAlG4cJ<>v^sWcc;< z_mgX6RJLa8lyz<-l9Uu2*^taZv+b!UX4^b5R^BhT(CrvB3ebgajVN?Gu<^%{P~%}J zej9?Hodcp%gzNtbC^@$akz_R(xD!QdwuzwUav|(|Zr>LbU4JpU{-`j2adD8&p^|jLITG_cuh3%XN$3(AWxG0=rHT_N!hs)pO@2(BMQHL+>)!daEsxD1 zAY0KVX%l%}HufxTn$wNx#>P~L#X|UX>}IyEq)eq3OhZcmIAmH0Gv`Mko>U~vP|oJx zS-Uled@CIN%77Zb#%zB(Ga27|caRIoish3m-1n9R1+ykI>i85x278N&TMDqlySuU_ z6_2($YafGss&n1pBD$@Xj9f|i*%pS{L5BqUQ&aULQb(4QpKJYbwo*%8$lieEqZc{x z*{N=PhAnUSeHW5klT&x5{@RY6-Vx(l2o3mlg{Z2Q$}mH;b`TsKy!C^_g~Do%c@6JC zNn;Rf++Z~bq`7u=y=}>=2;ewG(n8s!|EQEsI7Fn2k!7;>&e-s3X@yPp{CZy-ml*-#*r^8tm9B$bp- zk)3FqUHQQIB8iI`2_Hg{=k|=of`}qu<%4S+W{fw8+pr)6Fq5IQB1Da4IWn?L#pGZZ z=9SIO#2stN$PJZNE);!U#@wzoBKKxZ*S~^<_Db1va6M!E3s%?0xcIV|<4p|hjZlgW zYUT&lqj|GDNeLj$bW+ogQ}?->T)C!u{Uuquw|~ls2!&B}c*ak~@9U}cqgB%6fJZ7^ z8X|RTae*pnnS@t}<0?8Kqb`9%)b5PqJohcouPuD+h|6pcnK77EjA-OwumnN~MiANp z3WhHZsHY(`*Eu8+I{y`@xw zdcX)-1{Ke{Q*40;8RED~o|bNar(auYN;^v;hh0K6zJ_7V&8JA*YGOkRcl?)ChfwOB zYB`$dadOOFcnP}}Zds|Nmgsu8P_bywKuIYWA^Ac#VZ*PHE@y!$;;3ooJm;wq6$vjf zK|Llcfk_vMClH59Lx5?T6JSPyK64ti8YUwSfkJC0{@FOR5+b89o^Lh`GmgkA_!n^$ zAnzFum}O~1gdS{=Zm=M?x-rtaOD?#`^Rd*-3)ixKQTU7ap)}R)5hi@5j3Z1qiCgsF z%mQ*@`{r#~L4|BhUmvnUwpsJ0%ql6{GqvoSlGri;uGGYVwV|PK_K%gs79}Kp+$-_z z*Q#356H~6~pE9X`^316#H@v=T!)tXd--c4w^h?fyt^Z3GX28aO{X@dWe?atE>Fl4j zXgC>34sL%CO2}nJQEBJjiPmhDvVS(2k$ksGwo+8Gt<#lk7tFsZmt~SYq)hU2$RvwJ z|LmI=g$jk8f5ox^(Ji7g+lmj1Vww1!lQ+IQ%A=5N>&St9&m4%WbZ;k8sPN6h>wFQ~ zt(K{Yw@)NX_>K_Xg*m-n$zeB)G=4oPxA!%xrmxtspSwW+K!s!*6?M^+i{rht4GHYH z#oyf$NXf%pblXel3h5x_+~-qz`<-=kj^nfck-C~V!Dw&KR zOSW*U!fGgZ@ULcrD`MrmsbOEB26vJ`r6m0AD+6oUta=4kB#^|aab9@EewJgFK!WX6 zS?u_1IM2m*jzS$;aC*#&t#t??)_Q);_7&zfR-&^Hh8X%h3aD*+kKBuJDCZ?+*vO6^ z0j%8l?Z`|ko0yl1i_UD(T`Sv;kl3t>%Gp-*=e@i$238F!NpdNuw7HHb4XGo-RNEME z@&x)Xra)JQOJJ0tCX;x?11qEYi?m|Qy*m{?j(0Mel~`{s$TS#{rJYQ%-HOYPYn5gr zl`!=$e07M@j-dp}6;i2KN+!K8vN`w};<7rZF#*;5u`m~I(T43G?@Gw+W)f7ngI8^Y z4otZ=f>5V1-lh46FYIy`fKRUTe9N9sUsl~y5$_TKHcv>bTwh4xG}K&)GAi#zY&Zqa zqnj3UKxS}cW;$8T_kZvs8!)aanlgMN9P*L(O22)iyj8zlH47b2cLU0+jaWoQyGBXb z6ASlOUou`+vaqpU=l;D*zkwRc#3P6&8`<66&5g%wA@PP`LdrwvW7YMi8=|GFDd@(c zEjmxRLul#@b=#_HW79`9#_-_ly6u9y_#^`|r1RL7v+e2{5{noER*@DYvBvOR?Q=*G>S@Q!PiEWF96xsixCW zFCd>>wH~{MX+iu&q&!o>2r}uiz@lF4ZBGa7PD9NVShgKbtQ9%LOc_0&m@)SC_m?`( z_>s5g$++aeOYq>+tXx5TxVn&*!Z=_Vge&MGN`y_tipL!ayG_B+34ug`D9F+d8B#h##>=X7>KX+IxY-+MYoB5u-D|NPbCT z4cIVyV?xRz@vwmH%1o#^Q0@)ZNY%9Y z2RC0kwJm=)p-=%@ybS&kIKLm@@K5Yf1Y#2(`w=Rv+TZ*qfe5nx!;&1GWFdy-=i(3% za4Zx#k5}h&)F02c&cA#CSBrf+B4b-^@7B3|!t%x4PIeoR;1jjA)>;35o9f2)I33E! zqNu@MB;lYQih1|KA*;KT_S|#CU4jDjD~Lw`#!^knDZ5Dy{hC#nB1m!PGx_xz=oXRF z_SK32wTYaz+U4{ zw;k{nmgG}y?Rr4OA>@Lwn~4Z7L2ZbR5rWfVbzmyelBnp|OvJ_f8fXox!`hQ-N-##T z*-$&az-q^)(kqT@wL3$KjhH66R}~4+je^!Hw9-Lxcm!%Bg)~x!Zz^-FZwO8)q-shR zy3th{RZn`$9?^hGtH?x;Df7sQs9B|p-D4xkHCAalQyQ|{g}Mz?!YC|B9M6!@@6n9} zEo?U+bbfbM?BNid@+Ol9+I&wBiI}5T!e(9qcw=$`a~4R*Ec8>r8hy(ITsCMc`Cbtu z(}2rR=Z`}^%_+jdTENZ;nO~Qzf)ATx@ivApghOP}V3dF9 z`(XF$%O`p{UPdX2s`%Njp9YY30xrAY2d}pb%ee2v81ra|NsA}m$TMn@p4}?lFtMS) z2LJa%U5UuFc!jDsvwxq2xu@JKt$|3>*tz)_e zJ}Bo+Q+|M%cEnsrkgr>kA_uErKGIl8<|Eq= zo%u-lp<~tlCe24=1Gb^P#>D!6X}9NxKSN%p6UW2f{NHB#qQS7m=*!(7|e z+}1(=a`TbddqFt)^k93ddY>Z29E6tQ(H0xhNDO_>QQcDrY}lO@WDsz~&_~ zcH`0H3PNiK({w!Xov$gDR+U_IS<2v~=|VR+`?DbkOBW??;MO=&XEZhhakoXYSh(1{ zBJ9l)>oukb{bniX$#r4=#?9UHT7MM5udAp=!UXuV|0Fbr)FaOLQ3PbAO+IDJraebt z>EtbH3z3ApTjz;G^vL?7G^nCGht=kMuyJ$(vWYdRJ(97s(&{?;u6*=m!^OfQwoUv5 zGLY2cn1S@JP1_t;n^tIUk)3Yi)T(Eb#<8Tl#EJseDH-Hs3gu8qk#$~9>v-PQw-HR^ zwC;r#(waHFFJki5Xoy`nxoR^@BFo42D!!#OS~cOr;__M1s@WfwG~5X)>(d}~<`ISE z@%V5QMy$qv-TBkJwiP$fNZI2gxp=|}D=5ajOUS4UN-7m#Kt|P8v0Smj{H|(kms{Eu zY$K+_eFFk`74P&_#WN<^5yo7~Md+lUT8Rs@b^~sE$!xHfb-cO?m-*$a{STus4XxNZ zrKNv$bxHfn9r&FT2!>3410BU1?SA-hyiPwPIaHke-op~|LG7wuM&T#)- zy?#Sr;rMVF4+?`9StMmB0+ba}7-cStz-kdUr0S5}qFM!6dem?T#`u)bRwY5SvWnnD zP2u3e3GiL2{hytVNMFdzI}ZIq^y8i}o=pUuA(ISNaS=ZZjG;qAXPO6oPy#j@>x72Y z9+b&xbbzRJaY)h>Qnjw&OMn0RT^K;w3z^qKpIjV1>F+t^ zKMN`{{=o6#BjxNu}Oehdgls?j`qc&|uz{Oz>KkqO)3)IVWy%RBS)+fT|# z%6>FGpQM$2R$jC<`^NPA_Cy?;j-TITCs)*Wi{6uz*-3Rv(o>;iEKpjD&Y_YlUw-e7^974h=l?C`v6n(}0kq@z%~R-Te!(NwSr^N*@(Whu5IKzmW3?%9t=B$swO%`h#F!^q z;lgW{pka!LjknTlwMoD9VoDRwNv=F$R7L>= zZN2n<(IF3TL4~1rtJ?L^o6FN|UiSV(om;lb538_B?TnPgl+09o{!&H@RpjN*yR-!Oh(d+PHLX9U8u6+M^s{Ga8+G@ z>iHLwGEE0^@D%C#(`@ZMM^;ED}G)|DPI*>{QQ%WgEii-3X}sJ6 zM;0JMa?JWKQCG@NSZCDwPg#{Teo18EgX>8vAp~fdKKJ}UC>`&*uS=%o;W0Xz8T9dWj_iwxt018kAdatWJDb@O`|e5`ZZs~iXNlC#cTx#8ng8$PUSIW94!XIApIX)8DUbJd1_+@DrZUle|7T=?+;(H`g_Ffw!N z85spDio*8}h;FB9wX+kBMf*0cR%@7>Uw%3|omD3s+kNb=D^Mzip4YM!n>6}whPB8R zSc^;t?RHH`r17*+bX;=g`m90G?Q_%CFDZ(?i63=E(T#I6)*stF+6{j@W>ECuxiIn{ zP!hYkD5K-3w5rrMlTbF7*^x5!q}{16pv`syjy;K=`)WglZ-iSWhEgh)4~W)Yms6e^ zj%0sa5_?v(VjJ}T*n1c7Dypk}T*HYZLCFPBR7gSy7jGqaBVaHAf`m%B6fY=WO%N^K zz7MIQQ8e)!0}0j=;*EGCfPR7n3yLjm#fAhcT5OS01>2t_kt%rQAZnt7GQwzd3YW&!!Rdj&z`;4yWV%bE7<2h^4!eG^D`SK-TG0N_Rk3V%o>ro0EnRN zS-3&Ioys*->&7%rx{c_b&`T>@dP3JHnz{+I{{u54U$Wo-uKa<$3jprky!jzQR}DTj zYzq#y6YJ!|v$afyhX{X+WM^XpiMj8X;TPgKNJa#PE4w2xEci1;&pG(CmA*+h{ofBvY#n%T0|i)PGz?vftZ0I!-|Bs^tEXZ_7N|E+enpVKvP@7kl2} z@&vK${aai{RtDHW;urF$ud?tPG#!(;zAClqGlm)pTBz87gkXxZFf&Ee&pEg*J6|l$ zoVRdh;AnjYu4R#zZ&-${#P zTVx4)i?nTBlihy(WdcMafW6YoN?vgaV8?)?W#pkES!?XTcOuIRvl8Y=5j`#u;Dhl z8Gz#}sp>Vlb9h{?he-=C5X-zrD=n6&wrvS%1 zt6Rlz-pb6DTwa)Jjs*e`J5nKIY{;zU?Fb>>^yk>XW3!K+!kjKv$Oj3_H=18(@gBGb zj>sfvvca)YI?E%Fl>>{YTcs);CPe%(YvAmmUuFBa%6u5|u);yaz?Q2C1r{zNAZdp3 zF9E#7lfcB!Ca)kns1Q9TT}V-^F1)l5hQN!;2|khKwh}fAZIx=AE?e=~Ye(H3&JA{!^5*1&gj;&_n4l$MxPKF1a_?=yR;-6cR^L$IYp73y$0t*2CoZe zmtGl;>$d9Rl4Hh4e+9pg>d;zHYweRu4i!b8j*EXJIetNM#it|Z_Y1!>Cj3_ahM7q{ zcjfi0%}d+(bV0bGf5QcE`@f*)vSVRtl&m#U5@14*5Z$u(!u>I~Yos-0aN7*njcg4!4+U;}pi9Hl z1i;D@X0=-GjUUvO?B&|3M;0YM(|uvd$F(J&)Z#9Enm3cP#e}YfT(suNvB{>+4PUUL z^8Sn-YciN$>KNK^4EUmo1y8|-#J5=O%DGE3TC3@z8($0E!sJ>`)r56bOcFiL$G^Sr zmtzi>$08?YD=aYe*6VNyGvO;-B}l5(t48eZ9At598g?mXemeEsh&^@4sXq1s$ah1Feg#tznta1Obmv%keSy3DkrXsL}BsHqh z5^`<#Z#s>JCR%Md7piRL-*^)sIO(M1IIGoiS}1jgPv>mJ%>IqD5w9KGoEND|K9oR% zc;T5+3AwN>=Yax7BoKBEA_-^SLyU%}CR%9tk&^c^X0|@i|B6Lprrdn-52j??(0S^` z*Y_^E=c|Du&p0*y&AI!&{1rXECE~dtkN3qj7TvmbpvdemjNXh>H3^djVLLp&& z?h%WLLkaf#Cv!01Rn~^SZ%exN@@=b4H@QB(hoh+N`IlW?wMnJ&nw3G@uPC{{v1q88 zhzYwHzXL#Jaiu>_Su%6krOM7DNePtQxZb^T3)qK!nt~|b1RBhK)xb*1#76Fw^g4kv zd1B}0NyNGcdvHO%D-8H_3Q@%cp%sS(7X0YmvO`M#W4;@^aT!hk0|4xS+;jkA-*@9rp&FzE!rFckwhfU+#7QvWFH+It_7%H%m; zl}X!=-ln)4EF43cZ7)0!m9=d{);|#4p-ue(sD&O(MFKh*E>3|H^)0>1f-N^{2VKFN zd~WKY3E&>UVyMW#k$a_%RpRaB>MVZic?LQrx$xFuOfCXI4^bcNIE8dr)wqxvjbD2G zG4=G|Ek%{JzR)Sgh113gGZPi?+k&A_o6JOuY6*P_^{O8iH_+u2qDmwV7L&S`!*eFPMCQD_E-w6Agj}<4O2zr&3Wr`dExy9K=2}9z7 z#4_Mh%LwNL07e`S_l6anJl1kxfl6EJtU^_fr2(ps4$4PaVn{<+`zcL+>)$;rNOlQ5 ziyx5tJB;^+@l!N>2EQFLCiX6?0#S~IW53y0NKIg**!)fX;cfX7zQ(;+0J`Z|+u}CV zeYGG%q|eYAOul0-6jeyvHUm~vR+(RAm4Luauw)1sg~z;wnY-&I*`y4&6a8SG2#FcV zReC>IDkA~C3nLf){ywa~j5g}{O5y8kdS6+4A`?U^d*fIcfHuV2&NI+9ul2FDZ5sAV zzNaj-^d;Ivp`{u_M_xxts*zX-%1SnP$Sw<2VKU#u3KOhnx-eI)2SOLFHec7Ps24=9 zS6L-~b$SoYX|l@rA-HEIiIERa_=}NpR%-&K%|vclrH|Dp^J6OMb@+FCw>5RbFLLz3 zqvjX6eBRM(tKA~>l4X)@lTFyj(_cclE%@=<*+u5VE^@85i=1@N4KwcAHFj}xGR{rI z-`mQw+fEWtuIZNSw!OmBKLbB*5Nunq)=OGl&D?s)cZ6T$&%1YzGM0P1dli{mbX;0l zQ7*`~jv4sT)`d-a?fW^81b{hM`fmj?^0)Mt1}ENvNi$DIIPPldO+c3ZP=JauE`zL+ zY!?@t)A1EaF}2f3tn&!``mGqjLUWmaRZx#h+N^3cFQ>>Yc%4m!XuLY$R_%S^JR&i2 z%cO8YAMkt8Ed|-_=Pgd4A6xO`DEbi%s>vgh^ii?EjF);)f8}PFNEtT%wUO1I)bF0s z&YVuA5bJJj*C(ZlN?x!=2Riu+lYk655lWgS=*S$w>TS9SWf@viFDBe2fnu}un{fi1) z7l_4>y4kc27>KXtQ{l#nls_$@_L8eO^Z2+DRj$S{KbEaYsK{7VqAAI?opzno&) zmc-#HEiUOKI@WZ)o!GW>y!?+BSgMk|U-P|Z5{Od!v3o6kn!+SX=Ly69KUO+b1tjPSj zDSWrO|1@%{f3tgU65}UIxI9M< zGv4l6#vR^djz(w9=ZmCt>`in^-eQl`#-_!S{L2&V2HS#B9BPwZQd&mVzwyIHy9U(FX8VivlN>#&-3EN-{tVk$Vm zU<>3-0U09Orqxf$w;b_a%mCsPz2)gIzAw~U#+ogRQzJE+eeRkLhliiVNUM2G61J27 zhcEIPZtiovfBx-<=RUwZu}+3lkY7Y=HU4bJomz`e7KZ;Sx>5rq11^9bRX1nNPYZSk z$j~T??`V;A%F6UHZX$?K4=Cc}RwSHO{PXV8z=+aaR)=Grb}ohlBaHWf7XdwWZL%jv z!0mXSQiS)PKo~F1d9zqCJ%%rY#3fu z1?oA2F(yQJTU@aN7ZB=Y;QKA))ieYyke#_`=ZV5_)RZ#giy46jrOW`!)T^yuSbVe0 zx3^gn3(mYd#esXCsu6&dd^xgbvSonzGOluR;<6Z55`5Ie!|!bX+pFW>DA zdS8HN16Qh+4|46n#_I;?!uN!)qI67PN{5|}OuW3D<#wl|%tvg++BymVX4S8@eO{}1 zE_dRXrG!Ig{+fqh-n*fbY)L*ndbT8Y9zF23McR_cVmB$3qz>A$Asg{<)6ogwp@N2w z=cMJOcWpQ$v1eZ2t_`Q+uRensUYe7Vx458THU90Ct}UBKL_+1qWj7QbJE|?`7n#)S zt?rnI8$0g)*U;Nrvsw=y6h@3CE-LOCLB$OrUy_ck_m}^*UrW=iY?mKhi$2=x7cqcH zM##cz6+CoV`%1hDXgaI337Us4kHXi1kEwwzy__|p`=K7o&)S)BNr$*Z>= ze2LueedxH4#vEqJSqy$sY}h4rpWIINCNThvmSpx+K~`-e`xOE9P=N&jJC1hwnA7u# z&`g?(=QQd=#>@h5>ci1LmhQ&#%XRm`)Df3>P;49u|BWCuTF=QHJ!Lg11Yr(Nw=oH1^{$~IE5QN7RyX|oy zHc)ld<0#wCKzf$71g{hc_?s{%@Enok7U@Yx43zqzlzLc6?o|UnCr-&D+zk>-QmI0F zKy^IGrt`7H8m&(_z`Ja_=&P!B7DoUjU@=|t4)0w$9#v66_6l~7BlpgWtHbcVRy1wnt%h9Zkw)uAS&d$Feo5JCC82;Dig?Kyp7Bm|-EW#Hzhp`m6sk$3L z5!#yVo0kYO(3c#FB+fNBACnF;gbSG~_beiAj*FYF`6KPrqgch_i`&dnQaxJ0Z3WB}HK<_*uOf!VtDx8G7o*$GgEzU8sZ$`aQ zbe*+B;)16+z_4eSy@GC;^*u|mR_=7e1JZ;Kt?VXoh#|Av1_De7RehNXFl@qb(=nZS z0gy4+b*T0`RQ@an#GWGyw5Ys>G`zP%P_RJx`TV{brtOh?k{Db$#r-sR!SA~xcFxPL z7xV~xT{*4LQ@e5+W~4kNWTx^U%LQ;HN;v{4D!b3PISa9BNi4zEkD{jrB^BIrQXnY5 z@?n-oj2d79-4rfHJ*&31vJzHZhHWhsi0az_26Yukj;CGU(uCFiopR< z0bpwdwN_xcY{cFqjm!n@3xu)g$~CSqN0Dt&9N_DFm5iX~DtK4}BqQkChxgR|lG@)> z)Jc{bS4d36OL~0Tv7X-G6;so{$WR$doq;5#+=9Hh?~m#f$NJwzf<_&Wz$NNH)9$I% zRZgVUkom$V*~wWuW)*<}+Ao%q^N!C>HU-XNY9TG4zU#63{S&bJVtuP|M)LI!V+ZGC zu1pAvWWFuC!$oa}m}M)yF&y_`Rou^uB9%}_Ru0ao%r3n)95=HnZu+WG(N~~;TpUV1 zb8<=7DJ92^kN!R`{-Tul^B0FEoiVxO*tvWlE4MNoIg6$ul0VBp?1;Shjx=r?q_Y2dffo_`qGm8BF}|%*eB2vTQ6H zF<w-`<@VO^OQ1zU;h67#N@5gG4kTw3P*!kKs$U&CE>eIEB$* z+dTZ3Nk87H<6T@L5d%?gtT^RkTr&I5c&0Xlra>D`0ZxuX_5rh@`BsE_i>mFb5~q+q zpWZr{k?^)G>*YV&+JUP3;}rILO0@ZTle941hR8{LTPqEm}fx7Te^Ew($TTq+wi>Y3z*9pX``V7f)eRgV{<8iPTVbs zMCfcSQ5+zvV38*|3nDR!hL9-DCWsV??}fE^(sG%`m+q~3JJ!uDeb zA_Y*E_`_O%RcK5ZvLW(gOo?m3`j`{n_}a3?m7N)mjqGUNO|#y5H{Mgnc!^?7y=8}V zIj(k3O~XvVDR5;JT*s2R>J%tBHLNvU+i4qfftsnyH@&S+?$U9L&k5*EA(CS$o_a0} zEI@EI>U3D+{YI zfk2n#Q-h1x`KC^hK_TLD-&uPNzBZLk+dqRQjRGP7CSb9|DMKEE5e5l)A*!VH+MUk& zD}iDL!NU7dzQB5^)rkleW{P|86tiV^uiy^vcnY{SAYr!7>7*hJKTYJ{Xt zp+*vu(!hgZN-vzoV}nbq_cUf+%EhveSLYiA6oCp2%K=wjiigTc-G7Hrd`y!v$bLYWkJRt`Ti3SMXNLoSv<>TX+I*neH#%Sa2BXE;+-hS zb@FBU7%#MSTlGn)Rb-=2kq5yrbU%YbNkA{Qa_UZJgHcW@tnGqKyz)y@=xjxPJX*-& zm&yPgxwC*lW)ptwIto9&Np=V(QcgKjQbPd}d78wTs&RJx$oZLBKJbi6C;a7*H@~?p z`Cx*t7;?{(75H5aOlFkl)rk$!azG7WesZV)@@*3|8bE^sa4L)7yn81a5Z*gBh}XH(>vzg$8o1~ z+bUvOf^V+KU#5%XFB?kSdm zh1!mEGFMhyoS@X3H0iAjvEzDtj;olWP^8*rlDTnqckIX%7Od@F6O~pZ$;mZQxc!rh zciX4^M1;!vE)yOWq?WCqLX%+Pe`>DNX6!BNrksyUG`$O;E`yxQ%E;^=ToyZ!wZn*{ ze(UMX7oL4aqP>YW7%%MH_OQwwX*}j%V^ss~7w9Upm$e*a#L@U1qN~J8;Y4kUpr}(Y z%dRo{e!c$T@sf|ZWy>nBcx;-BRKhDzEKcd5Pt{g1^k8kHhw*?z{+Rr8qS~^_$J3=o z67!Rou^?g?{+;#4|EwoFE-5BrIM!-ILKrJi<~=LZNJCmLTG1(VT%@LTf%wU8b>tEJ zvY?aKVJCR72fqN+4*fNrcM83rq9itQDe#tsxnqEqlr#Wt(NgCX)?xt%+)|ZVMa(eA z%s!Gpx{9jAbJ0U92`!(I6@*q55yvL20_n{~jBU#g=z$eu4d93WPoceSW@>hr$t zJG&FXW`yry!fQ3wf(za(01Tse4G9GII62l^Fl+p_Yq^BIeu^IH3QLNuDSU+odGLMQ z3~P9pc~}Wgqkih@P6MRlQ$b~cSe##DdmZ;0?3lrLOW*bS1c}#H${wy>Zy{bpT+;Nz zYn2&@b2d`e3jy8;k@ylI&%G4OOoQl#;RSMebaBb;vDQQjMMHU&9BY!BuS{ykOkk_) ziN~AnEWYqS%+#cUqX74m*T=@fMGAMO$TWg=hj^V4k_2?z62hM zA;LLZLKjq$gW|edC*;g6%AUR@G;>Qu`Sylm!DW>+)_-_UE@7SGxkYWWgx`P0ZAEQM z_{WCNuFw5rhJ+^12nl3+ZOLJVDqmG69!Dz2bNjr{HCAE%xHDc0ox@yDP6Z!NeJwP7 z)mwYNZYTKiUA2z~_bSvRXEz_yMnjC!e~ta}KnKq<0&kiv6YLpJ!~qqT)g7?-9_v55 z*fngKb>en*+!$H``cSDK1<$2Pts>sbdD*jgFS?|2t4BuwMqA9Q6*p!G)Bo}$2)?{f zoA5T{ec$P6q6C;wTN0d=5XXt%ScS^!FW8W5_O&5N4zwZJ6J$ejC)tqXkBqe;A(T8% z+L8bjPf|r|H(2;Pc@X3ATM+`l8Y_R~(gH*K?5pn>BjPUjYZWh@!r8)pR>FL@>Y_ay zr=XE6zavlqS-kYJ9B0kiKip^y;c}qeKXTUoNk}THAf2w4#0zJq!EInf^=+}VI zun9ljzwB1niuBEDnI%fp$S!h8WdY@>qpBmD-3dVLB=*e(uWdPU-~l=GI#69ABJ=2F z!@qQF_SY_cLijO>_F)rC!U1vg@inyLmp#;r1Hrm$2ejON1v}?c2kn*oBQF@DX@Wi9 zt+VYSPJkyTa@Vqi-Vu%=vceyN%dqZ1Ospa;9ek%GdLw01_j1p1I_l-{-}dyop_5JdrMQek^=ARJw+yYEERTEqQ?Y#PX7j=ZGy%fNX- zCfTG|4wZg)1ZR?|^6z#hFtXtT)VRvH+DOb`?0^z@LC{c26Jg9sIXYzN0Ly{rFeJja z&&>>}V*r|1DcQq?^_^=(eE?6O+aRm*%(sXMhfC^3K1R(hy>N(A*DdLKc$`HI6%`um zp&EaX{7tpD6C{Pv+i}SfhJ%^{=KP&Ph&T6o7@zdp`#c0-m}bEGxI}2YJcbSn#-Hx( z=x-S}8Ny-`Vz1lE*C>8hb3C>Y7A%u~AC#6@PFx@;RTinf3BFuSm7?0A5oV>Sdrb99 zjhm?Snq#_R8If2b_9lh4GASBcv@#2xxOh{UzH_Z}R^}1gm5?ublu2P=by>65f&dMq z+2dDDZ@eFr%rc^IS~R#uZMMZU8m0Hd zmAx$u*1Y=lXZ?p^kJnR9_v^WQ%8yjP-KDhTk z9wW_f8;@;?tzM~@l@fab_jB}_lv{}nF|M3{n7?ANVnL!@0$H>0?*=lh6Sn=f$ zDQ`ri+}e1a6nLI!y|qiE0ZCzLADkkc=CJeHcAn zdEMmGEQtDuMI3CSQ!pz*KD?^pBx;XcR65nltk2?h*>vg(;-?1WWsGY)G$$+XR}{=P z_V`(5qZ_J0K~|}%)if-3R;zh*s=1XWHvVv31#$=;MN$|Dgj4DsDqhoTI8LECDw9t! zRT(!fKXOQ)+2rC%y9dysq7{ z9w1B0JH$Qo>R4cD@2H_=*cB}Qs>&sJGwX`+Z9icn|CS~A@p|p)52B3lbSwr?@tf$4 z=Z(+`UIAcUA8b62OSUM>0ym%*qv6PV+d;6ySL7)p%c!GLApsMSonyvYE6&=p-uPx& z2Gwj>K`}xPiz>@Zy%H3X-%EkS&W1L~v@gvfHon3iZ)60WXrT zH3K>k0KFQXl2W2lUvdttbs((*IgYiZN4J!O+6PzDYgXl>icR%sNHb2mdw>E4`h_PT zA>4psGd%W>p)k{_&9<9yG7F~apbTFKr>=t&u!i@sFFb}G8cIn|HNc06Yoy#TR?<^8 zJ)X=@DBwY^C6v~3@*ZN?6$f1Z@^7FPF}=v*-S#oslq1aVyXTZm1x~i4n!ftHa$Zz) zAg$mWKcZ=4q47IlwY*j@I6;c4L&@tY-GV#>mHYUiizs+M~=-qs0qPNiUSIN$sXPmBW)4 z^AX-5=2sifN49&1AQG4ODor7X3`mgyy3uy+`Zc)SyG;|x{LAL22QORbOSgKD#n5K%_6itVyTS+RXjqR&E zLOg5O@cuLPCEQmy1>~T@%DYmyE(~F)eC-*0Cu8{@)Swgy`WA(9=Cg3pC3G?97NwJl~utdvmRB%hD?w3L@EAOsvDD(8f(SJ2X z@v&SR?fps*8Eo1>udS1{QQN-U40X(o(jYkDTWg3d-6-{&z!(hhbX2ASX9n3Cl~{Qv z0*M?yfM?^azlu33Ul2FR2oF?@2N0)$wt?6vJdzIfFoJH&jD!~sW=2&+$iUmX z#!7f}QXMK{2p7^)Q%(iL7IPIFBZ#?(+O^pX$2A@nth2^@+|k0V*Ao#2Kg%^M=9BxT z#I=rZUHF0wg~2*eP2`#^nc-GnR)2qCfA6b&nlG5C7}1}e=;%;hU(Tt1FMSm09Q8Y` zQwi!d$0;Px4zs7W9t7L)u;HHLF6tj0*Efo{!|)O6dIaN50zKP9J3D}0cOR8%8DDw5 z3gD6^q5pJe1JpP>dfsKzd*g~<@jutuv?I=cSGR>SQ^eTveOOq2|GO6~diTn-u??pk zXg_Cf{?NWB{+Pe03~sQ&G#g zZbvADfk}C0%x~!HgH+T)6=Is%+EWwi6M$BXrboUEPk=T~3oANSVk^N(89?_q63$$XuRx zA+erHs#;RL+m5vW`w-rBw%g+!&Vz?Wnd1?ZWd#9e>nn+)|!t?YSWI z)8zQeszcSMO)fb&x#aVr=mT-_$5n^+51w4|+2oQ>CnTh!&Gy^NH~p=qIUa4d`zC$T zCvoYdWt;v~(|jP5bkh?9!oTR>u$9_u<4;=N;{`Bt4F96bj62SuHro+PE-CH)$uSZo zZl!00Bl+iNH#D6RXdf*KS4*gV{z_M{C#*_q;6k9G&?Rne_KuS zmfFcjJe(B_AzL_|2lkR~IGbLoP}7Ar*B} zF1Zq3k<-QzHfn1?mu;Kd{{iVc`3;LCU#(7jTj(Y`YfFv+J{i4vN(bOmfDZ{_A|DI~ zACkb@wo1lr0;+ND*TQiHPxDVVZ=58*vdK)64F1o3;sAjm$^jD`pevNv}Xx_|JBlgqc{rx5u#;$l}g*#UDJv(g>3b} zjO;M4-jG*M98Du6m(@#DuseW9@G5akd#k?#$+?hQr%%dk`O^AU>5~P%NE`a{auF2Z zk{}p|E$xeCafw67d!#1%jFr{hro2dAXoR5h&MO-#TK5!|{uiyT6@&g+aOVCqYNJ(8 zBn&yTHhO;_z!g$mUSqAjx6_8ld)^H=RaLQtuz+0{;@GcJbs2!rg%sT6wd;t!ZdZFn z8;F``hz&S))8llnWTS%Ql3deZ3WJ3T3g2v?k6co^zoAk}DSynt0+m~F{N~;pt>gAT z_v3ARVEbGB(mFn2zAZ`=9Lm}LN)((fXTv6IGB2}HaDrfP0}B;wu_jHz?1x|ZZTZ{u z_G^G9!rB~2qF}J2{*z{y)O}aw!L-04{uHc}qtC}p*PNC7&5)Mr7b5T+$Z4U5Y*H#o zHve`Ll1tOk3(Ck*s#xPn*pKST3iO4J) zo|@L1%b}`@Rt2NvPs6dvJr*8CUi_2o-?)zsh6T%+QK`;Nq+^bG4(PCc|(XbwM6&<62&9CQEC6n0aTzEuHVZiJrA=UO+JOBe|zc-q< zWC`OD%*z!Q?%Uw94{=XCG@H>4@6-#f#|@{*QyA5nYMujw6uq+ThsT#WI)OmwQhx%2 z2d|Zb>MJCdg@Lzy74DctN@4{z{`u0LACE_1Sd`j zh)mxXnjSSyYCclV5<*m6tZILNwskg0-&@vnaL-y)v@PC!<+4rLD>h|4(7Zf8>63m* zZ}dyN>WXEXdau}Y;teTz&liLTWH)r_)^pdWp0%UWHvXz0oLU$@A-iEmQqO4xKZ)y7 z6uG_E;GFX8(o4f}dn)7pF)lKw*Wh(`4bHjihUNLYr}hpqi|iL*7Ws&WS>y+TcI%L? z{?dk|epq%x-Kml0=R!#xyB3GaZ%l|(qM7!{iIFX6rad4WS&6^;ha;8fsXeTFq$~dI zyor%)`>C95)`-jn z1kRoXI1^pj%GQl(oOB!Eq0mb!TYBbO#*-Iv1=WF>kuTZRWLN&c-UT@I=FJZsfd|PK zfH3|ZBrjWDkG!2baTHGFz5^6yfhEC0@G;Vh;c4_mPH#L=c89P_@$n!r%P#F6Btxv* zYU+(0H7SEEJxDAFEK+;8-dRgFWeXD1z8)ka;X$%NdXN;yqr@LycLQRgDl8p=aOyfxXhwJzH#FUHi2TJj%ktB=kzHg>bg>1!W-Gbm6F8-K zARmuN@Id8>+qTq1%brN6K7*cNw4zVdSEXYXRgC**Vd;#9idp|GT6fxQNr9>FD0HVK(yy~c0U`Sso{ z-#8js$rwWg7)ZBu4ln&^=Xm=HEh1&6GaeLMWejMnJfnqzyT94$redNmCGzBkjy9gn znwadB&nhe(G{@*VNfjQywa`maH4v88?;xWG!S=8ay4jIivG{25^P{3Y-O!I)xEl5Q9CU>Tdm8vh(gJclR#PA*Hsv(Mq z?Zl>{ccaU_X#@hi2T`A-cpm6GSSlM9kes zu9X3)xm1Ex0?8kVVG;OFTJyc~HVCc&=H&KH1sX>Z8-0`3Jek2&}#89 zk?4$GqawwfHO+%Yq~t9uXxIjpZH|yoZtjy<_DGk8nF&4Hv%5xCrr@_bCsQ8ol$n9y6d2Sseir5~u zn9Upcc;garjD7O+YZMu$j7o0iA)Z7|k}xP-V)p#M$wG9vWWy}#1W;63WPS4uvNJ$Y z=|q7G>jQ+7D0OOz@h91PU_l>;auOf)FSlspXq1I#NSf37Y7V%>${SGh27WL&DihJR zJ@AwHI{IK%(FN&9V2a@tlqI}!2k(jJ9)XS|uqvyay0ew16TqRmH|Zfs{v_zC&4iU@ z{O{0#7`_@#A^g7{+hc6yj{*Y@`B|8}!(x#Fo9G);$11OhKzt=j2sI<8ej^^~HsMdw z0q1{;!8+h1S-#{_ilN%JuE}nHXMgoD*lA}qT#hs_Am1q8a0*_B^wQcRU@@jTB8JzY2(%SVszsCnxY@P3hHr#Z0*4!8 zvVx~MTJtZ^qUh_UBj|QTZ?!85zafiWKZ!;z+y1PSlaMUh1F_+f4Hr2H1vh-S)y^(? z7Y78IRj$(<&xM{vF zwX^fV%bWm%{Js1~S9oyJ^Ki|YLK}9Y!e)pg1u|Acsz;e~UY@muG(PCNW5(6$fgxUq zb=MI~2oEr`WdM)9c+IEOSmI`|WfJ<>hfbwwFq-p&WF%DNNIaHX8A-3PSfHgPk*Uf; zWCD>KwAAB{wbQX%@!L*c7oe}l*M=vASwLH~40dBtF3)8OcAM&cLV=s%$q6w0#m7tQ z>*`VLHu~UED|TBxZ_Nfi^Z&=uF1xm%p}0?^`FEh&c7tmBpM=PU zw83rUB>7ywg!CUSYyJ{WB-QseKO9P0y6E2KUxbp<7d_bgZf-(4M3Od_lHx88G;g>k z^RC*97H@iWdGqD>+%V(9e-)`&2+NCeyZIifh$71E5nsv;Bo+5PtM#Vbiirj~PS8S` zJ>HC{E=MI+UMpE7bV>Da!`P1{}Fqh$^!AD1rf0($SY8izI#mo16DW<@Xi z&m@M9D8IkqRBB>1g*tXsm}Aw;;#Cl*L{!dwDohvw-=8ayIVv|sZ9rNON4hBsv7PBu zq!kN$u|bzn*_7;@>{QU`@Dt)l>KSF0^8AK39bAI#Rcwv!NKK|TJss`k-q%82|Mip$ zq}sM^-FfRRFRiLT8*LJkRMd}@UrT4EoiIYj`(9Rh>QV4!FSp0A+7i_bXu}6J_#M?G zz-SzG7t+SuwjQud8k7uK|3K7kAt|{XlTZ z2FiaMLu1lm z{ZxPuO5|1y8Jq%CF!I_0BFbNJDny*IX$l?dS)P3=t3V`)Af{xr3q9fnv*OeWJ>j=N z?<+0NUULeRm~21=I+)rl#*Y5YR{fL)65;`<;}!SH@H zL%S?t43F7}3Cb=No*;ree^I#>yxs<+#(;>dB#0CJhWDu^bWlaMir3T=Y(UJY>yp(- zNv*EP4$NQ0K%2KFiM1`JSA##7TH;f2jC zKy^xUt}u2%5HEp#WGLpsYiflHP64JQ-`P0RSL-j_Hw~#2V!DI`7ox{60ty2y_f7ZQ z*T;MlD@wrdaFie$!0{84{U=P=m#3=WEd};9WDLY9&X~z4!K2V3`$({_7bh)OgTsH= zyMX{&zid&cdJxF6Gf_pBc-2Js=P%#%YEAPyC}-<~KMAsj z_R*C^{^+)~PrD{^*55ul*uIZ&+bL%^^kD#T zd{&P$g~D$jA0y*6Isf-kC@3T{F~r)c3*V{!3}{B^6I+B}hL4d$O95_z%HhicWyc5C zu$c?t6s{vS*}AXIfkZcTV+RsIBOYN7JAZU=n-mDtI=+w&Bq(2Fq--i*BWjB2I37{x0F9;*YGzYw4xdO|P9#H*uG)J>3dVlER4O$gX0I9@Lsnva65#x07tr8>f+n*yH2-l%M2D1pp_otJm%}(Fz{>qf?6{yT1~z zcId(Z;y3q<-oUKYul)#0m)I+;J&w7x&OgNe1w&-eM=~*G=_4T^y}jg~`ibgG>sVBs z!d?~_qgi1(mJqZ012{9dxFVLMu!S0o+YqExF)ig&&j#Cz&ndFfwGavm$Wv!=oGlO= zLjWhnCbt6U4#_Qq53d z$}m|SlO*!+=u=o-Wyo$QJiNctM~;Wp0vYb6{UeN;F#Ic&BfOO0b*el5{QyPePB;YL z_#&?aYDgudR>!pP*EXm+o#=$7zYMOoB#}KM{%^ZAX5Ki@Vv-YliFn0E77(h1U z>fdE=M?0vXss043RlM*SN4~(5mCgySKSG4SLKVg#xREOW`0x^i={c{IoRnt7S9jhi zgxW}{}2AO>9>545o!p z)5<|QNmX8ssH#o!R=mIIubHLyJIQ^C+DKj9Y9CigTg7G-1QSY)Z5V&s?(aJ0m)4L9 zl2Sp8nZ2L8aa|P-CA^i#Y;SST3!mR>?Gx5xT!pm6#cf;1{SIGG_UlkKNyL7~sp4c-X!yu-A}t!pnytX|_X4n>v&3 z&q~-ly<5-P!k*I#)AC*{2#+ib56^Bmqg&73V|wlyeL`f=iG$ZwWS5qN<1VO*JEthJ z6P0X{!Rx}=rB{aIx~;mnS>eu$WehIrltyRN|WCOfN28Sa%QOcIrJ#slp*+z7a48vc!6C<~y%q=q)+m739sbHAx7!$d_dUFBQbH z zvZ)?VnAJ|w%@}_Y=Fj-^DEF-#IwXo)@mMbPqoa3*7_U{>6}cHJ9q^{%WWaEWE;&?p zjL4`ljN=^&AFWPB=M_-StSnWyjeu$+6x@DxN@D#9{M)x*h#ynv$Cq^`oq-&`@1(0N zq;JV#7%r*%UJOW9&hsyDBbiG8)z$*4O&4rGYmsPIUCbof%kmi8Z(xO6KJ>^2qV4KC zEusyC)5xcIAjT+T$p5vHy0_3NK*0|KQC!CtFGdpgZ&?`qEkB@1uZ5@WzC5QTb$IGk z({fr)9-g)ac(LR2Adb!2f?Z3V%1c_?xOV1q$F&@L2XSmnO2tGIdYiMVE%avTT%Czy zv&;;f@xKo7|G#ZO@>XFUc2b!Fvh8n2ri3IHFc(wgL~Hb3lM<2)0R5DZUyjNr&w^MH zqdFDT_nF*0**;KGzJ_p%7M$%s^svphUJPdhUMRv+`lVxY0-PGN!dCpg!gtd^28mPl zLJIfXmSGt%n>M`QC0F)?)7J}aXDmgh;xgeaA{{Pd^dp#@EM5(93Rd(2ZlwyY*m0G> zAa=KUb5KkU@J2+f5%zyGErkcBo|yqgNX{vC!ihO35Vw&FN_a(xz-%G2e$gHL626Vu zX~UMz5Nd2KUIp7G8+Oy<3#|r&2Ba%HRv8%ya9={|rI!kKM2q4z8Jc?{W)y@GEfqWQ z)xGeW)Wrt7HX=@D%`!hL|FM^r3!6VCb;qS!Sql;izT`mKC%&G8YSf5lzFF*g4z@|$ zPdx&F?)40;9iwYo7uZ|`ggC!$&eRE+2#HVv9wlYBF*oDR+$e-QMTkhnNO?R35O!|r zJ&WF4+kn$2b+=d`_lGH*2E~C$J9yrb_9J&5~4A=ic(Ng=4MWOt^gqugghorWD!?vGgmL4CD z)cvy{+7n=M;u+gM8`AI^oASTWr^hFPaQo}fhQIYoy{T`H`}_7NTQN1gc?hANvS+$n z4eU`-_tE0$Owm)j>jl~E*RWsYx_PHeJYebme2UiSMt)xZx!ag=@~&A+?f#_#SiB&_ zV62J<4;%3nD;3{oIMrwW5{su}yds_zrX|$>KHeZSUGL`vth_qk$C5IPRiet}f9O|ICy0L&IC|6zML3yYkH z4`kjEVPAh_W+_HqRgAlWv|>~orNQP2%SM)-c^F=Nv1e9`J<|aG#K`^0{vsP*besZ3 z#@!*!ntox=30skBkWs#y8EC1e0>lgQHm=oHO0iWgto(R$r*g}62;rJpjjvW`Onp_^ zE;C$?ml|UOgGfq)#0DkytxSkVJO}riySM$vQ)F>L!|FL{d8c%3**qc=GI)yIwQKC+ z=95spHVuF8EYEIRDB7|egptS-T^i=WNF)c&{_ht=d&AlP^>FiP)OcXQ`gyVvD(997 z>uzD$;>BA+Q!5MC&7F{Q+XURDdsM=+ERy$4np@QNJR6Bb@9mpb_CkN+XLgM|VP%^X z>XmqV_+N+{L{1Ql&ZvR4qgb8SkB@6d58O5SxeK>7<-gJUY=04CXUEk4?tm+v^zbks zEXW}1XCE6CjzKX1PWy_~52=mrJWbGOZXm{!yy8tbiyZnajeUjOSEyqWqwIVS3NW|e z@?8b-&;pGVV6P779Qu-|W8+042C%_jr1KANY%SiK|LSK0#=X#|az*na|CN8=-#cIZ z@$BA3_dGLx+n+89U)1-`m(So1SxzX%D?0GuezN^jZQVw*b!BvtjG#nC&hkRX}-Bk|=HCfH0!HlM?N+47#YD23@3 z{g$^9r_dN6Ir;R(Q6E#u<5AAEBWd8qj6sci`2Xhh>w5LP;s%GMBw*%0{2e8|gl09i z?rnlE)1k6EfzF|G5a~uc@^^|a(8;UZDACagVW2hItc|{UDcpOG6_xdu{|)`v1q`GZ zAhw1j2Q#;IbOt)R+cLf~_?0Pk(Yvkb<7#TT8ry-JR3)kdH)8(%*G{1Zj|f~mYTTd# z>%8i@_eq@Uh z9`;1M?Xj@t5&OM}bsf=yZtQB2$mWQRhMS{u5gfXwx=iEa3%klVq=mITUKa6yCSaO@8Rs4=aV+Ov>`m;Y zR+Q>l!IjRIgoxQWwJz|xK(n$~LVefQuHj>hgjF{wXVt}eN*t*GRdB7}KKq2b)kl9Sh1Da1Rn+*L02Q^k2}V*4CKi$+rqRXHEUPmO z*)+XcS`r$=av)FH2>DBT95VKWJ+ihH+=kMq4B(e#&5HWg_Al$v0^rS!zb*)mFAR^( zZWsfj|1mvl$E0n1p&&f8FnreEGE1)x$6Z_$=?bmny@PY^%`W{(IIe$H+^I#8l`#6h z=l7xG7Kipj9@$Y8{bgMItI6@dsSZuT@z;|}+6wj^#SQ>1-*4mXb&E0^>cGHVUXYS5 z8vyiJc4ckyOR)VvHl>1^v~{$Pi%t5p&v-3#P8B(4%oepuVW^)<@>AyO+)_(*%*2=lG4L$}1bLc&k|N1HB&5tFbS(4QP5_1V z&&xrCH-6lz?E<*}TZCg>9QI-0x9V_F+l`t0i|ogb_tKC1EOyOpYNC$e71_z!P3zur zc_8AC9|Ntm`AbJ1QtdP#x)nhF@X=hkl44n ztEQ)r{X!#Ev%SP7|Bd8Avcu{39}wem`4_vU$Of#=C^g)6(Xn6eJc7=ym%Q^`z*C841HQ2NhuvkH*wILf(`>9}>DrfcVt* zPMeB-7}UC`Ngb?ZKEg?Kd$6n`R|nuou%c!6$g5+@k8vn$Ul;7ym>e`g=kj}IoGq9@ zsPqd+C;Z(;6X_1o7URy7nYh9LCIg>yBxRTp(Wl*WTm`{k9x=S=SbbFQM*~HgiT2a@ zMTjSlrBK!NNYnkRW`^gbw$@!s2}u08_71#0yx7Vt7VX~7#dq-5dppKitEu}RdH3IW z;tP5E%4Ow-^`Gr+oM{!A@(A8S*2K#HV_zHiw!gjaPxz!*?#8~G>1LIg#9>*lxD`0@ zDf?X<3xU3heqZ;7$uL)6|EGJ)+5;3NkksmT*^bOK+J`-k=ggJ(7?;Qlm3ei6-~skH z0f{g{Kih|t_DsP;^IJd|t=_13(69ZsN`!^fRMa=XDGj(do z7r*(WEb$`IJ^O>&lD&+m9$A$5O!tM?d|X@dNo{nM=H1p-) zHUwLdbnui@nnsxdqGOKO22?CKr-S_S5g0hD<@7Q6{y8<2$meE0~-`^c!ji?NXQSNy8dCy2y^hAqU*rqB4v&1QSWv2*6i%2ydtZOaOD}9{U z)6zdkEzA>sk)^tecFqIBIWlzhyg|$Nz|rI(wukhxi-gbbB<19PgY^p~58_1ft;m{E z-|BgJh!$conyrQ{xQTYzos+{AC-uUZ6_rlK0P?0Zi;nr!6>wgxi?Q_IawutFUe=(- zZrujv^&iKqTz()vgDQ*mR`+)Ew8m$wCt2~>zI^2NWr-$5r#ik60j!9r^bVzBEILKl z)%_8!9S!vfvl*^vc%qhInHdwIy-wlAA}lf>naBBSM{p(b*)_~RF{P$M<=A_W1dj{D z*bxcEud>)J|cQr!P<|4hCwU`}miU6*aVnG)%E5LuJ*WwG?!c9faHEe#CrX zucjv=-+C*Q*G}D5H#)VyYfvRmsYG_HA%kYmgrH45pST+SObs?HA*COChR0)yVr}dh zJYR@C%jMBk)Q+&zrCA8#OTMnOq*|e@nK|M2YrmnJP}npV7Lyn*;=<0eb~4qm|jQi zBaNx;iGM|C3y38>0?jNY%!6PVI)sPLJ^R%$K86&Xt^#IV>V_6zUU@OyhfP?@8@{YM z^+SHy-Q%R*bQ(I#QeMw%*&|vhdttpn%8pp|15WYaQ>t*bS;kjH+ZFv05E?PjU%kGz z;-o6PygETC@{@BfCHk1{KrxFohprZeY@y=-tipS$UP{BB23||piKqe4TUz+HojDW} zQ1^&K$?Fj$&|K9+=U5WppVQg#)U6$JABfGGse4mbjHcG^5Qn%rQMmm^>L*<<#VE!} z!!K;-p&$NSs>D2!R!Dc(Cm-{*P_^J`n9(x*=6^WLs4>xFjp$?l`fKs+jIjyN^vkw4 z&FknWF6lX{_fc!BojGCuX~!qNk)E_BJu!XivP~Z>-}KL#<^l0Z$EPRmp0aGy|5?82 z?~kVBttbdTHzxdS|AuZ!J$sMnxeG1W>I%XS_ixxkg=-g6|FtEu|5|qG`JyFT*T&C( z%4Q_>!$S@CqK56gl+d;TlcQujA{()hz5fwcwUts+?QLkPeFos!js)}+=-Hk)`K(tq z{{T?#m{1G#VarD!w&GheOVh%U@@+NEb)uzq`5QIOO`@u;JXkdOf)Guf)VEJFSge8V zk9X&O(H)moR+Nj5+8s0S<2$KD8hx^K!BjZ&$ALMk^-)chQUB;u1)hjZ$hnuvv@v{~ zGk?Om`KSElAY+$)J>KZY@Fnx?hAf6JH*vKV*52h%UNZiX6F3Gh<-W600VWx=@-Z?! z1$`KKwUk;ODX~Rs{HCM%_pA8?PDOc|4a#W~6Kv+zZHYqTPg*tb?QJfdWa|T`cFw$w z2rzr4885B!)JM=s%*$TPK4Qw_QCo#DjA}ZT-jl=QNsC2jElCJJ^WovLHq0Mi-!-cx zIf5UTV~g8={ZzSL;9`+wr{@Bbd3ii#E!KU0x}wM0*(>YA%v^pyf&GcSPAM_ z@F4I+4%>L~bd}tl9X4P(Q&(-SmmsTuS>i@5*_Kom^g+NB{l^$F2%N?a=wbtrrz2i< z4F; zmx)jrG5?i#;X3jMiJXEFL?FjN?)WOAaD)>g2T_Q4drh$E-gB}sl}t4`qc@8GeEPXU zs4b%hy!|s>*#bv)0f~05qmst<=2Ug1pUt1$*9#_0cMiLZjGbJ(b%>l1eHrxrH zUZ*l(`~gNI9KPu2S{Z3E(b?cixtZa%Z*bk zrhsTe)Rc>zRg`u>?{_>OwUS~;)U1-45r+ql84$ugr;^tc^?xPQQ^(>aR)mP`3+FCV z1zjxGSX;qRdKfzMJ~sw1%najhbEmlxg>FXbW>dXZ7K0TT^w3_#^jRFl#(vvqc6tA zpE!BRrehvx{s~%OXCz*A>9S32%Qt;h(>y*t>CB9zSH`U#7ufu>6J{))NxXA4={l9(iy4wbZs$uf~@#K<^ilULY_yyhLZ-X5qeE#1rik1SA z{pA5N`F~%U{3kB`k?{Qwzq#>IwACKdAdMoe#@T4cX1S2;S{$lBFVrw5vFFD0K@EH6 zq-=bkpkW98swrsLJST1Asa+eo;NMOe)bP}tl=QbjrJa+R*pZabxcS^j2)2~vahG&s zpU8apM6SCJ63NaYP-EOTyB;kGwhd3&SXt1J|3Oi7_VARv1qBWDmJ7-1ZJ)K4pFEs_ zg3BpVi6H;hwe1r^FHN@=z#hLgx^_f%`+8C5mJAz`t-5!1rSSF$dAa?cvgmWdetu^E zH!lU36rh3xAS0ybLSqUxB@8-5sE-uQnl6(JP>S#t$=z)UHe={B&cH#i`NRPdY|?h< z{3LZuY7zbz&Y;UrLQt)69%~$B389Vd;w2HcwiquVlJsNOP-4c^Cve);K)e-3oCdy% zY6y{K!U)^QJber%kA$6yxw9ik`gjHp^O8mt-r3C>+r6r4EmfStmk#WhT}M9uJ{rh5 z&_K?4Ge7-@42ZoQBC7)(BFl{qk)ik6H!c^z>Rf0S58vKJVAkZ*tO)E83&w7vQ{Sr( z$?P-ND<#=1D`mV=m~Y*LG4xcDWoKDVW{Ai|j{7?`u(_Y&aT_nE^2v^5**R8J&9Rpa zt;twSxQv)LBC)|hDv2H1zOCzkU|}uC5x8WMwKM8}bVLi1u(EB4&P@#kYcd{>1!)vo z+d-Nnn~WJii~?&*0yZ{lBc~3F#*IyMMU5qi^e|fCw5pYWMv!rXjIf)o`6JnSWL^!O zS4G{m16ppsVl0J}#XlX;;%vtz^4uH`Ib$|?3iVuIq{K5agltq^0k|V@h>8)M!;3~0 zR@*lxkkN6VwA>Tng9A8_2uuWCUL{bPT2!2s8VvXNzz)SFC%Yu=P{(?iG?V1s>0}FJ zdN2q=cz&1k8hgi^oe43qJ@G*cuaSZ>x}>avj~05iUB2)0Wu54;?K!*{@%e+3zsaye zE=6^y$FTyJ9NHN`SfKx8|5zEF9VlIr4s#*qHC+foq`WwtV84r=jt;yAK*^)0mjZ#$ z^f9-L#7p9Q&?3Snw2cX~hjg8c292{4dk}yeam?1>8D`esZ7V}f6$RKaH2aaca4zA9 z*d9>ZCOE@1+#G&90+Jyzh+&Dn8Z4^eN0La|jJF7)l2btg54Q~EhkDi-^Tk8+ET!<_1Wv@ zWS0&XRNA=6EXl5A-ZYzi{iCvHwzP_@AC}#)?SxU$%U}y~es;rb{5Ux-e(jd=(b7Jl zwb_%SxAqCm#$Pu-5!(OisAvKH?bY$o>-toXnSaOeMUi@#`RDv1li*+7F>j5&kenjB zqIT^vV7P-Bi5+8wQ)FwO#E#)zBdsZe+qU4xv${sM;KwC>5|)kzh`c`cC5d=K=WGdG zz@60A-8vy>Zc+C1EuooPD$2Jv97_fwjSxX{nUt3ZGWU=0A!I1wQ>R8elwY0NTYJCm zI0)kByJ{d?>x}q{meO;XS9@#AA*G}&BF&&o?%43ym3O~tDaoAwuCFHqP(NET{#qM} z*xClv$g#cwT0t*(gn9{FGLNL6Saq5F;>@JvzxY)2cDD5INa4?Jo0`d8<=Tg)B$i7{ z|B?9dNG<&V);xb31=dQFARF>boA8}!!uM$gE9LsPb! z_McJf=$TEr%xf%gb~|l)yawOEsktp8W!R)rh7N_Ct@c`M^#!yod!!vL)MG2&acoT} zRWs{)RJ6vNlOeOFz2 z$i*Zd;UOCNsZeqGRi}VY8pjn@2JIEc57Ech2uWELYzT5%e7H0pC@j$Kc7Rw_Kjg+@ zn1h8ASPL19)kN&(mo`^(&+)aw5P~7?0zk$@F%lB;iMU}HrgU&g6<{ThvYqacc-aml zGOSN@)69HKeWyGMQ?jlYD-v1MXpta?N3j&vWa@yWz=&hgiaX$xq^2y^f`Fiil~o?% zh@AIsF%o32o zybX-n*psvM3%rO2KOwuCu&F4*j6c zs8X{p#Q`Y?pe|5RhUu3@z?nB0K(_!$4E%@J-!>X@bD|eO4p{6>(w^s4Trb>EP8fYX zIey_W1AoiC#aXb%-;7S7vGpVx3awLgLAUa*B>WP;E8#V^ z_&4FT@6X~t^S!-iS;c5e$B0NQt!1n@1sO<2esva)?wM+I`&;{2E>1}cfmb$P7>ALl zs+=$7j;YE0r72|QW4EW>sSB)Dk^>|!L}ziZ*D8-uuXJvJ@{zpvY4GmErorKoGK1W= zd$J&ZP9gQ!OGCuQRM}JPaYdyUTYJ@BIz{DPqqUBV-kKkjGb~X7 zMsO_!^pC~L9>ka=C;g*Q_=TrY!^9NpAC<49+NX3JfK$*vCS%9<4t!bvNa18GwT7Iw zgIfDa)N%PxYuIdKSuxEb*lRL8G$SH09eWcQI3QLLWR~ZiR>9PoQD2I|G z2x?6gW*GA5anF~7Tf;j5%Ra*sRJ18XmgSxNQQnA+?qZChQpk-{sy@qyaqOmMnZ{Y- z)c5Bw@-9C}{oquH$dBPj9Y$XIkHZKhhwSh$Rz)U-*b)x^$6>@V>5jU?Nap`M^MX`t zfMsebzhb5%x_1*iq*>(luM|5oT8+hLLS*T9)w3RRacdvnTXsIL%62GW&q!EXy^t_E zpoEh0V-u&FAoLOjo0?}=O9?cgXw@uM2lr|rfgC9{rIyN>%CrLsZWm`<_xaweZB&^p zl{j!9e7HB#&TC~Eo#P`AGA;oG6Z@EB6SMaG8@mY$62H|JLQFut7?dX1WecTwR7t4O zwyGrDNh>|XwuUze>IK(BhEwS6E|OpFo9JT;9;@Y5PkutkkxRd5UE0Lx>^DabogKw4 z*}1Yy_OQy#(wuOlxKE_{cj$+P7HoB=Mw*|)k2j&^^(m3&<)Ngjsvm4VkehJT_+`x> z<|gbe;`Z9_<|d>UEo*)&H!=Me_ePEWk3Y_>ulx!4wqqcOJRL=A9sL{X_Z3957&t3RZainGpfsoqa19u4(G#uJ~p$bX}UrQt(KhdTY?39Y%bG$ra*Ga*(0z2B9wTm$K`3 z;4u`fWoVAfqi79^PYYj2S}x%g7fQ3ZO92vay<9M5+qG4sBQXgVd40g6Y44kYRY7z! zZ-fyfJ8^m#K@KtEJzb|}S&;C$Tap$Olui#!L_Rjt;BhL-hCY8 zu2;E*+u79@KuD{4;*S{msu76UnUH=ZZUH^Nf zv3C7i0`2+_7y*RMuAd@ulmVsP(Rqd9YBlJ~?6PPfb0pkIf-LXiD4NSFK3`T> zAJDqyF_Kalw=0FT&V}Jv>;E~<%FJ4GTPAC%K{LSf_A^-@CIN&(1r9(_D?ZDxJIc>$ zoZbCl;Ywz}X~$!TlmrqYvHV#0q&F5uPy`9BxIia#)gS}D%2YNj8OAy&m_ZT?75hnc zphg?9chpe-`hDyc#>%g=9pG?UJXx{cr~tr(v+BrNkNI>g_I&cHR&CZ8F)3bGlAySklgv| zh-~D|VH5^a_-k&c~d{FULX>TL}k!Y;jH+y(oalgS!Tu-BAoF-E`ekOh75naH1Ged_%{V zF8G38&2{&|qrr+l#*?DiWrL~8b= zTikYI%InCwzl;8mhK-b0?2=xG=WW|xPBxJ(N6jX(@r$D;k33V#BPEQpYS>;z`)uhs zvSmSb+w?Ihc@Gsde2ivn>0KMnNbH%{w`;?x_^Z#LhL`4~}ZVK%Zsn2j{SY~&=%Y{Xg{Bs}CnBT>2q7a>L2 z4l78POQj(Fn9CFP~qm^JFt|kGO&~_-&o3)eYMRf zGC#n}pBzL+p$_O0eilEn6go>U#W73xubq#=k1t+~IJ;?2b9d7wy8-`Z+TTJuvdc^T z#c{F}N*E$K3fNiHW!U!Y0E0@QJ((a>sn|XE4M{_Q3u_mzqTHWzV=5vWe7;GYV(L8L zTR?$Tg)$kbA5qv?*OzzQjwl4dcO-E1U2aJud8?Mi>S~jA?Au}=wl;Z(1WXJhz1fx% z;x8BOPd-S^)ExK^kqmc<+py|u>dmP0kphpXV=&U4hvV<=@KKOd)W3cubYHjUQ>!&@ z9CyWKX&paU&Du{g8`)uV^(R5fm8sI3$m#!~Aj*q$(W|GFKj{v)6#gKu0y4_qe{B(d zGh4-qe1s}MxsEOxIh`-E2N<^NxFWNB?ac>wa!);51tbOx$uf%R1^Oue_?UL`6*H$C zh_1I`W<5m}K%i5y*PM;#iyP{~&=#|7h-qY|9p3P=pP6&ERux)A1}xcyk|~SK@m*Jm zX^u;RY)4SOrE})?@;3DfpnQvPs#6c;ATfl+o`_!EGZ1e!PFead4Q>Z z*0=%-uoocb!IeF%j=B0UhqvZ&g{*mhz7bHURRx=1)3}qr{4&QhgNAkWG(Hhn<)P{WEWPtiq9m+f6URZM2} z!aM#)KcQ(?4+jZl>Eo#{!;RJ+2{tw<`VzG_tTmALpv7Kf5yMACIi$#LVOzmJQ*C?)>< z#i2>?A~|+$;_;b@yWu_3UekPOeA4|HNjGIAF1=*grq7peiatC4@WG>p-p>8Eptm>w zHT3q;`a=VH`x%f-y7i+j?VkZ*w9Fcjxq!5i&jPdw%gC~IV;U#jMvx}-(#n>eu)Ak3 z$67B2?&gPr@mo9bS3bpUEt4(Fe4BES-+4>SnEMXt4$T@#X0J|%M)HnejU+qXJ2*KG zhSkUHBf3lGDhae9{gBG0nl#%9k$r@>P-0CdXmc=PWTJ_+)=7tPdWQykB-*fTkcKP8 z@zAQ^K`cS#k;Gs)J|ALqMmpU%Kx6;ph(5; zBK#_<(9Nag@5Ti^XGh0Uho6p1Lg=c^BJ!+nOde4U%- zu!r34wueMFw{Zvkp_@HL?t~6cQ=BV8T~T>u*=nnJL7|W!zcOq`jieq8%%fL}$RqX> zIK1ONYzW&#o>r&D?fFUS!^S#xy{D^apol%SNh7HPX7!Jb>m`>JD+RPR!1h1)6GhT( zRN3}UKi)m45gSY*&%~}A-B78p7lLM?GEw8@=vw>j3a_+zy#q+Vm&N5mAG@sy9?JQU zb(Gxp_?p~(<@*P=>^zO{OghQ^FL0CZa?o$C4%t-s;#D_+_#gC}qzhw*kist1V5}2O zmc3=Dm!UW=xU5Zv*E~~;DEedY&qkd*+bpwf_g1chGMfdb0dvqPbR(clw+&&h8u+>H z(#*>RV&|{f^4>e=`>9Nx#Udo-Kth+?@;HgrvsbX~fFr9e<|-9m1u4#8R*$^9;XmtH zj!U+EW-8mFwMfiYyH?EbwQvg57vCf{oWd*2jnKdblgJ)@6P6Y(Yu-CNtvy%xkIYyU$`^gI%Pg7X3A0+i zkXHQWE^IyE=?{V4!1f7UUpW9ElK)n}mMtS#RhW!6*{^GZNKjJDLWZbvX$$Ht*)T$+256^z5)}z*q znXmTr$)t^|Phia0etJq3S%{RKTvgF@9REIL8C4a<3HY(%Drm&KDKs|p`u~pu$v;k& zzzq!Arzdkfw3P?|rnVhi_ZD;?dv&>)s<2{Q7}SLPOKucGzOR-Y}% zFOSR@;#Xbs9J98{`0oD!4kRlmi;Z>w--`pupO?c{tBg2DYM3p}{PXkBjW8X(*AW3U zrHh^Y2=+EW<+nL#@ZQ{LN4H{O5B&)7Y_i?ED*{n~4iN||!Gs>b+Vcbr!ogW|*cEh_ zb}(#|5pDNj*hDT`?%>V{PVD_8ri@xjAEnXLA}I3gg#K`nF$z)FlaUF>)?S!{CZ83b zJ6VW0@+z>XACh%?%}s3mcA{Agm*a5&Ru(hGtOGU)nbq|c!(@OoJi6>BZ}!|P$|g{S zKX5xl^o?mMMPP1}fPaDI{Qqt5O~9+F&h>E(Cz1pt13{@m5<&!=3eJdu1QH}vG*By| z-b$!rd+kLEYABj;Vj#h)Avk~&0rZLk3eLS18xpKIP@{Fi+LA=7C}MLEH4#Gcd%tf@ zd+)RMK4``N{|?XN^LX}QXPWMT(YrV`A)c+{oa?2n^eC!>c{g6Sf8uLlJ)wslqfc>kfu!936zhIP}R_Or< zZ0&Ajy9r>G2Z?iB3%ej6oYVGyso%)Ipcxs&O=Xb5^-gPZES^C((J${3jeW&=aQV;= zvVGFR$RC?CPA#N+=C(im@9sD9lCR%LYzNMAUy#iM%qZflGA6BPkuH(tsAf}E{!c&=`R!V-y@%Fo$D7t`Nt2J?Ox0{<2a9U9dQ`I&i`Hv( zfRft=By5HDu?m3gIN=gGQ8vpy`QE>)N-MidWZfq%2T`R}W7C}csvTQ8t?*{SlvT;MUj5~DuH|#YZ)OpR= zgm87M!`4wGb+3LBx=wzS6=q`ciO3p3LxDycBN@Md&TGk5{ASdaP?tnszt(&*+xSbD z5J{^?p)@qECdnoxQmq76Yde}`gA`=hClrt$Qd0q|0&*J)E^O6sVyMXwO`DNeK2AA9 zc9$j{{9=0*i@O+gMpdzq_M>?#WEO;guSgXmURe4g#kXyuw@8U~`9V-(1-SXYDKwDn z?;5U72?Vf4$DuL1|qujHKYlp^8*nvJE65vIp(! zt6~!4=j?#x)9@AJ5_4<;Y6L;-y+3^1HRM>9k)@J{eK1bT<5AQJSyuuV5h>v`U6J99 zh;bLf(_plTrfc0dTzX6WIm{os^U7MKcOg*f-m*+Hdns};0EV!K=&4v-@P$FZgC)q0lt59xo=%6)IG|`GE+|vQT7^3QSO+p+`ZoHEZRwbA8a$ z7WSRQGt&x-zq88DK(gun6lvn7d6@|mhEZZ802o7AN!c#z2heK5YbTZryQz>A$MB;` zq7T1kn4z;YMbtyis}kb{G9&qp=A1$uP$UKkZoJ(*~A(vK}CHt})-e@M_wj0M33TUHO2g8x+j2G?WifO~MT7M7EoP1x; zPN(*SM`UMUL-G4%T&jhh$RNC*z9%5=0@;j>7?@{4f1~b(SFv5-!MDohu_eMZExO5We*e(gd>l9KM40|@vfyM z9f3ywPwyF4o;YN2kX__6dvY_0qzdqh{8{D>UKgSOl%c=&T*?=*0yLI$0%mO6$N4h1 z1*FTBVcFBQ}mBB9rzZYl&H7+rePGWyf$L z*r)@Xm0%;!e86dBg7FO;wR7ee2A0%e&I&1M8i=knM~6~B2xylLIx40da_@|cu|bBB zjVy_SHkTix<0EH4;DIadlOwrWx2=@j(hBEOv+4z!AF zSy8uq>WJ2TBU3ZZ$F)vuO*}ni)83>Zx!FS-d-WQUJ9q-fBvesl==Rw6sSjIzA%FN2 zRB7$7;Wn`=B_bXmRv7<@Bso+NcfTxr2V5RpLV6Y>(@UEcG?}q?v^=qOT0Vi|2B2A| z>zIXdA%+5{T;&HT(x|ri`{dZ_{!aMj9UwL7@?TX|)bUzD)^s_LORg8q7U+ z9qbz2!G>efW&w;k!Kwb$AjSZgS?lE&5pRjz_$t1WP~k(T_Vu5Jk&kByma8LVOXX<^;;SwE3C|Z5j*-p zm;%3I$Ho>j7G^5#|Y| zh4ngV9&lQ8Kc$I^|2Y3e*+(+Ow_+@E!&ct)_5~UATCw}_m*eYP4n+G#?@GeCV0ySg zubC&=@)TlJ*^I<4IG-}{`tr9KtgasQNq4bYL#xGrdDql)u6d)o&VSS>AN1zKefBhG ze#P_NDRFnZjqF&_ofou{!cYvSh`DIC9najPQ_NLTXbX*|A^B%T17n$z5ox$lpOlOE z!)HH(tAF#nvd(@m^^eq zWFEPs<=gdRD?1cUlcpcr+JoL{EZZ`Eam(TGAXx*%_E}z9XQAlFHWwnu*TXHNVNVI$ z$R9+KTQ>UJ!STr!Z6N>4P*zLc#;*3CQO zoL#^)F+BaRr`on7MJZLqq!BkVKrGt^o|ar4LUq?Moa(2&12Idzaq1ZG8F92mR} z=iI$>gG<&{P-4s>gZ$VJrHQ_8oPuNej+Y~smkITKyNKkVwgNf^bE-{a{+yY*8gGEg zzziohe`H`tOEx;#HBxHHlJ_Ri(wT8s8per+D);7`#e9u$|E&{05ogi5?q9f89=mXD zqqT4?`@z#BiH6#?*tU^%89&Qv9Z%pYZ7bET@i<q5{&ddQDkL-ihS10CkY03 z+`lG^+_XEM0yg_OW3$%bPc4Z!n`Yy2BkQjUn?Q&0)3ouAayL79p!Ik17_fsJ3BXdt zDyzaNZM2bpm*jce35ur$TG;EsSvsPU5f)ck_)`g0oJG(15!w$qF5d2jlM`N8N)Vm9e2OrbPOI@hfr0=$VA6c zDqAeP{A6h8!SkHup}Hi>5^v%yM#)%$y`zbrErD|E9#+7W_9R8{!r%nT)SwOsSHf3& z(2G8JJgjC4dS@ji$3%RNJi-9?h)AJYNOVsRCenmoryfkBapN_K)?0+|Sw2EAvEze; z5n8Vtd)@$N$^FbTBlY7>I5}m%R)zbc5wI9}UZ2<91kR`ix=zh^>lu`ts-MgYWRu6J z71R89&|sN3Uktj>O|V?XSuv%EyGs=E_C6SVOwwpPs4?o+#FWdQt1hNX;CirF zSqXV3zdjZ_tBT>=sF2nS25Zw}%v*ppKXYCQlnKwt&N0!s12GLHJrzwOn-PSQhcM*S zIU#JPPESsH+A(T^4K~RFCq_~`xHl`{99DB{?4Fz`SBq`8DNwRrT+Q`)?5Dn2-`?4I z?1-DP2FyHua!J~hlGKTjH{#;IPKp0`acC_ZM0!s_yRL-g(^hOqThbDaPuiK4v^gvB z;`3K*NLaPukR>g@j88gl#gD@W=QKQ@l)fx4ZT%zp;avsc&j&Z$mXtniOnUz@&s^Po z-1Aoym3%$9b89hVfObv_22nXPET21mEVv&JE!Afo+<@7xP*N!f!v z*c0C$rEL`E^A(@6oqzwlA=C1hAKG_G#lfV@J9NS~^Z{38F_ud@v7tBHS@a&n$LS|F z9QLb`O;G?M85|pI?O&n62c$lDq`tmUr-LI6#Jw>!Rp3Z0#h~2i&IP1&s6<4C{ zZz}Up;56F6C<*S!Qr1Rq9r1d{YZ`CKaN2!Lv9!Fb=_nvuyD{fRddRM?WkH~iDI~b9 z=a}1KxGDZ&-a^(*z0GGwo{xF^b-ZP)hpK9Ut(Ik^wuuNZ7xB+J-x za#z!ZkoDa&HwKwTQp#<$@U=4_4FbO_y_S&d0k#AlVDgi!t=F)b@BPQ|!ne;4sgVkF z?a^#VC=IhM3pMOQ%QJU;EZn`K@t9c9XV6zaGGevl(;FQ83Z|2=C_2rCx5N$yK&DoF zM-=K*^`o&9WL9s-p|MLw-z7};CdT{ZiwA)KD^wcF)W0xiy^@JkWrmY}h z`%#~FoDe?x6+f*)kBC#blNB0nz9G6`(4kL+S-@ zVb%JlFJ=&cd4ISy(TY79`01qqv2XOsAX{C~^v7^I-9elSf?;pq2nn?88hH;9;j$@7 z72bQ!0G5D;&jwMxPr8-b_agh{i1}+N1 ziqo>L%V7m0re(IyN4=J{n>1BnLQ;o0pr>RsUHx6@vo!bez>Mb7({=95n}rag%eMHw1IJ;)9JHq+EvS1zSNd!EbgrhXo?vXCM2-}I}k^2LMMXt<}N1~5GC$_fH-At+7dEh9u4-OoqmLfW_ZFuSZ zl-y%uAWZ-Dss|5(WUuk zRh5~aalu;NmO-sU&tMykzrS_$O(3DN2o6ljWboB-Lc<*e#_9lFG<;YiIoxiN4 zhp<3goOrqDkDb8B4=zr8cJiZ5yT936^TfPY?zrUqCx?yv&z$=*w|5R5^3%~{=U+5u z$1|5cc3jJkYx(Try!m*tvrXy#RaX{K6sklXv6BRbAq2Ag^tdH3|xVmFqO+Ik< zha1Pm3H5ZfX{r6-b!kAEZxslX=2PHCx%ErcmDdgtyx*e8)%@YbL$M6PVfrmQ?{=r+ z-4Jqo6zo8PkYj1jenq7LRc&;UU&i-!c||_)!rGF?!N18~zpkY5q~GLB8;_vl>tL%$ zr{S1uRdhIz%;fZ>Om#b#ZOvL>BbS*@o%%KWgor3SB!vZLFau@TA0%mUs1|ct=?DbG zSNgebV*-ZQ8%zM^qf|6OahaNxwEFWz6>3g>;QV)WekVDQlz)E?ByTaM^KN;Ds&i)9 z{TxW>k@Q8tj3E(Gvl*CMN+@Dw1?6acAO{jc2>}vGR1Q+=S_yO991=$pI)lcQic~jJp3+Q2=L*3>GP{CLbi0ro6MZPz}Opd--Q#$0azs>gO3>W(EKWYA=XWhTCVG1|EM!d&X2HJWT_2WtIfJ3aoqW#xAsr z#IYVsxhm-6r*X*N31*shb(qLx;S>=*`_5Z(9SdrbN#y5FKF4oW<#d zYIkw`T*xS!{f1R~b&hffuU(^QSbJ%YuV%K%a_DtdTm@2 z06|^q#I(hC<`poX)V@j8 zR1;w15NN*mg|*fHRHpu@gLq-ynM~zWY15Ij)S|kb-h$WI^a?~rzavU6XZ^z}TzE=R zbnDl!_+@#`V0R^3t4AJ*@6)_7R$FakTzU8Oqy`-gYA95OQ?S2^4a)qZ-mvf%^|`ZZ zQdH`f^^wzQFrsZD*`-_AM2e(&3`vO)o$B4)42XD(cYAq%e7rg#w-rTH-izpCW$NdS zTTJ}idG=J6jb_85AW$0lA#K(w+O$Tiz+T#&z%ssZsW2Yd%#Vo?Pie5AM!2NkpZBcD z#F^?bax3{_&_QtECCw=|aOAE?++~3QIyQrF6ajgl6bdGsW#k*^h(J_#3q8Vg8NbRH zhA6BVozSklGxblVQP42$G5l(wI?J|a5HRhzs7``CFNPCWZ4|{iw1T+P5aw96lJlkH zsNlyu)0*RPVI{vu8ViK1wWOq$_`!~$lerxcyy2_b%e72ot5CLU-Jv2a!(_4=G*BK1 zHK_HGHgoXTIy4Vqm#|7l$-l)~uVunx__svM%;d0(_>}hPAHZk?Uqn<=d5I5wj4$#d z-U#)hU`CjxHCy$=|K0+r^8Z%8ne_Vj4=y6zT^-A;9zuE@=93=3ul^iyUCXjS%Pu@i z+U`hn_hhu$jg-&a7SUP~%LHff=nJ3xY5HowRFz1T3qq~7(}Kvmz0V+a9m}Hjah~0e z5+F<`&l`P*_}+eb9{%2cz5?GnT5O^>L&OgCOLrJYH8(!wHJ=LC)K=`qOJ;fjaQUVh zQi+hBHXhy8a7y6*20^7&kz!U~bn6yrI5%t~bSrpfPJC+xsWW@Siyh|+{a7<6Y}1HZ zX1!bs3`Vb2bjOHJ^Ef|dc=T-+cizY)zpjVz$hsGv$q!E~43E!gDCm{Gr67G-LE8E! z^TQ_;hL3+ctMnJ)xU-5XdZIVmjl)LVm{a=maNOYPxFd@yR`(mWZvG!ahb#`Y9W%M4 zt0?jaAYO9(vo)b&9Dg&pq$9tm?dZxF#v{FB8IK(O!x)cbz6&^ZN>*Z7uY^Y9?w|f~ z+r}SPgz66pwQf8H|KDHZ$fx)0?E2$LFW>)PMA?-1IQ-wY=b-K}GhP9zMiXu7zuZ`S zO;N`a?Cc-85k`_v4ko5&%jn0({HcEJ{p2HtL*G7ZJMQn?8~HIF-5{s`9lJaCK6ny| z+zN&)D`3Ut*h?$MGe5M2$u@!pJDRsQQVYu{dQJ0OtM**l{l~&4;Wegkl_tl~WRyVH zlpEI9Q$~j?)O3$*mx^*_>&=-lOWdFp6*)Q$>(E4&irx59>uv&%q~xT6Q^5IdEcZ60 zx@^FUlQa04Wyug~;Pux@+tavZ&!gMi$s!qlqFhqjTMi2^6? zufpR@n(AWo(w??e^O#Xqk_Z6i#u86X&-^&G@cJzivakm4nlT7Cj@F46t7;{bOZk9J zNIs9%ldXJSa8EW+U!U*7$P{DqR4Z8oIRaPzo>|AXX4LT3h-g=DS(~BJTP%Fcs7pc_4@ID{8~8K9 zDKLn(MsVQI5R{9&e4z)M$wxx+R?!&@ouPmsXL)xF7RdQfyO#!)!U0}ogFvhn>Mb*W zxf6TL8Z4Z@;5=@BW%&qwF=OrLM3D)F)ICZFeM|cJ_qt0|%K%QPf1Vjk3my?~xmw*7 zQRp4b=g$!B@SI_-jIQ-%QGS@o&j;@SoK4a|hMy18LgIPOCbuFhYQPT-z{YZ9IKrk6 zrOiF+N(pE_NpRC@Wn#{^VqvOY#m$7yAvpKpT}@D>_S zdoPkL8ba%F1(Vktb6h0@?O|mLAD|5%{|SvL(mD3xDzVmTOyODuRC4&=do9|J3aq^7 zhr%_i>@i|v-1US7greqgRwYCJ-zHr~>EZiSx|s4@wt?-xhB~cbyYriiw8_yayQ%is zz2ixqnb#Tjvut@>@fKEu_pMuyNg6tr6(6|>D?5F|0Om-Lkbf!$f!X!K_V(`hGgB)g z2aI5wQF`DAwzsTK|EKne|8C`NGX|pXT8||y&&|)ee%YCeH@vj6<^1`-nsI~lAGvKz1$>{`C z&@cO0#Q*;Nvg@TWh%ZjN`u1jmB8LdhC#&yfQR0@u$CC#2y=vvc-PeP{3;p}cq1jxb zHKKabx@yK3_wX?m(ub7sy_?xG2<)7%iNvFQOoI%C=2-0s5V94bP4=_Ih7$$h=&h6Q z-6h;byx2`2Q?RWrt3T^_B&w-kEIHZlUxvZW&Pnf z|GwD>%sf7ER8JJid za(#u6Wibp$fYKtRSbL*6wpN4E%1eVOHe952EYf5FU9;g$U3y`I+uMIq8yrch?@bsk zJx55C8H-;tIdE6_sxy(~2}+^APYO&RJ;_=5CL!@kLjuM zJMXBhf9)7T0iBmkP3t=OQNq;xH2?VvJWG6FiwQV&`{r@Udumk7;EKdF-~7yi3s}4s z7&_n-DPGett~sA>V(T%-;FGNMjq!@S1I8+z;fNk)pjU3i?&k%Ab`O-nb`%FmtZeR` zimELEj0|_3?dj|k8(^A9Z5NU`f(OirQO7BmeIjsGbJ8>Af#d|(^NOtWF_e%1wXJkL*+DF!FQ}Yf1^C{XzrC2_02CnV`5@zWZ`qZ*iE1 z*kf7{PQKK;SPlN_5R^#?n4#Y!bNLdBGX0l^bvH?3%gQ}#h7DC$)n6+T*fK8Qg=Z4G zY@OFaxE!nODj$sSPTKHAT*4RA>$~|#q?tq;NE1bCwuG03Y2#rFViH=;$58t2A~SIN zS7Yn130@JVI8Ayad(&k;w7|aa_!n*L67jA#XiU#cXvKev)_KD7BSy_tVRNMCZeU5-+Z~t!3B9 zgo`JxX!&$x!q%b{(PsX3PVKNgs-EpzJCLN^Jb)|E{QXD%cpYE{RkX1d@{xkZnrj%W zWbRxXxsB~28~*vrk$dH7Z7|Ob|0CW z7$&xl+(+1~(^jMgH6^#PlhQ{HtngcDm5@dwJFtEGp+)8FvgdVD39nFZa8(NOao zoeP$y)442y#NHaHbppKNQ!PP2(Tf5-5VTeMqKI?7l~$6Fs_@m2p^c@8q%9zgB5Nl| zqkaanBmwq^q~k7Yi^P zK{}lEVHVsa-#RQyiURNZlqe%<}orz{Z1SC+EiG2{037bxup5F=p z+qfVMAGkg;{c{A%Q_lWd`A#$YaX0Y*ba#dBwiR9|`DMc~)JtyPH7~^IwuWTWyjeH0 zqAb<7vlI+C>#wWmfONwecZpDJ8b0rZ|9A=7jR1EHXfC_AUiy!q{$r0vagyI}L#_bKS@)O|r~_ow zlZ^?eq#{)D_1>eT?+REXMq-h;p+Cf_L0vFMfpot%>ig7kn^C2R1k^t(I?Li*Di}E{ z+S{tA;RjzTB2q*;%sbFR;2C41<7}MSKlPX3SYq8x;MZ!HU;Fg2_@qhMNylX;W?Z;p zL&mBN1D3Shfiks0Nms2rq@i!G^l9VLa@XaD2Ns5Zl+*AT%GAcDZ^=)~eJnp5|MpQM z=I4~2ACB8Sq2m00!$y=1TX#cF=}*IPuT{lu?AiFm@^P{JBj?BPkG#vrKl0hrzF+@H zJHa+6Pqtzk^DPbjM=N6N^dVbLXXW94UQ+RJLH7HjhV(yU2-*bn>>!u|EhN`Z`Dh^z zUEKl3iJ`)mY(jE#X5Tf!m!#{2hJ!$&RW7{0lS-&g6(%I{ca3ml%)9l@?t7d7AVc^VJJMw162kc$QZ%|C)H_4>er1R*IdSD#sny@N*9g~N zbEK#UxBaymIfT3-@!vyGdA!5QL}LfS_d)Y+(IgwmEUi7Y9f}dMk~BP;QwJ$Dmj>Z! zkSE5N6eB=!3j-)NDKm@SMuS51Uka=B=WM zY|LhCI_c$J@)Us63GRPIHX>N7XtfS6sZ1F$mE=T)2IcS;NDblv%Jb1Y8#)a32(&a}Bd4%$(TL4xyzKETrXMgg8rjL% z!KdJYeiu(=gM1+NvCL{*&h8;U094# zD^PHPa0(b#RWEAIWn=7w$)7c7%O|GU?M|RUf-3|@{%NO&Qv_T`09Uh&sPwB1t{W%6;5^qMA)FmEZswe+LG(q^PJp<>xjwMjui6dWH;8XQKxvi3I!iEL z$ZfmQkY>Z-WS1-(wlB&sMVTIR-eO}_L4-SXM%RR8BzACF1r|+;2`mh_LKV)@)EM4M zR-_9gyyjDGlkcWul5svuPzmA_>q#q^|te2{pahx;Vvy_?Eh z+q#0r;%kX(3q8NOHCGl@|dr z>Bih%Nk5qPE8i83w&dC%OT|#422CfU0(@7C|3`gKvH&v#+a`P)s|zFuHWFnz2sMUV zj$;sPY}grXWgAaqu^`^ z-LaC4A6Wh+lVf@*HBlDcg26P2yP^X+y#)da(x&725+Zt_YjR6DMXK0>B*23=2_Mt; zCn1Mh*rNQaoomXeL=Bjx8mi40E|4OzZ3hQfG*DcI(nhK5v5as2(}2pCb1>{> zHeL4(r?3z#I!fmCiQ-L{5KJdhu{^kJMrMlgM>|B&;j%Xla82H2(Q$(euMOUiIxuVQ zSZbaSTR;?U8@VEy&oqB=%43YKe5#}{MCc)|9$0JWhVW~epI4{8AGioIVQtK*R+*7Z2l{H z+VL3D-IG!>&U!hYNltXd;LuFkCyOMU(ZPg}bXxLh`Vqj#Fx_lflq3(lZ(h^%?p`-% zCwy>Guk>Yw>C+0+a-Yr*k1h08DfS}_!UB9%F%CE>U;s^d;7s(24f zS;eq*;hfS7!*RVHI;-TMiIKD(WcNPn=0$jilU>3{6GW{DlHi4wJUHtnr{^Kp0>`N29W0NePZ zJobYO0&Gx#swY5#zJwQz7?S}TPSIo**WJnLb;);;&j&t+l2|}uZZOAhb!Mb!W0J`}!)bZ;qu9a)Y%LVijEzZ2qZ;{*p8OI@g3|mOHpyV{XTN|ms@QS?A3$lJO zx0ixO+w~7j6>lNP|7Z%=1TLqdHQstIQ*~<@6Yv%S;v1GtNC`59ys9k+7AVgJdqNVk z6l@6zACKoY^(`tLt>95lY5G|BfWm61Mf>t*>^P~3$w#Yv?VV}|YNEGLX7*{!YPG!4 zUM}s{Xf7=;CYL5Hza#?I%ct7f@QW#We|**5rQKxvWkXUd21~gohHnLmz=5L-=EKTo4w=>Yb5A{)YnuVNR6hKLU1S{QiU~w?nzM@IsVhO7wU0SU zNL{Qa(%;?>?b-R);EP8GdMY2;^)ZS#%V&nES6ys}k?undkxDk(Xi_Fo$-VUcltVn0 zBYTL9iMHU(?~U*=8X}-YxfdM2=sruw+RhDEeiDCj^QC{HEqp8!QvnrhV9roM3f*QqoN^imdVpLi5jO2NBMCxrfvw zWMdaup?FQC$~CT*${mUx!lBqpv!jr>(f7JHk zhyT|7(vm~|@sB|V_3QJ8dAFaKH0Pr!!=FDH6H)VTsuOh1htaW?hyu}7EKh5QJ&Hd$ zen_O)X=ois4ujgUecTbzRY`uIi@SEyrheEU+3RD9X$mdH?QC#*oXPy zS0F&@px2&z1`mNAm#B#@^dsWmrT&`tHimdCgb);$e+q(QXFxu``^g_EJx5ZAZu)*= zXUkec;WRXq#(dNHN&5WQu`@K3b|`IZPobgY-|HPuv#5h>lQm@abO*bVc!5o-u;+~rMJG#cL1&XTVqw)9cpt3CE$J+)kXh)J3fnnr z<+PTmp9UW!ZN?D5NA#@I17McitPj~lL^_Vhoy@*GFvaBt(%$o-n9L)7%#+`cif;ON z#Ua}VMC51@No2;(thIGt1LA|pVS9xJdH)ZWfuKA4t3RAR$+v3;O>M!*b@xNr+rk4? z_LiCO)`9Ecwr>4h)QP)6GII4JLk+VN(>u|1JMXkm!$o+xWpSwflu$!KV*2`wp$*$+ zrL13)-|!axSDW9kaaP*;BYQUVz`q?awBi0)DH-G=c~VwlS5iXb#*-^TWN#ApvtGN_ z{VuB^^PsV%a3yJJD$nUSd|<-z!WAveuqD|(AYp64ik3{gd~QHO23}78O;%}AIPS>R ztvfSh@3qYX`tB6PZ+|(V;q5`SR}JiY^T57cFgEQV*wPK#|9Jp1z>vmn8FB_>w&T zz4(&+Ggvi=$C`afs%{{!$kb$A`<9{I|9i9hzbfP|rjFaZ#9qjOMw>iFDm(WswX;5| znsDdd%S-lKiGanZKx(zSN4|BW7h~rnU zhDb8+oXSo85Mxg&9oOf@FaB7OSo89tNMl7})62E2BU~EOK2n>CxRbNSK2j7+CFr2G z$WxsLFc_nZyAk0iuk~gh6LEq$7CW#z8guV!P_E5WWRD$G04jJkcC)X^x(w*>Vw7Ao ziaj_Q_o-`jevCW^(T8Q%R9Dsw$HLn?MyRBlc8XM`B};FK%KAMm^6;0^f2_Z`6ehu-6(>f5xS%gS)jY=_EF z(9$}(b#{4{z^;xz#{CePPhPW%`C^+5fo`;o6cB5nvVEF7V|n4O6oJ0~q^IUUMa@@* ztj)2i4K-1;ex5i;BovVxlaMoXxhJODZ&Q-ccPqq_ts-T(`;w!JvmkgxGROSfBbjct z)(A;*;@50;AfjO(#5`l0Ptyz6=TK72HzJmGm_+!qU9DZ8y(GfA2q1p~Y~a9LP)lZ#Jt4zJ9WVG4nk>-sdtl0Y1m!e?^K zy-@4YXr8{1QdsKG0GG+#|2Zz7%Dj z-5|rP40Hn;juB=cta`OAaYWIBDr54~KsTLXkfgm}?W7djvex1)P;!*<^LQ6Ik2Uq@ zN}xGFG8bM-v9@)K#-8-85@@M*klffn4|5FsOF6}8GEd5Vphj(;SJp-N?7-QQ{IsKM z%C$|4H|$)wVMlGt3Gqo+Wh70?NWA!`D>k&$whW9<+B`66&3%atgOk#?j7ncNDsBCW z{P4WOJ!}_^U^wh*C$k* zJvq{LdRpgfVH{b5mlyR3)eM>(8Gd?7?gD)0w_bf}?}R`~uKdlTlOszNMk9rxKSl$e zlppBvfA#HLQWdP4B%&x#HL1jz4lN^Rb(Oqvc!QWI3rOOQmY^Txm;kLP}z}?43OtFS&R2 z|Bq}*t|tyow$3)b{u_(pc#BM}-gaLlwTYX|NX#FZmB&VA+pNee<@4>zG7|0_AH=XF zA%A*q@%#{Tq4%i5x7Bewx&q?WqC|+mn-P zBd-;(7Nq0zK_Uop7D(863DtKw2tGibNne-5fc!sG*&?oq`Y_5sM_YE8iz;f*k8_*( zBwr(iJIj$P$@6RYM-6{!N#rel$!H3dSbO|qIOYv z=~GdK3GTX#4%J+j9VC_yXDBTZgigl8_^AFm2QlP=z=DqOm7^Bv+zg7U+N_ffaj3N9 zbFD5K6_`nhxAsd*9lbos)^9nY&JqAj5+5EOKnJXU^|KGrtB9}c*MO7tF(X~#q^#k#fWA{; zw{NYx%E{(xGyFynTQMkj81ffOsBbgTAh4FBYAp5bN~1)VGed@jpE6lC9K&$WT|()4 zh#lo@!e8i}7d!kwsjGfi=VEx4DHQ~(J7dD$`~5zFpPzL%o#VS6(DTJyH%?(vN~Kp^ zGb)>!$?;{BA((Tt^pj3c{^~KG)NoH5Z-J~iCf^a-+}-4GZ8bf2Lg>K8{g-I%hu>Bf zd){JmmRp%FJ2nd`rX3L4zuu0byzQIqYpE66`UBRAEurn`-ieztlGbD-W}LfX!^bN( z{G+zzsQ9Gb8Hrn`tl03kl^gzgS4!@x{P1H1;fDt|^n$0!sPrvBW_9`DI|eswOHAJ~ z@+^X7OC>DJDLqvnwVsV%Uyh2mKJoRZhZ=4KU-n^2sOhN55vpnXTS{mqUS5Ki@1}&3 z@$$F#hH5sAjr=4oz8L>IGcGSkw^CgbJi+LmTeXT9* z;ZyTu&D4n5lh(~=~GQ*Q(fm(RhEmIwyqg? zsT7k7&xNZ9j2lp*wTE5vU#dxbH<^#Smz5lZ!nAQ4r*u*M+mS*nSTSkDjr?&1d^}>o zq;(4}ykU8L_lQeg4)XTjQLwLl`a2&hjiVDC8KX#ekF*_o5X&{Rg3|me(p}{G|42=H zbI&22s*#&E=+|A`kog^VGA8nUhjIU0KEPJV2c~APlwimy7k6Ye#9-UH6 zHHKaN(TVC1{`OrN)s=Y(cv*Qd&P+j^_dGZ*3d&NW4NBl~CMDkF)wJ0(Ny9(D)S*=J4OAExvUkUa4@+qudAEZ*STYsu_ z2dCpL*lv`SP7YTdPHb6G>AC8TUBT=by>$VKu`-uW9nrdPWNOCw^tY{vr>A_dH)%+2 z_Rz*&y@uoto`4o+2(e?zCm{n9%cvU)IOLO};j_KSic}9eBW(>KXv+hQ$j%V1^_g#& z3L9EEWo>1jWMt;-)9gzA>8<{55v4n_uqJ;So<@umw`uNvVbx19I=iB#qOIFkCWhyy zvUvb9(#@aMbbgl8dVW<=$NK`MRpI4FgSZ`4=hurxWJda}%cdwLm?f|mne^_9AF~R< z%pcH&#$T#07k;a77zDkCk8$wH)(axsO*CSA-)(njEgoe3A^hJElx! zTd&UJj!jhV|0f=s==3Ggt(oS(b%lqXq}t-?4cSUN3Cw|~v%4ASn}v%A1nqBc&L%^@=u5%$iOL|SVu;qI4K(OC)xV%7s-Yc zlQk)Xbddxb!Y)M59W#L3#F4phWfx)|TyK39#FVmT$`@%wUxa!%@}4PaOH|^g#T@Lick_}9Ga#p(Ue>FFDrR6rB$)ClylkvlRsPR!wzr>Ol0NpztuTzt zS+ya1Nz2Okq@9D3Hp40Zf)yM3ui9|fuTpZK$PXWt)6fHLw#TL~8=JQN@%(UVVffIT zhPRT^r{(`Vu18VDb^V5oD92t20Pi7*m&2#dP13rtaMa|n&qK$U)89AM=@yn9-lkfcy7WNFu0(}3`2X07E2konu zGvO5I`wy@}-&e*5KrS->BHwC!WsH>?!_JRX7_n&yoA$yZUtGh5*7P+i7oODhHIZfp z^$8$oj){Pdazx4c3$uKL>?+O^o+FHA#pc0O8mAZ~FavS|Tga&`{GF}HZ6Jxiwg=X7mb zlqm_)yj?L&No0+&Y|55#SQ=sj3J-F%@&A=uCw?N%qV;0=Pt>qK!{u{Z&l1N7&fky0 zu@Uh_`_E@2g{0UgcZFcZAQw?c4b@dUV?Z_4-FZCcx3-cs35O9EC}?-_)c*8+m)8t- zgSCxbu!iC3QzP57)ka=dDavA@E7?L|leJy6CwPI9L)kB7<+RRQbBi2Bi+GuLA9pxK zZ=oX^MTF7jeL9U9yI=y$n53G#@h$c}ehURvcJUYHSPp#>@GB_f7ztn0F_GLYIm}{ee9Ge$rLOOZ!vj6zM3vHs$(Z$ILy63$&+NQXz_JlL?7cK zb@;YcK8gcy%D~9T&suegxDygDu^>1jEV)3){uJCw!q?c4f?sGLze^#NY`e!Zd@8sx z$g90IgDye>6dbK#woHM<6u;J_6_yGxRF=V)q*XM)ARa?3`1l+Fjdh+v<3Hvzqzlnl zds`|@C^4?q#Fk0gw2~{nzXudi%=jW)q3jBO>E`I0rJ7>+BM1O9TtkQP8knuRds=%P z_in}`d>L7FG25djmlr4ud+o-q32HKW|4~A%Cv!h3_pIuYXVz=Nlsm~`%XVXMulTaR zzv)$cl06ylfN~`vPcYtzd<=L2-JDmU&C8P>pttDzS|1E^KCZIoP+piVJgS_zYzATz zMT;TDLl1am-;6NfvLEDu$0@O}FAOMSgEZPj99c3f%eB>%lu{hq52F@=%_Z-x$Cwt^MI3p>2 z*%|56&gfe)0fv&-3>$GxPU+ZiTtQXb851hDgHihpoE|}^T`{p_TT$dUOsAa+LrE4s zmTb!(xpfj5NVc3-l~p<-Trqt>MazGMlFDWlRn#3>(efBxUR6}ld_+ad%23jNG-gZw zEFrD)M~g${@u1U=7z@E=Mc%>3Ecklfchf-bcxvxa{e`8!-8ZtHY)QVR;w#jtts;98 zKMmwNt2@pbaeW|0yYPnF>qC8CjzP1{{5t`Nf+5Qa*y4Wdr4{2DLu@IeV+s{^p!Cs> ztpr|Mwdc}iLvP7PTe*hh_Xf^E(rnXS7LpNWbIBjY!cA|XKN~cR9Ot^hm?qZ{=d$w% zFXwz&EDyphM0`PM{~^dnn`a|dT^LNRw!$lkxGL*=;AvI%kq|->&h7LOLVg6uO3%3L z$7ZpQB;}@D86;9<8xl$B5r%jT&i;*o{n@Hw_Gc@NA%y&#g^*vz5<=FGj#;Aixm5_6 z`l-@ElFMIk>ulheSY{-25pV140Up;#Pv6Uk@rA_#1>rA8xprmrdbs2kLF@vZicTSOm z02OFRKJ5jH)5(40iYkTxnHkxwORKm*D)NC)Mq+iK!f5Y)qch>cdj!-9;s9S5foK>h zTMCv)U*-qYDCmyXX@f__DfWe+)DB>n zZ{>s>B-C_i_W?8f`i>AHiB`5fXuU0%GwT@csM!O~9Lgq&j=zec+K?#H`-b#F4J$tSlj#c3#DD z33?03DAD-Q%T<_TkVzVGo2$t9nw}U~X+j6`rd6$1sKFDMEgvbEMpLl=Cn=b$8j=by z{jUOREdbHB$%^(2eFU^e&$I9H!RQvxGgDA*G7RqGGpiCxlf zAlv`=4j6zd3dhQM`7?7^FQB0hyEf7?LQW!Wd7DwssF753)&*7~tp=wkoB zI3USij(lFZ8`_N|%inA`m`SSoPv>tTDr@>RMIBdg1MSw#Yl}LTnqJy7hYw-~=|Lfe z>|9o|kB`5uNj!v%`%fP5(Q|}M!i?&UeK~ZJ&_h=8@wqRDE_$dnmzx3jcVo*QutVDC z%WO9zD$bxwP*r)`4Zrn4VX)*SqAc~+c39>M^e6R=&I{06hhEW15>U#Cx!F5?OgS?8 z7@qYC4F2jD`wX8qZTt&kJ}DpmU%Toz|JcVEl9SkUR!!&&YOu=VA$J}pH`Z~+3OwL1 zdGJGM>YbX2BYhU8b(Z7CG_G|P5Zg_&{ow5{)EDMuI?1n(bU)J}$>znrySIzVH zl0=TYeC7?i$$_w_@(SHo`$uYMHql#%CE9P!?H)BgND)~&bR~I#$q4rtb%FP&En8vf zaB)nrbg-f15m0~C47K#wUbJlMWRcd!DaB3HdAl~USn*ynwYB8EQdjxgn3Bk7N69-q zj*<{x*2n0t?MQ^XS3pNuw#b8LiR7e(E^%Ih)p^P_ zR+w*^0>XTgiEgN<7um}%7;ZZL#%zT8vUo~lz-_sEyt^9xBLUz5{Ugjjn!B4`5wx#V zCz(+e1OGi5jZFAnj7A^`vw&LzqyNun#3-O7igQT8-0Ol8JtDPcf21+5|IcW|0Yx0~ zwKf{*@=I3X;l;z+CYn9#=hwTc#$tvELsPFgPi8}0&s+@q;(zdyVV(q?$CJ#k(;vhI zofeTulkvix`txbNSx@=FfDV%aWzSTaZCc_ubFvlh~_7X0a%IvnCEe!r2j? z=1)f{46dnsmO(OJjs=c&P@sImG`2EX82OgozacNP3r4Q?lUD1C#uAJ)9J8l=d0tl% zjQ=+sDC7U)rV83}Z^c_Ew-~c|aCPaH3Fw!dwrkzue1WV#G;OnYQQPe9(pYV? z^J28krtZ$AaA+H<2*ddjd{yt>E&FD_2_aN_^PysM%6&j>P8Vq_k=1zXd-WNqe_}K3 zdA#l2r$m&?FT~28-TDMnF|Jw%od={jS3QS(F!3L9+;4HB)+Z5$c^@lY00bBz67GKZ zfXXeAwr^6=pLcPWnMnvC^>!rnGoDm7Im?@*pgRo6`33iFi)B8KOy-DLQYGd^dg{1r zwiJOydqF-ygXQpol$)?j=rAqe90jHR^!{QK6+=uZlidF{-UlShZq@<5=iJasJw0wU zD~Dob)SL~Yui7@3Ej^F6`;N<|rj^~zFL`|cc}7CdXoaYL3l1Rw$^I%mbbUp68ugNk zjwnoofzJ^j*_rP3M7pYjfIFQYBIA2riUqiJD!&ZfDB>ui+4SQ#XQ)z2lPx|fwY}~d zmw7xlZNy;tf@IOOQ7ggRGdD+-sJ)6HmZxqGd2w1PFzWrNFJnf_6&SCI7L3=6KXp^wQZNJgb>@J8gcg)}6X=-VvX3KuUp2NxG@+u$2t{>vpy!Nr)jc)W$e+Yq z2}f=ugoHampq%Vfp~DWEx%|+}nd{@k33Z_nhB@Jc4tD=4SSz=jOi~#dLtsr{uY32@ zHFHb^%xd5)!;iTgWW9@~uNTQia?y2!&R~}wX6u)rTgvPiC@G6oA-M&KeGc_!_H>qh za?voO@n1qtHLPB$E7EjvK9H~a(L3GNS_TJ4{AEER> z(={Ur%TC$c4mYHNTm(i2Z zb=m#1>#x5?-BHJ97+~if8Ta_AO&j3J<1 zlqV&=@$~#B*OqQvxq5p0emF&{=GxLJaxI)9#}MV#OHyuMWhR!b>d|mpLV9QJgo;d2 z#+C;ok}Kd4xiG(>*jV{5547?xTXkcP>xsfb=eCgq<9s@|eP2E#ZFvMh{L9#;-0%>d zW*PK zn{9a6Yb;*AA#!UR-%)`aufG7Tf=1(Gl-IFcr+^s@AN99kO(iiARjX49C(R@=DZ`8b z#8h%spEMbPsmSQy9@`^7X{wqVN)0Gyn}%{WY`@85+P^5`(_))6o4094lDidb+(>U_ z=iWyM40*AP;;h!MNF7SQBttT5P<^ZvLx5A7W7M(%4}QG@T2?aguZ=vYn7Afw>i;C% z@yq9Vg_4!g99w2g+Yv0gq`&ZD3{%KP-E;t`eXB1JPe^-wwfT9MczoH|7QLcUnQyQ~ z5#jFJ>`Vrl&=l+#?a|gSE$byxbxj-plq@#%JC6XIW7ay7z~;J*>^ZWHdfs&HfH&&3 zym@yJ&!#L#lvkuPB=C-kn8lEJ>_?v(h{@Q%9b2DpAn>Nemr1*?=v$!oKKi75i#;Uga1QW zjX=mpk$LpJG51OtemWP1DQEw!e5W?MU$Y9SvF7ICpTTK|r<3JJ;Vnoq>+^2eF2p1V zzGWKIjh129Sw8dUk@t=kSIQ>AXY`45*6=SQwvLeImeA*2_ooi(_2^=b#Qo>lOewVP z7Z$!eeW=|+NYn-LNv*HidefOD1(~+CkAB%TYCJF222D8TE4rGI?(Bu@yKD{f%!vw9 zYrf5_HGzY$`tLR^VhJ(hIRvp(cN{Hf9md|NteE~OGz-+8o?_&lSDL&;h5R|#|t*uCcpV8n*6h?JH3q=J0&Zzte5V=R*xQR zyJs9ewqp806_Ir}Wi_nrE{gOOJ=nIx)_4Xw*lXkOZ(V)UbO```J8qAB6?E6K51h!!VaPWs7Z zB?)}|;Nrw*CvTiv27{0w`^c*wYL*`()lYzuVmo21goW?koy0Fu63q>_Fg^3Bqc~QL z^@_?0pxuZeqoN=x$SisATcGx@pp2s zax!8aqqDHvOI0UYY%_57X|bJ2Lm6c`sevGzPDt7*)64FUf?jg5b`$vIR;=Oa*0CFuxi z2dd19idT`ktreWoDlU--Y+;2KY-shy&w)E;Q~0pso&phFN63Ig`ukI1+wf#IMuI$OKB-7r|Fg0-7*(u2Kkjd;+mn^7qF+oid9;XllPE>)Oii zx}(zL3n_X`*`No*s|(FmY`Y7o)sT12z%CnDI*58DobsJZuw-!+R#;@fMeAmJtOeYR z=bBSy98l@V)fM&f*L6o*35BHCbP%2Rx^^_J?(G`4|DbF7>iRZRP60Dx=K1t!#>^&; zDahdkkr$wloqRTk0=Oc%nq4AldkMwx7|-+)H^8Wstf%Zhto zj#W%;o(K}G2*ya{+2OkhsB58zG=+fDLbAKBKt%2b>nKkqzN;@}hF6MVBAif6KMe5Q z6kL3IeG_2d*f_9LVMaZgZj5fkXDnA)T+^t#K4dD?g^ zlrk>&WDlzEd6*NDh!`N8Z0}z4gwfTyNo58j-&GdrsZoY3GVdPudTc(7@PJk2z8Ee+I`&YR`*Bp>iWu|XOhQ<>MYOc=Zc+YtMV!}1~BIfb)` zn1SBBmPi&XxG|;;FG}qMjziR^p<59bIg3S{>jt&_!!@+@L@~e>jB|9>&}@9VkGpX$iwOGAn6wR^r9yuh@{VYQrH*T7DUybli#`hY!wacpgR~ zd1>n($q(-;2!B4f;Wqe2j!Ex7Cgm>&u0!_KvO{(=Tt^y0NnO(4zkOj&$LxZX^>^eq zWCE#uy9R7gwAvkw-;DlFyc(2|X9YH3t0VJ~56&ukprEm7jwqlycVJrCn!!|^)mc(i zS+|qH+O@@x7VlbG(h*q9_Vk`%<%vTUC!Q*hS^QlkhcJRzD2|ivD(Usip4@Ws4OzEv z(ugXqal7&5(C-9-StRt34~=;nPpeN6nwYy^z0x%YfQ2S67q2lTBw3`2Cj9XH_RAbJU%B`H4%WcZV6z|ksg?nA0A1*vw` z%pg*&J|?Nw7K2nvXv>*{-L*jQV|QS@#LmUX>z9gSYCdx_4&#qW1t z<>-}NpBo~7zsma29P9ks_>abR4-R&7V9iL3uuN^|(4CPpSajeFT2W|5y!lIgOb@Pm zQ`7g8?_7keA_=K+gdL33>=H4JJb=`OtFYNB?fIM?{7rp?BMda>gTI4cy~-95cJW4K zSo7e?M-ht0oH2!B8=iB@P)rYlye&G=pa)x}-d&NH9%M))y+~Six6l(9W+@z7_=miJ zMY=sZFQ&K^Ji``UN=8ulPt#r5xBC5RNc<3i3u?j#Z{Tz6Fj^xC7;V(=LrAF$hw$X^ zI;~L*I}KRyyz?z2zdk2!*VjpYN9SU-3T`9IeRlIJ!@~G-qR|xu!u??ok&caR?X*_qA7hV&9seIwtjk0 zjiOH>HHgG2(8fZ}QRj&0X-iSF!U}eMB>rAZeLYsj0+L{IieK2QmuFIns2JP(fY34P zBlVN)`qie*1c)~AnCkG}jJkLW1|-coyQb%%3sS(oeuso-}o25pRLQAXmlkmDPXK6atS5Fysw? zBD&MSHN;FA3@YHXMon4H(++{H&DR#rX}7;@^(w**aq&=&~M5 zTAo8w>`8-*s0H`2`OmD&ycSL)@#xL=C`=^x)P(Zh47W@KVFnlf8qtR>^W&n(Qv(t* z3RbiXReiFrzE;svn+NpWIe@(d|8hdZ+k@LUL#NZXe!PL&x9ihKSq*wdSuZq^3oKbucY5s@_A+i#*ZxiQG}uD zEu4aGG=!R@mm_aWP83y^%*9H>;}}2ICxzIib2glN$ zPaDztqtnxPFRR-AAm_qE2$6UB0kars%iOJ-D>`Bfl54h~|2C^XboeDB27L^|sJkn_4%R=udmHF9 zm}9p6J-O`meuY#z5uks3jC5O5U$i-onJ|D)dg3KCsbnpmjtWc^K}H4B$ZzOJOQb20 zJ22J(-Gy{g)16T!X7c8XSQTMH^Ra*HyoL=e9SK%E>d1!4*-IylI!Jc1q@1k4b0QhNV&6I`;l>XM#+AT5(t}L}(XK|8!6;6?K z6$uv<^n>x9mQPB$D*C~w)({5hPn0GG0Shew)8O7DvGJDDe|k|c1~?=Tz<%K2ZBcmG zgV~^l7jd?8SfynFfAX~PWT6{e&xW!&Yl}ZKcY(Ad210g+_~mFw3k+oMYoYG~2s~1E zM*k_V{`?5@HRf|mNGa~e9dyrKFgtCp&9y$>`bN;>J?7Py#iOQ@X0E$~cCQx!VZsXV zHeLjDa4iv@KBfm`^`+mSo1`yxW(KYK{F}bQNtWQAb726c;J)Amcq%)IiNu0It9qXC zNzaZFhJn6GxcxLBA>({UzQ^27z9s*Q3`m5-=A0Pi#8@_=g(PAsdg;)U_)T;hh%R?6 z6ibI-Fn<`q$oFJGQpf0R+kvu+Tz%NrO$Uxw5uEdA(?6G%{DZ;F<+mnY+lO_DpDruevAF2j zTN7WpH8BtQfXMUIybSeg%uo3OxXwt-KH4N53^MpX;`QB7{RR2^R~GD(e3M*zC2liB zS?i_L+007A!H1Wmi|E1z=a5)k*uKzE%<)N%kAX{pTOx_wE$dkg`&b8wMs4{)W|5%a z&<`#Ow8>~02&x_SBMM-b^-Ey#)ZNRMP<$LH62h->mZYX+=>GEwYrS) z46iQGaO+G3F=o+Q1HRlnBV+7~8^^=}$5vM@2+80zd2k}ehlc3Z>(ammZi>aPb?v%T z@2a|K_SKAzi^+22P?-H~>RVJgTK%M+>Eb8lzu0u7lrKWw*xX$ye`t|9@i}>7QRz8& z>q<&75yqW@gX+rDbeHU@>inhsNK09{eOpm!-Z_<<_=M7NeF}H~u_CeNUU~c@zb#9v{JYjT6K9sljou<*r!`JC z=R-HL(3;Vdd1ed^dy53>MqtDt!U-MrYiy@aUt2CCtdCc!pEgF(*m?~mI6 z@B&}NI(50t`hHPtz@9XrqQ+ll9~cypsME80NS>MQ3U?PbIT_1Of?f}$z8;4QFCPRz zr`LrvBAKHnpQNk;B3;s3TL07u*FafS`=-K>O5|)9rz8E=LlczSM<3KB%7uBi9j?o0xTUr!o~Qdsv8}e?v)*QncB{oLGX`&Qdq@t({OB)T^N5)DMXTDQ5l9iw zY4lcUl4MGK?Z?WaOLp5g+t+RwRMMP z@z2(TigEnS<*wSghY44TibS&xM!ekSCGBP+=sdo1cT*(O)jqOj-h+c;u12}2ukgeEGbj= zB6HW9tGJYpN{^d1?%1Y|OquV(*On=ABAs;ew^fju?YrYKiUc`gVQhv6C*PsKoH}KA z@}5lOZ-dI!F3$L}zBpTo&#kp&mCZel!FBe_U_>a+I81O~i|;O}novQP7lUSlEk3Z? zvH+G1yV`GqNj5y$F*vp-z_FngDkjCYG$zGX??bWaZrRw#V^eIOkA=pCx@2d!o~;xL z%g94a}A$)Gd5K&Lz&CQ2<(PT~+B{bTXPIq-}x5mn9Rh z;H5WDaUkAn@bMa|F|(IW=xyz65l}ni_9iaPf0_!I*WGkEhp_T}%kGD_VmD!{KON}y z19B|sd}mNvmo)OPLfzVH1zac9vHxAzjLJ*H0J9;9w_F~aK3JK4mlX(_i=l4ov%oR# z8uIUSmAqGzSjb7O*qSfvM|ZUA#A6f7-CmKRCdW5%4r>rYh&mCloXb>c(Tl@kOHi%P zhn!NNIZGU)ljgXRP?s6-Mah>tRfMaGoxJnP?D7sMZHCKrJK%*P^kN$3!5`N>qd9O^ z2_pAXAZIY}h%Se6{U8n0poA1+Fqy#C0L$nqT_@*597YSS0TxVjbYg&Z9f6Y>B+F~)aWfzRZ>^j4hIs6~_r@G6kn9W$9~!d9 zG=Zx)FJ0MYx#?NwZ-c_Ixg3+9mQ$Y}PBtZ`m3j5Yw$m1fb6hu1EI&FzrgrCBW8gIU zzNokJb^F2eC7Ul!Hn>s-DkLCO8>`5Y+Cdh^UKLbmZJq|!=@1_+r!6XXb#bI~LKbnH zEgc+51?t(EoX$5<6k)k0FyB{7$}Ww$pNXuPEO*3M zd*`MpeT+&SNYWBr<4m!nGku^h#bh#eJXiimEGfLiu!+BZkS)IlZ$Sx3vJ=Mum?yaK z0sa>_^y|!meP*Ll&1=9Zwmow#aS_jIAZg`7Aqj9~vs}Ebo}CB+$bTeyF!lICwxBIZ l-g(j9kY@ixkTrHM5}90hvAg@+FEii&?cK{~X7BCs{{X2|B6R=& literal 0 HcmV?d00001 From ef7de0aa05e7096922e8cb2d4d25017290347d6a Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 12 Apr 2021 14:30:32 +0200 Subject: [PATCH 33/45] feat: add persist flag to LifecycleRules For #1083. --- data_types/src/database_rules.rs | 6 ++++++ .../influxdata/iox/management/v1/database_rules.proto | 3 +++ src/commands/database.rs | 5 +++++ 3 files changed, 14 insertions(+) diff --git a/data_types/src/database_rules.rs b/data_types/src/database_rules.rs index c582d08fe3..9baa6aa860 100644 --- a/data_types/src/database_rules.rs +++ b/data_types/src/database_rules.rs @@ -199,6 +199,9 @@ pub struct LifecycleRules { /// Allow dropping data that has not been persisted to object storage pub drop_non_persisted: bool, + /// Persists chunks to object storage. + pub persist: bool, + /// Do not allow writing new data to this database pub immutable: bool, } @@ -228,6 +231,7 @@ impl From for management::LifecycleRules { .unwrap_or_default(), sort_order: Some(config.sort_order.into()), drop_non_persisted: config.drop_non_persisted, + persist: config.persist, immutable: config.immutable, } } @@ -245,6 +249,7 @@ impl TryFrom for LifecycleRules { buffer_size_hard: (proto.buffer_size_hard as usize).try_into().ok(), sort_order: proto.sort_order.optional("sort_order")?.unwrap_or_default(), drop_non_persisted: proto.drop_non_persisted, + persist: proto.persist, immutable: proto.immutable, }) } @@ -1288,6 +1293,7 @@ mod tests { buffer_size_hard: 232, sort_order: None, drop_non_persisted: true, + persist: true, immutable: true, }; diff --git a/generated_types/protos/influxdata/iox/management/v1/database_rules.proto b/generated_types/protos/influxdata/iox/management/v1/database_rules.proto index 8789631884..5ee66d0e1b 100644 --- a/generated_types/protos/influxdata/iox/management/v1/database_rules.proto +++ b/generated_types/protos/influxdata/iox/management/v1/database_rules.proto @@ -151,6 +151,9 @@ message LifecycleRules { // Allow dropping data that has not been persisted to object storage bool drop_non_persisted = 7; + // Persists chunks to object storage. + bool persist = 9; + // Do not allow writing new data to this database bool immutable = 8; } diff --git a/src/commands/database.rs b/src/commands/database.rs index c35580315c..29e12d5694 100644 --- a/src/commands/database.rs +++ b/src/commands/database.rs @@ -105,6 +105,10 @@ struct Create { #[structopt(long)] drop_non_persisted: bool, + /// Persists chunks to object storage. + #[structopt(long)] + persist: bool, + /// Do not allow writing new data to this database #[structopt(long)] immutable: bool, @@ -173,6 +177,7 @@ pub async fn command(url: String, config: Config) -> Result<()> { buffer_size_hard: command.buffer_size_hard as _, sort_order: None, // Server-side default drop_non_persisted: command.drop_non_persisted, + persist: command.persist, immutable: command.immutable, }), From 84b2ce45719c1e49a417af83ac5e4793304759ee Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 12 Apr 2021 15:03:28 +0200 Subject: [PATCH 34/45] docs: extend docs of `Db::load_chunk_to_read_buffer_in_background` --- server/src/db.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/db.rs b/server/src/db.rs index 7ecf2ab820..3fe6b69e86 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -565,7 +565,8 @@ impl Db { Ok(DBChunk::snapshot(&chunk)) } - /// Spawns a task to perform load_chunk_to_read_buffer + /// Spawns a task to perform + /// [`load_chunk_to_read_buffer`](Self::load_chunk_to_read_buffer) pub fn load_chunk_to_read_buffer_in_background( self: &Arc, partition_key: String, From 440cef59dff055d2d5f9e6ad93b0d64c6478e4b9 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 12 Apr 2021 15:03:48 +0200 Subject: [PATCH 35/45] docs: fix docs of `LifecycleRules::buffer_size_soft` --- data_types/src/database_rules.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data_types/src/database_rules.rs b/data_types/src/database_rules.rs index 9baa6aa860..42a4f3e17a 100644 --- a/data_types/src/database_rules.rs +++ b/data_types/src/database_rules.rs @@ -181,7 +181,7 @@ pub struct LifecycleRules { pub mutable_size_threshold: Option, /// Once the total amount of buffered data in memory reaches this size start - /// dropping data from memory based on the drop_order + /// dropping data from memory based on the [`sort_order`](Self::sort_order) pub buffer_size_soft: Option, /// Once the amount of data in memory reaches this size start From e516f65cbe1fd22c140b002613091b944bdb2603 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 12 Apr 2021 15:13:14 +0200 Subject: [PATCH 36/45] feat: add WriteChunk job For #1083. --- data_types/src/job.rs | 25 +++++++++++++++++++ .../influxdata/iox/management/v1/jobs.proto | 13 ++++++++++ 2 files changed, 38 insertions(+) diff --git a/data_types/src/job.rs b/data_types/src/job.rs index 0a308bdda9..5ad4e7f1ec 100644 --- a/data_types/src/job.rs +++ b/data_types/src/job.rs @@ -27,6 +27,13 @@ pub enum Job { partition_key: String, chunk_id: u32, }, + + /// Write a chunk from read buffer to object store + WriteChunk { + db_name: String, + partition_key: String, + chunk_id: u32, + }, } impl From for management::operation_metadata::Job { @@ -49,6 +56,15 @@ impl From for management::operation_metadata::Job { partition_key, chunk_id, }), + Job::WriteChunk { + db_name, + partition_key, + chunk_id, + } => Self::WriteChunk(management::WriteChunk { + db_name, + partition_key, + chunk_id, + }), } } } @@ -74,6 +90,15 @@ impl From for Job { partition_key, chunk_id, }, + Job::WriteChunk(management::WriteChunk { + db_name, + partition_key, + chunk_id, + }) => Self::WriteChunk { + db_name, + partition_key, + chunk_id, + }, } } } diff --git a/generated_types/protos/influxdata/iox/management/v1/jobs.proto b/generated_types/protos/influxdata/iox/management/v1/jobs.proto index a04ecc259f..a95006aef2 100644 --- a/generated_types/protos/influxdata/iox/management/v1/jobs.proto +++ b/generated_types/protos/influxdata/iox/management/v1/jobs.proto @@ -19,6 +19,7 @@ message OperationMetadata { Dummy dummy = 5; PersistSegment persist_segment = 6; CloseChunk close_chunk = 7; + WriteChunk write_chunk = 8; } } @@ -45,3 +46,15 @@ message CloseChunk { // chunk_id uint32 chunk_id = 3; } + +// Write a chunk from read buffer to object store +message WriteChunk { + // name of the database + string db_name = 1; + + // partition key + string partition_key = 2; + + // chunk_id + uint32 chunk_id = 3; +} \ No newline at end of file From 7b0b975433d6134631146f410f1eb825d6dd7d8c Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 12 Apr 2021 15:05:50 +0200 Subject: [PATCH 37/45] feat: add Db::load_chunk_to_object_store_in_background For #1083. --- server/src/db.rs | 72 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 72 insertions(+) diff --git a/server/src/db.rs b/server/src/db.rs index 3fe6b69e86..c2c51b2a35 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -600,6 +600,41 @@ impl Db { tracker } + /// Spawns a task to perform + /// [`load_chunk_to_object_store`](Self::load_chunk_to_object_store) + pub fn load_chunk_to_object_store_in_background( + self: &Arc, + partition_key: String, + chunk_id: u32, + ) -> TaskTracker { + let name = self.rules.read().name.clone(); + let (tracker, registration) = self.jobs.register(Job::WriteChunk { + db_name: name.to_string(), + partition_key: partition_key.clone(), + chunk_id, + }); + + let captured = Arc::clone(&self); + let task = async move { + debug!(%name, %partition_key, %chunk_id, "background task loading chunk to object store"); + let result = captured + .load_chunk_to_object_store(&partition_key, chunk_id) + .await; + if let Err(e) = result { + info!(?e, %name, %partition_key, %chunk_id, "background task error loading object store chunk"); + return Err(e); + } + + debug!(%name, %partition_key, %chunk_id, "background task completed writing chunk to object store"); + + Ok(()) + }; + + tokio::spawn(task.track(registration)); + + tracker + } + /// Returns the next write sequence number pub fn next_sequence(&self) -> u64 { self.sequence.fetch_add(1, Ordering::SeqCst) @@ -1717,4 +1752,41 @@ mod tests { chunk_ids.sort_unstable(); chunk_ids } + + #[tokio::test] + async fn load_chunk_to_object_store_in_background() { + // Test that data can be written to object store using a background task + let db = Arc::new(make_db()); + + // create MB partition + write_lp(db.as_ref(), "cpu bar=1 10"); + write_lp(db.as_ref(), "cpu bar=2 20"); + + // MB => RB + let partition_key = "1970-01-01T00"; + let mb_chunk = db.rollover_partition(partition_key).await.unwrap(); + let rb_chunk = db + .load_chunk_to_read_buffer(partition_key, mb_chunk.id()) + .await + .unwrap(); + assert_eq!(mb_chunk.id(), rb_chunk.id()); + + // RB => OS + let task = + db.load_chunk_to_object_store_in_background(partition_key.to_string(), rb_chunk.id()); + let t_start = std::time::Instant::now(); + while !task.is_complete() { + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + assert!( + std::time::Instant::now() - t_start < std::time::Duration::from_secs(10), + "task deadline exceeded" + ); + } + + // we should have chunks in the mutable buffer, read buffer, and object store + // (Note the currently open chunk is not listed) + assert_eq!(mutable_chunk_ids(&db, partition_key), vec![1]); + assert_eq!(read_buffer_chunk_ids(&db, partition_key), vec![0]); + assert_eq!(read_parquet_file_chunk_ids(&db, partition_key), vec![0]); + } } From 26ce520cf1714f83cae54156819d9b12fbe755d7 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Tue, 13 Apr 2021 10:21:10 +0200 Subject: [PATCH 38/45] feat: hook RB->OS into lifecycle Fixes #1083. --- server/src/db/lifecycle.rs | 251 ++++++++++++++++++++++++++++++++----- 1 file changed, 220 insertions(+), 31 deletions(-) diff --git a/server/src/db/lifecycle.rs b/server/src/db/lifecycle.rs index 73470f29b6..1108de49ae 100644 --- a/server/src/db/lifecycle.rs +++ b/server/src/db/lifecycle.rs @@ -20,6 +20,7 @@ pub struct LifecycleManager { db: Arc, db_name: String, move_task: Option>, + write_task: Option>, } impl LifecycleManager { @@ -30,6 +31,7 @@ impl LifecycleManager { db, db_name, move_task: None, + write_task: None, } } @@ -65,9 +67,15 @@ trait ChunkMover { /// Returns a boolean indicating if a move is in progress fn is_move_active(&self) -> bool; + /// Returns a boolean indicating if a write is in progress + fn is_write_active(&self) -> bool; + /// Starts an operation to move a chunk to the read buffer fn move_to_read_buffer(&mut self, partition_key: String, chunk_id: u32); + /// Starts an operation to write a chunk to the object store + fn write_to_object_store(&mut self, partition_key: String, chunk_id: u32); + /// Drops a chunk from the database fn drop_chunk(&mut self, partition_key: String, chunk_id: u32); @@ -78,10 +86,11 @@ trait ChunkMover { let mut buffer_size = 0; - // Only want to start a new move task if there isn't one already in-flight + // Only want to start a new move/write task if there isn't one already in-flight // // Note: This does not take into account manually triggered tasks let mut move_active = self.is_move_active(); + let mut write_active = self.is_write_active(); // Iterate through the chunks to determine // - total memory consumption @@ -90,33 +99,44 @@ trait ChunkMover { // TODO: Track size globally to avoid iterating through all chunks (#1100) for chunk in &chunks { let chunk_guard = chunk.upgradable_read(); + buffer_size += Self::chunk_size(&*chunk_guard); - if !move_active && can_move(&rules, &*chunk_guard, now) { - match chunk_guard.state() { - ChunkState::Open(_) => { - let mut chunk_guard = RwLockUpgradableReadGuard::upgrade(chunk_guard); - chunk_guard.set_closing().expect("cannot close open chunk"); + let would_move = !move_active && can_move(&rules, &*chunk_guard, now); + let would_write = !write_active && rules.persist; - let partition_key = chunk_guard.key().to_string(); - let chunk_id = chunk_guard.id(); + match chunk_guard.state() { + ChunkState::Open(_) if would_move => { + let mut chunk_guard = RwLockUpgradableReadGuard::upgrade(chunk_guard); + chunk_guard.set_closing().expect("cannot close open chunk"); - std::mem::drop(chunk_guard); + let partition_key = chunk_guard.key().to_string(); + let chunk_id = chunk_guard.id(); - move_active = true; - self.move_to_read_buffer(partition_key, chunk_id); - } - ChunkState::Closing(_) => { - let partition_key = chunk_guard.key().to_string(); - let chunk_id = chunk_guard.id(); + std::mem::drop(chunk_guard); - std::mem::drop(chunk_guard); - - move_active = true; - self.move_to_read_buffer(partition_key, chunk_id); - } - _ => {} + move_active = true; + self.move_to_read_buffer(partition_key, chunk_id); } + ChunkState::Closing(_) if would_move => { + let partition_key = chunk_guard.key().to_string(); + let chunk_id = chunk_guard.id(); + + std::mem::drop(chunk_guard); + + move_active = true; + self.move_to_read_buffer(partition_key, chunk_id); + } + ChunkState::Moved(_) if would_write => { + let partition_key = chunk_guard.key().to_string(); + let chunk_id = chunk_guard.id(); + + std::mem::drop(chunk_guard); + + write_active = true; + self.write_to_object_store(partition_key, chunk_id); + } + _ => {} } // TODO: Find and recover cancelled move jobs (#1099) @@ -129,8 +149,9 @@ trait ChunkMover { match chunks.next() { Some(chunk) => { let chunk_guard = chunk.read(); - if rules.drop_non_persisted - || matches!(chunk_guard.state(), ChunkState::Moved(_)) + if (rules.drop_non_persisted + && matches!(chunk_guard.state(), ChunkState::Moved(_))) + || matches!(chunk_guard.state(), ChunkState::WrittenToObjectStore(_, _)) { let partition_key = chunk_guard.key().to_string(); let chunk_id = chunk_guard.id(); @@ -169,6 +190,13 @@ impl ChunkMover for LifecycleManager { .unwrap_or(false) } + fn is_write_active(&self) -> bool { + self.write_task + .as_ref() + .map(|x| !x.is_complete()) + .unwrap_or(false) + } + fn move_to_read_buffer(&mut self, partition_key: String, chunk_id: u32) { info!(%partition_key, %chunk_id, "moving chunk to read buffer"); self.move_task = Some( @@ -177,6 +205,14 @@ impl ChunkMover for LifecycleManager { ) } + fn write_to_object_store(&mut self, partition_key: String, chunk_id: u32) { + info!(%partition_key, %chunk_id, "write chunk to object store"); + self.write_task = Some( + self.db + .load_chunk_to_object_store_in_background(partition_key, chunk_id), + ) + } + fn drop_chunk(&mut self, partition_key: String, chunk_id: u32) { info!(%partition_key, %chunk_id, "dropping chunk"); let _ = self @@ -251,9 +287,57 @@ mod tests { chunk } + /// Transitions a new ("open") chunk into the "moving" state. + fn transition_to_moving(mut chunk: Chunk) -> Chunk { + chunk.set_closing().unwrap(); + chunk.set_moving().unwrap(); + chunk + } + + /// Transitions a new ("open") chunk into the "moved" state. + fn transition_to_moved(mut chunk: Chunk, rb: &Arc) -> Chunk { + chunk = transition_to_moving(chunk); + chunk.set_moved(Arc::clone(&rb)).unwrap(); + chunk + } + + /// Transitions a new ("open") chunk into the "writing to object store" + /// state. + fn transition_to_writing_to_object_store( + mut chunk: Chunk, + rb: &Arc, + ) -> Chunk { + chunk = transition_to_moved(chunk, rb); + chunk.set_writing_to_object_store().unwrap(); + chunk + } + + /// Transitions a new ("open") chunk into the "written to object store" + /// state. + fn transition_to_written_to_object_store( + mut chunk: Chunk, + rb: &Arc, + ) -> Chunk { + chunk = transition_to_writing_to_object_store(chunk, rb); + let parquet_chunk = new_parquet_chunk(&chunk); + chunk + .set_written_to_object_store(Arc::new(parquet_chunk)) + .unwrap(); + chunk + } + + fn new_parquet_chunk(chunk: &Chunk) -> parquet_file::chunk::Chunk { + parquet_file::chunk::Chunk::new( + chunk.key().to_string(), + chunk.id(), + &tracker::MemRegistry::new(), + ) + } + #[derive(Debug, Eq, PartialEq)] enum MoverEvents { Move(u32), + Write(u32), Drop(u32), } @@ -262,6 +346,7 @@ mod tests { struct DummyMover { rules: LifecycleRules, move_active: bool, + write_active: bool, chunks: Vec>>, events: Vec, } @@ -275,6 +360,7 @@ mod tests { .map(|x| Arc::new(RwLock::new(x))) .collect(), move_active: false, + write_active: false, events: vec![], } } @@ -298,6 +384,10 @@ mod tests { self.move_active } + fn is_write_active(&self) -> bool { + self.write_active + } + fn move_to_read_buffer(&mut self, _: String, chunk_id: u32) { let chunk = self .chunks @@ -308,7 +398,22 @@ mod tests { self.events.push(MoverEvents::Move(chunk_id)) } + fn write_to_object_store(&mut self, _partition_key: String, chunk_id: u32) { + let chunk = self + .chunks + .iter() + .find(|x| x.read().id() == chunk_id) + .unwrap(); + chunk.write().set_writing_to_object_store().unwrap(); + self.events.push(MoverEvents::Write(chunk_id)) + } + fn drop_chunk(&mut self, _: String, chunk_id: u32) { + self.chunks = self + .chunks + .drain(..) + .filter(|x| x.read().id() != chunk_id) + .collect(); self.events.push(MoverEvents::Drop(chunk_id)) } @@ -467,7 +572,56 @@ mod tests { } #[test] - fn test_buffer_size_soft() { + fn test_buffer_size_soft_drop_non_persisted() { + // test that chunk mover only drops moved and written chunks + + // IMPORTANT: the lifecycle rules have the default `persist` flag (false) so NOT + // "write" events will be triggered + let rules = LifecycleRules { + buffer_size_soft: Some(NonZeroUsize::new(5).unwrap()), + drop_non_persisted: true, + ..Default::default() + }; + + let rb = Arc::new(read_buffer::Chunk::new_with_memory_tracker( + 22, + &tracker::MemRegistry::new(), + )); + + let chunks = vec![new_chunk(0, Some(0), Some(0))]; + + let mut mover = DummyMover::new(rules.clone(), chunks); + + mover.check_for_work(from_secs(10)); + assert_eq!(mover.events, vec![]); + + let chunks = vec![ + // two "open" chunks => they must not be dropped (yet) + new_chunk(0, Some(0), Some(0)), + new_chunk(1, Some(0), Some(0)), + // "moved" chunk => can be dropped because `drop_non_persistent=true` + transition_to_moved(new_chunk(2, Some(0), Some(0)), &rb), + // "writing" chunk => cannot be drop while write is in-progess + transition_to_writing_to_object_store(new_chunk(3, Some(0), Some(0)), &rb), + // "written" chunk => can be dropped + transition_to_written_to_object_store(new_chunk(4, Some(0), Some(0)), &rb), + ]; + + let mut mover = DummyMover::new(rules, chunks); + + mover.check_for_work(from_secs(10)); + assert_eq!( + mover.events, + vec![MoverEvents::Drop(2), MoverEvents::Drop(4)] + ); + } + + #[test] + fn test_buffer_size_soft_dont_drop_non_persisted() { + // test that chunk mover only drops written chunks + + // IMPORTANT: the lifecycle rules have the default `persist` flag (false) so NOT + // "write" events will be triggered let rules = LifecycleRules { buffer_size_soft: Some(NonZeroUsize::new(5).unwrap()), ..Default::default() @@ -485,21 +639,27 @@ mod tests { mover.check_for_work(from_secs(10)); assert_eq!(mover.events, vec![]); - let mut chunks = vec![ + let chunks = vec![ + // two "open" chunks => they must not be dropped (yet) new_chunk(0, Some(0), Some(0)), new_chunk(1, Some(0), Some(0)), - new_chunk(2, Some(0), Some(0)), + // "moved" chunk => cannot be dropped because `drop_non_persistent=false` + transition_to_moved(new_chunk(2, Some(0), Some(0)), &rb), + // "writing" chunk => cannot be drop while write is in-progess + transition_to_writing_to_object_store(new_chunk(3, Some(0), Some(0)), &rb), + // "written" chunk => can be dropped + transition_to_written_to_object_store(new_chunk(4, Some(0), Some(0)), &rb), ]; - chunks[2].set_closing().unwrap(); - chunks[2].set_moving().unwrap(); - chunks[2].set_moved(Arc::clone(&rb)).unwrap(); - let mut mover = DummyMover::new(rules, chunks); mover.check_for_work(from_secs(10)); - assert_eq!(mover.events, vec![MoverEvents::Drop(2)]); + assert_eq!(mover.events, vec![MoverEvents::Drop(4)]); + } + #[test] + fn test_buffer_size_soft_no_op() { + // check that we don't drop anything if nothing is to drop let rules = LifecycleRules { buffer_size_soft: Some(NonZeroUsize::new(40).unwrap()), ..Default::default() @@ -512,4 +672,33 @@ mod tests { mover.check_for_work(from_secs(10)); assert_eq!(mover.events, vec![]); } + + #[test] + fn test_persist() { + let rules = LifecycleRules { + mutable_linger_seconds: Some(NonZeroU32::new(10).unwrap()), + persist: true, + ..Default::default() + }; + + let rb = Arc::new(read_buffer::Chunk::new_with_memory_tracker( + 22, + &tracker::MemRegistry::new(), + )); + + let chunks = vec![ + // still moving => cannot write + transition_to_moving(new_chunk(0, Some(0), Some(0))), + // moved => write to object store + transition_to_moved(new_chunk(1, Some(0), Some(0)), &rb), + // moved, but there will be already a write in progress (previous chunk) => don't write + transition_to_moved(new_chunk(2, Some(0), Some(0)), &rb), + ]; + + let mut mover = DummyMover::new(rules, chunks); + + mover.check_for_work(from_secs(0)); + + assert_eq!(mover.events, vec![MoverEvents::Write(1)]); + } } From 1c6690ae3bb76cb0e30f7a9c7f3b0497b9dedd91 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Tue, 13 Apr 2021 15:34:04 +0200 Subject: [PATCH 39/45] fix: fix constant use in `server::db::tests::read_from_read_buffer` --- server/src/db.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/db.rs b/server/src/db.rs index c2c51b2a35..685e922186 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -989,7 +989,7 @@ mod tests { write_lp(db.as_ref(), "cpu bar=2 20"); let partition_key = "1970-01-01T00"; - let mb_chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); + let mb_chunk = db.rollover_partition(partition_key).await.unwrap(); let rb_chunk = db .load_chunk_to_read_buffer(partition_key, mb_chunk.id()) .await From f43001314175daa72c79ef92baba5577ad4e0a98 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Tue, 13 Apr 2021 15:39:24 +0200 Subject: [PATCH 40/45] refactor: rename `Db::load_chunk_to_object_store` to `Db::write_chunk_to_object_store For #1083. --- server/src/db.rs | 16 ++++++++-------- server/src/db/lifecycle.rs | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/server/src/db.rs b/server/src/db.rs index 685e922186..a834335d92 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -447,7 +447,7 @@ impl Db { Ok(DBChunk::snapshot(&chunk)) } - pub async fn load_chunk_to_object_store( + pub async fn write_chunk_to_object_store( &self, partition_key: &str, chunk_id: u32, @@ -601,8 +601,8 @@ impl Db { } /// Spawns a task to perform - /// [`load_chunk_to_object_store`](Self::load_chunk_to_object_store) - pub fn load_chunk_to_object_store_in_background( + /// [`write_chunk_to_object_store`](Self::write_chunk_to_object_store) + pub fn write_chunk_to_object_store_in_background( self: &Arc, partition_key: String, chunk_id: u32, @@ -618,7 +618,7 @@ impl Db { let task = async move { debug!(%name, %partition_key, %chunk_id, "background task loading chunk to object store"); let result = captured - .load_chunk_to_object_store(&partition_key, chunk_id) + .write_chunk_to_object_store(&partition_key, chunk_id) .await; if let Err(e) = result { info!(?e, %name, %partition_key, %chunk_id, "background task error loading object store chunk"); @@ -1069,7 +1069,7 @@ mod tests { .unwrap(); // Write the RB chunk to Object Store but keep it in RB let pq_chunk = db - .load_chunk_to_object_store(partition_key, mb_chunk.id()) + .write_chunk_to_object_store(partition_key, mb_chunk.id()) .await .unwrap(); @@ -1157,7 +1157,7 @@ mod tests { .unwrap(); // Write the RB chunk to Object Store but keep it in RB let pq_chunk = db - .load_chunk_to_object_store(partition_key, mb_chunk.id()) + .write_chunk_to_object_store(partition_key, mb_chunk.id()) .await .unwrap(); @@ -1754,7 +1754,7 @@ mod tests { } #[tokio::test] - async fn load_chunk_to_object_store_in_background() { + async fn write_chunk_to_object_store_in_background() { // Test that data can be written to object store using a background task let db = Arc::new(make_db()); @@ -1773,7 +1773,7 @@ mod tests { // RB => OS let task = - db.load_chunk_to_object_store_in_background(partition_key.to_string(), rb_chunk.id()); + db.write_chunk_to_object_store_in_background(partition_key.to_string(), rb_chunk.id()); let t_start = std::time::Instant::now(); while !task.is_complete() { tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; diff --git a/server/src/db/lifecycle.rs b/server/src/db/lifecycle.rs index 1108de49ae..3bcb5ecfb6 100644 --- a/server/src/db/lifecycle.rs +++ b/server/src/db/lifecycle.rs @@ -209,7 +209,7 @@ impl ChunkMover for LifecycleManager { info!(%partition_key, %chunk_id, "write chunk to object store"); self.write_task = Some( self.db - .load_chunk_to_object_store_in_background(partition_key, chunk_id), + .write_chunk_to_object_store_in_background(partition_key, chunk_id), ) } From f5f768d7505b2a391b5b76596d2c585c73eb5293 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 14 Apr 2021 06:48:09 -0400 Subject: [PATCH 41/45] feat: Add a dedicated threadpool for running queries (#1191) * feat: use a dedicated tokio threadpool for running queries * feat: plumb number of executor threads through to command line thread through command line * fix: Logical merge conflict * fix: another logical conflict Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- Cargo.lock | 2 + query/Cargo.toml | 4 + query/src/exec.rs | 41 +- query/src/exec/context.rs | 39 +- query/src/exec/task.rs | 368 ++++++++++++++++++ query/src/plan/stringset.rs | 2 +- query/src/test.rs | 2 +- server/Cargo.toml | 1 + server/benches/tag_values.rs | 2 +- server/src/db.rs | 2 +- server/src/lib.rs | 78 +++- .../query_tests/influxrpc/field_columns.rs | 4 +- .../src/query_tests/influxrpc/read_filter.rs | 2 +- .../src/query_tests/influxrpc/read_group.rs | 2 +- .../influxrpc/read_window_aggregate.rs | 2 +- .../src/query_tests/influxrpc/table_names.rs | 2 +- server/src/query_tests/influxrpc/tag_keys.rs | 2 +- .../src/query_tests/influxrpc/tag_values.rs | 2 +- server/src/query_tests/sql.rs | 2 +- src/commands/run.rs | 10 + src/influxdb_ioxd.rs | 18 +- src/influxdb_ioxd/http.rs | 95 ++--- src/influxdb_ioxd/rpc/flight.rs | 6 +- 23 files changed, 571 insertions(+), 117 deletions(-) create mode 100644 query/src/exec/task.rs diff --git a/Cargo.lock b/Cargo.lock index 2cda0a48d8..7554a29440 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2650,6 +2650,7 @@ dependencies = [ "futures", "influxdb_line_protocol", "internal_types", + "libc", "observability_deps", "parking_lot", "snafu", @@ -3273,6 +3274,7 @@ dependencies = [ "influxdb_line_protocol", "internal_types", "mutable_buffer", + "num_cpus", "object_store", "observability_deps", "parking_lot", diff --git a/query/Cargo.toml b/query/Cargo.toml index bfa7d705b7..92ba3c980d 100644 --- a/query/Cargo.toml +++ b/query/Cargo.toml @@ -29,5 +29,9 @@ tokio = { version = "1.0", features = ["macros"] } tokio-stream = "0.1.2" observability_deps = { path = "../observability_deps" } +# use libc on unix like platforms to set worker priority in DedicatedExecutor +[target."cfg(unix)".dependencies.libc] +version = "0.2" + [dev-dependencies] # In alphabetical order test_helpers = { path = "../test_helpers" } diff --git a/query/src/exec.rs b/query/src/exec.rs index e3a3611f33..4f58a247fd 100644 --- a/query/src/exec.rs +++ b/query/src/exec.rs @@ -8,6 +8,7 @@ pub mod fieldlist; mod schema_pivot; pub mod seriesset; pub mod stringset; +mod task; pub use context::{DEFAULT_CATALOG, DEFAULT_SCHEMA}; use std::sync::Arc; @@ -34,6 +35,8 @@ use crate::plan::{ stringset::StringSetPlan, }; +use self::task::DedicatedExecutor; + #[derive(Debug, Snafu)] pub enum Error { #[snafu(display("Plan Execution Error: {}", source))] @@ -89,16 +92,24 @@ pub enum Error { pub type Result = std::result::Result; -/// Handles executing plans, and marshalling the results into rust +/// Handles executing DataFusion plans, and marshalling the results into rust /// native structures. -#[derive(Debug, Default)] +#[derive(Debug)] pub struct Executor { counters: Arc, + exec: DedicatedExecutor, } impl Executor { - pub fn new() -> Self { - Self::default() + /// Creates a new executor with a single dedicated thread pool with + /// num_threads + pub fn new(num_threads: usize) -> Self { + let exec = DedicatedExecutor::new("IOx Executor Thread", num_threads); + + Self { + exec, + counters: Arc::new(ExecutionCounters::default()), + } } /// Executes this plan and returns the resulting set of strings @@ -212,10 +223,8 @@ impl Executor { let handles = plans .into_iter() .map(|plan| { - let counters = Arc::clone(&self.counters); - + let ctx = self.new_context(); tokio::task::spawn(async move { - let ctx = IOxExecutionContext::new(counters); let physical_plan = ctx .prepare_plan(&plan) .await @@ -252,7 +261,7 @@ impl Executor { /// Create a new execution context, suitable for executing a new query pub fn new_context(&self) -> IOxExecutionContext { - IOxExecutionContext::new(Arc::clone(&self.counters)) + IOxExecutionContext::new(self.exec.clone(), Arc::clone(&self.counters)) } /// plans and runs the plans in parallel and collects the results @@ -327,7 +336,7 @@ mod tests { let expected_strings = to_set(&["Foo", "Bar"]); let plan = StringSetPlan::Known(Arc::clone(&expected_strings)); - let executor = Executor::default(); + let executor = Executor::new(1); let result_strings = executor.to_string_set(plan).await.unwrap(); assert_eq!(result_strings, expected_strings); } @@ -339,7 +348,7 @@ mod tests { let scan = make_plan(schema, vec![]); let plan: StringSetPlan = vec![scan].into(); - let executor = Executor::new(); + let executor = Executor::new(1); let results = executor.to_string_set(plan).await.unwrap(); assert_eq!(results, StringSetRef::new(StringSet::new())); @@ -355,7 +364,7 @@ mod tests { let scan = make_plan(schema, vec![batch]); let plan: StringSetPlan = vec![scan].into(); - let executor = Executor::new(); + let executor = Executor::new(1); let results = executor.to_string_set(plan).await.unwrap(); assert_eq!(results, to_set(&["foo", "bar", "baz"])); @@ -374,7 +383,7 @@ mod tests { let scan = make_plan(schema, vec![batch1, batch2]); let plan: StringSetPlan = vec![scan].into(); - let executor = Executor::new(); + let executor = Executor::new(1); let results = executor.to_string_set(plan).await.unwrap(); assert_eq!(results, to_set(&["foo", "bar", "baz"])); @@ -397,7 +406,7 @@ mod tests { let plan: StringSetPlan = vec![scan1, scan2].into(); - let executor = Executor::new(); + let executor = Executor::new(1); let results = executor.to_string_set(plan).await.unwrap(); assert_eq!(results, to_set(&["foo", "bar", "baz"])); @@ -417,7 +426,7 @@ mod tests { let scan = make_plan(schema, vec![batch]); let plan: StringSetPlan = vec![scan].into(); - let executor = Executor::new(); + let executor = Executor::new(1); let results = executor.to_string_set(plan).await; let actual_error = match results { @@ -443,7 +452,7 @@ mod tests { let scan = make_plan(schema, vec![batch]); let plan: StringSetPlan = vec![scan].into(); - let executor = Executor::new(); + let executor = Executor::new(1); let results = executor.to_string_set(plan).await; let actual_error = match results { @@ -481,7 +490,7 @@ mod tests { let pivot = make_schema_pivot(scan); let plan = vec![pivot].into(); - let executor = Executor::new(); + let executor = Executor::new(1); let results = executor.to_string_set(plan).await.expect("Executed plan"); assert_eq!(results, to_set(&["f1", "f2"])); diff --git a/query/src/exec/context.rs b/query/src/exec/context.rs index a5d60f2e84..042b48347b 100644 --- a/query/src/exec/context.rs +++ b/query/src/exec/context.rs @@ -25,7 +25,7 @@ use observability_deps::tracing::debug; // Reuse DataFusion error and Result types for this module pub use arrow_deps::datafusion::error::{DataFusionError as Error, Result}; -use super::counters::ExecutionCounters; +use super::{counters::ExecutionCounters, task::DedicatedExecutor}; // The default catalog name - this impacts what SQL queries use if not specified pub const DEFAULT_CATALOG: &str = "public"; @@ -77,15 +77,27 @@ impl ExtensionPlanner for IOxExtensionPlanner { } } -/// This is an execution context for planning in IOx. -/// It wraps a DataFusion execution context and incudes -/// statistical counters. +/// This is an execution context for planning in IOx. It wraps a +/// DataFusion execution context and incudes statistical counters and +/// a dedicated thread pool. /// -/// Eventually we envision this as also managing resources -/// and providing visibility into what plans are running +/// Methods on this struct should be preferred to using the raw +/// DataFusion functions (such as `collect`) directly. +/// +/// Eventually we envision this also managing additional resource +/// types such as Memory and providing visibility into what plans are +/// running pub struct IOxExecutionContext { counters: Arc, inner: ExecutionContext, + + /// Dedicated executor for query execution. + /// + /// DataFusion plans are "CPU" bound and thus can consume tokio + /// executors threads for extended periods of time. We use a + /// dedicated tokio runtime to run them so that other requests + /// can be handled. + exec: DedicatedExecutor, } impl fmt::Debug for IOxExecutionContext { @@ -102,7 +114,7 @@ impl IOxExecutionContext { /// /// The config is created with a default catalog and schema, but this /// can be overridden at a later date - pub fn new(counters: Arc) -> Self { + pub fn new(exec: DedicatedExecutor, counters: Arc) -> Self { const BATCH_SIZE: usize = 1000; // TBD: Should we be reusing an execution context across all executions? @@ -115,7 +127,11 @@ impl IOxExecutionContext { let inner = ExecutionContext::with_config(config); - Self { counters, inner } + Self { + exec, + counters, + inner, + } } /// returns a reference to the inner datafusion execution context @@ -154,13 +170,16 @@ impl IOxExecutionContext { self.inner.create_physical_plan(&plan) } - /// Executes the logical plan using DataFusion and produces RecordBatches + /// Executes the logical plan using DataFusion on a separate + /// thread pool and produces RecordBatches pub async fn collect(&self, physical_plan: Arc) -> Result> { self.counters.inc_plans_run(); debug!("Running plan, physical:\n{:?}", physical_plan); - collect(physical_plan).await + self.exec.spawn(collect(physical_plan)).await.map_err(|e| { + Error::Execution(format!("Error running IOxExecutionContext::collect: {}", e)) + })? } /// Executes the physical plan and produces a RecordBatchStream to stream diff --git a/query/src/exec/task.rs b/query/src/exec/task.rs new file mode 100644 index 0000000000..501846e434 --- /dev/null +++ b/query/src/exec/task.rs @@ -0,0 +1,368 @@ +//! This module contains a dedicated thread pool for running "cpu +//! intensive" workloads such as DataFusion plans + +use parking_lot::Mutex; +use std::{pin::Pin, sync::Arc}; +use tokio::sync::oneshot::Receiver; + +use futures::Future; + +use observability_deps::tracing::warn; + +/// The type of thing that the dedicated executor runs +type Task = Pin + Send>>; + +/// Runs futures (and any `tasks` that are `tokio::task::spawned` by +/// them) on a separate tokio Executor +#[derive(Clone)] +pub struct DedicatedExecutor { + state: Arc>, +} + +/// Runs futures (and any `tasks` that are `tokio::task::spawned` by +/// them) on a separate tokio Executor +struct State { + /// The number of threads in this pool + num_threads: usize, + + /// The name of the threads for this executor + thread_name: String, + + /// Channel for requests -- the dedicated executor takes requests + /// from here and runs them. + requests: Option>, + + /// The thread that is doing the work + thread: Option>, +} + +/// The default worker priority (value passed to `libc::setpriority`); +const WORKER_PRIORITY: i32 = 10; + +impl std::fmt::Debug for DedicatedExecutor { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let state = self.state.lock(); + + let mut d = f.debug_struct("DedicatedExecutor"); + + d.field("num_threads", &state.num_threads) + .field("thread_name", &state.thread_name); + + if state.requests.is_some() { + d.field("requests", &"Some(...)") + } else { + d.field("requests", &"None") + }; + + if state.thread.is_some() { + d.field("thread", &"Some(...)") + } else { + d.field("thread", &"None") + }; + + d.finish() + } +} + +impl DedicatedExecutor { + /// Creates a new `DedicatedExecutor` with a dedicated tokio + /// executor that is separate from the threadpool created via + /// `[tokio::main]` or similar. + /// + /// The worker thread priority is set to low so that such tasks do + /// not starve other more important tasks (such as answering health checks) + /// + /// Follows the example from to stack overflow and spawns a new + /// thread to install a Tokio runtime "context" + /// https://stackoverflow.com/questions/62536566 + /// + /// If you try to do this from a async context you see something like + /// thread 'plan::stringset::tests::test_builder_plan' panicked at 'Cannot + /// drop a runtime in a context where blocking is not allowed. This + /// happens when a runtime is dropped from within an asynchronous + /// context.', .../tokio-1.4.0/src/runtime/blocking/shutdown.rs:51:21 + pub fn new(thread_name: impl Into, num_threads: usize) -> Self { + let thread_name = thread_name.into(); + let name_copy = thread_name.to_string(); + + let (tx, rx) = std::sync::mpsc::channel(); + + let thread = std::thread::spawn(move || { + let runtime = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .thread_name(&name_copy) + .worker_threads(num_threads) + .on_thread_start(move || set_current_thread_priority(WORKER_PRIORITY)) + .build() + .expect("Creating tokio runtime"); + + // By entering the context, all calls to `tokio::spawn` go + // to this executor + let _guard = runtime.enter(); + + while let Ok(request) = rx.recv() { + // TODO track the outstanding tasks + tokio::task::spawn(request); + } + }); + + let state = State { + num_threads, + thread_name, + requests: Some(tx), + thread: Some(thread), + }; + + Self { + state: Arc::new(Mutex::new(state)), + } + } + + /// Runs the specified Future (and any tasks it spawns) on the + /// `DedicatedExecutor`. + /// + /// Currently all tasks are added to the tokio executor + /// immediately and compete for the threadpool's resources. + pub fn spawn(&self, task: T) -> Receiver + where + T: Future + Send + 'static, + T::Output: Send + 'static, + { + let (tx, rx) = tokio::sync::oneshot::channel(); + + let job = Box::pin(async move { + let task_output = task.await; + if tx.send(task_output).is_err() { + warn!("Spawned task output ignored: receiver dropped") + } + }); + + let mut state = self.state.lock(); + + if let Some(requests) = &mut state.requests { + // would fail if someone has started shutdown + requests.send(job).ok(); + } else { + warn!("tried to schedule task on an executor that was shutdown"); + } + + rx + } + + /// signals shutdown of this executor and any Clones + pub fn shutdown(&self) { + // hang up the channel which will cause the dedicated thread + // to quit + let mut state = self.state.lock(); + state.requests = None; + } + + /// Stops all subsequent task executions, and waits for the worker + /// thread to complete. Note this will shutdown all clones of this + /// `DedicatedExecutor` as well. + /// + /// Only the first all to `join` will actually wait for the + /// executing thread to complete. All other calls to join will + /// complete immediately. + pub fn join(&self) { + self.shutdown(); + + // take the thread out when mutex is held + let thread = { + let mut state = self.state.lock(); + state.thread.take() + }; + + // wait for completion while not holding the mutex to avoid + // deadlocks + if let Some(thread) = thread { + thread.join().ok(); + } + } +} + +#[cfg(unix)] +fn set_current_thread_priority(prio: i32) { + // on linux setpriority sets the current thread's priority + // (as opposed to the current process). + unsafe { libc::setpriority(0, 0, prio) }; +} + +#[cfg(not(unix))] +fn set_current_thread_priority(prio: i32) { + warn!("Setting worker thread priority not supported on this platform"); +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::{Arc, Barrier}; + + #[cfg(unix)] + fn get_current_thread_priority() -> i32 { + // on linux setpriority sets the current thread's priority + // (as opposed to the current process). + unsafe { libc::getpriority(0, 0) } + } + + #[cfg(not(unix))] + fn get_current_thread_priority() -> i32 { + WORKER_PRIORITY + } + + #[tokio::test] + async fn basic() { + let barrier = Arc::new(Barrier::new(2)); + + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + let dedicated_task = exec.spawn(do_work(42, Arc::clone(&barrier))); + + // Note the dedicated task will never complete if it runs on + // the main tokio thread (as this test is not using the + // 'multithreaded' version of the executor and the call to + // barrier.wait actually blocks the tokio thread) + barrier.wait(); + + // should be able to get the result + assert_eq!(dedicated_task.await.unwrap(), 42); + } + + #[tokio::test] + async fn basic_clone() { + let barrier = Arc::new(Barrier::new(2)); + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + // Run task on clone should work fine + let dedicated_task = exec.clone().spawn(do_work(42, Arc::clone(&barrier))); + barrier.wait(); + assert_eq!(dedicated_task.await.unwrap(), 42); + } + + #[tokio::test] + async fn multi_task() { + let barrier = Arc::new(Barrier::new(3)); + + // make an executor with two threads + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 2); + let dedicated_task1 = exec.spawn(do_work(11, Arc::clone(&barrier))); + let dedicated_task2 = exec.spawn(do_work(42, Arc::clone(&barrier))); + + // block main thread until completion of other two tasks + barrier.wait(); + + // should be able to get the result + assert_eq!(dedicated_task1.await.unwrap(), 11); + assert_eq!(dedicated_task2.await.unwrap(), 42); + + exec.join(); + } + + #[tokio::test] + async fn worker_priority() { + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 2); + + let dedicated_task = exec.spawn(async move { get_current_thread_priority() }); + + assert_eq!(dedicated_task.await.unwrap(), WORKER_PRIORITY); + } + + #[tokio::test] + async fn tokio_spawn() { + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 2); + + // spawn a task that spawns to other tasks and ensure they run on the dedicated + // executor + let dedicated_task = exec.spawn(async move { + // spawn separate tasks + let t1 = tokio::task::spawn(async { + assert_eq!( + std::thread::current().name(), + Some("Test DedicatedExecutor") + ); + 25usize + }); + t1.await.unwrap() + }); + + // Validate the inner task ran to completion (aka it did not panic) + assert_eq!(dedicated_task.await.unwrap(), 25); + } + + #[tokio::test] + async fn panic_on_executor() { + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + let dedicated_task = exec.spawn(async move { + if true { + panic!("At the disco, on the dedicated task scheduler"); + } else { + 42 + } + }); + + // should not be able to get the result + dedicated_task.await.unwrap_err(); + } + + #[tokio::test] + async fn executor_shutdown_while_task_running() { + let barrier = Arc::new(Barrier::new(2)); + + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + let dedicated_task = exec.spawn(do_work(42, Arc::clone(&barrier))); + + exec.shutdown(); + // block main thread until completion of the outstanding task + barrier.wait(); + + // task should complete successfully + assert_eq!(dedicated_task.await.unwrap(), 42); + } + + #[tokio::test] + async fn executor_submit_task_after_shutdown() { + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + + // Simulate trying to submit tasks once executor has shutdown + exec.shutdown(); + let dedicated_task = exec.spawn(async { 11 }); + + // task should complete, but return an error + dedicated_task.await.unwrap_err(); + } + + #[tokio::test] + async fn executor_submit_task_after_clone_shutdown() { + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + + // shutdown the clone (but not the exec) + exec.clone().join(); + + // Simulate trying to submit tasks once executor has shutdown + let dedicated_task = exec.spawn(async { 11 }); + + // task should complete, but return an error + dedicated_task.await.unwrap_err(); + } + + #[tokio::test] + async fn executor_join() { + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + // test it doesn't hang + exec.join() + } + + #[tokio::test] + #[allow(clippy::redundant_clone)] + async fn executor_clone_join() { + let exec = DedicatedExecutor::new("Test DedicatedExecutor", 1); + // test it doesn't hang + exec.clone().join(); + exec.clone().join(); + exec.join(); + } + + /// Wait for the barrier and then return `result` + async fn do_work(result: usize, barrier: Arc) -> usize { + barrier.wait(); + result + } +} diff --git a/query/src/plan/stringset.rs b/query/src/plan/stringset.rs index 73cc850854..f0b359fad5 100644 --- a/query/src/plan/stringset.rs +++ b/query/src/plan/stringset.rs @@ -211,7 +211,7 @@ mod tests { let expected_ss = to_string_set(&["foo", "bar", "baz", "from_a_plan"]).into(); assert!(matches!(plan, StringSetPlan::Plan(_))); - let executor = Executor::new(); + let executor = Executor::new(1); let ss = executor.to_string_set(plan).await.unwrap(); assert_eq!(ss, expected_ss); } diff --git a/query/src/test.rs b/query/src/test.rs index a4e357613c..38c5eefadd 100644 --- a/query/src/test.rs +++ b/query/src/test.rs @@ -410,7 +410,7 @@ impl Default for TestDatabaseStore { fn default() -> Self { Self { databases: Mutex::new(BTreeMap::new()), - executor: Arc::new(Executor::new()), + executor: Arc::new(Executor::new(1)), } } } diff --git a/server/Cargo.toml b/server/Cargo.toml index 80da4b5069..218295b62a 100644 --- a/server/Cargo.toml +++ b/server/Cargo.toml @@ -19,6 +19,7 @@ generated_types = { path = "../generated_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } internal_types = { path = "../internal_types" } mutable_buffer = { path = "../mutable_buffer" } +num_cpus = "1.13.0" object_store = { path = "../object_store" } observability_deps = { path = "../observability_deps" } parking_lot = "0.11.1" diff --git a/server/benches/tag_values.rs b/server/benches/tag_values.rs index e39257876b..a57bd7bdee 100644 --- a/server/benches/tag_values.rs +++ b/server/benches/tag_values.rs @@ -56,7 +56,7 @@ pub fn benchmark_tag_values(c: &mut Criterion) { // predicate. fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DBScenario]) { let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); // single thread to execute queries let predicates = vec![ (PredicateBuilder::default().build(), "no_pred"), diff --git a/server/src/db.rs b/server/src/db.rs index a834335d92..bba15a92e1 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -1703,7 +1703,7 @@ mod tests { // run a sql query against the database, returning the results as record batches async fn run_query(db: Arc, query: &str) -> Vec { let planner = SQLQueryPlanner::default(); - let executor = Executor::new(); + let executor = Executor::new(1); let physical_plan = planner.query(db, query, &executor).await.unwrap(); diff --git a/server/src/lib.rs b/server/src/lib.rs index 8d6f49235e..60ca7ae25d 100644 --- a/server/src/lib.rs +++ b/server/src/lib.rs @@ -191,6 +191,38 @@ impl JobRegistry { const STORE_ERROR_PAUSE_SECONDS: u64 = 100; +/// Used to configure a server instance +#[derive(Debug)] +pub struct ServerConfig { + // number of executor worker threads. If not specified, defaults + // to number of cores on the system. + num_worker_threads: Option, + + /// The `ObjectStore` instance to use for persistence + object_store: Arc, +} + +impl ServerConfig { + /// Create a new config using the specified store + pub fn new(object_store: Arc) -> Self { + Self { + num_worker_threads: None, + object_store, + } + } + + /// Use `num` worker threads for running queries + pub fn with_num_worker_threads(mut self, num: usize) -> Self { + self.num_worker_threads = Some(num); + self + } + + /// return a reference to the object store in this configuration + pub fn store(&self) -> Arc { + Arc::clone(&self.object_store) + } +} + /// `Server` is the container struct for how servers store data internally, as /// well as how they communicate with other servers. Each server will have one /// of these structs, which keeps track of all replication and query rules. @@ -217,15 +249,21 @@ impl From for UpdateError { } impl Server { - pub fn new(connection_manager: M, store: Arc) -> Self { + pub fn new(connection_manager: M, config: ServerConfig) -> Self { let jobs = Arc::new(JobRegistry::new()); + let ServerConfig { + num_worker_threads, + object_store, + } = config; + let num_worker_threads = num_worker_threads.unwrap_or_else(num_cpus::get); + Self { id: Default::default(), config: Arc::new(Config::new(Arc::clone(&jobs))), - store, + store: object_store, connection_manager: Arc::new(connection_manager), - executor: Arc::new(Executor::new()), + executor: Arc::new(Executor::new(num_worker_threads)), jobs, } } @@ -703,11 +741,15 @@ mod tests { use super::*; + fn config() -> ServerConfig { + ServerConfig::new(Arc::new(ObjectStore::new_in_memory(InMemory::new()))) + .with_num_worker_threads(1) + } + #[tokio::test] async fn server_api_calls_return_error_with_no_id_set() { let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Server::new(manager, store); + let server = Server::new(manager, config()); let resp = server.require_id().unwrap_err(); assert!(matches!(resp, Error::IdNotSet)); @@ -720,8 +762,9 @@ mod tests { #[tokio::test] async fn create_database_persists_rules() { let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Server::new(manager, Arc::clone(&store)); + let config = config(); + let store = config.store(); + let server = Server::new(manager, config); server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); let name = DatabaseName::new("bananas").unwrap(); @@ -778,7 +821,8 @@ mod tests { store.list_with_delimiter(&store.new_path()).await.unwrap(); let manager = TestConnectionManager::new(); - let server2 = Server::new(manager, store); + let config2 = ServerConfig::new(store).with_num_worker_threads(1); + let server2 = Server::new(manager, config2); server2.set_id(NonZeroU32::new(1).unwrap()).unwrap(); server2.load_database_configs().await.unwrap(); @@ -791,8 +835,7 @@ mod tests { // Covers #643 let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Server::new(manager, store); + let server = Server::new(manager, config()); server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); let name = DatabaseName::new("bananas").unwrap(); @@ -825,8 +868,7 @@ mod tests { #[tokio::test] async fn db_names_sorted() { let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Server::new(manager, store); + let server = Server::new(manager, config()); server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); let names = vec!["bar", "baz"]; @@ -850,8 +892,7 @@ mod tests { #[tokio::test] async fn writes_local() { let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Server::new(manager, store); + let server = Server::new(manager, config()); server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); let name = DatabaseName::new("foo".to_string()).unwrap(); @@ -892,8 +933,7 @@ mod tests { #[tokio::test] async fn write_entry_local() { let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Server::new(manager, store); + let server = Server::new(manager, config()); server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); let name = DatabaseName::new("foo".to_string()).unwrap(); @@ -942,8 +982,7 @@ mod tests { async fn close_chunk() { test_helpers::maybe_start_logging(); let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Arc::new(Server::new(manager, store)); + let server = Arc::new(Server::new(manager, config())); let cancel_token = CancellationToken::new(); let background_handle = spawn_worker(Arc::clone(&server), cancel_token.clone()); @@ -1008,8 +1047,7 @@ mod tests { #[tokio::test] async fn background_task_cleans_jobs() { let manager = TestConnectionManager::new(); - let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - let server = Arc::new(Server::new(manager, store)); + let server = Arc::new(Server::new(manager, config())); let cancel_token = CancellationToken::new(); let background_handle = spawn_worker(Arc::clone(&server), cancel_token.clone()); diff --git a/server/src/query_tests/influxrpc/field_columns.rs b/server/src/query_tests/influxrpc/field_columns.rs index f0751375b6..b5042c5d80 100644 --- a/server/src/query_tests/influxrpc/field_columns.rs +++ b/server/src/query_tests/influxrpc/field_columns.rs @@ -31,7 +31,7 @@ macro_rules! run_field_columns_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plan = planner .field_columns(&db, predicate.clone()) @@ -132,7 +132,7 @@ async fn test_field_name_plan() { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plan = planner .field_columns(&db, predicate.clone()) diff --git a/server/src/query_tests/influxrpc/read_filter.rs b/server/src/query_tests/influxrpc/read_filter.rs index 9bc289c331..2e599b8a3c 100644 --- a/server/src/query_tests/influxrpc/read_filter.rs +++ b/server/src/query_tests/influxrpc/read_filter.rs @@ -47,7 +47,7 @@ macro_rules! run_read_filter_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plan = planner .read_filter(&db, predicate.clone()) diff --git a/server/src/query_tests/influxrpc/read_group.rs b/server/src/query_tests/influxrpc/read_group.rs index aac4f408ee..27cc2b34fc 100644 --- a/server/src/query_tests/influxrpc/read_group.rs +++ b/server/src/query_tests/influxrpc/read_group.rs @@ -26,7 +26,7 @@ macro_rules! run_read_group_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plans = planner .read_group(&db, predicate.clone(), agg, &group_columns) diff --git a/server/src/query_tests/influxrpc/read_window_aggregate.rs b/server/src/query_tests/influxrpc/read_window_aggregate.rs index 42e64d219d..21486eecaa 100644 --- a/server/src/query_tests/influxrpc/read_window_aggregate.rs +++ b/server/src/query_tests/influxrpc/read_window_aggregate.rs @@ -30,7 +30,7 @@ macro_rules! run_read_window_aggregate_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plans = planner .read_window_aggregate(&db, predicate.clone(), agg, every.clone(), offset.clone()) diff --git a/server/src/query_tests/influxrpc/table_names.rs b/server/src/query_tests/influxrpc/table_names.rs index da106479f8..823817cf6f 100644 --- a/server/src/query_tests/influxrpc/table_names.rs +++ b/server/src/query_tests/influxrpc/table_names.rs @@ -23,7 +23,7 @@ macro_rules! run_table_names_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plan = planner .table_names(&db, predicate.clone()) diff --git a/server/src/query_tests/influxrpc/tag_keys.rs b/server/src/query_tests/influxrpc/tag_keys.rs index 30e93f3117..e61bfed568 100644 --- a/server/src/query_tests/influxrpc/tag_keys.rs +++ b/server/src/query_tests/influxrpc/tag_keys.rs @@ -27,7 +27,7 @@ macro_rules! run_tag_keys_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plan = planner .tag_keys(&db, predicate.clone()) diff --git a/server/src/query_tests/influxrpc/tag_values.rs b/server/src/query_tests/influxrpc/tag_values.rs index 09a08ec163..406333eb61 100644 --- a/server/src/query_tests/influxrpc/tag_values.rs +++ b/server/src/query_tests/influxrpc/tag_values.rs @@ -25,7 +25,7 @@ macro_rules! run_tag_values_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(); + let executor = Executor::new(1); let plan = planner .tag_values(&db, &tag_name, predicate.clone()) diff --git a/server/src/query_tests/sql.rs b/server/src/query_tests/sql.rs index 4044cd9690..2ff9b81366 100644 --- a/server/src/query_tests/sql.rs +++ b/server/src/query_tests/sql.rs @@ -27,7 +27,7 @@ macro_rules! run_sql_test_case { println!("Running scenario '{}'", scenario_name); println!("SQL: '{:#?}'", sql); let planner = SQLQueryPlanner::default(); - let executor = Executor::new(); + let executor = Executor::new(1); let physical_plan = planner .query(db, &sql, &executor) diff --git a/src/commands/run.rs b/src/commands/run.rs index 42111e9da9..3bc7efb002 100644 --- a/src/commands/run.rs +++ b/src/commands/run.rs @@ -106,6 +106,16 @@ pub struct Config { #[structopt(long = "--data-dir", env = "INFLUXDB_IOX_DB_DIR")] pub database_directory: Option, + /// The number of threads to use for the query worker pool. + /// + /// IOx uses `--num-threads` threads for handling API requests and + /// will use a dedicated thread pool woth `--num-worker-threads` + /// for running queries. + /// + /// If not specified, defaults to the number of cores on the system + #[structopt(long = "--num-worker-threads", env = "INFLUXDB_IOX_NUM_WORKER_THREADS")] + pub num_worker_threads: Option, + #[structopt( long = "--object-store", env = "INFLUXDB_IOX_OBJECT_STORE", diff --git a/src/influxdb_ioxd.rs b/src/influxdb_ioxd.rs index 42ff6ee95a..cc5f18ca92 100644 --- a/src/influxdb_ioxd.rs +++ b/src/influxdb_ioxd.rs @@ -10,7 +10,10 @@ use object_store::{ }; use observability_deps::tracing::{self, error, info, warn, Instrument}; use panic_logging::SendPanicsToTracing; -use server::{ConnectionManagerImpl as ConnectionManager, Server as AppServer}; +use server::{ + ConnectionManagerImpl as ConnectionManager, Server as AppServer, + ServerConfig as AppServerConfig, +}; use snafu::{ResultExt, Snafu}; use std::{convert::TryFrom, fs, net::SocketAddr, path::PathBuf, sync::Arc}; @@ -124,9 +127,20 @@ pub async fn main(logging_level: LoggingLevel, config: Config) -> Result<()> { let object_store = ObjectStore::try_from(&config)?; let object_storage = Arc::new(object_store); + let server_config = AppServerConfig::new(object_storage); + + let server_config = if let Some(n) = config.num_worker_threads { + info!( + num_worker_threads = n, + "Using specified number of worker threads" + ); + server_config.with_num_worker_threads(n) + } else { + server_config + }; let connection_manager = ConnectionManager {}; - let app_server = Arc::new(AppServer::new(connection_manager, object_storage)); + let app_server = Arc::new(AppServer::new(connection_manager, server_config)); // if this ID isn't set the server won't be usable until this is set via an API // call diff --git a/src/influxdb_ioxd/http.rs b/src/influxdb_ioxd/http.rs index d41af8a518..520b128b94 100644 --- a/src/influxdb_ioxd/http.rs +++ b/src/influxdb_ioxd/http.rs @@ -739,17 +739,19 @@ mod tests { use data_types::{database_rules::DatabaseRules, DatabaseName}; use object_store::{memory::InMemory, ObjectStore}; use serde::de::DeserializeOwned; - use server::{db::Db, ConnectionManagerImpl}; + use server::{db::Db, ConnectionManagerImpl, ServerConfig as AppServerConfig}; use std::num::NonZeroU32; use test_helpers::assert_contains; + fn config() -> AppServerConfig { + AppServerConfig::new(Arc::new(ObjectStore::new_in_memory(InMemory::new()))) + .with_num_worker_threads(1) + } + #[tokio::test] async fn test_health() { - let test_storage = Arc::new(AppServer::new( - ConnectionManagerImpl {}, - Arc::new(ObjectStore::new_in_memory(InMemory::new())), - )); - let server_url = test_server(Arc::clone(&test_storage)); + let app_server = Arc::new(AppServer::new(ConnectionManagerImpl {}, config())); + let server_url = test_server(Arc::clone(&app_server)); let client = Client::new(); let response = client.get(&format!("{}/health", server_url)).send().await; @@ -760,20 +762,17 @@ mod tests { #[tokio::test] async fn test_write() { - let test_storage = Arc::new(AppServer::new( - ConnectionManagerImpl {}, - Arc::new(ObjectStore::new_in_memory(InMemory::new())), - )); - test_storage.set_id(NonZeroU32::new(1).unwrap()).unwrap(); - test_storage + let app_server = Arc::new(AppServer::new(ConnectionManagerImpl {}, config())); + app_server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); + app_server .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), - test_storage.require_id().unwrap(), - Arc::clone(&test_storage.store), + app_server.require_id().unwrap(), + Arc::clone(&app_server.store), ) .await .unwrap(); - let server_url = test_server(Arc::clone(&test_storage)); + let server_url = test_server(Arc::clone(&app_server)); let client = Client::new(); @@ -794,7 +793,7 @@ mod tests { check_response("write", response, StatusCode::NO_CONTENT, "").await; // Check that the data got into the right bucket - let test_db = test_storage + let test_db = app_server .db(&DatabaseName::new("MyOrg_MyBucket").unwrap()) .expect("Database exists"); @@ -812,20 +811,17 @@ mod tests { #[tokio::test] async fn test_write_metrics() { metrics::init_metrics_for_test(); - let test_storage = Arc::new(AppServer::new( - ConnectionManagerImpl {}, - Arc::new(ObjectStore::new_in_memory(InMemory::new())), - )); - test_storage.set_id(NonZeroU32::new(1).unwrap()).unwrap(); - test_storage + let app_server = Arc::new(AppServer::new(ConnectionManagerImpl {}, config())); + app_server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); + app_server .create_database( DatabaseRules::new(DatabaseName::new("MetricsOrg_MetricsBucket").unwrap()), - test_storage.require_id().unwrap(), - Arc::clone(&test_storage.store), + app_server.require_id().unwrap(), + Arc::clone(&app_server.store), ) .await .unwrap(); - let server_url = test_server(Arc::clone(&test_storage)); + let server_url = test_server(Arc::clone(&app_server)); let client = Client::new(); @@ -874,20 +870,17 @@ mod tests { /// returns a client for communicting with the server, and the server /// endpoint async fn setup_test_data() -> (Client, String) { - let test_storage: Arc> = Arc::new(AppServer::new( - ConnectionManagerImpl {}, - Arc::new(ObjectStore::new_in_memory(InMemory::new())), - )); - test_storage.set_id(NonZeroU32::new(1).unwrap()).unwrap(); - test_storage + let app_server = Arc::new(AppServer::new(ConnectionManagerImpl {}, config())); + app_server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); + app_server .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), - test_storage.require_id().unwrap(), - Arc::clone(&test_storage.store), + app_server.require_id().unwrap(), + Arc::clone(&app_server.store), ) .await .unwrap(); - let server_url = test_server(Arc::clone(&test_storage)); + let server_url = test_server(Arc::clone(&app_server)); let client = Client::new(); @@ -1011,20 +1004,17 @@ mod tests { #[tokio::test] async fn test_gzip_write() { - let test_storage = Arc::new(AppServer::new( - ConnectionManagerImpl {}, - Arc::new(ObjectStore::new_in_memory(InMemory::new())), - )); - test_storage.set_id(NonZeroU32::new(1).unwrap()).unwrap(); - test_storage + let app_server = Arc::new(AppServer::new(ConnectionManagerImpl {}, config())); + app_server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); + app_server .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), - test_storage.require_id().unwrap(), - Arc::clone(&test_storage.store), + app_server.require_id().unwrap(), + Arc::clone(&app_server.store), ) .await .unwrap(); - let server_url = test_server(Arc::clone(&test_storage)); + let server_url = test_server(Arc::clone(&app_server)); let client = Client::new(); let lp_data = "h2o_temperature,location=santa_monica,state=CA surface_degrees=65.2,bottom_degrees=50.4 1568756160"; @@ -1045,7 +1035,7 @@ mod tests { check_response("gzip_write", response, StatusCode::NO_CONTENT, "").await; // Check that the data got into the right bucket - let test_db = test_storage + let test_db = app_server .db(&DatabaseName::new("MyOrg_MyBucket").unwrap()) .expect("Database exists"); @@ -1063,20 +1053,17 @@ mod tests { #[tokio::test] async fn write_to_invalid_database() { - let test_storage = Arc::new(AppServer::new( - ConnectionManagerImpl {}, - Arc::new(ObjectStore::new_in_memory(InMemory::new())), - )); - test_storage.set_id(NonZeroU32::new(1).unwrap()).unwrap(); - test_storage + let app_server = Arc::new(AppServer::new(ConnectionManagerImpl {}, config())); + app_server.set_id(NonZeroU32::new(1).unwrap()).unwrap(); + app_server .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), - test_storage.require_id().unwrap(), - Arc::clone(&test_storage.store), + app_server.require_id().unwrap(), + Arc::clone(&app_server.store), ) .await .unwrap(); - let server_url = test_server(Arc::clone(&test_storage)); + let server_url = test_server(Arc::clone(&app_server)); let client = Client::new(); @@ -1179,7 +1166,7 @@ mod tests { /// Run the specified SQL query and return formatted results as a string async fn run_query(db: Arc, query: &str) -> Vec { let planner = SQLQueryPlanner::default(); - let executor = Executor::new(); + let executor = Executor::new(1); let physical_plan = planner.query(db, query, &executor).await.unwrap(); collect(physical_plan).await.unwrap() diff --git a/src/influxdb_ioxd/rpc/flight.rs b/src/influxdb_ioxd/rpc/flight.rs index 86f1056fab..09866f414e 100644 --- a/src/influxdb_ioxd/rpc/flight.rs +++ b/src/influxdb_ioxd/rpc/flight.rs @@ -1,3 +1,4 @@ +//! Implements the native gRPC IOx query API using Arrow Flight use std::{pin::Pin, sync::Arc}; use futures::Stream; @@ -19,7 +20,6 @@ use arrow_deps::{ Action, ActionType, Criteria, Empty, FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse, PutResult, SchemaResult, Ticket, }, - datafusion::physical_plan::collect, }; use data_types::{DatabaseName, DatabaseNameError}; use query::{frontend::sql::SQLQueryPlanner, DatabaseStore}; @@ -165,7 +165,9 @@ where })?; // execute the query - let results = collect(Arc::clone(&physical_plan)) + let results = executor + .new_context() + .collect(Arc::clone(&physical_plan)) .await .map_err(|e| Box::new(e) as _) .context(Query { From 8f1bf8a96033b7c3361636f74703643a971ea55b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 14 Apr 2021 08:09:40 -0400 Subject: [PATCH 42/45] fix: Remove mutex acquisition in impl `std::fmt::Debug` for DedicatedExecutor (#1205) --- query/src/exec/task.rs | 37 +++++-------------------------------- 1 file changed, 5 insertions(+), 32 deletions(-) diff --git a/query/src/exec/task.rs b/query/src/exec/task.rs index 501846e434..5b01d6ddd9 100644 --- a/query/src/exec/task.rs +++ b/query/src/exec/task.rs @@ -22,12 +22,6 @@ pub struct DedicatedExecutor { /// Runs futures (and any `tasks` that are `tokio::task::spawned` by /// them) on a separate tokio Executor struct State { - /// The number of threads in this pool - num_threads: usize, - - /// The name of the threads for this executor - thread_name: String, - /// Channel for requests -- the dedicated executor takes requests /// from here and runs them. requests: Option>, @@ -41,26 +35,8 @@ const WORKER_PRIORITY: i32 = 10; impl std::fmt::Debug for DedicatedExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let state = self.state.lock(); - - let mut d = f.debug_struct("DedicatedExecutor"); - - d.field("num_threads", &state.num_threads) - .field("thread_name", &state.thread_name); - - if state.requests.is_some() { - d.field("requests", &"Some(...)") - } else { - d.field("requests", &"None") - }; - - if state.thread.is_some() { - d.field("thread", &"Some(...)") - } else { - d.field("thread", &"None") - }; - - d.finish() + // Avoid taking the mutex in debug formatting + write!(f, "DedicatedExecutor") } } @@ -81,16 +57,15 @@ impl DedicatedExecutor { /// drop a runtime in a context where blocking is not allowed. This /// happens when a runtime is dropped from within an asynchronous /// context.', .../tokio-1.4.0/src/runtime/blocking/shutdown.rs:51:21 - pub fn new(thread_name: impl Into, num_threads: usize) -> Self { - let thread_name = thread_name.into(); - let name_copy = thread_name.to_string(); + pub fn new(thread_name: &str, num_threads: usize) -> Self { + let thread_name = thread_name.to_string(); let (tx, rx) = std::sync::mpsc::channel(); let thread = std::thread::spawn(move || { let runtime = tokio::runtime::Builder::new_multi_thread() .enable_all() - .thread_name(&name_copy) + .thread_name(&thread_name) .worker_threads(num_threads) .on_thread_start(move || set_current_thread_priority(WORKER_PRIORITY)) .build() @@ -107,8 +82,6 @@ impl DedicatedExecutor { }); let state = State { - num_threads, - thread_name, requests: Some(tx), thread: Some(thread), }; From 04f594bf94c860b9f7868ddd7f2fc5675f79fd01 Mon Sep 17 00:00:00 2001 From: Edd Robinson Date: Wed, 14 Apr 2021 12:06:35 +0100 Subject: [PATCH 43/45] perf: use column_values in read buffer --- server/src/db/chunk.rs | 55 +++++++++++++++++++++++++++++++++++------- 1 file changed, 46 insertions(+), 9 deletions(-) diff --git a/server/src/db/chunk.rs b/server/src/db/chunk.rs index 270961550d..6f1ea57efb 100644 --- a/server/src/db/chunk.rs +++ b/server/src/db/chunk.rs @@ -8,7 +8,10 @@ use query::{exec::stringset::StringSet, predicate::Predicate, PartitionChunk}; use read_buffer::Chunk as ReadBufferChunk; use snafu::{ResultExt, Snafu}; -use std::{collections::BTreeSet, sync::Arc}; +use std::{ + collections::{BTreeMap, BTreeSet}, + sync::Arc, +}; use super::{ pred::to_read_buffer_predicate, @@ -28,6 +31,9 @@ pub enum Error { chunk_id: u32, }, + #[snafu(display("Read Buffer Error in chunk {}: {}", chunk_id, msg))] + ReadBufferError { chunk_id: u32, msg: String }, + #[snafu(display("Parquet File Error in chunk {}: {}", chunk_id, source))] ParquetFileChunkError { source: parquet_file::chunk::Error, @@ -345,9 +351,9 @@ impl PartitionChunk for DBChunk { fn column_values( &self, - _table_name: &str, - _column_name: &str, - _predicate: &Predicate, + table_name: &str, + column_name: &str, + predicate: &Predicate, ) -> Result, Self::Error> { match self { Self::MutableBuffer { .. } => { @@ -355,11 +361,42 @@ impl PartitionChunk for DBChunk { // vs just letting DataFusion do its thing Ok(None) } - Self::ReadBuffer { .. } => { - // TODO hook up read buffer API here when ready. Until - // now, fallback to using a full plan - // https://github.com/influxdata/influxdb_iox/issues/857 - Ok(None) + Self::ReadBuffer { chunk, .. } => { + let rb_predicate = match to_read_buffer_predicate(predicate) { + Ok(rb_predicate) => rb_predicate, + Err(e) => { + debug!(?predicate, %e, "read buffer predicate not supported for column_names, falling back"); + return Ok(None); + } + }; + + let mut values = chunk + .column_values( + table_name, + rb_predicate, + Selection::Some(&[column_name]), + BTreeMap::new(), + ) + .context(ReadBufferChunkError { + chunk_id: chunk.id(), + })?; + + // The InfluxRPC frontend only supports getting column values + // for one column at a time (this is a restriction on the Influx + // Read gRPC API too). However, the Read Buffer support multiple + // columns and will return a map - we just need to pull the + // column out to get the set of values. + let values = values + .remove(column_name) + .ok_or_else(|| Error::ReadBufferError { + chunk_id: chunk.id(), + msg: format!( + "failed to find column_name {:?} in results of tag_values", + column_name + ), + })?; + + Ok(Some(values)) } Self::ParquetFile { .. } => { unimplemented!("parquet file not implemented for column_values") From 5bb34e9a972ff2ce526fa067f1eb32b09669cc66 Mon Sep 17 00:00:00 2001 From: Edd Robinson Date: Wed, 14 Apr 2021 16:41:30 +0100 Subject: [PATCH 44/45] refactor: use read_buffer column range for time range --- read_buffer/src/chunk.rs | 36 ++++++++++++++++---------- read_buffer/src/table.rs | 55 +++++++++++++++++++++++++++++++++++++--- server/src/db.rs | 2 +- 3 files changed, 75 insertions(+), 18 deletions(-) diff --git a/read_buffer/src/chunk.rs b/read_buffer/src/chunk.rs index 7ad1c7cd08..08c6cf41b1 100644 --- a/read_buffer/src/chunk.rs +++ b/read_buffer/src/chunk.rs @@ -316,19 +316,6 @@ impl Chunk { Ok(table.read_filter(&select_columns, &predicate)) } - /// Returns timestamp range for specified table - pub fn read_time_range(&self, table_name: &str) -> Result> { - // read lock on chunk. - let chunk_data = self.chunk_data.read().unwrap(); - - let table = chunk_data - .data - .get(table_name) - .context(TableNotFound { table_name })?; - - Ok(table.time_range()) - } - /// Returns an iterable collection of data in group columns and aggregate /// columns, optionally filtered by the provided predicate. Results are /// merged across all row groups within the returned table. @@ -389,6 +376,29 @@ impl Chunk { .collect() } + /// A helper method for determining the time-range associated with the + /// specified table. + /// + /// A table's schema need not contain a column representing the time, + /// however any table that represents data using the InfluxDB model does + /// contain a column that represents the timestamp associated with each + /// row. + /// + /// `table_time_range` will return the min and max values for that column + /// if the table is using the InfluxDB data-model, otherwise it will return + /// `None`. An error will be returned if the table does not exist. + pub fn table_time_range(&self, table_name: &str) -> Result> { + // read lock on chunk. + let chunk_data = self.chunk_data.read().unwrap(); + + let table = chunk_data + .data + .get(table_name) + .context(TableNotFound { table_name })?; + + Ok(table.time_range()) + } + /// Returns a schema object for a `read_filter` operation using the provided /// column selection. An error is returned if the specified columns do not /// exist. diff --git a/read_buffer/src/table.rs b/read_buffer/src/table.rs index aadd9ed272..7f158ada12 100644 --- a/read_buffer/src/table.rs +++ b/read_buffer/src/table.rs @@ -13,12 +13,15 @@ use snafu::{ensure, Snafu}; use crate::row_group::{self, ColumnName, Predicate, RowGroup}; use crate::schema::{AggregateType, ColumnType, LogicalDataType, ResultSchema}; -use crate::value::Value; +use crate::value::{OwnedValue, Scalar, Value}; #[derive(Debug, Snafu)] pub enum Error { #[snafu(display("cannot drop last row group in table; drop table"))] EmptyTableError {}, + #[snafu(display("table does not have InfluxDB timestamp column"))] + NoTimestampColumnError {}, + #[snafu(display("unsupported column operation on {}: {}", column_name, msg))] UnsupportedColumnOperation { msg: String, column_name: String }, } @@ -151,9 +154,38 @@ impl Table { self.table_data.read().unwrap().meta.to_summary(&self.name) } - /// The time range of all row groups within this table. + /// Returns the column range associated with an InfluxDB Timestamp column + /// or None if the table's schema does not have such a column. pub fn time_range(&self) -> Option<(i64, i64)> { - self.table_data.read().unwrap().meta.time_range + let table_data = self.table_data.read().unwrap(); + + let time_column = table_data + .meta + .columns + .values() + .filter(|cm| matches!(cm.typ, crate::schema::ColumnType::Timestamp(_))) + .collect::>(); + + if time_column.is_empty() { + return None; + } + + assert_eq!(time_column.len(), 1); // can only be one timestamp column. + let range = &time_column[0].range; + + let (min, max) = match (&range.0, &range.1) { + (OwnedValue::Scalar(Scalar::I64(min)), OwnedValue::Scalar(Scalar::I64(max))) => { + (min, max) + } + (min, max) => { + panic!( + "invalid range type for timestamp column: ({:?}, {:?})", + min, max + ); + } + }; + + Some((*min, *max)) } // Helper function used in tests. @@ -612,7 +644,6 @@ impl MetaData { } pub fn to_summary(&self, table_name: impl Into) -> TableSummary { - use crate::value::{OwnedValue, Scalar}; use data_types::partition_metadata::{ColumnSummary, StatValues, Statistics}; let columns = self .columns @@ -1435,4 +1466,20 @@ west,host-b,100 vec!["time".to_owned()], ); } + + #[test] + fn time_range() { + // Build a row group. + let mut columns = vec![]; + let tc = ColumnType::Time(Column::from(&[-29_i64, -100, 3, 2][..])); + columns.push((row_group::TIME_COLUMN_NAME.to_string(), tc)); + + let rc = ColumnType::Tag(Column::from(&["west", "south", "north", "west"][..])); + columns.push(("region".to_string(), rc)); + + let rg = RowGroup::new(4, columns); + let table = Table::new("cpu".to_owned(), rg); + + assert_eq!(table.time_range().unwrap(), (-100, 3)); + } } diff --git a/server/src/db.rs b/server/src/db.rs index bba15a92e1..7a27a4b3c8 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -514,7 +514,7 @@ impl Db { .context(ReadBufferChunkSchemaError { chunk_id })? .into(); let time_range = rb_chunk - .read_time_range(stats.name.as_str()) + .table_time_range(stats.name.as_str()) .context(ReadBufferChunkTimestampError { chunk_id })?; let stream: SendableRecordBatchStream = Box::pin( streams::ReadFilterResultsStream::new(read_results, Arc::clone(&arrow_schema)), From 59ca090aef7ca2eee953c5db65e07a64e0d876be Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 14 Apr 2021 12:46:02 -0400 Subject: [PATCH 45/45] feat: Use single db-wide executor for running queries (#1198) * refactor: plumb executor into all Db instances * refactor: Route all query executions through worker pool Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- arrow_deps/src/test_util.rs | 40 ++++++++++++ query/src/exec.rs | 26 +++++--- query/src/exec/context.rs | 31 +++++---- query/src/exec/task.rs | 3 + query/src/frontend/sql.rs | 4 +- server/benches/tag_values.rs | 4 +- server/src/config.rs | 25 ++++++-- server/src/db.rs | 32 ++++++---- server/src/lib.rs | 63 +++++-------------- .../query_tests/influxrpc/field_columns.rs | 11 ++-- .../src/query_tests/influxrpc/read_filter.rs | 4 +- .../src/query_tests/influxrpc/read_group.rs | 5 +- .../influxrpc/read_window_aggregate.rs | 5 +- .../src/query_tests/influxrpc/table_names.rs | 9 +-- server/src/query_tests/influxrpc/tag_keys.rs | 9 +-- .../src/query_tests/influxrpc/tag_values.rs | 9 +-- server/src/query_tests/influxrpc/util.rs | 2 +- server/src/query_tests/sql.rs | 16 +++-- server/src/query_tests/utils.rs | 6 +- server/src/snapshot.rs | 4 +- src/influxdb_ioxd/http.rs | 17 ++--- src/influxdb_ioxd/rpc/flight.rs | 12 ++-- src/influxdb_ioxd/rpc/management.rs | 7 +-- 23 files changed, 186 insertions(+), 158 deletions(-) diff --git a/arrow_deps/src/test_util.rs b/arrow_deps/src/test_util.rs index 87ffb5fd4a..e49f198f89 100644 --- a/arrow_deps/src/test_util.rs +++ b/arrow_deps/src/test_util.rs @@ -29,6 +29,46 @@ macro_rules! assert_table_eq { }; } +/// Compares formatted output of a record batch with an expected +/// vector of strings in a way that order does not matter. +/// This is a macro so errors appear on the correct line +/// +/// Designed so that failure output can be directly copy/pasted +/// into the test code as expected results. +/// +/// Expects to be called about like this: +/// +/// `assert_batch_sorted_eq!(expected_lines: &[&str], batches: &[RecordBatch])` +#[macro_export] +macro_rules! assert_batches_sorted_eq { + ($EXPECTED_LINES: expr, $CHUNKS: expr) => { + let mut expected_lines: Vec = $EXPECTED_LINES.iter().map(|&s| s.into()).collect(); + + // sort except for header + footer + let num_lines = expected_lines.len(); + if num_lines > 3 { + expected_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + let formatted = arrow_deps::arrow::util::pretty::pretty_format_batches($CHUNKS).unwrap(); + // fix for windows: \r\n --> + + let mut actual_lines: Vec<&str> = formatted.trim().lines().collect(); + + // sort except for header + footer + let num_lines = actual_lines.len(); + if num_lines > 3 { + actual_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + assert_eq!( + expected_lines, actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + // sort a record batch by all columns (to provide a stable output order for test // comparison) pub fn sort_record_batch(batch: RecordBatch) -> RecordBatch { diff --git a/query/src/exec.rs b/query/src/exec.rs index 4f58a247fd..3c755048e8 100644 --- a/query/src/exec.rs +++ b/query/src/exec.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use arrow_deps::{ arrow::record_batch::RecordBatch, - datafusion::{self, logical_plan::LogicalPlan}, + datafusion::{self, logical_plan::LogicalPlan, physical_plan::ExecutionPlan}, }; use counters::ExecutionCounters; @@ -35,7 +35,7 @@ use crate::plan::{ stringset::StringSetPlan, }; -use self::task::DedicatedExecutor; +use self::task::{DedicatedExecutor, Error as ExecutorError}; #[derive(Debug, Snafu)] pub enum Error { @@ -87,7 +87,7 @@ pub enum Error { }, #[snafu(display("Joining execution task: {}", source))] - JoinError { source: tokio::task::JoinError }, + JoinError { source: ExecutorError }, } pub type Result = std::result::Result; @@ -159,7 +159,7 @@ impl Executor { let (plan_tx, plan_rx) = mpsc::channel(1); rx_channels.push(plan_rx); - tokio::task::spawn(async move { + self.exec.spawn(async move { let SeriesSetPlan { table_name, plan, @@ -172,7 +172,6 @@ impl Executor { let physical_plan = ctx .prepare_plan(&plan) - .await .context(DataFusionPhysicalPlanning)?; let it = ctx @@ -224,10 +223,9 @@ impl Executor { .into_iter() .map(|plan| { let ctx = self.new_context(); - tokio::task::spawn(async move { + self.exec.spawn(async move { let physical_plan = ctx .prepare_plan(&plan) - .await .context(DataFusionPhysicalPlanning)?; // TODO: avoid this buffering @@ -259,6 +257,15 @@ impl Executor { self.run_logical_plans(vec![plan]).await } + /// Executes the logical plan using DataFusion on a separate + /// thread pool and produces RecordBatches + pub async fn collect(&self, physical_plan: Arc) -> Result> { + self.new_context() + .collect(physical_plan) + .await + .context(DataFusionExecution) + } + /// Create a new execution context, suitable for executing a new query pub fn new_context(&self) -> IOxExecutionContext { IOxExecutionContext::new(self.exec.clone(), Arc::clone(&self.counters)) @@ -271,11 +278,10 @@ impl Executor { .into_iter() .map(|plan| { let ctx = self.new_context(); - // TODO run these on some executor other than the main tokio pool - tokio::task::spawn(async move { + + self.exec.spawn(async move { let physical_plan = ctx .prepare_plan(&plan) - .await .context(DataFusionPhysicalPlanning)?; // TODO: avoid this buffering diff --git a/query/src/exec/context.rs b/query/src/exec/context.rs index 042b48347b..5f7b84262a 100644 --- a/query/src/exec/context.rs +++ b/query/src/exec/context.rs @@ -146,13 +146,13 @@ impl IOxExecutionContext { /// Prepare a SQL statement for execution. This assumes that any /// tables referenced in the SQL have been registered with this context - pub async fn prepare_sql(&mut self, sql: &str) -> Result> { + pub fn prepare_sql(&mut self, sql: &str) -> Result> { let logical_plan = self.inner.sql(sql)?.to_logical_plan(); - self.prepare_plan(&logical_plan).await + self.prepare_plan(&logical_plan) } /// Prepare (optimize + plan) a pre-created logical plan for execution - pub async fn prepare_plan(&self, plan: &LogicalPlan) -> Result> { + pub fn prepare_plan(&self, plan: &LogicalPlan) -> Result> { debug!( "Creating plan: Initial plan\n----\n{}\n{}\n----", plan.display_indent_schema(), @@ -188,14 +188,21 @@ impl IOxExecutionContext { &self, physical_plan: Arc, ) -> Result { - if physical_plan.output_partitioning().partition_count() <= 1 { - physical_plan.execute(0).await - } else { - // merge into a single partition - let plan = MergeExec::new(physical_plan); - // MergeExec must produce a single partition - assert_eq!(1, plan.output_partitioning().partition_count()); - plan.execute(0).await - } + self.exec + .spawn(async move { + if physical_plan.output_partitioning().partition_count() <= 1 { + physical_plan.execute(0).await + } else { + // merge into a single partition + let plan = MergeExec::new(physical_plan); + // MergeExec must produce a single partition + assert_eq!(1, plan.output_partitioning().partition_count()); + plan.execute(0).await + } + }) + .await + .map_err(|e| { + Error::Execution(format!("Error running IOxExecutionContext::execute: {}", e)) + })? } } diff --git a/query/src/exec/task.rs b/query/src/exec/task.rs index 5b01d6ddd9..c888642e4b 100644 --- a/query/src/exec/task.rs +++ b/query/src/exec/task.rs @@ -12,6 +12,9 @@ use observability_deps::tracing::warn; /// The type of thing that the dedicated executor runs type Task = Pin + Send>>; +/// The type of error that is returned from tasks in this module +pub type Error = tokio::sync::oneshot::error::RecvError; + /// Runs futures (and any `tasks` that are `tokio::task::spawned` by /// them) on a separate tokio Executor #[derive(Clone)] diff --git a/query/src/frontend/sql.rs b/query/src/frontend/sql.rs index be1cc65337..59ece13dc1 100644 --- a/query/src/frontend/sql.rs +++ b/query/src/frontend/sql.rs @@ -84,7 +84,7 @@ impl SQLQueryPlanner { /// Plan a SQL query against the data in `database`, and return a /// DataFusion physical execution plan. The plan can then be /// executed using `executor` in a streaming fashion. - pub async fn query( + pub fn query( &self, database: Arc, query: &str, @@ -92,6 +92,6 @@ impl SQLQueryPlanner { ) -> Result> { let mut ctx = executor.new_context(); ctx.inner_mut().register_catalog(DEFAULT_CATALOG, database); - ctx.prepare_sql(query).await.context(Preparing) + ctx.prepare_sql(query).context(Preparing) } } diff --git a/server/benches/tag_values.rs b/server/benches/tag_values.rs index a57bd7bdee..b1b0d7aca1 100644 --- a/server/benches/tag_values.rs +++ b/server/benches/tag_values.rs @@ -56,7 +56,6 @@ pub fn benchmark_tag_values(c: &mut Criterion) { // predicate. fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DBScenario]) { let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); // single thread to execute queries let predicates = vec![ (PredicateBuilder::default().build(), "no_pred"), @@ -85,10 +84,11 @@ fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DBScenario]) { BenchmarkId::from_parameter(format!("{}/{}", tag_key, pred_name)), tag_key, |b, &tag_key| { + let executor = db.executor(); b.to_async(Runtime::new().unwrap()).iter(|| { run_tag_values_query( &planner, - &executor, + executor.as_ref(), db, tag_key, predicate.clone(), diff --git a/server/src/config.rs b/server/src/config.rs index d662109920..c061cc0547 100644 --- a/server/src/config.rs +++ b/server/src/config.rs @@ -9,6 +9,7 @@ use data_types::{ DatabaseName, }; use object_store::{path::ObjectStorePath, ObjectStore}; +use query::exec::Executor; /// This module contains code for managing the configuration of the server. use crate::{db::Db, Error, JobRegistry, Result}; @@ -114,7 +115,13 @@ impl Config { state.remotes.remove(&id) } - fn commit(&self, rules: DatabaseRules, server_id: NonZeroU32, object_store: Arc) { + fn commit( + &self, + rules: DatabaseRules, + server_id: NonZeroU32, + object_store: Arc, + exec: Arc, + ) { let mut state = self.state.write().expect("mutex poisoned"); let name = state .reservations @@ -131,6 +138,7 @@ impl Config { rules, server_id, object_store, + exec, wal_buffer, Arc::clone(&self.jobs), )); @@ -253,9 +261,14 @@ pub(crate) struct CreateDatabaseHandle<'a> { } impl<'a> CreateDatabaseHandle<'a> { - pub(crate) fn commit(mut self, server_id: NonZeroU32, object_store: Arc) { + pub(crate) fn commit( + mut self, + server_id: NonZeroU32, + object_store: Arc, + exec: Arc, + ) { self.config - .commit(self.rules.take().unwrap(), server_id, object_store) + .commit(self.rules.take().unwrap(), server_id, object_store, exec) } pub(crate) fn rules(&self) -> &DatabaseRules { @@ -292,7 +305,8 @@ mod test { let db_reservation = config.create_db(rules).unwrap(); let server_id = NonZeroU32::new(1).unwrap(); let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - db_reservation.commit(server_id, store); + let exec = Arc::new(Executor::new(1)); + db_reservation.commit(server_id, store, exec); assert!(config.db(&name).is_some()); assert_eq!(config.db_names_sorted(), vec![name.clone()]); @@ -318,7 +332,8 @@ mod test { let db_reservation = config.create_db(rules).unwrap(); let server_id = NonZeroU32::new(1).unwrap(); let store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); - db_reservation.commit(server_id, store); + let exec = Arc::new(Executor::new(1)); + db_reservation.commit(server_id, store, exec); let token = config .state diff --git a/server/src/db.rs b/server/src/db.rs index 7a27a4b3c8..83c1bf40cb 100644 --- a/server/src/db.rs +++ b/server/src/db.rs @@ -33,7 +33,7 @@ use data_types::{ use internal_types::selection::Selection; use object_store::ObjectStore; use parquet_file::{chunk::Chunk, storage::Storage}; -use query::{Database, DEFAULT_SCHEMA}; +use query::{exec::Executor, Database, DEFAULT_SCHEMA}; use read_buffer::Chunk as ReadBufferChunk; use tracker::{MemRegistry, TaskTracker, TrackedFutureExt}; @@ -223,8 +223,12 @@ pub struct Db { pub server_id: NonZeroU32, // this is also the Query Server ID + /// Interface to use for peristence pub store: Arc, + /// Executor for running queries + exec: Arc, + /// The catalog holds chunks of data under partitions for the database. /// The underlying chunks may be backed by different execution engines /// depending on their stage in the data lifecycle. Currently there are @@ -271,6 +275,7 @@ impl Db { rules: DatabaseRules, server_id: NonZeroU32, object_store: Arc, + exec: Arc, wal_buffer: Option, jobs: Arc, ) -> Self { @@ -284,6 +289,7 @@ impl Db { rules, server_id, store, + exec, catalog, wal_buffer, jobs, @@ -294,6 +300,11 @@ impl Db { } } + /// Return a handle to the executor used to run queries + pub fn executor(&self) -> Arc { + Arc::clone(&self.exec) + } + /// Rolls over the active chunk in the database's specified /// partition. Returns the previously open (now closed) Chunk pub async fn rollover_partition(&self, partition_key: &str) -> Result> { @@ -850,9 +861,8 @@ mod tests { use crate::query_tests::utils::{make_database, make_db}; use ::test_helpers::assert_contains; use arrow_deps::{ - arrow::record_batch::RecordBatch, - assert_table_eq, - datafusion::{execution::context, physical_plan::collect}, + arrow::record_batch::RecordBatch, assert_batches_sorted_eq, assert_table_eq, + datafusion::execution::context, }; use chrono::Utc; use data_types::{ @@ -863,7 +873,7 @@ mod tests { use object_store::{ disk::File, path::ObjectStorePath, path::Path, ObjectStore, ObjectStoreApi, }; - use query::{exec::Executor, frontend::sql::SQLQueryPlanner, PartitionChunk}; + use query::{frontend::sql::SQLQueryPlanner, PartitionChunk}; use super::*; use futures::stream; @@ -926,7 +936,7 @@ mod tests { "+-----+------+", ]; let batches = run_query(Arc::clone(&db), "select * from cpu").await; - assert_table_eq!(expected, &batches); + assert_batches_sorted_eq!(expected, &batches); // add new data write_lp(db.as_ref(), "cpu bar=2 20"); @@ -939,14 +949,14 @@ mod tests { "+-----+------+", ]; let batches = run_query(Arc::clone(&db), "select * from cpu").await; - assert_table_eq!(&expected, &batches); + assert_batches_sorted_eq!(&expected, &batches); // And expect that we still get the same thing when data is rolled over again let chunk = db.rollover_partition("1970-01-01T00").await.unwrap(); assert_eq!(chunk.id(), 1); let batches = run_query(db, "select * from cpu").await; - assert_table_eq!(&expected, &batches); + assert_batches_sorted_eq!(&expected, &batches); } #[tokio::test] @@ -1703,11 +1713,11 @@ mod tests { // run a sql query against the database, returning the results as record batches async fn run_query(db: Arc, query: &str) -> Vec { let planner = SQLQueryPlanner::default(); - let executor = Executor::new(1); + let executor = db.executor(); - let physical_plan = planner.query(db, query, &executor).await.unwrap(); + let physical_plan = planner.query(db, query, &executor).unwrap(); - collect(physical_plan).await.unwrap() + executor.collect(physical_plan).await.unwrap() } fn mutable_chunk_ids(db: &Db, partition_key: &str) -> Vec { diff --git a/server/src/lib.rs b/server/src/lib.rs index 60ca7ae25d..14418e87cf 100644 --- a/server/src/lib.rs +++ b/server/src/lib.rs @@ -232,7 +232,7 @@ pub struct Server { config: Arc, connection_manager: Arc, pub store: Arc, - executor: Arc, + exec: Arc, jobs: Arc, } @@ -263,7 +263,7 @@ impl Server { config: Arc::new(Config::new(Arc::clone(&jobs))), store: object_store, connection_manager: Arc::new(connection_manager), - executor: Arc::new(Executor::new(num_worker_threads)), + exec: Arc::new(Executor::new(num_worker_threads)), jobs, } } @@ -282,12 +282,7 @@ impl Server { } /// Tells the server the set of rules for a database. - pub async fn create_database( - &self, - rules: DatabaseRules, - server_id: NonZeroU32, - object_store: Arc, - ) -> Result<()> { + pub async fn create_database(&self, rules: DatabaseRules, server_id: NonZeroU32) -> Result<()> { // Return an error if this server hasn't yet been setup with an id self.require_id()?; let db_reservation = self.config.create_db(rules)?; @@ -295,7 +290,7 @@ impl Server { self.persist_database_rules(db_reservation.rules().clone()) .await?; - db_reservation.commit(server_id, object_store); + db_reservation.commit(server_id, Arc::clone(&self.store), Arc::clone(&self.exec)); Ok(()) } @@ -350,6 +345,7 @@ impl Server { .map(|mut path| { let store = Arc::clone(&self.store); let config = Arc::clone(&self.config); + let exec = Arc::clone(&self.exec); path.set_file_name(DB_RULES_FILE_NAME); @@ -375,7 +371,7 @@ impl Server { } Ok(rules) => match config.create_db(rules) { Err(e) => error!("error adding database to config: {}", e), - Ok(handle) => handle.commit(server_id, store), + Ok(handle) => handle.commit(server_id, store, exec), }, } }) @@ -614,12 +610,8 @@ where let db = match self.db(&db_name) { Some(db) => db, None => { - self.create_database( - DatabaseRules::new(db_name.clone()), - self.require_id()?, - Arc::clone(&self.store), - ) - .await?; + self.create_database(DatabaseRules::new(db_name.clone()), self.require_id()?) + .await?; self.db(&db_name).expect("db not inserted") } }; @@ -627,8 +619,9 @@ where Ok(db) } + /// Return a handle to the query executor fn executor(&self) -> Arc { - Arc::clone(&self.executor) + Arc::clone(&self.exec) } } @@ -733,7 +726,7 @@ mod tests { use tokio::task::JoinHandle; use tokio_util::sync::CancellationToken; - use arrow_deps::{assert_table_eq, datafusion::physical_plan::collect}; + use arrow_deps::assert_table_eq; use data_types::database_rules::{PartitionTemplate, TemplatePart, NO_SHARD_CONFIG}; use influxdb_line_protocol::parse_lines; use object_store::{memory::InMemory, path::ObjectStorePath}; @@ -781,11 +774,7 @@ mod tests { // Create a database server - .create_database( - rules.clone(), - server.require_id().unwrap(), - Arc::clone(&server.store), - ) + .create_database(rules.clone(), server.require_id().unwrap()) .await .expect("failed to create database"); @@ -813,7 +802,6 @@ mod tests { .create_database( DatabaseRules::new(db2.clone()), server.require_id().unwrap(), - Arc::clone(&server.store), ) .await .expect("failed to create 2nd db"); @@ -845,7 +833,6 @@ mod tests { .create_database( DatabaseRules::new(name.clone()), server.require_id().unwrap(), - Arc::clone(&server.store), ) .await .expect("failed to create database"); @@ -855,7 +842,6 @@ mod tests { .create_database( DatabaseRules::new(name.clone()), server.require_id().unwrap(), - Arc::clone(&server.store), ) .await .unwrap_err(); @@ -876,11 +862,7 @@ mod tests { for name in &names { let name = DatabaseName::new(name.to_string()).unwrap(); server - .create_database( - DatabaseRules::new(name), - server.require_id().unwrap(), - Arc::clone(&server.store), - ) + .create_database(DatabaseRules::new(name), server.require_id().unwrap()) .await .expect("failed to create database"); } @@ -897,11 +879,7 @@ mod tests { let name = DatabaseName::new("foo".to_string()).unwrap(); server - .create_database( - DatabaseRules::new(name), - server.require_id().unwrap(), - Arc::clone(&server.store), - ) + .create_database(DatabaseRules::new(name), server.require_id().unwrap()) .await .unwrap(); @@ -916,10 +894,9 @@ mod tests { let executor = server.executor(); let physical_plan = planner .query(db, "select * from cpu", executor.as_ref()) - .await .unwrap(); - let batches = collect(physical_plan).await.unwrap(); + let batches = executor.collect(physical_plan).await.unwrap(); let expected = vec![ "+-----+------+", "| bar | time |", @@ -938,11 +915,7 @@ mod tests { let name = DatabaseName::new("foo".to_string()).unwrap(); server - .create_database( - DatabaseRules::new(name), - server.require_id().unwrap(), - Arc::clone(&server.store), - ) + .create_database(DatabaseRules::new(name), server.require_id().unwrap()) .await .unwrap(); @@ -964,10 +937,9 @@ mod tests { let executor = server.executor(); let physical_plan = planner .query(db, "select * from cpu", executor.as_ref()) - .await .unwrap(); - let batches = collect(physical_plan).await.unwrap(); + let batches = executor.collect(physical_plan).await.unwrap(); let expected = vec![ "+-----+------+", "| bar | time |", @@ -994,7 +966,6 @@ mod tests { .create_database( DatabaseRules::new(db_name.clone()), server.require_id().unwrap(), - Arc::clone(&server.store), ) .await .unwrap(); diff --git a/server/src/query_tests/influxrpc/field_columns.rs b/server/src/query_tests/influxrpc/field_columns.rs index b5042c5d80..4f9502f51b 100644 --- a/server/src/query_tests/influxrpc/field_columns.rs +++ b/server/src/query_tests/influxrpc/field_columns.rs @@ -4,10 +4,7 @@ use arrow_deps::{ datafusion::logical_plan::{col, lit}, }; use query::{ - exec::{ - fieldlist::{Field, FieldList}, - Executor, - }, + exec::fieldlist::{Field, FieldList}, frontend::influxrpc::InfluxRPCPlanner, predicate::PredicateBuilder, }; @@ -31,7 +28,7 @@ macro_rules! run_field_columns_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); + let executor = db.executor(); let plan = planner .field_columns(&db, predicate.clone()) @@ -132,7 +129,6 @@ async fn test_field_name_plan() { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); let plan = planner .field_columns(&db, predicate.clone()) @@ -144,7 +140,8 @@ async fn test_field_name_plan() { // run the created plan directly, ensuring the output is as // expected (specifically that the column ordering is correct) - let results = executor + let results = db + .executor() .run_logical_plan(plan) .await .expect("ok running plan"); diff --git a/server/src/query_tests/influxrpc/read_filter.rs b/server/src/query_tests/influxrpc/read_filter.rs index 2e599b8a3c..4d7cb97e20 100644 --- a/server/src/query_tests/influxrpc/read_filter.rs +++ b/server/src/query_tests/influxrpc/read_filter.rs @@ -4,7 +4,6 @@ use crate::query_tests::scenarios::*; use arrow_deps::datafusion::logical_plan::{col, lit}; use async_trait::async_trait; use query::{ - exec::Executor, frontend::influxrpc::InfluxRPCPlanner, predicate::{Predicate, PredicateBuilder, EMPTY_PREDICATE}, }; @@ -47,13 +46,12 @@ macro_rules! run_read_filter_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); let plan = planner .read_filter(&db, predicate.clone()) .expect("built plan successfully"); - let string_results = run_series_set_plan(executor, plan).await; + let string_results = run_series_set_plan(db.executor(), plan).await; assert_eq!( expected_results, string_results, diff --git a/server/src/query_tests/influxrpc/read_group.rs b/server/src/query_tests/influxrpc/read_group.rs index 27cc2b34fc..3efe49df51 100644 --- a/server/src/query_tests/influxrpc/read_group.rs +++ b/server/src/query_tests/influxrpc/read_group.rs @@ -4,7 +4,6 @@ use crate::query_tests::scenarios::*; use arrow_deps::{arrow::util::pretty::pretty_format_batches, datafusion::prelude::*}; use async_trait::async_trait; use query::{ - exec::Executor, frontend::influxrpc::InfluxRPCPlanner, group_by::Aggregate, predicate::{Predicate, PredicateBuilder}, @@ -26,7 +25,6 @@ macro_rules! run_read_group_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); let plans = planner .read_group(&db, predicate.clone(), agg, &group_columns) @@ -45,7 +43,8 @@ macro_rules! run_read_group_test_case { let mut string_results = vec![]; for plan in plans.into_iter() { - let batches = executor + let batches = db + .executor() .run_logical_plan(plan.plan) .await .expect("ok running plan"); diff --git a/server/src/query_tests/influxrpc/read_window_aggregate.rs b/server/src/query_tests/influxrpc/read_window_aggregate.rs index 21486eecaa..1e9786a163 100644 --- a/server/src/query_tests/influxrpc/read_window_aggregate.rs +++ b/server/src/query_tests/influxrpc/read_window_aggregate.rs @@ -7,7 +7,6 @@ use crate::{ use arrow_deps::{arrow::util::pretty::pretty_format_batches, datafusion::prelude::*}; use async_trait::async_trait; use query::{ - exec::Executor, frontend::influxrpc::InfluxRPCPlanner, group_by::{Aggregate, WindowDuration}, predicate::{Predicate, PredicateBuilder}, @@ -30,7 +29,6 @@ macro_rules! run_read_window_aggregate_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); let plans = planner .read_window_aggregate(&db, predicate.clone(), agg, every.clone(), offset.clone()) @@ -40,7 +38,8 @@ macro_rules! run_read_window_aggregate_test_case { let mut string_results = vec![]; for plan in plans.into_iter() { - let batches = executor + let batches = db + .executor() .run_logical_plan(plan.plan) .await .expect("ok running plan"); diff --git a/server/src/query_tests/influxrpc/table_names.rs b/server/src/query_tests/influxrpc/table_names.rs index 823817cf6f..efaa10481e 100644 --- a/server/src/query_tests/influxrpc/table_names.rs +++ b/server/src/query_tests/influxrpc/table_names.rs @@ -1,9 +1,6 @@ //! Tests for the Influx gRPC queries use query::{ - exec::{ - stringset::{IntoStringSet, StringSetRef}, - Executor, - }, + exec::stringset::{IntoStringSet, StringSetRef}, frontend::influxrpc::InfluxRPCPlanner, predicate::{Predicate, PredicateBuilder, EMPTY_PREDICATE}, }; @@ -23,12 +20,12 @@ macro_rules! run_table_names_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); let plan = planner .table_names(&db, predicate.clone()) .expect("built plan successfully"); - let names = executor + let names = db + .executor() .to_string_set(plan) .await .expect("converted plan to strings successfully"); diff --git a/server/src/query_tests/influxrpc/tag_keys.rs b/server/src/query_tests/influxrpc/tag_keys.rs index e61bfed568..e37ea11ac8 100644 --- a/server/src/query_tests/influxrpc/tag_keys.rs +++ b/server/src/query_tests/influxrpc/tag_keys.rs @@ -1,9 +1,6 @@ use arrow_deps::datafusion::logical_plan::{col, lit}; use query::{ - exec::{ - stringset::{IntoStringSet, StringSetRef}, - Executor, - }, + exec::stringset::{IntoStringSet, StringSetRef}, frontend::influxrpc::InfluxRPCPlanner, predicate::PredicateBuilder, }; @@ -27,12 +24,12 @@ macro_rules! run_tag_keys_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); let plan = planner .tag_keys(&db, predicate.clone()) .expect("built plan successfully"); - let names = executor + let names = db + .executor() .to_string_set(plan) .await .expect("converted plan to strings successfully"); diff --git a/server/src/query_tests/influxrpc/tag_values.rs b/server/src/query_tests/influxrpc/tag_values.rs index 406333eb61..a6205ff25e 100644 --- a/server/src/query_tests/influxrpc/tag_values.rs +++ b/server/src/query_tests/influxrpc/tag_values.rs @@ -1,9 +1,6 @@ use arrow_deps::datafusion::logical_plan::{col, lit}; use query::{ - exec::{ - stringset::{IntoStringSet, StringSetRef}, - Executor, - }, + exec::stringset::{IntoStringSet, StringSetRef}, frontend::influxrpc::InfluxRPCPlanner, predicate::PredicateBuilder, }; @@ -25,12 +22,12 @@ macro_rules! run_tag_values_test_case { println!("Running scenario '{}'", scenario_name); println!("Predicate: '{:#?}'", predicate); let planner = InfluxRPCPlanner::new(); - let executor = Executor::new(1); let plan = planner .tag_values(&db, &tag_name, predicate.clone()) .expect("built plan successfully"); - let names = executor + let names = db + .executor() .to_string_set(plan) .await .expect("converted plan to strings successfully"); diff --git a/server/src/query_tests/influxrpc/util.rs b/server/src/query_tests/influxrpc/util.rs index ac27fffd3e..f83b6c8d63 100644 --- a/server/src/query_tests/influxrpc/util.rs +++ b/server/src/query_tests/influxrpc/util.rs @@ -51,7 +51,7 @@ pub fn dump_series_set(s: SeriesSet) -> Vec { } /// Run a series set plan to completion and produce a Vec representation -pub async fn run_series_set_plan(executor: Executor, plans: SeriesSetPlans) -> Vec { +pub async fn run_series_set_plan(executor: Arc, plans: SeriesSetPlans) -> Vec { // Use a channel sufficiently large to buffer the series let (tx, mut rx) = mpsc::channel(100); executor diff --git a/server/src/query_tests/sql.rs b/server/src/query_tests/sql.rs index 2ff9b81366..e2016f2592 100644 --- a/server/src/query_tests/sql.rs +++ b/server/src/query_tests/sql.rs @@ -6,10 +6,8 @@ #![allow(unused_imports, dead_code, unused_macros)] use super::scenarios::*; -use arrow_deps::{ - arrow::record_batch::RecordBatch, assert_table_eq, datafusion::physical_plan::collect, -}; -use query::{exec::Executor, frontend::sql::SQLQueryPlanner}; +use arrow_deps::{arrow::record_batch::RecordBatch, assert_batches_sorted_eq}; +use query::frontend::sql::SQLQueryPlanner; use std::sync::Arc; /// runs table_names(predicate) and compares it to the expected @@ -27,16 +25,16 @@ macro_rules! run_sql_test_case { println!("Running scenario '{}'", scenario_name); println!("SQL: '{:#?}'", sql); let planner = SQLQueryPlanner::default(); - let executor = Executor::new(1); + let executor = db.executor(); let physical_plan = planner - .query(db, &sql, &executor) - .await + .query(db, &sql, executor.as_ref()) .expect("built plan successfully"); - let results: Vec = collect(physical_plan).await.expect("Running plan"); + let results: Vec = + executor.collect(physical_plan).await.expect("Running plan"); - assert_table_eq!($EXPECTED_LINES, &results); + assert_batches_sorted_eq!($EXPECTED_LINES, &results); } }; } diff --git a/server/src/query_tests/utils.rs b/server/src/query_tests/utils.rs index 46bda879d2..67e7643f25 100644 --- a/server/src/query_tests/utils.rs +++ b/server/src/query_tests/utils.rs @@ -4,7 +4,7 @@ use data_types::{ DatabaseName, }; use object_store::{memory::InMemory, ObjectStore}; -use query::Database; +use query::{exec::Executor, Database}; use crate::{db::Db, JobRegistry}; use std::{num::NonZeroU32, sync::Arc}; @@ -13,21 +13,25 @@ use std::{num::NonZeroU32, sync::Arc}; pub fn make_db() -> Db { let server_id: NonZeroU32 = NonZeroU32::new(1).unwrap(); let object_store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); + let exec = Arc::new(Executor::new(1)); Db::new( DatabaseRules::new(DatabaseName::new("placeholder").unwrap()), server_id, object_store, + exec, None, // wal buffer Arc::new(JobRegistry::new()), ) } pub fn make_database(server_id: NonZeroU32, object_store: Arc, db_name: &str) -> Db { + let exec = Arc::new(Executor::new(1)); Db::new( DatabaseRules::new(DatabaseName::new(db_name.to_string()).unwrap()), server_id, object_store, + exec, None, // wal buffer Arc::new(JobRegistry::new()), ) diff --git a/server/src/snapshot.rs b/server/src/snapshot.rs index cc6136b1a6..3ab62565aa 100644 --- a/server/src/snapshot.rs +++ b/server/src/snapshot.rs @@ -279,7 +279,7 @@ mod tests { use futures::TryStreamExt; use mutable_buffer::chunk::Chunk as ChunkWB; use object_store::memory::InMemory; - use query::Database; + use query::{exec::Executor, Database}; use tracker::MemRegistry; #[tokio::test] @@ -391,11 +391,13 @@ mem,host=A,region=west used=45 1 pub fn make_db() -> Db { let object_store = Arc::new(ObjectStore::new_in_memory(InMemory::new())); let server_id = std::num::NonZeroU32::new(1).unwrap(); + let exec = Arc::new(Executor::new(1)); Db::new( DatabaseRules::new(DatabaseName::new("placeholder").unwrap()), server_id, object_store, + exec, None, // wal buffer Arc::new(JobRegistry::new()), ) diff --git a/src/influxdb_ioxd/http.rs b/src/influxdb_ioxd/http.rs index 520b128b94..d7f001e90f 100644 --- a/src/influxdb_ioxd/http.rs +++ b/src/influxdb_ioxd/http.rs @@ -12,7 +12,6 @@ // Influx crates use super::super::commands::metrics; -use arrow_deps::datafusion::physical_plan::collect; use data_types::{ http::WalMetadataQuery, names::{org_and_bucket_to_database, OrgBucketMappingError}, @@ -523,12 +522,12 @@ async fn query( let physical_plan = planner .query(db, &q, executor.as_ref()) - .await .context(PlanningSQLQuery { query: &q })?; // TODO: stream read results out rather than rendering the // whole thing in mem - let batches = collect(physical_plan) + let batches = executor + .collect(physical_plan) .await .map_err(|e| Box::new(e) as _) .context(Query { db_name })?; @@ -733,7 +732,6 @@ mod tests { use std::net::{IpAddr, Ipv4Addr, SocketAddr}; use arrow_deps::{arrow::record_batch::RecordBatch, assert_table_eq}; - use query::exec::Executor; use reqwest::{Client, Response}; use data_types::{database_rules::DatabaseRules, DatabaseName}; @@ -768,7 +766,6 @@ mod tests { .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), app_server.require_id().unwrap(), - Arc::clone(&app_server.store), ) .await .unwrap(); @@ -817,7 +814,6 @@ mod tests { .create_database( DatabaseRules::new(DatabaseName::new("MetricsOrg_MetricsBucket").unwrap()), app_server.require_id().unwrap(), - Arc::clone(&app_server.store), ) .await .unwrap(); @@ -876,7 +872,6 @@ mod tests { .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), app_server.require_id().unwrap(), - Arc::clone(&app_server.store), ) .await .unwrap(); @@ -1010,7 +1005,6 @@ mod tests { .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), app_server.require_id().unwrap(), - Arc::clone(&app_server.store), ) .await .unwrap(); @@ -1059,7 +1053,6 @@ mod tests { .create_database( DatabaseRules::new(DatabaseName::new("MyOrg_MyBucket").unwrap()), app_server.require_id().unwrap(), - Arc::clone(&app_server.store), ) .await .unwrap(); @@ -1166,9 +1159,9 @@ mod tests { /// Run the specified SQL query and return formatted results as a string async fn run_query(db: Arc, query: &str) -> Vec { let planner = SQLQueryPlanner::default(); - let executor = Executor::new(1); - let physical_plan = planner.query(db, query, &executor).await.unwrap(); + let executor = db.executor(); + let physical_plan = planner.query(db, query, executor.as_ref()).unwrap(); - collect(physical_plan).await.unwrap() + executor.collect(physical_plan).await.unwrap() } } diff --git a/src/influxdb_ioxd/rpc/flight.rs b/src/influxdb_ioxd/rpc/flight.rs index 09866f414e..68bf8b1d89 100644 --- a/src/influxdb_ioxd/rpc/flight.rs +++ b/src/influxdb_ioxd/rpc/flight.rs @@ -157,12 +157,12 @@ where let planner = SQLQueryPlanner::default(); let executor = self.server.executor(); - let physical_plan = planner - .query(db, &read_info.sql_query, &executor) - .await - .context(PlanningSQLQuery { - query: &read_info.sql_query, - })?; + let physical_plan = + planner + .query(db, &read_info.sql_query, &executor) + .context(PlanningSQLQuery { + query: &read_info.sql_query, + })?; // execute the query let results = executor diff --git a/src/influxdb_ioxd/rpc/management.rs b/src/influxdb_ioxd/rpc/management.rs index ef51b8de99..5999eb7b44 100644 --- a/src/influxdb_ioxd/rpc/management.rs +++ b/src/influxdb_ioxd/rpc/management.rs @@ -126,13 +126,8 @@ where Some(id) => id, None => return Err(NotFound::default().into()), }; - let object_store = Arc::clone(&self.server.store); - match self - .server - .create_database(rules, server_id, object_store) - .await - { + match self.server.create_database(rules, server_id).await { Ok(_) => Ok(Response::new(CreateDatabaseResponse {})), Err(Error::DatabaseAlreadyExists { db_name }) => { return Err(AlreadyExists {

    EeUtif?8%-!{|$feZp$W-#3xfSLs%m=Z5-_rY-v?x4fScmf2mE6Ra~Ww z%-q|uZS;2ePY0E!L$7{ob;*Hhc+6VuCRPbd3||iso;z5h`sstZSRN&F%$m*!^&53a9NcU!(Ut>wTM| z-M&gq$~%vLLK|Eo>vH|hF1Wz$MCi3{^I0oBZIt#s^prxb#6eG<6u@syg(bHAwg*l9@ zPAjLYqNkBwLr55EO;yKIAl7I9s+s={3wk=oka|-?i+%j@IC-(#XTN9lgN%9#dkVmd z&0X}Zu!4Cb$0++&ctQ!oWu#hIr6Y)lrspl{tJpYaX%$N!5w414GXt@IbarA-!!$c@qMkmNAO>2wTphx5iKeHD|*HuSd!-ZIAyZh6wS*4 zEc1t8?TW3O)8mRB0kO-0wC@u8SLfkx^bV#WmeO=mDXOu7B^t&Fv3c~c>-U!bzkRWn zi;H00w>8WvYa0TQWWS$HB>B=U9jCoI=;wlGmfy(F3eVol{7LW(JX`zhl(!%MJvCHw zVcLd__`XZ~q4yt!Vy5kp9#sS*I^Z6e_qBLgn_pkImtWpqyz!)w>nj3OmgQ>jf6M(T z^ijYR1VmybpHT`|68@^OAdDB;3YhytqKM3lE`$?2o_>$?Hh+vA-a(T>xQ*cjY*BtC1@S5Eq4~jHP1WRoe+rg>8|bbh$57gzq2OjM}xcMHyQQUXG%dx4Nk=H9VXC z9z?O0Zp4WP`3A730IJyj>RH7$Ep8RtA!?buaU?)lO;zPwiO71U3`oMz;=il~cchVS zGhZ}QbCF0A1il7CDVFqsR#x{G9^K+W_!$6Rn~hG9S)Xq`URI-_6r%3Bovq}0S5TjC zm8h-%Ukkv_phI-hvF)uNo>c!^dYiNUN$a0qP=B$GsHs`Gi=yR`t7b-lG`Ly^n-}dT z&n^xe?of8=H*(;xnW8Lg`7NKG)-hs?M`jlwH~8#H2hf~$t+duQ>~4sMJ2Jld+*@Hb z8Z1@w(WULCPOyqW!j+bb$XZ3xda)P#bYIn-%ouCp=K|Ge2+<8z3oGD-tQIEEXb zvfuXIyJwZK3&AA%yNWH$fd(iw5Lx`lTg1)|Y4s;>4;EiMOs?*Kd={Huva3)`=34(` zV!MKqge+1p!@LwY(JYZrDwSXk_DF_m<;k3Xtj;G9>Q~;8ucL0tKE4`y0a@%|ZbFAs z3JmzgE6(j<2Lr&jG3bt=2E42^0zZ3N!$RNs!>MkIH%H1HP-T^*m~6Aj0h>@+C8#cQ zR0|LQ&w6|jiccxMLu8Xp)+mUN(cJslBbPI24CagRPc1-0QU*}YSy#J3B}Dj$j?Is= zS{NB{yVnd7VD5}1J)plQ^>d%CC$pW(lG!g3^9qK^_^8XNjRqA83mZ@g9RmI#-%(o* zC~Z10mzwB!5O4}T*G>mXxyaKkb=FZTaOA*C7T0AN2s!>yOt(%*L~1()g+<#p!zaiE z;SxOUUZaNLrc)kxOSr(r{B%lsVSsKG54(7>T}F2`uNiI(Ixn#}|wy=D2`rz7`gNM${DZMb{n^RbR2Y9tR2G`zxq`w+G+SlVs zzAB28`x55$NVt80e=Df8&x#_Ya9e!$e&W$SlRR3|vWo?^7FxJ#EWK(xw=%1AXsCR0 zzw*Z4!>$nyk;{)SZ+sdr(J{O8b!{nMqa*fl3;n?af76t{qo`yqugg<&50le><`JVx zCyiOyxHXv5daQ8zZ$L5I9-)lP#LHJi$Lz^4icChi-Rh>jf0V#w>F|L&hjVQIzf;Sf z8N71APyJ$abo}@+fS(V)n8I z=9k)X8(G96|}j8nN9Z3`v^t zat`-o6CcNmuk#aCGl8$LMLX-?cx&ZDf@j2z6-h6zYH}yl5R6rNb85K8zM<7wUYy!? z?#cG$N;ZmF=I-!+r?zqj0DmrHS6=BXz?Odrn)> zx-C1c?6#^3*^RBN6g}fJL=BbIH5<57d*qonHf*vt=y1~{_;7^FE>#zO=t(pEu|r$B z+L+2jhM5D0R$)Gp_f|3o*jt}p6p=oWLta=E+0K@cOpQa6Xoj#NlErk~23t5UsT1{t z1`HDwS_(SI7(=~Rj-Sl9JHr$X&j^OexVnP8p7>|*8cV`2kyYW4*qTO0mVr4aPwofh zf~2{!5lFH;O9vrV?S|ngk`q|8v2%q8_>EUs>2GN*poUXNPN%8Nv^{`TBSs^L&ta6{ znA8?TV9o;_oIB9ENhGk!}Pik>%SQd=Y)_m>gN99 z>W_n8Q^OTLd)+E@Sj<$mA@)dLC2HstmP&c>YS+$u4^~-n)LdLT-yl(zD^B43^R4|@ zVK~8KxP83hkV32@k#=TGY2!1CE_CuLHX2vYH4*TNtH`4&)1}V}QJIzaBo}!YYe}tU zgVu1`zK|SVAT=WZE)ouc5~*=?Too&ylZSxR6r(Jk^XL?WWUA~)?Q{kr`h1*XIi5J2 zODufHJ$RG>zj?v^XNW}V0Zom>r+yqPa~?x}gKrkKyVm;f=&`_i{O4vIf+44#miX0F z0w+^dv8zr3q5HaKuYZVMg{Q&4zr>eNH_RWN0TYs43;mXj|IUT}y0iV^{G{~t z8H2*Rr>Cx86bNs}|7rr^jnmWCAKxw975~wCj*y;?Z6$3JGqexu+c0ouwaoqqlZ& zm#i1UEhdy?>9HYFz#*s}5wIJ=WDZ_AY!*^2Y*^f za0-Yh5z*?6-x1+TO3D@}*pjsf!kW&lWUa~2rpqdeTBU=3C0>4(%@&$vRdORd zu|08Jlq(TJMH9#J4$O8ThlS9~t=n zY9dGCOYrgre#x=t!yX7&fL#DH=)O3+A8W#Z(GP5b_7b{D5J8?vU7WI6Sr|BOj*Mj3 zXaud6b!t^X2Zzmwkag*>NudaN2rhgmd-6nQFy720Qp+~LvY~~(EG}bq?c@c7_?T^2 zhLHF6^uPc^Rwz>0tl5m=aOg7Hz{b)+Mh|4IWpNRme^Bp9^vLnFgJu%+Q5A*Qq|O5u z9jEuOnft=@l<1hDv8R|6Usdop(YH~*x1h0o*4xo>NpA=i)e{{(7PK&0U7w+PP$Lix zUWMZ1`{(f1qpDhMa?DfAJ@)9i0Jr z;t(FkhI1=$A$HI<(K*G7If#a5|$*khdFk$#OK6GjX_L_=ikaPTdA?0 z+NFdiYTPnF>|KUEU++-1)L73d!5b@{Sll-98)2uS=;uMdkDnR~Hvjl_ko-nodnpjQ zupl%hC!Bv&`p*3HrTJ;=pAUpiF9@CXepcz#A>Tzs<=u`MTsvp*&^bA!zY6&VRQZlC zDqr1uaP8fH@E^I*-+c18lJ=s=Q>bi9NqD*1UyS!Zjw@*m?A$dWG3o7$Dfh?7a&1svd3u6D)j`#UmreFon? z59s#N!Q775zI+?)Cf8;6dOMrHRn{5d{@hmVW;{{^DxOODzF|gF8{EobV4Kn^t~z*4 z$7=;!W#38#MXW#Irr;>QC=E*KBIVR1I~IfCxModbHc_*t-wU!T3L|dft7%mvz^{@( z8YFb{@sUPs0{_AT?>d*WQ zeE7g>{S%!G>hts@<%9UH^M;^%s5q%qW%m}q%+si&8k&Sq{Kb3>;y7T2GFPJYCIO)h#9paEY z&OWDTB$r}~dVpJmx1V5v(C@eF^dvcTEm1}^sujQ%h$xQ(ujt@U zyS(B|LRvaR!q2Ebj-IxF z#^>NMb0&5ya*Cdz&KiqsB|a-QDWeX-e^7JpF6|Y_7tpA`u2ReruoI1&Cx}CAoNCs9 zsV$AiUI(da;N>CRJi`p)&^!ZQgr3t_@u_X$Aya<$@n;Yx1sG8Y7nu#iqY0d6_6U-h z7TD^rYlx@ne5XY%D}*h(yjklldz=XPpsuS?B|QtyN7)Yd_1v>auh34GG&t4_F)GMx z%o6!08=I~#K+6HDMvSmTuPl}ymlwCD(_Dt-4TpfS$lueLq{;f9C%9jlLr7^}TbfXQ za8WFRvw!9nIOQ;Vh&c8;Nip-f;a(lAXDFWbkBuo>4cF{jA=*xvK=Un>mo~~E)M5)z zeFtT^V#qlw9m%#LdpdlNCyp+?DdbyR<@;?>c_m<3<=~-}Ii=Tzd{e7@SFRZq+2l)j zaG^g1cK_YRmmGOv zuGp}D<%URT==#;EX-m&fpLBj&?h}E~JNcnE2ZS#IEXz$>|9Bwu`~1*LXJ(ZagnU<4 z`X&^X9}8jRt%GZ4~s{&QeBLIxxFuPAaegpob)U&WX2urRW0Mq+ul zUV~euyMN2PoYvoGC+_;iipI@&`E+(-23~$YEGeUUPU8dq(bV*4KT} zcm$PI)z|%h?7a(k71g;uu3@JnLC6JAR3H~3AYzI2h8RhJAfckbQ4yqt;0QBO%CDqfq7h=~xA|NDJw%~~_F*UUsK zeg5Y>eIB33%w(9Iy=Ki?-}}Aq_kN!)|M zg@Q6~;M6=mK73{5XsXHTKJMFKehrjV0KbO5(gLmV`)=VbQBj6d;Mb_Bw~=3Cd;H5% zm$tw&g#I*=ihwAT6VzrP(tUOtDL!ejbs{`OMFTXX7j927_wJyzKLr>+W*CLORzV`3H2HjzL|A`q*2dLdRZU6lz2_%+wn$<3w zTtjw}$4GT_nKdrOC*zpo*p)bio8;v_mhW4&hMP=0Wks}fWkS^{^lzi(eJej*5Z|+Z zbS*TG6H})+?M@Jx=Ji$Y%5P5(;Q%$f`)T&dMIJHO$HeZ%Q|LWU*aqbM^^SHlf;2=WDjbIC)=Yaf zfQSTTP{EsD@iX_s-7EU=dN%I!sZ_fLvZ~UnXN$?|5sSXzP^B559N7|V3N|IfXBkY% zYpDKwq- z*Izy&^~kUs#LipWw#V($F!%KsD44HKeS$!M^EP^hsnkq1l{%s*i#@x1mjl#(%?wx$ zB$MSD{X%}i44b7K#E=hDvk$38BX-sMl`3JzLl6xt-O>{&*o+-^IVmwu*+`}WzA76w^8NU5D_Xss-PKs9H$LXR0+b~EW7eSf%=4YD@ozzFvr+Hzt zzjbZhO`XL6(#U`$>J8E)(-ZLlX+$vjFli7$lHk%`1PKBy0HiT`GKnC!6G$YGCh?>n zNF$p{Uyw$8lmr;~n+5I|iV>2m+!Ug$=80BsHnNIi0Q!!^U%_7k5SiD!82H8@vf*gr zoP`HU+sP@F^!FR#*$%v}COM@>e*LiXI*}VA6gtIZ8%surKzrVoK>H?;Kuh;0&^{=k z=GfuD=Gf_h&@7z0D!=~oLtr%YqhXbQSa{DKv?bBj%}PO zqcFt*l%A$Xu4oSV`F zgZ6z3sZvC9DMk=fwP-{Ta*&nK%bZ)d;yKYA7V&_Z&TD`7=Z;-Yl#Jx?p^2-%eP_J! zAE8-3dqQ_|1R& zc2G<0X;;5=V(!A~rB_`zwR>U3nR8wa34hFucS0kg)_E}zopUpRQkb;<+NGD=w$8?p zlAy)H&FWzk2q;a*--UFu%!3SwkJODd&+kExRGyJ4ub>eHPEpB1%h<7a_hYkh>R+%< z^79YLxXbW4e7}u%K7)^ID0>SR@n&YuRMJH?eLgn5`B-D5mSi%lew#mkt4+MsF-x%= zaj?c6Bky`m^og||v2AhSX^qwwGnEsl8tZK8KOvu~vsH(NZ1&8LmEiK-301U`0h-vg z0?-is5X^D=^LzJFkG5I!ju-{E@%eQ*LDrF@1DySbdN}*yC7YCGY(~jr|YIEA0S>^-p(aJG9pb&9I85yvmf@fqlBin-W(# z__p#XHT#fqGthkMdSaP6zNAYWSEc-&KY~T1m%>V)T*R1)v{~)j0Ucd(_qk%XXTD6K zE`H@{?C0kb-Cdf>h_&_7j1)Yxfo(s5xp{Zy5ZgxL#}xWeuztMjF>F*H zSg#T;EG|hJwf*a3t?c$r!mO-7$P%ZpF}?XPcRtAdjg{AY+2S&Dm-1TPe*21XDf>27 zb^F+f;VZhinMG< zEPH=R{fbs>2GI;oA*6rP7|-gFrq|69J9(%}*ec(1^cZ*mK~MBmmqgE=Mx)awyE?X3cOnI!_8iW&-=eWu|X;S?@uIANAQZTvVZM&#U+Eu2Y1VT0L= zE(WV*tUywaVl6Itn7I)Hdn)Iwr7AIos3K}rWs~l?sumH4!;u;;D#hDst*`5v0MLss_dZiN*kNpo# zzdbhaE0Sr)49+6zrEZKv)LTHazN6y{73Ml`!{NmGwc&q!d#L5IBkn?SOh8%N13_hN z7kHGlMSs{f+V#zVXTQu(YdaQ_$X|qj3zJIYA3OGAtBW?3war9X+aO^>vJ(c8Hwg-E zTi?V|*?cWqt0XOPqc=ojkNeppGVTX))cSaj#5`dYSqfkOiE|09g`Qj2)XO4@e0=?9 zLUUO6BQ8u^pV9M`46ap6;o}LJJ+EK0cz+xDc)YzV@nd04lJY=NDi>j$WsyXaYLzEH zrxR?_MiFiff43N@MzUaHvE*ag$k@eWNc}!oR+b5q62_V$17ophIJKtfuCSIn+C-EL zi1=98e#qI4znMM>9f;^BUBPCdUoUV5|2O~W8~v2 z%Nu^gII%PxPI(FV5m2X;%hTLta{3rtAN^}zV3v2@qDeh(>R3e^`&>tH%025l@7cH|19mqzg#*GJyfCYf%XxMX-@)TD^$e=^@6m!50nB#vR z(n3aa997i5bSLfPqmZhvE6;oRrxB1$2xQve<==h=cgp5v;8ZKGZ5}wxU4S<~1}?P{ z24VGN4dl&9(8@!$9bbOmYO69LuQ*eMyF6tybkj+05iW~a=t67_GCDwULBxC zk<8GXk$P45!BebjJx5cBPuq*v8;vM`&!#s3JqQ_M;Q zF=@RhNS(5==>T~MoOuD_+tA0VhF=1m0}~&qj{|dJ5c8Y)w*0TDgps#6bdG5e2WoD) zzxQBz!z_-HCzq_zuwgBjbGeaks%U7T78>%@I-B1OJ503abUNcV59C|SmtMkj&j469)K;8bt zUQ=N{vI=U$O(-bKE;&b-jf{)Tmi{9dkDSdvwLXaU*DLbtq?La=Tq482N{DPu8{A5^ zBb}>aL-%dA@;?E-lG@6D=FvkI?VC>$$>XF%@^eTe3%W)Ir4DYLw=|R|s@O_b45-^F zTt{jwm&hk~{$qL3hsy{bT(dNBZV&E+_UVcuxQgJ^3riF8-grDYx`!yK(nq_jC5QYP z{O(H89lI;a*sdB!NN3~x%NN{P8`>t^B^R)-#-1Ia&qcTG@PwR&IJM@jao>_@Nu;|V zbVdzaF^b8hOg>&Qdf=|nPmSDJ^$!9PzFs2_&$j>;XB%Ubil5yJ5gFB=DZWedW3b;N zP87^BcklhJETr`gjf^ZcCo& zHb0?bh&#YY{qJeN)*Wdvb8GB!%V-2Zr74gv@Se^~pw*7h88~(fwW`BZF~GxB5xpL*Kp{!0V)yr1a>_C#LIpv zk?7eK2trzp9Znq~V<8)9GZkdR_Fr3ebh~exJ4|{}U*Q?XvC|)y?I>G5m0OnpH`aY# z#&Y&FFF#$cG;dS!Gu&NDbcXUpZNd@6Ge#p54M|d2k(wir&VmKzv6h>(I~Y;Nz6b2;7MTjh#fFkWWDGVnmW6x4`s=m9AO*5aL0n%tAfq zo;ZayL>kG z&F9s2^zJh@W%1@?2j?sqys%R9}LXN%^8z56a>uiz;3IHY3w0w-&mGR&F)9~H!=offJV=)KUcEW4awE$@a zlbV2J_ z(RJ;zvivjXDZGrP*=LFV*@yCBQwSvdx{#akTHw5u(qNL{@)tyNipar9x$z$=J&NEA z=G}>ngl7Ng--=BxFn+Drwt;8Une>Y-edCtSKA=@)c||X4!@0nMT6^gf@lgT2*r25R)D7aEcDqHABGbRl z4Nzd7WsSw@2ioa0|G=^6Op%lHG^j@EOm!|+nQ=$(OGbvq&P{rMw0fBuc2*83*01Ff zf6mYu*%|Cypy0$FZrH5~#J)Aq=BBIyIH%AlNHv1wmoQvzC6h?$^q-tu9bI0L|A-x| z*|#+G(rpG^(?f{MeQSkh`V)0~BJ1*=b^^XaMZ4)dR#^Nk3cQ=$!5Jq6k;I&Ru1 zviXa`?q6=swXgiNzUG?VR`^RsgF@rw2n4Lz%bPm+|IzGLJG{$ugA5GTQ zQ@Et#C0ZK%a0MIe@*vv5lC~JY>X1%p>e%{9wZCKj1_Wg&ymA|HN;`Ws-xcAcZmSs) z5jMav53C34HJ!n(nulv4Uld@CeXmVd(IsEKC*J~dNG$tAsV5-AEc-D`6k>ZX$?UxX z5$c_`53GKz0m{8x2SOjj1k4%~C`5--mR>U^?_>X%a4b`iAv|@&xPm@Qwv0(V)Y0gA z#gUu*iuU+ao}LIoG|QzRuv#W=g)@8R%!PWyJ!(Dmb*#xp)O* zvjl1!CR?@J1gtz(4)IAE&EhFZn5>p_`lKHGN$Spq5tgz*7b&hrkY9Jk2%hkF#>N>?Q!H+R z0k^*yN3<7PoEj&`>;=VUUaW0Tk2Ia-&KSiptSC~eswJPu+m2L`+R2NG*H-@h^dnWK zX4#PZ5M^p-EiJE@6G|-Yjt<+}hGc_fL-L(Dwo`WgBfaiLHt{$6`j@V~xPN2q-*3qK z@60C1AQ$_}AoHYOhK~%AD5(G$B=JqY7!BAc-|X;c$sj>UaTta>xnz)28A8?XL%X&h z86-rV3>GYW7i;7{j6a6=Q~61$l`yiD$PTTtk*AW>m_gx4r^u-U#+I~*nyIhW^D>ae zmVk0*eB4d+M-A&z9eqEa|DI<2*liSke4QlLguUE|P#qJOw-0}(=})mFrf>&Yl#%6Y z9S3-4=T`E2PLSx|Dk_5SXlQ#Za|RWUD=ntUjMLZ|-6N@K`^+`F>j?2o}hkLm1N zu=$Q55SEpD9|l>(R(!l=xj3e1+rLdVP?fn&cFLD~yNm4ZT4+7q-+4L-9R+Vh(0h1=3(`x>*M}s?wEKK=PVrn-IfskQ_e6$*z8Fn~QHCuVr-0EaJb?i?-!MMXyL()Hn3S{!4Pf+7SiyvRQU}G$k=SPxxtubvrh|TSF0*-j=K^ zwn9_9ecQ6y#4=ls^>JpDoFhb={>gHXPoe_G{YiXGB1qcDIdR=S^pwQ#@=FO~wZ6PQ zyX_acAKT?#IxzJ-P+F#qs#R>IN1C8A_%vUm7j7Ia9;Ct-nzx{va>88UDm#WqWk!lc z;KDf6OMeH=1ACh>&*cT+V2F|2Z| zFXosGDoEffLzcpheMQt}$G#yl(v*s8Rzc#0b-%6la1-DO@R;`_52}s&4$JOR(4bD` zi`}UKs33>&xVf>hI))08+9C)CVd4XlvcHoS^wsGV^$yEZ_7)UJ)<|S0VYK?M-j0hOm>Pd5j%Fpu;i~`%hiHwa416$K2C62af$2N!Mp2u9~uXYusmP zee%<`Jegni?SQ%`VbY)1YuCuM+{g3FJ{nWj)MeKEiL(lezL`{XpfI`uCj80q1FJ&A zPM%a0EsQRObh1bM+pm>3ra~auFKI%*#2J%TZ+(5u*4L^VzYQg=@0-{kEwf*khDK;< zTUJtIZ0ef5Np{UnS^5LYAT2Zh|EoqK?d`K(SmuY%-l{m*yy3e2kaReK6Hj65)Pa}U z=XYD^*=|jliOfgDcafmQ401^-|AL)J?XAn9SD|2xu@{Hc4uK`{zl9F@`f%b1!(R}sXsw5}7+U83*pB_Z z{h=b+yDc~3=j}g~f!p#5GVw1c8Df3JJK}hlIHvH~zXKaQL55WgS{A*{!lKFK;*WTC z$u7%3l59i#?IVeqn?I?n+^2u`EGS9Nd#cFUK6`0P?+GOZ7nL7(?1Yl@mm*MX1m=Cl zf_mp#79+WTWHfjsF%9;jD6{FC=mP@^*uHNmTr2K@=vu_lg%sw;hOU6KW}Tl3<2ZPB z+*NG`ykSlqi?&}XqEh%kr&MtYc+HHo5W8i(E!r`YsB-3H>Gb8b%Mnf0+-*S{8ThGS z%w8dZG(-4iZ13TjVY2#)1#Shj*DByfn#rpt`%5ADZclwg8ZVDQ#JJLp#$ILJD51b; z6jD!W&v*W0RHrGZNXy28mmcC(%7E4RxfDM`bX_a_{#n?$3E)lk)gHF#p6I%1V>wm> zl!EFph_vX&N>69?pAAH$o%Z~nfR4;jwGJ&{SK_4*Uw-kKz#)Y(24`dpcj~n4n65%Hcuq}3;qBZuO%8r z5ib?XjOgePb|pf9>0w7g@SRh)HF}Vm*0#Xk2TDs(K>fk`}Qbc;dWUtlSw5UG`cd4(nuhE9y(-!aU+*`3d}N ztYFAeJhfaHLWs%;Wc%>%4v?*;pyIN`71xsXaTpuI9u@<5Q4D?m|u5FB2jaX@g77yC||_Pe#c}Ws!CG z-+;16#h&bvvwK9k;&*3W6dAV%Uj73L|Lrp(*fc?VIndj`eMY3N+{xT^(r4r-*yO9a zymN>x9FPhJ2WfV|#eMzW|chsE(_E6V~EirzRE;vF^6sf`G$L;dahD zcT_uDu{F6`v7G>${>6d4*Z`y`jBYY5cvu51`B%brug)?;{v~p6I(~KJCrPo%#lb#t z7-gpYl9J%PWXm^W=mYT#ydr0M?q{N^0+%!>A)2BbR2)Yv#EO`Li5R3NWAXOSK&s8< zh^Uo1Wq*{(F7s_)W{R(=xu#eID?qKK{B&}Sca z{g`2tQl*OYl}Zul7apgq*HsOtdsPjv-X$qE2)4i#*MNK-6V|KykyOT)UrU^U2zk4VnH@Wms^lc?%|lS>YcWv zpLz`3qo?a0;jlnbY-eJP=DJjE`bwjP0^6}DI3{n$?)@#~*WYyHT4tvlL6G+K4{HY5 z|LxT5)-y8_OS>i1Z#yFrLSOFMxJ#10Y}xpmjJjb*jVhT6*O7O^*{w@NL$>{}fLz(7 zZhk_qww!U1nve3Med6NtUMg!m8J)aWPO16(uplL5dgJ>$!C;D&-jYwJz2Gvvy2t6O3m#qL!AEPPT zNBH`(C?l46W0WutZvEf*y7j;B?+A&Aqcp&+e-FF$?+bG45BvH-wqN;0k9$rUf79SQ ze|2*H!lI57PkC(GduPA7?;kg^6bY39yDjhunHq<-2RTmK$v5X&dE!j4?uE+a7o3BjnP!p?fI+kWic2dFG!L$#&K$XOj*h}?yhIUHX8uvz<~P6Q46MV%I&){H_WcS^dw{!jTFy6KNEv@Q49u2&&td!{eb|FX}4|CB)UL z-cr<8a?;UDetQE8!o}m!@3-S|A+tcXpfZwK}AvFQ9c+?(u z)VH7a5?huBsP}DwOv|S>yOnQt7GzbyYWKlq8o$ z(myVYZt9!Rf6VH}K~|6L8|TiHb{<~EZFh8OBUi$_H;=BS!mxECvfHMzX=H01y#C)! zC4}_Fs>K)Iew!#4^Kl=kA^G0_U4#G07rs)15;YS35$p9B=%tpE&ThYYE~zt(jvvT& zeTLV2LH07Eypb0cjCspKh)52#XH%0`)V6JigQGT1FE1->y+OE=wBW}@^y4RVyZ3%; zyq#7{(nV)3=ZoZL*#A7bu4xFujOIIDX`2)IWH%jh-p546HBWDV!i^LqLoJ?c*Z{1q zoJ1dvQ&5zIYpVBdxOjWTf)J%X(^bLS?R~~)@U7!gcaNt(9lxe4R{hla#6NPW;O+~= zG30%|7;oxWlKz{`NTSk=1o$j3_hH_d##n`HTYfKMxPtnd=@d1=Sq*Pd;^3;Ik6HxW zPQ}SR1JA0wT(@rWQI~EKUSBo+r&&+_AyIFamA2|~&YC{$GU_+Po&sEmgl$cpH zxcMh#nPRtp1mn|WGS6kK=&`u%jCzc$r_$5cc#$eMJ z2da1iZ@^4)81ud=eD5Erelq>)cuIczk#S(r8ann~W#Y#aWkyDwF!UFZvO7#Byo^eX zMI?%`((u;QbSCuI;*!e*FZJD#Mn3J`Jd{|wl0>}qiI)WgQy&5odH@Z`c3mL5j4b)> zVjWRtX?uG5jS@L`p)Zx`a(k%CVLb;wcLxR zgM@bfQoITqZSG|aUOSjlKDW2e!Lcc{K0|4q5c_Goc!!RRhWKlTj(vB*`MM*HsTTxrd&ca@00OF|Wzg{vEzh9~a6`;Nvk{O<018{Zh7kZvR4Y#rGST6U2Ym%^>&FzB{V z@}t?H+jdqr?l>o9OGSQNGMckZ>k+EToD{vcZ^Ek4sL%Vwfp-t5-PM;v`BzS>dyRcb zhW6_@p&y^xbz0M)5eJSgy%kgywaq4Q=G`{CLK;uLDSZD+Cu}Tb^6b!#P#5CQPFVUC z-QarBi7n|Tb;qz%#xa@vS-pNzckFe;cXHiT_Xmp58y7I~N7aeP3cvcEe2luQ1F(Z6 z1H^jaVPVy`DFUwi-owndIrucO6eA6W&&T9%UKeZ_scrl1pQJwqX>UpE=}#&XW+z-0 z$4>rViJk(+LNdfx%7EXyu#Kqje`Qa$j4S4Mvm*DHj%-*3znshvYSmZ^Qe92Q^y2p) zhg0}-!$iN$5)IiZ77;|F-=?20ZAtR`a!p%RU`^W||C%-;w2Wr|`73KIw|_WdHH6QW zqrc1oUd1O{V#t*gDp<~1UhUY*m-ohfYHHS@j4kh?@*TY?mM9tBqhPX-`U`Cb*@d7$l$=ywXE_p1HjGg1y1`#?C?~zkNA=h5e zNJ=H9;DBRu0|cXe*OsU)P}&-4J_ZOTtEyx7ESVISbd|02N`}HR6l*H(AyeQp`(-ul z?7`7zxTLbYHdnYL)Sf55z1W03kpX7nce3u*ouD0~o5S)G2V4+Afo3Eu(-W2D)Eg%U z73@ZWJ)f;S!uQ`-UiS+jQMq#-v%o?&}u4ogfW0Ai?5*09a zpUnLv&vU|I!s0(%GE>Y}>835etDQBPGVk@U$}XsIjiXa*tY+QU^)MPb8ve1m{#sn} z&F36|38k7NI^)UyhqYg=?nsDXzGOQI0rqj13mDGnk` z84D3q)5>iG`y>xpG2t(a=UAUj^UuUE~h6e@Er z4eaqJ5=HKRDvx>QB6v^rOnG6+6s{-!Ggk`7)#%#koDk?Kh~wxF@kbF}`gh&{ihAFm z0&|x>oqBZO3*kR-W9F2WT7_Yh8l+E`td$Ky#~Avf5;UDrwqfPx)rE@S)!S?c;2A{- z|CPxXZrPW?ERbP6L`G-+m^&>VNiU&l=Hb*L*a9`2KFvrZsaJrAq;OI|0~<#h2{mL6 zb{3ax{@9orpi~YOM7kQsn;|g!hdK$g0ru{GYFHK|+bq^vWX{pp+MIBaajjP& zJH5W|)BbMP`clcIA$JIOR(f6iXK(DPARiY!tGXd*&KBn8Y;PX9=4>CGwXwVJ&l_eX57Bw{AF8TSkrjf!s}**WplRMzmr^Y@uoxHNHp>B zilV(NbG(0P;-jKV+ZQXMJ^S@-gm2-5)(3i2?(qd`>Cz(cQ6Z6RTTyhFZ}&}A;*Urb zc}Cxls>mF%jA6+sFNRJRa>#N%zWBw^^vB-X`)xZZGJR|#VN1fEc`^Oilr{fjBj0V& z6qgKjnexIa!M^x6_I_)}p^hCtJpgi;p__0f;oJ)!^9D8g+5^X1To02?`uFO3yhSY? zfBE@XEZV$oO8|)=oBqB+VFr1dOpno*7yEgP-v};g{yR$?l?%U?IA)*qUtxWEgD2HjJvXEQ>Mh(Gr@<~`amP9WR3lBCeaXdpUL~q`!od17k1$Enm z@8m;BT0!tQ(dzr#?e0q~PHdWr zs-`61CNY5RHkW-QY`eaG(L#mIs*m>Ha4I#iiWbQx>a!YPF!0YyNbtuB>q51Y$@oqk z3+BiwTy#d@V(A54hTYmX{kTGd_Mj{Kis$ac zN`XBTp8JBbv`VV%fh&a%uqB0y7{BSf$*C|zN7A?rNy2PtN*#L(z)7syI51e;r`sR1 zd4ggwRKRQq!Nt=uVmAEAz-(-P;r_io24)ixHoq*fS{{ykv1J>^H$}`^j3y&yeyNFr z(O+3-C%DRs{%&LBOLzd7O=QQ_W0Sy{!GeT_8~#D8zfPO0N5mn-`n90Blrg^{v{GXhUcVMXooOw)lD)8nZrmd5U*YbLAt4Sr1{CUSk=0<{ z5q_U}Q^!W-M9W5ewY9bDVaA2&Z7hYv*rHigIDFq*gXiR(3o;r*7st za3;9Z5>jHiFPt&`MY#CP_@v3y{ZWtcGbHEwui&nCGsb+lp{LnWNK9AKgB1VyE9zRQ zXPY}IMt^titK#oA#?oc@MpxlZ%^#r)UF@CookiyoCx0o|5L&HaJG4ic(zpni(|C!+ zFan>Y^T?SAU+iMvuRgOt*#D~34{Ue@;iQda)k?unf}A>oH#h-m^M`sZj=H;~XU0L) zRNLuwu~JC%fTcjBxjYF@Zt@M9)P8Gwo3R{%tbSTv9T5zR zcV_Y%;#elcOTtqdG~55|-`{dsQLjrxoGwfEqhaCXV1 zWpR5d;@%$@8Pt35#@h$y+@5hu^U38$h-J(C0eiAVukBlIZ>+6L-Ma1B_TxyX_NY+P zwjua``+&NMw`7#0l|^dy7Do5(^+reh!pkAl!}5Km>rDTKY>FQ{Pe6sr@~#q$72HxRZ6t~L>iEaY;t6D0C~WtCSi!+Lw55qE zDqlyE0^Fy3FIplxc(xK3Yy%s&AsU3+u)zl4HlY_`7eV88PTkzCdzcpK8PkE=f>UMz zvJv36plvPIa!f_NmTUx<$S!nVvJtZ8-^*4QI`p#bR}xYbdxaztuk~ibhEyb}2ewU{lGgs?Cn<1^E}u5O_09EkJI1j9 zV|m6MUq!%<)_E$t(m<^2MJT+k;pCB5sKxmtT*6u~PqN5m&us_0BmxuTgf#bT7My7q z;LZ+X2?^>~L4U4m*k%6B9IdgS59{UU)jD}hEy8g+VPc7rIqyK$7UfB zW^6bsz^x_N^o4B{sk*55jQdsoogE!!%H_uiBXk(*RWnJ$y8RlG^r9`5B&{Ns$^t3} z(UCoZoFlOQ@b*XoG#%2aX%3dIhM%>19ki}YMY~8RFhgZrNfAScpxSH1gmxV{FcC3J zJf(6p6fQ%IJ^BN-4E0B6L|J*KaGcJh_KBFV{nblk@h>SLTV}p(WNy^ddRFKg1%`01 zsdA8oOwuvpwzkd8GE+;8&KYr%gAq%OQAj9WVf)Obe+VBP zK|Z_l66zQ$6FV4rkL%(2JB!2d=ckr`eIyyt6}f(;vMi=d0^$jJXefD+u z-3y~yH!jVnlaBu*Ck!Yo+A}G7O8)O}EKQ#akCOO|g!C!+|AF+x(zRXc?u7wK?zl+$ z-u!5PC?a2}Zk&yRwZ-{$6Mvmik_2_;y0>PxT#)l?j^n)$+TXgizJEdKBV(v>Hn(i{ zz-UiS+_$`_mH9L;e(l+VgTsjfmnEJhEJ)%XDEbkDlf~jV>4Bnde>|8Q{!gNz7Ej2j z;J@pR7el{c8gAnfI;KeBZ>$1~Q9x>x1=Igqj1UXcmT4*G*Az>~ESOxuZIBP$v(4n2 zBG(}Mk5y!H>pG&7xq596-bK+w&d6n9T?Le*kBPPAr&CBjqH{Lw&f+F*LsD4GS$ceB zdBcxba9NsOSw1rXKbE8I8fh~@;^51B(Q1%@V*TeWz)A9n(WRB) z)_(mPd5TQ+39@;XssX=#P6q$7Guz0avLD-$)|`9tdGm*C!v0qP&e_N`g)+~w-G?P79Z*NC)ges ziD<&y9PWZNtDWN&?+TPfULzM+k!+#3vj)ckrf>n6-^u`bPLNYR*#+!sSUK1h)ClUC zh%x81R7{I|C{EeNSy)SG9GYfGo50v7q*4pm8WjVrf?!_C#EQiv8DOWhU$HN%X-`{0 zJ$N)=Wb9lD6FUkgu^0UDC@6NPyDo6auCt+BI3!ceCoLWUZrfOA>l@Rn2T( zkbxu464KAuMcwu>f-!d9pCOyZXOfz-%be{{7`O@Bvn@4fQ}>6~cJQwpeJC6|m=%Hi zvKu*e6yE^eDT3XbMUx6W>tniTPB929=d3Mc_gzGA{G#JU+&rk>qHoQa8(hg@L;<}b z6LRk0PaDI>Ig2K2Ty(-~``;&t-k5xV-8E{rp!z^`7X*K zTve-CEjjfph2jT{_7Zj+f~b@JGzxiRST}+_i%;in(?~lq=Hu6dK1m;wS|p}&?#$FY zjS;6z-XM$p2|zi?GNS2NwvS8pk@zzPU|H8k=jT&ixHOd!W$UFGDb!!C?I$UTwcQz= zwvEJ(DfHv>T1`fSsCAA!f_#ii#_`8lw|7H2pjfzqts0&3#`BA;1^8wQw{58ur|`-? zJm0^OHWXlAC+|)Sk6D#Rv5?HsMN~nMWgZcQ~Q&pNLth5R`J^m zvJ^>cdR`tQ2NtMA^N(RZNqdY;8quKis}cm-r!vHKb{kPQuM z*&WLb`1COZpoh90WD6d{KFiOI-9>6|&1;=JC57xY{irpB$Mc`6^i-8`^!7VaXFdIb zr7BazgXY(kz=&@JU3JV$P_c=qz$RqV7ru%Oe)lHyt24$n{J@28*`f!ndK1Txrx1jv zk)ufA?W1LrOD_scWx8W237M|=x6eq-*ik_S1s1&uJAS_&5}puA7t2DC=6ztt#Z054XfAaVBrY) zqx_S?ktuJ~wzRdLl5=N9-(ob?PMchmIzIY(T>LjF@gFZMf9-^|KY$qWx><*}tV;>% z&Xy6-o$XPN?rb0bu-(~ipE?f4{tL5PPX)2|XlnMxn+CV`UVZr|-=VCu0l% ztADAKahiccL(?mQVW~@7;3*;yAt4{hE7<{A_t|Yi@TizI3H8)5EeSSi4xt~Gg#4j) z0JlRqhRF5PNpE3IMB6k1$CTDFiWQk@tFO zMSh3xr*5f_!Rj#==Z)R)(gV6Z?E~F_9KP{y&;|*5-YWwtTQ^n#v1Na-ZM1-EC}kTU zOW87(-kOH*yR7meoN{B^&tuV< z4hrBY;!U;*5t#545!;!TTnl< zKV`Ym@~pr7`pV(nrKr=PZA^ufW8hp6+oRUv7$MN~FVQCFFJp+vkF59^IOBDqNgIGm zu&X~Z_=1%zFvNF|b)u;pZN=Ugnfs@UmRj}9w{>Da@${G$813Icr>Z1*N*};Ty!#;*^1mmU9;l_A_I@uY(w(u5|yy@;s0FDa2n6 zS*6G9y)@Wy6PMijeH}N|)FsWAP=Oj=dI6Xn3yjAAlSafjoMKIo3j}lif_YLwF6;j6 z+y(TE0opQ9V4m7C$+i9e)^ub^N1x}G_kF3l-<^@^NgekdYMA=}l6%I#e#X2_&rUvL zM*0(TK8oojY=1e_17mC;0s|zPpp!G((yHr40?K>QU}cDzGt!D)C_{{EDewEteq!g8 zyrCCihvwQeN9^xU7By+N@lQy3QaPFp8EUkrpmHZs#uH1-MOCb=`hgs_HvfaO`6`pC zR}Px}qM|ZDKmAv1szF_BP)(_q9jO$t4fG=HV-kIjBa?w4u}3I@mRyZ7(yhd)i+f&d}{&aQYf#He$tL|vLFO;-u=^c%Chm!g)y|?l2 z!xQ?$Lh>#A&eM$~^ZPPz=Sh6v1>w4O1vdNF82wLKwRgwi=`SpEyODe{A*<*02>FOJ z8hgIngkAqw z7@p|sA&iaS?B4pdGoIwP(0ME-MH7vRTC`tdgQIpf9DDU7TX)+hotZgavsj%yT>+PMEvI8%0Q%w-cB5}+^&jQu?*H@_5>3twS^8iOzXoc=u|<;2*9apktxbum0$mPB9=Q^@p4xS@dAp4$H?3yE;-NN-5)sP%Tcgc zAQzIXrVA|^ZPle#8trPr8Q(k`No?M+B>HEjP$#}vHhlMGIZdhOq|TU{({%hfY3spC z{cx&Ds%AG=LD+kTsio8bMfzA3?kYZz@q{|Y6AExKgE57|29DTQZB3}TGMAm82B7R( zl6MvL2ALC9;7lbNEpW!U$a-N-kALL;y>-m3**22_w}FR)ZHVUK5a()FP5XwXBUC84 z&g79b$GCtajxq~bPvy>azQWC-by8maUgzzRr+?cEhx4D)cH!FxS2v)`759yb&PGMs zb6n9@Kl7TCnkH^yg$t4OZ(yI)`?oP}5_MBPNba^9W^s{&BQkSeFO2N1waS@97ScZ6 zu?-}00()Qca_Chjp&kc<3#Gd>KZJUGjEi8(zux)K02s+Z&r{;V;sT-*QMAi8&mLtKs1yPeVJ-B&#&Tim z6!oXF=_QTr>*^n8A&bw_Fy#$gLTx4k$YOMS_3rVEA%twCnR923b=2DUC>;}okE!_t zHRf87ypoxF_sXn}#mg-BO=sZDtMtY@fx{Haaxh%F1qF7RfZPtCnp(g0gHek&or=B5 zlBrA`scgksE=u+tTunR~GghodUCv)qzQSs$)T<{u$jjNUrlOph;%!Lz0XnoHuYiIC zQIMt~t$lK#Ofu{OuWG<#=L6gX84f{3>K9y5lXMEW#|WeKu7?~*?0Rq{&HX1|c$es3 z!6BNhFpxlJy{Rxmk3KSHWGv==)zq=!=5Id_qs8``dA}@+ySFm#H-(XkV+U`n7@Sj) zU2=6<+?ADa(;piZ-5wWz|FTf>&`CvICl~!_eDpu#;?GZsKWkZN;wh7gj-HcvOh)4F zOIL4gt8TnBKIyLXr0dfYS6#At>sM>GM$2A*eqBybUEAn@y0)Pnb#2@JB|6ERyzi@% zwERN6b&_Os51r(8Zp&tFuzkkL50(d;`xB?ME1x-~-d9DAXAY>(Xg)sqs>rE-yKz+m zFiG7BKdI|0jQ_KG4rO#!H;|9vn)4Y9EJgj5Kn8@yc-l$MD`C(fICUUZr3f*tg-qFj z&xgUrZp+@%CVW&NkF=0ytlURF9u|(%pQMZ_DI>-o!wfXRM_`yZ@tORAWbO~XuS{Vh z7KJ|PfKOnXg1{LHVWkykZ4yesAk7?x{^aiq=g4OK*liT(Pbv@te4`iw=+!35RL7Go z9!>)?-dl;??XL9E2}@H%Yqg-}B;K2^;8;m)wP7 z7|?jiqTEQ^_TUQ0)@YPGoc`YcFV}qPOwy3}l(?NweqzhW1?02(tO2tt3a|{lLRMGB`4$bTz6mS8xQOg$_#lV2_#=1mn%#;zJ-s6m9}~-4z?5 zKDO+w4f`htc~jnCGXbf1RB@Kht>V-`5uDqSu>~q-lkU{GM73_xzf{{GodU5W2p_l5 zL>{x%KMOcwt^PkChQ?;~Z|%r(_P8PMcjJXTF!Kb_;J~-!f{ZD=V>0ffjeVcBFs+-` zXSe-)3I}w3Y(prg&P?|=tU+#MjH?Md%laj+A(BuUlNMzP9S4^8y7Mb3yNM%tiBd{ zNHNMO0xZ*-!Zv(C$k?*S(=n@+jIo$@ctl9E4;p>qabrfmzAg|M!&QNteLQR8B=0Ni zTD{i@il$?E1~t2wxAevr86c?8Z;KORZse?wEn%BuT*Z69Pjo%N64r4380Ys1F$>K; zMZF~d>>_q(hf=4o526jDjX;mp@$1rc0*#Q#Zy#IO%JqK6GSr#2h_rrX5Z*wgpe zOWrx$giS+c5e2|bhxgeS+f=>UTEN*mY-pI@-aZqqk;9LiYh-y#ub|#+6M}lPon-FK zmihbxM7z~BfLp5rxAs9wXeaDPS{7%w-aIB{%YFHE=_lUPSloS7bW&XW26&K^^bKu* z2gxmcL&f;tjVnXVFN})j<99EN_jZj;+S@YgICKIyDiWR58~^X9ZX7fsC3i`F-A;65 zyQxR0Idf8UTi?Xe`@7U#39FLqu90;qp#CmjFt;`|K-9E(z_*JQEUfML$)0iOqK!&7 zDw4`GBe-V|_6jF17JmO;QpqQ)zFJ-s_dpSF2jPrmiPs1}k_0|}bXnq`COx@x|2i1v zhkp5D=w>E{aw;cmtYq->ARi-Y7;_j2GqdHRTo-D-nAk!*oGwgL_$;-OYunZyED5XM zt6w^`am^kmuM{GatGx3VVoA2<6b4()KSj`(l-!2f z|2MvF|L+I7{kI3W{SRli|MWn&{~i8SZfBq_ssF@`rUMp-*?*E?#nMtOw}1E_R`|R9 zgQiOlaQi3DD$wm8n!&&-H){_zQUi!*t@5+$vPwB{)7y;Nc{*V|a95`YAmLpvK-AbLJj>`}8Hg3o^%8su=&7EmeBf_d}SRBW0UpCyDt>3AXQ1|}QqlR!w8hzN@9Z(iaT1illEM6!C4NhVP(<*`844mY!ie5GP!l}-A@Co}*=Oadmq#j*Z7-$%vzOx+( zuf-*sxCG0vHpo*Kv7@OxV4~TEjU|9s8H*+a+SBL{Xpe-QKEo?sXlAIqS6+&mZF5df zHc3+|&dM&?Rrn-y#k?D+G>F*<9pQfGMA>A%hXJTbhk4`^(4MBjIOo*(6E!A9&9N`M zn1q?JH(?)J8?t~F5*aXizCwdKeg@y6L7RMo9aBn;t6FT`@3L67rED{H*!3`5`=-ZQ z%XX&Mcx-G%qh3?}Qj~ zCyODEpdHC=M@|fR|Gz{G`G2A{TT?Ef9)Fq*IzCZ|#`BeVGoHfI9Hji-1x9GaTN&4% z!H(o>QF-Sjssuix+AvxbocGtqSQRX&rl_~a2EMnGuO|cvfM)q?(Sd>8!YN!*dI+G4 zL^Mw6BB!=-s`b3pv`f_{@JEw%aroWq)~{v!HFL ztDud{H|AYsLR$Ne#csI5?|tzLjB zzJtb;RUw9O!9q@40!0mv&Bl?G)?5$!c^xb(Jm84iP->NKj5(~`vi;)}AslJ{EG02K zmv@EMZDX?AKFUm?_`3ZY#%$lE<(-PH^gb_6u@FoO{Ve^LV`Ic+IrL&1gFSUyK*d8l z{d9Wu4OcxyUYO&ijbbIIj|vi~TqwO9qh?Jqj!wae)zTWm2Zm8DKzBGc4Rq$t-O=t- z&}9m{`|pk>LR4zOuQmWji%4k;mcRj!#VOW;$x-$I6I^J)Ym7CF8KT@mU606{p1vtI zVe|(!Ex192)n-a|qcFs8M5Os4V1d*qI9E)D6WemS3`E-ZEEeSH&dGGIcNzm6cN2qy zP816Y)n=@RiTz{pVuLy9E+ZHTSez>{n1O@EDfsZ2-)R-Qq7#)~7(ANdfh808Sy7*I zNl?f5OV$PAGt^X7?h|Z|9q3Dpz`|#4`RIet9NK6#1NE5Y<0WP|Y2xZM+=Djy4SrDW zCB>TPQ^$e@8@FE+d^!;;6eGOUVoe3bMI9m48*g}9(P*X$*NcI&7E)!dNr6wATE`8%sRj;Ou=#obzp-#u?k z+@^}SC&xv0^cuWz-r$^h*(Kx4;_et1xwY5eoLdKPoI5a71zZ2mCl!5G7>&fmFYXaP zcWG!Roc%v4jFzCn?X`Qz*8d}E>z}ylLgDmZ_WHl30o&p4)qw5UgI~1o?)vc?KO=zP zWf=)BExh4(B>aNI|1reucsTrTJK2){bhYZ{GQFUj%3WbVJubR&J^IaZ%GUecZJC-l6y#%vGF z{hNI2)B<~Em)^c&T*|&#=>TJe+bb5y#%rnR=ci#^!GHOWu;s|7Ygn+liX0=ws=~*+ z$uSbg(3hO|fQ9>>l)}iY+?BkWy-B5Ms4)Aw1jdPO`D1G3f&GhxbcUskr(fR*{bRQ< z+28s&MuME`7}=kE_8j$U*ShP~NO5`Vc=+u~g~>yf!fDmzj1r#~@{W#?^Ii@%?mhEr zb{q33n8M0R#${`Mb$Hry-I-_Fy&+yR$PL;R{UQ7(s4cHl2lS-OX*5fD7QFx{j>TLqQ?;h^)>zY&ufnY zzs(ahQ(?>Mzw?C@<{eDD6WIAhzA}z>>3S2X*1$ujakg#T7()maOD-d^F$MrJvsf7{ z<;;8er}Yh2_okh({U_kpe#iV8^8~0#6WT7d+iSoqPCctVFP$NooQJ7sq1~yTVGOZ# z7FY$>oL=Vl(^y>vy|x@9tM17W6JSJjF5pHgcJ!^T0{!QzuDYb>A>sC?LOEbqdCD`2 zAXQ_4od63TH!_}*`D{bvD~$Byv~|o`Tr93D2yxvGG zkoelqaf?91_(}u5AY_3_FLj#!Bn4t{D`97O%Tx+@EP^y3CF?<&k^tDt$Og~YU0g#v z05Vlvl?_(PF?M2XB~j&vg`qj5KwV}iJ<2WT-2kbG{3aSe*`rb|?TmfhDSn_%$KIa3 z#4E!ZkXWc#+-Lt7dY3(uJ&{WVlVzx`T0%MkhH%0?#GEA)01fSHlsSxhv5>$E&cB?M z)9bO#ppeCm;O_8*U6?|y`-)S;lo0Er1_|~z5<5CC4VU!X>*d!OR1k=<>zwYf=41ct z4J%D9^gBbVXS;=yo-4^2ePTH^TV*0}h&De>1V3vEEbT4-`n zm+{eODoKG+)lB4&8G_seX*L= zhI++Fu0-2+W#j<+Ozz4X*oTVMUcd3TTwUd_dSt)*x0fY;EF_UB4-}=cD)KCb9f(gW z4yrb4JlA50tU(9D)xZ7pUj<9B@hXM41EQqyD(E#*@q_gF=j);|`%*Vj{JLzfQ- zTk!g1VZ)Ta+Ywvmcs1%C7h8A6>j*GdZC6%v9Vdj2GvnhkTk*4l-WY#I{;qSzE-=ta}XKygj{Nmcb*E)6|wn zq+`q1Ta`VT4v;HMINkV4#Tky9c<4pQC?Da7JtC3!vm=RjIYjYFp|ib6I5GU&-dr0M zzW=`R+LwpW4%~J%`e>iQeY80)JJk|*si&}@zNS6pB@I*at0waVM(Bnw5+et0k#lo8Ut#MVy2E2 zOqPgSHQ=;Ix9^GC3v|RK?PhEh4^H6{s{+z1iGIB>|BDrwsi%eBR_R_JpvG0k)y8jx zv9s*ZO#rQc!!OVZ%9Kyx0zF_j1t$~{HS0J}5W)J`BqDn(`|ZQi6<<`;`ymobd5wk$ z>v2h#+DdtOvLK`O#>vSCtJLo+DaMWgT8l0Y){)^7*%Z?LXyq~`AjA-S63Tzg(l3!A z#3kJC(Wb9;kKuh?53?^BWn4^80dH|VeJ7(GpSqdim<)0@dKe7UfRRl=jp~p?-vx(=hJ(IanN%TE_L_%=cWsmfdH4)zB~o2~vq{r>)g{ zoyDM_FJFc6*{J0J+#MxB+^nhgi#Y-Uk;DA7^cpN?ANy4XZe!d|((|giAqQ{XS=B(g z$I<(bmtgFGj5-X|GyK?Mgz6b#H>lt{isS?*`;nis?u`C zMQT3EkM=Z7W9qBsu+}gAi)GP!xg%TMKYlU%zlKTX$E#)b8GS#31IdJqWlXr8@?z+8 z7W?It^YO(mhNeH(luJF-B4=I|Ij4XmhkXhLPUQw{kqh}4B>Ueif=TQc05@_$1el>y zsTX~tX)QNITgnja#w~dgm>Nw;;1P1|@I7!K;e?pHt*l#9b!=oz-~?}&{2?2R<7U`a zX9Hn*fm?}`BO@{(FtA)uaT*YnJjEvdKvZ1ZR@xiAb%2&V>e6aRFbcFtgru%LQ^#a> zf`2Wmf$1@b9I(<{5{*7_V&=k6@_euvqTBZU{x<%&giv&_wTK0QFShuyE%!5~(8eOQ zXHQgaBG-whTb+w8cZXv7EN}+pK|29xRrvVzzNvOF@^YqId&xz$w&uHE`siu%E07u#Q>xNLg+}VcJscg{m;k zF8gvs3(8BAx+j{Tt9}$ZAivC`paMSg*rWsSFYVEu6k0jer6G+yiX&@(3e7D1QnBde z98rtCzf$p+Di%oC-a7AwS@cc@&^)+=rC}ycgabC_Z#h2}6K0XL7|!VJ03qZrxOaeRjY;@^hv((cj{PluF}#kCqm;W7pFj^g&_&W3%jI` z75rMW&SE41(Nw&wLsz8?qf>xr0WldM8bKp_fXi#VdU;4b2G&B1i>ZW%mk#qgjbJO| zejYD7hpaLMj+l3sWh0~zgB(+Jj1`K4awpXzqibGmxMdYyNAue(zT+*irB5!D0Nta- zOL+oh@KQCenOF_=jOZ>#?-4A$a@z7MpW&W{c6j82edY2BlP~Mp=qad2tn%yk;C1qS zu&=zoAR`2EWk6yG88F2^lw9v07H?_zqg@C zxZ(H@jWWoBjdLdC%thyHl0nvN{1754P;1wc@NXU5MW%Ac?AE#-iFxm)5>EJH)s&hK z8s3>q?;8#$h_2bKr_~)~itdZ_p2uW>wY&DfJu2Kk;Gv*~i{KX53LjLMQzE z;d`pP+&AvjhTnU@yqW0LmbzQHs+=Qj{ZnTYC^=)Ue_7k`Pj+m}jZ{`FAi-p6ayQwa zEvx^(N;g?u`K4tWiITN%?&9q}`D6=u-ESeUopcJs+2*dj3rl<)BY~IRclX%-pk!uU z`M3`YN@mrS&;GD*XVq2o!7&F$ z<-rYqLM!Y?N@aCh6MlqHlB_9|X{6xB%;Bvyfypo5m_}Ct9T*d)AK{HBqH23YNLU3f zms&Sw$$lL+b5hu86A7xEGk;9$g;N-*7Z7aap+*aGA%7-|QVk)nG(cDkdL=riRB z10901G;{S-Cpub*DBZR4$QMM6F6JZzz(?GA1bVZX#wVl!wpFsM)&bTLsksj6G#Q9l z1dqx9fc290^7!dli@tHB80_;Gm6%9C`LaYCV{TFnkZOTqeiv>Ka%KP3Kw9&-GFjW_ zQn5Pzvpg|9#O8v)^HZN(c&y#PQ2Jn^p!@-8?yp4-&{{Kw4no{%+*_D>&;qPQi z3Kn@g`Q|iwiDjy^vzlq6)vKOUfcfmxN1=xYsBY?<^`>YJ>4}t4+1A|A?XIkZZPUB; zT2atzYC&4=v-xG`6_lNmU3W^iUVX;&LKEi5AejD_XO|R}#hqIjcY0yuo!*0UB7--U zWtUuD7T4{u3yO{!AN>Pbs#S$H44G8aHmT@PVe}C+1xSuxoLv6td5bd3-W^l+)_}Sz zlX~sS?X@B|ZObG1WpxAU&Q0pI;@nt(o$=kq5kT@3Y$)CD=0+qNmo?Veql2aCuZP-eS3nMir zMjD?AC6!)Z7-={m(zqs+G^6U?#(l#RW{h9m`04P3-RQm6G(2(l-FGyW;dgi6+xW)t zgyhc?(%Ozk54ZSG)2tIl#hq9Y$vgUqMI}VQB`?g#U7uAL-J6%z_8URKHJ_8#cC&DZ ztiq4edxWYoCq;*ylajj#&XsT8I2R~{dADnpCeH1VNaWC`D~di_w(I(SJ@4w*vwf^= z!`6XD+T_Mj``NPS&Jj#C<$-}4b}s!G`TLtU-Z?Rtfg|U{|HB6VPb}F_z8=DsvZ(g~ z`9fQSn}Th+(u^+Yc_>3@`Y0nhRw8Lno zozBg)H!ve*i7RcGQGN1cT(XX>PAqGaSSC)q*BSr0S@llnVdcw?la1DJ$*29ta=0)} z;kWZe_?y;r6HZB;nIA|yraAj-8X*xdO!SmV<^{_no6IuFH*GS>PXlF=6N6=v>tbY* zU&YEKmj}uuRhg6{2ugjQ>ZJq+_Z4bnO^YB$lF=G`wIT2;|~O|^rk0bt6BGVJsco~nmi9LEyrFS;aGXYRi`E& zOmDKhN4yc*Q=QP8LE@E81Uw7PTx16k@~H@ypUnLv>}|-+3&9lmS38lmBi@B=8QN3o zX{1>)2Fmm{>+i09>Aqe>&Au~uQA`_H+2g_<@d_yCF~sboR}-N0B}XBeeBRz84#z$- z;6R6{VNU8o&Tuv3aT9=kk=IZJlAUTM`RTtvCONv1w9f{lXCKwroP?8tgO*H++KjqV zWOIO<1pHNGu=O%qF))N;4eHTIXF@wBA^3~6BLM!QnUyH>bpKO%%pyKy#VWt?w;n0Z z!{hVPoMXieDi913_S-%!;qGsn!SxH484Tfc(^JH?* zB$Y>V-o~&X;M6XWfm3Wg{;>qgIvW8RdvjoYtJUJd1=Q5?M>g6tRC1&`usy!0egCTZWm~^kv-Pv;#?#Oa zyMNMz{)sdGWA)a?>c)O3e(RUC{-MOW0l;n}daW3dwqpV({aIcaFq zTXy%R?2R{Nmz)EvRuOmNxX5g7kG=7xj5CTEt04)@jk1UJ7#00lTzu`>*>yYN+&=|^ z#j~>OI&b1%F(;JReoSGcHZBs)xjTdU1JoS#*68r}Luj9EI&@N>bGXXsfh!{qap~FA z5d&9@;8RauSzn80maV%Pn|->l5Akg?=M=WiW@%(o`rN|SjgMwfV!7X!Igu3^^WV!$lzk@m$T`ZPfeReiKdc~OP$7$XSj_a$!jaX z84`aFeDfuPU`R=&5ww*8F<-Y42*emVd9}65!6(@)w{0u4ip1!XsI=L*uL{);6plI@ zYlwkx3RIFeETBk_FOenb3PaeV0_<{&5Q-U*o`vMFc(R5JP2%ARDgFD&9`dUc_P;MZ zQcfdgDq@8wRIsr;GEPv41PFvZ=QVfa-20*ruIGD*u4~h#q?Jnde$=m(I*msDIhAA~ z92ThOv@ZnScqb~{gq@k;DHAAhdM&8$g0Gj#4W@a5i8OTVs3-u1QBg&lY92al{Nahh zLAjmO*?xG~bE3_rlvY7X74AqnqhrK)pSepyv6kX@K8I4oiTkzH1>SCW=!MgN6(ylq zA-mw4M55%7*Zg2cJH#DMol03z?`6|J`VpR{_a%$ac*Na!3c9Lazu~AwgyD=l-W$LO`Pe zD2yna<89;o!E|L&q#t4n$dScW&zO!s)vRe%*&)lB$DIJOaLC3x(PTm`osRF5s@_)R zI%%U5Xtg9uth%-@E$SE*O`wOny zELpejLA(k$7Fe49D)6v-^yW`+LNvK^^%eD!v5UI>V}zJQ-Bv$*dh)1po4>P$DYlo7 z9K{wrasciTS>*O=WDB%>^H8Gd(0kNRyRBBAh)vg>9ZJ*s5I`OApBo4D}0S=>Uq zYko@qDXH15`_mJ_jYS5f4sLDfo7jF%*GNms;MN`Z@zk!79r$s1--K18LHzw<_;bZ( zijBmCFDy;$fD=g@SsQ|4Bh$k2Fgmc+h$6S}uFR&Hb4g<@&Y4 za$gmjA*o72MqZ5Ri(+4*ON#&JXjy7eK3@pK-!jS0)yt};z#&JL;sFA9wv?v^L_M^qVX=YXuKvmmAdL8jO7VPZ&wXkW+-(_D3(QUkU4HU zom@$>b04v^iPNP^Np|i!oI(SDTG(Qt#7YEM?mWBP)vjR>?TDD|wX*?d%;wrGxfhB;F!2C5g9#KzuWPCFA)#uw=4Tir z)V}i5`iYwaUcgvVi#5Nf06RSpm?UT-h;m}IjM*>>n+2;9uC#LhULQj#De$7u%0?XV*4?1bb%#q#a0=WVYbi~BM`PRVUt0no`gEA*Ok(e z^ssa!C4ws$Pl%2lY7`BL2Qp%?1$nr~6H5*=LB*4{s5e>a#q!~fxA&QmpER5>OVoA~ z<~0PyWS6Jb1rP8%T3q7+8HA-FjE;WAVWI=z*ccEg_ek|@(~N)?NVk^STExbwD*1r7 zxuN4`L>?wWf*ues^CEHy^cF41w}YmfvxZQ;Sy{j*!v7{ zLYzYLK%~SZ7UsK zwKc%Ah+#7FF-|AMyWgo^d-8=VNrxS~Ur`gpKw7x{@ZQ-f%ys@ra!EM+{N4@h@Bi^r z@ktZ1l7?m__P>1f*8cw=d+!1sMU|}$Yv@Q46_OAX6_XGmpyDBpH^e{!1QW(6Fe;*c zO1$EX&Oydo5J}r@Ai)a|ynwd=(IYB2MsbESBYvR+BPuF-@Pfw~l8B0mGigO7N=W{9 zt$nGgu3c5=$o&8RJo7w0kKL7`x_j5Iz1O?mcfG4&UH?U`_a~>_Js|D6} zOlDzYapdTd$PxLmooQM9MrUm;&MbVUIFkI%F$K5hS6&cF+Ep680Q$(PA&s;1D}NP9 z+EkOYzUP?T`wyZW@~`07ns57Q9NYCbgPLKs$!r$MDo85L>N>mU1}cK;h7U>6&RRln z-Hl_Lo4(|iH&1}G|IE|~Hwo}|_U9(fhq9%}l+lBhjAkNl0KZ={W)P&TJHDq2`YG~a!5X0}o zvb_zKZPOiHu-)Ph@c?I#TTN{aXp)VuQ%Wo9NVSdhluTO5jS8OK16#AbvwkIZyX;>n ze0+lPC96M|U><#GGLwH2_o$@#VGZ9_s(ZF6?J<&n?bu#cS#(bIMke4Y$Mo6|Pmn?4 z&2^-}5=IOnNYev|40UBd} zFn}Xa>X9v5uxW(G;c|eRgUAS)gIES3O8W_a2Qx}W_W|04-SM(5Vi4jIL%glf8H4Cr zvM6k$?rWEDTkN*Qc=>BbhitM4w}nks)>}kgfN$Np9kgZ|tni`u1#HfuL7OAyaAM%b zJ8=q2!HoJK{wSvuOvkQZ2xOCH{8&&w)88--jm`w@LpjwHLat7jkR)tJEVKvDpPQWF zf;HA0M^7edOoIT!Z^t1JaZ*p_Q7DDS9}eq5(>EkKOnG*r;EM0? zqo>smL)b0%JP@*%6|FV7A;)BH-ta9XU&*CX>CXY&_zbI22WCwt)`59T8VDgvk3ly! zwhb#DQ=Nu%gv(`u1qPv}t0FB!Mr_%+KM*6uTQ|t1-Jduym1=Zhv4 zj%Om#f#mJqtGf^PBNq0-!Gzp7sFj z3uJ^zX~4c3=#s6C7mZ%Wd+{ndB?$X^HL|d7hG=A=qA?<=sTNUmw2))F@7OmH@#>YC z(%Z^`Gf7WG2h$F#A7Po5coF?RI28tbwvl}zZ{1%a$VlT$D+=-pW({e)HNUbvlJsy* z(!Hh8#pt_MnO|8NNxGpX>Dp;y;9_l@4V*D;hAr}|N+7BHczhP|p+WU)R z3-Iz{9KEkNwsS^iVRp~h@u^vbNB4{!h5z**9D8v_df~$2*h>8E$ewMN9X&c))MHWW zv#3ZbA6Q0a|AUMFtDvg)i>O{ZFul6wd}t&)OQYepBdy~@XS6 zPsGb(dPY~I4{4ln(d?U>LIb%&nn&}s!yo>-qvIIEZDr5M=JU_X`J!~U>YZk1&!kQfhu$5$ z=WPa17CO^_4Xs>>3VU+Ywgn*b2pds#w!(KBg5gm0+>^T4n8doUAn&aAaYEL+U z8n7I*i$-*WrxZYq@E|!lIe9O6h=G~{(vU(jQuY99)H03|ocIQkLU+CgP$TW8@vCT{ z_H|aYHh!3(BId= zEGp)W5&%vQ< zkmDZtWO4IdOtYCl|Es9iEYW0fD@>)YH>9?nxU|m^Zlj;DZMh z4lHFNt|yt)u5c6neZxZ?U7H}gki+ZDYlBg3X{voAK-`2r za82QjOJVZmgCr0xH*GHMKyO7QFtLY)91l+!2ekFY(ipfe7m!b*oVXhWJVL zcWt;3pVxlu_wOGoc%DQ+hKY$o)Pj2VQ~v?|iaVbv1C8pI{xk{~w9**4Q5)^movLv* z^di$MZdbY*GqZ;1*DE_NV+YGD-?P;#fLoyMNd(ze^d~|P6?d~Z5}|zUltAD%ED*T) zHg+Zv7>GBsf>mdlFH*kL9y|t5SlwAd{}%kb40a}vw84+S#2VvTSPX|6RZ*qjF!=gZ z25F0k0W7=>=*q;r1vm+)uKE|6i*#RF#?Oj3w!pM)|7v8g7INE7B&t*)uJazSR}g6- zM;`H7z;Xc%cgwXe@RH+*;dpzL>knV)#@Ag2e0KOeWgI=@r@ink))a*+439aTwBv*C z-ya_}alSG?gOjY9tqgjo<;EprS~^XfvFcPzTEtv^Uh?*gc&{D&smRl4f4; z*jga`r0Hbl2^=SobYi%y-y7kvh?eCn29AT1r(s0NE=Y_Igppo2L*{oNX~bxXi^#DS zORw7du07a&k$Gj#C5IeKFk=4przj*1*zb5)J<6y=)&T-daSa{vpYBMaLdx7yH)v-Lp>yFD9!Rns#?~l6_>qZ6Me38`kA5 zYF(b3_T_-I*9W9tdf~El{TkLC_M7y==ZYi81#XU0aJ8W=km4Bm*Wt*BvJS{%85U~D@9 zwX>*zZLut1%db3D)U@?%{`zY6A89&06uToetMjAu(B@+%#%GPq?EGtbXgXeAftT+A zw;!jiGY|uDiLs<8ZkB6K&P`aIe~Ld|9@ip(oq=nN_oZIj(&rbc29(wdIYq{GA@= z_u$>1RIHOX%eF>CC>bg9d*mL9`?FZFtx{<=ja(ei-doqdTx0}y7T7~Ht}d$yx+e#H zO)^djEC`H!p&n=>GGjF)ozgYo08v=w#9h4M;84dI%P0*MPGLOSd2D*^r}&Jneznz2 z2MPO;?Aq$06vCWBbjhm&V?qrJHF-y(%he3?)a4VAo=Tksc#TxuydaO^*HE=G4jZUA z`{6hwYqK}3AC-h)-Fu+4hRmINOXjT>4ZniqkDMPm0RX40dbfO!-L<}4A;ot! zQap!4i_d|48{o%=Cid=5+DMDp93j66bc9?H>ARaI&K(w4p#%l@ z46P9A?*`e^JnXQ{TseaRnO(LJ8?8oRy4U{5V(Tjk4>*)|={m~|6MfJyygqnDPRIkv zAky76P@-&zG7}0hq*s9zl8EX?qf|%|ZLOsJ*(eTyh-=cfCXb|?&bl2MU&&p${p$6Ug{R3r)mNqO z4*%+saaigMs~rx}#DDAolTV1nHQFg9Yq0bh9cq~Q+X3`Q8$UyzhNRg9FWVCdC;o1K zn(eHtOoUPK1RO~wkjPp+^!C#vPIFwwd>fd(a?clS2M~+HT~v)q6?{e#bcIq>Z677@IZTSYB`wAG%^=br`)Z<1G(Q?HXTG z9o}*vcWm}Hh0KwB&425mL74L%!>X+8*lK%{Zz5Fx>fq`VWh` zM~$a6UX|kleARv8&BOk*=&xm2$!qR=eAo8BcE7ygtyS>@PD}ct{{DM1qw`Duau0C2 zxI(6`)pCq}qgUaU{y0&%r51H3+|YVS0M}enqWK6Q%oLekhM6Lh?JRV7YbDMoLYAf; z|5P}PQIot=0wKi4MLfZjK2(A=I|z`60J3p&#JK8lvZgBz&0#ZO2UL+jS_2&Tx)gDQ z?!_6AGIyVKCAH>iDg;Fz8dB3lFZx*Q?VHtePndeH~D)y#GR5{@EO2|eqqL8eR? z@Cvv#>MSFM$&txX;obX(p+=0lMRF3I{*5()`e~7uNmw$yGUylac5A=v#fd6zreG<`=b6VO5+7WswdItLSMP@PH;-y+y49GsaGG)ek;NOE0KXC z9;|8c%^zdAlGuBLeiebS@}tg3JGO;qu=Em;YhP<-a9#Aos*% z;i;{fPQQN##YRp%$60nh#LWQMLjKrL#`=pDo-3?T5brK|2o)dx#2XUp+b_!=?MJ*l zaZ4s3@0PLDW;?I&e)SR8^82-mHH7@;N_D@V)m^u*$Dz8E34;N;;4`pgdusmLdTc;k z6wpIz9`hbJO=={mq{H9iAF8Rv0lXVsHiKmi^$UAEmuNsj&7b@+oxN1T zu)W}r@(ziRR>hqs!=d0#RMbX^yo-~RC)3l?_Y`{W^ls!1 zZOAXM$2nz#5Jh3B_R5#*=?@SQWbw@)f*E;rM~jC#@eMx4-5#1k5?Wq(D!yPxAIPOaOu!%Sn?0 z@3Jh63Ph~9Jt&;TWV|bCu4EW2 ztF00N$(Nje^Wud$1qZ^Z$rGd70klVFDuL_p*qhVcrUU zSzp&;vjwXG7^jtvj`Jy2G^UN?Z|T=2Yw zoCC8n!UA!ZssWQR03Kt)?^DY=K_}Vgt%TSD7!kv!nO{3{vwkknj93RW)Upt820y^4 zb_;SSY7=cWV)&)bHATM{7qK9pxTmr^q~Er&@lnfWyVvo7VTNkFizorB{+>ZKi zRXF*~jOzXBNHV6Z;=73z-9AJi??ND^K#{T z;)Yuy5WD_)GkJz}5d9n3d1L$D?^L;**?e-$`Xj;?{IY$L1>4vu1gi@#ulUYP2&P*$ zCfrpU*w=$x;}jY^OTZ^03!|ls7CXm>s(K@h@{Rbe@s-87^9f5|m77_06n~M9*?2jC zUTV7$4a_9>rr!go74c^&Bq+lX{qdYLTajM;@n#>SM(tbuy&`3Kv0p#7llQ3~8|bR* z0z{J39-6EgZdEDP-qIs--e=n;isTmZ-%J1IZx@+`MXIX^T&<7o{{Vo6y&HGU$X}Z? zxOqE6!omSP^VTLfMk23&QsMIoI-a4bGYbE}Veu6dJaYZez9i1bgN#Cc!{VtoUpIp! zlW~BT;$J6*8~QzVMecGV|A8IX$ix}?fJ=u2qrMBV!^vUEI>x=Z%uxy!4j{k5Yp)!ZgzYBvB+UaKrZor8iwSSZ=@Uq4vR*fBFv6d_ULV~s2k0^IZR*_OA+4YO# zQ^PjPf7H*3rc-AA2eS%tR)pNL$lfr%=`BNY!;c2@L=VwJ&@+cr09FW%$ibFOcILya zUxXi-OA#`UIi8a@&NEe&SYlK=4{hBU2kt=WWTKOq1k1YI4O;9X(ahOnkkP#gUnU4h z;FQmK3TKMw*vg0NSXY$%*#^R$Z~y>Hh;`)?q3;MTLgG=UFiK|dbxfwkqmIGl8AHXW ziXAi1Yf6H~4&7pP@fccs)!YIIsGo-yxVf8y$j~>A?*VOV2;G^Z@?I4LhR~DC`rTTCEm7E-24vdOG4KEKdv`h)uHIOZS`h*_JI4x>>2qb+GZatz5TzcLHhV1UIWT?@FNw!nh4Sh&&Tkq%L9V+B1vArNtetopr@Z8qE-x$Lk_$SA!L(2P1(+#RWO-$4@!&A4 zR@F}E=-FO%@o9oUD=I8$MpYX_PgT_>ODX*rwI9`ZDd|MOP4`&BrY0v4tI-~u5E_=S zCo4xIO|Ra`8#()tbqxhTlkM3yt;+BGwPhXokf_^o!&pr^ps>)=t_eKhh>OXw8_t~O zxLKc%Mtq`_!;!qVQEilC0H8Lwaz?iDSRgcFr0gKT9QX^v!AM!Eb9B$R-tnW<%!By(mnWH2_zmhd*C1x;>wv~C7EKvA94c2X+MnHCo@vFat~sw}dC zv1fi*mO~G-4@BFCNef|AC(lId5!U#Pim5Q>;DRi@rT(Mi{6?wvQi4hgJnON>cUd5` z+B&`^Bx1AeoH7pXg{8ZdalD>}j^`KjRt~{vH^kD1s0s9qsxqh!G?dD=*#?W4DFL7p z$6vn(Xslw=-ZAIP@^!v}<{U6`t*~HA;8#8zUyT$CUHN{jyOU*)vA+>W!iDu(9wxvVvi_D{k*f!5e z5z8GNo#SfhBb?8&eYRMSGh<{t^~I07*Fv#IU;kEjceq2eJ#<-y$oZKscxc~&4RYlK z_AR~~R86j=k5Lj8qq#4}>G@+=M+n$;W$vcZvuj-!Yi_qAvfUYBa?4uA`*eV0% zeAS5?s3E&&wM6(NJ0i4WWiGXxb^lyr0kNJ`B-yz5qRyKZxu0six#OwszVv{P0~8`K zAnAVT;@3P17*U_*4Bn$YME8leSE>4xz%e98&WWDNMmh&iGD&q~x2^ldoPFdiMB<6`@HLJ;ujhNJ>6z;^K7& zENZ<3t+8`bFFk+Rx{l@RzN>2;pPV){C+(1&l=ZXH21Gs>8+mtN>@?2NRmC1+$!Iip9k6lm_IAq6+(SB{P(jjc&Kqcpk$o+H0Q2Q|Q-X}_%4UKaly z8;+a`G{;S}E4CLe*;Y>It#vy3Xjdd9hi9ZjcVrIfAQO^54M^Gc%Vn+PCHdppu{EI` z+jOfP+XZOH_R}C++n4`WHBmc#&{ok``@gP;KE=?h-x-5COM3R~AhxUOi5`~|*|hYF zh4CvyGwq)6k-V~$Jqu?YdCpfKo|VEp&gbIa77dzG#P9cB5j~K2oR9{% z$uDKbnz5N3o;?X;-u8hJJa80~0W=w^&zgFY-(UNLuPr$w8Ie>SQCr<2j7X|-V9B72 zNSY=*{IxLX@+x$DiL_#Gd_qlRc|F$OO73&~tC4ixJ^Jdd5uS&(2s zpy&7E7P(w_8kZnN3ly;g^E+0zD{JwTVV2&XJ?mp2(d6Rn z#D8kNXNjf(n&1$}eB5U)z=rQgAL9XOLI!=MX6rm+3F>EOLU$@a0eTazf^qVXTs@X! z)12(Qwqd;^!wESk@CK35&nzDyvoQC4>PNhlkpZ=DsyoRTEa9v=>;3fSsqs|DW2=~? z>R1ibCcXcMtzHygaiLI>;$fxcc6+9uW5Tyq$oz)g#vn1CLx@eK80@6t%&aFy6`2TeNYw<0r5fX!RN z!JFnPgqEG+Ih+C>#@B&hsAxf}cu6bdwcs(HHxBxKJUrIeN{c@Y=uEb#2!_W~T<}<~ z3HFZ>E`2)P4w4T>z}m=U$~Zt3?S5_sa3I=<&-24?cE=;`(C(U>b3o`Bdq1kBp`tH7 z$KXj)^1NtuN)r+*E>L*7x0*||W()J9?wEr1M?c7(K61^a7jNCCb>T&?qB;hTjd!B2 z-Be?pW`2XhQ$wIL{RdB1!;BV(8mLX2O8QvCN@rcdhSzaR){ zCp}~Cl~Np{a($XSaP*cdL?TYcJwkp&T*G_|P<|&WsuKUH^RL~P-Z49^JUjK$UoBhLTGx6sK;F@5s~$ZtHV{y5MAniKnQNC7N0yI`JUTG8D>Z8h zyd&#Dqn&h2^*jkIYquB&VToc0nZCG&~f$0Tpa7p@L1iMs5V(_T$X| zr^5Vyjb;9SeDNRN%DEYRvy)Nawh}!6epnDH65jt+^#fxIgllBZr)BZg{Zq0biX058 zZ4SQkmM>QCIi{b$X%AizeU!1*lo5lLj9{$w+!f6eZv3K0=iZGAZY-mEHbN9dXrR4u z1T$5=Z!YV2Ra*a_{K!Y^gCQGT;M8>P50Ly|#X$^F=8E@e4^|v}$|F0NZ}Np~+OVI~ zWlDyqd9vnLtBhz1c>q*Y;PWrYn7`xuQ^Ug2G8ur{gj zE1b^@OVy~?2i8_@EzJdL{;F|du^7{jF3XSbV5MyM>YOpuOKu1TudLeJEwU&;28pj(|pc3+!!kkhk$@Wkp$%aq6=PLRz4xNq7 zv+V?#LDYdmh;Ntv41OfUY#En!9f=|BRwVS2hV&?f&D@zBJdtDf5xbQ>g!(n4p3Qkl z+3pI0Nu1)X*2BgOH6)VYxEGEH!uFokko08@YDkI%?P^Hrl#oDfY0^z?6QL`Rm(vKimt0b%Reizk4iY~!+sA16SxXV!~#?86kt?1*#4 zFp+ybfl6biH|zZ1+!**TT_34u5^XCSWz+5g%T4in4bQtc1cppik$J)7g|2SVWg8Lz zF~~Zh#kRu>nQ7BN;w!oVqBu#X1+7qbqcHHBYz(lyBD_qqq6#qN`GMgX(1b5`WI*+-yJjRCrKgOzO46cpf}ovodisft4%tbpJN zDC3~I)Wg`)u%b_0%9(X!8vi6}84~#}2UPmv-r3#ouQ#5BUay4HCe2=JT}2BU9amlc z?zGOcd0W8IZdP}U>y zApu>x1B0SLKq@^sgEh0H@`DblCs_(#kfhFH7RRA4L|ARs&pqGm!^t z0<~&m=Kdmn-!9)HTR!qZg$P%Y{|M!G3m?Iu{N^pG_zmRFZ8rD+M49_jRDC95g4*gD z-J0zzMRV<-L*yV5D2EKsyeL+^dwWUc|Dk_dJ^16|>)OGs)jpCkZ0M5s-Tj%7YZ@6; zvNm09^_CEcf4B^au&RngcJBWGfOsUx5OPq&NR!+ct>X7^*@4Zm!gM;pEJ& z|5!=8beuYAZ&^i*2|u1FOf%zH9mP?f1s(3{@YUij$&UaCD8O+VpOX_9?jrCObI(FX zh!%>=t(k=Yk)KcR`8z>91Bnw<-ZLmSF~#jLLLz4Cd=*uD4K6Dgr2(iBePRNGBpZgW zjrdFGzzowIr>F;3Db4ht%kT<38j7N<>FS>D*9h&V@oQ)pVFtmPHN&dR47=fpN9v0g z4;z!r&#-&(5aLb)6x7cGcVx%H;<&lL`KaqM4cIFBLg(1MckfkuM~r7gY!Qd{F`!n? z?a#iv%fAO3RF&cBm&D(f9UDCyJYhf5ed%XyBm_ysdDeS6XZq#34hb>ui05BF?V6T7e?^zn?>nR zUIEZ5K|XYe#MdD_dodvasLao{23WWV)(~Gp7?Q(;i{xL9IcFbqA=Lz~u|pe)R79?U z<(|?&6Jx1Slr?WvrYwo7niFz9V!ER^Q+L4efNes!?FQA>#8;HDeRGw|2VZmY+vs&I zOPt-+l9Vab|&wLS;Jy!R09_d74LWIqR z0>*_`53rz&4bQelt^91uTk{(>0dtT3^)&h47-96@(jSG;H@tN0$K=K5SSQ2@e5V`l zgjeKd@`@a@YrlC#E}ymkydvA);Ig(Z*;@N>ux$SUmhH6s*wUk-t^A=ZqKbeCK6qN{|$^pYqsZCp4Kbc6MsA9oM`Fx+`?4@%i`A-XLcTjHrl@q z!Jsm&D*4I7UjC_Gk%|-ZV%M&}J%{kxv=O<~FS_F?fVKA+tW`CRZJuy5p_tH%E84PD z`)JSlwd?OArDE(LRza@H?en@2MyB)o@&SFWTRwMJ=huKd?=4LIR4{Jo4_0I_emhn9 z(N9!dlU(I&jF<>%4 zHq?^J;fz&Qkj209;@(NB6_IxTf?t~|e(cZAuQexh)`n+Ecut@I(%6UC z9*$DvjG_^9?Gy%^9S6{b&O4_z27*rB60J>4?1$F-zvP7 zd^IH2!LT(TI$+SU6AyFFLxi@=#{bBR=!>cd+iOefcz6|Ow%PqFJ`E7)k#4VrrJIC<$*m>A4h$0F^aW)8&T878aZAM8aAR6@n%X9e6h_BP-iLgp^pLZRa%Dd&@tE6EMiw=TzeIe1HNjkWW{Y+eQDvU~Xefs-LNt{7@A`4U%?a@0 z6;xK@{KGHH{m&;@(1{rfb!1ETlqX4g4%mGk~qpcjRS0?A7ufk z9KQ*NAZ|>+t=5_FVY}S&onQNafj+|H$ zIpLk$%F823XO%^J9yX-$jv)ni(y)NS|N(HgG8$wbIhrm#^}yri?3%T8+``twA&FcC ztC5{WnW!|24$c_TF>`*X=*>v$$WU6r4ZuK1Wu_^+Oy zdUG!(dp=)MfvPkF@h{I$EqeQ@-tjeyJImnu`1$SE5W?}G)+R6gXjm-H?EpO6v3)zK zE3-V;w`T3znO!#$Y%p?b47&D-Tig3NgE3u9#*M6beSjqa!L+9r>>ez2g|w`?fF2H^ zcXn@l6*t(kAQ|^t1Fqo3oGy!f4Uu(qJm-qti+Ys)XUU`+&l-17QEcz@RpaluC_R1f z;Oq-~7EHW(&zZ+feMnw-@+9aarGJ8>brdWr=4BSqH}!368X45L zZF*o0Thqw;;$}CF%X`VHVOuec__eaiGR3#8(G=V{I5Y%BQoWB`7;nIuo{>6Tw`yCZ z*7#w^qtnsniSGV063rsP0)7?b?hnFCI&SMMu4C5`Y}WOC?=N{V{GP|2bYAv1IRO%= zT6lq_n2?MG7Vk$lyK4iM=Sldam`MH^v#p5NaLO=BKf5s)4rPcC`LV*!igb)??RR+$ zEJXZO@Y)?+1O0$Tb4&l@h=4}q5=MXRqB9sobhZu1?D`rlFX-?rxf9W)+~Jgcyop`* zVJ?v{uzIIn;Njaj2e+F=WY&<1G_mzQsLZPK?8&SlDAIy=rX8@R!+>IV+dtjRn zn=ttKH*qRIG4`fVKMp91AxmUQn}r2}om!Sg z20HzVOtah6&~^hV`ktUQoidAtDBC&1UJ%uhpx#85+f2QIUiM`yK)s1VDjomI$XUO? z2FmC)e_BuND;#Plfoxbt$>qVj*K}$!M?(lZngRswu0KBh{&+X%fK|k@BWdc+@AP+e zSjDQe#Bq19^Z3&5xycbpTG4%gz!ChK#tR1ck3)3Id*EV?(^Aen5O7oV-#zfOIL84y z5+iBU)~WP|B~5jVF?Xm+_souA3^_sO7>2|d$`7@7*MwZiJf)`7y$0DKUBK)Gq|*@ z?L+n~2q^=m1$3=}?>{za^L!08ASZ@Dp?RBSL~AXd*48G>NHD1#w=QwQ!mngE^{TTa zDDcFTl~Ct(LG<&ouq_~F(3WC}Q&qE$(3`N>44(y+1iXK6C-fiXA^;B(=v0GNgl)cJ zk}2cDExUFz;Z}KVBZ-3I58&brp~lhf|bNxLp5wf?+i>%LyTE*>dZ zyCOYv$;hlJBQp!1D2}`_HuCDg*jZ^={a{1#SaIY}Vx2_YujMVs-7)pGV0p)*)#Ir1|*5J9Kaf!{ePMPN$pZDY-18~<&$b7b3!6U4e}73 z(g9H?MPcqzGdk2Uav42CO)1RvQAL}d^~ht!YgyMuhqwjw?`7caCg+4ktZ+_l!>of< zI(vt9kIWE=5l_HVL{ef~4*vVEH@YqNBcHFY6`A^0!7?a^xVq8k|4X1ivvX4_6t z->HTUl~j%pNUQgP!3z!k{{57{Kb$3N0^wRt_IbH&;}~>oBlk!S+%8lGl9x_Kf3unu zaEUBBr+Q;wT3##1^xE*v|Ba?Dcx7I^IhwlpmAVS<3^!1RgsTPrSJHeg*WMS(LSo%` zDsj0KD=2~iUJyNOcrgTsQ?W^l;MtIW$03+p1}D;50+1TOh_S4~3WN^zQix%EWW!f5 zZrTIQL-`+J1(+L&sOX~O;jyCf!q9__x=!gHY(gm_#>F1uBAFvohDrq{QJq1@!@gm{ z8m{4?%J35Iu$B-wQ}gncmWEG7&qm$=^_8qEV?W<>@DJ?GW4O`0uZHja!>G@iZYk=R zG&!B+o<34ZP-3&5sWtkL>DUa8A>A)M_cB^na!8fPav;5{tVdot@EQCfuuo$$qb2X;^sI_s~RSC@x4(M_R@5rpI;^pWl7 zRM87Y)LvV~D~hO{a$-n%J0!HQ;%p~qv%)P+jb(@mDm&nnXW^8f8I6W*z9u>hBoRd? zL}@&#f+}%7zr8HN~ek+`_4(pQN zK!a>+n!b{HyK?fv0WfqjW1%Sg(klygL)rYo8fy^aCmHXP0y_VyMmE5kX0@swVnsAN z=CF-xW<{hK8|(3bO9*B;A_F}FmT{46WFw43d`q{7D`|G(;;i43eABIcD7nsB(q%qO zJlBp?uRR%HEUUDzm>pkxlw?Ttrfto&CBqo)1$*~4wSWITv2oY0-Cu0nJIz1r$JW|! z{JeeHIvVHa#-2TJeEj0#^ikyl%K#SRc4ix;j3`Gj zXY|z-(Qnz^WNXo&e#JQS*7f%hyH(Lc@IMPvuNI=n6n_82!qh)bTtAbXL_*iT61ssg zM?r0QV=WhdJsX5T&c93K13X8J1O9B zev~!eOq;m?&+gR|qI8xT}+gv6U zcjM(8dij~qZ&DGfK?_LLR(i51n@zSUAcCcSQ(q!lyUPS>McBA7-fgGTzx zC5L~uqh@9(Bj>ve3TSVeQU+5J3O>I27rK#jIo(K7gh!@Vg_}}c-bMY^^4C74AlGWy zk~@Fxs&v}R>K6=b+dDiX`vQV?ZKr?v1W{bP6*B`VuNH*q ze(Nqmhg&Gs28Ksy#1fzbBCUPUoQXMMo>F}pR6|@3){`>$D18$;eyI5pg=?!2j?{?#Ir&=NmiiDO{TyNEE|bTy1rXT zfW_U&rrE_-DxIijS`t)|+{Zl|*`PNYkydV~K{!Lowz+=FozTR0l)$YoV{rBe^Dl5C zlx>U|L>SRvH!m>V1|+x3MUvX>OQ2&D6u3>`8wrS)coBY2A%hIB#S<+8Cd$JIr5LD8 z#S{CDt+w>E#!AN=bOSk$PUZ*Jmc`2Cb|EA8x!0fXTXPHFk; z)g(Qe2e&_Zs@SdLCQvA36ll-hZDobE+Zp%^Wf7TS0h@k+L6NLpm(i+>%P914TN_KId$S2272>?#{)Ct*=w= z_L>d0r4D$-#m5Mw>|WA2!4x|V)EEjTt#PFqW*%CZ3&R^g6=P<=`u!m3G!E9|WFZZj zvTRl2xH)+cecJDzXcA|BDEOj3qVB5Uq2d2b{~FSEdOzwsWAT)}_b6>@-tg?4VEHso zDWW8rdzr+cJgRnXywhaUT8fFX*33aEN z(bcLFmXyf4wqh!YbZ&Oy7+dYx80tX~;K=!3Q$^NR0&5k~zE?(%Alv^z{l@V7 zW3P;!@Xl+yZfgoZaN?EGQ?87jF8*3FXi5p6a5lgHKjS`G-jzXnl!g&b2n957+=TBI zv$B%CK;)bx8o3%9Zd6bDY6()9O`(vVMBuurlFH}3*ql%xa zzlaC-ISM`XE^ORTC>l~bc{-sFnTc?U%C*kRjS^(SO|VvWIhie=C~sjAM;|Tz@fvviZyGC1@jnz^|K|tq zY=~o9M6>6_T}j8JfUj16D-Y8G`d!9bC?hjm3{YaH?-+Ltx|Xg|^jaFjl%-c4)7Jhx zSG0v6yuZ5WjpIZT9P?8WOkQRfn6+oy@N|j(cN&A`!%!P~$RbXOC;r`fkI$j~*rykSNj%ak{&X_c%yhI#~2#_;bTOl6-6WoSe%9v-LvhT z+wEk5j$9kSg>i?gI=KCre>R<>iRMJ~DgZ5DkLX`zfie-jS?>?yxY>VPt->iCV>#02 zf@Y@VqpO&&n3!hDP=G!1R_b9?8b#+R2iwV0)B+{qnRe!Po*eW8J$&MpVqih{2DK%} zDSIyM4?7QI#4F|{UFYH_KM}p6K+2zN%o2P_WbeEkCf>*zVEB-P!v`;LHMx?vTFhN= zST^pzqayiVo3`p_0eOYj~q{ycQQO#cPU`TsO z9?ZNfpz$@ASsI2_+IE5mQ_ri{1cwcyNP8vT68&CWL_$J zME%&BmMzSN{FjhWVGaW%T-1(MzJxlyMR7ZL&bE`rf@3spUjlu-|1s z)xRA!_o7g%jF6?Axi)vmBj4;mcR7i;WG=_~9{GK8(-%@-u7M+gPMK8=M+BCl?PNNU zm^03w;8J`DEw7EGW}F+trQ)-Su|do4IyG7JlN$y;@D$i`ay64`)K{B zBs9ip2wD$#`GrJI^XzeX?HNbHNd&+NkvUVN7yQd#u%WFeVX_$bB{wjbRu zov^X)h>S}oD4cYljJz`bc9eu?Bw>4a3BEd^F|S5!NQLcwU|9IkytPl_!X?cQD{NP| zQr)~wX>_HdKE{=t-Pqy@(nz!o2=5H>2XahjBG?dc%I(;DDW>@PpX^=(aU=(LGG5T2 z#CmVE9T5A01>LkgmVtD^AwW0UjaBKpMT_B(>|TRYf@&#*V+kG*M?7J@M-3ZdV`5mE zMwB!!J(|B>S3Es6JX6$9=@K?;Pc|Z64j+)dX71e;{)(7N0A~^DCDj>$7g`gD7(_>5 zhNw*87^|$!7L~JrJJ5I>-j z=8|Pp$UOk~Ha$BZ4I(`g3wV(61kZl!0i~<7fP;=maA?kjc<$CLO7P%Ir|L)%hVep; zSi5D~fAdM+X$VkYEr)OW77|SlghneSPHOMI>PjZuguK)+?N9KB$P@}gaA=IDtw0bL z5qz@eNMjxbt`p`*oU%U>M%##D8NAx>7#$KyPXn=}?}JfSW{1&}--d07tZtOaGbcxm z-w5TSxj~hF${ijjvM-hXqvRmK#|eO)$yBI?W&}lt_x~%L1C6~0_{ovFGByB|hGs6B z*6Z=sizx1mU*>_DL~iwBn(Yun)6ydRwIX7Xd4TPsE~2!^Gfc_I6k@BC9CE67lejSG zsr&P}lXU$C%axIZ7KEFF+%+IG9K!DYuGrnExttLzIPzwRBO~T7OE-;6yq>l|>BsX2 zS);1wzQh^-b|-Pwt-~oi``*iubkLy~Gfd@%;>Na?i{B`-d|2-nTu%S5e%-G0N17ioH%xy&0R&-@s zS{%85U~D^ilAN1#RB3dvY{`~id8+8r*0cGmtGTQV{e_}E`wZ#0YixSq;^J7#QHxqP zA38cd8U3}dD~mRr5{ivY&03p1IJSL8`r1Xsu`T#tU2$yvjLfx1^^Env-;Nv{du&Gf zD4Tm^?=2~rTZdTr~GI>wDrWc!6Uvs zu&Zn75k>zqfGi{*0^ISkkbHMV$5{opF{$IlxLq{+zNS#0R|35MTZpZiGIr3CvCL78 zzA`$FskW`myHPsq3ucNHyXB_d8h*I4iJbJ!sz`y^zSzE(?a&DK<$LyMGtzbzEl0J? zF5WMFy~cWA+sH{g4e>z0aidBr3Lx@w=$S@94QszRN(n5@*jZ_;%DDb@` z;GrO*VEL@~fQ$s9jbDZCxiBt*>HpVcgHD?=RJ#^x?hA>wm2gyWtwI?yZ%$B& z_4;#h2<2{D?nuk`!)!!eC7S;Ccboo)3xiDm)gJ6xEcGuI`}iq!v=PgJmn?>i{vEmA z4SY)C_E@q{sWaj1mH2P@hTsz?zfCRP-10ab3LuK0Q$Ao~gCbQ#SoR=Gc6_}?=tC0e z2MrYnXApDM{hCRUrt-aApH7r|FoW|yh2dHgX&)X$uhw3p0Tv=IOWkE0V)iJInd5Y_ z6mv&+u)U;@b*s~K(7?2mIV#~V#Ws@Aiqq$k^4PZ-u zJtU4)f}VKVjNhYr3JWRdI#AN8KIINdzd0047!^rhQSOZ570VA@zh@*^^s5W$;qV;U zeIkj&W7mnarE5l7wj&lxY>svt$u_|~of9zd1fj`x+ydbM$hT}&0}Snlxqq_-qOh;w zlnLB@%0jA%umTMP!VsE?TT(_3sGHAW-pU9Ctvg8q5??SOv}5c6=q4JpfP_O)^PChq z3BYXnQ1hw+m`t9R2^_Ilb*xn=I8(;N7OU8tznY$6sUGoi5wa#H9wGUpdadAN+!x_q zk+!T3RFR;i)TU{e-an^yMC8dY=i8$f3%P(x6{u}Nu;ssa1!taFC3l^VhXv3A+StI> z5!*AOhQw)#E$`iuzZPwx4Qj+Eqq{Sd(UyM3oqE%A*|N+2Fv!2_^BZuAwCtkDL=Cd+ z63J^Wlx6rN_x@tnMk+Th*T7f==KYXgDP|ZgshpvNSi_r-?frvNmO1~f*IOvdn#>)B zWcm{#&-PhW-4|43jZWEr?b+t;zxHeeKXZFFyN~4hlcS+idnGp|op$lp|L)kzyeXRx5L z@y7Ckn;33gv9WsTXRQZNZB_H;8;h?a>c_KeO_}Ex%;tZ)ePd{+xLmlr;C38Z-Zq5H z^P`6f&@g(?lF|8#j=nNFfZs0}Gid9WLw9^nypylbKkCT>%GOB4Nr_jNGi2_3hHlrd zEz;L;Lp%~(UNd|EM;n)>1(bj)B=MBno%|%~6Uo?-t{OY;3sv*rp+LlAj8mHS z2I(OwRr-2EJR)@&vI1FQM8j#)2MF}PI-o%NjPx9XvmE~ z8=tU~0Cw@*Kd;gJqk(Lz+UYNelqohpkI1~W_r3)u;e!K#THC9-dhTpdJ(tmjMJyt$`IBS`crYYPNwH9h;5RZc>fP+NT(yNb|uHCcrXi~G_= z5=|mceCgSW=7GP;`1ap~I?LI|GCwu%-OZt$!=FE%hD^QiP(+0MH22>eg)b?lXp;g+pG7Fl&%CPX&TV z3BhzmRwij5+m}L$rf%Q7E_Y@Nr-twA%Z6a#2kx(K+H@RY-Oj7frR`+y(#B~9__uOSAth2wJM}pRSIUF2J+phhH{(G3``hplcnSG#cc{#d1v8Q&$TuK>T>9=jr?I9H zYouZW15{?59Hfa9^fJd-eA!$yn6qD0P)}yjutsV?!NT6r!A$5d>}>(1Vavu;l*z!6 zP6?X9!YGK+V;BR0#TGg790)Ao=4TJeP2Q=zA{cN)61ni#E&$JF*QEz4~4BYR+JvUJaW|my*g#^`cP=TtI zUk|>OIFhxLw|gkTFrP5i&@KzHmo?cECpc~8Dq=Q`|J%UpBos9+>iS~9;@!6`yTs&V zq!K={<6eUVbjQ5~9guNIRA#|k*~J;7NM}u}KF8mMh%fbr)Ao)S#{k%{GSDH^g>}z- z>-SCyiAz3SyNxdw~zxMb1X9=jUC* zIv%y;LIvfkY+y(KfH@4SNNm-d^ma}xU}19H#hQpyn*4fVB8vlIBt5n6jE36XRy9C? zsqnY$PH(mT)sF$gI!Tx~OZmZ4j&zzdO$ROSzr9WyRLQci1fmF#j(xc z(q{AuwS!Ar-#@kLfgZ6dQnEVpdq!8J!`Sko**AyVd+fUAKZu4|;-QONF=}1Y1#ghh z1iTuZ*m+(}bfLx)Js z(@hx%66=#=Lj3IoW#D48&@PRYpvdGnF_wZ|O+B&pe4UB%Dq!W6#kqgPm+H7GH?!&} zmUMQ^#>)Zpas(W~Ft#KobLubkdoX-03?zBebbb{zw%@E-kzVq+4_1RNJJ8Pm`^&ON zZ>X6~c90o4x-r{r{l*K=5y(65B_U@vb!5wIdrFVtS^WMb`5v2fpg&|$_JrKFZxv0L zJyB!;lTHjPgUCC{%xnG2VlVRP%NCy6u^puphZ9mOt2|eQtkX1dyKn$WLM!v3&|I4{ zxOqE+!omSP8)i<8RhNETQh8addfLZj+$^rCeGz0}G6R2xir@}xFp zv(SQ$32=za!>)aeC8)kVZQ95FEx^7TBwM)Ub`*g)(!q~Ai?Z?a%!SAn&9=_gvDp*A z86{wdmg{PncA&~Q-5f{9&DK7JL(M|PNyr)7iY>LQ-m`5aBJm~h3sA+jT2!y)WaqUF z>m6Z=YI<)OAlQ^(hAjq%)W&WRI6AbDhd36ObrrePYtEgPmr721{2;)K4%Fxwb|x$} zDa9Yd%1G?%=^S_a*RpvcWPx%YVl9}b2R)TwfMGL7A{O32EgLg=2AIS5it43+vJEvW zc)F#u3_qGXQTdH6qHW)rl>!ocQ#FhZKN=ol8JNwO5B?i01~U?*d^;zq#FGooJ}W%BCm3Ned9&4#B_W;N_bBDcPXn zrlOr&8l;>n+P>XB2-hFhRu5mqkb`UAn{X~zC$O>CN<%h_oA)n31d*&1z6Ubowv!KKYpzN56PFc;Kj(k!Ao+|$H&<(FW|5+9O1QB=`uN3!CO5^{=g~dV$@sZ^q`8B(e%nWiR znHu0qf>7Y=N~E0!t6y@vhy!3WaHvo&5sLWkZvK5qz!W5xxB))h%eZ(oRCXbTP*xmbfE&mZ(QUQqT*;kJWf$bf1lSLtT&l$w z@akzuzJsF6Gq?FOGH@^3@)(BD4TltaS?dK41D+NAq>?p>f|X|8-e|ggGXDU% z7^Gm{nKA5O$H}XB@jjS);wmENW4sMm%jo;c9Z}Jjg21qcydttrWI%Mt3xT5AB%Dv8 zWv~exX12WGR4_qfz^n{O37J`yfcS1X8OksZRQ0}=jg4Jxcgj;_HAZcyPCPbc07 zR=2BY#4z^6bpb?6IS8k2U1|f6P(tsl2SBeCN>hT11TgY#L7Ec&Cfo?{E&N^DKBr&e}Y+ z`{29tQr@`u;H)JjSyM_f3tuRXj4X+qo*z5@;H-XQvmmgG4o1JW>io)zNYWX#Nhg&> z-|su5AUdQml3#gIB3z%y+;wPgF9XvsD|w0V%|(pKD0 z@@}WqRyogo*>`4 zmsA`8QfbWkNmZoKJ4!&2XnDaMET(l#|ulXUo>s#$q=zU~IdaxIYQBY`hvHB7Tn|bc~7=MM#xoD9`$2rreY* zr)B)%07kI?r=fKdW%5vS%Yl^J7OK#$tQ0oy3_~C}RR|=*5J={*Kyu?~Y9>cKg%jmj z??K9utW##&+`JqU5MN~_)pqUjE@MM}yS+$|ZcBI+_{ly~t&1~&(v&_@l5Um}N%onU z*pu*I@yYG^-&jo>eQZd!6+Va?$$6iZs-&&(5x%!uQt?wrtwsT+yNhkXj)C4#6RVHeZ#WwBjwXuZfSbeD@f!Sot-I*P`bGfK3gEaJgcFu`dT> zIy_*KjoG0f>v6;F%~`p=iW= zS2HKNt)SR0^8r7lSa=>wsr(m|m7Aon5TEid@QA^xTdw>Ih3vT>(YGgxO`4BPz51B8 z73@R8qyUPogaoYQ^5@AD)euzxm}wYYLc6#vn+`Y-x*X~+u3mB;m#6B$jmV@9*8MnQ z1|de{+o06bI+VA5EjxEgwOJS0#dB*i*XocOspGQmP{1*nhuQI9kf*Qm)1VgmMtP4exybPy{N10vqm~ZvM8PFY4#O)Ta>N@}=NKv}n^XO(` zh@}XV?>X~h+Dg;@$XgH${20M!_lW32SvGFC&Ww*?)eE3vNC4@mxq)Q>0uv_Aetu8L z7jTHJ@p<0ppJ`1cVef~P(zc_?pDmYEf1lnEo??)FMaGG5Ey{{f@>ml>D}|lDfk8`} z(S0H{Ji&iMkQdsWle5cYqG>(3P(6mbFFB(-P#YOuS{`>^N_ux{R87_;?br72A`36W ztrlnkQ9<{pZ$46O>a?D~+I4UUCC2SXjlqTdkTSqP%Z{fycV>y1ZQk2nfskhQ_pb-` zyK5=^wv#$SdM^=>Cu#wR;n(31+HMn^P9P!0T3k}jYX(k{Pp(@gBJT-+pPBj83k1w8 zm-1Y9ED#x%Wsi3qv4zmscax(dUOnw_POjFr1;SQ-Y#-)1(lgVwfH*!TNGaL={hlzC zk^}akA6xU4DU*IeGi~^f^b2aHJs_Z&_5&Wxw6}|9+Evnjd#u%x_fSZ$DanI zZ2RT1){S`i%z%_^y!>`}YW9LVTJH~~)z819^`1~#_WXNW|28}&yKGtOyZGBZcf_e1 z+fsC6yP!BdTlUiKv+P$>n#g{n>i{rqsA>zNs;x=bk95}$j1}!Dj%Oz&cQ3)wPm1IH z@KTxmKQrpZaQh)&`wAuRI%oFn-Sv$hLu&?ZtpT9RlUsuWN&8Lbb=`C)iT=FAk}v!) zqE?p(?%|HW2T5 z;T}$hdi}76gp<-tvSyBKr=8JzptP0DZ9XFvlG6A@oL!gV1>ul13*^_TkUKHB{+HaU z037mN^^-UX7@$(L+q5ak={UrNSQ8_Dib*QOs-IloY|geZp*h>h*u(=}|E1NRq<&oh z4bc;4%Nrirx^G@2T4CdeO!mo@zUx0ZjRxKNV1w^=jaV`zg7=|N_f204+yf+#?PIti&P zq4V-wKN=iY5t`~mYIlw}#}Hgw5Ceit&Nundm&ln3%O6uTkC}{(+~^@W=>9nj~Gz*WqZ+Hh2FuXWKF> zZEVu&nxvJb(T$+dW)3NsnO`|Rl5|ID^p>n41-A@oym?S)!H|g+J116rQ5KITCC}}Z zd^2dY!4oS!DT`NvR(o^Nx|i!(KMAE>*B}3-)}IYK#^vi?jx_#DRFHe9uB`}lZPc4> zDSERF6LoD(a3R?&>>`UkEsIY-J-u*lajfa%;&>Lm^EvB3m_1A2hk_f*8*gC7>#U8T zb47!-o68}l?BBWuSfb;!qtlsH^CZ}AldP4G4GFPUF?19oMPP29*K=9uXj4JX-E|u} z3ix-=*7azdzfsCV%OB*N??kbgMo|_imM)w_hHrz%2$O(VOI;VBL#Sz6z%Wke9(kE| zzNbhUze46AWlh`D^LCL(h`4G+-*1UUa-k&$#YU%eFFsy03|5GZZ-PGLRzQAAJf{hu z?Sldmr%a&lDD)ym2_#ZsP>2@aI9WXrY$w9yvSdPeU(DYPbJ>*BUT zZTR00evuXdUr#k|BIsjrL+2`f9jpJBr+j&5A98O&*c17%NlZmvNaQqUd#s4R> zllx@X^=X@v+^4>7VC=Xf5i0w4(8jigycq?3-ZYS`fwC<45<>zGrG04%{A3{!OQb~f zdybUV$U0YGUVd2xiS1qQEQY-vHB{aS$j3R_hm;b;fVha^3OF=p9_`Yq{=tS1YDLUY z&I}s`({2GIb5u$I33RZJ063BOvFI7QmB}wJ|66z`O+qm*E!trTiNFFJ2c#RAlc|si zcfOscUAXCvuF{T4lhbLV^#N!r3(+XfpRah~HC#%ifz`07_lVre$cF?fuOKj zTZqA0F^Ie@=0HzPfs=_3KTrR5{l~=fS=1?u;?02>DK_~bvEmL%IqUPW2n)_h17GV# z(I>OtJBVgqBNE~7hau8ld=_LCD8S)guLD@eC||an`>7M#&o@dFN!EL9PVWKc7$wXS z5-o2ssKqO@kR*nhdt>_+3QEPMf55A$JADIXuM579tBBbOtWq;S(-w??uakz*vPvou z!KMgW$~KYZsO!5hL{MAaDoI%^B^pI~jVpXe_IRP+GLZu#g@vMgtr@8oQ&M76;d8JL z$;EE;Yg6-8^t}o&+ep>WGb}c4qP^bpa>$S~&XHX+FX3RS5T9^7qLA3Ue*DHVexAFvbeiqB$ zk1;(qE|nnE=s9)gedM&q=P3r-kT~LrJNfOFSyf!6)_=cMsr})t@P1^}KK&OkY8QRh z`^A*Z&I8k{$O9o@ltfp?mEVD9%%{<6mZM~p*{M; z0WIW>PY{h%f1N9PrFO3x$POgy@$z|oxpm~BD>{I@7;}9is@!t=tjgi{T_?s404%GX z`&cJ2ZKsk4$&!lX2P+O@yN$WxecFQ+2cPoD4zl}boLgQ{!$9SZS3-9Q9p5~9k1TG@ z4?tjaSs6IFa8mQDRn$1nR!^pJ7D_1yL1oO}t-HEe%$tL5J9JtnF;(OZ<@_Z59x(=S ziwR|L=gk3jn zLPzU8QC9S|91v0Xo#^tkocb~8b=&kWK3!wb5`~wF0+8A8&vvXYjMn-nAp5uH--w;J zCXj?d@83D=q1p}QF z+8c=!8aC)_{5dp-0?fSZB7?nv>NQ}cC0qp(lU|@Fz8xLDQO2FFe67@-+%45zK4K^O zBzGic!w@JjvrN03JGo9^zl~fh@8B|Y(%Uz|0%5*Krv&KwsjIUw;!pu@CwWDhF&bHR z=^`}*1CV$HY->W|rDIP6c#!?15L1ZKsKm{wT#!vZgNyk~Id4CF#x+#w{57UsIKJbO z$^3FLUjCY2a@2W|4+IurDuj|bI_xlnVleCkeP#}qt(yidCGFsz$zQ?`DkyhCQ<8vL z&n*9DpP}v6jkh(RLswAgp_Azrw3@t=mSTcw%zHsCqBuF_n^A$p8e`8?C<9Z z)kq0e`n*VH93+Bo8SY6NNc9EN{W z#(_c#x3BvGO9`hF?c0~NIHIw1cYVK4O;0u>H}()}0KwMu{z{T}`DD|sr?snKM7M7< z%+J6-zsvifgBIp)6e3%}Z7!|O9eVT}Ka@9bjz&#Veu{&2Wi$1AWuTTAl%V2{c4qr} zoVb>#YKz28QhhQZxUTP~!svg>wu_dn%Wqhhx2Sb_a@v;z(q12sdg+DB*7a*xci3;z z3!f{F9FrgGad6hw(OFAIXRiHyaU`Q8a%g^ROIp^H;!BcxltphrSz9>2^885B_L`&* zOQVDP4r#n?NWpEn*R>yeNI*-r2ZLI&on>yxHm33qB}B&bi6bm zwdy}pqHFN-IlR0!C0dg?q;ljr(T&@4E02jpSKxmGBhi}e`IV>jiuT0cPB|x9x;?j$ zIjcwe_y~PISs!#;DR_k>)2W{vW#gz&>>Y9= z@GMee{{}kco-<6iN`j(`HK|fHCF2(Ua{P?_xGQ`db2fvfZ|%H2tf_1;w~uA6+&eBBHGg>&udu zu*<0ZQTvL%<&{O}RBvQM$;vUkN-7h4Bf)I7)u}*u+l6XypJL`No75uqB)HeY zOSBcx194QwtT1Q2e2Z1^WeM-7q_DNH>-|ktDLX0+9Le6Z1!GlCQ=1g3DDU?cRkf=Yu1_>icV_BwqPm7!%)XYIqRm8G9gP3in8Rk2p9Vy zgvapZHtb2(AlWbSPuYw-#;)p#yCU!oql}KT-=i#d(Q9d!6A1FHW)nIu1!&#XK@du6 zI#g%KE!`6t6rgq3QfCHT{#~En0HI`{VWbC+V<1gH3)uBXKx9uG*OYLNa{tIrgFZh9 zBCCGNLHru$!aaI{f%b)$ye~cmWJbGWGtaD_1zbdod?JU0^??VUXb3#7`A^UQ8m5aF z*8kURM&2#yvhK!|aVqEKh4>+khIVnyneGQ&?MSp1WYJTmw4)woo$2Bn52(vK7rcR4 z3}*Z=p=1zZFpmcr3jrQ{oU)pqRemsyi>@Hh62n)$o`&!Y?*qw(WtVvo?`8KG9abmU zJLOSPGpB+faun$nL&8QJf;~u3WtICPna}ZxZ7njYM%l3Xb54T%WAU6yO4n;Q8^*+1 zgskh`Y6adAK}h-1?a)EC@V`z>wps7FvAsrk;-<Ofwey#WLaov9{k~=g4u@_78g;WfqIBCNXIGGb>Sx<>TivVB(Rb@(WPgi z{Ml9m_6sJR{Rr)y?r9BZs@(LPf43e3O#8hpn>iC{-$fmX-XCt*JiVzq`OJ*!PoGIn zE6+93ZpYprWNIyvq2qtjMBni?CJmbG<6){+sKYnK&AmXD1*Ixw~?HERl* zXV=5+adm##DoE<;=4i&2fYH#5?e9Q3K30&;%T>=eBYYy# z_W2g3#Vqlorp2HXA^NEKt5MRnQUEELgJEFB%S`oK*T0ND1&S<_3P8tul4vVTW?jF| z`ebT_4X!V!K#hi$wkao4l84&bZ$~gL6`SESWV3-si=ZNXHo`cusb^kW?7x+~L`5BhHJ8vQ)hk5!B6IKj3xe2tLOzeWw+6}wX|sPkjl z$Y&+;dl4807>B4)vXN)D9tzaE(S-1U5eVI-CeS-5GiSR2FhT+!izk{QNB7(mkHZ}N znx9fiWy8lXyWnV zF|%Qi*jlh=2z{6jGi+rg3^O)&{`-gF*9bGt#Eax_CYJ-N=I9bE5w-XqHs#$20LD~BVTLL)4x?8qV<6K zNK#RSmjF@}U5H)D0tR}QqC@aa!J1)Z(Da5V!3f@?0VDvJ3K5)~lGN%!ni%W@X`XNx z0*Q}>V~B6zYELixtZ&%}K}$KSyX_{Ntccb^550CeayN!{s|gFDO>g31!YSpN{u)RD zjt$$XTxXkhcYfEjZb&qJ|1oHf??0#hwtoeKmi2P`PovOQZvMyOijTQs>+1QbH}_(3 z;^#{$z7S51^HaM)?Y3^*Oxj2IKyrPQN158>r5_E8#aaDWE~JkgC&qr@_h07pIfN?G zuK9LS6?x@X^bpd!g{fBy`C|&de_~PQkND>Rl3DhW2qLE^|FL7k9zm@;tfjlhRWMSL<$hl%atcr8FH$ex0pGK3>UER4)`qQdIvcHte0TD7M&E{$y-6CoyB=b3AlUstUI? zIA+4~^+dYJ@4^WO>gGrOi;1`VS3^U0U6t9Ga#70lQ!*QWa8W9iceEAVh%6lkD)%bS zhFdvtS4TUUu6P#kHEPiXm=dKM=uz|0v1zTt#<>$?xC#te=PmsyvTXw6t1dZYjO!~rKNkXoi zv#wmiN;0+U$>>lXHx_or&fj$EP<&ns>?2$EFC>?SNEzC-y*`?nAN5yFw(%1S3uaee zu70x$FmyHBHv@L;yzlb)tVP($g2CuGL?2rYCtllOYyY!2!_MJC^ZK7^>wsv7iHLtrX~OGVZi|55*@r7_dNw+O824z8ogA*Ns)A(rlB_l-ggJywnw4#kk7 zMny%?!JkHPBE~Mz;-1S;!=6g8Pu+5reGm=}SQ%x;+Yr2hckn~Z2$@HHWchn~*0iC> zsQ$Efnn|1KNxP8@a>SqNV%qODT{o5=U&nZ1jX>O&BYIHLlk%4dt?V1<4%TE?h$y*w z-F9@9^bEKiyMnJ5z$;3mtA^S5K;{$q{t$% zqh>J=nN117INsVxorr`#xu6^DaTI&zSc%yUlVKUs1u-bNzp2g?gC7c+X7M^c{L!=d{V+j$Xgdvf(r0yCvAaf)SP_n@eUA#XXUJzZhRT(Vd39yyt~TkcuXL3c5H<>97f z^=;%uQhWT&{u_tSjm5&97=wj7?+h~wH(s)E((K>r$Offk>pzl%g=b8Nka^^_;OKst zQ}fxKsad@{)|{2lHLF+0n$z*Wo&#!LzB4&%VNT5&{H;gF#x29b!RnJh+?_bGS<-Op zPS`Q3m}t1uW~bjzE!jRmOSbufhC77L+D~<;nT5{UnPbDtg?D83UqTIM!ZqcodFy9M zs>s8OWPaz)5eoW?pY8Bdk{>Py0KB0W6Hgs4&S<%DEc+nN>T&r&Sv(g0bTk2??%DmP zWHS|X?1u1(#15S@_A4qL^Ye=wx~-5v;3v(od1{a#gy~TiW6cuuY$;?#Mci(*#_b8W-43sB;K_^51j# z>m9($lSks^+d`7Ys{;W!UL7z%Hk~GkGH*3gek9qU;X&+BLSyy$*;bjGn~VE;MfFQ# zQxGWR7M-d_jgC2C)Ke$hKazft+Vg*e@*;_Em*;A!$=r9r@0Ix-;JtvLs~dLrWFbHagoL}>{udP5H9oHxnpJT;hAk`vj79zB!dRz zEtz5SOjZ_+qkVW zZ#gCKi9|CtF0)c9Nt-rA2UlXF7tbnE?!s3rI*L+>U3#!BiOPFasTteyo4XMc%c*xn{P&U= z)RrTU&aS^1BlS+Uol9ej&dAoYbpWLx9O0o5{Bf}7XZlY;HWv(`5|hXgHv_El$>575 zwdEO@p{D{PLp@{D=Q%HmiwL&bjwd8QC-BDIr0_+=*EjqkoPs4D!gst?x4P27znHb6 z^_AeFbwua50~r>{rou3J5p|4i)VB;Qzp%9F;pd;rj>g&~6;WA~(!WsfQ-R&lSs zMZjmmtJ4QZpt|zJt;fbb?=3>5Shi%xFNDg&N#Ur~BbF5>0cNJVUCAAeMl3)EKrOmK z8Ww-P2UEtpYzG4BH)A)?pq>{zOt--F^IfBGK|=z^WzJ>xZ`fK618lIi&#;bmF?*J7%viPMF2(?XxNbT z!zoH}(*~r$s@j0$Deb<3>Q~aSI4e47#(^~yMcE?Zr9Vj>i-*~$;O391da!lZq1|Nf z-&k~8D6qIZ@K9d3487ON24~?`FqkxW!Z)|~KXdNgZ3AB( zkdZlSVAXvYMdL$(C(8nh$A*`{DY7V|Xly8OS6Se;={b?|K-?cX$343sI0Vj-dnXjY zi}C(I+~ftp3fM;O9*@ehgzGOCHj$MLf0fRW@ozqv&~@j~t}SoE!+-pW1}T*+K}R;L zZ8pl;THj9&CIPSA_f)W=Ze--gfw+A9?~*`V62y}~?-@*jYVub6?Uj+uKQxjj;~Z%vKZ8~>w_|uf%E0EC3xe6gkE9p||J%?nyHhB< z+%k+jYuo1)zFb`P_A`?r2MSkyUD-Jy``r^6ovj|0(K3b6S#u4}>ZWYpi)SQrTkkJS z_(afj$&VMLFha9vw#ErC-e7+YIpJhKG)c}<_39{S#g^~gSd#7s&>cb6;LoTTg(;J!E+uETw<2n#m){GKq zYV8FBbvA#Hcf0`nq;z8D^y4XBdM3x!362;+Z&COhfl2&u8VBueoW^z3 zHk^jtQHV%Aahf^XZO3UUTn^<~(Ul4X=;pPjSWp;o8SWl$Y|T%u<+wd}Fg_qcek4*Y zNp0Ee{jz<;sO2I@E?6kf@y!4w6;q5lWek|P;T)CO!$Ntx=-LbcRLWb$t|IwmWoRvy zI678PGZ(;0>cKqo3ruA>1KZrTs8|FKnmFY(yj2g9zRDfm>lhY46X^G-NRZ1MHb4JP zxZt>q7-PpZaEiW|y8G z+({&Wdhn8LKr*x{a{=ZMqd0v&)q?R<0!dbAjQ`d8Oa`a1#OW|Njly`cuZ7BJpaQc5 zk9}0?&9=%=o-ZxtS}Ss57o>cb1~oAHiCuG5ELdgx!X#%jKP{8D|2k$H3VO?S{>_GM zN+M*k3PHw8PW|ZZN~{5}^_QI<)o9B3s*dGl#4XiAh!njGHV(r6j9DRpt6|F6z^_lv z(drWp-o2A?FY9Ufc}6eV>Ns{b3{A7sxM}vg$B#!lylhln!I23CU*<)Y!s|XMu75>v z$e9xgB6*PoXtdoq?)|rBXLi5XubRz^M;rWa;n8Yas~$?P8FB(398CS!!PLJ?ulQxT zD;oBohHYraa8vTY=4n7{!ZNbh>cMu(^d{7gaTNc?IW=#wxN&f=F5`Rgsh#IE4j49c zb0Grz$a%fk62Ik&veGo+IFh}yPvd}#*;wPlckg>PprMwUR?$gO`3Rsbo{qdGlg8 zj#O7a|0X0NxRwH%^LDfEN_M6^# zU04&*Sh>4xM`n@v$ff~ri4lmb$zf@y5w*d6q&gu0r3}FgCx@qwpQD0$k8dQ1HDInS zz-{w1Fe?7Z6L$qg%(aNJkuKNr*LW_ z>WX7H{G-(3-FFlFZ%i9dyPLI>S$#VGbY_(O2njV~*^gLUl~MGFX=F*szKdD}{b!kn zg67wmO4G-@-C7TnGnFj2I1+k_x^QEliRcMfOjk`u?d|EGh+r5U%B$BscQCdZ^602E zqp(tj$+h)8NyqtFMGGteuYv?}{b=5&(t7u89Fi1bZptchLkhfP>Zu>En?#VtYn(}6 zdh6M8&5$vb9v-m=g3+!&t@10YsjG*lC;!C%fX*wHtn4pOyJTfH8mO`GMXEY!erB!E zc3wR>we^QlJDxGH?%iTiV?LjE5%AJW9pZ{G?1RTmO}v(e2yTUJOzB7rJCB# zo9(+f_2rz38jQp))r4awu~HTm0rrrEhZ>cLM7FUqOw8wrtn>6m85lH75&%uVy) z6ll1qG2*p6(WhEynCur(;TkSdZw25wS`*P9ACo{D?eORy7?aub`@NFk*sqdBm~0EE zH5zk(JmRUuB%rXl~T3;i|j>Ms0G~j)l!dv!;j8SS-R zj|F{n_{!h$vu>y`mUjGjGy&pbEeBkG$sQl>c`0Bpx>Qg~%%0?s1zmb<@fLnQqH`A3 zzakKq9}~ohgjxDKUcX!EHa+cx8;oA-aLNz?epWOgDky}eMy?78_84{NB$B1 z`@0qfOQd7u4WmmlJ7S#dYhQfd1aMi#RqvgY%j4gE8y zBL3%It+UVX`7yishZsAY^=j}uQO#D$$CF+SUi(~Q)^frUJSfR(o9tsb8L zq$K;q-}E0|VWRMwoJlytI9uSc%JR=oks>lUm%Tje5`fbu^k)_oaacfoN zT88D;%68eU5m=yO<(Yw!{Kte|UVg#)96!(bn6zHZHRxYZ>pG2yYR{ zVvX?bZpn)R^@_J81ES}y0LMa*rXO-OzyWv)!HgBpA?Jae!K{A2ML&Bl3QTIrB#RyR zUeRU4Pb{GHyVvKmp5EzZvE7k4+A*KSgxmhZq^~P14W0+ZxgSPe@y>rdpORiSc(Urc zc6Dg1@RBkMoDX-hYLPedb|iVUU_(u=ETsZET`F4phd7)@JG-` zk{%NOQ`Z^)mF^Zz*;+q=&awZ5$Z2KiML!IM$uM&H>EVVKfYx>bt+m_K|Nbds zQHnM@qxtOg__SwJ;8HoTx!a1L+)~YL*#gO>No&xa?dytQwrI;%ysTeM#a-z|Nuh8W z`ev{18Q**KiiQD}W8}W|N6+Yf5exMmzajiImzhl&)_>_RKK0uhYV&98>u~>3u2TDS z_ANw5d59p}=Sb3;IzHXET9b8DcDWVZvE40OUHY-#AsMjvCVD5J_g zD_(#SlaQLF{A>YIW2o$b)JTNskJK0^n@Z;hDl=01^vqz&r`|vfdBB*4{>8;)=sztj zgr|CI*<9%{l9F~wDsCA5%NfkA*_Om?&GxZBN=rlnyRJ)K(QqqB=qPbm?(J4*Hu4hj zH;hDqSu8{UATAQksC^V|=ufs`u^2Urrt)HXtgZg}`K@04Rbv%RVDW5+xisjaSze#O zi8=UjS`e5-_vRQEZ(&O?W2QEjT*9I*qoP(v=uitJy~Ow#*85BDLQvjBL3!&wptE~; zZ)n~!I-}*|K1g5V^bvL(VHK$of*CdyoI=QD)PZT)q#a00`B$_sO$s;NkVS=Vh%miK z5%KLw2#MIfd?^KYKDBDl1$tFa$WcW51r_j?M+LOV)Dm8nHkY;#OhyJOW4y|0U+&25 z+M;6FLe+pHLQ||1t(DB?Uuc}-Y+m)COkW#jIQ-RRF5AvEh-gcqEh*Y`)e-9X4IIerC_KraC>zb_ znP%1hWs5wxW2i)49OcWDZReV1ADRb4i&n{LO#cSvHzSQrA?$D04z$x%Q%R z?Srz6SupDqy=g@1tFdK}50osD`pv6C zIpo6`c|9kibNoJzdz(hy+&XOJ zj6x+o?P`VAyECwJ~hg4(-ielHr$?%yq=4VxOa@ z+^X(=etwHPjK$m9XEeeRFn3+I6*$-5_yYB_7<=_dD!%ZmVuQzbQ`ONq4iaq0`xd-qKd0F}HIR})7bR2nijkCZhQwG{kzUyL4dAgSgp1;cgXQW2IM9pFGe-%S97xhOO=GR^V|K!~^qBXC|1 zaKmk(Il~(^*p<0|@5yVR++W3?wT7ICX%)+^AUry?n`rw9UC zMZ6mLsTqz)dyR2P!55X0FBNQgNESqA3cxosf?pHE$Hj4(=#deFHH}m*&Xj`eOsJwieJTAAX#0lfyq zYG5!m4EFTu$GlBp&Y1eP-DedFmhAE&}(LTS!{>mV}ucj$HZm@WIZb&>yDQKGXLz2 z$vXio7LxR=?Et04&D&{lKHxw+5Nlc;MCi9 z{4$APijUv_mYGCYMcMtcM)FA(Od=8XoaK%$H!_KpbPd{5`$@Cu##%({jK}9=Gc1CG zh%_pR=FT`nLoL}zjp-)n^v7U9;E&-2L83ZTQqU`Qm}KG%RG4!4U=^p#{mG&+7=H-5 zi+`dpV6zJqOY1ve0hHgBs5LSY@e)B&zdV0xZHfKbvYdiXDQfLwP-}m`mU@2EIAoPs z@y|o@6t-3Wer!M>N4I2!{Uned6%C*ca#_E`#WQLrCCCrk_!uA0lEjxEUW0(cx-4&@2e=4+n8`s@|)TSl9yrhk}Vv(o~TRS0;gM?-)c(Y=M~+KnSn%? z?nh-v2I3UHvM87i?X^e?mEDqJJ8bmK4FjEk8#b=Itl9JwEpV0@TC@b?zvUsOOyhPq zO3Iznw$C`yt?WZ$PD)maE?28y2BX4cow1~kw!&2DgzpYatWW{^aZ#6Hs8(Zk%Bogt zKLGX^HxVw#eH@Xmxv~Pknl@P1;OmQ`p@rAoiwc`o;72cb`o(MPVrqJ3ZmW!AxHuy)(M3FgqU5! zVka(3VuWIuEb6rQ23q4+EgIUi?ImF|D^v5&W$mpVK9EfJ^rcno!8f6x+f5Knxo4Dv zJA4tB>2@&*!790_oN;&uD?Qo#l7~MW*|}$*)G;+@U9d3Yj<$z>uyOt1o8B8d_7`jK zyQKK)-%KTsC+P;EXiGZvs#|a-MqXnL`nv0X!{0!m7OgURngKAjrx1hGBz^lhVf)A*FE$A~+gIa)^JKnZtV9k-!c(4e~%@9p6MWSxfPu|OV@A5pa^$?TGdZNuE zM#@{!_naRF2f7ds2X)PZxn;HS@A~}~x2&eKp5c6#iC(kHo}=KE=zxh)5k4c{NQv?MyS?V#Lxy>&5(#pU<3DY`&UR#G{ zzeu~^s;pJy%Sz*J!$P1>S$Qly#iB8=U45<6Z2AJ2cIe!OK!D>jCg)Y;qAx0XBYNHe z;N$vPtzS(o=sI%ge{1KqV6Xxnk)KT{*q0Xx2jb>+j=N_;a2sTiALm7ipox6zmz!Uy zZ1`AuL?$e|OxQ+-sy0534%VY<>QTa015xBh$-!-BkeD*EXI^^IS)nlX)voUmZde&i zyuM;_!@;5P*NLVzS@g> z#xEO%@AtEzFT-Yp3(&ktH*~UQ3^&wABROwUi)6A2zUIrlQjzd}64Om>OAAf_pEGL9 z#8yIRr!yR@3Xjj6!`NlCsAroyzH06@4=g*7O*rqnN+jE~vA&#F3T81aHt~UtT4<@5 zGzMSR`&aIm-eS3S%YQ8YuHKJjp}y_tJ>&z-Ypygzff+t~mI%ucG8f^6!NctXJ2l=? zeJ}lNAIC^mz-U+hbH0nKKevgq_DZK4id8RoxNLQDZs`U;&&Z^haPHu=-Xph`JwOu4 zl(bW&J~H?I?&E}eWZ;}li{HNU?d83H%rV=qmG|VOe7s2>V~-xt1)!YK@&!Obyg--H zsobJr*s4%0l|9^O>ECygM04lI^d2F=GW@w5KPZO^ru_V~mO2}2e;nOd8@q5!OMhAN z?8CHiF5J?F84_GJX968qsr`yh!4*Up_(reU>e!08b(o+c2VTMpx zOyL)1fCt#pLfy$Ate~6a#mUgX9MrL@(+Ctb*9Pdin@XMyDoN6P*_*H_h->(1IEA3V zLn@|rCp>Or4_U_k9&oz+>lJJX=4S2Gjlc>~;ahGc+(w`{OaCcF&HhDgW}gtvwQ(xi zy9^#iWOKPH%Mjkl*bcxA#dF^>v5sbu5@aC*OgU)_Dd<>9&17VNWwP2mdLmB_RVT@~ z>}D}gdZ|IU1VOP%87wQiqmde{^4qT8+qiDA8<*>; zJDFNOK6sEiy%|<3z89P%YrPP7)3b||NTh9QB;|@vK7#ZnUz}g^w3vma&1nSe_J6a1YCUF-_1Jw8+2=yf zW8gONym+P>?2op1l&wA6Qq);eQCS{oTplb!%q`~?YczPbe9RPgPD3R$lt5E0>P|_p zWi28XOvCTtvNoorEnTmD{#J5z~T;I_lk z$;2DC!K~gFxAl1I5LIri?bGFrKKxD6&#mdpP~vtZbBGM9T?yr841<3`iYauZ(HN^e ziF>P7O#(c*x2Z9?H=?pEkSA6qG@U2tCbg2V{MaBGzS(&$PHD>mH<&k88+%{#u7d)% zmfp#Do~7f;x&St3jKPvnUh{vXbBSIPUe2YLdl5bxfuCv}Fp*64<>xX`mN_{(#mBrb zEE{z~Wp<&jII?ACSNz(%(&?)4+7GOCxsFare}NJE`@wl+esk5r905;0+L9OGWO+*Y zCr_7Nuf4Q9@EQ~;%imnAsJSPsAZ81l>YMd6vViB)XAxnx%p%M>NW!e|OBc#2w@V;5 zDQUV%6lZz(MQ#_hRlW`+4^ySrwz^6uYHN%}ZE*K38>_s$P5c|yPHeCuHR)J&m52r# zA7{Sf{np=DeXN)_b40z-0S4!rvGFOera_Jf&OmQkyQgzM(YrW^HLtoWS&cznoT}v8 z98-`NC+}Gb4ayJDn2jiC#P9}#e^MLS4E~3-v)DofP0;}t>&H0KM=%WD3TsaEUBdVM zj*=3fwSSuf34~#DVP-!hMy3~U`9OI3DIBO{U%C-d#Z&gkK^9jbECSrUilc$(094N{ zM6#;*(19qdRpf(HZ+3fn+l#A+aYNuR)id&@n;X9JmsY~eM2Cs26)Z}s$=C#Nfuvh* zxPVhaKp73g98;B5m<9$~Q){@$w*S3~H|^$r@VI}- ztH*chSi>FTjOh$@8?f{=#elzPhi!(x3cQknk{1uT1KGzzmS4goGRc~`dL}9r1D%5A z(41mZtA@Bp)Phr4E)cOrTk6b~Km7HhgP|CuABP8l48dg1+rrOSN#K-%?KwsoN2szr z;qT!C!>f-C$$T(coPSagnzJaYvHlK3N!Chpua#CKjB0LHsfRv(!Zmp}55T z>4|ToCtQEkip}w>HlMVp;b#DDD}ER{F{9??#I8%TQ#U@J6FN9L^sjz3ixay}8PT=- zh~z&sb)6TyIVo;(MX&cAC=dzz-5HZnZ*gm7e9H~hb?vSGlmg0wnR3ko_f!Y_l~KEEuU6TYKQOy`(?7?e@!I%~xkO84 z@95K5H;kc8(mi?2e_24H$%3ZeeciPrVV>a9yfE6DiZ7NF1RnPjNUqs-Xf=pDKZWGJ z(Z|?h%+-BO!L2Kn@6T^iU@4mSk{zLo&=H!!6lD-3DJGA3aRwFm16YS(T5F6aV4prF zk%&v5nam%6ge7rlP(YT&#A$R9QZ74LE?yXMsyGQzy)^P9OaEtax0-KCPc1&3zew`~ zc-e8nYhM8@yZf@Ml=nF8|5HM`z6?E}cKg>zfTk<>CR}C}urIq(#BqHa zZ<-4za^!d6gadW+Bmc$u-uzcXLw8-3*_d)s%Jow+8-H+7>RJ*49*kuWspVmY6%VHb zQ*}qTE4Lnf#)uEFnm*t8Yo5Et%%TV^YM5+s+%#;!8>N##9oKfJhQqf!hsL((ZUmtsDZz-C3>>&Ue-n!xv&ghryp(j2CPOK zk-};_Mz(3kZ9AIK+nJY(rrHkeSURBav%UaM0j!yA`&8fndoA7}wO&VEpYaKIV?*4) z+AqI4Hm@QnE%*Z`hP~7iOJpF-&J_7*cM4cV7|=-)81~~Pl(}EjhMzOo3PHV8V3gYK z%YhcFz=Uyuo{$PA&?(sXQ}k^k-jD!uiZ5jYb;1xj5*8!R$6UcC8^;;Z7w`qJlSpaH zE>wnCAH`>~s8LpzxB_Iz2@zcT5|-%Zwt;%5JA-e@aEKtPUE1mDV;t{B|6YPoguzA-~CzaG_5i0LH+f zF7@!~e2Nm#o?>&3pn1)*_x)7rdA;Th)y5nRoN48|q4z?`=4Ng8Rnl!_prki12H4 z$bLVCKt*xY=-T{y2)qPeUftN$BB*@o*rpOnI7uLq^e%6tGhdd>$K(5SxqaolLoLKc z`8LZ2`xJvqgQ%0aaTod$T-2c2gV17z2%XY$>eM&#+Cj~-OW31CH^&wzP#+8O$_WA| zUiR>Eb=BaS=TOUg@Q@@J1(}1dBsLr-z7QzD0|W+O3MkotlZB; z%POQGLWNfj5FtJC{UZiSV!Jj)OT1PyhC2J|I6)Q}HnjN~Kva5@vY zxx!mrCLjMB-A0Aec(lbz^~- z92|S1SDu0ke)Wa#H;Zm@#)=F=&h)M&eek>7$V1)6RUp^uXZ-m7ON!z>P?{((~+5M5v}kYXdp@$_{?W`LjqW< zjJ9DMe<>I%kZb0uDTK+uc7A}EE{gAJq?mQUjElsSlEq#nkSVuASqTZShe^=PA#yBo zZHWIgHcH84%*`KI_28+BY%&6T1frnM>b;f;G38LQrln-_Nzv&hf2Vkb(F!eYXAvkM zv(8`e_NP#^l#-or4nl1;x1#3^wzmVX&orvQ6cnMUpLWwftp<+@hS}}`)SM*U{8Q6k zOlnF!j}Zbloq07sAPTzZ#E!_A-GYs~>CvAa8#azJ0*;?24S5kw{EKYIX20n#I;A)L z=>=-$iW_f()p~MbIBQ{N5@p>h3zNA*jqeROyOPd)4FjVp76?op`M?Q5jeDI?0#I`a z6@Z3#tR;#wKHxxcA|0j&U)&FUY{@R8*Ky1NUT;{+7=s}O6-N)8yJ8Zja(~5tfs_%C zwUCUcrBsg?PR(Vj65phbxCm%V#atp5O#Ll`I$p9UpNprqT@Ib*Tul_f_7IH>^`e>W zORgtPO`jaT5=2``3lVKS+d;G?ck1&kh_>G^%sy~tr-V0pC$8w_j(A2CascbFuNQad(&x7Q!EL?q2ux3s|*Oou`iC=cOGl?pnchO0Osc_9AjWx2LTwlF%lVZZz3m3eJSyFTJ)?(n z)dIKrNxt!78(7QXy zVLSs_hQ$a{Zu8}Usvbn=0_>M!H!>$C0r0>Qd2DH-z5*AC;X{H}-)Y9+&e{RdBt?Yg zhVnJiX{hqG_LonXK>`JGyh;1&NBUTen$BeAY$Sff|rA=O^wO)Eq*Xr8`!Gfo}Yg#N?nO~SDbRB^TkK- zN5X}~PT!cPC#bLbe{fSP*@Vh4!yWD16i9Fk#!nJ7{T%T z@T))6ibUITpuw_(sxt>}5JM)##A|{)R_hIYfVf(Qn3xblsz=%oy5rQ@g0r$I>!PtT z5FNgVC5AL&SrS9WfWi125mFoOv3!P^)?%AK<9diyQqWCa#Zp*bZy1plpW~Lz% z)-Bu@&dOXmnKlI>@U+K(5W;=`f;-Kvnt`3L38{Kx*J>du2=}*(1G@HJzPnNMoVAd& z*J6Y%z>JXgd$$FR zGm6HA0&~WO??v0Sdk0qC69|vRU+x~5d3Q$9$WUN(S>WQa;XUZV_KS?7EMTzdKc)t3 zzYq=BU_R1c^kmDKvTHnhr8{75052wl0Bp{`Yuv0=y;KEjtema>x0p+BtKdK(G=BD@E z=!eyci~Je3JKnBHIuzg6uW0Slr>V{N;RynEodJNy_wgD&J|K_rU4}nyA(b^&@mkhX zIF(WRZUUlM@Lr!Dgd=CZm6JVDiWyTrwR&cU0EPWqW`m^rG^QM~_`YjuN}G1)7X2UE z$4UqM6CLH?I#T{r{E)#*BMnI;?H{3RCC>$|}&ECU* zR{&w-4|}%HLQZQ}ZT8TBAp;!0_1c%&6<{Dh=ieVu zHUcUNC|4EoI7K!iN0&U$gidrQY#nnuw(7rkN6miAjy|ca`?>jAq^vBj`SsE1PXH-4 z|9!L=4uIGLB|b419$38P+NjV!8d@MIR)tmZxwNIa)FPwMQpfTla_W3kXok3XLpz*C zE^U5hEq6$7UcWY@<+{ntn+{pOD6;zMF)DDfx-RS3Dw-|=#8IeP-6x&nO~qwl9BGEt z>eir}x4_H>)WR2)vN0is;?)mPoXEgDi~GPuyjD`XU0_(}!qvFZDIxE~mDMb_7f$g9 zie~GwY}kDZAm_Dd!nC=C9a*56HQ+Lq51GbxXp_VMgLzZYvyQ2(naS8V!%~e!-x76< zi-`Ez{^9a!d?{iRkg+xltNQ(vC37l5)vQfnPkg#Hc(CbSk_5rJ8;)TW>P|ZjmN!X} z^@6F261(2IinLowS%z+FQ)9@a@s=i`09A9#@z`ToyoH`J3rytY(j)1_Cm5MpisZeS zUV1;tE;m8F=|`?%jX@EGY`kt*_BcUSd;n8H@-8M!=K#}c>@~ayb#Irizg3c~sWkP4 z_@=Q6(M$*H&hD`MpuCPLH6y3hf7m!>k43|Mk@-vcX;90@D(DayDjOaj{v1+##O8zZ zIW_QIR3kgBhutxo?Id$-91OLidXf#lho%zRVe?Fio~qc@@`N8Jy=9i4r9an!MdM(x z5a`ber?@8sR+(VooiuketN0~5ll%I(RD}^obv5_5sBM7Mj-Q{Q!rS*8L z`E|x!4uXy21z9PF*-sw&POu0@nV2{B{U@TV^@vdiIiI&2Y}UefgtLG~mWwU@OJ)h# zX7iQl$#rxC&P`6JK1D#b5qOzQFMA;a&A^O@eh|l3&SMnM3F8qC5}ZJdQ5ppab(7-L zcctl$h%}q1r?!XoelSjgVIxv|248B!$3xHUSNj-8nn>&~Zu4c`6rr~eb*JEp}*+Y4PH%AD+COZuPndwf)G?F|&# z+WmAz>d?Zg5*FaDh|G3+)cUK#V~)Fh12e!-p&?O$m^(0|zEM$unb;0x+yF{UsXPqv zJ5GUDizOAoMU>ve5#@p$711krH4!wcFhQaW>zH{pO1&F+H8#lqS0C{!5em%|dui0( zlFP@4i-e5RCn!VAE2W@B#B+e`pf?bw;rr5l?B<5_erX9x3=|0mdL@gy@s6EBnP-20 ze*kBTesL;g#ORQ}zuc6K3E|LE$T#00z0`}ZnDkN_g1kw3srov4lOdXs(HTAHKg(+! zMnE!Z>g0&N6!^*XrX3Ugm5{WoB>nA+52U2h9CKuXkvcm0T&K03G%Mga%D0sOf2Rxi zO?ECw-VM64FdTWH@j8f9c{t6pnP{}O*IuQyH1!<*XJ-7CWD!Do@EkK6_>CT5H>+gF zZvr^`@j=Q)$S{j=io3j<@daFWX?|kIl8>HSzALT?Q%gkKF{ZMJw(Va+wuh7?Cde%% z6m-JPs*o>=1CObQ zF^3MaL;*X;JXI^OQ3Xk6_%=cxD#&aqcXrsnL8xp15Ut$KG5nQ$7aR%+?4eLX(thr~h=R$u z2myFk`Vyz)a&T3m!y<_{6bvf_`ZpQ`zG|hX2s^-t2Xys)bJ>LANwTUzVf7+ zE+==LGAcEzDksz{H}r#yn$O_-II`=`oYbrrazb(Mosl^^qv)zo;LzCcRow<=77wg? zAfxCfp}^~9fh`?tk1VTQxOxA|&HE}F&WTIBy?5gH-U-+LWX0x&%7$KXiEs2uT>JFw zh7|mzSK|0y3D-|pvH9(lo8PKz_$rvVwr4_b$RS^O90g^m8=n&L$9^^Ir6MwU!G)mW z?i`4oLdjW6b84o-(EpDx^xw5GSn~7u@CGn$H-K?_p-+5oFm7KCP3T=Qr{U3H;<5#E z8XgHI_Fk~K;lrWvyctFa)=9vqE*>8m!h6fXim-egK1~14?Ky?4iiF5@ccsUxfm2`i3 zNzbc)p8iny+8@93boH^q@txl6op{IYUv)g?)k}8V(kuVZAD_|vB8E_ADy|yc+nbLL zIj?nGS!v0;CsLhMZS}r|k!`}3BzynD$Pf$T?VHqcMt9M$?Njd7);+ublx)8Eu^YlC z5=47p#B1w50rI$Al({9IThoatqE5mlvg^4ur{4PI%;LKV2!2P;|6}_!+L9SHse6RO z0KIy?qKSEW!f6ILNT<$!>LB2+9oR9jqQtNQdemhu3T=Mxn#qf8n#tt>$d8ct=opgY~2>imIQ=S{j!N17x%FYM|SLAfO*RYt(V5%z64!o(aS3dNrp z^4ydJ5}T|m;L_lY<7K&nCKH-;<>X;R5jz?d`=i=a^|#?Ivi`dj{~+UYkRFEbfMfQ(kri3Nmh;TV~AgPQdR&TOjlE%WWvkV-6qDia=^N>HS;9e1lp zK?5|Lng-0w;h3J=>3?uW8u*wPy>(O+ zWhfzD+qw2gDq03^FW7;Cfmp;TdW$gSG>@E{g|i4}4r49sK$iJY-Jj*()n!%lH_zb( zltbj**W7{;+y`*uoHInPD*0cIj;M*V}d-Y1Am-eUfl z9_G%xJ2?W{)uQ>E1`RygxnwJ3TggR#IBPe=jIi#B9clnXo1lhBD1J}6lBzqoRMMS_ z({tSNx>vW$x!7`3Z9J%4_-V9Q!GkMIOYA^aEvBoU0Hw_C(@#Sbm0~Tf>hD_XfKziP z^*)?!Ec?`p!s;xvZ*{I1xU(5l@vLy%YjavzFy>01sdW?GEBPumse)cKMSySxzKcpf zda%%!&wRU$#cg^ziy5wuAH`GX&f|$M!mD@HgRPtPtgD76$*7{+LV?BQfrs+KWv33T zDjS$tmQi$5C~!l0;M(U#Mz#jx9$Oepf}do^i3KN(i@X$wyCga8M+<}bXH6(L@$Q69 z=?S~8T(P;Ovf)Yqz_i5M(-M|lv10QVD>p|%3wKS5k5SY%E>=-nA9GQg#ly+2Y{gPC zxv5V=vBYw}#mn2`!(|Xi4!=BH2Z7`nq3~+_uU{x!2A$-Eox>gRw+k*0kKLV~MUEuX z;YiXgxm0##YYjek>L*0Qm4JpL^T=$NM@|(qTnW0eEx#+hC@B;!fxCabfN~{&2{oKa zg?RJUgSHBA_q0!z7VP1gw8s`CJlmO1?Oz%hGHmE(qR%3Q(vsw=Ua75Dk0nWBa~WRV z#4oF6^|<^Xs4K=~UKs6^v0qVdXg`QHbX%c6LQCoVrDb>TlqqW9wD_1d*>_roASs5= zCnByQHW5dhmkp1(@mcFZ7ZJB3_o(g%SM}h1varI5h8zG1Zz<$PbS`9N@nt0)xr8Z6 z_>W}A!AlzuF%4|y2RZzs>+!PVNW6SoMAaN9I9*5hwot2@_dc6bd>cTXx2BS+E9#$s zedLTnGIrsJp{JE<86qp$uyejY{6Wsxu2*$$5965gT)XL%NRFi>9gU}qS8vsnE$iV7 z*#6_Pq4)zChiS7HS)6=N!DQO9@qibMICjw8qd27Z<%~E!g(egjE8FZJshY9@KCPZk zcK>-rdDf=0LE8W463w|Fu^5HsmbWW_3^KN8hlSmgp8=|O9&50Pxe7*N?Ox73R5 zh;)G}X(RXF*Muz7b6_ifs?q$`WYNs-?Em%|wWh7F9Nd&hS9y zB&{@Y^+i%8Ry{1Yc3WrOK(e!PYlkF>W>g!>#2_G!lGb3xQNs+fWMGc5vXU$um}-sa zZ?H29(~;+^@*11oum^ONO`?r)u3zD!zGZHi#}`5XgXZ1gQN? z{x7pRgYy_TXFHw3=foh~&bPvAKRAJJA~-}B!7J^^jS*QU z0~aSp#E$^WA&V@nqhVIz6g)x9PclZmFA2i=EsAcAo^o;)0`$?7Y9TuP%&WT!W-s40 z~nDBw`)Ly!k%FB(iUuTVSoHr|=XqZmf)z*RCo#@YT_En+JyP*!cUL z(74>tn2egyCwJXBy6aLnExwo&Iwv=D_Iv3?KMe&g%?oz~ew#Beb52Ilb)i7N^1$hN z;nm#+R?U7Xc+$e)fk6`rTJs{$qj6hO+#f1}`8fV+LP7I;wa2Fab$Fl9Uq^>_^sBic zvFpyPu1m90H~uOoRMW5KVzdUhxa*XQ6KeY5FT=Vn9hSOrMNVkt=+M*sY7QlIopMns z;>6G+=VesgnNf5R8I1%^9~+*|Ok35R>E{=6ky}J^ZbMMv_J0C#)fZ;eY=gD`WOPSE z_w4q&{8MFHYp1+$H7eUOA4#Xa+{;hcF{)(en1u~VvjOA4xa|Ss_7D`2eJ4Z?KqGmV zF!`^*%WFmV>>+5zHW%Od-F@qiHVx0L_=F;U55*)iY6m9W)e3ezW$Mx zHq*(Iu~3muJ++^-oo<+gg(H0)Yb@{Bm?n)u+y*0@k%Yc;H*r&>E^z`fbxc||{uo}5 zB*BIzvPy|r^`P+GS2#)hYL@iTC@^%geD_u98B#NLB-PrP_U1ECx&;y~Kg7M{B6T?# zEf*}L_6m_Eyc|g{XF&eJkWe{A%1SwPOnu+1&yD^ne9W6Xd?R=6lGI~rqsupvw$@lh zZS_>t*84~b?LS$AB{G{|q#9@6$sYgNF+Kh({5}4u^s0A$dgGT?4PWmGVlxSxQ@l&Q zd=Sd7;agsTI0`FfZ$+b>Z82-hCgGpZn>I!2VhAM>z}D}sZ22t1$*R2>&8lHLpS9$d zEX`ynvR>Tj+}cU3o64fi44ZZJfWogViI{yy(+M%U&}EOa_{$>^hB%m__z4%bN!6NO zSse&&)rh?%g_{pizsu-%=wn=obRRyO9AH1L>_#XZV&5d|DGkz<{QIy%gryezr(gXCL*pq$okpPJb zPIzGFJTG~Qx(6z}R&0l+IyvIL@M;d?g&BztX0t?KYQPjr=Z%y!1btDYd$C$Gyg z4uhzAxfxPh@0z@QEtf{A->1hzouaoiB|SuIvdn55aOy3>X~cK(!sHXS^@e|}Cp+G< zTy(z>xe9B6;QYFXb~!s|84MX_O)6#7vR}sIV}!yvZa}Tq6$KgS6iOlKk#UjB`mR}G zOg&}seI&~);&&i2i7$;m9Ag2>i!w(wNVswE>W*Jw9*J(~(lo=v?H`zpz`s9c6YGy3 zW!ndW$nun3vF3r8VbCI=ZuH;PF*D1KG*L8O6BhddXE}aQU*eZ} z-NakWw!3O(fqko2i?#v@j{AU$u|h@+>S?ItDoGp2_?zY18I7YY0*(tV+N%{=>xq~J z&J{D%W5xnnPBp?o{eGxCF7V&VwxdmStR5A3%Z8sAMIMCM1Xp~y-Q`H74)mjrQ(QLY z-sJ4l7^<~nJ9u|=-e$qLwRc4jXqn?)55R2iQjzm3UOd#ClZheeTxoNElBA_=-U=m& zPGG3;h3~+WF>w3eHZ?RcIPLf`Ynhu?C#NnQ-gV0G)U0Q7LT`=^{iR>crO;UdxIL2- z`s3)(?-wU$t;z|#Fgo<>el;iKFT=X-1Xf#~6MD2?&2ExJjwYbCL;|&pq8|zD*0J`B zo7temvWxsEIk@eN2@x_Q`A2fF5HCq3`F?UR2`}$^Dp*lBGVo!Wlh zGnj-HZMWiYuZ%33GIn9Zw%HlYXQjtmUjGRb&fZUMk*z1h*HVF72@2e*_vS=00R z`qgBMzS(UMNmhTH6X}k#i#pUyipO_8WqQ-U1Q@6>yG2f&A2_oI2aFQB zkK*GqZw#OP-o8UEt>kHV+pEF57;a>ikFP4{t2`<7dF^V-*0jwNusy+mZQpL)+AV{? zqjcYru!~N;Fk-j0E;#?L-;ap8HLaX7kv!Y^%6I^BW@w~0dEz>`GB_oPyza?U^p=;Y z7acCckc*#Xd(m6-#Vd$}Dv@R+XC`xrVM(Xl+Nm8FK$axs)}{sVa+LTu`{fPBk=ZeE zTiN%~n=P{<{iA7VA)th<+Bec+_K(uM@9gzYo`z;Gl6Oy+1#F075Bq!l$HBrohSxt` zB--nLO0?HMB3*+QNrnjE{*7KF$zl(7SF@-TyAWeq6W_VQ(}u~=3i)Vh=%(r->^n9P zoXBbCy2Tv9k0?WK&~`^0N=MLcc4vWs0q1Y5YEc{@(-(@nwYj11me;d(sVAuEy zOuu?uvos-Df|oyIKN8Mo_wce0QQJP+4{9)Nh|A!YhvtHaiFui^;=O3-fm%qB4t7V> zdAASv)x${l=Cx-dCWv9lQj zS1O5M&7dbZbL}^ONd(CwlDCz6G#je<=q)n2?FI^B^dW+*=BXb26zF5XBDd>+44PFA zU78yh*;s5VR&rabzSICql*|y>TcbzZglHh=m$$JXvK^nTT2y{?+e;71;12i;dZ-Fx zLO_N5+XrFaGZypcL4Lt5CQBuP*)1EZdJQr#|8SSS4XLw?Xpv**|9X(FLOZkbQPq-X z(C6|h?y=Dvr?`rkf(AGjPcxT>#yjaHvqVZCsBWPxEatv!K&FQHUz*Be-T|qZi5vCg z0%Iu{?LZKEpxhUc)tq|fqO{I{u`89>LUiLXmJ-|8A1qB+LS?a~VKv0ejZuFo1)J`> zqwj-7GTCRYT-kYa+0kERvkbcpPNBYx6eq?jakdtd;Ii|ePsJ=X&G5PG+1*Ny!}zjmVIHc=32bmiC)+j1Zzg4 z>ssuNY`x!4$!I<=J)t!bh)i_PE(u(5@>f+4rPrjL01yX$ZCgo3^J%@}m*uW#s7Lec zJ;H`04KFwJjQ@1`FEW~!^o(CN3I%vS8~U<;OSY2jH7AleNNx3}v$;HK>fL$G(|d8H zR~mX}FX5LppZ{#=pTRXBT<*?mtoEn=` z8={ex?UU@Bny@P2NJVjKz*@A9FxCR6X4!F4vX&vK)E{V0t)4kFUrtTJXyViuCwp*e z0Li?#*wGy=Hu4yp8nu~|+aPeK?W^!tW06RHpV@n}3w@Q3t%v+e)OKLYBs$HzL~RvH z)X-waGTIxYQj4jaBOW4-VNTHW$zLqRdY2dx>jfC!YG4VPpP76N3u`tG$6?6 zs~V#eK8EsOa|aMN+z(2i<_x1h1~J@nCjEOT8-rLDC)u{4gIU%_;GFi zdiM5bUjZLGP8l5lRLQbOBas>G3o~tQ@1TMzyM{?&xgM|;Alq&GA62l&Si;+MKBb&I z3J4nk+NOzWDK*RV(2G-m+HEB_PJu^J6a;hp3m6NvwXF>~y-Vz6QXB+hooOrkui9|yp)UZU#T_SL9K)wQmy$-R|rWg%b=G%_hm7ZIp|fM`v$}z zPJwWgu5K@Z&14gM6dyH&2TP&bUU<1O`!6 zvV44G-D1iUTM3+8vJ$eb?WBUN|6nM8pceDFmx;Cq_Wqd!DyGbq*u&xNS$|l49qJwl4 z%i=JfJlXMY$aap-0#TfXRYpKBN18z|1K)>O1U4l3ygRDP$^}OqW49DhhGDm*`^62} zs*iQ+J|_9G4W|ywoIkLtJfrBAP+(Sh;I6#zRX8&lm79-LHjIl)9GsTeDJ}l*vl5S& zB(mqfK@$0BZS!z)A?cWt+qQ;IP4JDZ!^^XUg`|RR=*Ko| z`oghy5f=00*!Fd3X(~Mf4@mQcy^`NV*zcu@d|||&OV&N}1$s;ZM`S+0xM6o)@H3{| zs!GOZ&c>;gJH~!>pz%6LEdA^x*K9jf|KwY=PAeX(wTR>{l9i2f*kwrwHtZEoErl^XorNPU3MdE z31-ag)1HXxzING-w6(4|(WC}LfSku}`D`YlyuEdx;0W!|DGONIfETSmx#g1yQdHKm zFF7ImVh+?>B5+jA5{;w20o}%Jp<>`UxvhJ;Z9x+m6Dkw8mE`j^;=K zg)0e|>qwcWlj)z#LQLfd&OiytQ#OxI8F8tgO2%3E!J;Bd1#k9N&hld@SM66;!Tsob zF$QYW2ydf&&0Ii*k-%G|iM~C;bxd3UBJu%sS7|D!cRn7YuesjSOEc+wq#Yxxh5!)w zgKfs#3^9?y9;6)$1AwiiAu#}7&e$8ZWhv#Nc?HGis8FM{mc^hOf_f@q!)5N)bH|8` zg`d9!xn=;}9?yRez(%4Dfh4aU6#0U@;%M8r+;R5I9JFehLzP3itAj!B1Q2Og@neWt z89n*+vCwxKzl2zGZ`n;Wv&`v$P-J-61uQXO5}ZOi>gc|A z^QZ8=RqrRie=iQ2#So-#2F+}s-g$ABE-phD40v>J#$f4EMo;Qp^pdkv@8Yqre++aX zf`U>o!wx&Bd`&wCnaMOzWacLsw;}!K&%Q{`W8a?|lDkVfp9(-C+-Xt6-Eb61PrN-nVcFyrn*;k& zyXT~CT$2;}s$b0-2pY4y?i`Vt_3NC_$D>1y9r_$cMP%gPp@@9rTPq@Ed6XwwyJI#g z#xfhBh#U<1$wv_>o%H<_kwj1VDI)J!#@v%PMC;pg-v-Mkgm-kn47uV-XGWO*gPcP0 zS79;@)7Ug0EA}qf-wVFZ+Y6GYD_S(1hL7qP2~S~M;=qRU4;>=PkFQ$L?Mu>3q3##M zCIcTSfz+qyEp2Z;>cPb^2iDRH!D)4Ac94HZYuET(`iyU@hJ;{pb&NjR;+ai8el~x@a-00sukBKxdw|}xD>DVo=Xd<9V zdFh493v#l8v)#zAK3Wjj-5`i8iq)LW3a6EJD=O5q_VJwYUfYgNPrGyFVy%TB(j0TB z9Zb+lDr@Pe5f_OvP-)m-{RCy?B%+gm&N}vtpK=Wp97_;QbE%V8Ob4_hVr56{tM}2; z`kuUx*OX4})wglFa284H-M4W`Sq??l5vwJeT@a>YD|>a$Hb=mAn=bjvSo3N=H0 z7P#l%&z%_CsO%Y|QJIVd1vYb7jM+DK*l!~HAUcHdsr-gcZ_~_6z~EN8pVJ5 zRsijRbjEiud=0v-Y_%ZcYU3~RFRhAaMJIu1#T7e7Q$ODLHQ%PRN_L9mn=#p85n}j3 zICZL*CQJ=5=rO%ANGX{5qfFMI4ve~~yH^gyid@sq(yH{o34m*#v3i7HjC1T8M7tVu zsR`bGpApd#(m;}M@Ev8#Z)Y&+Ar_U_t3aV$8{22(Te#Yh(xIrd1a}u# z`ewtLh*Q?1{9WWDD6^vn$c^LFnyZx93YBPSq)07*FT?B+(r~FR}@The0KR4z-;XtAhbN0bUsn7g5P z*^1D3aTdb_O0Cej&f2jpeKHmml6M*vm-I1Asb#k&KHN9H?%I>PF3s&aB{wzecR8Wq zxuJ_PYR)>jYxmJz;Y|@90E3azjG}^2;NtSYd3oUvx(&<>53CAh6kQVvoc!FS1t*M) z{3a0hNkwoS+(lX@6dcWq{0at=NpbV?&py1osq3g19kf$pbqpT*I~xtOTiY#%?2QI(Q{f7kh@!gHJEk{Xn)wUn zzRs%)UM#q;8RIkW=2JJ+m6o6!GqQ%E>nLm{-Qjy|Ad*f$S*nrj?jdxFTDOs< ziYJVV+?#_~TjI%*M7U$)OJ$$bw{e|thpg_{w=pRU+(-`i{Gm3{o0 zTbNKnYi8oetw&C^hIE_KMfMe^=xKe1{admgvUdH~t%|>_1vIvET8o#CO+Hvvk%P3? zN7MR*1O(fS(2H{7-9}xr}L4JqN9~ZsT3AjB%imdx*>CX3-x9;RYCh~Xb_2v<MoSc5RQQ?p5zlvQh7k7CtuMM2>N*|MDOwvDx{n__0Sz>u>5tn5Ovg zM@yIA%rAL)V8;+%9)b16Yio!7@I_l8J;H^ZBMRbTPkNL4aVl`p4 z|0-%bC5_`P-)t2!MU!Nd)e^aVO*FKfbI*`JF8W7i*SHN=d_D2K)(hW%C$jRyOD^9( zvF^DOj%Hq5_!6>Ywuc0fKc%{DnTad9G$p-kl~eIrLrvbQ1Kc~(I}jf8z}qDyFJJH9 znvL9tuKz~^$;3Q5JLBd|q=t`|C&y5nQ7^~OZuadDFRs+TpY|tm7I8_4j0;`*V$0pE znsrT1u}q_Iv*f_kL!=bdlTxh|vb>On>8hh&eb&_vO+!}=0?Kb4F$Tu6I*C7g${0JT zW9HocW-+)*H>$y{I7JTfDlnGz@^<_}@4ww$aJ2PC+3r!0!tM2;Pgc;&fW`!CXB=8N z+Ww6)slg+aU?GJ?0R|7Rf;Rfts!`N9LqQLjfIvB=YT|4M%6Yk5h>27L5@h+jba;*{ zvTVy^r_e|+U?b}?K8k`dvLU;SEu6Y)LeoT+SuB><&Sw=%K0p6UYNBqM+BC11Z`uvX zmT^vsLF@z8`sk{@Y|f?Tyu2~$7&~!&-LS7nk?B3Ylklc{3%(QBAKSF&c=v4^Rg^z1 zui(gpf-mzTOW_}q6xY8ZIONO;1(CeS0wBW9aqquX8l!L9)gFD@{&3;Hp>Nx*eX_q* z+qNx#9lohFhb%dkUetf5XwLSZH-vvJTB=>#AI;x7HWU8He)q&X2^r<1)a|#zkYrA; z)MA@^BD!!;(u<$E(k^B-`Ox<1ZNlqnPn(_d*6AjXIKXo zj^V$n_hW%LP|cH>=d)Gsw#r`b);_!c>O1QMqA-K6;v{UuQ)`C=xYDi~7>|JGRA9Uf z0^`*KwTU%Kd_Px07q3I;{+Gaz z1PV!NV^gYJBkigX^Tq~n{4$qU>p;o@ATY-S1uga{+16Qadm@IIMs9+}Kt){)PwfGr zEK^~b{zg3Jg)h=on!&*_N?pG;qeVKBkl(+(FB>RVFQ!v&{NO4OTzKlDsG}1^T!`e5 zzK9M()G>_wgmg;VMyL>)<89JKS!msvk^P?gva$RM3$K1I(kZ}kL=MGv+-EHfDrByD z5R`e0>;#O6L`0V2iEinU_VMJz!yx;v`-Nnd7mS1Z6UmEm5viivNX!BjN|)F}Df;T1 zzDyQ~S3j^WM@Si7p<_uDIF{pUkH^^f(-DvD$Zgg16x+;Cb^lt2z!~LJ^*yFJkd+Is zp?FHW_;JakD}tU2rU|(>10>W_7H?VKE(xl)53k6o-DBh-R4&8eb)+70y@=#=T6gyT z*fvGtNoMVtp4(#h`FP40W^)bl-pX=86?4i=IO*`@aW1^_lqezgTn;dN=qed~k?5vu zN?_r*w?h4!D;~%8vqbXwHEeqRV3h2)lDs5Rv&jelI}4~BhTwolp6|MTNao! zHhfywftfP~R^63RG$Ir@YiziVP}pO^A1w$D88D$>H~!0u+|E{!Kk8iC_$C^k^-OI2 z1ip~{CqzyI5BI}Rn0)>v0oMvYl9G1!W4oVRBR_y^*OsiY;WXhInT>vI(_!#GFQ+EoBH$YL!)zoLMuUp7OYyTE znEhFfp4m4DJCcqtjlA)DZpQ`=F9!Xls53#8S(zG5zX@JU3nP_Q676Ip{dPpOp!1{O zG|-Z9n3Im9?3jWu7#^`x(m)AuX3d~|h>|5CDXM(vmkEW8b_gM=kRm28Y=*~7(v6Wd zIq3^{4qIz__tz% z>MMU6V0%RtT9aY(tMEvmQE2$x1xfrM7sKK8J*DrM-mFHTWk#6 zw|Cx?Bg2AC`l2BFDUmJH1H2tdbjCUB7-MTrSy~Xo>X=Q5>{4#-M+5|n1vW^TL-$^Y8(`?*)f4s`LBzmtm0c&8QhgI4T%0x4uKY&&JagOGsf z*hxtoS72S3dtzn2@^TS6(|l(>|@tjk$;eI>-O-K zN4N$Qr)+tm2VX|6KygvZvgn|UJZb$0ms8aomm4D{Y<7%|9iUa=DhZFVTR{FO2oJ+; z!l}7rYO6r3?b+9N!f3shEYP6H-GD6vDQA9`g`^ynR^}&KVJ*~1>19E>@u_5h%nyK1 z^VLpsMHdoYR&LHYz?tm<$paO%1ywk=XFonRm0-P%-%r zBi~K!LcVGs^lw$Ckj9=Ruz@UtRI$ApU57!hdcGa&$EbFt!UsnXPw}%?WL465V{|eDBJMELa$oih~B#<0n>%(?NcR`&!ena?a zrp~4e>%VjuNA|zHp*DZUz78$T#5^tkboMQTNmj$we+t90=95t1bB@+ z18fw-*MG0_d*lZ`8Y!*%`o`>+0A?t`K)8x07at?S%IU;dZb#iw$R!z?(vB_}FbD7U z_8qU7!JYWxcFDx!!63_pvs!v1K7|P;-ILmNfzTAX z4NF3K&Hs_k;Ij!Y=hDl)NNCacx|(l`@y5P4u?`!wlU?H=nNG0~m9jTemyJqJ*`)^6 zh0>r9$@1@(J#=qf+00-{n&`xK|CBrsX|y6AoR$&d+X0ct<)iRjW)0*A-MfITJ_bai z47W)pSy_H~o)pEP>F#iUktFD`!z<7=p88~0MxGV_E+t``>YM!{b?zPhLV**WEr^C^ z4J%5rUbMo6i>+Wc>-RX7TYCdVleR3kBT20`CP+|4lKX;#t(h$HPZ_|QND;1s{< zozq#|C6I$7?JAqrp#N0Kfk|I{aHi0E7~Y6enNYaQx^ju=oX5aT#}_i*Xe{iEIKS!C zq4>NOv|?-BzmPVd2-!%!K3Wh`R%IJ$VK$#2v?($r;0o*!9aDtuO3~_j#F=jno|M0Q z^njrpN}w{exygt{9mmp(L>%4Nmfv(nqjZuifpRkY%|S{fBYQj<5t$gGYjkeg>Yi(l ze=FhYP;2=!M8|si7@xz)!;euzByQ@)CVDGl&f-TOi8Z#GR#s!>i1tAK)kmcpY<<9Z zZNZNr4XrK4P!A4YikjKSDrloJReaUSRud|>Bx)6Vc*(p~+A6qPn=Q4gGUA31vCK2Y=dWI}GQp40l~}vZNW-JX zka}|Y`V+A}lT$?|g_{7gEEgMT)3EI>#Nw!_!@YbKUY5nDtF`IaI@xo=C%f?&k z+k#H!jYmbDDy{E;FndYlp&W*p0C0?>jQ~#MlsWEjR@3qQB`wK5h{-NwkjJ3|;b@M| zH13m&0}QskE8Xzu*MI0=>V~+P((wmnTeh;fWJW?yf0}Y3>2x6DuojKvpJC-dui;So zfILR%9HZ6^_$yXlHoWE2A8-wuS1;`zD6&Wzv@o&?Ba)I(sIc2wH9}`~0v2BlN-g{H z(mJ*&DH_#z>lc3tCse#TKT;b`*!F5=K{u*78mkn6(?!O)ymU34$}74Zc-5WM@w6*|WrP)Gj2&2B=RyO}Mdl$6J zZ!f!qL5JZU^XAtoda`A|mdG+rAD#`2(>Aiz_XBKpI!Ot{Y$XsVwWuO(rzeBPVes$p zmSY1*5n#aSeAk@{QAbF`z?oNE)M<7$hN?$Z#p=uk)HXE+Y7@_+H`t);jwG(!q^`(9@>6fl zGo+ML+c%O)fRyr$cY#vHTTMKK#KNNb%CYXqw!1Q?v)fh|Wt^78pmCj*UMiyTuILFS zAVn&5Uw4m973o6!Rl9IAB+;gmF0v7v#7u(xI^m<%0XrySaLibRVX2^@?bbg!DkY=C zW3EK7C^YpiCC)$qL3sHdH#aQ6hQ1h{Fwqp!Yh75o%CqbHeCiO%5aTs$oPryU9#-CG zTt2^gzB?9b!N%zX8SbS-%Ia+(_1Yls1a$FCHrO9#^Y~(2mM}pGc@+%MgP>Z&cmfll z>$EQj5=5w(Y;By5nQOZN?CdXXdfc;3Ou!jlkMjS=-n&3YQDtqz8ak2$B_RYwg(QRs z5pjrj#6TJd5=Imll#9FrVZ7j|^CIIdh$MCsNMMu@ukl6z=M_frCF-d2jcDlLh>D7g zqBxEu5fv3Qji8BKlKKlxK;%u-F>d1e*u;n&n72g~s_lF#K51-L(!i|5^vhRlNME&~*WHcx z!40xc()BBk33cz1Hf2O=UaddSyC864c4#-;Acv=I_owDPa)25(>_HuvVu4kc0%q~7(w9FnI zoF%(xXPzy+{iP3y)t2q0K8bCzdG;p2wP_<$^6vGAcF#%8OYa;yEio;xcjwT__+PL7 zp%>?*fE$(NYGbzOm=AY(ZgGF@5&6^H8QmBsBI(K$%tgoQNvqnmt=<4 zwHJoFi@t2T&QCowSa|!-TDJa^Mh5)uDR8!t`b> z>{$M`K24j3Fk(nXU$)nE_iSJ}##=GEXK%!fVtYk;qcq7{MX+a@%MoRQ0N_#x z6-6tx!8^c4d3i;``O?7C>5mb%9#TjwIVssGfDs+JjFc2723-1K^Vn05^xUGwb2@1=0DYM9K zDoofPTLlUGHm?%3OxpnXpiX3`<45&xMlxyyR zXzh4e6yabYWni)!2}uZ zq41HOKr-KjugT`84*Dx{&1BX|7P>`_xxiU)Qo`eNRIJ*%HZyvGO|T%9Vj(l=K}7g( z*g$eCNN?nTCL5#B3MONj)UczxhDOf?aYja2&u90Q$n(Zd|p+xe(prav!6(nf4DG3G5^ zcIkWO;yeu{4!W zM-xssZuAdCO~vww{BCYr+ss#&CBDBb@pB=OOnI=V8>{%vWdr;hdTc8{&@%Zs^0%^3 zmtwY*JP|s`_8oCB_@azVv`&%D@WB?XFCA1J{@_$$IpQLPgr&&5TNg_)nUV(bN2|{z zG1+s2wU|t@YapU*dPunTx9|ANr!%fTK|nIz*PJj33ESz=hiqOi{luIcmqmNkbtYxk zyidq03|Dwri<~!n1NDQ|?E{1+QXfO8Ni3$-Ce_1R7KFmWb z0x;$-7MVgpby1JE@FKjt>eU&mNxA4stRa)!++}`DW4+QF5wq5<%l{UJcpY#UnLqVj z&vvfOPdoSh$I06QP+;8P5?ir{o!kW{-@z zInctWa?1BFIY2CVKgzfFiN)OhEh>T0F{_aVdXrfMcj_@q6VNpEH*PM^GDL&B;J z$K2icTeN9g@r%IG*`XJc(yH@Q*FWJ8>>U~Sx^L*7q_io+(s~X{`D;tsBHz`?@slcj zl`w(aH@@iG!tmX3@yAs9njwDtW_;1tV-wORtl04W$_?*THO9v$9osu;55$h+R&02q zs&OxYp;H2P^$l%KOxr#f|E1C=Yv||_5*DJ6jVq{J3fUG-JoK&b?3VEzgrDkH4pZf4 zyocDti{kYk{2}2Ha}C?|T*KB>3^vNEgsp);PGLgX+f5%saAJUlT7g-+1oRn@uM2m^ zVg_MG!Vtj;oa8A;AZg!Zz$qiP-|z z>=^Tf$OThuwcN!960rC~4*?_yv>8My?oN~`*1Q!$%YcNKh1HNZ+v5S&kN|Jap9+_j zHB3=HZ1n`lds5u3pnh`_eN_FvIoazo`q%H|3-bDO{!Q_WP}%5D3yNok%4U69Si1yN z+?J*)3#RQ*$n8&bDm7h1Vmj#+xetjPp_Sp5sxL$OlIbStN}zT^4GnM#DoBdORY0LW zN$-G0M*&;zl3H0=tz2wTr-lTG!!C~Y)>hb{K>Q0rNyXBs6r&?bMJp?E^^3+87H)tQ zqzI&2pM<-}hkf{uOir!}qZ`}x*gL{6hz4z>Fa^{V5wC+EP$@_VWsC;f8ai{yl&s7$ zBWC)0J;xTLVvIel^qwBfXt7Uz+d#OB$X?l-VF&r!NdjL$I|KHN0%|vyOJT0I(56g7 zR07$*euBSk2ucGxB7n>oMshO86BhBcB-vbPAiwBNfnXMwJp3nPGFIUzQgvGp!7*vA z)X%!p*G3b>_G&4*KN%E|5X|P5p&uf}&&;9G*H&qA1HK|R5(>LcrZ z<~|H8DlZ>YnB6daT1pkeV7Em<)Ies(fDm|iGy|d+R37}jWZe%dd&AG|3$Eaf4Sxia zMbs4q_mEYRa$c|?c}>6X4?oM&7^D^lBDGcamdPJe&0V^t(i%0LCQzeBr>>m)PHBYN z*N$W=wG{cY>){HxFi*zLGmgAm}k+?mG>2 zF?9`Gj^;IU{~vD0c0}DqK5CD9ETR7UBiS!IH#>J;PVH^k#bX0;50}T?GdlPHEFX)r zi$@3IZY+&;M1=CBbReretn!e|7Bqmv1P$JAQE8ZT)V)^na%TTwUKQVbjBm>QIi6 zkUn-r47#E+s zX-xRPd-;;H$A_=Q&t4r~Y%_~Yem4PK1kw&AjX(9z4I6&}_2f}#;r3%H*)E@dOP#N; zVBB0txJ%~WF`I1m!)NtQd5hGNmzI}h?5<$J5q0JBKCr;imb z$x{>NKQsGd{HaElkxW6leYdxbqY46s8ks3#jeyajv!HV1W!Z_=G_l7L#)uE^Rw3v? zx)pmyS}I2o97WZUkv|ANhC)~~#%dj!$vhb|Vl(0q+5@)^f}zVtqPnda|LigxPvISI z56AQlji4yLrawYSFI9Uu-g%vWiX=HvIyQ7+=5OZ4luCX!fn70|Q%`N)lCF|6)Nbjc zIb&Qp^v%8%#nR2ccA22r(7Y`>Ac*&Z+lX+hNhkc4+Vm&ux2Kbhe`?c9@)-3?_b4mL zhb%^EBc001dlV4@^}*iMy|8$wMUr)v3dn-%XXjw=2h5-(xo0!&*&bDF@WKA*-K2gF z0QX5GfI}d+(+j^@d$cj~1kkcHVoxk?GaB$>%6lby5x@3AcAAZZ5&~QctAK+4|fLS+YR}8>v|^KSR`gJjHG>A=_Flo zhRTb_s0fmQY!i9iSvXGU&+ME7#d=gf$udX<52a{F9kas$Ru7FVoItwu;k7I=t*EB( zG6um0D9r*!mtYK|2q-OM%xEBt;FdIKC>Qh+oB?fjL~v$sH0&l?T_cWCRS#+yl$J?4 z;MmzDH5(n#Gc>3I0juzy){CDzM$}q4uj}$8dNgXj%oY=C2!|KwnrOfy z<8Qh-95F;QNW(j?PUY-gP$J%cH4u^WcLGd~ASdJZ3?)sd`1SsnW>~x&g9(qHtQj z^P&P7b$-y1n$s=zoSAyR^$l=JB6%zAqo11|eapcsa+T%?kiHcP)BvF6R8w!0xl&L{|+{$6JB zZvt@_6$U$_&Gv$v+y&XizYfIpt%y6hFu1x$PVF6k^&PX!*No2EZH3_{;^JRTj(@(= zHx9=?j4x{S_uMftA@R-hq_ydZ=@VCM_#72KbF^Ic|7(62Ke!YuBCQ3yn-nTc6%Pv~y0%`n&z1ck#a}e`w>J)b%HK z4t2uMPU;_eY)(qLjdQbEkLWmLxu z?X1v)51=c5<9@(}=0>4*a2NH!khI zdly4A#U6?5`9g5E-gOssB{3tljvJ9&4RKccE@#)^$QWZQ!dxXdi~J#;OBI~L78-m9 zL*e=7%PG3Q71!cGZ=v_X9v`$KM@>j zWMz?K29P0b2C!Vpmej^agr_!>ixmySt{;OHbK74b)K@Elsdfo8Ym)1Z=FP;43A;k` zORcCSG=gqW5w$F`;;o6oZ3)GYxnQ4>$QiAVVVp>s&e%n6a3`ub#O6*0ypL1Z({{OH zV-c82XDu;R4Qp=7Ti11=_sbocN#3{y7h->bAH|KD}cx$JgrG;GUBN?S&UNz)EnD?{&WN zoa&OrSca8_rtMB{NEr>V^}FEdOSBd2nrhgt=AnbUg@cIDh?~Zr_G71+iGDba>)}i( zY>m`7$|s`n z+FavS*&cskx3VMWLvsAfDcfeB>BYeLJs7yLQ4E}5*xvegFl?vqw=|L)2Y_g6>l+$( zQ)Y2$AXo#MaA&WC9q{-sha<_r&cRJ7Ijv(*0oMnt;hzueDM`FgNFzV5F51QN#`~8h zJ}yq}sSbC4^uSSdF9Lv&StPS+t~Mm_7>UWRq0EhL5Tj@IIgjijg=6iavAN|epj_~p z?+*gTEEWhWp`4662zki0kRh0&$`C@;ziy_*uo6ojewaln!^vzxxAB5$(_6wUlE5Vq z_s&mJ$FwICIO8=Re={#n{(+1VxwEx&*7vd@A!C$i$RtnIHy=&AvQ|k_AA!1K3+iWg z65X?t3+e~?@J~4BLrlr}F()<@R3;^dU_(N`ZKC-Uw^jq6|EFJCXo000t+KZD6&7A< z``*ijq>gMzVj#0miiuk*^(RPBBg_FuMKIzGT7!QCe;>3Yqq&~r3t#tlK_;P&`IoS9%#3!YsG%q#cFX+gHYfaOdSYdKSZy) z8Afnn-#rCFR=3ro%zIzm>q|IK&YN=S{r8pCZ5}{uz@e)qrM8{@q^JM{&;@tq8ZwB{ zIh*;Y7&atdKL=SSJ0L{mg2xx8x9UY^JtZoeOD+iHdYLQygB%Jk{2|^6)_vTk;s!-Hhmkz zJ#dPwNIX>OH^BvIcl5NiL1ZS_l$_jxAnV5!J-MJ_7p81#hj`gB#} z9DZq+KxtndDYwYZ_1`WZQCRf-_@eI$!_}a`lH>bT`Uat*EnFC08W;b0*ZB9}zN4`l zTqJuZjqRN{ef)|I@2uSLc2(mKzNEFi64O!6_UeNWB&M!^M3k@f4LvI>+EV`hZ^3AL zeiBAI`;m`PkHv84tWCZPSPfZw)7adb`P3Dg%4)u7JPKuBMfF>5^7kV4W@7KulC^ya z*&He=FRR(Zxb2p48&7SMwurg&`FD42^6eJ4D;=AA2TrYQ$|)teWNZawOHFKywr1FS zGiV>i=gU--4Oxj(-@X2_%~s!em2VZZUTcQ!(05Tpnd7eSz=OmRk~u#@UC%DSFaMe{ zVs?xNj@ikq)w$DU92S6lYbMQRET~4>uGUea0~gK2>Hr>ZpwC>S5vkckEU!?vt@K1dF5N#lnr>U z&ZBEK)px{h$~J;-64{jP?-9-;q{@u#7Rl_DK}e?eeJjn44(}#_fG`wG=J!G=<3?1_ zAIrZT8KvoCi%`2UQxdI6K+JhH4_2Z5U{A3Q$eO%o=*BfEk=J zLp7sciM}3CO<-<&uqJ_yf@+HYa%35#nIJs2r6NCV#BcaQI0a`F!EI?XzQ_?~6HPA;jVhs{!#^GW;wI!KWTf(Fj?F=D?9-w z6`<$RRnU~Q{Wj07NuwBNvWuU$J19JYAV}S6L#J>!Mv|a?Tm0yzn>{ajdp1;=H=IvCX3bZ6vtb(uX|2)T&@( zhIx%~)V)!i8~!T{U(X82bGw#> z|1Da`oj+9zDXFTEyA~4FSFD8uA>~@ScEb{Q^Jh}omTMpF!EcPijbgRY-Z+;XO0Ij& zcO%1$+={Wa6^sHO=40$#Yi7i5qMh}~o(Guea)=ux^Yusm$qnoAcX%26(|3)jg=ABf z2xd+An`mZjp77?hp(h8R0)T?DGbUU^%J1LoKeBt*=~gM(@{v|{b(A=TqW`**bgEcO zZJKIbHT^>CsyR4Cz5s~-E6V3nVfY?G7Tfm7^o0zCw%5gn;r2s}n^B@*4 z!u}%^qK7A=QTDSV`739nXEhB<4iwU9oBxO`{4gRV1Y{N@*2!)K3*;%K_(b0tA^38{ z&H#9D4)un_v)!JZglj!Gj%Ur+p{Zcge!*;7_<^ZoTHjqO#vg$Zuvk8)Uma0^oq!;d z7#XS?L$<(caMvdJWQ5m%h*hi+u3dslwvZ3E5ugjehCr2bT}L?DbNP1q=hCTD&0Z96vi!^FbV835-Yp^*U=F0j5aN>j7x_y| zvm67q1F%!{viT`2T?4+=Hg5yu+I>?x$8inN% z(U{a>(^G=2%N!z>W3CHsg_2rm2Kl7)2nANF;s+KXt;;slnuj5ka?Mv15*=JX5rMa2)eCQvvYvO>yEYwc-IZX8aL;x`=nrl zl#z!wpJZy1sE3w%!Kua6j7esGz-?{p(_OZ?&?_@MKI=qXvi&l-VtA|wdM~23ue=og zOATu}My>TW!VoLbLDgxNOw+i^;cL2fPeZ~*_NRle(Q7?G6`V(QthXsUR>&$5!Q#_S zh4;`SP5?YMZ1Fk|Ys-$oo?EWbdQrve+=RW%xT@$UC}f((Rf%W!q^%9}+78SpN*i8y zz78HpvUI)(0mtAF0;V8$+gI2}%rZ&o$VIzJ_Ekv}x<_nE~>sHI!IZC1-gxxZ%! zB*@{i+p1d12SAz+Ivm*+lQa!=rcO1SoQ4+Pdlv~sIiQn zp&|^r%h$WOHLMCnG!zk`*fsZdKCFz!90^?owYK7u8ouGy%5v|vZ|8-q-<(7jK^!LI zTXV?VlQdhdMG(-FLI4SkS;O(q=R^M(?dZ>_tsz}}KgSS7CJse9fx9Z1<6XHq*^H$V zUNK5P2wJjI&9Uh@7(2`D)9^>|pW4>%G*+H){<`&9_(T4h{UMtMJ$rX}HQV_!w7LdD zG9!1R^V@=o5~FK?5~T4m8&8g z!{7j5NIq~`qpOgNqFc1(6Uldeu#~;E7fA79Z4pgwW6O1Al7O~-4aY=2-O_x@9JbPr z>8o%_rImd*%yBve6ldm_6_!8;@{)s`hR~6;dQM_G&j zYU_lILfiTyRVPRt?qLm=6th6U)T=p`#RY~E)Y@zz7*hjLptN?HSW{(GvJr74RC|p- zNk0+wkTje?l^Wt^?3lc1Iz^Q9O0ZaD{qqounBr8CUck0R_lTiQX9c!<6%}>4%C-_; z>Y>R&RmZf$x~SS2);YV=Ii?HCH+d@mE`~L)0A-LaTJLGJsGN7Y=WC;vgKnbJgj%D) z*L4H~A90}wb>s6;F<8c2?#luDhe(QC%^}CM0&>E;u3k*X;QRNB4Av#y0GNC_(7MrZ z9|f4Hn;RR5IZ5EfWb7l?XDiul_H`6;4dlCa7!j6!yD3p+&FQ$U(b*SFOx9|;=4yGV zz|zBQBu&K_a@s=wxQkpOM-(419xZqMk>k<6d=mC*knHH41FR+y?O!YE|$jtp-e4Ynf)CnCu>b&Mrl7?kiE zAj{aESlk-nDbfMuV%F#63Kta3E_Ap;#2e&a4LNCKH#)(+;JMHvW$-(W12DV4_3-`v!zGPTZ&U8 zG0Tswek?$#8q#?Q$)|8G=u(nJu>emTPZ!6e!G2vab{AWWT@ydy<(FVZ13KDh?3kUk z=sal|LKYz>vidGvwOc#g+S8X$6gMgF+@r8KM!Rgf$#3Jk@2KqFG>3n8`La%DytJ;U zzVGj|*4Gx*pY{9fDWiy3dd&)Qn+IA!?t3`ZEwPfaULDVm1*W*=1+}cpVk9)c5}2z` zM~bLCswIR)2*=XStwdwA4b&dWp!))L$R-jBjmVjF;-KCwSU|y7gWj8LftH0GuyBHK z27PmA8=7P9;ilN(a!_iUM`lu4Me9d`S`)olJtBruk4=p;;sQWSxcK(cYVN_PwmA_3 zkc&uN=_v0`Yt0#7#I(ygbn*M7**X|PuQl4AV_89xlInvct-G_ML6!<@$pobCy!Ttt zrqWGqN_7^`l5}e%#4AC>>Q;q{h??~=UDW&`U!>6jWBrPQ(+a#90g+$`F<;0L1qP34CaVW zWe*A)E$lM*vPXJMYON-EB}&nKp(UbZ9W|ka&e&^eQCBU5)h8}Ug<*EY}X?XDZek7HyE3^*9%o<^sw94T-v;wkob#o zgW{*Rc2!-=p~H0>a)LJ;F-OSv>JQ8v*XKVLBd!)BM11|nV#I+85kvpSVuV7jGcDj% zNu9Higa5y6F_KlfL&_n^^rOAYOkoc1MzV>FSWM`)%G88y)!c|CY`ALHzS!W+E&?-$ zWJ*b9rL-fU19mv6G!@A=CA4uwy(8#L8OvZ)m_Z-pmU8CR`9BUL1O8ui7-`+K{iQ!n zS?ZgR_RqUNX@0r=uB&opjR`*7Z|G?Y7Ipr3(`A4CjjV#vGzKt>r;^W78966jLII2P zFx`>!Vt&CfRjt(nu?ejnKng4cGh!0NCh*Kh(7~BCOMBVx6EX_4g~xcyiW#MAc&HE~ zhjmj%z3A;WfJ++wc{QyaGW8t^CV&WjqSamm6}sxzq7u?ws@g#c&RkAOBQI#;4z&O1 zklCJ)SlgjSH@46e4~@HBL8284&Ivz0gZ5bm&ccsRWEHYW#T7j~QpFW33!(h@V$g3; zVGlRQuPTSA!K#s-1alUvo;UzBiV`Mjsc0oBv@lq$;NDV66h~W@9!6k=xr&xGuBjU_ z?I_=QU6;sgjX0Kxnn0AWx8RRX@+rr%|BXa_7QFir0OQi`Ke%l~4(YXmVva;aieQnV zw@KME1h@hT@E5*e5=kusdS#%ya#srNnQ3^@aSE(58+aPikkX*@8T+#07*d67sS*>k zvdeVjEIWOYAg5Kjj1!_5C8=-i$a;V|$+$SyYMZb05}8z{=v=rd3V>WWqKa5TY8y-x z4POo6j0_{%cI?|n_^tMc;Wz*G9TR(cRk8K)sA9X{rHXB*7yCxkT!&k*wbm}p46QqQ zOn90_1NVj)q z@9rh57IeCuXc;uoE{thDGE7!mRf{fcd#?z7$6Vg_{C++C;PT$N;a;yIwkO}-x4Ufu zRl2!*`cJrdO*^{_I2}Rwy_BaeeRR(Z;ZCA_LbR_)Cb3U{S@Oir!?4?xt=E+6qoWnu zHQP>Xhd&Q*QgBwZNiDfdjgBYW{7(DfRnJlZ2hU!?IPANAq zN7!U`ajffnY481|iFFD5Rj8#lJt1RD?df!(Sl($Y1$d#mvJoU7-;9J7Q*O-TJ9$6B z!jWR+RA0Aurzuvg+#=fkAWy+Fw=#?Oqk(F82*L4)4W?6F5A55R7pzD=m_W~G*g)Aw zdvU8W^8d|TvWu+qw2K@RTP+ErjMBMg2a@ll0|{L+FSe0Cyfi*qkQB3S6NwJ!FX9KS#0xcCG*b}-mf$@Yq3bY^cEYOC2kFxVd zQYda~I;RZZLaR3XO9XTp6DS^!PQB?Rg$kcn4yoBZ3e2NLl3YnuH&pKIc43GiFDSP3 zr&!pp6Nw7h%2Re96PKqp0X;RnueO zQV_jGzyt01l6uBV&)2exrr~+%?5`|#a)yZFVe!LrCy{lv1QrliYDmm!ZxR`P^T=g8 zQpmEGlX_P87_#*+kM8En3cyI%z2Xf~lW1y1*=LkqQcDO^Y(;&qVpR)baAYH88DQBY zQWXe;u+HK?kOa}(i5zy3Nw>e$hFBA)ge9wycK1_^-4O-q^-&7cXtF|digyHM*RCRF zOQzY^u9}#!|DY6oM_B2J`G=5q2*j53JI2L9d~v!BJM{9%cO!@UXsxvH_S4f`g|!8W zJ;o(1!rRaAjK$yzD?;~rDzPlQWEA7+fv_(N4j>`+N8*H8Bo`eE+mjBozq-y_9x=cy zNnAzXR?Q0otC>k&H9i1*3P`)GyJ#;OaT5ua^}L1hgx+P-i@|FPd*{p*4Zw|G8!?fk z2l>D4sRbSw3Jj?SQRC#8T3i{-@e*%dc7$5RHc^vslMGdIsAjubKxt6G6pMUUB#KkV z;H>e0#mb-3 zUY~|#NkHKJyO!52XO-l>Bi2iM<-Bn-y<8&8JX|8Dy0}CxI*+@tp=drOui77)(JR>a ze;|$A0cqraCj?(e&1t=2Wa^=(&`JB3D;mEWoS0s@pz%Im(( z>4hsAx8Y}ZEeOx-Jv=z0)7_0P+>v>E^@Yneyt=Y+@*UUCEZIJ4S>p*fHwFLypftO6 zG2HsI#)tRyN-TM#Q)nJ4+j2(-Yd-Ubd!n-K^+4mP5N|%R=-Gq&Q{xyTxH$Ru?$wB< zYRs~sNn7_4*Vb!;r1mg|Op=U1Y6GkYJnc2#S;A7JjE^V2=9{*rDUTc}gXdfsJimaT zR?mWdQwmtHc`+Yjn|g1yC2{1L%Qb6uEYkkxoZay@k%qe(XTqsS%SiP0=4mv_dE>vKnO zdoCLU(f7)W!e*!*9_DE`(*E*^8)mGI1F*zMHwJ;KZuZ1d9@fMj(UJLZxF*&!X+xgw5 zPswdM@%&V_M{Kw`juk8TvkHr6S);#N$mOrccNE*T2qVmZf(t0b-yYpW_Y z1P*ix$k97n#;`@lBiLKTD$TH^c8Tbutace4A*oX^$C%+3lu1i1T+-mN!Bu-%YN3Ut z2+j>%vuP_i^mQK#2v{zc?!f~^k67_@#jmK7r^ zz?y>ph-PcQid)=iGd@QshV@pM^To_R02GOEOvPd{09MDSI}++E_o3F|ws8gEj0;8E zwCxeFbv_Kyw2>p_m>lMI%UVEMNgz@+g*QN3_hX0JEF#~4pgq$tX6h6*`=n{>c<06LBu{ZRjHiH&>Qy61 zZRDXgqX!%<6Q;FU45q~%t5sJo#I)?d~u?u5o^dy~v72cNKQZ7md^>{#2gd9btk{OmQo_6IrjZdQsK8`?G0Enmu6g zFWq3VdQG9g&2Zh`kN_jLPQRyai?pnEj3(>U8zXLF1s+nNZM8}lp@KcdWtcHGSgbc8 zwM5EOHW9XqND`eXNZK5G+Rip2KMtMgPlp#t$-zOnOEP;MIWLm>c}XKZyh!?ad6C?0 z_99ufV=VmtS2oV>Iy_v29@%peg1Z3N<^iz%pFRmY0N5&z8xwwEQD$*YAou|a+VZ*v zC!?TkNY~(b_}}15g14gBEi(|DjGvv|wchSUGV`S2aVM7tXB<6X;lA5{5_p?b^YIoG zYcad#W)}I~(QR#8?#%ddS@^8pDQ^L7xYfCRHSy3XJq5}3B{OV2^ZQN7XJm5tir~?N zzn&WQ#*AR%l^#a_HQx{h>C^qqKKzGeo*BwwSaLlVybGe6r0q zgE(Op^3~46imzk9CHEv7e}{H#1c#z4<`n3o;F@Z~c(G+{h*!ryRzp8-$99{iX{2xn ziG||OXU$KDmKZjM(I-}vH5|jlU`a+r*^C7I6aF4;-#Q63LEAY9HpbA}sU)(@4YBcO zAlN>NNwCfKB-jpm5NtnFKQ?xvHxg_G#Ulk?LZMa0@(GM>`&hZc9je;xe4K)%VM+1$ zK-mdB7-vniaMmn3=2KH)5ty-jQf|}X!QIj)<6dVrC7z%1WrQ8cN!Wxv?MOQ4XZXZo zSEUzrJ?4lxO&hr?iyaWQ`S~KHrs9amNDPvxO3F){84BK~9ukTU^CnM&5E_6>@$+BjXO%OU%Flx_ zNqXEF6tgNP68bn(B&GEtg0G6+mvq#t+f2cN!AM0?cQd2t&+#$5{ps6svPbQRhc2F^ zTSSFt!71`Zc8aW@arG%p6umhiT|e7UIvScumt9Fy1R|zvm5lUlc;oIw^*PPVKWjldh)| z=Kt*xtfLxZ>Dbu;Sy?UhVs_#$`GC~Tu)sVUoy@U2c23&*CtGiJ13aQxi%UK;}UDXUGhb>93 zL?Z3y21ddyXa%$tu z*Gr2x{&j0{r*z&8z8%Q!yJeir4wl^!?i7H}kJnR!yy=g?Pj0@0BsE^XK$Kk%hzG z;??@HwD||FX3ITtYRKv%sD}K^OAYzsyhv`JG8!z}9oem?L;Co5x9r+EIjucbT)zFM z${}C2WQ^zc+kQR+h5xxC_$Pt0F6eitpmXO|a`z~CywfKvi~MR?_=?_)Q#xOeedx;3 z>`yrFq)YaKb>d%l!)R!iP5vF%XEOLoIx}>vFdPwJEbYwD@z)OC;@vJ=GFL{OBDS}t zD?v+f&YJ-iN3GH;B(92oQoq+yZW>V>n6h_nc$Ji!v}ib?pgJZQS=9ZXSbXc^MQmO` z!xI_+>yP&NgB~n>@YfL@jv5(Z%R%?F()M$E$fUO$+lDRrg=R7tphwXop97!xinG4y5{n5jb1OO?yvj4L1ZPr&#CYwl%!s2SHvJIDvi)|iBg2~%F(OUk{ z$cAk9xAYiWTyRO*3CE8uzHq59;K+N%D$zFf&9*gPb(J}Ybhea#*%xQIcyg{fX4br? z*of_$UqlHUk;PV`S4C;IsLk?~msweR(4Yth;VX}U*Kw)Z`J{Se>npU)#JE@5L+K!OGvma=d=n!%xwctODuihGTIwkH^V;iw`?EDvHj)H zm61DkjL2*&slBY98=u8D&n>)+wP$W%ffMMsgx$fMd3Nzw__o}v${axGMif}Eh8ZI( zjah9_SdW<_tuRn9RJK?}lFj;hHus6-mBdu$DKcdgV3drmjVK9Y0j>;FhTt0c{}`{r zC4)TVyr9#lN4gclXnZ={Hl>~xL z`ze)GP?4wyCHX;xu{>#&lw ze=E^xe|+KioZQ7ZwH4XL*979`Rm9y`7@Uk#i*jlgp5z-haC}kXnDB{>U~Sx^L*7q_io+(s~Yi?uPb( zFU~qrV#tU7En>)(r9YKVYiJ3~J^KWT|J@4LCT27?On!?LCNPd9E~@oUT@&-)@5Y#+ zc4YmyTS%MddvSGBn&o33qjrs8R0xVbeBj#5?r#dzmcqwl`*gp4<)VFu_5)$Ozbx_d z6jo`bJXq9?LB_ev3SrS5Z*fb}44Bjwd}zj5m}RiyBiL5I7j51Ggr6t~h+@P@Vc2wM z-Q!N4Ok7wfev6l$8GJL&`B<`Y066> zDn1pS=f~zPC7^XlK1RzflE_k4RkCnY=4lA`>0`7JAgFBoQwTi^bNQs<1+m*@hjTb- zy@hJl_{BZbDqEArBN=2oa{KntnaC^do!KWKay75~+}8d8R{2Ah`|Ed;D3X@kycGf+ zLG`8J8ELCw^ldez;@#FWTIEok<}dm#UjL3q?&Vh z>q!-f?SD8^Q!zSdL-Q!I+=f6-oVrGLM>S^Cv#?)-J2O5g{owxp>@@N{6`0v@2tevV zi_z04PRwmSF6u$7mNA>4zcF1>D%5l?1yopO9F8SQ#H3*ZPBCZYOqI)@tLw?%?;AH* z_+u!xkByNlrj|6->rY0U@Jr;+iJnaS=Fn)OH*-9^RI%xpS1hvfP7}gHL&0zY> z?6SI^*=1`3#n-Y`70HQ<>NMO#4WgcD^cst;FWM(0a&18AnS#}7ob=4cDWD)M?ha%T zY4s0&O_aFNZp%A@e$I4EgNAxX+5W(gT;!;Uk|Idjj5>}g?+}!x&3o=sDRb~7Aj(^|MrL#vCTY!MQls|Ex4`EFxiuB*jn^v z+nt{ZVl3FdTTbiTrM~>P1C2v{NhQ^NLtD^{?btxD=2L$-4L!Xt+4$j5*)gozon9l7 zDgWK0nVniaB>T`5=G0n40B;{Mc-yml(Zt)O)-QKH7p3joN|t?lsZ@1>{r1c3N#~SWd}H z3MnMDVp1CC%Y2gjixLW2=ntu)#&1PEzLSvx9~d|9tiFc4SxTm!c>0$jvZg| zWvVD5i@oRB^vhWABZ4X11Trugd`R^duQ5pCTcJY7=(RmYh{yCl@oD|8qyEtOZzQp8 zeg5WO);C<$gLd7vGr_by#7rBH3{;LDOa;v2HKJBJCY8(0>V|j=NR5HBO!R7&O~1eovdS**?4=c6tB6W7Ro+x z)$`^^Wi45Rs9Gq)7s4s19I0h86uTm;ZMX`TU@D8LDCeb^x7G^s!SQerq1fCU$_f*9 zi#;}cfErgkc%iYgPgm`E$4(@?14ZRlWIH^2%A90r! zU*()M42}4d#d*18yf|HK(N#7s-LW*FeNBVs%%wfY$nVkH!0q1iJ+h9-7er@!L zwMR~FX?F2tfw-OJaUYEi_V1BXdwWjq?U^?|9yx5^)QYGmL~eM4Q6(s~X_+kSp(UX4F+U*FKq#I)^$FN!;P zbnpQ-6{($*U3@OuQkTb_+_^q{HNa5Ucu07HoxA6>{sDd?5BNh3C*R$;<+$PDNhn{t zzA#vKjxRJaF>QT%|Ip4kDeLd{hu+2is{Emib5hrz+&R<Y=!lvg9Z6{{xl2{H=k;F#u-E`-UoS%q&h01T#J_3_sf|A${bE z#{OuZz0e=H>)jh}0dUHUMXW98#8ekB+SBBHG4AHV)~A;;G+59(wd9$;R6ezR=y9uC zQ9s4v#{I%gGNb$23_flF)a1OXd+=(gc`d&QMLkIBsc+RBPG+mjlqDL``&CQ@4%u zj->v^7EV-Dn_p9P+%P-ClT|ArP7?ebee3ArvDTrPY(+xgtsbq|3f-{TO~`l|VV;70 z;P{Abl)D&)LF{^biXQ`;1D@(jIlT9HYxmefDp1~^vcxHHFQs)i$Tn1N#2OM>Ef1dr zvyRr(;wq;>oFWXYkyj-=(A*iG5!MjUut%IU?6 zt+KY|msz*HiaK!(YWiVts+mqq1Ucmv4`6_>0jd%V7gtV@ zX=EW0eG?TP<0dhKvG93QId-q0N~zeTY9#%!hfb048=H{Gha@5>2VEKHi5+?p<7KVQ z!_$d;Ah?9R!J=bjc`qnE!5o94#)%qBkr}@4NtwttT(%XBdy8YMo}?Lm^kdQyuh3&+ z6;P41MmPeWVTyZl6NtR5@W?sAWd{m76KsGII7MxUxfsnc;@%5`qYayI42em!vtx)Q zMKoe4=p@y`P{^cm6d;2N(i}KfC~2T%uw2|9ASS!_Q8{su$0`!1W)GQ zuj?Y^m{FCo8pv_e3Oqjd1((}gO2N2&IrM$fHj@Si|4ksR#GnkwE~00fX1lOkHPn6Z z_Kek}e0P&EQ1h3T`x4m^%Z5kD0mPJD-2VJLSUCy zTMO~+o)`iC48%cn|z%J#aurB-u7YZH{+Fz0rQ?a zkk>x{-3LiG71NHS^yOiXW9b4$ak6mCfAOhp{SP)@v|ma>7~4s9jDH#szq)PHK}G8s zP$8W<>Frj6z%eaJpc=I%yG7cu&QwS<8Sh!bQk$%i-(!Ymc#olMWJM`Et~eEze5$GC z4ob8OJdQ%QS4d-OEOgt%-Q=Q?qehZLkV>~cSD`>DvSd6-%8~tOy#1Mb<8A5)JDMuq z3W|?MXu#{o1F{0HptzTalE2HX%=~0#MgWle6%F;Yit?ii67jviUdh9|{FiFuCU>{;Y=d#CuxvH?Ms7Hc!jXguxIG<76nb!5q$#99sNC=UEm0|P z8SfGvwKhM;&Zh=(Aq>B)t`*L#uL}D}FiAGs#{4N|QBx29IAMCK<�w9V(wjvzQq57Y3WTjxe%t5c@h4Mthk=hHEs=7Pq?)f7 z8TB6>(y=3dbcz=Qt`L=p+@+QRsbjCCJL5cr7%5!-5yMKdyJzAZ%qf&fu2n5$NJbti z#u5&lQC5ImK|qZPDWZ!JX6SgqO-rgM4FuHEbI{ck247eh`YWJ_LS-ko5o)d&(--Wd z8WIC=!zoz32>hJ=2}g1I%c}41B}PR=hgJ$mD{yXzX1j)P*34bI3gv?>^g74DAsr%- z>q;XAK@X2W#E!eDA37EZG8Dz~p;i#R4yo($0+C%{1PS1@Bmp#5g_SO&i%VW)^sMoi z0uu7nEq1EDK*2W0W#Sscv(dK&87jyRwQ5AeXF#+}ro6?5W#ze4%HYRo&(yCK(jU!> z1If`)?+6CPBlklbn(G4y4FFH0I5fw%E*QuQBnZb-*nr5s5bIQez!OPl?c&ldJ*T6`Rg-82AjQml15aA{0_*9fW}v;YSgoCMuM_wK z2+4a`g{WHHReuo8Ywjj!?s}(C7KFFvd;*?A!YglGO1qrHibpS;>K=6J5iN#K zzW!14X3KZ}5I3*lif7S>n^R#P+ZgunEuv~ty}FK99r1u;FXZhwYUVAZ-W%?D2f7{! zcWxyYW3}DiGPJb)G}OA4{g_%e-@4Pr7x^X>bs7`?b6ouK;~&^?)ZLB0j!$|sBXRm= zD>k&Q-0*!>;}}%IWh5P&k+AW|LAL6D2V}c;=1{4aO#hMn*kx zk2$$77Inx|bV*Cye@Pj22lbL?<1>D$UXm1?)S%8l{ZY1uhFH`^ps=cowM*|(t|Z;W zDO^(X%>)6jMp?+d z1j8PU-&UY3QsyHf<(L>vJD%7gPrXaC-zKN*Y)MMrAMf(_rzh`|mZWK9WD!l`rG*HTEQ)0iR-nR##L z!HG8RGp>r~_&NqedscajnUlPPr{QKcI+|Nk!7ln= z{G)zg@_`Ja+lEFHQ`50uJE4Yt;35M&0C$NN+#w{=oFy)KfLTbLx>W-sPc?CWqceU@ z+`M@T=L(IXZ3=g*h#*Ut(Y978D^?bR&9QUs5+O8o9X903gwPa09JQ{qQa4UKg-dEw zvfi9ynBtjdSKgnFxLZ6RMjNy=%&_!Lu1f=irUnRrY3V1@o7HQo8=KuF;on|v>J+sge*s@l&V)K?x98O>!}%~EWO{N&1cen>YExpiq`>&CIQH!?7~Xp?WEICaa|*2S01zoVIo zt`3du%+f=5j0RBTp_sg#$hMOO=n9U_UBDkVl8W%QLcH^+<+$!>Ebh`g+%2vr&*`l-Bh^)dY>ZI%R zF|7yD4BGvGKQ;!fPQu?93>v{-+JPcHN;HEu!^WVIYl)3PLuRKgg}x|JQpOvp8-K-y zCe~KwM2$)@`O(CLtiQ+f^k3~spxs_QI%TiaN@^RDjfHV6dhAxQ^HJ(Ruy?!wS*KVa zCRXItt&m%}Q*nDNY)mNp;pS7aBiJYm-BV_@S-0io&@~gT_ zT(e$Sc9HlhJ^s(%k&ZN`o}~TGz!x8!O8k*qjGKa>V#|#D_t)JMYyAZgR9k+d+ zBFHNPl}6Pbsd9eWl$O3Y z9nab=)WjPPPtNvva~81qjk^m8-KyEQSG;#_v$z(oRLfsa^zMAu(yuRH6p2gZF)tW( z4_v8aV+M+VQ{3$@l}Zu2Gvp`2SO^4>aBrXf^)3&;B8JVJ-E>Y^J9ZPyu^M8Gx^z6z z*iFXT5uz^zlhIGX{E{mSK(&aQ4sUmj7bZugl^(If?OU+@dPh zX#a@jf>s_ZVzB{|6NQ)>5sR#wK1M|B_Q>T*T-va2OC7nB>_2kuBy-30$ z@z_Az!{u@Jpo%RmC$~7ecyu7{#`3u9X8FSvaq-W0jemTpZ&1$oqA$i5?J5l49v44p zsjsp>bdY0#sS>AOLM_>-U)zW1VBHJtvqg~9yu ze4%NHX@{Qfli2otiZA(;@!{JZ@l|db9{y!q{5bsY!npY4O=H6U-OHDpO=8RNHNDF0 zUD|T*$_(5!GPLffZ6hpmk{?U8E%z*ZgY5_?wlD7(Oi0JooaB(?uZj*VBQuj%D-+*O zW>3iW>Y}576h>^E(B=W-E}y@A29Z@o$30ktHcYt7F;$7j3uBG$0&Gl}zXshTH07nL z+VrWf?m2t_*7nA$glRquOxEkW7;b@6E4R%$WYIu$GhR3pO5Rf%`kT@Rc#G-GuA!8uuZ|4S&XzWeVppO3KpM<3LNbXDi} zs09X>U(R6UX$sy|orj!LQTrIave1H=b7SHj${&K4$Y=~k5>M0q$KjStpyj(|@i=g! z#&uRJEn^}Z;NrL#=7#<~v_;o0ffJ{vI|(*XYu1t=M@JqqH7VRlSW;#{a6jIhr0!uJ z^+*a#MH7Nbae$%s_{cyE z08B;D1%w2h;yXgqXa;UZ+xmKZjtAAo{Fb3&Q@4Oe-dxyQs5muplGCrfp9%t`6w7f=jM9;yo(~BwOIQw{mu< zzSNA1oVZ?k)0C#?7NUQ`=H#vbQF( zy}k5?2xLM&M)n)yM&46b?MQ}<3e#=KtRYw14a#hnv=+aJwBkqS9VKi75EZ{2nF&?L z7W*WB0ZDbWsPnq~uXcb>t>S40x(!6KjVO=C%_zkrZp()AWHwTzlf+F#u%b9PS^LTB zJTd2Z6QyaXJ$a(%<&P7Obl*O@|@iA?Bc5eaaUBtO{mXpGC~WIIq3D<~;lIYkUzieq z?lRvv$Rm%wIq}%c#2t`B9;#}*EI#SZjHK%`5|>}PV#Bv9H-rQ4ymaI!x1ZUgB>nwv z*{$bfCYE$bsNZ;Y&^$r;qJW?mKUsO+;x8Hp~0Pl zEh#yzGvO$?Ina0pj7#qB6q=X-{CM1~7K!v+;Lo&(m&`O(Xp4?h#->?K?MD{uv zE1XGM&j=l0vywd--H#=Y|7!-n!7dq{7;{ZpmUy)=CrRMr$Co8OKYrs}c=CsQ*S+Su zQP6M|V{0oIV?4~qNRq9w2q>Au)-;)n=O5Wa-J%8UWd~SzWBu1Nf7Vo7Cp*&-SrJ8> z;S}1mH824eNyzDAY~K7aybvgm#*7|=#@Zz)5|hN9p&E@(fdyj`D9aJ;u!^@R+_a$+ zsk(I+ZSqYiap|H>_A5x<5YQDAl=B$&k(CVD-PDhWz$I!CSLl9iPoldKSv?+Egx$Iy4x(gmVOCmg}($-xhjWs zURh1wSuG9H#|zF3*{_v?^h)E~#BQc--6j_A2II7D%PadfIofGAMe=TBMOHb`dIWDv z7BF#~5dCeZt%|EGVrv~l>PqIa4V#XrvJm%hiN_$%H12sCM+hTscNoD?O=V~e@g+_H zs<}#^WT6Evfs!lcriL1@h#CqR(aAKg@FrnL|7dR#N>$suNwf$gB4;SW155DbiqZs7 z7GvvDZPBwMg%lz#v6FnZfS7C65dBmPCM37znY+WcPUX~c-J8Fxueq9?BuR-Hs{@_{ zZ5K0Ib{*e@f`fB^=V{c5U>EAA@7b2XP=_Sr1`u3a>`SqfV^;dlwzwW#@;*oTI+lnG zdp*=;n9mDI&=oN;Q>bRqMlK^}o<AmSdgM-P1y8BE-dO; zfM|E_oom)`4;8fQ+2h|YvR?R7;Fwy(f#EuFD#s3gLCi z!`d%$3t)0N;~kZ4)VmB+FSBWli>)8Q+b;Jpg}EI28p}6bUv<{&VdY8Z4{}S3twm!A zyr!IFGR_x_F5s0&REgK!IX>bAhOp9WIgspB3@Q!2JV4_3Ksil08Y>iDCvvf6iKhoh zBm8VrjYq!IB4sA1PnqR%EZdlxO+uU*o&3>@j8$J@Sd@6VWliMw>o(*BYhQZK9~e^* z7?m9w38%=BY1JcB*FWPAoKX-s{k_cM-vr_=Dhzf8a$As-yCA#x*MYdc6>%pQ23Nx= z@{Ygyj#=hw9x%SBtuXvVT>Pua@y}QK#^Lyf@kOov!sb)Ud{-yOPpb4)o;tp0-}s_$ z3&VHE#UE4YYX)of&G@3Pk4;F2o82@-8xTC?XdS0_n9ovj(RV|1}+gEk_aXzgHzi6cJeFoECxF{F*l~dzsKY7xx^4j%`>$e*Gv0fH7Pu}W?FfS)U}){HC}GP8(c=(=Uw&ytJ5O)NqLOcY%~a` z-vFIU>yt45|F92l?8(Vh;Rnk7BtlW@NJR2=fuk*HP2>h8<`mv9V?~6Hk42RgZQlQz zl|nmpG<(@euuzuii;fdi%)u;_HH)1=4UaXtYIF9xf@EMGu+6A!)YuBJEmp3#@Re`> z0k%rCu+o8GB+P@Wr$Ek;@tr(oSfTpX*{5nZ4!JoZ{!x(F5tfPNh7en{qgkNP8Jr*( zF;->J1EU83)uS!_?Vu-AqzfOB8oo4j4nSsv_Dpg=x<|F!{6z(u(2PaCChC|Po;RHE zclhVp3$YZMVH8N*Py77aD?J6&4Y&4YLq5aEQ6&{pUi1cxTl8)0^@4?*=UmRhF@-5) zoUM*QF_&fx`8;a9h;kiiIFD7`4AIp0vWL5?KseQHdDg=N(a{J`_aajmg3yP!#!>tu zey*Vj(4w2rO+~{Qeuv|oC(b~#F1u_SzXU0bIh~@ zC^X4hMNClXG>DL2GEQz26z>A;OT4mX(Kkt@UUoqwRhekR5+L77ld+TDGjKCB+S8v&Wj;8?8=C^RF6***YjBKm$qMUTPD zNDir}VLedlIy$u)#`+PDE^@-+l64A_t3RNyK>K=jq){8*)^*;_BP})7j^$E-W`}met$%phc7JN#Q~p5wd#B{ykzG7F5Vvo1aB`2F+>)Hy z`Ps$43dC(LkJ|{h{^cXQnrz?E*eyQk`rb)ndnZmGzhc8XD>uAd)%b%iX>G5>bYQtx zAA~1=>iS0ncIz8@R>HZI$A9K#ZM93mq8&XZJk6hy{&=_S+BrF`Jyu*k#cBjlewi<^ zhCF!ZBu&t(PR(N@clO43aJ16TJ~<G%B>#*;+dp})43>~aB3*mvlX z!;_D)!7a|Z*p-Z}Lsw9;xGsU~^bQS3srcNDYMahATD(=#C<;?22RdAZOK#mq^c#nJ z21d?W9Q-%UR{op4tW9N*Y^sd_`s1NXm=Swc;paPX%{^jSrNa(1XW+qNT~3 zMaO(b7BfdxS2Ku*G=mnlAcx0A!UAELH&&MI10Aw z8`9;LuJ&OgCIx;|iEeW+0MI}UKCCWn*JhQ{r$(t631uRoV_@;rZ4ssa5Yu4*G#f1s zhbBx)A(zfWQ&GkC3$9`dKQMJn>$_`dgXFFL?A;4W&SjE8fdEZ1h*Mk92UdyGB7Z0` zG&5K3n1Lxhq~4p`pP>oQb3R&pW*HOd&{XZ?KJ_o`=x5PmL0d~B*v0Cg)LyR2C%{L6 znAMQRi4a@gl<}@r7VG5OWcF7}H;vy2Us91@x(SwwndEfEB`Rp97!1K{B&6+754=H6 zqDbRyeuhN8125C7C702tu|s@M_MWC1D+@9+UZyYLI?}%pvTm(y4Z7~uYx&h){jA1lNgX9C4A%yee^4bCqF^NusMfAg$?cRaoblx z)9U1n0u>HINugSaa!J7hqInPP)Sg_>Pwx9H>!uJl+=~_Hrr2u&d4@n^E@rdK%1eE^ zBH)BX%4&(HPcj!~$t8jpRZn^t5E;)xzQP&>C}6-4ncJASkw&Pv6WSBK0EiVw9}CrH zEQ<_uUYAWa+!h4TY?zkjNi9(_E|G7l5zo=&1QTcGm{n2|TFB@H!6KWjncGq8O$VZN zY*@AhL3#&8z5$@wcyF~>PPMD)n_3WAJuUqJ#V!SJ$o#e*Jx8T1JyPC~HNUEEseESM zh`49VNp^A+%PIpnUX#fmvQZAi@aF|Z6%TonFhj@eQ++cDZ4 z*nJ6cs0}aKA=E0Ea8UocC;_8|pu+R~c$X!Ju1i8oS)gTOts1l`OrVLe6;T9XLw&65 zNq9T66M2-7qpw1G@1)@cx4HX5BF5dH!(oD3t2SiCJSes~>xp8^_oUb=Jt?;QbQ8rE zx3$FL*x(1!P~w5k$nBn-31d{N6_DOz*UrM?{7cFm2pM|7|@sOdIeJwC^Dt`Br^OJ+x$#6CE>a1jDW3B48^`e zIAY+`#$uGhF9_U55KIZd?4!y|3u7%zabqc;{_ffe5R4kJOwk!)Flyca`0%Ffy+3{9 zsg2{){`sqE_ZD2+e)1pkZco_o?WbFQ_1LSUyY`+zP3KH8BR^86+-5^5Np#W??c%mL zXWi)od+ir|nhMS2(5quOBu0l65)CShfk6?`g=6ef$oP*3@kf(zprKF)qn3L1-Bokh+&Axq+O!ro}rD zzNM6EHVwb517}>4;h~d!kF}vXZc<-gomz#zcM-qQj-^245t)1b%aQ{&p8}TwmZVe^ z64b+MS(G@RGDWt6^ygx!aaOUEV@y?+K{ch46U`75gT+xYY-!AAUwg1;O#{I~c9|9u z(POL@njG4LBbP1uRV%SXGDzHm>lKZwEdm1Ai?~$^%0Xndh#P1`jT`1*w~GC2>cKcg zJukT^>0{4O_)=TGOMh2<{epXW6!5R4&WDX{I`mWM(e;3QPf6KhTH6=a4~Ndxu^wkF zbq@0yt8GQ(6R{7gR9Z#BK(b9l;}7#;J~~s zHSJwMv2C8#C9S$3ZAw9E-k<$}p#_2SvqPtKN$WW>ZTpC1-#ApTC5{Pih>O1^CH`u( zYIC^zzq2f+yZ=Y?lDrzSb@yt>HpbkLt#CHCW(%KJo>`n52-Xeug>Hmi@+H`e{4>Qj z8&GZ&pxiXH;J()%+C3*VFTHcwa|U>NyGr_ekYM&^Qe`^+Ei3F7VbK;x+-?(96P zhkLoFYl zBIP5bf8@o*$>Oca|DtZ)-}ZhmCgS%Uz1Xh7yWMivMQvByLVt*ig9)o@Dh!L^Nr%xm z`aw%1X1op^*|4k~Ts~Ni0h(}=!KG0-d1u>-_DfR2q9Z51mP`*jfgQq8B2;70ZrHJx zk3gfSg;xIeM#ZdEh*PBR@B()E`ET~Eh}D7n{##SQw`?9sg=|@Q_gNoy9UrfMKz-7; zo__zEcP3`9UoJR`g8ELf^;*{R=Se#f>f9E)@iyd@FL;Y5zy57}cSqaps%Y_K{?KQv zKjZ={L7(>=Z;w|{$fGvp7#p3aimKogyCEn=eqP1SYTMGF1*aDNf9$;rbXC>WHZ0*p zl7QqwP^ypwh=8{O)hc2z0fK~z1gc`hwoTAD zJoA~G`Al~69%dA0MAs^-Y`&4CEMNnnpqM6T!X+>x!lqlDY-W6x!A<1~#%;uEDV(xL z_}IBmG4bH(UtzU$p9h(zpX)y@yuwm)4%2kswi*8;q(Ayb4LZQH9@6||)(<_#t}5Zukj%8kz&_ObKJDpM2sfxCgosqKt$an(p`~ zIpvSPc{FQ3H?GfY|G`u~C5LW&BD#i)-xwP};zwz0uaFD~yoRf-yn%Es4shT&eO;o8 zV(>mx=Hy7si*K91Y8G0{*xk-HgbO%+Mbr-vt?lz{#%)%kduLD}#u{ow0+g1FyumZ6 z`zak|4>~yErwFLpN1$5^V3A>T*u>*1DPv8%ii#{hNh+ZK93!Rl4~ zjy@b7a$-S82WCMC2dV)+)GU*iUdkp4NbVp5((rY=ePiwjM#p$etG;=FM2!tM3UusE z0fAJ&Ek^^y)Yu#R5^Hb@`$>O%Y&iXSuoWCXa8zvs8g^jAUjlt&jEm}Pmhwwhn&KU*= zx3aF|9B^=HGcTE3lrp6#c|!D^nAoq9Vl6GTMjbD z-0)eM^*a($3v$NBbj^!A2zg{_X7P>Tm_6k&{}>+`j3T&)hYWi-qepP5+r*$!xAXO- zZvFqw44k#pO<9q=+x@WN{=elybOGoXSf77iMsZ3wQVnOzJ$>VLja}7T4p)+kyG6Fa zm1F|oU4Ph6W{r3S{veFBd{mOsHL`$s8V{9Zw>53>v1bDBh+yy1d#x9yBpt)+&p?q{ zS-0`W$m-swLEA&KRyGm5@h)>4(DyS~fKVlu6-!yExVXV5$OoU6x|<*)4JLAKtuR#) zm2#N^z6rQyK!$7M65nnUw>lr9Ub9Dt`3u;9@QjSt6&Pg8bb~w7fg|L^z)qk&0LSun zbjAAzF14q@>|73O3u@d*<*7w#Fd-L4Bc3&Gpi`g?HTGKj+c8T=p;vl-|M_dHUsmeR zp2jGjBTL$jFDN;glW@g_^=0Ef&Mlr^UpC|8 zyxK?bW;X>EyCnu5XhI&ABy!rC!=BY3p7~xnkKk}=X2fM)L1oBgoFF7unp0ICXegFa z)e!n9jO^yUbCTSUHOP@3vHvh5NOtCfjn{1XPtq=pVI)~!tM1v>ZmgPN!zB*3HTgsp z+T&D{Y5!Th!*nLQv+@%KT_xBCkFhP$9*0I?%=8+i{b6AcK(A3a?H7kZ^oHQnf}rdY zzR>P|u^(>$W_F69oWHRy`_rbn=!^aWNj%1#0zhyZ+HI7P00h@P{h3QvY8CEaT|-GG zafb(tKziBH1K#kYR`3kB%wVWt#uVyHl!30GzRd8`2@n95a7(tJzC7xv5JfR1#R4vO zG69hrqFKNm5^zNfupxng^)()1MAwr4F|nYCp*UP-fUOOf@Zk*aAWe=Y=0&`}z3$Ls zXkm%3MbBn%$1{1-Vaud^4IpJgj!A7K&bp*tWGb3Xzlaj-_I8Tet(B?S+ z_@ZKq_Bs2cmynO%kcI^AxubKv)+1&Qmb{nvRL;_tRe;9cwN^wi5H9TWwPXig)g6aCFst2$E*0?61zjo0G-g?A?&N7EbI zFDFyLZ^@{w{X0%qp4R4PO256R8s%~hkz75!- zu!*Mq&yYkj-zBMGk-pJva$@G{l^`m4gH2_Qq4}2`9IZJ^n&05qUS{O1Aer0>)N3pd z;bN^GYvO0xkC|g@VvcRUPZ?Xc$Y8jTlw}qdg=4O&h`BT`@&V9TWJql|vv^uKru$ua zMQ2Qi{x&9dZ)Io$EJ)fX7oEt9{tEt*iLr|k%MJyXv6TgsvEA%Z#5*(}vR^5TRE;H|vi!n3Z*Hsn7*?jE66_U+F<_*79FqdiZa?c|h>l!+O> zDDC*+_Dcy%eD{vXm5d$wAf!_Wt_20^7D$fbz%-5*MaMiQH;aa`5Fw_&E(rVDid!ejX19}n;A^h2|X6hAJcA7Nt$RESiMFyv%qg#9&X)=-wH{ZrT&hMh#f zsJ-aVsC^a0sMQ5AYJa;b9_-$cpgwKY!DVctv}J5Hn+&^1tm#{+~Drwb`3C6g3R|*?>*8MGcq!EVE!78R5TXu$8UKGxA+l!B$eZf9gKH zUv6#l3M-J+GDBr?$UD!jBmtB?_Eb)gBgWlhMAkACQM)(4F6#YOu#yb7WQ**#O1zr6 z!o?7bU;}@K{um=fP=}N)QYBW(88V0v#JIm>?_{YY(*QL(nH5kI@x4Aml{K5=<7*#Ne+UEVC^m-B7FdZ!(g0#lghG^Ux^gTjz{=_jTUG)pX^!T=K?;L>zq3Sy;; zk{q#!&~bSOZH%lWDP?CO%#VvwDj^M(5-@9gDuEr?F+T)MU_g~pIYVak`tOiM3ALr> zn-k%f4Y#UgXiJ}f!4W%xZKy;JEp96dM{Q5hFMx(G5|5MQ(_QFhJ{EbnEN1KM->|8; zf(lY7y)>X#T>^_%B24HtJA=4HT%4&XlNvC`G+%gt)nFR>*w`7?*KCzxwKtUD!bzf$ zjaa~5_lOY}TgUy*m-Xwwh> z@>=&zD~xeJM7DZUs6%Knu_`nRLv05tp{1X-#!8Nv;+|3BWAjS(_He;AGFcGN{z5&W zoc1{I>Mumy7#xDc2FYQOD<=SO=9v$R+j?>fSNiTirEmHVDJ`SfNab@^K-kpr+*>`0Xr zlR>I1N?Ul?P!pkc^rP?%n5C`9j@7|FGRdc<_CrDTk&AroBbV>GJ}&;v zw1o9(@oCqu+Pru5=8vnI&xc85TKq1UM*iQ`oB#fFQdUh)_=U0I=LgnzhpEZP)SXwR zWL4*cpBz}fhg?RkjX4j^+j^l-HU7&izMM=WW6tZ=aOj7qdFvTldT(5$TdyH)k3#?W zR8D;pO4%Aw$~F}}+3v}U)O|NpKQ=yfQ`+GAJ+qTGJ&{wt1OKbasoyd?Wz%`x>bv6K z&K+F;-0Y;QEgiKJcfc{WWF&&K<7I% zQcBL_Lc6wk__06zcnubC_D^0HK7NeWI6sF^W2v2c9GgnXcZK+pPe{A|QWUw-cG2zW zW1okHZ}OHyLM+lWbDq?HdD)PV{_C+U>=#!b7ziWD#cj1qb40MU+w8O*oz#G#FmmZP zh}as?fhMa#rpRLo$+JvwsX)QA1(B0WFO^IWjFRt|+U^65XikMyU|XIHc|RJQd4AkVA4Q zH$A2X!Kx$sax2NWB$saiairi3R*tlKmTvG@R`=Jx6U|)tQiMt;8hqdz(O;eSvOkDA z|7AR&j{dtypk>4h&VK|!N+pL4glx57Mn%}$am!q0^)#d^Kh|C-Wlk9kV1`O?pB!CU zu1(u277S@yWnQ4Z9a|cqfiX08Z!?K5k+hoecnDE2o|Z7IJq(({MjK}G*Tj~o3GU## zVI-=J5q6FaEHh|=x@#w445Q3Ghn#uQeTSp;jg zuLzfFc~$ckYX>pvYrrCL533BoCCJ!Hl51Zl5_EgU+VNwD;Z~>s_TmxPtQ?dN@Gpwk z9K4S+4n`4Mw)R&Iz(-64o{{shr;ZCV7ihFpyIU#*8w8ERSj0wQ$X>goMTc`vHxkcqbd^b=r{JfRPatwDm8MlUi|@s#3~p)+UPc zY&nmy8dKBB$x1J))tyJ;S%Yp~?Zq!!&oh!48^2)Nw&|HDL1Slo-&yNvAcB}FnN%JH z<8X^iVYwMJ$zOs#3&@l~1xdV?9UAB5n*Gr!jc)8*-e;LurUD0Rht=Q$N&Us=o;v5` zwt!Eg<(JK^nE7R^M#B8s}o+mOsOZAch055)lfBZq7D zTJdA|(fDx~7POs9ldaj*^T4eIJN>U&la%}Om%T-hprR_tE^=}7VgpbO9f;SQ@)<#w zdLI9BdfMo%vSW6-vWR?GIsL^dS)cMoRLhRAsaMRfJeIppSlFEAs zB@!p;=R=~CKyKaRY{rInyZvcu>cFR-wq{Kn;+}gdwpql;`vmi)FOfn-8Wb)d)*5q4 z^7Cr1qAp3XK?jNRS*2Xfa8q3ZilgIKD-T;tCo%{&Gd1mEcVs*ajOJTDh0fVjCy9u% z^!|#n?5j9vZ*S?J(s3w>JF#|rwG?^|OhhiLI3@h60#frZl9!Yd808|hg-~cgO&TvN zdJy3-|6#l_88+6|WT<>6ew)%|z4@r;Mo}FXUld>Z)QuF9w7r2+wI6Y*8b_?)&h$fL zf-}<(Tg$;EBlV9GHf_8t1717}YD&T4IT#KrfH0zRdl`=G*(^jk4vP>oelpVOMj@m- zUoOHsuxq7+SS`TplB%PPe>6Mb;*Hjh0>`Y4I3)N+qw0Sw=unW=38)0oe%H&$DEH4K zhY%yN?f&pJA3=Vlke8bGwxv)j-b;NW=G!SmP4Cry7OxbmzD)+!997>mW$rtsJXAAp zaOr=5!#TpYg|8wSr79`>gt0MuWa-0Q;K_6?AThr#UicVHYeXlTzVTbLr#u2gMI$P{ z%Vf@>Pz;#5z*p7EwINy1olYnKd0{}&4-D}2zbYRk#i0?i1BGV)^^$@q&>^H?^m(n! zuX++8utqz8Z`9-Cm8J2-$ihniEhBEG{ji)RJ3`&T)v$G7!&1 zN)PY5rf%Oa4|0+Eg4 zO8uC~#FVj4_1PF~qyIUpk?jAq8u|Z;)yOt7#4i{-X!%$cWRAWgGLFT8J6Q-x^Y$}# z+jB8ys&86N#~l!zc=2k=W`u=g3Q-A)$6B>UCL>0w++G%^U>%sQrp|e+LW$x$>t)&>DUTn+f7lr|Ufw$5_v@kn> z)jsk=09=(#S3zrq)nI^WDSJv${=2*%APwy8o_#nzJ3dQ_FMWVNb=wwnWBa6k5@p&| z3pU{z;CHYA;!5iuB-@W z9im0&*I_N7(0>WQYKSw9xyU<*v?5UVi>Nt(L~H$vhasNG*wh$pnX6#ur~lYC^Ll+u zCYdRBrb4kCcnEzQM?@x`#(2-RrsW!TPy+b8+9l1(egTdx{VY881)=MdP*!f-jCNI& z$2_vJ@f%_9m(YQ!?W+F5*GcJh;?9#JCm76*Z9NMNOl~44^7lx&2uHr|;I{@U$JSfU zkxH+rRTX+%K;Av)Jj6}xoO4^f=10A98`~3k17QU7g=0V7&7|+CkDt z=R>50z1p{75-AX(r=nMQZ;Hdob0UlVL6$#S69IA@%~D9Crqu}oQ%|8UMGzo{F@Hdw z0qRN5JcBgo#6$3DJojBp&xL5#9grdu55e-Vt`ON7Qtc1_W{Jpgtwe;ikk8&9(94yS zTjj5jE0;~GEmn}s@Sgm`zJ0Bw;VGe(gYU=04oZ$ayDF3jtC7ShMLj1(Uq@f;q}boT z(>0(C+hv*mY8$rGtzT<+mK(Ccs5&lf;;QDMJx53X3kt}Yd6DcZLiM-6F7id#MZTXD zN`zhHLn}g++eSyf7ZaP0|6LOkn+VhYpY{zU!tMWV{M)Oe+bnI^uAVqBuV~NY=tVhy z8kT)vKwh+a4#bl9|L55$?YSuFOCHiTXGtjg?Qrub=pvU7tZ)2TMsW|=j(nUGO{KmB z3pSpRcK1cz9+A}8tnKNZ732?6@|&>@mG?7tNk6bOx~v}+P}P4iXVxR&nlh-S&8w3( zxHY1q>d$0>-};PRZ)UJ~GKpVL?BDC2)!p_TBjKgL#s6G`7qfyTlLVXfQj;fYtKxrY zIB52AQ?`<3MH((en;)yzK188I3a%OT@unjQzEBOr9)488I230Y_M3zv6w75pP-MR; z;F3Jk`L(>%(AtF(X<}GQddIOzXI+onhM8Si#aWk_+b}dl>dnoNUK8Z2ATCfw!1mWL zU)*MWw)O1{wAOJjHKyX^VQ;vG1Fb3BpIyT(l-u=V*A5mBo^IiOE>y|J6htby#Y4qw z#LlhT0uUSJ7#&d39 z(ad%7At@_wG@g6sLZaZJ5BH64-L^RTe>lR+e+{kVE?OR~SP?h$;^ooD`XcX%kJT^j zX5+VhsU9?qAN(R^#1vm;jhd`Sm*EkHDoC%;f-9#e>a%hXDqx_q53DW!wv#zBUmJn3 zceDW9rcNZyF^x;WZ9~0q+hP)L@+~ui<&Q!OqM`#AYC|wdf)Wk2MxU~Du8dOuGm;UE zi#C9^jx7Z@TqWp)Ed07-&_#UOCsUN_(}E4o`q=?_bVOs)_RkF)u zsnE}iwsA|NyL+qUdtfkYR!v$btj%DgF+g5w!__KuwT)7NNVi)W z3@MhhW<}zXWS4buER{})tpx%w5i!dJp{OAlLE}#)9?C8MRhtLih9|cz9+N_B{+6g( z%n)B|b1fJiC>w>;Q;mQ^SCV@J@rE@o7QB>wIlrU+LZ|7T)@lUdV}cx*{(Iv$lsT{g zdg+(o4!vrl)X*_rX1wE0(Ev?^q1==*O*dg|O)>~*ZJpI}){D7!p#t5qw%m}%<^11? zcTxa>|8B%WF1`775l&#^)1_CYoTl*6QG`s7c_<*&Al9jV5sTdq>EXNX-;g+CEL&%1pI2_bW5T42M`~`lJ1?)O z^SoZKJ=>N%=k=;9w%xt(zKy@W)q&JFjVGOR%=%;N_eqdb+Y`t9euQnNAE^m>@IN)U$2k$mmi)?c}%osdvkdG z+idQ5alc*@`!TlKIlN`?$S=EeeDy^7{-xwU@)j_Fhdgq}XVxGf{JN(+Ty>NA7^%wpi?l*~Ff-Bb-qKRbMlzA^@e}q3(=oDq7 z%&!r6u*MUeo)4NPCMGUpKM!GXc7EBSOVJPxaKc1~f^;)q!*U4pnVh{+j0rb0{QbhB zvi`!)(C$^L8+Cjb<;=FBjg=j3*$KF|Q5on_C5p zy26qCRQIPC^3zq>qZVci@U*+dkurza^*U5 zt;h{ml+YO+4f)*?iC@?jl(0!uB?4^G7@W}=iHh6wlPcqfa3 zrou{>i6Nw0j{e~+T_}OMT~+Y~ons*H0Fpz{ zz~MogkB2|H$a)nD=86(2z)F=xwhgQZaLJ^8xmOxlA!j&xo>EAxK^hsqFaVX|v0w=; z9D|_DM`>?0e$6(2;qKx<8d9ig-fvWBY6>^LKGI+)trdc?`JX>uu{*He^Udi;{z39E z?K_CUyn#(~y~j*6a3miSy*h6@$FH`hSVJ+XroyGyPO01T`|fZ!ekTXyZ zF`r<2sQV!)Hwyj;$(}V;fK0g#U1<@~?{QE#4Y@`C2Yu?ZLVk54ODwYg`_<{nQp-yfUsSVqD< z8SyKpuG$>)SxTRrluhe$!p8>IuY+eucIwWnQ?j1V34by++|qS=aen{IVRMJnK9pHJ zF&y)3dCW2xh@gpfac1%OaLm2sF+ZM>6Rn7e{e92aUoQy_MN{ohCl|pZ|KXU}sY^nY z=%~GCV*EXQ<8PfLI%role<+)3$L%;3o(;d6_PtfjUyg`Rt6b3hWGG?fk_F9=hZ543 zENlK~L|j_ls^<6bZ;vmCYHcF#?tfY|+oUZYjtcf1p}0PKN0%3gMDp+)IV;%5f6;U8 z)Hd43bL6`#_PtCrk+0_ncsSvS!KX9hA)bY@o~+@oCDd-(g{%;O3YzMsx2?<8KwFjbC+_GmM zo=n*l=Rfi#^j)JR)IMM-w_976lkp63o;tUYhj{zo1JjFZ!!4D2`Rz3CC`R=+iO08R zEzGEy>)lU#?bV--KkDxiiQVJV^u5Ws5APfgAAgwm_YaG)+FD)}!q*tKQt~W5)reE;ux$-rF<-nHOiF$QpAB#G&JWw5(9;}Wmcd8?|9&xnOj>RE8Pz7Tf z4OZ`Zx|Eqkv>0d0<%y&LV7O%Xm4yaqw^V)xa$mkadoUTHM4me@tO5_@^+g!0+1M!p z;Sx+F@?=|3b^2TRwi?37bkdmToI3i*n2_)pRcVUuPug=4_Y>G*(|>0$VZ3g8cC4rf zi{4a@ZV7bKm1Sq;HWa3b4pU8OGPYHTYPl|5@jo)=BH68vUlIS(x;BRP?%gM+vt^bO z#a=V-ozl`rpYazxVIj2+{nXb^tNpt7kFV<7b+Q?))-J=><<~NMJs$Tz1GLs-DjK-! zeIVvn&lQp_Yp|u##+fy9e=?o!`c5WVz_|-t)Qy*YzMx}n<1g?LeIhjah`oQ>F>)MZ zRTJW4ykzTU{Dkf6mTt}dQZv(ETgM_%6o4v6%79z=A!&7H4b%J)G4n$Ktpvdj3C1j$ zVr?y^0%}u5MryX(7Qh%UNNu7DcyO?zfCk?1tp68H24o9((>l~^SYeg#0K+o2Qy}CN zXg8}Rp+5S(L{y0RcBw@6gOAcsY5mWW!L#((SB=cz*etl3YJ-uiKO;LEz!d55S+tMdWoYlPAB zzDFxf=})a$9eOgzUb^YO3vd&^w0*qfkBq7M3f?ZU2wRSJ?aRTUn`W*%{J{kxxueET z{xcm^u%f~8M~fWsLY!im;#4qi!-E=61sb1HyXOqi1m3yRa>f_S+PjMa#%CuI;em&p zz|QLXGqIxhSg`o74mj)#>wVi=H*&%(uBI*kY7*-Q78d`4VRLRB>a}=?5jhxNtwpBF zttX`Z#X3_s3YIBB)}M0}>^cMGhRLZkgJL896gQ12{8X54uxA?8Y{%PZYrKtju6?BA zPy4nNl0>rj$KjY|6)}(IMap64Up{15d1mpQ;h5ViVs2SGI=U5h{YyiMXr|q5O3|4U zqJNBuy(TI4@};4C^waKge|(RO_+2-y+T32%d?P?zdcr;F@hfjwwfWHM&C%x-(jYtRKi$Bt_+A)d$!x|AG817hJumEQc;H7c|?!+aq5x)3!5vP@f!NrFR1TM<#EX zL!4IV$FGI%Wwqb1iix!q%r`yDuW_cWO_!QMxz)KmKvo%z8CgBT33pM^#%cwW|}jvG$RXNP|faP2qsyLQSCG=pl5b5yj@hU8rdz%P%pk z?H9sp0gov8Rz+FUnM?|nq*s&`#^Fb1tB5%>^_R|yw)>IRwqrW;?da{rmT*BR`5(8a z8nto&1>WTzm6s4h`zs^?2~KRuic}WedB5J?!NZP*hz{4m%R3|_qlTvQNd zOV1x0c< za2q%8ox->Lk((_6ZryBEW*up@I>CG8+{Xu!2H;V0JBVRJ=30Jt_I=qJGn6j$+Sz3m zy_tLp0?=Ctb8!aFvV>=O%u7+HLGVzFG%f#VyMH=bTsTs@;^Z>|1<1?6G2@SITQ^oB zj)wHK0WCul!|XaXGf`qvC1BBhc9BNcN=Hpa7(-`Sx|Kma-rR>8b*7+fqcizwzBY`l zz1SbtNFqnho(n!2N!Ys-nmJb;jH+RO25$z4l&WCXW@0Y6!Lu1KuY)(*{FF*pxGZLhV#6JhEU;L`y2 zks%0A-}(t9xT`AmdawvJK+Nn-b9<3(I!s@QcRU%Qr)F9S|tSWZ!5)wVuU`l8L?Y~c<$3?gqvE2L5uX- zT;paM4B|HGpAo2=uH%u{{1N;`a#ISuCS95n0!k^LQ4NR_mq4vyvHM!`-s6JOF7pwv ze1dCIi6Jx1z{xY}U8kylHQO96aUKlgx^~dpp5`N`Fk^qf)U)O-)C?yo>hrhh+w~sX zHC8Iuib;(#reGXysTMndW^g4!wmrrJo>8*hj(g>MsQd53E&DYn+*Deh%8r0o%QJHz z;1%mgjI&ISj-mNMGZ?w( zwa}#kf|c>>>t74qvi8Y+$FBJH);{0y>5^S@XmV#9A|sdZ(7g2I;1gFY-YaE{ssppc zaWUJu%3G5Xt4g^rV=9V-VwS3V&+W zzkxrK>M#7N46vLm8r{h?n9$Dkk6Q?g%A-~-Xe$R83%PGxIyyYkX?syvd` zF=Bapl#qEFnwx?5(FV@u{dgY{CusOo3S3zAqzFjsn3zSEq9RL=CB(sR<(O5ZYYToG zfEgX-6FDRSGcpsTmXJW%M1U%rAwvgv2P$2tjge3x7>cO`k0#UCnt&9Hn~Iw?;!<3a zy|*Nob%I-dt;)LmhIOvN>sp7^oK1%4Q$?ocikC1M!zvf7n^zZO1Lm(Z5TTKwkX?K| zsAfOSq36df75P>+Nc z2OLtc={~R|X)gtvZe_dr9t=jw#`0T~lC_Tjjetv5ILX}f)FqH?o*A(e(r`v@#4Sdu z%3Y@>R`-bDj99%?l$f{|J5O$3^DCuM1D8yHnHG2gi}GsMBnr*_(Ax6#yUF^j){0Y- zM0+-bB85tLJ`mAg$#smO^&yg_X#l$U)*>6@g&p^AiMLG5P@Eb`${oIuFPyo%J$8h;F=l3$PPyGJxKspPAGl)fnP3;_a$23cdNgmi-p_1g`=({@kjU!uGNB_|`90$i|HHU>*R|p`iL4$UWa`kq3xvX( zaCt8d*G3{D7=oPC+*_6mymY{(l?JxzDzVjoKN`K(YM}hMVz{&`^FnH1)LJhoBu&>8 z<`1suZSV~cTrL%1&__miTw2&Kz-r-qd{+4il=yI0J^SgoaHM#Ef|Ngv8Ho&eTfz9L7#!{A_nbf z>9^RT$fJM?C>T=4fIt|28Tx2ijWzr-1adYhCQcNok|9c(;TasX>t>ONgZ0U9Yz4AC zovHD12fBh5g zS&FPY`rn}qADZ9sCXGMY@>Ufz8TmEdhvc31RE6Ai#CD05i4EYJb@#`j1{;{o=?xRk z31nfu7P&4hg(npabi^AHQ8hg6rT$%|Dr~ujoo-m1 zvfYu|f&6iXyUa>VY2J9ahDvEYG`IHv9Z-^d1P~b%M7&2&zQTn zx*vNGi#2tNN2YUx$%r_aH0uzT0M7Ia8Ap@^&m+xs>N>wTTfX6vzCsS-PH-q#jCOFE z?Z5(_IXflKVr}}58BAw}ORZtKrmfq-H|sLee$lW;GA^D8Z^)D>MadJQ@5ID@l@z;o zX=no!k3FUob^mVMN|ddoJkcDEO*oK|@McE*tv9XO99OgX%qNl>Ph|IJzvYJGKdGzb6o>V)WBeajL*M@#H;mxk(MLoHh_i-gdPyDsL2 zgwKyoKWB8L@Qg@wdT;!{v#NRU$fT^rIrWYAX12}l8EWl6Il859e914m*53|Dm)R|{ zCJC**r_FnyE;O*55ZJJLC)VD}bl0`pLf4B+4@_)ZJZawiy3R}Yj31fVKCxTwVZf#A z_xiVN8_D9q#0T=){*UKII=8>$c7bd->t$-F6`mdx}NqKZx2s|FvP z5{D!LvVxw*(o0rC;ubbr5xu!^zXe}m zTL8|Zx1l{%kFQD+acbh<7-iiQW+i9zjhxK(%SBb&3H&&Qe!LIIFaCbo8tj@%^?EI3 z%!25({QUkEdA{^ob=o){77qZ2B;w%!CFE^$NC{aNsDvCEsD!N3Dj_p(n>$tjt|3T? z1=4G!b0>$(&LOOpSA4wz)MhB`b~AOIUO9D`9j|RWkuWH0z~F}N-3Mh29M6pg(km}E z{7&9B2+n+w3EfH2>?xUVb4-Bjl(}h;i?atRU)bn z8?cBRMwRA#_ zl^Mw55=5Ir@v$s4QL+rN)&0Sx6N1#z2dE<26<&o{^D?!qG6gcy!nlH?lX-3}ZAn;w zZAdgbO9)#H`Z}d=FDv4rh->n(q;_F zLp@=Ffq+>IW)G*OMFDR(1> zP}Y&nBq1jYV?e>>^Rjo&B{hRjt0{%msZhW{!vZ044E7i zWf|(nuLY;bMCVcuX?js_;4{cN)yYMtjzIb95^xg`vcVnNQU47ob2``ZZ%n%Xoz7|_ zC1x&i+gGvTvsInhR>xSCt5BP7&Gr3`q`R|6R-F7_4`*Y-O3nMd6sMDq=3pi+s>~$gs$e+Hhv^v~W!K zwbvG%F(LZfnAp9Qp$!*KE^41#bRsYME0nk;#x6=MYZE;8gH-A>uJUlkxp0tFC zY4Nvyf7RyZs^)&N32*jGSpUl|^#c=9caBV5J~Cy~s+{oZvEg41tUnx|T5x3wg2nLT zm!4lcJG1yovVV*@Z+v7%FF+yumoahXAo!3>MuIq_co-PA5uy5f;SBjF*!ch7q|i*@ zwr#*|w~S57`guoM%3A8*itW8IC8t%cwrJzr zhlE4E7gX!nGfMne35YgoAni@G!JM7}i4W06)mHZiAOD2!)c0U$(xX4SY@~jt{%jC! zXbru*=&YxUda`!!%&Pdag_f@uzecG0B^pu-n-HF`E}T@h>?Dy_dW?Dz>A1#X#{ zO&?ij_|G4?t{Y1*&Jola>>ydrskMW&ln2|vn8Jzge!igO$9hi_2^8gFkIdI42936I ztg)|-6PFMVH2rZr89(M0pREwC@`mh-OHxec*M}{xHS>tc!{<{wHqMnz<1epOzDCHr za-#A6?-_qm+9X^`Y1t2fDf}L@J+NA_iPh4Bv0AahYMG=<<+I+7)zV#9ZHF7H6yaD9do$eFK=VAv}^$z^79j50HZumroz5D1sR zF^NpJ0%vBNHh^d;z6EEj_;>RkZTAE6=)U{`9CP(q^z$_t$BJrYm2H zkOx?c4^9IsG?#PQ9$^7CkGHIm<76INGtYte6v3?GEK`YxOQeNl;}B=`XTHWm*o?r8bk*O(-zpgl0Y?Y8G6=f_ zMc|r?S(YC>Wh z%D=A_2{dO2Y~5^dl0T>2Q8*Ie{qg}a12Y;AC5)n>3G_9a99-gYrkh+3}3554ettOI?OzJf~6uT(+kdvv3OdO zoZ~spT1chx&`cP?rRlR-_$J6v7Xtw<%`YcLOCFj_J&~+7TJaEUHrD+{GRkT{s8bRT z&78s|;{?}dWJF$A!Dx@dik4<__|Z&#b8it#Sq5iPJRx&G{LWLi!?FJ)IrfkE#vJ?a zZt1xBf&9FpuO=6LnHOCSW-KvwP-ST71(S=SdC?^Rbve|vs&GN4<(go*v)Z=Jkq z^E<0Izg^XQER?VwE{^$N&Yn)$IU;4#3S!R4yniD%TH84y>6t4Ss=?7LDQkI7eIZD- zKY&!*1^MGo<03Do3~9T4Y)bnJXvp@jctM`NpGeb*%~{VpO5E2Em&8BNlew_{%Zs3R z#HCl4#Ak1RJ~7&QBmp8VzpS0fgbI* z)EfW@(yW5B;6{FX&U3FElk}pmUw_9}{zi}pr?cN2SkYEnnS;UIy=6>{0J0T{17*ML zj3p1Iz-?vQSTcMZko6>X2yqzl>rGFI*Ay(@LEjap$CSCqEL^IJ%O{V1)P`-_6QAi8 zKxXW1G2o=3R`HP;fT~M(H4_PigG>|4rVQsq(B1^L)DOkpI&h6-oZ!fh7Z4N?vTxoKbQ8`Yu=l` zWXJE z*>8NWq4lLlDB`Jm^Lq{X8~I0$smOrHBOlB$D+kzqZieJQQs5O~8(+Xw5n&rMn}bUg zZScIHy`)KtIsUy4ByO2m%`-9{SEvK|qHUg*1_XG(gJT_a-aZnmmi=*Ti3l~jNppR2 z%WaH#g}1gXh8C0gkpQ%eAsiD_oyqd<5;7E$mB%t-mKdGCg;S&7_#qZj(wuPNK9m`- zIhmOMXp8znr{_5!ync{KBaA_f+5!}k0tzeEK{K=p^!v59wJ)}A0j6|z&Jbb z2n6EHd03<43=I-imyEuc9Q$jLpTEB;rQh(U5*%=6*-Qv1YALnngP690Rnrx=VHFEt z^>?qetbiRAw#iCKw?a>>>gmwEIJOn(*r^r=3foQzEe8mCMa#zTR_SZ2kjAx2B|L^7 zFqa!hx`|QimLOG_Y8$t2%0<0@X~ zNwfjbY&z!sxTVo;N+i!znT@dK5zMpOdYmILG)@#+Yor$kNIN2qsjJmLOw2zOJ zZVxw40CpUOA4MDOj^X(6jd1gDTs>b3B+q^O3&~FJ_m z1|^rvE@AJLA7|0Dg(JgEyQYE@O)9y=^DdQ&`Aoe*|5cHt$u# zVZgv^LPlxcPe*)q1?ej7u>@4i95Y@^a1lA+#);DD$26iEfSvjkgxNB2t;Z@l1hC^t=I@y>;4)!$Vh0zxH6*6SD_o{g!^Wq1Vqc{^sD_fLMZgw?JvSpv*in zWf&cdevkzT8|yRt^+pkN1{-kNrn2OmTfsOW5$OY_O}k=}J;O#{dy*LDEQpdHZKc@!41LU)UZ^iU;*pLz!&D<=-X|Q^&L|)r+WL z2Y+&Yh&E!fo{k>9YlacYt^H*c`60`m?`3+x@pJUq_4H09Kkv2IoL?S&`&6ZScXM-v2JEm#m5@*)u?eWmp7WX?z}B&iMdPr10d`Km=S$FgoWB zAUjaIBuWlS*ix>g0$AJEAiHT8nkh5tGbS$?+KP}^lkDfWp&a_mzydbCMhqcW#uBE| zxQa@~V=BHf5dtp78D&&z6lKh%USm{64-9K4c;Vs)I&W4?c2Vy;=e*My=R^8nA>)=h zF<142hPgcCjRhU5$zvYZLy9xWf@R12qT5NsQcJ%xOr3&Ts5zrU5+L4|zo`H7$m@6~ zbe-_D-Ni=nqNUH5I7ikW z0n>7Zv85O_wY$nqMZGq>J~$pGDEI#c$7q&-|3j09Mfi38lZyP9ONMTRovx`+3SZYN&?6N8Cw zYXpv<6D5_}h8AjYag^0{-Zr0aa$-C+Z;v#ND(&n@cx*u2Ti={M@{3Um`-eXq8-8zK z{q0aeW~DCAO4;{7LN|cj4hA3YJ6lLsI{ME z7H2^fIb%Z6p1kPKK;HF?y&Pq3tcon!lhbF{L@H}*mikCZzm?3)i%27ql6&$ZvSaqG zmCKqBj)=Q;!m8%|BjR?U|60q4_+5`LXby)GZhd@N^Y#&OT+8+?aCfL>n||(SYQ~-2 z-+b5Q>Z30zEeMdA;vwTG1-L;s}18686*Kj z3hKt7&_;Yn?47ncO)_J%HswoaOPz|6ir+3NRuw1<1;qT*1V4#4Lc}3d=jq*}MQx{T(NjHDsGtzfz@fpAu$u)6!G6BaSul+UL z{MoEiQ1W&TPH08xjJ_b0baJ1+GP2aaicLn!73K4&G|R z4E))EO|?Z0m;MYs^YmsdRmQ`2-E6$y^WyzwpsTgoGf#e_$ugBj7?T_Oz6BH@(3Sj& zHm{D@m)e_g{>yxAoTm1bm!W`)693wvm}X9n)@*#xC6!ChZh4u;1Ax^ZNunwDuw7V#xhx{Dzi}Veg@3CJj)VPv-WE zFN&{w<3eKX+P`yAeD+)XqXC-&j7O%KXR$VLt~dze5weQ+g~3y|wQuX7GgM7H;=N6DVCOrN(rYH#3Y16#Srf3TOfZyP!&>tDPUl ziaWm6Mq)Ux%kpaUaI!dm^#z#R4jw{@Y%LO0ahY1u`uu`n6{MNF zq1#w>hYei1TDln>5uQOeO>wsyjhJ7Uc3Xf8B1tA``0FnO2c||ugFJER=SF3jCf*g7 zhFI+onNW6PwBX81v?f?U$u`YjLEj6uOct|oh^<9W7-=cJ_R&UKFtmoAOs?2N0@+S2 zsbL+~Y~L+mJpn7dGjGL-#Eo=n<3G?!ogI5_@3{X7cUnNDxk%L5RQkc&LXqKxe))9T=GTyc*@3+S?dSpMQ7xswD$&$_M;H!yo8R>+SK=3$$KOl-Xpb-fO_l9 ziw+guBX#KS-FRivrt+Nn?7ex>nZh};?&F+jDxl@0b&X#%UA4ehFWE^Q*y;vfOtT*Z z+g*MH+m{z~%?9MVon&3Gog~#5 zS_nAaxA>A_Cs{ftVRUY>FUb}S(od$7&wtui$&~BLQZ3nv`%FM%ZG;jrRl6oY?YYJu z!66Xa=`Bm|48!ryOgN3%&cD-)Y>fu+m^Kkmq3m+1+>wIoNs3SMa!cl zE8;3IBI+$#*0*Ah!#MI>#L|JqZKvqpJLS5qo^me&t%h>J%x0?47MB*Dk`YNddc9gv zp?S_xehuRy>LmqtV>?&WCA#J0OcL-eMFN?L`uUWJAUrpVgzf3Ik5+Y#;KHFaod}IUx98H#BKv zDVIATFUMI`0}IGuovG%yNQ*!%Fr(@!Bzm|GTI60VLuK0`DoC%P7&FV4XaW<^(D%5* zYpLEA8^h0yw3*cBnN)$AORpbaRZW%$Zn8mLvOPtrz?{3xQt?iHvleJFc-CU(b8mD@ z4$QVS2nSqC!oVC4Nd#N^JepsMn!f~~VHgU=$;f_L>I2Phz+jx5dkLp8*|Z5FMXg(; z>|>4b0mVk)v)^v;Ds|lBCFV8!wXTO$2@q`pke?V`Fw_@jemO?U!Eg+f3KLg;nML;a=J49DkEVW8cy0Qe{{UV) zd-v(VwJjfxGMZfLjsA&WV>0c!ZrlsH6h4vF_p>IjbZ7sT!UyQb&?{?NQVkq3F`W%X z+J@I3=bqaK(tGtF&yl-Eyv{y`KJD0AvPg9q`R*N&E7`KXPwt?CToz?s$FGqvdlwER zTzKcgQ&NUA=_}%_+}0sD_cE3+^EPW!w(7X8O4Uq!g-dsh*n{SB93t3ppy@Y>quXHg z6zO(DuYLOfb!g3y^W$g6Ip%D?##|xC4RlM#vlsa`Vsoh=_j<8xr=WtwE%~H1^^rje z`FPHKGo`D_;;7E=f5)gt5EEoJY^1T~nBp;m4BA7N=(A%WWZUetvn9EfoAsD%pdIc5 zj3(})8@-(Vv4aE`atPkTJpR>*g-Z(|;0nx>i7;4`&X;*IA`xZDN+gMd9IZ;-Em3c@${DQH;*PZDY|ie zX8X6-G3(f$ixftVzsY<_Q4IV7n zU_=IZikqOp2uXtwWYx{ju8X;NS|axnG=tHA*a4%h^->%F2ioqzCbnAAd3H^i8X5Q( z2{o8UxfHfa(lG`EylB*@$IxoV<00uB0csNY{KtF^CkL5{u!i|@@B+n_NGUC`*u22T z81$tKgc0tl7OBeScdmWpYq)S~(gliQYf@j0)POm|UF;nJ==Xev(3(hDKFUy(ReWVM zo^U}g?nN}rHBNW$6?{+`0WNY(KlZ{ZypgZBJMmkQY07qk zf%zS5#6I_e6JloDAEVh2aEXX2N1r=uwG{99Fe!Oi#Wz2y~8e>rxx8V{9Zz<$a(rWONU!v^Y`Q z0hXbF!d`I#8T-0N#5}DSAj@(>eLD;MsiF&wb6J?L4F+2CM{ug5U|i>}!`lkOQ$j7p zKMuz%tB83tFH#OO$nqh>$}@}a49DDF5p&Dh(b26idt4eygu8#YDMe>ai2gAq_L`*F z%a@iF#Y`#ca({e}jQCwQuG-vQ)qG=Y!ei+P_oT*GhJjrdEV1W!IKv)Nn37o)M)ye~#G*d%X^ zW^z~#d1S^fBt|O?#%MzWF&aEE4R|7jk|BZC*wUn69NM2nr7Bd^;kcj?bctI6HIV7@ zZ|z9MIqc1z0U>h=$j*>iOVotZiA{tlT>e>u${8=%`0w1MNWm+W05Ml&z$oLD%A3Lr zz1rqn!|J;7R0N$IcV26hHSD8HWFt>|sgILCTLKney~N*mWIJ3w zo||)r@$d&JJJ0WTH^2Wj%0k%$V2TH`5)Dw0W{Y0V-jx-=&^WDqX^3HXCm}R(A~^M$ z-f58$Iyi$4ksG^{yZI z*6?RgeVP0jpVCrbryXB-cGD3|(py)?S6s3R$zrUim-3b3nQs3=87HWR;vrt}n$JE| zsWD-d`z!o0Z8uL%DS7HfQpL30i65WgA9?nCy|z^qLRiDIP8RRvLB~mjU))x~`S0dy zypJfa(0t2A!rVaG7UqR2(;xwpBjZ&FrJw{xMHWA5sUS?vg}`NO%56pn?`=I<{sWN& zl8Rt1oCrPRUpPUIA}2O{{v~7}+GMwwzTlec60cw#SSHmxY7s2?jvT!%sS_O<=?-($ zZ14k;%hKv+4?-v6y2Jbpx~cFxBqp(&t%@h{eY{H?t-4seo!nbp>l7*U(x$@0a6>}Z z7}vi_w9MMHf9^B1X?pwt%m&7V(`|N2o#QV~zX?Ae=l5=Mr7Vk-(qZif!%}g4h6e0r{l>%g$dlSSMLa$?f&LDK(O?r4CyV)43p=QWwu{# zEPpkOnY$fx9k-A-h&ETJb_mt1U6b}>xoN2&dwYFOdTV1l6|sdlm~bj}j+(nIz)*|i zo9FaqfulJ~MT*mWjSXP#`epvR1kqwgd#?BQ9U=>n*2UAkgs0@ci+Meg-tSM?l?2b6 ztKW7&pp_;*hjr_>OTcWTP!Q18?Ktx`^DE&^;ydu^$tQT^^vG1NgY<_0(P2rfs<0Ciq9x|+CNbS7L;_ruJ-YAdR(rrv}{`4RR$+7?k z$(ubKB>(m-t0dcHFYQNRB>7ZMed}2hqOu)ZC4M}0hyUsQpnUB5MDqrKwTT1ssCmHP zod2oiR@z-N;u6=vbmUlNs8hDmp1UN}xuR?RT(n@j9Y4x$Y}W$a^+)4@%T|2!vaq`k zUAirF758APy>H^M`#G?`eOp=S_WCX*Fv2J+-O*(>^mPq&aF%Qot=O^;ER7Dmip7r~ z?U?n5!CU!IY{XjybYeSr+GZns)B#DrY-C%bC(HH>Be>7@nEl)sZONV7Qn2>I)q4sZ z@XvzI_WJj!W(RCR6q zfR^^iZ7A%@q+e-bZbNzqKaLUBCcM|%(W1>plHD!+fLuCkB!~OkNFE82L1w4#P0r0a zFmNfdy~PE2Lj0I=aM%+$Jgkt7{9F35!APWZP8MVoM7MG1nj^xnyj;WMpKJ{SJpPH1 zQXc=>er(g}A8vBINpP=IPUOI2UPMY&d%A;zldP3jBArUu0_&`@ zIqzvjk6ff8V{0bI^hFhd@4+NP2^$*RSn+}xs^)x5BF=(w=zv^EQ=4C?_`r6&v2TZG zWI(nqGkXHGwo~;D+5FEaXP`t4w4W-cf~f zf+X72yWWk0Uwz_Ny3Aw$Gcw&OQy^!RInfwHQ$VQ&AB0DMC#iXa(Q**kU8KhBAVUZ? zJsC_O2Gha7L!+x`FPYocuwM{L%FnCju=8tRHxdo5GnazUVTE+6r3>l_)AS3qGU+FuC1!pttH zfktV?|12cWBsonq=eKdmq>ofw0ZkH~L<`yJ!;(byxoy_Xy zo2Fd1Lf>$PoGY~{A~PujgnRPn40>~0Nurq1JiQsA6*5BG`^_P=qu(6G*5f}xu{AtQ zJ=nfJ#Ww2!Ff;5y|7PD_2l}#n@o&TH-(~sYWBqz{pc1o$dd}g8t6mjBUvN zoSptVYsWU!9pE1y$lr1S97Mw0czYhhyFJ@Np9{oSI&s*1Tw2{SglnkI55Kb~bqv#-3A0j4OaezoGy$JGe{4XPRL5xX%BLxTs zt`b&{5^A9dTcEu^QGqf<g+G54+pLv88oB z*#?R4hkzLwiD-e@HdHz5{3Mw>W8gxT`uIDwe+qpKLM@Uy=DNpDH2#r7}NJ(+2Scm~efWm%H`}my%iL zK~0#YNXAmVZicUj)X~W~U7LM32sv2k3d*W(WNw3O!hSS#tlH3d2*;q5s z1(s0-7-rn6kZ`Exr0fsnhky|mLY{TPi(=w0%O|OuAvj^oBPH@Ohe8nw;t~LIP)BEW zMX?9EI+u|b0_xPjW)*tEDlvOOu-(G#tg5)sQ3s(&BpEP$VF&3yeW_EVP+FSbn*t+p za80$Q_pSgEOHcdckcl4~W~kaq0liFsAwoDQ)^sO!)Ot`}p4OPp3XjPn-(1k@W7s|! zM3O`>`_MiRFDfOO2R=r_I7;rj(-LZOgH59$aWh8puactBh+hAtcww+$p6C49V;qhV`ZrQBAm!%75|Jj?8vH-*fQGGldseICYaFXhbu{C_Pa!e4% zcDh~Kmi(K$w2`C#1~~fPVmkWwUiJNorY+x&gzC zOW*hsNwQ^1NB?Q_9;gcqWTfTQM|;w|`ITglPc7Q0f~o`PFq-RThnYlBE(Od63BHv;F*JljBzTl3?CGUQA zxV?18hU4%;^6sZSW!~D();sS#3Ec`IF&oLHz!=!J&ZAd+bcAe7ZDnd=r9%Nl`0qEB zM?1&8Qo>Ftl(R7Kn5s^rOK91~&L(6cX`~SdJK4)H+y>kZ7_HmLo3<%aJBnj+`ScM_6s@;_Y~}4g5$jRr{w9q!ixW zupIIC^$+CQsslumFS2N|FiaDfPm%{EU__jgOQzSpcz=7yl$Rlc{hE@yBvh zu`+6HPQ#;YOtR^DgXr977?W&Tk02=6as;XQ%0RA7>BYmdKlXX1Ofv*`k_UhOR0=jco%s%MF z2;f!s;}g3BX8)ygM000V=7#q?uTS%a>)om{Gj?ERMiY!#F`f)QsW5|?A7`LEOvLp5 z7>^EU&#^A6`|Bfo%2&|g*a0iaQmLY(8aImx$ukBmrV6IaC_r4=ZtagHq*AtUn{=8m zO`-R3193&a;e8Og1V+mm2kPVs!uX6!jGTcjKC=!2qGjAnZ|WX9u}WYnDvCVKnJZlk zOwcF{L_Eiq1$#}}XxT^^uwajMrFjgpbtK)C-Nab=+BwL`-n|agmo?>mtf=lk;>BWJ zMDk1I`-BhQpSKxWM{_1x%dKerzOqEyh!MXay*$Z>Z}8Y-{yu!KL8mtF@n3LvNC*Bl z=~8-9`4X&5N6>6DBtc%FZPEN%S6}m)5 zKwi8ZMZH*3pAa>ebCYi^)Nsyom9Z zr+ph3&qcqn25r-e^ZN%iXv+y`&^FPdLE8?gnS2XmPK!+u>H@zB2<4%d}=#v zBs<b{{rJ+(;Vpwlj@V3QA_V5h zaL^0Pjra>{#Mmyxr{e7?JgC;U;td?u|Ce1!K0h5-l7aJ;I8wuxwWqOOvQX$HOM1uU zlBl!}ddZnWFIfk@!a=7KQf7$%M?7@_W%uJhi zd$z=U)o=T5LpYE)m;>(nu=A1bNe|Mf6afyL2E;}m_#wEH;Nt}{Nz_?(^q&z(x zagYj}P@p6n287W7XDaOHk5zW5(qd+~flH_4Js@hoyy-Zr8H4^*LsL6%I!@G3;TDMsGIMv*Dx`T0G6L;Sxqp zRA{NuHjjC6i9xjF^zOc`Q;*if60uSiGPF}P2vnCxJ$*m&SkV4d561z#MY#PIeyX_ zJ_U}vtLjMMX%Rd+BVX=W?Bf=vNLnkZOL9CE8bRN>!lK@weS!_@5qTrizF2s8iz%si zc*-Wjmk>Z!464UB;Mp!(Ma6iCV3#xtiC7cTL!7D!pJvI2*?E+DVoLW=uX3%^^fr_o zFnEtC>V1i&h{~23P&4l?VP)XQheE)sHii_CT1Mu~LITNt>h6t0!HybPBIy|t6K#Rm z{EXcl1K%UA!I^jwa0#omweIdeh|^9N$0D>voC=M!bDf(Ln^@_o=6Pn{vCV7|wvHb$ z-q7UmcrBf(FGq|o0pk>7AG5IVC(0tRp97rUs(5nf(tVsvq zqJ@->T*8{2lglbS1*tUgpqmsKFlRDIy~gfNI9AJOi;p#ulV8H3-y<1WFS&EgQqQ9k z87_%z>?5gD$EZtH^avyQrB;M|oO&%DV*SGCSCdI7E+}YLNJ$SB@5Dk~U}9mYkyuF9 z^V0toUy{<+V<-LvUy@M%MUw-3NiJB}TKUr4F)pIr1=UNjODyQpXE1L(gtcyiGvdC_8+=D+>(&97E9e*(|{ zzW6VG<#p7LZS$+)cQ!pup8xgI>Az!lQm7GNjGAc6y0-N5o@oAh#v~LxTtd`SbV0wAl9vV&Nwss-S!>!16AtgzY-p=i6!BM02DWA^ zDHh^P2_RgI+pL!a_oXA*hO^SKY|qIf!Y_o>BgCBX)?y*k$km@auQ^qf=4(>9xG6SM zYYFE_EL2%D=wS^BVI@6;wnRPoRgy^YxcFP6>vGsurR~m)l#=uK4{4i+AN$jf&_t3; z3N3HDHdDQRmz;;49G~>Ck%}(MNF2QZj*%liZ7rV@@^g$F!LI)4 ziRUxSy1^Lz4dV5!;x%^lZEnG2!xqv|4q(9~fLD6u;HA-rfYW%pO_VCh@Lkz6<&SXH zo|zqh&M1pSGKndiQ@oP_;ihQqTnH>D45*u&&oR{w4_a|-Lr#($h! zJiWeb#>aWJkKnGC_xiVGqr9;|lfCFZXpx^;r0#!A% z#Zn5dqG}aVBam+LOc~R&xaEOy_Ud9F3_A5D=Ua2>QFeWetfEFqA#U*{CL}>+2@GVs zVZqbX4pYX-MFsC3S=_U}ZoFdHUn@TwzO(AD|EiAP?#+Z%v$8sr?cO4)t9HP~Bfj!Z_ z1+N@sw2Qw-yKE8AWAFHrwo;W+4c_!&LsAg^p^&6v1Ct_NWPcb&PYOok8K6SYCz2~!zK2&^n_i!TCg-|32PBJ z@5u$@gyp{XsNwL!E#sLiGjfB9;YENUo7BJw9~~h%x#TssiIx&ZCC<@TuhBV+9%BR; z#@v^^OvN6HwLCu1kdx9&e({7-CVPrQX8p_J5Mb8UQJwAN>_73FbN2t++=QUc*pGuysQk1vtFBU$4k>HBh`Y4E3fv#NQ9XvCIpcFz8@f9LF66tz+170}skQXf{s zcI0+eLAG?eDzp8L@l+O8JNMj42T?P{V8E;Y+F9d|QRnA-$mM@neZqI@doV2MA$nw| zey9HIyT6>%(J}qirG8ay@0A~K-EjBe4=&J47lmy6%fEKa?LJ>>I@}=YEml2#d9Y({ zeHm%8=nf5XKz!gH3R3NL!^50`WkrZ*In`LV^4iZ zB2FXpmo2op3XssiTIy7J=dBQ?(el^cSJu=O`%%-}-R+|P9E6b!n}>F-ivA;qe{98% z-ACiccc5OsV)SPGWgsZ3emA$YJfa?Ki)v}I2!+s2@-cudc&hyEOiuY7HBv>_7T_*zvF^=^7&ph7MijS%g6 zPLySd)VX0c9F!=N=GrDsQwKdTykFS`jyr0Zs}1hVM?)J@a&Gv^Vd~q; zf0L!?z`dk4Jx)ahiPQ2!F96N~yMc~942EP#VR#6sk#J7mAatC8 z9M(D};xs_T+!hP~t#)^F!>6}qjFzs|^P27J(4GyqSg9}7&*U;P%SBN9Y=l)jk^*zjIwiTt2Lb7_Bl}jCmUJdOHX!b;Te_z%&rL1JP09LQPIy#q z_{z-si@K-w8Jh}Ai^yQ~11QTZE(*t7RS|P(UgU$`Lxx3$)P^&Qr-fs>uf4YDj0w@- z#>DQe3~jh@a#8!_q7!-1U!h+>V(g;CvM)w046^?35@7v*>G^u=|FZu?Rok>aiLbyi zG6oH{Zw$qkoE#si{xm0=3Eb9L)x1sg(oRHu+s(rIe_7wSm195w{%FK2Mvc@Dm&8BN zljHpT%Zq^15a++TBtDy*BN@8wpTC?C$(oUw?FEe6+Uk48XMdPX_~%Oi+|AR|5`(I%-tT-;mE#SE@mieZnEBM=}X9-)n3SjY3W?44P!5*y{lbqcjnzKq|7 zH(L2l(25XU$Axc2y@8Bg4kXxE^@;aqcFYB#+S5QSpw%29+2F=I9B@s*eCnZ(qJP*hW2T=;Q{lFRO9=*Oh{sKyu^=8n`Evee!``;nE zvQgXY9jt|9ItIj7w)U{M0yD!d&^>@iiig{->2mCIGPLt&fwtq8qlImpv*&HdP6$yf z;^RXgyEkK*BV|7G*KBKa+L111A3KR!U@sml%Ph>W8KDK~!2D7U;$y3L(-xnK9e~KM z9ZXqdib=15P%yv1f)tUR8O+)uJue#$^BsjA6C>+6xUA{a)C8I_D=&M^yra>%O$et| zm6MhVqErr7ID#cL2U2`d=4(M(ImR_f$jb;$*Ix*KQ3Tz#5fHdcU4jY{dxtNCSg2H= z5b0jb8-OBlBv8B9i(j_BkBF%?-#NoMMv=1lSOwB}@M|AIeVMWtvVEZM4sWf#lbQ$Nzki-Wr^aswbZUfjZztJo*{7r1 zn&e;T)M(D9*HKl$+lNP`6egps-=luxnEq6^la<;er#jNm60*9^D@A9*v;bW++Gx|! zsohN%6VVKhe=|#^JNjtV_YF*j0qbd!VUla63C^1RQ>>UUN834iRs<~`S+jY-6V0n*6AtuGc(Z@}tv9dQ+^1&q*>@*p zy_gd|KeN7T_tc%EQS|NS>VsgE@Atmn`|Y^nwx%%t?vA^fULTQ=`~{%cF-t@B@$eWqW^CNC zRTVQ2Jz@Udx#S)BgH@30w|%{=2L@co&?!oJ07Z|(a2?SN{v@78qiG}BN5Ury5G^3)@f_dZG;2~5_IP>j&uJUnIx z3(pyuc_KC7SEiW>dU4L!c}xmy*N4N5fPf>}m0&}LJtQt+Vo|8balO;)17b<~7+MsZ z_=>kIoT~3m`?n>9dHMXTLUzePp7X0{DdajxA%Fjjcbyty>Y8jGmce>h z2B#@ZNAqx?S)`-h7~?D4Ffn?pX2e zhcd3#0MMW)1Fjc%%^EB~-=EgmA1)}d&i<6bwvV-}T$GE3;{o&7O5+eIS2Fx+oDxl- zF-{)Mn=y5{f3xJ3%}{R~yN?(ij(jq>3GC;0QZFiFqJs)ubY9Ci)5H>sj8%>B=Z+1+ z5a@G%3#O9c3J?pq9M{7wpOs!deiv96u5!=YSU-_6ls@_v{NQyJ{hqCXh<*1#IZbF) z?Ti!`WT3jl3M--k{DWc2URPi-f*-f4&6hIo&iQ7O{DFH z9BZw(6Sanh5|V(a8y;KN2R3|%%+bneVe;rP%OPVwPTYV(Gv~DQ*n#}AZldqtcn~n? zh`C8E`vch|zV9h=Sl`#;IQeCeZSD-~ft%YhS4q*}+tk$_NCVKdvorH)qoV{g=w1}F9@Gh z6dv+sR_P_-xHF3@x`SQ2EqB;$Ii(kc;|5g69a~(n4$$n@KZg!m8fpQd)=?aNBrg7i z}^!~_K|6Ms|vz*4~Xm}s5UL`*a;QOdga#5!GAfWr-gy1;*Ra! z@a+|-YwH=0^l(M@Ub$_uBU>ZN)nrTU@@vo_`_xdRFfn~o#-PZ~IVqd&EQq{;|J4>m zw#-S}bZqxXH~if(gCY;kNy&H{(#KP?63Y%xXxK8mB7|Gl$DMQd_jR{tMTQ;%^ATXT zcgl0x7K)bI-@qYKc4Qj{vhCx7XdjSmFNK?qC!o3DiMcmRm;UP})m_I#*O^;G(**Xq zaT4y*GfG?+5Zfg_hz-g}R89pcBgvk`PZ>F4+iXHF-X4*uJ-<^llZy;PIEyJ%dDj7p z+>)69iMY`F6`zk)qNPUgT}DhtbzrdjCvSXWEu6faJ_d>+?|mDPgZSsI=i6zB1X8S9 z1n-Q_zkKvU{VPB7pv^c+eGYFxX5vSp#Ujv{+UesvSgs_u6E-Td42N9-@KE! zcJAT$QM`}s(nu~-vIkL^h2jT;GE?aYL9b!9`{LNW6U1MAksFQEJ;pvy}(ZY@)V8@B9{~m8W#38}CCe zb^&uiVN!7hv|+-g$~rJ(?F>Ju`j;-CV#AupD>z`@_V}IA-?H$rvi_A{G_1O!w?blT z0RdV%uX|ju;;!lkF-TW|F4Cg1YEo4?BiiDk*7q8^Mz+1HLPbZ&r$CAll^Ie8cC(RH z^zFUWcnEUsWV4R;ZP*;ncH0VW0ADFl~{q74Bzhb@fKWZWH-UC^@{CLkjI zml`m8EG6eF7%~Z5bt1wZThlNF^@nHM6C$dNu_JvM9umn%t09PeJ5_b0<4s3wMIerN zs;=1oOVOrnSmhg_-w3Qf0zm8a0Gq62oaL~hzzmv9Gd7Sg!%GaN(*tA^BAZOY=|$90 z>A#e@gBik9-_3aF_2YHHUHL2Z5V#-mSeYA^>NyO!DH}fPQx@tHr=)mENPyh0;O0YR&hC7SKIOQLVL_l%WU3!C za<*D3{u{bYZ(hh_pAB!6{66;mRuWgn&4eN}>K1|M!6O?ca=!W=j#a>V1yG+C^ZLt2 zq#v2mUwbbHqFJ=xyami76z-V_!{3auam*eAi_Z}+yIy7_z|+U#Cb5e&mo*FG&iO9e zUqx$tehSS<^FOZ{VUCRp+4>%|Lbj)(^B*p|>tL0#UG*<1Wn1&(O4;7Jvh~bi|H(Mb z+(LU5`oFh*Kn@+-t}h@3W7{7fe|($ek7a1kHu*+wNA~Qx=JZV2NzA8(b`>EKUq8rv zK%|OOFRK{I;=n#d17{R53_OdE5#he6gqu#Vq%A{I&)&Pm2(~C0O#BoSf1AKGaZ@@9 zB1~M}RU>vzcJgV`*5CRV?A!b?{Psv#hZYx3eobjZMe@+ap+PHYX2m!Jz^z&iB}Iw& zu4ER3Pi#LqC9(bprtsRw;EyTvMT!^N%?rdybC`nP2w%Cp+)Bd-(e2yInPANY-!C zKX|cla{d3YN3!65XlJ!0Z>@ABwROf!51@!}Jz2UN6_*P1v124>xZRGCj{%9wZzPqF z!gb71Xyp!2&Aw%Ii#stbEclbh$QNXMo9X6ngfxMG zD{MHWM9vMrsFIL0T+Jt(kh~9aGBMi?NP|v=RFmh}2cVBh^Wl_cTcwI}>9BW{Fw85Q z^0d&@YlYp1)1#IU94953>fIWw(eRgc9Xg>&(zufpB6L5&&6O_>h^jCxOqu-&Lu=N_p3Y_gj9 z8tbZ+l>cD^E@3MSYa$nsqmb>cxJxHPN^7yu&zb*~vvSh#9^#ZLoV=R4NdD-}u4`r_ zkHS$`rk%0W*|uMPsCA2|nY4bB^-pp9I$Xj&V750`zR7q?vjt{$+{U~hut&Z#gC@sW zUyq>sz}xOrS@r_GO2_z<8m0_Rd6h%pb=3(;4UMIkI(A0_h9ixK7NF=)vfdL_R&3o? z=brgvroUd_+W!4P)=5^jrY{K4NlpsVNgij`N%nv4r!|YT>m+TI+kR!}zvTp&m2?b< zOujy=G!4FxyNaVb`zGutT-{VPGOc}Z_lm74xos1{$Mv65vUB)~4^+LC7lLa#gzBvb z!1X1bXBrG5H?Toudr4L0DyzeGhcs~<<|UGB&MhZ+=HnIl=$LJ-)C#U{!K7hT3@dMY zF?74Ayjn=dG;iOu3Q)MAmdUi=9`S~e|0P^D95K6an z&U==C3I5Zya|uPQ#Ky^rw(r1X>JkSvSdF3d(y)Yiwds7;i2 z@iyp1xG7zxADG($F!zuzm|GPH=5~>7Wai#f8k}2Z6r;3_0*>9frtSGXXOG(o59(Yz z*WyZVOaGZGCn*f~mV$K|XLA(y8APW*8XbGgG+R{Kk5ENb-h;f;7b+R#rvJh*2c*~0 zDNst$)p3bKPvCv(A%(xT(J>Fy0it4oU1lfx1{TJ!+{iT0po#_9LY1q%bQspsx!%XzqXev-3DZsr2Xz@$?A|sD#Fl{B)@CMxKI<5tM5%E%$WtF)e%Dj@ zU0**HpGug|r>qUWc$thn3{OFvBHF^wV1PX9L0cdOw1;9cbkolTw8f9@22HMcA`WAe zYnXf%YPOdh>PXodH=F`io9V=A*^<~trkg7WK+P2lTe>J7;QuWxI85|2G)&?IwZ%-b+oVSc4Cs*saN!b5u$ zaAazWROR%=@BJ|~S66@7F-s%!lHQlkd#${)?0hIYcupD^7@UG1XJ^N+@>toS2ck&G zs5DX~L(b=Jh$wTSg+w{K8ByvPck!pqLYTr~tPZke^owJNK$F5ves7u&Gfx@yiC0#U5PvE z`6$CaI~eq1@1#PkxuCBk)H zKS3+0L;uQ|;og_vH!^aEBdiZR_yc9#|V8Jccr-7Gm52I&RR$%zu34rx8k*fCzbqk$Fx3a z%SWfr7@d~)P(k?B!thH2B4^1vz4qpGq>e-^H& zKRqY1>ez~=M?*;++n0vQf1OZK1<%OQXIE^6XXJ6=igozkfN(_>tRqkFS3cTg5!>Qq*yHH)_Ti`~uUM{2mOG;ym{^5{V6X-z#MHkoIxMM1j z5LDYZl}xAyz}$ugAqb1LnY|%kz$q%fVpjqyIuNMFI#yw^ceKDsYhTw$KNEVXvq7;E zQq3Z~bHutjIs>mpG?Y8-sUotr`wK-0ivhBc^O()#|DR}PB?r=ms^cCrJ^;w}8UrH+ zVzP`6m>&ao%?7OIXxpc2_60t$Si}}W9VY9zbmmr#T-~CAk6`dc-gPoZfr%vL(N4kw zm+T@dFxy|o#$yh~*PcpuIFLNS8E$KuiD_qHUV7JP-KD7=3$|oXY_QssEid%VusQ^_ zZzX3T_ndNIHc<0?7l1ZF4{O#($NemQrl#WjqcjHI(L(KN{ z7+7Q6Woaa=iG~^z7J3yiOCxQMi;DhKOl_nO0HC}qSu6|z!Kq#qsQOYb29|ylhL$lH zLP9=`DPRr*kvKk2V>oMkX>d8e1u)}XTgB{;cvTIC$Nta3sLI-|I8P z=yokbb~;ZuIBHA&-C`O=kRKpBCA9Y?2Ned4Z9zd)WsVNW;h-6|o^$&Bwn}($3X;WS zV^#}TS$1l~B^v5kgQamEor3*{n3l1FuaC)o#K5cF!^17c7PAiaTC{D7Y&We>GTM}< zkY5W}YI%p3)%04qF43*n^~rvOUleOC5tH6J7LF|5iASc@+Q{So_s<-eJ8WTYU3E_B zRpGd~)p6GqSDc4a3vyvD7@9nIN=f3x=;pZi8&l%1SQ^@h-fN+$CEX@QpNxw?a>}yJ zhuqn8QGC)XnTZ#lyLxlmn$3G_n#{WBOn+AbC`s|!lUrx5jEhm&i4Fn;pwfjm5NKhdd<=2S` z;{||^IUSTnt=O2%D<(;(jJ9;G8Xwcv?2=Zd2+bsYn~HM*j4W-VhD5wRu$B=iF|xL& z;qW5gqH_zdD#$+MF!kq}!kLF4hA+t8pYUnkYJtI!^eRNij}7}O$g@WWa%x`%l&*EjfCLxqeo--U zEt*|>lb>yks4nShdDY!TJMzCEBGQ1Lx}a}bOv*1)+SuCC?(Tk|{Z!sO4cSkQET+Fc z7)mOe9>%aE(IqXbYK=h>0svWY@v2>j=Ge^?1^Hfs_$+(cSbm(k9^6_UP%`mP?rv-( z8qq*4SVjg_4iznVWo<=lC%1EIVdI6=z_@TS$hm!5F-U;nKqVtJiXqwTa`1_X<;%{c z+OxKo;pqRX=@h~+%K)TCk+sjWAVN&GNj4U>5MD;r5pDEtovNls@HsRt#>c562g%+! zH*M*%Pb811-SDqn(}h|(SL;A)VAT)@DfW>jvTF1yqB^`4_`U2(Vqrg4H&9&8;YHmd za7el#l{q4Vz*=ZRrSi)#``}Kbjr(FrAqipWy(exi_Vuui0TyQqmf7$LJiogpn=ae$ z6_devmxv;V_}Yv)`U{Axvpp77q#p&K_=S8sM@%eZY?(C|RJeLKUB4Yzj0`x1zRP3W zuwbJybEqyAVs6=;ArRLX9-sw)p~MW->4AnxF~7$m3&vS5-qySpMVNH@O0#;ALx05~WM_CbI}lcF~xDMUAQ-(+F2bBz&3CSpgm zzJ~Z_d`#*)fDy4W7=95>AzC29PrtwR%#K_qyJSIPZegLPES?^{gPvO+yZB4eIi_OI zj+?MObNu8=RfX6Ptm543RWjG(IvN(y^3BR`QD72tUtH2Ey54x9WT&5yd2E4@+5XEt z-SjBi;cu!(QyMgd>q5`gXL-NWYpnH+XqiC^1lUA&^zJi0Wx>J2X>S}noL2WQ!D)x@ z{!uXPn7@CmOblN+De`x2nmxE*uSxxw{@H#~^Po{9HWRhgc5lyW!V(@BZTN~0flY!O zN9YEp31Aj~Z^>bd%@&B`q0_$JAi&0RRbjZRFKK6`B;R$$czB5*d2J!2Sc(fT zFD|KJy=T{jk5>_vGP?P9LHJ0zFD1H+>e#SQeHuunh`?kzJ4BC)4k_I|DW?vH9F&h{C zblunV9+X4-v(0*dD=H)xXO6|-H;=F+06|ci+IClGyasxat;7uV5;`P!o!eQN_cf^w zp=Qo9wo#i^?EWQrDIKk87@@*2ZkA2kv;m$`n<)*?c&`qyL#K_{lfmIaETqImjoZ|- zHs!yfdo0?5H?r$!nK?DM+uDI{^Vo9BV{-#agVRXkvzhI=CyMTIK0uB1R@0vsjTJeX zGDQ53lH6*>&W+Ou>Y33R%4e@bUAW@GMRqIzeT&O@|Jg@0rMu{*l^^z2CV_J z!+qrlg2F0%+S`XhmF8D}#i&_CdsYh9F|`3PW2ypW$%S>jvgtEqHYB!SK1-60=J5fv z6aE0Ho+>Y_CbERl3pI~JYA7Zyi;dG*m?;uT)|iWck&lU4Lry=S*ivNIH3YEI1Jbpd zsVMqr&l!*J5y-a-!0ddwz`qiVL{31d8NjZNc4L1Ja%H)y8M}I=U73QlKw7#Du?J|* z5mtcibwms<))g>oL}(d{4FwuT4gsh;>ut~WQGl;_)8>t=k-c0!hzx|kEe|eM_I+0J z%VN&Rc7)c6xvTsQ4oPge>duczB58%bdcok?UAGA2SozH3-^NAG_Z4GH{y`!)`1&95 zaox5X-3j$#!56*%x%}oPcum-knJZ`2pY*ZUPjKx;ncu!6q{AK=3d+~!2f99CmLnZw z^=h;xv6$Da$M)as*jv*u$E|%+@yGT_EFYT8NJeZ~F7wg=>kzRf=;_8#mGcS`L4&IE zQ=nrCHb77rur%;ZRJK>mW;=qGi7*G4a7;Pn{Z(;y!TS+X$I_hA3E{Zws^Wfq<+x~d zT>R5L;~#?55ytx_7W(mPmiMFWlPtcCC}uamCOrQMS^$vXj%z=$O6tNR_MFMg%0UcUH&j_X9g- zcaXpTmN9Xxh13QlV-`y}B+T(AQs1~_>W#fH>d}UcHd+5JY{OwhkTE%{`8(A#A!CZ* zBjRfFXXlCyoMe)nM>PnO$NoxSv*VZI!tCaeIJFi!NzU{7v|;N*$arPVfaU`uQZvq@ zf8CroGUe6S!nMzDD?81V$p`=GXBi{0`BGXc0MLJl3O*#iM) zgan)AlHgkzG~k8|_ulv;WMKK78@t_nZf5E08$KsB-$(hYw>Or$}{3C4;xS@+*t zkr?~+#YPMQrE?;mT^cmQU?*roAF_-Fa~MmNxh_g)eA$4+KM+$Tl~&ks>SN_RO?6ZU zI9M-EdBZd^@$|x0;%Aca(EabTWn(Jp6tODmT)359P-}ycnG{(Dw)%0#xwdg*907{0 zG00tn^W6F;I;2X{T?D^cWMOR;N7oa60b=j z(naTQ#Wop74FlQ1B845%G0*$Y;rd0tc?--3%F4)^@T~|0&jh=sf1P27 ze^Ap8Ye@8CNGQ7bH~6Xo#s8 z^9Pmmen1sWctCq>WpoMIcX}qh@CY2Y5wE#5qs0!{0Pp^wu9vuE9s9dDA9D|W%zRrX zTW*dNRJDX*O3PEWoLIv1g6fPJAwp<@*_%}XVFD1^E<$K^2MwVWB(xmgBf!xA-XKH& zGtGwn<8BVn3+rFAm z{_EkUi2!9Q2SjSF%PLI{S7d%t9DSm1LI!+B28~R~n~yGWZ(Mt0qqHAeLhy46E4xK~bX7UZiZI7|;F?t((MjmIMk?o_8Sl0$ZivRhy z;@@WW+K|b|9VbN&0sg35@L)U9TR!yKQ+x92(Gjx8y*`%yQ*U~>oK!A~($bzD)~UbU zb>KLQ=5mr}%3UOHN4_T4w06=#w!QYZQ#uAX|FQggeLb{RjrEV;KoEve41S%E>wL|~ zP@*(+q)fFLH5#DU=eT5E+CpJ zeR5I5&=CGOZYT@ac(K4G9!;}J-6?l9Cu*z~Ba3%`vEl@A*;nH(u@R2 zA@lQJ8BpC;S5tto{?O{hmOMb(MrPgf+q6;0SrxKtzqgBj#~%N3HE?6gjAoHVmC7p; zv{&=dtpCc;@N9@eY%8Z>D>?>nI+$Q<{#AUh9W}k0=kUK>xwKntY%ey-S6`+E_qSS5 zb~Q)YTS<6l)ij$l8+I94ijaQT<6#-NTF&?`{}8EMlFs;Uou3t0sDN4r&{n`p0ZrtD z`#4)2K(Gm)Fh^u1ettp`X||w44e*zze|iPi6gl=Q5pbycqEzj>vGwdDHnU%L^R}KG z36+;tSJvm@t%mlkezY7LLeoCF5d28w#)V&)A>5Lugj7rbG>?`wWG|v>ElGQ4;cPv` zkf}Ta>k#j-tamjILu>e|}S8(P+-gT5tm0dZlpF z9n%t4g9Z#(jka6&-#f4x?d6`BgOlcFtVUJ@JPyEWW-*w4Fn|G4v+16pA|9EgOb9Hj zmL+Ir4V#s5pEdG2l;Ow7Ldbv=15}Pe8Ja3+H64g**&T|^EIguP*dqEa3ZJ0-3$e8! zw$p)ht^&#WwpM`*kG-QGmEi{o1Z=Td%}#wZb!&E#!!ibrB^2QQC@$5bD92YH8wF)= z6?O8%mE#I0kC~qGIx#E|FYXtGkLEE0D^J!U+@Y%5F{f=E}yK{tetFHuE7V{N%AW0wKhu#9MD^I8Qu=KL(|8| zbhmTGv~kIXfu&Q|bj5U+v4-L0k+W8K>+5-=*F0+~jq-Yt@or0;+|NcPTjQo*bUKC4 zp`F`@2Zpgpqnf_f*RbJac4^rydK4P5E#7N)GE2Y1@}0K!@AIafwOF)q)JvosC$hA( zZ(jQEdC(LBZc|IUbynL*ix;L85g7R)x817{?5_wB- ziVTA8KNKnpKY(`MZ`071u6*_bQDL<;jgT5|Tle4=ISq5=G4|-^UMK07L^K|zBAq+# z+4p!y=wDz7vFP}glp#AdcX&uAYBSsNf^dZ9c+L-8-S#;jH3j~5NM8{cQm(S`AsbqQQ;U-H0 z3NgflCk1EZtCwF8Vyz=qaYdR9Z5+|K|HI(#;f) zz;-jR7R)>GUZ*%scDt}1DP6SSe!YanJTj4eScGNlFVR2Z%~T_6n2KaZQhxbm&oDh<2x9maJkMpmtlIoq>tW@B$k^D_LINCaON?TGcZ8n7DRLc^HH>1Zu#Q!{ z?HE?^X!udh7>~j-vmbBDZmKGz9fstM;=r+MZ!RZLJ$ zo>R60WS zX?b-8;eJKoqjMr(!0B;p`u2jfyhjVd@oyeC?ADyp^TKi8OsF`ockZyV+`4%=rN0cv zy<8QyrF+BPmE(#__D(7Jt~k0J>cQmrfi{_Vft+LRifbWOjc zN&ONpp0axLYil;7aNtw|H1GgDZh930B~*(bz_VCRi7`7 zUfPcfzPg{0(|*|mVs7f@UNCP_eQ1loc3wXJZ%z3A%wb{QPHk`P`uC2Q= zkL+H9MT_Wxe2l7zDVB|ho&T~NBi|adpKd~B6)pf6$KU*fM`~J;EfX#z)<1N(k_=z5 zdkcd&O}xbtag+vj0mK4Qlge99p9&XyDIax8B$@BxQvibWG4#GW)r;~JH*15yw17(T z+D}NSX7A3Ut1BB16Qo>bb>++i{ISx@)x;r|T<7H7c=yxTP`vlYhf-hmv5bUbyoOvz z7DfjvMdeVkAmL{*f=>0^_rzj70dNd^?4siOD3Hl;a0~V#q4f94sSKjMgpw$InL8z1c@$x};?ijf$X%(i-TCl?Yg{>fm{TUX zHAW^0AMZ?0mE>~3_aCG!+5XRHOW=ZsRv6X}r8g}rutUM4uKx;;q zJj+R13mNDLRaRd(ZsQSokgBuuB_U@uh;lG?9iL2*VlxA(qn%l!I(w?z;iaJQ7pA8l zj=-{EZ}iOcm^*b0UN@%bN{+3Ln89YyE^MM&IkJnMe*PRX?iUUTt1dsTx#jUi^e@Wq zy}L61RYrJB^N2on1|CA60$mD~mY_=wLtG216KZw0haPL}-&@BJw9;;EMDHS05WBUZ zx1b+IkqQDJM>0&AeH;rRp%i5kJpzceIjxw7x0E)>xQ9%SZKRCuB!XxWFQ+ zy}~gBFf3LoW?XH$D#(fxqXM$9+&3LDV?4|+8#~XuTz7D!r(mh8x-5$VP(I>N(`+K* zo@*qjppdslW4=MW=ndli@Ota%{t$ELlX6rh+^9qjONpeg_(`#$eivxC$y|SB&9GP) zeL0iwa_x}rvayIk5Pq@b90KklJCft**$>G9QdYxtU%)mU*IRm2Oa_jI9W&7p9dq36 zgXEy}mg-PnfQ+1z%8NS3m|qk|a$5K7$S7E4mjHW2a=TLo`qX<|)#bS$&=UczCEj-1 zf*nK2!4<`en$AwU3t3HSlC586P2X7H7pIh|kVkfm7h6@{=#^-Lp+B^rocA>h8BbH@ zn^Y9xTys$HbZk@{9;-Bk($P&eyB>4SoQ&S}dzFX&J=xnm;Y$+kEfZ zU|!SL<$YSdb2jJn^+UtImQ*xP?f)1Lc@~sldeMtrN%2xb30D8oFFY0nnLmK(m~PGJ zX-9%@91t+z%e;T>jaqYN9AmU&>!5aQ2?wpEcFKa*^l<^ck%tBOM*hz18#(MJZpT)2 zLnyKAh~kRIxQghoJFuGv8jC_=#{SNx8Qqm3=Z!W3T~=cnW&MF=TO)U=l(hA|3HAS>-DX#-SP?V{1Zd zS3++MyH)2nA101yD=n$yK-YPUl|kKBqr0hRJxS&w;;txH6Xv;^470GAK1G-8{f2an z7MODSgupTqx#hI7)MCQJZRcjCyh?b@)Ky#fpYWE(z)xd8X4-gtz$MRoLZk0y{e-5I zwPZlB;ZJwyYQ2%VeL<1f=SXfgHyyiJapje%O8;=Q+SRtwDJeJ05WU#)w7Gw2pNXO? zTU;vY+US=F2)FVVdJCmbOWkp%I7Y~vcdy#m)+@FyZv{V_H^?iJ>iYALFY2Wt%mRV5 zil0}#*w9WXs23Yj&;h;J;4t9s#r9AP_XbqBD&u@1;>m7)KE?8GS73%L zg-9)Duq2miYZrEf1QMD+6ea$3>?*I1d5cw#%&B6Lu^6C;IbzUylVgI#C6KZ{6Tnp9 zb9}`mqMD8J;q{}68dmk=D61oyl30FIf}pPE%IJ!FEe8m9U>wFTs7HztP(&uY@g^+n za9ihL+ixHQYt*o4EQnaQ#S>}sRosScW<%!X@gx&#@WE+MNNX^kxa>Z9k<{N?@*Dl2 z(q4b=>0z)0NAC(l9HI%k;a}WV*5X$2h}(z^uLBC$qy5xsg-YqS{{SGYXQU$zan}qg ze-q7isLy%^ru87)Dt z;Wi5e=dbZJaeIQhc6=54f`AQ8x>FhheSir}D2BkO@o+}>*7CtgFZ!~^(WQj}*wnnH z*q4J~wIt@g-*N!54evT{*3Nt(VJPY~$$Z?*faPF>@IG-f&DOZG_et!OFQS)NaPgcl z*PVN;fwGh2NMm#2!H{*3r)*|K5Q&*~Jao$rhQLB+V)znbIiPOYZ0q0mpuw}JuI(KB z?4TgWk%9onkxBm=$B|N7+cjH7+qYF00k<8R(C~xWWk0#^%Z}%gUwQCPWN%;c$?}q2 zOQ}-oo+XJ7^<=~SFPBGqJ+SYP&gZWFJA@@DYhxbD8>9uBj${UdH2Mm@AWiW~-XM(> zmu8T*T!J*qxWrQ~nQygGvwe~c3zFHsO5fl+Q${1T`eP>VXq?z|WT^Dx7s*o^X3>6` zP#N|sqT1kX5<)qQnc`|2orRp2P@z`Nj2FG*jE0d2mo+73uU50*;7+{Lb}6O}rwNvl z65()*eEJ!0w@8pm{$`Q5N9&fnit4I)M4+W6Ytxa8pQjygp^B#sYOzc$V)p|Z4MdRsB+}<+w z3Vn<_)dkEW{-9a*N zpA)%l3In|Gu%fBx?KYLC&Hs@aGZDJ+C%2z9om1PqOSk#dwF#Oj0tK@xh7hOLvP6s} zE^g%=COg$-O!9Ir&}nty^GPaTJ7-~w_()oft=nK|h&-5NEZAX$6FH0@47qkYrq;2R zo{~g#K#7`Zp#x@m&e|YKGc;0xENi2=T0??y=$1)zWgHLdC*zX+TP|U3q`7M|)|?9h zurQv%{BzmE$SbC$MRzSOIZxUA#WGEN+(lRKjh-b~qS2BAP&=TSUThk?95<{tLiBoY z(#bO3W*}J78+c2`>Bbp@NdzlLuvq~DNo!%qrq|d_>cTc(kUKj0Am0Oh4D-q^=}wuL zG6E{f@_MwVv!YbafO%qyQ1)}wjE%mCn;z2B7CW5L27=`zJXXKaF?P(bW*j-2+*Y?~T}n`=?5u!J*$;Vi%Kr8W?wMVF zT|!0oUb$_zLj}34Akuj3olV=07#p1qzsPHfe}D-|#@pzKeQH)>+2Md^!z)6k^^C8N zJLmBI%HqFn5j-S2i=(~zC9Eu3-Lz|DTKkCZ6|E_`ZL?q(`Et1FBsi1Y*)1|HA$|LZ zBd%yfsT zBKz2lHnlNJ^iw#EoHg z$nT2s@yUIQp(8PTgmp98;fGd}^#H#TfJ*|9TyK5)^r;)Af=nVb19e81JWI{zd{EnJ zQnl14Fj^%2qo)Ij_&8o#oS&$fI$%T~UKNWO$T!-ss@A0x^_Z>UXkVq_8cL!^>|1zg z^A>Ri>=?Lgv7Umc*8kJdSjD`!botuY6c!5w9D@w0)>|_m#IOV9!kX@Q%@do`c z`R4P6s{h`)8TQ!EY@4C@c)36FH|!YrYr0GR-;y}0@gdf@DA1RrfOJs@%|%)lrZdQd zm!T>&<7w-9e2y0sWYo+J(2T9#Ef+G(aWikDHMj`fG?4bd5h$hP6uC8;vcN)LkB{+W zokdgzI?T#{8v-UkCzxZhB(%2L%Xb9=NW75V^@12-7`p(}8_T)Q=40)Kp8GNb=ItXr_jN+kkV4dy7gBo* zYBN7(2mcV4lxsMmv4{i%nBZA>_0I42ZRa54XXt;Rh$btTcCH_FivY#&iwqdR>Q^U; zCL6Ab9U;N&0?B&reZi*GUe3EKEj0LLz&VEVG&+nRWvRjH+&9g?rKKMJNHma0!D z#-L?YV-9{HJFUh`ehVBN~kPMM#SQ3EuMO@rBnDEG7NG* z9Q{Z~DV*#?f@HKS{uM2@O0nsDAFWl<>wUIr$oPGxowRcO!vo(lLuKk6=MBMlDk}#t z&zcGbYP$};3&}wPYSDiQP}})qftqyn@BwPXQxQ;OGZIgr2G%BvRE<7BjhHAuptj|v zGNPG&?cFW=+J|S#kCD5XR_HOIh23n$m*AmzH*=Sd{ht5k+D-T#Od&!##+ZsTX23NH zm!UjOc#Qa!r`_zn|22F!#)r(A%@u72WDM%EXB0jqydW5X6PKqcttY`VY7kfEQJzLd zoGt}r>OwC4L6#St1sEnfcs+?4MHe>2JzJ+GWbNDLQ=f*HwKD*q$<&w+K)b3-sz(Sc zpS$T~EH1p2o!3i%;G)tI1V4)Yyr`HNv6+W3A-@oPB~y zA8t^{Z|I3d6i)BjS?ok`ofWCP@+0wmTBAWNvomFu(>ody{FX;?6ow(7 zS^|3(FeUB1{#jj0Q5(MmZF=mTiQrRg@s?!CzNMgDd92fz24cnRd@!9Z@oVFZ>(Ml> z-blu&{f|vYsGg#2Jz(0q!eJyixi-42iX~qfqOod=DJAbsjpAvk;Sw)9AKNqLQtyd2 z*|5Pk?D(Ml5Kto?u_9X$C8V8n5!vW)Vt#haZFN+_AxZgNy?GO7Yq0k7;WY9yb{dH; zJAY!^8yf`3Kx%imm-tJwHlWcgFs)I66%L21ftIgesO*IhcyYodH3C=L1GZ}6Nh0@n z-pV=HafYD6sgqt)z}pF|7PYdl8db?t$Za7S_b&CpP%f7a2e#Rpz}=>7LSNj*v(*JZ z7bdSB)V;k7zY3?|v{FOV)Q0R-uOJ^l4%upNo{}bTGj~1I>2)2N>7bd=Agcxq?~&xu zy56%Yc0@6EWg9DCL0vIqPC?+Snt5dFv2wYirrpGrs0d}TFo3(l+o<1kIZ5mz44aXe zyoTrzupC#!DQdi?*9EEhvfqJj3KosAvo)K=1h@7yd=n8+8)9mTnPm|b`qOxc?l^HK z9D+22vc^K3;(|zNL}VI#5Xr>~+r|#NJ{Fp4zx7s7SBCXA{NLrviQvY6PeNbWnwbgz zE86xLOBjASM2+R07o8!@fjlpyt2R!82bleeY+)^;AMRDC&1m^eU&u*3^Jx6qmk+&p z_dCbaH*&5G)>tAem|L}!&4Ik3GF>$K?f{txh_(9DR|U#MI-fgd8{{pfRi4)(ZlSF+ zTNqr!fIlUqL;zd3#1{#}sv`yo+ZI;VKz+(;6O}tx&sDO^V)?}c(mRud z!R)jczy5~Ytd$C|RNTl{m~&h}mu8dS)HevA;Xk|%l-cqb*8_96%`A`qd1~diZ^y+C zOpQOXHk3SMN=fq6lAaTze~XKMAtnClDYrE(j87WeKQUv*>digZZtihs(+w~j$x6B= zD{ z5QS+8eEiVT#HXjM-}X)8=?~zY^fwpjocFU8^me;In?Ci(CCIO0^8LQ*52p{J*a<_G1;l{ z28gD5K{OdYfg%oJfEgqt13R|=ylw%)kuG$z>O+WMigXD$YH*UBCXx@$vwI?5e~ZwR z+vcBtdt-+9aSkDv12}Q@S&u~Mz`P6G z7{UQGq*4It58+k?gu?7>;BD!>a0bf@hZ^G$4f%vcs3M^`@2=x%jy5Q5fS}w^83?hq zRzjN5fi>S*EhYkzVxjkmoq{kT6^R7_it$(^j<%1jKRlieYE1bS7{gBdGL*FmL0LBS zVmDreOXM4CjO{4gLvsZyEAE)N{LCAyeU6fv`?`_lz136p)ZI@8Hk zUjlIFgp|oJpnvVWFA;!#N3HA(_&J2;z7&w#q$G{rVOs_QeFD1Vz{$A9YhtQ5`Jd@n z11l6Wx%>l=F(4L1od#sArxENQMGd53SQDOqmDQ22izaI_hhVm!T|sgv(~fB6AkfZ@ zJ4xDC3rZk~evl$Q=#o~Xp1NDua$K?NX;-u&U9LcaJ6vBksqQ)s+t1t@nkKv)Z=BS&uxHOGS@Xl=zlaDW4+qF& z&fX1c!hl|EFdvD8junWYV$!hN_|ppcc-Z_&b@Pw;`!}$~A2_3M;POH?L>zlr#d!9` z*j_}(!1jd)_Np$|yp!2DF)7fy415$Xlem-2_0Taw1ax&}5VCc@Y#T>GnhU~_nc{*| zI*6@uv2PX|N6ugJwpJzEsdLI0Ir3dMti4$7WB3SLi5Tx=@)TXt()b_ei-LbIiVd6d zH{V$OvhM>JZo51yt?bxhb{m<8KlZ0T!Zg3$Ua`h&w^Lq;OY+H@L>AaPKT!;^SR1e- zd$(c3C6inI`>^3VQ`gqKF9$3In^8KWSQv_=?H+b#fp{3%o9=qes>b&2!&X-6#j#bc zBRdgzXh%yCox1J)n~xCQ9%!XhUix#^D|(f#krDS;Xim=IJ?bp+?-V?sZR_&+bdw7r zm90CAO872n(PeKswwX&j#MtC66A(G#i->*RoP9(g8$}$pdIxJ8TDYAz z{9~;0CY!EC?~8POkANyFh#)z`4RgdBs$)B4Q*_+1cPYuOoGLI4nJjF;#8*rel#2+` zF*yvj(!T0ApPmwnp>3g6LvAuuSI$;g;yLJtuyzkNYTG|?fh!f?x@J? ztp>(rrde9BwI-AK*~prQHrwfpJXmXHl=sXEJsmn?pgY=XoN1G25`3tnYo zalgjO&f91Kp5yxM?(|q1eHGiM(`_!~yqyID^>W0JhGt%}#6Yg~Qj)aFdN)PrM+_^p zMOE3TbO)sgxb{yAM*!EbZ3V!!K;a0xgZT39zHRhmW7Y*we?LCZg6OOfc%RsqWE2$uxOAl7L z+8^e3d~@*Z`tSQ!*!5rVscM>CbuO$%4os+6^=UygCoVpJTW!zr-?~s@c!nLQ!8?DCz*jOE zi0eg?g?=Lm5^uoH3)n>wm`fzd->n@z^eq_$b5;(bOZwMF z$Vd->Ey_pF&$?vp;fgxgrHw{X#VM+knkp_DIXPNVl@LQ(ssHDV`A zM-sd2V2L0N<CR)GCO#LpwqKflj4m5Y+%_Ck+2&cxu?Ea0;7d{tI6p*L}%_3)eQ? zb6?}6ugfnT_;GDk)`L4X&;EU%Q%hD2e(O7|jR4 z#YSST9no5v0k%(Ky2!9WaG*Y8yeMxn2LK|weV<^I6o_yr!DRm!T%tEl1IVCLlT^io zR_JheprMNZbA&t%V2-vz8<@l9>H7%W_5#(+hglSd&QufAX7tuVWtiC3VgZeHEQ%EZ zm^SRH%rSHf26kJ<*sbklwAKC;=OAK@UX;ibO_rI+8-SZ4d?CE{14#H6L=n z7^iMA($>8|D-pqECOkRnv*s;Ov-)M1>E3lQL6=ZB6Rg-# z|A`4OF!jTJd@wO8-?;Z1rC`&z#hRr!RmUwQ@N8mN66;t+&w2%t>=JLooXSCG@_?)l z)gAMtdMsks0!oAF%E4?(V>#(2nybjTiO4s0P146yylniZ@_k`B3GR_9<)zCW3DJgO)3YNC)iOzp}iecc1Yo_dIiC?y!Zqb=5hgSB2x|R>xgeTyY*w zEyx9j6Pk=J+ldpSo8#hdOo_i@X=o!_Y=@?nbekA`GA{neDa$q=a%a;;@ky^_CSHt| z+ihz$@2zc`2&rXeQjg4p_va=R23Y(L46^vY(`@mdvSSi})S9N*J;z2%U_LS@p<)+W zu+2pawmXJG&fxF&d!|Dyq*>3rRA26>RE9*{x{<6ir?u^k}8 zdmsDAx2pCL!Q*a$rcXUQ=G_*{X$`X$ybI>N&`6XO6?Qur zH+>8~s-^Q;LMObhNqC5w4EzKEFH3rV$IQ9Gg~U2Vm+V|3tlG>RS$hULQPo$@3^Bu3 z)*J3!_=x%*MGdpNaZz1;a#6$35dJu>OMWD152qAS1~kZpR4d1!N0c`eM}`9aOdKjI z2kvF9?8$-|yz>a`8RXqnMSvgn%G(wYR9JmYUj0$;=W z(GM3i+@8dR1AkC)+!J)nr#4W5ldB_J^St<88MVEd=g0S2Sv!EM!>T6ez~f3a%l&>}iwoSpqaGk`jTjJ(9M9vP=G?=OR^)t>U0N_MckEsy4$Ol<~GER%?hQ zKcqfWjD?)e=tVjp2EH>3#ef538`#&e+3KEd)+XhM;$8{TmqbsmLh`zu(571|$wXHD zXcc7y!2*F&D?D?r0HW>w4fJ(8a!+JC zFMkx5>>vG-IMA;ms|Y(ZNEq6V-B?h=W?(@P4jQOee(c^1PS^0>Z*N4$OX59f&OUN_=yM^Sv|+pS2Xqya4M2>)Q&tPgckUj!hJUylX;yx zeq870Q)~0Br!jr1a1he7#>xWdQz$U*cr;ZFkVt37{&1M>LOCM;aS>Q7!(~)qEY&5> zkk*8S64C3;m0}Knt-+eww4FQ^BGou0VGD*USw;W?5iQ_ z%}6q9%+N@B3^&0N>RJu#EG5iJ#B;I_HfMSH-dJM9P4LO~50^b5<&5PXj{$keA0jI0 zwQCnkBVtBpJ~?1E`)f)tr91b(E6dzGrfm5K-@M;)@V!We1r@iA3-I|L?BVmj<)p-{;i>IgvX9qvWH}s+Y49-+557Jqge>qXB;0Ne_6$l zH?LjU2s+58pe?)a=g%nqj3Xa~zBEQl7_=WYm@L$BmAs`0ZDijT&Er{0M-9E`)Dc&< z8;Lx-S~8P~E8hC}^Dz6FBclLOTKr3@&!}L;D)F$g-4~td+`ZAtlCkSb5FK4#khKKA z)~J&N)*(kvchhYiTEc&ClCMFK#edDZl&jY^=6^x&EdMZ93$_ywa7X>o+1=d_OhA}_ zi=^RZ^$&|6_4%q=**H7p)iV;G%llNdXq=|nt8Kc61LwK&7%dvp0*JV&8Q*5oiu#=* z7c-79 zhF<@fwIHSu+f~1VNVN1a1+&)Vpf zRh;3=&ns$}nSt_DM5MEmEPc?dq!W{QH#oHa=-VCI}t1^&>S} ziW96bU@R`VFxZZS6J$Op4kjV7ew;GC)*%^E=(9cT|7eJXa|1fhX`IACrSs=mdb2d5 zo6Q6pb5PzCCL%Goq$fLrTRC$zDiL)`YF#j(4O$oQ!WwD?t?~9sc;f|@hMG_`X@*&+ z3LIe-7`2&3C}RH@_qvcE{5&M}14hk(eRa-8k2M%IimBazRAW>yYV{VQ=Dq_xE$XqU zea>wWt%c7>DzE6wg^XzQ2k2lo6HV!B`8V^P<_c9g;}x(Z$hh3CfMm9D_j_p5iF}sf zCf4Aqb`=(vFVw48&Usc-e>>d6kP{27?NpQVcjN}f2Zq@!PoYe(@tq!$qIG)~k|gBf z3tt4=v-d7?55V@RxOH1a6~7Ajw9X-?v0sUI&IZ>U=r^@v)Ocpv4CyGGf-0nbeR}x= zSPnfbc(E}8)gM@;Y>qXFQ%v-iLUvticDFx(xr8YsHdvibux67!rOiPMqnvXIqhq>6 z&tE<79oJv$)z(dq_*#e*-Db5^eyo5ZsU`6`(l;~a`e^=n`G@G}f`nYcPzfsqy1Y2p zVlfNBrai+09Xs&)7$0L4S}IX|D4+5CBJ(@F73*JLD3%1FHUm#a57)i(75 z1J^HU!vl$t0ZHlGN2MWpt06n3$f|uY2TJ{IBnz$g^`&@|G4v*5mJv>E66$R7I%%u$;*3L&vu1 zhS_&Fw9S(pvu~M!&e`vj=b#`hG&y@p^qankWh=Tx<|d@K4}*sC^XKBa9J%v+iH0f7$h+O~5tKcCR*U)=WA zp#86VR<~WkT-?KBcGLj438A2Od0QZ0BddI=mdDrT1W?vPj2{U+G8{u=H*WGRqXcI$ z8A79ea0Fr@Bg;mG?E)Y3q6)m(dMmy2!jzN9G(yZqGR>>I8xxmGG)ZvT_*H}mm2831 zE2_U5qGXUO3z3}VO>VqAg|~VF^~EkKJyNY7%XI7PzBq-q`oOG=vCqqT-uhMi8-5AsxYgYL}S_iI9C3t8_@1zc7#kmUgc}=QDUWSbc!mtZcED9 z^e}Ex)NsGTb9w94O<;O*s^tD^uPJvPJS^wV#j&Ino~!nUOZ!1^U)*Igdu*A zYoywYjYJG~GPR+<_5A)VUUDyFep`=vOv|No#B4=dI7MzA2l|ymcqQmD`$DgmU)n<7 znZWfNhA@Wjw_z=9{r6AoTYi+@(qr?4a2lYC&YW%3f+2WXNG5-g6R5TGbCX}1xz5k_ zyMe?m=U&8$F}-)n?NlBMzyHPyoA7zj^Qwy5o+#wNG&3W+d1x}FwJr2UvY2k&{CnTQ zF;&!Bn8RDV;CL%E2`XLsVh0SZ!zKr2e1icTGTdX?bF=)OyDRHoK7oSR_RC={Ih?H} zIUF6LBww-PqtmmX#p-p~X9y(7rW!C%SZ~Dsy*>sEY~%-LV3|1Fpg;mUydjV%&=Hof zUi0_rl$tM35{S(kv9MTio2=u~GnsN){%&T!39Qr9oY%q;UcVaLnzk@diG&zD0&dOR z^g=%@GUIEVKRj3yS<@11#y9VsS~lY|tO6;bDHUG2c_;raKw(B&&UdZ~Hsc#}y=W1s zVFr*#SXg+<p4t}oQB%6ontHw(qCMo6Bibu z(32RbQQ&9|;w;TJLuq0%@%pn%pTY!cUsEu_EAeG50*s@@i(t)CYkK092Asn?835;o z8UtxNoU`6*`>)O+-8SchVcIEPS`7#ss$K9pM8;`HRXr;mhy?=4*qcai7`^sGASpOx z``3_Ag$AEZJGI` zz#HCK^H7LU1v10d%>jZdpB1{KlmCi3xakS&0BtIe`1Rprq;Jd!S=Zx|7Pp6QaS>C- zG68`QK|_*Uq-jUb{$G=|jJ(0VQC zlLUzCBuV6ggC~i6o(mBh*ubC>|4H?%QlO7!}^q1pJ~wJSm`FN}>A;O~SrN%9v7Y3;`>4b{hon(Zyx zw)_m>?hu?i0srq95Se^^R%u$eV%4tV=+3?gI|^4fRiRtk;O-S$(XDMF0A7D!%i$|N zknC0Hl&zsN*pZ~}`booXVBYM~t(B`jZ90U_KDJ$7z)l!{=$V-Rb}G?UWp#xOlW%0C z^6a|ibg)lB&i|;1n^fl`iEPdTQVAg2!2AiU!SAYqO`0pKylWm?@*9MXD7GT4V(OAs zFjL|l-een@>d=|T2o)>0O zXY}QQoc7a}644pmk3Wv1KYnZJ%2Y|6@alA^FEf7>pz?OzjDI&)|=8+>^W-4!dOxId7j;0pXf?U8}#TzX*#*v_E(@rahf5yhNjmN{gtb6r1ig`G5J_$WT*V~_t%aF^W8$hz z(8eJEZp2afiA{HM6{L6kMl)}=8WYnyuU>nW>C+r`ytrBP%F;a~lVAtDdISkwz*|s+ zz{L^e&sQ8;M-Pdo|wB}IH^#2`MFIb zu8kda>ln-*3_h_|@rLPSOcj!#HKBU{#&c++F%K*fJ@uV`_*kSWU3{DvABQikrhTo+ zi1x2tQAA=pe}mr*$I}x_w_J7|wwi(+6PuRhOF~50^#SAiHUbAVAsqlspb|Ru_?S;u ziRKsj6_LENT6)>!bDI|&=jXs3TQ$q8_YlYL6F6FTsjy;v9rl9~3TRvbT!{%|Q2WaH zi(84e&=Ycm)&q)}F|hS}rTe{cv6TuE(SA(~nPEx6DP{KIGjo9(iQuxa$arm>eYCso z_oCljYb?YmewU@RQoh2mPYAz;(_hw97PH$rW*}`2SNQWbtgUbPe&0C4d$V|E>GjZiPt2ggivpIVF{5hSu&t6$vvUf_!cg4}==zEe=^!OsZ2o&~)AymI4Sf?cCZjg({Iu;O(l)IiNEW_h zK;(%;G^)mbDL=I_$*g{3D`s}Pv+0>zvu;`b^QBZ2ciyd6uPOg=RNKlAGJ~kLjHaQC z${y;*B$c;^q?8&~d{ziu;lMum183y3DDS|f6^9m))NsQ0bR&Of5^myl#Jwd@KLJ!7 zR-1UFKpDOG81dR%3;@o;3!V8%s;?w+>luk4 z7k92<9K7-u`t9fA9cPMT!omL5k%wV_Hj->v?`>1)?7=agKC>#IWK9)eOyxlI^xkUS zoUS&rWxcs+va3H^tTo9U`&KvJ=a0~kltRNw-i*DdHmF`s<+P*BUmG?w1i{=*p0bD5 zQtAjC76%tlAY3>tAYAz0;X5MiCoQeg|JQv-p1*`ljWlYePS3ui8p3Nh(X;l(if%>l zY#oEVBN*%(?ZUf+Ez_J)5VBt~jZG~cbtoSLB19Omt5b7Jc_FNL!}WDuOrZS{EPQ@Qeynguj=0i~H~Wu4?ekGC&L^H@I)az%6bj)onu2+>f_^02q2B!*j(Q|3d4^f+54XN#0_H&3H>%eF52+_r-_F*k1b++Qsg3#}WHuS5(mjg^l za>^zc&LMTSOvEs5K;4@tN!E|cN?*`Te@K((M;7j8p|oQ@iS3HuvCbzrk_qD7$EmIf zAw_3M!zb_tS<)MB9`N3GOfD6SG1$fHi%U9p-8LQ^aSV5kbpU@Y)gq&SkGB=AEqj0- zoO@zQsuyfo=S{~?0|)L#d-|F1t!8$6oh(fpS+SxX763mFL)7_A=E{9HEb`KN;#dD9 zT9Fa|v{t0z+;3Y)mv@{vY*E(W*=Wj^Hnk*mV)V7R_`NCdpTJB6<{~|&mK=U+!b(nBtjddQs8(}YB_`?%6m=uZ{(BSVqf zposiy|Aduiu5N09x5zGdi<}s)$UiM7(s)co)0$Ay#Wi;|?H-YE@x;|lpN&Y^QM|gT zc|_ulJ8o+V#x0)@Dp{@rXs0!&u-c%kP70|xSz9sQ?}ORQx;6BTQF%@ z6@!i2UJTtX@Ww(qCN1=v_4SXu3T2fGidmQmvdGpW4`IQyCfh9fEQ<{Du{-WOFgH@E zn!9-@Bd4$0-T3U8(vQfA&Dh_AHZ&4mZTzccCaieAtUvSTNO>elVO}_{^WBp7r)Io)i!!HF51y+m*|2=2o4~5VEf4_iS(h zpM&ZxF;0q4oM=$r6lHWI?Ubor+{wo95yIv)ZSv&))ysG2ia=Lk8|A}dKwPPlWP`3G z2*kb2AnxNJ5LfQmV_W$D1H`>-1aa9Y%){-VHUPX?ov(Pfi>LQN27EH?{>xS*)SO7`d9<=`b=1wUZ5g0r;3w3FO`}se zc@%=et@mG`9l@7HV1!v8yCLdh=*nn!_OW}q&mx;V+yl_l%crZ&K3GqnEDo98*50Rz ziU)gm;1H#ty>t1@PvM&FzCy)=ZTA+G#UdMh44oupwPRWd*>cO2 z)s_)>B@#t;9U~bSFwx1uvjco-m>|;_7KP=Nj2Ntv=hpnS)mW*I3Fg~$ZKV~8+M`pD zX<)hVaxWA@UIQY7+7j4DjF1H(^oY?Z@OsQi#DdlBp)${TM$dx7GRjZGB=dBJ z*LvKn8{pKeoj(yEh=csfi6=0PjZ@wy`giW5vdL&^1xyR!`4Q0cZ)p9UuTV z{09NJi_crVIbrSQ!|rVQ72F_K|15lHPUP97^yT?!n;t0$?=B2~H6U^qpxv1CK4Vh; z+?u{1bVYLf^qNo&NVadLl(9(6RmUD8IJTGPB35>IS-v9QiM#@nudn#PEs zY^#`#%hi+xrw3dE*9P>=ON0?YK@dr7v{leXqE3&3HqvNPu=v7941sg#TL^_O?kp^-^tvI$eF1rk++E^(i6`XeVM*tA}*b20)H%sgO95~m% zw1{l?$1+dlkIT#@QW~lOmzDeBvaB7sS>+D^=z26PS7?khl^xwPgVc~6WG+Jt;NKP`Iv`fq*gkvzU$7vFDWBtjn7Fpg z|JN-@&gs>%?Qyac#D9oQ9wF_zSXt=td>7}6Kk773h3Z=|#B%E4{rM(nsNflKPnK*^*C zHXJHg+nzXMJUVWFQ;t?{vu-+At=z(O&#g;ITRu8{#^|)XhYG^47KUFM5IGY>TVC3x z2MfY~Dhxk$N>*u6IPSu#xM>qAj!MrRc71N$bvdPD!f_`~sMt!x+b_)&T*K}wpup@I zUw&OeMfYC0ZMVaPWLZI^5w&dFP|G$Q=Kt3eSJa;xiWDZMZ^{@H**Pa=)13v8H}JpO zg2dOl)zkONZA{n&QYPK7hI&Tv* z^_nJ=?>`^&EB>Q_obWy=7HA64N$E&E%>3c+NO&;A#R>6EwdEuE z6Y*`-xW`=4_P)|gLQu2U-AB%47%1sOt zR4)SQ9oR&SSci8>2Bb6na^%>tfmvIr{@D$`a4MCybsD44gK5(Fyhy3@a^+VKbmoV7vdsv+fo4p7oS9oS7ih*reF zZlOZ`8=JQm-+thU!V-b%h1$=D`N+@Md?dQ;{E2OEY@jWWBcS)E(g!MpWkzkSN;YqdY_QV4QFl zw?ky<5zY^q!Za*WLg1#0Vl)%NExloaW0i<`^TqIIK-x<>Y+E`dZrADUkD~WwT`WWBk zL3cjhZsIxpb$k-aBn^ItRs_0!%%|KbQXvUFsMa_?iZ6pDLcHQRdcIrtz~7)0vy7N^ z_w3it16d4&-74GWjCz@gBVHJ(o$Li(;XCHCdRb%NRBK*o=Rsovi2P>JV{jEeh}fH0 z$AVn3y$;s;E`Gmo#}TixHBuzkTdsa1c;Nuaq_8aYpSMunASq^F-eqq*Yg|!>sn#IJ zGg1GFYAA;W)?xnh1lx8&guB^gJC7%n+IpFZ9aqq2YU_1^B8#g9R3YWJal%&buR5Rd zlF&gm=D&hS&1=uYLOBB2u>BK0GrDwXJpDQvyEX3b$rssQjoNB!JLNaJasBW>=}6~} ze|#7y9chh2E>FDWJ}x9JAN(>TM-fY@U@~wtj zWQAMa|0dV}qCe|yRFXUMb^RyRCzhEV5N|T4C>p3a0Ub#%hp*U8ZRFgPoTliAapqz< z-d?4e@R5b>AQfprXHaxAq{)Bz=!NoEen#3un--BNd36fH#`4TWw19LoW(rIRo#edo zJEg)iz>TDH-g|HkV`;V1=ikC@=t+IQKX3!RR-Sp;CmVYewH~R89k%sXRw3Eq6p63@ z^4+WG6p4}CmaYEwWc3fY<_*xG<&-^Gb$t497nKfL168RqEjmHdAk>MwsC1|}#-ImS zO9Fd&Sdwh90k~)&fZGxb;G(eruDAsX%IXyfLkBO9F77Mp>@#!+?gv!*Y~=-v%^#!m z7VqB_uo!U@HXU$8Y&V_R#(qLE@PeCEGdLz$K8Nc>)VclEdpo6ld$Wq1q9zi5}Z~v9TI@Y?J8I)?*uy zv@F0j?Bh74Y&-S7?|@`9hq+=)JfCxVePixz!)&a{z(9v-2)2JROA;6dG1omL86)Ts zwLmf`WbJFBV?BpgA9`2>buq_H+@dHM6N}jfSS!7I#O742%ZQceOtjE5zVo3=7;s^= z)P6^LE;61~-Z<25gQ0fp#J7%>z|{(X@#5Ivz)K{8Gb-jLEU|ShfS7-f~ zg7QFjcymB)JwH@w3X0nB;OK2(q>n`+g@t@UL_+@+^hpga-)dIs^_E&_u- zLg&~3d)K?6v3PSh?^CzvM!aAY7r3UQZ0IbjC)ac4!El*8<+zP`-Izt&Zwv`XU&a3E zxXnyA4`f3xva3qKs!Q(}?B}V9ONL&~l$a&&w!QGY4=JRJ-W7z=-PW_#JP(A%fN(m? zMbLQRWwg%OmpX7~v7naTPF@`U?fq|?$a2(aV#79a?zYCx!xv{KY&}@s|C5K@(*Kdh z%Yqz9ZVhlGnds4_?bV;QOIzuT2{8A+HK%PzRzmU{39vy)Z%>*sZzU|(gOig)s|D#QtB9eRzNr>AzjT&n&FNwdm0RTNP(`*HOf`uZ$pxAkULtD_37TZ>yuxxt>?xVBb-xrZQqD&5ZdQdH zb)?>MGz|L@z*?1Yg1obi1ydDs|K}|$l#~D#U`$IY-@Ybr?82y z6X8}7yAK#3nj=cqfids!$}55`vDNP80Af1gMIwDN$1s*oL`U3wuewEtagDtwVY2H) ztoRfNOhmYbAV8K?AE7hT_O0J>JwAuRaX)$uRh0B)4eT1>S=~pQxhN5u8Ge?aW{lX? z8fMOtOS9<|3aj122=FT8B>Jh@b}XLnY=Wkf4iug^#}+-bIZa+YgTjt&kKY;n?fKm1 zsQ#5-G_1O!w~C6$r5m%sbL0@1!a6`BX#;{td+fQVWgl8Ml-BuPYMIOVc0+Xs0|`R6 z3Twx(J0m!9LNM$05{aFhR+ihn+Y5LFWENAK@o>(bj`N)#c{f7KfT?@MO|MxpF?$eZ zMfb1nK>;D-(S<}%84LIBBXyRVml9FM&eJEL&Wdpq5VPBgpYd(B&7{}aME3DFy0$ld zRE?WMuq&9S-UM{av2N_RVsgn|3($d4k4zpN$BJN0MxaHSUhXHLXR~6Uoghu{Q*>(v zY57{38#^F4#(w~5el!He8z3!NrEh9kkL(&A_|up}X6sofvz`CT-kXO8me>PQk97C}uTU?PO%_nvca-Ky%oRfRhE{o^;!D;OA z2h|5kO1~`lwTWZ>Qnd{)sx*sX?P7Ar z(~RXGZu1tv);P^X&MPu`DaDws`+*RXXg-9D6xz4$+t}fII(EdYqo#q(If@#{Z+{LA zWYE+0<{ypw-h6{Qq1wB!_pB4ltwp+Zp@` z($5NIvNZ4}goQ+0J(qBR{ql6cN^E`;?s};?51&{IzhJ-1PpN8APqAS$WB(kO@yfyZ zWzsf_AI6&ty{maYHkU(n(^az5xD951^d8bU@+!G8=;e(>Ow{0(*`(OS#(-zYw2l>N zg=go}qAyKiuFlmvQoJ4SJN9=}0?V2O&|%v>NqBR&VKwp{>A98u49+9!d`C*Wd`Hp} z_QhtVewReSY3_h5KsFk~64jt>(bh3QmdiaGwC#6q&;}OmOT7$I8T%8b7-Gdww6fChPTx(2yyIg-bT?&wz9+{M|bWV24cENDYO-{GRekNLP0cT{HCWkW50~e?(_he; zhpX5W}-P>^v1p~->K&D|0~;~(9Psg&^MNKlwJ$Gu;SDiLMke*Uj>t~2KukDnc!7bBWG65#XMCJQz!(uF5%?=?iSupjpeZST6%5woz z>!yFqu6pKJMvg75?OvH|^cRWvr0@aukvo?(lrg#X&e0;*63V|hf38+35sI~b&idsWUHgH*78Rp!QjXOu9B^%dFIksdoY%<2CG2&`HGjxz);8$tgk4wj; z>j%BXWmB##*_dfM_nP2fCYyQ|^qpM5WYfj`8u{H#hFOY8wi;MTF=R>Oz9UBu;?Sc? z+tMFldE?EirbtH*$|fjmV&UknAGD2#s%YeAz75thzYWh?`A+^{qv%cvAyG*7nIOd^ z*Am5^p=hlE?@wou9b*}=i;LDIu5N<xhF5Ftp%p;LKl7woKfHqOBp=qGa%+&M*g$Iz^ z7HG@H3W!sOm8E*=m-)A|^h-iwsrZUKBtZmHH|l7aj`eLwtZCvKfxHMQM-K@d<#Ks{q zv&>+4{^Ys}b#ib3pY4BP4h=+3{t!Rc^kmgPqE^}j4fq67az%W58LRD+f zUE5};_?pa#ZRoDe$Ew97_uDrKQ_F%gHUBUrW7ocODRGXJXh745mVq}$jg@WqJ{8Md zOxDPO7l?K)ny7ZL0^z{tZs{+4M@O(F)RMSc`vt1i z0Hw$s{FQ4COkNGO_X1(bb2H*kSL+(^NuSxm_aT)c@OG4mD@VNp-bat2)!0IYsmYHe znTv*CnsNGS|8B&cyxgs+tjI-=I6>*D9We4Bak56|dE|-8`;J~;V&b!wB@OK-qGk3Y zN3Uh}XS3ro0-uZweAFj&b$rT}+?2&I75PJcAk-%`6poTZQzj4X92}jJU3*)0&TUyG z!vnsN6~3XPgF9g=^82ikTp0NGJGFL3R>@#t>OVR-P5S(&_nO`py{U`B=T)SaG^-WxisN*w9@t^mj5t!$OE4oPM+V35tS3McA!)H zcMqg}xsa$e>bLCV@W1<#wo`i!V>sm`etcorTT_B@*94CjTEF3a7Z2zAGx+u55q-Ce zcwyMa>Td|H{6uao?{i0(CQR!O>}}`BUvXeEBrpM6`8{XrRg_F`$sBS*G(@Z*S)ah ze1c50VRn%Y{slKpCr?PAD{>BT_wq;o*XbVJve9Vgvk&9@u1f1yzm59<_ zeDuEyN~VU&r~SLI_FfpBY+Ga;m0MMj+MnoBa_SGujrE^5vttGE2b$yNS$4vbyRkkJ z&JYr)NN8c*&GD3tQ=EF_$b|0w@EJ>T%EF3SbTY(Yj-h~T@p=w@8p9o+GP>@9E?vM{ zH7ETNzDaDLcnsN*dAUT^(5H@8a#)!N0|cA#lRD_-B}?n!|DW26{FCB0GlZir*`D#X z&aVI3i$vpNhF&ZKguTdXqV1bC@Rln95GhsKE^rAPnUL~}{u^d0%tWg00GKvp{K3LZNXc_$Vp6>}bk~~mQjRNN z6AVZcMlx5Yx+DxpaITqSuD%A}`(-?^&@7Uy(ecQ`uM#yCiGCyQCjsRla>WDO=Hc)Sycus(on#U`vF#tr^la%mTQKB13Z1ILC z)rWLTD?CVOVKSVu{v%&vFOjmiy4Wlp^8SgHJ1|0}x~ADjAbi9vPxIRu08s9~>J!k` zPg3qqx5y^Du<4zEzMbAbvth01q&;%oNN$5XiD9h7UIufX`M1|Sb zI7U8=LLIbLvf&k#X}0g}2n|NqEJ&oE?VpQxZ@XEL zHBDxYm*EwZ?r6}-l#?v7Qlo1=0GFg}Ga5WwHwVEVYd!=$Gt?gp8kUpWB;M7?4xiD} z4?9RQ)jtL=1I{y6t%2mM+Y23hlDKl)9H7c%U5RaTD4-=Re~UQvpF=Prl$}3CFjCy( z@EbeVQSDm&-j97TeUoBNSmaMYGwp;4#ofk)--5GaV$7>Wa~kKN(OO1aDmrO*t66>A z1C6)E#LrKUzbQR#$;73reY=x;<|nUuK0ok%pV0F#=FdynGAudwPx*oEBLhvBbltb} z2f?>hOY@H$zCC^(^KH=bB<3#84^8P6Z2YS~zJ2RLf7$P1gRdoLw_ZImx$T9F*whP` zHXaxhms&Na@nL`bk_B@b@At>2E_kr<^FgtxXs7)V{_Xxb;o1eLPU|!#d}V%O>N81M zwKK9?dn~qxe#)xTOEet={n}md+Z+DPxyI>-?Pwi=4dRTYA7vB#F!k#h(>Aj(MHqM4ej7S)mhd7_^Z{tK#DyEAGb5 z7{@M(J3*9mb?4WZw+Ez+i9`!brSX9+rHJJZ9%j!Fv&hVpE>^PtN78~8_}tTn(4st4 zoe{mpRwO1~6HbukYbdZ801Ch-^K_NupKqbpaElkchPQD8Z=lyCz9xF@MZ%v0(f;7G zd=_~&%1(k4Koc+Pd7i>A#-s+=C4%wXY}eU$}%= zHV(h871%X1DRs1f7I|;?sdTY^T)TJNh;`6=WHUToPKJ-1tv`cbx5(Exs6Am@2gvR6 z&E0{x7S3Yy8bU&>t4+Mzd!D*$-eMV!{qf9X`SA? z^Mg6Q4=$*C@3z(tR`MeX4czvcH9>Qv`Lk~(bLbc9^Ufp165eOC z4@{us_u}wpBSrkmNp6oC0D+&KzXFv7Z|aXF8AXQRjecZQY{Adv4cGjG($m8WC`C;| zN$Ef`<+@Xwma`cN^UA0<8>k%^RUicKkST6DJ_6b>26Tnc#XX?5BsyTaSYH z??M4q-mWQbF)sd6+|ux`>xihISE7XzSpBaUTLT4eayTNH9`ROY_^kfjydSqTI~pH| z;W(mwp}|=wNnh3fjyepAz~G zOYwspp<<_aR%Dp8v9Q=FYRnQQNYJP@(hdm;ZNxjsu#MDDagQEC_o>wlih*UhS-SAh z)W9cP5=;^w><*ZZnIyTZXfkT~5AjF!^xnieHI7xymb{|o{yC-EYO6vfuvjMfCRIUl zNTD4tj3G!ouWBpFX;~161k~n?#Vxd~UB}fBb!wA@So21;)$V!woYWpT>rBU4&B)@? z)E2J0P!;k$9#h;XGZ=GAU7&n)l{L}Tq;Ma4S{!*}{(CNA(-^@9@f5B^`^U+5?NR~WvD z8?XJMTe-9!sT=GM&4E7hi$woMXd`C>wMm&I0l1G7{Ry}_>q&pry5ZpqeKF(lzYBaZ z3G2p$f7i>OkX01E0sr>q@KyqF)c|ml@)L(inPk8GH)`{~$}9|b!8J1U-E>_FC<<5M$1A(}tMKE%!HKzZK^pFB{Oa#-l;;z6~_$<+?F(*T2%`aLbhDt3}n69%z+y*_66}N`0D-5ouYxpFvz2K-^}5 z=Nj;17s7Kn;td&;NUD$7Jj-VQy^_X|kMl$L*m6fvD3G*{YyJ~@$%>i&q%>hda;Iq+ zIW;T5OCGIj`l01>b?Sv2v;E0{gI8U|uiucbL2Nx3XT)KT8(iNSx|Esw6vh}*$K`sx z>bb^r&hmk9~NZnn0!OOX_6K!%ODElgKYGNmCExc)*s zLj&{Z5{WxGZi`I5tJmyUAkq(!&*0a^JuX%y%TBQvAR5(hXaiXP7vBpLlJ9Wb7=j!M zy)@C-bvUfG?_u2E`p&ZUr5$Y)trHdZvcO~s35&uTMy97)VcI5w(>kLD6|@4JcI3JM z1q?lIG3&MhuMBc*EULz3VF%Cbmf}TPo97WEqU@gHDtN|@M z7hRFi3SdoJduRr|0BcH+6-krOYX)noK*AamC8@sSn7{gSUk#PDPD9N{hF1pMg0W=; z!z9I+=2T^d$=D9GNDaIW7Lq=c644b(sssj}mmL)|$w2|bB=gN?7~x zzD!ZJneq@)NlK6z_|=?AYQ%5}!4kD0)OCT6<(BM}nwMfQZP67CFg~il)M}2(Wuabc zHD|Ma%}Woi$r$L#qdu~Rs(LmPQ-2$I99WP*WE$r=T4Va`;PJ4gDn0(~^th`gEnOX3v-;Qv8h--{ZfQoKQ&#A8 zRHo%6uX-*&uypr0e<5MONOX)c*@t&5H5x$oze9xonHEPyoX6MYzDj5^-%^4lM z17Pir?AqJ=`l|rW_7oNGE(`~KG52(fxqX3uBT(7)!f**7*|-Y{oc&&aGyDtJuoZ91 zkCoMIl4l#(ZFu;9eKB?CWrbv;?7UxOg{Grtc9Ab;B|uq8FaJt_vRixkr{jM&Kk09N zb9gu(|MupX@b$gQ6ZW8J?Ie(NG5#jmD7(zJsq4Nq{}c3=9fNEA@&EQdp}Jl9;VfTF z$C5swyq)>sRQ$LYSGVVfd*VmKB{JdNZ=_Fr@Ug3dPcb-~Jf!d9A&l2vyt=+_SK~44 zXZLDm#I(gl@IxVp1}G)g}eAQ^7RlL$|=v{NMtzjZ{m-`FpAt7%A}@KCJ60_ z37a1I7T~cYw;?#ksaSq3FV#16LL4GtblYi(0w?#ZEU!D3Jt*5#E6ej@i5MiiKAq2` z={tG6VMxN8t^K1AfaG-Iw$uEH&3SJtZP${KMrb-p;3bM~+duoPO{kt6irMI<(vwz0 zLgE6?)^2;M98^!k%5klm$52G!Wa*DH`f$xW(~w9D(&(lu7W6qgbJY@KAUjJsEc2f8 zBx?qit8U7#zZFum3Pq(|WCTn*=@Q@c{kIU2xQQl_UOyBwxMRqelDu){>sU=$GNN07 zhF7cGyBi%xlC=He&;89BuT2ifV>qAc-*fk5FhGetu+D+gA%orWam=AHPkkq$Aei z>yKg#5lU#XfVhOfSeyMeAyt!NsT?7*^Z+#Y3v>Kb7-vM+nh(J-M(f2BS!!0$i~f?A zS7f!xrr8_t<2l?vn*&hcEyBA4os1gp25q&|40Cj2xU8{6W!uc(W*oxoU4JrWZ*+*K z_|a^-sWgpm;w?jmE0Dmh8$Xg33*tpf;6x+4)}YsDJNWyMw|G-gBVtAx;u1pJ<~gsn z?MV^gtNyD1F>bJ;0Cu)T!eX|o*-z>|hR$y4J|p1PMoKu`oO}ZS7no`$Aq5t= ztKMv~0>*!^SJEoV0t<#4VVDbqp3kN~A}z9l_%SUTWJai<#~k(5VrW3r$(8Fz(HCcW zOl%;Ab(x6Ct&>|w2$dLh0S(_wrUl#0Zrl=*7Z?6pSz4er?-JVS~^9ntfMqiaTT z&={ZVK{*{D%F4jK0y79TrQ8j^w&G$nRJN_6{u}0;tiQadl9U~m_3{d~hglq+Yc?TD zTSSn$=B8-H8(8&wh`E}S#59!-w%U?q3g?;t5SRG#I%6}g#wrsn<8<%9G&AcY^}twd ziO@;v7xItPNfxrmRef3N%Rs|7ola6v%|8?vM{cy{7r}*>BJPl_8Y7RVCN7l#Br`Wx zk*qLZds}ff%x*t8r8s4H_H?KtlP44>jS0W!i}@}wX2(MR$^k{i$4w~idVXT=i}`_5 zvqGKGPkVUE;^E1w{+u63DhM2(75WeuEq|o1b7Am~9@#l%StXYReA_B~pN$UoLqF|1 zvvclDzp44Oa{qM+F%zr&Ri_md?<*?)wlMsFFXq@Pe={)KH$}x?kBv>8uypmu%T{k% z)EE;Je_ZeQuX@ET8NYP(TZGrX$CqC7FOj*ae`DGtuG&S6spG)e(DYq52PGfy!Q`+r?t&d8n*YqX~Q8h!|A#iHoAbfOTU8S%Cw=`yWNE4nmAm~=MVDLCv;iKbyC zNH$W0j7TTi&q8Hc*)6uz)1fau&Iu{sk&n|^;#T6 zV|%n;cOx=9*TBmabg7_Z2!I!A*alYhTNs{Y98w9=gk;{9twrMKM)BEe(+HXsmN*F^ zQAd?}X?l}OFSb}bWI4Pct#EDa-oFRqs@Bg7*9YS^u3uD4#wY8%%G^vNZcQE*myDx+ zQ};t!7v4zzM}MS^IHJs1vyfYAC_EA6o`XYLbzs(<)3`*pl=3;whqhfuhf+g|`maAi zj;-TS9U?KF+qR>q>^&BOwe3!f%NxpPNLx5Dt}YjZGlF!(G}5N6vFb*wK^gH?(Pa_l zSE7u?kQ&yvaDo;R;V_fI_KdlCVv0Pt9oNVY!2vOAA!k7gNzLS3>9z6rxY3cSim;%C zG(vMn=q#HgM}>A~=mvBsAhtD7lf$p*HI)lD$BM!)M!^W`6+Nu*l{p;ZQoTxa8$#a5 ztOe*b1n<1WWmec)R>grDrM2w1Oif~*7u*L-6<9`=nuvV?^xAYD)_H>%UTN;aEv{%7 zfC2s!qFJ*F^CV7Dudj!q=}^ZnFQ@Kgjw5TMZ>%A%8a@hSN3IsR#HbXH{|gR-ZBt)i zE_2@F(o~_Z(yR-0KTvsf^v+;yRbG4VbKW{r@Nl9E%A8>;M3t8=V`g0rSb?5>7w@Fm zeT@N6_on4&W^bm)Kim$^(G zd&Ij$SvhVA*euB}%wWo`1Z6TwvN>l1zy2vaV^spcl%9)AP*lQc)7_+3kXC-+bbKIA zz7;^usRj;k86QQwk7i}?`$+ujk$YrnN3LKS?3YKE(rrfL1i3T(zxN{9J@3jt@sJPs zdU%IA63rP%L!k0$%g^8Ct{17jcie2zQ_-FWE##b+*uk6n-;i+7ODpmnIBNHUY{KmK zJhdW&=x@@dg#=o|w*Z0HKo4RV4YOBv#9WY69(Jy0d#`h-d)Ji7Ba(A#^8>vL0w-mK z_MmEQc*>Ui|Dtv3Y)bC$1!mC|& z`HyUu{pCB1-AZqX4R-0C-TIG=*d-S&ZES&2ggsz|F-v z78f4_m{33G={v~^!v&$GKDqNXa^-Msmu;Bb+vSG}si9rAX{~Q3(4ZFqc$tK9wso=v zHS!0HO!$nqz`2BXTX73X?IJ*)dQCDiei(Ua7+l|hwfpwbMZQwp<9?aZ!arMz#|-Sn zmv9T@T?CsLBgu2z3ZvS3BRZ`s&+`jnEd}Xo`Wkdx8x<%?KF5v;&7wBkQcZuE=8Xi| zM;MrtWJq4mH7M^}Xe4-uD#az-Qr+ZU*CrGp^IB`G0MN3&STjPPvqhB&eWZhAdi~J2 zr*YT|{N~8AK^5W&)DrYSZLtNYE%yX!)!smDc_dIPm=*0CxqrQZ7@L*7|0d3SRuM9= z=DMw6f(plwwiUv1>cOf%(9zi3BVP_w84>Bq@I%F72En8`I5~&OI!Gk}(?m&x(Ohb& z=566)?Qv7H83hm|sW6uF)YkYwMh-I2EpO;0X{sr`Gn=MW@Sl*Dn%VSdMLyF~W~thd zVqgyCwFi?Kmrt57em)ufMTb-*+l%||_?vVFZJO|i$8!oJu zeb}gD_V|~%B^#w-CjjH_H6=3kb6pP9OV*g#7BPs0PCc8%J(};)j5R(<+rN$rCgc}m z>{=>se{&DV8VcF28PPxfbfP_v5Cfaj-iqs}85<#QOMcg4EYD+)`Rm~?0E`86%W`3z zUjqKf_eE?@Oipz9znFdS(J-TCD;I2*@lsnc=ROB}r}+X3i0 zNHR@WCpT$Y#Dd6VWsRh&qmLgkZ4Ke4K21wuF0XakvPgugO={ij)Zo#a}o#0tYcu#y)ef=S~S|MruLV=TH@7nE$&D5Tuc2~WRTl_G>=wR*Ja21|4VL#ISvpF zOj<|j4evt7(tdFl_T4D^D5I%bc#f29$!MB#I~ydvzB6>%uvN15O}P8-y*hY+wUt}) z`t~IKW4JM!tE7$I z<@L!p!U+HigANBR4a)Oa&d{o-~7nJnEC4Ah&Q&Wep zm7e}-CnO_mqrB~%s!{fgRS)K%+Ss#3t;!Rd<)w~KZ#rO5l&O+B^DW9ViquAxy;oK} zbsK!Pu%jlNW(1v&7$|BZU80+&v~Gj(om=NMa8oUN3)^pdAYA%nY}M)XsBn3&%5{h0 z`{ri$tMA&iZ*HH_sGK7q)}s$Wh$Uh0%f?=O3GcwU6nO&UHuRhR{bqLd(?$iDh4i3{ zA7e$AtF_3XqVqb2jB<<2NZmPo^}dx4M#pYeR8i{a*(G*kVICsOp{z+>GdOeU>s+{& z#a_ZI&s{o(HdXAuWd;l4WHny5?MM_gRT@r>&LV?T6ID~qzAfJz=_2{{jI z!$jO`de_?`2{a-0E~KE8v@B}X@NT6K_F!>X5(&ggA9=XE;o2S)*p#AecJ+1qBeC!` zYl($2;(9GFgD*hI&=f?FNDfQC;HH&lWQj6Hi>yhGs|sHrr<~5QilXXF(8d8Wo&VCd zKxSA}G{3+=wT7R7dE6@;ATOy>$mnS)H95&qrB%D3MtrP;CjkU$%Gt1+lYaq*YD$&8$BXl5%5ROMLwEnx>9aCZU1H` zKLKgic_h4=P-#XNcbeat;pk_5bN{c2`!$T{fwzE`_4rjBWx|0(h4-3>UpSD+y+xKx zMQkyWr;@1FGW(5ffhjd7tgRpgqVP(Jx3#Mgy9Dqhb3f+JsBnP1OM@+Tpfn90OS9W1Q@m|ZYCy{vVzmM5C^hK=){`a{hoKY~q zbesAv)wt%Vtf7LlwmsLVt(sL+>NNT?uj@3eY1aj^1T*Bc)R|?+hCmw4h>c0CB_99D zqn2B9S`rsUBv5cm!-dxh0|EO&CcxH{hneV4W`* zo_g|d-zgQryiU949-ew4n6_hZtq1k}kqlKx--T_;tvBD@K;Q?eNAA+>T)37dm3hwyBCTz4@nPoHzN?Bm zHWA!f?x44D%i?0VV1Ra-xG?TIy1KYHmR~=!Fz(g&KCIr^vXKa%pD22)E2EDLC4Dqr z+sj2-&*p9A>93{DcsZ1Z_fKm>!)M-3?Imi$ED{j~84q;?`5x+-YO&Rxhqw&2I%^y7REO*l%hGkLZeHddAj8IF~{Ggvh$ zm5Lv}cK->KnW6mOMx}zx6qjh(KFyqC?tRA2xWQoFau?tdILEe}s`M|`3Tv$&2kZ6ClzXU3e z=x*8Djc(Z}BAj^v&YQ||r-C&+Wc%=#WbQqc*>tgyFkf<+ft|0V-qSO$2-!PjKNQ(1 zn}nE(NponFEep4t0|s%{qwTu$&Z~?F%)rVT4%3I_-;Uh0&DTaeZ97!OGx(WQsAf=9 z+5i;imT(_YXSXd5%YRaIo)W$xe5^;O8O5(o71Bt}b0p*dtA+(|-srYn5IRO#K$3r- z!f3W2RTB*15`y8G+Gxs@;8~OCvNX#4M{!=}kStV-wN;pnnEJooBAWRnyhSp9C3t#W zX!T!j5j(0EI6}Lkxk^w_HAl@`4i&iOYe8Dq1amUCL+Ng6kyH;QdBMR-O`SC5C2UZ`3^hpvk_Q}t!`4E~@^gGoWEGGBZ#i7hd2GrY5J$!nfspdF?Nzb}DKp(B+(TTGH&6@`kd}uU zW|SfvD@C~i> zon08*41r@XyEc$jaz((`_4>l%W5$I4?2Fk^=u$wP} z2MHTj$`HvSSQG-iL7xo7yHIL?&=SK8ehoT3kj+0em4M6Qk4kz5$++1-UCGUF>K~#W z3ZOzXRs?JJB9)E_J*8+uC9N?PO=hoA1JgB~MkK1BjE@_9{IV|Ok1Zsn;pO@13tU(= zp=gXG@ziTDczF?wn#38(Ifo0b3N?*cS-x*Sw;q1)u;6|WRLg4BJ3Uy-6MTu+ERb0Q-9LdNjXhPgOjeB zoYQpD;N%rG_&j}bUowcap+uxnfRXzbK2EPD<&rY^(u(KAnZ}7l!y8+ZrBpuIj%ONR zr{0H8fjOg?A`*V^{ACdaj5DOYmp1;v90g3HBwO@pgiDzu++@oUrJsyEYJz&8}m|=DhgXa$>2Ys0zYogKnfe=(`*SoSqH0YvZ_N3ROG(<5L!MT ze&hCb7GpSmP3YHHH>oe$(OGG`q<8H5Jw57;``3_e#bl5bSZ2og<{lqGky3MF*;AUS zt1ev%rOD88?t3ZwvfDNp-X8lu2OFXTmx|Bryno3aPxbX55C^A`tHueZ zkwuN4Nb`}{b^qDM+HCAEJF^+a{-@{vv9>hr_N1)VnDp4RiTMA4w760U=yFE~(}ZPY z9+cxQLxXkVYgbJ{)03bC7i#mp#_%7+&6fYEo;XdkjsO8jIvbCk1K%+d_ar~`Nohq*tIg^}j| zUqqTkLbv)Kb04|X#eL);=|-$M+b9@O|13~%bhgb}Y=rK|4RlgF(nu&<+vDv%f?Ezx z>a082aoEfLuQh1djDkxURqI3K#fTDId)G8g?}i}ZbEJdosk}{gVk;|wmujwXMo^}a z^e7hqCkPEUktu>ZYin1BiVT2K&rq3cz?=h!Rzi77#I^CI2g0%!8`ZX1#gecWB!3Ts zuTasG8Bm>}lGxg4jT$-CJCz%mVIrdcXy<7e8&a&uiD8FkY$788NXyPG?g-i-7_?Pd z2>8%XIf3)VrdUVx53uH+s|gy5>SlZNE;(d0wUQm}wZ(vlijmf*Z*WDX%zbO1glI=4 z=B$E+Hfb917{q}^eL1F>^6P_3(7wLPV9 zEzO{)on(Y~CkFt>bQCP8M2sB?5RI^#FdP6l8{}(uMI?R&0FRbAn#nKs@~b59ORUqN z1s@5n&DSzbyPR7A;(i8a&Ke|ZcP~pnY*TSBZX-rHGtG^ zpRsG-xs+z+`+t(zWZim~kA0Gk=cs9H+aI1fxU@QYKUh?`gEu4EY?=tj;sjUwpSkg= zOE|$d(r>sNb+(68eq-Mz1$v%)3HummptQ^^fStz2{oFlNfKsxdqestCiTAiz{GaTh zlq`DGRVn%5?D&yC(c=Hz2lOV9wbCTA0X?)s=%Fn;v61G5CZ zyR_>eDsHQj9{+Xl_#ZE%lGSJzw@sLjRG<^vfG)vxiP^1VBmsBMlb^%wgQ1xdT5CK8 zCiBJh8*k00`nT3Ub&Jarjo3_{^GlUY#Mscb9!Mn6gu!-3?V(XFPr?rv6M>IXB z=sIdu6NUb6_=0Ri6an`J4op&$;4?}UzFm}vtGK11>=r4X2G$fy+XkZQy5Pth*mPom z;UoHKzfbR>?WZk5Y7R8)@1$6`O}OPZ?lc@0Bq_%$zj3Z^qg(#wUfw2y1vi;w&#hy2 ziD;Sv>Lt{PO>7iA@LQ=9n~bL2b&^0vYqrH@X}tZPF}(e8i#JU3@*e4lGkW~#i|u7? z+wY3$p1P=e)7+TuOBVGB^#`TuEtHgul$^p*n;$q=VYugI_KrN&-zj37yevx@A`&A30klCmxt`?7%gSgb`++b z_w0~ed7?H>1jojf#osr_&kfhS(CuJHQhWC4d^)!eI6bcH(aXrOt(C0(EmYhU9}Vvi zS$I880M4kntoQzoJ_Z-6iQckmX8(5TWnB(;#JUF%FYI30K@I?Jfa13II;yk*gxRp} zxo!um+$vNM!XYeHY^MXD5Ye|al`a9Zx*w}0+H>BP@cpggt0Vjz)TC9Kvn=YUti21j z7}TsW#%4q;uwAW_bD#Ro$hnVOmTRI!Nmj7;rDT9L^9Gkh(V>3yg>!>R8ajnwh{XiW-ZQ6k58guMzM>TjH7mRk#WlzcL`-;TbD|12 zCw$!H@Gm^*`9ZsoQ}G~>MUOk_A*4w|cvx^^{0a@vUA4(-z?zU)Xp(ed0|i(cXxOr7 zmLUN#IPCK_V4Ob|Ze0Rpx@ER|V-K>&*8*vCF`(xgal>}?YO-aYr`*C3w*Qd>p&eH9 zt5JQV{T6S>d~6bnrwp zXS+4K_Li)YVFBOiXx`R6yLQGS4*jxs2#d&esgsliZPbaa`jlYf3jl8L@-MqFHdulB zwe!aX*P(vx)Ie}K{?{iEtUwvtdEJ6t@Negi3y$6f56ICA8#mvb)p`afv}cmCYG-7( z_E>twt{+XSU6k{CSC|G;ZL7p-L_&ox>787aQw^OuBvin$t>41lyBy z7e8;?I{zL)(v^+Pxf_?3HD%|Krt)}!T!!~uJe=`E2ESfBqVJXw#~)?)krx?8m2W#N z?JViAi(=w%X~de7%1k2>IXjiH2*0MBSd_#vks7NB>MeNLrWsHyyIc5!XS>QSowWOq zG_~AKvS;y-(jKt_s)3GtO`x`hE}2ay262^WDTZJ6-={sCxiy4w3s9tHHyz;&8C_z$ z%yv)XGb#(GE0VI8A@&-WX(Yy~JxR2EYjT5?35Q}S$QU+2)~?NK&3Pc7af$y6Ceg|~ zE&S8Q_Z4qCFMNaXITHHe(!B7k0JLaH&{2|$lp~e2@n4cJ&ac?Dtusf8AFoFmcJIFV zoW^o8AGx11Ur^!p$2q?!{EUH>H_~ek8D3&yDtu?JxaM{9!vDuNj9(wPK2jt(A#MNY zeeV8{AcJs}yGCILhL@egIcrzB$kXzWhz_*GMm-gO`THJQFWw10V#XLFIcQgehjxLZ!YtUQb877lB`s;)=$8@v%sUtRJVpkqmW0@Xyzeo z`D?6MUPNrKovBsPn7~faQk2`#N*>8`L|a{C9+!swmibr}I&g^B-uHBd(wx9^c6uXxiAuNIyCT(7D8k~M~5cK5p^pf3^Umcka0rvDnN%#G-=Ri zMoeu#uZ>pCOrr@4c!i?OlU`f`s!}x?0!&4E34rm7gwi*B(LUAwb&~XI(o+5!oyK=X z&99)nYWVYJsG)cs>L>8+cfD6O@OR&&N9+`aj$4G4XqSRSiUPix2p5D`T}LsrXuuwm$V;3z zYdiI3;tH(=pV)BtQG|67t5tJfkQCDa<_~|Zje}dZNbsGvPc&aFnDFgreym@xCnp& zu*p~1m#bRo0xu&(I`>6K7ZOrKzI;d9ZS0b(o7{vk058j?T>5W!ORe2#CONsEHqXXU z8tt&R(oNk~ls;uUkRs4A-?TQzNq(C7H{M{z(Pvt@L##FnwzDmUCY;PIQ*VXPEBX$# ze4u1I{1JmZTP}#Ke&sF(Hw7zg?v&A~+O8vQ%`KBDW_nvX>9pxCYW;1R4Ou3toTb=9 z7+n1aBltk6jGTpO9Pcx3Xbm+}K=76i`NlwR`d zfbXKhU>6j;&B@N0lU4GofUi%b@07yea`arg`)~eZ7y6q~GCg2-N=(fHq$P<2@WGatR35pXvcPf*^Vu~=!}0?{K_9! z+9|fa;xhdI@aSN*Xr`S9Pm*&u^m2_?J2s>Fu4Oy4v0a7| zOy;$%k(o@8na<@Zk^)gYWs_7!p8WEFdYKc; zw$sWIa3R;+_RcaXC<{}$+sh*v50KOUz=Xi{6o?tFk&uX(o&lP?BQ8OuB|x$YENpp4 zKv~;EqzWwZYj|5Q($N95`7q#&T~0xwL~%FL$A#5Db*(EWtAFVo*?tySx5t@vH8trH{@@7z0qM|V-tUExR|sNDYQiE39Pu}Tg@d~Qc43E3VE7kECgJ8OMo4jH zk$OlXt)rnyrNWL%cNxux@S_MaAlvPl+;Ljcn9|(sLN;=g&(MQtuz`6zB_qEZ^R;31 zXI(tc_Wf2E5(vC@uS|P6(OVenQJC20){9duicJv1SOdv+OP!InNk(qVmNpQX1ZJ-6 zT!j1*)SShqsTLJjj>!iWtMvDYq?V~~v_bN?1~yAEbZVeUlbs>% zu|8Pdscu6OBbvh$TM=&^49O(FVkH*w$H|-+CMCR8^($G;FQ7Jssm*z)KmuKgeuzZP z<~D6~aMW;8CN=DlloXH~MgxwBnc*@EK%);VF%2^)9Zn)abDq*vw7%9^om$48P5kv> zSYdwnIhw3YhE%t5Cjfz{&GeUg0$qa5zsQ9}fj$%2`c7QyV z91`nFv~GX;m2AalvSS4BUf*~1Ec0>(f|?~^PH^Ok+7G=<(VMvx)JNjaQ+sdw)_t4n zd{4)Y*yr4d?I)J6Et@@l>QC*&c6suaLCLG0%n#f@GH`#N(93ZtTL$63#CxP~S@gRdoLw_ZImx$TFwVw-|iY_FNE*e2b5!_-o0#de}>#kLu( z*ycg-m{}Cw*DJ16+L7b{xoyu6_w>c&8NQMCynM*0r`oZ&cqf~XJhC9}8R1Lv)#7k> zbYcUH8%u?2R1HfSchX-K_iV|vMm*E=4g7r7XOn1!nbn+3?hFa zydn?b#}V}7tvFX{XweedPNb>VKxP2Z#uxcWFu*brdEwZpN3Qw`~Mu`k+nZnJW^iw&wEJ*Hhe9WI+elJzivZ+3i{fv z1>1Jc^Gb{e>Pz2^S$!R_xzwbrR-1Hle67zo#}?WHU*zeNkZ zg$)FM%v`clj4^o>sIMuG6Hn}dxaBn}=0M?^z`>gNjKVc52 zGS$tYH!TB}f=33Vg8k7Zy=i#X3K2E0N&0r9SHB!5{55)89ufD7w}}?)<&$fklwpz> z5=&A}Jk5yVQb1}y9b#F;s4+}5ih5)dm97%qOxhcG+_YB}?6A^h2!p%%c|@C^Hb%Dw;Q!Yx+S7viaCX0q`O5LUW(E zL^dH_2J}KJY0X_&Yo-i$lZGeUr|c21JL(oLAlE#yZ=@Igg>R%cP^-p=jVLroGS5>k50-7uo$D;? z+!i_}v{iSaFRz%3F<~u9wMJtD2J`LyHW z-cF5Qks6nJ$+d5(C>VGX;{pn+gxi$HL7e)sD)F;#xD#;-! zTLvfRR_6yE?i1QZfb1gQDWijrk}4AaWtIE_-KZ;kr*x^``*RR!r_FEpaZK8QPYx%~ z@5TMQ^zQ!sh8*YwLopx{6s?`(B8Gf^{NAUH*4s{M$d5*UPZd{ga>y5kZLIo+^m`fI z-_GFAm3~$zlL@7p*kOb;-v^$a?cR=U_k+k`6A@%%$Hel50~z5fuxabU!tldQ63LZm zkzmezP3dbzIN@XSA&@AVw6U8eNg_c-vPGqX>`_>cuwoD6D>X9K0=Dq%d?dIfR>`!$O?lA3GfMGcMUv(*00BD{4=^>HM{=FhW8tF5yF)-d8 zDjvImlkDq=*s@ea36?hMOfeGKH+*q;DqG`NLt&XmBmKsGTRh|<^T|JKK^{v%RG7ef zswIp-{eH4-h){f}08BEypUhF&C@Ipb!$pQezFG@fCjkPTr(U7(ZUG=^x_Ohzhv09uiH zpE>c8*f;=;Rc14PEXN#?1BSle6of}7gRO5v`EPBc$N}>7m$(mII+ad&5NqaX9+7Sr zLj+=NiYu`O=nP`b6&;8AgIJsH830QVE>mL08J@1F2B-QcHL)@J%ALR#m?>Zffh484 zt*}EnC90XtQM5pFkzcUP_6jyDG83Sv2bfZPD#$8KSL(yARVxB%sfNatiYT)Y6nGzx zBCA(WI(%rv$vnd~0EfOoI-vG9tZQlAlOkp&YdRu#3lcm2eb0JFMI_q}&`rZmdJA4$ zJZ&9D_|I5RH@?CSyai} zzX`bkC~0as(wwwuV%757XpL+*BS~KksNg^F|J0UXZqp`nWl7Jd8A(6_%_YUIs&Wi+ zJk@JMUTr>Ay^-7-0IKxhSROdc!{srk(y-~i7t8pchFB-8=003v8;=NNCblGXe+`^I zS`;K#A3#dAdD9^|h{@{OCaT&Hv47mbi8B*y*@g|9D-vOcI*dclfoHzin;WKg;`~{p zuH8tf759KqcdBkX`4Fh=rL`N{5(us*^gz~q_2FjGt&lV5@y<{_A3wFwATsW z|AgBMTmQL$Od5+@&Ux~SHL`W|m)t|UXI|gQd0ek{`08LM!Vt%g{+=H23*qP=^H}k* z?C5`wcpd*(ao2O7+)3!ir9E5uNVtjw^z@k6`EGs*V29Izl6qUDTqFa77;e??ZI_jI z6m41fHDl{U<=uRur6^)BNIKyLX}*R~jNgXiIQdQ!E?!IB`P)pM)%csQR$MUP#fsg&M2n3@6 zFN0odQ5$Xn&^Ay8lXHBqS*qH^l1=~HYl?#GW*j9aN~?b|{i>o8w|U_KMj6z=K1RSb z$dr@0b=5v|)r0TR2uc3Tu{~RR_!{FqR;miOn|XH_Pc7Mw#|gczdtu4>#$VMX1%DF3 zhOriR$mQCy*#X8hN7B^kp`uyd;==flHeb9R=g}&FA5kh7R^Y zvS#3Iw1tf&Q?U>;J%-8)w`}|Ym{?(44yeR<|3L=UR$T2nlW8?GLN)^HNHon`6G%Dg zb7gSST7C-?6o^Zt#AT-nT#B5{xCMw=jJ`Qkwr@jnSJ6YSAS3YMkA=f3EQ63vn-WLt zBztH>tBQfAH?<1IR4|L+7FL$RZ^$y(B*nF)HViqoby$-u-ki*ksa0*8X6nxcaMCjk zyus3>gtb^AYAb{b$Y}HL!?6 zo9k})t9x%Y){V(uR=;UJEdBA#C)6B~woGWTt)nZ$Y>u%flF!G7ial53g?Pc{I@127 z6QmGm;6La(a<&#e8yknzhWyucgbzHifBgT?btI3iAr~AiYsl(f?rN!eY4$z->k?un zR{5(=D=OYsRQzpW_yJ$cu~q)&?4sgtii*D;8=DH!?c-&uH!W(60SVVT{;OVbOU5r< z{nnz!z0{KWR8m21kSfUJ+$V%Evh%W;XA|ibtQ+hP&528C`$tCXl8cr$QhPSpM?3Eq zS)qoLgN@7l@mEzn*tmC4>{VlyHhwuMc5C6%#->4WTkoIK7{I^X|6t?$gJRP@hFkx+ z>2al9QJ?l3KY9qnm&QDM!oeTMv~B<7aFPepw!Kq33-VC4b`QZd$Er0b1i5e8;$e)= z_Ff%4z<_N_Uf-VixP+=TqNUub*6!KIt>9eSvq3}X(Vop2r%jePtyFS?E;x-SD|ehW zFbbz7RiY_5#}!0j!7YEfry{Zp6>e@AEgeM_QoRXhNY-aQ8Ypc2Z94zhf*@9O$Q9fnp!02IN$rFs(IVNr46M^CNRU6VPp-r{!-kR*)$lJ zsI=lyGJU9FGIYAKBUvFhCODRyUVykA9$R*`T5` zS8T~h8N?eiVXz-TkL~K-oM~+jC&rb{6vfnKW(T)wWffo@Rco#ru_MnRslu~Fvo}$S z-CDHer$UD4&@YG6!v{;ki%Lby%OOP7s4?O;3F-H0V1L`u(>6*SmpF#g~OW0Om}A2#c$SVo8? zA}?vaVnQsUjn&`R{}B|G6Qu@}dM${hQ*=DtMzrhjeVQ>h*KTz*PLRDcrOL0pCQD!n^VggbE!^7ZkDug@O->g}j zOj^vA-5I8-z`(dAQpRfh7C|_p^4YQxS|KX-Lb0UJO(I)+t@KZbqs5q+Q{6U(@DDly zy`%?1wpsLRkUuWbEuy~CoW?3+BXKm@hM-*8dDlfYW6*)&dBUzFa?en&agT%9bH1 zi-#nyTACkNHZt&3pU}Rzl*y1zE&*yg`&5!j4i-|$(ZOj_J()gWdS5iOE()JlkzSG$ z2-Xeqhi(C&dm|&R{a=ax>8BQj*TM1M)@VC)dR$6w?=GQJ@V{RDLa)zA%w3otdVWT7 z?#W%6)(nB2|FK!2>79nR=G>o-I=axxV?L^Hue~olH1L?=t+n^2hgNnJq9iSLNx{;_ zo$&S_)Fs#gZ~v(a{COJ!jc5AfOCRVQx+E4t&*P`H0DHJfCI_ts99bMbyLTdWa&Eh{ zqC9O6dq(DM$!O{)+iZV$^SycAHvdFX6?bNqZQ{bX>jcpj%dekV824(?^BecIB%FQE z*xGx>=2S2QnX}%1pMWIu=ry_KY^p1J?j7%5Y$qHe#bSiV_^>gkv-HOSc|baEJ3Jea zANP>cYAcDtnpnNJ;q{B8ki<^lDd$NNlC1p4-pyko-{)`qK1Auk^Mf47-aNJA%-5Np zbm+=<08znJJGKjtEe2-1Yg!31=HEKDb|+IX+(E6`s0gprfq-l9capjkJL|_-o3>v_ zIuqz?(S0KRX1FZiG=I_>SAdt`&c9)^-sa&e`RiHKoGtJChZsL!VidBidW2ukmamEQ zQmmWKR}uvmdFL<5uirzJZlwld?zFjL>I*VSh+ORBfABEd1YT&ZjFlm!if@$x>A=eXt_O4}Bb>mcVhMClt z8GX^3t>wUIy=syUY@2Fw;C)LVCFJms_6mJzq3P~zxQ(fyFn!YHC1so72IqvnO~ob>Qk(3DP(Ma< zr7{*_@!5E7IJ{^JLj}xLQNakBgCwjGX6Pc?g1g@okAMx{enDH>LtLkjziQ`hqKK0%5w(Mg3g|5YCCl+t!_4-dO8y`tzK;q7wahnj zg(O&AO_Xb10;{{zqcn>sWLB#XOmJ}aoJ#S%BRJzP?0VdCll$OeDT>B6y2UKkH*S&5 zsMJWvkx{ZpNoPo*@ z5QQUa~{ zyA>_#xE%;Fw3V5cQl*I)plM9k_U&#-gUl16a6*|3ZO3#rg@Qx4m zgK=beR!MQdH?-1sc42UHkL;Xac5NW5$4XXA2TNWXJ5>YD*sBjMz$3dA1Vz0 z!58yLLd-o0<=-4V1Xup^2qy3Rv4Tli`Q;**B$9`$N#K&s-b5^tZg~fEjmAWP~v@D6Y0o( z32IAY9XT0CgtCsrxh4-`mvfEn8_Z!)Xst3m=? znoHf+<$5>5>O|4bRgEzs=J8HkG=G;&rGX$f|3(#hnwQWKt6JKU2=uD{WHkze;}oH! za_$_V8gm&HLV5(P-Vq=z%?Pj~13(4R@C^C|7my}$-PWO!83ftB&Jr*?OmTp;2<$7r zC^fK%{i3bFZ7tfUEJUMO29?Yb@R$-AS{bSj%NXt7^R)kN&T#J>8S`429NDGaXLJmpvI_in z{iEcyMF|CS7OQC{FWoM^wzP=@vG}LrF{AC^w~(dVmR;oQZvk!HGd2_0Vyuw1mFSI9 zTzU&goS}xFn#^e@_p`7VqhL(Cs|rk4yJ1dyA8qNJnz-;y+jb&<2O1Fze+}eJVvtVu zzug>5AUS;=3zJM*7oou#ZV`E{$i*zS4RZU4C=nC@%_KyX1B%*=@#Pk>3%~dZWPf*$ znn3_$ps~nP%-ozUrI-%7k(+v?3~-ON`?n~BP)DEM+bL;O8b2lnnn3enRa3E zw!-l5+0OqL3;Y|IS}Wd`KWOV%ca2G`)(P`XyK1n2md50(}wnKD_A~k^=s7< zrR{J*X$MYgPd+Y<3A_CnMEuN8>%Ky~Za*`03?zBw6VtX24EfJ;6HHF5-g#vU;D(E0 z@}r7_i0}3FXhwYh$!p(eK^8IrOiCe`C}}{+nio{#9&Tn`oM6_>*VMMuj^gMI)-a!^ zd`WPNmoEvuO%%Z5^9-1aeM!KCv1=yzl2EsB2VWAsA7ud*=W##*UlKyvmWnstRQM8$ zV%eH7et5yGuEut{Q&qPmiAy-K_Gfcd*+$vCZC>~+BlB8C#ci;l|C#*#(bZ?W-k0RF zQ)EFJnV7jZY9kY~Iq_Mmp5<`h38RoL_jx0MxFMb%@=DO&rfpDUo3{U`FG)>@Qx455 zpNkN#QY4P8R(oO_HzPM>pSL=_1D}>~=k&JT!!0f3exRkxsE`WN&h{96keee?>YSUf zr)}l^!inS!I^?4jk(@16ecE7F<{j*60jvqBtnG*sfvJHkBGbP>mimZUMbJ{giCD@~ zFl5Q>wmz|fS*yy$6j*G0>4ESamye#MKpAwmU^J@DDqf><-)y;u=b+R&}A zVAgM^enGvDIagp0YE_hh_klxRWOi4?e;mV*uZ})x9$>6YNfaZU?Hsn5jwxD72*bD7 zFa=kk2XyF+)vROd6!7sovk5wWM-hbTuQUrPVS+C0{T*#ctY&^TRA%f9H+IFB!&Oyg z^c5M6L3LS0)sfn$P=K-{^5Dx8eXY!(UyTNJaYn}R2sVcGka4th-*WJ_2ciQzHQJDk zbR2sBcvy?r<8CR?mIxTdmK}HO=n7hzr>ZZ(0zwZ9qjxfJ?Pe z4_QNi$z7_Sja>ioxldx1v`-kKb>LB71~q}3 z-4NiJZ9Ph5+Y8=#IJfo2g=-9dArppKPkEf`EsvZa^=0-9qB_8^O*C}9+e<2fKd#~R zKT3U>GzON@gLR`QTY1Jlr7yD|lroCw*t+jf2f6>3+!*jZSn0d3FjxUPts*<8BCF)u zfbZ%`-<8h}53dDXw$Pt2ps2XZgyLhzg#YG?xgas-7YqI4PcJI&bX(kU>2X^xTe`Y! zQR8JX@$=K-Z%T_>a_Q35-!5An4&^F8W>Dx~~L~ zV?x1zQHoR}TarVC=~^KpN3&o8n>*2}JUoWH@d&xah(>b*AvgUjpWRZL@nhHF_>pWr z4!v@k_Hp+2yg?ZE$@h?Qbw&6ITeB{YeZR;5{u^@*Dn7a!0M__xf0uHw%M~AXfx(B( zz$IkeJBIbyIAysT?Us3j4ke`?Hj{vJ;S!kpMw4z$2I;mM`Ve={4G(GZu$iQXMA}S_ zjU?PiuKAJ?=nXYoBy#`4DDi5-1eTPzi5;vPGgu>$5&tO~68t@!G&iwZB$gAoY%MV* zH({U^9O&va*F?kM+XgxX67#n3v58xz8@e=ykrjAS-%h%;~gOM~RXNI!dD$DbR z^2cjy%1Cbi78Ep2?e^~$3J%CB0#QeoDLKL=KEyo|luCHzCw z$)ZE=ZeoOpb^pAANTsTmg4}M-1JPmo`fck6NzO2YRxy5j80WKq`x#M zB4ds++EziuIeX@m-tLiM=L!=|pcN}%D&kQflWBxWqadsqtFi}%ur#m@t&tL1c?K*y z1&CJpbZUoXdnh-kGxi;O^d{5m-W}u7vPujkTMTMPhoQ7|4;OJ;Wc)XRtwDK>^LVP9 z1V>@~&>xSJRpf+?L6l1>3}?cEM1;}2xnMoI4lDRJajP0uUQQO@SwT=m2zjMkTMjEq z&AxN@QwN(SaB}3DRuBtyab_z4Sy83_ksZ6ZXCEmzqkCMa2@5Y(qPetGG%o{LPMN>L zaWw)-A(TGIf;!E#mX|2mWDJrVG3yTFubKH<)^FO;ul!}}%2>(9pE?jJaK@+Bm23mZ zGBt+bK=X=~f+cp`oo$cNYzVkC7ai#bMvR=Ls*3*wk13n$o`+y;%*#W&CE{q~%ox@w z?7bb}1?HTW%`9s?ZOh-17UNMjz_T0MhNPHRAAsfj=W6C{!5{cvK*%W>io?Ky)#IdC zrK7w-JjBSU%L#04mPbdGTFZ`sf_^EhiOEY@el-pS7lVX=(^DaE({Dw!0gXQ0~?{er)f!tBRJces9_8cNaB&?~h;6D=rl# zC2v0F(5>z9pG6Y6?8kXXPOScXU}!zFV{iBB{*~wm@WGj(O@c7%-Mth^g~mIG0L%NZ z6DtCj56NnqJemutLfzu>K1m{K?W-mCTym$hX3TMF1JIOB0-{92TIBsAOZib#m^ zTPkMylhVFRq9o#-lMB&MoA%7TQ?mjrhlpLfcif0|NM>d8e(Th$4M*>iu}1jyk%0tL z$Kg_P)0Y7CHAHUZqUU6HBafXgn1a}~-_Mk-+Ol!59-iJ={G|z}8N$g&jIrKGmqxK( z%(Yh=_esr78(n{BMrQ7xDY~ui{`>UDTY8KwDHvCN;t69*E?9sn8NA#5*V3DD2-iHo zc)v}YjjJN<0)RJD#T09r0jo9Niukr#oZ?ahrr8;fKvYJSD%vAcMSMUJc@&*S6=l&S z5iS5(=xWF1u*E8&uHkelIcWW=keE67s8Q)#{_^uqH_r3;*` z`Sv(CZn)+jlm+*6{Qs~0$Yqx%H*fr#EujtxRYTq$_qqGlw@W)>os!^UWnMe;y}GiG z83Sa(0MOV+?2O|}q9deWfD|+^u@^2a+&;V`NeiJ>1dUIVS1Q)4vx37~_-jd?75Dh9YBnFS)7yM;g5lPeO=PVeu zhKP+jDFP0I$;&$u_)BQU=`=H_ubRiNKl;uiH{#Pq2&u()Co>CRkN`>51T~3wb%_|X zD=96f+c;2Yfh~twHuEuRg{x=KW4Rd2eMcAqS;%e1S8AMd@ZT|)%#lP3jE_0w#G|z5 z{Shcfgv3*x`qlwdC!?2g1TD4S*jd#Pb3s!1moLP`kIjr9kQtYH#nRQOHLH6)(D*P2 zvW)nfmK_`F-Zf?Nh~(Vb{6O!5z)4x5J!q*mJY`FMa_$THftXFF=G>iCGAZEOH##^8 z6>X*2wRdHe{4(I%P~ltCWyGxU8Cf~AvukH%m5dGeo~ZCWh|X&%**PUyC8GnrTPl1v zPRkEh`eI(~7W2#k|3KJE?kXzYSs1?47c+5zzp5X+MaII-G485yB!FD~=Ay<=U@O@R z|HW>2f^8++-pPnf9lNw~AhePH3vJ}|!eHKc{?L^$A$budBp)aG6JSCz>q&pry5Zpq zeKF(lzYBaZ3G2p$f7i>OkX01E0srmsAi4%ej3l8gq9GQ*l+>*e+|SG|JiNA7xSe>mQ2WEgf51%C8Tt z^LHlmkdqgDOV%Y@$W5|1*yEbuNeo+hj^Nj)T@yTG)9!tDc90MKjqCllu-`~d<=EOv ze##U48Y!=lhdG}Wh0T~;qAHwAtG_yA8tzM?0B_8o&U777Phq$djrd*|wrxtpFN?P@ z??8X|1kX`(@C~+lxv$_8+ygk1QyK<^fB;+ERsKo z6RBq8%I&pqR%A#h_P($S%2dIQCIG@EP;HKNRf&7~w0qxh z;B+|ph+IQoxiaTQ&X}nvU4t`T8>L~V^v-PB*o_05vXaX3x?DhpUI=A!--Vhl5ook!c{OIFZA_tAb{&)5=aqMBF%vPeYUBGN6;i}mnXsTwjWurqmMJcM%5?!zjnDQ&2Qk!IdW zGu?3H`M=;eSwzG%LmY>7Ela(k8BP<<8TD7zwNvx>|CMb~6|SQDdRg5HcLPq+Ot$|f zSE~lYNQe<7f05aHg5>{2sU9f;i~O=`>8@oJ9%@Le{5MX26CcX*?J1obe-|9~ww#<< z_Eb8V_i5K9@;LNvYaYhCIf-^LzT`o){WnGnNOSnqoF`hDn6;LxhG`{g7T_itFKBXh z{8;WqfyJ*``DP1v(ehmjmL(SX)MCYHPE3Uk=+aF_S3AqD5h2C~KokomlJb+Vk ziJGmpd6`L7&3p@%w(I@?5aiJmbBJL_qUI~5dRZhQ#1SMmhe(q@tkbkLR<2w6Q8t={ z)}ikXvi^zI`Y%Us)9D|xkDfV}VzQdAvDEJBwuo1VW|4@sj9Rw;kG*#RucFG*hBcf> z5|La8N^8uGD0rcv+Y2ax1PBro4g4sgjwZAh?AD%$7Yw4ra|k4$5(2HDA|UpNEjUEc zcBaJ+35>L&B7+yykt8DGJqJM(Ate91*1lBLIlHQ&E&cuT{rx;V4>^@eopW~W+Izk0 zeb>A4I>|8dkt0{j_R-9QBUQ^5_m44*Z2L*2Y%q)jHO2&tTaWD_8!5^s!$@kL=583d z!rw5Gi6jrh$W1dycNse8y3lz7(iNOKxq$J;#rzta@Y}E~Asi>8On^%{i{+uXeupf6 zOFpy2chNpyB|JpZT9&A5SmH(p$9CeIZNy@UPHT4Trm-O<$>e3N!Zt9>MBT9rb1ikw zR;}+$W{h}1LB73+FChFrWGanc%R?27-LS+-SWVO#J9fHIYh*=2kGLJ~*_FauHZUMa6}I1=8DX>A$qfxbQN107$~|Xqc)XIHinzn%z)Y`Bduh2PO%RE zW)<}&S6icw2CD2^r6j7Nlb$MD+3{(T5PlhmHOgh%i8Z+myN_+LaagSQLZ`M7JZ$D> zSdq#22Iye9jLI?MSdgsO%AV7@tlr%C26UWy2FLNN`f6LhHmCjSNsKB7u3r>h&Gl@; zTHs@s{!~`{APh-FNT&wOc0XdCMg7xKg0kM3-bWyFI?P%Y>(V+}VYK4332}V0q&HLm zfUBItlUMKskwb@dSgsE=0Tcri8Lv{0b>&4Qv-b8PDqhqP55L2h*+FpFqKnC+$+9GR zQrYJyb+B54TNzkWD5ktr-C_CV6_|blpAbBbtSF3vWfI##QD}zLI?x7w1czk5to2SZ zH^9`fKwOTo)yW2x-nM3$e)L6m3&ODyrE2R2ol20y6b**Ojk2?9c<2!hJI2-4&m6^5 zRz0H$A7N009SIgco;b$EgY8J95N#gSK}N{%FcOp9wAC7F;C;vjFoJ~Y7&Lw0f8P4# zg{iTtFTL*D#mhQ3k2@!^$91*)?^-bF>fCjIcww}~lX4aUPx+Ch-Qbpw0xV*#qG>rB zdgGEvlf4w*os!=6x=P1Wn)$C!K1VYa|6rffShx0BYyMgj&6?6e2JMM7#KU8MShADp zDOYeC@BasEc8_|`>aJC2(nZi^`*t^W$gW2*%ziqAudQ?Twqmx6 z{P4)xMIKl_@=Fy;uG}L+VuPwoRv_AwWmp`l8!2%TmCw zfgIeO7Pku(Y?bi-KeKyi3%viw_6!tdfh&Ce;8(NaFAz0sE)BLHJ9t$2%vY;#FC@pv ze~1@J(a+&Uvf>-2!L-4nRote%NMQ3T!Iwcti9;hd{Q^Qv&h04LE|Pget~(k+Ze~@c zrI5raOdY!P(MC@-dK=pA@HTS0vvX$c=?GvqN#B%pF+$<#(!Y%%a+KCmYcI*f5>6%t zuo!?=kG~4UW6MXPW0ZRZ8-Z6pFOs_SPm>FBzv{aXiL)|GAvtJKKF0=u-^?U?|2kiL z|59Ij|GNKE_Wsih_Wt|)?EMvy6`k)ADKJ%E_2A|W|Apu;9I$O z0K#L@{>I}{8)op0Bw?6k@Q+Vz$5VQ4RM!RuzV*q{U&={c32gbRTRao^AhzpDW z*NQVNUu-vK19l{(98u^SWbg3Ifd&kZ7>pb64x{9?Q0H061H-q*A;{T820{wM>;bAo zAAlO5mOZB~RDeEqB;cSL$w?#{rl>fMoi^>R0{V+v;0lrK_gqkE9nxnd3(HQ2rKkR- zm%pE1` ziZBnp*ne2zn`?j|dRT?Mh#$=sArQ4v5Sa%>iwf{k#T2VK(X4eIAv6%TO8jJW)&f|B zjHEL<`A|TOYNW4>GuLIvvob?GtCSZ(D>17xKH`vLK?tIq9J0ERD`wYx{n-f2RM8<9 zgpOVNTmR`f(!cDSOG0CIXXdW$TNu76Kc)TnqzYSy?RPi*67cM(K=YK|BSOWUHBJ4_ zPs*K_U%&0PoVIB_11(wO!<*9LXFb%d{@S?I_MGma)k*zpr=9rbH`3xh^zgNz$Ji%w z^7*GOJ)g;~XRmE2y8V`Wi-|o#f9*dbjCb#^t?cJL`fE2533QCm0A}`jJ(EeHBz`?E ztJlqs&)t7FVF(X#lW`%@DWa_CO1wmrj0h$ z@O{Es1Q_i``d&tiW=p2Kac#&oQBykO#}VWuGGk-*r-rJ&@$~l3_~;2{A#?9j8QHB; zewv-T0EfVlJ(2f%->R~mV^G4DvafcbKxa80>q@qQGWPb5OFV_w(Mtstig-f2zD81` z*j&A>{S)9uusc3y!*cBTyuWvoj*r=M2d9c}N){r&U^XaSx6U3wuZbX9J`tN8E>hXO zcVXEC0NARE!M)W8P11n2H!gDMgyCocR+e{3#TGuJY-G>P2mTt0um0P-a6>45+uv$R zddDKy?MJk!QfK2%%{nNtB?V|#t}rc(n10d##C2Z-8N`2WArk>stk-^I&{}Y0er^KC zd6ZMgs%)6xOSz0&P%($OI39W@98^I?+oPj|jIC`A%GTZ&X~Se9&hSp2WYd#>vZ-HHwp44{HeDqXwpJ!!tD<1H(U$c8Ws z9{!!z%>6|4e>9JI?R+|E*LrReX-L-dy8L-7j-!kPLfe8sD}H_&V*jTI7T~rP6__eNyS<>}SNfNfdFDSpni~&v89dJj2nfr@lpatb}8y&d?*!i>a~V z!?mj(@I}O6np`<~0fHgm)5BW0nSrl^X8_)UD$eZWr{t9y-N`w0BUY|;(lelXE=|bGkJN`wcPV9+3H!(hAkH+NubU{ ze>l^))JSfQd9BVv7d|`dF+GKx54G5|z&Mp>Z2Z*pXj@M)ozysqTDuyvHnlVTki`xjtqq&m`;HTT3PauDCONzRpxHTP zR|jMIR>hoD7+T%Cf9<`02^_O9(1Lz!9fjeiV`5)PjQw+Upa`#j7+=!%cEiCLN3Oo@ zji0x^t!(m`g-!3>o6|NRGcIvQ9Gp*5+Y`nQIB*1cwmNDNO*9)3NbUu0k_rlaF8T%J zojN&>ap2)=Lq`*bbljL9=mO`8hT5?YmmI_V&s_03;o*{F&snyYcq@;_+HsW!rLa?m zXi+e#;ni72!$)L!At+&SeO~K4%BFkNM6h?uZ4pfMW>GJP7eZLk`%k*B6xW47<!82vJH*!1j9wZ$O50Zfu0@#gGR{k6QO<}#;=n&cK}ZK(;*S{7G*24x+?6=_wU-V@s^y{1?5oY-E=Yx;7RxVlHjXgSnWwXLld z%*M@t4P6qBln>^7p-WB-I7H;hMK;(FHqfC0maAIChR)Ji5j1jvJ^797MN0B=eCP2dns#PmYuHtrQJ}1ca`UAo|iMO~W; z(M?Le8f3Ro*Jh48tVj&+!<8(Kh(VBgvNDKiEdgvDCU?}7O6=bVktphRi?OgU+I>8A zWX0RO;ywlm;{#h>V$~ym6Bd6OPv9xyw#@2p9MJ<^7W9qCbQ?lleTYtZt6_@D zBqUu$t&nxzLBNbiyfxXDICBzd`ur=AID5&7ldS+z#9&=o8Tca6K#MjWio_XCdzl&1 zNP{-~`wd9A$(*MF$pgLyB&d`E2Q40SHv7Mx(Jy3V~(jXhq4Uc`+7KTa;`% z7tjc19VRsVuv=XHydnGs7J?9Uklw&7PkvVSVMR3+X65fb=E7v7WCYKb9x{yytRcf8 zbg7`e=h4fGweUv(#c3VVh@UWKj-mw)k;>PJ$4Qz`zyQ%$GXXfCG+cpfoS5tlJs z1Bd1atCPrO0)MQ}C#w_gDH-t|Bo_G*B-_uS6_KJ%eGl{iTK7OCw4$*c?yQ)$pS29@ zY#%sie&)E+0pm+jCX^(P4Zjr=`+ZXErwaq?VB+6n0z&(^<&#%#Oj*@*1r5O?DI#}H>QP}>H`TKA14L2!D*y* zZcf{6qmni}m|vf9(jxlP#{&u1R4;DYHze+wu`8QC9}>5#aAi~Tkoa8>%x((ecMmLX z+CC&M@k^NYpSUnk7Yi~?^<;Z`j%u5o`y=f~hRL?fZXMaWcW)z51BXbq#4oy|u$$$9-P<|rxYcbSoR~0k>&G_tiFWcY@jz!Ue4AVM&5G-LJX~_z!zDc# zUmQ~tf4qPgz4-Oz6J|Ws8K>ln99iO;PO|O6eupk{MQ8>4>HG;ZHdt42aBB!6#8pS0 zv8@f3T}&A}F`!md!m_J#Sa3T1f&G&tS6LFw`qRH4MVc!BV^Zktt;l5gC?Utsc&jt6 zE-r_Ln>uI{0<7y%&`{cqKYm?eLBqfRejGV4@nD8!Tw?Pw(cUJgup$X3%vA!k>V*PYUlxi}PJ2yhLb`s|Tiu`v|er)#(X14t#? zG{!#ypSk;we7%=FwG;q$t9Zju^zn;vn@f8t5tzZtpFB0Smp{()x8ZzI*l?O70^Aro z1jh3mb&Ifj&7tpQyoGveg9fG1)#rfWhGpi@Le8mIjBR^qEhCrW8%}91T0fuBFE9jT zj(LncEJ&r1hCBp_-l3H!YPgA9MtN#)QW}0p;f|2UjAMdN=0H1c-9wt z>Z94#UN(l*Sqm#X7AviIpxtv`*ynMIka}4;EIlUj5TSlmi{?lxh!g>A!T!vmP1no; zQ$NUI;t_La-MOEjew6KnR5Is9{lL_7%9BUtf1OO1=ujaLiTM`HiN1(rmG%7;noPAD zI*~B1J4E67&s)OKuIVlg7K-(nVqTOB-9WUyM`B}*n74BEkRZ*L$}%LPXqkm239kc& zVWq~Fc!0s6VPTTV2{F3XWE1lb-uPhZ^a5h7DFQX)wYZ1~Q>=SqbLmtxFtsgr7vf-h z#>_L!M0v=z989=)v#Z}o7is`SJ4lX(oofRR@Y+>*t?fS?DlSPKapNvnL*}g7n7yd! zaUij*gx9m;ueoC7#y+by9{;ms7t3NG$*A zm~MrkJA3yZRGw3ISukdIWz2_TLjAz6-Q9oC-I=$yoK_LIAu)DRb)fpR@g@7mmwa0o zUKA61Om(0I$n2Z(C13Z5OP{cE4bYBlKhb#k;;DG-6Im@7<)uASbHa{ zdCU1+vz0i#ux(7ID1{epWa`eoWBQrD1lt5#(9rZ*ck- zb|k!vM5`o)2~on)n_DZnj~YLYQP2$gSYzETNGwRt=X)Nu*Ds8b~6|dlR-RX~Kt7bkuCv2kDSxsBC5J91Q5!XrT=h zOA62E4%%!FQa)3DEqh6nP7XT0ph|1&joum%aY$_ZBF>YtFUJKi%IHEs-#VJx&=`yerD6@TH(id{4YEXA^+7U=rflmJyg?4j%p`k&*~X zYpHr;ppw@PuB?NPE|zu|Fw#}Yh;S!zhtC3D+j1|_nB}@INk$Uf@Qz#)*hxgn7Y*|T zhhX;TCmCUz5bxyxIM7ov!oHKyPQ5uOg#T*qf*-cGb?V_g)5t?j~g3)ac*W=|6u67-8p5sJwunL^e;QVXJ|0~8*)kL z&fS@1nZeNI_}$q(8#bMfVzpy(>PwFvQMUAgg>>o5@4awJXZrNyoVHl#B7aol>@U_I zI;GEfELC)?V*8?7`{AwkXMDbp{372VD0gXPMaGvx5V@7h@|pppZ#>=cPN%Nb7O*Wn{dx_jh55C6g4w=L&T6t zL>0Wx_H}Sjry-ZjDr#csl#01$ne~*G$0T@YmOVO2Hfl{L>gYhSL4hqHnAm}EzFZo? zlhGkE0AbRL-98LpuWeW+h-ish(Kh>6)FaO5@DIw0&@QM!8bMJN;do6JmIT91ABD+G z5-Q~ln}Y^X$Eq~oUX78H#UaF_dnWKVTs~ScMb|5eEPV_PA=((`BZd5)+Od_-^lQh~ z=-ZC1e5R=#8!}?bFA^Bomtb3weraa2hYT_f>dzesr*6pT*RY#ux7@7mA4dW<6uRxM zQQ_dOD^v>YPrUD?6RFm9qu!_IA<|ZNz_~y=^iqa+0f&-kk9KCs<-)FGI&6#Y8`o@= zwS&`$y{y{2qv4dPevNCoNV0Vd6lOfmm_Ws1&_PNinW2)66ku&1juK-5C}8Po++drz z;z7d+PJ=6Xe-Gon-$Q9bG)$BGqkb)LwR_UHx%4@>3T!|#Hf+ol-Uk0d=;a!dolOtE zN8&dbnoV$URPkgB!DMz-8Eq_;p@%J*9u_jEQ4p9VVKFFNH(Miye2gNexE) z+s|w@lK1mijpY5)YD7$73(fu0Y9x{n7u>N(QEtX?gP%kFYBWZBZFYz5sc0>4_Qe4mDfudH887VR4rel#))tqLg3GkB zShwmyNLP5tHhd7?_|S`O*ZxEQG#mM+*$Co*|BsuEe9uZt=?3|o)Qd#nGOK7%Q(dq|~X z!0wZWsMBNDoc%WqkAcR#s&Y<1f`-Jt@d)Y|D-Vr-*>d8YNaPwv$qzA1m?>YwbJJG{ znONO#VfZcuJrhath-heDpIuTCNlqeD$#NLGNI{pD=7{fgGvX$8(I;lLeuRJ>{Y*1~#3WC&8I+A(@=qWUWj z6DfM)>xm7E6Jf?_(Q_IH^We+s$fL4drCcD*k+9;fk#6cyXwmHYYB|U3(=M)H{|62K z-W8|KYN5e4WRK_uFnN^$glx#ikVlZn4Zta6Oe@G^Q)Q~0a@WJe3{SEcUOik6DJoHE z)JoExUyx+7es{9%kH!)*{~+d>J)vbE5^I!HMb+QcqlSwkS;Ayc;&DDwu}X#yA*LOKS3exebP1+SKBv+1QY@Bpz_b}TWNSHqs|j&a&Id!1$$H+_$q#GP)gn|9 zZk^r^rrNDjrnw9>8sr_A;E}Amey#P_u2UA4{i0L$?jzMHd-d_CkZL_rGRVhgoE3~Y zsWMb|UZ8$98e{)gR^0N7RyMVOJlhNM?2KS2@7$dF#uGz-U#(h)?73Dp8FI+?%5%`< zI#857KD-|ekjuK&&x8ZypfRBpd-B75Vq)`N4K|%d(#=ULKB{XC&N%x7;$2=(>-AL{ zOY;8vSM_ga(V_cK?llXQk4<%#zd`CaI6`ue6;uZdt66-6IM|Zek^6?qk?{ zP^6Y&gm1y6d8HD`jlSot@@=*Ignd?}25uKAEVq{qLDN`wLPr~PT3*#`V6TRyzY3}4 zuIgUR)0k0PzOdW&NOOM>ON}r6v0Y2d9#h=g@+&zUc@~u-sB`LEzF`j(nMM{zj5*?l z)<=HbNxCr4o;9uG2X~%7+KaiM?>D&V2S3ZQ^10YoJj#ku-j<>H95Z$aXwu~R#mYUjPdKXS6gl`YeEL$GD8ZNzG-zJ7r^Rg=$lR0wFr7!X_vaE^x~5}%9V z1mpk~5b1G;c2w3?af~5+wskmQ>3@y7Z#;edMFhIZN-Q%$nNmr zj2#$PkClZ_LAxQMZB4w7oQY(*+bTlQgV0|%=s72&mhg&TSiF_}(lT<^ku<&4`y4!T zBF&j@=P9ldY5GfBz*Qe2raXdu)FVe_RhU{1&RPGV<60;ZGLz2wTE^O|e8E2RZ6rEU zsW_cMX%#WxeCT-bxJC?>N+7a747bRTMM5rsTA*pe?GsQ1V}7OJu{;<}>|a3Z+5e4E zcaM$D%4ZD@F!#NRiG!1MPdV?{SXdTu;*xHcwK&k=dnis*$fJlMFS;2t8d0vEe}r;n zoA&4;R)Kd7iqise;kBK=UDbCK)*2OCreGw1wpg{aC_8wS;r~sqH|{{ z^vFt}*p;Rq&9plkV6L@1^TRIITbII%X0i1~r`D|XnY}wzRETZFmdupvL0xP6zROYB zw)DtVw*B!4x{t`pHYts)8xp9$4ajcS!a&~OK>bztaVNY)&I#0yicj5;-miZ5w4@D- z^6Pitznc8|P18~~oYcL(8-90UzxpSpB@HKg|EaL|@10bk+#%z~5BMtaCAg9tjbo&Y z{GmF~xvE?JV&J%$3j&=I(p`%mrN940vJKnW%RWL47dHtzYfE5gC3WJey?xxEJNVGG zTPn)8*B?D=CNNF8^dG7FVqtijC~eF8YGHVw(na1fKhM99Hg|IN=%c-6+x|m)2|r}@ zdOeGQQ0A}evsveNGjmatAnS@OG7tF2D4rDl_@_Tz-KxcvE++raAG}}L67j{f_|ZBa z5%r<+B$}|CT34q*rLE+U-`;iQ^V!K;f{7=aW zih}FSl{J!pOWH9s9vJfT$A!w0Tu8*)!moFc3khCR!0g>l8;Zow_W?H-sc}78LZsx+ zj}t$wg3;lyCl~>^xkQr3{PgST<4NxvARg{|fY`E{d`r*%NI)vc%VZq>IUQ zl}Wc#8#>a}*)(e<)iD+wVsCy2X(9g93Yvwa%eZgyHQD@8+T|O|n=r{vioL%u$PxEXxpBZGV6}2iWMhw*g31Ls-ieOW;^Zw zrHSGTc1>gUTkox{E6r=zGsRbax|%zZ8PFm}4s92|OQVzbS_0p^ZO<(VOZyS(sJ+9+ zym32c^FWI+5(XtQr7R){N05*WG!Drjh36{5uzs)4hsKh2edGOR#6f0yl|~SpPk#8E z-C>7Kg++~FvCXo;nvimCi|GzG|A??zy7BP1NfD;V7xT(&iX7mbu)-(7po}x6JGUVo z6&Jvq2U|=4U+79n)1zd?KFL@;Po?yL>)GWNk&#n$Y04=5`5Mk*m`M zn4B`{g~Tr)l(|64n!=zpCABS533hxeAN`XQ=d2-b3YiGm&&Mf(hjIbRVu${{)RAjY z>sawr-Ly+_FoxKtIuNTP7+8OL$Oi)7&d&PsSvGthD2;qCfYP{b${nSlbSjgOz_<*?S8czlKgi_WR~fx} zP{Xwwhr9-|YiWrGO4~GJIBK&(=Uf*$kIA4u1*c9fU~=eUehtFxZLnd&bDq}$6Y0iI z{T7zeg>K1J!kQ$lWr@0mO7>Aya-kQkH-1!`Utyjv)4fm@d+D(T<)+;$fm_y!e zHOjUcu6+6ykEyIk502%E=xWIJ1<@p;4d^4bPsn!!yiZb8(eMuk_H;c)I`t>^N*0 z$VkUtljYb1@uee*tK^{C-$rzYjeaM7=kTLzS=A!^M#vcQ>Pbv`kS6lC{3D|zFl59_ z*|OUf1lGvRv%V^yjKR7liQJ++3^ZaZ5TYo09YN9Ppb~MWz{s>l0!Kf=+3d(z!HU?x zfWar*vnkl?2Un6dj2B;-e7ARLW_*UrP#1RNBg9X{qzN!tf$z3wK&GW(E=`&-C*g=L zA*qomrbhu@d`43O>t3Cru(1o$U>6MiL>n+CuL|Y|8pAG_WHOG$L*fN9$PQQL=T^^rexv@=-I$R=tH7%Q1JmKnC9qWE}elJY+>C>q{1+_9ibYdmEgIr-UKlY84*R+3`N*ZHh)+&W%B!A8Ngu2?f zcVq{aXq4XH87EsNzBTbE{XT~`PR@<9xTR-$Sbf`LL|@h z{I#YuZIkdDX`DH$qj#UtNe{hveE&i7`qx(Fl-(GNnOPNcTVd#O9Gcs|cFu``BAENf zj}32(iJhJldjrh=&lq14m{8JfZ20+@*yG19*?82VrmJHUUeAcX=CYL=+aBNeeNEF? znEhuY^vH<&aArbQ@cmK2cly>}n~=IQH+5-l%7#DY2kZOR4@Db*p{bLHb_fsyf`khH4s=Ckb<*v=eu=byNGFvicz%_CS|5*RcZ@)XOQnHH3iz1S&8)DNFrURze;Q$ZS3k;1Ri z!1WQ%6Zx`)5e0X7dXF6F&mr)(Y!yHKTH)C0z8nIt@MG7uGP}0NmtC9T%dYJ)vuhtJ z^hR8hH+Vza>knrq7Tm>uQ@Q-nm`wMJ`~@wWnTC{82AO!p&nBiZF{Ax9V7w7S%QZPvNo@$YuQ zdzm4ZK)-ZKYw>Z$ogHz9ti-s$u0PSSmyE^4);I-*oT?;~b^t3+EYGiKOj9DTb!h&v zUO0_V8||&5GFd(N{X&uXwpXB78;gCfL6}S7wkL+eW-9ePE{$E$drkx^IzWpK*eC}s zcx4iY*|d}#u(e%0iP)y_i?T5QW3-~Ee~}J=w{`#F95V1cv;)N0JjXFy&VjJF(NQv{>?m9 zbvTE&5dd0?sUr1cKwYI0C5ki+hgx=o0RF9xlLj)cZB02nt{}$pGypV5d0^K_=oNi) z^gJgWoiw+PsRdYZN}Z^~(gWdm;PH&#NKdL7xIz*6MGnyvagNvc0>c?1 zUvalIDU9fuC}#UJ=vW&>^c-!HXCXctvCL0z#Q1DRpFkob&ibs08 z8+A_eYsff6B$N&Bp%)OedModY9@=c-#YxXgSB*DCuU0({kx5C1+Qg--Vi`TlHd&)a zhStUK41UCr&agD`C{G^UxoiJ>#lZ=ItRvPXJHOAaiE;6-rzfmUk58Yta^t6uZ~UaD z=@hv5r^oMttN;HzzVU-!B;~Hk4?Z(0_@}=0$0nrqIX`vhc`3Oo@`Dfdt=~~x7a<-V0x;GrSfkB)qY?D3M$RP56k^-e@$964#d^ahOh#&7-7O37b zB79*?Y!UvuASN~uhX4PX7D$BC|L^d-S4OneF37B3arD^m<@rg&$Mr1)Dh&6_e_>$; z`9pTgjLVpW|MzFacZ}{H>X+QVtu-yawC0%uhbS{hWruP6R3aMe@I zxplt$TB$PUcPI}n677=jg^hnCHHODGa{;5XTQ(@crZ91odln#*%npGH+)ZRuw2snx z8<0_q#CTPZxbD(kdr8Botsxct6DfO?xrVmRIaBJA4w?PCHKGJ?LNs_RN)+Pz`dN!*{ z%&Bx)TY*#59vp2=C^j-Ui+gvL*hDm15(! zYd);t%09>{&uv9b(IJD{%m01Vq9Ls-?$A9J3DwC+P2^G}eFOvB*PA<%3>ET$td znXuQdb$??$p4Je1LttyR@`KO&l5IAMHIa&?M2c;OlBAi3HH+J`<5cU$*-(OXaUgCh z5`{2nEQb|VDpIs*&!_0=2&hH$7en9zLDL$(wXi`E1LWpD!m+hsh0$~f$hxk%kF0Vn zmO|(W2>c;2%ti<-rW;PNjAu==AFME}K>a&f)d@>`9SmEOmshq)+sVVk$g0a$&4OyH zyTiC>YRK%kj0K+6qRJ|VuNZ7gN)szE{&*f`x@_-Gml+TZD8%E#_m+6fN?!%9cuuR- z5+PfNTZ^MN2Bt?qs5%X|$i5tSc2Z$JOh3ty`2dj`6b2;7IfKTnePOXTF ztJV!jQ~55v%*e+;zXoc=MZ}ovds4&A+{H%8>2m!n8=?Rl68mrTqii2mJjw$0xl~nv zR5NqT_GJfqI93envU*eHMP%)&@|GVCt=rf?bjuOLXm5|~oKcjOGiYZ2+PiYf#sy;@ zt&CX=z=amsWjSSIf-$#M#@sYDKU@_P`{$mqPc8@yL}Rwk#+U3Z4Bs6SJ840n8vU|& zkBh%KE&iHIM1Sm>ruSvb?6@}`<;HB#@&po39Us1LR8sDe{CaX9k&Uv+#s4ZC{X>5J zm(x;m)4SK75uciSa`*a^@LyWL`j@69?HS*jH3YJ<^#la7KR6&pOiZ)oO+j`R1(B*`~nUYG+av&0PMDq zEfZ^|Bay4)DOmK6;hwafVv!7#T!_Gq6Is#47tnq_VN~~6D=(B}5%iAriRj}X&ienU zm&m^=iY-`(Z8TbH2Lpd%oT39dTeJYiNoO4#@h;y0=FrIFfRfj3@P?eVw|A4X&)oCk_$V((R#L@bYA39ipS! z$Pp%=6FhlFG?jhkik>kKC6RQ@%28)SqO?(w$ zyNABqtWLJ{vOk%!Xo{#EwhjR}a6w@J0l&B5X&!>;xd-?|ytHkZu@_D8`8b6hOebiu z0goTk+WFF6`}evCCfRiB<=x(MmW7a`NBavQtMB;MTL=mNkn)y|U*IGxV$tIw4$(