From 27e1b0eb859540cdc51483d0622509bf36a3d19c Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Thu, 18 Apr 2024 22:45:44 +0800 Subject: [PATCH 01/22] feat(storage): implement iter_log for hummock (#15666) --- proto/hummock.proto | 13 + src/meta/src/hummock/manager/versioning.rs | 2 + src/storage/hummock_sdk/src/change_log.rs | 113 +++ .../compaction_group/hummock_version_ext.rs | 3 + src/storage/hummock_sdk/src/lib.rs | 1 + src/storage/hummock_sdk/src/version.rs | 17 + .../src/hummock/event_handler/uploader.rs | 1 + .../src/hummock/iterator/change_log.rs | 695 ++++++++++++++++++ src/storage/src/hummock/iterator/mod.rs | 2 + .../src/hummock/store/hummock_storage.rs | 18 +- .../hummock/store/local_hummock_storage.rs | 19 +- src/storage/src/hummock/store/version.rs | 85 ++- 12 files changed, 950 insertions(+), 19 deletions(-) create mode 100644 src/storage/hummock_sdk/src/change_log.rs create mode 100644 src/storage/src/hummock/iterator/change_log.rs diff --git a/proto/hummock.proto b/proto/hummock.proto index 4a3dac7ff6f45..dc376cd694b8b 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -137,6 +137,18 @@ message TableWatermarks { bool is_ascending = 2; } +message EpochNewChangeLog { + repeated SstableInfo old_value = 1; + repeated SstableInfo new_value = 2; + // Epochs should be sorted in ascending order, which means earlier epoch at the front + repeated uint64 epochs = 3; +} + +message TableChangeLog { + // Epochs should be sorted in ascending order, which means earlier epoch at the front. + repeated EpochNewChangeLog change_logs = 1; +} + message HummockVersion { message Levels { repeated Level levels = 1; @@ -153,6 +165,7 @@ message HummockVersion { // Reads against such an epoch will fail. uint64 safe_epoch = 4; map table_watermarks = 5; + map table_change_logs = 6; } message HummockVersionDelta { diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index be510d2aa7557..feeacc98b558a 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -347,6 +347,7 @@ pub(super) fn create_init_version(default_compaction_config: CompactionConfig) - max_committed_epoch: INVALID_EPOCH, safe_epoch: INVALID_EPOCH, table_watermarks: HashMap::new(), + table_change_log: HashMap::new(), }; for group_id in [ StaticCompactionGroupId::StateDefault as CompactionGroupId, @@ -569,6 +570,7 @@ mod tests { max_committed_epoch: 0, safe_epoch: 0, table_watermarks: HashMap::new(), + table_change_log: HashMap::new(), }; for cg in 1..3 { version.levels.insert( diff --git a/src/storage/hummock_sdk/src/change_log.rs b/src/storage/hummock_sdk/src/change_log.rs new file mode 100644 index 0000000000000..cec5ef58034c6 --- /dev/null +++ b/src/storage/hummock_sdk/src/change_log.rs @@ -0,0 +1,113 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_pb::hummock::{PbEpochNewChangeLog, PbTableChangeLog, SstableInfo}; + +#[derive(Debug, Clone, PartialEq)] +pub struct EpochNewChangeLog { + pub new_value: Vec, + pub old_value: Vec, + pub epochs: Vec, +} + +#[derive(Debug, Clone, PartialEq)] +pub struct TableChangeLog(pub Vec); + +impl TableChangeLog { + pub fn filter_epoch(&self, (min_epoch, max_epoch): (u64, u64)) -> &[EpochNewChangeLog] { + let start = self.0.partition_point(|epoch_change_log| { + epoch_change_log.epochs.last().expect("non-empty") < &min_epoch + }); + let end = self.0.partition_point(|epoch_change_log| { + epoch_change_log.epochs.first().expect("non-empty") <= &max_epoch + }); + &self.0[start..end] + } +} + +impl TableChangeLog { + pub fn to_protobuf(&self) -> PbTableChangeLog { + PbTableChangeLog { + change_logs: self + .0 + .iter() + .map(|epoch_new_log| PbEpochNewChangeLog { + epochs: epoch_new_log.epochs.clone(), + new_value: epoch_new_log.new_value.clone(), + old_value: epoch_new_log.old_value.clone(), + }) + .collect(), + } + } + + pub fn from_protobuf(val: &PbTableChangeLog) -> Self { + Self( + val.change_logs + .iter() + .map(|epoch_new_log| EpochNewChangeLog { + epochs: epoch_new_log.epochs.clone(), + new_value: epoch_new_log.new_value.clone(), + old_value: epoch_new_log.old_value.clone(), + }) + .collect(), + ) + } +} + +#[cfg(test)] +mod tests { + use itertools::Itertools; + + use crate::change_log::{EpochNewChangeLog, TableChangeLog}; + + #[test] + fn test_filter_epoch() { + let table_change_log = TableChangeLog(vec![ + EpochNewChangeLog { + new_value: vec![], + old_value: vec![], + epochs: vec![2], + }, + EpochNewChangeLog { + new_value: vec![], + old_value: vec![], + epochs: vec![3, 4], + }, + EpochNewChangeLog { + new_value: vec![], + old_value: vec![], + epochs: vec![5], + }, + ]); + + let epochs = [1, 2, 3, 4, 5, 6]; + for i in 0..epochs.len() { + for j in i..epochs.len() { + let min_epoch = epochs[i]; + let max_epoch = epochs[j]; + let expected = table_change_log + .0 + .iter() + .filter(|log| { + &min_epoch <= log.epochs.last().unwrap() + && log.epochs.first().unwrap() <= &max_epoch + }) + .cloned() + .collect_vec(); + let actual = table_change_log.filter_epoch((min_epoch, max_epoch)); + assert_eq!(&expected, actual, "{:?}", (min_epoch, max_epoch)); + } + } + } +} diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 19295a5cf1246..ded56ede6fe65 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -1315,6 +1315,7 @@ mod tests { max_committed_epoch: 0, safe_epoch: 0, table_watermarks: HashMap::new(), + table_change_log: HashMap::new(), }; assert_eq!(version.get_object_ids().len(), 0); @@ -1378,6 +1379,7 @@ mod tests { max_committed_epoch: 0, safe_epoch: 0, table_watermarks: HashMap::new(), + table_change_log: HashMap::new(), }; let version_delta = HummockVersionDelta { id: 1, @@ -1461,6 +1463,7 @@ mod tests { max_committed_epoch: 0, safe_epoch: 0, table_watermarks: HashMap::new(), + table_change_log: HashMap::new(), } ); } diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 1090a132855e0..aa095b6c66322 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -37,6 +37,7 @@ use crate::compaction_group::StaticCompactionGroupId; use crate::key_range::KeyRangeCommon; use crate::table_stats::{to_prost_table_stats_map, PbTableStatsMap, TableStatsMap}; +pub mod change_log; pub mod compact; pub mod compaction_group; pub mod key; diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 4fa87296667ae..3bf6fa02df463 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -23,6 +23,7 @@ use risingwave_pb::hummock::hummock_version::PbLevels; use risingwave_pb::hummock::hummock_version_delta::GroupDeltas as PbGroupDeltas; use risingwave_pb::hummock::{PbHummockVersion, PbHummockVersionDelta}; +use crate::change_log::TableChangeLog; use crate::table_watermark::TableWatermarks; use crate::{CompactionGroupId, HummockSstableObjectId}; @@ -33,6 +34,7 @@ pub struct HummockVersion { pub max_committed_epoch: u64, pub safe_epoch: u64, pub table_watermarks: HashMap>, + pub table_change_log: HashMap, } impl Default for HummockVersion { @@ -74,6 +76,16 @@ impl HummockVersion { ) }) .collect(), + table_change_log: pb_version + .table_change_logs + .iter() + .map(|(table_id, change_log)| { + ( + TableId::new(*table_id), + TableChangeLog::from_protobuf(change_log), + ) + }) + .collect(), } } @@ -92,6 +104,11 @@ impl HummockVersion { .iter() .map(|(table_id, watermark)| (table_id.table_id, watermark.to_protobuf())) .collect(), + table_change_logs: self + .table_change_log + .iter() + .map(|(table_id, change_log)| (table_id.table_id, change_log.to_protobuf())) + .collect(), } } diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index eafa99042a506..30be483a72f61 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -1288,6 +1288,7 @@ mod tests { max_committed_epoch: epoch, safe_epoch: 0, table_watermarks: HashMap::new(), + table_change_log: HashMap::new(), } } diff --git a/src/storage/src/hummock/iterator/change_log.rs b/src/storage/src/hummock/iterator/change_log.rs new file mode 100644 index 0000000000000..8d6187b8cd474 --- /dev/null +++ b/src/storage/src/hummock/iterator/change_log.rs @@ -0,0 +1,695 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::cmp::Ordering; +use std::ops::Bound::{Excluded, Included, Unbounded}; + +use risingwave_common::catalog::TableId; +use risingwave_common::must_match; +use risingwave_common::util::epoch::MAX_SPILL_TIMES; +use risingwave_hummock_sdk::key::{FullKey, SetSlice, TableKeyRange, UserKey, UserKeyRange}; +use risingwave_hummock_sdk::EpochWithGap; + +use crate::error::StorageResult; +use crate::hummock::iterator::{Forward, HummockIterator, MergeIterator}; +use crate::hummock::value::HummockValue; +use crate::hummock::{HummockResult, SstableIterator}; +use crate::store::{ChangeLogValue, StateStoreReadLogItem, StateStoreReadLogItemRef}; +use crate::StateStoreIter; + +struct ChangeLogIteratorInner< + NI: HummockIterator, + OI: HummockIterator, +> { + /// Iterator for new value. In each `next`, the iterator will iterate over all value of the current key. + /// Therefore, we need to buffer the key and newest value in `curr_key` and `new_value`. + /// + /// We assume that all operation between `min_epoch` and `max_epoch` will be included in the `new_value_iter`. + new_value_iter: NI, + /// Iterator for old value. When `is_old_value_set` is true, its value is the old value in the change log value. + /// + /// We assume that each old value will have a new value of the same epoch in the `new_value_iter`. This is to say, + /// For a specific key, we won't have an epoch that only exists in the `old_value_iter` but not exists in `new_value_iter`. + /// `Delete` also contains a tombstone value. + old_value_iter: OI, + /// Inclusive max epoch + max_epoch: u64, + /// Inclusive min epoch + min_epoch: u64, + key_range: UserKeyRange, + + /// Buffer of current key + curr_key: FullKey>, + /// Buffer for new value. Only valid when `is_new_value_delete` is true + new_value: Vec, + /// Indicate whether the current new value is delete. + is_new_value_delete: bool, + + /// Whether Indicate whether the current `old_value_iter` represents the old value in ChangeLogValue + is_old_value_set: bool, + + /// Whether the iterator is currently pointing at a valid key with ChangeLogValue + is_current_pos_valid: bool, +} + +impl, OI: HummockIterator> + ChangeLogIteratorInner +{ + fn new( + (min_epoch, max_epoch): (u64, u64), + key_range: UserKeyRange, + new_value_iter: NI, + old_value_iter: OI, + ) -> Self { + Self { + new_value_iter, + old_value_iter, + min_epoch, + max_epoch, + key_range, + + curr_key: FullKey::default(), + new_value: vec![], + is_new_value_delete: false, + is_old_value_set: false, + is_current_pos_valid: false, + } + } + + /// Resets the iterating position to the beginning. + pub async fn rewind(&mut self) -> HummockResult<()> { + // Handle range scan + match &self.key_range.0 { + Included(begin_key) => { + let full_key = FullKey { + user_key: begin_key.as_ref(), + epoch_with_gap: EpochWithGap::new(self.max_epoch, MAX_SPILL_TIMES), + }; + self.new_value_iter.seek(full_key).await?; + self.old_value_iter.seek(full_key).await?; + } + Excluded(_) => unimplemented!("excluded begin key is not supported"), + Unbounded => { + self.new_value_iter.rewind().await?; + self.old_value_iter.rewind().await?; + } + }; + + self.try_advance_to_next_change_log_value().await?; + Ok(()) + } + + pub async fn next(&mut self) -> HummockResult<()> { + self.try_advance_to_next_change_log_value().await + } + + pub fn is_valid(&self) -> bool { + self.is_current_pos_valid + } + + pub fn log_value(&self) -> ChangeLogValue<&[u8]> { + if self.is_new_value_delete { + ChangeLogValue::Delete( + self.old_value() + .expect("should have old value when new value is delete"), + ) + } else { + match self.old_value() { + Some(old_value) => ChangeLogValue::Update { + new_value: self.new_value.as_slice(), + old_value, + }, + None => ChangeLogValue::Insert(self.new_value.as_slice()), + } + } + } + + pub fn key(&self) -> UserKey<&[u8]> { + self.curr_key.user_key.as_ref() + } +} + +impl, OI: HummockIterator> + ChangeLogIteratorInner +{ + async fn try_advance_to_next_change_log_value(&mut self) -> HummockResult<()> { + loop { + self.try_advance_to_next_valid().await?; + if !self.is_valid() { + break; + } + if self.has_log_value() { + break; + } else { + continue; + } + } + Ok(()) + } + + fn user_key_out_of_range(&self, user_key: UserKey<&[u8]>) -> bool { + // handle range scan + match &self.key_range.1 { + Included(end_key) => user_key > end_key.as_ref(), + Excluded(end_key) => user_key >= end_key.as_ref(), + Unbounded => false, + } + } + + /// Advance the `new_value_iter` to a valid key and valid epoch. + async fn advance_to_valid_key(&mut self) -> HummockResult<()> { + self.is_current_pos_valid = false; + loop { + if !self.new_value_iter.is_valid() { + return Ok(()); + } + + let key = self.new_value_iter.key(); + + // Handle epoch visibility + if !self.is_valid_epoch(key.epoch_with_gap) { + self.new_value_iter.next().await?; + continue; + } + + if self.user_key_out_of_range(key.user_key) { + return Ok(()); + } + + break; + } + + debug_assert!(self.new_value_iter.is_valid()); + debug_assert!(self.is_valid_epoch(self.new_value_iter.key().epoch_with_gap)); + debug_assert!(!self.user_key_out_of_range(self.new_value_iter.key().user_key)); + self.is_current_pos_valid = true; + // The key and value will be saved in a buffer, because in the next step we will + // continue advancing the `new_value_iter`. + self.curr_key.set(self.new_value_iter.key()); + match self.new_value_iter.value() { + HummockValue::Put(val) => { + self.new_value.set(val); + self.is_new_value_delete = false; + } + HummockValue::Delete => { + self.new_value.clear(); + self.is_new_value_delete = true; + } + } + + Ok(()) + } + + /// Advance the `new_value_iter` to find the oldest epoch of the current key. + async fn advance_to_find_oldest_epoch(&mut self) -> HummockResult { + let mut ret = self.curr_key.epoch_with_gap; + debug_assert!(self.is_valid_epoch(ret)); + self.new_value_iter.next().await?; + loop { + if !self.new_value_iter.is_valid() { + break; + } + let key = self.new_value_iter.key(); + match self.curr_key.user_key.as_ref().cmp(&key.user_key) { + Ordering::Less => { + // has advance to next key + break; + } + Ordering::Equal => { + assert!(ret > key.epoch_with_gap); + if !self.is_valid_epoch(key.epoch_with_gap) { + debug_assert!(self.min_epoch > key.epoch_with_gap.pure_epoch()); + break; + } + ret = key.epoch_with_gap; + self.new_value_iter.next().await?; + continue; + } + Ordering::Greater => { + unreachable!( + "hummock iterator advance to a prev key: {:?} {:?}", + self.curr_key, + self.new_value_iter.key() + ); + } + } + } + debug_assert!(self.is_valid_epoch(ret)); + + Ok(ret) + } + + /// Advance the two iters to a valid position. After it returns with Ok, + /// it is possible that the position is valid but there is no change log value, + /// because the new and old value may consume each other, such as Insert in old epoch, + /// but then Delete in new epoch + async fn try_advance_to_next_valid(&mut self) -> HummockResult<()> { + // 1. advance the new_value_iter to the newest op between max and min epoch + self.advance_to_valid_key().await?; + + if !self.is_current_pos_valid { + return Ok(()); + } + + // 2. advance new_value_iter to out of the valid range, and save the oldest value + let oldest_epoch = self.advance_to_find_oldest_epoch().await?; + + // 3. iterate old value iter to the oldest epoch + self.is_old_value_set = false; + loop { + if !self.old_value_iter.is_valid() { + break; + } + + let old_value_iter_key = self.old_value_iter.key(); + match self + .curr_key + .user_key + .as_ref() + .cmp(&old_value_iter_key.user_key.as_ref()) + { + Ordering::Less => { + // old value iter has advanced over the current range + break; + } + Ordering::Equal => match old_value_iter_key.epoch_with_gap.cmp(&oldest_epoch) { + Ordering::Less => { + // The assertion holds because we assume that for a specific key, any old value will have a new value of the same + // epoch in the `new_value_iter`. If the assertion is broken, it means we must have a new value of the same epoch + // that are valid but older than the `oldest_epoch`, which breaks the definition of `oldest_epoch`. + assert!( + old_value_iter_key.epoch_with_gap.pure_epoch() < self.min_epoch, + "there should not be old value between oldest new_value and min_epoch. \ + new value key: {:?}, oldest epoch: {:?}, min epoch: {:?}, old value epoch: {:?}", + self.curr_key, oldest_epoch, self.min_epoch, old_value_iter_key.epoch_with_gap + ); + break; + } + Ordering::Equal => { + self.is_old_value_set = true; + break; + } + Ordering::Greater => { + self.old_value_iter.next().await?; + continue; + } + }, + Ordering::Greater => { + self.old_value_iter.next().await?; + continue; + } + } + } + + Ok(()) + } + + fn is_valid_epoch(&self, epoch: EpochWithGap) -> bool { + let epoch = epoch.pure_epoch(); + self.min_epoch <= epoch && epoch <= self.max_epoch + } + + fn old_value(&self) -> Option<&[u8]> { + if self.is_old_value_set { + debug_assert!(self.old_value_iter.is_valid()); + debug_assert_eq!( + self.old_value_iter.key().user_key, + self.curr_key.user_key.as_ref() + ); + Some(must_match!(self.old_value_iter.value(), HummockValue::Put(val) => val)) + } else { + None + } + } + + fn has_log_value(&self) -> bool { + debug_assert!(self.is_current_pos_valid); + !self.is_new_value_delete || self.is_old_value_set + } +} + +pub struct ChangeLogIterator { + inner: ChangeLogIteratorInner, MergeIterator>, + initial_read: bool, +} + +impl ChangeLogIterator { + pub async fn new( + epoch_range: (u64, u64), + (start_bound, end_bound): TableKeyRange, + new_value_iter: MergeIterator, + old_value_iter: MergeIterator, + table_id: TableId, + ) -> HummockResult { + let make_user_key = |table_key| UserKey { + table_id, + table_key, + }; + let start_bound = start_bound.map(make_user_key); + let end_bound = end_bound.map(make_user_key); + let mut inner = ChangeLogIteratorInner::new( + epoch_range, + (start_bound, end_bound), + new_value_iter, + old_value_iter, + ); + inner.rewind().await?; + Ok(Self { + inner, + initial_read: false, + }) + } +} + +impl StateStoreIter for ChangeLogIterator { + async fn try_next(&mut self) -> StorageResult>> { + if !self.initial_read { + self.initial_read = true; + } else { + self.inner.next().await?; + } + if self.inner.is_valid() { + Ok(Some((self.inner.key().table_key, self.inner.log_value()))) + } else { + Ok(None) + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::BTreeMap; + use std::ops::Bound::Unbounded; + + use bytes::Bytes; + use itertools::Itertools; + use rand::{thread_rng, Rng, RngCore}; + use risingwave_common::catalog::TableId; + use risingwave_common::hash::table_distribution::TableDistribution; + use risingwave_common::util::epoch::{test_epoch, EpochPair}; + use risingwave_hummock_sdk::key::{TableKey, UserKey}; + use risingwave_hummock_sdk::EpochWithGap; + + use crate::hummock::iterator::change_log::ChangeLogIteratorInner; + use crate::hummock::iterator::test_utils::{ + iterator_test_table_key_of, iterator_test_value_of, + }; + use crate::hummock::iterator::MergeIterator; + use crate::mem_table::{KeyOp, MemTable, MemTableHummockIterator, MemTableStore}; + use crate::memory::MemoryStateStore; + use crate::store::{ + ChangeLogValue, InitOptions, LocalStateStore, NewLocalOptions, OpConsistencyLevel, + ReadLogOptions, ReadOptions, SealCurrentEpochOptions, StateStoreIter, StateStoreRead, + CHECK_BYTES_EQUAL, + }; + use crate::StateStore; + + #[tokio::test] + async fn test_empty() { + let table_id = TableId::new(233); + let epoch = EpochWithGap::new_from_epoch(test_epoch(1)); + let empty = BTreeMap::new(); + let new_value_iter = MemTableHummockIterator::new(&empty, epoch, table_id); + let old_value_iter = MemTableHummockIterator::new(&empty, epoch, table_id); + let mut iter = ChangeLogIteratorInner::new( + (epoch.pure_epoch(), epoch.pure_epoch()), + (Unbounded, Unbounded), + new_value_iter, + old_value_iter, + ); + iter.rewind().await.unwrap(); + assert!(!iter.is_valid()); + } + + #[tokio::test] + async fn test_append_only() { + let table_id = TableId::new(233); + + let count = 100; + let kvs = (0..count) + .map(|i| { + ( + TableKey(Bytes::from(iterator_test_table_key_of(i))), + Bytes::from(iterator_test_value_of(i)), + ) + }) + .collect_vec(); + let mem_tables = kvs + .iter() + .map(|(key, value)| { + let mut t = MemTable::new(OpConsistencyLevel::Inconsistent); + t.insert(key.clone(), value.clone()).unwrap(); + t + }) + .collect_vec(); + let epoch = EpochWithGap::new_from_epoch(test_epoch(1)); + let new_value_iter = MergeIterator::new( + mem_tables + .iter() + .map(|mem_table| MemTableHummockIterator::new(&mem_table.buffer, epoch, table_id)), + ); + let empty = BTreeMap::new(); + let old_value_iter = MemTableHummockIterator::new(&empty, epoch, table_id); + let mut iter = ChangeLogIteratorInner::new( + (epoch.pure_epoch(), epoch.pure_epoch()), + (Unbounded, Unbounded), + new_value_iter, + old_value_iter, + ); + iter.rewind().await.unwrap(); + for (key, value) in kvs { + assert!(iter.is_valid()); + assert_eq!( + UserKey { + table_id, + table_key: key.to_ref(), + }, + iter.key() + ); + assert_eq!(ChangeLogValue::Insert(value.as_ref()), iter.log_value()); + iter.next().await.unwrap(); + } + assert!(!iter.is_valid()); + } + + #[tokio::test] + async fn test_delete_only() { + let table_id = TableId::new(233); + + let count = 100; + let kvs = (0..count) + .map(|i| { + ( + TableKey(Bytes::from(iterator_test_table_key_of(i))), + Bytes::from(iterator_test_value_of(i)), + ) + }) + .collect_vec(); + let mut new_value_memtable = MemTable::new(OpConsistencyLevel::Inconsistent); + let mut old_value_memtable = MemTable::new(OpConsistencyLevel::Inconsistent); + for (key, value) in &kvs { + new_value_memtable + .delete(key.clone(), Bytes::new()) + .unwrap(); + old_value_memtable + .insert(key.clone(), value.clone()) + .unwrap(); + } + let epoch = EpochWithGap::new_from_epoch(test_epoch(1)); + let new_value_iter = + MemTableHummockIterator::new(&new_value_memtable.buffer, epoch, table_id); + let old_value_iter = + MemTableHummockIterator::new(&old_value_memtable.buffer, epoch, table_id); + let mut iter = ChangeLogIteratorInner::new( + (epoch.pure_epoch(), epoch.pure_epoch()), + (Unbounded, Unbounded), + new_value_iter, + old_value_iter, + ); + iter.rewind().await.unwrap(); + for (key, value) in kvs { + assert!(iter.is_valid()); + assert_eq!( + UserKey { + table_id, + table_key: key.to_ref(), + }, + iter.key() + ); + assert_eq!(ChangeLogValue::Delete(value.as_ref()), iter.log_value()); + iter.next().await.unwrap(); + } + assert!(!iter.is_valid()); + } + + async fn gen_test_data( + table_id: TableId, + epoch_count: usize, + key_count: usize, + delete_ratio: f64, + ) -> (Vec<(u64, MemTableStore, MemTableStore)>, MemoryStateStore) { + let state_store = MemoryStateStore::new(); + let mut rng = thread_rng(); + let mut local = state_store + .new_local(NewLocalOptions { + table_id, + op_consistency_level: OpConsistencyLevel::ConsistentOldValue( + CHECK_BYTES_EQUAL.clone(), + ), + table_option: Default::default(), + is_replicated: false, + vnodes: TableDistribution::all_vnodes(), + }) + .await; + let mut logs = Vec::new(); + for epoch_idx in 1..=epoch_count { + let epoch = test_epoch(epoch_idx as _); + let mut new_values = MemTableStore::new(); + let mut old_values = MemTableStore::new(); + if epoch_idx == 1 { + local + .init(InitOptions { + epoch: EpochPair::new_test_epoch(epoch), + }) + .await + .unwrap(); + } else { + local.flush().await.unwrap(); + local.seal_current_epoch( + epoch, + SealCurrentEpochOptions { + table_watermarks: None, + switch_op_consistency_level: None, + }, + ); + } + for key_idx in 0..key_count { + let key = TableKey(Bytes::from(iterator_test_table_key_of(key_idx))); + if rng.gen_bool(delete_ratio) { + if let Some(prev_value) = local + .get( + key.clone(), + ReadOptions { + prefix_hint: None, + ignore_range_tombstone: false, + prefetch_options: Default::default(), + cache_policy: Default::default(), + retention_seconds: None, + table_id, + read_version_from_backup: false, + }, + ) + .await + .unwrap() + { + new_values.insert(key.clone(), KeyOp::Delete(Bytes::new())); + old_values.insert(key.clone(), KeyOp::Insert(prev_value.clone())); + local.delete(key, prev_value).unwrap(); + } + } else { + let value = Bytes::copy_from_slice(rng.next_u64().to_string().as_bytes()); + new_values.insert(key.clone(), KeyOp::Insert(value.clone())); + let prev_value = local + .get( + key.clone(), + ReadOptions { + prefix_hint: None, + ignore_range_tombstone: false, + prefetch_options: Default::default(), + cache_policy: Default::default(), + retention_seconds: None, + table_id, + read_version_from_backup: false, + }, + ) + .await + .unwrap(); + if let Some(prev_value) = prev_value.clone() { + old_values.insert(key.clone(), KeyOp::Insert(prev_value)); + } + local.insert(key, value, prev_value).unwrap(); + } + } + logs.push((epoch, new_values, old_values)); + } + local.flush().await.unwrap(); + local.seal_current_epoch( + test_epoch((epoch_count + 1) as _), + SealCurrentEpochOptions { + table_watermarks: None, + switch_op_consistency_level: None, + }, + ); + (logs, state_store) + } + + #[tokio::test] + async fn test_random_data() { + let table_id = TableId::new(233); + let epoch_count = 10; + let (logs, state_store) = gen_test_data(table_id, epoch_count, 10000, 0.2).await; + assert_eq!(logs.len(), epoch_count); + for start_epoch_idx in 0..epoch_count { + for end_epoch_idx in start_epoch_idx + 1..epoch_count { + let new_value_iter = + MergeIterator::new(logs.iter().map(|(epoch, new_value_memtable, _)| { + MemTableHummockIterator::new( + new_value_memtable, + EpochWithGap::new_from_epoch(*epoch), + table_id, + ) + })); + let old_value_iter = + MergeIterator::new(logs.iter().map(|(epoch, _, old_value_memtable)| { + MemTableHummockIterator::new( + old_value_memtable, + EpochWithGap::new_from_epoch(*epoch), + table_id, + ) + })); + let epoch_range = (logs[start_epoch_idx].0, logs[end_epoch_idx].0); + let mut change_log_iter = ChangeLogIteratorInner::new( + epoch_range, + (Unbounded, Unbounded), + new_value_iter, + old_value_iter, + ); + change_log_iter.rewind().await.unwrap(); + let mut expected_change_log_iter = state_store + .iter_log( + epoch_range, + (Unbounded, Unbounded), + ReadLogOptions { table_id }, + ) + .await + .unwrap(); + while let Some((key, change_log_value)) = + expected_change_log_iter.try_next().await.unwrap() + { + assert!(change_log_iter.is_valid()); + assert_eq!( + change_log_iter.key(), + UserKey { + table_id, + table_key: key, + }, + ); + assert_eq!(change_log_iter.log_value(), change_log_value); + change_log_iter.next().await.unwrap(); + } + assert!(!change_log_iter.is_valid()); + } + } + } +} diff --git a/src/storage/src/hummock/iterator/mod.rs b/src/storage/src/hummock/iterator/mod.rs index 83bf2de61b136..9cae83be3f895 100644 --- a/src/storage/src/hummock/iterator/mod.rs +++ b/src/storage/src/hummock/iterator/mod.rs @@ -41,6 +41,8 @@ use risingwave_hummock_sdk::EpochWithGap; use crate::hummock::iterator::HummockIteratorUnion::{First, Fourth, Second, Third}; +mod change_log; +pub use change_log::*; mod concat_delete_range_iterator; mod delete_range_iterator; mod skip_watermark; diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index fd392a3e023c7..2bdbd70c17d56 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -47,6 +47,7 @@ use crate::hummock::event_handler::hummock_event_handler::{BufferTracker, Hummoc use crate::hummock::event_handler::{ HummockEvent, HummockEventHandler, HummockVersionUpdate, ReadOnlyReadVersionMapping, }; +use crate::hummock::iterator::ChangeLogIterator; use crate::hummock::local_version::pinned_version::{start_pinned_version_worker, PinnedVersion}; use crate::hummock::observer_manager::HummockObserverNode; use crate::hummock::utils::{validate_safe_epoch, wait_for_epoch}; @@ -459,7 +460,7 @@ impl HummockStorage { } impl StateStoreRead for HummockStorage { - type ChangeLogIter = PanicStateStoreIter; + type ChangeLogIter = ChangeLogIterator; type Iter = HummockStorageIterator; fn get( @@ -490,11 +491,16 @@ impl StateStoreRead for HummockStorage { async fn iter_log( &self, - _epoch_range: (u64, u64), - _key_range: TableKeyRange, - _options: ReadLogOptions, + epoch_range: (u64, u64), + key_range: TableKeyRange, + options: ReadLogOptions, ) -> StorageResult { - unimplemented!() + let version = (**self.pinned_version.load()).clone(); + let iter = self + .hummock_version_reader + .iter_log(version, epoch_range, key_range, options) + .await?; + Ok(iter) } } @@ -599,8 +605,6 @@ impl StateStore for HummockStorage { #[cfg(any(test, feature = "test"))] use risingwave_hummock_sdk::version::HummockVersion; -use crate::panic_store::PanicStateStoreIter; - #[cfg(any(test, feature = "test"))] impl HummockStorage { pub async fn seal_and_sync_epoch(&self, epoch: u64) -> StorageResult { diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index 5215a9eaf18db..d9a83c8695b14 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -30,7 +30,8 @@ use crate::error::StorageResult; use crate::hummock::event_handler::hummock_event_handler::HummockEventSender; use crate::hummock::event_handler::{HummockEvent, HummockReadVersionRef, LocalInstanceGuard}; use crate::hummock::iterator::{ - ConcatIteratorInner, Forward, HummockIteratorUnion, MergeIterator, UserIterator, + ChangeLogIterator, ConcatIteratorInner, Forward, HummockIteratorUnion, MergeIterator, + UserIterator, }; use crate::hummock::shared_buffer::shared_buffer_batch::{ SharedBufferBatch, SharedBufferBatchIterator, @@ -44,7 +45,6 @@ use crate::hummock::write_limiter::WriteLimiterRef; use crate::hummock::{MemoryLimiter, SstableIterator}; use crate::mem_table::{KeyOp, MemTable, MemTableHummockIterator}; use crate::monitor::{HummockStateStoreMetrics, IterLocalMetricsGuard, StoreLocalStatistic}; -use crate::panic_store::PanicStateStoreIter; use crate::storage_value::StorageValue; use crate::store::*; @@ -206,7 +206,7 @@ impl LocalHummockStorage { } impl StateStoreRead for LocalHummockStorage { - type ChangeLogIter = PanicStateStoreIter; + type ChangeLogIter = ChangeLogIterator; type Iter = HummockStorageIterator; fn get( @@ -232,11 +232,16 @@ impl StateStoreRead for LocalHummockStorage { async fn iter_log( &self, - _epoch_range: (u64, u64), - _key_range: TableKeyRange, - _options: ReadLogOptions, + epoch_range: (u64, u64), + key_range: TableKeyRange, + options: ReadLogOptions, ) -> StorageResult { - unimplemented!() + let version = self.read_version.read().committed().clone(); + let iter = self + .hummock_version_reader + .iter_log(version, epoch_range, key_range, options) + .await?; + Ok(iter) } } diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 3ce49380863a4..91b6394ed160a 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -21,12 +21,14 @@ use std::sync::Arc; use std::time::Instant; use bytes::Bytes; +use futures::future::try_join_all; use itertools::Itertools; use parking_lot::RwLock; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::MAX_SPILL_TIMES; +use risingwave_hummock_sdk::change_log::EpochNewChangeLog; use risingwave_hummock_sdk::key::{ bound_table_key_range, is_empty_key_range, FullKey, TableKey, TableKeyRange, UserKey, }; @@ -41,7 +43,9 @@ use sync_point::sync_point; use super::StagingDataIterator; use crate::error::StorageResult; -use crate::hummock::iterator::{ConcatIterator, HummockIteratorUnion, MergeIterator, UserIterator}; +use crate::hummock::iterator::{ + ChangeLogIterator, ConcatIterator, HummockIteratorUnion, MergeIterator, UserIterator, +}; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::sstable::SstableIteratorReadOptions; use crate::hummock::sstable_store::SstableStoreRef; @@ -50,15 +54,15 @@ use crate::hummock::utils::{ prune_overlapping_ssts, range_overlap, search_sst_idx, }; use crate::hummock::{ - get_from_batch, get_from_sstable_info, hit_sstable_bloom_filter, HummockStorageIterator, - HummockStorageIteratorInner, LocalHummockStorageIterator, ReadVersionTuple, Sstable, - SstableIterator, + get_from_batch, get_from_sstable_info, hit_sstable_bloom_filter, HummockError, HummockResult, + HummockStorageIterator, HummockStorageIteratorInner, LocalHummockStorageIterator, + ReadVersionTuple, Sstable, SstableIterator, }; use crate::mem_table::{ImmId, ImmutableMemtable, MemTableHummockIterator}; use crate::monitor::{ GetLocalMetricsGuard, HummockStateStoreMetrics, MayExistLocalMetricsGuard, StoreLocalStatistic, }; -use crate::store::{gen_min_epoch, ReadOptions}; +use crate::store::{gen_min_epoch, ReadLogOptions, ReadOptions}; pub type CommittedVersion = PinnedVersion; @@ -1123,4 +1127,75 @@ impl HummockVersionReader { Ok(false) } + + pub async fn iter_log( + &self, + version: PinnedVersion, + epoch_range: (u64, u64), + key_range: TableKeyRange, + options: ReadLogOptions, + ) -> HummockResult { + let change_log = + if let Some(change_log) = version.version().table_change_log.get(&options.table_id) { + change_log.filter_epoch(epoch_range) + } else { + static EMPTY_VEC: Vec = Vec::new(); + &EMPTY_VEC[..] + }; + let read_options = Arc::new(SstableIteratorReadOptions { + cache_policy: Default::default(), + must_iterated_end_user_key: None, + max_preload_retry_times: 0, + prefetch_for_large_query: false, + }); + + async fn make_iter( + ssts: impl Iterator, + sstable_store: &SstableStoreRef, + read_options: Arc, + ) -> HummockResult> { + let iters = try_join_all(ssts.map(|sst| { + let sstable_store = sstable_store.clone(); + let read_options = read_options.clone(); + async move { + let mut local_stat = StoreLocalStatistic::default(); + let table_holder = sstable_store.sstable(sst, &mut local_stat).await?; + Ok::<_, HummockError>(SstableIterator::new( + table_holder, + sstable_store, + read_options, + )) + } + })) + .await?; + Ok::<_, HummockError>(MergeIterator::new(iters)) + } + + let new_value_iter = make_iter( + change_log + .iter() + .flat_map(|log| log.new_value.iter()) + .filter(|sst| filter_single_sst(sst, options.table_id, &key_range)), + &self.sstable_store, + read_options.clone(), + ) + .await?; + let old_value_iter = make_iter( + change_log + .iter() + .flat_map(|log| log.old_value.iter()) + .filter(|sst| filter_single_sst(sst, options.table_id, &key_range)), + &self.sstable_store, + read_options.clone(), + ) + .await?; + ChangeLogIterator::new( + epoch_range, + key_range, + new_value_iter, + old_value_iter, + options.table_id, + ) + .await + } } From b29ef06c68ba6d231adb79ee7c62231817b6ab93 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 19 Apr 2024 09:20:09 +0800 Subject: [PATCH 02/22] chore(standalone): add connection string (#16357) --- src/cmd_all/src/standalone.rs | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index eb5e1a67e9889..734460f20f6b0 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -202,7 +202,7 @@ pub async fn standalone( tracing::info!("starting compute-node thread with cli args: {:?}", opts); let _compute_handle = tokio::spawn(async move { risingwave_compute::start(opts).await }); } - if let Some(opts) = frontend_opts { + if let Some(opts) = frontend_opts.clone() { tracing::info!("starting frontend-node thread with cli args: {:?}", opts); let _frontend_handle = tokio::spawn(async move { risingwave_frontend::start(opts).await }); } @@ -214,8 +214,9 @@ pub async fn standalone( // wait for log messages to be flushed tokio::time::sleep(std::time::Duration::from_millis(5000)).await; - eprintln!("-------------------------------"); - eprintln!("RisingWave standalone mode is ready."); + eprintln!("----------------------------------------"); + eprintln!("| RisingWave standalone mode is ready. |"); + eprintln!("----------------------------------------"); if is_in_memory { eprintln!( "{}", @@ -227,6 +228,21 @@ It SHOULD NEVER be used in benchmarks and production environment!!!" .bold() ); } + if let Some(opts) = frontend_opts { + let host = opts.listen_addr.split(':').next().unwrap_or("localhost"); + let port = opts.listen_addr.split(':').last().unwrap_or("4566"); + let database = "dev"; + let user = "root"; + eprintln!(); + eprintln!("Connect to the RisingWave instance via psql:"); + eprintln!( + "{}", + console::style(format!( + " psql -h {host} -p {port} -d {database} -U {user}" + )) + .blue() + ); + } // TODO: should we join all handles? // Currently, not all services can be shutdown gracefully, just quit on Ctrl-C now. From 708ff99728a83b189a2c7f58b91d00c0b02b475b Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 19 Apr 2024 10:44:13 +0800 Subject: [PATCH 03/22] feat(risedev): add command reset-rw (#16382) Signed-off-by: xxchan --- Makefile.toml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/Makefile.toml b/Makefile.toml index 5e9d65c00dd31..8b1a88707bb70 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -128,6 +128,17 @@ rm -rf "${PREFIX_CONFIG}" rm -rf "${PREFIX_PROFILING}" ''' +[tasks.reset-rw] +category = "RiseDev - Start/Stop" +description = "Clean all data in the default database dev of the running RisingWave" +dependencies = ["check-risedev-env-file"] +env_files = ["${PREFIX_CONFIG}/risedev-env"] +script = ''' +#!/usr/bin/env bash +psql -h $RISEDEV_RW_FRONTEND_LISTEN_ADDRESS -p $RISEDEV_RW_FRONTEND_PORT -U root -d dev -c "CREATE DATABASE risedev_tmp;" +psql -h $RISEDEV_RW_FRONTEND_LISTEN_ADDRESS -p $RISEDEV_RW_FRONTEND_PORT -U root -d risedev_tmp -c "DROP DATABASE dev; CREATE DATABASE dev;" +psql -h $RISEDEV_RW_FRONTEND_LISTEN_ADDRESS -p $RISEDEV_RW_FRONTEND_PORT -U root -d dev -c "DROP DATABASE risedev_tmp;" +''' [tasks.l] alias = "logs" From 26629e7be7f2611ce8ef1b66e70485901042ed74 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 19 Apr 2024 11:17:19 +0800 Subject: [PATCH 04/22] chore(deps): Bump google-cloud-pubsub from 0.23.0 to 0.24.0 in the google-cloud group (#16389) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 45 +++++---------------- scripts/source/prepare_ci_pubsub/Cargo.toml | 2 +- src/connector/Cargo.toml | 2 +- src/risedevtool/Cargo.toml | 2 +- 4 files changed, 14 insertions(+), 37 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index edae74d715f75..3b6d51bd49ba6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5472,16 +5472,16 @@ dependencies = [ [[package]] name = "google-cloud-auth" -version = "0.13.0" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af1087f1fbd2dd3f58c17c7574ddd99cd61cbbbc2c4dc81114b8687209b196cb" +checksum = "fe54dd8b6eb2bcd5390998238bcc39d1daed4dbb70df2845832532540384fc41" dependencies = [ "async-trait", "base64 0.21.7", "google-cloud-metadata", "google-cloud-token", "home", - "jsonwebtoken 8.3.0", + "jsonwebtoken", "reqwest 0.11.20", "serde", "serde_json", @@ -5532,9 +5532,9 @@ dependencies = [ [[package]] name = "google-cloud-pubsub" -version = "0.23.0" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1da196da473976944d408a91213bafe078e7223e10694d3f8ed36b6e210fa130" +checksum = "0b2184a5c70b994e6d77eb1c140e193e7f5fe6015e9115322fac24f7e33f003c" dependencies = [ "async-channel", "async-stream", @@ -6484,20 +6484,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "jsonwebtoken" -version = "8.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6971da4d9c3aa03c3d8f3ff0f4155b534aad021292003895a469716b2a230378" -dependencies = [ - "base64 0.21.7", - "pem 1.1.1", - "ring 0.16.20", - "serde", - "serde_json", - "simple_asn1", -] - [[package]] name = "jsonwebtoken" version = "9.3.0" @@ -6506,7 +6492,7 @@ checksum = "b9ae10193d25051e74945f1ea2d0b42e03cc3b890f7e4cc5faa44997d808193f" dependencies = [ "base64 0.21.7", "js-sys", - "pem 3.0.2", + "pem", "ring 0.17.5", "serde", "serde_json", @@ -7316,7 +7302,7 @@ dependencies = [ "mysql_common", "native-tls", "once_cell", - "pem 3.0.2", + "pem", "percent-encoding", "pin-project", "rand", @@ -8343,15 +8329,6 @@ dependencies = [ "hmac", ] -[[package]] -name = "pem" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8835c273a76a90455d7344889b0964598e3316e2a79ede8e36f16bdcf2228b8" -dependencies = [ - "base64 0.13.1", -] - [[package]] name = "pem" version = "3.0.2" @@ -8408,7 +8385,7 @@ dependencies = [ "bytes", "futures", "itertools 0.12.1", - "jsonwebtoken 9.3.0", + "jsonwebtoken", "madsim-tokio", "openssl", "panic-message", @@ -9229,7 +9206,7 @@ dependencies = [ "nom", "oauth2", "openidconnect", - "pem 3.0.2", + "pem", "prost 0.11.9", "prost-build 0.11.9", "prost-derive 0.11.9", @@ -9643,7 +9620,7 @@ dependencies = [ "hmac", "home", "http 0.2.9", - "jsonwebtoken 9.3.0", + "jsonwebtoken", "log", "once_cell", "percent-encoding", @@ -10402,7 +10379,7 @@ dependencies = [ "itertools 0.12.1", "jni", "jsonschema-transpiler", - "jsonwebtoken 9.3.0", + "jsonwebtoken", "madsim-rdkafka", "madsim-tokio", "madsim-tonic", diff --git a/scripts/source/prepare_ci_pubsub/Cargo.toml b/scripts/source/prepare_ci_pubsub/Cargo.toml index 52d6e2460a3db..ff3fd2b311468 100644 --- a/scripts/source/prepare_ci_pubsub/Cargo.toml +++ b/scripts/source/prepare_ci_pubsub/Cargo.toml @@ -13,7 +13,7 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" google-cloud-googleapis = { version = "0.12", features = ["pubsub"] } -google-cloud-pubsub = "0.23" +google-cloud-pubsub = "0.24" tokio = { version = "0.2", package = "madsim-tokio", features = [ "rt", "rt-multi-thread", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 51437f82e3ec5..224c877dc6366 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -58,7 +58,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } gcp-bigquery-client = "0.18.0" glob = "0.3" -google-cloud-pubsub = "0.23" +google-cloud-pubsub = "0.24" http = "0.2" icelake = { workspace = true } indexmap = { version = "1.9.3", features = ["serde"] } diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index a73fb078dd963..78ec4b5a63e45 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -23,7 +23,7 @@ clap = { workspace = true } console = "0.15" fs-err = "2.11.0" glob = "0.3" -google-cloud-pubsub = "0.23" +google-cloud-pubsub = "0.24" indicatif = "0.17" itertools = { workspace = true } rdkafka = { workspace = true } From c3b494c0468363ab402a1f4b7ad181e1cd0df7eb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 19 Apr 2024 11:18:45 +0800 Subject: [PATCH 05/22] chore(deps): Bump duration-str from 0.7.0 to 0.7.1 (#16390) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- src/connector/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3b6d51bd49ba6..efcaf78f5f6c3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4227,9 +4227,9 @@ checksum = "56ce8c6da7551ec6c462cbaf3bfbc75131ebbfa1c944aeaa9dab51ca1c5f0c3b" [[package]] name = "duration-str" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e172e85f305d6a442b250bf40667ffcb91a24f52c9a1ca59e2fa991ac9b7790" +checksum = "a8bb6a301a95ba86fa0ebaf71d49ae4838c51f8b84cb88ed140dfb66452bb3c4" dependencies = [ "chrono", "nom", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 224c877dc6366..18c70436cbca8 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -51,7 +51,7 @@ clickhouse = { git = "https://github.com/risingwavelabs/clickhouse.rs", rev = "d ] } csv = "1.3" deltalake = { workspace = true } -duration-str = "0.7.0" +duration-str = "0.7.1" easy-ext = "1" enum-as-inner = "0.6" futures = { version = "0.3", default-features = false, features = ["alloc"] } From 43e8df9c258180e2067e265dfc6d5dcdb581c0ba Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 19 Apr 2024 11:24:07 +0800 Subject: [PATCH 06/22] refactor: remove ouput_exprs from LogicalKafkaScan (#16385) --- .../optimizer/plan_node/logical_kafka_scan.rs | 61 +++++-------------- .../rule/source_to_kafka_scan_rule.rs | 2 +- 2 files changed, 16 insertions(+), 47 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs b/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs index 18ae05a135ff9..99333eef0779e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs @@ -24,12 +24,12 @@ use risingwave_connector::source::DataType; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - generic, BatchProject, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, - PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, PlanBase, + PlanRef, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; -use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor}; +use crate::expr::{Expr, ExprImpl, ExprType}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ @@ -44,27 +44,28 @@ pub struct LogicalKafkaScan { pub base: PlanBase, pub core: generic::Source, - /// Expressions to output. This field presents and will be turned to a `Project` when - /// converting to a physical plan, only if there are generated columns. - output_exprs: Option>, - /// Kafka timestamp range. kafka_timestamp_range: (Bound, Bound), } impl LogicalKafkaScan { - pub fn new(logical_source: &LogicalSource) -> Self { + pub fn create(logical_source: &LogicalSource) -> PlanRef { assert!(logical_source.core.is_kafka_connector()); - let base = logical_source.base.clone_with_new_plan_id(); let core = logical_source.core.clone(); + let base = PlanBase::new_logical_with_core(&core); let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); - LogicalKafkaScan { + let kafka_scan = LogicalKafkaScan { base, core, - output_exprs: logical_source.output_exprs.clone(), kafka_timestamp_range, + }; + + if let Some(exprs) = &logical_source.output_exprs { + LogicalProject::create(kafka_scan.into(), exprs.to_vec()) + } else { + kafka_scan.into() } } @@ -76,7 +77,6 @@ impl LogicalKafkaScan { Self { base: self.base.clone(), core: self.core.clone(), - output_exprs: self.output_exprs.clone(), kafka_timestamp_range: range, } } @@ -107,34 +107,9 @@ impl ColPrunable for LogicalKafkaScan { } } -impl ExprRewritable for LogicalKafkaScan { - fn has_rewritable_expr(&self) -> bool { - self.output_exprs.is_some() - } - - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut output_exprs = self.output_exprs.clone(); - - for expr in output_exprs.iter_mut().flatten() { - *expr = r.rewrite_expr(expr.clone()); - } - - Self { - output_exprs, - ..self.clone() - } - .into() - } -} +impl ExprRewritable for LogicalKafkaScan {} -impl ExprVisitable for LogicalKafkaScan { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - self.output_exprs - .iter() - .flatten() - .for_each(|e| v.visit_expr(e)); - } -} +impl ExprVisitable for LogicalKafkaScan {} /// A util function to extract kafka offset timestamp range. /// @@ -320,14 +295,8 @@ impl PredicatePushdown for LogicalKafkaScan { impl ToBatch for LogicalKafkaScan { fn to_batch(&self) -> Result { - let mut plan: PlanRef = + let plan: PlanRef = BatchKafkaScan::new(self.core.clone(), self.kafka_timestamp_range).into(); - - if let Some(exprs) = &self.output_exprs { - let logical_project = generic::Project::new(exprs.to_vec(), plan); - plan = BatchProject::new(logical_project).into(); - } - Ok(plan) } } diff --git a/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs b/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs index 244278bdc33c4..bda4a3068e8bf 100644 --- a/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs +++ b/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs @@ -21,7 +21,7 @@ impl Rule for SourceToKafkaScanRule { fn apply(&self, plan: PlanRef) -> Option { let source: &LogicalSource = plan.as_logical_source()?; if source.core.is_kafka_connector() { - Some(LogicalKafkaScan::new(source).into()) + Some(LogicalKafkaScan::create(source)) } else { None } From 57310af6b492cc72f8e8c6af5a8bcaed7690006b Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Fri, 19 Apr 2024 12:25:14 +0800 Subject: [PATCH 07/22] feat(cdc): allow decimal to rw_int256 and varchar (#16346) --- Cargo.lock | 14 ++ e2e_test/source/cdc/cdc.check.slt | 23 +++ e2e_test/source/cdc/cdc.check_new_rows.slt | 37 +++- e2e_test/source/cdc/cdc.load.slt | 33 ++++ e2e_test/source/cdc/cdc.share_stream.slt | 95 ++++++++++ e2e_test/source/cdc/cdc.validate.postgres.slt | 34 ++++ e2e_test/source/cdc/postgres_cdc.sql | 16 ++ e2e_test/source/cdc/postgres_cdc_insert.sql | 9 + .../source/common/PostgresValidator.java | 6 +- src/connector/Cargo.toml | 1 + src/connector/src/parser/postgres.rs | 171 +++++++++++++++++- 11 files changed, 427 insertions(+), 12 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index efcaf78f5f6c3..4d2f9864883d2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8364,6 +8364,19 @@ dependencies = [ "indexmap 2.0.0", ] +[[package]] +name = "pg_bigdecimal" +version = "0.1.5" +source = "git+https://github.com/risingwavelabs/rust-pg_bigdecimal?rev=0b7893d88894ca082b4525f94f812da034486f7c#0b7893d88894ca082b4525f94f812da034486f7c" +dependencies = [ + "bigdecimal 0.4.2", + "byteorder", + "bytes", + "num-bigint", + "num-integer", + "postgres-types", +] + [[package]] name = "pg_interval" version = "0.4.2" @@ -10393,6 +10406,7 @@ dependencies = [ "openssl", "parking_lot 0.12.1", "paste", + "pg_bigdecimal", "postgres-openssl", "pretty_assertions", "prometheus", diff --git a/e2e_test/source/cdc/cdc.check.slt b/e2e_test/source/cdc/cdc.check.slt index b90137948be1e..b6fb7f2156fbf 100644 --- a/e2e_test/source/cdc/cdc.check.slt +++ b/e2e_test/source/cdc/cdc.check.slt @@ -46,3 +46,26 @@ query I select count(*) from tt3_rw; ---- 2 + +query II +select * from numeric_to_varchar order by id; +---- +1 3.14 +2 57896044618658097711785492504343953926634992332820282019728792003956564819967 +3 57896044618658097711785492504343953926634992332820282019728792003956564819968 +4 115792089237316195423570985008687907853269984665640564039457584007913129639936 +5 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555 +6 NAN +7 POSITIVE_INFINITY + +# The invalid data for rw_int256 is converted to NULL +query II +select * from numeric_to_rw_int256 order by id; +---- +1 NULL +2 57896044618658097711785492504343953926634992332820282019728792003956564819967 +3 NULL +4 NULL +5 NULL +6 NULL +7 NULL diff --git a/e2e_test/source/cdc/cdc.check_new_rows.slt b/e2e_test/source/cdc/cdc.check_new_rows.slt index 5a97a21ba973a..e3e52c1e35d7e 100644 --- a/e2e_test/source/cdc/cdc.check_new_rows.slt +++ b/e2e_test/source/cdc/cdc.check_new_rows.slt @@ -77,4 +77,39 @@ SELECT * FROM rw.products_test order by id limit 3 query TTTT select order_date,customer_name,product_id,order_status from orders_no_backfill order by order_id; ---- -2022-12-01 15:08:22 Sam 110 0 \ No newline at end of file +2022-12-01 15:08:22 Sam 110 0 + +query II +select * from numeric_to_varchar order by id; +---- +1 3.14 +2 57896044618658097711785492504343953926634992332820282019728792003956564819967 +3 57896044618658097711785492504343953926634992332820282019728792003956564819968 +4 115792089237316195423570985008687907853269984665640564039457584007913129639936 +5 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555 +6 NAN +7 POSITIVE_INFINITY +102 57896044618658097711785492504343953926634992332820282019728792003956564819967 +103 57896044618658097711785492504343953926634992332820282019728792003956564819968 +104 115792089237316195423570985008687907853269984665640564039457584007913129639936 +105 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555 +106 NAN +107 POSITIVE_INFINITY + +# The invalid data for rw_int256 is converted to NULL +query II +select * from numeric_to_rw_int256 order by id; +---- +1 NULL +2 57896044618658097711785492504343953926634992332820282019728792003956564819967 +3 NULL +4 NULL +5 NULL +6 NULL +7 NULL +102 57896044618658097711785492504343953926634992332820282019728792003956564819967 +103 NULL +104 NULL +105 NULL +106 NULL +107 NULL \ No newline at end of file diff --git a/e2e_test/source/cdc/cdc.load.slt b/e2e_test/source/cdc/cdc.load.slt index 7dda34fe08518..1cff579bb054d 100644 --- a/e2e_test/source/cdc/cdc.load.slt +++ b/e2e_test/source/cdc/cdc.load.slt @@ -193,3 +193,36 @@ create table person_rw ( publication.name='dumb_publicaton', publication.create.enable='false' ); + +statement ok +create table numeric_to_rw_int256 ( + id int, + num rw_int256, + PRIMARY KEY (id) +) with ( + connector = 'postgres-cdc', + hostname = '${PGHOST:localhost}', + port = '${PGPORT:5432}', + username = '${PGUSER:$USER}', + password = '${PGPASSWORD:}', + database.name = '${PGDATABASE:postgres}', + table.name = 'numeric_table', + slot.name = 'numeric_to_rw_int256' +); + +statement ok +create table numeric_to_varchar ( + id int, + num varchar, + PRIMARY KEY (id) +) with ( + connector = 'postgres-cdc', + hostname = '${PGHOST:localhost}', + port = '${PGPORT:5432}', + username = '${PGUSER:$USER}', + password = '${PGPASSWORD:}', + database.name = '${PGDATABASE:postgres}', + schema.name = 'public', + table.name = 'numeric_table', + slot.name = 'numeric_to_varchar' +); diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index dc59de5f48005..f1a301baed800 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -268,3 +268,98 @@ SELECT * from person_new order by id; 1100 noris ypl@qbxfg.com 1864 2539 enne 1101 white myc@xpmpe.com 8157 6974 se 1102 spencer wip@dkaap.com 9481 6270 angeles + +statement ok +CREATE TABLE numeric_to_rw_int256_shared ( + id int, + num rw_int256, + PRIMARY KEY (id) +) FROM pg_source TABLE 'public.numeric_table'; + +statement ok +CREATE TABLE numeric_to_varchar_shared ( + id int, + num varchar, + PRIMARY KEY (id) +) FROM pg_source TABLE 'public.numeric_table'; + +statement ok +CREATE TABLE numeric_list_to_rw_int256_list_shared ( + id int, + num rw_int256[], + PRIMARY KEY (id) +) FROM pg_source TABLE 'public.numeric_list'; + +statement ok +CREATE TABLE numeric_list_to_varchar_list_shared ( + id int, + num varchar[], + PRIMARY KEY (id) +) FROM pg_source TABLE 'public.numeric_list'; + + +system ok +psql -c " +insert into numeric_table values(102, 57896044618658097711785492504343953926634992332820282019728792003956564819967); +--- 2^255 +insert into numeric_table values(103, 57896044618658097711785492504343953926634992332820282019728792003956564819968); +--- 2^256 +insert into numeric_table values(104, 115792089237316195423570985008687907853269984665640564039457584007913129639936); +insert into numeric_table values(105, 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555); +insert into numeric_table values(106, 'NaN'::numeric); +insert into numeric_table values(107, 'Infinity'::numeric); +" + +sleep 3s + +query II +select * from numeric_to_varchar_shared order by id; +---- +1 3.14 +2 57896044618658097711785492504343953926634992332820282019728792003956564819967 +3 57896044618658097711785492504343953926634992332820282019728792003956564819968 +4 115792089237316195423570985008687907853269984665640564039457584007913129639936 +5 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555 +6 NAN +7 POSITIVE_INFINITY +102 57896044618658097711785492504343953926634992332820282019728792003956564819967 +103 57896044618658097711785492504343953926634992332820282019728792003956564819968 +104 115792089237316195423570985008687907853269984665640564039457584007913129639936 +105 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555 +106 NAN +107 POSITIVE_INFINITY + +# The invalid data for rw_int256 is converted to NULL +query II +select * from numeric_to_rw_int256_shared order by id; +---- +1 NULL +2 57896044618658097711785492504343953926634992332820282019728792003956564819967 +3 NULL +4 NULL +5 NULL +6 NULL +7 NULL +102 57896044618658097711785492504343953926634992332820282019728792003956564819967 +103 NULL +104 NULL +105 NULL +106 NULL +107 NULL + +system ok +psql -c " +DELETE FROM numeric_table WHERE id IN (102, 103, 104, 105, 106, 107); +" + +query II +select * from numeric_list_to_varchar_list_shared order by id; +---- +1 {3.14,6,57896044618658097711785492504343953926634992332820282019728792003956564819967,57896044618658097711785492504343953926634992332820282019728792003956564819968,115792089237316195423570985008687907853269984665640564039457584007913129639936.555555} +2 {NAN,POSITIVE_INFINITY,NEGATIVE_INFINITY} + +query II +select * from numeric_list_to_rw_int256_list_shared order by id; +---- +1 {NULL,6,57896044618658097711785492504343953926634992332820282019728792003956564819967,NULL,NULL} +2 {NULL,NULL,NULL} \ No newline at end of file diff --git a/e2e_test/source/cdc/cdc.validate.postgres.slt b/e2e_test/source/cdc/cdc.validate.postgres.slt index 4ac6669913c58..1cf42983de49a 100644 --- a/e2e_test/source/cdc/cdc.validate.postgres.slt +++ b/e2e_test/source/cdc/cdc.validate.postgres.slt @@ -167,3 +167,37 @@ create table shipments ( table.name = 'shipments', slot.name = 'shipments' ) format canal encode csv; + +statement ok +explain create table numeric_to_rw_int256 ( + id int, + num rw_int256, + PRIMARY KEY (id) +) with ( + connector = 'postgres-cdc', + hostname = 'db', + port = '5432', + username = '${PGUSER:$USER}', + password = '${PGPASSWORD:}', + database.name = 'cdc_test', + schema.name = 'public', + table.name = 'numeric_table', + slot.name = 'numeric_to_rw_int256' +); + +statement ok +explain create table numeric_to_varchar ( + id int, + num varchar, + PRIMARY KEY (id) +) with ( + connector = 'postgres-cdc', + hostname = 'db', + port = '5432', + username = '${PGUSER:$USER}', + password = '${PGPASSWORD:}', + database.name = 'cdc_test', + schema.name = 'public', + table.name = 'numeric_table', + slot.name = 'numeric_to_varchar' +); diff --git a/e2e_test/source/cdc/postgres_cdc.sql b/e2e_test/source/cdc/postgres_cdc.sql index a4de0e447a0cc..0f8961f5289d2 100644 --- a/e2e_test/source/cdc/postgres_cdc.sql +++ b/e2e_test/source/cdc/postgres_cdc.sql @@ -72,3 +72,19 @@ CREATE TABLE IF NOT EXISTS postgres_all_types( ); INSERT INTO postgres_all_types VALUES ( False, 0, 0, 0, 0, 0, 0, '', '\x00', '0001-01-01', '00:00:00', '0001-01-01 00:00:00'::timestamp, '0001-01-01 00:00:00'::timestamptz, interval '0 second', '{}', array[]::boolean[], array[]::smallint[], array[]::integer[], array[]::bigint[], array[]::decimal[], array[]::real[], array[]::double precision[], array[]::varchar[], array[]::bytea[], array[]::date[], array[]::time[], array[]::timestamp[], array[]::timestamptz[], array[]::interval[], array[]::jsonb[], null); INSERT INTO postgres_all_types VALUES ( False, -32767, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, '', '\x00', '0001-01-01', '00:00:00', '0001-01-01 00:00:00'::timestamp, '0001-01-01 00:00:00'::timestamptz, interval '0 second', '{}', array[False::boolean]::boolean[], array[-32767::smallint]::smallint[], array[-2147483647::integer]::integer[], array[-9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[-9999.999999::real]::real[], array[-10000.0::double precision]::double precision[], array[''::varchar]::varchar[], array['\x00'::bytea]::bytea[], array['0001-01-01'::date]::date[], array['00:00:00'::time]::time[], array['0001-01-01 00:00:00'::timestamp::timestamp]::timestamp[], array['0001-01-01 00:00:00'::timestamptz::timestamptz]::timestamptz[], array[interval '0 second'::interval]::interval[], array['{}'::jsonb]::jsonb[], 'bb488f9b-330d-4012-b849-12adeb49e57e'); + +create table numeric_table(id int PRIMARY KEY, num numeric); +insert into numeric_table values(1, 3.14); +--- 2^255 - 1 +insert into numeric_table values(2, 57896044618658097711785492504343953926634992332820282019728792003956564819967); +--- 2^255 +insert into numeric_table values(3, 57896044618658097711785492504343953926634992332820282019728792003956564819968); +--- 2^256 +insert into numeric_table values(4, 115792089237316195423570985008687907853269984665640564039457584007913129639936); +insert into numeric_table values(5, 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555); +insert into numeric_table values(6, 'NaN'::numeric); +insert into numeric_table values(7, 'Infinity'::numeric); + +create table numeric_list(id int primary key, num numeric[]); +insert into numeric_list values(1, '{3.14, 6, 57896044618658097711785492504343953926634992332820282019728792003956564819967, 57896044618658097711785492504343953926634992332820282019728792003956564819968, 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555}'); +insert into numeric_list values(2, '{nan, infinity, -infinity}'); diff --git a/e2e_test/source/cdc/postgres_cdc_insert.sql b/e2e_test/source/cdc/postgres_cdc_insert.sql index 84ae9068c187a..2ff71bf389615 100644 --- a/e2e_test/source/cdc/postgres_cdc_insert.sql +++ b/e2e_test/source/cdc/postgres_cdc_insert.sql @@ -11,3 +11,12 @@ insert into abs.t1 values (2, 2.2, 'bbb', '1234.5431'); SELECT pg_current_wal_lsn(); select * from pg_publication_tables where pubname='rw_publication'; select * from public.person order by id; + +insert into numeric_table values(102, 57896044618658097711785492504343953926634992332820282019728792003956564819967); +--- 2^255 +insert into numeric_table values(103, 57896044618658097711785492504343953926634992332820282019728792003956564819968); +--- 2^256 +insert into numeric_table values(104, 115792089237316195423570985008687907853269984665640564039457584007913129639936); +insert into numeric_table values(105, 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555); +insert into numeric_table values(106, 'NaN'::numeric); +insert into numeric_table values(107, 'Infinity'::numeric); diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java index 6dbb52b287488..d91e5f885d609 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java @@ -540,7 +540,11 @@ private boolean isDataTypeCompatible(String pgDataType, Data.DataType.TypeName t return val == Data.DataType.TypeName.DOUBLE_VALUE; case "decimal": case "numeric": - return val == Data.DataType.TypeName.DECIMAL_VALUE; + return val == Data.DataType.TypeName.DECIMAL_VALUE + // We allow user to map numeric into rw_int256 or varchar to avoid precision + // loss in the conversion from pg-numeric to rw-numeric + || val == Data.DataType.TypeName.INT256_VALUE + || val == Data.DataType.TypeName.VARCHAR_VALUE; case "varchar": case "character varying": return val == Data.DataType.TypeName.VARCHAR_VALUE; diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 18c70436cbca8..6196f85877bd4 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -80,6 +80,7 @@ opendal = "0.45" openssl = "0.10" parking_lot = { workspace = true } paste = "1" +pg_bigdecimal = { git = "https://github.com/risingwavelabs/rust-pg_bigdecimal", rev = "0b7893d88894ca082b4525f94f812da034486f7c" } postgres-openssl = "0.5.0" prometheus = { version = "0.13", features = ["process"] } prost = { version = "0.12", features = ["no-recursion-limit"] } diff --git a/src/connector/src/parser/postgres.rs b/src/connector/src/parser/postgres.rs index f2c541423eb71..fa6c9a80952fb 100644 --- a/src/connector/src/parser/postgres.rs +++ b/src/connector/src/parser/postgres.rs @@ -12,14 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::str::FromStr; use std::sync::LazyLock; use chrono::{NaiveDate, Utc}; +use pg_bigdecimal::PgNumeric; use risingwave_common::catalog::Schema; use risingwave_common::log::LogSuppresser; use risingwave_common::row::OwnedRow; use risingwave_common::types::{ - DataType, Date, Decimal, Interval, JsonbVal, ListValue, ScalarImpl, Time, Timestamp, + DataType, Date, Decimal, Int256, Interval, JsonbVal, ListValue, ScalarImpl, Time, Timestamp, Timestamptz, }; use rust_decimal::Decimal as RustDecimal; @@ -139,10 +141,31 @@ pub fn postgres_row_to_owned_row(row: tokio_postgres::Row, schema: &Schema) -> O DataType::Decimal => { handle_data_type!(row, i, name, RustDecimal, Decimal) } + DataType::Int256 => { + // Currently in order to handle the decimal beyond RustDecimal, + // we use the PgNumeric type to convert the decimal to a string. + // Then we convert the string to Int256. + // Note: It's only used to map the numeric type in upstream Postgres to RisingWave's rw_int256. + let res = row.try_get::<_, Option>(i); + match res { + Ok(val) => pg_numeric_to_rw_int256(val), + Err(err) => { + if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { + tracing::error!( + column = name, + error = %err.as_report(), + suppressed_count, + "parse numeric column as pg_numeric failed", + ); + } + None + } + } + } DataType::Varchar => { - match row.columns()[i].type_() { + match *row.columns()[i].type_() { // Since we don't support UUID natively, adapt it to a VARCHAR column - &Type::UUID => { + Type::UUID => { let res = row.try_get::<_, Option>(i); match res { Ok(val) => val.map(|v| ScalarImpl::from(v.to_string())), @@ -159,6 +182,27 @@ pub fn postgres_row_to_owned_row(row: tokio_postgres::Row, schema: &Schema) -> O } } } + // we support converting NUMERIC to VARCHAR implicitly + Type::NUMERIC => { + // Currently in order to handle the decimal beyond RustDecimal, + // we use the PgNumeric type to convert the decimal to a string. + // Note: It's only used to map the numeric type in upstream Postgres to RisingWave's varchar. + let res = row.try_get::<_, Option>(i); + match res { + Ok(val) => pg_numeric_to_varchar(val), + Err(err) => { + if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { + tracing::error!( + column = name, + error = %err.as_report(), + suppressed_count, + "parse numeric column as pg_numeric failed", + ); + } + None + } + } + } _ => { handle_data_type!(row, i, name, String) } @@ -227,7 +271,58 @@ pub fn postgres_row_to_owned_row(row: tokio_postgres::Row, schema: &Schema) -> O handle_list_data_type!(row, i, name, NaiveDate, builder, Date); } DataType::Varchar => { - handle_list_data_type!(row, i, name, String, builder); + match *row.columns()[i].type_() { + // Since we don't support UUID natively, adapt it to a VARCHAR column + Type::UUID_ARRAY => { + let res = row.try_get::<_, Option>>(i); + match res { + Ok(val) => { + if let Some(v) = val { + v.into_iter().for_each(|val| { + builder.append(Some(ScalarImpl::from( + val.to_string(), + ))) + }); + } + } + Err(err) => { + if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { + tracing::error!( + suppressed_count, + column = name, + error = %err.as_report(), + "parse uuid column failed", + ); + } + } + }; + } + Type::NUMERIC_ARRAY => { + let res = row.try_get::<_, Option>>(i); + match res { + Ok(val) => { + if let Some(v) = val { + v.into_iter().for_each(|val| { + builder.append(pg_numeric_to_varchar(Some(val))) + }); + } + } + Err(err) => { + if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { + tracing::error!( + suppressed_count, + column = name, + error = %err.as_report(), + "parse numeric list column as pg_numeric list failed", + ); + } + } + }; + } + _ => { + handle_list_data_type!(row, i, name, String, builder); + } + } } DataType::Time => { handle_list_data_type!(row, i, name, chrono::NaiveTime, builder, Time); @@ -289,10 +384,29 @@ pub fn postgres_row_to_owned_row(row: tokio_postgres::Row, schema: &Schema) -> O } } } - DataType::Struct(_) - | DataType::List(_) - | DataType::Serial - | DataType::Int256 => { + DataType::Int256 => { + let res = row.try_get::<_, Option>>(i); + match res { + Ok(val) => { + if let Some(v) = val { + v.into_iter().for_each(|val| { + builder.append(pg_numeric_to_rw_int256(Some(val))) + }); + } + } + Err(err) => { + if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { + tracing::error!( + suppressed_count, + column = name, + error = %err.as_report(), + "parse numeric list column as pg_numeric list failed", + ); + } + } + }; + } + DataType::Struct(_) | DataType::List(_) | DataType::Serial => { tracing::warn!( "unsupported List data type {:?}, set the List to empty", **dtype @@ -302,8 +416,8 @@ pub fn postgres_row_to_owned_row(row: tokio_postgres::Row, schema: &Schema) -> O Some(ScalarImpl::from(ListValue::new(builder.finish()))) } - DataType::Struct(_) | DataType::Int256 | DataType::Serial => { - // Interval, Struct, List, Int256 are not supported + DataType::Struct(_) | DataType::Serial => { + // Interval, Struct, List are not supported tracing::warn!(rw_field.name, ?rw_field.data_type, "unsupported data type, set to null"); None } @@ -313,3 +427,40 @@ pub fn postgres_row_to_owned_row(row: tokio_postgres::Row, schema: &Schema) -> O } OwnedRow::new(datums) } + +fn pg_numeric_to_rw_int256(val: Option) -> Option { + let string = pg_numeric_to_string(val)?; + match Int256::from_str(string.as_str()) { + Ok(num) => Some(ScalarImpl::from(num)), + Err(err) => { + if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { + tracing::error!( + error = %err.as_report(), + suppressed_count, + "parse numeric string as rw_int256 failed", + ); + } + None + } + } +} + +fn pg_numeric_to_varchar(val: Option) -> Option { + pg_numeric_to_string(val).map(ScalarImpl::from) +} + +fn pg_numeric_to_string(val: Option) -> Option { + if let Some(pg_numeric) = val { + // TODO(kexiang): NEGATIVE_INFINITY -> -Infinity, POSITIVE_INFINITY -> Infinity, NAN -> NaN + // The current implementation is to ensure consistency with the behavior of cdc event parsor. + match pg_numeric { + PgNumeric::NegativeInf => Some(String::from("NEGATIVE_INFINITY")), + PgNumeric::Normalized(big_decimal) => Some(big_decimal.to_string()), + PgNumeric::PositiveInf => Some(String::from("POSITIVE_INFINITY")), + PgNumeric::NaN => Some(String::from("NAN")), + } + } else { + // NULL + None + } +} From 4952e3acd7cc7c03ea173486d430b8564d298799 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 19 Apr 2024 13:06:25 +0800 Subject: [PATCH 08/22] doc: document different RisingWave keys (#16149) --- docs/keys.md | 65 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 65 insertions(+) create mode 100644 docs/keys.md diff --git a/docs/keys.md b/docs/keys.md new file mode 100644 index 0000000000000..b0baa40c8716d --- /dev/null +++ b/docs/keys.md @@ -0,0 +1,65 @@ +# Keys + +Document the different Keys in RisingWave. + +## Stream Key + +The key which can identify records in the RisingWave stream. + +For example, given the following stream chunk, where stream key is `k1, k2`: +```text +| op | k1 | k2 | v1 | v2 | +|----|----|----|----|----| +| - | 1 | 2 | 1 | 1 | +| + | 1 | 2 | 3 | 4 | +| + | 0 | 1 | 2 | 3 | +``` + +We can tell that the record corresponding to the key `(1, 2)` +has been updated from `(1, 2, 1, 1)` to `(1, 2, 3, 4)`. + +The record corresponding to key `(0, 1)` has been inserted with `(0, 1, 2, 3)`. + +It may not be the minimal set of columns required to identify a record, +for instance `group key` could be part of the stream key, to specify the distribution of records. + +## Primary Key (Storage) + +This discusses the internal primary key (pk) which we often see in streaming operators. +It is different from the primary key in SQL. + +A more appropriate name for this would be `Storage Primary Key`. + +Besides uniquely identifying a record in storage, this key may also be used +to provide ordering properties. + +Let's use the following query as an example: + +```sql +create table t1(id bigint primary key, i bigint); +create materialized view mv1 as select id, i from t1 order by i, id; +``` + +`mv1` has the following plan: +```text + StreamMaterialize { + columns: [id, i], + stream_key: [id], + pk_columns: [i, id], -- notice the pk_columns + pk_conflict: NoCheck + } + └─StreamTableScan { table: t1, columns: [id, i] } +``` + +You can see that the `pk_columns` are `[i, id]`, although the upstream SQL primary key is just `id`. +In the storage layer, key-value pairs are sorted by their keys. + +Because the materialized view contains an `order by i, id`, +the storage primary key is `[i, id]` to ensure they are ordered in storage. +Importantly, `i` will be a prefix of the key. + +Then when iterating over the keys from storage, the records are returned in the correct order per partition. + +When the update stream comes, we can just use `id` to identify the records that need to be updated. +We can get the whole record corresponding to the `id` and get the `i` from there. +Then we can use that to update the materialized state accordingly. \ No newline at end of file From 172662fd3aba01be8de667aff66d6891c4156e49 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 19 Apr 2024 13:07:12 +0800 Subject: [PATCH 09/22] fix(cmd_all): immediately exit if `idle_exit_secs` is set, and meta node has stopped (#16369) --- src/cmd_all/src/standalone.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index 734460f20f6b0..beaf46bafc498 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -192,8 +192,16 @@ pub async fn standalone( tracing::info!("starting meta-node thread with cli args: {:?}", opts); let _meta_handle = tokio::spawn(async move { + let dangerous_max_idle_secs = opts.dangerous_max_idle_secs; risingwave_meta_node::start(opts).await; tracing::warn!("meta is stopped, shutdown all nodes"); + if let Some(idle_exit_secs) = dangerous_max_idle_secs { + eprintln!("{}", + console::style(format_args!( + "RisingWave playground exited after being idle for {idle_exit_secs} seconds. Bye!" + )).bold()); + std::process::exit(0); + } }); // wait for the service to be ready tokio::time::sleep(std::time::Duration::from_secs(1)).await; From 66ac1351cffb66ccaa72934874d33ec816b82aef Mon Sep 17 00:00:00 2001 From: Zihao Xu Date: Fri, 19 Apr 2024 01:14:23 -0400 Subject: [PATCH 10/22] feat(snowflake-sink): change to streaming upload instead of batched bulk load (#16269) --- Cargo.lock | 1 + src/connector/Cargo.toml | 1 + src/connector/src/sink/snowflake.rs | 183 ++++++++++++------ src/connector/src/sink/snowflake_connector.rs | 78 +++----- src/connector/with_options_sink.yaml | 6 +- .../opendal_engine/opendal_object_store.rs | 2 + .../src/object/opendal_engine/opendal_s3.rs | 44 +++++ 7 files changed, 198 insertions(+), 117 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4d2f9864883d2..bf24db465e5f1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10425,6 +10425,7 @@ dependencies = [ "risingwave_common", "risingwave_common_estimate_size", "risingwave_jni_core", + "risingwave_object_store", "risingwave_pb", "risingwave_rpc_client", "rumqttc", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 6196f85877bd4..db0064d9b8bfd 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -109,6 +109,7 @@ reqwest = { version = "0.12.2", features = ["json", "stream"] } risingwave_common = { workspace = true } risingwave_common_estimate_size = { workspace = true } risingwave_jni_core = { workspace = true } +risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } rumqttc = { version = "0.24.0", features = ["url"] } diff --git a/src/connector/src/sink/snowflake.rs b/src/connector/src/sink/snowflake.rs index f4901b025effc..e4dbbfa59f17b 100644 --- a/src/connector/src/sink/snowflake.rs +++ b/src/connector/src/sink/snowflake.rs @@ -13,13 +13,16 @@ // limitations under the License. use std::collections::HashMap; +use std::fmt::Write; use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; +use bytes::{Bytes, BytesMut}; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; +use risingwave_object_store::object::{ObjectStore, StreamingUploader}; use serde::Deserialize; use serde_json::Value; use serde_with::serde_as; @@ -29,13 +32,14 @@ use with_options::WithOptions; use super::encoder::{ JsonEncoder, RowEncoder, TimeHandlingMode, TimestampHandlingMode, TimestamptzHandlingMode, }; -use super::snowflake_connector::{SnowflakeHttpClient, SnowflakeS3Client}; +use super::snowflake_connector::{generate_s3_file_name, SnowflakeHttpClient, SnowflakeS3Client}; use super::writer::LogSinkerOf; use super::{SinkError, SinkParam}; use crate::sink::writer::SinkWriterExt; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; pub const SNOWFLAKE_SINK: &str = "snowflake"; +const INITIAL_ROW_CAPACITY: usize = 1024; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct SnowflakeCommon { @@ -77,9 +81,9 @@ pub struct SnowflakeCommon { pub s3_bucket: String, /// The optional s3 path to be specified - /// the actual file location would be `:///` + /// the actual file location would be `s3:////` /// if this field is specified by user(s) - /// otherwise it would be `://` + /// otherwise it would be `s3:///` #[serde(rename = "snowflake.s3_path")] pub s3_path: Option, @@ -94,11 +98,6 @@ pub struct SnowflakeCommon { /// The s3 region, e.g., us-east-2 #[serde(rename = "snowflake.aws_region")] pub aws_region: String, - - /// The configurable max row(s) to batch, - /// which should be *explicitly* specified by user(s) - #[serde(rename = "snowflake.max_batch_row_num")] - pub max_batch_row_num: String, } #[serde_as] @@ -137,8 +136,7 @@ impl Sink for SnowflakeSink { self.schema.clone(), self.pk_indices.clone(), self.is_append_only, - ) - .await + )? .into_log_sinker(writer_param.sink_metrics)) } @@ -177,22 +175,25 @@ pub struct SnowflakeSinkWriter { /// the client to insert file to external storage (i.e., s3) s3_client: SnowflakeS3Client, row_encoder: JsonEncoder, - row_counter: u32, - payload: String, - /// the threshold for sinking to s3 - max_batch_row_num: u32, /// The current epoch, used in naming the sink files /// mainly used for debugging purpose epoch: u64, + /// streaming uploader to upload data to the intermediate (s3) storage. + /// this also contains the file suffix *unique* to the *local* sink writer per epoch. + /// i.e., opendal s3 engine and the file suffix for intermediate s3 file. + /// note: the option here *implicitly* indicates whether we have at + /// least call `streaming_upload` once during this epoch, + /// which is mainly used to prevent uploading empty data. + streaming_uploader: Option<(Box, String)>, } impl SnowflakeSinkWriter { - pub async fn new( + pub fn new( config: SnowflakeConfig, schema: Schema, pk_indices: Vec, is_append_only: bool, - ) -> Self { + ) -> Result { let http_client = SnowflakeHttpClient::new( config.common.account_identifier.clone(), config.common.user.clone(), @@ -211,17 +212,9 @@ impl SnowflakeSinkWriter { config.common.aws_access_key_id.clone(), config.common.aws_secret_access_key.clone(), config.common.aws_region.clone(), - ) - .await; - - let max_batch_row_num = config - .common - .max_batch_row_num - .clone() - .parse::() - .expect("failed to parse `snowflake.max_batch_row_num` as a `u32`"); + )?; - Self { + Ok(Self { config, schema: schema.clone(), pk_indices, @@ -236,32 +229,102 @@ impl SnowflakeSinkWriter { TimestamptzHandlingMode::UtcString, TimeHandlingMode::String, ), - row_counter: 0, - payload: String::new(), - max_batch_row_num, - // initial value of `epoch` will start from 0 + // initial value of `epoch` will be set to 0 epoch: 0, - } + // will be (lazily) initialized after the begin of each epoch + // when some data is ready to be upload + streaming_uploader: None, + }) } - /// reset the `payload` and `row_counter`. - /// shall *only* be called after a successful sink to s3. - fn reset(&mut self) { - self.payload.clear(); - self.row_counter = 0; + /// return a brand new the streaming uploader as well as the file suffix. + /// note: should *only* be called iff after a new epoch begins, + /// and `streaming_upload` being called the first time. + /// i.e., lazily initialization of the internal `streaming_uploader`. + /// plus, this function is *pure*, the `&mut self` here is to make rustc (and tokio) happy. + async fn new_streaming_uploader(&mut self) -> Result<(Box, String)> { + let file_suffix = self.file_suffix(); + let path = generate_s3_file_name(self.s3_client.s3_path(), &file_suffix); + let uploader = self + .s3_client + .opendal_s3_engine + .streaming_upload(&path) + .await + .map_err(|err| { + SinkError::Snowflake(format!( + "failed to create the streaming uploader of opendal s3 engine for epoch {}, error: {}", + self.epoch, + err + )) + })?; + Ok((uploader, file_suffix)) + } + + /// write data to the current streaming uploader for this epoch. + async fn streaming_upload(&mut self, data: Bytes) -> Result<()> { + let (uploader, _) = match self.streaming_uploader.as_mut() { + Some(s) => s, + None => { + assert!( + self.streaming_uploader.is_none(), + "expect `streaming_uploader` to be None" + ); + let uploader = self.new_streaming_uploader().await?; + self.streaming_uploader.insert(uploader) + } + }; + uploader + .write_bytes(data) + .await + .map_err(|err| { + SinkError::Snowflake(format!( + "failed to write bytes when streaming uploading to s3 for snowflake sink, error: {}", + err + )) + })?; + Ok(()) } - fn at_sink_threshold(&self) -> bool { - self.row_counter >= self.max_batch_row_num + /// finalize streaming upload for this epoch. + /// ensure all the data has been properly uploaded to intermediate s3. + async fn finish_streaming_upload(&mut self) -> Result> { + let uploader = std::mem::take(&mut self.streaming_uploader); + let Some((uploader, file_suffix)) = uploader else { + // there is no data to be uploaded for this epoch + return Ok(None); + }; + uploader.finish().await.map_err(|err| { + SinkError::Snowflake(format!( + "failed to finish streaming upload to s3 for snowflake sink, error: {}", + err + )) + })?; + Ok(Some(file_suffix)) } - fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + let mut chunk_buf = BytesMut::new(); + + // write the json representations of the row(s) in current chunk to `chunk_buf` for (op, row) in chunk.rows() { assert_eq!(op, Op::Insert, "expect all `op(s)` to be `Op::Insert`"); - let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); - self.payload.push_str(&row_json_string); - self.row_counter += 1; + // to prevent temporary string allocation, + // so we directly write to `chunk_buf` implicitly via `write_fmt`. + write!( + chunk_buf, + "{}", + Value::Object(self.row_encoder.encode(row)?) + ) + .map_err(|err| { + SinkError::Snowflake(format!( + "failed to write json object to `row_buf`, error: {}", + err + )) + })?; } + + // streaming upload in a chunk-by-chunk manner + self.streaming_upload(chunk_buf.freeze()).await?; Ok(()) } @@ -284,20 +347,16 @@ impl SnowflakeSinkWriter { /// sink `payload` to s3, then trigger corresponding `insertFiles` post request /// to snowflake, to finish the overall sinking pipeline. - async fn sink_payload(&mut self) -> Result<()> { - if self.payload.is_empty() { + async fn commit(&mut self) -> Result<()> { + // note that after `finish_streaming_upload`, do *not* interact with + // `streaming_uploader` until new data comes in at next epoch, + // since the ownership has been taken in this method, and `None` will be left. + let Some(file_suffix) = self.finish_streaming_upload().await? else { + // represents there is no data to be uploaded for this epoch return Ok(()); - } - let file_suffix = self.file_suffix(); - // todo: change this to streaming upload - // first sink to the external stage provided by user (i.e., s3) - self.s3_client - .sink_to_s3(self.payload.clone().into(), file_suffix.clone()) - .await?; - // then trigger `insertFiles` post request to snowflake - self.http_client.send_request(file_suffix).await?; - // reset `payload` & `row_counter` - self.reset(); + }; + // trigger `insertFiles` post request to snowflake + self.http_client.send_request(&file_suffix).await?; Ok(()) } } @@ -319,20 +378,16 @@ impl SinkWriter for SnowflakeSinkWriter { async fn barrier(&mut self, is_checkpoint: bool) -> Result { if is_checkpoint { - // sink all the row(s) currently batched in `self.payload` - self.sink_payload().await?; + // finalize current streaming upload, plus notify snowflake to sink + // the corresponding data to snowflake pipe. + // note: if no data needs to be committed, then `commit` is simply a no-op. + self.commit().await?; } Ok(()) } async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - self.append_only(chunk)?; - - // When the number of row exceeds `MAX_BATCH_ROW_NUM` - if self.at_sink_threshold() { - self.sink_payload().await?; - } - + self.append_only(chunk).await?; Ok(()) } } diff --git a/src/connector/src/sink/snowflake_connector.rs b/src/connector/src/sink/snowflake_connector.rs index 30c74045441a2..432d222a8b426 100644 --- a/src/connector/src/sink/snowflake_connector.rs +++ b/src/connector/src/sink/snowflake_connector.rs @@ -15,15 +15,10 @@ use std::collections::HashMap; use std::time::{SystemTime, UNIX_EPOCH}; -use aws_config; -use aws_config::meta::region::RegionProviderChain; -use aws_sdk_s3::config::Credentials; -use aws_sdk_s3::primitives::ByteStream; -use aws_sdk_s3::Client as S3Client; -use aws_types::region::Region; -use bytes::Bytes; use jsonwebtoken::{encode, Algorithm, EncodingKey, Header}; use reqwest::{header, Client, RequestBuilder, StatusCode}; +use risingwave_common::config::ObjectStoreConfig; +use risingwave_object_store::object::*; use serde::{Deserialize, Serialize}; use thiserror_ext::AsReport; @@ -35,7 +30,7 @@ const SNOWFLAKE_REQUEST_ID: &str = "RW_SNOWFLAKE_SINK"; const S3_INTERMEDIATE_FILE_NAME: &str = "RW_SNOWFLAKE_S3_SINK_FILE"; /// The helper function to generate the *global unique* s3 file name. -fn generate_s3_file_name(s3_path: Option, suffix: String) -> String { +pub(crate) fn generate_s3_file_name(s3_path: Option<&str>, suffix: &str) -> String { match s3_path { Some(path) => format!("{}/{}_{}", path, S3_INTERMEDIATE_FILE_NAME, suffix), None => format!("{}_{}", S3_INTERMEDIATE_FILE_NAME, suffix), @@ -155,7 +150,7 @@ impl SnowflakeHttpClient { /// NOTE: this function should ONLY be called *after* /// uploading files to remote external staged storage, i.e., AWS S3 - pub async fn send_request(&self, file_suffix: String) -> Result<()> { + pub async fn send_request(&self, file_suffix: &str) -> Result<()> { let builder = self.build_request_and_client(); // Generate the jwt_token @@ -167,7 +162,7 @@ impl SnowflakeHttpClient { "X-Snowflake-Authorization-Token-Type".to_string(), "KEYPAIR_JWT", ) - .body(generate_s3_file_name(self.s3_path.clone(), file_suffix)); + .body(generate_s3_file_name(self.s3_path.as_deref(), file_suffix)); let response = builder .send() @@ -190,56 +185,43 @@ impl SnowflakeHttpClient { pub struct SnowflakeS3Client { s3_bucket: String, s3_path: Option, - s3_client: S3Client, + pub opendal_s3_engine: OpendalObjectStore, } impl SnowflakeS3Client { - pub async fn new( + pub fn new( s3_bucket: String, s3_path: Option, aws_access_key_id: String, aws_secret_access_key: String, aws_region: String, - ) -> Self { - let credentials = Credentials::new( - aws_access_key_id, - aws_secret_access_key, - // we don't allow temporary credentials - None, - None, - "rw_sink_to_s3_credentials", - ); - - let region = RegionProviderChain::first_try(Region::new(aws_region)).or_default_provider(); - - let config = aws_config::from_env() - .credentials_provider(credentials) - .region(region) - .load() - .await; - - // create the brand new s3 client used to sink files to s3 - let s3_client = S3Client::new(&config); + ) -> Result { + // just use default configuration here for opendal s3 engine + let config = ObjectStoreConfig::default(); + + // create the s3 engine for streaming upload to the intermediate s3 bucket + let opendal_s3_engine = OpendalObjectStore::new_s3_engine_with_credentials( + &s3_bucket, + config, + &aws_access_key_id, + &aws_secret_access_key, + &aws_region, + ) + .map_err(|err| { + SinkError::Snowflake(format!( + "failed to create opendal s3 engine, error: {}", + err + )) + })?; - Self { + Ok(Self { s3_bucket, s3_path, - s3_client, - } + opendal_s3_engine, + }) } - pub async fn sink_to_s3(&self, data: Bytes, file_suffix: String) -> Result<()> { - self.s3_client - .put_object() - .bucket(self.s3_bucket.clone()) - .key(generate_s3_file_name(self.s3_path.clone(), file_suffix)) - .body(ByteStream::from(data)) - .send() - .await - .map_err(|err| { - SinkError::Snowflake(format!("failed to sink data to S3, error: {}", err)) - })?; - - Ok(()) + pub fn s3_path(&self) -> Option<&str> { + self.s3_path.as_deref() } } diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 07da6a36a0e3a..27daa718b64f9 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -564,7 +564,7 @@ SnowflakeConfig: required: true - name: snowflake.s3_path field_type: String - comments: The optional s3 path to be specified the actual file location would be `:///` if this field is specified by user(s) otherwise it would be `://` + comments: The optional s3 path to be specified the actual file location would be `s3:////` if this field is specified by user(s) otherwise it would be `s3:///` required: false - name: snowflake.aws_access_key_id field_type: String @@ -578,10 +578,6 @@ SnowflakeConfig: field_type: String comments: The s3 region, e.g., us-east-2 required: true - - name: snowflake.max_batch_row_num - field_type: String - comments: The configurable max row(s) to batch, which should be *explicitly* specified by user(s) - required: true StarrocksConfig: fields: - name: starrocks.host diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index 2aa4bd458806e..d7ba829dda632 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -33,6 +33,7 @@ pub struct OpendalObjectStore { pub(crate) op: Operator, pub(crate) engine_type: EngineType, } + #[derive(Clone)] pub enum EngineType { Memory, @@ -219,6 +220,7 @@ impl ObjectStore for OpendalObjectStore { pub struct OpendalStreamingUploader { writer: Writer, } + impl OpendalStreamingUploader { pub async fn new(op: Operator, path: String) -> ObjectResult { let writer = op diff --git a/src/object_store/src/object/opendal_engine/opendal_s3.rs b/src/object_store/src/object/opendal_engine/opendal_s3.rs index 28f90a48e9ae0..c83498305d8e5 100644 --- a/src/object_store/src/object/opendal_engine/opendal_s3.rs +++ b/src/object_store/src/object/opendal_engine/opendal_s3.rs @@ -59,6 +59,7 @@ impl OpendalObjectStore { .with_jitter(), ) .finish(); + Ok(Self { op, engine_type: EngineType::S3, @@ -128,4 +129,47 @@ impl OpendalObjectStore { Ok(HttpClient::build(client_builder)?) } + + /// currently used by snowflake sink, + /// especially when sinking to the intermediate s3 bucket. + pub fn new_s3_engine_with_credentials( + bucket: &str, + object_store_config: ObjectStoreConfig, + aws_access_key_id: &str, + aws_secret_access_key: &str, + aws_region: &str, + ) -> ObjectResult { + // Create s3 builder with credentials. + let mut builder = S3::default(); + + // set credentials for s3 sink + builder.bucket(bucket); + builder.access_key_id(aws_access_key_id); + builder.secret_access_key(aws_secret_access_key); + builder.region(aws_region); + + let http_client = Self::new_http_client(&object_store_config)?; + builder.http_client(http_client); + + let op: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer( + RetryLayer::new() + .with_min_delay(Duration::from_millis( + object_store_config.s3.object_store_req_retry_interval_ms, + )) + .with_max_delay(Duration::from_millis( + object_store_config.s3.object_store_req_retry_max_delay_ms, + )) + .with_max_times(object_store_config.s3.object_store_req_retry_max_attempts) + .with_factor(1.1) + .with_jitter(), + ) + .finish(); + + Ok(Self { + op, + engine_type: EngineType::S3, + }) + } } From bda3b4548558484942ded604c8b29a8d205f97e7 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 19 Apr 2024 13:39:11 +0800 Subject: [PATCH 11/22] chore: disable hyper hyper::client::connect::http by default (#16396) Signed-off-by: xxchan --- Makefile.toml | 4 ++-- src/utils/runtime/src/logger.rs | 4 +--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/Makefile.toml b/Makefile.toml index 8b1a88707bb70..6db1dbb604b12 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -90,9 +90,9 @@ if ${is_not_ci} no_rust_log = not ${rust_log} if ${no_rust_log} - set_env RUST_LOG "pgwire_query_log=info,hyper::client::connect::http=info" + set_env RUST_LOG "pgwire_query_log=info" else - set_env RUST_LOG "pgwire_query_log=info,hyper::client::connect::http=info,${rust_log}" + set_env RUST_LOG "pgwire_query_log=info,${rust_log}" end end diff --git a/src/utils/runtime/src/logger.rs b/src/utils/runtime/src/logger.rs index 7dd834afbcbdc..7cc4194600b00 100644 --- a/src/utils/runtime/src/logger.rs +++ b/src/utils/runtime/src/logger.rs @@ -216,9 +216,7 @@ pub fn init_risingwave_logger(settings: LoggerSettings) { .with_target("sled", Level::INFO) .with_target("cranelift", Level::INFO) .with_target("wasmtime", Level::INFO) - .with_target("sqlx", Level::WARN) - // Expose hyper connection socket addr log. - .with_target("hyper::client::connect::http", Level::DEBUG); + .with_target("sqlx", Level::WARN); // For all other crates, apply default level depending on the deployment and `debug_assertions` flag. let default_level = match deployment { From db0208307905da21fe1c90cb6c8ca734c4cfc852 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 19 Apr 2024 13:55:08 +0800 Subject: [PATCH 12/22] doc: add doc on aggregations (#16144) --- docs/aggregation.md | 49 +++++++++++++++++++++ docs/images/aggregation/agg-components.png | Bin 0 -> 503060 bytes docs/images/aggregation/init-agg-group.png | Bin 0 -> 158795 bytes 3 files changed, 49 insertions(+) create mode 100644 docs/aggregation.md create mode 100644 docs/images/aggregation/agg-components.png create mode 100644 docs/images/aggregation/init-agg-group.png diff --git a/docs/aggregation.md b/docs/aggregation.md new file mode 100644 index 0000000000000..05d50c17967e2 --- /dev/null +++ b/docs/aggregation.md @@ -0,0 +1,49 @@ +# Aggregations + +We will cover internal implementation of common aggregations in this document. + + +## Frontend + +TODO + +## Expression Framework + +TODO + +## HashAggExecutor + +![aggregation components](./images/aggregation/agg-components.png) + +Within the `HashAggExecutor`, there are 4 main components: +1. AggCalls. +2. AggState. +3. AggGroups. +4. Persisted State. + +AggCalls are the aggregation calls for the query. For instance `SUM(v1)`, `COUNT(v2)` has the AggCalls `SUM` and `COUNT`. + +AggState is the state we use to compute to the result (output) of the aggregation call. +Within each aggregation group, it will have an AggState for each AggCall. + +AggGroups are created per aggregation group. +For instance with `GROUP BY x1, x2`, there will be a group for each unique combination of `x1` and `x2`. + +Whenever stream chunks come in, the executor will update the aggregation state for each group, per agg call. + +On barrier, we will persist the in-memory states. +For `value` type aggregations, we will persist the state to the intermediate state table. +This state table will store all value aggregations per group on a single row. + +For `MaterializedInput` type aggregations, these require tracking input state. For example, non-append-only min/max. +For each of these aggregations, they have 1 state table (`AggStateStorage::MaterializedInput`) each. Within the state table, it will store the input state for each group. + +### Initialization of `AggGroups` + +![init-agg-group](./images/aggregation/init-agg-group.png) + +AggGroups are initialized when corresponding aggregation groups are not found in `AggGroupCache`. +This could be either because the `AggGroupCache` got evicted, +or its a new group key. + +It could take a while to initialize agg groups, hence we cache them in `AggGroupCache`. \ No newline at end of file diff --git a/docs/images/aggregation/agg-components.png b/docs/images/aggregation/agg-components.png new file mode 100644 index 0000000000000000000000000000000000000000..a41fec9fe884f62c9318e5e509bd10193247cb1f GIT binary patch literal 503060 zcmeGDRa70@7B-5a!7X@jNP;`THMqOGyF+kKfZ#4c6WoKl2iM^4?(TAHCTs1TyVu{& zzaG!SnXOG`QB`BqkS<^UdauuNGNMTEui(MJz>vhngcQKQV3EPVASYm70B6X%clp7< z5c$mo1?9vA1&QSxY)#CqjKRRfKF6!Vs44bgrEA7S@tgU{Wq(|PLZbxF{x}LDM(-aU z1qJ)2n_sS+EEK*}n?_GiRTzVd7z;z5h9;z$9t$JRxm-_mDciU7zW;jodZ_vNaeL5b zIF+jw5$qLty@5S)Asl}xh3-qdmu)yHsU5`5kWef9C`sU)iHYPU!lI&JW&zu+^No0; zcyr34$y0py&qd~^&~DsdQD(v8H*QT@!6@Lqs%Jl9Ji&sQmJBRf)B^P(IbS9b|3ENi z7xbexW|#K6?(RnA_B!or!XYCM<|REeQ|+rDIVm>bO`aL5JyQqY&k-9lCb)- ztHq(?e%%-!`_zsfyb3Rn2Y1eKin=e@Bo~E^k#{jmlZ;;!_Gv0da)#!OPXu*zPkfg< z@sNxj<)yAEJiCOKR5JrJFPs7Vj|AwBA7a6iBp><*A`n#X!srH#M3L{rvdb$TXh|Y4 zgWp47+^GpPyk=L~)%}vf#;$ASm>Y#&@B`+Id0i0mvR%Vf%k9aIZpPRzIG!P&tQQZD zksZ5*m_dxurC5_#B3NZY)$6)Pq__a90(;uF7#F%%k z4eWfJI??1hNnJ>xDjNaxkZ8V05a5tv`jU{b0$8Xu*HhOW)WWh1D-dwKU=S<$MlVEP z_jw4N;K0DQXQ0CIFOAlLxs`!o@IURN7O`{sJGt5Vc?z+*!rRlq!f>2uP!YX(6_Nm( z&M$HkxuZ4X%CL+-WAyl(PGr=|Ow%hwfxD|G6xf|f&KQg%hWO?M`k|fR0`_$`*~(8M zR_xTz9>~M%jeM(=Ujt5p?b1fY>v+V~Uf)sRuV-?j14m^!3iu75RAAheo;LjN1W@3JQ)I`jl_Aevh%MnKMsxS!ugj z`I%?M*N79M(-+6Z#W8Nf?}mAXbuqkd{mm;}SoCDEWD{YTna*)(ZiG=8*h$mJ?E*t9 zL#e?kjziZjFW-thE%NVJFW2AtfUS(PFfiC%aA8kDwRxl5+z{Ok&oZs%31a#>in=O@ z`}ubo!$5dSnzfM+fhB<<=pk&cLoE2QoxlhXBMSMsMj^iKIAC7oO!O_#dz%C!!H=u< zLM7l*4}AvygFnv-SUyZo8>JnV7F1%pvmJJ6I^7Axg`dj`vp@0ck0GO|kUucW-{YfV zfBbwyDj){+GW0cBf!GVu_u0KtM!X7v#r#ubno$-;d=5|?pHf9Rqvm@BvtiuAJ>N4b zAx#Q|%Ue1@^$V$+vA>Fx;b$JDG$YXnqZ63^ro3ZQh0OCwN2n|-Z3m%>2r`LEq-T&p zH0*PEIRy>0WY6mvt6~i5p2F`M#b`_0MQY?zp7(^1> zZ$gJaO;$}FOwL%WPi9+4PJGBu zQ)(=^3Y5svP<$+)obcRxY4Proq%Ne_9`ewh+0p(xQR>C$Md8n$${jIAoE4yd}F z>zdQ7mbK)|)n`F?mC~IxlxLll_SJb8)!E94ol~9DnA5{)&iT%%+F9Ef-8pYpWruM3 zFx$|{k~1h>P*;MFzg6Hl+qAf641I@otZ1_F+qI5QQcG(Kf2)#D*^~35>z1u955oJvwnZ3j*$8ooK!q|$PX8iPB!sN(2vedG0MKr_b!tTPQ!vw>+ zy7$9m@vgs&NJGVC(uB;S33msU+?P3AS-VNwgj@754O$Ql{!rJ0bjsOc(R zJgBS>ck_3PzSTLu$HG9!6>jLF3N;Dkl~f#HOkk$9q&ZfLE=f{JRqRP(cTbp~T@VrHQ-XYOuRro5!0*+jI;aL#z%`g@~Q743}t^+^80>MXQ zJbcbT^gs_}Y^1VR-sAE^(VJ>3YU?+npN}wBl{S=)FC0E@2FIZdh*m}1Y#nT+t~U+c z9*G@B>yw3wutsDBH7q$~)rZ?_u1pD-*JhC2T+B8%)kG_*_(xuWiFC&>~x~%^=y7d#^vQOhPa{>W%Xm!dZQyT~WOFQ-z zp7~IoP%kD_1VlI*BrB9LWC>Ueq6^{~Zg#S|w{`SCXpKaA`{dr6x$c~err;YQM=B2O zBAmmfU{1X(#UjLR;eC+*==JgSH+w|Rw^oDwjMt{?K%wcz$QX%WqGpkxAtfG${|l??`hz?cAwprjVYdHC>oab z?#iG-zk;EjmsokD>q2M|1RHS(el!=F^+k;o=M8Fp6*m z;suQgt#(;;$_q~{G8)DjM3F<9ogVGi+huFlu%1xVP$OerW2)IMrZ+~#gFAf}12iKC zm=&M)Kb2LyuVgiD9*O8*isp%mjjLo+eZ%(dI#^6Ij@BaCipgzm1Gzk)PH2In%d&ru z(ksTU?~41i?Ry=;7UYL$+t@h?pJ-m16WV-=8-{oGd+p{L<3i(ex%0BRERX8?cB{6` zC%8?xL(D@a!zs6RSe8=>v%LpZvqO$FTNop|U+|b*wz>P84%}L?Ij~D8+`0J7y$#Fl zPZ!pDT0cH0D1Or+rL3=S^Jvfgbn||-<4QOSt3>mnBu>};>tKCx)4_nj(i%c*#Dnu; z8KmS`Vimo!Zi79peX`kX_@?)OOX;YFeBIh1N85+?3d~ zQ)#%`{P$sp#R_NLDbvzc9W=S}+Od2G`OSNz2P9S2JKN$#qox9@@w}mqq1u{{Rs}^B zrBurj3mV@qcWZuFm$Wka6fN1*HCx?J6MQ2`<1%!qQSwkyJ!YD#u6j`tJ#}$F*m{4K z-o}Ge9Z{`$HFlkF2zj4oC+y;BxKFyi>56S4*KG*NP{>H&WA;*6JRI{XCa>E* zJGUQc)Ct!0Jfgi-+#l&rtf5DF+VPbgf)3*`MEf+IfOGc3AfE z*1cQH9zFW=F!AwA$J9&q^d-fuHU6*Tx~j66f633}>isl)65Aw`I> znz)IyG#C|d3=0MY{t65lI06U0yx@5MITit@0E7H>9s&$3$Q%snuX|*GU(hEC_=4K} z{)LPQ1cL$odkK79Ga>%I8x}be^6z8F3E&!-fTEzdIPj}z=wNJY<7j5<^uUBI4V-|p z6H|8t1H*a)`htrqygmW?pEg%gb5fI*;xx3irqegFH87@gwYCHG1IF#j2^?A*JLwa< zT3gvTa=P-6{FHfuT1-WL=@~gVIOrLe=$V*kfjej&-E5rnU1@C`$$oe8 zcRxbLj)o5Ac24HDHpHNQ^$l#Dop?w|K@a-R=l3{`UCsadBpb)SrUgup9(0ACk&c1> zKYasDxj|<+<;-1;t<;6gt${TIp25q(&cOZa{(oHg?-T!MsrKKNObo0{e>VN&(*HJ9 zb~JVnw6zAF>BRfr{rao%pBMjX$W0HL`X5X2d!2us1s0kYo}2zZo5l;zZl=%#>?6Lp zkgO8$3rHF01HK3Rq4@m^97F87-=R`5fr0UZi3SSMI@qb~7 zCY6rT2_UA05|Y>HM|)2KYB7+SEe)B9uPFR1S5EswG-+E;XmcH(q> zn`ar@Kj%5Q?&lk$7MjjwirF;Ku;Ar-g^{2e(5Y;s>{*WYo(1UxxB`3%wPvWypUucnW?$9GbQ^Kb(gXxBY+F z0yKD@|8MaB$G!f4+secCbY{G>Oc+e((Ja+zeifksAzPwS@+FLbOC^R}mN2&=fkvGa zom>V{Rr73nq-;(uli!b_x_xNsMHJ)^-!77<{#zcg(RF zKWigLl2W$D5$)%DqpI*Qnw`Q&0hH9v(q=+^N30RcKA9Oa3TVKeI$T zm5`o(nl04XCk`8l)QCnBR#?q4?a#wsa(cU#>bCMuY_CE+d<*9j zB9M>3=XkkAv^HB|>?njjLGPlEWMMv$AAK-eNxpEHuTfn%n99K{b;-WR91^PA;6y29 z&EFH**ysUz%IA!q{LwLQBty!|%HBn4?+>5g3j4}`egEme;P1LX9uWlhF1uZPjfh)UU#Lpv78=J)z?qSm)zpq*p9ar|UgIyL9-Mwn>!_>GdBD z=NMqpPp`kU)uJw=+ww_7xXWgu(-ebuKDe-)xYf;#SexE>6FY7VRshmbYkMO{!*O;G z{_QaFJPnUs*ho_T32rYfsVWEj^3D1=<_yT@^ z$H57fl!~Qkgp9jYX*HCYJJNEx^wT~MTDs2<`&Cx!x!zpql#a<4o+78*9h?mdXsuE! zJbr@+`j79ifkWn3OWSDa5k%ZjqE7suu1#~Py+k_CLu)EMMLO>X2$ob`WodMp*ay`G zw0416(v$1;ZG%Iji9QZGecEncW`AvR+S|-~ zh&~=CTup7)=uaafW|^O|o}S>2U0fsXJmUD6Au}vpy{CndB5siBn|z+x*MPlxcw7(m zBP=~3NN0W+6heirFb=#vmPedHv}{Yv@-_*)wnY*31oqS8Llrn~Tjv~h;BraFJpr`$ z76~=@ow=dA{can$r|VDkVBJ788oi>AWRoTMgQH1F<{g9tB~>eKlm9B zBoz39XUBi1OX3qndD%wIjG%^KpZ4!iP6sK2eg{-MQY&D+$4WG7%@&CWtRZ2dSijk? zcUvIC?*|1_!fMpo3{}0HV$jO_KRU;Mps65Oqk)8>d~)>)qr%9!fdOib7pOw>ZQy)- ze3x z=<4hglt`p2Nx0?ve+G4bBdJV&BUrsQu3Va#O4G=$%?hJF(f}BgguTfkO)HnFcrya_ zmEwFW?EgDT1aRJ+l{emqAa{v{C+QQpXxkv8zqo^H&4JER#Xz-xbF zLQR5%jq(C(hlGa5fYr>2w!?yiAq7uP{Kqo?AN%!7BJ0>f)Z*y zT+z`6ZtC{Z3g+L^3mG!*J)xoTx(ic7-?|06ADJd!Cw1wC>?5LBnzSI9wrV_DEuC5X|9Jg?;)e4m; z)tg+)%LykD00g~9-nOs!uRy);0D{%bcLDdSgMwT-3iXQYo4Y&CAs$h;m9}_l)oks? z5Aa4mzX{8*B?gRhQ?h>icPjL)8tkqo5kNagc&r#Q%6jf`?<}X{)_Wq0X#d)PaUD8^ ze>Ffry-=eeZXdjNkNG%}5Pg%AMtjB##m zZp1G4Cc~ZS{%z<(fqDud|B`*>_?SK%k1YWPg)pD%;_U3Dt?hAetEP_MzgrhTH4yy7 zQ1D@4VNrolyWah>&_b-m^VaC`?h-3}4gTMWA~5D?GSpo^NbC8UcwjVzjgntj3J&I~ zB|$)!!)2i8U(ZU91hv2q5|7m?3cww>NNQDP(Q$Ozb86cV|9&Xcpe~I2;}~-(cBV>F zoQ{`x9F7xIv;JWig#`To7v2ZpolO8|Ih?Ku*Ly!UEHpSv9xXPK%VzM#tCs2X?N66& zwmv__ve_=>8#k@-{Slt}?a-rAtI)LjlpFS_l-aNSkRJd{a=g&zm*V|#ly9u(s$wV= z-}a^RG;Z|AOEM@|rKD(Hw7M;U|C)?Y2!Hl}WFW_e3Iw5S7CB|)Q7I#nu)qG#aQ${j zPu98+)GJNG50;uWBM7+O!s57|tbjv;eH(x@;`)!D^Nocgo&g&dG3|>HhR2{F&hR|@ zJdrOi_>x@az=+n^@t1uiE`%n1V+W8L{0obV;Z~5pwIDLT#YHFMAEJYr195%4oQTV8 z(z`X3vf1$o;q2koG3D?x0awCnF;v|p$y7OoTvyq#Y;Rd6W3jjD%y|KW9?XbXoe0+e#54*X&h3=z@ViH!M)S@VwD7*YRQF!&Wk|p`@55Hrl?{}H< zXncDNv)L9+eRkX|iskZ(UWc1fxiAjo07Eb+4ay^_e-RF-ZV*dLOH8K2_|ENVTrMGi zcz)F3?tp|DoP8mYKttKm(t_qLSe*I~iCCe5NGe3hg0xwv(`TVdTes9zY{ z8eoc{QGahpg#8~s4zifD5VGk_s2R6h#^+b)4o45PZl%(pnTZ#>$+2B-xqI94 z%K-DghD*AiZEk9U@`FNCC*`)w1Te3|!;QT&N+=d>NE{118eDh7f293CT5|kyXQCjs zShrQ@>He5+ebmZkH^)9;ZLCH3M;+ed{`DV^Qv=E6E-3AA|W_3A$xp;Gy2x)AmUms=X`XXI^^ zC&Yuj0hU#v@q=|TyS+jF-dg+by<+wIEBc)1>$TWfT}7)2Ibf#-HtWC)66Jrb1vtEr zr5}J7`{-LANC8Wxd>zdqZg6U$A=BV|!=kZ_jVJqG4de%hePFOm8BA@JgdpUWCx(0Z zS}skMOf=#(>Tg4p*#<Qt3gTb-AP{%gni5ixFT*4(XM1vG2C@Za7$Qp}NfqnIld4X8yo zKhhEtaI0T1HsG70n2h^-0hlw{_2FoN(#FOH%|ZVKvJfmabZQ`TlbL?6G&ls0 z`$ZH8vws2P@W=c-7epy(N1Gq4zH!c9kNJ-Rw!nb7!%>HIHN9bX7?b0c6s6jCy(q)_-!8j@4&>60j*bfb zjrwBxK!k#hI;<&^9WfOC(YODdEqoI6)^HlNU=VWrbeXPXv08cd=bk*KZ@SLsBwBxB z5&p23IE){{2OlS8?+I?h?+9=m?Ol!*I7DzyOhD2oRK*|u>fOeV&**zhfSSdO(F>n|SJ_6j2;G#x1VnLVbdqv79wr1)js^iapio@5Jb#}Lfgt7e7TS05m5z-H!g-$L90bSj~ooi$>r_Z}i3D z?MJ8q(3}iI{T~RwF58#GEK~kbcC%67oz@wzn<7T0?>;;}?+0a(FikRg-Ij!wMGLWl z$Yqzsl+CnXGUYoCi}?KHl6)STVId1}JsY*l-i}{t%1J;H`mKmROC;QAB%P=5mY0`T zq1oLzXbBz?2EV_r|9|J%rVKSWjvhysE{b(mWf!lx?ge;CMzKt(a7Kr96W@mQ4x)y> zvs;31hl?=Wu5W`PbP_aci6m0l%X3U7{bMNfn~1L+ua^wfD;!!kdJ{wnmHUZ|e3YzR z0Tn!$@e;WrYN09lk`Rs2hc2wF=&B7;ooLe0uRZT)>n`gWRJuYijgJ=UqaCz$Kxp2# z2!8O(J4t}T%(WRA7?5mgZYFRq0wj@wocbRSP8Ai0*Ru|w#uOJr@)FGTFqo0*It({? zd;oBnGHf8Vy=62x=DU?)x{-`oqMB!kkTY*d3n|igjGn zuxRJWdx=rCIO@EjUdtwo8n_IUZ58N2TH&v2>%Q{V83av$c-&WXV0Z5f)`VXU5&>jO zlZZ!eQ*@0Fa}D=YJgyEXfnX{eu6D=+B+;lO5YPx%aX~0Vg$?xD4I87D40df_Q;Wm? zROnx#RoV6l0XthEN8;@6((;z!`M$zJKV`?0%Zdx=r4zb|b0qe#Q3eklX&)n&QE(q` zf4Ii?0HU>okLDcf@r~$1*_;(2=K^!KEZq}%ndB{#n^E@<69Fb<7Hc-ihpRNzZInG_ z$)+#f29>3~OLfO9Z%(Z?4YjTw+M~Ofeu0X*1~BiZ)one%+8F_SsX}p*Bb0M}1J`jR z5Y6)V5%Rd_XJm=TzKu8Rfe5I;cTNO^C+_wB|GG#&GkUoiEA@JZ%@Cu(D2)##`)G&9 zL2SU@{Lu14&$AVy9Vx&9D`}-6gLHq9%UV5y)YPOiUg6(M5N8n2pTHyQZgA!~Hd?zgJ=mCX$fS_xyrRLixTN zv{l|;3DXKCwremiCMkTHU7kOOJ?qC7Yk7zf^110(@+9eejmJ8XYj8zQxwp2yIQc?% zkD{&8h^i%5DIrkrd6uAYkz4HH{SPZN*st$z7NnMO1mR;=9_w= zetT>xO<>gO)_E)GvU$`RxpA~8ssZz0&Ed91q+WTNe;b9GNFHfHlQDV2@WZg9f_bH# zx!&`#bOFVcblMiGx1jR5cbLa?^Wb*=gYtF)M~>BVbE0<788Jxze~Rl}WFs+o9U)I{ z0I{5SJeBe{#q(Je&^0>RmC-go5Z0M2@;i^zZgA?$S#@>%2jt*{cq0)PmBMB#QB#a- zv|mV9>S)-64p^pcIK2(;>N|AkEY6aF8#F?NS{42MA^94cbOnA}q3NsmY{!%k3r+|x zO}nz?2wiGs>;rfkm*9EE%l4j6l@czr2!a*aaDv>dRb0^mndPjBY`KcC)_qC5^&5Th zI$z9jVW`Y!vDAL;$v6HX5yg!-teto!V|Wgyo$u63q`%xhe-(zE*$T+@Aws=C#=+3~ zBRKheNRHJO=YeyGAu5R-{xdB)zGCjP_zVj${#OWL__Zn~j}JA>1O>;J=3uF`c5U~K z%SKOYtu$59)_P+u>1X1fQ?cetR2hZ@%0tLX*ygoH;>k+lXzp4ts8glkF`Z%$Yzkg~ zI8bAabp9dY0r~30ikB`%TLNQq@bPTbQwCm+wwbuQ1%R?>BQ-h#5MHIkDrpq;RSWvBTSIKyL7Wz$8|>+-6% zdN+p;*rWe1UXg37hdSFDq6Xmiq=JDDDS$&AO1sP7fqHpQVt3MuX+CQ3#lHLGDPw?$ zOQwG+EpMU*jtsAF`-lklKFfV|HEVmUxYLSbjVP&oVjLrCh*bz8l& za*aTsJM`eC+qqGk1?D+rlbRt}`TmO< zHeaYKZvb%V8^EkgM+}^!{;x~R^phwqHE8V^F49~HsKPWUXj zCwGMC)M*-NF3iae8=q-3t5(3{3bImyJ*xdlbavZ2O@UAvOgxS1v)+Y|*+2;w3bZrv zCa=guYf+}4_re4clWnx|7WiY+b!AR8GVOr);TIzD4k-J}p*`orZo@It9j3lpZ_Wa> zE64&B%GQnHXc-Y%4O=5^vmQJ2dA%wc;2p;T0-(ZQ1t@uLBGs$EHj?K#yHif3K+U~F3c=sPE1>3r)U2+E4LVrAO zIA_uAe>yWh>y0#QjS_-eS3h3k1T9(~8ikfk-5nc%!(K6Y-8to%W8=O51uNl&tN zULYAg+?+B_#$8MQ8HxR}5kka3&h_kUQ__64${+Da`Q4YQFGC{FIAG>u?% zUpj69!&y5`|ER%aDn&xzP7@cd8Ci5V-u+HFZ9l;jru2}=hehe$3I&Om^)+a!LFID24AVKSaKzSV|LD%8Zc98bN4)Jnk`F4kV1w* zvXvc>a8%^5P93WIDw5F?k@3IIc1*Rl2#K2O_X4DB5F_c}JV@%nJe1W-%8J8zFYZU= z^N|m|K`zGe7myfkfRs$H<0BlC{hH9^9zdpuUmq=!Z6mw>ld0#&f#P)Ddu=&Y{BtW- zh9>CQb3Ph9u+SAmUiwjBCUI-;yC&CknFVlE`f}LmZDpG%6lm$Jv(o_fnkt}0J7;W3 z^Wf$=L~|>y;_+Ii=0kxsxy*8uZ6FQGhYOENksKp4zI)M!=OSoA8m$V|E&8)c*!QD& za!q0%l37~{vZbQcmN7Ok1rT{)XqD%GmbZ@PIY$}#&->oicpz)N`~jJa0vMP#04Y`h zOM&w)KizfenZ0h$-%7L_oTtv<^x%U+lUNOL4^-{y)uQN_TuR7Vf3-(9NqqJ8Y@Q}k z-%)}F0E=P?Aro*eRg02z)p21QL9wDQI{XeF_CP!}IRX}qT?bZ$ILNRVMDTxvLIyGc zY8tgQ&)8>4okLvz2$D_FdZWpL9Hr?V+HfXo5-n*>EzpA{Hpw9Qw{iju`C z&#OB z(Uy9fz8~b@v_Jz;4)8ihGLcm`XyAXnGf4AsxIPq1B{K<2W~9kQ91?D@92y9MoaRT7 zm90YGsqoyimk-R)+1v1~q$9Y+-{Xz}ig zl@?z7#!#%h`-ei(+jEmRpJSfr&beD*_b6`n$Yss*`wMhR-<=Tn0|cEyJ!ZjufOhYp86C4Z*DoRQMS7 z3tO^uuV0PT)O&%O8C!%g=HjX`3`zFK4d$q#uN6J_;VCrsDTNwF$!ALSrN;1BTEuf0 z0LIxv)ym($l})F?fo5^qkpp~^!lBv-L*-#^cE_w>WLdz5w(W9!=398W4@?@WwvO*{NwiQ zB4m+UN9{Vg>=ad4>A+yh=~;a@T^3Yfdv8p5Mg`jCVMD_=Ah#B|Fg`wn7?U0m~f3-*B^YG;OJYcz&N zR99mNho)c26`3C7IA2o+ZOLo5<3p|GA+BLYCb*{kS9eco`z_H@xH0=ho*)yjQU&oC zSQ6Dm{ge<#so48fFet4i%?bzPtW>c9W}=91)y&-o;E`&3HGWoGs_p{o&C-m=%GCtn za=e78tE-DC1TYDgCn`PXe=wK^b-yECn25H8mfYI9*=Q!Nz}V9(;1+TW92SRo8m$Jl z30grF-_^(c-qcTxMnC}CeVd|s^*#BHX!l;zMVpxGLk4R^8e@5~(|+sBj*)EI8PDnb zjOto2pF7KCdoxaVb4-iNfqX^1UUc% zEg&*nYptVl=Uis4ba8iy#{tj}VLvON0bkHR6G7~kw%-~ZI5zHT^mTi!9O&M!fP)CKOVBmr>i`E2RSlbut? zTVVuz_13QCq;xO=l6c-n{#VoVbm}-5rmOt`uxPu5Uci%%jx~LXHs}iJJKOZ*J5Oz$ zv?2ki$n;Ag3_E>1cqw5kk$?<^Htf{ng>*(+J)e6PZ>E0wY z=1SCC=`UiI;Ps4#-o`6W4Y3JUX$j!W!jBzHEXP*ft*X>Gb+0tomdd3_k$~@ zklxbpW)$=nHe0&+T3b~ZeL4*kl*lOA6`lX?BH%@cqky!k+tWk6=eaZO?&w)8YQzQ| z=#F7>2DdvRCi^+86r@fk@K1bBv7T;BVuZBNJ_%uzMhhq@n|ZVI6Q^CFMu}aO^M%Gz z#w$M^8iaXv;^?#(LNmvGQm|-FZu$^HAJuRA z{^YQ+_pPb05m#;R9h;NR)i{nZ0HVkEv)k9g6+fuzR%g!>3z4?z77{=t08)ugGjVL2 z#&W4Z1%%%7|?Dpz(2ncZ^6%hXDQG&~@^O$3h~-#CY?fK1CsG)#wp*cc##P z3jDczq5fXXSX6E0KCv7HAy)L=l!66$41iJ8tVCG@`!~-uteDF#CLzE)tNFp=5~L*t zR&05hKip?D>90DtFWbd;=|4oK81IFZ?6FIIWiWO!T*s$rL+?`QhN%vC2B{$`5>!G7 zFL86V9y4MGm(P@9mkPLmj zmzczCIt(f$Gy)1cD_o8n6`=qkUz62#=>W0zrS;rswmAB1vq!DPaTJMQwsMim&)h>R z3XmNBmIPl%xSiRdwVqIN@@sc4)%BTIpz|cK#E@WPGb1`UH`k2F!rQl1r_yiazujm= z@k^u-^x(wv%B4*4)4Q9pOkGyp+_ktg8(cj3Swp@73eV0I5a>=$&KN}+TlcjN%H->P z`(1sHRrWkjvxK`LJcikbDlPQMZy9_{&fgl|&Yr8rv`);^K;Ff@kWFtvHL_%Y(b0Pa z$^LD^<}5(l`U`XQr#Sjp(X{nnJf;Z;Qw@L2?ubgmS5PpZEQloqLN-1_ z;UaNf+|2e4j|vUEVsMbtc22#;w_>mH2!1`4&H!Bx?3z7NV_GO)Uy=mE8`)kTYh z$3Y+vEC>w{#Kr&(SU6G))E^=jaJ`3sp#uoasBYm}iBUJLED@xv*-&hHtT2XJj3s9qoQpQ@+oY#x z_D~b3T#}gP>W-%e`H)!vVny}*5`Kd$K@LHu`KYa8?}d6gq5En zm_ha(hV|=vjEuGjtMyDhtYXpLWguQ@hM|wA<&t^f7`VBW(ajpa7k^$9?(*c7EgkS( zKA{vD-^F1S-?apV}aBVymYim9}bsy#c|!> z9$Dw*hzf6a!zBa+`PTD0RI3#kO)#cdw<36g3qnqR415mPfMBz*qD0aD=*9I{O`MBhv>&CLP&);PXr#D}aB zAh$JErpu=p#L-n_+xn=ID~lqf?Q*suP8-4_|FL?1cRWuaR|>a14rOa?f1mn|q{i^A z$r6$JfNVN9oIN~z6{MnU9EJRQP$@)TEO{c3;zAfs;dZO|;_c@L`PE0Gm-Q3~go@A= z^JHj&yqm-AnQ^_{s=s>|veih;n^36~R@xaUqOYw?BY$8^yLaGH)Bq~t*ID+unUR->>4}G8)`K>dx)|GLASj(6FMOP{`T8#e#ZQz8=UH=bnN2 zQ*}}a>L@*Sl6j2n>Y=dE>4YlJxHGK87XB<`5CMo46Fjan#Ew#T-&pUx7Km7V@ylbq zW;j;m!{yv^IplJtMIV5>_a93wc(nSJujs`Xg@mz_7QGw-WCe`uka3L{sM&Qmm?`gd zvM;pUMK+sA<#5bacr&jnG+C%TURle935 zkVnHC07%4>WaHpWghI}6bar5G59e!DN4V^o!!Dz+MnCI* zJEK^S3YdoxKQp>|K3Mp^S?6?6EteJA2Z4AGD*6+DGd2iK@yIo<%jrEwRAbQ6-7%)otV|eE9G>o z*+L+hGxu}XtrjSovJwWt5sIwD7nGJG+lCuH^A@zDsCIk5s6wEy^s`c#j`kzAB2hAj z9dULTWKzdUcX(UxCXhdp04z3HJfdzAiANg{W=!T=LwRD*YA6B4JlNR;IFx)g3+&Yd zX?L*h!NF_C$7Lch^MxMU!)a?xOnvdj{Ea4 z^yqSb+~DITFFY1azJl5%Q1cR%@aYp^gMcCb_McM_?s883apK0$L|!E>jv(gKR}tZ1 z*F|J3Tc*&N$`sa_3x%MWs@0ag*qOmJdSvmC2-V>)!nQ$Y6=l)r_5t~8!_`GSm(fLb zoBWM#bpbKX8=~Y*Aiig^zWTu7b(FC2Oq?~5uOJsB3$#MVR265?L@@4$y^S z(k0f%{lw)CLnOP5cH&8}xa_EJS<{7pRJnf(BKW9Ue`=vzA&2**mMsIYd-H+nEVMIhY1(B5G zA`$qXhh(%@JPzmH)jRJe5U&f|UF-@@F8Mq^s!9vL*t7X_Xx+Jg4R% z0pxKfCoq%t;b#d?+}_WVHV{L0fGSY#ua8yR$&x;h0yS=hNI(g`at+$zi>W_qP*6XB zLp6~3HW<`HFj>Z&n|LAI6P!tb1S$fPfBVBVcuV0}=^-^-1;NMMq&)E(H+SmWY@y-N z)L}zwSWu$8;Y#~;uZ^Bc_*YI`i*D^d+@6CXfxQ?YyTfQzJfAr>XRiK6CWDtYkzQA21O*p?AzlD@ zS%G#5@~%FTkmse;bQGO7=k82-R>B8{(iZBOXWJh14!vE-3=&7S=`-TqFaG{003 zwW4if*QfV~x#onxc#jXIGIgNMEAbH(W)K~}_t5&hQ>JheKJD~rqY&;=jyKpQC(-PRJ2_@gpD@`}I`-u3sMJhFpGm3#j#KZuUGjI%9YCR$vYu~xmw;{7 zZT0(4@J!ZoESQ6OOA*FUDVgZpb^h1L{u0Gp3v5<1#D{a$ig`ewA&rXhQb8c-sXYKj zxeb27{rZR&NRh`##8VZlyfDJX0bK0r7(gT7vRbK$DQg^bR(3fN!}h}Nx&ft+LS8CX z(p~oFThayUmAPHpqidwOm=ny-drD-komHWsf7oDo3Q##~`*8xrHVUWD)OU*PwHy!? zfdg@uFKac^R1pLelg!PSj=cX3UnAc*_g@=jV6nHYxq^FEdu%WFoSpo16hG7P6i?w4 zk2=1>_feh#Ib>o3Xa`KZd@t2!5eAB}yZi0nW+qEe2|KD6D45$AfhVX>D&(hc7j{YF z3bJ-4T=xvSxB_03E6+l|XFTI^JKMN>6VQZ3tHB~>O3vwt2D@@}L9N0Lp~#-lmk{tVTvyJGJdY3%~$%81cviRwMIlCWAQbW*!(kClJ%!*wI_8 zJD1ScA^Z4z-U|k7=?D<-F8n_LZ?(zlDwn~IwJE`!Zw*z>L|{UF%e3y5`I#vI8A<~w zfc*6QLUY>J97vc1aCOZ@pxpp9b&ONxU>;EJJJ8^W?a;9+k@F+G_fn_}ayb&1m=swV zrrur6$Ux4(8eX~^+(@54FbsGxF{uc_zTWHp%HJ2&5epS+mzY?P-t_})h*&&TJiJQ# z1<_U})PZO&4AlCcwYBU)zJ#c0?Dh~GHPk0AlG|punxMo&n;WzR+pWNZtd1%Yce8EX zl~?%VIMg??`z%>V$mfB?J~GGL8oMPtUCeUCnp5+E`gDBksAUpt#rrpT_TUi zRla*M7iqK#`uTVkVfK+|JEw>%%(Otn@C*=zCl#p{TryhVslPa&1d%c-B^m{SLuoax z=s*fv0hK|YPRx%FP(nSf&`U$cm-yR;A6s;4Y?n*7ih3Guks^`x_Vu8gUKqlES`nE5 zio~^0v(dCbuLRt6k<|@95&1mGTZQum5#+x3o=5y0jBFI31xm;6$i<@gR|3yZtu82u zFB+osJKR|w0raCx!2Uw_!&>5-a}UUQwC{dWNI)gK{4okQxm08(wpI~f9SR9tkBRvO zT5Y-Mfx_WOxp-QEk>IbkKYD4PUJ}b00X`Uu;B<;FY;XkWIx;hIr60gsQ0gcb{`~Mx z4Vu`F#GT|Bp5H3$)QAk5y)XBaH}#td_F;Vm+WMP4u9c!X0f%j1k0S-JSJ_OA#IdkU zo;zD+%Rs?7jlT_QoB!^+6Yuf7k8P%H+>3!-&~>1sCg5a2=n8cgqStB@h{8B3@)C!r z8$|9d-2^~ahpA51i>;YVS8e*DcSTE38+<5q-Lx)?M4(HBbE* zp&CXtIQPzOh5ep2>T1DsJ`zH z7$$}e>5v);3F#8ap%Il70YN}OX$k2DQB=C5B&0!7NvQ!OM7m2tq@|?$ITz&n_xC*S zTJKu3SPJ*vIcJ~Udw=#-!eHOCr^OZR>;2%py{JB@vf{qu9)6YHhA`JC*?mjN$8o^8 z(5*IZYGa75z1n#-lBaH8MrvSw1!!*WC{|U9VAs~ox$m(z`&v!5MHC_e?-$pDPNn7e zRDnu|xpt0`#s|Ews*(W=m7O(XtUE>OObnf^V<8Rw3xK0pCZKqUC6l88zgEYngzo50 zG}{N!?cVr8$`d?wFFK}0J{B1e8x3Ax@g;!Kaq5+KjA-YmT?^X7LnEm>T)r`TLaFhT z;4b;*OrY(;+h#YaO$Pl2LA5U+E33$+@U9t2hza@bAt%Dxk*KU5hl>O9Sl;G%8eg(iQw)+`sb~k+ z4JrDYb;PNAAFbZJohV!&U$$s2)NK_OItMi1C~a;CY44j`pjB_Q(mq{YLZ?9}{9P@8 zs3^1+???lOxB^!a>FoH5X&UK%6$2)+xY}aBo-jU18|*t9k=rmnP7{;|NYu=@kqR&} zc~P{(ttUpz6}QY$lXlFt{m?LeP5GR~fK&%gmW5I*carFu2QfIYz* z8$R}8mPnk}ecOCC8Ynnif1`~9>4pUiX|{O_{NNW=7lGHg)J_keVTTidwzh^^si=2` z(ay`|pX{N(Fy1lRDktqLJnE*1^^nrLdf=?~XfysO8>&68uJjc~I=DwAQJV5OMQQJ}4(e4Mt1wE4jb@Mk{!^szuTJtO)d4MAj!U1FNUq<7XR9O; zuJJxQcLvfifHMH#D0I1t(fBqEi#D~Se}G@6#niMT%G>_MsKNg*&%G7|K4d%w6UH;1 zpdn#VHnr}j%wNb-EXNFHzUgutXc}n2+>!JfTeK}+Alsm}9n@Ygno$=wlWZ@R|DRseFxrBGpgjH|o5Lbc6Ynr|0f(5?BP z3@A1c)OR(|+b5ke@Ch&A{?ohG*JHk8oz~h&uKg+V(3tsLGJj7vb`R>qwbI2u?=%CC z-NiP?NoLV5L0wk6>+;d8zidID9wghK7HtuyX%tZ^5CIn`BY1F2pkLmYN z8-hT2us3WDpPD~MMahfM!?7`O!RMwK$v!}cWU4cL-;Upp{0JJI@(8z_wx0y6wZ2|- zi8(pi!<%;;7CY!{ojp#L@W=~BJ9IXcN4!E^@t;P4-C4|Cib(q*K8V6k8z=VI{uIdM z{YQsv&sRicAGWp~fU0U9JNkAiI#rq%=%XTV;1Hbxy4!|0)tAtGt9G%U8g@%}+{kNc z5k)ckizF9HzDzr=(MJ`J3JI^n`=D`=lDUlubmr&Hjty;-4C!9w?L^D!>M6)C3Rwt5 zfn^K~8Xtws>AMm)HOa~SYYfOFDz+?dp8lwj_D#7gM5XUBzZB{{Qc~{ z)T@GqXknF^^DGjdGBOt(3-EWkLeT6R5li})i`$L`3(l>JJ}U^ci7N`cCS7i1D}-tp zuNCuLqf@*OzcJ2ZNR_!1rQw^I%qaBWzI9Weeb=s=t&PGLbhmhAj0ILD-$F$t0>}mm7)2`AynpmDU|=EPYk05d z*&bWtKSrgXiYcncB6^R)bQkplC5!;dhf;K79sz&_#aDf)R@!KPQNAYuRiHsfCMnE@ zORxMs>h&Q%^m2du4k!3HMwi6%`QWj9DMMJ_qO|a#@Rr-ZKoUD|KVUU*PFrqA^VaPj zVW`N?9Tz;sg910XLMQ~!(CmKc0MGzIcqU{1EjvMa)}PD%eDAB}xBHa`uO8xD17*`1 zkJLNiq<;Y~#-*T%k%@V_L6iEHLq>ZkO*)_J7Q}d<(b8!eAt%`N%CG-fpJMSMfn1=t z=%0e`z*KU9ITdP&1!C7CWEd=Z)p-+60vWu+!3d9BLc?Hp;p~D|@pXZx9cuk}af*h( z36jRRlKNbWXR)JM$;~ltR(u7tc>JVWmG_!Y>$bH`HZ+(Di|vQ@UK4fFSeu3~p_eIok63kQKaNhfY}g^d)1_$)m|P72iDW%P znSvE+BJ0epv+aIWjP=KNi&72y^>LE1{6*DVfIrw#-B@Ey=j!*iD_P!sIO^qa8zBDM>SpoeU zroTrjEO0mP_FhW?`JvGY+Z1=7hbb?dU}DpFkMve$vnBnbw*Bv-fVdT8<^`mONXfRM zu?!3hT096t+e+0d?H((gUm8rm-IMqf5GNHo!a$0i?nrq7#)C4Zev2Hw zNNbpI_U$D7-DjuQzf59|LoJK&MvOHq@fZd(o0xb!=UgkNXL{=Hj(Ns=PTdBTTvMu( z@Wk>YB2x}3u@2ch?JnEB1M%>SFd-E$dN`}GD(&8z{&E|=ous0=qzhF;#70EywN+BO zy=O~{N=qE_T&|k&BOyRP2H~TE%wb8~$|Kv>T+QtJtha^=(6M^>Y^<28ybDhsf-qKS z!97jx4Qy$j_;qSnWDDP7j(m@wWc?9P;lUPHK>C7Z(Wso>B6JIY>#l)@ddgt^VLX23 z{D)=APY^v7AAVrlDaG-AsQczBYJq0bI`gJ?l07;T3{4@_6DN^)P=Ga~Ow?OAD3@clnCNcJJmykzsu}VVMVm z!d?>whE2n3*BaZ7=Kg+7OoFQ!S#n5h_7#6lTjnFpv*Y=s-Qz!gNGK_w>R~axo&}+Wu=BOWRFx#rmM?X_ zcM0b$c(&Oz?|S+9Qr6h+`0u3xF#p3JB_@G_Rzr8EG7W$!@<2%QhTihPD4#JHE_`HV z`Tw(FAwI#9T~9MInD$7?v#I!(u=C zb;y6!WPD_0{~pvASoiS_3w0k0p5`3XMVYUw@p0OZR|^?Md)*hG1e7L>hOh$Pd}0Lr z^9iy@Fjd@W#@L;hK8Dnsmq~NTUcat$!z2`0eDkoIhe3ZK&5hgHQ7u8Ukm(h= zA9_PboT8?LxcGn)^;VAcz-aN~COBQ0JFZX0+u{D>m5Lz8Bar)kz;qOK{T@f$Q=Mz`Y>P46vv8;s%JyD~UjVAwla;3D4i0T^crY4kQ`-%x`s37ug-r>$sM8^UfH zi8@;h($;tyZL^cV+559qV?lL^Di8pWHAJ0%n@2P^j98eUN?)<)&(bm!{&`2y12@?g zc=GojpuiNTJx#-v;`uWVuTaCoPf!~ecx4sqt8}v06zy=6nb5>LE2ZguTZa%|Q8r7&WJykW}8wOrrBN-hX| zsIx4p9<^uYjr!`8_mNGwuX{Pi+~=5;L@8@pY78I(LOQ7ja%Ei?A-)5ntVd+w8@JR2?rT5e=p2x&4@U}aCGP&M9e?Q7<1gyPtieljn3FYYZ$1M&PG1i;bql~bCe>hzgu#5e$T zW)@JE=0KwQF=I1Ii)wEP6q{!Zw!h=jV+I}HEe}eu(!T&XmnO!}@FgC5eYPEp(Wd5` zOJFBq`k^?+9wmk`k1QV3{|2@xQs7P`^PA>tm6WhJcoZg@Zoa_1X$egLIp2ww7e-q_ zvyvdi4sG^WdnC;>JNTz&rbTy9VbM-MED#`!;|WxsqU15SpU4|Y z(@#?KF*Thc-Mpta=wP~&32FP|OC}mgzUEBeeb!G@!PEi@PJhxh>FD;Tq!~0=OAQ$K z$=*wNJaCdr7oggEOFSRPkDT;(TM%(xs($$Fd3Jh25ziDHby}Z_m8WCtG9Ws^ z_sHJ~E<2R3+YHc)_ihvM7}OB)1j5l^Z>H=L-?~W4j5^4q@8hkHHZn12EOa zZ~iGz!V|FJ&!O_)8~~sG+;rokUVS8$Q2HK%02t??WU=IIHQ$H&gc)6dw+$6!L8}dG zR^&!q6C2USbo~8XF$3O5eg^JNg&b{?NJ#XNqZklq@I)~`8bDFoBbZv>*H&3WU~E4n zN+kRd{sgded2qcNCfsZey8vUh6~wZSV~?@p7AXG6x;gI?DrVmh{n|?$9I-Zpu{1^O z=$nhbadpza_eb1w`la7|2~wpP={>!VLzW1_B^oCEJFd95d)$rb0aY;%RjpPb+sTPI zel)rkfU*ZTn-3EE*(u&&I7U`Z=?S(5wiOXtVYP8HZcGwmsSR=yxT8Z(5PBZiw9Kp# z?V}ar3q>VrUfN0Q2mjRLjVgR4U&ec* zeR?~p$9pme+CI3lTx92cL3(#o?(Q>0ALhu8czw{kVCQV!&%d?rmF9F?|et_HUqY5TZ$R5Yq8z zm%=mvG?wOyyKW$oK+=I?Q1_WFT^qNM^$72*XK5yIb}id^J;sYozmWuq+R-n^@tM4; z)ykP_=p+;=k$!>651JLLmwC-l1yX#GrT~k#m1Ptg;f1GZ5`QP=k6SYUglt{E=zYEF z_ndNFY0%v^yXg0plas|1zcH48NN4BgYsA;5eg3=}c*Z}NQ*Zgwli#Z`wng+D;3<-h zY+%yPkPfo&=ltCEtmDFV5%YjynXhlolcyn2mQEwIP%+}k0Sfhr%Z>0BNf0qFyVfu! z`8aa6!F+iNK&!xwq4az^DF(0Y2+zZ%Yz+YQjOo1gxG|u2Q+V|0Wy)G#4ux(lj$Ub< zv(+YsQ6#sbxNEUd^cM*X^)WEehR|U#f=Tf=Aw8G^5%}fGbC$*ZsWPir zH_hVjc1SYKAr?W4h3jCdp~AA>FmVFm9vB9fGsRM{Qt7x9xszoyY~wT*I4{Y2LX8IR zU;z%fKmM~_3K(Kq>e$BD*TR%$aj4{N5PjG+GYf{OjwW2zZg&$gQ zFWvK+e^JXaq}$a{K%?>W5t0QXyzUf?w}}*Wvb<$CjmTNs1Tv%nQYB7=S=LC(qTbrY zQ5N1Y4$!3-335qUdtqx=Vpji7fD%zdVHzN|%W)ZFq9GS`U8Nfg+3KBnM*k?Z>yLa8 z#pt6z0hdb9-8!@uj9R|N1blWd6mUGcf8E`&Ty&!P0^{#9oNFnC1|KW8vEu8am!<+! zMx2wvUZ`QA@r8S#E~LRx&^7Om>V@AzP1DzV`@ z)C2X5j>96Ie^!T~#R-?q59+DPUYnx1N?=JDvmPjPDc7U0Z68Wvgv(5$yqOjKIFxS! z)B`!g$S|R&Q)Hs8n@?^izr-asovd;DfCg3MOU-*_lKSg?d@KNjH>YdiVsHmUd^kTw zPl+g)P8Xzd1jrr8!6lL9#P4#Le)8cqdsx>Dz@uPP*??+Jwc?RjTGJ2rXmN$?E7bgQ z;UVqG{wUb6kOVH7aotc+;b#-(j?PTj=2&ZWf5-3Ee2iEqJQL)-L$iU6I z-unmZCu%##(vJ4&{GL4@;$%h~S7wwQOk1xalnpw*J{`uYT*EwBVSm4~d(-N^@~hpC z1x0^i(4=9nLQn6_a~*ykS*s`s%TP`1_&a*ZjKp>KIOZ3Rkem>^V0`!cCHj%`K}fy# zeEraSSKpM>Do;bH4VxuUK%e{EEg1!}UvH8v84Q!b43&?wz-MLy2ACktdA@@hf_K1` zQi1>4SM}U%mMUYnm9m2ka7zS;)2NUBnO#Q9?n`K#Vcc3O<8D^Qq_CVG^5J%otyO@B zIT}`vZYc2@RZ^%w@p9#rW0e?jH;1y-M`tfy?g*kU+Ewfwy9{=p@aav1o-J)n^53Hq zs(o21B%k?Q_dpGzAGgv&B_=A{HfJ}4Vdk2X6**x^0qBM#1C0orq10m)p@LiZ9b_gY zFK&?azzWYu;Q)ca$N9x6hx5_+mIn`d03P%doL_AGh_W|+=qh{WBLLuwl!rJZ?aI)g zEf{xU!EZUhHY;p3m75sQ%e1x79SbyQt{2E+)CyV-w27z~nuAoG^;F#>!_^-ehV=*s8) zuGl*5TT@Nkw^UXSv4r3{AXBvs{FgBeLgvKWO1^*nQy zwfC6nUG7NTj^g-*$Q~}5YRkb{F$Dk}&Vl2T8W#0NSBh~^;Cqz1-ZS^J+M)iT7nI%u zQwP3=(CAHdYC~#Ng(Ve>P`}bn+PHPTnBAmt4z;++Q~j6oA^@-RBEN=Wcn?_grO05Am9sjx0C$D{_=WRtsA}T1iIo9 z&3pY+Uy>fFj|*2qGq9m_V>a2ym#yg1HU24cJ#%k^Yl}tz8RqcyEbnV5`JF89sQ^Fp zSIm_87%J~i6)oLV_=L|yKZitfwnz(LVnc>0n*{VD5;90~vTJ16<&~H4L7|VnIuL-u z{re_JiAQywx{@c`{VtPBe&|@tPRj$*p?Y9mm*esPdzGk*#fZRYgTb!n?M^cVO0yg-fo_H#Pa0uT?ubtLXm2p0O3#k?@GE}R-gd{$vdmFiCAc0 z#}Fqwi`ST`t&2i>rY}~vX@jxUXxT#k#Fs6lvI(x|9N&_d3M#cv)SwRF%0z$;{k|%8 zg=P3?sWIsqvdQYpZz)IJ!KpNRcIv+D^5(Mhx-U`}=0HK0`h<`aXU)Ug%!2hCLLcA*pE3l2tIgILD8F)pX(YmXk7q$O*%axE>JNwMVQdjpnD;ggaX-5p z?*V^{K_uv#F9W(_ip_7EX75*2ia;Q-Y;xd6^-EqA216yf*UvYS6{i8q=2h%Edg zUAW!yC7qIjd-Mhtp`pg0^E2(+ligPxdtRm&QBNN73NY8Qc z$5U@XhW;qZ*S70+gJ>u z)H!$A1*2$etAYV&PjC?-aonjO<8N77C!vk7^mcLCg9pC4Zim8}sCf)LR8b`_Xf2hm z08i=9-ulB00h1PZe#CG%wbE3G z;JkeOz-OOjLp%efJaLcH_NR)6Ja_imNef784OSVi8Jzu$sqULgl>h1T z$G!L-r{VQ52r5;bZGml-*aeb?O)baVVrQZ>)_OhEQ^hLkOF0+<8F!)vepa5HdT;?? z8*#nY0$2I?!v=wP{;W60C9?rg9Z=-!RQLG2SMn{NJ9#(bOlbq1X@3ZP)fql-lm)o3 zm5-lXo%Pran5#>=&c}dtF(Ut)wMnE2!3(Ds7FfY>U6SBGpb_XbyLXK$E3t^jZa&mA z3~Vmj;%^2i?;3bq^W-O+abd`QfzD$E44S(Mg1062r_nuB7tTwV5bXmrQq|KHK7pDP z-0aa8GLb)AZG<#58`TS7rj6YC2B_-SA3hpr`7i;RJ-^z4k?vnNZlc(xp!FiAL*4~e zsf|ANk62`TiEA1Gyc;Eur=J{k59T~O>S2M<7t)F)2G>=_8Kbh#K0NSn%wtR%e?YdS z^!!AmMmSIJZXJ!rRL$)kU7x+F*1WEP+(@N9W}G>1B&pl(ab`J}AM>3AHIlDBVtJfmId5sP@N20SO(guD$b}XmPA%@w_LWkE~bfU(xH|`{-3+Fr|efeT++HTeCYa z>h9Ujzoix%YSx?hSut+B?uPb{jWpebCQeT3t{r1gXhHE)V?vohkvFXvhy-abVVgsG>;!cowmk6Iaw`5w}$s3evQ zPmx8#Hv>;eMI{ULe`s3f`EWn@nEc~;w*>GS8S9}ru!Er@(=Lp{2ox9s>4y@4PUve# z1w+7@u_w+J#RO9bR$Klw3Jj~)-x;S$e3ugW6QMz;KelBraxO# z*$Z3EmYLh55^3(9w#_iosdN?iadU~T{xCUN@O5F`rx#OovWlOUmJ`h)uu+0Dhvf^* z(T1l4cT9K3&LY;_%u(rXC8QJXRb>cuy6?KDo2wmd_sp*M>b*xx{l)mxwTh*4!_ynQo-Oh~ z8W1q`AvQupjccSARJ;*nq`r3Wx2z1ZSl~b}K7N0Qz|=L=5C*Bbfn3n9K-e~G7=ZOj z(4tRH&rU5*{rCBG1ZYHr<_H;Lzy_AA#ZOiwuXZRnm_+dI+_?49_g7_w9;W`5`0B(? z5XYaUdDao?Mw-m&P7K4ICRMHnN8_cIa=?lTGhJ;{;X@w1MK|y`eHv+Zwlz5_cSZcP zP*imxg=bhirw^a%Bs>~Bg`2KOe2Znwcl}S-%2~F9)=_A`$Yidpru$0Fc<{ZQTYk+L zcsiy=oE1L?LatBz=%!7V)2rMmdfr)zO4lXvf}+N`TK&EMX!dw(?FTv3r*k!o^e5fV zIs6WHmgA~f+`pQw)M=*?7&bIfGDskVe*Rpn2}5 zo0yD?6mPe9srBTe=%KU^RW|ub_XF{cBy6lTD!R=2S9l0-R@^C65`2xr`5@xS#@s@$ zKo|kHE^Pdq=T~8H+8luX#R80+77X}UU3i9y8`~Gq?%H}R*Q>`3ccn)gCr2EJ6 z3LkfOQ39&z^$0-#?{cbp_7$}Q4Z8Y13o15NUZn6C8Y)o^wx1#^&xa~oTi%aER}SO5 z^{rl=-n_D(ukXEjS9!`!ZB@745-rJD`O`zd)GP#hEfz)ME=-U5YFP+qHs8J_fgiC> z{D^&4KgT%Mz!>mdCs@rPLJ<4s0XL?l$z}+N5!KRMoa>h<&(``7>`sFhsFXj zzHX_mbFD?my!`?h>&Gv->xb$Y8fP0-eA2Sa-gj7#{#hKnIdtf@7rXub%F~qOgJDY* zt&KOMPlydV-aF7W4+jhE9TX~+2B6A5jWemWy=zGBgmk&fz*805W$h5TZjm|kd~TF#L#EASef+k8k>RJ{J#<{W#i?DPt#Qr>|)} zTlYw(l{{y5Q5?YRN!z``fY}8}0JC2;qcg$0WcJsVXkeCO2FU*$@qrf;@D=DS5NIvC zAdO8xm|E&{=DpmP86=PM8@j=-=NeFViyav5XlQUFRYV_5-~Pwec9UvghyPE4(=f$T z8PEWn^q}|V_KD|ixl~H++7otm8D5U}LObru{h!Omj=O6@1UFMA9Wm=b&CAEMzuSXQ z^T7j2kA7B-rhZMG@bHY_!=H2Y>(lBnLx{$0nxLYH$&rO^x4qTc#j02pIhGA_$%$>B zqWB2d`(?gRpVJ1ff!9ZtoDIMGw|IPHy6fzI+Ael7Nd2O;K$`dO^=sGgrMl>SndezJ z!r{o5L#cB*c4mwEvK?=|Hdg+OaWI$8_Stvk+a;|;DhIsrno=WNQ7L-2L5V~_%k=nB z?NTT)lzOclOr{@*-w7T?-!k2SdBu-xR+ zt|?7byI8DMkKfgruIPW-RH$0l==b0~htaKU{9--kK_vNbt&zfu%DFLa&Si6DmwT?? z#aGA6NVTRq>7L}1kZi)C#}^5-ejv--8G77PC>!3SUAFTx1sNpRQ_FTrgsZ9F2G>z!6|$4kY{Tc~zpnbq&QL3O9jDGy_PMD3XUD-7y7RXxL_^psY)w_i0##jl(=biR0cfY5FyA)F8? z74Z7BD@Jn7ciou9&egtsXP`pc>LF4F=)c$DnWOOp<~w30>|)uzr{6#9YkNjN%VH%$ z_g1}K1zr~`WO2W{0;i0=LsH`GPb;ncw5(Q<+{1mxC0D6;bs<6Kh(0#)o!?mVht zIMe^Q`Av-b7QE(Vb$?&Uf@W6uuiu|lJ}jKt6J&0V?EOA_u~z4k{7rVb%1y^CL1e~Z zoWrZ1OiB8O_sD~{FV3|u4y1iyIXNW2b1=rHS{F4%fT3wP?UjHRaTV?ZslIScP|&nw zjR3q=5KLRw2*F^aWpmAVavY2!G5$4Tj2y&-*SWXf12rEXmS3A?4?&E!D1+fdfy{=r86i9rz8F zkZoh4GuLmfUw$W0KRwYfOu6&0KS;Ak)8dNh_;8d@$Et+I??r=l1kWpXUGA0s&{Lt& zvvtGUvG5?RLrkaeGn}U>;JBm+zAVY7TV8e7!gBxl;p6ck{ABF7liuCF%WQk}%cG~G z9_zpFxrEl};4qw?{xW`k_Q}oX2`X}FzI^a4wyYfdXZ8DW?m<;5V_H2y7r`ZUgAr&#az6zDK(WWSk*>qnk^=4EoY_Q4|7lhf zKT;U8B@(pvfTjeMF@rM_`{@L$LUnfBzP`R3(AYarVQY95ym9Bv+Y-!k%1JULA?ZN%`}7sPZ}l4~RxgpT{`XIR(pR^AW~c72*gO<<@vbFTtn+*dd1X<#;W`B%*W`xx^?>>JCO>9!^!pW%*y^nywc)3QaZe^BBLzh z2iHb+ZUo_-W|s3TkNKZUY&}hoTwzMs?7RB8quhx!S7p`pxtnA^|(0CrWZFiWw7(9&9jQv=AVr=Sj zG^jq1rPZXNMe`Glm?Hm?Jp-3+Nz9tJQL=8A%Dxc=hY3W$WuN^6q8)0%+TeG}>yewM zQw0*{K!aypJG<}*lU|E#xKKQwXip6DJuva>ht^~;=) zfKDPAWa8{0e6-NEy|Z0;u=oW*++qnM2&voN!K0mVbBHVpL`sA zI?-qluIl6=icc$VXz`s(GzxY&L^M==d_b}pWyDw1iXSHoQt6u$l+Qk_Jxcg^?Y1)d z8x`J}wK~TmF3R{O{jE_I<+%a$Cz))#c$@7_w326?FAaMHDG2nx{q+08Y1=WS6m~r8 zMN+hQe?bY;{lHm*{yZ-}mpTJ+oNT6JF<5(NcS;^q7LJBr+miwY&2$T=J*ey}SAH z*}_VH?!4i(?P6OVIlWu|S@|t*@QbiXAB?>%G=V1pB@8K$8YEJNEOh25!Vu6;Cd7BM zQ1*~K-{m_Wy@8BP>f)9EvrKXve!%YJzDDPlysh5z#mVX_pD`>m>*2z+koy!=l+cCu z$x&xp1q0BvTFv~NGu(7kn?f= z&{EL%xhzKEp<(5m51JF7f~nT5iXYc25LnID4SvIT*~-qn`?abrf++YoGkKGQi-Gd$ z!<$){tbpfQTg2ob5GAJ7Iv%C%zqL3;|0tln9Eznx33741y(632oEz(|63c*aOQ)WqC zU|63lY(0Vi{WIC1#;XSWOeVKBDjMjr0HF&yJ(dVr6bI&p&u=@azBv16-BJGRIpd_> zaqSbsehD8#wZf{K0qAq0x9+MW?EX~TE*iDEZz~^Q*G(7uZggvk@QFiS5Tj_i{fWaH z(e`{Yv2#Zl7mE5Y!h2!h!6PKRlTK|Y;DpC5mT6!8Xd>IvV%L$!z7^KAN3L1l=Zl($(qkN&%GD@RoeDnUcMnMOpX5aP({9J zw{m>-!y0<>2Se}gY9=}*cKBt}+xR-r*T5qodAOG`@MCDSv`r3?w*C6_rt9Jc!>#C;M+hg<*`?^H6$Q`sSD%z7 zwt76>Z4YiP_oVL`O5Zf-&&@X;IscBw_fQ)BijyTB>A~6}ae%jy5f9e0y*`x!09GpB zzJ1$Xt(bbik9+^h<&sd)=2*icB0j*v7}44~Ix-;kQdU+zxbZa}q7e$(wR`6l7NMj{ zDtfVz)Z1_-ft0a5*9p#(xdg1GUx#nzh0h&GxCV4g>;iA>FC-SI)d4z(M!)t&{U->y zq>BfFpS|~{Cq33P0gmcW8q9X%1SR1vTZZ#Y4GN4g(09{=ywR^MUNaa5Q6eoKsTWk2 z5t!_%dTnv|BS9_i+!yl}zmgXy*mAtU9cFZO;1dM$gi8t-R7oC~L@HK}zj+B%qSp#& zbv|m}_M;*r&6fvqfd?^V{?(5U-0X|EPsF=+LPRLn>}V1WEskgtRtU`YSVEm)$kyD_ zO^qV@0!Hr+Dmb{-5sCvv7wFP~x6;D|9PjWxBEm&JS@!=kwHr*l4!@LQvY0Z5Z|h$B7=S{NABly_sG}4`{cRMyb4`HpI*@f zh1P7V5#Y15An@Q`p(!dZ@L(f|WgRF2(EUneEiU)rhs_k8oU8(>haW%-ZvUXR;XNjW z;Mc{(#v&I7*zk8BI&$h4DjnfOTv= zA%atpE{>JmpcAq41rD7#m{cxJeLFZX)@Z=A=5@pu|4d74g?crt%dH_yiHRDJIqK== zXWq6dswR&6M@?|QbJqvOFA^3!^^u4$n)!vzg|e|@vO6dfVE0;(9$dw7*~=Watzw&a z<$DTwI=;eyPfFE4z`+{GNW%Aifft-;s5mt!H{;D9$PARI`2T)qfPf!B9P|Nb_sJ_> zkUVkU9W^`IeT@EB+2I3+i;8&z=UjS-)?is(`u7A359A0;*6=FS_P9Hnyp$HDpl9g- zc0rO55JIPf$RzT$OVub&)A*0Z?Tp1>LK(SzDyZrXWUDd&rxv>mn8Z&Bvb7gfCPn~R z#*UN2M?L>;1o4Q^1{Ig?CV|j1ov^~iGc&bw9(oNK5jj;JOdy*?{*e7`xGjCIoGukK zq^M9}{)*sn(ez<-c2=TiSaR}uAsKfGRCYM#D$*2Pc8 zXuwItq9s%mV25pxbeLvi(xG2uN&MQb1Fa$rUVV_JZRiyf85OkzG*4D^baWfN;9rHP z#sz&j#cy-6ztp{WE{7agDeg?uH)st+kbt8pydwSYn>j#TT#+FYycKI-i>c8Mbbsxy zHevofKG+MeriRqlqW@hE{Ug;9V!^VS-mB1M2oYba|7eq{j00YAk_;*d^TSUcKD-L> z99M|3j&qDIP8_ZSmWWMpdjKS>|DlT(M9e&skay725zrG?{(Yj-TmRFOqkd4c%mJD} z5TM+a$17}Oc2PqvgjbFdY%0yFj_=_mKS1t48yuXZX$O-6D%iFp^t~xnGOB;=EIbYS z_jpB^hNdQy z3xq;fZ&h`szvP0L^YRl7k#5fSmFEAFE&(E4C#ml|q&q?pJpt&TcA5@`1oeZrk}Qpy zrG$uvQ}O-M8MnC0chCGbRt$^FWkQ-*=D_{ytN`jz;40yE^!OUdZ->38{gCj6cato&Uhk4-d z!D@-RJ_ptT^&Y?ksN85j)MNISI5|(zsHv$zfX+Ks=T&W}TK?>ZGCMoK(i8)VpP%j^ zWIvaaQA__ZkKiwUP`=ylsmM7`aL9mb-u5(A{oM^wCJfXJ5ugM9reezf54Qsa?E`d& zT0GA7r53rM@HSX$YBw$mbi!~d=)5uI2QF8lu7F^K?6zTuJHRjy;meI4n^Z#INc0T4 zvVvOH=PzemFrF_qh(|D>3ykg|-XDRcJb55@LOZMD!(WV06u~Vh6mKtLzW+`f{3Cz? zqLFP^kIBCb1hf^(@Ht=TkeUw{(mi?lR0A}Bn}d-CLm22HXfp_uVzvJjT4+n~W61h@ zKYv9J8Q1^&1fW-9LVrglKz^Gtcr{q-|DQF@wr;EHCF9q3rMLd2@`dmLS_4JY_olTN zY`zrA#$@mYpg(L=!@gt!L?9C6wk$xa$N(_E38GGUUF>X^!uweE!j&1hODzgAk{8cB z{|TEx;YNc?A~lI?o$Hnc^=q-b1^gU6|2I?-9JaK4qI+;(6d;*`F0s6+-^^ z5^NW=mM{xIO0Vm^KL6bYgxVZRdI)w3w?G+CHYm<~k2Qlx-Btr0+QrEs!w~1$dS0Zh z;e`SU)6md3-Zi=8md3v^x?QsTHr?f25V^-7a@z=mZh~`#$v_E}1@H^(Thaf?5Su(w zTcoGb-sI}dXS4*={2BLeT@+%#%IsJ#ypahrl9T_d;{bo@IuKX6eruC_Kp?+!pWls$ z5T<-(?mvc8Ce{HX+LbkXfZ>v(W|PMfyVwik%L^Y?DPBGJa`oqoaTQuA!RKH;u+r#Q zhzq&`?NV#V?CJmIlk3sf|G^h^P%ln-QXh4J;&tlmeeGg%@La4f0!3q;soBBzpE>>vkL38fw{wsroK z1M&=GU=)LZIC3zEqwuf)0>A04dL1W(kM{Rkj(A_W6XFI)7&_mzgwoxaZ3z*mD7j2m z_TEMA*Wv=sG{m41`)4nRN@AeuCZ6J-03OOwhPGls3QQeySoz;IzyqM9$o)(_{x5Mr z+kuH0vR_frvCr!!z(OQna6nEy8OlFvM!G+&J}6|8@iJqRK;{xfk@Ww|6Emt8DrLW$Md4qDA(DVJRdQ8}5;#yaR=}+nm7)sn4cvAS zrk18V)JN!(PaIWL@ZaTKn^u{I{i8LGj&EMMoCR=c4%wtHP7H^41a|p5NIm%xK@K3c zD&ZH95LH`+fp;b=fcfRA#7qh~XD@oMfuKQ}*y$f_{Fizl=><}S&XzsrZo;LaBPwP_ znek=0N_+ljWvF(QX(XTdci(E>kO772>&kcTqyx_@x}Nd^Mhn=52SYZk85UZ|Vby#H#|B876Nl?TeVZ@(#4zYH{>dOr~Hhm+<6|C|pB z%pbT8enx6?bkMp|Re>Mj^?>bi9Wl!P#g=882MZ~hmX>)_l2CT|;sa9vHzQp?2>uTn z1)xFXcudiQ_5V`Z5Nm#d_4W5#fcC1VXF!qOg%S&OUPlR+i3Mb!Aij)A*2ZB|)mw8^h=r+^q#UN_p2qcP~Wg-=BF< zSfKTm=C7~&m+lBPNCA)q=bT6MrI70rf{XF*^Dmy!7y&Vih}ELNKgWY8g*a!d2ql;S zIP-ZpuzMmxd=Xb;XJJfK}c$K?P61q4MXe(g#U(*(IPSGDOeL7qC98HN|&kJqMX(&U74{A|IO zTq~K9CMI&ky_JhA<{onBA{}}n-SO<{;=ysT|c4;eUaX=EP zAB_6xZ~zeKG*C{_0CG@u<;Gtkfg`*F9pPNa(7%WG|8hhR0VDL7y^Q>OQ>uI_7?%T8 zxFLTcgo;mQg_i6;j?WQ?_>o?3;9rgfuDv|?o_59{cW~H@en=oV=D-#)cLg;#=!1$S zNg;ml|LP2AJ;+F~op1C7_gxtL)Ci;f*%G32H2>$UR_FWg#3ja zGTQnA>Td~r9t5jWuAIfBUdq{J=1N!2R{57by#-Qg4?i9e(q{JV)!Y_yK1uV@coX^U zcG=Rql`lOby|v1;y;H~IOhM*5tYT~eH&o9uNLV}!-z%s}iN5O`r006|;ovG;7O+J)x?fD;`7LZ`i> zgE{uQJebwaS}}f)K1|W8U6g_-hZC)&q=bx=RQ3=1*>H~fi=RJ#vfa8Be^t;Nk)7<$ zk!0UOJpe|#SWeb}ccKjBe){6?E)D!`1qDpdsFVzXq|Cxr&^;zAKCOsN^Y8etAa9zTuV_>x=6Cv8I;yyXnl^{#Na2AJ!8Fe7p68 zqBHVo>dBu|>m#ZahGJFv3-81&mcRRC?$YT+PZX3%IDdU!>a=y`;geO|lesba?O`14 zaq<%J@{P{0rI%)tCE^CX=t-i2nUb-kAEaX|cq))-XKzY7C_q%07no_M@%z_IY{ z^YF(Egw~VC0oGyOCp+3tHpYq{qpwa>MVX4Rva*^U1%>+ga{Ga|xu}7+=CA?pmb-Eq z4dfztlAOnI*2#aks}TZbREqWk{gv#}G3p8d=-?g8e8QWT#uQG3c|ncnk-LC+$J)&qY`F9mG1Rj%eo|Ao~rk zKJ&5nxGH9%Uv&s)J@IOrvY2x5`0iErek4^L-Q9Z9Bsxx`LVqXU(*)yWZ|UsaivBEf zd{{)-lmeZ5vh8o&#pm6jKLkxC@I^g0><|kW4|Nzx@#_mez9mzi#3t}7fAi=6C$e!fed&{zXLMq&!RYsR?c?V4Beq_5|htYi9 zxtNm`YeM%%+iZJ!rMpiFPi#eu__2?E?Tt5LMtVa|j*{y7UT^QL+30m8)xHv!4+H?p zafjrzlZkFs_N)h1ZwvY-%kx*L>hA{?e6ulp3NAarcrMyDTh(4p3kGK0*g}7s>vp|) zXw_10D5lEa_~$a;R#*AlgxgZbA1nGhtn~e(C3$E|k7AL!y3AlbBC#!{JvOv6FrP zkG#CRw-XhO!@Z#J=}+Ie(}5RBRSmddjvZc`^F2p^4^(a)XJiau;Wdm7tdACWLe55MpKGsPH#T*Alk%X2~8f^COf{n_P&fv^N9BC z+}SF{lgez7!YzktWBn;iVaK!IZ2oX<;VS-j#n@>$VcFF-NygKx6|dp0;=(~KX-wmuEg@#prmNI_a%rLP_^xc7x zTj%uDs?__(Lg~qoi6g+*M|wp*Y|{BP7E6W|IfTHk%YN_h)*d?ro+@tyK#j~6kK}^+ z!Q@D}t37G2WG=OxJ0#z|oV1rMsuY9+S!VMRGGz&Zgtc#E!qSqF^FoQ{b3PB=kr0bZ zSP(t(%B(e!8EE<_he93i<~a3LzVtAPZtj`;t$^rz!b1`dv&&gIF%bkf(|qMUypX6i3|zs>l%YHqC1afN{L-ABTuCSXOhTqQ9L zy}3VA=@d$~-R?K@k~=$HcW=)}$Wc{YPxf}StI~|J9@!RS)om>L#no?z!?+pMWo0by~f6Uj?QNU=1x>>DHA+5os{YI-3~% zoAN~Lw72|ZJs)xzPq>(Qb-X%yU|m_^v-E+-f8;A?{j8*esc%9^e!^2LPl_P;JNx*S ztBkK>#P{#k3Fi~+^@V3oE3!0hycHeOt>{$LZ>3;Ym(}N2WGU92xV54?P?89Y_^95Q zI!79*%TSV@4Ru=?_1KVtZ3Oqm}8k#$dWKpUI_23O| zWR#TIjMs(0+gWhhxu=OqOl96|z=%0F?y6kVWb6%>=vAt(C z!Cj}HJFD>$uV+n`y!JjgxROZX9{A8jIa7~yr~ih2+Ccl6b=Kh7N#55<LA2m~O}OCg0=wPc{Q(U_X&l5E_IKC8 zg_)OkUrffgcNJ1pwwnVIjxtU-{Ggk1LGczMBY3$V4UPb0w}~y!XM`~6(MRkUxwXPX zL&G8kuVLQ_jqN_A4O>8}Ebjf*pXfMbUE`t*)7EI`y8KVZ*!fRzZLwMT(&-0_ z2bXq7=ldBvw*;!-v-4}=eU=)BFWi|XBZqES#+c2-IbFP|J#kgPom1W^;Jx?cOMm41`P7MJmgYJsBc)J{k6SGlvBp#9w`gOX#wQ3aRtRCZ z_8FrNA#N2}$Btx9L1aa}V^W^`mCtGWbNWu(KQxW&^@OX%ESc(UKR*}{BSAQl%pO8% z2!{g(`YMD^A5&hIegK|F>40!P-WVMR9bmT6Gx?zNp-b+E#r-A(+?08fvb>B-0l%IA z@-z2>I7#MM35h|^MkMRIkmc9$`KGI#`Jpfpz4jg_Bcnx0C%jM9x$>4v2(M77=o=?f zngETF;J^@ zmGLRc!{wB^nLNDDbgD?0vBE;w*p6OjDz1z7Ig|1EpMUr+iKF>Yd55}cnmA)cO~{^! zyF>4a%)&!O1w9|>wMM>t!aeivpZ)9o32b%a>x==!pXW(se?$tUQQPJ6pqMV=)^IgJ zE}^fMHJj=9NWl+~HNG}6od|wB`L6XAWq~s@Tz>&X<=TG4yHuDs{lT)6lf63e{Yscc zbt@TuhMx_qEyqyH##fz>W#=Yc`G(KmDzj9*L*QjVI{ObFRsmq=%R~%9T?z!;|3x}= z>K76Hr;&~j_}726f|X+odU_MEo^k5m@Z*s(lfUbYOTGG1i1Sw<-^#R%o&Sq3rzw78>2{KFz-kpLEx@bt!=Ir+^&Icuc~jyo3{`{|JIvU z>yL6Y^2m2GbnyESFNACkiH&JqoBw$9F}Na(czBmQ{323AMR~=Y1UG~Xr?OC#f=YbG zw2dR2N3xE_KmjfU=j!WM_#qxXl7Kx}f!9flnvRpepj(2R3EC%9qKzn*{k(*~faWOc z3#K4LH-y=`{|y)eDy;sYA$r@ZizAFKGn-{DQ!5+9w3P;IcHof+nmL%QsQ9}%1)vOP zH|gTktz=%>(N;NeOAOY;X+6Z~4XWrsQ6-!>5{b>EgnDK1Bn1;yy_nKum3uA8-M6c^ zcPoGU>QZ@KQC17kcvokGZ!X>cDSGVG?r6q8B9UDWhinJ(51?YW+O%!ep;>Vr|5AHE z-|k=~J+nQRX)AE7C63^`)N@+G4C?!NE^yhlQ)yM4reVB@y}g6$k}zKLwu&e@J&jWajU~C3fuUiMoRoUZ`C^~l)sOebcV#3}V#pXRFQMwh zxa%S3$Taiu-zKBu#j@D}aFD&FYYyHqfa!zg@gpJ}Q_~SXTWNjZ8xp5&kIyodIPd{? zN;&r!pT6ISR+}70kI}NSddH95o5kVNk)!9WM&h(SVjR>$oDz0dJv4AtA}3S5w2(FL@=7o>7Uhnd!J>zE`i&O|KLB;PU3(c>~y`psJx< zaMRpFrgYmnJEuqOV};8y1R>%-$@{8>;a9Nc+Qz~0Vd!!y@rZ|R#-<_|cPkfi8rBSn(tR*1L5)$6$E4MGk-`|ZPJ!PkA4V}f!3DO2#L+>i0A zZQBT@{{$ni_QEX%DqMpQ!vYYv>w`+nllvD551I0z8SjY@AfO;~RhgFXe~q@jbmN@$ zlgA?A?`7x3TLH#Z@q#OspR#Dxk*8(^KDp*NCDu*R4gW4e3r+0mrwR(zjN8!4rm-mZ z1ui+x-2YMI2wy2@CF}cTdKUhxY0bk{dv^Ejdd%NHd z6xg2-2_r&!Ot1abW~euOwolt74rkqE7uy9Tfa8E{e)rsI*mi3{@Ony7S;JS7#98cd z*^s0g5wwD8m@h>j#Y*VHdlmS%*p2PEJTGFm72grS57FbLJ*{qx73${1?DyVW7fK-i z{jPi!9JzGGrmc}v%zavCyH`^l*rKG6&joNxFxLQ1hUd;?n&h8-nWrgwkiZ$x#nRjF z0o066TL~2aBRl^|$U?u*@95uB?FZH8V!u+tPb(2}B;T0=QL1U_E(f$9S_BTKOC-TS zpUr#PGD(z!7+0a&03j0WWuEG}m!I5R=^=q!gZdl?(Y39XGb{^)=*oPny5M3wY^ape zfw7JZ!y~2d^zu8X*NpAsuk34$fhTulBV={3A!Sy14L8ICjVW9T&gT1ii?Oc@%5SoG z%+ZUt4bV3dJS%J2`A*bf6t0j6{nQ?GjnV1;rKx-L>p#6EwBk`m)kJkt{EBtvluEAo zgTOCW_U0lk7Qa#|VWIALCQO@*mewW>_5+9!%uFzD|L$+`MMj2_6&;0;W4Ml#3;uPK zkWe=C|JP`*_d|rby*BLKC%`(_)jbh>ldt-}d=*m`0<{2!;C>1|`}mM?`WV7QAf+Rd(p z*UO`sxjlt5>uUW*L(XNlYfgh>Uqf$ye0yR=teKmx8?InvLy0o^(}*cMJjSi$OcQq^ z2p(C?4I^c4f{(-TaeUndHaQwL)U&$GfliFIt$}qxj2t{ zfl>aLZH#N8(+FqMBEcxqHg{{AzY*zBm8aaT#h++e(!GZ^!8QCp0R2yfSFY}-=Tq7C zk5qAtBmioDs2~gl*lFg)e-H>PU?(GN`==Oshro8dlBZ9%OPW6$e}RORwOK*ACkKv5 zj~?bWZgZA^(t6FBuxd=Y_QrkYku`@#zOjMm-7Na`f$%=O1OE4I4n1?rDpxltDO`}je2hiU1>533SVv4F#fQ+6*}@L5Gdn!tTs4l{eS&(;W;i=D!eE_ z&qNO<56A3=0uktzy|k8pe%WMbWV^j@O+B3?&|_zkq%?t}^zpjh7+oBlK>C>B+&5=M zO6`-YJ%fD_^;VDFd1tzcfiMx?XR=EkIbs)a=--wXZwkc?)`8g3!1n#Zy<5hN(+(|$ zSUSZ}N4hgo;+Lvkwo^|&5|5prPs}qsFm*3CH5-#r28poX;4r6~h1<}zy`mn5^jW${ z8`G-2+m1Wf!t@a`?KxhI)}9MgyUEj3l$4Qt6hjhy; zZrYeXOJcdgf?af2nN`xzdu@*%TiioVgD~_<0wgHt-U?nTO)QQ1OCieL+Kd(B? zZ+)P}D@r6r7HQN}gQ-jl!^5q_pqOs@>~X%%z9h|M=88tPZ9V2(axB6kEt>usM|f*5 zm~2HCIzr;=^$jw}Q6Cc5TY5zrIk;>@WPIp&evFeXv^ZC5$Vo|MfhA}$eujOR_%PiePN2uFaldz9 z&wSyrtH&h9knE%^FM(h!YVlZQ6ZgMH7&~Uh_+NX461~|}j3_q8RNeYh7CCGT@)u*xu4gGVF1Lc|XZ@3@0c>D4b zE|%20$67$wd+*xJtC@{^#ukxeIp3tYO#dQH2d!`MQ$qt+)74*T6A&v{cf3vp&f?zx z&f@o}x##hKj1#38uOHA^wv5@-*wo{H-4kW6x0GA5X-eKvutm5ar;AkaG|}7zb_Y@K`9u2_d?}Bc7Su& z;F;-ku{cNu6ht8vXu0|U4Ig(pl2*EIZAp;ATZNHhxX|*NCawfxFPGKnxsSc7;s3-uyIWuR z*gDJHkc??vA`QLZ3<(vN*54quT~X` zQu8^!;vVGQs=HFgM>_sv3zwRc8ush@=vm%=*UBKo=QPRCL_Z79J6|SA#GkV+_l@&j zRsaJtDG@x{YfN3Coeyl6e!IViFToPaKl-n*_qIWjn9{9%!cd+m_jat}Coz~D$q1E2 zYEY#=qMIEHm+_t0KJ`Ix(rTcI=mb@|teAareKO{_=zVe(OC?Q$rwOC!H2L@JyZwHZPKoNL_D%4 z_2j8;nM>v-wKgnYGlnT(w2`UZ44Zqa*vwCOl9uj#YBNGWORE%{#Q$JU)R7&8b?DLb zwB}K6upZ#A_Jva~B1~upQ1k^r%sCDM!IUygrwWBtzgfeAe6^9L2(FjPi1{BtNlC;7 z$)=WpUPpR%^~O{z^^Ew{O_}wUEK@2oiaOG8F|FAd zPM$TK{+K9stj~oXN9<(Cxl^%uakenM6AEw zJa|@!r#{o~Kojo2Mr${!aKil*fN8de2gnS1wW+ACH|3oijJAu;yy`HOL1FecQH&aO z=2CZecg5vE$oh9UiOu4$r*r(h)fhhntUZ3v+7&0s1}?q4HuqzSc)y##R9DEq!-`2o zIeu|xXKl+;m+J%C^X@dq6_a#x|3SH?Sd@x_UrmLT5^w*_|DgXf!FI{kg`m0T9?Y>k%=PrWklKn{ zfmYws)&!OUi5Ww27;Ez&7pC5s2+0HK zZ`*z*)CKmKsxzpB>>6up6&SC8)rIe>?Sjoa$N1U={33FjerxkF_^i>eu~Sm@7f5x||S&Yp;<-Cb8>0buG`Qb49d!1xE4 zaNzbY%vwF${|6h1{g6d=fqlJA7AXR27k&t_NyTq1IO4U6%|Vf2g|bT>_a>;}8CdEC zQ^$Wg0wq78gg0UP{_k%lu!b}RfyRj{;1XPk+1?%FS)wL@LdP238wA`|e;h?jlgZ@! z=@bxpMr+AYQtljvXWr~^(J)3U%bVF!YypN+zFK;HRze;-n_z}&ATwGnR#;t z1_oJP^usT_@+!vszk0Gp&1yHyszueshux#0U4e0Lq9DUF=f?hIDQRaU#h9e;vlB+- zN7&hKwkWkfmozzPYbKoId|;T7Nekc z4yz`h6{@;z?iZd`A@|C(7PbG&EjfRufv|JamzpLxywyF=|$Wd z4AF^?@9e-c{Vl(A{{j=#mzLp7{y_hIU1vFD-fDHO%Ho8q+ASbs;9*$5({Gtv1Vt`N zEJ-(>)jD3B>h-S_ilgo+frM*B3&x=VsN-0yi(&e?ky)w&>NO>f_>jZG;HpgK$blTs z(IsKamh(5;J3GauAn{KEti#JUe9Hxhba{7}>@Qqvhw4lBO)GX?kDMx`ZZ*iSh2Nte zT2DHk=oE20$M+K)DRcX_p?wn}Fg7sUIfFX$ut@ zrwNm&SXCIJp`4XL@%F*>1-vOaxZN2eJxnk3Xbbq&o4{RsY(=Ajxn#X*n8gs@b0K|0 z?(0;MP5J5wutc7NknAL%LPbbghDsf;@|igs8NQLfO+TOrz} zHAg?tR}I~DMIrR#l5}=Izp2nwS1BiTL?uLeYxl=iy6`f21s*M)Js7tLqcziR0jhm+ z_==<{Yqa%l<@u(*eFB69Dl0Q;)MUP-HzH%49pxxf~ zh~--|YgKWlMLBAHx@SE`D}*?2HI`kJnCzg(4rU>>K|=!D`IEL~y4rHfhSg3INXlK? zOwcLw4Wgs-PLI_=>Kr9?7ibe+Q)BoA&*|p|Si9^8P8Qw(%}u8ot-zJ7L~1 zHYq-R_iP_DM?nrR+qX&M6KTA1o&y-ZXP(dIjhAiu!a-t3`yKxGHJFl*oZ=rlg5pv$ zoTH<&&y0K?< zjn+F7_w@@!$1X*Un8q>C;oo@2mv`CGIa8Ry=CpmP*sL;3FCcFAQ0Nk?@&1%y5ek~J zS1`o^LSvJoxt%B}u&vfWff$a3{i28RVgw=?^K+#q;nh#jdYLVU<;@I)_Qtq`CyHF( zJ`nPk;%r+puLJ-@jr-e&cIVxMrzR4JtVF1Y$NpGjjC~6@K-Ga{sW#NO5B#;WkI->8q379w;(o_-GRgspN!D;vP&3`_;rfI8p&75 zx$6vPb|*gQvip9?C2oeusD7M-1+L|aBLG~RfJGrn^75HC5d`ePBrbdJ+CR(zw7*c` zHGnAaI5rIbZImn{-ab~W$e9oy=X5Bq4rdyo(~x?roKV;W_Ys~KxUq|ksRMp^sx5$1 zODc?qy$~XE|4Cs&^bN)Tsng-#dvjb9j1zx3rhtnt;0OG^D9Y|j=>v>EaGZEJU<+J= zxQYU7`nkxCkenZ6j!1;8b-b>m$s`b-cZsUi>rZ&?l=Tn+z}Cx~H&|Bac8^~POrdnT zJ-`#PsyMfzaii5OZ?MCg1d`dcqJ^OkYfK;o^Pl`OlqogjY<-tFTYc__0I^o(Kv&~zBIOid!8}uZ2TvlWwsoawg zmKotp4FnK^mMn1V#sAwGXq|`|wdmgM4{1~KBUc9Yaw|O*XjdFyJ*{-l&^3Qp(d1t3 z*|=hnBBINnc&o+)E={O!GA*{07{^yEMhj5kbBaS!FqMJxQ@Zgt>D!-gRqX&$ib@{1 z8K7R;JJ#Y?4FHN<^D&ni!QTsK9)PC^G?mKV>|venl-LB)^)^dA->QfUW@m*} zFMG<1%E^92uKlC`t=5P)tZg>sbCK}z>;a$^1)UM_q^$b0&nY5wV-g~!4u7F*KN{n!%%W?H7-ReM=zBKx~6>%5^ zhr-MA&Av$=*wlA+5Z$uc&)({CoegFqH4X;dV?%4433I*51;>LG zv%?pPmW@lPt=CaO?7NGCQQNNTki-q9U__7UjRflK1gu*Hy6{(>A=V)%3Vs;|!drdQ zscMEpTE2b@+bbWZ^M-n~{i8txYyW$$>Z!1~N#8>Yy?Kr#D^N zyc_MjW%zWU)P*oz4m2q{S`?}?@4GSS_ zOV-5@Pv_f6_zixXKr8qig8=m5M$kUb<%O!$^@^zCRDbJTHv41P95>BxoACjMWEYhB z3JyqItsa4&IWnZzSvz9CiI1ePw3#etKy)DIg4VHT5zO2@)J*E_ixhK`Mxw~IG)7ossQ73>I@VJxzBjeoIc+r z5#Pzk9xUD~|s+ywS=?`uqf>COBz+Jl+^13xAzyQG6D@fc_`=ZQP5I#p}h9 zNQ42~F0@!@K}%0R#G3lC-N9J&HQ~wp<-)-Incw5Sf~Xqx<8B?5s2@=XKt zLK&X`zzw)zY-db4MuRcBIybfPPmpeV?#UJrtPX?8_(+Us(~AgJ0*v5|Sj*J|RHCn5 z{giFY+_kQt_4&@MIbcbgvJ`M{dMT;$iX6TIql>$uj`x%owex{W2bW;IF)qOpaja2v z0jf)l#^)CoVHZCkBu+RG#jMhPv^vdJbqBLIi*mVa35ooSr| zuZsMtcmjMh5qVG{Kq>=@M)q}mwykwiNqPwl^V5O!5cYfA_kaNS|+$`iS(oeY`=Zu+(?ql$t$kzGk*b z?Xh#cZM-rtqnxZ)+dEv<{Boea)NZCabO1o%`W2XNb6J^(D6bvEwKk~%6uQdP^G9&? zBPVDJCDHl-?}8wfVzpL!gO()!j)+A;>0q!AX=c2g^ts+te@d}_@)z_vf{uEJrNHj{ zYEyVP8LW8vE{PTy9k*Id;sD?qe~T)H;k|jsRIEVH6f6;%1<(lr$tz+9{aDe?U}$9OLw{te^1uM zaVTEuscH*akCo0W?JBeDQZr!mF6ZtepH&-U-X?2APd4{o-7vRSHzymKAUWK8!G zLqD>X<5>w{@cAzH)PQwkVC);R{U_^`s>ViUge}QXbx6-fND#)Br-0-(8D*D!1SD1kw?T2>3**LuPtRDhJ1bW4}AGOXUT;vl1PY%gHaM|7u^C}U=^+=TVW`A%T^#|3>qHJ-80!sWy7vtMEM7D?$W37V2#1H*LiBx~W zmp3-r*&1@b5es%%;5cgYJU@1f@r5(!J`Z0KoXne!&H5{TMx;3IKuI z(30hioXaXD-w|;49>!(xx%F;#rVSyJWgknx);R||8=y^ghLb${DuGv6o*hW(3kgOi zmIjlZNhg-5cj;g_1+3m`xs$sq7Gih;e|^`&!pd?!eEJxOooJyOSf)rM zIBWr`Txb|ZPGl=uTpzzH>5TP-fSW3QIk6~9wE{3Q%9239?Zt@E4QJlnKl^>)1v2=H zdRE$4V@qwj07Z-;HsQgcXI1D1uxkA#@<8S!ssdtk3V#cDqXC5{fdp#~oTh4(i`vet zQQ}EdtHR4~2=*Q%i%@B{vC<6|fSLq&zdp}AEg$4vMS|s@j9B5p3+eZlg(7Jot;3!4 zGg!i@XNdsu;NZW@bQ zE4YG5_W|vB?c(KQ&w6vA$hH~YiS104cdVk& zn?<1TDro%8O%ef;5JuTOM72NksQj{6{k=Xzbm zRy%2_4yF_0OM_}1#qkizHQQz(ODg>EfthK3T_9})!!;P-0$R&8&^%);$^fOROja}q zc{qL;zAsjTnP?e44+Sm>U~bP04?5fJgd=6BCUSW3Z<2Wbc#CRBdWKAVyPfztI1oC{ zQR{BzVO^8-ujkWw6VNXnp*J0=xMtO!Hmae$M3#D=UtptNd5#E#DScBZq`*=7Mre-Q$jH}#p4oIk` z^rUr>Kn4TXu{drEXJcZ*P+TOC!AR)57>ojuT?yzhf!mOEz!5eub0Ph_0EN(*I<6uL z^NwLNG4C;i%&{)&PgXsqxtcDnXbZ1d(#4?TMS~-;zAyUz@1;%#&P4*&(5Hs*D(f=; z@ILoSVVYkkB1hQfwS{L((K&qSVZVZ8dqp_lF$n#LP#_ZFHgFLpau^{{D+(eAt@67- zu=~18zo`aZZ#+}D#`)6dvy6;V5mIU5ch9r`G9SRANh1}S^?%f|sd(%YZ{iu$MI4vy z69_y(82v*wQ=)vZs#p!BiMig!{AOFZZ(nQOdfRPtqMeACVwF}cafhQq-a?f8-L-$b zdR(Pq5Ls+rt1KnPh%t$vD8IPtJdMSvbp?=7r2b`FVm9o0v?du-mY5xSupoOfs821Jz`WkZQDvSY zii;(A6h*^E-8KXiqu3?f1-Vwc&Kb_EwajaMiXm0SA0Yb$#5A&}QGdo*Z8W_s#Jcql#{yGIRx;41$zt!s~Hu%FVMQO#7oD9gln0vZuv7)P{##I>)P|- zTgTUZ7y&mkb|ko`oCSP`Q!EC}B)n07QYH|lc#h8VDE*wrwNnTP>Oi2cRmPyGn) zn+9WOMVXc~42oB$qM|TaE!XDRxLDTV_Y5BA+99oIhpA@l=C!LSyy~Jbog}AvKZ47E ztgJv|c7LY^8Xth4$V_G#0bfCVt%Mi(i~761Y?)jF8MU&*zeK>HClO7Oc`bF``g*%3 zBTP8(GO@YZJ_na!y=lpOO^?-UWMhUM9u_@)ap*(J;!Zyfhja(Yq*BOXji;0&`P7{d zoG&_|_B`RdEKF2S7{y^CBjf93)RE&I z|N2S{2(rA=QU=ri;ETaGrX@XE{k`*$xfdC{d4H{D?D|uKMEJd%<&_WK4Z5Q&)I~TZ z9yhqJD;It~joP>*s)45tC5E~F@q-2lq|yJS_F9nNL`IQFh?e$>phSejXGemLciShu zuk~j)$xXwq-J8k*auwbln?dO3`*#Vu#=i~SZ938*#39pdTf30EhhZ~AO_i_5ATeun zjE|$&6%oxwfpv*mV}>^B#}pg$yRgN2&Ly(~;&E{7x;zCL*4s%5E?3c(UtpeZN%Z(i zI4CVQ%2Xd?rs$&eU|;ahX^cRQb+5=};ux8GJ;h>Fo7noAi!kh~)q>{|nxzMbX5p^< zeW9d6c!@|Ch_YmH0~2L0QXWN0#|~~=)@u6|B3zTeOw!k{G2|{CeV1A_C+A@gKgr_% zmRks;{5>0`%cgEghJ*Y6Lf~F`lVQLoTY+c*Eqogg4<@74G#4;|@^!VVIJtV1C*&Iw z?y_|s@kg+9>BIfW=(habo#woL7z%{QXxW99LvP9e!17zQVDfGvf%HfqTXePhL$Y#- zv2#5`MjaRFi1

fu3aV6t0@j-98oADsb~v7 zCs4;2xd)NZtQLt}cPIUEcJ?%+r zik1G(Hn4FkwT(I|a89ElP4nFWk~<0og7Bp*`F-2DsU#*Vtcu=Vc#*DpLBSbyn_LC6 zEbsVeRVtA@0P$B;O}?6cZmLF!$E@tzF$_)NKS@%MX$e>RA_ucR0aOFWF5FKZpVnPX z&3(4}w;T6HwYxS=Ms4ROz&EgxT7cq2>XmI;0jk@899*nVS3V(e3M*B=HCL@pM?qW2 z^k}S&TmZ4c>HowNYH(8EdnN}!*@xk`7-Ht+wLd0&5dZ)BS5&l@bZ?&*OYYi*m_;4} zLrQh*j$B3ss#=?q<^P)n5bAQ^&}5a_O4DkGGM58B45jsT#U`isDL~-Cj+*M@r_A)R zZ&-5!G$^kzB)(5x>P%*h1WXn32)aQ{3({A?I^55Yt23n?ty6*NUM(+Evg%AkzaIS z6}r3DmUygR)dRnia?HgQ;kK@c!5aY$j|Jd~F#HtZjXb!SeS#AGX3S%LT8b9Vk>$oX zknKuyl)I0)QNq}Y9`Zo{A3gi#9d>JGqy3^=x4n7PhtJ(q)sXU$jnSvnzAo-TDtlN- z-rpLr!htyw-IRd0?!H&2;H&=`Zi9M#JgAmiu4NptA2CN}=hw$J^+Ia`c%ev(=U8aq zt@5C6!-6?_6&e*Id6X@0M0RK?-@#-7TI<8b5LL+^4tS-+4zNW>7ESQscd;>867mu3 z`yS%aN=hatpy+GOSx;{dakGr-XS6}9(Ogo0&@zbf=A>@<5}WjzV`;aPo@0p@l-NC3 zA}PvwDVTS9wHdeZGXbZBC=AU-d?#$Gzq6Y0Bl@)W3ZI7;0TIdYuxb1Qx!|l4h?u~Q z*(#WpGmK{v0+47Rir7smxPyo?30_{@B_&LS>^^=xTyMC%6KGY7vj&=4A8}}Iw~V?7 z?X^lNG{q{4=s(^Y(TP-&CROUPe3#u|s;k-xx+3x1&`ViGPqJT98KR|F||WB zny;9inTK>T9_iCOi&`_8_J{+s5#JHPn^&=YxZG<*;~XCwxPu=e!Q+Vy7kaJpbZEx& zpnL@n_jlJ_(pB|{-57KKvkaD65zLE2{I!zOfA-Spm%;V3e|k)eD5l23vBRW0WLeU2 z#o4$Bhrf?)4*d7FnG-4K>3@4G{dt2%ryOyLD(AsdK>R(zl~JF~b*XRPX$f#gX(HRU zZ9g32e*&gQU~0fRV}v}A9JO;;0qRJEB}Cy5Ax8s7dv3kz-{$a2cpp0v7FS#9zW3T4 zZ?16K5~a(e;f-3sqw{IpO8}?~VZ{AsgiiA3?>zFYVWxrqCSw&HebT-~(`yTdZ1VO`{Q3fdWFLt+jH zzIG*>&T~?}tZS05WOuO=V;;r~K`u%}$OFLYW}5Vn#rr=dV} za0P)q^rvPXabiU}MWxiuy6^IQDoK9eWLN(sNAG)nUj0K&6{WQNr%clertfM znRnEwFJV>&^kB=9!=JA>5^vJ5o%8%@-!^X;o4vi)rG9?=9n|+{tVILxi&QElU9`q@rDx5by*27bOma^$EE-MzQdD!9M856%q!rr-CK_xM!-q<_tr1c@;X0C)>f zyGp#Pm$jX`!b9jKVBvTmhvkY)P*4z^z4r*Z)r({I(~JKdv){CQXBih~!nGPlaRJuk zMZ&KjP5-tpARgVAHx@p~T!oE!hWaw;G`{ReKd)gdV0wRpp&qGu^yX z4y{lBHu~@#II$-^=0qmuS_QN|x;IHaYWOGy9;sXJx)=LD`W9tp%^ZqB`2EABJ0c;&hIUfb{s ztca$V&|}IjSIIz#E{!7{$#;36W1*fwbe6xMiy=lXc_j7wt+18-=hFB;4B(9?9ODD} zVGO~`s@o*h671@u?~8I63|45j0%cJoFU&f(o}3%SArHTK182ACgBy@`=!PRO!79iw)@P`ynkzUjTSvD%N9C4fYr$z zoTt5N#OdgJ$K#xUa*zHaZD5bve}2EJ`!k_-!`k)7x7A9!tNNUs^?RdLNeiBO_Ilsi zi<#KxzrnM2&SOKOw$HMR3yVl+KGQ!x&OomVtyQCfzlOI$OMSUxAEi;#R-UkegTiQa zsFdwya3|}=qsp{>|2G`B1#6&I7;(rNivHuMc-Pv;woER7*q7}3ED7tml@c>`svQ3L z(28-FXfSU{^v=#^=pR;Bg8OI|A5Xets-9WzS4>~MpZCWy+3H;k_ijqmJL7zuxqQV*SDRrz{)7o>N_9-i=MP%ok} z?;*PhA>mlNn17XcMgERJ*Z1yMdrH`{#!3K`^Af4ujR3=UU-?mE^X^ZZH)EH_9!2)Y zzdN~8<{Em*1I1YN=JJU9=}|kEBGqqbrqfSR9pm%ivw<8qz3)+kLvPRTyAj2k5b;>J z5-s?h%>cmKa`QRr8LyrsXh2++4s(fFY+|d|1v*Eqbj;EJo^4H`*^iP;a&NGfZEg>} zdAA|I8xG_i8Lfw2G1;HmS9iMK*!WZJ4o3^ac)x{C>;g&*FK$;OEVlehv9YTJr1!{$ zL}c)vQ86iv<7pUlZBx^dp6`nbTwRR{n$5oB;1>Q1TFh!H{+Du6p5$@rDyPc)eQlb@ zRjS(R7Qwx~$G%7^vhDF^@oJF1LjKz~B7lC^ewCSigYCT;CNvb!H~>l;$i95(F096l zKDy$)NlEi^9d(iHb&rLy0)ABHbV5LcW7p`c;3ZF^_M1jcyfPlEH;-`ak0$F->hhF7 z3R>qL8!!!EJCQyovLdIfbb-Qhg|RyQjuta=0E=14Ak^!Q|KY^!wLv{aN#&r-6LTTC z=tB%;$$l_D`tm!8UreBo@(K{29~Lcu0!R{0YJD`xEH6XeNNaVSt5{M_~Y=DmUrf70m9~cyuOF+iS}`B)i9Qs zu11#aJKC#BV_UJ9w*nbLBfPl*)y~uLw6BA?z9af?Ws{?RdNZh0)-wF1+A@j0?l-4f z2Ar$P>rMWj%w-;yjE&O#N~BvpJEF&J6v+e?KH_X-EBpT8Q|6IhvOi;Ac@}?cy#0k8 z+srPB46&nDYj|}t*65zTDrtLjC`u#1C~HnXK$~Z;$f58}gJF6;=Oq_ z-4}oMtWnROo*L0`Q~U}I}OUrx=4iho-_jbiEa+~XN3ZCGsTCoT&P*Z93>NHYGl8WkYfXt)jimbos zhAr7(oSa57@N!MRq|_TQsQt@`N6~M}%7G7&c^KN{9!rABikeQ{<=edX#X@*(P+LfV zX}dx`?CfeKu)DGFsTz^$z>j5j`_aox)TdMheyuHZ8i{C!uj1O42ZS)aRy%G`GxVQE zB8PS)JZy4-ior{o(^Yi}k4(ATQG|GKD*p6f4I|=?zIDx^dwIOdxKQFmg-vtmrSTyb zi2Ey1kgrk-;yYX2MQ@9wG|ld7(b6BhaNKxxkSh-}%k7?d(Uj(`1e0zNMjjuiC!Ru9 zTR)*pTI@X15%wfa{3b{Bd*Y_w5ZYtqZ8^t#{5SY{vxA51jlt6NvfA#iQhUiC+h!$U z7G=yozgRH+mMUg3_4m}>A8P-4^Ztug-meQ;Y;$XWzn2%ma_F}`XK zHt~RD{=+{uO~G8~@hopFw@Rx=vxkReHrm1R)$@|A^&?UIb{+nE7F2wQlq^0ZdOqM+ zJ?dU$Ln>U7ih@rg-2W}t7*PFEJ@yP zTfp6K+ohQd^=@62*zl5q3@gG$U8w&R(tQZBLi1aj5Qf#{2EL=}UmveuBP>EKpYmtX z|4uJ=(Au2LAPXqNVbpqZ(bQG z7ykYo`GUldLGoc<0n0Ub zro+@2y1ce?Bbm*9x4pj(dY(*8c)Cx$63(FZLjTnXGsKeOd)a5Ndy)4gh8zD3t2D~I zNa$5sx77uDNX!zdB}1QH!VBeQc~>K1&GMcqe$;tVyYg%P>1I*&Ru6V;HI*dMf{nSf zDQEul`A1uu@N>+O29LT4RhvtL14z997O~lvEq1z$wLd=8312%`+$MtGEk47A@50FS z-JJ1XLMHtqIR8!KKS8Q#>LeHpPlxmJ#(-gfx;k2(+1q7hKJ9C=TDxbQNdeCcciu_X z0iIB(6b4-JK3tA}9(g9Ym4>8{=WHi+BVZtZrz~MnKo+5DHCYk!FDw3i=GgwVtaDvc1_{srzcW@3`hsW%_DW-$d?{ZxSHat$7PJ z)*JBaal_B160@c852~j(qmCLo{mu`yAn6yAh7vJshO(+W7BcoSTMpgyW&E`0`g^rm z`VIQvd$g9m?J1XMq|yw`hz+#jt@NbV|QD90I8wzxM5nfOF8W?clg8tL%pHxlom#t@r{SP|E zrN{CQ(_5JK#@o^!*xlLiS5@WbIJ}Obi>q4f#|Ud|PUt3jwtmIXvQib%HD8)7YMtcg zz(vT9u74|@ZD7iqu1;5dMCB7##j+;5LX*XS6N#8#pd~bUZh7>8%g;!VC1xtS)v7!Z zOfgwaZS}C1kdeS~ZHezWsm}>k#gY3a+qh^e&Mfo2aJ88P+YnpPS_R_?n{aV6YJK%E z{3KR>=%1r5Hdo3R2SDozmSQt#p@k zcQb=@gET{@v~qObL=d-`m@Ax%n zTZOJSaW+mU`6wPA$~I1BPu%{*RJq-JsEDUqV!?Xv>B&ey@fQF7HqCK6@pi_Rw>Y<& zE!;mujO?m^Gp$7Fd27_xH;50FspyZ8GwyFk9o@GZP{)ujO$4ul zcy9S!5Fcj(Dm>r~{#78GOhWDnulJf%AdtDajxRo*E`?k)3QqsZ@LerSemMl{F^mPK z3Sn)lSLswAwm|3L_KBBHHgHv81;%0)j^m(#wgcNezXEXXA6pQ6(xB;*xdMh*php$! zW@n&eW$c${(w_)Um{7ByaR#%t$Yh{5RB9yQZ%hTC6x-7O_^5|@e&2Pgal$N~;HRfCh|Rt5l-n4PS7O(BYes_}!DSFE z&RlN`O9FCM_g>QOfq2jvKZe;~oX2R%K?V&NYs`Yj`3LG&{qt*s66v7y0^p%Pmdu`s zNl+0U_b8|R7Q_C_0F5H+JXmNX#QilLQWqZkRdsp4K)uF>Qm^I%x8zsW z`4|lj)VwU0FZSOpzJ`|5<9jV3;?jm!Y%yxF@a%T|1_wCPi}}-TxEPMf3h7Tb@xQUZ zRr2;r2|2-{DiizuyHdt{iBl8fGiRD-0EXt5>~n1UMz3lnf_I$Bxmv-ds%pg82&us= zxF!u%lk$R<(UVC>rqm_lEO)U4cqo7<&kM>aVEp*@ zw{#hNB6xwN#qE?zq&|B0VJ~{YUX8@q(f7?VXWsAGwZ>$-eBa%xDwXl}-1ZSkmIqmx z&rmKSq1cpIc2ix4y{79h)c*p2XLdP+cABMXaxe)z%B;1s4m}*-BTr zHuBj zt>MtcXH)S|LA0Zk88IxP7@f+V@oAg#ZBC%it+Rzkl5?`I{gHU|J$GECJ^{h?``b)z z|BQ9UQ=(CG!9VHy^>w~}CMtpfA=#9AZ^Q9_oc`dl`qt}}BVr}U;P5Uc%r31pnxn{^ zxYlYSDQJ#}v1c7uOMweH!5hCBW_6jWH1{ ziL8iA&nVTz3t=AsS%=GM5?l|6imnK7%beI9=fYvbp;G^eLBI*LaTs&^D;|iTgpRsn zB*Vra;Lbqfwdy0saj(d|xgs4HF7p?RogdFfo0wF<$!BQ1_8#{A+z{2wI#o^$VS?jW z8w9QKOYK{L>NQ14w_u_Sg0A0~$<=)e)J>`aqYTbtIw1WFmTnOX(g&wzY+)uh5Ibf`PFo^OV80(pnH~JgAV3$JVd2 zTfL~?M)nhpr^l1Md)&(5GfTcuV$nOD(6Osoi23!kjv5iE2fn<+mQRxL06o> zf6ze5Qht?E{7o6SIaY20#fQ7Y%ljRtJXA;?X)d&UXv8@qA<7fXbmfl#%mD@(yeev} z7XP1ZjqhuNEHJIDxzpr4r>OM+$b-QppWP%@>wI6uAnHYcXLH#T?tYyL)rrA~uVmiPG`upCVSi^qg9_d#R!gQ#ss2pI29 z$!f~v-ADT8hcLpu^k|5A2S+&WL2xXU3D=?a{LQV}qL7o@npufUBZ7==ywnk}KCz?D zzW$~h9g?f;@jd;DgT*DV9G%zsyWZ#P6; z?mteut01T#j0L0oJ>kF60Sa8w{5JOX_O>oPEmzt-rfW@Q+ubqgHuZ3ud2E1I#Y|6W zSh!5kXoinQ-$EFOJr4*s+Ix>2{(}L{Xuxb~co7rSJpL$U;>B-HS&P7FgG?rdt{d3m zb>n{zC&Siubu*1t-nH}UWQ*-H8(|ePsu6~06qs6K`Yfmi(+$%GIWNxf#6&<5R9%e@ zQeW%^IhGUmOVOJP`rr2e+fKf{X7xledo=quq$*XMeZ=3cRLegQ-W*-)(W0V!*YN!W zb9?<~ATmq*r!oRi@}PyGYO9V@9Szkq&!nrQd=fB?c#bp&b0c>M_QIv`ZmoAdP_ z71<=7pcoI-Ykd6f#f}U0d6o6Z-TSo}ZX8@pSUn^sCvlKAkh44DL|^YH)uM|-i!#=1 zZi=6t$8;t0V+LU&Gh%m9CjB+>-=z95Mp#&@Fz!cX1sL~&5sHFZ#<&K|6A3jBDcrBd zwMQgrs6T^2_Q7d4*B%)b@bidDj2Zq^!E9`EmrKR$Plk$FJzwk~Zet_7oZ4`hr&v^v zla^Wy@(`llTOygV5m6kd(!y`5CwA1|{0Gx1=fCj1I)4kL6D1(9+b!{G0d>^?4J)pZ zFs~E?m(4u|I`)`!{^;^AyX`5`Uuq|xWHtKZfzUr4#6}fG> zli9Ow_c8Y&Ityf%!(>JMDW;q1`Fu{uewven?LB&{1NP%>u88acrXcSfcxW+M^7Q-Z z#mXPQG_5yxu4g>eUvB#5uWl0hOTM^SEAJLnJtdpTr!p>jS|6w57b`n#K@B~R7k3|2 zkfQqFU)%XUy(V#gg8Ek!UF~ls3(0#m9p-M4oL;~J8RU=R`99^V1IGwX*SS8}*TX<; zB1lVjDPmD?;%B3GTc)c#OtoZ8vkJ5D<2+bJl99=IQo-$kdB#~epZ2qLvQ)$3^soA) zk^~i=HzuSY^&gS1u#%|=`(+uaWF6F6bqxs;I?%8ta%VKD%T~W0=g!yjelAazx%8SY zKw3Ts6+~)#iZRc|wBh(}G%KO+X z0VN}~x>8HHi?h!3cP@q)jCvSK{UXZY+I6C4{m#*9&yJR^)z{VgL%(|@3cq*d*@s?# zRl+~<_7|_}FPZT|oSOD;o1vO71t$&GbC|nvs@ng9%@uhgT)2^YQ@_0SMC66OaxpRv z{yTQv)qpT_XGeh8L5kt!!!iPES+>Y)*Gx9ZcPZ(`^@mA5rE;uz>JjJ!lud3P18Wk_Rq=7sE-wuL}!*qFGI9=IxWgY;rT~?qtX8;Hdj% z=ktG$$u6j7jG})TOukWEGKy(DYiv609LTKX>-={0FBlu98pq)CL#ir|P-Klm3_dek zTXaa`zfQ6Xt?jLXYMF2prXsunqp+g}3JrbD)iaO|e7hXdN%$ryu1@~Y%w^H2Esj4n z3Lk3SA46UOU=ms0XP;WvLrHC7zY3@#z|lDTTNB%dsSm>$45tIZ{6wk%vfMxjUh3!O zXjIo?%bLPaa+4`o-oZF~O_dHFJL^aZ@XE_<`Tlv>PG(#BW6ewqor6Q-I7QFVP_JJDSEg)U0b%+EW1x`&BfgjWSY$&D zITe-+wzn;B{LE6d8a2<3ApO(AfTWi!=V^vsxOFXO<>)bYzDXWgJe7%xS&~7_Ku<$g zgir%BY#QH7G!V)sJxgSu8SBQ2^I7eG7g$^7^n0=o7MFyutN%7K@Bs>&;Tizi<^U1s zB#EzCbYy~%EHy*cy}w&JD}EvT5778^d6HwrZ&P9@`h=P|jPAy(iyziq#&Gy438$Ihm|b9@kz16BpUkbNGEr?!Y0q+{V-776fr z*EPkju}7v(EDlGbjRri#Ph$4f_i=X4w;k%ZWyCgGH9uB-E_63<^twI&=-Myks|&q< z;@C63Y!aC)Uio6N#dc2{F|`rg?SKmmt6JEcyGQw7zH%c96(fNzqVByW+P!v=mnlm( zK=6G!dB!wBjpi!@qE8v8BI!Jcx^`?iZ!ViloF;rYNF!qtCuS`L{z4z%rCgG(s-_As z8;BM%tQzDw8%fdh@?4&mAla^f$1xSGx}<*^I@GrrrXm9yNXzIF{+G$Iv)jc^Biq9*?IG&l&lW$tr8~v_noVs z-uKzEqf+q;FaDS?I=(hpme>&<|79vk#v3ne>r(l0pUlt!WQ3xwFlw0Nzes&w@KTk& z$lb&FIJy4ogHhtzt{7+s>!#i-8?%KqV-P#bTpQ}$oOZ;agbWN=uzH5w?kQJ}Z8Wjr zDXuG*LFR8xD2TVDL+okw2yKw5?8$K8vGYTbk`T#hxj%a`aE2>wz_V}Ih@#u`2~)~^ z?uQBhEgP5j=j;wpjXE%Sxu22#E0px3q29{M_@wl)W?b-*DgMmtzpXBCmRsaj^r3z$`{R%EMdTGQz=b=Et87dQg^ zwk0Sk5zw2{&F}xuf_^VC)J$m7Kr5*IWI{%1lPmsi72NE$j_lVZ2>N9U#=ORyO{n>z z^-dpP0k)UbgC>{=Gy`mB(t|2p>$(TQ8S22q_ zB{IeM;~vs4WD{vXzHTExRidZ3_4Uy35>5aY;WPuejJ%QDA>$liT_s+3iu`jL<%ji_ z;g#c>?6zz4{1NNO!G`*zayn3^9YN8vCTMrs8&b|^H~1-0zDc5`k1s`P)Xv0kqKeZb z@cI!R#7{fS8 z{@?Z2ods3TN~c2F49Ezj7;HJdC$Q`L%!?g;o=LPhh=GliX8d9D?o_Zj-k~{ zG@pTsNpynvRtlH?oS2DywD{A=_$7mXvZ`>QFGsTI^~FWe_{|Yd71`q>$}-Qsu<_L6 z$&$1#vJ20wi2E75)<0WB1HQYe{Jk@;O14_}<;riKmsg+a9T$)s<^apa8?`M74&U4!0!%xwhZ*{J2R#V8?CJK_4bFK$x)>6cnZKsA`Rg zD9XS(S3iZ#t@&ttmm*vH?AWkT;w$B|@9sgmnT08M1C5ZVKhD2Qn65^eIZ_KFxhho{ zR3x0g>E&MJv~DV1F>%M+=NJ=aWT@8l@qu0F*87x-gjbwQJzup}Nz+leAGirv^z#T@ zzH9V@#ab)859Mj7FkaDAQU_JDM2d<3L4{S<%xZj1WVy+8 z9OGi_c%|t0=n|G_Q`68b<-Qix^2W<>msDWAoWEThF94* z{Z754Wdj&zviD4j%9*YtpH3zgyX*HaKlmDtx&LaaS#9r@Q)9E=-JYH;nO-k-Rl$oO z4%og{Rg}KDw>NSkFV-|!Qe{^_n%#DxM>jbcATS9ID#KFW6T&Zl<}^*G#>`;MuIE>H zSc!Z!9CS}sK4OFm2pSJ-$4+K7WA>CCm^+-u@BP^19UnQjzI9&lOMp9(I|^Wos*#B zR4#18_cgE6r1@m9#mDZvJJNIf8lo&^GK z@#y8SyVk5n;sN}ucx2vO(gNB+2x1%nafc)dwwnqAryU0FwhHrE7_YU&X|@qAccuMx zCe*798mojlVSJ``ywXHJix4{Zx!yqR>mahJF6aI&TA5)q?q(rQog~%wXfRSR8t77H z%V~rPbfoyU{SM|YIl&U%(Ek{SfGVC;jzpacKzvzKGzf!Gs|>BaN4bo;o{Z%FeMAnE z57|flG!9rCxhdx+SyRHMk!dOCYzmqL_6v5|MzK*tBTW`kQXQbx-U&Nelt7$GOEul| zQ3(L3zyp8^y8i=Ixac-W13NmD1CEs22D=B`@jD!c=I=`n?u)bJ$@orAiEO@2B)u~Q z61ji9F9p!z2XFXnpUmu*Bo%aT6C8z;S3DSX9i~FK{&$CGYZN(KrM^=a z>mPnRve2%dVs01jMp^S)+#Wa7V>gp#sS!WIWQLYgWdT(C@rs9ZA&MW2KZf`wsNflI z!mRpd8$)(dXZ2F70ggu18;)}U-I*jK+zQCSIXfL+0oRFMw7V)qAAa9McH1qqtFL>v z2`M`@RSuUo)ZoAelr(o0uorM&H%pnD7_WwGhuKO|Y}7S`T8>h4aIY(NaNtT}RuJ4L zLHc#hnrWxo0J-vMm!H3RD`$QNt(4@Fe-2e`biS_?j!93Can{f1=f0PUT^RCCzG*JhW^Tn+0 z-(L&YC7ikK#*-V;l3mBGY^UD2UEO%7V76hftUruZl)oCAX8N4j^s>Ury(~dG>Y4-t z9}lsL^JV4gAh%-}&%Kdu`heWza@eQ}t+tCfk(Gv&*;Zif+_Vb3Cd|Fek4pi%_88cT z62CG$TbRA94(tExGCYqN{L}F{5#*l1dHmrL&U|4lKh8iZ$(>9F`x7QD3fL$CPN&Y- zvK!1!n}Wx8&z8x9$okLQX_xw*trZiLYuI&g)9CriKw#}>H97QhlQHT7e01}1`3a%?d(N*RToQbMTkMkWRKzb9P zog8Rz`TiJrH_GcYgQeeTQR$r@+OZ_kv7rF=+!*)R8T1iOaPBJTBm7xh=1dHg*a+gM zO7doI%zZ=;k+nBEF}sTzFm}&Zh5{gMJaZU0+eiUG9>{hD`EU{753_#v@+Kei7L7G- zCdAO0FD0A7**xJ1yZKRdF9kF$T|d52>vJ{WH`3hU-L28W8ut--{>qlcW<|vfMSY+a%xF6xlI4|f#?ICngGW>qA-lQ z=){Ofq>RvJRa1tB)$`RDI!;1rs-=^fRr1T1uZy7P_vkV-I!fTG^d()*WN1&Mu=^Hw z%8tzMqBH&EmT>gelyk;XCyaj7{V0|~{^q{lz6yEVYa;Hze8%5g#4m}0_&nTDWTMoX z$@6TLU<;0-r@iF-?=!u(P1J7ccvkL3%=;)8y#b*N2{#PLusvzjJostbl%+NI8e=0R z#z$bX`$Xi>t-VqTN?LUb`@r=uD=oi=pwN3gw7c0uS?-eGa(}DAU zcK^1n)5~APIO?Jdqb~(|B!IibSn1_1Ins6ZpQS<5VLj?^$anU?Ij7K(DTw=r#XJb= z7b>uO7aRd1Y_qw`6ge(@pKW((9ay(x%_X0QAhTp{KUTO=Zd9}SYW_-KzP{G}6s&dK z$!$m8^Wn%4n>azk3io8bTtzUrnjLRMEeOeF1*^owa4oJOjuq>F zk8-Vh2OrB+qJ;5+lbE&?By=!plXXKcsG=D8)xO4@S`RcSr>K5Br_`gJCCSF1dL;3q z^66*J2pe^WG8X-8SAI9_rlc7lYjV4L!jvaLZuv-U0)!gxa7BLOREqhe|33v?>q{U3BXT}tj~F!+=10-(1h2Y%4(0d+G17~QOH zpFIm5KYzomNPXsM_ue_|-kC+_1oRdQYlCVbVm)*wMt_tRtALCJq#IItD$wn^vYdr9NA+ekpk!AGJ46x&n=AN#ai zk6p+?x51o+F!;EAQvi7gkFtEwi~8I$(U?_);*UMePyN+E0Hiom3zdRq-Ur3~+c_Ah zl!>Q8FF9Xn4p~&Us&(2OOUTYI7?dBGl79k?l>Kxl>p9(QsND|R-~;? z912ru7ceK;7ww?(DAt`L(X1f5Vv|=Bpwk9Bv-W4qp=bMg!cv* zhU(&KtPaYDsC#YmTJ`U-eX`NCT2AZG&B-$n7YXN&gpIr6`_ESE zG+!hqpXcl@w(4sIQp2U{W>r~zl-)?PQhD9)?%;6-qI>uCbu@~+ptc5P`Q!L|Oy;G& z9*8_`otEf{ljl@sl_hOx_kHc(fJrQ(FpLh9VnxZfjHT!%-{{nsn$Irn@lXYogXSs!d$yvLHlqoxR`zwVB=uWi4@L5jFj#VQF+qEbrc*a8E^jx`_T5G&cO=`LWjueE!@xKo$Nk zVA<~EbF_g{ZZa-5wXcUsu2_l%Utx%$m%-TC+(A=oG2M*SS3JkjqK8LBBPT1*ceZKD zV!{Thp~pPVf3Q&R+-7Y>h<#%2*2rka#F&hYDekl`nx)7LYQ>F~X*c=TjCtFh2hwQz z*5zXUI+RH1RV%Mpa5cYiX=%jUJ{&4A#beqrvU@HkL{jqBB10?<=-DB@bB_AH@O95d*G}OF8h`gziy*A9azl%^PSi zGrW#l;I?FdR9`^dl6jmRvVFU^GJOQ&&LLjyz{1WiGu2&rBZ|K6c!FSL;R{hJ@vChh z$L>7LaHNcnAv}?%>T`~Q^4-Uoo^=#(G{WbaER7@LfT==hP=7O!GH3d{MO!+7v-t4J zz-c>K_zu}Dj%v;E5C!FvEZeXr!{%aKl7U&tLlB@HL-4std{P664>T(rYFa`(g6gID z;Kf<8J;5CA@6m!nTF_r}|90W)sU!i?7!lBZ|lTUba|Gn@5{J#xuD$) zR6Je)p{&`nyQIhQqynzY6?z?u;hBOC5E6rPDVk=$boec67C@JDLBu{A3wNdq|kldN_wD8z+@@RNPwzLPqC+F$={qn2D^s4h87Fh zq!LrN*164tq|`Ua;jLXPVPq1)%L~`}~`4ox?Em*GtG2ve0*^QXuuzztl z4hmG8Kg;O_b_4u(-jDyz*nJTye|!V{eqd*#e+tRbTl=WyM2Y7_ z%p}a`=a=&R9-UR*+RABVJ6T`8TxIG%Ov{?SwK(rZYf$-e?GQLp>EJwW!uYRoe@%A@^U1TTpf=6F{Ch0ZgSeywPHhX@8oIRDzRe}PKd$y@v!y>p7obh0{JVJpA`R>vDMw zB^L4CXSP0n1%{>Z<9t8XyiX@%x%}|^Uw6y8(RK!6H>Qg{Yy>#Ez8IKRur!pM_$=PX ze=C0V@Gv0`k^8%~GsyLK&)0+=5dr>)7(tiAumuMJNyTdwwU-00`0&zwk73|%I%mb` zBPp^rb4tLxfpImjoQ>Y;^#^B@8N>WnK^GWz+&TP5*2*g0T;vN5!5q)K+3UiG6?Jb} zhzDr~erG#A+HB5`fv(drZA{8x?`_grM+atkf_Bl-pU(!UXY>>P~I$F69pgnBL>{^|8-R7v6_(Z+HJg;QBA=3 z9kQZc)#j;~iTMOM*O6oaTom3x2sv})r&nyU93VUrv*qGZzw$ixzk?CCN|fentD+p= zJs|S9Y%u-l{iypr-y`3}e+z1hoWsv)8@|ZGn17m1I@$E6mRt`awZEpLX~k*s1vbGn z#s7J{_jOIJW?XCCVgJ|Kl5&rBNx{+NRLF{DA16<^sm~h;=mt5{O5eZWKvVDklW0^~ zNs|>vbY(_Ru2ma$TL5&A^WglOi^cTkS|P;I%@ss{d{)Sdp8B}+KqRoG&xc*lJ;CT zh2~nPc!wx;Jm)Ucyy0{l=$Nj~d-!TI!B+~Ygc()5FQ2Rie;={i5lfY=g9#Z;v$H=3 z7k3zXFfr8TqkY_VB7C=BR#mJT>B)7y`zon^8(h>CW1i(9^>+Mk(Vr%J{kx4w)(h23 zb!s>uOsHsM!zfpU{WvewU4GbP$zHnek;7!^9EbyH?(aa^iK`!yQ+Xw0t29f@qNo1) zxRE9(svo_jVG45mn=9r5*m@hQSVZZ; zArd%DSJj?ke%?~=kH(P!sb6`%vee=%CyJW9Q&z6t1kUN9A*}G`iOLv@#!>*(2SVXU zIX5Da9r57&_b3MJcX*v5 z6eR-{XF~OerQ<@`NO|Ww{^NBifCI|lm#x-RE!}Zc=(l(NfgXDtQ0%j-z%z^ptop8< zgrPX{OD3Z=zPBFZH^;NnqwgOAN~Qzk+vT2&U_azH8iUr}mGJ1b>_CkeW4> z{ip_z0*J@MW+@6)tE$)+X9*8Ql{zq=Kiv0fgCQA%V;uLby&y_LnWQya`GN%*lLP$% zItR8~qpJ$1XbCuDvqO|xgGOEpA5_uxbM0`gHopZ0(r5d8$B%wJOc=BnPU}voZsXqP z^fhEr*g5JP@XHmfl773Ek9+qOefcN55bsXy&J}qt-{}P0k^A|)00JyH5$z6nkME>g zaoLX`AD`?QeflDF_q;05Ln=0#`NO!BYTbTN4Gx-`8d7Rte<8-Q%wpvOnwMKfHzI-RI6gO4+q*e4fXDU5W1@gIDQ@~BAT4Bs)G@K;*NyViFH6f z1-4)lUl(LJvcBnJPM}I#U+SjoY9|KGRZ)q7di=MVb=S)^a>Q=NpF+y|<90hibILzW z<5y~ZwCK&r;IHNVC)_Ngk(%{GH0;Ei22vV_&oH`NMe7{m)LG-;f;IMaey?l}`$0sp z1g}^!8P~8N^N_P4rfBOzi7-rYTdEH;B+lIL$`Ek^f{?Lr|;I3MG*^{k25; z`PQ+dZU+>#8Tbq8UyrM%8wMKm8gQB~>6@ryj9X+MiQ?|GH#@^x7hfbNK{GM&A@KO5 zf+1*=_6&kWZLn*Mdz2>@+(aZ0tkC^}j}_60!X|(jt&DK%32Xct>LnbGEGjHOdBh_G zK#BdmG>#06l7OGS`zf-%1u6}7ct2r7jK*MrhL{8h$WF{tQ&WRza5PLr3 zC>iWHDIcs~zck>4+G1tje18JX`pGeEK=)jTD^fZ#Ryw;|wP*fM7QnMDB;H6I`ir;< z5piOUMF)N+VD4cbS$jR|@w941tmf2FB|%+gT<@NYe7o`n0`kv<({Y`%_dh3~I zj!3&EobSI#_!6ndR~tBU=As_!UuEB!R-Fr9#ZX>w`7)$Uqq=(;um?5E$rIiEghXh3 z!t{52OX9$8Uj2M^iqvfu5G!k?+^yh4Rh`W*bf6ro-Ab}>vE$lemqX~YcDm!@6zX> z$Pt_~cYv(YC?Z`he5(H0H$uuq2d`#KbUEg5O0wjw@b?jc21KUgl%fV{ky2UrGGvp1 zE4Ve~x~|l}=q?@}&5ZV%oDUDlqv@?*``+vOG5TtA@oVJzm(iK1?ahv`D5UAYaDQsd zVl**;UI0XZTT(C9gpRDgUU)uRIPUDfEc%E_VGY5n$j;N3p|4e_LHGsV2xt|EEk$uiKETuUz^lFibxh3k3r zVVAIW#&BuI68-R*LZSv;o(10&&`a+m_DuwhG`Q@ zm`a(A;f-6;mqpjxVS7dKvAQs4O*tmHNO;Ku+G`y?Y22P~plg`JiKr=kA5sREW(PD? zapV1RL>wzJDqL34Y2tREfZH*1R>8RNJ+rRU=!YanB>@19n?tESnuNy^Tyyi3lgZ%2 z8!y9qdw0SGXdwcq)Vd{f3rwAj`=WhmNIeG?TaQZ)9vg|7+#gSeY^xAd87B^2bZ)w_ zo1+n6KFtNL(<(eh>ueDjQ#Ee3GRml@1B=#;PNqxxUVSx$n^&daDUq!c_O4-lijL5+ zmT$aY?svJ~qIyD3xmyRZtR2R80H-41U28i}s1sxpZ3}aPv zD)&jw;fJmDBaZl>hMV(lG>kP!_XqN~kB9V94a9Q+GDl425aEwE^#=#0AuL)?i|>Y3 zQ;Y0NFPe@LZHs4$FFwOYNo(D9#6$gB=4nBTgHI=CBRMYdPHD=?BG#)CyGxIejw`K! zaPZTqb#7BNqkKk`+JwIv<#PL~sVwH}vM9g5QRMFI&y1UqCQtp@@Pty4QNLJxSA;c= zW6LTb7*iqeuF;KjHId}9(tg@-?-nE5t`TvGc;LiEz&Gp`YW-+XXY?@$j8W46CG%2u zqiV*TsCeaO@AV6Rr0;`N3*gc68vHj$)f$?QPoDhEH*Ph_pWt=-TylPQOV;DN1}o05(fb7kSnbi+fPS* z^=GicDzX!yUzAl(Rye>n=R#Ku_z3|=;nEA}+dSv6t_VQEedBSQ-Sg?(+}ufxwy@Xr zrV2G!%?f~;T<0I_yTt%K{XQXoF9uiu++Y+Rh$lmd0z4cb10r^n)C*ilc07smAxb*} znSUFKqvG~Qs6k>~0R%2EPD2T#W7GEdH|)#|iR1o?_@d}0r!f53YV!GaU?Y4q4uHX_ zMsCt|+#Ljqui*L<|$nEx1J?VoCU^9`Lkcro&hJg=zo~f)JlI{MVYNSi326FgKNEQ z$E!7p%R?I=3tcJo2f}gWtmr#G_~WClz-Ch+#q1e*bYK7)u(|~(l}NC+2jN3j-Xg)U8U9AE82h?LUZYdO>RiA5PJN~SPgK4 z2vU8qRK7Ltxz$#^N}}HEjN27Vjq-|{p{8-v>)jJyc(l2dhHx;xW;~^J(Z<- z4bJoa2q+TNJW|2oC8zlZ_~QNqqvG|H&@nJfVel-80=0%hnZ*$R{|(rt*kX!sCF|Y1j*0ePtCghKxGic>yTd=rTm@v6P*+dXhRxI;Yy-CGu6im*-k(}(J zrx$6?FKfzYrWW1pi<@nQn@hjF%;Y~FbJ&m(T&Z{mCJxtKjL@l6g5l;wU+V5oAbjiMq%bTzAqpRDsE zu6EVy+qyI-XL)^7Fu=C5a!Cv*Rfb@QC&QYUujd zU5S^a=vKv7I^nb~KK9nMDnm;L39R7CA0{3fS@6@zB>1QRf1+IxX#TR1x4Ywdc{x|K zI;)n4Y=-CS3raM1V~0xWP=p=BfbAJc>*{FH6xkEezk#N06(+1#~uUWAxqsZgf<;m6Ay7DF|3HcDtI%2AFhG+TS+ z2~G?1PY_d;oXWbj;XLiAuM6zD%|d{g=HfSP%nnsgPeJb=c5k)RDb7n>U%ym2_L~;4 z*rvDOb)5dDmE`~b&Q;pP__c5XPeD-Ze=YE;{GpsIhz^thp^jdy+2`tS8Z=Cms1IMh zL_{i8C7^EdXV{c-2z(9L8bRr|y#nypu7J5c5EPE{o~G;2>+~352|n&>i$+%GT@r}> zH)9uIO8K`xOPkWbTAnCuvx@X(Y*90C;VW67TW!5`3+rZ9S9&pR*&8@6I0q~$7V6h$ zOEFNPe%w*MPnm=jNE*#JM|^;7CW%!xv!BC3M5W&LQp8X|F^eHrFDIVL6AroF*Kpu* zpp1~pc(DOFXC~W$+*%|qFnUV+n3Z(3LYrr3a4`2f9>bqz$IU?Uyv7m~A{-)a8={x| zLl$CzU_GyM{k|&$wBzdmWzQJ{}MX>(RC7n;Aavs)j|{J<9kF{sb)NNkIIw(&1?qJ7=kxst zX%wG=R0oyUyGmK&+C?~>XyR!XruPwjciepW#d&{XVzm)2FlQ6vtep$QuP_NQQd z?60TB{ENfq^K`tD@afrm!{^<|`y4d;_P9`8cAfpLAtJ?bLE6r-X?s2Wd0bxdfl~0n z>#Bl7iG}GJeTLH2`p(riPQsZI5c4Pb%sZD~S?1SgFv6I`QHWh0VWnSJRml$dpS_J! zLM1c}nqRY@@DPG%Ae6c9nVKfj^wp+@TFRpCh-o#m^(wsoa8h{pNTO&i zQ_)YaN?+QlOL2F)^9t7=Q!Y8V#gajEn|$?=^i)=~Q?#nqD7 zX5EYU<#4t4FFa~O1|_>k3YB#=9du|;?O=xad3H}P%yS9po2U(KON36n$;K;qs@)(W zxnoB!!qQ1mF9tP->bTc}t@9T4)5yCw-~qKwb|xtzWMpQxILde;CKOC`hoT0Vg6Y?T zDrGKTg8XaUH@*!mXEDW;FO+a!dgYjlGF_FG%dqq4|x^R`}vYwT!fYg(nu zu-uT?#U30cBNIk!-uRITaC-DvX}6UvDi8k{?R9hoY)pZBe%@d9dk|NaE@xS$o9#_! zTA=r0I}{u)`LBKcn{*ie?@0%ph|T}%WL1GiDk=v{_=bu(lc_hLI#|g%nA4b7q7X>k zuQ%v|drlGp9sf{$%aF*547d+`h@F&KS0HOVt7ElYo_%DpucbfUX9ro1YKI<5{BUHA z_ZgrH4MyNJ@%pe}KreO!@n?6zsOO;sGFiw$;Q=GEZFwF`@I3&7bd1afMz>4P`$44) z#+LcaCa{2+0PX>~(-!FYm&+=CCkM08JG;xyyF%Y=3L|*f{TTc{F8~%t&vH;>@v`Eq z>8`Bi;#CqHGq2bE;8^i#0kgSQaZWl=X5ScADDbjwUBbC1B>z6iEEqFxpnNMvdesWx zG(HfQg`1bsOCUn|FA&L%|0^%2za;t{Tqlb2|Ioaa09NI^S1=1@GH3D{#=1Z5jX3-c z74?*TlZa)V5q?wIOxcETe5E}%jb_9yow3^_)$s_N&SLDugKGsf~$%?sb9psr-=z|-p0D$1SBOo&C$pdDW&w=k=>HvS+5RA6+_vSk1;IM_=mWnlYMWa8 zBSO5ST1VShbpwiEOh1g^6|&3XTwX!d!{>$cTFJyd2{AjMw6$g1{WuruJAUB;{I92~ zKrT-&vTggV%I-fxyrv|-5N0~iY_9LhxL`mL8LZ;lrEK!;c$#D9a2gGOO6c8BDYBw~gr^Jq z$5;NSsTpx64NMty6yR(d^R+Cx6fEi_U5F#>cv3U z9A`__&Dg0KG5d&NV3g5W=x-8vgLUT;X8qVoy*NNYM@|)dXzg~)6X?A0OK)xL9>+%( zBo61Ec=`4jX)aadAHVCN#!EZKK#~{GF#P3*dgME}M4B9U5_EiedL%!kZsG?$nSbZ! zv8p=dfK{r$9$MEc_D&>r&bu)t3rRoLrm=-J~P4u z1}?(W0)fY6Af0=qDRojlo08f0N{M$NPg*%v-(_bE^-&%(}`)IS~Q#6pnNM3`xPDaZwr7bQ+d^tsDQW7*s?#`%0Pxr`Vksm+^?Ar4RR!WrDPhvuxG251^70zw)1_zc0-XvRqZjMeC68}w z@4xDkf5}$~!sR7;qo@BTt3ffR1Ah3*9tdoyz*idX;4(qtKpX} zg^8$G1$D$tZwth2H`L#1|2=#3B!6P6y=AqP_};(-fgcT1?Go2`?O;fX##9DU0zVrg z2J|4Is^mLjeC~q)9$hh1IzB}vhX5kQru;{te0}pTp*#U!q*}$u)_jo^{z^+qM_9sF z=F^m69!OyOEqu#EJVYy(LG{i&Ooe1$jlrZ5z0N~C#mIF?F+69ALOTvfuPO1oxP2O; zG2e@PB(Jn1RB`kd5X|x#u!mZO;5&-yBD`Dy@8F7j5&k>(>p+vP#c0|GF3$^#G(nHr z$yy{M835)azNRo)+66$!40eVS$CEm8QL|#XKB1LXz~?*n>HnhYFQcMxzcx@9n4xRv zhM~KqLs}Z7Ln-M{MkI#@6%YlaTM-nbLqrLOZV&}Y>Fyp<-kab5dCysA`N5@2MO}OD zz4x`RfMJNTc_|U)LdnLNJ0ff}W+DZni5xf}4Y)i{Oq*8D6_tvs#o;MZ6!Y|@G!rf? z^yC8z$NEBcC0ETc=dRvLMrHcAzQOen1V*$#Q}RiaM2U|};9~>U<+6UbOYX9roYD$_ zIvlesFQzI24rTh$c$zTt2~$-rZjASAX2R>dnqq#bD_Sr@vQHs+OMx*?HNMHaj~a%Z z7eg`jH<$0{;a2@Lj`LVz`*HfX`NS*>gqHg5&E|g8`z_{j%~woCg=q^-6SUR<gw3-GbQPdFGN5bla)bQwM2)@PXpM!`W31P4=nD^hKD{Tgd55w5m46D2yCySi zO@O2g4vW=Xf9~-i>R~edd+6{(E{_Q+u2|P$Hw^TBNmNK`@c@uZ=K$Oz@{FYV*}s1% zKP4M~$%_8|ZGDDi!iUwL7{2U0B}+|B9T_b#=4Dq+#`>aQ0f33(k7PZ7H2a64)_Jf1 zFo0fOpuck0ny$D3)r_fz`IqZ^YT;fKW`ovRo(WawYc41Kkn)kv1VzjW}{PiS3kXLw_HFkp-GlCUkp8CdU1y4uULF4T-We0S;c(H@bwYIeHFU*^Jq5N z?;AW!Rv{TrznTS0>JGSyRgvx39yVf3f3W)v%cmx_j;7S^Jn$=mK3Whhx>KV3Qi}P5 z*KfO!S497oop~ci4Y7V36t%PduDdr^N5?=Qa{u>7J-3B6f*Z+fR~JW18>x30$>rao;T{b15kA(p%dPbEbeC%W2=E%2n~LpaGC z$X`=<@6}EvWgRI+h6}1V9`0)+M4#-hqeAp{r!|`2-JwvD8YNtmTblgycSfzqDX@mV z%&*T83ehAn3SOUU$ckthj<~evbr1lvP(D`uqDS+CfyUaL7R? zP1r;tIEa%HHgBFes$qm%4J9%12?>1cC{)ufkJ{TM`y;8#{YcaF?@Kn4o_iK#xXX{I z-efu3HJlWc1FJQ`wvn=D3+Wb z&y`9l@|$ zQFtn_{zM-8`)Hu|lEg|Wirg~l@A;L)R%8IB->sXWQZs`Hr@J+9UuQTFz>d{ zqkgm~9R{4#M$0V4fm=hwjZCXqJF$|@uFN1En!FwYXAji*GcVJ7DpG!ZNobLr-uDvx z4N%W@!_=NB@l%zm%qnn{=DOC!4)|oZ(d}V1R3b?7T2BN#g8FMQpQ&nbbOCO7Z~pVO zXNni~=C^8#^{8*3%AMw#RA=4qy**pQved&sXB@o^k~L_n)QQp zVzq~jLn}M>ccvruKM$b%F#0s^yIozk0vL*;hCYlYxk@lnB?dUNDKhX_hX*`*I0G-O zZpJlHc}L&sv_Ick=Q7Ov9o+h{OL{IWfHLe|kVbgr-^p5+OXA&xe?sqLAfYTTt?rXa z7qM5lb!u4c?`_;!Et7n<*N^jtNjlY zHCq^dOlTmTchYvF6qe=Qc-$invbe!Xe!NXo+TgMfI(5u4Q9qRk9bmOrQGPWd2qoiW z`}G3bL9sYX$lQNXNrHHqS6l!rOHPV>5{9>=Y%0utAR-NSa#fidseW{iPe6c@u8aNU zXzMF~K(@z1Kkrd=teTwkQ-X+ENS={Tw{Fm(lVa(xl)jb@JpH4Th&cl&XIqmm!KGQR zs~OXwXM!a7<$Ep185(6tT1O~OqGcwkCO}6522qRcp?9$~TVgs4PfSAy@8t@pM*dVL z!mv;JBj(WdLH-G~l7iz5`{mDD<0%?QL-$=whlySvAQHe;iUIyz%Vi4KKA6-hn@wCg zBG&yQqL#{z=h}fO{mz}6sOIE*=k|x*TrLlEa*%f$4Hui7BFytWo2t-wLJW6(R&~Fx zq*?Ru^2SO&o^hRi;yc}27{OoJ^y|q?pGV`SVo~F*$8F&=Chyk zCM8gRs~*zR@D=zd@rlEKszqu(GQU&GL^mJGyV{g%cX2R0lLGcqyy||X;SMZPbaBHkInbGoVq6d2_2jiPz%nz%%u{rSM zqXb{Erp_5U3?--Zk*Yg!UGnGX7NluCkRN!oY8CIm`EawS(!dvMmC z%Ph6vO}^wLr$b|PB=$G>LtwZ;%RCGZ!2c`*N6cw`KAy`K!n2SgLnf^R;|bX5`;rj( zxFx}?tux&%PbLI0m@(l{U;Jzb%@w&w+7)<8FsGPHuGsEhTJ}H=Ph4$(%nl1E>;pfl ztpd1Lz}TVlEuJ(e6`WlU>#w|pLpN*Sq}S|Kqy@{_vBkM7$$f%lLEC`oji9bE7MOHa zanv1meELSOtBw%m@h|v6@@dV%D{8@l(#n=BK!w;8?igVr!U#Hznb;YyTg|`AC_i2>FO?P7Ylml%M1ht-lz&p&O*k&m&^ohmEU>PMYK8FET@+(Iaqg3 z5O6H%!IVI5{7Ija%M88uB#b4eOx&m=BALo;i1_K&yX<|h+UxM46(k1x6oibf%>rZ*_%F~g$n;GSYNTSYH zH$>Ba{$hD{HM(?ft6q0rT4HTidr}^b?#{SjXy%H8hAWKiu+EGo2M8ol&K;X#fZb|3 zv!pSN79JLbU+$4nOF3*nXD%H!9Oc5(&vqX;&U8z=u((%4@!LlHl#=bzjD^o|s^gO} zM5!V2bHr6YnPFH@N%C;|eLm@g)4mNitj;f+aet~BrAi9E>C(&@EODD4> zdGuG4^uBc9TmvGOUS-Q_tG|=g0MC`mr3pY7>HW(*(b>3J5XrC_($^8x|;W+(+!1tTnJwZ$Qv_h!i0(k=IM${ z26|vOj;r$dCZTU36O3{pKl}{Yg#A+B{ljm^)Z|>JkdKv~z*f&5H9|&9lrkLa~?F?uHQh=@HgYVqLhomuw z+%@?L+5qLqSGvx~gg#+K#p`3^pyi}{JQ5PAe<~h&On>^%91~6xA^uugB;c68?gq_B zmSgZoKZ)?|=OQ-mxZ5tNa{2E6TXvlk>BhUK4@T~z<=iT2`)05tt35Bs&Wa?$JYObseW5*ym((OG3G1rKD{glWX&=IsM zqHLc(uAGb5t@Pqr&_b$UmSukG{fDUsJhp^}&gF0{wT>@!9>3Cic^B^6gj;-v^_w_8 z)vyvz=MOixEIH!ANYn?nC*8ewIW@DRwZnLEP>V*7aiQrWBe~&iys8u*d;9l2BFr)k zk`?)04L^puLjL@e6?r^CvfKxr;mex#Gz}}hyA08V>n51NR-|vKpd4%RG-A6)GD|iu zpa;3@zwTGxO9Qr;eerh=hQaz@B*s2U=#Js(hqVj(aL=-j`gZx^kX-}rDV`BP%q!p) z*wO&&9^wbcWkGw+no}tgjI3ZQ&SSzozUgoBU0gsh#)aauwbk%JwpE-orrhs!`2iX? zzwxVYJ2~l`U36rS#r!2S?gWXU({u!i;wY`GJLHsMRo(-ckSXr+OYrNAik23zhFR)D zagBbDLutLn;3vCf7%y5mZWS1(8U}nM!<`L4a|dlTf4j4|Np@#9=||3}hlcM*Hp8*o zhvp|+8L&O=)EP4as7mn#9?+I@<@B)=tQb7U-Sb4E=p*$7?!TtjEmcf}*!_8cEZX4aB= zg(HQ5A$qW*gCz#6y%RiF4GC44q85>KzfQ^U|0WT6g6#4Toj`6HA#96=ezD7+4Hob4 zP^Wlg)f*U0+X|1IB9IA5y}eKWY-pIbQqK@kca3&PY&*(f09NUto^o1c`8}CY!b2K( z+C#Gbiq`LU>a~z5G$QbhYDbx^zRVnrsBQERihoz#Kd;vLo^6+QjN- zSn2a4PG{e{H>zZ@brQc;8@+w+VoCso@T=QJRNYQMlK=L*QRiFr)Lj8A;5z8GQQQCe z-^>4$?OgO*?hB*wsQ4{s1_>ZTB7*w)>0>k z#1lOv@n>$aYp6{MS;ug|!(S~@gng(D$;N6R?UXw=z<97NwwnM?KDyUYQ`AFUw>6W? zjqgBNi96w$A5x5FZ%TgT&Yq@U55W+f+1Cee#fk3F+3gIr@fUhsiisAZLnq3PEA+j$ zvIi%H$(R-0aE~gcV*4j@^S$}8N4!|@XVs=Y4vpL?7OT40$}hw}<^q#32HVp{91Zm@ z>(f2o9Z16|;Ses}Z!LQ3sf}-TlNnP84fS_G;=26wo9k5A?z;+w7mF9YRSIE)OUx?# z`lt}HaAYx_+DCwOd{6StX4P65Zfox0pvTcHf2Fu4iXUML-HyX~_6e^h;h};-1YmDs zdb2n|X3~M-7RDwvyjPv;q{c%Gno5^KH)8f!YPvcox{RwfM`QJ?^&iQ>iAI_MObi z1$C_do&)M*BW>RKeO^O}8s4kP03E-1#YfYa9-X|wy_m|*vK6nL8zi@P6wHJJ)4OcW zu{t|CexB}tZIL4MODO@n9oLJkzg3GUr}PLrBmwocoTap_gBeiSj9*_CRFRNg57cpxw?t>&BH*+-tX?dm0ySMQnV_yMOj3e0-@W ze>c~lqDc&qwNv4)BY1Dh;#clatMuz3J+QcbAT`d71Fs^Ajwzyw40mXq=R)5mjE*W^ zdfu1y5;nPQH2D%-oH~KIJx&bkRp<^Favhd?F0KLJJYYr+6K&NM;^^RyK?inMBaWC( z43`B&K0%4I^`spJ`W%2-g-bI`HKawH87JGSU^ghTwjIg|>6z^Xh6kXu+T|wkI}!&J zmz4oDC`J#*Q4-H)lkY1EN8&`kvkCX{!-;kV$L9H2W&tAJekp5eSY#09HdleWO>`|u z#cJz<-82CX^?BB^$M1?B7u_3zz+FK_??DYoW_Kt!vEJ1BGkF_PZlJR*{b>9qmEOjO zhik%mWwc>x3oy#xe{D^g#90XzOL*dDb1~Hi-aX|~!^Xxo5qbBPLkEWph9C4+vODwq zM`h4=e`mk%-*Ipr_?3p<&B)u`+}s@hYg&8Vve!Jdn0Y5bkb!1E_QII`XO+F`VSxPc zVEW$u_AY(pJGwJ2^Oh;4wizAGr7CjJQkmib9-Wm-FI;K4S<}|jHuXDexo6OwGjBy# zW20z?1Omj~U~BCe{%TBC0QHKn#;d|Zrd4_4>U+IQHWK^8bxgF;GV0`KxMB}vEOwwV zNbHiUg8w3UnsN@5o9CgWGywO*#Hm>vMCEXo`O)g&Bc538@ZgpLt90LtBM?4Ag|s_C zDdYGR@mT1spdnZ{JS^R-f!!?u2XX^1=N|1wzsW$-Pnvg7lbyXpsjb<4DXMcpn|%-H zNj9l?Aaen9*hM>=ujwOxKS$jX6LivOEmh0zpjsDvxB4gz^vw+lepj4lmkD5#ET1{J zmJ4``$o||kyMyII8L{J0KN9x%Dna@q38P`uzwxN74T~jO?Gl%pM&ojchdS)< z$bRCf;YKjAT&p4P{^h3k2=wv!{9FC)HY`_l(@b9+OV)e-4x}u59M2UB{!~0nCTW=Z zyTSDPomjX{@ZH~Q3r>CN0Ada){+30mlpOYaQ5MpiEQ3i}2ic7D}jqidPUL z#kw(c;;OrU^(jITOZq9v&p+2gT4ucF(ZnUeW*Lc8!}u`tdcIsqWf}cG@JTb_kG`tq zpRIq~5RYu^+uX$VP@35z4@q(AT>RM4{64LBBccxz(e;cIcTvKhyXrwy_dR7vW;Ca) z+k>O<a$#wHDKdMx5Ei3~}dJ)60nlSWd* zun}=VYfd+&8Q-Z09r(oRw%76u+nnGdY0Wb4-``$vsb8rguIzBhh{OI!}J%%@v&?Y68wZ`*oQtI%)eNEI(@Tas;&QRH2(#i^v^m3 zvDI-a(|tKxFH@KVkFP2{=Id|`C45j=@1V(6oJUS3uJhpw2$oor779ezXg+5JTPi7e zd#RFx1v9!wG1{}q>!8&S!{(u&Da&{yI$1AD#3b=cY;fz|R@4-%-8ERg3y&_t4xRf0M zbN8}NkfQBqR^f7?j=8+bN3~by_!3`AHisr0PRz$&ddty9C)Nf<2>Z+7)%!}PlQi7T ziuUNU0hDlkWzsreTOY2P+K4|ml-JT@S(P^vHiYkrUjxT_ zA7+!d=ssO+a8q~Xl<|Z5GcN2uPl0(P6G8QKV;8z(EBY0};S6p@2fw6Hn6-f(=8Tar1iyzEhjw zcV?kZqJ{5*!3rAtem*y+++7ZXJgV0j^jHo#Cvc3Y!pO{tggu>BS?0%xO=>kdhl`S% zXvWhfiuEw2QQFHT;x5|2RMsZO>yr9NzearzY9)a^@JL#jW=40NFJ#sH!RL(0a6GR4 z@kM9`>+%hQGOUNq_c^y_An?IRJP?fGp-MOmoKVHq5r0Grq(f5%p6_w{*kJM&6& zgIxV{Eooc(?}7*Pj~`TdZQpr(b~0}3cDz0UGDIXnL_LY1ZOhdiW<4n;UMzid?C{&Z zC$A1xu-uN=EcAMryAu0oA}j*@b_hye2P8$57uqBaVVd=L=rXyF=U5DX=3%7mFvF^l zj@_1+ZWlpR?hC}h@puxXt);6Rh6&<5_ayUFIMA7&5yz0=T4;qwF)H14qC-D4SkcR#uM$1_Cv4Ct6wf4tT-)*m0{<2k;M}< z;}0d*ida{l$0AeVc=c1gt>?u!$~?ea{vzpD&YJnQma+(8q|$a=B!V!xu2u)I}JxyHL?Ht%6n<3>*_dqLEk5bZr* zk7;g#l$l!TBZW(L;dCx9Ti;dEvV>yL5^tNG6IKMf+?$as?mw|UV-XOi#kX6@qqpxH z9#-X#f3mj$s2adX1%g8qqh0xIUdhDn4!x+YMrH)0uO9NzB=p=Yz;ecDik)4Dv8J_z zq^30oK}tTI+Y$QXG)&XFqZzos!9`8*SjCrSH#(8f$(J8JMUX*_+eMP>X?A98qHKwF z`fIvFL2&uyddO`2@cXVrr$n*0{JVpbFLZ}aBJG}XM0*0agKS~8UFCP0V}v9T^D12m zHLw2Sd#PsQU+j`GuaupJ1pd7EDf}OE78B~;&}T~e_rb9O+X>bL3EZ_UNuVyqJT=~} z*uBGqlEd!)r#NB6NB(4m7oNcm2Lu)rvO>}2&$7%P|NWkNdnf`y2aCWQM~lt--JKqi zeCPEci=&!LHO&xisQ*Zn_ zJMEQ6L&-flmjs|qg3s27Nv7pPLecUEtKtT#MPD(+k?4uTs!R~gye4BuV&x*O+#Vf3-K zPU?lHHwPUWgps6ciY&S2Af5!w*a|2}x@23;_SS5-%NNr)3ggC(U_~bycPW(8RQcCj zy^NM3dN6x208A(FbMeHmCs?$ z3_bM!s}2&_6ehWbr>+NFK9^yWP+rW`5r?T3v4$(ryhcsFQF^D>TGd*74}JD@!!;D>2K>&zz4Dr6u$1`aQEmh?iWO7#QP{Z!;wYy0pYq}1_ z->Ygz&fmWM{IX;X;w>efNh1Pg;>~_Mp~-#v2=teYR1P2q}u-e7hJa8ZWT#X zTS<$FAhq;?WDKZV#2vQ7>|U8HSkZ>^KxW}r!6+{DZ_GldpEK5m0|nH z;pcO5V*W+-kQ~umNm5?`ipg*2V65X?Ph%cI^(F(%pVz}}R%#Ab8(`(ifJMWR|M2^> z_rPAlXlLq+TUR{YXwx_Oez~{xzJJ%@cK-Rr( zQQPHI5DPTwh#G;D6hD#ZS}S=QK$vCl+K9~Bub_m4jf?yuRLG%jpFcwtSr)>^+tmy- zg~Ye(A`Cik+4Eo!Ga8)sLo=@g?Hum^!NNV8VkwNPyGK6ZGRp$iXT!H-cqR<Q@A$(@yc6Y889xt53CShfF)s3hoE!UNJ>KA0n^9qbp&xZ&17uvO^DJH^?xho~t5>S_FnGM~vJN?s< zF4YXXIw&^4J4Z{1~GW!O08_M&xoPk}5= z3gibe9+mlTqQ-95Qd^w_<&uWcIvhGkA2HX9S;3UNe45_J-z)yBp4$nl5Dz*Y#u=i44O@YGk25^2A^nS8me zyfZk-Bc{4i)7sNOQxD-IANU70!5?$-iL{m@T)Wq|yJ6efWW5L?9|wO`noIQiq}dX2 zU$o(nWFqa>m?n{#Fi*EKjD&mu&K3Op2gCRX=YhUtE$LEHNJepD2;O5}uJflGfk?|d zInu)!{SF0=)Jb$s?O|>VQaE@tdak7m_P?DT&5UwSY_WtT>;PjyCZq=;O6d7LbU83oTN3_;Cg@<8X z(#_<<3(qsSg;Lrai)obj=6Edxe-jnZ#kfd~)$7@lCSxLLfwNWJOOP6i#W>9h*D`I! zXw}PG!i&V98zK;@S$wV+`?gP)d78+6s5BgocdwKwaquxf!K`-RcvyZoyN%ui%j zySX9{QqrgC&c-MI01T(utSfW8NaJjpq}uy8|J0uZ zKwKnCCrOH(St2_$A+ea`W4(ST4$WMfCL}xSE*Z2n+Elj&g2;E%76D6tbi-GU4616A zUd4gqX%e{4IF)t#zl#7L?+{>0&`pO{*x5Sp)xE)rhSiW7uh_hPA&(}7H%x`kyK_CE zsE1T!#IYgG%|^1n5usNw?cR9KXE;uOZi$>5=@eG>W?f>=EUW=JdY9gc@JtSMsB+c4 zKqCV?%kCEE!g`1{-ESnSwHzaj2D6{~w7ZMienE)}@Hgmwn<@1g%6@HmFSKw+NRzZo zDlG_#wMP16mB2CfW|2GFi4J`)2|>E3*b;$-_<{FGhk5s5eCpda0&$T1jk$?^Kz@Q% z_dXACL)DeWo^d3Nr{XMEMLNt@N>hi*@lPsD58h6w&W(aI=5-2m${2vf!9uV_u_^^Q zQ;J)fSq}jYv%!jG6Um#5Wn3oCBvB16H8QQ#>@359mS9dbzn^KpXrtT=pEj5=f0lJr zbHyh<)at&e3=8ediw5XPyy)2O@)#BJ>IVyB6r6(BhnIlV3lEN1Mz_S#)dhnuX62Z{A`yCwYNOPl! zv{rL(*a=Ql!`HA_?wCtmpp1&@&~z1cOX}s}ivF3yj!1|{7F$=bMZ|YtWb@+5nS1(z zC2Xqxau_c%dfiHcE$Ycc))JcK6rW#oa>f7Fof-d%+qu?x zgi-VDf$0D@&GdMC8e2*5(C0L0=}M;ZopPAkZ=5 zS-1x6SsfPnm-DK+Gq%AtCfVq|aEytQMkn|*W|M_4ys1_2$u|_9hsUfED8sJUfY=Mn z+M@N_l#eR;^_@wh=1ASj#6G-5RDQOxBS-Hb^w*~Iq@LJu;>BsToyjR#4cAO*bv>QMPhQiEWL!guE@iD$~l^Svh?xS;UPC}3wx zs2kUrcZfepH+!2y$3L47=km>*nqkxtowiRt{FpncnD}$mG$!U^j9fbQDzu0}c7>x)H7Mp+6*{Fd5b1pKa^%Yv}6V0BkBtHzD>CDA#)6}gc zw#&}ByYOW@1@R=1o1Y8}WHWpmFSPI?nh@qyoy{>FhCR!=HOn zyii^UHYWDsENDIm+g>nr{eAt8gbO%PwtTyOU#+g03t?D;;}z4umrS8yfW}v0p$w~D zTbRimB@8dp=5iqs`{+Y;HH+WDuair-IM%y&t@Gd>_T<^j;&RsS2D_K9ZEWjdB74ap zax||f{5f1wO`XjaETIVDZTruYk?z%6DlBILOJ-zcW=u^#%V zc#I2`N{lcjeV(T!JEH(+HR1b zMt;!$zfK2lqFu#qp<3hY0OCWFpZpWc7P7>bXtVF~>-CU+9a?GOY zi-8&SUME%+l^dG!FSIu#B{lP2JI>dSpSb(i&dL3;IIC3OFDbRZC*HVXT}wsfZ3?-8 zW_Vk#S{m>>uJg}Ih(&N@z29-n;eUSuej1hk`gVlBBrErixy2mJ$C{KDgx&~0xocz6 zE@uf+%Wa$ZdvMqv4W875fq^fVeqQYOIQnK%gF#pZoP`bNqHx5i<7u~RA%o_H?A zzd895zI(Kg`v9NrZ-w2S`=9%L^|mf#b|*QNjUiuOj;%sCbgtvv6p*FcuZLM0DH*H_vAvXy? z2?cKId1KHoTSeVBv9Z};%7Y)SbYLpo%6KwRPn@70Vpf$!`^l-*z078wv><}0xy*Z2 z3iqm`@CqAR!m-3`Mp#y`82L1!QH9?!ITFqZbTSrR>PB&+ggACXr4F{qR7k)b6Q3-5 z{H2!3XI8dThEy9?ohowg_{$!9y|-OdC5BJl2NT8V_A^sw@@buEuqun1sOvR{ugP2N zbZCsYbJHlpCK;h4)W+A?Bx~(_;;{`|>Q`?hxkPzdqhij%zAk3Ie!Lbj^v?d;al#;hF!Z7@`PA$4|9V@03KOO1{^ zvW!$rFYSy51Kr)3>Ov*#)IQ!fakeUHZIT%XS8_eS z92k1~rylnSRstBXvT^-8XKPeHBzy+9Mw|JHpY-UW2d5?qcn&f>U#?NXu*V#i1K)c< z)MJ2eJ$k<;NO{P49x{2>aB)f-8~%2GIculEZZ7K=HOa?701K4>)b-i;Lx0|#5B#YQ zpeB11B>D?wPNEgx`X=)dg;`qNb!YgoOGDxjd?dpes<79oh&h_2XTFz@Xqag_KJiNi z(Sb{mU;s6^AWYCLd#X-9I>I4|jL~(%X?1^$ya}jJJaDcuX^HG3;#XF}Z=H0!fs^|OU+b)Di?;~ z3uK2knBzz8)BVumaM>L+%Q;e#uL>cx?rBH{*tNzyOXL{eLtYs%PfA$R#-P=6pKqG- z9yNWNK6!UzY7Rvbb2RLntLIW{ARAsqBoV3n=D$E9MAT3z0K{$?-sRG;=(r;qp9;aP z2uQq%RS?Nw&X-8%mc{}Lj0o|!Nn`Wj(0Vo3>3hsoPUCz!W5Yr2)a5tM%@3)%Pi9`e z4`b2(`?3e}!-;qhEE(mt-}7pI*E)?APEQ>_R(+HwcRsgc+jJx&80s&|Bom+(#5mby z{H1t)OLS+~tuCxPbke?X`Yh=9NBv#m1|ew?kw4~|B`F5H8MURCHSZ(Vlj}c7<0pIz2=SFs72r8j}Vgnam3S_3s_MPw{FA#-VHDJe};(>j?*j1z0eJsi@@nGR?#)1BY+ zY31S{H*Z;P1kyOM*x-I4&-QxCVFX@gEVFh_9RI?|epd>1_}0OBrg6bs@Q4RZ z#fHZQmIeAqsi%g*nznd6RPueb;NV{}L2?Fz9$;RW6GtbicGdA6WPPQH{`u5V*EU8v zi;xpf^uC)x#pG+kUpP#Xw+9E1!FzAI-it!Q_s@HXolC28>SZ*(Hj9ZwyvzN!2-0tU zW}M!ge>!U?OoL8e1V#A3i2M7HwN&D}2Jj{3dWdHNoIfo{nQxTv0r^SfEiNwDUOT}! zyr^Eog_zCX7kQ1?dFrN+_Id5#SMIZF+9)sn+A)$+nvOA`>9dnCJ;BOv%W*lqm+SeD z*{4H!zyQAch%c*==F-svOLVPMLId`K0#TWP$j6?JxMFgx#Pxn_v2!M}mTQ=EG~xG7 zWYDsjnipDuhnyz}vo=P=;1^?nO;c9fR@C?`Bm!ia(k`5`y48lUa~^HGURL+W!l$&*Xyp**9X_pPeUv_ zdBu0%{n=SDlkvN>RoG{}xZX!J4)$%yp*@uo?)txadX0*^2?-=?KG}4MDAf--pME{8 zd^5<81?9US;KAJV`<`P7We$0Yd5vyP?Ig*v_{wJb*edq(opK_o$ioyP;cpg9%_{#5 zvaZlRE-XFc&%ZiNw|?fNr{53K5Yl4|+i)|c7a{u-0pTJCNaQUGh4LOMc)ZoR;Dkif zJpv;2_gOL?quub}N`E@dUI?AVBb#@qSAZ+`w?|PICqj^Sw#*kUy1&@$AgpXSVRjM; zlf^uVl;(;qD6*N?oyooy1y7Yt(>-km_@#)_>|c52u*Xw%u|kbyct5!FgWpyFttE<2 zxqsF(kv4F8Bu}-gnL|guSUFf!OlDFpP;H&d>g!`Bg?$vn%r45({^EG!jvWf)$fMf8 zNrlIQQ!_IJE-Hi>d%wg&Zk50mvng3TskfxS<{gjkuG69-4IMfb90)3(F!K$#u*O_| zTW1+1d`Hj9Z$kz}j2hr+MZyo!^TB33G|rLHPN&P2ussR2}**mSnx4J8R{x|drvVDHSU^w`x;BpfA+u=*pJ2*{DfdAWT zi=hxLFg|^mnaw9|;=7~9` zJi`*~{RyEA!Z}MQ*-*+;WnzW>z3g_1#$XDSf5E7>-#(>K=e}bj>)iwY^mXq-kS`8O zIx8Nd@sOL;nP`rmZFgKk>W5!n1hN0@ua~u|lL9Pifn zK(b;&0Mzz@w=c!#$u65OdVc~!{V=c3rl@6W7M8~}&Voj_t&*)!ZxKWrbmPy7hwXteaz=~Jk7IZ&MXx;$>&770n6 z`WrZ+WnHAUMdx*=kacW}tGeRLPvI6=`PO}-Zr!i_>Jk)hi|thiPKu3^-BS&CNZQ8 z3tHMGjVqmsy+2+2W6~D^-pvmFQW}v1E4uW?%0hK{lB&%8K>hiNPKfpZlWI4%&vkF8 zP2fuEF`x&>{7ruQHrk~6R9eyUj?PeM&1E#_JaH-=n1je4npwo<&ODZXrQT|`u!$58 zQ&Rc`I47BSDW_pcuhcnq%aQiqOl+O&FwZ+;_zR}@)FMy zO;%>N3=^I!F?&#VB6q%E>B?Mf)@g9DzskAG|GdKYFLdH{@b|xXyY2DLg865?)%5e6 zvpAaXca?T81s+uYbkwgXdfse3J?y`&%;C?X3WM|F z{vxuE)cKH&d?$_@Dg}FTG6=vfdBGRQA%>9qQ`l2RxTb-OkjLpsoVZlTn9&bJ;jKYK z!Ndk6km&;L@?7hQWl~mIEvM5Ag78*JI8{nCLHH6!=;9MJu-qfJfaNu8mEE5UD{k}Ky-2>2L!Y>j7@dB zKKLJZTQ*20QUb>tgmfhb?K;MS`q%%y&zf{s>X+>D!4LC_2a1ElEWJN;i`q4K?YNOt z!EqLgtEjRxpXoPNCl#2A|8a$p{JP}JdB4Lqx9e}z?`y9;qu7;5kv~+=Z`d5e$3J*{ zDwu2!@OIUr68BBfDA1C8xxJx!twZwqbz+S&w;Si_CaX*rXs~&7?c0*_{JygRPMFs$ zH3b75;~xEejg_cNlbR1Ul_rMXwP)8fHc*bX#;MsRx~kkEk1j!5P+QepQiT{_h8L{o zJ}HNZ_Ozcb8YDcl$3eS27~(!q&h#}E=6LZTUm%qdR^ln)I=;2-o1YN=}TO?Y2Umlvl^gsboiVpM%wuv$98LUXTn}=lYqo$V&-_QT3E@` zZ~kMvn1RQ-j@5j{Z@Eeyi<_rku8uQb<77X&*UOsYDf(<)a3wDO@tNnHqCZ|}ORdIQ zuVUX&o{|>$SN_@hPm9a+Em6GpVh2KL5r5??#@v-o(n`vOKkBy++ZF^JZIw^F%5nW< znm3$^)|3!r!OJchiW%*_6gU1+>BKWGh?{5emyF9RD2@!F(p(}SBvpFhTnjD-Z#P>4 z@ZB8;uqxbUDByqU5B}Cx_x!5ZwPFCD?zd$PR7-B=rD+ft7347h`!CcIO6woo^?~(<7!IT&q5W;6*VK z2-^&NjE^_leX>=T_HH1|;GJFh_9G6CK^^KkBXjB}jvb`CuCj0>Sl)8u*!#xeI^N+4 zw&zmVX}EnM;Uo8CMm;pg$?J$RCQ~LxD>OS^fes@vNo_+5fd@~m4+bJsg`777yd|<(=ueKxl*66be=%2 z%ZCcoZ(tj`f}WMBJ5&zWB^b(#^$gH~2jO;~7_-G-pwniB0;nQALKPZQgVbQj@u+tVWx_iY&ON}1w(bvgLS#x?Ee z!=<`;%FY`t(yRl)ETPD&8hi>84YbFXKI z7=QT0dynn#5|yfFFu>3E1QFBHylh*3SH^mfvhlFsln*g}!{r-UYaAi4X88L1oo0M! zUn(OVPxD>2-jVkEmm5za+Lj{$=q<>t%0qBU#$#}(1f7Y8y|dpfwj{OdQOo^@3I63J zE8v#nK4q3$-TTbX^5y0RwGsX}6oLvIaeLX|Q+JRU+?K+`RMcL%9j?ra~+GXmgmwPsmFJu`KA`*7ltxEPRzaD z@ov8Qt6()ND5$`nxzmm?lr;8_xmQ976Addqz@?An!C`pTsq}ljdcilD%KJca`_dm3 z-Ew(uE>Giq1P#6t$K?T-f3I-fP`X%&7SGkGv|65JNeJ|TJVd1kzf{a6ehA!&G(2fP zxroaI=Lk8F7Nx-~B%fgWcio!FMBXl;c?N0;60jJs6ubnwAvG8ckS8;U2#AH$KZy3MoH! zMp)v}^F<#z8|YX0Wq?8}iH4SyqKIJ{k@2R#XwLd$t0YW62}4J`11w%KY7O+)fD1p< zuYBOS5ZSw*k1zMqeqA-nx_PCP5^-_*Wqk!NmBr2(^j*f) ze!GYdLSE?Q2M_|u%1#-)K}{h zcY3~AQh|XTm9veqLN6Wgi)F9&Ekx>SNtQkbVGLS9=Ght}uV0?h;m`aDE1pGDv7=$4 z2|oqfnZI161Hw|~&5n|Q7bS<-V(wxpxTQHwTc)_NF@>sj|4H4=kIZJTp`!BP#(yO| zvVbywe>tjs10)Fr+3qbB8_X@2^q(|});SHp|OHmdq~kQC`VWi%-75^v9B z|5^xUOlr`;<2ba1aSG(V3tuuW2iUH-gCd37&SYta)2N;lIyA1df*}#?5^I_Y(~v#5 zijJrGwGnU!#OUx>3ajTc!wDiYb?obR`N_m`IkG6lWh%euNss4GVB0WS3Z;NT{G`7g z1L@-8xg37A-bTWnmVa64VrU%bp~T)GkN@uM;Ej_Pjw(CDG;+reOTCce75pH3$%zwx zW2Z=KcA2*1S|!@4c!Kpqw|KuGeUTS(Pl|3Kxsfyoo=+?Ym$IBKp_G8j+!@Ye*;c=z z63zs;zer`Sy0_z7Y8L3OUWz=pY}DLg?a4QOaIUH4^qWPybQ82nKwN;4rgNTU1u^G% za`7u#;?LERS$tBPV$`VS3+4F@BtRAw?Skyca9(hi-=HCbkd|io!>AB11N09GDKMw5 z_{utxm$hxIHD4BJYH;?f)4-aIapUgUM1u50^puEe@*2n$GHTGkG$Yi^r)eJbyd5kb z!pF6lso=erdQ?_kepftud~>-E)gbOW*&6=_Xtk*u?gEhw?cPVN8p548e~t){pxOuc zxCK8FI%3C@UJ~=kk|HULT;7yQV-~?#cuK0n=mLctJi!uE2{{z{};d+VfRx-l6@0F$GK~uQ^2n z$cOU4X{V9svSLAsCJGgQ$>wog{V{cO>7G-5Rr$`?JGcMwhH?vKhwr6EfGmhIV$leAROl7LKx!7;5Lr0qeiv<;!1L3VD)&n8wA#H064?U>&_NT&MI#vS)w?lSOdRS+c2aXj|%XcN* zW~>?{;l)(&;ykD2@#OLcL^#`$-hjF<`6PRQ+5pr~T8J*hO%^J&jh##E4oVwE;s$Mv zDmHmJAk+$etky8dAoaW=5W3r4Y2v{a-=x$M>)C+vtU%W_x!B75tTr4O))Otq(c2_Y zjp6HA{(D7x_Jh}K>x#j0AJrrkrxy>=ipPug^8-O!a77hfa%^U@iVzjCGjM;y^;XpS zHygK7=&4k{2e=32WKJGEb_KjrNC-c{@tie2k|ZB3aNM1(26P%l8fN9|c ze#@4U&^PgIBSCKe!<}}~Ra0RH#i(YWtzkL-K5zLYS<381 zfEedrYTChM+vZW@?-WF5cbjJ5-c*0MnI zYPHs4jnRx`<5Gcye>c5`u#m$d%AEVkdtR%tN4BpHnDuJG8#fmBvtIqg%V(dzTq^$w z5c>|S`n1EDee`juvC5-2JfAReEw%qX&Gq{Je|(AJXfLOgzqd(o4)`H8tfmU8y0Wlo zjerdVR7I#pEZSvw*|8J5zCjLJ{1d>ilkHva%^{5V9u&GKcfF}<9o_<{E_4!-lcO~pNM zWv-Bd%S3WnysEG|?Z3zS=9f(=Lhk7H2HAW(WsyE2eshRh`r+CL9FUR3APo|87sE^v zDn)4d?N&|{gw))C!qHrB8=@E3pN;HLer3GybzErjIqsp49tP4d`{|p^o27g0e}->q<|tdxev=gL7lT9puCQPEVL zDmKsaau687Bf+cNk050I1C|=IJzZEv-4=XNZ&oRKhlX+@Ihztew>h=W+{K{J)=2w& z&Mc_Pz@%kh-t(!|l_au5(S|KIHAK~b{UiO4yiN8JJQYy{c$nFLOl|js<*}2OHX&WE!p*F0G`p%4bUUIE}ztjJMOA2u3wP&4pdx!_vtfS=r4%3Xe?J;`mp{Rrz&h~b^50a;nBzX{{ z*hq5=uY*N?cAvx%#FWhSm{lfm;~BVda8%hRV$*{2azwm`X{`8j6Xtm!y_^_{W%?o9ZFkSJHeeQjqzdyVY8 z{YIs~x{s^SC$xRU1?NJC9xrrVeE3(FwZQx**qM@QaNwzK*s#JVrhu$lwi3jb`gnd_ z)Dnr*a6dGVVG0%Yp(R%Y47^LsgHsFYr#~^VJlw!tx<)z9i`m}bN=$qPd|D0=>oL!9%fGnLv$6dn{`9FhxP%Ga84^zm5No;Z&WAwdYbW zaW0529#5J?k3^C3bPx@4fsbYWm1#L`e6!D`D;})Wbyw#pXv$J4S7%}THI~z*61wO2 zE#2=XNz2!Cx*`)L#B0AXVe;9VBizxq2l3I7KH5G3Gxrnp`8*Sx4e?IJQ&Q0By~h4sMdFGtl2wtpZc zP+G_6hJRd|_V)B=J&p|>HN>pusNBd?W+Oq>NHVIt$8ll5Vq|kH@Xz7kPzN6dcO;qL z48CnIjXy6Gt9DU5p*F!kkfL+~T>XGvfK8*J%Fxe!DfN#@obS^7B=aL57(%Sz zK?h1*=l&!xhQ0Pwjf)KB&0l!9BRA;X9pc;>u);+q-YS&&hq=rx#H; zd4KAh^c{HpWAb&&h!Cu?HH+U=4aRIle-mbEpf6K2^gTC|oG;&L)~?wbZl!!Iu>ELP z^iLMCfp4Xxl!qrx9(AJprQuO zt=kd42|ej-mbSl~=~)BG{KE_Tz6xr_iXe)kS`p}%N8?4U@3iSK-ME2U+NH)=bMjjk z19KgRSkD2~=b-CBTw;twaYOw=Ti`_KMYenSZ;3Z(q@$pZg;R6RCi5(Z86CNX2I&!; z!2yk+Rg}4_G2FIOUbz>I%mHvRl?HoMYeB5(gj?ZGwRIEW9&HQBL7iOUic+1J_gLnP zpSjjA-v=6}frxBLD~Y-Owf0jym0IO~9!-s;pE{uyj?k_j2JV9lPkhb0M8}@Z%E`(< zT0cY+mj%=x4lF*Yg}t?}pl)eEFDu)?fkK{c?Nf&y(X9M|0QUySmbLS2vsfsfnOabM zy+P*)G%V&$h@uqMfEYVkX9S>AfrT2QL1CepfJLh1wAPK3!#PLRrAE?kjd(pZerEV^mpAg-R`u6eT$779N zZcCMGeRF8>Nz}45w6`1|35O1GJ1Sk7PvX_3QOLSB+iqzk%t^v-v}Wp>uCD10qUF5! z@>5A>$zgRLAKEhk!;%@Mk0}Pc-WU?chWVL7X7fAbkkmJFjmE*Kz>)(vh3Z%SoLJzX zrj-vY0lYtpN*z}1dez53{}T_3E!S* zwo0yKEkjP^{yBPp>nu;|t53S$$jFNJ@uFy;<(P}U!g%dwZ0YmG$#5I133569l7+c? z`*a`KM%n*pq0A6K0D(}PDW}|?5Dp^Fut9pOD)TDz{YKCax3YzOfl}76Ud&UArN9Nw zC+m{}^S~h05F34VkG9%2nJ7HtAtYlxbw?83L1}XP5{uX#zuOG4RJUD3&(tA=;vWx1 zYZFb+bgOLewt?Si~bgp!WK}^>b)rwC0u1h4J7YW7Bf&1o4@Qwe^LV29X8Z>$Q3EOO(Kyudmml`>TTmW**LoK5GC|Kt zF6Q>MKVk1{nm!p!?k!Wn*>uS&e*#GT7-&5?D7(w3XSmRqMm*W`@TplA`k*+NXbMc5 z_QEmPHh3x~nrvRte`B61KdSQu;+4{3O7KFxV`JO+k_+3+o^(hq`Be7b0xho-G-j#9 zK+Ll`$6?F_)}k6jn#Ve6X~8Y4FidccnF(K(g-7#Cew#0V16OP{y#fz(yY=!?9HR@I zLGT%yk4&WAHTa&M{Y3)F#2|t8oa>2uBlm`f z$>Xz?XsllspwOCl0k|OK+0+L*`>2p$S6B4`o3pd?=5U5|rWiCXDYzSD5Z_JR-&i>q zRQCD$MpFn$@mY<*ougHD0(#}KO;$MzV8hMg>!g0?;O{7l?ltlCck1(`{681l#W?_K z;tz=yP#6(Wj}bO(+6coHXPx0_Q2I&2G^nBHCZlmZ^&UO6Zy-g8$L1se9r|lPU_lXd zH$9jZd;5#w!t^F|tkmbV=cYQuZ87@TA$a4@)A-ZZYjJx0cr<%&2oC_#isThum{&!RIIb)P~9;rX?*HJ-T_ujtSb(>OM4gPHweoCI%cW`Ey@p ziflMa$dRGR3x++R`O2WzM+a*|XAE1-cxQ($?F*86LDE}8+TBB=ph#d7Kq_!Y_cy1jk&m5#4i6!;Z~LC1O;@hMVC^bY%y*^) zL!?hx&du<@hftmyZ4N3b3&l~JpZ@R5UC7|EN#8|mL3b8_69Has=!Pvjo^_8eET%t@3< z?yisbT-xgzzrRmF6H5D9rUGi7_C)*d--^ozY%99{9hQtZ=VPyx@2R8RwB1MbW@A#X zIo_L3nRmCA)cWa{KJQ4Gv0|5JkB#1P z#@l1!@CsIwv0vL2T5`_%+4!^&!SoFL)?P&@*wFMmy7BE}uO0?m!ce5fRFx;$1BzGQ zr5wi9HSaN^ibVq~t9#=9W)pF5?5ofV4mXd|n^=R`%-{;8+GGpcze+_PTX;j^t| zV`$^Gcz-Ay<4h;a<2m2TJlNsR?dOu*0yftHEi@-0X9LDiny@RMHl`MJR4OFRzM0v7 zfc91J{{^(#8UlTv*}s8KluS{R!AQB(25HByI2Y zozWCzYQ{YU*~BS zr>T5d8tQW4$<>S=RLhYY*++o)v&#N_W1-LG@uqZPa*nFEO_R!!cHObAH954mR2C6S zhO^(&`&0ziK~7njm_>9M>xO6CvVVV_85*PSb26@0tYmg!_xP4yF!b-y(I$9n;#gCg z?sfYt3(Xb<{UDwkH(|xLAH<9r1^!=9hS2G{sM~MRED3^DY&SxjvxkL3oK26>OhM;a z^oWLogj@}OqjrAe7}f!HUKG~S^vJdnE}Cw&9btg)l{bWlauP4>*-r!WtO8(b@H|P{ z&NbY}l6SRm*$2-ZNVJZ5MiY#U($Aff^f6RgZ#3^fGdLqZ9)0cg(F1FQE zEr0nqnbmyqBC50aNLS)9BT448t5BlB!p!DHKW8}cZyOI;Ko1_czkW0^uYX=c+>l0a zaH;+?u@A9}qQJ%c0Bz}$u@`oPOe0Ndmfc6xKYvmiNFjNT$-UFAOG{;iV2HIjEE@6@ zPYj#QzNxCszM1FF3w~*PNwDSd?00{L0|1j)fPKcC<}TGxvlivUG(PZmSQ>XE439py z!JfzV+);>qVewl|y2sC1Md=Xi3=MOROf4YRG#sY<_5eyU(syA!DlMvNt>Rqfg^L zrLgP61>zyE6gw)Ox#=#f>|K;TZ72)JLDf<_M^;XDgAs;e30iye?X~?p!;}5#%329i zocd)eT3A~e_VPuWYYWy)fLe~?puP&H)&r>*v?dp4dKu5)u_7_JZ*;MnkB@PLwF6#o## z?($Zi_Lg0~0RfEXlCwOIcgrwl`fD_g%x+;r8W@lfLD9$pgA(A{`YWVtIvZ(Jc~?1; zI@hp8Nah{S&B6U~NS-gHvDtEjj@7|Gt1~}`iq;TkUE~pp?W`x_|Ap2F>`-H6iO*-D z)V_B!3Rn{uIu!@6t>%}dtjUL1+e1h>moHCu6^a{0@31 zA%N9%5>XrE93s@BCx0A_YH^#XFP7>w37r|E*#TEuIc6necBPr6O83&|uhQ1=%54Yr ztPLNJ?KH15&l~{N9sKu`{SuV|i@o&?AjeUa3CRgBiN*6}?!o^WPO$!S}Krdyb z^#oD#b=zFGMfUTHM<3~00*bB#I(eesf6YiRc(}P&*971rt;3t@=Z|?=enZ@n$+ggqcR_U)`^`A>*vi%hlh^tIXADT zFs0M)%1D5_R~iOlJCD#PFA$o1CWF9)!x>qb zcSZ+8s9pIIng!xP4!c+hHa{|pSK|DHU=idhKGAzOA}gi(Fzbt75TF-;?oW0yH^YY7z=_H9|xw7)yyDFq_~TKNt4v_r*N-89-c0< z_XtaE^Y8F$V)!^MqOaJp&_V20o1E9#ET)?d+OU2ziMh^=RnlO!Tg>VbmM#ulQ9&jo z;ZkAPuH>Kb17-%+=w<@CX!a8bv0GuDY@c+a4`yM;% zrO|R8u(W3^g20v);x z5Wa`Ub)%6ex>ASs!{!5D!fC{DG{)C-=^Y-;ElsnRHLNnAcU41;k1(Ht^%( z=MS*odZVFD$0`@$H75M?fZb7;gK#v>_g^Hf`FFlWdhuqF4(a=aeE5NTWI@m6HxkWz z8h{(4qKRr1CVaEA_h;qKzXqZx0e5I~*$l_KweQm|8}V);cS1FkGg9-y7OErf^{vPN zWtnQXE>`HW<_9z%3oYhr)fItlluK8 zg}FMH@}^|0t7V^=&+p%Dsy{R5x43ogVkTzN(Ndje7##4C5=Tc)HF^om9aOVQa#21g zBs#v{tKZKSw%bC(UtyLsZaC|uFeXRWCiT=NCUZuvjDe=Ze6rZ**(rPk%A(Umsz*3( zS~DIq9vg(3o(JnT(9heC8W+*X1XuYbpJ)pQYv-y8h0ptLTz|z@>Iw~HZ8}5yd-di% zfe}Tn{<6?Wr$aLZ~;|U_4w)n_Ki4P=~ zL36F(NUZRv;mluY+99lps8u~{WS#Q53fkC#idfiw#AZ*(<^wGXdPg_wZ`yo8JJcjk z2ZJGt&HXlww%!)U)}jvv#dsz4%5*<#G^;5m?;M?7n?*Fqkp3X-s=T&pAK&pd`@AhGds8a*$A;DIHi!&cdhQEnzh%{!-Z}n zKTu_LEa@^-tD&K8+L*WG*!(s_~ z`%|ma<*Yh{wdh_^mdMI4=VgM@*p-w7vm2szQ8Y%_%rc;pFLDx(tX71un4au*Va!w= zW8QI~5fao|^$sn2mF>dAt57bEKbd%s+F>>eC*(Ixz6NI)w-v5Vq|c<%>;8MvQrKZp zb=NO->b!|vY;@D-y>{2jvu5hjbHZB6Iep}YhR!hW*KZ8}p#Gg=Z0+gx56$`&PeoD! zij(^l{AIjb>+j5CNX#i4&)HCG1Y>67<^co~lsoQF6h%}HCLapDNMeh5Qb1#57;)tr zX+a&bX9oYC=YBZP3B(vTRv4YxqZcV>Uu^>nwOqgaR3r32E+8AEDLut$TrDy~u0Mdt z2>E1+77*=PoTj=MF18d;CtU4E_b^vsLu+<1`obF|s5`)k$D z_w1NDYaQS+(}(oO=YL6nmJO#AagV{r$IsO$)S0 z5^#9;bpt_2>DBlT^*rC}3$+}br~3T4tb8_jibar5FTB&XoBRTnXK%lK6!4SCpC}+W zMJE@qd)SMyetwWsya+FQENL9qj4~B%D?^81{1EJSD zu%^^XH{^~EH7@xp8C`Y0Dvctq_(NoSFu{5nr-E>Ufvoy`$a$IG=5aJXbZU%C!|VE# ziaX|MDAM0n|8v7S3c98Ufz+<;_8}%{fOfX9KgEO>&q(|QuAkN6+6P}E<|?3N;b}M6 zq5eXCLG#SO0pU+m80A=JUo%(WG9Sh0PNXa-9(jJrrNr6pJc)0jz-oQ#fCauRlG{+k zWQpZ$0xW!7HNrINA&@-9MighN8Jv#Cxx}#iREm@#uAd*1Z04mnMa1K?8){^9I34Jt zokcy5zp@r_t-q#vj^6!-N5T7iee2!ziC?+v8V%cPTO+VNas0Oe>d&bA1t?NnVO}^K zJcQa>wEoeZq`i8cZuUy{4V5$ntKt@yj>(EO%y(SCK+UtYACcE+JZIXDEown;r@u4# zoF0ynghoJT!zmBbL1X3OtX0La@T2z0hgD+kJEh$nzb6cn6dl7)9QU4KcHdR3g9p`b zIBiBl`C9;ew$?Vtmat?nv~09G8yjiY8KLo6a^LUp@hH9cj|(VsSdubMiLil!(Q=!L zbLFT}m9B8PrvB`Z(&;kF7l>Vnf__&Tpq+p~J=~5%)!6yNUaTgVQW0SNZw>7#~tJH>JdvSTS6~W!ko1Zubhac6N?8CuzA(Z*VPd zEX1YGcfBumc%(m06cvFt`M;Xsh$IaN>SS9mxUt5ag(+z&__D5J&5M##b@wgfAaX1A zD-$4&)XS+9MNMnS?H9AjO*^s-&uUKHQ_FTGev|pZJ38smXNyD|T!@1j9=F=)Mn^x8 zvg|&cUPVWFD7mZKf=w8UM>8-`%T zkdbAN&pZ1zRgErd^l>q0SWqy-P8}AsoTR|E zJ+#rYRy=`|DYsI8&v@BV$3v2X+oN+zSyI-WS(-(>ifQWhA~kY2M{>K7oE=D9DEIb>`LnjK160D{@Gu;D_v;0>>$bMTOuW=@MXT$0aYLH z;%1{Oav0i^@g@#5ERnbA0X5>p(Kf`Fo|A1>ypR*zj4$+gn*;Mv7KhKHV)YN$h#nU{ zT<8oGvghbIU;)0w861b`u99LGm0w`&n5XUIuob&)7E;-WL-J zTP#MaPiBluNMK%9EPq+0`e(BA+gY9a{K~td#Lac4!=J7Y!19u?S!j@m?WlTM^m9t)2?gnUQwyjr^qPU5kGt<&gBeX=(mdvm$2%b4$6C-86N zG>#5l*%BZZ%M`w2=Udv(nb>E{ZFsOZcf&m|NsNcTRKjHgO<_y~rD}uxW`9SENPL@Z zgpJ&pXU11fe%#e8Q^q71=EEVa0kDHhj5lk!1~Hq=d#`V$o=2Ohto6n((dz5{o=)&X z;k<6fz#GTbsv^p1FT32qm}~HGUXrv;`*yfYITfmC1dZEB^ZM|QW)Kpd6u~Y0U$k{E`!7@Dyob%N7*Wr&Md49gU6|tCQY_QjIP8lJDWYp}YPt z#ES}BK%t6DgYHEUqqi}ZuJNMGVx3H!d^RpQx%e+&GrQj=67(Vc=j{#3QI0yaD`Mo5 zydD%XqYqZrYE@B0yE6<_iaxaoUh~0zi65~2ty!UyieDdDdktO1IL0zry& zNt?Gzy1w^!Nb#8>sI2JVjm2^r7AwsA7_SQ!Tm~4Sk$K>gllMB^ahwfQ$NpVxsO%V=zZsrm_)XK? zm%Kt5Fl#a>M7j?TX3dHIMf{)}_JdX>@cb&g`D(w=Bt@DHrb5hJXlwOB`}nrkB0|8L zz47c<{_!!^Ga0ei%Yuiaxv%K)AXO&aILk)R(XG(vMu(A&WszD+jV%v=_Y0Vk^!|hU zRlhMf9R3IQSDKtrAkfjI2<(Q`Xng?}cDwS38Uc=bvssf*Vh;tv)%{EegPq zh9-8)LPK(9T_icXVP53~#)Ew(@ks_}*D5o5`d0+aP?5J*&7Eop8maE!W$2i-5+*7R zNg~MgtC%cbr~6*K#S?t@#CFqXkOOkIk9KqU_1TGh#qyD(h31BfJs;YuXGV+hL!*P~ zzTf&VE52{sm`oDNp2O>e{=2M#LUh6a<{Wk+Ki!GEc=-i(3hMDH7ZXO}JN}mF9^Uhy z-6h}xH*gaE;g$W*StyT(_6m!_NKuPSjFB;9AH+|3oVwQJ)5G?zePv#qL&!4uJh)t^xN%*6%i$H@7hXp;;5WB1$7gHs z447i_D}w;Vkac<+#|cctXD^x=ioxomWryz(P?asqL2DsvVJ|PqMu7SzJI<)^S!=3r zy{V~T&flAmjRFMF;={cvqu%z-)8hmg$<{W4IpRxCqgqPSALz6)!OyxXK6y%hv}xF? zFK!#^UslFv?39H2DNN9#aHt&l>7D~)k-<6(Y1NZnSEviO_@L~A?fZ&)$ zdinTuQx}8s%r+S-5rv@^rTLX&vk0KxFHd@loyvlqPjVI>4rufxan7u~vXZVP-zh-U zcT2Jb`|D^XwYGhj?q_7JDg_b;OQ9iBs061rZ%ptv*2OZ?11@*w8u7A_P;SASCwuXV7t_t4o6Y|isK^iV#ubR^Ik(kuEGko({zS`?Xh3)Udmt=kBz zY+uOB<6B8Iq?YbX6jM8*Kobt#^c9p}`e?F+UPhx6JTn|%XlP3brGXjFD{VMVCr7@u zY5JA2G0-9S9_3+ui+K6aRefjjb>JC$$Ejr5vgBeqD--GU4rjgw_!q(`GAJv)prm+WE$B{*iisMjsG$@>BZ0s>zVPm{0{>CTduJ6&-Q&frhSJ~U;8imae z>os|`Pv*x=|L4g_;f7>4pNn2wY;CAjI?3u>kBbjYI-UgF=s~8D=fsqvX^fvF6j^)3 ztDj%|`S%z)riYKW1pLBHxWZO9IQTOElRJKrTA-_{m2O!q!nYZ;cQ{eQrTA>3ABH?*2qquZ zS>H%Sf3T=E_JGTnn`g^@5dpKw9hWnNtSe&osE{fiCF}-pEut5GqDOk4T!(cvh&W)(C}@^4onqe@Sk6 zU^)0i(FB#mm;CChLKjNLqiH7{J=y&cpo?gPvg*d=W2$CH`-s}{fbJnGfMgiMmLrJP z2W3HE+7U|#Pl_5uMS5D39|WCat0PoMiQ9O;X~u&tsY}!SD%@a7%)ayTjXM1|E;yWM zIeLe%Ou2W!1}2J)`9K7>6TJX!2Dxt2S`M2D2vQ=^9mJYdvzdZvn4BP*I;${>;8o}< zTMj8lZ1JQULi<)HkahBgrQZ5WQa(=Pj8p1br^v z;i14uBwB2%8gr@ig)_-d8h6C*=iK38jGuM?FB*<1YPzseNb(;F4+Qg2q|z_RQWe@m z4B6}2QK7-X!MtzUAk$@Lk>j#B9Y$+`RJH-tzfwS!#iokVl6l=RKZ6Sk1aT1WQJ_sd z5u|6tvG*~sCn=@~L5ohC)pI$skV2-2eCpsT+iPD5qwi{^-3h>vX3yd(5dT7I=GOxK zbDCkD_sDglgvYP5<2GA1dR~cmA&s))QPDAt8X`0Ft*J!4lx2{5^l%IaS#|OYs_2BK zMBxxJ0TJf=>|0F#_paxOcec1v`EKDG^=l_pjOnwvz#ODe-3fl+|D2qSz;^ZfatnV{ zXb$h6=Tb%fP6d3`P{MR*f<#Ot*Wz;?Bk80Y#e0T+oIb~FZU1ywR5MLuQ4i|ai0D@g zf@=h9pn8PUfi$9q^XrbL{qYK_?uQ)Adjo+FF(iA!y^8E7acU$SreaU^q8St4@3ha; zOjy0Lv*U>8H2nT`@#vf_{F<4XRFXc#P=gId_b$EA+WgnorPvyej{#&#nd#Nx;7Tc0 zFighqjK{!CH%Lt^zWZFT$N6#`%UBhYK8@jLl_(nN;KVGLXrARnQ3)2TLU*+PLCgT^ z2%D%tT@-co$)dFL2*uqB?mJeP zb++eRI@Dd`1238_6@%|kv-tqo(+vvB9W5!2$h{E2O{(<%`w+CON9H=QG1B}RE2jd=%$O2p7z!tv|(g80|BO48Ggl_mMT+CPy52$$@L zE6_TxSF=``-(7%k5HBv(#vIEi;UE;l5Mqc(l7j$U<1SEyNZB3FCw9~@^Y#|G2H~O( z?e}_Ow}0N$ecgSOZ^+gx0s;;nU3!02DB?JtF12A`ZJypMmmvZ+R^6X#EK2T|_is@? zD>yumoL$lBjbPS~uYr2R_RasqnFT?)tLkqwvoVkJA-qznDEs8P3niYQ(UG! za6FL~av3;ehdo8KS4hG<9n|iCDDyy~dEQa8mJfFIqFFOikMIQX0zuOx71R2v8DHkm zt8FKgbKl~r?-_X=a)hyBHD7(*YMTcH>TJib%&1VEZzk)nO*S0KPU%;l!5`SDns)Zr zfP%A#*zMmLbbtep)+5aaj{7}2PZ6NS3WYGyyw@{C>D2dzRyOQx`r`_>1N^NjmB z=(P&zK*rii`x43lavEszDF4HR(T^coKB&uL?VoElJS>N4=>ORK3kGiyc};H@X(QR4 zyF?|RpV1?2vGkS(Ru^!glC!wat|J1H5ubfjBEcvC92wq~LZ(K$h(&aSz|OXdMM|6#M&mvyQVQV{+_#AleiP2%09GQyu3U_uIE1_Czlr1 zv_8o#a}Y1ksZS`XJ?GOqa;oq`4w)^*oe164+mZ!@;c3-6FIcBiNFju%@cU!f&GY^Yrbe39L2)QvD2p6Id~VJ+I;PU~ zTz97#5%>CUl5eutk&Xv%sC;%Yc;EbWJ3eHNlx+#v46yx;m!9I2D2AJq7t;&%U|_`z zN6ovE*X;cfu*%;UJL9kCtjejv)2l!xl4C%bz|s7&bQaJ+}8vND;OhqBWD`dN zD)QF~yXjRaE>FW)Ih_Vx-6KV;%>Qs@eAO&i(!%mZuDQ{Ug8gQBribViMVJJwp$Oce z-mhbis6YLKI$@e5uTocIwI?dU_ZlMfN}fHFnY9J^MihC<02|YH$+^TNb4P+J^Actu zf#WQ{f1FEMEUB1Onm7$P<02OQpJ-4D{_UZEdwO|`>)c12sB2{dU0zy&=;`xZy(1I4 zIB}Py+otOf&F8N%odu{qmE(3UYiy(DdNbprTb`G}cZ8tfPK$VK)Fo)g1`pRulgikb z-hNPSxjOo{f&LEy%fZ(uNz{uU)KcbYC{v!G`8{Plbq94POCf=Dje#`lfi;Hq3GE&` zRY5--hghW+4XrMpA{!=s5a+Nfdi2h;PY2~BaWtST+9lk`17-Tu(>D==?Ts!ZpK{cr zwNihI1_ncOfiFs|G&G3R9bct^efk5N$B{I{5WE_TRnsqmCkk8cXE(fFe|KNPQAx{a z?!EybAuZGLs{bD3>Ui$cbFyXl_n~`q2WQSan`;8=za*i%xT-dC2rmUkNeMU{@&ywO z97Q?cZi(NVe6xt&u&>lc7l1B@GLlEPhh-4SBb2ie?4;YkCie3&8S*tAC=%HJ$W6@R zcIE4}-0782<08a5aP6B^6Kca{QG!{PI=Af|^5g$w>b>Ks@ZUG!aBw6XWM&*hS((}E zpzM(qDWi;R$L3@mvSnq@lFZC-k{QV!q3kXDNXD^y-uixi&-47#AEjRLy3hT-@9Vnm z>k^$v%Ah5JHNeRYXg*E`PGAQb$au3aI$s3dXqXmaD!*S{I|!|RF|lhN*rtfUzQS5I%38>R*+0tIiiD`M8a!byL! z@p{3u-Mgp#+80|R!~4NV_0We^AWW1u36)da6JuD%h^M9@STW8cf{oof zaN4*$ns$^9;@g#ol3mUAX5lxjiwXOSWoPshuXi9W{Z#wp@#C`5Yd)u>0*wldEmKeD zE)yr)ctoTYo0*n;>wm}&)@s8R*^-evlx0T27I%yKjz+X{ST!S0eRb0(uJDC_kGlc) zXR3SX%Eyu=7hd8z_-v7-L}L~wqyIIh0%=y>AwyQmeA@li9gy9s2I}G(2V((mvXbsL4qft)8O`(r}b!UzzKwH zx|B3M2q3}Nf&}jg1SS}joum&ijlVPtu<@a0sJ*5SN*98M$a}+~YeMsG!2hD;HStgX zgZeNzru7y){4%lD5`uG`Ev2Qu(rp$L-0c587b8lo3p757?LVtQ(ZsvOCI*LZ{+zCCDS5}C?oHEO8hn`wOjuY;TO|6gHTu|6DJ>l^?rI+% zVkp>?n71g=@H@jXjFVSMh!8>Bl{HuM?N`d!jOZ*P*T!Gg+?Pbjd4YUk^79n|9DLqF zV~Yz|P{<%(YwuHu9$>;D!o6@5H7-(hB@kcnxdVN#f=9i*KZRisMgR}#@)|i98vhw2 z!dUHZO(aQNS;{dDAJVFq{Y43>qTV1BalQCR9IS8JlxK8nw_OyfsFt0I*l4`R8t1B2sP@n8uM!W2=DEW9+9F=(#3P#3-2dQo(u0FPS99{U_(mFXtw{o0ZdFZvK zW>b)%O*N9AaeD8yfgC>V8AJJpaP~R^l{dx$KiwMJ9#@5rk&fGc8=w*p5GVv=jLL@1 zj*QCFzmeer1(vB<1boEGb$hAf{zi8gG?WAaRFjcOYpbyXvV%Xrv@Xp8e?YN`GAYW{ z+z5k!4Dd6_KvGCw&BWB_vKn7xTDSW8f~Pie+j*3#8-oi75H#80L!z!EN<>&Gawl~Iwc_u}3?w>!SZIbnMwmRs(m0Py4dwQPWX+(*d z{>#`|ZR7t=;Ygzh1WnVUgdZTxMM%?6j9XC-fr^|EhpM7a2w$s7JlCv; z>}hRVN-PD4l-~pCN^JelHpvao(R{l)u7PW1)uYeqRsM>#tfs54q_W+Hf?Q%I$b(Ew zoO5c+6tMa87B|{jFX-%OlLPn*XG%GXh)qm&?F2wQs>>GqY&9=1;OBJm_l38;4$M&o-PkH9j>?^`p*<98v6brHO@x~7tGio05@YV2p3n;UG*)LU& z^LcpWEc2Yt?WKbhf?$re_>w*Y(2v*Zudjye_!2^m9Dd;ayw@T@Xm6BGRBF;099YSZ z#((hf5OS1|HE(ws;yTF`@S*@gP~YDVK&YCx>G#Dr$L;xOx6H+kC}V-@vXA zH_ZABa~^26chC<`rHLabUiB=O-T8uY8Hqtr$DMYA+h=nxfyJEj0H+CycZK&4Di0r~ zG}c`uvkut<;Zt9FT_}8UdhKv1m(W#B^Tr!Qh}VHSn?^c?tDP?pU#+=40GaY*_n{Z8 zvW8;h`a~m;Q75wnfWerT9k?bjR)2X~ogRsTXspR#)h?q65z{wyc;q!{-{T0{#)jAZ zvIZdlk{^{s!(Wen(r&u4>Plie%}PMEgpqLiQC96ltfzYm|DI^HAxI$U@ofF0=>$_Q zYJjb-oVF%z%WD4^?Xp@Rmf)ZGf)@B|2eyj0uO&-#__C1D%-G-#F^xUpQF3bx{8-ECMeC2UxBbpr>`|eKylHw~(U|s6iKt-z??Lch1kj_tw zF0JaG!R2ngew}^bEAqg3gS~Hr=uohr*VU|CcduCt%5F8$W?xKHNARjig?vl)jJrf< z_aYJk?uoazvO@Q&%TFFSHkF07swkNnHBQB0(^es4P5Iz$BSIi^^{*HxQQvZkH!}a0P%@_4i z3lGS1Z8Krtt=zr{O!DNAE?%rHQ{pHO@KdB*o)o$y(6nX4mlL|vC&`zJohD%1)E93f zPA#}Q3H0vQ7Gn}DL8V~NA2CVHZdKg9d{4 zD~Wv5g4pb)!*?&{BFE=LPzhYzqg9eEGYQklRKKFO+O9)EE(V-~piW@0LzVci+We1c z>C`wN$0spad@-ou{h{VQ&s?iLz&>$|$qa>*yzDzz>so>Xd>RN6R{0T31c@u)1nH%9 zKUbkt0ex!;ej~_7dL?(dRnsMQQ4JKXQf3<;Ojm{unF&j#L7EhZ=xj)g^r~c>Y6qVQ z$FD5%J;!=(4zFi&ko_)-8yJePKgsXyax zbopnjRPDvW>77E}JJNT~&hkF3BCK0^gbcR;b#UFG4_uHMJGDridl-e5OxTBqj2V-> z^@4@!cOHv=qZ*`8=jW*rM!Da2gNw|2C_LhOu?iKZYHh)`&ds%gm*0aX+?ScT3f)e)PZL_p-mL zjxN%A5imYr8Ylbx)Z3F>ceZCY>iI7|EnER(7Q0zvA=k(b$bI6}oMt7~zIcv%Uv)2m z9?~*5{btfF3RCfzLJ_)xj}@?SP`yS#Vxv#h&K4>RZ=2TSXk+r-b+%ZThv9%l55+BS%~IKZbxKH9na(Y3LWQlCxnGws%A- znV&l+3Z38S+-&zzP_U;hFw-RjLA(pDL8$IfM`GiqD+pLwXP(IxJffl<0Xvb`lYqWe z!hJSy>dyFh_eqi#89m#NgoQE54@`NvJ!}G0mUVJWI{=bp&?Vt`~=H z{NHY%qcFC~>3g*+Wr+fm$~BDsGBmgvB~j~A*TE3`SHrR{w9{5F$O27Sr;+qVj_o3H zP}!U7L=t1fy7%YLYv8by8))Vr``6+I+Q+01#9O{;Ia^_YFgX_23?A(+MUoV+edWrO zTw-xAu&GiI4Y$x}Q|E0K)tYA=4U{Mf>Hh80;qO6RH%9*MPx8xq*7lVtokJivP+hgAe~(j%ZYr2G823Z+G(D!#JCk`kTO zZJc$El|6db=&e{8kI$Rq>Nh}3uspyQpyGfiLNC5HU#DJVne8?tx*7K8^a+d{cuo_o zPs%&(SPd-iL(zhL7z)GHkPg8Ekb*Sv8Y6BXFfNH_oDu(${8Cyq7=wzZS2GkiVzOHB zUWh?4|6?8pmX~AfD&SDnY`|wcG#jdoL+)4T9`pP@HGdg-%8@TVcI-@D!H+1Qa{S*X)Y^s=0X_^%=r*4m< zcb=9Mbz{6sQ#jX@bftec)Baq1bjHL^&?MO6;Z>^rp?=kl`qIy~-K#bCD+3PX190JF zC=SrNx0S)+5XYgacM8C*N46$vBIP&vphmp^|M7b7zwz3kaqGj=5me4IX$vYy@TZr0 zPa;u;N=8CCtR)uGS5O@0>KeA5GlR-yUU@-?6mjI={)hT&?It5@8JpO=tsfaEiOPB! z)S~UQiuVsUN8;CCwAA18N3CCf$lk{(n#aos%G((mqFe4J7 zN`B#1VK1|Y?bLsD^hC*G{d<8&8zEi%+)n;Z%h^=9Av=l$pX_?D<--=>vCx!5w7;YL z?60pi0rN`_)BhIkV)j-qX~)kZe{$T{d**)B`8kxNiT(gaf~Sa*lDg;M7FE>8b^B&~ zr<4st>OVZ{GdgJRA|9tjOUwyM$nECiLnL|p&o&Z8s-Imu53O0hfM2HTz3pg9vtv#T zs6n(eef&Ya$S?b@*zq2v6}XT(^>6J%ut0!IX}X=?+V6uI? zxo+OqI<%Cx!t;vx@czdRJOX9(5NbtHQY&|;LN?*3I2{-Uk8lQd!z`v+ALiQ5ymzO7 z_q|Eo8+EH`+Z;T0U8QMLxX>Rq&sST998-89fS0>dr=AkTw?$B0ssG{P+l2*R`q`eX zuchSA6e-(3T=A5qTFc%=ZXAK@-)IdMgxATG zd18|0`I=AFVej`88@;Tbj3D4AGF4GFPjl{kks-6b-oH5B{=0TQrrCF#n-!1(VZKbH0h=0e(Qh@Aw`SW<)ZQqBL6qWD08; z>y}!4CMoDy9HGR?vlGRHjJXa17o61zJfoaCf=a~VDwO~#P1h44@jEVL%$`?XIqqc( zZz*$rtNd-m>l-rB$ZXy8UAC?fea;RNw8hICWGTa7-uru_cLy9vU7B29CgE_VuB|q9 z3fnNKoGawHuMTuv<<8%c&kaDS<(>?4ufK#>(!nGwhn7BCo3hQk*j50GxSwHZ(-55K@T{Q?SD;METxkHlU7GS&x8s8b8JkDes5i?YE1{x>$E=7j14P& zYDB9fxJrFRAo-bLuk{#1oB8>boIw*=^Hy3RCG_iRg>zNKy*lRK65h-xdVxh2hSfGS zH-?sMlvH9VXJtm#i8u?m1(RWYlPDpUurebVR^u=B;XAR9N|{wx=-7?>BQWBI=_apy z$qof|AMF;@ddtgz1^vp`oS@+CpyDxY!U$AKk4u~JM!}6(f->VOVs(W8$ms6*gj4aS zRj~Xr+&*k=W(!qN50v_~{!0aO_rs5=jwZFrM@KoS?rZ}EqCcgSl6`2~eU}}``9`!V zMSTJUI}*#XJ;oQl6*sTv@IH|?9~1{WFoN(}Eq~ zXX^E~&r4IQW9`lwsZ`4I6oHI7BJyTxDlEe}a%XEJ5M0XQ|1ctc2$NZ`N_Lrdgc=#s zeA%0d-+jtDA(1L!b4N z3CE2W+&se&gmPn&mN2$3?+!JI4FqAZ5)6AfF>h2AQglrA$w|NXEHo086oX3kf6ku# zm%FouWSX{1F)%vg;@gX8kkE7Iun-q9@#wUWJv8QK+Jx6U%CphmZdU|r*l$EK5Nii- zo6IZA!PHkNcYXwrJ6gao?oSTASL!4|#4$}@c%e6(6pzTyw*-pJB}g2tAVC+MFr*i| zD?ou=*9;t*GkBT(PJK%|X1_y&rI!cE$CL+2aLbz&_PM2GCF*!rh~Q)#DZN0OP~?Z);Is3LRYdr%{ht7p%^vazvWiMx%& z|2u7HV=#xDfk@_c+>EFee84AZ+|L2USBEjb`fTd?b^DW^lVi|d*P&J*{o??D3+$$A zD?(8YM}#lXv`-~_IR5N)`b1Vr;33fM=uN^Phm!co$z!9Rc2vjWSW{ljuAV251ODa` zLxkc-vJ38ir+a?jow>f~9#jZ&K4)C75V z*H^Ot~w$|O!m@!gK61FaVV$%2c zJK(w(sjj8;U=EyYYbTSfmoRsoBkk$ z>ommi5NhNujCFS%+;cFn$>Q6!;yI+J#z{K?Cs&l&TO(;O;5tgjN3DkUjoekar8(X1 z%s!uvY{A2GQ^yzMI4|LPiruD@?b<33D9^iU<}_g`mvN zd>fok&-gmq?3o}7!JPO*ies(9bQ^?nS}^5;kOJR%b1B)U`ANvvI%Z38G0<`UMTfv z77Uf#qu8klP;LLOq|mM9?1Ebxsq4KJKS$CDJ`eQ{UYi@Uw`%c=X^k?JA308KDe}Yk z>^_xRgKH(4%Cdxe{0|sudKuKt>x%033s3vl8Vo{*?g@t{j(ut(bgZ64lIw0ADH+PM zJsTKZV(E}&I~9B=P;Yh5kvObqOYWx-WC2Ox_9fLGe#2378hl!Q5L^Hw=y!GzsKy9= z1nmAqxaH1ril#>)hCE!Y^4%1<;KXB+DZJZ!7~knqawIrB7RDm5pA-7Ezx5aV`5EMR zfp6vR0uxb@{QT|Q%8g8u?P9M71N02bS%qC+G0B$JMH_TkskwPGM+~^((}@m$=ql%R zO7M)H&@6vyAsiNlNvt~(f6-g*>RGJ@W0eGTXndp}US&tse(f5W_85W23a*?2O)qQ( z^#}HRB@Xp-ULrHdRoU{HvIpp*=z^)n`A6xF&l90(Ikyl%aa|##Q=tq8=x%1M`?t}O zFEX+u?vmwS`}AjeEu#(h-#AIn1F4j5r|jGwtSlvZo~ZvVn6uJ%ZP#~v9i#Iu_h2}P zUXO(a>Y6OBh%JSp z0dijZXfT|8#-SA|G3fV=b?OyYv#$~XOQgpWn4r@9aADy{>E9e3Cki_?_qmo{$6MT- zmniUMD;wp2ls6n}Wqh{a&mQ_raBkZd;WsL>u3inpFn&AIXIg3}cTjR)y`ySWs~7jL z^~KU&&R`URDVeilh+F)w7CIiz4~c6l4&4QdJY8Rp^7j(4UZu)!sdK1fl60_gOT3drjeu=sPjAsT=I)fQbK0KXX!a2>wb`Wm zKQDkVZ#PcVt=!0F?2cI>Ar-;9bb)>VV;J+y?PA2)v1}$3#V=eIb80BqY_t94BEoIT zmn>frBqMWwkbJ97b&#Be_1Tixi0<5*P7UfPf<)rs>+*V*qv5mq){P->h3WP`Y(s8o z8zEEjnmNV=xgi+wD(19zP?o_MyW7^G7TmC|;Qf#C?H|@N;1M<|3>K0!2R_7zGGpO{ z`>_GRktd-!&1kdgk_E zFSQYF!H<7R(I6|{{h7J}!|iY|zM&dp(>JXd$I19i zqe}V!AgBj2Q><(C9qG$bR8x+$i}(p8;N2mPuOJuePQ)-gu&WaM)fC1$blCtCyYl9H z3M8)|2una0Ri8XKI13_pwM!4fse|+CERBZC^%e|t9s*xknpclqo#FHfHO5gG^N@Lb z@LPQ&n7EVgi3~^ejJQTl`e-1>))VML$PbRj5Tz=CO_*LIfPBAT@m{xO$%_xG>P}?U z55Vh21vu8Ci~U0R2k`mAofMhGu5vvi*z0w zsQ;n(=y_V*{y@uLc%G;tWZWonI~i!+2R+)!#P5Eg?1}$3$*JDKA2?MiziYAs<8*~b ze(RD{7$*K!$1DiPW~Y8Nn+T_|T%0u)lL(?4VhLyg7Fb-iT)jjN5vR%^L5r(U?eGm) z=r^+ajQyH?R66}vxMDKZVt$pYNEiT_XS_<*`i_n`80OF#d`v5HO-8WpI1hwel%&{n-)m{+T52bd{>b$$Q0^1?JX#JkkrU(h~LY|k#?ZB z4H^A4lWvjrb+Q&CUd3Jqk4eNP8->iBa+9vY&-?&^w;%Y8?39i0nxX0oq{u7<^Njuw zPsp~y#MQ^AV^1&T%2L9Ym`!Exq|no+C<@3q;oHBRN)mZU8xY z;vh0s>0FiY0ZU%Seg@ssQ_dEEc)^j72Cr>(PEJmb^W28^Q1~A#E07P5z zRBiI8SG(ab>cV3#+po6pWXyd%_FwA(9m|5lb*IRC?Sps9Py#bWUOKf4wJ{?DzCepm zDhNm8y6cdl?(j2UuIv7-cFIc)g2f>GlEat?-_8J3Q&K?=hA{?PpVy=6(|uk z#s%?G0^y^+zqnif=;9LJs9PM94``%Fv96hso5mCjAaaaBE+cO;XM-ou_&ujL-Y`FW z^mPvg4}nK8#Ye&8r&Cm6kZR;PL(@!D@c|btdDR}ePVqkimR9u?)`fCOPIv8;eQkPX zQfODTe5^=`Q&;QQc_endidlkod8?{?#V5pnPy}dklNsumEfxdhaZ(^RS_DgGV58|o z%r`C`+Vf`o8IVZ+qOA*8GwN&tpBj1y;^GN=<`RBJ5Xh;i%t%7IxD75@+@)U3`QQaR zDlGE?L+S1I;J{_2Y?ihJu00TT8729~4TgJR_I|R%|1qBu^h_x}%A|5q6H@AHx~=>g ze$?n_mJfLY>iy7(^}{qUIhO@X?`2A$wQOZv16%)9{Li7SmsiQaF0_E)aPa-Tnu|sa z&n|7CpF-q0v?LgqJfd3vBVIAUK`ZfH<>CyMXof#Sv z0dq!Te1bv$s2?uV2w)jpDE=+hg()QQ!M{haGMv(Nmeoy!T)gSCmil}vmYu$~#Xaa~ z-uRx#?`lj(66WD-_wSdu+BE{b_xc7-o-IV)&CHnfjK1ASlc`dtTWXYHSuLPL4jN}b5PifZ z7?z3GwhJ-?o=)-xwKSna;c!wAk-7j*_5gUn8?meKL=n`ShR}P%qQTysCVXX1YDMs$ zi?NoQWO;9wu5AA(`jw*xETTD3V)H0z8_y9%KjO>CfcwP4fTJuvYs1i*s)#?_M8yJw zhKPR2svmXZtYXMPz|b52tqkj`o*R7&U3**H0JDdH-LLAjvlflsw7t2p?O&ZBa=Wyq4#?Wgx-%DmW(GmIsc zGv(w3Q-Ivj9g0(}e`)Anwy3kjgTVBYI(u$!ACS`BF~B2;rsatN6Shy zTz8K02$<^>+v=cLP%9I>Yx{WFo2b-+Ky>RJj-fZ-&h6*MqFLX66vf%Tf8wm*b^WrV zT*3;6g7^Z>V&OSwtBz6@EqsqePX$nzyTt)6?3G)Yu^tOFfAzF_=8{p=zIGbm%NeD> z>(yRX#&-fPOA>{dNSIWw#&~fK;PsdubD7b$=Zg_ry&u>SSA6R6ReFVGGKapD&b8D3 zW&JnV7S;MFO|@%>+F}V7+O5(i$LGS43(4Kx6X8&_?nJm06~RO}Ctz}ZkQg6+e`cpH zmKubqfFLQ3QL;$$tJMrted;`auRBe7b)IVzP(34qz7Y^i$?1|CwGDwJ1NjmntH34G zi8vm)Gbm+Bsp|YPJ*eiYZ3t#3q6+89`~Js`DRSj^13HRL+cTS?bH0^VHVC^A8M2=1Z8$N27ObQCKrUKG05 z8Q`q77aF|V7=9e?M6B&45Y7+E@KEsNN~vlJhGYVud*o@t)ssTqfUEd)Q&UVS{a!u2 z?eq<5+zX(AlZ%X!^>MC(kGa@TB(=%_*}uYor22L%Sw85d*xE9;d{ak0ld5mL#{DGV z@;qx2^KjjA0p@MyTH#dTypR_>clJxXc5+!>SMF$@Kx$t%R{g;T+2wG!t~?AFY~23v z_){Xe^hX+h;RBt6eEsjF2zWQN&vak*{DR^cH~(FQQhC*L4_g{)(gyF zT%j@{@PG#UZBX(>$}GMTtuZ z+qhTI$_!=J(xB9n{x;`iF50qA05}1sZ{i*c>AZ+jj_C}eO(#D0A)N|~ySFsXWbV>^ zGR%NYH16@MZq#3yh>^M*4Yt@y`Zlu56pU?Vh>$*^mJcD~@*&jkX|r!~4Opvj+6K=Z zie9A(kcL}U(FdzhwpRk{K#|r4*+#!S|*GRL|qVfz7nC$ZTK8MKQdG zFDrF78NLV_kpy=8pemSv&iqBy$&xaf=LLw~fcv$_U{mBwHhit4-A8Y!%M+F)F zv8~gD4(lbEa|SabwF~oGZJ4PYBQHP-urM&^(Qj z!ucJ8W&Ye1iSG#)7f8}DYi>1C8~1{n>OiX)gY7z-dX^#Umeh~2^U?@dGi}rF#hH)} z-Rib5y<;AYtb#vxJN()8j*s<)padkpzE>d1KS0_5{TBX-BCAAUT8D_uioMM-4Qc)= z=Ct~2)NQLTxS9ct4?6s_4oZ-mov0xXu65S zm0q{b?;v#q!Y9OH9y52kS3&gUNu2M0g!uMy_#cuQM#j#v<=7yPf zVk*>Mc6)jo-p5{2iD?)CfIV{BPPeBruQqHxrLoA#dG(1i9pGux0(WaBgZ=P`H3{>s z<(V2+%NmgQdJn7}ihJ{Cr1^oHrmj`3>C}M8Pu6#T@sJzsbx{deEar${4+8rh-P=MA zB4Bi%U%eEs)Qivl#}iYITjBK{S=Efh7Z}(C)QMm{J7f*~%h>f7XMg8BL|fEe9d8Gy z0Lf&#LCw>_f&ti4Ko}2Dv71M;V0L_G_P7!S=C!A#$&9~xRc6TdE*-z|BSnu{)IPg@ zv>I!@)i}Rj*!-dyfZ`>*z&4p=Wf>#?a4hvgT^@qRjL&pdZPiUBb=3MeLw?Em$%6?l zsHiHW{9-9{fBmd>bPwYtYPm&5=63D7-TyVC`RD@qVe6kUl!V#$y?C;`E=Km3PPRu2 z+*LA!_LlC8Nin?oS8`+&c%a!V3W@l)-RKj)Da5oWw`p)s7Xusxun!6a`jy4fL?ksD z)nY{-8_myx~$j*V-YQ)Ngk4eyF2ZLQlB2TP1Ai zN3TguOkFl|j``+&@fp|a!?B7$mb3@zNM*`4M4`v)gNOkk@LD)I&1mzwV)E&dYm*y@ zGQF;gZrOH+Cfk2R>%tZC> z`#rRmV0G0n2h2YQOoW2rFNM zBtb=6v|aD>=Zcbjj{|Y8i@qm5eIqP*M!J|cL6-BZSl@uY)yY=*Z}#7ob8H!phz+A* z<#z*raJTAlw6Wr+K>I{V&z|vtL9iDmgX8owa~0q;1v3kr0qQtW{qh9Blx)TaX9rmK zwUW7Y0<)yy_oHmkEiDL9&gu0JbC{4YicO7t>_{nmyFWn9^#71a1$O9i@MAgVo9_SL zYG0SbE_e`P?Q!bwQaTH6{!pVIGB0L)Wtqb~E=L|n@coxT5b~S5t{IK|P^c8||sRNlfc{SS~Q8&;(~i zzziaH9~W&rn|nmgIF3O+dw22od?#?U9wmjZZWLNI%8dxE(iDQ;<_>d|G2q4Lmu#1K z31kbZh07XA_x{mx&5?Vyq}3ggOU>!LEKCv|3@dV02ALD&X+?d2NTjMGNY_md-iX|! zO6ZYSCVciOrJV?f4Ge91Kc9*KDU91IXUofZ=Fb3zVv5`K0y%jL%X0M`JaxK!p1R{UG!0cbH7 z{=6L7hT&!IGv~*&|Lab{{;~fG2rAJLH$`ezI=-v@EA^`s6$kKl0f`;*WE_IK7zzzR zTVHFp>g`s8dmcV&g2W5VI`J)sJ!$U3l1a|etqeHHE-ng8?-EZOYTz=4?ixiL3W!^8 zoeo8J1ieJKu1>+*aRlP?$;iJT;+(nly9Roq8zW8v0?lRFmS%gs;F_S4?&NV`y+b2U z58*!n!4K#XCMWZEO<&5sTZwG{bI9YptKiz=!3JSM)YE3y6n65hF8lm%&O0ZyokvR)cN=QaN{1lo~g}bAHvmx4!jTyH`+6hd0_$d�U7`f?Y z#F*5-;-D}c7@S8gQp#)&r~W3RR!5M9tibw0(OtiI`B_L^rc^ZLRg;`}msJBS^NBwy!aDhaHuGHA1B8Kk(> zM7!P=Z%f?qr7mhm;)*OWvJV?#kq>|9fJ+B1%`}A&BC@xX9QU`>X{zJkL-UkHWo7Sx zzrCKJAx*a5p9lMsdo3=%pC#f!@yYR-;{p}9?O)slnM(ag9d_5A-opQbh;+??4s|JD ztQN3#~RbFhk^m6>ARJq8u+8v6ZUIbt~fbz|Ap! z1P#Pe@hLa~zZJYSY!w-f7MoyNxI#8bX+Rd`4WLkR9=2%Ia=X`CDm9x~-FMRLa91r4;;$#`6@FZcnPYYe7d z#00r6BlJ#|OUBnuVTz7(I8Eikb^l-Kwre1>s{pK-ew!cai#8t+hrpilX`RhdLhvMA zRGM0gMGlj?;aua6iw6;1HZzSd?wjTrGWd~Mx0sZl*-le@Uo>*(N7EnLjgkVflkORJ zl*a>;jRy7D4_@j(WsEx1qU{u z%s9)u)GegrfvV>k(1Yn;61!(-@#Mv4HjIEe`T=1(-w}CmKbS}1HimjH!;DAXz;bTy zY+d#m;1(lJ33bPgNLIsAw-t=7?-O455|-}D4tCgoW5W{9)2~-{?6qq{N*QG3uk_kO zE@~SMyh|47tEoCxTZfgKf(Weiik0d!4YGH0cxz)hD|f+qUwG+ao5DVr<0ueZuJ!=% zaPZ$}dxInTI_;^NMf!LAUK#>Ei>DL3UX};wwe11#lx!no zv2e#1C&&T&gLN^1QJ$Ofppf3LMoJ?85|)SWxW20CiP3J#gShvKzTPD_TC&L0^`wM; z*ZBA?nt%v=2>QIP{7-+WVQL%ubnsKunm2Es7zGowvxFq4&AwCb*s@xZ)XKdi%+U)3 zdhY>_4#pRXuP(@H><5hKd;+daWIdk3laV(7seU)oh}~xL40`x!PwcFuVb3U*jEW5b z6MWTp`<;8`BUJrU=NfvfvCJ!q z67m3C2*v?XLYq)$btRW^zlYh-mExHWJc9iDu_cD@HB#>Yd^6_JCV&8jMqv`TX?MHh zKP#`3fpozC&_e%4b4)!Y`|rPI*-AR+EfXKjOEpi#Lsrm>$au`;(jd9&1s;20ry3!W33}FZNOP++x4B z)^sGQ@+pbCxxIAOjq^*+w0QJWHgZr8#KeKCh@T&rHVES_4%t-OlQ1en8byA*7=FNO z@xXBRpy&vt>2Gx|ED*e@VD)iw1;xkF*2-edWR)_Kb&^rwqK*3|FDM(plw~8$pAuI; zhmj^%bWYbnmK|9iQ)i2EBK>*Y#5A&38_;6NzocCzTv$&dX{b>E~ zP^D=#9c02-$`_W&cf@XDsWGeRFr1-j@Jcwnj!GZDzh^k{o~SUE9|#8n0&d$bvP8-D zdql}%R%tZFt{D*^S+5b7CJff*Ixam;=4+LWqd_%%0_$fdkBDP!KGs)BP^aGQg9zK= ztSI#DwJ6!6+?_X$;mZv-trq5|_auz@t>fn8-x3fTQ~h}*jny?G1Wt?P?FnRP1%1SP zQU0s9fBv{_{M>Y9{M%EjWj!uDa6hV*%=?VBKWC4h8RcP`@h33w`Mdnrmj8FjLj!5p zXI7(09@}7u%^kC{^MOA0o@!ao|5W-W?0Q~-M$7vxM{l{jg;;GrzJCSPe))VQ`dr(D z$!8(WE$%;sklHS2`38F*vQ>YYP?-tX#jfBS)GwbWr3C-Afc#%Ea-`>?y>c%ox{N~R z|L6)tKoAOO$!%R+zU501tUjO-!C@N@uh{$}+oDAk3zFg&M^iO}P0kN)hEXD2QjipN zB}O<~e>#7 z<)bu=_y9)d{dN$4a_)4@sa?!-e4q&~WiF-KV+5a^PB%E~ou37r_TMyka&uW}OHP^R za30aevV&~c13V+1-*G#>Oh>DpgCbnmY*bG882K}kD+e6Zz=Tt{hU{Cy7^%k~1@w$} zp;eyxD~pkU1u3zBVXSNAqt|1vem3SwhrOSY@W}&dJEA_nEk>Ta%YqNxgn{5t@#o83 z3HD_VA5W&TG~*h%gq#_ymXzoFqH_v7eW#v}AIq-qhVKXJi)llw4E~#-V@ZOY&spCm zepf$PU1}+K6SBY4d1C}~qu2Oi_lx7n&rSDHS1jh7tKs~Wym-o+Zi#;n*K^T-pbv%h z7OB~k7o5PV7v{vfwa0?~<`LvX{E-2-g(F>*BeN3evYesf1MGP9OcEyimb7iusYp`< z%8Hd1-2uWI`t&`u^5eql2rd=Bl#pR27TkI_yU@{BX|UJvup)@cNpHRlA?(7~^2MrN z3(*|-?xNrkg3;O8%tp3QGvIU~zf&ykgL>3aN=OlicohgbCK0l@W}8_ zf+th&EhsBrP#D({rg(~A^Swn41uA=tK$1WtN`OyN1cdpzjS0m$^B$sN=cLV2j`g02 zs|1AeSS+*}*9C1WL{HWcBq%NjsEf_+5VLtWZ@ur@8TVzYMl?h5A3)azsC(J4Lb#Lf z1Xe}7;J|;SBMWQ_~Wkd z?&PlCv0)}s{+NERICxv3nGLz4x?<}C!$Dt=Vncgkg8hQ4wu5>w<`JI=aWaE3r`@e| zR_=FlNJXQq9cu}`vgj|G5tbVI$;(T$*#o+x$17;Z*C!yzGtAkt_9`eZvNI8OY;XZP zR<=@E_x<^qpz2KS%!wT_qOYUYhaT&DtAR#;Q?e*FqH1)Cw(4pqbD7a;PGKh9M>4RI z0Ws}W{GU&}nfo(d&O7r3HaiKu$*p?L|9vEo=w2Go2u32Urfss5!K|JT;?5D!*`z;^hKU1MwMF*^ zAzsY4Y|QH6ho`p@25eAvph|(`KcOpaCBGdUcgbG;eApYe2CQVbIKtWF<$>JGwyxKiz5DPBTq&6#KNgaNH*R zU`wbE6HcfHHrCd}Tvl!GZFZN+F;WZZ`Fqn3FRXYEMU{dK8y*JH0FB?oYUX=SBYk1= zvn3P@HIAVhARsxcTNZc!)OYv{48?FUYE-NeLrQ7f=gpt+&HN&CYX2wUNQ`PFqDRF|B+%h z@o!!8%c^taeJAZAm2}}V(<(dhdNf-ZLK~Q_((dAujYdQ_cwkjUYyz@6CJT7}+rF&W zwd%&V)?l6OgAEp@ZZA-;5 z!f}Zw;n;(_e&Eum>vj$C9>ImRf z1Z8G(Fu#|KwyX`wBRA&(#wRm-a*D7HFBd-EG(eOw z@Q47GBp9>0ir@c5jOb02j74N?k^=9~#5Wr+WaNSK_}!}TbaFl|u+W2s)+(P<2z)td zu(!-Qn(XGy(=a-}_CnZFA~0=%Bkh9}o*<=WRXypHSUy}QLo^bEo2y;o)3NkIUk>Pd zh%P(SwO|lFr)OryQcytjeDd#ws*0Bcyz1(^kMK$gO5?qAS}BL|z3b4U_0ODfTkScj zBmX^nXazSIy$@C&gcy)B3N^~l{Jqsw{n+i%>qlO6!RFqthl1QM5Z2a)9={Q%!*9Qu zW1W+$TmA40rFF^mL!8w=6iTv#^mO*?5A8X|JD z04SXE%7Aa~60Y<`(Om|dIy#+%EyY642-RoM*+dx`yrnd{8TaQj))d*XbfFtlh>gar z`S%t4VWwJwz!aI^mKN=0gIQaZBPd`ar9%XJ*^Yj&_N~I9Au*?k0*;Gvr=qYsz|i9+ z695aQ2d}UJkE}!V55BEf=i~qqw^9n#I~0pK1nT%@X0Gqw-+xP!8<+o?^@z^@^DjC> z=h;SIBteP+Oj%6=%09yjMOTGx2id|ps4PAa`G-2n++27<$9Y5k|KaPaqq2O1D5blk zyQI6jMLI;fK~Rwv>29SPC8WC>=}sx>PU-GuAB^9|+1+#WA3phf-zR47%-p#X%1`7N zPH!xJTRTBTNFW4hzfX$u3FN+bjvqVlAP>wjS_RY*3VslUJJ%K9yo%e(@k#^C@$uVf z3AUQG+DZI5@9!`q7XzRoY+Ci&LZB4+q5(TM9OSu+Tw}eP03Fm1q{ibiksHv* z?mYS@y_58woAa2A){_=y`Xk}~%vMN5@6~t_8yl;tr&n~Zv|(jb4)C>$C^6QdQBU z2t1f%AgNs=J&iY%c5T9K*VIiQ;y$8@mojG=@zf{ABRjFdH+3s``7>|8+~OVh^v04$ zA|xYcOdk)by05ASm>0wt>zPn(38XiLo)*4|vsD7<8!$&+7bGM8U?Lj=g}G`XdX9cu zi_|IKCB%8-Z?;1ncj};KS62m;3-`4eTfpwKYKHC^(Ho)plCi35+vtH^&$w7MM4-St z^I?tq3hyIolWk=bi~l4>bSWj?Scw7$^&s8crSluWWeb`_1*Myopd893bay&BQm|GJ zZtX7h;~=^$jv>KlFzW6oeN>9_z(`5&?(QQ`V$3{@cP5$5PRfJS!GN|{a=759>}Gd4 zmF*bWzk34h?f$u?6QpqD;;mC#bL_J`g|a*sYglgREHM?FAI$4E(qxvNG4}PmQv?%D z(gu^yxRZPm{~C*?zfqLD{^18Zhc0&xbOaYZs$Ulc?E6P6LY;S=Hv-Z33PVB{jCT!m zq+*ccGDMN8RPdar(7d^i5dR|>Ni&iaKQZ{MI%uQq;11VDiUgc^oz$nt7(HC)t_MXh z)L%rfXDz3lMxDiU(kv`a;h(3_w7Q{Rm@L^IpkLtAtTtHWw9*2pQXfMuTGP!q!)1#J zA0E)*RZ1YZWf}X`s*c`v=yf?CHi%*J+@|pS@>BJ_3=s@ui_MeMwu#|=01STb+98{? zq#ednirny3^&WASq$}70wbh}eY^IlAZ?^gu?YWI;6ia87#bad0H>6w^SjO2?>-fnP zP(Nwarc|zgzX4(7Hol>0Y!P&v9GlX4yYrS&RrC-96@up8nywe9w-GP`@g_*QYMXZ> z^N&@DAvw@`DN&G27DE=};*Ai;;Z4PGzUe!jz5!Fy`mfWW>Z^=GH7zfoE88!t#~0#) z8)yxEeQ}@!rirM+l-%6Od^{j7?TeH|&+P`c@)SnSLen@<=^(y2jZ+&_Dv_jcvZyr}!`QH;sD>53Zc zF4IpgFn#l;YA~%>idbH#z9tRoVkq4g^@q2oGIN#XXjDSZ_h{Tm*)_l_3dg`Z>u^2o z^2Ak1S)FCbn@ZZnQ4RC@8Y*&-sgQVLb`mE(Q&$wxq}xQI4y*HS_MD=ek%2uOjA$9b z*3cDmTyUo8sVm(RT^BXUuKpa8cj9llij?h@D~SI5R||h)h6eO<2I@I{#|~@Nj0p&k z4Vriqpp#CX9n3mL=G_s)OWxetN>HjE3BjvOp>m3#LT`;&b&k1U7s8fvJrRJIH)gXn zxWb%)MOc}{xG0^WGcF*8nuU0^(!^;Y@T49;Zpl$EmHDYjIbRi5u_W3`6K#_1(9dJx zWK|Ddxw#mN7v~;iv!GW%47++fOc=Bga$21Ew6l@`em6(nE$H~A-TocPN{Np^s`a9gU>w0@KR5g${kq5~NBRcBSSc zexF>cXO7grmi0AtR(wetM;$|Qql3jkm24?@!7LRdozy~_T=vWN$Of2c!%wj+;DJ}O z@*Rf#-V9fjz(npi)Yg~o##S(b5*vb*NlH@USJo4TP zrGqrxNVi?|oR2EaA2$6GpqgeZ`K4ZCm7E!8<3UzjHMnA1PhmLy#^Jt{F0 zA2D#<$TdjI4<$2Q>R%9STC~>J|JIlArdM8{S2MM{jCiw)K`E6(5&L@zE$y!YvQ+d- zlYIi0txzSi-boP*C#kB+_g^JI=Pw`e?1`Wa&-D&4EBH5@aB16;4TDfeb%21w7tuVZ zvgb5_nicv4L>YrK1dc$OSURAkqh#XKE{RXAymlpKg%RF9rai;Ca{GpbVKb ziUnNIhL^>HPqomen;=1(+7q^KufZQKO98Ew-XD*4!k5vOD%RRqV2SuL2ewxXy{|0| zTl$EjaN35YAYQqxm5K&rFw5Ur4c=JKC)O8Go-=;cp@xiY_93_HuIYG z#HY0e^mbxU0HiUbY*8n1oO|GQKU|irqCh_!-K$v?d(?UkLWi|M>dEW zItG#e6#k(8EY)%;h@eLP$`vgC0?6^UzPA`!w(I=Dv04-b1GqW|soyvw20-rNu(ssz z{P7Nonzp2*#cRC9cgp-Z>kVI=u}nd>-`^f2^aA(*lL|~|2s7v8J<+y~Mltl~pfk+3 zwgz-dtov+`f_95C05qh%YC$7ldUh%qtF+z6VMff0Sh9V)M)rNr){|ixc)TaJDNQw> zZ=M?Rp=gq%uYa8PlJ_MO$2PZ3b`RSr#Pu2H-^qVIi)pkR@dwP{9(-5P@wNagOYcUX zmW*&=*^-9iAeHJB(fxkrU72v=Zc`rlq*pPj&jk_DG;D@@%&vtFEtOG|DN$#Xx<0XVV+6riI zm(a&9Dw^ms)R&LR)ShH0ELWSta%+oO?-0z$?-OFP>38M4gkRn54H_0d^fPhV=M!f+ zws`QJE>6A}Uf!GYMXyBZeIb^LKzXBeEpT68QN^t96 zHD2UW5*~o6iJ81Qrx~>T5az7IlWpATQp{mSux%18b~PWE@bec)fU38WIZtIKP~#qM z^G7lp%~nuVSAT?(^vVMW>UKra-0+Ync-}EFSSt^sKY!**2IG7b-BI11tVhy2O%#ZQ z@tPf`kVOYf3L&p(D5^)1n@vBmPx@BldFEiSkyQJejkg3R+YewGLUo_Qu3eTn%x}vH z{3s!}MJkY{7d-CfYi{w_FYA{cgFC-Yi-27vWA#|sIBnT?;E0D3_WcxHI7M8^0 z(x|FQBOLYge1~#7CKCHG4#0zKZa%V4sBuc3*{Ytk$`Ckc;85{a6C`Ed*}>?r!}+8U zy68Htj$T4a!BGXp{^~~Z+_9-65jV;5m=2y7{!0=~>+QwMIl$AffbkiRD?s*eFV+4; z{jeeIV)UasB62p(2RiA#CkMRi6ntiOwh9>1ww0Y|rNaFZl^I8eSGBE;JlYZt)!|Wk zO-)>Ei8zK2-4t$OK8%Y&qd2pEzm8$$>!AOm#5;k2Lm@rsoo=3bj|g~l0^(~h1&mV^QvM7Kfh>hjI#Zf^}wvUk7qZ%QAa7IQ&9 zd_Ck7y4A#E(biYV(FJk4?F-JgZs*%0 z7-CS9V|0`Z0E-z;qStP(^Ss-Kj1chmt?mK>A-2aBbiz+IqxI7XK=g{*YouNP zu7{7e^%j8)%TA2;8@^uQpksj(GkfClMIuxdM1yXap3B9)!ejgdb#5&a{AM1zUdsdU zhg+-wKVn*mV%3G_*ZTVKq(KAHioLK~ZvEQTUp!R+F<@SoU-@SkyHzae3?QEf2|MDcUgU< zeSD_mLyB~yD z@x}a+;=@Geq~)fl`nk@aD2jznwTz$@1}aT{I9m~75Yhd(muCDre4#NK)q1i{5nqcO zn8o@)QKOf9#EyNG`B50{O6hR>HyKgRljHB{2}y-mHn*cK6$l|}PC8Voq55rvzrpt> zqH0Gp_>4~-zd79;k!=iM;CC10aoLVUq7~?$Pt==D1uilH9Wqv#x_0%xX3G~%BmX5F zBt#t;lCd?RV;vgoqsEyiuKW#g4K=uaUJF0}@|%NIf<%GmkJJoTWHr$H&Ryf&I?eo; zJyCAPz~iwZ5(3jkDnXbdEu~Y_muVOHF{62*=z)-}+!xpj2PkjFM$!(ult0pgf$jnS z>Ht2XuSJ(`8Z0?C)8Eojbt;)RmoEO|3a}lYWD~qTh2sPS!OFAAbZK0jsvp@pR^R=L z@r?@BB)NNV|K@%h!ye^l2;*A38&R7beL+XW&AMp=BXev2N^q5cmZ>?J{S8c}m8!S5QEPz~D75wFV9&eSgCs&+xO-nKAQt_8 z7UITq)YG6|9dcxaf{FnKLI|>XpEetb*)!5(S~nH3Ja){yq*2|RroaT^=xk?*K)ead z(ep(4ti%L+u#Ia0qgBk^e7sczu)ggPdlA8_=3c;ukcxck3Ug1MD86I_EVQVlbz^lU_i=^qvq=60B!(%J%F3FGLa>4q7qvo8 z&$=3}@EcjzU(>Uc1mz;x*bxgXrH8e6&p5TX7>?C&6pyS>(k@I+uYcCW3 zE-0A^1W1|ZOFjB%Dy;8Dz-)$C)sGGN#M4fm=Z5Iw`-o&9^-$aORymO5BL969ms$5t zbfNj>qb^;Kvyc2Q!(}+-blVzYn-`YY1DSvZ0zAq}Vg6aKEEJlEMgPT_kLvMkiQIE1 zCNIFLC`E$5z3-YdTUfDwuXN7Y45s^xI~G{r`}Z}Ak+sN_qVWA_Tvz6T^q^iAE*F7B zKKz0AHjjg)gO0M`@!~F%DQj!Xq6uG;4J7bIBL=@`uXcjIsToXP=q7#9K6oGlb?x|` z*Ej0hgwFpd_ck%>A=?eUh{~=MkebsW z*_8eG@sWQuWc0B36yyE4?CTW#S2?$TJA|eL9@V+NSHahm0++DVVg-rUSa0XmfQFiW z=AZ~m{*lZr7w(0|%bgpuoa7ff6Ggrc9|aOl*HNmfd(prXNDD)1%-_Ne=p7Ohf9Poi z8g}XYC}TZc7g7P6%l6fig{3y3N>9iuNr^o|i2ZkjSjKjl786Jy%P!EWiOD>fVakWb ze2lO?gMO?lC=>9K{?ZwvON@^X6geDJK8JqFsSHN~Ow%q74Zq@%^8sLP}33_8k5%~0xV-Qw18|bmMTWcYB|GcOyz^k z5clfkOWWi!A~!=+`*+Qx#FItDD1v#Iz;hS>wt$=HPBy3YrzxLPMvVfivz4-ES}p62 z%YTa(JTCV;Y50$V_YgoYq0((Zy6=ayxwPKmhPz&!8`Qwpnt-_CDN&Kz+R$oCmhAj| z!y*sc(iS!;$F#-UbHq9@c-`DGniLqaM*w<;^jjW?UW-WeCnVe>r7&rn7Ytl@?3dZb z{otC%GYE0a&zocf`xOH4pv+ty)*^{N3)y!If<{7KuBKS;YRG9?@I?sZrN-k!ABCUg zY6)|*9CT~n1kksYA2EeNh|D%VcESarr>jPM-}p54k&*XUfEjY+BHZvED@Nw!hSFTy za;$pqQ$)2wvFhWzC0ex4I+czG3zc9Hps}luWW@tfO-SKmjH4&D6jPCs8QQ#?d3w4Y z^YoBBxRz06`jIBu9#EF0u=hxGk5uzFzxA0RH1?1jLSjo?dqkg{6wurjQ%@Lt%T9r6 z+3@w52>DdWd9V%%J;Si+TdthDi~bHFp<`-`(~d)s{Tu-(2_rOdV8 z`tOBu|NLm_ZQ{ZFNVacH z!SdE%stx?71#op2zapb)5LR}YaYSi@b$^-LMoJ3A-7>(0qA93qGC~7GYC=-Rq%iXa z$jMsbNON`;P7hU6KYe)K4VeWVmRGczJ|`Q-d(3IaRVJ|>#f4UG1K~kB=l;0*WYo;g zSDW(!TZaz&69l#l5wT!JmOhJ;>o*Vq*S-A<3Db4(CrBASk}J1qQ8hbBU*&F3mR5aQ zq|@KvP(6aGI{28cT>_**Dl!TmO$cmw%PknQ@;LL@VwxSgyf}kcz@ZC;`V%oNT+__M z4V~A)EhH}$dP`0?B4R%X-Z)LxzUyptPS7+Ha)UHDHocOu@VeI50m5^`%!~i_2N^Q4 z21^-!N4w@nYGozRr4K!MYbyg28>gnypY-v}LKCGxty%{~o4lM6>EVPXBi%qn1#+z1 zl)Fl#P>6UkgC_Yy-;-Ow)eXM`_BC?W!?_&v$a-Jq`dxMx zFk)4;6YzMD)+j1wCAgb5F?CmU#IE%-P4b=JmN=6;3&kf8umg3WXS^69~fFjks>4 zv7;-Qwb1K>?26!=9Bg~v3(OwOId8)%{IRs|#(c$^jlTJvs$C{6x>Gf)LMs){=)xUH@H|-!W~=K3NMl3{mZ#+rFY+sO91l zNU1~~xvY>7IU`|CDidGjT_`iHaqhA0)dpy6t5$!jX zL5Ltno5B^C!?gR)0Ai+pS@OmxKRvZw489^ZxYoOg5^~ohQ_%~Vi^N0Zuzlwm*nO}RMdG`o+I|gQ-^yUIUC3N zD!buMK*L3e*rhU9y~>tPo3*M}xFc^o9fgZ&C^I+u{pIzcEs>2RsvYydqi+EOp7+Wi z_m(i%k5QsjN#vMoNAs=g0HR6Anv$wFOdzS{B<8{Q^6UmT%twIS@?r@na%QKer}sR7 zHh$HR7RdV~9%~Z&gW{T*r$mHkkgbLhn$AJ-V5n{55wiS?-xB|iM%oXK5dT$f* z!wCVo&4rf-%8CxuOvz|p9Sg=@aWBy8029jqoGX>V38lGxr=@;p`_Mg%vI&@UfW6mn8Z0@v}`%wWj zNH^&&2@1>IMjYGOB=!f~S0_g4&eyBEY8CWfL=Zk~ewA&D%b-F+LfSft#Qj-{G?$fY zejaBH9wY)8dVSD9PUG%Kq2ZlemRj%&392>02wo=L0#r4tC*+*ClTI4Ct3ul`!T$?y zQ>OG#2)bOAGU68`lRofy{P{Oos$k*+n-W?MhvHs36AR&EF7^ULvuyh<|HCnEY_pU8 zLU<&i#Q2p7wue{Chyy&?9;p`Tfn2E#fAP@=-PO|iR%3RSjshtFI)5(^rzJFr^BRWB z*|-+TI`pBa{Hug`yvzjq6 zs9}G~yA<$)xzG_c!p=r~T)@oGiw<1s=N z$CoN>kkSM6Zzkx8;#pYA=ZbVH&?gVXihaixbX3 zLVqL>3lWsijlWdyqy<$~ZtvuA0*Ns!Z{YJhN8{oj&su~ablCPCnVOmor_~%U0jr@` zK5y*_Xcnkc<>s!7D_r5j%AYZYL%H1L8#UJ-9;cc8i)RywRoMzp87dI$)B9AS)ueM-P z57ZY^O*E;`VM^}GYq9llQqqdZHF<-VUW$6harr5ChY{!1)g`=9Xi5&_ZRldr(~11; zC+1(IRyG8M@hy|9g=0dmsplQ1p_;4`kglm`%Df3t{KD$JU$?{T-zx@3{G8+MVCLN1 z9Q&K{O45J>v8_`OxL@!Ny^!)e&CI_x(KZ^2NHP-&Z4Om()AnP7nC?vy((yP zxu-n8$Cw9VbL9T?o*m8(IPg6LL>c{u7s}(*xdv|HO1E}wji=jKGoH=5+JZOxQ^v=S zBC6-;|LH^w_7?z12ig@31#F6|B@0a0{OKnmMQ-GESGk7V>VB7S|HA!uKjCd!T82a( z~J&Wv&AGkKn?(AmFYS9zn;c_Mml;YV~m?eskfn{H>U>itps zm>?5Xxjvvc`m}tLdJI|AC>Px?h_kY?IxShhJTOn(L=Yn~Kn<+u_UntmC(sj&WlvYU z9j%(w*C`kRl=gInk_Ptj!APah!WP_fk;c`(54nbN5q)6hJWpw#zldadJg}lA7bfrFqd-0g!FD)% zaGs?ZViX_AZCeEE8^yZI1v-+Xt8>M+czfRM7@UHZY9WFE>GPtB;~<9O)bDm1{37EU zfT|#Yt7mfK!m*F5?_^Z8SEyU+bhPr{lD!=a7giFFbyi0IqI7?{zh=ImAP@w)W}s8D zC^h#Ow?}i(%1}Czd6LCE8}?QAJpK^z0$X$r7}H#(8YTF&9#R@Bo`kP#w;jeq$D=>Z zbb@wtta$?CQ;+5G&MZwdO*#$B(QHIKO;awd7TYoK2|3EX;t?KOdHut8p5)#iZyaUJ zQlGi*pF|`Va9qsKfA3GSYmO%H2$YI`m5b)i@=8m{+lGiMfGG38(dha*E@*V{r#rbK zC9@%_f%y&gm1>J^bqg<9d#}Ix_{0#PkNt%72N||vdIuTJwFJ)qeJ%_s! z+_cvNSb{@Ecp`om?37cj0M)#x)#EFP)4KtyJHa)#H}EsEVHtc|0{R2S0w*^ZJP{-dK%ZpZ3zzl;*`twsSglCeF6oT3C26QnD{ytPRf7T5Ad$MJCuxxE* zs@QN??cr5v3L`^?$P1Ea!9rU`wvxdEF@c?hyoMNPxa{7JvMmX#UPt5jL0q0+Z(-g` zzYZS{ChNqI7=FV}VCpLsIaofOxR0dXEo*`ehDVEozhwH7PtlnDDZQhkqxmt9QtCj+ zv@~9zQEo9)Y4&cuu1cGfgd|MA(PbIv8J8D=cHzg0I{TX+xz(SCoX5I8FNuEN=pRDh zE5d`E9Ny>-o%qDhU9x~zo;bz=*=t4?mqMXbZ?Qm=t4^0PaIf17{YA75qt;~=fuJ&h z?!b}9dZCi9beYP%JK8jrR&9bR)BXY`F~gSX--*J51X4epNqlw@T;}RY{Psa*VBLQo zC+R5Y4m;|yw2t|CVmLGs`6#kgdAfv(;p%&iZ=@(P_D&w%jYsE*XcNuqk zL%aDp(Nd!;LB7LO=MKep3|Ma&AGjaNkVp!AGBe1IJ;VE$rI&2|dUS3XC17X0GWxj$9i)!A9>aK5TZO`5BWYY6;m{_TPv10+74B5nj8iCtf^VWHC`BIMHd{FI2#i zKoCaM1|c+1Lav8FWlt(eb9?X*sjO2~c_VTUy@#KZKcl`5du3rR+vhkPhatngCgPwV zdX+SX`~@w-*gskhDN!)QXu~k5mW*nUSY>UZO0BET=>w-qmUMQ(Dl*v0gE+6ZdCSMp z6DzX8=+Ck@(RocROi#q+G2)l?TXd?I&)TW!i%tF*78GwFeIZdum@MGf1;!t7a=HDx zIb>VtkENHUn|*=Wy1KTZA(k!qkhg7r@6{3%SxXoc!X-6r7lApN6H2RRgxDq*zMt`m zvW`F%+byIZVizw?>8cl(tnMK60x?ALs%Y0KE41}98MCyfDx>Q%q2%?_+`?a)R?A@1 zc;;~}PBp%=MG(~m=V!({!!-oPp9?1o8!w^)OX=S!?C08n6bsrOw=7b@EYwT15 zjn(a4@phPB`J~8@he-Bfh8iKRj zQG#dAD73^Cu$x8vX6Rku*H03A@6HOv9kc~C)k$zFR=w0zbz7kq@F9v;y@D}2W$ok@ zjNk}n1XE%{J>64~#Ytb?oW9r5xqzv{Ae|EOzVuM+;HMa8QE`fxFH4uS>eFoS(CldU zmtRVLI$TVu-3{@2N=pUd`}2MYg=3^XqdNj=+Ya=~&od^#zA%?3XD!A~z%3E{IjN;@ z!K+pDn)oy~i|Fd=?&8A?B|rKOa8OyjSG&VA^y(d?zp_&m>o+F&z3qGT`ro_V|8^DZ z8X;WkXilMN-E^pDOuYQAqs69o`yBIIq2#BfmuOd_(;3nml-WTGcY`00m zy!ge;c?*S4)BCzK0wscU>&z=142HQ@Lgq0)O>T6UfLqB_gdu*H9;f!^_sdCkV1t*# zP@X_5TJfSgCdl(k@{`#9xfZBCwf-H&8!J3v3_dA|&?c8KF|#)6&*T3<_*9hxA~WeY zMtrNeS|j}d=YMxoCcw)ECs(6{1gJyXoU9K#<6!v)|IeJEd4q;3RL*z9PvV*N#KASS zy}!LeGqO6kBfxqcI$_&_=RuNvXlJsPCl4#BlsIVuuL}j2gIq~i)O20?1x18A5|~CO z8ff4N4w_!}tc(w=^T}gO8X;rCrad)5rzglM%DNFxdZBB00ljj<#|B+Y<4eI;&fD!* zUIwdxA3TX>?pK?H&>#TWcb&$I`XxQ31+Re+mF^@`MASKvh8x-{+Xa)api^3oVbOH& zKsFJ|=C3K9;_~LRTQ>xS5O&ot%|PCNrdB#1TEEHda}zln4Q-7m0;n0~*XBgO)c!4unnmR!q-6K0N3Ch9aWQ z_5lV%dH-rcm?oe_Bv%a!A1iTo%`~VM{sNNCh-=sAt`xzS;cJVhC_pWlt6t_Vz%9^p zkA(MlS7r|sV3148k9d0qRtA|}XW#e?Kz#qZ(Ys-yL*InT%qN*+XkS7m@*utC{GY~_ z8RGsy<)^!~^P4{=_Hgm3G;6o_g98C+*R;KeCbBm$FElb+f-06R6W9b24s9mz2Sc)$ao3ndCCJft#h4(CvnkwdcCy>D=Ag*I#GWykZp}jP!1*9@^OI>eHp&Acs>`c z_o(G1&MZJ*wz4Sj!;=UfcDxm)Q{>8wP{$u*sIfOtCOm@pcc7aQ#`0dlO_mr)C30J4 z9zXH@XPzQr5Nhk|+fCS)kAL+`C-d8@e4`%<{d0T1w760|+Uid_H(or{yIx2_(8X_0 zkh_K@z_rFmV&{xwvyAQP(>yo4G@E+EU95W}r|52DV12r+uGe~)(%giM+&+=)xe`B4 zV{^CuYW_@~WE!$4G4^G@!(RO-eJG8d+t2e7kyNi-s`lN9;Ku_x8N$d*-oDw@o_ugm zV(j{jURwQ}XVmwtYJhG5M0yTv9UyZ_cJ6T!*FrOl9a2?&wr2fdE^ z#;O$d-*QnrE*HGEbBpV!Cv|8-2yopTUZEz)0&-}N|P8NnXzzP2f%cKG$1^I&{Tw<@xTY{>hQ?_C<)3wRzFBrvZ{R;O)S zk|;K+pkrK{&h3nm(Y)y9QjmjUds8>_EJ-$|4nLsq;h3b_@SKf5uvP#(FaL<=`5;X1 z4(E9gwHoTC1yksbVM9t3TgcUH5zW*X6?Nn;{CPube}@+UAMEr-AxgbG*wS=lF)Daj{{Ye^Re5%&l{|NW1WtNUMT%n9cp;(iml}K z_`KQ7SR3>)Wl?p@8}$f@M>`48+qWAEdtR{i!MRr#?qo!;)I*>JiR|6bVwx#mrMVV~ zRk7L8(o3?wm>v{3IJm0bYQ2q-x6&c2Ep4Ry8H%Ak@8@rLhsye8$V~A`@h%2K}ocS)790ry|)+P5plbc zApc$CC}D4Le?)RdMr0s{`26nrEc)KhIpE)EWkkngHGBe+v4KE7+z2#U;Ar2TP&k0r z*B^!FpAMk(M|Zlh0$tGd$Oe=IWcsy;l<*no2tE8-b5uji>aHHxn+fw(dkPp_C-Ku* zS#NUOFYV7{1U;{09qTI1R8n1Y@;8U`iOSzNOpJeh%N-%!+Ja$jY8&Y@y&O4+xdsCt+czweAOeBzM?;s&vtmue&($ zxH1vw<7=#Xq$7}IAbn{ANbR~IRtut^5qd_eif_Lnde5`N!4=SkUa3a^*$Hdw*=*Tz zH$QW+cm7i{5t;x|JfE7;4H_PNtb(bI*JSOKOtfrajGi@>foBrl$L+U`U#bjKew^PF za$le2XQ;X>mu#>Ib@Yz%9DSPbo#UI&ELW5br4P|`{m)dx^}kqiHF=^ox) zeR|-I;t_vx&C*p}?Xg+X37__KC4!Noh^FdHEg|VxZ~aUH${+0yHf*8r?hM9#4fq~5 zwzl5Q&CQ+M|HYAHbQrYy4ZJ=+KCI5iCSVpDGo3={#y>Pnwg4To5jZ$F2oDec$x%cH z1>9DgWKYeU1~G2ZLjua))LP(bwR*`^N4N38g(zQbDPKfc$?#*R;Ft066I zOzRa+bk|T&uoIZ`B)7iezmQN7;q|;qlZ=HvISOzeM;EG7CNQ${6tP78l- zkf9WGUmR0#j#7#-7=d^R(tUgH)&=36ta^b+Vu_#q zFq7QZSR!%bRhMjjKuYvmt2B4C#xngU)8C6pkDcF?^-`xXgnw%Y8*GfrZAXmtW-7RD z5sE(}{*_pBlkreG{+`qv&I*4tUG+$pR+|7?eu994?av2)A5eG#|M?t#Ti(0iGUZhI_q)c!EGgD{JZE(Zsz zv%1Yy!s`K~uf~KB1#CWv2V;>>Uq(&T^?sluRCMc-HT9H=MTocjq+8hEv6$bn8AMC?s&Uf*b>5ClGp^jNN~FG zLZi}GMV&g#Ip1+%UXV1z5M8^g!*Ss5#&$n|9KlrMV%DT_lCY!HMU&gPj6EH`CEfGZ z)9>|cjw29yu9n%ab`URbJ-H&7%wa8UeTZu|*-1Ab_qB+ve>x5<>*dh0^X8P#bTzCD z-r;JNUZUR)c8BORs_5s{Di_Dp7n>O>^PZD7g~aZ-t+Ku%Ky;9}c4nAb4J~qHDK3|jvr4EU#Gy%W7J*DvIovVVJXE;5u`|rkOd@#-}&0$r$Ls)Zae9#Y7SzEC2QB z_vNLfRBjAJZufu=UR1{0G>+{t+T4l8a44q!3b-bJxqz{Q)k>LTqPnwog#)j5$n#3( z9j{an%pihZ8QR}x=iaOpI-q~c3h3pY@X*jE_G0+s>|6HXAF;W&w>KW&hi7$;2Ud}VCh%bXsiTMqIQ0h6H*V`X|<7+Q?u!#_qe>M@)qcVo4=V#(;Y#2WYKbW zr&lmSyurz3p6cp(sHJQc}_+0Y}tOLY}Ut;Ux|oTr`scNpNc7_o5P(-=gp#Ba?SSM7~$;4ezX| zH@w+L+E(m`(t)LLRMgNaxkW$-?f7x!YI19uCBOUb;6}?bQ!!ST?q?&SZ}$|Pr=iH+ z6Kokg8$XB!)6t)Gdf)04W4_CwLdbatrOR@g@0hDGd0E4`Ta1@2_g0{*7V^kATv;i< z&Qnj_v)*&MkCOPpZl2Tmp8Q6{TtxuUK4RXuu^J4)XuPUfINGKR(QogvQYPl>aPo;| z#WD*u^njC+4h&846sPtMm=sH3Qk)wA zZF|0}MH>c1u?r#>v@kPQ)3xAWMohR zEPRGTX@mREQ+y5=1%rMbV-$dpq59)ll-w|v7~cF%Wko!J#?tY^>zm`gt8TAYZJCX# z7*=xjtpw-e2MCuSR95E5{p*O9AGjFG7&!B!vdIc#{4t>SPsZub#aE>f(|IN_in zOs}E2<7DuTK4$?EeavSIgRB1D;d^SV<-i(oOiX7Qm`V?Pc%x#KD)+%j}# z{PnZ)ZiO4DeoqndZXMQ)=#Qw6nF(PlLPCyLxyT<=OOfXV)Tz6DGAS*K@I2fuLSNvN zC_RHR_mq#EG<35+4lO^Lx$&tgtDW;e(9{|{ zrqBJ8_idIcn&=z4hwe0MCqIt4B|OmGW6mUiS&^Klq{@@PQqN>9!pySJ&mgzKbnhnK z6d+kG%yx@+zAOD4%J3V+J17|fA`2e31>VmbrSOxDp;;j`*xwZlL&$Tn)VD+5RLpW7 z>nb%?$?ozAZ$XLVcV4(9#OwH08H;9&5eWh#2ua{N9VMI#j4hT!NrJ0)wLKvdN|044Jq>>{yHY08^EI!{4Nr z$jB-J14WyYnd^2csIU`!s%FJd(oGcO!_6IDQa6 zcV5q|&a|;10SPOKc!$`-UpW(wdqMZ&T?V?{AsHQ8H|q49-UcisZ0CssvQQ z!Uy_LzD<=(mdz%eY#~&u4bsN(yXjiVk<(0#Kbk!(MYj@G$;4?tor)mFvbq^kI$P|8 z$Hgyc7O4oEE0|s-->@hb^nR=gwfT5CRnH58U-)LP8@$WC_h)z7jcXh+OkZ`X=k1oB zHW4&wH*&wE4KX!v*T*H6krglvGTHv}`An1xB!m%be()73wx3?J^>!V6j<;y|U{K1B zr!mC^A`9z1J7f4cS+{hv(06a5qv=)z=aAeOXP?EMNeCF(rdSTY&vvoimRi=0alQ9k z<|T7DaNMfG%Nd#<(WmyKAEY6s*Ft_tCf@$3s!x0S`JiUXLrK8k=Rxh+8anF{Q{h3F zxr!P4KR_*4^=sKoPoa1r4H6Ei&*Ie=+X9U~25j8E*bkFc$jFG@;NX>gNVX<=BP-Er zB`so2mAI)tekMVSTALYyAP_-8!xu4w%3*{&jSx-l*R)zR#@D>nl*+vZ^~VsEnnN0R z-f>M7rm;FWpBNu3Og$jYa$$WF{1c`;bB2V1MUFrP_W`%-e9y&x?3sx3H?sek$Dgp+s4SEyMT|Psx=3+CMGjr50|eDh^m8PG#ypEv?O(c@BI% zo`*a&X4A9Pdv5lrVAU)iT>hw#iiqEa9K}Z zZ8@EJr>jf$)iiP)W6(ajj}Tu)KV)Y`9^Ke& zvo^UMp=OI`5h)6t80gbPp-i)UNF&)n7D>PxuiN2z5h?Z#%Dubdr(;8Qv25bYrhWwb z(TT8uZka%@qM^?=ilB(>^{R`To5mujPHymh;A6eV0(bEyaX?I@v@Z>7`wf?W1BTfy zE2)We3W0mG7S)M(XP2)-p^j`@%Y&E66$fvQIQ8VmxP%TmB*ZW>z6W%-m`=XInG8!t zqJ}p==D`3`;z^wT)d^wK;bfv_#E7VnsQs7HNe^dwurxF@V<0P1wYfN&L%u2Glx4y0 zY=Y#BrZ&^Ob+()0Savy?QS0<8@?(_UZy5Ql?P^Tyh{9Ha2i`sbqP@3P^y2(4j8qZ3 z#6*fCiOMPO9J$r@vrD$|Q@&1^3`Q5zo9(`ZeQIvfK2hYe?4RHJ9j4toSi`j(lzW5rkwsl@~2q2zEqWTzmgaoaSqrLQVFenSJ(9zLZ-hm3Tj z&~~U>WvpMhcOSh=L4=@qKuYqKrd~b4fO8k*<7b>@(YENga$iB8WZOq{&xpKwd(ajh zjMhhCbekfQLFsY-3;x*IRIDJ|G(zxuboKXrO+TF6@sD+NqvN-YpD8wce6U~i2^)8B zelcQGX-^lWeO9^4Ps)|Q)*Z#CCzHU(&3vAp;+pPe<~Xf%i{Oq;C&3Rt{l@bBi#`O7 zE*&1C_rK~l?#mz3vuaD`Z zSus)_dy8TIDlMdT%(u$)Uz*p@vwzH(<;nKl4<(E#(jRMDq|#x@3I?r;yGdMs;8aWLk0icyTV3q!_jg^4Rl8f&GPMi(&51ej|cB3L3 z#{A9Iks!D|++AEQ2a-lG>2N3UVq82dpZEA}{Earxr9_=QXx~^_={V^SAguG+EVscF z>eb_5GpPI3IIhHgao34RPdT*VBnSmhL}F#dT-jb&Hn0p5G{#(jnd5DIEezcZVR|&HszBzx#Rd%rLVrcHCvpo^!>guF^JFJXH7@5B}MxE4aN< zCHunQ_p55C#Bkcf9e0(|l)=OcB|`cNl%IS6##*oz=(EMBsXf*D6qhf26;&R?-+IZd zbG_nRx?vVmPt~^9FY)A}PzrgKcfsY5#rZr!7VEn|HX66=joV2R9Xb6fuPGtCfs|PY z6`K;!>FpWjPu;pGvzv%uB6iEjnos7=Ny|NJbT8=`RTk_s31NxyJu0EXs|inSB&E!@ zTy&Shm>H%~CqwQ3fD`3U@`(hz`o=nA93EF`MuhOQbu_@h%@&tQU2Jzp5`x_sfsr+P zONQ*Fn%^e?OFCD1B)v~4&`_wJQ*m^@H@Deom4=Z2NMP7ozw*E7#3FjqTm7M*v|ZiY zbh{m`9u`H4p*n3`+$Pha%XLK4nTD{itAa@?MEywcjs9HeN#R=i4?>?$$M+rDMGxgm z9}F3PD2Yyno;>27%d*VF2x6)m?5(h$R%B|NJMoG?+RAOoP?oy^z37CZVzi3-|lI9l%-QVuBRspM~65OJF8L%2HwgE-l#QR!xPt~W`&bZ{yI^&^~k{RKUe1Auizq-UuVZic`Zr?rB74*A(PXbegS8dZ58tG>+S-n zp`n4L!}uITvAivi%h^qXgZd-&g18|mx7(HrCE&s+EiFAfIjM>eJ$d=(OYlTUK$l4) z4%R6#|5`l5KR}hRCeZ1RY<;x)j+2|4oMd_EclYr1=Y6F^#d_9sMcjK_rXqp zWxothAcqq`epeNT}9vg8cQ z)aB^g>_8}z<6_*-w>w?`WVc`QmKzD|1u(Css^uUItlD4|487rJ)tk#5iqDC+0BOOIM>tc=x{E{&3^ZrK zr0V@4O4ge6a>lVQubW+vk*+ISd1}uM5ol?CZw1|mTO#znXrx$g zM@x$;0n0|uSi+j@ms|N00G#;~dW|ad1cZF&4B_)!EFFx?d?iao6Bnju(IYaodKNSG zBVOW{CYL5>vu`s$LbY8tGrX=XKoeFL;eA8)`Zr}35JfBgt(jiERVJAQ_8@1;(`ma- z%Sl{PftqBaRjUFW=g)}Ysg8*z1|?`Mkl$R9RjKZQ<#_xX?r9<~Hy)!p-J8SN9?kKY z{@xe#7ZLX4#Q-Rf&OX8!67yFAxW(>n%Gy_4mW$2RO?T`ZmW!bnlwu!ST*rP1>!Hm` zqlTP}F;rDPo1^8yEL!PRYM~d_y_7mDRXp0WJnPePJt-m}+5&)`2Go{&_ho?^Y`viIC z?&NDO2eS^c9}Llkkp?$Eh1Ah`HjMdLy)}}87R!~hZv9qU)`+A>B) zq@w9ZpnC1%%r)Xk9pQ`yS-6m^73OGlDJRc_QsD^$5BW8ttu-5zf?YIw6|qUO@9svH z9(M`_YRVtd^H~@2eURPC;;|pdu@k{oL9RX*9I7hPHxA{eHt=Z(}qYV|N?bPca zgn*Ee?A4Rp*eO=l+R|MgEZXP-NljRP?dufvOY_t+6)Kc2wYT|SCf6?w=tF%=JK>Tl z013!2g$pC}9ab)Rfb2t_-KV6`V|A%~;=82GtH?iZS2X-%5N3u4$VZH!LS|5JjD2g+ zJ@&T}Qym&W2A~qZ&yz}xr29ap)_{R&ayZmd5A$!g%YOHJ@`#feL7TBAf#4?x*k(p* z5U^k9z!WN$m^kUlQ*>u8ioj+T>!B4IHz3E+S=zfIV=BU3qtf|G*!nfbtS9=R8}`pj z8`WtO6)Am*6XAHpf{*mxn5ps879P<(HoVVQuxzQLNxYBGc@Z(l40mysOL#dkapdDo zo-gfb_HZbh!6u+b?bDxV>oAcp(ZTh`Mbg3eV*S2T$!huM2Oh^$3`R|VQnKS30(smm zTZ0ji(IE{Z;nADT9_OhR-p+TE}!YPUe2z6SxCAwc4(E( zihk`Am(S|Z!}NLMnfd!Do*FV#`r}zPH?tSU%Q*VzQ9lkXu6%a1et2~Vg=QbZ24{KK zrgJ_kMDXB6dA}*KO{4Z!Y++yXq;~TP@`|tHd_pq`$K$)xZ;FyDSjg>usPc#ktbS9` zZ)jM<9}+PLU)wD|bh&;%I1jM4-rwIJ6>$eSQ=ES0U&^8I`ZGRGZFb`cp$c+->Brqc zsgOGWg_tS5Tm~Y6&!Nt#VxpOeN2w&?vx1^~w0YNaJLiW(PVSkSU*SeHDg6e~(Vilk znFe~KcZp&bzld>&W0n7G9lr8Pd}S|TFCz#%bDfD;h#X-Jk;`_$Zo{6mcpaiilkLUP6}Bqyy}=x&!9{M07G3WpU{XJu z!>hp^6XZlHA`cafy8`tF5C>78Ae?1s5~iU3S^P-)(Qx@(IoSr9@d` z)`I@&e#_r1gFnQg#!}=u>)G|;4+tN*{>aBM|L}G8?|T%1R?w=|GXd4I-_^;Mkgr8~ zY*(iaT%MgC+UNlln`Ke+%w`YFi9NB71~^gN#FH0lkvWH8yzT5~+n#sl5%>NMQGdR| zMpjyU$W9->_PT%+*7vk+n*AY{&F<8>c+v4aN0XK7IQPsCnoyy=`#LlL2+1-Ea}wB5 zD*tL^e_EhF>$4mQ&yocF!$QiiUN4aS-gu6tm#nz+>M5#B;adjvmzFsiyND%gtU@0+ z(KZCrnz;QuY1BxPMeWgk8G=oPysHM>WKZlRvn9SGMhppgn$OzQZS0c|<9~(V8%1 z`U*PQtWg({1>P-mZ>}y&sZ2v8nWF@0L#K*dX_g%V6&00*0e{6si)VhGWnd(No}{OI zY`xD0CVaT2U-aF>g4DNP0rtbQH_f$bT+fCzN%d&jZ1xzH*;eKZjD)3RI(`(mrUr?z zx{9w@s&LRtN=eiH)HoL@oOXR!BdTdN_E|MjC`%~pAfxkD^SELm$|lnz>AX|Febe*{ zpl#zJw!E$czFbc)LWI-~%XD_ach(=YYJU;?JTQ?) zGMB{7@GYfH>5~as8^ZZ0Z-e0j?mGzck6td-e1YX0fFtdCo zEyj}77gYR|c9KTaHy7Ph6-DBZEXDfe8YbhC^*-{oCmk3>?=wb=HdH4{7FA z&ZR@cp;kNmGP@=yC7T_d27ml!cz2M2hF9meewQ3Q8M*3ksH$gGYi@B4b<_n_w(DFkO1-b)$d7|qQX`QSrD{ElMmzhOVOb@<_*IWOhwODYH%0wiHjOt*8|KP6zs7$p2o}+st8eMnzGL_B< zQ`i99nMYZz@saCCAaeVw#V znCwq#**{{|!Sqf8@QX3)eS!~fq|}+DR5yABb=3L0eB-cC-$7L#aOht?+#1!z! zPt0i)JmKTW|7->yX(HgxOX1Kvv^N^%3o(6j%3&)j;`mq4%QjThq8B?EMK|8y-wSXW zo87Jv0L-I+MZN=38E@a`?^Ca4#ja}4j%&Q z=#`!@{6=fee_C<@Qkhm)Cm#1(T;Ok}Y`woW%}LPwx35jZKx|wI3*w3GLOz%Nt9*^d zg3zbOiV@2{i^PV*I3y?Qz?^S#KuveQruEg!;J_2Qh|E3}cm7DQhL%w>MwCp(Ahb*I zL1%*6pfkc=1Hp|)>(az5n=qKn{kZhg-5R@V1zpyL)U5>}opx77&(V%9YM3v&317cu z*__Sfs$F;K#Phh5Oe(xDz6!Ox*&jPc@aj4Dvj`U~vN7a(H^;*=Z??JbA93&e)EF~v zV03(`R*weUR{nu4CDNGm|FQsjf)GBEN4~Pum|eJkp8U%4eGmp^F?-g?v?h3Y()(kt z`l51Dy6p!d{JI+569V)r42*DMl&~ziDr88|w2j_kMBlZ*mBj8qR<;iio2Q$;O3x5p z+A!gSc%iNb=tV5K_nbk<_*JBfD2N?>{Mv7Ai<<6TkUx5<-!Kl0aTF@WrW}lpPP-R+ zhgLRevi)Rr%o&yya4G5N7sS1f`_4+1(s7A5H$f!`Bb<+#5U?QY?hxp&E^58JeIm2K zCAD|={`I*r;jk}jXQk7^Y6+s&{wWCodQuEsxD~FQjg7AiF`71JqkDydM%iSL;4=zu z{)%6{%5tbdp9q)$W*6YeZ8h*cG9DhMh?V&rf6l=r5wse6N8S*VGjYrsaAYaY+5Vji zr(Z#AyjlnLO@F!%h10K)(s=*Oi_?S8WWSXGbG5->oP1G_CHu^k{hfn+v zu{**ExiNg(>5&)$1>rR#IpvaAJ)v4>0|wnqML0oShDTQfjm! z3B6r6Ri+O;;i@w9Wtyxb-*&UrZDH|uT&VrUbB&~2Kra{JXZCQ3USXR0d&0$ue{dHx zCl~BIr4E2ie@~gS>ZS4qZgUt{*a0aVbKhV*o4)a=`VO$Ll*qSVUINA4BE5$U;mfYq zJkcHaG~tM|?yDd3scz3WaT8f{V#GMZ=7tn+lIqw;aQZILYs>COn=gHaU;OHwsj3Kv z>S6Lgq{k3gnlm}Q3%lFxN)1H!|7>bR*dMy)k^43Pg>b!-@dzyx^Y2;#VGFrhmvxCu zD<`=3l9?ZF>t8VeejOfO-tC8LSOP8^=mwjO>qy49vj3#t+@z1lX9}k^BnVxQF0Y9s z1l6SkqvG2;i4))=&;fCVHtUZ3FFE`H*TTvxLlv}UH-TBRLJbi$km(yGh|fB!$MS|J z0mpbC|C!N-{CgtPhlGm;Kx@QiJ~KZzCwg+BR?+mN=I$~9IBSAedL15c^^SH;FoT(I zc_dcutt8~VQrwSsW;skJk=)(gji$>RZ5{Q-M2*N%~@CJ|w8zd;n;m1A-f4{^8-be-YYbF>vd40*Xp)Nbo`ET<5` z5Q3cVWZksYBG$?B=;3b!#B3BzDLkZum*N0b)7>4yEQZCwZTrs^LI`oR`=%HhV1tm{ zi8BjxBme3JSq9BP;LvNGF_izaGA=J+0?U%ehzz0j?d8$sc3yh6Qox}d+q)l~t1?IB z_0WLT&aJJo(}K_D><_1y_%$z}_}ETzu3F4z5@=O7hGj^j_&weWXG=sOMWn-K$=&ta z^zE5me-T(L-Z|94F&J?l6y3Sx=og)O)230QpIR9szpHh3(&qr&c``SL5SUEV8PDCN z(mhoTTE0rmO=3BHMiLC@*Un&?v0b}|dxxNT436b#PnT=YmI9@U9rXWq;u@a_3MI9A zaTiAAqqAFj!Wov|ez}9igV^iaC0ZQJE_nBQ>^c}L>-b_d`GA5LWvwc**hV{)(uM;B ze=$S1DDELh!?no!U$m1g_(gqMnRF~wj=Gbjjwa8@Zoe)6^4MJl-yo3BF4m@<}E>FjLY2U!%xhMjI-;-7h^>%JDKV_n(;$0(YUtm=>^pAQMA zmwkvN7?-J714*MqC*8~sl{5xYcXzHyvEbAQwBQ#J0@n9WG0UQdOj)5Vvly)?+zA3`W0_p^{^^n&38TvkBga^u@x^{*jNk?19Mc61y9 z%WW3W#MlpcQ;+=BE|6qC`Y#2evaMqSn(7zAW~O^F|$;rzTrX&eJ9pK8+ZpxI(YOJTlWx{Hov#Jxu`n~GeOWp zboamYtpkf?eXUpzHvpg>S=vUMlTg}KmQd}y%&tP|;&7A%O-gbiA)+lZV3j#jZX#y@1^S8 z1pMv8C#l~ z!YNUy`NG)-b{_{wJS<=lem#d8(k;a{8~z?OMAtr$uCrq82^!In$^g~F8O6u1keRq7 z_Md|(^g>ZB?o>)J;gm}G-#>D@`WDqLvb$Q?qFp*f7bSayO^w4T72=Q;{^`@W6PqM& z2~0TkN#Y`}H$RwYL`kT&x$_PHjgg$tqi$5Xy|))`YdBR1-0m#b`U)8r0K{+wQ}dY- zDni%I%1dq*&Uc;!NGR>3`fub2j7V>QO)*&3VfoL99eCl4hVeWIW1^>z`8QL9Y&JOb zOUi&6ucBwM{tWAamcUzNMwIp|dBhyPkR-)Bo`Gwyw-U_Hs~Lw~PH(b>l}U;C(KW<# zcKgd>NWJBD*pIbI7POD~pVrl(z5LY&u6*6mWHJ8apO7z_%nKNMSGULVvn6U(8BDxf0oAfl-2?x0Rflxl zzP?Y52_uMkt?3i*A2h59Sy7_D9lz4$keT257yy+s!g$h@DdyKK7O$D;`3^KG;UHP_ z#XdBeP|onv1#=UPoK4MUM^a=yGMb(}?h59?30Odxvq8UgRYA|I)$5O*t~QP-X7<18 z#6lN9F`8J8B#X$cnaMoD_G6hAG_RDO+gUy%kC%R{2TZu3GhgY&B&jgFE>yE=vs5zT zq}2k5PWW)jcm&t%ZS~UKlHuBUm5QyEo*ffYWsqhR6Sv;NN?(|RK_h>B*|IgV3?F9*gNJ2&?NH;!SX1S1l+4h8rh(SaI#!^Af38;V*l(Z;K0RPV?FHv`eAOCk zRzT(v)4Rnl5i_!!*$gPc*M={^1R%D>O1+hEdDQIQ^&P{s9gKiEZ|`sM2=zZE_e%T7 z7ToL2Xx${W(nP*03g$TdK;e9@8V(fuSxbhWzLzPPE7zG%Ii2EF%F!*}I<0eaavnba zBDm(m2|a$c_D!^2|C&asMNTWY@#%`dPGliG!C0(9@PrbjYtS@1tJySD%|Bes6M+Ug zcFFaX#k@eJUPXAc7;oD@HH63xG>A+JF`X*S)a?yVVA2+pOJ^8~_zNMlyO2J*$3m5_ zzK%9K+1?&@qCOLT^-c}=h4cDI6oQci-@UArC4!u8TJ?MuE<=3jzHfn7s3qcIxqcDM zerYsk91)S)=yl*R??VPt8H1jW!qS-iL-B+evs~m+U^Njy2UZWDxrh+qNCP5|5|4P_ zXpy#7O|3C|kM)ss;wFcKT1O%VJxD|^Kz56)Y561JEqk0lc@!2QLhe^%Hbjqhtrn45 zvBo9$MRg08Y@Ij#5 zdd%exypO_pB%+3A_;puVX8k-LTh`@k(m;2hgq)m2RaX^4LwYkOqEGV5t+(iBVQeTy zk?kM*U5nhevVZR(g_lqON~Z1But*e{A6o*`vp*at5b25h6RChO874!iP`M0Vn`N#) z0`n)BA|j7Yvq(1M;=eW!nvdlfp^~P2&9v9Fp+$MKs^>jfm$Ms%`t*9v_EF*q&7>pk!WGiLt8*_| zv){PZWiu;zC`J5~47mG#dqS1?&U(w_?Q1K4gapj;YWIXX;1nj^szL&zF2jpdust`s z-#z~|RVJLsVl8Bn5YhkF-dxTI34+7E0tNKZ4`Je!_5PT1cBeG@e@sIg0!WK}MlF>h z0J_j4KfL+Rj|B29o;-f%PXcF2W7V?&dDaO&`05YUw5ROdnHtjXdta`z$%tzj#_M)Q z;Gqq(cwLCJ&mUqIVaUZR2qMllW02w>EU$;^V7Cbx8PPb_H`gJwP)LPjXlXYSlsnV=XH|d1Q6yUMg5K5&tWp$HB+_)>^---J5@DTDiV=sly zWY}`b%-#Zhx?gjDk=|7t!bb|NR`-NgEYKET$a)hFN}V>Au5cJNG8Dkc-#5$_OF=hR zHKSo6e+3`m@OImyFd%#4Em(5O?dknWR=fU*H&GCPX-lZk_1M`cz>-fo@z1xoi01`u zOOUVDz(U01NWyp;PRPIRdcK=Dk4L3LvefFPdeMWB2T}QY6s5}laX0~8lP9>VZ*;b4 zr8fn6E3y5CWLepyo@YoI@x$`YZPKdD0lDb*Emmqq@z7XrUyl!xpyUcrLdEn$Gy5ke zCl9%U#0_XKFm0@XOpk=upXflUhc0$7^S6j`Muuo8l}~bI{n&Yg+$ddS=u2NWilnqo zV@pgjWzeTvYZ+M@a^(lB;j~v|afSAjEP1k?DHSaH>7O)y{UMuR!^I5o3)+keCGdBk z1F&i6yFDpXZg7S54^IfPGag&+=oaUgv-!6uQ2&l~;Bi1lfd)0IMjD>S^godS2z~7J zN5+@Q7C|?gs~y^RVtVrL<3o5u0@zgd`zymxJXY9+#!sF3vRN3bvlx500EZ9+sR{4H z34-<)ng*S4y&s;0mi1>Cscuf=-cw7QN;9izoDqfqy(=dD(^f00NL&P(rM4HJZ=-mk zb%v-%#NN*)apdVB&A`M&hB+Afft>t)ILAL*4vzWxniy3ETE1#sX{hYn-?}MQu|I+3 zgxsStIuJ);B;~~L9f&Cf&Laj6?0ao)h zrideCu(n4}{iM;kbw4!k8-;3gn8C0}-Nn=NH`;Kcvc*o8$y33=XO$czs`LP9R@?wYlcIBxza=~`3Jxzgzj4F7{xmo-b8CT>l4BP+&b) zBbw+W>v#rrx(V3PY|TDwCkbDG2Pn{gHU7n*xtR#D2g13j7<2j<8h zI5cJ;DD;HFN^z&nyb{k2-X)tja6)8}OKVM50*M=$fs#q zCEA}u?5V7(H!-=VEdEd#bNrkP9urwvwERz68$^0>I#L+(a9#Lp+==jQ+2+wFzW)9? zzndB#m0jFW_+!$}qm11C-SW3YR-+$ZU}!O^I)Qs-*JPhZ`QO*20vQZhSyc4m_V%{= z1K4-}P!*mC&6MEjIEq_Y?&R)1nLc+R#$uR=Ud?iu-PHks6e}MAh8Aw7^w-Hgo1kd) zBRh7e75qDQ!yYx==(t$9QrtJfc;&mj`-e6o+3_uVUw_0*{b-oN$QFx3QYxKxM>7lt z))1x^vu37C4wG>Jt^q6W&H0=L>_wFjB*Y_UQCKKx3{oimr#6tj-hDyA4G6Df8nQm~w31y4Al4U#sjfJJ<96?uOj$`jpe-zPUwb+3CL)FZ0;qGdOkO ziau755%D_vqU^acSy!4TDhn;Y-$@9y=Xj+EN263qkjCTW{YG=LcWc2kf8)2x^>8Qw z`O>Y^m=G}&?R%9tJJPo1P2dlr%$?tPJ@A8SmoZPxp| zK#f%18^M-n$AEZOq65wX|6FWoGSh*-hhl1VueyGBna=csUmu5^8|5R}H+>0`cu#wJ z^kEIE7}3HZ*V&raclU)jbBDwPv`cMOc@dqIGum-Ptf!|ko2;MGne_y_*IfChd#Ux2xe zZKf+!Uu^})DXRT@e@M15&`2Y?A5g)W1wXMkI?BGcKzHzlsfTdAsI0XEpwLlVwgxGD zC9kYZRPnv*V&2fC08iBSXPt9%_@99c030!;NKLk4*=>vg6AFbKWUITw_u3ojoryfa z@Q`MrvuJ)Xk}|v8u&vooC!+Dso^PmJIgG-LZz`%hvPKsCtI^yDQ`vI$w<+GCFXOkyvgLY+gJafRBJ9qxDvGs551Uuy(a zM!tGeEt2f_TuFS1)YlVyq=&`-Y7ZL3irGTBwh+ovXFBZPCa4MtOd>*7R!oRE41k{P zUrc!t1c1>eUvfx+!&|1=(OQ|3=r$OxeX}3e8Q4ns{pI~PDc+i3;T zJq@=Y&i<3QB;a+tv~p(Z^EQf3pGBc8g}CXwcXwDKnF@v~)F_+^S*O+K!B?r}8UJ!m zNuJScHiW^5vQ~v@sF^Dr%PHIF1TjD%_GW8h^jG;Ab$UVFI+TWgsosbsXifsmvh~YN z36p_ydXA^@nP$LN6YLJ5dnm=3-6zDI{GxMg3=`G|;)Vm&*M&jJbOGeeFhP(-Mw*KL z?H_8E*CVb~Vj>B|66iP}BF!fj!DhfccqdSX@aJgG(>CFMBl#+tOviJnG#=%MD`f_gqvPR)fnb~rQOsEglXxUA3j7Ef z7Wk;n2xSv+9!@HS)64bhSR2$cgBsvwKO)cQdbRYw1=Yy0LFM%`&fKyWa~mV+>3p64 znvDMMm&t`1Jh=WL?h%=~nTzAPxuN7*VwFU)vM8}WQLv-J^p3wft%5fRh#u;T|p@QPH82LGL1fo&ZQqFXCkrU&CDqo?Xm^p;n)pnx)5 zw=0GTywMZIBr{sQMr zz>kH2Xbb zyzY83{EPcSBSUd)+bScij^5EwbmsD>q1C{2l1f*cr;@$1^dF)-c591YVmTcn|9`9R z@yp}|WT159!{e%CQ=Fr1`kotR?DtGktl)Ry6;OlBqPFa={Lo8ZN+~%z((N~hs-JgR zOerj_A-ts_PSa|D7+mH=zu$k)0GP5&CJGl3%C4v3U_34W`>FSbB1KZ+Z~{*I-Gqtz ze?c1O;>#di)aZK#1Xlsq%H~k=`BQJh;lFKj*%ESK^Kl6$eTUfcg;D5VZ;2d#46`D0 z4S0%0;~5wT?+P+&e=fE0OUPar2h6;2{-_p7^z3s5SJnWw_{!HToP6b)C(lL<8dQ?Y z>qqzD)#`8I!S7GX#R8}B@|Txxo`s91 z_kTs%ZFYyrUn^Ky4~aU@>sccQ_+UE0RZOOzgV&cpx~WcR2!@33n2uvAEbiAFFR%6z z8sh^G#yuHz#(7W4A^x_t<=qq%2nd{SMY_kiQy@r)AlfkVJ6VmLx!7lhu&t4F@$LeE zvi;imKYAMNF<0;{XEGS`SHXIT z1d;xj+RgGs)L4Z6%q0+Wk}xr$0a0ET2y>rk1+D&`)(c28uoyMe>{VNw+36Gl7ov(o z^^tg=fRq%g0Rd80f-|DC#l^*X*DIE0Q=G36W85_AmvP{~^9A%`bv!&w@`GE$ZJ234 zzzLUuQaOrSiGzU8rek#`d(2bbgfrJNzlh)cw*GnE^=fy>qjsk>?SJ66#TFM`kXQOh zVAWbuQ_KAqQ?;8S0FDG7U7wknEjHt=&Jg@Z$eh3R1U+WEsl=Dy=Ry`k>;95I(KDC1 z6u_>c;rS9pzsW}k#(EL){ku%HxyCchwh(;zg-ro4=0$UQMa&I<%_)kIse4aaO(A+d z(snO7Tra#;h|57TNi$Zp#5nEpv$9zHGY2J=+Z1Hj4$j1n#M4fc+v;7Wf@hIkrxPE8 zjOEAGafMLj33kY^0vfYRBcc;|oLInD)f=6Oh~|Db1#L1_N-s38P^yLpPGyaU86urf z3da{!$B$DAR~I+FjzTB3d7GbNM%=Lc^Zd>40>lKt>-FB;VC}f3ZLvPyb@o9C2&P2s zYr&r}J%X~BcMi^#e6Dq#0&^WBGkoaXIv8FcN|~*k(pYOgH}G)+WD0er@w%*z+x|;-8=6Fm&NEmPmHv+Q%aaeg>dh43Z$10+dV=8?Dv-Pm$j}a+(qXJzVPtqi|$D zJ0TDPz=8mdj>%LA7T2%gHFuHVCf2~l#aSM+!H%M@aC5wVSWAfZWVXFYi_*5GHcR67 zZ+WB!!`G_J?P$4{}&-k0#8F{2-1chgGg}^!DuP!DPU*E_CA6r3}A)>Xn|yUgpV)ls&od> zs)7~4AO2#p@I%q1!+QL`86o;Ntkh-sg)Euk-j?w`#*!Aa$FG+=6nkb2LinSI1(H}< zdiTxB6}pm|k6qBTPv7z!T_FMFx-rRvzdP1!t`B^0NZfl z0!$nIt`}oKvG>E(jYt{~Ua4kVrVcxFsh3a8Q?BE2`N40ft#{|Dx8N3)NK^;HRH<6_ zLUzHoWZ)`k#{P?kLu$I6VS{V7sPf}(D�~^`YMDeR4gU3E;eEU^s`?#uuV|nV!zy zQ1rRf@_7qR{({|gn~HdDn0c7}gDQ;}BS`Zi5BUdS3q|N^Tt+Yo*kIcOez+ItB9uMln z%i%Ln`m!`SAMl0FzYlM^g7m?%3gjDm!_bJ+t_F@@E8$ZDO z)NyHM^5bGo33 z!We#Dq)e)sTcOQVLDA$Mt1lfDJxccHbS4ay#I1&WXMn`=5<1$$%0i7P4oIdAv^5NG zM{ACYHLC1nR{xX?g)e0P#iWJQ5r{BM6zM%d(!#XaNE~>9o(!n4XuoBxHa$8w4Js;P zy}gB0$nH6OlF)ItU8k;p9kx{vZ;BebkoBj|d%RdQ!arpGw*rxoq>(F>X{3%%{8sH5 z=y~u{D2_Y?v~SnJ;}*C@0B|GB>itrlG%?b1EC(prTjACJ-ggXQFVtCU5*iwj z{b-D)HCDa8$jhIt0WTQjvf%}VgmRjfK>^hrF$yK)Iz?tttkG>us@O=%TImc}o6_`% zqufjS-0gdiWNHGm<ES29x(!jNG#rs*+a!jD*|rq!HyF{kwymq0>9i(i{UnHQ z&*ddWo=?Jen9@=Pz1^eFdq`ioX>dQ$q_E7!bDEBqqcEs7qimfTqg$no!*VivK~pVn zY`hzQ`0;bBiU{`HRB{JrkpqMl^S%u?Lanfxke5hdL++|_?)EJ;TyOj9f(i^wP^7oYg>w_U&i z-|Kv{(^zxTe@vbNOiz5fG{#UdV!Fixf#5Y!IJE zbb*JMFb^5$)&&w|pmo(-GmCkWovI8au|M2p*fVtmqLY`qUfFPsWeNop;s@+%?eM&# ziX{2dn|SQ^=apOFDEr!Z{>%7$E3oWiPk^@A1!~xPaOTAtAi-}vPwq5V0mhjBG*{HiGmCt}W+OxLBHe0Mnvt5Gs6&!)itx--k2=T80Ly>YdD;ta z?7i;hZnsT6kjZ_qUh5h7emek-5h`%7*ir%R4*ls&YPDRY%)2i^7OzND2CN@zA40cs z2|ZV`q@E#b@-E2Rq%jRL?WyK$u}nZ*moyDB;@l1(yfmKCu8H_4o-0Wd<{X#RN6Uul zM*X7b(U>bVx5W_?f$5+fFlsS98&3z?pLQ#@>E&IMwBW zwW-Q>glhzknXImYykHm`Df0-C_#cc(MQKdu1t(+7U({D$$6WE!v4ZO3K) zH2v(;3RFdIh0VzVo4Px=I|@w?*1_@OkFBxu_G?!*2Ycp2F88EfA1OR(MidFFsHi6y zpObY5cQ-MyHEKl?NO4mk|4M%;-@9Zv8F6Qn)_m9_q2%msp+4L1hvN_p$m_uQ-rU(hG49=m_zZ)F1`-^hi2`?NzTT|?Y0H^2uHm(_ zbvFHY{^rT}QjS-_$v4&!crv%>2`7U%oGrXEWNS)&F)K`s;I;yB`+fNri5XdWpq#biB!T?21`3 zx_JEV%>o0vV4N{<%n_#)jj)i^>x=muux2$wjX`4t885q45YRNR_o~L4&~hLY`?wub zGvNYxq?lpOVK-)6XnWJGLI>e^9W!EJ&+UNYfKd28w)FE7ok!`C{^C@zvOKs_MnawL zTIpo@AYZqScYVdp5J&eR8X#M|!uo}U<yhm7$`H>xIn$^dQ{XxqmxrnBU)-lW+nJ2Y>SbB?|K&Vz!q{xU-mmog z=|Nx|g2;Jc$K8tF^+lHgzMt69rEinUfI74>=Shbf?g=$;M5mW6K%+qSg)QXElBa(n zGxu2_Q3pcJEIs?{?L3@mCbaFFffr>7=gI7*23RbJyIQj2hZ~Zpz#PLp+-|&nw34Ht2H`aeb`F>r9Gfy%m|KW^J3P_r~$(5QJD_7Xm%vD_aLyLGm{xqX|etpHtIwq3z%6PO( z6}y!C&67eT)2YIwi1@}&zXkyJ@Ig`p8`#y+zV$BOfxT6F+}^_p+)9&p4s}rzbp_74 zl*Ry)PSfiMQ}}HLI>q9jDmVjvQyMOk8Xk9_lww{jI%e(8R0#tPiR@9x(;0yJlmIu9 zNC#y1+5C8i)3DQX3@K;0I~#}HUT*Y*tU#s4L}K}4y;UYjj$cc=^%E56mlToskdBkr zo5_~zyHz9cfZ^LPP@1ItTLA*81$w;#`Tb^~9IrM$hf&~6UJibZ1^mD&2A!yDBFYT6 z%S+y?^_b(KDb4;Ba$(+Dy*A13K^6ff;`t@=Vieeei|v#cfmV>)!VyiQDA3Qvf-pqs z$Xga$l1}oUYqUkx>zFvwk4>7Xwz@U|=cv;C-G%1KVsL!@R(RU+LNTQQVKz}zNTeS$ zqE9;zt%YJf4cBiwXOVAx7Q{%4px<=8+aoW{OPTI^95my;7dNziWmtNu{25<_Bp#vp z?q2$Mv`<;CRzg(y{$z|xM$K+pyi(UNSfM~ch2huy#~!zMc65*Kjns{>sDu2hEdpzz zC31uf=bu}ZzCi>+ry*M{>v5mcChR!ZByKEQuX0JKsh|F>*_Y(y4hI@s2RxFZIjE=L zgz%YC++1vX!&cfaIUL;AcKP#mW0G9;;Og}Npb7K<%)ny5`>}41vGwwc8aQrl@r&#T z*~Wb>@e>F_FWtX5SX#<9uqx?W>i~6KH*`T}%$M%@D4BrGVtS#^oeU>6=c@`05=hk| zMS2{6aOunVx_dgI%9fMhc;0qoyR)pu$aPr}v?4_0{^MbC3z z#g$Rr(SYBk>bdZgn8}VdW)#{NGpw|L4Hpb}amXEfa)c!R42c4a=jt{|YL3=Q1?LGtC_wz*hD8WveNyt`P!kAvaN zSZ=5XJ~BS*?Cfk@>L=C6Ii4)~1GvISO|RQQI>yF+QWEfU%))ADdC;E@MrgSRLjH|3I9vg#us$(ehGU-IDs0 z9wj!G|0+8Yq|CgRydBMfkHiO=t}UWb^Fu4)25=J=CMuIAk#=c&$&MZVjP1bf&YD1( ze)x35BTJ%%1q7Tl&d0TCNIC8r9&dS@8#?za*G{K3VPX184b(Fe&y_{UZS`>k>!TZXv&4K8L=ymsO%Rti9 z+yjLtxp_e#xq7dy`}w^_d`fTauj~f{LUHxD;#+6Riqn?`NZ?7r8>4{z}hj>Nq6>}Myf7??$?A>v9yvDx{c|5 zl}x!UE50D{e65|vNf%|SY_VDEEGSr|TQO<3nDX5lMFJx=k0xiEke)ABUzKRIEO-j>woqmJdr; zubRo^T$Wi!`mEz%-gy<1QJj4g$)JJ%*kB$LjiJBSv2{vhU9)7d?9y)S*y>(867|4) z)#`dzLx*Kl07l$RD}K%WmX;Ey5q|JlDnln}@F<`V@Q^(;m1WIO)s^7KR}c@yXqSz! zVV+J~Df%=XFNDsla#l*E2tP}7;oaw5MBJy0)Oc4&8=IX?q1?781kpPet|t8ZBNu=a zdqU?U5~J8;Z=T5Qz>s)J>WKwy-|5JNKF?IT=5S0N!Lw@80N7;6EeH=7Da5%tOC=bd z0A_y|)_Irxhe5`+1UraJLeHPBeWO$PvO;6qHO#aafF9av2+`Bi!ws;<-7Q%$P0o8Rs zP+Ca7SZH$ijLvwfxOQbBn*W5+ir?Rb9g!R5u9#!o-=zKgp%b60oRUv8tFY;6j~?lE zI8@RkWg?JkC~>bU{|M}(SS9j8ziN+^k?|#3HHDL&*SosZWTmmXr9L0|w94Dy7v6u< z#l4fMxBnm)wzBa-rB!zlk=WyFuajBc0Ib6l)2SSv$#zA=!8pN%?P)zSQqrY2~-XJ~k{LogUF)J8Yh#+FoBq@AAC5rjP)>QNoS{~Rv#QqE-k zLY_rFvSL4TH|g>TY`n2}>DrH-oPHcayQM6Bpqz|T)J9Ktn#dscCOxO31gSHAy+ zAFUvvtaxbY*zzkJpFN3fu%Xuk?U*s4yg;p?vL?8ULWTyKDGE9dCl)NBgO^8ZLzQ*> z!)rIEF6REP1t%s)d|rDSzqtd~lJcEf<*<5*7}b)|=@)CwRo^6$Wyc*V zYPq&uEV{Iqv;6wr1D=ox(#gjP`#qI611vNjV8tMwyhB{S8kW2Q^;we7N*rNqsns$3 z|B?09QBi*1_c$@6G^oS?BOxguB`w`CAgM?r64Kp6i==dSBdLV6NGnK(NQbm^JVb$70jA8)+NetW;N$5-7w-x}FtUl2Fd#g8uvhYLBw&bn zjOGmOrEQDZ@j3kZs7a*hS*Fgj)?}2ZvTs zO_nYC=E$D&Kk2+q@!d*=u4+hr$6FbDsHS(5u?ivH>%m!?vK2ic^n8_`GaUT@9V#r} zc^$h-snR&}>$Ot(Mw5zwL~)#={*uDe`gePt9~oaJZfrtgPnsNl`o~379)M|**tzVA zj7)v0A6oMSh6LB464G58A6N&{X|*<=C*1K)Fc9TT>QcJlN)vYeX|!S>c5`NG$|cE* zbN}s?%GH@w(v|}3F(f-36$3vW5Lo};1{>P3jX^wgI`7x^MT;p2 zFS4RW@w|qyV3$IxK&9;CNcB#NMxLh(%g()V+woO49BFz$qK)cH+xr^(koJll2WMZS z&KXBRFs5+#m0p3aWPmJsUA&F~HA@4N*$<`CUcx7a{B3+!4;%kv!sR#?(z6obw+FEr zl0?oH0`K+lZSS4U%Kl01rXi`1>}=`Wt|@Biy@GPKMAI>iejc;)Sx@lFeig@b-<|{{ z*$yW|6O&vE9$8sY_{>Mb$5-D3h^O3SkNEIPJL0V?8oQY_=&t>Xqyj%Yj_4pKCBp@} zYc?>V(Yqp@*aDEsG%w3fL=273LOZBm(UbTB+jfp+9>;H;DHNNifDgeFMG!_OH>1pK znM>w+B%(>5-H&iuenW8)>?`?>Xpry(4U2?X6~;|-KC&NfqLBT}r30e``0ai=BMZw~m5 zC-Eq@Q@=;kASdg#AT%{D4Kf;(KN~}k3_YlyH5sI8&C>l-i-GS4a{f{1=gW6TTJ~eO zjXyy~?>A=?59_x!yEXbs&z526MXqODopd<7jG|0CEmr?n)ZcWO807R~8-5&+#j7Zi z9Citd{Awle=-fBQAUnh?t1G3V`-SvuRyk$``)htfmp~m(c&E>dBTQs z=Z1g2=InP2op^%YCZux43XTt)v$(9sZnrE|FU!Ne^y#;2rPVyTkp}|N6l@waA zIEpHpw%Iq)(^-0e={0;;uZ5O}Z|Njl1|>gZ^f~;TS3gi6=a<--`u&HFLRd6u=u=hV zd=0w3dfceW>&8sTrv&kf;bp64oi^E)Iv$5uFES;)Y_mRw#_f@cg9A3Z1}oTCviq{? z-anMe`vWm&8Tb5nJKu5S$^sYRt&d4TLZ7|}@f#}qS6}A_ii6QP%12xg2ryaM*BP(W zUzS*zx4BYk^4&9?Y%?tM`1<$ga0>UG50lhfe-HzM8dniy!l8I z&s)ujJHyJ}_Y08rd-I~jtjH+Kt0A5(TNOx`G1jPxg(PyWlFljM&Ej|_4$dAic&yzK%R4OA}@Eh0C zId@5KcVz}vE3b_|1WwKxRAPT?|6nN?rTa>P5rg#^+}EGy)7sbR&1c1jC*SRqfX$8B zaIOrpZaMU*I{DCcXAK++gcR0Tmo8l(3e+##46*krzsUG+?gXF^T2vu|m1x7>)4Ipy z3s{7{?oy6}2B5wkuXotQTQGb&v9!^3V5Wy^LYdD!w`rcX<+tF}o3A@W zHar%%4AS(yRsSl}t|tFr+?6`aJXLnKh<@pYj&Ih&Bh#zSe2$7K%r1eEZkO}qY1h*Y z`HXiVmp-Tk2k+ja@As{j@<>WqyH!@owunzNyc9z9>vCY0X9`>X%`%YEf8Z>d^zMqU zWRP|fa!*1?qWGhXc;ZZ=LjX2waObJju|n*NovcT3Y+;3U>qYN{c17xBsLgy6da^DR z?tAkMD6ha+7Xxl_RF#s;1-Q+5$PrbeNLl{AiBA4$qZ3Se%Qb;W#F>7Riu5r%16&tk z^gd{x1cQ_}k->VcV{eRy`FY#qnDG}ii3`G>So6(=F^wBh_=Iv4Oc6OU@@;DA#!F{M zEaG8y1|l}d5t|`Nc$amiNzplBPZhv(I;FpRaQBJ`$k6z$k*07dP;VW*SoGkm-Xx^7 zZxg~-erQJ=i{%3KYTPJHQ+B4ASTqP&&Y}5-N$5hd?`XC8`W)UNxnys}SvX~*`+Ew-YNf^Rm(G2RNxh(p zGnV=H?hOlZ0HinSt^t35Xr~ zEwX%G%-cA&FV)}89mwI@KN?j_+krE*d&sb=8=3=dyGE`wj$vA$ft?^Tq!ztG@Qc)f zh=-$$VmIh7Gzj8ni&a=NKe@WxYvW)$SwL)HCeSYr%tdSWKWF8hZ7({-0gT_Ptx?|{5|aoSvfba`uRzKEBrFsy z{Wqf;M&2RD?yCDzRD^`u(uaB$8I1EjMIo>UDXEoDA7n|8@9KcGrt#qk-P7-eeTu?x zS!z6?%d)~y(R&GoC)^o&oIAA@hi9sM{Pas=Rd%)QU3~LzD3~UJbfn~O#JpVHPU|Er0-7mPd*=6VhL5S^e*rh)(XFMFo zF?p6)8$?^;y}mEXM7O&|&V)CKXl6H~NiSUP!~Vjp%Y?07a(EphR9QYL>%}Se#UP=i z6lF<%*c3~HvsY1=NasM`oC4e@a2BX))tDw>{oILvpZUw*Um8498r2>@2C0>_D^P79;8_6J8j59^~#G(ET>21z_XThn27Fx=)-;(P6qXu&cl#G+yB z5*?o>!tQ!n6oi(1Q<^-}#^Pe-DJkX7o44G*__0OB^GHrS~JbDgnkOkx!5Y-=w`J-Bk< zPMa)RpHoU%vd8q^4f!m8Sy=@Qr&k9zp1bh{Q?>H~HyyopZAnvGb3W^l+&(W7oe1fY zh(4^nYfnKr_%zW1Q4gwYJ_xTb{afY}1wy#)q6D*$KYC}8K>nF%EM`&H;XlOplYO`s zpV|5%+<#rVc}+V68lut^CjcchItx{wUg(oKU{qOn^LP4<)C zf>Ve?MDdp3`9V43we3^57UxEIX+T+#_}#AU2QO-;K*SO=VDKdX>!?PG)M}b9M#-{I zgJ)&+={0CniM4SO9DZ00eJ{Ihx!s75`I0<$2gAuGrioVAwcMI7z$IJT|L9I4J-;vd zQu_BNH0{G^?C+KyPzQf_@TgMWN8nZJ;Oa{5`$=YB>CIKN6?s#Y7{~|pXf()4H5N3;rpRJBcBV=njSZ!V$T>jxE_zCL2$B1}(;&+d^~PNM zX}T9WPTLy7g!*N@gBXwC2VD36fAJx9Be8@;;JGlJse~WT2C7h#N}HP~cdyqP<^lNr zIPK?=c-(o1+$!&bCjUC8Ur)b-x5oc)$qXs1pa9v zX^`>85ROSDw9f9N3xlE2re|FqB>owT%hl$qnMU1TaskOeY=s*r=o|3gjyi zy+(`@Kz1NGgAWJk;Q{c(*F}AI4ZV$NF7{C(6>z=!W*I#t?thSMgi8s63%*ygFKbXkW02TH%}$ z+12Yk%lQmv==qbj$va!3m5pE&$t1MJgG@EXjS25;P%e2-y&@RpAX86f#GP| zdvU!mc9_R1?i)_8Yr)r}-WB*>ri1z9i-uf~(OC2Z%YaPBwQrT+jqzKWQa;eGvlnro zxn$eQEMm#qJ|&Ttlaiz?&>0#q-+x>F`sPm;v~4f+E&2II#q*|*yNqEBS0JwibyW%h z`)upEzNgVT`5ON5kJIBGEZ6!p)vvbwzWM$j0TcS%aVvlE-E+S!qglJDZ=%eohiLDf zo@*;ZhQQk&9^VGuM);=W-&<~9MOjyNUjX1@hWT#xAgP+jJ&@$q*KamHT_12j4p-%o5eXY9M+DDc|}!dd^^i*&g-w4>BW8P7IK1 z@(et?=ZN)(jylW6D2#%Q`DeXBSbrnDLIp~^1$`;@5ol2KzRlqw~}r?;<)sC&8j zIlsL=Jsb;*%^>CMsdkb1g*;`yr(7pLH`Uiw5tQISL&st9*z~eahT~r@vI`Cf%Orj# zE3PKt#=D9a9E>xQ>NJU)AMIp_nKi95@Y}?edd%g5u()hl?^76!S)7sz&m_ z)jxn{5CxtK{;}{>gztzczQ^7BE;EzB$em5M0gp|i+h>_~i`bV3oDY6v+u{g`MzHb= zj1jYQlw>}1y|gA{3HeMc?U~%Ebd-5&L3G+o8ZIhP5Smi*2$ye`%7Smc<>**+=T$U2 zQ<#RHqR1R;x^B`}jf@$R{Hrltf^z+R-F)43H_lG_vA>F?#UP>mBN>j9~ z-yUjRpI&_ud-v>_3(a>&x{=k3t;!@IoA(edpL7XxmI|WVV>H;eXGC_|J~N=MJdu-r zXx1)TFkwm(mL);2g>q1z*KUu8?&y1qVVpQ6D=h%~w?(E$_lH~Qcz zr{Hz9a?Q^!J5jtRpSz`yq0jmL{%*b=D~l3)oQa`IET3j6?p0pIzgIREl0>ok-zgG0Tf%^w+{7jn@bD zhEz_r%IxM_E3CgdWz9B+E_IBRTr}NCBT_%36N18WO{dK^H~FHgU5O-AaIyr0CqSGwIowOJ+1P`Oxc ztgw1%sey0oZjbi~mfw$|$~V`O)GGe{-A+;87g9^CF`Qgw6s9nv$Y5|sKAe4Y;AA5} zSL}%}o*uUo?ZIIR3vzsZ^zmpNvY%C>$r!fu!FhX^WKI5oKbEO=&Hcj#XTA#RVe7o2 zt*Nk@b_NnUnRfwKZ`u5)nfq>!!}4)SJy_bVPtszEWe8+?4W)deLDTpR%Kb|uxa%Y! zjbTiJ8z*r>;{3U#e_lBHm3C;p>P3t8^jy#-H{C8_adVU6Oi$9_pv9&E%hvO%4P&U> z#OCP?iRpGEoBGhRClxMm&UR6D3F@M&+r__$qdY2D&)vb z_L14d8lQTt7*kyjB{Gt>`<9JJyYRPo#>L&X-%Vv3d%`vK`*_l@kW~Y(ed7@)MhUqQ za|r5p8Xwya!FBudG^>O-9$uN}p#OZXNhEIICQ#&koN&{9!W?E6`Uy3C&nLm}_w}Ut zkqq_FPlxiYk%Bi4aBUxmv0_59fqc(5t)`4Co30yQj&LX*6^`z|8KAYT zehd#k&`hb?TwNP~f@1{e=7Q;%XW*z zLoQvfc0V_MNNU!vG{Q2_)~kJ?^396&WGGoQIKan8-S4E_VV}<%2j-79k0rks_@0;j zZ#!HZ%>zkpXs%^@D{2f0aDL#6m(xBqjf75M-zuh!pOaR~;vri!Q0*h*4HVM73g&$) zDnR9 zHuo=Eo@BrXib;Y_hSEqNxmRGerY)Vcg{mhvo5|A*%oks-Iz*=EN6-y=nZuGB4*J<5 zi^TAFdwDDiu=>P|c9Xa^{6lxdN$)h;rFg~o`=KE_B}c%AjJ-rrgloBjg7L}YTFSwS z`3Ei6{7c`CXjN6vNwQ`&#?{dZsIe#XUS;~;`8~+Z-;;FW;XUS%KghSq51^Wh@=$yU zsx*fo;Zow$P2Sr&%^$P)wJ6LV$Eyc4uVdUx*i+WA_`sVVYMIOL@jLErM-lb6Bd&|< ztW~2`n%8Eu`gNqzM3z_to1#OpIg{PDPwqRi3zr{pA@ELBLw(i!3Qn8*`&)dXT({XG zma~K=7&Yy)R^Cr`)Tlp38zAYZ!kfVPB)a2&ot((d>Uei3Rm@=YPKNyzn@?*?s$RDp zH#t*z^C;jJT93^@Dr27ux}MacosuXYea{=X(p3Q3`#V~&_F}j?dNGc|KXwRN)K12I{wlYXC zEErl^@CQ0X8}hs=gUK-MVg7MJ$4&F4)4=}J>7DNIT&BiNvYORM{%frtTU$s-797Y= zUU#Htr0(@OMHDBcq_7laW$pwKG6wPtiHwd1{dkRf48#S?Gl`QJYCWiS-A-QLg-eQpnC+oU9I-OdvBz9~0R8#jJ;cx>f% zW?9K{@7J_s^(no+m{)6e`>~7~$~@m$`T~!+ju^{DW;NE|<3{9xWS@Afe>oyI=!;>j z^-9rU)=}hWZU|JT6W0u=Us9eoiXg`z@xczo@Wadr2on>jk;K<$JF4n+FdXnE?BC0h#y^scE;_9F~P1f(7Y#yJrLfk7KC@YM*MjLuD zJ7-Z?EFq6Z6;$IG&pFEcjPK6HSM>^2qD!Wq)y55k6ENq56I9~lxi=00-=MP&j8r8K zJJ7uZ)$*?zobqGXrU@k{8tiJPpFf*$yM9(VSRHFyUH51kK%iske<1F_$rGb{uf9!; zWiKM$DtU(_%na?MIsWzQ*N*0=6WliK1x$FS1>OYsEM`O*v^sR;p$yV>pG&IG9kX_f z1L;*pJM*nQll_TLE%i1ry43hxuK7|u>=XIR2dxm$@{!4neZC%_k{1^(iK3Pp?{>0B zJAb+*030)aD$t82;%li9iD79%z0}7Ej2%d9$PpLY2ho;w#&O4;EYo{B#HXn=vX`j| zuI)poN5iD5M2xAMr_P!szKc-_18CH7qVr z{W&S#L`HLK3yaf!=u)3()rZbmxFo0X=TFFFwVUiWsxSgHxr$EM2u(W7+V-(rvHv;0z-{+CUc6!}=Q}o{TbPG5 zz3!bQ?KXUyJ)p%P)@q<4EZ~T+fgq6OWDBhy?dLNA?-QUi-wGl`&HpK{ zqQZCRE$a1xEh3`H&`8RdAIoNz;18C~P*9E}GBxjQ%b)LvMM& zYg>_(b^%-kDG{TOxrAC5f?|!0TH{;R-_(~Z2b`CDB4x=i;haQ~V#*Rn#tUIyrbHT@ z?h8LV-{}#bGE$Wh=T%QS=(5#DkSkby#s*ar{Vq@4Pj}eR^S&9;9<~T}z2VRg<3ATT z687|_(==O=847?L5c1E8@5WF1UL-9z<$uCinrB~rnoi3UrcZH;$IVa!}0VjWxul(6<|gN@&6E$EbUZwyy4? zEgS((e;^nCGL5=y+ay}?7w8D{e#FjG!VTjq@-!aT63-4OKwWpldHgzu9k3`I2${n2 zHX-V)@EOFiUO#gepQ!SND{9nri*E^5tn0$hJLczQLM}f=IR(Fo?peF6+`7NxFH~l3 z!}H}@u&fd9^fmb586T5TC;NtwD<8FI$F;Y;7S7yKKM1XtDP1#jDalz?sS=g?=Hm)G ztepn#X$os+etAuMn)KrCvC|gS{S4xbPz9CvgH^?WUlx_|eYI`_W0tjFJ`3+L+V=h> zk-`Wez84!zjpu7=k&9E|F@sV(4{LAHeYh8GmoO_W`s{Mb(D;2YYlapqBWk^S_D++- zG+>d84s`C^A!BA)2@ZRu8Je9| zkO^qy$>P_3y}|0<0ic@}AXqOv-5KD*8^>Vg&Te-Ped!oa>w6}V{nHFI6Np!7!_m-h zR380UnII#>H%tq%R8A6+iZ3`q-U>O=rZSi3b(<#_S*spPbhE zdwj5kk;kXK7dDF72^`E?nGQOm#K~N+-_|AT`d?8oENx?Cu4%C`DYbkoQaIFpo)1tm zgO;LdwQr<*9LE7p<1xeEd>fzx9dfiUC?1mZTgIDsPeXTf*&;N3H*uzo66#KM04;c2 zf?CJ4_IId;+>D}20Q?v&67QHT0c@}9nDr2dlmC14(6v(a7xl8|r8&Z4!xoP+Qmr~1 zjJ{MJ_HEOSSK{rvwBwS9BZ0qwpayik*-A5e93F zi?r|*Ru&_;n=sMPX9!G2V~qs4eU;Bqa(m-^-)9^3!He+=-y>da!s7 z#ldzx1kRtQBO-DMt_5$sNDk<5UdfQu294qGT*6ls;yJJEO;ZZfUn&Mx?X30o`q zwN3$rUl+4PSm6xL&$GMfwW$tQqeqHxmT&vyk|ri64Gqx%LVc483^A?NDhJxSo9?{5W*#|&VJdDv4M^IT;KaO8=|q3TzH z=?hNoU->4JJ>|WyEuyaO=Ll@`CmLRKgp-%m5n>@fVw##Lwm z8ek<*%oPA8phnA6hY}CRxAB#p74a-OUu`-&={6T`6Y>I(1b&M&g4~Jkmp;=R`btiS zx~vrMxc|FB6klA1K(x14#%dkMdIjJJZtQDzg2)y2JMH9*6v7No z@=TVM#&np%*ejh4dt6_JqmWCZ$zEEUm73XTNd=q?M|&cc&wOJ$ojwe*Jk{yhEfUa@ zXkU~ccuRGvqb4XcqN9+`s`UBt&B_=*w{g`d^E~A1@105kzDUVJ%AjzapB2MbyCfmD zmzE4%y4{R?c;v?0@)`FoPYV=Gx}sz?@M=uXq7^X6&AHSApuT=~hHNnkA~C5YTcZ(y zC}y`?5)M}1sz?uJ`aMzh?V4Rt(XTOkz2Dr^`RKlB2m$rx4iRYX5GyQzA$fOJc)Lyo zaL4#yN}d2YQ)+q<>Y~z2X@M@lKy8p~xb8=3>aA~rKN>apI8oF0VI&FRuOCZrP?I(k zQWx7NOcs_W4CZOa3q*XqX)8jKuyGh1Gu&vF9&UVeHJ)n5~C!BcU{(M98;jcwLIsC|vSK?P0iMlI!Yc+B`cI|2=7TaI~uN!AgrqkkCVSYYu%wSNYB}(t`@rO|(5mS4#c+e-0)gYB7F0 z4BPY}!-n1!X%&^<0~WX6UXVH5KtW&5XBZ*RdTW-)4JG1lkgR#Uy|B8_F%d=Ei{N=` z`8E`EKigY$4G-iN0dzvuV=N%H;D!sORX^GN!It#gHHGbixNQ`wpbsS9{mvJO^Cu+Y zwD(oh$=^q`SX{Qwb#|bpXNRLO;Wp)=os?*nyglD>?ZkhS?15&tvmmnA?5U(~RQY@C zi5d>xYL2E#I}ha;jOf@-O&Cl~_t~7R%xd)ACP10;1=he$K6)kO?|f4EU$GL3onLz$;FGR5xxu_&X_0 zBt%jQC5qGDy(3VR?mn=&Tkg-owm=|NKkj38Gw)e+u-==Y4U36AI$;jd=lk5xKvW1J z`Wm3XcWL%C*s);c(;MMr92xgbFSaQ9Ej~$~9Lm(Rl8$p_J1zY!<#>0v#`)oxeVbH9 z&Y1DNocGp#Ll`>o?hg5(IIx5)ZW*^B%}yVbgcPIE&W8MhvW@ERo~~baTGV9)TXmxG z&B-2b7)RM^cV2t17ON8V=@A>+;!n?s3rdI%y#G32W9`HtG}buap$t&@o$`>&a6vX* zYu?*L8b3eZeyq~7e68P9ikb2WmE{doE^$NmY3ZmgwY3{(Wg~nWLHgN2E1;N297poW z0|AY39La=ROIjW6h7Unvb`wX`9!tEzw|33@TsHDqg=5daY-$iGBVDct(~BRh8&q>j zBCmgm{uklI4X|~I>v@A9a)jNc0>>Ryo=8=Z_i4w@M$?`2{>qk<8E7~mSFh{FhzuZ{ znmP{e&9}`LFLex~%Rce4^^jkpAEAxihR6DUdR~Q5>PhQ&Rjp21_1pKf{;}CQaYB^- zsrGceT^4YFE37sSMSvUVy;rvya9P z|9<8a3Cc8T%K=`|FO+#uQk_|FL)><#?0?WY(kU*~3Pe3`x35%uFay*%0VWj+0w0|C z)KCdz zrLkisjzbqi`q@|XhY>{t>1Vh8gJQKr;4;9Q6*^s8*x#-4eMMKckmAj;FlJn3%v=j_ zTeyxIZjC)95yt0;`>*@95{#1RH8L`yZ0p$DQ%i=AD(ye-wbvq;juo);6-dwycVeQp zPzF)L!sTYfhKilZmx+xDEHKFjMe;__k8g*V3J`)}BHx&GJSXHn;$78Ac%n65u!Z^i zt!KC?KZDn8kCnv1D9MK?{!@}I{+b?8@zbmSi49}KYHi!z;0UjHB*OhSO(3H|0ISV3 zULzov!F|bdt4_xmik(Qd1sL4YWT-0-g6c)@^P|axcoN_Z@^YnqQoql6=~dTr%0RRW zi3ubbR<~yktW9o#y)Cp_H7n6s=e3;s_$HR&m~noCt9$J?n%wImJGiBX3SnM}t&#nS z4&cxDP^DqvQ!>OTwwy-MjQnn$pj4St8NbzkZZkr@`^2KOx>4JxXkt|SS5ZEq+BwQy9A$cC+RkM`S4Ae7aAn3w$~)5*dXOU=KD)s0qLc^ z`{6%1yb$PAxlLSpI7I=?v$xzit4UY?tS<;s?GE^=Hoy)h9$0m+72xcW(xKTa((>CR z_o33-#BtRTQgDAxXcL{f4k!tYkGH=s&MSHo%1_>5IYPeRK|cAMD|V~nD8)q{#tHef zQ=63Z&^{}G2`SR*;4uIaEeHbRA9@bBYvt_)fbK+4ti$c%I?&X(MY!8P9L%D9@ ztR%D}S&=k%tj3D(0UX)7A`RKr_6Bm!7j65?Jxaih$LR;A8_*ata9NHpNZ2aLtC&cR zJVOY{YiPVp$(QaYk|xor)YiAkOlMdv%Vaa}*W&s9OEl`Wa&9(kJ=yQA30jg;;c zAzm)fUpi1$9nQf#py!~;X*KD>ZPXflo9)fL#&7AaO8Dwj^Kfo53#5%&E^!BxUwx}9 zSzol!A6qz>?IxM5eo*m~0kDVGj99C*4|tW|4OO&W!Iv8;D?oen$dL0@2{zrXk~J-< zH`Ok6&*>XzW?2f}n7tvZa8Yr6cSgzM@XnGDFIx6@K2SML#L`cz*Gje=`;>iql|bk* zag#9Yw{xXMT2!dst2rLNwA%FM$tcGeJH=*0F8lIZ&6ezZzfh7;A;0Z;H+d2^*SBM1 znv;I=XEZ|2U?ntGD%)<}w$)eOmj6W~|8?xxvReWrZedRG+_BG5HSgod*5>yDx#;r4 zL9f@3H2#u<;N*}y51ul_1j((q+m2S=mi=vVOb##XbfVKBgelG&EHK;8d9354PZ>7{ zkkSF}p!iGdU$u8BqlRY`+kCIh4p#b*3{v1Rsra5CZkg_{&ISrm6@K7#bEp3xhZ&q( z44FD&ZZz!`X2&Jtn#^Fx0Z==?lo~4*Z8Ni-2~4Luc?Z-Y?0Pi}kD6*YaHR1Oz|;z; zi>^!#FqV3-(s8^g4Bl9MOHB7xr_Sn$WJk02Ms(h~e@Mk=#qet8{ll8pIoS^IN3s6j zA0-<>ZhR1YsF%<6a(#zN6cUNy2Q=uC`@P&%X}*%=Fx3 zi7O@BP>CH04*uI9H3f`~BY@9Ikv?ryyGd`X#QBoeW_&eh*5g@0eS0;=Cl{ih+P%53txpZh zrwECF<4TzG=2%}Z?mQ5d99$g^tyeSd%A!p}I2u<-rmKttop)xmfZtF?%e}imaCr0< z%$)Dv-X9d+!-irbdG+Cm?Lhs&nXDWdOCI%t3$r#xT!Of2ml3cvd7@r=2omfudgOHb zAQ||uIvm|ii%ET_$adJL|LIc}GTNK=TG>ElUSLw}Ci3?&iMo)Tf-a^<6QL)-{$c=- zyKP0JK#t4^O2}{*2B;r4HWKVv$qKSr0p)1s`kf}{4LU$g;~SaqyV{q_{Jt0ka^rI* z|Jg))UMY^tqAcI1yA8`xSC9Jd$hQ-sg37;r@o#4bjoyAj&!P_)m_PwoqL**M=EU@tYx$rT{vf|LptY&91b;9)6) zQRI{DethH`$_U*SqDzgY7QUZk;91gqJn_uK_Kh%L#jIqYh`m8pAPy^yzdqekA?R$b z?-o^n37|D6L3&<)_#;}7RLa(V_w@ei5by_HS&364c(rU3`FA7dhP49$2Mj5|i)Yu< z_;uC&Z&j-$!8@*YEHr(Y9s!NaiF+tgLNQ@2SZE-+SDg$W0#<@7S^;jZL%Sjem7_^5(dQ^~GNJh%&cyl~}Ob~d) z#DO00Z51YW&3|Au5!yf!XA%&~$b;L;9Q-cg5o_1Dzt4Ms+~u^q0q~9{xp+z z(D~@P!wQUNV!^a=sz6iUkQv|yI^F8ilnyXYZi~V6E8Z?2K)G^MXxAv}fLy${99;o^ z9AC_Dm8*i@Ey^-}gg|rBO2)lYhW{cIc4oWqUNb$CfvQ=qs}4*L?p#HcEBxL159JrY zUP%N4Sqw=?NX&rd`k}nP)Sn_VObh6w!p3=M3zlE3yo}GB;-Z=W;}~9Q+|n%7Q~)N( zsg-lazu79AN8a!E(nt3Kg>P2nNdpgvoc0jcy}1WSH&7tHUw^O!__BN7*ixW&=X&I{b6>bxI$0*8kI-~o{G=)pY0Gztzx4#vat zq@rO7SFhS6Sl{pZnA>KmYQfd)0SWd!u<<3a<5d0#<{%8;fMhZ*kD?7R=e`{2f2@c2 z7bG+M%>{6}W(~`0SI+)uE$Bl1U4BbNOSuq6I9cIf1c1Lp4U>?D9T)-=OTVp+@#jQ~ zfF8?)Y~2bM2Xjpr9z0OF!=cX%m|+8_#zg0Vz3WI1xlmf)SX{`wg44&tiTOAk&Kfo_|W{~ieTMjULWRiMY_e)$d72u{)Kf8ksn zhJu*+VEv|ZF$NfvD}awfA6t9zZ$pB*Xh*<5%s|>SC#b*do{%b)oH0{-uR6$$@7_UV z`Q1zl;2-VHKV}Gpih90h+w=e~Y$G6Zig`aE3*Nvi2wA)_p;-@2W#yrMv(fEt3`8QR zvr0&vq(o^doD8g{MC>c1>5aHENmT^xQFATvfvlL*0$LfK=)=t+El3|Qj%BvUo!Z`AqUL`UCcul-}j1HMD< z!+|6~K}{_QAbips&A&di3s+(OP06KE5dl?SL)oL2Ps00=)nq6;c>`{AOISn z1avnT(+0bZ>ogu$`#to&%1{)Xpj)US+ntt1;O688)e6TYuK>&k}zhN?d|e+_Mv8Ius90f`A27UN&LR$)BAj4`atB z3wo6ac=irfd2OU1hTS+p-_SSzzPU8-k7s$CoXkwcZxbd1f%4#xVI!J$KMwTXLKfcL z5fo2gJw|y9L>v%zPMjp34qxQQiMtcI|I7(p0a}Da&*M#;^gU$6`}2iH`f}!d$@cep zwM?HAJ&>TNEl}$Q{{6>Xe~>HWE--X25rOdj#zNG!8^|6^1??C7-`h-UfF=o%F#5W@ zLbgA4GdK_2?!SyAd_V2AUaBg_Gpmcsnba~6i6NgP%c&#tzb8>AZ8#cHG6&mqm|Ck;t!h^aEf=ASazQt?+j@_i~~F(k8IxUK8YQ+cEZI zB?tV5l0jdkkegLJne-*#9c%;KB41KUk@w$Dg(2D@Gj*@np4HhP zZDO=A@hQpNu_loIK;R3(46_H8tpgym+K#zqe_NccW>jK0T#jQ*l(u1XXxf@^ zV9#R)E(LX7ZUR8cif#uSAto@Sm8?06>XGV9y`7;uFdC4D;bH$WAi-opJpb=kbHPx? z3%~@L0U(}QO?Mi^38bmM^%wIZx`AYmfj@+jG~(Bcb!D($=(N<|g~BFAWj0A-)8UYQ zr|tkVgH6L#L2UZk7i66qhe8(s(if4wFJI!*Y(FXh)Ri&=eYIE~yh5NsIJA$x>;{g} zI~x1U*`3vTNRx}^q@Yzx<09mC$%f->&$T>z`u^oT2M325;FcpP%1`)5fGLg^tDNn& zH&+8(RyIII+2JVa~7E~frrn%lA8oj8QwU()1Krl14dnv?c84*Mye;*pr|FRO`6-3HM zkuCJ-Z^nXxlMb^Vcu?O9Y+?(*6LR}rxjX)!*oz)1_9BM59jbtB(>iUho*)&n*b>ClX5$aw%0f40HxcsXrbhfOJKVBwaZy8~9>_9YP4T z4YxuAS7*|rdG9AkTOD9v7QR{SXwN`|i{^E@n*eOqk)&kvrO&TNfD0O&cYt3WmWu;6 z%5YJ12J-A6yono@!r34tX+q9FVPGfvw$3aK{(BH5NT&CJ!a2|r6#z>BZm*MVT}q)I z#lKm>Jx1^}F-vBq5ztjWqAf@&uO(gY6QBLAy908xK4e=00=^`_6xuhOaUF>`44QR; z6|jYwhmZCeCP~;sunwQ6aMtaS>#zVp0B^woU=FzggvarrG@fDl{|hq#48MT5ydNAN z#-dEmQluxQ#BP_>|JEcEB)kg>L3_ZWFE-CZwud?|tR2{AnFG)J`*J|i^qT{Oj5oQ% z4$KThLTI2pRN4&4da~vVfZNTTmGbSwV4^&%2Dv8F>(MR$tw}RU@spw`VA#+L`datE z#F>6;p4tC)4|pn*sCE!@ks?ox0>#^;*+fn#0!IILd_X4$+pPRiXYa6X8=8QxFskwa zNN|*pC+8gW)OOMul+hTnA=%vD%5*r9(H+1@a$uh*bAZ2y21LAlluz}@KaOY>0+6SC zi9pO3@!#uSy*U_mbpQ8Wo}$@*^g0@rD*&C@8k4T@Z~TK3e-{iEf=qitR9z(q5@Pc( z6p~)aQ6hln9fgPb(IqGt;%Wzl*hWIyZ{CU{h~*U&9(s7?XaqXb$ZvIkBseAx+=+rI zf08R{9t2=#^#Rrv8>3QUJm?Ydq=xoa|GS-gagfC15fFeBR809<7k$xcFU?XIL9O0u4+?FKA5n-Z%+wKy?4d6wX1HK1vTdVH{W7BSJa@iz7as$9K zs5?*!Vk1`-JN#dEZECxg{j!?m@7>!#s#|B1@}C*NpG(btzSr%4n?iyVJ7rlMRH+e< z3_&Sw2DX5NoBwV4G>{3>UtU$8V#u|JlSRIH69=pskqx$6yG9^mq6=yVjtr|ngc=8+ z*nW^2g80HB={acoR(zOfp(X!i!(lPu8&4@f1*YI%a@6j zjKJN$Mw1#70|TRiqoy3Hmj6B*d<-JFCg7iT2+AeK-kup?W|d2GH`-7hltEABvptf- z7`}m_Fi39md%O8LX+oJ5*pwv+HZcN_4-^19X9a9-ZjS>q!OQJhsWI-fdx)R+L4b44 zG6b;)Yyx()WI5zLIO{U``~Va_Ps`5#9ash;2;%6-xdc=+FnNn&=CFgl zIJ7@o*1!7$S(5!&Ps$s3i~zj70Web}ANb_ffZcHQsP~RMEC5HGn`i_9il{2&bxSNl z;+g^KAn@LT2h!w_U13Bn9lX*&RM7LN*g+s<`Qbm2WsLYPcDn0~<%>{|fv?)iP5db$ zz>BsAItTw-7H1i$jG{hXQc{9c79*{ZOeipVXgVPvQ=tdYMYB2u7&}z@ z8YRpa3KhaxNd+I?DBDeqJO;2O>~B-V4n+~ve||eQ4;t8>cS1YwDJ5|vfL3a--rf}T z-!mJAhpb;B4#`|wKqfhrLE_4+jDttPE!`bO&Bmx09|LZJU`x*F2PDh_XffE$D{qw; z#Ka7ML+%5BV6Zm?R-2QAX&}S;#_QE828{cAfjEU#K|!GgFbn>lTLc{ed|sY& z^YEBYlXc7E&8pb``;8`OzyKfsiN`n| z^f3*9T3B*cDD6+%9)_5L8Hw`voJ?Cdc3|B7+?_3`_}`i!lJ9>5DHceSBb~RWh9OlU zXqePkk`fO?W{g%|;$_$n@R>n?tmQ@xX;NIG{Vo-N++1cmHTOuc-O${1{gz%cK?wJF zECzW?(HG||PNI?0b@m@hKq15cuLo9a$#~>k{&7HZre0$Rp7S2P{c_}^eqha*Z$6L+ z`~`wyJ`9BX3uuYh=`Is`0T!=;xEp!Xnp*yZvi%nY#3M+mH0F%YKzO``g8?Eh#B`hY zQ}6JU;3U-YCpOaE4e9JaDeZeI8%Y^lvCsADwQLrb=)>Ttcc6){2!4){k59{F^&_Wr zvxp`hNmzkbAE*&lK^K|dq7j5%Zqx1va*<^y0ExxfiC>NuM%v7`{>Wz|=Y0uh$ImeR zXZ03C{4W&d(qW9OfEPKeSsiq_?i1AqDKdW33HGb zV>m|GTgA&}*$Yw0j3$9F{R}x_2na7S#C?-M!3O&cFcK!+X0D9x+?UJhxcdTOm zOX!GRKo%Gh5)-?z`?!QTw+a5Nj2W@G?>dMW6ih2vG@yATG&J{5P(*{gAPB!TD7^$p z1TQaiT5+31d{Bd2O(~xF9ta9=Jc)QOq_0^{gxit4CEKPQ{t|djz!k^>G%EB3+m0J! z!2=1PRXcJ@IDZ4V(sMZOsKWwKSn+T#g3RZm^X}~QI2Ol$S-S*D)qO88!Q=rjnB?e{ zs($tF)ICLmi_U_ID-lnr7myG-Z$fgrOIj#bRCb9wXC&#Buva1}+ zBNq4WMb99l@hqU7Fe9^=FKLg|mX?-2eJI<$izCf|c-fW&6>*1C@>#98+Wsy6ou28y z?QC1mU*yFHoPfr(dE;hj8SW+=T#?`z@esgr__=}C4j&ki9L{dgoUHl(J~^mZYRvjb zkm>ZZ%~}!uo*(~?(>4-#+g4L|c8vkqw1Q|J16qsk!_Y1 zokT#y?Ck(3sVmZ0ab-k_6F+OPnu&pk5t4gtU?MX&qjonv#%iM6(ooB20IrY%CU;uW z9u&$q{edYl6c&K|?~!U3K)VwDa{wgls{2=|mg4BDwj(>J-XH*}i)s`6cYA@S6bE2e zGXZV}xTE?*Eg03;a)lH^vDsP_B+UQIJHLNpJ`jh%+X=BfLI3%1k2xPa?R9zP2G63F7; zWMg6Y$^U;$eFr$!{r5k&+swGl>@A{@?CiZmM8k@rvSqK^$jTm>MJPhqAz39WvO{EK ztB}3Q|9pDt`~O{6SJ!jZ^YnDT@Avzh*Ll5O=Nv|VlE*HRthfYK(0!IZF$a@@$=U_! ze6o=wh74GS&)xwe675e3ku*@tSQ=49EckYw5PHK7gH&F9d()!P1dAL;4)q0_IcAXP zL4FY##0;5Q%dxQu0yYm!AEUULI!Xt@mz1gy+>w%x^sc}FP9qrONI>k|@h`qIq7Xm; z3qIQNrIcTv|5trSc>Lk8{EA^xehRSln$n)D%s&=6SK8Y<`U~aXGh0&rS*E1Wj~AAW zFrHQyNf!D<4`@h7oE^7eWh=7PG%3vbZ}sH7>iA^a0gluahBc(kas>W_E`m#T?F&J< z*$P#Y+DPPIYuJ7K4NaHa4fB1J=b;>SXW^ANp+Ot8R(JesEf9$Cv5D_puaUp_v>GhZ zOmQ8%nW|ZS<2{;O=l_&>yz8H?Ii*+0!>*VV;O6EVI9C4$EtS|ySP1{Jzvq%{z4mTK z+Cz;V^L?{@GF&1E&bP0F#R34+29z7&p0=BP7a09FaL&H1XnY z&32#g50qQ053+AOqhOPhxffIUG>sSYVmA?jlva+<%@p6&I(4Mh{^p=z4pTOTU`G>i z(F?eJjJ)!aBtR@k1$RvE?d|34RP!tP+dw*NyNyYl6UI~^;U9;{wD$`DvZCIS5x!*>*jB$I6L6#Agn8+sGa4@Z5XakWcYKgKnB|8 zH-Ja1&mg+|o|}tHIVa5+D1*oc58QsNIpBsJL9ejy_}r_P5NkIe^4Goi(31%&7b9T{ zj46Fv&2%sV)6jxQI=vSkEne-#2T(l9jk8#YkX$)7BB0ZahBh1gQ%fR1atgO4A7O^P%AXvqc=1_dIk0&6IM#NMxr+~ku#}3a}1dD ztmlMF_*9Bme;OlQt3v3Z}RU?kgH_XJTgYhVVfTKLzb=V)YYH529!vy`j-QH9Xa*Zx=+3i|DL?Qi|05 zvek>Ju6%!dB{u61I|QR6A@`XAH^~D|mEfVEf3FDh15NT){9-$#);q3Ja%!;)RP@D3 z3arG4K#~Ysq;th6pEDHpEhQy^!Cxw!<~<|*Cz#8AcMgYAtUXcJ5=ur_=>5x}71jZw zP0*JsRf8Rk4z5d8&|Co}=jbNPvx>t0$TQ1DC-*{oMDY_(-ia?jptU`^|Q&0`>e=lE4P{FB7Nw8*UBkrI* z6c9b=Tbf-v;Olaz4TFe$CpQ!dY5oqb4mg#+AuFBMnOy-ba)LNCQ9&4P&<1{NOh-VW z_C7!LDIMX2z0m-$_I3dGXlDGxNEphb5#PJGT{w?P&>B+f(;*aBjptyQM=A`_sGal` zG+jXfp?ObtL$PNLrjKuz@avz2^`3F#=gI>i<>fx|K=k8H%8vut?d(9T4)K0#1@LeL_w{Dk35>2Lr6#7=9U0F2j^NJT*W2 z%2bHjroc;_(m#jQksx8F{jP|eW(5>n?SH; zuVpo%7*oEsg5ohaH0>Nn_M&Jgc82hl#)0gJfr)jMw#V`iSxc|KIPimtCc+#AU%SY2 z!>_Zm_)~2EUKAnb?w^mZs0L5BC6eG`Fy3yE@4_sA6iJ9=7I=Wg@;|gNjifo2_#gkskR&;t7nkbI|4TaL3= z@t-&mw9bj$uL35awr6p^CLBgi3bpWk&Ia6dPedxYjMq|=0YWJv$S%}0i5^6ptA)q8 zD{^`(=AU2;QA0O&RcTED&Abi@`a>Y+JGX{X=X}tlA!el3sFn$B%+KGW)^X_u&0aC# z#1_yO8Btq9GC-q%k8>07fRrKgg#Ydr)O~PIR>?DymsB{n7GW zr@F~@d`We5IEdh)L;gzA`*QYqA^j77xj=O7 zWdc)$BF_9XH-JQs&_!u1ihx@QVV_1sh08VX=lwgJBTC8H=<{=M8-ZfBe-6&U-;A@9 z|Il3sQay+8ei?~pkXm^_I+CQYL=0PowBk3J4U*$eZ=2X{b>=GP#WfY@uh%qyMkI1) zh;M)D_&0CF>dj6~9BNu&(p;I14lY-qD;G;z#kk`skV65Xp2PEtd+Iknq*oW{`*+)u zSJ$8MVf~o^2NZd!{j{Es=TM%5@BUSWa48Uhg0^@i3MwyB_;L5^6WYfsb1p&+*06YI zbJFp8%`WbgB5-=u<8l#a-Y$|x$xV_BpcS4rTLh6{m^t;KN?JD5D%%-r@U8if0CZP; zu(CH$xG}F$FtWI|CrgUjso^2q@(*xvIe>57di_WM06Abw&@OG@OI}Ey21+UFH?CXg z-=ondLPK`vg_1(-^ti-01NWUuM_|*ID?&fcApvc z03ajR>j5o{dBN%3G8ocE>?NIIjuL0++_T^Q*xR|jtZ@l@4=igKE#$Dk4l2;$d5gAV zjmsa)a~!wbbhPnc0=zGmX-B@n{ILbC)6Os@p%X&1*?1Yw>>ki4Ukq`+f2lV?sBaTl z`gb`tSd{fXxC~}LhvDZ-!y4FvyDV8J5cj?-Hx}^Un|=#Pc6NUXlH>}@dzB=)QGlrJ znQ2ShAD2qX9ayu~+p-kmn8EnhTh$Wx2~-HVwCLvnC#L{UlRod(D{E3hLn29#h4(~B z*l|KKuPfL=L$eRICM^#ar%Lg5-2DgyH>MuF?vJybhzt+uX%X;Rll0+EMMiLa1a=+(!X-hL6#paD=YH=;-{iHGvChx;O;n9R>Kak_vej z#U&X1E3VMYQ|0<_6hD9!Cem&!CvOJ6-5&L5NBUz7UV_ot`3L_Y?GtoxL#k=4+$UkH zp|JCcFjmwLNS%!zHB(nJC(NIqgzZO#D_8##8?_GYOp=^$AOg7#Xz~(o6m7$x|M1PFtl1ms2_v?ilX1wm!zDov=Tz(rZGqPNl-q7SfuZP1HNnsox_Zs<(#M3Nagei zLl4a%#cH(x<97Yqld}7oDqHWbjg?!6f4G7gpBdE$ua3zud{cu@xz~aA`M}~i zf$Xew(Y}TZXg|UE=sSMX$lkRcz0ePEdJmID8b^S-P+|Xjl)&h*_vLF+k2M80nYdSB zM0-#DJI)n^eOS5Z;*3{yDw#owk-60Q`F&!a4#$+!m(*Z}>@0KA(9FQaop%?42K6rV z*pI$)$lN7)RsG3!=&EWSMU331YJHkg;Q_ON{rQDM!snt<72iQ#K`dLKq^BZ%tn+=C z!iyS|DC;ShFlTHTO96KoDWdUP$@x(cL!EH z@a(;LlEa$o^&P)|>vC&Dbt9Cuh)|DMjb0cp&=#TMk9O_x`uAZWs)Re}*p`NV(>(xY zguoL~XTVf8z?!Bd_q%x+=ebcl0Zvm2X2!|W2WyDRBDbvCgoz@{j#FHF=FpDG#B=s+ z4}7%;E5$F(a)z8?a(C3CoW?aAUcx^pZe6um%akP5m$i{XNu zV5u-3bH1cm?0@GT0LR-b^muf%qocn=wl@vkJ|`I^OXxfo%lFEMHM+c=Q|Z4$#E_RF zwQ2%7{AkZQPWCdpS+D`yM;yc+o$v;@x*Z?p`g>BG(Hxnqh3t$6Ma3^ghTqKFQB(z% zjmz&%;~|FB=Z>m6g!Mw-5){gi8?C^CzJr5s5+l#9fy@i}yMmjSH!~Uakh&SB|K>xO za*V|T-B~2tzvJML-_8ll3u=$qJ~~3m-Y_1P2! z5D0CY#aClZ{9ajj&EVJELkst9uYS4EKH3fxJs`*K{xI1eIO0lQnq4!>_}``D=;2=u zatnP=7fY+}itq;2>7~dKO{?LYww|^Loss}UyvKN(#Q)CCo7Wj+%gp2)ubm+3^Z!?@ zfa~h{Cbz;vQZ-hMXBx#k1pJ2?^<;#G`p{YJISkRKQR|opH8w{7lDkg|E!Oq}cRtEU zMit$iB_u;VB%&8crqM^Mw3HE%u8>xwBw|`PzhiDvQ>xe)a7I)RQyM%}IGt8QSXjr$(5lxW!Mthu_k>xs$+vaBYojqYEL0d875_JV|$flU0$! zMVo)E$9-9P<;s-=d5(|&Jr^vRSqXHl2qwW6}Q3US!l3uuCrs5Ke^qy_3REw!3~(oi;{|V0K)_lr)XBt$9L_?nH4-r zJTi{KFv_1-pb|;7a^A4QMl<_M-kBDt2aK9gT0~$mFn2zyn4=ir@$;*yyY9Lpp>ZN` z&_&7#V{vjPDnN!T>Xpdaj$xA6DJH@;!cwG%>Njx|aX?@dQz2Aca#LzKF+#=1l}0vF z0`gNQOs^=3RctUuqm;yOdEUalIcw3WT9-US({tj^P88@UlUc!ar2Jv$P^`nO^aF7Vb|Q^sNN^P(ry*3n6Z zL8ey+&-oHd;YyO}A2Ou(cf`MXx3oo>vaeZivH$n@w0^NcURWX>Y^wF@mp;$(<42#w zkFLsE>vnRk2fI)j>0fDm-49DV(IT_INsI8M*>Uump`eQNxW_qr@+tmC{X`FJiI>j_sa=C!4?>7M z1%!>k2;GaW>AR+UYgj75$rilE&j+8|nsF_RYGcq7nmF?*h`o)LG!lpF{~ict?!n>- zAkGX(EI(~^ng6?gK18N;k~oVzS!ob&Kh}iZ-pYx|N)O|A6Llq0S=BHQqIVCm0vnmh z#$>{U%q7OUhl*j}MJi&x3rqViZIypLCZ(wRS;kb|1{;g5Ra3hXD8d`ZFhiffS1>=M z{%`|gZuA%M!)+4L2@~RZt(_--T{+E?B@BDbcm`v|M@lVBDAr);)3r?wDmFHuj-{8- zrbmg!66jNdFW{%$w+0cL4h-Ui{7l-pOG3-)W(ow_Sfj3D1X6$FFa67pXusU*s(T76;2Uld)E z1;z^F7q>5{=@wkHOkUpV%%;w7fy%Y0VlhG0!brKnb;wk`!QG+u;c^b2eTb3N%0uN9 zW^w0fg`i93<6yfnCdMsMeu$|aYanaqY$mC0wUXpT5nAdqBG}aa%WLBov%TPa69iUA z-*v@@E15LjfHY(SC;7zsN^iV(2_9uEjJXDGb9*EikOd#gV^W*_OXqdM|eN6w$(O}8Y zlL3bgupvFtP^;BV;(ymkVFtN1`n7`<)t87uIo*a%yFCbVL5ydG)nCApnl5_VP*i7} z)@_J94VgxOjOD+!kl!1b)W3X5*j3u!@BDxkuRcrP*jD2;f>*S$V#$QU*I@K@a)~8G zt1M}EEyTbq`Msa~DN27unJH`kwTz;@eF=i(Z``=C<+v?GL`fJ3Wjj z?GJ#-+**zkqt3RtvkELXDr1u5R#w7w7nLm#pTjq-?cC1G%xTEl;@0z?KtoG%SU!6g`b{CfYxS6v5#*4Tj-u=BjHkCoD*wz+kS z(ccpH4&K<MpD_oFpyp|bsV(zW_dTQt7W;xDUk+EM-Mw&_=Z&)mN*xOiT@ z_`Le26s_ldEVj;Eg1efdUG)JeN_voC{uC%@0oMq03*yeEnFS`Ez4l7u>eUR8pY%X# zHqYOGdA-gMDY&+RNc|!vY z3D20k%tDrsZ2TVM_#0GeA33ZLLx- zQ)6uy^9R2_wCAe3(gbbcY z102HIg+s)Mb;VxhUU%PsG@}h<=EnCsQ;iMG(4gd?-ig|}$c0->H0m!j2JNK^Ux>n{ zGw=w>H9*oW$sj*OyoW)B99W!okbhMHc=E1+J}}gsN&VM%fO~-mDIHf>@9v(LxUkUX z_V;;UC2(cb+czC$n%Xvf1fqlRF6&26%awF9$5@h5Uby$8`5jvSnSy*@+$gbXXEV4{ zPfJNjZGZ>13kVH|@sj1Oa4h3&$$zqmxlk%Tdq{O+O+g8dJ6r^opak7y$RD)m8|8 z6=F~N3{sfLZJ1@$9sm!*06Bw}#h;)FoC~wJea_w3T$qZbp>=<78j29=ii0H(*GmqRkLGJ z-TfU;F?C!cYizfKVK5}v)?z0*Rr))i{@@3aB@wpSGEyyj=w$1$UEr%fWw5pgr~ zOje?o^Q&#=h*ws@ImxQuj4a(iTs6qHpk3og!s@s3+?KdO&J)_*{2WBdip`||b*SVx zqwl4sg9({-RpHhD3{rgY$o^VBKm@62Y4j~3|GHTwbQF#oTbW-R#B?^}N*uSNg(X<- z-&S3)!zp;saVkC`OZ0XkV%|Wt4?z_VtxZSrrs~h^Q1=H*fdG;>9kHv+~kY&PH54W;r#1`3+D;UL5Q2`bS zR?n>L9S!alZ%3X>ItzXs3Y!D_&DyE$CrDav|0oU4^2jNy+h1`R5OzDtp)r zj|so2=>b??0b^Fba@+amBqrx-ZpIC7;BcgmO-av7Xv5R59;l(}%Q*rMijlL&S z^>Zum^5>VBmMfiI+_i-YMol&WX{&@QuyL1lORnP@=L6bXo;yB$A+Z~HKOR2ZX@jUK zLHUwUnH^9FEFpcyJQJ@>^66u?e5sa4TAccAY-_8^<}(?fWG(2JeU}G#6sI#irSJ;N zyEDYJStOwKBF+%)Eqype&`lU*%Ey&UhKuK%xU)Q{&gwOd+b}je%*>2sK~(J5tkw*( zez3q)Re+tn5VyiWl!}i($azvbC)Z`V85b%clP)gRM8nX?amQuT?H zAb9n~jU1U|gtR?l=&Gu!QlBM{m@GZEdX4&BnXAEpfCC0#8Z@WwJ(!&ke0BMjayC)z zfTCHgqw;&63m2}zLftt7&KYpV2#e(O_@gWR4tE@ax=4dp^rL2;gtL-paNz6`q48VW z+PdJyFS^L~c6B9!d+<3n2$BqX$U@k9@m#-ZGx2WX66eRUKet8+ezj^S1d-@KzDQgdY%+JBSy{?CIM~g2#e0mKo zO0D0kEY|JKAcf{g1T8XSBD)zfuPz#ngYyNGDo^)?QsN)qM_>7c&8!0XRetl?C{2Cyj>33$& z4S6UCnTOC|&gs_OJOf?eDzIz%$*VlIQ+I9-WNA)=%ODljV9!s>xGhAVN7|PZuxR%A zsT@T-#8j5}d$7d-mvyy<0d*Inuns$hJK!eBeCE*4fn@*#z)yo-VvA^;r1b0Tttocw z;N}&5^C621qF*NQQn)naWnT zl{n3VXQqa$CxVul9?s?%yiCZ+Eaj>K`>(#0o~{W`|KmccnF_dc{NUi2vs?M=vSyA3 z8lhTw^|FY@C|o*7zWPMv6ce)f=>tf-l>c+XlFAtmuKL8fpes3C^?og!ph-Tf@a7ND zWTY-Y@G*aZSV(F8X{eoH@R7Y}hfp)w;LCIBRNH9WeGsWT zfGCf4&}ZvoyDAw4sQm7CCP?gwBjVx{R}?%P9?+fb&a{D5mjlr4ALp<@i)PD7MtukQ z8^KA{-SQtfb3)vZ1`P-tiLkO+>MoXw{=>nMSn^s{ zm{|ZFtM#{G%k$q~_rmnaY1rS(`q~VBcSOR|lBQ4mR@K3YS?TU~?yq&7j}pBz)S*vB zWW!v@5*@xd8H5WVHYHCzu1F9Q-B)tud|o}sxEip%JZ2l5jG7Qzhox0y!Cnx3PGK48 zjY9EOjOl0KR15YHR0n1`~dkc+D^NaL6WB>%T%rN1^KWVaTL0NzC z3r>0~=*Q|Zpm^?xHn&r@#L?og3nAs{qHFGx*E->q$|f0bR6CV%P1a{C9xBFft1{-7 z3=NGeam6_cgA6BGy_m}@9LB|A!$BtF2e4A*>phZ=r9b&dF4l{QGXO)zD60R|9D~U- zzQP$`n1w?u8IK)Y<*6;T(h-K9RWNZeEB73+kXx64@zc7kEeJrriUckT&Zno^Cp$Yj zA{^sD(w#%lPC+Y&bYL$H&d%O&0$)8-0I- z??t1|)P5{U6Rs5Wbyd&VQ)t!kzS4sl)Abvt4JR6E>+9>$;nSD-R(F0}@bjJENu3j{ z)7JaFDdSvUc^>MMOx>M|O|3Mct3hO1AkUK;I!@GDNiGDrVIVl^8}^7`~_gJ#hLBKJF@UtY1pz z;46Z)XNh5VVX%M#ba$-7)ZH0af>$J`7>pi$<9+@{Tl8%^PiPwjqoD4i0*wR|Ot?`< z6Sg89>3wt|94OPT03gGprwHMXQUMOf&tw*i%Wt8wHTnhklVjrw^8*F&n7IHHagTGD zsO`5(zB^G{e!zga|K`mbM9L5E@7eR_lrI4I<4OOeFyeF&L~|=>X0e-Ul!Y(#mQR8$ zyj5|oX}jJ;uqRm^|NeC+07zj~5a1ldN2ZAY7*QKO)47c)2XJ8XF>B2EBOY?EiF8r0 zHQ6X|6O2I?D{D=CL-(!!mB0ejeN(BI3-xiSdG+qiKX-LR!hr(HQLouc=>28iO)ERe z=>^Y}{;kcq&JgfzL2nRkNBa4Ke%eLZp(egA(%(m$7e(#*nr%b?_bTUWBCHMt0e*pc2-ZQ_K`nhfnA*A6uF)4Zp$1IWQ^%uqT7n1pnyYgRD zjn#XW7G0~bA(oMm(RsPaeUG?@m8P{_>GHd~Lx%IUY{MoBR#9eMc^-5s2~m7W9NR_=2kIf+jJ!p3t2Q8 zCGZ24Hlxp7yF9D^VWz6fZ*N`EdlT$obRb=*LbqI&hoDgwnmGl_C8r(TWbrEzc= zF3LyTiAjE12?u~Gtq1ZoHJ~9jVmf6?6~2}8(LDVb1J86HAX3g3R{4MuPF>YlW(MeB zdIv_97>J)tpEsMUiNKamNHwON8)nuNzyfTkvD(1J$1*LT0T;9AtUq>7)`;#QFp1;mzN`ok3LlUmM#-DPHZsS&585f zby2)og0IG$%S1;4iGqr6{SRZegrTX4_H;{acDZfo>V3Wu9-1|{g8Q6Z^q2WqJ%4D_A1C?ZmeXVvr_ZU__ z@b{a8MyusBJc)+jUOVgH=u!_(Rsnw-YE#ZznxVX!Q~3q@Wh!9U4QIdo%rq+#J8>fM zq0%RFQup_k)4&9Y;yAC~bieZBw&Zk$_J_#79RP^E75TBJtPoActy(LY77-Gy5)R}P zGuSRK1p1VHHbjXsmnGAT<3}Qg5jBz-`cy}g$(S-)`b#s!s6ud2Z=n| zO1tJKWj7alog%R8YGfN9QhDwXM-XOu$#mHQE}33lW+)*X%G9}5WeXt~Mer8>#qCSQa_@7_d9F^x zi(Wi;qqVHa$ufp#;}SHmy8!LmoN-VmIMpy0XB5dk48EWUcbNxnlwJp0&tD7i48*9^ z#Sd98XhFAR)!I5G<#v5X1d9NFV;mE0Bvq@z0GsvQ>~qxz8WD7|RJrOg(a-_R73RTw z-iM}VDanAZ`=^cBI#JcP0}64+O|=D5`5W8vKZ1Fq8~zr#028={_e!_ZF{VK&N|%zjL)g^5n&*z_%8@ zN%A4|zAcz}_=<017V34`c?MqZv+$+c8HD1T!(#Gqhz|vxlWrN|=MF*;EQ8j7o5Jjw zL!|zFVS_j6xq}L;ar1V%Q22IBw{eTbOjVWud zRC!zd;O8tN56&7bPkM|0K2 zVq}6H=g-`v@bK`cm1MEImv6FL8K~!}QLxJBTdlsw;RFk`nu!;^AsU)`VgO7QA6{D3 zotm%bONbX%WX{*6Wvg@JR(yVgA)edd%v6t;9_oo&j(yL$TEA_MB9kuM(xBh!xhKT% zrG)>!pIS^%`6Um_q1G1~7QGt~qKhG<8jlYFI=3GoAn-igwTxQFrNg9z%Z)X+tEhdK5_3x_5EFfI(dx!}W1v{zD!lOdYprHc6)8 z4{S})(A^Fb@{d$#;i*Vh*y={D&{pQ$NIxUam~ca*IRIU8k|A_aXwp(4j^EDoLg_Dp zTpI*8{Q1#sb`Rqp{<1$;&p+Jyf=>d`pHs$$P>t3OGkq9&-28K}W_>Ct_0-j_L&Sdn zT;rjX>zr_18}%Vo&J5OnU;61ilWpb{zfFKDWPKXVUM+J^>+ErA?vBJ-*~hf3ODdqg zK05u_8KkLle!_bSN_ETcbhELm6za7=XYtPW2+ZW2JbN|2aY!oId?*N=m4ZE+%>$%% zD>TX;sV0Cv)f&sfbYuAJZFO$WBy(=*AeB6=N77<2_aO*oEhy!lfac3q_Di|Z8dV87 zH0MMaL1F4_d*gSUSVashAEFpzN=M*1J|~aH1u-w=-+B3R2d?^ z0|8i|Ea5bn11=wokmqq!bhX#d8$|MS<5cS#8*gE%u5q%UnXAf=`-(-$#QGH)j8*og z`r3h%SDIQcLS=z^?za8uty@6L&hVoI8O`ejed${Nq`68e)0$;c>7#OeZCzSQ+nsIq zPq+1*{!}Cas8Wlj_GBsFa`zGM>kT`YwW=y2pM+hNueFHpbIJIqiVK&Ke3bHKWt7Lw z-hLpM6-bGo7lQ4b_D%msbE?&*f}JPqI!|y)DW>gSE_ioQo2cr}u-5(ily_@SoD^jD z^BSM)n14ng;eV#?SgAh7>ppZom^%z~-eN&4@_j_38&^1=$C8{B{H{)^$-53ii*odz zDDl6>W)lP%!OLfQ{P=N^u|6kxM7aTsD-*W;$Q-DixOVf#I{Q>-$!TaJXaiNoC^big z*=u)|#fI!mCz1@J)BB5_8Y(D{2LR_ zfb-eiL`lCc5LUqc=aOcCiA=4R=0=r8n@Z-FN;yzx`Oq$lc`SM%j z6|-IYqRqjfzMBk3b3Opf?D6!8*6~3(V_EVrH0^x?B_a$9 z%anbJ?;T}Vx<{Euq#2IkQX@4J+g4wO07|+@WyJ@)&~uLtV^nQ2cl!g#JhS z`%fu@f3&opgPu1&XZd^W990KXo;KtjDl$F+uaK}|}=M?*7 zG|bmBczzaM*vk!-aaSG3gL@V9yzunYZl;DoU{M8xWYl;sQkvaj5`ms)xGZ~ZJHe~4 z4(7{4%|Y&r&bFrOo9pXkItDLNe!6usz6*lN4wm0IJyE)O{R@sF+fi4FKv$ZgeCow> z&hZ*J4{16*3)B@bc&En^ew+#epJ^F8%=Ic!i3kdH$qyC1J&>lt zTV})j3Rf>nG6=X)aI+JVs3&(M=ohF3czK-FmEoRFMDu@Q2;WRyeFEu&{zCT#5x*dD zNYcNdgc#0v5p^N4fokdr9TJW?<}idMQHpNv>u0JKB?zx3Laz;_xqNMHfr@?D_YvmO z+xvvMGuRVCj?M*Yz5UZvHi4e(?3sCx)h=I3(Ip$3C{v$#c5&LMZkbv#FW%1Ji4wk- zudni(fcKSYh&kRF#qsk@g77++0N+A&LpFKTmeF~t&c5nFs=wvU&P1N2&(+0u;5Lbu zm^XaBmw88j9Sn?+N#O15~`dHi7HRF91Y+Z2PKZkjM|B3(-c!nP-$z4j?X&~fVW09D5? zuHBIQSI-T2D8ja1eBHTD)LczG$1g{J@o}SC@5lN63ImRj$7gK+4ecdYVmJOcxpHA~ z?ANb#vt%OjTv(#^Cj*!91j&V%SoL@1?d|=nq(ZoKL|u8s{Pyh4e&^=1sb4BI2i=B5 z^z7#+(XMkHPsG&>^EHAVUGV%&OCaIjI5YpOSAs6qqw3}dx|_n73m*2uE7-gjNG^*=`>EV$U4tqw6Dx|p>6Mg%v3W>5;$rTlorw?|+~6EYw$ z|H1DITKoX}B7j*fIvE9ZP&@R@8(r()gg|$TOLXb24c_%!l`>Pc=Js|%L^rzrRJ|gf zSUAwtnp+DqD1vLRiqg|z*Ikv{+gd*VNzgtndYMZ4r%s{tU*soAf%|K}-oe;5UxRVP zo2KB|e^uvYO0VBXXYYtU3N4m9E(+LqtUjr_+|Tubs;w;wsV%wSmOjqI{W_y6!K+`O z$5OqFId?-vLj`A+xnoBKz(^_6;M=rJxt%wI!GhNUJ-mE;a-oS*@nq{`?r8$F(zh#| zcEEr>0A80*88ub-9#lE&iUy3mZ{<$~;?a{J$4$N8B51sK~t=#DfV0g^uDadT}P zJvq|;&;1!oPloghU=o^DrjIXjs|M4Hqm2N6hjJ)~6`sm%>V8*y59BzgzYL8BUIEO) zSXV)wVai~U^P>mZ^6s~caZV&&K=+ijp0bj4Lzrrt?isp%J%xNs(Fd9Iw zA#xjisgR^lNypqbDeGqt7^s=ABkhNdsqG;pZ1tH1H;2{(CJ9HhF4K;yBG ztTu>n zp->3qL94w+ruYlC)r#Bl(18IAPv>05lF8GkDdCM8rzthy)5sj&d#yrnl*rOvVKsZG zA{4)60?*YjfpAL=l*JPu&=8u{Kws8U>QF)cx%ep#AVYD2+0aX3dQ(CB?TZ5BkkRiAY`UU~kW~fZ0^o zeZ^>4+A6p!^YvCr-pj!%izSY5iOA27Pvo&CDan`;nI?aug%Z@5j{E(g+-v?^z6RaN@yi|_Eex(&$X%UI zLz_4H#I3~evI7ye^yR4xiC4a>j_UK2FGO=^#OWbjw`KbWD1BM+{w_|A{~vmU9DlXv z)a+1Bv={T=dSihwv<4#fmc*XD?5X(#?0HI}PTR=xks4vD5S++l zF00-5#qt0?ypDf2M3Jy(?!2DCR(+|orLal*;@JsAYKxo2OHJ}R$o}b~)YhnT{Cyvw z>emN<(66E1f+exQ-+uD9abPXU4Ni6GTOIMZ@W8gXX5qbe75450FG2goh&=ta&#WB@ zuyU#(No{N2Il@0d$0r!3#cHaG3@wdMrZ9}6?TTWs{$@Iy9i!Q7Ol0IFL(JPdq4YeL zLHOo9Ws`Ke#Xo8X`9;h94CE7ZR{wwx7rdTI-mf0@tro$!${1DdMmGuS+xD4)^CII`*vy~$LMk`%y8 zIiz;qi!bV=*)tsu>ec8>xrr^8wru)eV;lyNtrw7-J}nRwAMh}OLjf{CaHvHt2J(9Q zMGLpg{k9nHm~@pB5p-uWT&kla*}WIIWrNNCec6dfNBI-DNkdWh@CBL*#d~$58ObiOYMS zHiGFwQB1yJbTK9U(TI;uc1^GFi@Usy43&)h-qQoZ;$m{>PCB zkAD+Q5NrA%{g_$*B;c4RXx&S5>bOb`T%T}D3-vKkpWBeUNQoye3Vt^@)GT!a1Y-D` zvvP?0J07IvESi1(w2Vkv`ugJsnI;GO)bg3LiG+F^E$IT_ z$xwvS<(CQrW-Z_x4V5yMh>|nyTNFT{!T@O44?PNqx5I~dIp@Yd)owsu=6`{uKiM(* zyYpmTsO$WZ<7HnjuN-jX@tM7@DmQgMn(L(2posto&+rmqn_=K@cvN@1q{|IsA?a-J zelncWU<_@KV=MMK7a-}mn)Rf#o|V3Uw1P)sraW3$<9lN*x$<#N za!wQVQjKjrK5rZQ`KZxvzk==cS22q!I|Gf3Aez&Ldv%W2e@?gHK^3I-y*EC7@%6gdn>@Jn7}M7)^sjfXt*U!W zUhk}C-g!Rr#Q#=LGQ9z3riA&{U;^!B$~19Lq2J7PireW@FNve*bnY4-$J<{on{b}+ zUGEm2d&-4JDd%VUuC9T?{Ec7uc4n_-%LbR^3)K!`!Ro-S`%nCfnM|I~CQJD|zgQP; z^6~DsMISq3$-&o5=Ziksaq2y4h#`4;-g;u)r!Cd%KV@b57Mj5O#k=VJ!{L%|L(fB? z%ekHu*X7O_XmH%Fw?LY85D@~p1D)qf1f{2M$$8t`j}<^p;h2mdrs8e})~#EDw|ACYnXK< zjYxP?Pr#5l3Zv5Ab3yAq6PR+9!Af&>b@mR~`x_Q0gel9uWxBb@U(T=}T+nENGjn2*at_g%6g#c#`!&T|=bGaP4c#n{rE zneqd+6du#6T8{rNzG>5ZNrnfYqjj*LzeWecGd~iV1wX*VqzuL>5bP<(h;RYY^Gd*3G!2VsE00?b6lYPQ-cUujT-(+nmHS- zbRWwludRe15kkRFd7{-f0IUtCmQSes0L7-Lir_v8trEaLN+?1Ivt#^+H-G+AjHKdO zhv;k5nT9G}sHs}RChMW}lyau}#L_C4*#C0yl98W`lF-bV&6zc9`la)t@~jiMx6fvV zHC3HiNL*GDyxths)aDQrnz%OfipJGOmkJYQX~87#!26rANwxE|*%`c3ad+tL(|0h= zOKEYhFGk;7yFN~<8CccpDIO^(?AsD>#?+&gyDRAD-swdDxSG$u1Pn0JH3Ii{UPLij zgweC)cYbNtuQ|xJh?i34eCKV)D*55{;jN)(gEIDG-mFrUJnsjtIQUsCNd3o-GEo%{ z`jNzD)jr963p_n36h^K-B%O)=+L4~KZDg6<;bC~M9pD^I$O~l_VJ?x?j0SJ2d93)zR{1TL+(D(pCg6dzZMFL zW0O~QO+B3Ovcbl$ho^a!Iy{yS*yz$%Wa-nJGKd44WN-Fcqx2kByZf8Q}27>Dfbpx z?_UeDbZ2Cv`;%VH41e3b^*-hFV@ih%Is-Kp*EOyBv~!$?w*^K97WQvR3I}SPnCnd% zxLDs;=nU7n=UL{W$eC+y%*%t^%?IyU{{QJ_Qt=tMbHAMOiw5*H3uATkE(V3+={1GE zJckRIxJ}xXE{K|5%9R_yT^=J~5-R@#ze6s;^-22~4Rq}?^+7SmOg7-6`_L+21wDgL zN-<*WK^V?vTTP;%1>q-3NKss=WN?7@m zJMDKeDvej3jJh|xFZS>3{vvDN`fa?WfeE^#OUuAH_~h;%#YX%y9%dUVTNz2UFVs=|fp zWATg`v+>wV7>%`=$XzzM+ukXecMJ%XY&W}4D;Q`}5ut))cz%njoTTr|BVqjX@tb;7 zXqH6HVz(nmfo1l4JoI|}JeJ!E5!n>>9=W=XQnlr}c56aGE3VVNY4EbJrH9v^!YAfzF z+S$Wi(`WSML&d8>cT!y0FL(2`rRv1ak2w!7av-8$_;F)1U=Wv$Ze3Qr6 zU$N+)*61N_jgc(xFa8(3y7Pa~pZ8|s=Im^Y(Mey*r07M!Nb@6N`al!J$O3iie3c}e zdX@H~PZXT`_NV0Y2Wrz`cMP>0dC=zF+J8CpU^D|89!}WGcjn3fdbB)h#=@Z|bt5aYd9l|b zUfRS-Eb=Zd!G&F$LRn%-`XJlhDNgnNAc7a4&fbY5y-m&dv4`b)!*3#%01LYaLw1K} zaSqX6?+>KkWi4`Ft4cnxC^&Jc4adaqw=sdwS8wyW>*AHH3sheNiXXQ7Z<0I79IVj> z6veX)3oNdCl2=?gtA=Cp^7iBJSM#xU23&WgrjycC2ocHY$Ln9SrO{YHjiC@=e{!wI z4e5Nx3Au&%M*boPGPC!-f%{q6WQ#3Q>yHc-Qg1H*;(kt&arg8o?v$8nRZemnom8Ty z4{Q}G5Af~j6ctC_jg6RPhyRxHZBg7}YgAL6jyIzHH>1Q-C&%wcd9ByJ+>Q3w`>-9E z!NRr3nvq;`*tPCe#TK*sk8StevE@3SJkQ7yAmUJE^gxqV-Q4?Q&Q;ZmmUC^RD|IJGFhd zA{0zuh$ia!8*X>c};j z?29t%CkE89Kx8W(h)#kJWK&|GGv8-Nk&XkK*%V<99wA|2Dt$|^fT(EmM5Qn$1kHCh zXqlkdu7H#C$TzZ9@*Tm}#wiezrGcUxfY^yp@6F`<-s`zQs{3RJL?WXzMumjKX_)*r zg&qI0>LrW?Gt}s1najh*lKMQ^w!hrVPQU#%C%*tktn0p8 zohoywufXfe+Rm?GPmTGnS6QpJEIrq2iT!B0#*$ynyXxk*yu>$tCQT~adRBR#Y1Tb> z`PnD_sQ!LTtc+@+X!`bIn{BcGrxQa$PM6WpcgpPXlkK=a^-5vBsehUqjEy_2t9+k;FkUf3oFaP4j<0mmFTuKF#mjd| z`&C=Y+ocZ>;)c%^HgEL;tbJKhrhk4|S7t|UqJddn+7Jk!747S@8sej@LU*-b6HTjS z;E|Pjv2=oCw*M`(I?butRMafWDmM}-l1(AsPeycvtK#&ksiA@DSB?F9FxPhNZ`FRp zr3s)7TcJ!`}M2x;|6=VS;FXDn<16ihsOF$#-7OF;6BjJrzfeuT^POtO^5^02je44_*}!le5hp#K=eqh) z?vEr|Lgihf0u|LkANrRN6cf&Y9JS3h{5Hd8(`VGDULS-w&+#xURPUaQd`PUTe`yVf zM|S5s();$0 zxGJDwC^qebRwGBh!&^MA>lLkmBCKc9Y#hA_Ka(GzC#?gzg!z0nq0rYP=eOJ?1jvC* ze$;rfE0>E!<`qGBN992TW?=VMMFykTntMmW;`GzCSexZ3@yw8|GmRUxdgnd*bqw`3ED}uCyHA}N*kxtCoS!@0 z==rUkB4pK=burZ+Q%rcEja&O2X=Qu(4Bf#W+g?mnmFtL|1OSBNAy;Rg8NH|@$@B<* z#6!|myPAy5E5BB=PLoDp#om&Ja{Nd%0;IHHvW0(iGJ;rtJbpSyW;Q8FXSchaP*cmG zdC%v^Y-}F~!vQ&z1R(WE#3A>in82QsqHn^l9P?1w0eK2wBucbFMgZ=?V?Aqt(~Jwm zH7lZiK&1!|1O(E6bjktjLLW#F?0Od#NI*XE6Q2)ozCDUmGQJ~mn36+&60u^y${gxl zC>4E4_Uhm_UU;gVIkufOEYaCT?}7^j$9pNSgy+2<6{6a-CWk*G$?aTlHUEW_)>vB> zDc!-T(nW_D%db&7{rvNDK>(ED`^$`f;E(GAU0+x82p8wg)4AjGIjfiH?Wm3f6kO-} zjl}^CDp_I~16%g!EFatSz^Ww5r?6pED5HB&G#t$kU0ViUgiSy8x_HGC>780i;=}$|Z!MSH}XYq2wg&HQdqUo}d5t&av=^pKz z>59eS1@jwj{5&X~=3@jXR1Wz_8f~k%?&w+;O-m1ewDXpsa@F&DG5XK=CFRvUv1nH1 zhAqN?#x_F3k9!6_lKsy`9t`__d1*AQg|d~mXY!OMQwR&HBgM=d-6a|QD6Ra>0Y*Mz zFio4jOKSf4_i0w1{Q{Bqv<@0UoMj(@!_kPAUFYc3pL_1NH;$O|dxz0l_-aLiCl`Nr z>u~~DEJtLZcs#%vH31C%if9tvexF`BXmSSLgvH~!lXTbe?sog6Am{xUmchktlk9UwRHrdn2 z-OYtTtfis#Ha4GBmOch#;<4x-G7M=~r{jw>RndG=Sd^oDi@ZaExBV1-)Zuvus+ukz zA9546A4%hb?FU7ltvMSfcs{X&p;Xz-z}M~mxqx^1ci~HSbX(v7nz!(*w@BY!q9N_hoGLdj z-=UF<^}?~Rh~0B`qp=K=?N{3T3E@pg{CF&mnxkJj{|wJd+;&@#^r2gpZBn3hT=kXQ z@6i^BKs`%I)Kk*tV1)9bFGhEw@+L8YvYa|iA1RV6M3kpAU(m@Mb$vOfFV>_Z2)?TP zz}I-yRl99l?L1*IsMoYSLf@Y6^~Xa)H!=8II=NR!83$nzB%!p+MSt~O`KlBY;j^;< ze6Zqoj3Mutm(w0{2hOWMp&!5rAPqS^m19$^DEK)?qTlZ6$9+{w=P>urrXN^%Acfj5 z`mS23)fEBRli<|GxyyQPUHMXU$QXI1zO^_N(9VHnY%{jW^G)j<0+T>AFtG=tnME0T zz=D-(kPsTRRH!eIeo5+uLNk-V?}u?g?W^+T<7Mt{gV6sD(e$gLTh3H`0u;CRtKWja z#Mn`=I2#4ARshdM+FCw4BHha7W)NUatN{+8y|d5CQYMA~dZ{$5GlSi6@zdSL)WSP2 zEDVCDPpkgAl7eKZu+FG9g0b9gQf`yEC!$RSA@7a2_(&rqdm>0VNQ=wd9<~QER11i~ zU5ipUxMalH6MG=6CRk!UVL{%oFH~8fU8{1pb$}R|Q3DrmddL}Dp7qgE8%2qW#-DJE zW>3vq8U5uOrUX3ZTNAIv|jjuSH6XB;;=A= ztBv!((69*AFtFx&-%zbmaB{~mg<90;FFU*;!&29>DLd-8(rAw1U_ZB?6-#4<9q+vL zO2Tn*=(mwlhSalIIlA@Ytc==^@7U2Ss(FFIth{#9-Ln^w5r zZ7AN0@T|Q)3v854#ZUY=Y1bCDZh!eBGh21f20OO5wXj(ty+Ypgs?-h3g24KetOJAp* zB;%y2(?4LmO@T;+`?^X-AXd5Bhx0N3vZC(+lIy1hf8+@lCOuF6`ruYDIhg(0t&t`c4pJGv~G}n28@(mNDuH_18j*m9_^UL=zx-oCjP?O2JT6+

4_oX@Mg9|hW$^m9O-WVovF<|mypE0SGkTybivNpMDM{_33gmYgu=~4 z>#3BQYn47E;Y;VR$$G2x(NUGslIeK&x_kh#UNYBr?sD$ixY5B)(a1nS*+9t zK7p1|YZUc3W9wVTTYPLX!v_4%;tF~5*P|W%jXPMXjcO5=V_Fd(s7#jo4lRf9SwzFh z;na&;Eb(gBE5++4t51B+@G1PaV=57N`z{=WepezqdqbUfWEp=ps^x-1=kFTVEanr+ zF)%syNx8=ShEm}@?>|HZ7#GUplXbKFvz@$xCK#~#{{+Y`{JnJt)x58}VTv@|x0T~E z^Ml(cC3j_>c7AD67rGev$4E)XVg@OK>Y4xB19rx44LKijT4K$`yDYk;pA7# z(C85u`OZY-k4Ilg71ezfH%2n)kdsD`+wEUvV*3z(TaPAr_Z07^aj+CsN}GWw26=N~ zh5RloQKfr9`ZI0LysmgXGaKe^{ItoqZvg!6ky|^7SHEH3eIvA86lGy7r%#zSr-^dO zTc;^fR9f@fHEYiDMUA&TV*cxRF;TTX zoEEHr69gZEA#;NuQz1&}*F|=3Ubv?4+p;-ZSx|a@5K?7ifDU0yn>Be_1*G(`Xy>@> zO} zWgn%xW@B+1r}Egy#mJ9ZXQSB^rQON&_mS1H1A$lziiBvTH0rmv;Ciu^Oe%x-9;VVYfIn$ zs`GvEI;bVycM;i`b@cVwhg;tcgz2|w`py(Fa526lSnR`_tZ7|lFnSqoHfOz%oOPJsZg%)j=EUzaMNp+fCOkq#`lNgHZGPTvT3m z-$!^Ws+08tApSEUX8uerb?BJXhMfL|!sonX@r@KJC*@|HcTTKSyJmySA!VPjY1A9? zkqB7JaxzGvw1_wLLw3su5Ohm-0|U0R;Med2=>2VoBMIqYdguvML9CN}iBcyLzuCR4 zjhy7aZDa{KDv~`N- z{cAxcY=3WCxeliJ`N-+%?>Zby8@xQNF%8stDfpTaytHHJQumtfgl&kGoS`J#O*{^k zR$g~oG&Z_dMpyTW!B&Q(>^_#L*1vW?f$a^^Zm+As)*n;3&w~$0C^^-K2=D>+lvtd! zhW`0aW4WmR!%HzS{Y<_@=V>pS%)1oY0XHfwF(tCC)DL`^AYUhXm4V%YVyRImdpe&x0pDv3D9sOoydgS#zeW;^em2MYDk0_noYO|N zy@n5Iyy(2L&z+N^4+71lCmnPxM#>L5*7fg~nK14OB*Zt4$Crz;#)DTwf5cSLI+7VV=~xS(fXeDJSeMu{KL&JFGBwJ5vy7cqNUj?8Lhjm0MtyF zX-^-Vl(;2eNqIL#A>vKur2SH@KW@ak$5%_vntm~c5Ecx`=!);WK5bEwO|$G~yY>1+ z&}RplKTKKj5cSuEqN&ny-B)>YPo7*Z5*#}wU+-$%$D{9;;?jt%pWzNBOD`t0O&7r`{)iZ_I4-wDQ7ILlg*gLqqou1*dd%_bfI>TH)526`Hf;KjQx!x!Js`xcArW za0Y6G+MN4~y_@X`GuxQjl;1}v^3tPK&U&;8gs2=wQe#51-76goZ1yY1+&F9}T566K zX_r%p#TXrIJuO3KlP9Iu2QhT((Yvk-6{zM@_7x)OHwjA13aK+5=^83n!tS^iQ5%sggMMfu(L+9ObMDNA z<4>g%-^cwHHAC7@V%z!(h^`%a9T$XjrIB2*&q?lY?n?C8qnN0iz+}^R+|g0UH@ZOR zW2%rPXEf2%lcJQ%eLH?Y5y0WU)ULKJ!K4=CUEvcv-Lkiic=0A$pzB;*obJ0f6dhRc zhU{`>jy7*a;xX%;s1eYQ&)$#^7_ z3@B5@|GuLeYt}_3_>5O14toiFNBdauIJJ2;q7HW~vc#VyLVl%m?!zOF`TQZ4?$p8f zlbx;k^#slPpn7z4YWBr~vc;BI>-mwWYt-xctvc6**Qw2l^))hVbF)e3s0(%X)A;M? zOK`FkIe>KW^LX{RAu|-4-yi#Grz5#o%g3Lp-k00x55fEcz}bq&jN>|Tjp2Mw2B)>^ z`~!n|wx|&XGe>cQ&WbqdI9!hRueE=*-a!eA(>%ZOvNtOo(Q}hcu>WSb$tot=JV%?OA=G98d$e=TP>#)B) z2+0)j3b&qX4<>2|QZIT@n|(-Yfb7?xm2#-GFhb=ROi_K4<(%~KJqhSqpCftRoD=D- zH#~cNnU#j8MzZ?E}Z*Y@p=XR zD)(;2p_CO-6VemOR>JS8*RnV?@bNJBKGbOB4_80ehCCcOB}xx(3;q^sn3ZL%(qab@ zMkFvLb&YVhN#r7h#`X;)+s1O)llFuYNq69`Z2x%CXn%Q2=A=&~Ykj=PSU0UpZ3n6U zK62TWC&C0Dkbh$EpS6)9($4Q~TFZK=O6Nw2ZWtHs6U=lrVOdE`o{-aIUuzNIl|cZN zb(0Sw<9OG)Up3;eL6t>eP*o*VERo;tUIbX_c|f}S^Q)RFrW^!TQ32F{D3=*nLlVfx z(N^c9F2wlu0^7;eAdG0&gyTPu;b&VAH|6KTqKwj!Z=O; zlo(=25Tkbc&(V^V4s!B%4f<~HWa8S%!dJz`f^iJ_Fw@aDY-#xKV_mOCv^|x-BI;R{ z5X4!O(A8*|L>g8S39Pp$N=35>j?lP78EXjT)whi*vp$lCBCwumd}s=vodkYAP@gDP zp0AxnlHGotMAg#lPz06mc$Rw>0``vqfC^q(wGI3q-jEM=;VZy^i`EpGl+W%7U}!=U zxH&*rn1Rg_n`R9JgovoANC1s+d5~M=d*_$yz)nZo5L9GKkc0-()fxc%C9GUdHR=U+ z%yJR%85KLB{><+6(#tFz)`{K^lP;$1JEf4%2?CtQJLv!|aw-8HL|`6-L=H1IOU`3T z19W$0)s%DAM-Q@_PueSTQDB9V6Dr-YmKj-$+Rs4%R>bvN!ZXFqB@uT_BYl_ zF^8J2Z__||tb=-0`kP(^Y+mcs!C<3JCmnlh5ySghf%cB0;lzYoTW)3Z%aumW=`(z0 z(a7gB_uRkfz@MwJj6}^BzZWX0<3bP}HRueCIm-;}Yq_FmK&T}km->gCqh_`pBs=to zz)2)@{<|s|qT(j4&}+lbH4^-0z(r~osAeZd2DoOv5fX{a^rN}bsMd=&+%Dw2SS)>g z!L*pxu!cgTAVr8-yW!=JICRfzA~Xf4%n2&>0Zg%fO+UhDca0$}0T|u71-%1J*LLUvs|N zazD}`UN!d zL4w7c8cfp3a^#m;B4kh#UNqXlugS>-&>56F9`NoB2U9rYK64guA^?_BdI#r5b0R2H zcn2$BV7+fxhsUl81PkLNWkV8m2Fm544YSp+N7SqPBCpl6Shr2!E6=Hf!YIR2&t8yMd{wd$`?;NWq#fcDAfrw$J9Fn54wQ20t5=k?Rs zmNoU~?lp|aO$>ff!uanm`X>I9L1tp|Sv_Ju*mjh#UwG3c`ZFW2gtd&*=O_C8xbpsu z{Ed2*Z5e>f=Xi+G!}tNvW}_-9lrHjCeUyzczU~FLFWKI*y?H2 z+mf)aWIJ8XiRWU$ronJ6$@+jIXb(un z2G3<+qNIUTEUKP-vM-?%guK1gb1E5%XLt(CZY<}0q5eEM3S!Hn1au%KHVO#fvaG=#)vS}8lVG?TUkzT!=aeict!@vKO#s=b8qyh5 z24J^C!eecBFK>@PyW$T4Mt!+xp0rMWp6V%_CMeK@GWK%0%_9W(l&%ATD9Bo+gtE{Z zvIv`}%C!mH&bHK6>xXy@>5x;XKt&;XR_5RV3;Med&yM-}3X(Y5@XbphB;d0_6GQJp z0d33n@SVhh>yTlshWGyO?K#sQAIDza=bS48W)6aS6nf2>Iqqxd7e%|G0B9Wus@uB- zg&XVZdBTd{TE@>|(<6We4VE;BKsOx7O>`~!Gu%1BLRfWafI`$eFpzB-WRbiX3HcfC z`vnJ+rW=qySAj1Ru-icZ;i#c6KXYnw9RM|AE{J~lbNKbB?pXON7Mq?>?Hza=3U_UR zC7ks?*%sqXk|GpUm z7g$C|&&Bg*XTz$+{&c#wT@!Gl?x-drIQmIPK18*TN-8KnsLMY z@L=U1T<-9RrtrDN`g?WMsD%J!mIn&Kgz3@r!2#np)c_ASw~{OqMJ|xaiUVA8qks*8 z5hzZkUw2C4d!G4W#SBs#7GQzb9BFVI1BpZe;hyk-k1Un&9c$IAv*nyq#?Vy`_JgFs z*lFGlhaEg9)qxWp!i%O$>CgVtR0{aL<@fmVOp5^HLbfM#l}uN6_ffUFA_f6Fi9oLu&cak*&799bznzsOBQuY+trbJK89Z^h;NNwuK5n*+CN zvSP34>dd@&Z&PA^H%i#j@qsjE;EG1!mz&sXw>ONl37G*h*pPJJ{fY21s69jstvq(x zj$AiPn!!hXDnL70u76toxx@2X)eQ^S0qFz@04X{W5CxNJ?b_)8+h7CC**XiHmXE(- zIgp@g4xrz{-XTZ_y5_ZHeQ%W#6ey?6`zhfUN-TR2y;<7M3>?nKp~90+F;G(BP{?C^ zHDQ4s>8k=mynu;>wchlSHOV`0xp~qkmyb!M3wV$egqwCIj2*6roql=}{dn}{IysQ@ zC%(P(8U6OMGGzgOj+4RpC0z;LV`C|a4c0u^uQHnp(ls{Q>S>aI5!M=T6WLw70a1z8 zxzy;0*8Z`8lm63#G9L-J55fUg;jEvI?Etb*ETqiu+u{SCVFLr@yI4aZyZc)=69}dL zH&{guXuiDlLIXNb1iT@y?$Ziu1?K*<=L2!op5$MlL|z>uGJe#*Smu+(SfB&_bW3gl zM7?kRqD%dtU7>~uDPVNvJ?Q1WSy0^>ZE1=a44?R~W^OaYF#3qLy8VTj=;M+IqyC-Y zBasgC&)X3T#WJ6{ps@b6^$B!;@25(qsCX*4;Q(h~a1@%P8N9wiI%-GBcRo842(APr zyW<`V2=GP;0a7~){vXt(LfcTwk?v=>APE(7975}^1cZFmz(kt+Be1WB>U4dkT+er~ zKqpW(o+uXx^YrP{_=Xz-dTlRFPOM>^A_j23O|Vfh06sn~-6W8XD(5-8iX>E~fyiuN z(ysi1%9;W|`(!j0IsH9;eIbHmI}@+|k*f$gLx%r4>h8a4vO}fK>HquI2stg`3Ww(+ zdcu5qKpl~^*0DkuUk*^D&~8I5TJ`0A-r3*>#m%xqXDLG=Kv#?a%&%WX8WB{ zDm+ucsvY>?S3-8E&@Q2u(=m>Rk`h4^2)DDZ>d-Y*1NU%T@7Gv>XAHEZU)enouMmAd z9P=;cmmca%j#n&O{FpA{#Zx!MAt(Y&bqwDZ zc+hnQDezMM)8ue2G=R>W z&-IuR+;;(hK~6ls^V^Zu(oc#FjHsaUARobDSEc((Il158^2?l8`Vp%+Xf4IE$akV6 zm(95ncU))$0_tJU+@05A%#))3oZX@nv*o1OezpIFP9Tzv7zJeNDwdC4&?|NjD22Sw z#*7-|G&Z@O#C88>Kmia3RdSHRwJhih?qlD#aYKL^WiZ%fe__S`*$U)X(;v;WLG#2w zONn4^hzmV7AR!M9xTZjJgt9(EH(+!M38NBlK0RPsK}SG{6DWEV*l_{ll&vC)=f9#s z&w1Fmm2Q8GWc)owyddE7iWT&@l{~LlOk9`gtDbXCXwerh;8=@ zwm$83*BIbE<%?+z5-<1gGl-WzK1p(&(#UU1L}5VL5#OxQWH&X}CCH#FdJHZwcnWLi zN{?U;p6=dyE0Ie=>RC<;K1}OCTufph*dr#M%*qT34szk-A5IArGV5T z(Com5@lcnbLazzNH-+YI1%|{Mp7*z!LD?W$8XM|BUuU!MPIn-kzjoL0=G-FwGC1mg zPKq;SjDV9RP$R=U-)fKseCPvqSU@J;!+fUfOn4~j83n*6=_~z#|L6MwRyN%^Mk3Gj zo{<6C{66f9HQdMhTq3-)TXz2M&bTDEY(4w&Ni(@>w-z@BI%3(k-o7XqXe-)49rfik z7#!GbmxKxoz##^XYDoQRNvU8$p3Q)wEx;HzNYLf5coH5M{&3~f#dM9N^9KOUB2+d# z+wLU@IC-!feDlEfl#?sw*Cq3zEGR)&+K$7s&JfiCg@W-;Yu!0y#&|DV!L9@;pveBO zb&)19FO?>fTlaslk?>yuazn1Wv0-gEX!jN3j{`T~yJ0~r%YRg8w3us0sa7LpbI?V3BzM&#qW z+*hD9iPBR5*Vlgd=WmM{pu8;JVI2G1gApybsHn&?hSydPqzKPG^tD2Xj7gA>0Z%@A zg3ZR%5t_X2F~AOh<`V)nAnKC^M)9)XaqL>AJ9J3{uh!dYm}ntW3>ZU8Mf=Q2%r$d88}7kj)0Tu_ylY=BE@ zCn%$2;w-z+uG<#^EMWqH4bckN+k%!RPeGRQH!`%EgJ5+kU|_BB)tLo)%vDL`nBefQ zC_!=jN1y2fdqa_Vp4}j76$MISaO8YwiR0x;e_(YL=)kr!(f+R>640)2e?X;wx%|Vd z9<_p{2&`s6?w`0AFDakXZi4$ZDYlW;i{SfvnVVTACJDl!_>Bl|g+DV;Hw>UGCe@Nn7gO1i7^71U81%;p@i_LyO2sM+lmbh z_D^o{8Mg2xk5<`Wut?#sr9il=C$~3}?FAe$U9XmOipSRL6EavqDn@)qA{F%}1%4g% z)#ob_6U)n?v>`8`)R`_>8RTIxaGr;&6fUPYKqL7F1b{%@S^&vrygfh#Se*C{V8-ll z1DQ`ha5b9*!8*HvfJalD01qs}f`M=cke$)~<+ht_4TYsZ+Y>j|GCHtWAAM-ZTNP8@ zsg4%JdiJ=g3*SOV$#$6uKjd;6PW-QU`*)slGmK6-GUm!gM85I3;R_mm`{)7RY5&g> zwZI{NRSa`l%BPuD*T!TD6T&^)D-Zq1X`K${5 zuL+qd0Fzb>`UY@Br2=eh*FVQpP}fty^U|b=v&o1Ab?#Mz!P}Wt1c z{UWb#)(*Z#6NJ{aW2C>j4#sKR2rrl8X{B<(a5WR}#e>x6$PSZi8 zFuoNqYI=A4YroO3zJp7a3Kd~0-Hc31)F|GiEU zEGRHi%9G*?pvblHSU;XP1&+s&tc}ejsvh< z6N`{gvwlUXe$u6|RZZ6BPwVlp`o(nBY ziG&BOf|SI#(f(P=KhH)dH!@$us@NkCqM61l0qmHogJ{7k{=PQgrj6-jOZ1Nx3b=U- z0_dZ{p<6D&4^+^m%RqiXFb`qUje;yGD7^TIQQwia^at>-hy}loITa8vmkhr4%T2}K zhe~1(3sJ#vhh7v-H~SMChXFtWZFKPM@_Pzv&eQ}M|NG~CN{fDg*w^31;-j&cMfPve z@rCTa2K2!ZxwN8acQ!D9gNylfIt?fqF}_wT#T?1KwRc~p0H1V!)&uKbZSX(j#ea|{=%pqp z_L_wj3z7CzOj!Z&z)y_6&8F~I$Sb$hK7G6|;<2Hx+QXn=)mOMU*nbez?-ih<1h(oZ z75W`fUtNyY0Pf>@x`7PfSm=zg1`JzMWqTPmH(C1Ef-LfUahx-%4_TN3cKUCMbP)^f z9v{Ra`fSm4+O;Pj6C2kLv}V+imz+%2DOG^UlD?`BMn0_P@0fd{DJXy}+uwSyP$3b; z@Cnp!AVxJBh%TkUfil3N=W%d=>`-5SRX$A{DgIkYR?dMO5BM*g;vckpIM_eTbQ$;QAAiJ-nq7FyVC?v!Jh09yok#o>M8r@s3p^hb^94Gm)|?c zOMwBEkkZS^>*k!X@Kr`Qa7+9Kysv+!#Vl2 z(FX72XThesLyU^nHvdT#RT}#qiXm=}$peq}rz+jj(wD_^=uS&ZOU3WXQw=HErh8q# z{T<@Jy;|iE%JsO7Zrghj{iSJIR;%3I{-{lTu7<@$K~5rM14}{f8!D!%FQyzU1j#^) z(pM6RPV5;~3??N6i7uKrHNtu>j`?bdWgw|suW{$jY`WHAEQ3Fy*Fz(9^>0<(=HZgh zk?_~~4i!}$)$@js2j%Iyr;WPecX2mkqZ>IP`d;6UuIsPE4N{Xh4q^tLJ_KEvs86sG zXYphT{1u?^b~?YBwsg=4c8*nlS;NiB; zk7cAOcFDblJNi}{l(3;#C~vXFR{L(UJo{9PUuh)umG|v-J(C1pU70D-h5W8!cYKK3 z;!EB3CQ}aTZ|jV_HsL4EX~aBZ4Sffl4x~;>JG*WReOoe4I?jDIeTAvf1G)3mrRY=R zWrLcT3+vA>A(x}2EmK_2EGW?x$cltNreB?J$_SqZ$0cmka^kE_+&G>T%)%vL!gcX-|9z{C)wZUU zON^mruw!g=9*q2*h1_Y(G!I@IbEP{LUC*_-Ld*56hI&V)f?cLbk;$8F$eG(a<=ViO z6SpKo{t-*m5{^lO3{7p4hOwl+A+B!@nL&&%QNf?3H69I|+}{ytxa>w5%n28RS5nL> zj4GJ^(4BmMzTa!#9F9QyG`~wep!Gu`tA3y+V6buU3eUI-im2x79~CFh(2*sh#pv*H!MX_TXjm87dgd;>?AC+u^dfJRYfV*^&Y2p?CpU4k0X=YOdgCEZD}1}w~cs% zNy3)j2rzu?vgvnUimH58svCT*ZnLNA+Pv%(4Rg zOKIfpuI0rMU?xnN8NUjeZ!_afOiWhsQi6j|gJr@htc$ZuQ^t>+An3$MU)!;%zdnYH~0umhofFt7o{2k1@)2;Ar2(TaSxB{DwC6wF1Nly0=fXVjRy zC*(at0MnQv*an~{|B#5!G)&F=vRLb=P)ad>My#Hl9Y@OpL`M7CJY;GlOxbQK{~aql zapvsvKrYO194(LUJ0{P?v6-->3Z)@f&*1pKs!9!S1>BrzWed=d;b39NoM2v^Q0M;qMrtINaMdL<=RGS$Hf@SE41?9R6>F# z*Vi2=qWJN*Y5N;lxz_r}AQtMPJu>(zA#bhg$c)!yV!~;}c&;pHYBV#WX6 z_R&+(8Zr-9ScBQt&$ zDNP-F{S-?oy>Zsu$4+$X!C@D*dW=0{HdGpOzqKKDNEHhb@d-bmNu3%AoC*z2eqow! zX8#4`8aLtiq1j~bS%!(YaLRX6m4#kPCJAY=Rzf2QNV(+wCWfM4_!ubqm7DnUd!7_- zfB#$aTv=kBBcz3}haU!IJhxP6H(?aH$4h(Y0b6UPN-xX^ffk)zSc#i0J`rh{*QyTY`XbNQqRoL>MU+)}ky8q{Fk-_+_zWLxc zEAaAam_KCrZ{ghF1Ti^I)Yte@s`+~%M)4crtd1|U8pqd5XWAaG{yk7Dn0!2#qdqSL zRy(&{+s=~}J=K%>Fa>@03pE55 z{wqQ6`7R_$;4r}Mm)U2sD?uuMjY=s@aMIWX{RrOCwD~T6u`pc=j;H*|j|@xeL4ALw zHQzRmT7vx2JayWTG#OIk#T4Zd-z})NX8+0Kx-!^L??G7Q?GNINbrzhg+Fz;cHWL9jwML~GKND9 z*hl?(D9L;{`8+k64C2Rb3+RWA6LHx2T9B@2GB&%gTK15|&ccG>7gD1OaV$JMYeH7D z;&Z>{*w}$xtz8X#&l6pouM>o{yeaPjmFpP9>q04I#31iqA;*#@&(l?4tJ~?hvxu-> zrzg;gSLJE1VVl)7=4162NL%-u`CwJ;N3MsEZ5H+#(%_5Fj$1W+v+oEZyDFh6NnX}Y z?~bO+c!*uEk)U&@C6vP4l+#h6=JTPLvuBj^A3gK2Mj5V^Iyz$(cz~D398bXXUS6dQ zUB%qi*cKf}#r6uD?n(T<9d{>(!x2Hr$KsEpPJ5=$S!v;1(s?Rpeqa@Q7(8YG{Mj-^mVuJad#qns29b`wm`y9= z42&IisOUtKpF~^_wmFWx59M7Z+E2AwHs@dLV812rsi|l_p>to)FrpE??iRIoVd8Sos8QsZ(^R0gkA?$`gs(0 zJF*F%=iKw_oWr6+w1+mZ%bw2AED2}PzFMJs8cW1-{`FO zhXqmUq9nV<_t0U);zY31#%_3K7Tl;lEm5$rc}VWI*GHZ?MQe=`kzD&JLQ7iqyE(f4lzgki`I@XYudEkVY#h;HQ?ag<4utT|%OidA2?HuNX&U5nzM zb);F`ueSvTnSVysobJKH7oVgG>UP#S>`T`R=t-nv46qQ(pMKo+i%euO? zT^Z3#$M)~Rs|e#Lms?Zl4Bs@J%(s@*{i!j@dF<>4A0Y0&7RIo%=(0@r%D~dxJX8c% zw!j{x3j_h?CO1=%!pa6bwoXw9(k{zJFn$xmJ}J``%SvXMoSEb|2aUb2uTS;wKP}yj zFcFIuvfb_3s)rsG*}C4P`;0%g#Z{)Z#GdwyN`ly>;i;#kRj`D^0Rx z)r`owoRRH2yVO$iSJmN^`B5H9JdxtOZZ0uFdS9^W;a>VXAv7E9Md-b2j4Xw-(=i>o zoKJ7jxsrY=PaOWi`2})=@{XRaS_qOY=oaKLr&fuX%2=~A?S4h$@*U10+eFYVk5|DD zZ?OVUXi5-fahxxKjZu}zt7B2oTO`k{T@0>7cOyqq%r5FA67BzEQO*5}n zmgM54E)BH9{=jzzXH#cwz!DU#({*^(lTf7Gz#6Jw zYdg-B1@3bvHy^v2`KD!_g+brCD2-BXBLqL%5*XZm3) zkop(e#0`e`w&LzX=DWX-#)u=w2J*w6_hRm<H; z(|=F^S1RGbY!gjOkZXP=smm|>^T8J*m<-rWo0xxEk#;%`((P8xsCN!s$|&s?%OCQ6 z+BGq215P$2yDRvK-wh-i8=^(Hc3Lo%tyi|$+|vR9Eeqv!Dn*3Y*^#nO@~!|ogM$Fm{cHzqSo zQ}!bnKxBAuXNOv=m@e3{P|D_RBuw23Cn#=RT{}dY*b0d~Ar%zH2l*TFd2xBPgx|Nk zv1^DD(c&kDASIJ+rO4wAkqYm;#_ZIhWe z?$X5nWpofRy-}9WmP&T@i$EBz!ewGy9@DezvlM14di8py$DBCN$Y83t$SJ+rWoKIp z%3qOZZkXr_#OCg)v`%4;Z88=HjnwupVlh_gQ;V^F;P03ayj7L@^#xPhIMt~1!i|Si z!WVQQt}L$-HJ+%1&?5Vf)H)!MTE}jfP^x^kPLkb>uaA?+GMfwid9;5Ot$~ z7yh$R@UR}qUmDK~O@p&fHTjRi`Sg+Pd^BySg`EE}eVW_-mtxOBANsFKidur5{!s@? z#R`74&CjFwEAbSur3TORY5jfq7+~ZvOK{pUb6a!-9&?$s-V{}!ZSNW-tW5A|Fn_nCs#!@Io~bArVV+o?Abb^mH6$KEtyW^?+`~Qn02jLv^ zlab?a$PO7LGqU%JIE2V5hmf6)_4Q0xpDpL9D3-j^8p^*7+BYufY%WJgQ}xpf5H)D;+HemFPKX zz&}5L>AEJrII%P`w$$~WBvKy{d=~kuT2Y6SZ!`1BBBb<}bUsuPbF4T5tBxoBZ;7P| zHi5i2>CNm9pl#3$%>DBzO7!#6Cn}D-cxSIZDcwv3+LOf-lP{A;D&CXllBb8S$$nLN zuq>P}?HbcpH9q$K@b?sF697xkH0#$J9CaZkL+EMPDCXI{#}!5Dw@3zuhl>OT1_oBh zniXUzOPX#BVpU3^-!jpz6- zGhuvR2I!s!7Q!!I1Xj!4+x$H?6sEX|NxZ3kKWq`Hw_~!36hTm%*(+@g-26UfmQc3+ zD!1|8opI(yrMQY#J(=dPtNHCKWMP8nQe2y-uVNc2bG@eHBPmuh8EixGS`ngq4~}+3 zSrJWe!uYJz`p`U)ln#2HmTzWGgE$$ND(e%m;^EUQsjQADMqb5f97EXj0q)j5 zjPTOhg>+R~dDkil2WM1#0!`yaaA&GO&~vd9d+W6vad#+pr({rwpOKUtKc;mkyiht_ zCz&eS3`;(^DhzDhn2)OH#TgfhZ18G+6$>0A1}5Q|kxev|u4n2!C&>+Zj`y1^e5a3; zGEBtZJvY30PT#Q9#H6SumnazevXX9vOx(@l=w&`qSN`;t?9O+t%AoT=&zPxAQ^qSz z#}k`G?eQj>%TSV|XEMBOkd}p~&#!X*{LPTtHvmTmKfvxkihK5^uryiDcQ^4?X{NX9 zubH2pEDV(#EDtcX2YakN`4Pf5&t<(1PA4}1!s=i5O>u+-y+z9V@czI~NVvIN6voR);Na2GTbDHwUrRKEj8D0qlD){b%`$yo{% z{BCAQryK!>(+Bj8Aft=A@^q#RqzSiOUrr>f*WME1OZX_l0{uZh(1&{FHol88Git?l zO=8j|WrWT2z7i%@mXoDECm{ErF#d)x*2!=69}nMQ<<`O=|0E5C*T%yLijrKnIXkB5 zmgXE7R|NoSQemUndVq)XYA60rw+UW$KfA)ulO)47L8Qwg6^dp6p3A*=kPp(Ts~-8s)wS160_?w>2Kq}b&+ z?r{`ItF=pR9jvz?1?x+_0B%0vZ2vg-1@U|$ZK zrp(oi-cT0CzbFAd@hw)xS`rH4SVBlodR*Piqm?^=$^Q9e*at#?`XpGpl;O7=Dc-Zy zHfcEnk^bj0FPro~)cYQOs@8uwMb*t?n)JOtWkVB|O^&qbJD=sFdzp3D62~1IPFtj; z7slm(9!2>*u|6**_OU|gTUv(eGXHOR1gt+bJ zW8xr#fruhdFKtI^Q;=s@5rX4G&ov4^D6p6fU2lHWw0>oz{)*jh^n42jAr+Kmz@79q zy?JpoSi;lUQMZ&q|1gekLSbA03c?&C-8{xFMt_ATntF`2`m&}YT}U&^2gaW)Lzs)i zcG1l~v8A{z;AtUtyN;IoN*o;zuUnM+xCj)wuXW>13UtO5s^~(Bja<5cGxfVRUc?sP zOOpJ6E%8g^6LxiB&eYxLwCI<=SNI(5_gCK>QL82WRv%fpz$N~_lYj0wvm3}KD2+E` zk3|!(evXLtFW=ak-E-)Slg5iq1Rwm0S=anRmn)<*oxD~21!wwjk9)_EEFy{~x|>T- z|8Sk|ZA8Q?FnoWjn{B5K*s)bb{7mnO_}${S(wk{+Sg%@sT%E_PY^Vg%KfGIE;O|wQ zY&q+nQRXIayGN!0mcR|>Lp7B_tHh}-WNu!^r&S+E6)77M(izASx{We}6HjSgPEz_`w`4NPfCV{j( zHdA&ZSBAuL!}vx%sX)?@wkhWqn@Y%BhT)a_d6Bmil~+9|uz4#9qsgvO4@E1!z*-)2 z|4{}t4TtvxVcwSuG17DyZ)lOpC2H|za-tqfwK+Mj4pXW?B1(mR(LCsGSb*~AAqFTX7n&iQrq;&^ty4CGWWZE&j;kdu<*YDqj*F)E+ zGW>Vc|5O}j-fIqdNl%mX6E$2V1=6{@=3VKE53JovvpDJ&nyMsBR)`4Z$`FDg?mW&j zXud1G`fQ7 zlL{+Ex2;L{e|OpCh^X^br;y5D%I0!G*zGR_jlX#QNfU7VD;x*7p6l+r0Jgj5eJad- zfQ@%>j)vtH2I;@8vO00Bm*|1P&Hd9x-3Y7pWUJ6DOi z_jrr#-t)f}aMBWh_mmAkDg~o^sWMDSIx@mi^knKks(?s*4ia8VSqPkmy~y3jcxxV`9_~Z zN%|vwcHUwwt;{D{00jyD)k9>+9Txm-_Hi`}chp`!Al`mw zDOUQ~OhU#vogi(=zVt@rz!Tu^cc)?C4(a^Ws{m!VSwunZZF#JE!JMBvukV$T1#_IA zln~GGMULakjl^$MUScyTtXA1llYII5T)qQMR!5FqZv^GFk4I%Iy{)LVGo)XhZ7*>= ze7aHW^}or^k^uFb>~U2(xrQACq*kN4o}H-#`8^h;W*<+V4EmEYB~{y#MSU(Qh!0xYW7?K?s5{SMbFdiyu3r?8E!rS7AFOEOdMft%Os2JuFA z6}^+F5=*q9SU!yZcGaNk_G`u#N@JOytoj|2<%$H{l=67DoE&u<(EEvu=;#2CxV-JO z@@hss^B@r)X|~%{;7N_Kwl*!CaJ;KqMo zHKhrDENuH_(!mj&XzTLevRiq~>rR5fS$6DfiC%PCDag_*qC+Pv+TT!)a0lmBPb0ks z5VsR;)n-Q493{}gS@nn&5NJ?osmIxeWs9wd41Tw;66<~bp7)M!B}aTMyN>RN_MK5R z5gB^b{3@{UJZ5fk9gqZk@R!SL1VA*%qOzLHb{=w`I;Gy@njQA(phq4gyB@JUf_x)v z>J)r*F~0qjq20@mmjYPQBawS;*O7V#mxQVhiuBs{WTqy|&2vBr=2CCVKD1sVcY6*5{FG>F74QetX_rdA5Ox-JgJFj7Xr=f+c2hq zaO)2C{Q)g0F70s-qPxhw`VuBZyMt+5Wi+(`RC(nx9^tLx;ZW(QlGfbcxNIOT>~Vv> z4KqC3_=G3MCsURC?N)7atv>}@Vz3+i2wS4yOKYIl*te-5IBg*cpxjn&iMpb<%ec57 zcQM?UyN~EXigQbQ&*^5l!8#V#qV06h(fQ13aWH;ZkJ%M`gAu4@GiX3O zKUGJyZC+K`&360jH#;9KQXph6|Mj3)xfy@Y$gBN&W3`s)k>1VA`H1#dtJDcdNZFhm z{iZQeGi+J`t{DphsKTR$cRBlh3J`nqvYw#gEm7mVbt*u2bAEk?$1Ka%90;kUBW+X9 zPbP7N_RvYL^{+xa1lCq+{*muQwuj1Yd!v_&HPrv)%{DyGc8seeG3HxncI01_@5Qwa z1Ky{VagW|Spg#O#QIFiQi|@fqC~Z?RK&~k0NKezCu_4V^OUCCX*Y*WSY;0RBhb%Ks z;-~a{RM;no5d+KR5kGg#mj)Wm=ZV7>i|D%`+q;R=8Cojb?BTzt1O zsAw9>&UdnTNxI|-)WH~0HS+r8O*^Nzd!W!M== zbfka2M-skFj0lv}$4>gL303u7xSWHYZs<$%H{Ai_^YXm15d2oNbXJ%bR(`(0;Tl)M zVqPJEnp}58TsJ0tv2Z@d%`-LO7vct^;+AdqJ)^){k?te1Los5KY0>rFjjw z2lq%Jub=9Qe81;=Ss9Xmwf#VkYpJ}!{dN}V_I`!fyre{#9XPaG3BZvUB|8G`{Ek_ z8N@@x&r$%})w)@c0a)5hB^p)BMrd}WgKp8|M}xX?PIjx43Cry z!PqF!^Kyt_8l6osuv#kLWccc1&&0|HRkr4h(!S>Wi5)Z4!=2^z%{@b?=-!5w9t_=_ zh9oUdUW%VUTyH&5HIn0KT6t6G&%}z_P}t2u8GdVig4stcNne^z__3ZQ!o zl@6q(ZWHc73buZ9DsQ}WJ!2k*^K&p#h{Kw<+I{LIG*H0#Zg&#|Ar*p4XPh@4yZ}=c zBzZk-kjzUu2bNJ80_gi3hOt?(@K_wltQ>ut?4(G& z$u)j|Tz0;E$Zbhz2(iJaVA$={m>JAgZz5ZIcYkauo6$$$mv+8xrjAGhF?P5cIQSVZ z4*o0=fAUG!4SxUa6^{O2>aQZzu4cMZy;geUdz@GFgor_Y8{3IDJzd=-?)eGaK#b_F zM?UZR|LDn_d(+jljrW3%x*Xo6#HtA}{V037&QT(gXEkxg|JXiPudiV5`TNb2Exyi& z<+tH?r7P6{U!`k$fiVha6J=Q3<#Zwrqs(p`nx6l|0E zB)kL-2yb6oUij1Q6W6Up`DwAx4gjm=M3o8;$rk(ynOAwR4E8p^Jg}@T7uqq#KjS3i z=^v)DPJ`~V-HEf#2hKhb>qKi7wL7VENR1u>*+?m+UF)Uk@26t`36hS*nR@qu9$MW} zW_eu&bAZaYaycXpG2Fl-sGwS!{OD;{HTHe2Z|e?tgvG2nY7*;*61E?HZ}PDeKFh( zAygzx62n)haT&z{CR1DmHqKF${19C{hsn~2Kj!_YFv1QXP!@0;31aV#5PRb6EVGp4 za?eIYKlf8z#XlC@l~&ryj6sN~zIHce>LOlwKrWD3Q*Z4T9#AA&Rj=tSGKR*3Y%bpHc2@MbvidxvMmMb|c zogTviR;vpheqdtUll|(&99xUv#w=5OIvXnebQw<%X6IH+O*3c1)YSQ$W_21&({%KV zClumz>ZyWMZw)LYW~dEY{_*T}trPf<@P1+UnubJkWvK+kg0jy2Yu^ah9r={~v>yL4 z_Mx~j?CIIU%JY?d`;N?ab$S1?Ei$0S^{sNA+0p9^0H-%npU`oIsJB#D=)YMw zzk2h#y%z~@3x!Kqrxm3^E#I#vo)k`kt^s}ZnMYLa4=>Qk@9fx`ufgg9F<_D&GbW>X z^M0yq?j#jG1J-b&iZ}gpHgOa^`%SQ}kO3MQg8_z~d z)V!KnP#lbkV6~eIIAz`^Pg6!vO*JU`wC*wEc!gomrrORoH?D=8SGD3##hFMT12+1q z-2795=--}MiRpDE$L~qUDt}Y_a0GopdVK&Z)wLn`IM#)h8z) z>es`zkb-oq>V`@hkk-cHN9;Y~(uBC~hIIq(1^}(ddN+{tIP;`FhJ%aK z%S`hCxIsKg#K3ZGa%7Xvi%K{93r|4OJ+5H_Li&e`AK$I&Rl!pc_{Zy2;*0wRftEW? zJAiF#h?0Xiw0ql*E7fzhGnxV8LFbBJfNM0LR6n}5c9Iy~4OM*4-C$j(!8cE1{vQnLX=M z?K`Psg_=a@{_M|o{ng-nN9d^edeo=Qr8Cd}t&z=<&3UG>svNx!HBXP8XX{Qi_Pm(; zYwFT6k}Z9*Vqd#eUU1|3l;YC|P9-%7L0}9Ul%+G#Z|f=2}r$HPq#>vV>K<%fT7BIBVuw0 zmoAwj?i+&OPbL*QpxV)jrXFD`SF8gSaPp5l0{}}xQSgF7@SI!;Wm^MnTZ2Rc4318E z*P;%&mLa0sM)3o+yhlln4SweIMMl^bswk+iFG-(a0tKm+c3Lxb+ub}1lyx#NnOJ5TDsPmztP&R2{^SA!L;z*mx_#XbWBVWGi4)!^p0QkE=gW0X<+XrPqx zW#AuNn>HWk9SU^L&TWaWm6hPHCNi!1hZZwGzrZn7m8y>%bu7XL5yDVb$3&bEzdTb} z3;?OWzKt)uprH{xVb!Jwz~~{lGK!MbK5v;rq2mT4C!uadd?AOKj$Y${zvPw=Cj8UB zuS;VZy=gyv|0h1qc7PC+gtGMpG5M6qOeEm4yFwD80P+gFqFUKmkA*8-1F%@ zuU+&tHz32yg4EN{kIY{erhj1m)w8uk$F%I$VDPM7H8mmSd^_nJvX*S}{ETblM)wWL zk1u~eNsh)PQdCO)zT#@GHLeNo@bqnoBqo1M{Oz3jI9LoKC7|ej^^&5@?=|~XcSqmLbs$Xz zNjH2MAAj<3wi$?XX&25a_sczH$~Obd=VBGvD(Jl$)2;^^B#6FttlD)rM1XL@|$o0>EDqw;FH zQw1L&hPhs1+#!sD>^Gb=7+X~hZydOLmAI>SRYrcuPYn(k)fyvx4* zJTs3WqFH9u+z0yg4+WP?!(hKrBB-zmzbX>K;5gCo9Y|@=c@TyOX=jmzviESOIr2G( z%MgUBNC5`3fAL5t$t>q)MLoT#QlVuSvE0GTZ=~wC+A3Wlv0O7}CkKvsj)6sgovC_G zr~E99c70V+=!2%eIKM-Z+n2$fEnAhe?!Au;?D6vAwoLsuX76U&3-WOql$&Yv3~im1 z*w0QP-0_wu}c=sE^wQU+Fv2DJ%qOVwtei2xt*wH&$f8!v_&AlXtwIL!gb5Tr~m14uTRwJO3Uwm zN)z-${$z%h*i1@W7y(#NwS0+vsOP&I=Wd$%SNg9 zTUN}n_q$Bivr00hQ!)-t_anatEUc41S=$S^<^P1%cR#);s@n5khT{TnKx9aG1LiJ| zmQx$&#a-!(&^`a@L8`5-$Rgp2Ep&RaV;XpGfzgxLdY+*zb#`*A*R}RvtdeDW;izJt z^ORd{@ZTG8aO_FD?IgpS@dAFjAQB|{)DW7Be&MZ+?gouObSH&T+LD!zFBZl(eiHau zP;uy*A4TJHhw|j~x_ks8HOKBAYizgH_&f3>T=E+t$ul%D0;1?y+akC>{(TV-~V&Z>K5ivW-w@s`M0jXPaz@HxT-Q_uh8Go>Ny6z zoZL%)n-H-YUMTs1e0I`8rdj{c85%GD4+ZD?Q_0U{##Av#Yu=~uB{M#@{kR*llc_FP z6%eOvuMu5H_*Tz1)XD~E@idLRO#i;p8;Mv8mKk=jS${^+siQS)v^2p4I@ z)U@TPL&j3R^yMFQ-=fr6K?Xj9h5&-2vYoHwonGxC9F6`Vup1_yo4asB6>rQU8RBWn zlglc5f+o5q*-0R~Ck{|h0rPKwwFKOLc*I8o1Ky;pke7vkzTT1mf<|?!GJdl*vPnv) zq~W@AuD|s410ynGDd_S|8@%i=bF#^0wtZ?6{3)P}m>Jywt^ma8D$Z(P5r`1^L^Wl% zV@@@5WE~r$(CMcYGJsD4&eX#kXC-D`3~?z|SP)&omY9arBNzO`#hWyv5c_UGwjtU4Y^(<>_|Q-IaV9D`o`6)sDleQS66s=i?st zJAmH4>earxn3eMVSmtpr??Umvd!5&7fWNCOO5BYYNd_*a7*OX;&uuJZmk+$#@Mq}Z zH}XzWiiGCvMnIL@z*Wasuj8yJ@p*R}nhA7~vDRuL%Lp}*Y203z<6@GEw&ah_6h{Wd z!a}>~6S{Im4YcDdkBkuO>7R62D-N%ViJRDfgLBzHDI1J5<{(iLY?mAp9ua4~0(qwe zYOynCLs-%n1oB(VhFwWKn9Oe|Y<-Y7TkKjzD*8u->nw;QBM0yd0t#EnyW0d{lGwbv zm&jC~?2})6Y;Iui7BCi)`0+7wNANW$7#qB_j+G|l5scI6#ID$R(2|a@Z9FDlkrwt$ zbF8L?r?WrRxsDcIp~G^%}4h!E2Q$d*ZMD>N$uE6UZw}H_Pca zk@jq_l3UD<)wPpn>J*}J-TF{a>`NT7t&-{93BD@Oh7zi>yHzpxn@{#1^eXJj$R6PH3+BzjKP!K5`Fs>F#fxa-eBb7n~oTmyJu zF}r(a@ikNY;6X1M=j@;2bH+TV1PWCac<9?rQ=j;>*>B}$ed3OT(0~ned|Y6&ATLwc zSn7Bn^Gq2~w8K|~3AJ947xOm9JiL*@OEUWC#c8k3`%DXW|DtNv7(XhqSC4#Gk~#FB zVykfvo;eRyz9*i{C4Q+D@_zq++%;gGecK!|N61}GxpFk=Gq8+J?b#d8Vgd^vO?Y z*53IR|CA0|ojp+qdwk7poHaX^7N8;^(!zPdEma3-U#Z5zo{sPDf4l(pzi;-@)W#8N zH6S7Vj&K(yX@V^$f!RnT%jh|iM@f}qSwuMr_irr>%>NBHcmubd`HO>*=AKq6H{Wf* zK07yKLa+hiK$j`L=~#>37TAoK3@$v2`r7yeFp=<&^fP}}znKcohDVMDufp}MDsS;; zI+G{4T@SU5LhYDyDnIZbdR$R!pnzm1CHz8JjWTemA%szmMuD0d-??oODn5m2;m+Zl z|14{^Li}6-eiki+*_wn$b|%v0*7CRWpfM^iuFo2d>gbzs-9RS6nC}a651dfwR@(L+ z4@c|mTDe)=$M|sU)H3&&Lt~hj00zT@kyY3FYn2NFOnV|wZe6pkd&Hg^n+-*}YDwEe|rfDVfty;dAB5_XihopkSfcsB^Zz^pKI zG-mps;TA%!U1Qv1<7&jT`+yl}W@Gdx_IMI-oh=FoFo=eJ5Ae~dy+^8aJ0emk6?u3lnkC&54#mIXBh6$%#aD~oF94Dd1wIu6UK zqAUBn4_aOH@o82Ve#$ZGcH>-Q_3}uQNXW4$jclhkdl|d;|3noa4j0<2?Om@U(wLx| ze2veDxYC`$IyBo9)a1_?#WLx;v56{}(v{KVUw-JIoqfD{kg)V+obfM5{S);cd2rV} zs93gvP`kFgkZ5IM#o*T8j6+R6y~*>G9}?fItg2Kmg&LrXf5w|S@|uam;jQ;D)-P4F z6w96{s=Oc-!ZPh5iLa}ei*3r{i1`7>49SN$ZgJ^%c$L3b0g1B1!OsNu-`6ER6Hphi zri=8e2efd&j|6I|BIN-D?TBxZb=>Yo<#{43!o&HcvJ_~FoIfz^8!C%gz&Gtj7g8@$ zd-XxFkxr18f67pc-K!O^BV1Z^FnIIDCV2BwsToKNW81iaJ${VgE-AYon+WsrwLCSK zMuq;o1eVd#WQpf+7O4YFa7=hvDoz)`I(>1r2}=v(Q@kFwM|!AyC0-?#vx55#tZP^v z&?t!jOoec=*~diovGgRGGAIUBVjTJ|I?qkhTfN!G9?IIzkG5{1mWkAkP=Ulyl6z>V zReHfl)gejZNM|d^l9^DZ!`@-94VQesvkN0l->xA0Qu6y)bQwUAVLPkCq9~mxF>eum z#+_ag=c;g2UNt8Qq19daa_#0j@YhI0FOf1j)56F4qedU2Tr4+u#c!qd%^#zFoF-!3 zE?8~Pyl?l%)S$102ED1*Y=DF8WYC*e>kCKB1F=IrpW3t>f>Xmg-|k{NMVv=f1z1fC z@d~Xiy^dfaTCp(aA8<^r;|Dt~ zov?BvK%W;e3==TC(XBAPi7%^T8@v`v$>C_+xp<$c#kD}RBgbekX7~)0vW*O{>7&TL zH|Do81F$pl7xzRs9B=+JzG%u)%48aX#TVFTiitI zCd0s>RHO0-zOYK0mNc&7VdlsbUjYd+rRS4UJOd5Y={X`^=YIvj3wXRm#+fw$BGhd> zd`^(!sXDr0o8SBX%{Q3r{%^!BKi{>lPTDjLr~Ssad_xZXHcU9$ESLWce$re%oz&CQ zQvmvVPy07A&(7Xt{vP`LaBTjteTHgkQ4(Omrqmui;yfJ=I!!!6p8!ma@I=7=M1~(* z)PDD+5w^_@oXeA^StqOh0DPSnO+X^!kuX@Qr%`znZiW}IW9K&13!?SxJZyqkQfvSa zk{z20jdvj|401n^%-)eb^#rpllbu$z^5*4X%$&O@wo918i*DTw5YV5vRw*O8*b>WMc(bd^ zQN+xze3t!G`-$I3<+a4?!kOh)L{NpFaUSz6Akwn#vY6!W75?^$&qQl9vRT*qdiIVu zBwoH{7CCq!iuxIyt-g_fD7c9n!>ZmQ&*QEqYDE*C3ce3ZGRuDlVRB>`V)~xFGlIBY zPG$GyE22ksAJ8B+=T15g-!1CqgndYO2bt*VfrF!8BAsdB8>a8wgUK$En?jBn7$(Je zJ2$$Jebk4K1_yWNkY2%HrLd?{`TRT}%B6n@#RuHMYO!CMoe1HPwvu`w?`ZfhdFU_`HBK zks^WYlq^|(d46H)GE+4`8bh~~w zyg&q~CLj+!>%(!1w|U;uDz2lQ;q7q4ewM*k38SMojO^74)2o5*DjV2TX>85!qSx6cb~fckt8Vnp9m=p z8f6Nm5)O#np{oA*UuGq6a`jt#RTOc0VrgKcfc?<&JID2N+S8dZT0Sn>V*ufV<9x=S6?Oq@B3x0y9jf)|sTI;#VJ1SKl(jDphx#W#PBu z!0Z7bN-NsjVz!>zaYMlfW}9l((ok9M#mEK(o+N1{WwoUpUh{F1Tg=|(Z)hGvATQX@ z+#t1Bm~?{z9BnNc1|%iCjnW+`)F^=9k;ydflirGi+`WxR9gCH>G`pO@v-4ptBWK5( z%u6xyaj}M@0>C5XA>cSfbDE)wv|=;xmO;JdumYH8(}Pb&`iGsUXG!PBXtk0LmYf>s z>ABe9%E&L!mNEr9R5yHey}BSoO$CAorZaz7t;5bt`iXtNkiGe&SG`NL;qJkwG9aS% zST(vkbV|KD=!c2AQDe$EC%9Uy`>W7tO^7AYk{XVI*zDJMy%7)x-XC)(MWA?wG9aHp zsEG}p;?maa4g#Fv4)ZJnR3BR<0Gkd295?BNO#Q;AP0|m?i9$p?pQ(WGXfqSTg1o%G zP!$fAIAyI~yrs7b9AGE#i))J+fBLv?axV=#alOtQS29cP2zA*y(L7hyq^$1W7FYdQ z@iy>0dhLK2D=BvtS<~s_wxQdkV|p^(#`Sbg`2TV|FYje-Kpz-XM)fq!Q&i2ojbXWU zX+<%#gpYSZ`?$;5vFDFN-@eG*oy{J@o>S7#Q@Bu_GzoNu&;0lO!`AAhK=1$3V&Aw8uX5r*ZC zHkVAC)&R5ay_%i{qJ=oi1{0TnzQf;<0A}jw+kmO#^P}>fVxsoV*Krmhww849BAynj zYCA1%VPkBGK7&4xv4hXsgigeD1tU4%5piJPJ&X%e!hu(T4H?gNRA;i zep3qAH{0&7yvR`H&!G(z26DStd$W%!Nia(qv=KbIBVrKoG^#VvG_O?QjYAq>YR}_; zjEkR+wW%r3=vZ_wa&x9_yI*AH^6u`>6dT z^Yvr>gnE(_u%p-R(J5xSom3+1EKt3Xxa1=!Tx|+PSI9p2?_FPTra)K z0&u$D_f?dzPpI*-W(L@ArSN$Vir#Y&#x*}zciT67b*G9PxO3Dd=lhB3YfAc`Tot-!VFoSo?Kcm|Gbe=-xr`99Tu=Z2TO8 zmflv{{Lxt*tD_It+g!uqCN2N%sQ{pQGjnJ&fZ|J({gG`=fgI-}Z*{Ys19t|wGQMx| zRqNeg$miv#$29tWE#xZpX1*x50DegUfAI%kXFZ$N%e$6O_ooVgPB1?A(;Je1^6J)r zYkmqF8#8)jHt(j$ct0)b6-mGzgGYW|DsNfG{aPWT!{-618M}MjUcPs13L0`^t2FCr zYvWz*T5X7ZyXAv4ZX%mQZ(KiWG%HKqdc&$H`Q3NqT34-JLp<#9HkqlK+q(F^WNHGO zfqoseyp9_uFLyIvG|~tM0$~yqLkVz9R1~YDYAa2&wLMh1QiMfx>I>GEFTtQZSafmD zz+k3%h0HH??`LjgMXJ2M%v_Lu`y?xWWa8{i3Rzvw`CaKpx2u!rxJw1dVs23ugxjQL=xhzH-q zD&TFhJ>nP%Elp@HdG=1+C_g{&jFNn}XMJDz`XRmGg%zyE>wYXQJLF*!$_t=5V5?Fe zUi^{6m`MqJ6B|r)@&sUtJT>;IJdpGMkH8o6r(Ar5joBgd+(Jgr9fm@5Em_XwI61ug z4c0&8TWn*zh)XPVEuNETsM588H0j#-bBDKFswt|8ejk+e{y~j%p%4RUbYIo4%EWwc zR)<)|XiZn4VT9X%xT$u7?v5Yk=gB2bH@u@a)za)eS5$xHXuWf3Ks(Ht9Po?-KuVU* zboz(xzE5wE4$=l10!%pvYiu%O`gsDdi53i72Z_;(-54{F;bLT1C89a`U<7*C2j$=% zVf4?bgC2xz}qKyv551O@!Updb5+lqlI)7)Q1n;X7zS#`5X|Cz0h#BSLY zNd<*E>RvgC)a&(2KT{I)4WH%ZD}WtInYlcte^RFH%bRPJiob9ncrU*zNs-4_ zwitS*{(lGfGzJjWPjm3Wwfg-_40( zo1J_=HAn}VJeneZd5QHZ{6}Rj5GdE{1AX|MC<{kVBfTnO%`_ydrGxR!6yp<>GTiBA zU28PT3$I-E^wCR_RSNgTDw2K>DSSsH6rJe5Z~1();i3S7qN(0&Y7{ZxYJdd#{oVds zENbb2&%=#Ny@E+GfH!>LtEEikkItrDn3|L{!C$v$OIH3d{g!FXp(FAuQgX@G*~$}28u@}#cm zsn}?WV8UU5YO|E$*s;_k*zl&qJ}`qVULJm^S3bYB#2MB3_6|Y6p}!T_u4oM)vwX$u z%Px$H%ZKtZ9(yyoYtd6(H_s0nbPeLmAz=c69GK%<9=^dm1m0ITU2(`h@T@&}MI@6V`xN+WZS|4vel0^En1+Wp`@U{O(VxaV(UZh(L6n8mP7 zaGw4WI(L1LG{0T{P!1wh!!-NGDz!h~Af@mQLHw?i8F}J?0ZQbUj#4tmPiF2rP30>N z1sOuJQAJJLLc_X%k&zsIZXofo-{%Ku%?4%A2$)w6M>_i#9w>-@U(<~srr5c?0PJ9j zm5^LD3PoU9Q!g?wc^SDhdRdNF%WhZhM-^kC1~d#NO+g|d2^3yEK1(%Js{CCjuqeNe zO-*=*8^$l(R302FeMh$sg<4$Gehu!qb#4oNr}p}xIp~ol8ti6FIjPwW2gl4cf_8(_ zpPXq)Uj@P*TPHar0J#UCnFPqQ&NSTm#RfQCsT9T? z__p7#lmWi6ycZ&tQ9eS+=Tv~B>Bi^u7B3Gm9{v0NwMzp=@eHCrf^!4ssj0~4G15nX(|^n69 z;fu-7eGR97H_T^)VF$m(Gk>1aHWe(+t3Ep`A4D!JzZs|fOBG(g5&uVEi-1;5;$mI< z-xzWJz3WTk5t&)&$79QXOHUf^6|;&5&`8|6vI68avw^&3(7yL~_6d=HIY+{Ov-4pW z+4-nuoodaFu+^P&$%IG3USSigzyYP+um@)+R zdU)vbMM=*e>GSbvstHEgO@w6NqiiQcgTMWUxSjxsEftuktlhN@~ zmD|#OIiaJKxx(8vU0a5>@^9xKulV7?PfnGIzkFFtrTDR=?d1b zM(!*2jf@FB= zDamZ33g}U>#)sg!#w?ayc?#^YUBz4=1^CSRH*%(wQCiGLo|5JanSGcU{*hsKNw7G5 z=1`tAThuDedp4YhKz{F%6hujSz+i<8JsRwJ@Z$lx7ppbykPXKJek&5A1EMlrh!BOE z`Alim<;}dP(8feS>W=dwjH}=`W{ zMS4OO$jx6v+X$1Q4;MsR&!8tzH56^s!!$|ir4Y8Ib>DYH73Sc!1tJN6P^^PBZ<_3X z6ymRmRlfWgOh^5|NahCM#d%(Eyb_CKp7#Y8=grl${(z<&f}GEs)m!qVEdeaQ_@ z9z*h%m5+A5Rp!Rb8_ednBPOYZxBK!#O>DxhM0bk|*;N1m$!1g!sCIak8*>=%MsmEN z4lziS&WzB?r%_vd#;B-9VPGr+i8jtF)vT|`bTAn+%5zcOX#RE-b&EuHF+ia>Ie|`* zJ6BKVW3t{aIA;90G=W+jvatMEglVr_GY*L2$lEi;Qq-H+;`EKBzkj#KM4|7VyRz>C zkGkEN<^UDYyhdTemCO8F*{zFnK89OSt;pcQdKC)gVQN7c1&-Y&B_TW)Rn?W*`FtoC~v*tLf$2Y}#Q0gELP2c2H$jy8YTo zTO)pDNa2+obCxbGMLIoE5F-Av(Scoe?hBwj^Hzd|KzpV((Rb~0^>U{GSKkM<2pwUg|PoSlj0E-+xvU@)yNY^ zqG*us>D0>9W=R&s+)69c)0^RiGQaZEBWG;{>+G*N}M>R5QqkIn{+AiAMzmUPgWbQ|ne!nb3m#OY{mT|*hN7v_(c zEn?^BfKm`Ib0xIOQ8vA!`=}lLyYTu+aynbe@gg2Y^-lqPoWXPMR6rta8n=X<%Xm{b z4>`(aHWD_n1Jm2*=gD9l;0?{GPB5nrZV+XJCi_0n8{+vAx03d=W7lygo7La(4;>?| z?48H&mNpJ3y}|$i>{B)kHVp_FZ`nn6tCN(EkOQ4QNaKd}1^(MD4DlQhnrRb7VRB7> zaKAtwv|$4;Dan~8)IJ*I0*snb-c7|$<&*j6yFNUhHTdp*ZPCE!&f8VUB9v7Y=5X_e zQ|+*EN;eUi9`P`6HrSZ2D%e+LHJQd*w&D$}!Tx359P-Y&rv{j+T1)>>OOm+-HN~pL z#8e5`QM_I*otqHLcTtWX(`Sh2lhyX@(aYPYuwge@AhO6^OswE8cJtMSb0mH_eA^Zu z&?3gxbb?HorOfz(b{_p-d~#hl3dfmf7;OhxV;t6S$(sEk1mgvuK;7`?Vd@bAKEMHdDH zAqE_sp5Qv_iYYVEBMVU;5Vz0bZ-0m6Vt%ULp1j5~)JOO>kxj`#p8$xZdHwwcch){t zEXSx`7bTR`?fgHo-aDQOzx^K%M-GK!SN7gi3K2T?&L%sgY{xic@4fd{Rz{hf9E5Pn z3K?Y`^MsU&FV6QF_I<1@CBubhq`263c`u3gL_phd z&p!fJfR==xvqR*M{BNIr+XyvgP#l);VECn!J~vjl<6mqYBw89ny$#>(^71_#Tr2Wq zGB)`<$)F&eUVpkdU7p(kf?fU@ZQenHW}``Dczr;q_aT5*UJoWRPA7e4w3q;SZExlx zJ#GISZW;U#+jA?go)0qLOgzd8`(Of>r}aF#bEaFgk)NQA(p9ny7e$qLr}Io1=cEC?M& zSBo`wVvYzIf6$a3pMu*ml0Y;-c#9wpIb}5Ajh@s7bpce!9zyFtU+!d8qwH>F{Xt;-;2dtablh0 zU$|^X8=xd2QjUvtWI;}a5Ig3njS?+HAqI%Ym0`c?JT2$y3~KeVdqMI>Mbu2(@yQ%7 zeauXp3#Yc0+FEK88bbmLUwh~trwl`>Tf(=KhbSAb{2D1fLooUiJB1rU17h5sSpiW7tB?;{+{freZp0(TI^BgC@BAb&|< zq@RI`&La3*;BIH`8Hi~xFgL&U${bbuc^-Xfp8DU}Do+#|YUb(g5G%a>^YqWp$=g|S zZ`eD}riVYX_K%{IzcUjb4shoJ}ErhB_g_ivCaV#E~LR z-k;Bb)vlqG#ZQO)1HQE`nWXxT?ViV$1i7>Q+MUmrfBM7RcQ$4uaPzA-n~_3;idPyt zxuW}>e*=-=NwNKFg$M`doD8h3*@}bq2X7%?&b|%#8Im7@>ad_=Jx_{ zJ;Sg&M08JI|;kd`HP{V*a|dCf?t+;V-hf+HoBC`pxk7n>rDQlg1Im z7~<@+kkHeram3X`sG=tJdQPD6)}aGNCNPo1xff-m7nx!Y6Vay1`$7v#s1=`66wn-u zF)Q0I4<4_+`!n6o=CF$dqZ7*G?AaXSgm@Wz7hBloc~p?&6<^LCeAn{16-UUP`Ifa6 zMRS{ro!_;{uNNnUFe0{P{iGdcRByQi1tXhH|Gy|7Cr@LPn%- z8p?DZYp-FMF0*5=K@R>kRn%)%KvQqni}5lG1(`wg%|`6FHMX_oqn{&+F7{5p_6bPV zfiS&8v;q5I1=8{A>Ox(ldP6vF5qRL)GkUq8-n4pbj*POSZVnQcdWJZlc||v_Zj$B@ zQd5rE)URH7RD7;=<2>>UMd_1%xhUIC%6MU*MJJx(a+rW5;;hHcH?K}I% zf})gDNgu~>c^t6sYD|`%;ZB)5?e>2i$@fwD2u0APv)|qRXNg26Jb4Pb>*%FqnaIIvPjxAn6c%9_mV&d60ESJdofeL9imDOkb&Dl;kt8|V; z!;sd(an&c$mKrO&t_;jgxGt)cKt#XX>cTuo$AD~Fuh9Zxsox;Gm*u8nfJnq`U1G4H#0{lZB2^1$X?2Jr{A*&LBV#rcE;&pD zt6OSF@Yk!Wcow+sqz;91lKEom=)O0eYiyi@lYP}Kvz4@r+VR4-SFXh^0{{Wmvv=Zg zS;{=xBH&C*D4rfsnx;x8dSF9`m*BweU)2nf?hMOx2VOekZ1vb4zk!B{0P2$SlQtgy z;!KXW8{PGu%ZHbMr0&8&H%(dm)oT6gqKrLOf3~%rSoO$UyX{QhdA!q6RoL+(1pzdI z%U42)7A6&hAa7ik?(-L)$|j_ymQ!l&mZ@k-O-AR+Ra(YvsebjHBT!Gy{pz4bh_Zc- zY?qb&PoD*uZTPS9yu#j^)57S(twvwhB__$NI! ziYBX4cO}>VqlzK<`>9Hp)(>sN6iR=oqfkw{0#8m(_Q54fP23OJ&lC_c$VKz@pvz@P zmT6O?(dnua&doiHvaibbRUEt*Zi3B91;#?%+5V7{Qk~_Kqk{yU6d+T~9K){cnWz6% z7?NT-vmo;k)ebvJY{^~UN@BX6h(^RsQRT^|+@7`dCU>U6aHQErE(6mWvf&So?&ZRK zyLf@dbs+ewzY3Dvn!CP<4XS3@|tzJTZ}6?MxR<9K1m{!_6yZ!{1yuFz>T z^dRyb<;76AMW7zfKWE7nr-6SptiIz4^76Je@L~FS=ba)4p&@I;L;H^(xI9kykz;7w zXh5?7@>L8fKK?}T(DbYsb9QnpmRLV~uX@*%5WM8#)Z8W;{I$6AKUn}6ijzgqKnV>% zT!WH05Zcg>Vds)|+P*v*uiST%TA87!i)|o7bZZ$Or* z`AQ!H6$9Mp=-!p@Kq;X$J-TIXq$qXr)3jFH2i(@m%KLVD_5&sAM`#NFn<;C|z{1>+ z8<^vk`tGNol%c?sw)i!R^F4t-{GZ!TeJmo!DjOcM6D|S zT^OYyDHP$uA;0gQZnWHei<<%)6al&RO#d= z_~KGI%V#=--#>9F;r7@5Rl&LU|5d>`(Ig&k_byubem4G|_zB#jQlp7SW8he`)djFF zdK-mc5N$9A5K z$F$#5{PE*C{`ffD%?r6lhka7s1H1DDpN1_`4~Z`!=rV!F_yvjaEn%0OCgeReUg{{E zSL1txAaBntM4#^Omk^I!j%hCXUX582-DUk6@Vgd7aP#&Yk>;7+VaHHn>ljz7c!fe4 zPja>z+=B)C956>lv``a>wW!Iv1dLamP1Zj6$v!eHDi9AF9ca<9h+{?fQjgaSOdm5gs|6 zpt^RPhR$$A#%lrlj#iL5NZyy;2evUw^Xz5{5F7eY2W}LSASk2&qK`7(n4_V=I`?!7z05lwdOT#5ZaR`NiK3AFuu_W;Pn*~L1kEG zR{4TUIRi&uV)|tc0JE!jiULw=bngr&cbVp0#CKAl&IMLYd9gaqh?dV(TX66IfCben znFsNn-J?H4g@G4lIs;Up1aUfeH748fZ7$cJrPps1Ax&4g;>Oj9?fAQ$5UU9_IP0Zc z6;Fn6ijTurzb3X!joO}h%f!1m?%i2QY?-F=3?O>p>uDZcgXH^^q)Fjh;;YXkeu}j? zZf&GONx%cQFN4f!RpUfu^n1JVsVMpiraTqTyXV9DyFC)aJlIvj7>~jW!`a59sZq>x zBws6PA=&!FLFi*T)1xG*uH0s5gfNQ?&XOf@%s-Nhy2?{p+tm6guT^Mg;z1Je+-S(B z(D^$n_hbVfF6_k7@ZeJzipD6I{VAxB`Tqr#w?;F(!|!{fdjBkG2wY<`$Aj#vVK);DwmygyjNkEL)hW+ZNCqr`sasgw$@1lxr5 zVjQknGo6yJe~m4UVa82J3z*s;5$iOeP1`<%8Pb5%C`LH_g>v_5va~y5+ri`Wa6b3i z)=Wx$8f9Wxu@Rn`{kb*F=`U?8hFfITCA+LQyeTUUECTRM+svDumLCE;Ov_WWkXQ?6 zGJvwPa@>)e@_4o~*k?5Tb~@w$&|dt6j3)+jJd-2zR}Du~w4jmNJH{Uhab1Cgz)0z9 z?pVIek*F(sHJj_!NIvyhV5x-gMZ9LC3gENiU6_~2z^bP0L_qCW6FD;UWiLF8b*SHN-^*kz8{a+27lv2|~$#vf2ulEer zi|FBH1LM&{U^;T~+`gxKv1}jLsp1JFTTa{BK4gCPR&qd&;QYE9#3U<(Ej%f&bkWfwniA+jGB_t%9=f_r%wD zl5bt}W0egdI3g%?)#)@tcJs%^U3F%u-O7gI>jh_r%lNLJ8>?*pi6c+Z`8!AnSWBI* z(R!?!F;anZ*6BmT!^Yp=p{A|1Sav~38_=5Upv8@77JpYe+(3{_lngTGOUo?ZRP}%9+!7K}5Su-yTI=zWE=_AGqFFsc;?zGBQp7jvwo92e-l`pjDo5(AxY zTyN{1RssQ|4hP`mlF>yxfj(`-vUCMyUTffgRyZb6@&FEk77wr0Xx#--X?#1v>0l6- zW?$Xz1h!!N-v@zVPVadlfH_AwZuusqKEe259LG=VIW0X|x}FKKh)GcNKR3g>{3Z)b zZ`)J~YiqZBJO?8Bt>5_fPwfZ$sbz;iUcdwoa5Ir@7A9}Z+qM`Jo)Bkzrg#Cr+@bN~RKT$6SWl|asn&Pbj!eb0@YBw5p=nMPMp#M)m{Qn(&X zY_$|J*Z44UQ?Ttv%EP}(0f22^?vC53VP_!5%N?@Goo(E&evH?yt#Nxc*Ur~|{@df< zt~vQ_7yHfe<8E)G)Rq5TI^QuRzEfBdc=luc#E|?9Xl;d)OMN?Pt>(=!D(lh*?+Ao$dP~3iq(7 zkofUox?(f+Mqr==dE+2v#9YI2RT~(Rc@VPS2ESQu$V^fJu#LV7-K+(2d6o zPPw|!mpt^#J-}5*GFAC7L|pl2p-B{nPXECzI;m?~d@|aYM?~X8 z5X+ZlE9sT<*W}!zyZ@%vb7t{%q~sS52ttDR^xGn6x`1m+f&^}d?MRR6l&WfYDh9sl zB;$H}T;3Z#=suYtBbJYzy}0TabggWEoOByiAdS2hyjTq1NsW|UdY$K5Pda;7coI9@ zMK`|k1CFJR#C4UvL+p}C8cu+S2!MS?O5A3`B=ZP$`x&sS^ydFnpgb2A{7}Kg3yHsK ztp(yq^a;ZpQ`I2D5TM#rPc^o~ve3CEmDN0P1zg{G-x+91dMIwc`~ATU)iSK#i@OoFHo-(AzzGYhMwnCbk- zj+>L0^3Y|4=jI}A#uVlIR=uLK=eL28MOAP0zGc(Kn!55D-bZ<@KNIo5vct?+Hf{4{ zng{-*eBX^0gISG-!IfoqCIvb3&gFB_Z;*~mw6_J2Tmb*y=x`Znh=7XaZrlCbQ;FbE42=>a80%P_apY!v>g7N@~h)L}uBzfOA& z^2`wZ$oVmSyl@Q#*%Xz3lm&gfo`|A|6^f*v7UurBKIVw{U@6nOoZ;~tMJeJvt{4#2 zIbtf&N~BS=kG(`$Z#u+ba0tSC@eodrkb^?2`x-~qWGe5@+?jwm`H=i_NAL%zQ9YKE z9lfrwtoczA ze(J`)k(Pw}hF@Fni%L-+jaZ0f&hG@sSWb?~pRCaQ`>SKsIG|*s%Yv`j#SnC?#?n6* zc0){efZBd@swVSeUC|4^9Sv-?Op4pvhJJ*G-!_F0*3aHZK8|sEBExpB+Ce@Cf$gLD z&fR0M*VT_(Fm`CE2RM%Pz5X?8X)*pVw4mBc@2OgyDdJ%U=l)>BvO(sRkv|u|NPp_t zq|Ndyea%4i2FbBz4L&sP-0ml4ViyNXNKBjV>)9>T$u~b=@$W~Nrd4-QkiEj)q+5+i z8ICXuZ$-YN#RK`R&DXdgg&gJWdm}<=9>Ln%@TohPc@#vDl$9QtmW23BWfqaOAr{zf zJ5YEvS zZ+ai=3Ia)mG#%<@P4T#$M)NdCb|~8)IqCoLq66IdVZpnYd<;!k77!XAJD9GHEWX}cB4=clVeI4mceSi_wJLOmz_EPhCqy0-Y$n?Y)sv|?`9o? zYgkhrv7t5>pfgL`5uVG${x|zL-?~F$bO}PrB(K&tyuF$^*<9t1& zPHaH|!eCJ7-+J7j3U63CO}cwBV<#v$DpdogCrM$SH2v~&$xY~T$nn9Cv4znnHHv?I zEm1gC>^&G&QIDacg5@foOs(f~hL%`*u62NhSl5vq;gI@VC!smP2orqCGDDvBBj%Xx zJPi98p?&gNY)`WJA(bXsS$v^V+>lK}($F-m)JfPMW7V}>-L!-)kWD4Clgqi7+q5R% zoUc#tVfa&);rFh%i&D&&a`qnWOXU?vRpS1WVg39E>qtxU{^0Y7vzr2;u+sX0N?-D8 z{k%oHboU~%h5wXwyhGe+|Fn(fc|Mc%|aK^im5Ta)AgJtbL2#^c%&)POUm_xTxI zrv<@}c!U_T1^t8z?wX5r+x)DMithd8zY@5?PyE}jgM&p{p|rHobmL<9;JN<`mbzLg z>YPI6%$%GkpL&6uN8%v_?AAU1tbaAof7kCEY52;dy9zirW5lROpBMWhK%)MKU&t@H z{T{roOKv}unf#y1y+jud{{P4gUvMxLu=n07kstwM=fikKEcWhmVJ3*2sO21EPZMdjtk98aJ3eA z|Lx-AiWV0X4gQ<-Me-}CITrj`wSy?RBP$*U4?q&9!%#OF;J5g$bvh``A!YA_m%1>iymUlNi?gjL`iy*4TU-=f~Dxdna$&xgu-)faZ-D+>=3-2O{ z7{bOlA;)!*=w^BaSL^h!u`IX713i@9koT}dP`U+VoDaJn$6@~ak}%UPl!OoGW1RWT zSC~_+IvV)vtw<=bJ^?sR%lBu8fq({KY^>QlD~z3cdbcyoNjfOSQcGCUnJx>R$uSof zc#bj>jp##d8COkn2z7Ix^>rjZ7|J%bhh$HEI7LZp z5^UnK((c~Zgkn^mq9y;VE4YxQC_-*9G^p8ZzqEdHxt-DG%~>7k?%V&HRp0MY ztDh|-!t)u4!qbP-^OWJC?ce*Gk$okZZ}6`ymblM@qK`Z0G!$TGH2?HpFb}+API+g{ z9;(M^?0(zHOPggm;eGR?gXa0ApTU=8|J$dfh~d(n7stjn6waSM1^2NWz}@2gBV1}} zPtIZmDDMrM_wGeJC?P-ipiM2|W)|AXgP>3rY!0T!X+87&z9-b9k9$){r?j zqS^35wQS#zzlw8#>B2UR^0bU?4qN{UKcm+@eyAcL)F#kz-K+3({Iv(h9kioIB^WSSZS?)uA85)u&4j$NH%SrFJDieSa2C|d!0d>WuBSUZP*HdRBRoqHlFlj` zF;Iaku`74vu{dJE&*i_jS!A3Nxh00QF&=V;`QG+l+;~S`1a9@+5ajg4QeViIZnU+f z|GJKJqlFZ|Kh8r(OkaI3u3yJzE&Q^;)-LisB;CnKei9b2flml{;@y#hZUv#N8(*cP zEXYz`FkKFN=8&eQ*IS!)<-Mvk4TkSwZxjt&ngve}zup<6Pq3zeY$&L}N_e0jhBBXh zj$dZH2QhsK>v^RXFoMWM+n*Uin%Fi;N$59k`@|oho|m4x(CV zK?kJ(Hi-sv1_{0OO``HoH_qp44Ne+bj5-Y!9dp`Q@i}PrM4@~!w65wa>Bb6meZy?U z6~<_N?_n(Fv`$_1=~Z7Ar8MiwF)-Y|Vzo`CwCV1mD0Swke~Cf8=C9=rnl_+lmA}4n zO6w6n+z@)|k}uI8Y^i@GW`4{Dl<@LNuzPPwu`I;B^FWaBdDoyjc2A#-@*7v^$?3+} z0@eQw1)v{4vx1qm?isgjDJjS}StybDWj?)+^_*~9VgF~|g~6m5{x)ES^vmZuc>V4W z_ulJ$IwdU~3WTqjUVW5O>iYWWu2}Rq0c}p3#H`kto7r z@{;;goBH)1v|A87GtR}zg>*HTnPIOB)NIPJJfYL7Fe{paUW3h_6;H|GUXVJl<(0tU z#nt*0Li5h$$1>w?B40$?&Xy6vsRjc*K&sMe}(3UEb=A35NKq`oe;* z{EJ2PR5BXsZA5vzgE5qWvx4TI&=?Ku_1T9)2IQ8)D%eS0`PqD0FM3_ucGKG%Yd2vP zJoSz_qds!C?Dm040o@mu+g>h-K^f>1LL#z`)idHFGF-v;jy1oR*4tJW?`LzFNL3e^ z!X>1TZG9bp#S^Z6M)G5=?sqr}<>G|}6=^mq$+QY-6%BOgvTX_?Dah^zz=e1r#)AH? zQixDJ$%5*jFf;;%{T^T0J!;|rur8{{`t`B9j~u-gkD~wv!ysUQAqSc4$}Avzu@M7VV3>x7*ic)T7=g`-K668h0bcNH>>3(e#F*sN*A{I~f~0(Vua zvZ%?720Q4cJ1ZuvCq z+}<&6)w=$fsne1n8?;~q?WuhK>F5WmW|Wigs_%+*bl*qRQMeb;priO~;|?0<6~pA*M7*J% zVn1&kpJigyk90;Ytnv#cmJh+|^1nsD2`lZ#q9p}usvV@oR@#(UBvrhsu(dpayZU);8oj`ET}=6u?SGnP z%f7k&NNvk!V%K^hJmlEzTN_?wt@TqU#eR_nOdc_6es{?j=2|hZ{v;yH-95u3a{0{* zST713U?7>j``J+AA)P7TQ)O5UNTR64WUs;CKaYYja-=YN#h-1P--Y5+iPyLG?(QWe_*WapZr9fTwVw!_jSwA}iWFW7fpVSfz3NCrQ z8PFM>JB-L*t1j*V?^*`O^`$?;6PI3NIyEAm%>H=gxr|Eh6-~yKa6u_^0`>gv(Ok^* z{{v52Huu3YZ9MCJF<`aQLc~Qgid0==a%56=-@i^vWhVQHX~(C1%Bqf5pAa7va+cbo zz5Cr0D}i`KGjYi$?$UKvJ5wyL!U2H4l1%c zI?<7vi93&+sV$E+L`cP+mOjJK;_X*Xeyo0ze#2wNuVQm+@FIm?kl9Ymk=-TbD@q9K zv?MQ4nVkc>v#uv}*_Zjxd88$@?8n6rhO%K<(QIvJb3|@NA#;dbPIvzz9(64*P0sJ0 zbQLw$NEIV_+yb-B%}!uvv(hSrq=PGxooY{!PrvfbJCmJ=A?#+O=E7%7{QU~iH_M(t zELY{Y*$bC!HP!Vcp%Zpe@OH+A;#&V~&SgtMPO$8`3_2!o{BGUK)Tm|^7!gan%Vv>X zmj7UGE_`96V^2oOsH>N^9z+JeQVO7y-H{XIjP*=5qsAUTH;LNYuUA3nid@xc&W1Srfp5mF4ALe&w8fsX=@bcoK>~?pbQN8J&Gn{CIvBe=?mh!mKCjd8L=R z>Nk4gQl4G!k1WSC3Em1)V7^%mkySt5=Gaw(yJF_jsu^OT5&cTzoCrUFY>ZO0UQ~R& zR;M{`t>qRyMjyAFaKonD#3DNKxB1-a`-y~Qk({N|9ly1%Ph}T3Ha6b>)eveW|7Se4gpouP zp_F?x?hjND(#zG4Sn=0#IgZO(#+kOgu^vQ-Gw0#D4}=8`C9E(D-u$rzUkdnnNMxU- z-lEfznoUc`(PDxP#+hC*z|+tVG;^Po^}-9SZR44B`kNTFXr*F5)yXlKG2Z!8mdcyb zgJ`?1OfRz8+N6`8UdYBZZrB;bkW zt3NVSe5qBWi)_y+H(`a2F{twbFu=uf730MWom+`p+|Ax#gy#`T!3&?NRm27TGWs#% z0yn?#pT0G1#~c&L^Wq2eDB~|iwHczXQ6%7-C}CY1D*1%ivv1NrXfo$@en4Ys6SoZJ zb-t0GR%4grdMtHKB#|Gz8}nre_6~aSYL3?v7@t^z~5P=16ip784Vl1US9si)SRIl6Q6gQ;xQfQNNXUWh#jdgBiU0Z(2KbJ|rPbKf090fJruG>RmVqobPqP zz~`=Dn;(tk<@YC<+^kl+{PEhBtAfpK<3Av8^a)^yEO1pFJ6<{BoV;WGjiWFbM;rBk z-3NR8r;I?<7K2`zJ51Pe#wLS-Aw?_0cDPZ8648Lf;t;LNd$GObSf>83=2g9%CLEb^ z;BN_h>_8Fx^4pA)7Pdt^K$>j{^_nM-msvI3^;-$^0s_kwH`HKf)Kv;;NzoEVgx(b@ zy+M%gms=>|v3LUxpG2 z82BE&Q^UGMTWlUia8)oZx6`7`(}gga3A7bIjvRB|d%4dRRgdJ)3!2#LgjFFwd=gaD8XCMa^-n#Es>2{*7B8W#e4+ z*OnF9R#yDv@G{z{g6R<0L!PXBnr7*XmwKYn_$qnc!8EhfbpV6#h&zUq?aN_wm~+0N z@pr`bR^C=2-zoDl_n-PawmBTf=sz%QH3``G<3grh!M8$tNFx;kQOfSUa>&?W_X77# z+EJ=K25n>9t@(ZRmo=@3j-&t!UJ%@r;(|W1xit_&o7SdZm+blp42C?Xta#abVQyqJ z>xo0H7#%0w_}1KocGIg*x08lE?g%ia33!#^ZhoKHR2&VzQyWA8d zjBKw#>F1b4a*X_lIC$bS)mJc&;)+X(3(U0EyDYNs7sXAv614PDCThluFX2#$zC9HU zUcEy7hCLx?$q%_nxw=hHa{Ym~3H(ZS3#N5cnNqy$YJB4#8^|iHm=M((1lg7Q{`#RO z>uqF~z3PRX&#$C$cXD~`x*bM6tO`{YT3MY2&;8`3-^+Ktde`V!3IdKz`Tlko@{D=K zWT61b(1WUEdIr=p-d_nhSr8U{qW1m=bQvNYkCS@Mg${=+*k- z8!5;B$wzU47P8OidLEt$#|2icuwYK`yzr@ekfvrZJxH(9NF%2w?AlldNwrC?koVPom$?Fvq)$C1jg1}u(+mHHoBCUYuVslhIy~Qn zg!GXtHQB3mu!O>>E_`6J_J`MZ{x%mB5y=9m|0a*^D_lnBTd~y`u^!{2Fv|_vv6ndO zOzXzi0W}GFmJbp^oT($2HN=bD#QSC?(J)MK$vpgn_2P+{y!v$;BQGJL&;)*=rQa&} zp;%nkyAAa8LvTjiM~!z9(G9_Ljv~lc<-@z4q-uRi;JCoOCt>PJ>CJ^V_^_)0$4X)Ai)cRw(1-(4jA$xk@jR6`xe>m>V!9 z6J=kOr;4L}5qKM<0NaQk4<{Gu%Qj9c+qcZiI6pFwea65%OA$K`u$!Gs0GnEo$n1bT z-xi0euN(*t=X`%Z*yOgLWHY43!|@R~Q8A= zv}PE+Lh&b$e$_njddq&Z_oDYn3_>{>#%_5;~x#pmR49R6~3 z1FzDWqV26>lbs>TYxXveIiK_7WV1fC;g+nOu8gXnH9hGS<&FqL+cn>Z#Jb4U8MJSp zN}=u+IOM~KWREk6=S~kth>fJPa58I*Su+%+^f}|x24ARLmCLfen}gOTBFp$xwD^X7 z%+`GCeR+J_*dV187DhOU_VBq2*%B|YFkPJ8L&|R<{)dbD|CJ;19(c*;h%hPWctCa8 z>~tNwZ_c6WL~Oe521Ct3)gdyn);Rn=5M)wlvMn3pG#3|Ms~sIFU5X8J^#swJs~=R? z3?ceytbha|&UOD#Lh+cH6(9d4sx?=t*jf+dgBFQ7CEY*eqFN0}zbRzHyz8z4i=?<1 zj@}LZu2gd9(Fr9MgZuE`f39`c#AiZM_e9OOnIdKQ^F^&vsA3|o3e0CARn#kyLu6Jj zL12U2Gp0Ri*~yPT@IC~|D^yI{u0bn6zWI8W;>z-?JNPh3Ji$DJ`u zA{1Ek3FAA)b}#q~4aE=?*Y{Z zc`kyP9AS_6W1(eN+#cj{lLtxI&?%eHQz;zHZQWZEWO zTxl`)KA!*X%iuzrzBgUbD}9-}hR#7w8Xf7OKU1valq1(Cw8XQ)VXPqrQ%laJ^D}(0 zbc_d51R?|Q`Sz-DpDB~+Tk39BnMS5k6y0xH`^RB(UUD+*(6?3zB+aM`Kc(R)Jl4$lNXU}U^{2TFr%Tbi|)h87^l(Tv< zi${AZ+e(rxxT8mDB$2l;Iv35@!25G671t%BFRK0EQw39Xe4vr^b+K7BEXHD-$qmBG zU=1fP&j1PJTiot1p^BLhc>g0YmJNBlkj8aY-h&fSuPLfDGZ58H?4aa#H9F++^i2$< z)hbnUxPpiM3pLw7IaY9j#UlaDKYKU+JjwD6^4=QS!_33-9Z!dq);yCNZb1XvF>%Q` zjCY~#8iESIoRa%A6@Y=c@ueQ>!M4$RZ7MMAZpl~Q_y*AXoXP%{Zbn163t9@hDIbxg zParn%mT6zK5>9?Z-sppQE_IbA3V25hX`)Qfl{K0VsQq@+r7c<+MR|Me)+)I&%I1L>O!6itETJpkS}K>#yPo^s`1G)J3`6 zcQKue!*2Z>=K|ETSYr4J?MK_u@Fq6+^W-70IF*V3VD+k%R(eq;)4>a-D>M*gPx<~} zR)E1ze_QW1fjr#kZa{|lKZ&-Whv_%MSLOMIr6~SxXkh6Y47=OF4$~Ah-bBgLvCUP#Ej@?1jP!ZVr`m3Nk&Zh^T z&qWf?+SMILiv&~Uew$4#^lr1l!VmPN8~tY`Yb=|r7{%v>giUtSs@9dslqI*k{;5n$ zDD?8T=?9MYnYj%4Dbr>pV$@UJA2IblES|KN_Ng%W;ud85##N3(7Ri>g5STg`9(mjA zW$hxAm=1QK;5;KurFGR1a$<8cUcC1|q<+{t(ztt!N$yAN-=_$Ej1%lFAE(>e?x>FZQt?7zo~XWL&ob^0TTGuY!hv&#>>vXb6cNap3CufA9!pO zqK|wywW8kM3)h!@#_KLZzV40XH_vQ_LIkb0w(O9vKL4tNV~<&Ld3L0p_l5)V7fma=VM<$T9HbeL*-0u`JiJ8DFQR`RLxy8(&CUiJM5HtY#fhF}^poN>I9y z%m+9Vg!T2{^sj}>{*ME~#}G?13D(G|zzP<8c^PsXTfhCt;|H4EQGKa8`Ax(vn+yD& z4|Hoxi^_S`sUxvArrjp2*<&{7$M-HTn_FCaGf=WFw_3;gB%ImzM%hc*=N-ij8q?-r zSnR>=@5Mf^t)O;{b*WT_MZ@4|Qr?%vH@M=OH#15ZtYmF%9-FV2@aYqH(tv16CNLkO zpN{L#zyyn~P6l^w1+J_w8l}!%jRt<8D2wP>f;D{bVjPFIF-4=u#{}JmJH+robExJm zGv|$Nnx2)DXN^74yz+J#;OGD)V$6F6_4*)`oz~RaZdFs>a+P#PYF!mV6AA4#lub(* zS^~i}^c5t^vbnXZpVf)Y$(zz`0B~@=S(~Ledd?;mmjvQsj-_bI1g0kvIH<~yj*-ns zRoU}NlmvRQ1EWgs{_5o<4c@$Iyxy^5`j%_-z4zi+yMS=VGs7g6J)DTi(enK%=Q_`C zsndjK)gAfscj~#vPescsFUe!6>nOt+T#!RH zjINTs-EOi}gFStyi3wUrm>V>@;lC*A&p$9*O@lGEwVPgM_kl;0V=*=<{8`iCFE^2s zDW|_5gvnm7l_mc{-rNB#@lKxIz@_7_^?^GEbW-NcJ!lN`3VIBbw*vn&ZSesz1nx)T zI@wMvFwtFpTZhZzxl*37>H1q`&BZFh{BONRd#2^AC~fSqtvWLNPESF$*A}M?-y&ac zVC6uRl7I^^*&gx7g5MIwkQm*^b4&{vNzBvbKge5y$GPFyLm!w_zHJ*=2tIS<2Z2}_ zq0s9rxu?M?T4+;myLZk0wLd(yjZUqVdGCwl3#{oj6QM|Mx6cvL-lZH=&EMX9v_wt> zA(gqd8bh;qEmwNb@pp3z@*Zfmo9O9wQ#Z{4v#CHI#8hA&a06;?g7N7_G$1Y$&<%J# z3$xrKJ)Rps^DGn1*83^$rM1iMOiQ_@agTV$E4hBH)ocP-h)BM5TGTN(31ES)d{yiO z*BqezxJ-fBJO;YXn-7Cy#L0UTl5b&7OY6?;pejoztrEk+Sdd)qY$!Wv5d)wJi~+a6 z4Lfe5%Ep}MF7DFCd_y+a&_Cd=mSUoVS$K7JnpvNI`k4(Cp~(9QWmYww^lrf~FFa(%G|d8?Uh7)2~s zE704s;`7cW_GW^_X;4%smKaZ$;-{hg(`_zJ8wz6wozhGExw<(MTin=l3_S`Ff5(KVF1l}Gic)a)te#JKAQ}|teWSxM_m7vd2J`%(=u#vT|z5K zHUvC)^Ky~P;)`o~ZKUz0+Thr5`tbP}4`C*DV<;^=v~;>QS8_K~jm?7VAopEb7!MI2 zS&{o1(^t)h`r-6SKTW?>TFT(JL>B%abJ<7&G>F{Ag#@z{&rHo7x~j~s3(Fq+6jHSQ z7u9&C#0~Rs4_Bw@eLpATRPCmIZtDG7_4oD+KN+4S%bqNS;@rvxj02P-w5hNahOZj~ zP$;=rc-%6My={Pp$>)jZ;&L%{Z@APExqmrZ2#6kN4m}TP+xx)0CO6$p*0deFOzW3v zE10{yM?GLau4(tf`ZF4m3zQrX_3Nc{Gf0ori-^e)IF^S;Eb(V}=$*2R{-=o}+rNzj45O?0;&{I`RPs%fZ@Q`oOig~w#vSLIR$r9)4c*}Gj1Z+#Mc6QZigH%Yb9<33$Y3AA}D zJ2uRCh@OOhH_zetToILq7EB*Xf(igAe9kAV0)$++);@^sIZAiSDfq8VzMf*f_0+DyCbv z=pY4a4dyVafO@E4!u)s}LHjQx;tU93C^RWJ1oq-N{6!xb4>?$RC$c&m=vOgu7?<;_ zKV;9Tna;OCXEOp=qJ!2H2zo-JQ+=^O-?&Lg{UJbi*)nk&@n!T#*%NtoJm5%|ojOO9 z3{{jp_gj`3`cZQq(u4c@Q7EMe=~(uoaYTFi?@wuy>Y}0$S0pWArKN-A?)(t9tjQTM zoPxwMgQr8Pm~-cv;jOt}pA<#zoCl*w-t;~+>+#$yW(Uaxkv9;P&s?6Ozl+ zRvMd{$Z-oOH{cH3|HySh6`(8uzn;BqUT1jlE7!RBXIu0p$$5XG z>Og;efvC45+U47X!l&?V|C7N-`|~eyhd<~XSFkyJh!!5T={IZm~oAr(|*BoMaJ*X>BLbf-+R zfyWKqW~!IB5fk`cg%hxJgt3=j+GoRkNbVyu`n=z{8$4D)bwoAM4%y85kwv+-z=*m8N?Zw&zHaD{4vd=&!s}_tG(H_O%=h!%YAw+tt}d%w6#~YyX z^NMw4U+X31Tl8UQMxfmw@74-xO&*HX>Y!G)nVzVMN;`bn`*d(xS-WhZE+0UwDzG;~ zk3?__E85BCBN}K74WLh08wEML{vTE69Z&V&|NlZE;aI8cogFf=_m-5Eos2>_Hpj@Q zY#}78Y}w=F*dt|=kz*X|*x|@_Iyk?V>$<+5>-zoMt#s?w`~7;qo{#6_aet(zM3I2M zu@;TK+r|DEN_;RZ%g7|8nzblz247h zS(dMvvVD=n$_OV+C#kR7xH4UQDm7|;1L?pT=3J6`f5nx`f;)=e3@y)f2gURF-Iqn} z$`bvFjINaCEm`5HvK)MqBFgP`apyc#q0OoF{c0+hcPsvA4dS28f!9c3BCpl+_n-Spc|q<=8{y9 zu#*4_X25xx?YYb-8AUhc=eclpsgUhg;PDp&pFMWc=dw=R(u;D2O3vZPboDQ74=+|#+=3X)AYV6_Kd4fUiQ0X*9}2vY_NL^zU-FU zy&o2k@SpnU`ya&u_a`?r;k>Pr3BfXB)bi>60QAGrPa7)9AMO_W{6FLk#F`}{TH zeE87iUcUQX^z5t=Z^?9Mpqp@0oG*}{AJhI~cz}P(jJC?L0=NKN-k!PQ9U0+e~6MUG>$-4G9#A6l|90|0_(2`qsZZTz{7j6F1aLiW<;GTG$G@5>b~sy|$3z78Hj z!0qUIs3#laCx+d^N_?8Ef?K(g<3a4m9%ge+Fs~?K!{+(xS@Qo_fnT*dQx%tfN%3#{ z$Rm6jkR8R0K@}Kcns9`op1(gK=NiIi_i_>h-dW4Hs`_k6XU^?!nWiq&FZaU~JPVq) zRoryung44BQdC@sQ(5?#O(?p96z+*uD8K7X?4$4DQDfg1alQx7m>1+QaE2J)1EM=5 zF7CGs0H57R`m?e!QeE+dJZniN?xu+HZ3c!FleWJp16eYXdn21k-Zjk*4N>=w052{s zUp#1J-C^x*SH(dXn4&{KxrOwB8jsUYdZr)&SYyeoLY54?~3$Gsg>SN=_mYcMX zGU%Ny4m2kHF!=&GHRepO0x0g2nB-z^E18@9%zCc z8FD%bDQWrp8(sAnuxC&oW4_P=V>*SSXDg_;7$v)zQVC%w0Pb<1{eKMY&eiL751vJt7G{)acEWA|77HYxBX<44ZUi^7#|`bhpWIb02G{BqlW zA(hx}@bvvd%yj8mpI(lC)AFi%$Pcep<>Rg~{$>2xNr?T<`IiGBpKU8P&pA1rijy`L z-T52;-)nNJvd~^}wzq!FoB}69YI{E8+f9Jo_a42}up`6r9%Gq@ZQkE`J~YdGWwHI1 zP0%(R*+(Ya%o>C{O1VOIBeNrQ=q!N(C4Bj{jJb*>MQzh#A>@n=m*UrR=i00GGCfeb@?1HjCpV92rr10#c4}$ znlg6D$!GFRQNt`H+n({@DivOICu4TXnc_XBCtfpn?T`&Rfa` z$-|&y6E8R6$Au9t&eMD{w|y_OM6-g0xJ$t0C>Tqs7YZ3RaQKU2GF|-U#uXsQU*V;S z2ysX+K3xqj;Uv-6R-{|Kl;f#~zE;YgR2&WN@sqtn`4H=E-DWxitfuhK@uA5B0_x}| z@mROgM^{zq)OJ&yUyPA~_^{RPeN5Df6N$AZ3O45aiVHRsw2SHTDC!s$4AacDiy?Xu zjR1akV`OFG?X6OMBHv4E!#{p3?X!?xttim140R`qhb18b=kij4IAu0XXe*vY@<0Ti zNKj4w0`d{7pXL|E-tvk_RLfsJ;XV+(uS;Y61%E}d*6p_yz}0T3cUecw4E1s8Xi+9AT_Y#G`(Fp>aO-jbIHiJED5-8gb;p7qDa) zB&?*Xu;&YWh+U%T*zVX2g%hhJ5~+%GM)gw<$oOnY?XLduCe~<|ue{hrRcIvXe(g&L zK61n!<4=+~skJr~sO4htJ7S~dZeI3JnlOr!rvKbs=2(fcal?))z6z&VoY1qp+MUoH ze9XZP%^9|bp)R8+XcB+6uimf(!wbszvWUzVJ+G_FHI>INW!U|^Sx>c5$8cTp>VNnC zf8MbFDeA^Q|B^znC2eyQjy#HSAWzPj!wEAzLitH_T<;7OQ$z-X!ey^IN{YAFZ?W(C zS)3mfDUi0w&zi-~AsG+q|5~6+ikHt=T~P*D;P#Zpzyf-&zU;1!({=!SaXxD`3C11! zwYU2f)1kV+5^2y2te~C*V~2XNjVeToI$=L2D1LHPY{Xo3baf&&jPHNr?NNs<9^lJ~ zdV@oaWjU;V1zSqto6BHR5f3Y*-%>l>=k&Bvk6|^Hc^GxFa_vAZ;WgPb{QA@j--SOA8Y%8B4=ZgSQyv5Eq&3|`dqeDiQV{d=JTU^m*0Kta-_L$w-**_NBEp z#c^f&Hs|z%+xaB1p;dz^R>{4mNaK_IC>W!TwIMGewZy%L_q~={vW2h8_wVZWbp?3C z-1CtjCD_ce?AdcI>T%i`(<`GZWdbCdor>TA-xg#%nYPr=c>XN+AoaLSt>Q1*C@Gm< zj$Zisy>ABpU1Pm&Up2~DnO8U_ERnYmq+3zj+seapx4_ifG!$6en@)e@><=4fH!N?7 z*%w>jvxc?(>!S}|{(tsqX>Nl3obwDS3Fg_5aYjXfR6&!rkTu>jE5+~vbK4;=}g#Lo}`esaU!3N#btR~ulF*0K) zF(K=fPGBN5ii2-H!>S4xW4uL+0v{|Noscu6=ywMxBrIiXL3BAjL`+j4MTvMp zzLb+adl(4f0G~j%^T#i=dM$6EQT6RyR+tKHe>!o-399@#!=txw)OnSfYYR_lcPmvIVZdHjR^t8l$eln zXU7{fCYqNh9{w5<08p_QWQE&#n~N$Wn2M2z3l33wWphlb4b;+>)JG&gX@(PlP>H`C zy;uRk_yTvi

;PMp%NiUyKy-vXhZx1UhM;(Ms9!JSW5TVYkgyY z7!iqlV2sS|yxb+HcC$i_ifDf9Z&ki~%ns!bI+}i=#0-}gFK@KEm0g-M0IMQm3e~?4 zOUtZU%=q|nhj)93N#~`6sjAv$;6?5WZP`zFtr6S4k4!CCS#Yax(e0P6)kFTIx|1dU z!~!ze2!3gNSvk)L5nxDlc}MjvpeMQQ^gZpeMq8!)x3{oah4a#Rp_HapeDkPptw>{Y3t?^Uqg8dowxK!dWtE`*drA zLpdu1-9<_JC|eRjPcscf$KDAVK&FaMgIQf~%59td zoqW+~a<*}Wo3B9~#9K07(kcyjZ0BO{-d{l+AMZKkB_TVYA=kSG2Oddbm<)Q;ZM3?B zBy0@>gGN$;2@Jy!PjhTJGAsY3D(i1#T{CY8Hz(VtkUx zUtSCox|;})e`Or|psc>4kxcHDCsgj8On!r`Tu0VY=2h`>aFHt4%*&~^TPZnY-Dif7 zy^8PAF)D(@K#se{v1681EJ5@_7$W&iu8VwFM<|EHR)&xFYhF@VR}iICVV;(w}gv4AxXnrUF8CpynKac+BF&GWe(0=HUu{Cjo&FnTe4~P;aDQ7z#vm z(VQQIWoZgeMi&R>xbu&EjF561fl_rztvxGJgK%OJt!dyC%0%_qKwD9~OWp}KYx5IB zx(Cq+K7{7c@te8rUmw1MkxL;k04JQ8NFq+{9RNOFS-N1fLZIT#?Tb+!1M zFTISCZfQ4>X?NxYmi%qSFP%km<}%4Y9s%tZf*cL|AVX^VXQ3l9Q@`gFyI)|fP!Ac9 z-&x$*XSs}PvV@TmuNyY+j!B1HTR9oPw zmt`f(F@NqO%)?_|fk#UA#^nUhU>lp(?An0c@w&~|e=!X7q->7Q#?QX8yeqW~V=XnM zSupEg@OWv^g#(TqdYQp)_9=@mAO^r5xls_QJ=7P28vJTC7aVzfC?%9aM-kgM|CaKM z=atJHUsWz7M~jXBpeSS%Jd{APFZEAj>zG`bB@@}KQ0ulgDC^0wa+me_$n?A6$;j@P zw~Kw&Z*eu0ek47v$k7Rk$>QZ*uBC#<0*#aj)nhulfB|NBep3x-7%YJo0Xrl_k`uEp zXGgf}M+(-oRz>ZXw>_e2b21Xo!II3*6STY5$^s~IaiH&(*CXnhPfni=my-~_H2G!j zHpdb&7|Z^oK|;lM6nl_-FepDxAy8`1r%oCi*~N%QnMH!T(}FL^lB41yAQJ^AUmitr zMcn~%j-JJ$fBMQN%<*UDSH0xnWW-wd=5wLRxJw+8SJV8ehoO{&PwL3j+M#wRAU zwX_(%WI z$Hs@AtGR>VuiN&eG3TRjm+XhKBeFY1Xj{Bx%)ZB!AsxN#Sa~QHSJ+R^%P)FY;oh3U znubb5l4R@%r2O!aA4R>*xk@0oqR0B>tlk&Cg~Riq@!fO@X+!s0v2C>bZ8}YZ_*saO z%V!eh$1u-$Dbiby#g-V;&EReza9pQ@009sSMF@dOm83gzocAj4^X$y9wx|j>Gu~u*2 znC_1M=hY!1lDB>%D?-o|ohql$_sb`iXif&rU{N!9Vy0n(QV@YvkjBh{3(V*5HP5W5 z2h_Q6J*!ti#cwxP?w$sJ?w&X}#8p>H$H-i$BiJyO?2)w4yr`^+Mw9YScCxlbGWTtZ zW^$970%l8rdza~lfIfA^Ho~#HD#7Kax?e{8vvDz9wQiBTLvRW1ij&}*Z|q9PDAwF{ z`o4jvG$ZOR$U%HYydv7LKy+Eg#P_EyTKxi6*eKlLkCp(bU-a9>?me7bT~Ze&j*S3H z-)2~2n%o;JpLYor_P_)mk0WHAlw@tOu?cPCQ;S*YFBX|Z*6E)bU?EE5K)#n=G=5Kl zKF95(HSM&T%kM_%_kh+Hm&WFFE6_d_76nl=GB)nR%^l09%fY^$F7qfP*3{J8YmV)! zj5kTg?nDDCq_Z5g+C8CNL!;U_>Z0BvK&c!f|u)JClMMwwGFj*<1%%w*3$77_0h%%}JA=xUnD85s(h zfhrK~x8=NbnTTecUd4MEZC_8ex(ZWuPbtd^#i8g^hNW|}24x=eviS{D!KuqhlU+JK zvLu~?b4TvAc_vup(pSFKJA^Rsj*{I81Ja!T6;YpaePK_Y%Tosgdf+=sVQ!t_QR^_1I{G00? z-H;cVh{@__1%-v`U>VCnMqfC}K13TQ;FXl5A-!Y|$RKYXa4rn{eOt33^qJodHc#`* zX_lX}n)~kVHO95>`Bj83goPsMU32qfKRlZ@Z?0Dp3gnZfe#~(nGz~**qlBk z!SC3wT(ZT6g?&3+;x5TA<}C+b!5^R3KKm$wXQn$ygz>mq!sMi6rf|uFi!n`(E?%cx z2gR*)Vq4T8Y!-G8{3ZcM8IXdx_t~>vE(*r5gDE}k>Ed!#g-E3Zih)--m?%3hyGTB0 zBcv#Ij~?AH5y?8CMHgW{2WryTBgeRu0*`8QQi`$z`ee_e>GV%Ryah&hWJtMC}gt z^68&{S2reFqAEtqrf`PmPBjt{Za;%X{X5tA9qW@Fx|0mOiWyqu-wn;uXq}lW+_F zm6X-CgZP{ddL^k#f3}F9(Nfs$U2d4*`anAMu3;&-P>c_+d0JeirV0n903TEl)3H0h zuZJpEQ8nPK>W*^C<;^n3bJgHJ{jE{WHm>ACBC zq^k$F!`+s4x)h7#e4RkxhJdm0`6E0tfSplKarEmsDEUr)J3n}bM=>T@fRV2>1d zD>Gy)bv+c8@&Jm=oxJmwmyH>%ZZ6Xvk6UT4k+VdXX-~b0$1T3*ct}pAhuW8VZ08vT zMK#5no6G!6MTA`=eisb{8CpF<`7U+vK=FsI-!PEb-_Q@4hCD+R5wFUz(zWnCV8?_U zmRk3MkGOqp|M48G(jpkUqW+jM~I+h51EhdI*s47=jeIhROz zh5C|tSg`M}GaVdr+?54UCC8;Lvd+t&oD^>6xxZq0NB`rRlEg`51~{Xo2#R=nR=1uz znrAH4uxM6w46Qu`uKb$Ic4EMpe5@{*M2>P;lZ70=c143k#G8nDXu4K6w9s_u&o~96 zt5U#abgQ@)htQ($Ch1te<&np3x6l#n;!P08H=iBNtYonS7B%`PW>5pj88^^E zI}K87or!Y+vndB#Pvy!>ZSvtk61CFF6|TS7wjsllcLM{xPT%&r&X$lq=bNhTeetkN zdRc|QPC$zKc?BSbnW>1hPyw0Utp{PVgPH_RHq<{M99uGne)^}!%BubJ%dZf^9&N(a zfVj8~jbiG44On|D zx|vnGOye6;6V6WMBlF(}oF!DQaw}2TIymNF!;&o|=!oq3;eAQCyzcFjAHLZl#8O_m zAq|I;zL{s?A9*C zMovgN(XA48EA10IUpnU^6@s@YE!4$F3PZUhzm@9hdjZK_ig_rrpefv=3o3G2I-xIm z->|W|PR?wZ1qdjizO_itCR^Fq76CQ~Pknto#WzU)meh~^B{R;?N3UXS8^cSNq;OxY5{DkDOrRV}UM}6rVm18wU}rRZ7K(V@2d+il;xR zqeNOhNztXScsrWQ+^x1tde1ST;8c*i$4>>1OTSWI;yl7NcRdXu$;V<#Ep4iC*?XRn+(=L zERKIgILE+5eJ4zZRQps5;zY&B#|Yw%QLG=$Gwex|q@~xqLrI31QYrzj9{Pms9+A*) zFmoCLxcR)^ej6<5qBCj687#D@rxU$r&q%gHXQOXsv)t~M>s^gTUE4YRPyF$Xi%*sc zOi4nFO-%;xS$hVCp)NZ8ZC#^WZNIG1f7ysQF}|!FZ^VACTzr@;bfTJ^%PYF!@8N`V z*WNC5$vG&zcCRQXXIg-JDP(g*vs$nt45_=WabCs(h~SQrQe3(3wa&{(v0u=OHz1&; z^YS&d`GX3W-Ee95@>dPlWVR)hX~k|ihR$r_e{Yo98x$8R5@xxCja=Wu+75#owwQ|O zYe7f+seMQPUiGlN%joEvSPW0ST!16YRo;dFpO@!yDB1PUJX}H#{rs8Q3s;v^c}?`$ zOvt=fDAVEwB@YSlgM^sh#<)f59aQ3L!q)Iv!kSQnGZ44BE!yWDLL-w!NPRtVs6q|> zMzvE{s!`voGW)Msy0p-)e^#`MP4p#&156UbY`>1dw@1lVKZCpCye%jQQ+27_1;wG1 z-Dd_QjDgi--PGN7-rG(pt>pWNF5opV&-lvB%3g=B{V(`Do$uJ4siqm!_ zAe|tgzD_lLwufRTdD+~=hG8T5u*~NAg-yT|@TBZiD1lM(S&^1RJ*W9~e@sBDAoN@D zGIMsZUHQ^nH0zzA-XA%|YUu2y z4viAl2ngLk2z&D=Rq@%IFBxmO#P=s){#Mu3)%6J0qu6U51_K6zA3l6EkiNo(0m93D zzS=%V?IBPX`B?HfSR7c`r!_bOSgKWOkZ#y3lH38duqvV5p>+lpLi#%A7itie)*~lu zKUs))M!@fqB0F?fvLjA~QjW8J)+YmdZe)$no(mbowU`!0m}}UnW3TsKz`IHxjYsjj z7ytCfm=*GUha4{zu**od6f*zf>0gvZl4xt9#-KYHPXORAiB;2!UYTf3hoA;d*gBn_ zr%QTy3dP_}Tqfr(2tW_U%|Cn=;j1*R@P|Z~I}*T*uyMsZN?JG-wi^JpgYFJ14r>9W zk0Mv7BGn3&iA>gu-HWv*KKZ=apVyagUQ^J5SrG=p#h_6c*I2lP{iAS88{)JK4XX;@ z^BCB|#=y&PfwT%=HH$tV==!T6XukNV{LcvIEC$FVlWdC{Y>1hcVQp=Ke ztU@Rw-fG4IgYCn_X{E+Gb&e(+Fz$9XM^baW^a1+W+8soef$-t6o zhCWYuV`AqEynC8q>u62k=P#~6Dbea<(6YF!ucj30Olo)n-|iNWt6Dx}Q|qni-$xnTJ|5`jY>j(+b>m(C)!uZdK?1-e_;BaDCU zj1@IpR(bQ)bSUNdH)8vpxt7%r+xXO`76^lD++xI?*RUfAdrV{>yXrY{NZt8QcnV2S?o>Q55^yx z1~Me|)nlce+}U3Ur|gq#!|LsyZMwERXbmWrsp*9r7iW#AulT+}^n+wDP-~}e0Uj(E zLW$N2W)5w89jpW5vhYER2SrQsa7~ow(8(iZm+e@6qBPQ}OlqZjAeIhGwE833v#PWT zUzN_*UB!2-5~6i1o^B8Dsi5kux38-~Xc^-U^O1FlgMYz{U@X8d&1I($@6VCMaCP}H zLcHfn0nkZHu=>?hjK{_7QD$Y|KPR`!sJe5=u}cl%rT78^*9z!5u%l$_16DQ_1<}`t z^Nk)>NjA}5j7=mIrh9TziYe}7QT$KNIZ1{D3cai5O2$Zyc!3+t+t_<+W_!n{(nRWy z&pRsfugvc^F{g-upkq`#x%wpTA2}wIkjQNJ^68rJSu^95uPXED;@g!$PyO?tS~=gR zeHlCN^SXZ*)HQ2ja5IG44I5SE|NKcJA|ZDotH4wJy%?F`r}2BaPrfaGX2EY>3F^9_ z3F=o~EMgIm*Mwe2LgkhOX^0$zh(RywphMb_54+Q|q7AFhLKj#foTIF22h$bc)ZhcY}IhO*FT6X!nMspuD^?TJSYGN@3m)Q{I9v3i3@eC zjPIp_T?CMHs>i91<38Z~|C|EKMCm)kqarPf`*YCBPU?LH{X}Xo22^!AHX`@}*AdnR zwaNjXr8ULy$tATW1}or_aRzW8{82KxUTSU8KNMo6 z9}uvL50t{_B-)Bz&POe$VYyMb?}6h3M|aWUL`?xi@k!T{Hl7%M5_z?iV}O|i(FIbAitI_eO047q{b zio7Pz6xx&2OdzOwjPx>;kX@T5MYra&{F61h_gmph=Uh@4t!bQeyw>eR)=UDP0_Osz zw1VUjD5IXl_)~l;@)g2FN(gQj9@ocWK!oN-Ql;p2+aD6}4qkySd7>m=xwzgBTZ5)T zwbYrKN|?Z;qqv9wP}>nGW$q~a zhN1BL+%095yK+a9&-MV}TqNkY9!#RjzOqPZ)g+eJ8QFHVF|2a2*C37ixYci56ru6X zFPw=@amvTPH>gec`1x1T9kZ#VaTy;oINR=7-~O_>GMUJSokq5DBSF^Ug>Puf*5SH+ zQSB7G%-vZP@X|bH*-6A|-&y@BPYBdW6)L!+WQW~b8HGC|lbO-{$AN0h*hP-juQ@>S z_t+>%%RUIgCFlZZJHv|}hYYK`t#+MS4{UHBj@%}O1UgT5KF<7->f7Ef!ey{a70#Y? z-Tyc3Xsu1Ja?ypqPe^(elRhH=kXk9)19;9=eC+70i(u~bsPn_c@b@{zDP5V9=dLHlSHbtmU1fEx3UYf54TbQ17F$bDlvhW+P- z0NYBB8B6{C3?V!*c2ZJ)0;)s;WBy0&^o(r}oh4o_^bJ-D@ZwKa+jn(6)~H85cRk^d z40$-`@k?g{CjeV*gZ7F7z%_VVyi7awb6!Q_gWr6ah47S1RK^QU;X9Wu--5@dx2}ti z(oifPd04o{N^{xl~c%Uzs^_#Q9X~7nZ&GKNa z6kni2B^gYmYQFvI2cUm28w}CBxBKQYM$fd(o~M&3^jv*q44@Hyw0zX-E-)v0+UtPH z>*sQvE!A}aPT1#5v*oZ~V1>Li4hTKBC(^GGC#5&|S*i;HCLOq#PKUYx(qEA@Mqw{8 zp(h7_x^_@l*}SLSu$B$$8oCIyZpbXfyeF|O`A{6FZxjuD&XPog<^cEft0iD-Ql#NR zknwH9%|xhLIjry;16bi+w<+viR{wuwz*LF`7-f5E>!kxtZ48!1lw=jmoaxxh?JEmee^--vlgicI|)gB;D$Kt6_+)A zkuCNkaW}cFFxdyI1uBLDRrl^IT!@fyTXf%-X(LF@LE_VF!#JAi*lA|4l)|kYV|XRTzx)&yCO8xB{NuPXKVU)|T}Igx#fE6u|@n=!;xJ zHQ1lfYO49f@24ela`CaE26uhQ=#P-xS ziH9^dD#{$fAi~A@Th19mfp~{O%-6Qfv@kEWJGuoMW@ptKSG1g&HLT}Y!34Ju#SwCp z|M>Q&G#2nyr?>9=RB4}|4#bVNxvlvGwO7}+u^o~Riu&|DYyi7i_~zGdKmN(DKp-bL zm4(0QNUdi<$-3D;t%HwaHib1WUDy$_FGg8OC>trUF$Z6RF;xocMvB5sjDv5pFjF)y z`Ui-sQASaQ`(+tR4{D*z(`@vmFdCnZ+Tp3cl!y`!WQ6J^S_#n1a6(qnCidU5PFxT= z-D9WwwyoWCadC|q4MF0W?QnO3;a#avL1^VhjD#tl6Y=_jLL^DB4`Z5LePU1%#+@N` z8%f_%PFg({SIl%Nb(Ybzh9WG!-$M%14G!$sJ|5B^lP{2H5x4egXT~&OV@p$dh_T;( z_0~m7gfB<6$@xW8#_V9EFR_vgwz2er*R|(Pwb2ky7 zRPXx6mG8+H+4v#VuxGn@?H+}XiJ(NsPhzq$k7-OOSmXs*PTGm>K-g%gdr>k3H#6UN zutK%FRb|C>pQvOrqL_j%F>9z&8yvn$;}mbv9%WFzQGcHW#Qr;aI56-z~goX@ST zjNnh>u=1}d#|m3giGG3Y2;eQV{qaa1{6Tsh1>+F!NJHex31e^15DIj&*R+uF+2chfA7EJz8#SC;euVh-05pc4*FQt| z`52zKxJ^Heu#tNgy=3+q8)AX@-WQ%(_8fhFDIz3L&xtS4PmnA1u`IikvGSIR*;p5~ zq0}dKHm6_p5x{`?`sjBoJJDC&@3F`PYi}p&hHWni} z$G_7uaMtjYTgL1ATb6f5&c0d(&{kC2dQtl6bA;PJ2IhCvyrD##zk+)2s^U zvQ0*b28}Rx7fe8Nc&8o~vt}SC%Ei0R1oG(-kr{P5&6hxX!8T2b8|f%%0T5H;F^I|n z7@Zcxt8(<^Z!_*B#eXc>te+T&1WiZX;HI0eU!m}2vsI#aQa!&M%jL7(tUmN6w#Y>*Uk(I7R;tOBgxzM z?m9UZDkF^iWT7jd9>Ft!Wy^c$qWb~AXHMA<s8$uW%LeaWNS2wqhwe2A@h57TFrpJ81AI8L|au+s0+HiURk3 zCVrPl8q<$rItKCFTbeu}er-ikaG@OI`cew^_7kM`AsL{0FI|Ua9yKlhjW#KN3V|` zmQnUWIV(&TZ>T{6-2j(L#M*WWc&bCZtKOWuN0=-tYQ2BpL(uZ9_}an!}+@Su7ZoU?bQD{YL8^17m>n(Xlj>8y^E0NkSKqR){-4)=K~`zuwRIb70_ses zVK1k!r@k#%t!e8QnXv2oO@*8<;H$2T#_9k68c$Ld@^Y1sBbK@il$EevW*N`(mfqD-1z=>YmAx zgtG`se@AYHE{sF4T&FI}p@9HN>9026%_Oa)c9rQ^*gGe+hFS!|((6UyoGK*yo`M=Ozr z4iOCrh|e=g75xUU^QvYs6?*J^{-Ly>9R)ZvR}fx_sP9W=|!R;K;p)x|8F*~o+g zbQMQKxyM8n{CzEd(}x4qvq5S4L4wsme9Us*{7Lb^wuH$ZTbNY9L9qq&JKYO(wr}9k zlw$#Y{ahWPmWEK^Yok{9^;=H3+&$*ROHH=klHSrQaSPQjW=LE0<6o1!*G2>VVf~CJ(>c9kNBQ~R= z`h>kTRshuWCp2em2ccg;uSm4WhVfk4 zdH0MGUK>j#SXoF!u^UfUkc#?Y+6;43o}9$3um4kI0gA2S+Pe!aICfJJfd?wJD-#*# z`hSne7Go}|R~(_p3QJM9Iob64o~zClsg6}so_l#m;UfHulL^Ik)Z5`8rt8L26B@zZ z+sPYwxzJO}$>{qCJ$ZKv$d_|sW*F1*77(%A8OceYzJxt%sg>DIrTMB|cn0F>);*>@ zFOpH-bu&Ub+}IBK!^z^3=vqDYpuL{1}(G115^&M#m=wu?E8j)wu}5COV%9*SaNz}^X)>_MaPZBmCxSXi5%x{g={u?}xJ3r1}Gl%8!6)+lLU2CG*BmdGk z5s!lJ8yN>mIpTy>btQ)n^-HBKL`&bc0Zx0{DgD#!Vmq5lvLE8wF!RL4+0lzV@_P~>av>+NJMfgg?yYRF+1{A_i;}f_zZjHMwhT8wd>#%4FxOY#r@z(6c0d zdrVf0Kg-HS6;c4=07O$3q%{rvPQ-W2JO#@qpPMWR`4_w>r7Awy1y)!<`RWi!e`C7J zM#|yzWbuY`K_I{?Q~dEy2L)W89wv6R@Pk675Z(yy*=ef&uh&>OmHeED>$D(d)roqj zMhDU0`?1||ANWa#bBCJaeVfkC?f`?5UcDTfzHF^>MI)BaA7_;2)l5hU`>&ls@@D9S z-6MGsT)-H8^T}&rt>cQ3onPWje=j1Hg_UwoF#&5d^GU7WP`p7OU%tWpH>U%%D=Df_ zVmDm+>EUrmO4zBzRMp8QPjKnC5Rk87I_C`xBDT5kV6d)GTLh!H*4Z+sb{Ozn|eoB)4>x~#niop@J%F#N13s)RcZ;F zhVIvhjO24;#N%N!I}{!LoL(YL69FW4@ye`XXyGR4ZB zR%G7UG{qxY=QLgh^DgJSuDreqfiS(pz0e4JM0$elB{&B>(+Z+~zCL7TI`o02)TcZY z+ye6}A>?#v@2$dV$LYzZ&*95Dl=A}5n$ALwDhTDH`SAErm~!!Fj+Tfd<>yDryBLAB z#Vp%zR>Q$Zwc(R6r4=GI{o&3h`sYV^^+Y5#>v9p)n)(vk__LPY8Cubu7QKhHC|KaV zjH#)q^qsBNsjYf)s>+1M8C0V;f0lbE>6}_fBi>|PZftp4>d-2sa3HEUIMFdNsnO@htr%|6f=b*cGBZ)ssj}luEAhT!u3B+|i@%qM%fTaAlMT zk2*`g7!SpICsd5@o7y83Cr1yF(W90i{p9;t? zJ(RTPpLAoXh3NPAIkrlFP)#`0`iIUey&;)70MGVY{?l7Lfz2_B-G-`)#FR*YppA_{ z-x9}J{;@vVLBB0lo!o>cc*Do@0P&*E(M_IQ7o+Ce?A_+-4Qo?t|K|_g2<2vTSDRj0 z<~{tGGo)GA%_3W*HS;`*63lU_e#XJCgOaRcUrv1m25;_+XK*_)8KD@()!&f}Ph@_I z)YQeup}4Lv9#GrU<{b#05%5tK@XRTZ!q+J1ISjF72=DykOX-8McOROg`L~iA5hAzpw=GQElf*OCt69b(c$XYDK$C$gb-Fe`9^R%h08%%uAh-Q5?VS zmNUKExDw9(aEI!*)lP@ybPO4NOkHFnoN!}f!)xM+qrdM346{IgHCVYn*n&G%?il5H zh&ir^WOzh0LdH(wv;5`ivs+ZO546BpQSS|i%a15y`mF0+nc`R!=5Rlo98r4Pn@_9% zj|IS!AU`F}XFYM{58=7rulolBv-&WWnaMbi?q-etrr?9EjSM5LYg9?HDOWZV|7g|j zI&IaGX%-b)623kcg7Ta^4Vm*_^DhmMQu=>fy>~nnZvQ`?6%me|y+?NT$jaV3DJv(|n?(zQIzuzDIc^*CLoY!@|uIF4<^%aHr z@k2`jys1JC2l=_)gqxFDpwl0?zXvy6{kUZ@ja(Y&N#2$|O)J+^8@fYW{NjM6h$)BA zbXJvc{QDn?_9v=QftPh_NAKE*)?ywnJl9`IAA#;0+NjNumerQWRCwyY;xKbG6973z zLjC8R*MEg6lBk?~w|vAnGg8oVM;7NHw2*l8P9*c=5u)8??}{RZGM$~)u^$qRwku{( zU7KD|HF?UE)>+|1vv=gVs`uPd?~&Y#%`k%1MGH9ZW*7}s4)~40mYt{}Wa^gpym|nh zIud!yHV}{O8%Z?81@f{YrkeEzwGts0EMUfxH=U15Zjy(@=-P@xVxCUist?&T<0}v@h6KxmT#U@n zQd4VK>$6SNU}UimR>GHZD!E&W4&&?DKZX`w(4Bn}h3d*|fuM<5Xxtng&1(M?baccL z^Lk_Jf-qX&gc_TOwp$o}ppW7ZWFJv(;+!Od3%wf*n!X1caO1-yuwXsiKBCP)@&&rX zfrzE`&s+p|0!W@r8MxIw&Vl**VJ-7g_1v4;03|v2=#C5t_3QfAdeHftd!mCe-8%Q4 z)_9>lq9r=}p%p+G;zsO;W~l`+)v|?$txpv&hDC3AXTP-XuHdcIA~6pvF3mL7ttLr$ zxXNRidekP{ZLMu1;D0n>L(UD>7s$3DsJ*{EPdeW=QNnQp`|W^aK7-8V8iD0MnP-dV ziVOaGN}pDdK0htQI?p9Qxkju%`JlWIXEC=g*_Iw?nJa|BfR;n2psVXoopYjpU8Nt= z3-I!(`T53=f3nT7w~YdXdMN}uUTysm5lVhTIXo}$?514A0x0JV?!>)6t6(Zj4D0yS z(ZCd|2^sbmE9=AA7}y6NM3*xr-)nN2j4?Aa+p0&had*^I>D?3%;4orZeiM9$`fKQp zEd=|<%X^jo(Yvgyo9@o;)-R9nElM2n5|tsEkC2JSe9BWnkkD&M*UKnu6-|&QiL}hj z`sSqu@BQ{D{QpQmJ6TsnZ1=*=(;5m&1s{9%mQ`U}D@cq+a5*-;*JLWY@5K-@O;OO+ zm9^_=iqm&9PK}VP&=X24WSqz>4+MA}g%WE(`zn zQ|YL_KfUPu-_PKI@hO-kv623vCnzFVA|$4qh-s_%Cj5Pu92dj_eMA# zfd>BJn{dGbCbZ78f&3=U&Ku}T{Dowvm!g7MohIv-JNJA%ynR|+byn@H6gg}o3(@xg zg98d6tow~dcmVOZ*OG15z)6^>Zx|SXh7&_+6eoOxf@5jbGBtQdhqJYBaqhycXs`wx zYnFv#Ozt=iFh3m#BS|O?;i}dQbab02$W;SU`F;DXLdZA)2(O_?55!jP%3%ncq0WZL zpZIN}%^&TwQ92ErO5aemW)rfZ!Ei<&d zYa8ivZL<8_Q6uzYxw^Dp>Rz#7N%m`S2f>{qE%=ik3R8v)8@zN-vsCP=TS$v$@vdFm5GudpgUWEq z7;_d2(E>@)o$Xlvoo@+jp(hU&ej;f4B(w=luOmZ3gilZ9L&JVdRUGU5nH-hR%?n(+ z`RaAG@uy$?MoPyPf9uu1W2Px>)cp-R%u#-4>|{;9yXX{e^I2|FJ5wQY9(z9>TDxgs zX0iHN{+%3@>*6wf$vnUSu9M;U;1)lB?8HQl*t32R_+Q#7e(}NbF%v#oIb__&V|56W zxErwtA9O$5vG-r_l{)cP7iiH>)e&3P@oTClJ{DJD;BLQyP*Uu(d>Q@Vy;|UE9%wD# z9^}1NSiDsrFk?u6m;SwAv?fH`4qgI~4{lQa+DuSsudgbT!A?Ds$3g0+#Fk~}z!m9T zcxb4AkzMv}&VH*&ZBo7C{p-B5+TmuS8sY7*L=*DKIlTSzBt*&x@G}E6`CuA19t(0m7l^yP?qp_WIUQWlW_rW*zB!2x-F%!fs3JUvH?X8K~n&tA7M0 z^&_eZ)?a6tdh-yq&4$I_{N{hfp5)zr_wOUX_*5t{MIK{?Rm- z>x@hcJud%G(4_*U+qmq57SC|QVz-&vl1U@Qgv9`K0(WEy9mN%`)4oh2IgY`reh(UU z%%qvT@nt_>{xM7b`Zy5DOOt2A@32a}b}ZLp!;mH%7X&Wm{U$VgMtj}6&4UfV8fqwF z!ML-Kt#~`PMTY0g@X<*0En>u1aYOr0vVL;_qQ!3{C`o&Kzah(w_;Z0@m$2VE#l+)~ zaf3+KBbH)QXpw3}bBe=T_$?qK{Z+w>QIipmDCA}y8pnM>+!vC=>}ak3@|I#@>|?aB zdC!k|Y!NytuE85yRrD6zYObJtJ?d%UO2^<*2((VL*R80q@(v<$7%BVvBUlUPA=tSH z!9@R9NVmqbPYnTdbQT2}N`40IWox97W%;!UnEb;Y+8@Dp7}R7`{>*4^@6>hD?~G>N zdTAuTo*I~x8}<}0BP#mr9gERIov>61^CVMdm-e9+98ke$uiWNe0_tXG&aKbK=Zg-fLID zT6$QL#)~r9eHy*}%Ru3*Ma9hIPQf>**jtW!Ce-P`g55!$|9NrWXzzNb=yZ>W-@FI* zSy=>Z(Id8b>^bb{9k3~I(E3{56f#{g4}V2yyE-Js8Tr7TbhSe`JM&p0*(+~F7Qsrv zJA1FQNP46M^Xp0;BojJG;LcXz42=X3_vjlCZ0?(*LhkI`IlN*9jk1>2UjNLuU)QHO zFr3wr`Zl`S?dF3I9gAWV8weUDvrQByiF+zW{?3SV14LgaQpj4rS?83omJ`$L7oHMs zP*;NTE7TV&@1y3Do05c+Q+`P0lCxYTRSC7by#3~BF-`ZNn~NBanh*KT>hCGXW>Ki7 za%P`?5T}e@%y%h;4}Kk=dhp`L#*fYI@qQ8(5ho=(WC301J z%vr+S)2PIiaOS3%m<6vB_e6Q9?{(chGS2p8$%Yg1LidjvT2sCTDFwOM4k|oa%@ewQ zqVGTLu$sr3(OVc3q2tN#{tED>&&gRbN)yjmx9)BvKE@nk%Awa3>L`9>=CdAmb}C>)GLA>=%5R+doL7KN2oSOMOd*2wU2g6qVhyk_R}B1yz>Jb z*Er0muqcN1)cWi02?7Dw^59U>SPl_do_kjmsup5+++VvAL-yHRsrM-|PV9=Kio@Ya zl{9H8062XohdVVd&>a9sOVv#q|5P4Pis2BR_LpzGU9o5+;2QglLnXOlax5dfRZOK-lH!rVHR= z9WQ8%N{SCQN^f$r8L z%{)Iyig`)&0RQf;?dD?8eu+>myyq-vfjOEZds7$}g*ly6a=cZC+>vDgNgTAkgbQcB zW%(PIYwg)W%=eys@LJc~mw165Wx!=!{+{VHo&TH=ulN1h7bov$9~Q5*RX(MwA8@b$ za0Nb!#>y+%?!yT1_RJhHX~XHbDfd6*^IG-=Q2HaJb@1JT+ss4d7NPjo{=bm)fu5buBJm6&}5ti!k+t zD4&;B>urMyZP9n-!JNs}+i;a&{YL$7v~@u28lN!LQ$qmPa-x4s`nHvSz{s+xf}x9MY>|kSjKomU&o@F#}8cGwq;BeC)x6seozRpl*jjvdq21gS&R2{ zK&OXD{Z1M+%YPWMwP=lHRdV>w{m+nf)xB2jH{L$_ay&XV+sPbr)DdSG_!}`!X#&-i z&wVjV_y?V^Dec1iTkSs+`ELjWBsU~VVhWTVe02E&vpf1-mGS4#0ew>k9mOZO`QgrP z^0}h{FN-*DwpAn?3Yu@>{ze z``rbtGZyn?!Cr8kncmTfJVZc|AwHg|pZ&VPGfuR7n&Y&U5rQ>{BqHMSQ_$Ok15@rx zxJwC#yI!Rppkfff%|1nu`Ig*_up9A$A2_|lSdoAhY4Ym|fA&;Zu(O_~lX079UNcaQ z3nV82VwweOpz+hq-!rvku@_J>)ITu^N@X_b2|8!Tu+J)dZDj!=1M3MFQoi`!7MFIA zy20^d)(rBdEc-NLMxt%4MuHw*yXcOm@OaB!ebjPeKNp|8oEEM)#fZ9CT8vr2M_}Q+ z-*8HyrN%QZBQ)baD3 zHQR9r3B}7iDe8E5$K%iC@xIG9p^pd!=ll?v1+QTdsnGL)eeYka208iuy{A5Re`KX* zXX`UEG9uj1!Oa$*zuJAW#BATD7S_7!k=n{{B^>zctE{GPM4SEPQ6KN+WFGr=%e5Kv zQ1_020AwQT_Qz4>BEP*KG`1PGVT>WaW@Qg=!X`=5MQsH`0~x-d92(?-JoHvB?s0UIZeW79NCNfQ?DF=Hiy`koHU*3}hdd`w zmn~saXFuvlmM=Gl4>;P-}SVR zgU2rigXSKQB0(63kFKg2VOiC|L#nRTJ0E#Cl$T!6ol5b!e>#+`qGdiN*^fZf^j4XH zq|(!dpYw|AOWGZvt>E#qGH?D)E%hX3y{0ywswKA81?WGJ$N4vs>d||~fZ!y_l`9qd z3Ye38|IL@i>EpSSA$Qc&BBZ!4MDp1^*#C=2{%bl5DoBC>LUM#bf4s>f1#6&JnVS0g zDHCbEq8cwx*NI6S5bH-p5yXQaP+Hn*-fSW$L7{}M4@)aR8ch^Ge|V?7J>=h!<)njv1VLRJk#1{s21($-4kfun7GR%q-X+BT|p3qJT{dAb`ws96+;+ zQYGxB2=PN##(>w`qcMiBPU<{1OF0b%Om1?5jB+IrHz>$Ryls6|zF89p8y3FJScYdk z8+-G$5g^&Cb|98~ilSOri>Dbjr1I_f-U)?07|-OG8*-SLo|_>Bj_hF26{$6n=93eL z2@$qf_1#9-Oh1oo>f*F%x=#?~BkC0qH{yQ8YkC)2_p_UE*?Cmuo+LFjo&sWG4DL8y z{qz8K1x(vd+NDyuXu7EJ!}Vd?9RjVq+ix#@=JU5cPqN;zaW8nt6#*L(8-mO$`X4oW zNx!+@6?NWuc`UZykV<2=)z9xyOf#3iY|H0y8)#sUhJ@+{?6)sxv6H3yGUJLK9o*^1 zyR#|Ez=qa45X6^lNNsgv*mwyi+y1kQd2#b(HC{~M)~)d~#ie_#-XexKB;PMeT$bEY z>jD+5oaK5I1Jwm8xYN-dDYz-4JAV&JADQI4=$Fks##&uTemh(?KRdVp-I@1 z`GXR_a5qDI@~8);Tyg-=L!@!b=a)+U z@FTe2)gI+(aXR-JO*85;hT=OqjPJ8>@{Fb28)3QVMOuApMR5T$ZxI02*pJ@lY>!Ey zuy|ST^pJKGSIMdAsl*+~)vlUQq7z9%dVIG8{@m(S!dsJlTAHoPtVS7V6#;eAAL$nP zn=F~X>s(T$#l|?up-60j7?)sPEm5N3Uu3DHqe2+2z|&qlvKoffMHy}+DkL+@;2 z8XbXVqgbn675>>WMkQP7M4+9sU-v!UZkzTM#VaqRyxSV3b%e7ykjObaa;V4La0jE( zs^S>`$cg84^!#$-nP#*BNYcLouhP28)n?ezgpI7eAFp$WSy@nvIyQN3#=|c@Tj?$1 z%-J4Jgr{2`2)kCf7w-=L87WvMSfZ{I+e?xaglR^m6}!W8TRO2nmf}Y`H-itK{T};t zR`z1w(F@o1%k|N9dBL#IMf_J}aYgJ)=Mj0qhr0VMEVDRF>we(m)z$mR@n1`As>mjJ zj5yZS&WqU%u%x7h&l02o@WmgKrN!#*;bKL3(=zyo;KS0@sFB`m;JI z*7qnM=%*C0sQd2;#DuGb3-#-=4B|WOgul6*bZW-aQs7cb>~yKX&w{{$nHJs zMYIo|kLs%SLA(>=Sc@W&LXkUkS+}{>Mi=*`s|;lV0uLOE-~g?iS)%m*8WXp@Xc!& zM#PH3f89~oJD(AXU9;0``hPtl;i&QS`VGBH(oZPcu)^jnckmpvJoVw=e5ehx6x<`c2?wjj%?}nwo*g_5A*A| zGY2^g_jFVZ8-VUZWHnrLG)&0(F#Z|)^n?u*`ekT4V-%S% zIq2Ve;YSlMZcz9(%jEU-S5t2t6473r93_^C!|HS|K9LSy0gI-~URU}=4Wc%P4syo@+-<|aU-$tAm2`B_O~1p0yb2N zUkSHKi=#e~kJA(xmr02UMG{`)ijN8gt zff_z#B4nS()&?6H_CI}+T1DX13M_ZuW1I7a-5s?y6a&hT_OBrq$1@13v#hMF#r09+ zu!s>B{}`-)5>j30ZnzHo!f&o4q}BiQCpxLD$v)`8`DG88*c1)VK@XBWEGO{wo{?qu z^jvE4bLpNMJdW;5zdqKZUP=|%o>vQ=mBfIlsj1owDv11Rh)1oB&-YO%x<5`K!gpO| zl5?~h{3AHt$$GFEzLV0e#vxTM-{ig6V1zyZM!j-(V{~jH!|=WHE+op61R0|0pj1y~ zb>l;Lc)gP6vTA#5aWU`Y1CNp0)g7v}7LhaS4L$3T8mt8sdMh)$+L_r?BjRfFVufNP zn%b(I`E@JCfTpVR)Tc??&n9t?`_5|H)>@s*NKZ~h=xTzB;lRAHgHS>BZI=D?fL=W^-QX_(MP82z=+NSiGg!75Wl z^YFPWZUNNW#4|P>u`P->uU$Wo^k~5QbXAEpsFOkC!nSmdT>*`n9HQNNuj}{3UYA*j zNKF)|M>PDN$sOakYqp! zaIpmJ7xDUxa^p&&42ea*Jp603LVX7Oto{Kk#~dL7BVxsFiS6kMpW@#~d+#5)bT-Ir zj77zb#v{MjtWRD4^9irD;eUuU`iLi72ir$sliFgT8uhoRQh!die?`rCoOuedoIL-D z`-~Pkah(Ob-RF7j?v`z&gkOv7hg~nk2b=isE50 zp>dLG3TBN=(7xPLQPQBK})>R+N z*B)Mf{SGpyH#HfUHWL3z2etCo*HVm2i;H=f#2v02@603t1cM893My)KtCz$|*p*by zULcIHWj#H29P3q0P2c!GFbFOyEj81$AbKqNwK#`We(`KAL2^XyW#&N`kny_AeVRky zcjV`{0{zpcAgueJpGfv~^&b~GivG3VWt$K7!*Hz-;en-7=Kv=-$Ihq%T9^u*pOT`k z=#MGh-F;Tyxi|l4k@Z%xU`tw1m04{oM#0p-D`|7>it@&{ng^3ZX+kb@l9Nxd-xh*1 zZUny;?l{@8$LRroZP>aaQ)n=g?di$7RN;k=*zA`yO5q#Jw*rrzs{PLJ6l;YId1zns z{GqlI8K`%e(sbUtIlOj~o|)79O`*Idi(a2g1}-_0CNro=f{DYn zFDfm95745!(mTvJY)+i3mqwT7h+(~bFCoSC!S9ar z=y1Rr$gXsmkq~c;eb;^D0+N723Lod1w#xozFr8(-@_7liv|lA5t z8!(PybVpJg2|h(1EL>c#hheF*{wnlmbC*}Z>v!;F|rXF}E#L;HXK zhFU0pi>L6ZV_jBu<5Qjo0me1xd&+M(+)e1j%|JGkZ`2bLk6Hcv?w-3|vYv3Piv|Sc+}`FO->5wlPNC1| z{+-vZ3EfdysaJfx4b-D9USOsgX>74ye5aZ$RrHkkD%c*1SiMsUl357BE*x4_>OYf= z8wE)PuYpZVC30#+bfokbf1Yu+q%1Q?vx&NP^rNco+k^bGxfkqZ23cmt2osMIt`U@qXGHL%x{y=V;%suKjkdTx2#e zx-as&{0X+NPhX(Y$a1JFf}s6u#FEIyx(o1W@G*?@C z4uZYWTCpe_Cj=YZ=!@wQnHnijR%cHW+r;JR*qHjGvR`rJ$j1+E(7pmH1c0FQo-WzI z8yS0QSV1_7ff!6Pup1%*-T#7XIeNFC6S=IAXh(G$S5+@nbMR0YTPVoO8&mJonNDv( zXA-EU;H#>1r7W;q>(Q^UM>{ioYDWw+Tm9eO44+84(Y{TH(X22pyvuzG3a&Mc+O=!0 z=(%fOrs_g6N^JzGiNRahy8)01i8x$p7`ydKI4)m5i15Fo6-FYZE$6w)o>#c1=)L`Y zDWOoP)D9g+?1BCM&r6Ozq6j~N*{Ch=(EHc+tN6r{R`rn*d&DXEUrG=%5I1ep)U z62KNlJTIaFJ>*7wwaa%=jVjIFa2nyHe%*^Cqk_ji+brjt*03RVXZ{yVsheC)eVmNfjxdxlX0?DYa%lQ_+yhGSMDovS2d#gLg zPF&-7it9-ZcZ2+WWfI(4z-k=-56i1Zy9+7qfqQ&RsK?^Ao(v0^& z*zvTL5BSrn)VAp5_w5%tUS5!$75lL7q146V_7Q;Oryh*%Wz5(GCG;n;FKsQj{jM#N zy}XsH9D0h@+V_ATBxU5QkD{!y+q&=x=_%m6tOsa*qa2;TowQ03L}iRQAtSS&+>o__ z#-MS}W6Y&2SF5kDk1JY@PZD)Hk(=t}sMlUe%ZFY6Wd|^eJLTP$ysg6@D_YAGbCbP- z+n;++^hPt->EtTP)@JpWC~>E#;e=aergRPKf$yd9 z%KbLKsuel^w?OrXv-V7rlGza}| z){4oIa}fhwuzo*d;grNPLJRG!fsGP5j9?qF4;{sWArr-yuPANzRHc81r-(R;mr-c? zzWsaA_+>nvjAok57oXV^cuQJP5S)wt0DS5pdFeQexCd zYM5xozh)sUPo;I{zq7FPcNWI@kCiK{s|#D7pfoSciBDu9e*6Dygbq{(4+!%*x8?6& zE_nS|_^9;tVoR%22~YwD^X4;ctQq#N>pjD5BUyBn1sn^!ib@e*Ya490ue$pR2V{6Y_BeR#?YYxy7_=4|yx`8k1GY)ZSy~ ziFdYpUOZ)dlWE5GY72ANu@UQxr9k%N-VgKzRg0sL^R*0vQ9c81;Ouf4sDtbKW7`76?m^#}m( zwCwEr3k)xMcc_szvn`;43A^Ag1sJ&dpPw;4U+S=}<*US3pxD;oHfGh`q)?e@wOKp} zxLtT>1=!)=#3PJrP2k(5*;@!2KamMH;Jt9E-k!2^zQJC*}*cx$j(#>EatRtPC(me)v<7lpdhnQxC2?1sXQ36 z30L4bZCP`o7h>^L9}deH?o6uo>(RiQc}`l#%+&N zu`B;0_W-jJM7R;AV4O+L&3)&lclSkT(Cxc&W3SDT3tuFkd?~1Zgq#2eBI>os#UIKv z``3d&7sZzOjp743ueC%mx)6QC^rCVl9=*u3XjRP6WlPUqXe=h~iwe?&$fb+Q@c~ic zT!eTr6_1++#7+O(MCfQJj6<${dEiJ>5^8^q+S{qR#jhY8Zsm`%@?_Bn@FBo(U`wd@)so6vOmtH7q1_2>tr81mlHFM!TQ1 z;ft0#x~rus6J*pNbFz&b2~9czKZL+;L5D0#;pU9o)~SVlyCa17S^vxV;@68|4y0eUU-?^p6a9d;}!(>nJF?kDzIuQVYqWW{ei%C zkJBr%%_Dr@T`BE#TpMq^yV3>y{_o7xj9cekE6N({bs}~I4b9n}1jFm+zW@_f_%%CM zAu>zS*-S%^hlr4{3u>f;*H9#B6LLY+05@Z(Wkd?!rA-L8Lyz3VrKk zZ*T9-q9|Q)N}YC7a#{0zY(lKec+W9epBs0LSm2bY{<6$H!fvX?xQ8#d%ko3H=uKZU zO2z~pZLNgpsHx!@zIZ{(DyX1gW!UyMhFm!(Gf&zryaW3YQx9#-H z(cCc*(>=lAvLTzT6^IQJ;aTY zi6ln$ijUqAej=w+&bf-1*+?vx1VXGMrYve{mi=p&HXAY;t9J*Per9Q;+4UxQ1c)^D z;K^EC1T}-F4tS58-JWZ2`i~F;r%_M=BDCyI)9o@ZS^JveN}M~x##Ja6|3}&@jIT|E zpSMcg;<#qHdN26Y{e1~*8y$rd`n7m>gZHLUJ&`$n2cxmS2D@PA4ar##T>)2swUDqj z%od39Fdj>@=;BN&oXUxSWwL_9o$EU6tc53YUROLpZ_y?u5p9N%188|O zI=8bpk#hV|b3*_xT`I#wJ@g4eDNMe$*DRU*YW-vbI`gd15}}mhAVemqVG(L?Xwl(f ze{@&F$V2n!lc?a*14RewG|C2fN}~sS&ICIe>$11^JEkZ+Exb$|3L?;Rs$+cf8B$^fnY0%e@>!AMktRN$#yX;rAWImB;hOOW1!)$N@ibF?hANz3< z6GyRpW^Hx|JH@+S#OaG74lWhsO|XrHKYPOOV*94MRez}@{NT!7PT#&qXueo(dz4+yMz9OCmt1!2dDFt( z{pM2!qiGmvT*1RPu_-@&t)2u^q;I}b{sRHtLjy7Ktd9n)*izHeUAVm&H_Fw3up6I* zo*)>Aq&{spw~dY+TljmL5tOlb=Jw7y=Zgr^IqeyE5UiwgN5Uq4XU)rm6%iwx)4D^*}|q_Pnh#es_Y z%#Uv$nWZ^>6tlbu-lsPsZ(Qj%3~CEoi1GNiKb6f*gqHrV zjAT7HFSfOQl=rS2$GCh;=nx=`PSKa`N$C?xNnhUAr}R%0D~igsT>1Z~n5ZI3Nr^>U zHrS;+#g=!EqqoOqmy-f0<*%s>!l!nJpOq9Wlxqo4iG zh2QcV!^T6X(N}peLf>vOMnJB1l%n7R}XA8dtSiFnn0e_;RP0iH`t=#!SR%4J526%df^;XA&||SXH`y&<{4A zb!Vba(Ws9gU5#PQAQ}etE%2*f4os3Dag|e3mHy1#ksT+_7fCIZsBa!=@mn(q4_fpJ zC_5x)UU_E7`wxrRwcXow?>+6N)vpzNxN4VS#{m@e`c2o|g#wY5H?r=Q41C{KhLrnI z>=n-yZ{13|W7a2rbYGUmMnxRJRrLW5ha4@=8w*k_jz!``&`$PkqFtZI4@ZuOODBW& z+xgi>qvXlH{-Bsd&@rZ|@VE;6*;yz$JoW7tg{C0VeA-0y1@#+2^DW>`5Oi02{`5(YwRFG;Dc)=~jas#K1A;Ahar`Sz6XISI?m5f@?W0n&cuHdhqJH{+ zg-LA)OGEb^tP9%Hhe(M(&f$F|CGJRng1-NbHo>g|JomWd`gJT$o@#gwn;iG*>TpZX zOxViB-gfp{)n#~@nmn+qL()W%kK5@eb)qJ;lAF4)af~MLHB|_7|JPcyyd_`uTj#$d zW*FpYrRVbZYe|zBr593{g+F~3pKqRx;>{~H{-5w}xm;uU^R=dl4%@S>)y~(2XZt7Z z_H0Fe5A;uiVi^H6r)2W}#517Cm`H_#L?6?uzEZH<-$eRlVu2hw&-{@k>L@Wn zXkC)tM&ct3ua`Ox9VK_mK8-7_vJl7tHuX`7@Cw~C)Jvw}D%fs}tf9Rjrv6rd+J z=&aB*n<~DTsHq!qApybGM7r%L{iP-Xfp^^HaBc7iv!PqgIK)q4 z2{>x0?Z+8TlG^oEj%D6zI(4G=$$JK<)XMYbIkN|)7emGiHIFt`aMsTp&!f>UKM0ac z-87@sL9N3{yO1V?hTo_dJSJesva+)H?;*`9j4^RSz9}3EJ0GIH!)PpgSV2o~@$C;J zcP!JqATCr@;_N;>=c8QL!LFeeFiFNmY|d;k1W)i11bxFBIE6a>CSOA$-dRA+*g@QV zqd%xi2ZN^R?_wmO?SF#=AF{?RF;qTOz6$R`Ky7q@aD-gx&f2Og8#8 zo~p-s2TNybOWMGv*SR@{Y5otXaUzCqNJnv`f2Nq0VKuQ=w_p@kboK22x9H0WLIVD@ zls#P!vno7n5(>G!>ks1dG5TD+F8<|5!6UvL5BRWaM=Vv}Tv*LJS0+KEIu ztlaesS6<9@bN;>3vTCEhFzH4LJoExK5jtmt&w*1B==xZ3em*D!0zE_-X zfnLlwU9upoZz=83)<#LDyU9U$bA(u?8VPQ9629ZXgHmg}_$;?n>E^QM{0+!9A8jUQ zlrWUi%uKyjiwt+&`T&H}bb=g3zmAJ$7Xr=j*8PPrLK2=ZElW&jQ9W!_a0#wkP*;cKxaS(R;6@h2Qh3 zCg+U6;jv*hHq2JRkxf>qDnnZhSk4P6M8eHk;W%bnz#hnwjq$R@{^*1^c(r0m?2mR zL2Gc|vu5n#f1}27Fh{D%Y<-}`SVGc(S?=Mc%y;N$`)*F?Epe}cc~{3%uUb~|IDc(g z1%F|znuUb_*QywGloVT!EsbL7oYi#9FS;)E0%|N4^eMPi-Zr0MI2E+yZYtI(=M*ZqG&N&W| zmN!Kn_0d&W!|Tq|WIICS%tU*anJg|yDEPXovq4;t$G<9AXskBSA@q-u;aqOSO#Q90 z2)z*;pLco%Ne>VM%L`rm)fu$+ItMrJaLHbrg|im!-XAu-#k7)6$tBm3uGMN$GSwHS z35gw|yw;=w@{?jRar$BXZq$g2HZ^N;3{Ny?$rY>4VZ4*c>DFbSX&0a*9)TrE7=1Q|*ztk-1XMfB0(v8Su!(P{NVudu4CuFsYaOzmmr4_d?>D^QP;doh zJ93z;DfC=KH|ThM`_-te?(B^|koQOK(pr!MVH-L0rJc*A#L{zExs%+t@6Z>;u(KKt zOcM}%&peDBqi{Di1k#T8urJf{qECv>trIAtz^yoA}se^!)yuHloD9U z=|&7_igUK4>v!yZQd`6V$z2&i!px=T$MWEpfs~248`ieeSa&2((P6AAAr&PR;;#%J z0ERq!3$0e?w@=qn`qYOZgCUD0Vei+M#27m3?AqTsKH8y4j*gB#8dIknKils#=0+wI zXN`^)9~^krU%r!e+XAmthNima4W^b~BZ|6O>)0s$w(Nf1j)E&(B*j!95J&^5WYcjT zx^}$K&pAbL5IjcJNWhnPrU${cq$YnY$#&76rC_3sEloIS7>Tv& zA}`qP$DYm!xAy}jp@_P4PWibKxcTbPZ29b0RX-IlB^H=~`bXAd@@!MI!yOssQ5@Kd zOrhMS3PliiHH~KaU(7SI2gerD^lF3*;IV}r!9lwFgeKeqF6|@ST;?GkkML96Dw;r3 zXb3+!RE6F9E^Tc|%jr^u-XL8Z+xu5XOi+>NfhsZ$qjO~|awD6H%ATC)HLB@@5$iW` zT@?vCm5noI(&=V$;_EMOO1U6-X7*+k&&lvkv2W-oPG(+w408LA8cLfO-;j!;O?jtN z>NpE$VzYB2@G>?dk%D8_^nYZB%QGaY^l4+3MYYt(I02mL^&&L`V~he{HE&Om6VTytD$1BAikQ+Pcv`GckRe;abSR*#yk}f z!s*X)23jX7ruv*s*FDJq8x4tnxt7X&Zr*bU2BM_68G6A0++-DR7Lp7 zjXt85G6p%(pM9Y8ht9$=*^UF;GA{nN^u)&eJ>ml?SE!8%W_KYA316#;RWXtnK2a#T z{G2J{go7N~`vzqogN)6zD3RMHQlLT@!Ng?lM2$1Y@!P5v08zb(o*pS39o-8`dp@#r zS${TL}+8_R9md=0H5gnkmI*%Yad{P zj>>f zzeCjA5$@NoB&bp+m?S|eF{Zg>T^uO&r&yZIkjBdOnkI~cwo!7iJQ)5xR->Zm5V)^4 z!@O7MH*g1RmAzD(ZwBw$M!(#89P1gci@Fz$&T}J{yC_m}b^cS7VkgZaJlx2NomZM4 zhjwaCefNoi8%YC5$3`*5Do^{-s+Kv&Wiz?8qJsp^TTe$-^-75*r9UvEk9hZseqn7v zq&eE?XV3xJA)Uesy&`5<5U8K3jKr2T5)v{c&tkCMJMAf6Dj|k-X+WAAV<^PJ zldcZj7!t({!fMgRI}RZ0<}F9gXtm~CD#Ut!oQ5mP{oN7Mn@XqT%JC|Hu}Xouf+XdO zOtaWhfzhP)$*_M(mkwhug4jh$g4nXuAMq!Y==$3%s}Q?kr~bp(CNdG83~wI@_;WUQ zvOG19BETvidIVyNc&AP?3Ev&=BQMrX_CoLfAEx4&%I!Hg7XQ$vx)g|<3Mlq|FjHTw zjQurN-a^$sBKF;@lu0{_Yy`$ik<-3=wkxV4P)Qa)DNp+u9GP_J^T^1R!^WIBrFT2g=UOQnV-M2g?qIU>~VU6T+Xr^-}#%jsEVEEVDqR zU&T~fKN$(fYnHiQ-(0VTVMT(7w%IYx>-S=+;+XmU=EC71zQ%z|06a#!3Pf12<2r;l zV#E$Ddq1JSGf+j{oCSN4jbgzMxsC5GM#IbBU2H8RQXccdhGyP=eG3Pr3fG)Ix`F}m zsRVhO2clC#277RyKZOJ?@EF2*v(M3=)7)j_hluOHG1Y!i z+w!zZc)faEu+m{wYm?d^Bn{ZC?p8i|=0jNFi3dgq^d| zL&jeMpnQ5}#8X>tbP}ASP?Qx@F~c)8AmQS9%+by_1%%|L2}k(!?wlf#LZlYGx;772 zLtmT08_R+zU5Ohw<|0Y(7j`Qjf>e$|QsDXBQ#B|ETGe}40x(UaDAeZc@7`L^D(8OF z7Z!AU4p?``r%#LT6V_Kgm0q{ni2KCgH`tm0j0@g{Bs16#oT>qTf~aT9%%Gp98d zF)LP`n-fLqhfCfAjO16Gkws{Q-{lfscNZTCu>rLe%c@$U;mxwp`a{ z#k+#3R2PzC=RH?2S>90i_OJAT>GnM(_KLj8#hHe`vjfZ;!wO=49{&YwSR=%q1RJs6 zKEumZKK1v(By5_g7M%IpV;;CLVx=_Y#2KZDR0?YyR9)>`y!+cl(5kz3|8sd9s3di* z_lgxAHh2Z@l<04rjE9_4P=-M$SX@kMJg)$!Lw~WS1DQ?`A4{g^^D@u6!$BbPr4;o` zCTD7B2B`cO=xA|!M+SrG;MU1Bpd{jgk{=68crtf@Sjeg))4_R;NQc|LTqJi{Zq2eh z29GYm`$VgJ?1-;V5zN3?0?xZ@s+fgu)Uq%=imTU(d*8BRS;+Hf$yfOX7NGV#6J&Qu zu##4y z^u&QVzaE+Y)71iq67)Q3|JxaC$-V88rr4Afxb$vc*p&ZvV0Ze5nTgAKbnKu;+!J*Y zbfn?wYhc&M_|+6B$P^hq#t~%EZ^+bRVz9=bBkLK(m8_TCP?K1pz0LsFCbK=j;{8-` zJ3nu|g?x2YFVx;84YPgz{4-!<@|CQfJ&jiR35wp^uP&90(bb) zRNHG))i+kaCNW8HI(QbsDhC4pZYA^vJURSjT#?t=V(Z!4LF&~}+Tum+QCu09ov@Zq zDO-FLInO-*~#mJAw3d>DA;m^KT)l7!kWUQ5D;SoRf2u*TlL>kBKS1GSYmw{vG zQ2sn>y?%;<_e19yvw1F!Jv=(=D*&cZ}`WbMDYQ} zsdM8&CfFgU8^RRA3$c}I&jxR%%|<_0$ zgaD7q#WNYi6}HvL=z za6ki%n35nAAagJkufeg?oNmJ9L95HR^+|pL5zOT zUBGftj13$V02F6F{7T}v@6q&B1S$Lz(uv8E_D;709daasB*0!U^~rgI{=npf4*S+1(r|ds`u`0drpDd^gxW{ zlPA1>T$v)dv#EGvrO>;7dVyOWeC6cuBLB2`gWf}b6OH2QX5rk|3!TTM&?sNiE&3!Vc0U*JZFgT= zzsM61cgwDSC@(FoiNf>>d5!@`dw%g^G;st*l}yY zj|VR4<$t)OoG?tOty{x4@N=E~ceKgG1FTa*Sy7r2f=tzf01f-Re#o17R7&|t^6)P9ZrBjuP4YHohFWsR0I981er@j0 ziZ<%_0KHSd1TfGSH)7%-SnQKJxlEyOg6~g@S*Xd_-{ulW(aWC5AmjZaVAWBBbDp#w zhte+`abT*Zf`zIz26B9~I#e^|P|aA}Nityo_?tinnL2=@BDnS>VBWz|cva$-Yr&4- zW!dBqT-kr}#BnlVZ!Edy)Bt1=P=)Jo*E_8*Cp@5i3S`N|6VmTNjySJI?X;3H*3@1q zZs902(XS;L(+S&P#+fD)oL{o zSN2S;sI%}+URmgxH2Eme(wCcVjgkz-p|TF>0@45q2)9XuEr_OMF1XD#0Z4JE8MhDY z_GiK9f+S!O&47J}2^yiKM#+=Q44kWN-u+R7&nN}o^1yriJ7~{>uE+lwlYBdAY$)y- z*fe&OyccJJX(_4W%K9cO&Epu{Wh4oL{&-IO$7)9^n^Da95KmpolBt@gQ2%y%v<9r2 zWzOtqUk=F7UeN=a<$Wn?r@Rbh7%VyjfwPwhaU6@{hGeS4SSz&O0-_o@FpXnUaJYi? zouXI75{N?5RNOkz+)@R8)U%yOcwySRi*9y&DJ>}H3;6z8%E>>Q>%>2-v3jgkV5f$haNy%15qLIo`LD!_6_{g~vN3f6=s)?9@F9~Z4=mcLZH#+{2jpsDo5NT^%B zTPt|B)6Tw9^tmJR(yS0tytBi@FvMyl7`2G-9Bz%n2v4trjq%6acaj&EPPS$-_N zS6Lq{eLs93g6yfJ1?F=|n7oNUA)q9gDX~G(}imBvIr_hLiVXJZ@}N?l(JI)2E*`*Z;@O z9ZcVO3NfyK{w1DnA=9q?<-!r@Kl*|Hcjt@Y&dvcyr}Lj|!zRY4OmBvYhIl^^ie7Gh z@0T4LS<_dBhs^q8Nl}|nJc!#L7(8L#!ModwK+zYSX+rPkSJ-M7-2lV5mtjhmjL|-9 z8i?ddVJK1Ti`72NXK0o`l(wgZI-Hzg!D2fWa{^%~?H`p8-jd{9Tm~QR+|GP?P(HDtXN2jiF}R zEaW$oCUB$(V+ePe97f0i3Oq?;YLf(SqiB@jnTaEWQ8dD_ z09CG4MovM#`{OBvuBg@}-a9-7WLD`-em>J;vj_X%eFLeGJ|MF;xf)U(*mNq+>2{R& z&Ls&E?L{SXk%U!?&te!TFKIes{ah1(Qg8qw5-V+*=pqTmRH1E!VLfe_?_G@f$>cUM zG5=|C@9k{Hc}55b1I!6^6xkKjlC>;0uKwYz0A$tCB)c)dWqsq6A8__`SOP(;$FiD* zqdhEj`4cP^fIb@qxZ%++(}17Z58Q%>jow9HT8JUjH)mO^xF4m2C{aw};oItVRCyO; zV8~FG0SOoF`p!1`$#wOQnIg3#85q9pJw{J^FX(r%sO}&&4z+voFOz@WLZk9}5SP?r`|nLRBp(ScH{@g`av(DQo@}0mhSn_2{=4 zRx)|&nv?T4Zef^Jd=#A>Z~p5d4|tjoot3(o{yI%riI!U2;oHs3T?24=$CLiQI4j7j zxwoz13hJ6cB~$b7q=KgE8*W-D)~l{p!@vwDL8>DBmc6&Sf68)OSK4B(hz){#W~l?c z)Blg-%mM>wV7z)gztf7T|2Yf7Et5#JbWu!AUzG!uRWvz}K77 zXVPEB)mR+Cnh9WEkt%;m{d@867vrz3K&T;t)I6MYy3{=>$&^qShXB~r7og%}Ml*pX zL=C7dlgd>tj@Iv5W+0Vr{mSvrIR+b!h<-NUi2!gILor@2txNhuLRH{87kNk^X~cxH zRy_q=YV7evnLtqk-QHT{pvJ7+mbljM>(A!=vgziD7U>bCHg6ulRg9VGFaTa5 z)ACwVhesk0U)OiMR~AZ$qI>h;XaUrS>kP=4S=I!8$TL#vd{)<4iuL zzvG00qPVgpzLI}=`vR!J1j1m^M?bJd=>1gY6LF8?ir)a*WV!+LR@a&%%8yd4%+cO` z9%9iJ@5|5}90n?{#rcqo0~&$w5U2N#LI_gJuF$v&62jWUMW z?!mT2Z$Xo~z~{rwF#UJj$;lq(EiHUs4;}(uR4O@esFuLQQwL7h0wIX}>cg#^2M`%qpXB1#DV_p-YH!C~LlVE+Jtw z^$y_qK>}kAt~xO0R2Kn8cRA~KA#6;r96-2Lrow$zahm1C@hP^2g+Zrwf^yo%B@_|a zSHZca|D-oXH4InwzQvQQ8(^4DS$CRz20IUdySkf0u6=wDyW;(ueubw#-6GM^6>8Rw zHO04oaT#kq{$=w8Plm2kX8BwLvKJsqc{Qk>=3nDD%g)fy>xSc@FkM|uIkJn_oS(YX ze4%!)-_P}DHP|4nR3umlNl<(EEq1TfhCaS0@FSG^aEI9->QUcSQudo^Y~}b>(R~`@ zkCPomol@0jf44VMJYEv5l5u40jq`?6j<%m2Q#>`A^tlKqO(@TgSP4pNA=WHe5j+LA z{+|T7E3nRF-!xTB1ti~0LVjCldd#&s;;x5B`BiSv_Xem9y!b-qx`FoX6L*?r zQUUufq8a|UtdxhhQy6$Vi({pQHVZaHsBJKR>CS+Eg_8nRJ6t&4&{r=>zSg{1{^Shm zl`u_U;j5E4ncU3%krF3*m*9oNuVvs|QVr|2aV@57M>PsvOG znOJ^-F1?ob;r|KcM9`mg1q0>LyJ~!B4r80`j{JGl!%VnwxpE3y`*I-l^+D}vOqQJr zZD|!h&cDha)8xAJg3~p`QRHEH|LaHwYE^XsEkZVlGlswkZ#eTUbe}8slSx++Hfx-o z$n-K5piJj}3h0r5-c*nb`2$XVE4Yiz4L_yUEjqsG#Ky?#0GvLg4|sdT<*!|s50&es zH;>t461hZuEJoi2!pQ5%J#A1SAbJN}_inn9Vo32!PdKv->l2U9j=#iM6k%I|JQAPg zkSe%MZyj;47KZ=QgJ#Pfbq;tV+WY4e?}q;aJ|}$2l+&Fq?H^oE)-?=Se3FG*<=g~v zO^-$}L3=njROs~DDWNp#)ob`U$7rbYr&V+a{2U@!%HE$KAR{2ENFCPKC_wPSO+*>A z$HKrNWF_Rt2NXzy?$7b6SJQH@rk4TRp0!k2{)<3Rd3C=mp2g+-?eLjPG4m^qFY{{3UNZJh=P#qjsRF&=%mv&XwB} zIJqizqWE$O|Czxnu8b!kivj;HzA9KAv7?BH>+F=d&pIRfyZ=SnA^;q^N>}p${Cxe7 zR827x9Go?x3IwD9$pC!9o~FKj3o@GSHtmjSq{2zg)W3JJ72;LZ`84N<(iB)#*8x5( z)qUW_H^i+QCRYAt>iMjvRE(V$kuphQ-N@6)B#3*`<5sJFxpWy$oOG84}4tzJ_alHQbW}`Ahy`WMvUEv`Freqs#Hh!nNXy z?DR5C#^7%5ml5=@q@#{D_tw+x(1_wnyBF>8w~5X8M$sGaB!=N?-D{T@iU6SaUvfBa zu<}U0+N9V_=DD*WwUYDG?ol(53~vE7!gde`+=|pZYl;=KWRvyd^X zg6~P(Z3E~3IY#Py^AYEtr_*$!AIEwI?4Q0)L|@Za)D;0qk-3;lNo+=g$8jSX|hdd$^@KwcjDc*9t+maLE0OCxI~ z_4J$P0CGX<(|&wQK)%#(mb*N^VKP})jEm!20*^WRR=!)5W1hdGk1UQ#@%-j(@xL%o zkt>o9etXAi!Nox(0m5yo0-o` zrJKDJZ9)J-6b+Hn1qEt0iMOpK{jhc-CTuH^ua(O_q{}ui- z3mw_Fy$Vm{xynlM4Vl^%1l?bY!cxlR!P+9n^VbPl)x9epatIAvCReuEACi`C6hwl5 z9cq-^o5<-B(~3skeHd9FCttrq8R}^KygL7TzSEcR;hChz4EK?X-$trTE-%TiF)-X{ zU}}rlOuZo)1PWDfq7dUtXie1lVW-bL%_8d_U%nR>FfuJGPGEQD~{(1vA_v7$z-kL${yrrqc4_T1_ zyBzNh)E~IAEOha;t*G)+GQnR>-#qhcpw#4+Vftdmz<2F?Q>A ztR26EE9eZ>50ajhmC+ryV9A)*Uyr*S{}AD9)>Gn|CMqpRa-u^OOxztJo9zdRHq0f$ zTSD6m2y>AFouS3vU9TYW3R?-YkqrOl zbc<^E6@id%pcEwZ0b3HUDd9HJW#+%!6~clLlgI(OL0XHdrx>bCNgp`ML$?EI>}V{`40IaQ zrH84n$=K1M0QpfUEaen9F@r zX|HyQ68b`~S^nnv+5d>;BdXx!bZ|elIg(kv?9Y0|pmzIlxwS551W7(o>Qvy}|_j|d4EyWm3=9#L));#p>U2a{9H22@?F)aXoX94d2= z#{&{cc;~%wU(CCpAuCVU1M!r+F(xXQyKQvq;$eIhRb^oQ9TCdj0KKS47K3(t7KjH+ z)V1J6j|7!XUX{1B7_Zt@KA?Jk`wHl5n1h&x#9OoZkK`vAFt#&1t5y|U00OkXZuhVc zNHAS2ycniCbuGv(51u5Ll!3D_v`d?bcVTf1xbCkn+gdb?whvx!f>09-N;6sMT#UXP zh|-&RK=ZUL4>BJJ_(1FeaLVz1@maG`aq}cCK-ar>WQ3&lF7{2v=A4~HsHl_q%Ij#n zs2PQ=>hKszVu`gDfTUu$n`P4K{fZ@^%m*w`HMCE|7DE_(*3FAQ0Ucu7Fhwh@>M^tR z@@{nmppuhUQDiUlDt#o5EPygQq%i^qCNh^f&72HvsC^V-A%0IB0_Ixz3ogV(DgkLH zhmTtu#2tX<&{2E=2IO4sI&?b-2_%Fhv@VJ0et}u7liII$;M<6I&gz9)P>CO3jgJ^@ z!BX5?G`TQIXbGC|oyxU!#}1cnIwNI6WgE0Ezlpi$T6Kc|R5;@Jr{9lEIe~fP54&=e zs*WK*AnT-)NPDn6I8+ZrGDk#?rb-6^?Ccp|k)^6ygdOofPmF>1v4G=a*eKhR9P9#$ zUBNoeJ{HzqU05FlQ7r2m(Xp4j;2)siF2bq3inGYz$Xr*;Tqei=>IEyYUWsLC_Sk=G zA1-6S3{LWtYR#Jyz6|D`s6Nj@3-z+5@6Dax&Dz3?&0dVOrmtB?=xAz!Y~7aMh%Wt$ zOglXO1&)r5E7ZC3$G^R+llm~%DD1D8OKO8G^7>`E($h=Bj&CaZZoPm?s=R69dbK^r zm+fD9(?RN0-@^uT@XbK7Si?MhBjEV=cJ|wde{}g0OCHaKuY*w%jFRU}(sb>R$W_v$iZWo}D; zXXH)1Dbv&BYAT`354RQ6;|K*NCUsx6C+0MJ=TNh3vxYpI0#F_#I@U3bt?GNGje=1HsNjvEIvtN_gGZ5v@C3R?I+0km2ToPUTj%sHwTDJ?9xfWD9VZJE~I7^~^EyYae1 z@8-&OPLO&iPQ_U3Qpds;5AlNvwG$Vo{Ilb~v67X7`?lL0U(1e;O)q>x;)^VIbl zCSXHpzf}efe{W`1uG7cco%)woRuiTPuZ`b7O~UbT-$ks1ehuQ!`G=g3L5-`6 z#OdRbuGqw|zw!4tk!U>m%a-uMsH8_neZi#eWCe))zb2j-jVm=bZTvjTV`P5{t{ttv zM{)Q0fHf3ft&CSUSg8gPx*64{(~BIThx7+GpnGo9_z2>qFKKiVp8YdtJpJG@an9ya zQ1lw`G#d}5*Id8*pBO^mfO(MA^o0Cjy4aVJM4RZI>Aaiz@!kHbMXd5#u&4FzHDU(2Zp2gM{H!BNmeNQ#z@D znm|}@5inrV%zw@VGUHPWn7uRTQW@xdCzkhJ#00z|ehK8+3E`f2_(1iqgd4L!o^W{LcjCP=`N@n>=g1Kw)9wO>k3g)rwplI6jwrN%w+erpGxkPJ}B z6^WpshC@qKqsp4?;H{)h(Fg@;ge2yGHI1-ofi@=vN>BqzjZXThEQR)-3UZ_2$`dQW zFK953Jb2Q{7kq8GuCFW~HH@(qQH$ATkd`41?hLxo*#F+)R4j=HFJW06xL5H5tWQ~Z zqo#TrQZJEa{oH_d-wQ2c4jqgP`#`*605!I`$tjMs^+fOslKyp1-oP)=hPs$m;J{m( zslNB7H4<*zOD7>>H-dQkgL;V&X)Z{?DkQc!Pae$8K@)C(Fc;$)t4)ba{}txcM-7`x zJ1UB#LxWUUFEWCwci)RiT&TTLCE0W4)vAcr4>he-eq=}ZH5{)yo@>v1aDQ^mC#u3P z#5=0x{jab=2m=cdwT44hX4$=DIae>bnPs{D{n=Wl?MPDNp19}CW(85Ch;-jx!1n6? zWANVLS)+0V+hmoK_?9>SaY^V+eGh8pyYxzxY4e`FfqbRc6oIhH(dXm|$Jnxqt7;uv z0@G)UK29Z8kMs2}vY+)hkYW7m>rydmVDhM13%fUC zNLX*HiF<8VXnKdBn30q+T`SKqE9*aeh6&+fA)*~;1yzyjGklje45`6*huwpJgT@6S z#R&{%@2gW~grH$h689_7&SPLN4&FAm3E5^=`08=-Mlhxdk0cJYmRKcl51Zij*@~l~ z>_EeMkWNYx{8dJkkmkQ^eeFL~ya!|YL+#6JE`#CILO;d6A!=038!%jP;U?s$@|0iK zbXLD%W9Za%AWKY_l2#)oTGO-q#`Q^s=VgibYmIA3RP8lr`oGcoU4XO%m=8Kl`4>nh z#rAv6Sr9Y$O@%)vuSp@S+<;CAbX|Gs@&u(@{Zh4!@%yO@5*sVi=Wv;o89kAWrt8n4 zCj3ln9o6~;8y3||#A3Ki;K|#{ksWGJ^zCaw{kxv`Afo4Jbm zTiJC>n#kbIMcS*B$dH8D`KL8;sDm0|Egoyc+tnUJ_URGw`VC}tJ{LBpf2k$|w<3>x zI5}=u7yTX!gip>~WEA)X0|x|1r?gZB7;@nRxy4QLMY?0{;k5!*33S1JXl7Vfj7=-l z;%I;XE#KMn`ziAi6z%;836by3QPPUM=*xM>^z5&^p)ZFU3&}uq0^xRq*x($JdDi4P zS`!myv3$RbbgUORNNn3*uxAchQBJaqSuaH>A2mj9;P(e@!L0PnTay^tPM!%}(!EP{ za};p5S0dG0zi5Bwm;n>y_W&XFx^0+A31B(AEVu?WAI(mD7-YVO(#R6a_o)l~A&SWSw3IO$4YKTh}KOupAu0o0gu%HtXv6q?9NA3DI+mbmHEw3+S!?XdNbg zG{1t%w^nLN+$1#JF{;{gCj+^Vjwg05naWF}F1EcB%!M9OXRAp+?ub1lQBK$Whq6p2 z*GM{+6F=kTsg}SSt&IZj?S<+bd2%`a5I)DXrx#%U>ag~jp;e;Pn0zRcJXC1}a78kg zrt{)jjb3h>rr)BBw0Mc{R*nL%UD5XGW>|t|+k(>PQATQYg=3MDlh$BjujQnh&uQPt zTMgp^vhQbJr@N1km)x}w?iK&5*blCnEc0P6IkIye0Lr^-2u2BZjh+aqLnD{<$G1a2 z65Qmc>BH=({i;^=m7x28W}r}K6G*fy{~Qj zE~>sRy{r9~Bo|l*PiTr$bbclE-=BI~fq1j-M}%-sZ(Y-tO?f^Xgi7DJ=5r*#1>nQR zB2+-q#e^^jHWCl+A7N^q=_k|7?``?$Wju|T4Q=a|e0@rVG<0WESJjSoDTHQjJ_N%y z`SFqi4kEit%|X7*7vc7l(KLkSABh%xsoTzYO9!Hy8Te-sr}2;>kekc+Pa1?6^ms3T z2Ddom`VLKrhQ@t?K>9;E4x-pdn_x=_)R6EEi4WCNWTi((mcfKE3!lQ+)6$*Mka+pL zA-*$x?YS*`dIlB#R5c&ztpsq-mzZc@fsc|P&lp{celys7%FGyrJ$qF5nS)dQ3j=W6 z%m#wgpgRVg&w$s9OGh4xmRG2Z4|lS}c60Pxeg2%U4g=MaVzx#_X4Ps=dOaH}=;ei7352(D;_Q+&iO zb56V{FKCu<@2RI!N_Ajf<8mY<4|m5zF82q~i3?DFN_rbg_KB@qFk%R8+3xL#V2_OA z#WB1=NJIu~(w1t+YC9PGxSs4&yh?5JrybzQAD~ELxM=PD6EBN`@)TvBEJdLv5vAnj z_5x?Sq+uTt&O0WHR8td0EL(E-c3$72Chl?8yHbz*Iaaw1D@Ui5Q?poj#2b=)iraXC z{I=X?>rvwa#ARER1pP+AC-?$VIgD;n|9%`S)(x5D2P6=^JU{MszmaEG(pJ&KV5gWO zHa`V@pXvNN8xLSJphX5*iGAjP3aH zw`o&{ugL+ERt~Y}ZDut4p=eM}#&f>x0ib zkLqUz&E*DM-FAMwd@?l+a=k=`()@?!(hM_L@B@yqki%EO zH1q09!N4&@uY@&V2-hsK)PV?U- zzva5hO)>aPfUxVp=yn7?( zA_yzrcyU^v)H&GLFHzg{kyNgY>42#XTo@A%`kA1Z5*>N?abkh>S0VSE!a_w;BU<51 z<<{}r`PL0%d`EFr9}JiU)1l6ii2%}GW@1F>wZ{idAIFwaSW{VVqo2%|O}p7r@_3_I zjv92-45N%(uwQ9!B}-%xjMwKk*m!HhLi6k;GVlBdzpt}Q-qDD-@s&kt25hEDlGXt6 z7Bo$rEL-ILUYr^1bXH%yDX{Zw=%=uCh4MiQg9BD$p z()qn>&%7YSPwW#dMvRXRGhPqthB+oGbPZ;Ok$EN!&&UxT`@JGzl}wC+a(ndO8YeQ~ zt}G#oUXShp|5}^!SIgvVHrM7@qbj6eMfAI5i#p{;74zwqDCo^8Mmn)OO~`e&;Hx`7 zl(y<8Vy9hvINu{YUK|$K1T2+}8VLkxeoiYy77>w1MiHSV-C{bE30Y~r^D=UJKSE8i zMWWTp+cvMN)0p8)A0a;qZCg-d+&u!Jo zHZ=j!>;C$pCtq)!0hYPy=5Jr8FMD*?N}-fSQsJiDNBptoO+TbZY_W^pydrb8 z!oM%^f$jebg{hTDl z;*Wg=aJ|di50zzjvzvcvczzqyxSf(x!|eIk>ObfU%Fw?em!~@CuFo3gKHg_=C0Ja4 zK>Yg)+>nrxOIvwFo)>D1 z&{*@Ft&3`z6$)t8s`Qy*Huzn`JQCE@w}fWKY~FEZn|`tz`=Ng3Rew7CYOr#$%`B$N z*3n(T#p373R*$VSDSuvz_Itf)ui^ay8l^W>D>U0J!x|H(UzWUj3#dIzFukg z0RPx`8RovbGc?^ML)|nW@zD&vC2}G?faoG}dZ(a7TN;)hZWjLkPZBj4_h0aV8)}iK zH8AjJ<+xrao3RqDG=_!vH<0!&>2lEK$6iu)3CTCo}CRg&GtA~u0L!8NIp^JJ?gtJviRgeLr#kWpcJTlD1Jhbs*kYCKH!N^p+!WMRjy|8IK`j|75YT1Pr4@O?B>Eu zu~@-mjV@s zNmaBlI1Eo$?~)jpk$sL-T>5^RmtkhKLz&zo5~~{#dmXtRh!gBr@7JB}Zn(Amlk=Gz zB@JZA4AS|?(&NVr1Q=AFvJ3auC0}-)L~Ed)#^e1vc^1z_Z4K;D13`%l+SKAYK}v0E zzEjlVHT92?f}VV(IP%)kQ(XZSx_>SArb$1fv=Nkg7D^>*^r9T@gxDDgLVC_qoj!o9+CsByeJdiT7i>w3E(o?b)H%N zcy(0JqCi_3yzV*Ue={;!^5w#1l{_Gg`HF+KIRO%^o2sjY>~>`M2fz^kD*TlrB` zdg)c{@41`<1COhTRVv;LcLx3Sz4F@rL+5_<5v9=lc;{nnb*g&HTZLx^Pl(Aicq9<) zn1%A!DcV|Ej6)zZ1Wy0LY9Gps$tdn+5j#1o1t?a(zgw9jhva#zj=E392rC=kM)KU< z1dXL~7)bp&9PDjkQ&F667;t(fc^%>(eD545y=$FQppkSz1ob-+&@y??0z4j2QY zJmOoU^~m#-i6bIx5bQkt%vHp?XPEUb zpc)&xN^EgeGsVUpB(}iZCQ_-1NwC}YA;{phaA(^=>dS-3caymeZ|=EEQ2Pe{?49Bp zK8{BJ^vnj(lkD$~5zjVrBKL3|SwOLVR^QH_x_X~Ro83o2bZn5RXVaUjJrD%@j3@1L zo&~Oi1i#n(gWa7HOClQPFGOWNo~**A6GWFg=xZ5w@fHflabh2 z$vn-wo7#uU$qj@sBXbx=*R2p5>dgti6m!kco(=6-NQNdlit_K?;=0!m4jX=E@PR;V@>x5|Y*o{fiN^Oc8 zn{4=R34sGc0S=}0G3W~r-pp@me9l{vm*RO_!-1p#6We28!)bmgWMECq%0P~uiUUAg z62z@`P|<}td41iyEB3mlPL>N?EyGVm`l)qe@MYdmXl05h*wiSv^sA62+^3!cQ2Cdm zL7YC>4r^~no%Zf3jX&mG&3)vdj;&1!^MUX^mqT7sV(455FhXNPoWeY@ z-mj7%+RK_-=e{RZ7MWPeXw-;)7@lN_OsQ#T`iv+Szgv2V{P>Log^_BQ$G7-DSpf4- zgx6=iWABHiFWh`g8S$VbLPDgpM@8RRm!0ZJq00p26yjf`4F2`#*?jH8lT{QshR-X5 zr&t8aIt0BfU(hrOtl~Zt#=QNksUXD1l@; z>w%qkJiT&h>hG$eSDXm{WwfHr#9Ryu<*TdH!=OPYRC}@d;EQc~X66LCC)i2@*_Ekw z?azNZoZzJf^8fuY=bDNPZ5gPdzYw(_pl?RO{cM-QT~=P|_Gd$ujqpJxl78v0eTF=; z9er?_2&&Rwj+*E{%gF|lIbZwz>{wl$$l`-t<5h6N3}ALzwL zC3f`hZ0lHnDD@zDKWyRrPGS7031xbaQ@9fJ@2cdqMDcD@joom_^z^eL=J60mVrRz< zgdFB~)QVERrYAu`%^0xY?x#EoH76LQU#DJb7yP2*8~(7uECv4Q$qoB+?4&0}1&4&( z95noFyIsp%{*@B6Qx_p2tvzN#D;M))L|XPeThRRmm2cezY9jOfhqHjHFugN_-vJDX z7fTQ-dUnN)tG8;ZDZ4qti3Uf3GnoOv^7ead?5Y`cY1wuX+U)DcvO;@OVWH%!A0Sa? zT)hN2v^H;l63R6$x_-Hot?1dq?H16H=OJ_EJ(3Jv)p!m&u8P?l*CC2C zo;_bH5;sq}54?ER_{+f{hmD;G^{PyyTh1g^`N{M!t_)hK_e%z8^+Gp9oAsypvFs47NTi8Kxp}Y zz?D#=5a14LotYhTgHh1*AGWXq7-{lkm>`80ne&G-m}hNPRh3>H7kCPPZ-zdl?Y#kF zWhkcqja4~uXvQNX9%7LK6_W_bVK4X$aT^p_!AFD_hQ(`EdpFFhw;Hon%%h(VmEu?* zNQ*j`d~RC77;vsv@g5VIjaK#Re-T21Tt$mne~BRkLqm~N_@~DD86Xr)wwB0Ix~6ps zYVS;3F==q&sb8r~y6q~>S@D3j7c3b+)T`$^E+_OXG6%f!JT&oNZEgr0#ov{57Iuxa zcL5%>5I;_2CKLgh{$h#FkuSRuPR|dI&uDd^luE0qpwZniiQoeIR2gFAO$IEpf#d0Fyk#J(r*M1Cq`4* zQ~o>gW~$tjEa_Ku%~arF6^!}e_w+hv{Xx@ka`D83dGx>_o2L`DxZ~G+3v%`fn;_kgs8Ak|=21-|HiAlPnA|j+95;aLzx}{xWJlW2`$k_cf zd411ZNAzyaR2N?vbd&AlQ(6R&2l1%LZBhzSoW~ZyVy{*219ayiF|pe50_$%ByPdMJ z?;r4#>g8)c!7Q|^*`ptFA-Y&cx4{;PjgsOp^xTq1cV*K@5l+b`RO<$mpA`>bSFNfR zxQTKOxP9EEL<>J`bKv^JovKWwek$pdgo!?j!xt#6KWwx%!rXP0X%dJT2I=@daJkg{ zL{4kISH}psniFyN5B!xYxRcG(9ERM*GtUuSLj5wcU!`c2tUMq1#=pq?tQR_88}K0u zyG?Z5DMa*GILEWzhpzV8(8(WpAIA>gF)LxW@J_lpjn={ZSLn@l)+*-{YbYq%y{1zw zsYisV#VjhAMor~beNzq|75?mbvwzkCV5`=3$DIW2zon%1sodAyG$Nx3PB7QtW9%LN zaPifZXmmI8&U>$f1MTGxk!8;`lBrw+R`#3cj?LW@%OxUJ~KtU3)=260YCLd<{G zuDZper}iutq9Qx!pD_Jkvo{M$q4hx4LK3RPdeN7=%M7xa*v)>{y8rUQIHz*zZbK0A5-4muiKT3c+-1k8UrcK)#5zQc>fQ@@`*x(ngTZpv(d!% z_9ooq#XI=SLJWZJIPUg5qDXE&At1=bV#c)tFiujIdCQ@g*)Bo>q?-P33T@toi7Fzn z0GrdRPb_~iszZ6u<_*wbM_7$T%bXt{1!uD0AQZTe5z?$_Phf|ygFOk*m|o#@O_C8e zsEr8nWsG8b^4IhcflaDDPAsm>*@;(Dpy(#Wy82tc?1FnKusm>(G3c(08`KMjv$58k zym{kIeMI~}zdadH=Y%{*3LtdfH4R$eSKOGr#P147LWj+LoZwwp8tsyBz8CJf>kMLZ z=OUxQ&dx~+V6Dl@4oizD2J`LE!D#JE=K&^0VZ+bPk>;I`8lu$6mug_EtBnsdS1%A9V0gN2 z*$rPrP{x0PthlD{De+LWr@4;Sth7qeq`;Q+6n&4f>($=%{ z{|q$Ji0js^eciQZ_X8XBxfiM1l3%5y&bDFfHa9vO0K)KS@Rn-js4QfRHYl9dulwXA zRm%Mvj)D{aYb5jq*)8RvFC`uJRo9E$Ek+)S_ji{xIYSw&PUp^|SG2(vH5YaN+55BiK?k4o=buxT){mEcRJ%Rj>>O@O>D(0>ka1lOXkrODGg^{# zX||W%@;lAmh$2u5{hot9Z!~dU>f8z`@CiFL|J^N%WADUsRY7fkq)*v$GOAGI-zjV~ z2~$c`?(&E%4~9+M_RIfaoYrKL@?y`K@y;YKz^94(4ka+<;TjiO^0J@znSijaRrqsyTRFFD3o8m=BDA_(=n8u&X zN1;KcH(@tFW{n;beOV~@qsm>UA@oQ2d^ysl=Xz-oWSnqUGnf^5I?(p{X~VDO>hYyN zcQ~H91ZzIy@ayFy^S!9^ec(nw5cTbKc0G$TXXhteh5YlG#aQFtrA^(0nQ^WLrUwQs zYcz7d>)%X6ZaU}0-cD9X;O~Z$&3I_B$Uey+Yo`9M{W}%-k9%bPM7eX-943m|L8$_g zPhY^(b=t-|>JnLs&39V%Nc)+LL)3rP)pWmqfRq- z0pQ!FVahf0mJHky;8X?w0!vpebwgvzpx^hwv$lABGtbUctHcAl`Vjro2=<3C<|oip zu|O4tYRR5+1MI@Gmh_c+ENSTO9cfLQhwy^pCOF9aVzZ6%K01q=ZpOjrfSO1M)?!L7l7z+4b%x$2Rf>E0A$Xbfvw{Datdpxal`8Df z)T^=|X%6JMZm&6y*^BbK5yij&6`hdR52d2xFyiac&oS>Lk(QMO_yC$=n_FR3+}Kf~ z=Ss2G>al>QH0!xYlEq6=rFV4qy{s=r0X<^bGp;76UTgCDNTnxRxs>=AXXMIk!lRo8{R^nlG4_^N6Sz;R350lmz-5ZS! zS-(%#`;GWocI#ZMo(}g;o*c@er^c?92~3jF1$DUJ!Kit&#d~p@fQs7tbwlLrN~P~o#7TQf&DS%fdwg}o5tW}9+VPHD?E92j zu4Je`#WLj@bfGgXyiY}#&Tmz!qt~tMq{PJ)w zCjV0mRQWk8l*8}0CYyo|)tgZ?s59@uRdD#_iP19kV@?VK56t5*dCh9l+-+S=+ns68 zbEBTk;h1XEI%XR?8BAkQ3FXW1KPQ?+N)fKskfSsD&E1sS3Oa?`kL$F29D%H@!Jj{W z)_OCOcEG(oz}eDUO@H-M`K)(L;avgw22(OO5_xmBxWjaMAj2wAeQ_amtGyHUK&3#! zF%Bm&o5BD3xwcN+?da)p=QTj9{7SxYN?69bceifiD!Dr}hhSF+de#O!trrvA8#G!pVa z9+wbs#AY%pX*Kx%j5_p(Ntj`RF`T~263|S^S3Rnom(sN%krcn`TIhr8aC}9c`+!B* zKX3_&jWVqkdbn|_KF@&bn#KzdeNE7bY>Og-ixBuKBjmdjD~|`**E{UmsT-j4bgAq# zU=lUXob`B*J$$<2`FfXGFcQN{3B1VgR1_XhUgElvUoML04<3n_%M%Pqmr_d*h(5kG zik+u}1sjvI${A;h@`H$jr3#*dIk+uf`l-0G!l|oH9J2>j%{KL3`(<54(Lw2O=T8@? z7|FoVTRSzocSr4S@E|YZaFnBNt}mJkM11sb-@H82=q+2xwH`~ezuqBi^nxZ4F*F}6 zJ>CkJ8(*lIlQI2cche9Y&pL_P+p424t{h$dN;piY{>R&r=1s~#HqJ)>1G&b?q=b`@%38)gNya( z#Gs?A8OuVQ60vrgn#s@GLc*7ycu3DUtyu?|wVX9rxQy(VXnM}SP`;d2!<`s8oHh_$ z3z0+5_}?@$P!i8KhaWTmr7K2Z{L3S6`ru46BztCr@WGmLU&CP+<2n@hnnEpslMH=d?tq`owJA> zQ;-bktKR4$!a>TjM5ZFpwtzRiASn=`&895l28~CL(cHckBwy*786PS+PZj zr);n$2z@(QRdz&6Z3U*E*M;qtzMTQ(v{`>N-lF-IR_|BZ%bXay&$DyF)N3Sbxk=Ks zm0LE!n;m~OOfYMdw~wi<^^M%fpc|Zf?X)&}R_wO2qz*nHL)L(N4f+(tQL0-+?jf`x z?A|SM)o?%p`Pz|F4#BQIo6B}HtdnVKrT=xbjE|JK8{v(UH@S+NHgWG)?Q}pg2Jnpo zoZKtov5;Aw_}~#e*obCt?}NFfHb7!pGw1tg{H-hOY{}EoqG4R(2vshbc?G4S;|^j<$kEb|D;IrX~& zD=?_G#`%l*2me=1e-FM|*d~OT;=pGa*TUyp9(!i^W76Q?vex5*7-NOEmj^ zGV8Ath?^4*5r^?e_5?#C*U-q6jYrv{hwQERZQqnlUPS52aCvK1Tld>n(aQ61$3TnT4%`H8VK?Urt4~GfLyE&luxH8L(Qiu z*v^#>noc$jC7^eumdTwxQm;DMZZ+RkjcTm;PsEUGfmJ`eyg+q2k11~jmj_|H;6UY7 z$P`XR-R6reO*`v!7D^m$3h;KJ)(M&{h5Dhcr7exER%o$b-y=bz1KJNAVb~3gdAF3E zO2*;-Cq7e-;nDT@x4$2%2wZG7WC^IGobD30VZ`nyXM9j*qub>5#o(bs(kSXE3Asog z1nv48MQ~;eoM5CZwlI*$d8KX}SZk~P^{K7hjilwqn`MmtQurUQMy&@RS-SSwu-_I` z9O38~1=BCEe7b{EV2c7&6T5deW5sB%tJhxaZZCy_D?^Mv(@<&7qo0d>N@f&tddy&} zp2p^5Pj)8R`tarDe97IG-p(-WQ&;_Sv#F5T6MK95DTq%qOo1DsEjwuyVT!Tm+7cGZ zp^bNL`b0nAbnD+&(;~Hbr$!hq;))5&j8}Zp=3F%|VrQbb;sQSZ9V4gtRL!eBQ;B6z zYA-18={4Z0^Ftwr=&l@cgd0~+s$KxM`)<(p*d*l;z%l&VSXAy|>d6uf&9#?yihDrl zO5gP4tD$~kcVhrCb{lCNPGk|0>wiGITI?5<|I9xI@BZ1a9FYXAn}Ccn{QK)=!j8Lw ze&3DX*0DUVu&q~#8&OA5FW^1fW3Axxt74-O;vY|#oJ0sMPWiPdJkr2=&$6xi!92$w zN4pUC-k$NgdEkS$;5#ckKd?N{A|L=sBkqi(R|U)EGzQe_B!kd~NErrN>#fr9`M~!r zzq|yyNHe6O^1z(AhhWz+#h$OFn65Nm(Ai{v3%FWtH?mo=V#(y*u z%lgX&n7g={-Kf+GE`3EJrb7$`qSwNUjgc~6u+fdGFkjXN2KxFi5hsFg>`%PqDvz;V zHt7ZW+F%iV{L=o7SlYkcxDJPIMF|@g3CA0tweDB^M0s3W<3y?83iNko4o?$&J^HoBTi_UyL~(h7Gwg+=3BQu57;r0?=QQbu%a zuc%b(EZ4oSU2(XP&ttJbIr?3QcvcY6STuZ7-X-nbqb#aqOqG!N1!YFCJRrQxMGl^` zF*XEu0?rYoel73vD9|?wuVKMlJ-zm!`r`H@9s&S?z+SWURFF52y&KqX`w%?XNdIe@ zHZ?b@SNnFYB`jTv)laGL_v$1XaIXz3s40dtcD1D)P2;<{Kg$DR(Jc2tQNW9xwH3RS z9jZoL^aS-4D;pJo9zBFLujHQz2n&h?MYhH$hp6RYb-vG_ekb(!@Ilj}@7`%WDZB7O zPr(Bgi_Luk@|y_x*`qeP!xMqpvHP-T+a=3coewbdkExzD+#Z_kAa2>ZrKOcqi?@Mn zIpV75`Zj-gH^o;1ZV)-L^eRH;C>Do$`F*MBg?-0KQ8V528ZKCZNlojWz;<=&l?dE^ z^!|^vv)SXyMl@fOmO47x6gu_E)tQ_pNOq_s#v9u#Y=C1=Zux(D&g7_h)TR z7?7;tcG{k&&=@J+_M;q9zg@oOo$1Z<<$RFe=r~u9MfJ;@*)lVg9f7;859^tpg|gS% zfst?4pMC$(-PwHoib~(=`)>LZNv!opOVtmgwA-xDyWOdj7!d62>Y=2{Wal?c>axKj zY#TBJex%BP(SNrkX1zay+@8-esevYPTV(u3JSM z%Q^O!OxAsC2$ed$X*)XtPtdIgAYO&0H!ILP-wD@TMvr09pHavCdJP_zOJm5Gd-CS* z(3Gd3<@0c@OO-zWp8YSW|L;~*I3w^6E!cZ`LE5}F0Ph?(_ehjY&C3%y=)`m+GKf9Y zND{{>Uv`c0aI&?1Vr$CZP3U;i-Jp%IVujdStmK2<;iIfM$Jk3lPY-j zLUW-$gn(~@Jy@>mBfRo)|B+3p4f-ot_**>qazy(tAC{Akz`jO12U=MwhS%KnJ;0a# zm3XCO7%n4$C3AHK!xb}Ol6%741{?j-=CY?c`$gEWMnjn;5t086Y+yJ-jHDNe4$TPp z(ZeluPpXOez~4??m{9x#tK%mv8fAXr9e-6%?2O!&SYHATMg;FXaB`%GKAKp${!Vr0 zQKj&FtZq8&Ktf2z*w8dYQ5q<5hmx$NbQ623o;FqxS5kizINAiOceSK&e~n~BoA?*P z4O~ohpp(VG8lR1an4C2&QD{?W4=y%EjuOLT((PO5x!xg3IJy*x%oitHcSn1}Uc3l9m*Bh#myfb;m8qj^^^{c&AKWf>i<``!9x3< zyXjQCj3_HE?uRCVKM5M~?dqn2?GqCrfY%;*_3iVWsR6&w%aZOY+}P0k`3upx15}-BaSy9o4^@t9`u+3{m=yl`)d&H0(NkQ@0cyoYyvKEp2Ja~U;IiWyS zMO<}YDPuzsmS>TOK$?Kx0OjQ7Je{YM1dltnBqQL{4K^hw*g#xNqa+$|%WvL%9v|!w zzm4;tEsJotBpp~@`PsdEHord+X1IZ#WxIPGb$yd!U-}}Y?NCWoG@|)5VZMj`g<{u1*n~E4Iclx#khUdS;WBLK8AuR z52RK3?()ZMdV_>Tf4-~+pyJ{M#p+;d8}M$4w4l@Zy%~@gTkAmVsT=kkBz~IpwN#;a zZXkMPX$Xc9JS61Ud-E#AXh4H-D=0Talcwy|$$I0i>e=`0ggS2A{+s%k6CbsVsiyD?oR-)00Nj{jqx7>C9U$6*{TW6>5k>4zvXOs+N{aVgLFa$Q z2@lfx2H)Ll`?R%7dI(_@d(b-r~H=i#-bFW{rJBiLJyb}UMpl|Lm z3&j6N1Zu#(Y<@D5ndE%wDl@fJ>|S&r9OW}mkZ|SvFThh<*kmq^`d$VMN$OJl`?c@E zX-u4CEdJlb>OFsoyUkYT>E$uiElGG^5J;NPXR6y;)C+&K(?1~MR=&&5V5J6jYerAU zLvqaO@llIN5@ZDPGhnvj2Bp0=oX2ncdi=0R$BX#1L1_Kmuyi@G7x`NOTJ$222xI-? zCbdWlcijV3Ef68^GdmIlf{^y{&j5~Rw;SOA@8i#A9QXnSSVC>ErH#>@mAmXuh~3?= zx;e~;aGAfIp{9pq1IOT+(sD%NrZ&Yuj%d-;W=q60UYB1H>Ak zos;$baU8Ze2roe0LDF|AEt>1HDy%qW9F)ULc94Di%FPex`N6>4#*WsKi5eY76k;3k zWzJz0*)6u&g%+Jkx*~O3z9)4@4n7&&2o>?UxU5etGNg@$ZGQyf7lD~UPM{Ff=iL2@GQx?6lu_<+?*^stM-N! zW9RmHbBtD-PV8kUZHVB7jRgWzZKa!w{K-WQsu-d&9ChsUy}) z_G^xjALX@<*@OKwjucBkyP8aPA-t~994yA~wF(0K_dt1i#D@y~FRl-;scawwR4has z(p0bF^Nf-OT+mFLaM;muANQ*|3E^ac)=cS#+CMvt@2w6zyS6v2ed0Ib|F*&y#)Y3h zJPdK8m#aj7Fvb@ZOXMdMw8+c-@b5&`DV|+84ZmMwa*6?z=IY~-i&gC>*D{2)=k{W* zEqblK=~k|Pz@IiJw6n_4x0pT+eBZlfyH;_PSz)-|1jMQRubxO%#kCjCri5`18r0f1 z+b^Gc$nD5z?xsZ6gck{6jbGiqEWkeA3MWeMgA)#|ae$hYlScV8Q)-rO%3q!?b($`} zQO*+_otLKX6u`kkv@Ihvt(wSh7irp@NHA?GcM-=w=7x+4hyaW(%cEAYnm0U4y}i~7~yQZgiJ6m+3~?8rB) zV!1tSB=F*|dVq)CZd>a7bv4$7T*wa7XtW=M+y*<)Ymr+)dY`7B%hR&CkF)(;2J)9< z)IPGy_B(aD9U8_D?-k>mLh~w0u)azs{@LNy_U$^rh3052{Yu1aM58K^Tx99i(Nqt= z6Syg^$Y@2hd*f>5`3GBB^HFmSWn>IuKCXADQFsp!1;i8KKDl?s(_&rvuBtU8&o2kb?)yvbKDob(}Yl67d^-IvYIumG~n7J7jC$n3?d3)v*+2HQ6`6sOE}|)p^;l$lu9f zu+@VoyPtDHcJ_tT!!`;PF*E(o=qn z?jf-zdx^|o-2t*YFvmGH82gGGy_9=Cu<{>D$J=BT5kE*##K&x(LVys&7mZM7_CWj} z6vbP_NPoR$(v>9*WI31WoD)H?NELyC1@Jj37#ty#R6INyTuW5HaOM1@@|g0pYtze! zZ=S!?L%ttFb8>Egh9&j}w5=2GEe3UGcS+yA;ngG{142==NA_>nJ-cp};*DuDCVGkB zF@Gtjt+{Mccvg56gwM%-ejZiCo>bJ6)K90-1^G<|LYpFUP(_VKBT{|yi2gl0LoC)! z$wW(1m6b}q0or}sA9&St0mNcbv!yh*E{pO0&i-)|ow`={o6TRP3){B7>cyHQD*g2T z8~0s)%7+Wu^)F}qcb|cUR0yey19_Yb)~H z`MTnXWw^ei|(E^R>J`Zpo)$Cp{UbPoR*tE3++E~m53td znW`{lGcKPL9$Gv(+imjxDX*=kN4Vz-z|qHD>g2qI2GsXl!lfVOw;VApM%M#C+11ii zxesA2Rjsg)I?b%BM8({QKPmR0YqGL#xF5&b$Lqj@tG3CeoiF9eH}5B>O|9P9L%*Hw zyzJeY^S8He+07{RSNAxjUP(Wkd;_IY*PH7kqHR39_8K4A6Zz##?H2wJmft^%(?Ul$ zWluf=fwrw)Fn>PaJy|9&_4L6J&21AP$UM;HMuam(T)mEs)-pFgj|QbD94CBcZ#E>9 zrE|EjncF#-Z*3qh>+AHIgz&eDbZk4eoLB^fdXpD#I&EJ>T4UO!fE*0qf&Vdx4>h3b z+V{RNb1tm)ZRE#edt(H5Qew~&+12!-hf&i?G^Uah^ps6l)C#5&e-Y&d; z`;tP6noYL&fnVQ7Y29}Wf}0X{6Mok3T31m%-p+Gzb07xSL|(8V4(UTMW$5V#^8MfC zURUw-JM{BNdd_H_28rI%QxTA9Wb}CbABl?K6ZTzjAs!2XQVP$`-Nb$9ve&C*oaIlj z%T=njQa+(iv#7 zo`Fy#Af&>Ex&RgK-ES0KqEc4V)PEWyN$>jNsI;cw;U;VxXYxMFaa;HaYALJS#m@+| z1_Iv#z6b6ZdweKXag^)kWZ*m*%&j((e|XE6htM5JE^p&N=~tG_5UnjVj04dEAA0w3 z9b+z@a!%hSQr22-YHCp|Jk9VKgV#J3vfW>t?c!HvHoriq!k&AxtD2fxfbIJeAq~{M zbQ|i_EP)Mq`B)iW63oJT5e|f1TbFV86)!Wu%Jf}`MMPI)CKLSNDPse_3hW&|)vGt5 zv?x4NBtRxBuH5}Bl-{g2I?Z2@ERdsH3TGp%xYm)f^jyB6#f$BEM>6m_(-1iTB`lf+ zHy+-S2~&6^%#`0UxH`B~t|(D6qH**R;&@aOy=9r?R~)E%lZ_KGwIV(#2Qr=8#nROL z8?SIFngzLJ6>rIbK7zDJxLMv@b_fWi{inWlc8?9CZRoZydh5#NQkc0lhXv$2A55}j ze56HtsWPyGyQZG#(xaIaZ-Gp8=(P{-nz{1aPR@G)um@)cdBqW;5TasDQL2}Oo|x*D zWOPi@i36bnYP2Oy<0zst6NCWhWDcAf}!Et zI`oC3tE>xmB)g@F5ntO#;K)bpslRmyOc;UTDwNbucIGgW`ZS@XPd3BOR*(N2q^HVK z1pl7?&G`3>sKrvq6u92ZoTk6bU(MRG-}}8D!E(eL&h4=$U$+BQg#LB1)gmV9BJOWr zE#CQaYznjhP4i6i(P$Nb_?v*=hciKeUUyg}>NiKJk z{-!;oNoKE^WsJSpiFhX}4KI=Ra7I1szO5wDKm&(6EiBq}tob)mV_K2TIfitw*#5zb zx^FkzHb)h)ZdmqrUL4kEk9;mIWEjZj9<`F8cA1}?c~JXwp@Oaw;BNwB<3s<3AgpJ7 z-UKd|Ym|dA%|1w;zPZQ#vOH5h413Qvmr?Jzmxo)6cBfh-Eyvjg8xMb+unk`sV19PW z^*&B~%NZW_>dS!GXP3_;(~|+%PozBu2_yzV|ftN?x&ng#xe8xe;`77;D- zR9~MhI2l~W0h-xI6~}LrG7q>Llv32Bf8T0h%if`68{qKc7^Ho*d;pvzhw}3vpN%Fp z9&%_sNRLCAbfN&CGjlC;#66b}r=N_Fg_%Ka?f$P{r;t0p?+xSzB^vkdx?N_>UcUNt zUyag~h6uloF)`@*49_w1v+=XVdY9i;>UxFinw7e7sG26bC_-vLz}gK%lK4#mJnHJY zl3R$D?g=^wqjUXOIJ@IX-oO&)RqqQyzF~pXJs)ZQkE#HX;g-xK?K3O51F1oGV|4E0 zr<2K6e{cExXk172uU>AU=+_AKZ@g3j2x|9`p0WQ+P(Q&+k)_a}ozj^;Jz)++#e)|D zKlgtmc6ZwLRIoQWIqhhpk@^V#vVA;(9Zp0O9RswINcerQ)Bl1aW7_V+<7>}$hSE;L zAus46ovzTw`Z!;8ysd52617G}ck?9+D2XWE(#yEBp$;r-PAA;g{wgA7&AYe=@{!gN-gZ4i{I=Mpb;yfMzZUTX28|&#|p*f z)%B6l0C;3lu9GW^mM`;J3PxHbZ9uy(%*8k?*I$C@>&UZa5PCWx?*k-qC0`tA>i0s# zX-`jCtw>T#tw?k|oCh_Y^k&~FQvyt~ow{AG+q&>BBmEtan#%o`s6gouZPB7ww1?BP*sm2m&hhMa?QvW0O#}Y3;u<>eU~E zV$}?#1leLkJOv<0mHv}=1b+HIW;{5%FdGm%^uF^ySC&vSC+0nl}JJ`YyVbsBmB;Af?S$il=@#5uY*rt?7kACMalfMjy<2kl<&z zQg3p5`|h)=8CCXgrX#fSt*!Z(HQ%!EWz%pb(Hi>hl4gIds8wi+C`HnEB%1&(`_&e$9>Ij?CFK1AGX8?tM zfNnIkI*{t15p%rQ+cdiJ->qoYNx64~nJGs}RW@79+U-s-uuu`~%umyzeEu6er!WP= z4HTx?@3X55mTP6`6BnLU{59Ywq8n+=qDF5VB$j$XNc!%}V6wfezk#Sll6qa~hhXv* zB_O;R(NSDHRRQwGcX>yfl|Yh8Pdd%CA6CXs?mlf|(~5=GL0Gji^6U5u0f}xWg4&<7 z@Sm|(6KC^4M4|5)6HS(M-_68L(&sDdjm|{jNg;9{V&`><`ONltXD0Q$_MF8p7~iGB`GRn7=ciA_MnB z<*_QwJPdH4Ww#7_uAb8(b=0Jbw;HaLrV_Tw$~ZuwWv+Nu!=(cmWob znlVQ%yI?$|mEJSbYQbf-%Fz{sY@6~ErbUyPUkoEO(%4%qX@PacI@bt%q#ANQ+Dh2G z=~y}gERh zB37>n33|y6?c7-~{?G2kT2p4F>g4*gT@kF4Xw5`Wf`1@4>_Aj=iPwzY z$pXH3gF@qG)l6}|EQA0kQg+1VEx@|k>iBo?ncl&z!{&g2@U5<#B9oF8UD$40jaS;erKWAWt_ z{9k2Wad9%_abPQ4fJX<_74zFRL)5ed!AiA>m3L@0SL{Eix6w^B=`y=5mL+}J`%VRB zADs>=*cKCF#jIwWK34e_Tei73$2q=qEjVT(ye9XUBAN3kb(w&ZuNDho&CGxOnUw+K zeV~fv6JCh)c$u75oU7`ye!{JMa2Chm*jF z(WW^p*u9gJZ5+63e^^RkkNKo44!jKH+v6EvVx2OG1(Pv7%-LHMz%{=(&;Jp=qyckK z3eM+H1(y1B!!W^|9vVA=N)}uC5qG{6Aqo%Z9BYq%sSfi6i@eKGwkVZtEVn@8{M8A@ zV+7{(faWHY7Z;&?b*f$u(yKgDKPg#2yLPx)PW9IuR*qTt4<(9bKUJiToA(PFFt>M@ zMXr#CE`x#-zH5bvu6*^L@M6_o&zoHUL0L#!9E0F>E;m_G!E0g&zV-;!Nw8pu3eh;e ze~X#!7s}p;Hoeg$bRvLXKPU06iL?H4<3n%-HF09mkgq%d`orSkMG+9I%@Vx87U06u zh^y#f{sJ_cNlwzb3s;QSw+XZ{GZCy;2)jf0=x%k_P38?CP4)_TFW|w0?qUQ)Z@0nM zK2b-@=DP4%U{c}g_5u-AlXrPI?SW850TjM*zJ}|{v}n?ShhR2j(@MT(VIJANn9tdf zSBxP6Hrju=9I=rtDq|qF*^+F;SB*B7r~lqvXaQK%u?~QQ0I>gLAw{u=szOfIS}d_u zgDL9`w^x7H)7Zu5wOk`yQ=b8=TmOW-tH4a9PnTNubHPEsXN$yn3IhYTA*ZeWlVlfb zDfWYcaFOp1mzK3?e#sYB{9&84w``IjU}tLjMNC6aGXzz-PPyJi)O_??ZlQ_8w!Qom z?sTO(w9IUDXaCt^eIrO|R%fBw3yZfowO{DH76NzDRyY*0S~6b)B4Q}$xGfXBJow1} zIdz8BTDOK0F&&h1L;y)x8AhjAPS8=ZafL~GOiPMtd^kg2W5(%rzW#mx++u7}R@44d zH^2t@x43Ba$0yB^{v_riiyFJ*pRNVdgZ&TJK*ObMiFlL*5Be1Aoo#8dgw2Pz?s*Dx z`$o-SMVTp`6u4sD@iRgStSsFNRC&W5EV%EhSKJ?Sg6oKk$YB+4?mcl{jIQ;b&s>0lkJlv}tqKxAng~`f# z#5(ifR>T7mG)P(`b%v}C2LK5e_ZCOZhdgv=h%(kpS(`+8g-5VT^$AL6BT#t0Nup92 z^tLWRA^>DT?AcV&TLw4K2BugmumM%d5TdYEb|w%v+DMkitX6$>va8<7_QRjslDl>Q>FxgiZ8mlu`P_~)Z>Iza3Ai-hBcI@Ctiw&&}{Y2WapMi_cu;2Q$WSinE;vX`}e|6WaxJ>5CjmpFdGN z4_Llq%^d6CN}xTLg^FsAe7?iG$%mhQ%p||jTnIC=i`-zJ=al2NWRmg`jMMHWQ&XPj zxc*MQM<93|2C zzlbfdkCW^zdsdh<4Vn^S-w<5^1RKT@Q1~kyRXDx{#4F*o(+G#My?K&_->*5j@G!9V z*xo=zR|P4Oy%}Dn3I?W|{Zz0G;YxAkcYrsl?UoaH&o}^Te23%Z{)J>IGI71z@Xc(> zNrX*Jj!D|5sfsEh*s9HYrtU4{ti-DuepT3P)g69SQY5_8BEu})tr|OLmr*jyI!(Qk z#T!RBUe|)c6VF|nBFEhiaLmnZjGwvV4$!@ai&C9q zcVjn#RUJza!Oj)|+bUa_^xg+9zV2@RPC|e)iqqs^VZuGr?s`*=E0mh%#ijmr~zvU#9^px+cReuZT%WE*GA1aD4bA9l!4 zR*W&fI6Jam)D=*H)xu{1Ad+wWZv53-wU|UmqU8csJJ|?RF_H{6ma=UZ0fBp6lh+5O z>_b>%t+`mhcbuox4L?z1*Cf5xCS8F~C_fjITFihq;;k!324)>C=i$a2u{E;1e+Il2 z7CrV*c-89|E2M7cr?tC{o^Ituel_=qup7Ssu%we~VCFskA_u;~SAkTyz?x~z`j{Do z69EaM+;{_gN~c^uw3YPT`&}RNnoN8&wm!@G4MEND(#1n7{BeF>uxCug}qK zO#hSJQGWlIon;#>yFh07Gs}Hk zS0p4J5bhIy8{(vyuuYGBesLQ zZ0=azehwLILkUd+!v8)I0#yVNvks8uzkw51m=I?*(=hQ*m{Ded$V{ z;lQ;%M+Q?jg;@jt9dq6pL}hGu|ECLpykbM^tVtdG`yG9iJJB?^YV8*O%Inz=0&^^7 z&#LPm1sVMvr8Vx-pvizRbEzkuQC}w5{)rK<|8Xk#0EDDoStjwDD8>JzXRlfIxF7M) zL@mM3-B$`GMX2WZZeDLn97}GgM}6_#_O9?_0d9#m-vwWn=;ZcRp|H#^L^`kQ+mD9@ z>Yw)DH#uOuRG6NFoN-PxMDp1ocXJ=s^_*j`%Zd7R68fI#POY;BbfRc@-w*z<0) z2IfQS_z$#UoJ1c_^UaUHBgie}l7XMwsOn9^)g-o>{=0rm&~lhXv0gTn*+}s0t0f|N ztY=wW2vFk`3Xhsc&+w`+&@ zGT!QVZhcz(?EUkZ;_TdzzHGnWaap0uis7zspO>3rXTK*2(tuG~H?AYgWk=sEM96p@ zV*`H$CqTu3LHr666qNi72Pd!j-4Oq9BkWg&BE42~D#A?@d}W0)j0QmB>XnGZD9&e+IMTXCUFw9@JO|L-7HnI};P&1i@&!_$L^*3B!1N3|LmHge@ z7c-V-IuC0V=$>yq%%KQsDEDm)(AV5k(cubQ0h%z=E$qV@b=XAoA(ro@-Q8c*te!)6 z4oyCb)`*s;P;zPvDjc-0>o3@LXwpT}^j$p`Boh85OZYXJO?hdT^DHSj?`c?m>p>ZO z;OKPLUbmvD>a9u4C_-g<;btLDkxWq}e=XlZPMe|5>+3qT^EXPTN}!td(BA({s@@M3 zKPL>pMLoJw-YXxFk3f&cA zlym>AH;Ogq$%si&x_oZg@iN4eUbV%tH8W4_cJf7Z5?5lK(||_RIJss)Sfb6#U7zWL zBz}o7YAqP#?8x=6)pV$z)V>K$WTM4d7VuzP2CKV@3yX&BZG)3?~?=EP?KkqKr2V7(;xxWdVZV4``yAWgWgZE2jmTo!1SQdQ< zd{oR-3_?Vup|IGqq|8p=+iNr-pEM&F&o9o4ap0wFKIyX@A(2*=kRQ0?9uo)$iOO_~ zAN)TFq;Xl>D$MGE0wK>5;4~3x|2Km%aV&n?Wp9vxE|)-gIrJz!sW_Q01$3wyTC`It z*fXAWjx1xv7!-lK?*-()S+$DII`BM!7}^Q{@v{yf8rr>;eoR%dW(RbOMnEk3xM|T4 z5b{JJ&C?J$$zQ;pqTqV~w3ivTTK-Iw(Yb?z>~)PvasK3O!cj*li(xA^s-TrC+zCO{L7WNGhDIWvYAb>#ttOdsQU;7?a(t6 zQN>YrWDXBJbeOSZut|TR{n>nzmw>6M$?2XCf8gt`utdlx&|oiY8o0f=8Ipiznz!b; z4dAmC(qAuoEqVFwDe2N<930>c{nUZL&&cikG^zQ%^)!~sUADd+BFya z`dhA?iHI=JElm6?`9;Kc5>H{(SL8Iu@_`Y;GyxHZYy{&2z}3wBBRfJtOc)>NufUM? z-!gxAraT^SXgOb(Kvd1{7tXB@4t`4&K(!fHD_xYAhC3+(4M<<-L;X%@Wd;fd%i?4ZQV-L75*UPd-+t8g6mPV+%i{9QsvIO-RuM`F%=mCvu>4QoZvYMHf&~ z*rw*$^p3}y^xLa-9mm>r9XsIm2DoOE7BZ{w0FQ)&IVG*b9M# z0(+sT^2!jP4%RLQzjGo!jgK)JTq)3 z1BZHQ^UfB0uBvKCKEMk|7Tn(GJeU!mcZul;KM1unQR{owi(nrlyGhf$YZQNlZ&-8I z45xo^7Q=T;05S?!2*7}UaeQ~rIomus{^+#--PWT_0;QXkIlKYk(X^#T5>&2@fr}GG7b>6L>wIyH4k01AT8-yDqIU@)1$-|ZHf>-e@mwgopxfy*c89rO2 zTwQK;-tvws9_CisALYF_NbtgdeK|S@QXYqxN_)+5*@nJI+ak@WIABw42u@npRaBFm zi+~t_3yAu{9DPmW%el2EC`o((Qf>eSz>T!pls?OUBuS-#M*a!bMUue%MIw@er|DKA z`ab2u70Ilzc&K$s0y4!T!xr}mv(h3)`27!2M#R$Gq61KuPe3;i^0!t$ViBxP-7H@c zlzGONu@23^8NjtbGi8h<*ZFGQ+=D0Y4(#gaub+3Ot}AztFCHO8rP5c1Rbk9TUsK9{ z+XV8XtR1e&-HAQ%Rf|=PA9?iOz_8)7pl^+RW2RKi9|thOR}^5Px;IHQ;EDi}epuuL zd!1@~^^|1)+BeNtk1D3jOP@_c%Ao45==YBB*|(KZQMv|h*(@xppO79vlWHNV$jp}N zdSM17v5IUZtdO6<{sS0@oAPQd74NqGF7S0a9k^O-n*6lV5Vy6T-kxp5BjEEM-&>wT^O|BK4zTcjKWS4$NyKRJL@>j)zu3(bN;In$o=jq1GGQ^VfzWmQ zs?{dS0*^9<;+llE0-%$Pp7!&0WXyi=pqZgHRhd$A-zmEEI}^|8noSGr{D!#AF4+xP z4PLY`7J|Bg+TV@%S=*qzdWw|&jm@$etxx?)&b(bhCY-)d=W)&o!H;IOp&D3D1i<0R zvv47z(+n;9r*)T7p%x!dmc6T$u6B4&`Ld3psHOW9Ya)XKe;rMI+Jh)m9%uCdPd|=k z%O1{w68qv+Kr@vNp>zRs4KE$xr)Y7)WL7&_tbd|3$q#+J#FDYgVm-2RHwI;jBlAt2 z4V?ovb06B2jEkjPI(@SvQ2V%s?MlbmCGTg|HlAhhK?DR#HUCVh;<|8M_9lLEGx7hU z>aC-qY`?Hk7>4ea?ru;(=`N)kq)Vwmlx~#n4h5t`8etTM?nXgG1{fGXB!*^C>O8!^ z^_}ybe^|2?3)ZZeXW#d}VqaTe1{f}?OE8;A&Ru0a?I~!KI)v>a5pTawOFwg&d4DkV z-TQiC=fx@EwvIudSj1IrLU3)<=O{m_QZpPXc1KemTtub0LY<}Dwvi^6M zYdC{Kgb>n^?2}g*Xh^Q;BM_RCQEVfMT<|{lfji)OaZgy#niFvRidTMI@=+U^{`qQY zNo2)6V|R(kj0zvbS;=O>s>BI^&aP@I##W)#XuzyS5IlVI54P0O?w0(zL$zWjxP}lv zivRfiH^%u_S)7`X{YgKvyPS2IB>?os&8Pe`2Id*+ohyQg*+VFzI6??SZ)dO!%JbcY zCJZK|9dp86TLCv4$>+@{4b2kcKG=xhFF1W;5w+JM{pxwj!6R-?lDq^9DZS>#nrH-&+kDSV!k+%5p$sGAc3Bpmt^c4r?NS%B9a;aHEBk<@3H+y$( zPkOpA8RK#F<=u?VT~2Osz~h(k<|qA&7$^yV2x8Xw#1vH6?zve(Yg_BE*C16|fARtt z{C;9`Ny(#orpPQ@g)h`#`WG1{r7IsG_rzc3Li%UpZ<{nCbkvy}I_l>W2kpz_Ge& zdplpfHIhVErm4ZnKuj;D`Q)y=^*!Va{}C=6>+QJuzGCs2JO*K*y65s8n=%kd-^ZRJ z@D2l|an)IUV$(@G2tr>^XXL4v(X0SmR}M0`6KjxRIHH^(B3Ru;@NEI?ktP{uTOK`= zDF4Ml>oAEMx?uGWtm1L0a1{N*)%VlNL?DlC(OD=z*vkm+tt5O!0KWnn>Q$Xg%8dH) zZR&Rd&J6#Y=T*4(zeg9ejN|$wjt8h0=KW%f_~v{y{>hPM;bKkcjouGP5!3w)wH_i&_~qalDd0kP7wQ@Crud5a0$uRa z@hF@dqE|)mRWOf!%)B-SnwklwnCEvN3x4stWN7|1|NZ`)jxV%sBiY@o43Ey!lXu2d zK$t5bnr+`w1fCfd*oR2SG)#^a3si7snTO>XXCS`nFhmDqlDg4%yS|9$Gi^yX(;?H& zI1;Iz(@NkJ34QMO=NlI%37EQ$L~k*i<=tT9lMsr5=cGVI^nL-U1!9Y%;*K-sSXCOr zw}W0RfBta#bv?=+`(@ev5UOM?{RQ3s0aX9(aS^${u5M{f?!uOd;<2iW!$c+DoE!71 z5F#zNt|lgvu1*q8UJe2g949xSmutC)br=wddY8yT7>1YKp-y4QudTgG>O(N9CWebu za@UN*88ND4nhiCzeu)OHxbC~#429J>IWaDO%2?^cG3PS!A?^L_tlf0K9;Vf9OSYtY zh@b8+;<59W0BOD8H3aUa)kkqIj2%$a7Yy{L+$st%R2AD!-w1^r=nG^a;(BfO@qZa) zL*_YAd{q*a_(C1We{cB}j0t+9_lLRKOoL&ePU!-5uMIy_j#}Zj{jG)``Rn!Gsd*fC z=NC8v$a&4T#TyE>NMGPzGlEyWd6)1I@Gc*kOAD1k^Y@2M7$>EF3tSkIw3 z#oKV3eIWY~{WxL@Fbdn=gni@W1H>7O(EXpy&Clyb(*fqIdi=FqA22f}%mzNYHsJfp za_9aQn(e^Uisx|vK6=sxp{?4|mgE`ZfAs7oJ~wbMi)QH!n9OKWO$!{<*q!IRKbzeR8&TY~)>k1j>gL_9 zUvrYwMQ?=H-_*Q}$75ky2OhPjX2vV`Q?2*ZG@3Ccm>GB)BZ9Fd>sL5C-oI|OT;K6c z$+2BIaz|$ma19X5E8nYqLR63T#sBYa>ay4vSiE$tH+~C~q^H2um%p(SEOPeuc!Lk$ z%wCd6`Mgn--}ML^E+}G6vV3`9iUt8g%|GFwUOAIcK=13R2q>A1ov&;sNtk$}Z+*h% zGjW5$KF$Gj$|%xNU`+`;m~N!$s38{EUchhkL@EtKiT!QO7bqu4XxSC?elqdFw?e7{EUx(Kyjx>8w&Wg` z(0!&JOe=v2+UOQF|=3Jf@AnZ_&(vyM# z#zH%4_RBuN;Qg~1mnNcw%p_eoOJAfkjUT$TjyI|ccN)T7*^u%dm5gt$Sx&}J-z}XB zqp-(USBCgm7B@S97Y!8tTwX#T{1jc8vKDpwNs<(%S)@W)+7mX4v3Yh*ysOKZpP0Fp zuZRkeno;`5aJBd+%-`ir(KeO}Vpu3_mkk<6+#P7Yd1lsw5$e0{K@Nt$zpvH+G&Fyh z@okC~Mx?K57|wEwel3!RC2r1<)+ybcUvOKrWUbs~S@|YYvc~i2a}-b3{79JT6D7cU z;W)Ot|MUiJ0O%sERZY|T0WW;wIXSts91M@-K20+|g_gZ(2Ql<4v33-_s{3_Ue0!q* z#mzxf?4KfuI50Q7>9NLf5S*t4!~NzDolb294qwDbpv^$p|1}hM$8tZDySr-9D7MkN zGgl@jI))u_$CjxGYRtuzy9#2yA)kc`c|n>!3-x9kF9PI1C@o}xph#eN51#@o`2>37&v14-cmei zn4Bo&sRw3>0VvHSszo+%&X;(l?e4z~zjAcl5PCp&SEXahPKBl!$sWBduEXQ1JholT z;Do9O0=nIx5V0ib7(IJ_Oj0m5-0|8w^0bqMBrV=x?Q=7=^yB zCXUx4?N1s)%(|F$$u2BUy42Vi_W{g>`fq93pZ}!(bK6T9Q7HdzVjGD1nsrbZ+W2|r8M*s3l|A=m{bjh@lN-EnY* z2k3rr4PF$Q*yaN7^!m9WX^&3K2pIuj%{n%oU%H?11sZQxy>-%{5gmfW&9YqioEJb& zD&RcUDozjG0tRHSl@5KOz6vM;5A$rHjt_o?PewoXJm3d%IG!al9Z3)GxAhgLis!c}aW#>QaQpqR+`r3`8gT|f)NYyW;XvTD++;S;j1Aur1%W z>n9jP_)xk&o-fkdvk0;b`##>#CSb=RP&9&ra@T8xj*`LtovprXk(~QS93A2^GFN`y ze%?KYp|-r^x{DX;wl-&6KLlivhVsbOpU6v4EiIh;9E25Un>cc8R1+f7JaTM*CY_>` z>d1j8a>dV4tO*$pp`~8hHz9#9o;AFq%N^du7GJ7s>6OEjqSDD2pFw@{mQkcbN{b9<1~uge!zXHJN0&&*AmLbA z;#)-GD97T#zObxubT*rkW)NKFrHmp!-9j-4ZFyYjOyH9kIhB82tE8!n2ae(JnGj;h zh}PcuQAVvArKxxhvZ}e}epb+%FX38ZKM*781(%MSpiXV>N?}Gj&mGF;dKJ#okdv3k zSxMiztSZ@SWO8f1NryjY13>RYIK-^*m!HY8zx%ic&%MKKYIhG@<(1sk# zbPZ5tWO!ioadBMioaUMM*WA^?t7$cO&m=6X8;Pn`*I&SZ{vH%!zl$Hfi`C33JfC`c z88|#M&i7)znmnyQ=f@|DZ0Cpe2921Nadw~kU1Ahkb5ydctwGtm6K~>oA9+0{779^q zei)A>$-^2wd9tR#?ZX=Ve)i$(2W+Iw1CvFqh~Y1L+Ksg^1L}WM3?-9=lrCV#UrFCQs>F`W(h0`7j61{^HEH$(j{G%~wH zKxp2=LNyGy=s5@{YRjJPWFl82HlC8l^1v5pgOfh75yMNiFkr^Q@sc=!bJL2(ezE&G zzzRnwX*^+MkVUcwE^2@-OC%iIxhnC&xbC4!sw1sRD$-Za6Svl3xFVGmnPHP=loLRb zB_stZbawpn03!0+=jl>ipHH7YWvkf=cCQ>7DiZNF(m|7vjM}0mge1zO*TdPCHWX1b zB3|`2C~^oX-#?ps<8t;4`oI@If#atYhLWuT8^UJTFG8_Dw%JKW5!g55l?j$)8920% zPvXMVC}r1)vfIUO?kIVWtG!hw7i|;o%8bV3mHEJ#-DsBEV#WcmtYCPt#w&gD)Y8cu zO3i*?MeS32k6ZRZm@EVL|To;XTE z=1(ST-T?-vILvKbk$|t1UB<-+QRLxqf;W7ps_C>ClXTD<&pQcCCC0h?^k&Xau^38C z6-+u^LZHr;e_U))@qn^}D$t+P%lS<^j>W=p0(+o8y(v zqTD7;WayKjD3ZD2#NAnGgNEOJhnl^Q9CDZN_8`J02kN;jh?L)^F+OYH<@9Pccc2N^ z<(zvk^v!2EpTaFWLma7)&N}MX%bSz1L15WNrBO#?@R%yn(G--Cd~_YBow1Sdlm8v1 zHoO#W57%&{vybo4*^C4>$emcYD6y+VmAUG$Tc&@ew2+M-w=rHgQf0{JUv)T-GMeoF z%_g5)a*!1B4vUB6x*O%sYuzp?@A_x$RP{z|v7#RD=)cPH{E&OL%Fx2~;A%Ah8v2Ju zhKf{#bVs5SaH2M5t&&dd-w`HZJlEr}OtzCdnnDu#c1-y~r50ywqQwqInhyA3miRcK zf4>ZP?s$^X(POGuk-Tftu-DG^yZNPj$}E%}=-ZzZ;R$}a5#gyi8TkLJF4%o_U9$Pu zC2BEhlK!3p1W1bwk>R#Dto3-BnrpZ)Z*~&hNvSd&+9&=vrX8#0q+6@^nQ8ToN#p?l z4qT=Rn*T_8{_5?G#QJF07o}+UHx3DEPhu-97V|f?2&ZU|}O~~Y@oCzpAI-;uCSR5DSUiL0p zfVs!UhX|TGL7@lo;%9@DiDWYpHs^75vq+$l_!a9Cp7Y@O`17`hY$eZez-6a9+ez(M zIF2;9s5<&;T2Tb^7=25-gt*NeaR=v#voJ(aNFDJzY^F zD_H+@DIm5#Jm zlF8=0N)W(3`ZngolkFWWH#RJOp(V&c<)TuTbrNIB_wbe>6G1gZjb=~!X}>IWE<5yX zVu=zs2AhU_e^0*)JQcLl$QO6Eo5&IJcw9p(t2mTYU;(+-D6wr!yP^7U3a40W#ahR0 z&gY5Q#nx(`=b?JF(Yv{j`O$D_d|aZ=Mqz5}4+rQPoen&dEWZU{XD7eu6#9O!bnve* z{aw-IHFC7*gc{!wsR$CQ>urs_4_H+G}gi@Ssme$ z=lcS}qc7Kkt4PTDPc=7qjhNH)?uUZl2~p2tB56zsg%4vXRdzLIb|(d6BrMAd-_7i; zN18o43zFTH5A#DAy;R!zR^0vVY}NuHzjv7`er-N7DzU4H8t-~+udc|1nJHb867$$o zy*DX>4)cHIRsI_qJ`p90Z(RoCI|+C0n9)&-n~ z5~wnfNFoTwo{t`9I^r^xw?imLJB*5jh=)BzqBD8}hYsD++?6E1!FgySEa3v|MV*&P zw!`$ga2$sD2s(S>nh(nbh10SjaO9fQhJa?RQP^4gJGz5@+>>&)%q-71OqF$0iM6#n z(pbI~t(i4u0|PHQda*ZgP1|f4{pI4XZ66zl@$z%w1k@W|ov@acKHdh6$4bkX$FPU-(07J^t5-|Rpa)OcQll>AL~6~Fh1OEFv2-_mypa=Lo#aL$ zCC6*HT4cKacCq37YA%y4qn!-PByiO?8Hom(@DpAf)e;J7`a78Z`M{2iwkcnqQ?|0j zM*hAMWp`?9UUoReSgd$2%tH|0gL|0{gjD0i^AjOfTc+dmq64ARM_9$>zomiw2^|9? z9pZN}vGvsyjeo-Cmj-72B1Y9!yFQBPlv7Y-L|ibh19b_+Aw@)BxZJ3<+m;7ql-uYt zn=qegLG;qVimG51TJM3wd^K2%CgfLwfUHZ0gVz9dSME1o%o>J2)Vhp$%`V)ptCwsV z3%Z;e7mvsKvp-m|j-s%8(zPX85zBo@ow=&4SrSihSoF_Ud; zMz|ja4YAE&E%%}e!U2JuHy=phj{IzT{_*N9O1#srI>mA!Lh{qUAd?-1@CX&pylmT| z`_jq%ZC$AG^8YI-@gU&-Qv4?~!0m%ELl-XQqQjVWF3pYLj`P5s6|r=ZA$w;L1@^Mo z3{8$K{P|Xz?>tIC%VWN4ni#7U@MMEd}8LXn6=5=k}Ys?piYmJ|3E$jHN zkM~%{yAIxBgH=+4X&qCqkeWdsPjy-I;-EvGt^}Qq!*b&Ri3A@s={gjbc{P|qvC&xD0o22O*1RYG8TFdzY<MIvUy=&_rHLP@%Ku#0okR z=XIktb1}`h)L3E$NsRQt$qX^}-f))u!N?>Z`S1M(f^>h0_m`St-&DB$Efu{O@kzr! zd@&}(smIdv7^A(c17-h?4Eo`DfA(B2Q59!zU=P zvPH|_P;W8^(2LTcJBowgv?_?o64i^^Sk>W*ArX2lG90^TjuRBF0l$ShzXBi9P#7sM z-~wmBlmh+$41ceJP$z+E)P(S4jgNdXHekE^6U^J^Etu`irnLEH2A4}tJFdW~ygUFC zG9%R;W%&1@7hR+sBQ*CIN~(Ge0uArsx0tCYE8AT{$(#P9CU*%mz4*?ZV^Alk|7;Ws z{kw>wqn04vYSNE;A8&Iw>FV44t=cOW8DIWMNm4RUpIXT{FDMy|uiX4$EMXV>Qm;`q zuJz|MRTc(n47sWWg65itG+nl^L@-L0$2Sk7zddMUQe399K-h32Z#RvG zvI|y!b??iztD>HevuQ-7G2Q*KQAf8C0JzOZ72bc!QDMWX6;<*yoZY_MX;_X&DeWL4 zm|KeZNv9qNrN{c0Qy~MfbIZZcnXP9Rm2RT_4Q4ERYa|$7zO%QT$O(N_`h1`n@9{F7 zZd7KocDf0^Jy2U8#8?4dIaZPNLLznZQsi{9;I#;~Lhj(Yzn?+|RqL11chRna_JXMz z%(4CgU5SH{;7n=OB;$WpWB--VnLp~AL#hf&AtucG!sx4H!0~VWKHE>q<;JGZX3bl@ zC*&`G|L$2o8Jnnmq?HMVUj5sOaCdD`9txti?~RJ|OTe-{%foRcd2i1UD!Py5cBX_Z z$FV`o|1JA%ubs=8e?vVI9uNuB`t`=+H4z*ZMC-+Ff$VQ4gdBOdkhD;dzldJ^sK@R~ zrp57Syz)@0fA?{n#dmN2V1{3$JBTB%c|Tksc`r=QtIA12spS;eu2T%0N6h7d4T~ox zmjSPK2i2D1AEyi77sGoZfXG*urm|Qd- zt0wp>u*xkGgl#v&OlWaufp(+76^*(cdcZ@%%RIuHeJGeV-a{Kqs@ZMLjVy|Ye7Ym& z34ry1+#CPnig4;mn-Y=yQE?O(&NR!wEHbj^E z6c!N(HyW;}vf+(%#or#9RO`9y3p`CDp`>RSZ!!A-W=1oDGYz63sy^>V)fYl)BRev?N!?W)z>Fj^L>f|z(G}lwFjhvP z;T@iK&J`z(Y>;p@8YrpKGF_Lj0G)L%+^?42hWRnpt?U2upX#+4R zD0aZuqFQ)td?I2>ynBEAGL;Vf7<_`7doU4ArDJB?*q8LmAl;;KeXzdN?c))FC0`r? z!`%m;YpGGFYjaXna7`YVJmUdqq(4Q=Io>MbN)-%jY)}s9`7#n32uv*ljE8ib%6UPk zA!nRo8xvk?s%X}@|Gv#vTDU=SWkDb8jA)4jamv8g;G}-V38Tn)K%(I@c`|UkHp;)n zn(>TM-)C5pOEm|q)Gkr(kxnec$d7m=$UiP2z@H^eXk5w|&t-85{q7sz_hyXR?5pVk z!ExnA#LmNti8ucPQMpwjQ*VMM#;1-ZCT-1o2(tNzD9t`HD%=)j3dQ5cL(Y0Qnp{@t z^@9iNI;#Av*b!D0N+#-5c%f9a?nRjnq*qJgM0e+Nu_olq?8>T|LYO5#@fqmaBcmd z@j9ks)qT`VE1db}kau!VCU?PmxkIS!5?XQAo?>eK*put}bYgAX<@7F}bm^6=+4dB^ z1c(X%T6Q+L|A!k)gX2{%DKm@nH!nU6a*y+AJoj9_)g063fJ*UJ_o1e4V zvVjZC{HWJZU!+7t1Pz_pEhmMmW*lHd5sQb-(bQ}{INPFjg&SS)N7{+^62tT_R1uX4 zV904+FAf+74z`FQZ6*OAny5Rc%v6h+0jyV(KR<%uguZg}a9{YJ!dplEWCR9P6!}SmxKQ{Y zDZyasRxoid4j}~S`XH%}bmt%oL{S98cwR%yw=EkG1$gM>!4SpGcRorBW2e@^%!sRR zyfXF0Tt%8;bJZSfDp5^N@K`$St+n8nhf22>o4z9aElYK(wNM6M+EL`{$e2dQSkqh4 zHi&c-Yo4{BvnH8pL`gu7`Vcj{Ssa1ZR&Zz92rN%O0~{4kZT0|?Iw;iA&+G6&C(-z} za*1Jmd#X^>{fFRa!NTf5QJj^MkAI$j2n{CWaP%@9PXfgkyr>Twj!X)+SdKI@s6rU* zb5th-yEUHv>xFRuO_;5p9Y*_3h;P#uhwX%oV-_vXNk7FXs2UJ1YguvWNgvRo;L#(uFPhP;6W9hH7AAl#yu>;K`JEI2U?%fXCfH&8^1)ZeSU@0E#>8q^X0txSCH}S_gERTdEsboRd+jp((ZE#hO6mlVCy&fX{7V%;mE-EN`o+|!1bvg?~) zYVP)jF)%VFx+z+#P}3S)I(mOEkn_s~p~ZQIXDD%eVMbUlO}esDa?GRk+o@o(*Ft!x zbslQVA^}_B9hoZc+K3eo9?5XsLj~;b3PlZq^>fuY(q$`aOp_O+U;T22dIu)3>S?qps7X&i_b_2zH=~CinhdLL-AA^1%J}-62otjHLhO zlFXMYi4F-4yZZ7aG%-Wj10MK48IlTD%h;$!y8k}d`on(Ac)C5Buyr)wVZ60Kw zfjTjxK{X*D1Apsl(pu^o9JN_m8)HE_^hxhCY_L~|S}zQ%H~Ew8*;Ezt2$!r*iIb1~ z+5{pW??gE@*%_aLvi3~n;kZRjH%2cc<|9dLrIZHrm}V2Nne=ub zZ@ut4cEt&7&|)LUe7OI|W2^Pf!*0iCeh0s(hf`U+7rHorB*6zd-fbVm=)(% zDsPCyBUwr|#`)O2w3bL16_+v3pi)gA)_Y-dmoxK(%A|_A_{((vaebMk$Z)EZJ3uSo zxPbaQO=g%hYGwG->TK~KF7+HCrtxAE1{>#3k2E0|I?&yJ4xAq&Ma^z-3TlgNvPv6D4Wt4^$NTem zD@G`K;P*$vOKVPknmR$pbNU7flkqyvO|@6=Y_z`Sj$2&5W%xNg8wTB=F|+0~{gkK0 zI4gtsC*bj2s1wjl9cw-1cpe_WP_b0Vsnse1?D;t8^I=!bqd@WI zU;XZYd%X=-kJ;enLgyk%ai>faFZ2P(zX2Y(_q%Z*vOZ@m!jITpaOY7O?fgsjFz&^l zlMn+ZH*f|ON+m$^0npOndA;w!tV)xHFdSewX`?`2?6)f|#7*A4A@Ne_4Qav_r8S*5 z*9r|_sb`jz2v+>@eyNwIP%0oOm~9Euq|?~xE05c0ZeE_7E8WLArxu#2KXMEpf9w`d zjW7iQZ4;p_h|I3^>*86a#`=FG9qXewFI%wEmAZeyq!%`jnmv9>VJpx7 zE;3DT@=O2oJG-GQ>QBooDf=LAzYqx+Q`ErC^Cv-k@8ux`1%59ITs%e@<>QFDA^}Ep zC39kJZPo^$LkNZoa4&Gr$g(iKMpbh2?jDIJI@<)~usVk&mS>cafv=!12(cr9c7?hYNStRWgf~LIreF{;AY+s%fSSQk`;Ye#6;)!H-CVJ_jm6zd(IImaj=(@gz!w+$PH5) zx|$#C)lWwBIDvT`V9EdjFLzudni(AiItLZOb_c{!LrxsJQPsH>MLX_<*8H}^{A#B7 z8?qp@ze7^JAy9RqE|=S{r-+?pbqKElA2?3SqR3@wv)$*ap%lJq<47AxWKq`-e9agm zj1Q}O<@;d^wI=#ix*0FWqA#x?{2OiamwI-sL=e4pEPn%l6OiG>vTo!Bdc$E(Jo;9* z>qcbeynnr))}Jlqd9^fbRJ^P)okWh>yEv;pO?a&rV&ayp{)Mi3ksWJC^8p5|O{)FF zx?|29EQ`v_?RqBV1f`xJKywbF+n|Y>s@^sDSuaYZY(-$u95Q1qroJKQV^MQ$*Okkh z+m$=+5`Su*bFREWW8)j7b_~rm6)7=KDAqv@ue|fsY-4{_P7V>NgZ67d%579Gw1T(< z)2Yl-Hl&}mV>z7D>q?3n;(wt@P5{`i^e#@*#&`|F!qD|IYX>6u_rI0hKe*08K;z74 zfbSCrCTAG*a21#NbH$=wdN9e>2GJMr${)mpNn`_8*niSgXz@tf>{ajc5j)+($J%D5 z2yd^Y(snAua@+1-VM~fGEM4L0kaX1VT8p0EZ5GMB2HJYoy=aPT9mQq&dLuY(&OS|L z$V0mRv{{7PF(6%GC(zo7Lac@BB9^SXTqFhi*eKQ z_p6jIrKlCW-9N97inl~QSnQESeh?}1Aa!;*F)oihd9W#;p;b?pEj8&O)B)ZKs*3+q2er0w^}3HrH}HVMlE@C_S7|jX z6iWoX?08lvDe9{GZNeP{&>r$t;QL4V6~_g)b;4IpwJ26DJfH!lG8^?9er_jois$JX zCz@{SaNXeqXQ*3*P`pn!4%^Q(cgfz^+f8ie-%_ngPsVnVu~UuW*q{j&ar$KF6uPu@ z$%5|v>4%XmwIiRsD94g@rkOA3!#-j?a%f36BffuCN6YCY!%<~}_<1RU1(!Gn(cf`t zkpVVHWhWUGPU(aSBFIdfm1@+4p^k{G$K%M0Me;5wb|p?bD@;6iCc7dRCrbhW4?{Xr zMyuWAM!SS+goI&yhDnrk>`OeQ>V*4ub25v`DY_yb#PCFNhxZ9%QyL3^XVqSM<{EwTzpsLJIVOa`$u#GNmJo|m+D8QPT zDgRizCXQ0!=?&&>$1ck`y9?W_dyojPmCp4irIRa|?Vji7`c)##sfxPnC<;@b9Cq(-gs_!8`SX~8~FR1sIyHXeK+$J@Sf;y$LJZvVm{}p7lL(yqC7SM_hP|r5$N_d=7K%iXX1>T^0qgS-AIgM zaXs=80sj(-th#u)#TZEeLh&R~5)kK*@@z$a1D79_ zTS6Xhi$@VlnXA9lJzMx8fUgN+W!2`^5k0CI@qtlgY3EEslVlT@expTo$RvcZTfV0_ zSu^*WQ~XqJCo5I%y)kKH=G}V7jLsXYF%vZWi>cUsOie7vI7%NmkC}8`%5)CR#X*S) zF((&(@1m|pgC)8eb39bBa%E#3g$Qm#SVu{y1a`_*B*Tiey8=T zyIOS^CJ31+KjCT_{Cio9+e50(85dp6eGed`z^(?3V(lNA%_~OXI*;*F;;c%Ot~%XF zAvA}f;9m?ii=(SL@g=WvYZhlOiyx~03`@;AwvB>v5yEYUj_>$W?$mc5&C>#3FUQH6 zLWjz7yQlrd{!;sMr&F5q0?|q^TY@v0jY4^p!A}?&)dlaTB^ujm4LG2!$tu(BW!}lZ zN|J_Ps6V?UyxC?LxKfcHHOCZ7UiQKOPAL zz=MXQpIDS?$<(=%;F%+RyqX6`*4LlB9rh)J)ovM51Es2WFldITE~(Y7I~mjebc+zbpi%8Gl7j_ycql&piJq%&66#7;@UAw@naAHNOAmNZk~Co;lUSfaU{JRl#jX0j$e4EBEZk{7Pv z0qhyUu@?$u92Aoo=5B{5<%}fS3%l3w>VAJR^%JvS00TyzIB{=X zPBnyDXe@Bne0!{!%mkKFRfh+6g}r_O$fto&duKVdi=XEs=0QZg2FqzSv-C=VUjS*;-OhqVymlpHIsCl zy4#MAkNvJ5(aQGVL7`BL8-HJ4EImEFqsi^~)uf7#!F68yUuXBmOeUdS33Rzxu-#bJ z7#p@!q#T=O6hsZdT=8A)T?7MNkDErt$@34^0EZ|IQQ^e%{%9<&y`?ZkA>z2{lZv5` zIX*&UXu#I9BjASAr-;1j0wHekC6#-VQxo3|SC}sI=(os$TXWn2OF$xTF!chV<|`|} zBKfMHr4EM~Et_M~RfnfV_LQGxjM%VOF2w?@3$U7y)L;_0`%sL?GDACUpNrl$`E>4! z(4r*S_ymDILBb`IG@qG>;7>YXN7npz^EF0IA%9=(tioin1aiUEhZ-5#l>AEbiut|< z00En@O3l{k`|Yxk*(f?XnoXRa!ok7uC!u)|zIzTWE#|V=!Xm8+S#54?hIdwk%dIy< z1}osi5su+!V}v#)eR9P(Ui6xE%R_NU-F$LKO5k0r22B!nF5>#Z>5j+SYqui z2Nhj0yCsRCBfz^SM;M6vDdk52_1 z5SxI`196Z32pDbuO|?>l(VS;(?Z)$lEXjo#K{ltY;#0$IQ!u$+1wM%9hQTLJh=Mf}0g4aII*3w!i-xf6$h@izh=#N0}( zXp@1VvT+b6$?RbuX!*|SiiL5>i|#aFA0Rx5Z9f%yRf&pMp*`^6)T6aR@MgT0OPBS+ zuGyIh9+AGcXr3ndhli_-zs|kyl&{JhA-%fba9aB)UIMg=+5qcMGJ}bLJ?u3^5H&Ld>#!jfuK?hD}Tfl*f(KFp@_tbK& zILYt&rGo%dArXF7nB)7$0XXCHyjaZnaz5}tJT_952lAPncES+!$)uaT!^mlAw+cWB zyb($NLz*%u0|2)&5fJ z2tQ@c1u`xObAsaOprBHA*I^c)ksWe4nT(2R5D!$^UAXtn1&o(An#n`j<~9e4A=Psz zI2G(;XWoG&=SP()FVz9py%dI~vyy1yh+Nc^a2}Tz;40I+KNFz2Q_#u%{bY-cja)Gv zGDQ-l)L_7rEbpRFoHGfnO44x!qv~8~qNm{fk>aRU0>Eop1d@6Tz-OmYH9*=C!^eXI z<<}r_=s7N!)5=tm6nw0sFb*YEzYC(bRqee|VO+%9G{Zs%ji@%48xkIvscwPD0mq6G zm$$J9WV+HS+rx?EEi;C#F$G^NGpdxm&?lS{-;vfpE;99QJC8r%&W&=$U|y;GT>@bF-Luconeyq=+mXrmOYy0UnybWccJj6yS; zo3>^Va0zMnT&HTsx%A$^zxdj(b_ELJVG~f<&7{h*!}~`xVZ){1Hd=fU`55#!&JCKS zY3RvwzG9%3@zr7q5Iw1kY$P7HSDa_e{|M*ZElA~yuC-zkVgp5I$S^1IuG^-o?g?Oe zKFe<2q$_+0JtnX}zW&b(q$DDu$1^n~B(#KYyvb0K%lTVkf@Sxk#|c$Y_(9Roo7ba8njUN}opM7jUgeJYl_Yw~73fVTZoXKl z>|auGweU^R{I`r!sBj3lTvKVX>jhQ({j$96V0z>j*Xwa`l!g~akcuP1qBmN=A<+79 zHW>d?`%vD+B5hiurhLV-^TFc8y9kA&9)o+x?%Of_sB4<1OTorq{?hIzu4dOukNLJQ zw_NePH-%_BNNb;d3zWsf;=A5%^>+fFh%9Nm!SxCPigFx5#zEKbXTJ}0I#@9l6x;9G zpEoPc?H+BcB=!j;KJp1Moymtd%;1~CM#@vy7yNA&ew}?0>3k4?$^E8i@InVYt|rqR zf(o5oW+Sf#`Q!B8Tg#7EMgVtzYifZ~tP)fGkE$(X%dXp#yuYO*52bw(M6<0EY2Gnz zwcBapGk>tEK0e^5gq>>X#Ky<+QR6&f)m;<&*N{IX4+{Ut|LxHBX4>?uaGA6g#upar zxu*QjQoMb@T%g~h*dnwZK73a1EjkvQ3ssp;RnMVRIvF3fgCx#y!<( zUBR+{Pv}QGGlTkszpDSE3v_PTg9j7RJ0fvJ{x3U*&0V zT#DsG?SQtEjFPgpiSj6kAFuRgm{;q(407W3b!2V>P4+6#vR4`MR zv=JIIa}Q%44r<2d%of4vYmfuTE<8nw>r!@s!!SnJ50yD&kvP?dqhcHUpaP#eY0(EH zo-%{RV!Y9Xz>l&VDzT>~M6eyX;;l^|ZH&CmLK&l>5+YQ=BlrGg@1FW<2S620D?fRj z6nRxfu0am|egUl#I|gE|Vj7-{)%}p@2f>k-I>@H#**AZ-JU-b#2nWFo_2K(f#PgrHvZuij8j}7;qbSUx$8x;_u{`7DySAsL zNYO<(D=>w=7SD-2m8x7^)DGf7To@zP$3r&a@L(UA@qOV?)5YgQ{(kBzgzljP;1FoLR-GNbVhA<;qenoM?GwyUC5}F zh{#s5zkvycecbGGG*#`5=rtw8%vt2vinhjnwSEUTeH-P4>{Pk=aUS?+U`C!eNMPjA z%ON+`c$X!L6NK2)TZlMZr8k`xwEv6a=Z4uVX=|Pt>959^9PFEIVuVSB{DPwY zD{5Rr@tDrV!x^z7U~GqVKMFEMe8CMHSN{)9 z=fIuW8ZGPCwr$(CZQHi(q+@lbm>pSK3!CFT!^AgVEJQe^^-QPWgVPdcOq@xt@ra~qgyL6cl?Ut*vymcN}nTaps9Sa ziR#Z=JpV|MnW5QwyZ1N5vnTxE@Sx~Mw|$;fxCu)ZzbpRw$$RmIHm?{2wP6WJR)F%C z_c166n^DECnEF^m+-RR#H(vedbz627)9b?BL;P%`ENiIFnd4LJI|6`6TkO(!{lHz1 z#Td^1+f_p49AxG;kxFfv!Jhj!o`1&P%g8=dCb=?oC>Op88lV>>h~tTRzd$`EH^IsH ze4VUS4DrX*#p!kVH2(F*{{WT?MO#*U)kQ;tw+dPmTh#-vDU0NU#WIi!XZ~Pth)-Rv~5Z{fA2wvWa zERcL0bA_!`Y$BbH1NBhHpPDH>$F}T3KN>${>SUZqkH96>%%chhx?o{^oKfQLS9>^k#iM zo&o&&kwOw($4CY}&e3%d=j<{2809O6o-5OxMc+>Qy_f8+dG=Erow3K3JPH5%_wNp# z>E2F58VXCNt`meOZvgnbUQ(wwuyL|Lqa^8n=8q~q>x2tA(g>Wy37P9oO?!8PPY~*t{@w>qHMc~}4hBpw)DJpB8TI#PVA+g3j%w#k ze&|a&lapg5`QB%O3i;^caf+5A0}$rBS#2)Qg4V%Y0NTVIAoMpbz1iLqC!0W&@T+n2mJTT2gvq{2QRnj(#=+CEkB>%R^S@H$PxV2+G4F- zr`+AD+vDCwr%A*cTzGU}|Np{(_TR!_66;LqIYTXh2S{H-P9Qa<$;ms6my2xJsu>L8U(FXNSuQoI{$qo0~W5;X}_9_ znV19A<4(G#$}qs@y1$|!5DHA5TTbPm<22d>RV7VXq{pEKFS%;=P49RXT$*t4QjNbF zG+6vP22LLFSK8DsJygf8FA!@sfR?~{+2dc6?i5U3O@uiP&()Mb0zEKF4941j-obQI~mH?a;m`?F1Rkc=>E^E77 zq^hK1oqyI!)-77mMq2Yq)+vDkq&Mv`hh9`S&=7zRndOJh+y#b$f&%inrWm5pehNV? zEx6~pW+sJiHFMdMU-O5c1RJ<>;vT9p8HyF=1M$M;umd6q_)Tsq27vAX>%Y^N1{0f= z0PKAR6_DzR!W338Qv%5lNv+Ge%qyfih*!Sdi$KE%vk04E`sU1rro%wLFl;c{Kkt0Z z0K@A~m4#*^qXmOkpVyVja#F_Zmaj(ifIr`;JK*nPEuqI-x>OHr!v5HI(@quU@n&@J zc5Fg**onY7Z}HW+T@a@0x~N1{!$(Y_3pI&^VS)&eRkSTS<_i>*1O6r@&`xKsxT;u)G=K;EpH-PN=r%FlhPu`N!7YBTi zbh!0iD;pw3HvMMgGG1&Xee*dxvIRwTPAMJlg{jZ}@+kzArvVd?MtA^}_`h+`iYBv({piZ8C@j0VfLA*OR|z|B#}xI z0W!0$KOSUK|KV|_vCh2n@$C*V$dD?9uIKUpj@fQHcuFCatb??ENo;RuBH@uIHqwE- zwCSf{T{M6F8k(#I-2;SB&%pr&^3Cp`NqR{$IZ>#*>&a=LWu*| zK9j8z-S*Amu0U7O1xI&Q_NJ?g4+<*!6idqz28a)1ljr~WkiQm$)JH+8!yMX%*^y@ESjd-3ZT(@wRkv)0^6pKKgtZZ+a z97p~qU+GQwP%-c#-Y*SQHwIW;X7rQvT?&w4;7qmZ5?w)dkmYY`qE8`{K2!OQepq+| z8BGDz-n(BuH_Y=vumWc81oPN9ZhrcnysqNI(Y@(uS$ivOJ@pw+0RaI{yS>03PA|w} z{J(m?XbEbBkUf2O4e)_MaP?TQLye^GuhEsZ^?7CguK$vY6M+U0N3hat#r#)GusSUc z0MX#p?8Ueo%l@NT)pP8x!U^)M>d-EcmVufc^_jMaSL)C9IzkujQ^@G8l*Y~L-N7FW zu8cy)fu~H|xVSV`@;&@b5x|y}_4l&<$sTv6r(-H+ztWPHL;`N49G740zox#AK4T;)q?A!e8wseZ2!~=|A zvmNn35DW1fv%4v4IYP0-4@QB*U@rA{0A>T5Ek`9nv2B#JT<1-W24Kab*Cm$HRC*5MB`@BOVB8ajJh2z5jjcNGwab8Houl7({uag( zfut94-|7tk!Bc{E5h29}!DF^`IZJ3OGg$%Emo36s5VejE0^x^wm0mo;ViDc09~88( zj{m3REvrNZU*&7B95+v0OqWQ&{d{~0zIuGRnCE`_QNyaC^8UOcrYFdj^In%Y$E?)y zfI8k?4`q61Ju*Zz=i8UfKB?520n zM{Dr1Th7aaEi;M|-hlV?As{}O@Cl6;54r8IPl{)C~Gn{Y4GKl?&i;W(jvnHIX zo_&BrJ{0@C5F6<~l!I-D`|5NUIT;^)t?s{bYU-(9{QRQxRcW!oS?KMfK&w?w)OfuW zVg8D{q8_Je(iB~T3qY9#n6nXTW}hC3gCz?_W}_I z09H$tN8Mwu%GL=J3F}MgTb+ubJc|;BMNo9(Gd{p|F7s8s`wvlycNV-LkRJk$BSNGD zPm2Nq+g!OV1I|&9h6~sbiZo3p>Z+W&in+hW0V-=3SVJJoyArYo-3YEYb@XF@r%4@p z9_7`pArm2)mEEFa5Mb(F@PZgbu{K(*-;;}1bmVpQ9*z~-tv;BvWMw7>mb!oaSfbr^ z!UB15Xv>f5bNA-jXBwoy^2OVU5ic{FRH2Jk83Si4N?uZ4N|EVyzrbj;TT@XUDh-`H zpGh>fPmjX|)>V(Is~8qZ>J*0qq%HLU3u7due1_bQ4K9IJn zMW}w@wV4g9TEaWW)eIvM4++Y*egi(jm#b7AB~iWb(F#r9g}p%}-dnO{T=s%52iP@? znSQ|oKpc>Ce+gSCLl)JohbYtXi1r9qEzt<805h$pR@a(h!>68r8d-7c*M2 zBZID5g+rBW3{$uo!uZEW#hA>M_q_1$dv+NnLpq$nDjpGsjn!O7%wMspn zasHDmD|EZ&X<5&*gUPV&Q>k5tnku-NT6_Yt?{bAt#Wd8OxBj6%41$M~ydphJWwm7= zJyzJ5jMdA_ckDW1iFEgFZwrzNnGmGy;UvV43bsa*>cCw{)b~T0Q32TSGGzqNoDNn%3NU7O_|H}3;ZTlMkxl4jSqjn0=|zJag?-$2H%poI^G zem6Dtdit`ju79Sg6<}-Exibr%EqDQrvg-6hCh~QtWtz8Fi@EW`InBkbmj*464h8ZW zSTrPN11EKOZ!SXKjZx*%MT+de%3U=l>{ZQL_2gpbPsAYhNt#pI@Iy{|rAjS)iD^t` zECD%lwu-iGq^6O}&pH!!xj*(*9Cj_l#hKkTl>U_6yzl-oS|w}OC*m`LGdGEob$b2Ix=abw>aR?Kc** zyZ<@Nh4V%*NvVO-s8kmat0e$JFxrZ9uQeHhS$0@S(w8p=HV$LG9;eTAzK6BQQL>&d zqkhC$QeF4$NcXh&pg$9*#0qA)7oCpO-XDT5N`5Jjlp^5wlJiQm1VrhBR?fGV!UwAo!;l*BxgfkZ0ZATFfWd%d zZ;6tN6>6kuQr&x4r+N#yvs0O%E>I5QeNK53UVck7m1Zpfk_sIV@C3d*bG_9eI+B0W zhyVDe=29P9?EYFgdd`XJ6XvO}+?EjIbHW88!EPqbSZv@bsK4A!INhhgy&dI9 ziF2kVi0Q^P;tYA^hr&8d!sw}=FUGsFdTw-}{~QdvK}!(-){natPu`C8$j|%TecwKV zVlA2_*1v>rV2Y&UNnS1$ z26fNBbS8Y1Zt`CfFxnGpd@L=fwqdWDc5r+RAU$R4OsU`IVAvsei%&uf{BfcGe3~+> zBxq2|T$5ajpg1L`M^_#ut{}Rz4EXVOGW$T*Na5>+q?3y&oI5T1z2Co!y`5nU)HPQ_nPu{vz;p61hskU9!Fv0mf)G;DZKlCc4a^<)bF{{1 z1kgRZ=hwrQ0AIobyAfF0{Ux(V-Yd~&wlZ~(gsEmnZ&HaL=Apzr2t$7G2LG+EhiX6Q z51N=iaZ`rr55BsTM6UlG_M$aK0OfeXp0}SwVLkD$sSHJf<_-KUOP$?UgM0^Zd7@~c z{hTMCb$iQ>-v7&GZww*KNRmcfo*&bIhm}_;it*NyOGclsR~It7|1A5b|W{a=K>GbD(iW+XY+Tmeo0%DTN%-#r=i_vI^k-d-{01_mBQi3?!*__K3I8F}C20IN(2B97pH zN$aI%S&aubM#IJfS%8Q%NyhmM*@7P=9*TK(9g|}njid`t&y}nz1)WmO$f=lM#@|zx zmfFq#S+$5Ia^n>E+qw_atLM!7kKJkg0k3|2%ROx-?pRvDWni=38vBTJchx&}(RXS8Ez~|+oNotthqk~60=xGB)b)Wd)_(DnX?gdo0^gbD+#ieO zg*s(st4NKYwZ=sGkW=pDYET*pGm;H_!GxQ{D*^F`|2Ug$OT!UBDoOUgRMJ6NV=pKz z?jjJ#b+&$9dvFGwCSz&*Ik*AIQ^+6`7fJw@J>+L5W!F6OFKG94uOt$2B4%dfbM~jm zgc89I(6Zs{N%1o>kk8;GmX2zJHj+#BGRRFp>m5j6J(sm$s|qL}q3uNE*Hff+uNZQYSDWj_nm(yqO+Y7d7n9z-I#F7#Em*G9gpM;EQ2pzz{%(QdBsSSiTd6&yQIM*OGYfl2PutuJ zFxRW?IDH@NMx)nK)yxcfmAm?41tVd;)2x!EH!F_rthGmGctrrW{;bRg?=o!E5*9&+ zEAc|bpdBaMJAh#!rZs`?A?yt!t4%vp*p(bbA7q~y>%3a$6bqHY7z71Ab+j#Y}gK+nm5kp4TAv(J-WORl?$!D z|5}i>_3^gHUW+K$gxWb4M}njR4V3#}2DAzGDvYb-><~sH zDvrQZFZO+uJxT90K|+aj6mBU$Z+0VC9!G8l!XZPA9|{G@Lg?k+kODV}(ah7ZeRpqN z*_ZypK4|V`4vv^WDjL^hBUJNgLh&)xl&m)sfcWRAZA~1}vvx-hfnnQZ-*{=(3$4{+ zhX0q>PoYdkhz!_sO3)#n)|mhu1NZ@wA&8c;M#!AZ1Nm0J+JpP}W~)RPCExB_@WEZ! z$2<}TRhTnmccRY_=4PshXwg%_P(IpeeD>S;T!!cZ(1Lv4Q3Jg8j@BFJ>@Hn((C&Z4 z@yT^1sRqMUww=AiHk94_kuF6s_J=lWi3h~P(vJs!8js7gNzXO{=vsi)cSD;odleOG zCr-!25;V^t)Q)ZYYE7prO1>fu#BPkp=71KjX234w`;SsJvW{;Q>& z)#!O}8e8JHWOh~8x)5CJif#B^>zasru;2uP4Jx@+XLelGL(wb_^c|$z@f2MC> z&&?q79i1=1bogPoO^v2-M`g>}!*x%^45@q!w z<&xnTc$_2%3b#K-?)EJgJMqY+fE_ml%HzL?Ce_{T(WA5m^VNGxWc$=C#*)NdaJ;|V z2D2S+`*!r^AMh*5N0_8=#T!BB@Nu+YB{d=GDL^og@G<=XAgTudhpOjDM@U{NbxfKv(S zULVCD&nRgajLW=|Z+BduI5BizAbcU_oM1}Wh+k@!XCU-@2aE%WYokU{P^%|^-wfdF zV0^YHRam>O$zlW>jQYe2J-b$DR2cy{s6Uc3eGw|B&VN^W4**3Dhg;@F$%?{Go1~>_ z#Tc#T_0x@5ydIS@=m(@~L$xnhOP8f>OrqjKRWZWYn{N806WOaBP6Fq6Xm?a0&7d`< znrreYVJ%s#>j-Am7A@C+!?&|_rjk@jc|d*wflPy5;`#i-?qm+$YfDG4BB=elTCd7l z0)E_=tBs(Y^*+L1-urgC-BoYD>qu`sbMcV~ks%$M?4TwvDe5ws;&lKLY;_<)w63t# z&;7z^uJJ?!{eO_gw5-5XO$)|NbCCGMQv)W!kYETZG9{59$cu!G5Tp_?Xy8iTRZx>) zvwbPxyA4||<0?&8(5X(=>d%}U5FtE=Uxr>*uT|Mu0-C%Ml!o6A0l*hI0-a7f{;}T^ z;%sdFHEO$MWnQLa%06-)W{|7TOr3AEL|#Nn1L`NwjV6)OYLdu&J6P#QSF&X5Z=HY~ znt54X_h&MCQfzY)^jnwjQ}4`qbC%quk0Q*Dfpmn16!Ps(6V%E-UE=@>6Cw`-N`Y~8 zfFNAkUeI=dZZurmNX?Fw8`{g;ar8suN!3Q8ic84eWeIfcS(P@z4EJ{7F}J8apTgr7 zesLE%rB!^*^7~$+%1Y?OYyXdQrgMoS{n&If&8FRl;}Yy(Unq%L zcfj>47)p`haNn zQ+&F%-7;bbX)6BJ`Ehk+?8h_Qpw}XL^m^Mlr zVdCJY7Iu$Q&Pe-)kihFGO$Zas}5k8ZLlx!py(9b zmXSBQZf;|$ofGPY-Lp(Vywe6nH-CaDBa`p@AZVeRE0^3^db#r+Le>s!dPngWOey5L zrK{5fy?2vyvsZhp8BImI-f1t1^ZoRSO=pB5?ZJgkxCR%sGlQFb$kiau3R<&*Bv+}) zJ4xAY@R8*YEv-S+m1S5L41R0=%D*{LWDN-6zwm&a$+G{^eX)VR@q4bmGU52CG>w$n z%Ewz5fqV0Tviq`*El4F2wb^HyC{Se+vrlR6RG;s=@O{tA+@AU=H zly$O6N%{acv0`eLzZUC(J}xcA|Ia_bsPW! z=A8Gldupll545NCX6s!8XB1#R1cn0vbajd05$%3EZ<>~h5{y`(=V_vC0DCs#-%=H+ z0f;u-*Vk8}Y#!+%(0eYAu`q*mD2M3rx6=ne^CX}`twIRU<3XiePc}F>*vrem9lZRt z9>u}F(nW@>#czo|VdNy)HuP{Zy@CG}%Js-4A$S;a`ah8~7p8qAvR(OFj$~l5BN7?% z9g#A#@y}=XyQYe9b>`2k<#5G+ffg;>dbjstK@}!S5HILyG!j-KRCwaM0*Dr_8U2+G z%G_c1*LTr(r>r&p`hc5=1l=Fkyvu`8N2Gc3UmQfXybH}Px*Y$~?X+QHn&tx%F`uUN zGVVy6-d7%+>>t>jTszq|h4=hB@^(Oz~xXZ+37c!WdL!WCZP$ zPXQ!T7Z-ze8(C_FYWkS0Yp-Tm5pzEHaYe=4#9Go$eC48PQuMWb%IIe-d%prEbC(>&=p&hrS<4gKWxvy$Nxz{-MJ&rG za$^7+!Ze9RVVhKWI%BK0m^_y{wI28lR49MBT%rM&P{8&Ms7I)phTq}NF2u+z`;h41 zWCMm7x5A%w%0qeK);(dmO=;fgF(>q6f27?;o0UDwmyMtqgf|bL8Z7KUocsA;{;pO4 z^KtI>S~bbS3Bn%J2lOCFRcC-jT;VWbnvFsMdb)7aXG7wf_7cx^5X%qltQe zO*2@}u2Wfe-wk&M|GcHzQC*X*kYWO0`r-ge0;$U#nivlIOfx_C%ql+Ewg%P_&qsUf zSRobvhpgMzfTxFjpJHw0BCS8?-6mA|)ZZZ6D*VzM%+!1YkeYSxN%X zKomf!W^2+-!FJwuS9FUrlnuU9_2(ZU6R$(O>Ylk*Nq+-P+VKB{p z9bWIj0B1b|Kc!%L)SSLjGFnjlb9*=uzJ@0%XvlW%|U`LpQS z%d&#uA^P)DUu0MsFyc0-Wi(1R!+zov%RO|vzOOaO!Zt?byAl4YeA6r4M++f%&?^hR zE=yTaNGEw$+7RQwi5sLbPzmX!!u8eF#lXRoG7cZD?zCDXoW^ zNirQY{MMLpR{Jdl0TATGTtpOJKThoZFab#jG`$Ql6J?E3idRojSSd>1cxfmf)wy#- zdP?1NBQjpGF!6m)UNnHr@eAln7c zc|Z{;B%laX43A6k{yLLfi`tbu)nVmF7$1cY<`DVf&9hg`sX98Iep>$iT&jdMEtNS^~&t@f5+C*?G3X zQV~nQuN4JZk}p0%Ja5EVBn*)7L53JLZEKZerrLF&Czs&8UMNW@i^=~B>>5I1R1uQ| z6>Y^^i#eub#k*^6#EayN*RO;xM7FopXsycS|IP;cTrZd|JWviF9=wAY(afq)U2DFL z!SOG-WKq{};^!5E!=ZW-33>H>?1zY%Va`GB2-1`0o{cXCVBjlnXQSyn!XUROkT_qU zlR12`d|*EKEWj?xc5j~gG*AJ13#pSO#f+Cxbu!%jskqh$46=!0o8z`mljTdzQU7hF zjU?#+UHN~uQj=S7bMK$K*ZqBHzpfYC+s9{_pFXApq9szH!=PM?&q=c{Ys=K%>g76T zYFmunD~Pt-lMKGe2DKcYdBhL8%k?Nm*&@B6^$di9esF6pW~OTvEWJ0Ox-WZ+-wRR< zw-GW=G}b!kN*_NYUQH(VFLs*ef{*>(<_YN@jDO3f0cgFEPK)*VIhCpoRzs66gmmY2GWf^M0E%kUef}YESSJ1fOrSV$1- zoMGfHI~4=Nud#*%Yi#NRfR3-wr2{B3fa{nQ{fjvsV=tH&%JOf%%Uq*oE#|O?bVR`( zC8`PFhrJSRf7nUm!ig2iS>U^AZzHlgAt{Ja-{@ng{pCwSS7Z7F&<-$zr8e!1Ilp&)#99*}3}N;<5{wSxiiA^^PBL$TF8ziw*bmd7GYxDa$3HLRpgk%WMU~hWzY*T8JoCr9pRPG zRnHhT9Gk+r{z5bs={UnX~UPX=7VmB zcw53n0DWJ`p^;|wkC={(f-a*;R-=P@A`UZiG$@uj;0;XR>OI=}#TQRcqy_Z)-)V#6 z7>yI#**xJ|k1L(#IJ0)!E=^d15y6P{?IwW9w^Uy3a}4}Gg~Ftabkq@B5XYGYsMzkW z(rwvo!l)5`q2^y)nmWti12IH2jGJ5p8a2!Vy4-4zjE<-rx%*w(;3W7=p!QIP@EiR8 zdh#H%D4AJSPm$(#g-PQnAOlvQnV>xhbsK3{9n%-=OTPi^-Z)+GtB=2twRB$}enivp ziN>6Vj0}uMLk`FDY1p+pc*ui-z~0h^*3@Qi1xECDwaF4_ch}pSi`eccCnb^9jy|61 z9H~tLd)BMUBcMjbRaMhDTrvu|r5~|?MizkreCB5F_-UBbEZ`n0O3xmY{{MFYRJVkB z!ISSJdXR1at27b0$KD$p&3O<(I<8-{?n#3nArr%vYpJuPVsCqg@Ks*H5G??^U45yUODM*Th&sGeKXBf#ybm0pN0F`D|bkh{$cNO@Gl6e-XaZ)&UXE$ zMxFUTjFh+WWc^<-Z*^Xd`fW=ztb%PG++sqjuUirp`h73rxg=#@hj=T0z-^5NSSHxT zuTp$O02N&9Gt^ZjmDb7FCq)s@pzDS0%HUWF0?}A3t_yNt^G9@ci|H7P0%<<aScAt&z$pKnvSq~Et0THeqE8n_Uvy32a-HeG{Zp;QH_kR*r{JT%)u*s&hvg%7?qu!7#^4evwztOG zs)Y#v^(u{Vj;H^Eh`uI(5c33i=}?y?87|8rouA7S56v7J#N_W*((n(o11DunWQV2v zMMdiz0wpDOUTnhKodGDIknrP8u2MxE7E4D`d&~H372WOvyAkhS(#$xkcJGybgJjG1 zI}mL@h!6#`8Q@NuMclPH0IV~n&Cj7sjM>4XE$YH{EDMcT8KF~w$8C?L$L;5rm&hzm z$NDMEpF4Ih%o(_}PFi6N$_b+UB?mwO0vDDCl1uAhSiOOHDvfw7hB?c9{J1PNd?qS3 z2M8G?d((y5Gs*sa@x&#U^fJ~>WhSz&sMlsjVNKr1ojY~H!IxK+rEA?Ymj>swI`byq z=B=>GC4BYv^xx@MX4wSH%ZyKS}iZo0$~l zU-b^IrR<(YlQQ{TLY1m7?q`=d81myI;q;BGLZCS==h#kc2^@a0+W>ch0q>1gwMB0Zn5q{=rV^)olAO}1N zr1`Cy%!n1aEAwqZwLcJW;|=^*uM9zU^U6dFS-*lI`mG{a zc_mci<-0AW{V^%#}Z=J%GCrpD9)dK+qe z^d*|&MHO8WSrLv9usZS4ZpZJFNmt5;bCVm#Hky6W*P>R!Y|R z%UWf9iD)q>m2~WnH!aYE%ul^jEHkWvTAJtS@oDtM2=cN!Y3pw?XZ!@`)(cYX-r?OpJ)8JYkv9We0mx;&-$uX1t2joU}M$ppc|@Zr!!g`{YsrJgTGPm08Syd}6un!=7CyO%3e z*NN|EUvnaC)1gdMHRO=v9^4Sg)AS`ub-#C9i2c?$$bjwN-aWgFk8(N7Qv*FRlOxp5 zgkr_S_k8~uud<{!s!_{)V4Ij`cuYit7K>l^Jsk`J59V}XrJzE(-CvLN4V80vt@+p( zkl!27x60PFfTC~InXa1ZK;f1BFCDi^H-N zU7;q3#T3|tf_v~+P2|}CB!;xSbVNLead7{(+l08FcD0RFY)rLtvo%l$oT{FS4MW(v z+NR2A1itn0*>v!KLB;*$Rk6l0cupkg1q3jw*X~G1`rzPSY|b#s4aLkQVQrI_|7if6 zPWT5-q02I@Om0^Sx3iEO8<;aZ9AC$Rxs{bZ%)gV_L0hWS6xz8ut5ZfusC)aAz>AU* znTH!Z$Q~Y7J@uQv9&_8ry;bRFUY4;h!!V$1UE^g^$4U*G2w}iXf9w~eVit$eNO|?S zR~%QZ)k4OtOAz2Z+fsaIfeLAFKu2l>*X>QP6&ClMq*aiC|HDtFl zkq{#zHCkUz?hBRAXo_$wAMOUYPjE*|$0@lUyh4|V!rt`LMlG%>JfWmG6pS{pycn9* z*@8-m3gspAV&pU z7|m$|V3?s^lyFWW(n^(NT(4Z818U+hAlAvmIm@#i**>|xJej*T>H6PI#dl*ua^rNI zjFdvXDcO0ydY*j^I4D(=ym<3g8??q+CG8czeMOY0@;Z~_I@Ab!`sblR@}myR1Vx@8 zWZ?gB2CemwXb)DeIw>CAo$SvM72Pn)9i_@WU6+_LQQNiA3H*AX9lqFZGcL*77Gj>E z9wTo);{m^cTOLre^14Ap#vZ08fV^r)i_9nU{@xP20@m#0eD-RK90Ox7tNS&qkKHNb zL&a?P=OMY8mK@vD*(qJvWh73RRa^va$kp-cgg9)&NL~ZHhOO11QlUMK7SLUo)498Z zQWY(|Bd4OqN|bQD#JU3YU}6k0=idnSC41e^v}<~D=H_olvwWFFkb#s*PV)X8^^ ze)g#Ta6dEOp6K51e<7IqV;H-aONAUCR>G&eigU)Vsv{UM=0$9l+!%Kc?)`SFQPL7- zLcafN-EWoau}z=Xwu(SF?)>!Oemzn&Prk(DL}OX9JuSg--xsnvC0{^vh&Zj9IZ4X` z?hkRY?%;5y_31X$vHj3a9!#`U-oR%BG`kHLqjyV%R5{5KvUS17Jl|(FExELG8qcYK zjF+?^YUK|uA)ILC9X??myBv{Csx;> zT0+Uw*UAV#hOpS(m`i-_D;SLmc@5bGxsZVhiRAfgBBy{y#3;=Bh?0tPoyLIrU_gFQ ziT$lAbG6Z@tm_b^uXOC_BR4c&|F##5ZRT ziV2M3>35Whi0AHa;+fvw1PGC!LP0HCEZlh}2R|(ds>OzKN4DWRXJMMQ(bBqL??o4N zI1lyLbmF|Cn7;EO-TP{OKz)n5vwm}(J$)6+Xzn;4_?!4!`5^srFr1MycinFA0ei1A zb)9dysXnEbnXu(@sv|cD3CITpVaO&gH`ivM%5@RlHbIUKigHziD#YI@#jbh1Qd*tXq(-3TwOIRZ2EahXMmUyU#RF3=Lo&w==^Oh4QGmzb! z#>ZZ2Jzz8A20EulCECtLjAb2deRRD!lPD|frs3Isn=*pRC{%egLH%?tkoz5l7Q`aR z65Ncb!|52L@^_ttqIY=b;75Mt&n0n#w(9ANg`k(qZOZPv&<3x7mn9NTU^n_lnYX`8 zmseWcmdd~ps+XRMPtr3n>75^32~n3Ms`c(de(ou@_fYiaL}o!ztiJU`B?zU@Q z_neFkoI$%|84-QpX+mhoItYHmF~?29TVl#@@DGkF$>c%lIVthsI)sFGp@a;dAY0L| z9lF?Ix-izr87NkA6Ljn@zNFfGF+kZYk5EwpcrIXS}YUyQFVDSv@i ze3|*39-_LC;=9-pQ4NXaQU>9p3FYCPxuIxw<*n*>FLJ!KYFOHd&R6PyF9x_5By2Qb zvFp4C`}ub0i0f0D!L|$P$=w1TA8U z>)EM%+dmfdt1DU&ajRyN|H!D!{FLz!#C*D>p$C7Yvq8a|$W;y$vHpq0QVGAr)WMEe zm(!vFAPervIv*h?<&Oc6GY>M*-aIzsHEV+K@Qfy zoal10Ye2%E*XSSvH{7L}U{E46X`8j^h)xl#_)Ksbh;nT`B{@5_^L**J5Q8CX`OucI z3qn6>(@|neFXz_V7~PFf@Aetw&R`7gr8399m>4Z} z5lf5rjDv;?1j9%Q%=*vc6WQ(Awy;i^u?9!n`sVHFie;|7IYBb<&w^T6v5#+B| z#u2DUxL{VWYcZQaDOH~tBC?CpV~(gkD!1N&ele?KwP@@=N_}P}!|!49R=}fzw?S-1 zq(mV4z8uO-wLHaHh38pW zAz)==1*?yBQBGc-HEmYR6>xgK)IR+*v&3qEI;!y6hZ;rF;;=5e4HrgK2>kwn9OhaM zbjsjh?9?S2u{OA+Onhoqd<&c-&&O)T4W^!_NNJ1ai(%VWWwM#KGa@a{gV|Na^`|oTsmx4b3@)-xL zm^unA-bwMBBeN{}%k)5-mjkh3lqJde*r4EQFh;PW%XBG2DiXwP&-yDX07{-d_eV@} zOC=d+Yv%1|!?{*GKPuWb!Dx!u$Lq+LHPcD!gzH($`xU43(kZYLz0u8NTlMo%xC^4CN=wiM15sgmTS|sba!`2 zcQ+DBOE-Aa4bt7+NVjx1(w$P$61Q}Bck^BDy`T5v2Y)#Z&Y8Jptyyb^*SWl)(I_I1 zFH&iRl?VahV1&S);S8?6E3_kw>@DZa$l6hpvBmebf$DxU%gag%;R)mR^pA3omgO}&X5!eF)*T5%NYc0ysh z9+|%{e!klJ(3cdB&tpGB-g$uW$7w6!bLnuSE?vnaN=Rt#IXk_yq;%_d#MX3vNQ@G> zZk9gtP{PrE_^G=6w9EI@=`McNg3p^#l1pk8%eQu7ZX8};w+#MO5yX1qhe#Io%>0|7IX zck7Q1o9~P!LD6-ol>$(fDcPyHGt_N?7wcPWEZ)}FpenyfFZD8#myFXg1uMkjA~GFf@q4Y?rD%M2aB^E(my8q(S^4Tm#u? zJhOaAgeq+x$Rr17fYj4S9}$Zq#YmT(cE{LzsQ6yeL?O?I(iVX$ z1Tejejn))wZ0M_=aviB6+5=5r(hT&fd`gx)FjMtU-R zH64X7o=pAq*rCM+{K-iJLP-3ddrEFrgise^*oTJ)Jz$;H6ocL%&>5xhiCRHHD30|v zD#Ksim#E|8in#n&tJdL@h#-;H9*FzCN%o{QlZK6)iD++bCBZrPZ|c4MI5(fhmq?_P zz#%xQJg&?`CFDu!7}|klFWX=+d85BS?5vS>_8s#|GIH}1xK^DeT1gW7IAV59nkqYy zIitC1c3b0=+iLFQQCF)dS%!f~fkL9W#kfJQcSw38wPQt|W>{WhUVcnr+PPD_&*l`{1a{95QaxDbg-=B4(*3!m{x)(hg55xw2;qvJQ6`@cq2oahf*$WKHZ}9T!p! zOpQ^Fed?3~{A4B0ljX`4wfR3GD`NO=b1@NjhNIN9-S|{OMT@CGj7LNl=d7FSSm{w? zV#cHHXE0FH$G*Q%G{iVH>aPtll%AsRYGx~fA(U7wX#mT`oH&y`L!#6d)0AxsJuzX4 zSuAh;j6|_Ao~{$ox(S$okG`u@$DatL*Hj|Y=|0PMDsyW#8e*g$F`O9W7o|Wdl(7_% z8}x|?ePFZX{3%pbmf>O*pvp|!LB<2c%gRe}jgxqi>dRv3KeH@Yc!5x9~gSKnf3c>V{{LunVu_sfmoI7c%yAAL3tF> z)x8On_up=&3uZq9@5x)^ARq!IW%QA`ED z2`!PJ+6lf-!K4oqKcb^P*z290jGN)3rZ+yy2X$tkH(1Ncv}r^(o%;S>XYqX%7Z>lD zQ-AwnSCnSJFCPd1B(+(F(Q>xMif`=5~iY z_J)C45YLgNr*YYOJtljN1-J1%_fPEv|Ic!A5*{TObRL%$4lt7r(5Dpk_D5pR5Osva z--#(Gg~W0265J*k$0Up;@aA~bFrv28+~1G5FM8yw{WuMaQ3>m74+*R@T>SNW9oV5? z9hnD>$%=(;P>xS(tJ9zD=Pa)XJ6?vYm-;F6NW}`hz>6EB*}~Qn>~2u$u|tMsGgfeb z?+S~+Dm%dJs(3?qitxFe*XYZA3Kcn}{mvA3xE@gmJB01J%>aL$9=#LQ8En?CSd2al z?m8=*so)x3C~5ZqWC(}OyplhE&g*~ZynLZJ7WE}`cA5_%(duw#XVU%wO$Y#1sc)n> zL~Db4Ix4*ukrZ%r*Zvxmn<+5m&G=Nx^97ikfi0JNDJj!L;se9Jt5;$o_mMi z?9+0!44XZBV;Jl3_RxHg!~v0_{y*EA8@R%9{Y0Qsb=uswsrg+!(BTSaesSSNlDtfp zqs6OzQFXE+!m|tRUN}KtV!F2u$d)=D@l)*iA@gg%vnpEcYt~}?#$~%@g*=D7vR9cD z$B5Wl(>5>PwRxRG$RfU+Ez}|+tcwS+yT^ytx~Puo#IH05+=)a^M&icnoLpJu)EqoM z_bXvjeM?_zMka3X1&pR^JWWmPsS`qkrXQ*Y+0bSRO==T7OJzIF%hptDi~*ccBOAs> z?`NaQW&>*b)3a_~$@vq)?|Em48wW+!3D}iZ7BCGKf`2oY6he6k`#|L`r26F3eG+t& z(}I-Jsji;PAc$HVs&RB#r#&hTb#z=H>~jqVwy4 z|G`sIyUot~_DRp-B?;zM#aL*3=c)E)GsD11bxAg+TLp3^ws9%MD?#T2$Gzg9J&Qs$ zzSSWpKMwNOsy-~m)6I97BGBy&8|jQAPw0#{5AzE2(sWQ~L&U5g=(`ZSg0VZMrF$xn z>%O`_FkCB{Us07auvVx%Q4WhVsnrF^#TAh;T^jm~_g%W0C2}6B72R-RPG_dzRq_K> z);#+6BFpKoJEbi&o0tErhL zYOW>~UQpxL6+%ZxM}tvBlWUh74Z_G&;cfVj#3*>10oqhfoV$9_l!O~}XW3SP9Zx98 z{SK(c#8~{r^a&;nD)ydighWgR@{{a(I1)&7Ale^UN3#`BwBK^DF%TxB{b&2I;hm48 zt!4DHKg$wpJ@m`gUew`ix9jFcH~&o~fef4LRz-sCEN>0PF$T5Sd!H zCUqx49NwLR{Ov%aH@OJ7HEf8>p@W|B}b*8Bctg2~iF zI@8a@aYMZ{mKO;7Fb7J%r}v-d$3cw4g7u+qx*t%4yoJHlwV(T@ARUw);gW-+h)Y@tJh#hU~N`Qy%Um|ibXC=O+rX9 zJmp2aCd%SHF)nx`MWw%4? z?Y*9$m){Z+1)TWJMEqPL#5ZZu?@F-xq$1?p>@b6Ogp(PFCMcA)_;L~Z%))XPp~}f?e7~h%h755xh=i*|3BmR-yuR`TT~x+4ZC7>Lk5P77i#Z(a z1QS>h5yfuMgpL(g8s$$;aCPiS*T`rakP%=b`1!px+shkU;a^64BKy|oes=}q;%^F< zf?$PIi4t!;KcQ10U)>Wl5o{rJiW9}20ZRfq z9X&x@@~(s|F46AmyRcV2x>)U*F<_7sXEJ-&bw=`jLvbyNsUggnlzFicU0VL~4BJZL zE{7n<9N{QDiRAXBY0ck&f+VWeGBjt!bNmqLsE9ta(=ymlZJR_euC4f zS(F&wR}1BKd7A4P&|T~)<|xLIH6j!Qs*lM{{w#NZ4*P(6${ft6a`l#lhO$gZIC}*d z+k=v{`v70$F*ZIHO z(8l|zIw_68=rEBnwQ)7P+$Qe)5sjY3K~UX3gh0(*?v0HTOi>;3Pa$dAgG* zDTlK#;Gvk$gzJB}boxBGZV>o4?kttZynFI-{xAR`Zg8BGW4PBHf8RHQCGrvNlPai9 z({}s*m>=>#UPOrlnlAL>xp~*P@{W?P#h8QZtHrWGxpVLj8dNqMXo2}CW6(#cdCm-2 ztb;7fO|12x*Po;zSIx{y$t0<_#varJwi@O(j4{0mp`Hl`)ukUpo6&M!l(?|#wi0IV z6D6{PK~SFbp_u54dA(sms;Z}jqMsKs0_!G|_KKIJ+rou#L$)E|&OHXCT)bOTg)((T zbKbe*q9mzMF;)nAaS-mJy$>vBHZNE1mCP8FeoKZkI*+=m@$;)sJ%KoKYQ{5@!Zf+sLOa7{2PnRW)R|ws% z*Dp0pwRYq}5t98hRfI@GMd!G@tSiBETs&cvbsNiNMmrLhr?lb-MHTE97Uz|4Df`>x ziXUV1T25{<;TG z)=6`t)-El)t4&3TeBEPD^b&dpM;kH3Q_-z|O#l9)ia4eopASUnq_3Ql?{V4;bG+%L z<7F`TX2x;!cPlNZ9fm)hUkR>14J;@)oJn^&B&ioH7oQqDX;*>zh{}Ym5;V)*0V&*D zBPA8|YIT*@x*Xn4F&5U9I*W->{gujrkqYO-hZdS24&%zDSSn`pQC>r7FvipSeu&lN z2c@;O@Op=B>vz(^Bp8xG!=J2KSuiD{o*HqbEkF2Oa3&#M9{=MueV0vwiXO3LdZeOGbVTi zI*@%|)X4~_yP(ye16{KLh>Wh~UkANeDkI-ULDA{?Nd{wH$s}LG9kSd|I&5S*MOWHs z$fm$?R?A?~tx(g@KBw$Sq%bWfM{bS@WHMy%#l!8!r(AMYwADi{@pqzK#jJaJ0dlSP zJC+NCG8{MRVhn+l6vyu{kQR;ik*k~qj*tuSa@Bq$Dlnw~GhShyB|Q~rWsnqFWn?we zM-Nwxi4!eKN`vN|Lm}jZ$MtRzElbzJjTopZd=^Ca2bOSg&z>b8zpi=Vy!dOHk4HgW zjY))w8v0@!leTK|**yp+2Vf-OLTl`5Oiizz4*|nw-LBO_i}Mal$iewcbOuI29rD&T zZC*`6ai)S=s9m$&b-OhgCe=~&ymOk+Kl|(JNE3v?;*>_;hxt&s*|RhR+g&%6viQiw zm8&?8eO9a5qYHQI2ZE~5ClstDxfo`n zJjvZNmnFlk&9z|ML6aixPBsxgW&kWZA8ZrsCivg6sx(#P)kQuBUq z$PXJ$e9+E=*x6Eo-a%9cH2v^9tZzM&ehuE~=5!Y&v74C&QGGy9`j5fh)vQ9FdmdokR4QRyDliqCK_rEA@?4<<)zc2&grZHw8t@{Elz7HqQW!6&+Ki4g{Od zzS=|^2Q{kNgesQ^BQVb!C-0FYKbyI(-Nx-!WgC-0qvD8grK>p=-v%||rvrpn!M>P1J zxgj)?V*C8omS%>)9IusFjdhsXIEaid{b>6z4I-ez$U^*CNmEvxT_H;l0Rvy4fE^Qd znHyVkb89Q_PPMalVg!|^ay6IsCo|j?WCD6M1wrr9Dv=(O-uB$42tlvVvQY3#*O4sk zG!5z?=Nw_c_x@Y>J=)HaA6Oy*k&Gd3NFnG%mGYKe^miOWyBDQOKNm7F#XF)Ya+_9v zME+nu?m;NE6$e9zITWGeXlBjgXzsgfI&M+$(BdxQQ(&tsS{?4V0&yPgsk49s3y#4P#bHzQ6P54V&|m#XTqnLN z2RvIi)zo?hlY0&9rktwF+1zYCSCTzS5NryHzfreU5f%Y$F1&uOi!1V!6-~!f5nlWD z%3NJhtO3weWyA?zCp&J6-mVbyMGV4fB59sCNDz?56h=8@eH#oFjnR0Q>d2Dk*=uSM zqcVb0899dp-irXuT7fyg7faC`Di);G(Puo@ z9ta7P2Mfqgla=%^GWXtp2$+DF3RjZ0&CC#`D>R=MPohV5KD4p7Q6OgXFVw-V-D~|p zj*3qHOXDQ$=h|!3kC=J3qlG_HabbEDxJSV`(Em(H0OpZ2!0y8{Y@i#gM5lm4t@V}H z`f*_*56emTTnHa|a*OZbO-ZoUJawcnuN}9&UDbaFm_4Y-O?_(G6%+BcX?vOf-1*KSjT(^Zf-G)EoQ; zbmLU*qU#nIkqkuzCd)JSz?#d+UTJ3}Gqr+pw@hU!(3w9#6j^*q^WiCP?pb{P`1N?k zRs#V@_+C$hgR4BdojOXvhHxCQ0rerA?n@rlpw8Mcwqd3*?Cq6CLfi_x26WKzk5Aq7*wV{gIeM$1PmO}6!vn=hog7%BR6((??zBAToVZ_8nnmrFH4hCpi}B0M7K1JPJ%$9m`U@snQ|kubJBqunk_H z_~3cB4{LY*k};GxwzJ1YkeV;Q2ILq^*|VwMe@~0m>?gSKHzfTCnKofYHscS4JV8V5D zboL;>U8vn)7)&~uej+CSxQgGYphRZx>=1m{?Mt@Y(STE1E)w~S43J3w4#Nz>MIi=x zKo1v)e6N!Zz=f%xHQwk@S6d=N9wtK>7YbBjaf!o2f{KuOGIn!lXFpKMq8S76nTQDP zOyW#HHg5{q{o?G`jxRFkJ*o8AoLNR0ON)O3IS2Qvk@N6n9TAhOmvzxol9l{ zBFyQhsmVGl@Iq~(X4I2qh$VUkM&#-;O7B*-CqyPAEowO1MKmPj7@eH;BmN*CO>6r* zI9pRB+$}JkF|nZK0N;c-6}T)~RcR?7wV~14+TOWK5`}yN1`%F*p;Ww4%bER-Qg6JK zfv&nmr!8`4204Ts+~|eZ;_MPKKQ44_WGA;&Cxp=qXe*#u9hyL`6sdXr&9d;rD6OfE z$_~_f91JZDA#Eux$!+kaZX4C;3B#*;d(;yPnG9+j;(`9&ozreCAsXzs3`YzJdJ)I6 zk(x2*f0P+X#D3yu2#hIO4`T|w_ zKpOlQ26q)dfFlV-d!lavzt$IHu-Tf~!g$S*gQVmvkV~^HF0>u~GKznLp@$24n61Tq z*{B8EMP~o}Ps5(!x!h~93(rh>8#>&-bkzz)*2!3onK;rlh?jfE#H6>h`V4-2$++X{ zt~=evN^><97_a`zQkq7-i(d~%ma@ETJ_~Z=wY?eU5O${~ipGK)y$f_cnn$Sybs}bG zXV*IIO>pqz1GDgGU>2Um7pIqoys;m!GWCO^p~1Ec;eKQ+QAx=}XwX4Vfmm-O3Qlo} zM@gz5tOOlmVFNFI6PQs7no7lu9DH zXf3_MMA;@yZk}^)ooJlNGOC{W(&sM569-|ue$?%(Fx>gl7NdfR6e4ybXyDch+%d@s z1p}q5K(LqyVT5k;wA?b8!~i($PxzhR-&z`pCygFQ=!kWR=|8XRl({>#+?HOLY{-Kj zpe`fYgAFrvIt=jUI>=h|r5wI13HI^HJs}-P(v}{Ssy?MpPMScE? zq5nS%Lwf4I^x|h5P{+M@?T4MHH9JtUSos0cMq!OsRS$>q;BRF+y>N(2Os2~Ipo&ZG zJt)MXQ>u!@eSX(=QtNPGXQcnO3S9c_)d~o1xm)P0H9YmqJ7q?8>Pe3InCAY!Kg56^ z@V9gSRi}?A)mT>_08m*0bHe$$Fp4_?WI)v(y~C`%<8{VnUXqqE>y3a4$A!{Bv{Ld! zC)8y>_7AG7h1z2qbb!3)H;6WN^1)^L5)J_c3x7dP_tu||au${Rky z2fo^}XYG~M?=AzR876io)@^HSg8^cu-wP#dX~7O@C3&zQlE{OXoQ5dIKswek^gTfe z*mGw9A{2g&fdY8u58odUh?&3l3{`ikaMsD6V@@jKb`3FGX+`e*|5u)_2gxA|PJyMt z0R{N3J%h`z&unTMR#prjrKJ9zFzo$9U>qZ>@=@b`VJ?Rl?fbX*%}5bX? zWO@wCqC^5(LxZq@EWTVA6`U&LyNXM4R5oeX(+Gch&Sp^!S5mOxABZxvYtoraEO>lT z?_dAbTXkUNdYEGWE%y?@SUB~yIv*XQt}ziq#-a+KJ^T>rEf*)2R#XUbBlFTcn;psJ zu-Rp6@i?Hdd2y{!S{QA{k=LzJMT_Hkb=hF8bqeZ2wgXg=2$SfH#nt|WnKI&6XqrCn zvIBl1R?o1zXa8S(?Zk)zykzt_83pO?$)sc-)g-!2w-9PvUAx6Ii!3``x*n7{);go#aq$ z{4ODc_lntIJ$?t#Smd=-%3RZ=WKNxdJW0hKUy*vmzNj@ts zm`>g?ggt*_!=g%=#NiiWgf0oSFmiq2x?}q~ZpZFRo7TwbmJx*6%rJZkX`mq=VLnD> za)WHgo71L0DU-KU=sV8suMx{exzO*eBSDLJ>jHxrcXfASbNLybbH{>NHN3)F_p!(j zO{>xlJzC%ZEPRFvB}|5>kgvcR26ke8$B)Dhl-0i~T3D9;cmy#rV3ii-w|}ySj)0$# zj$^+aBMgnyB`9s-o^_-1jC1-4!7X25#ow-*ARYAwP(&e0JVgLs@6JIrLup`emd zsN&(`&$;R<#0%x}`Kbcmel8OLX z3vGtOYHVHB5dj6Q3IajcEO`*lzdB=2WOvh(YxY;dLSLi$n@b5H#RzApIdDB(|M$&Y zHxwl&qrOLb_C}JIj5btttjDFVQH7EH<1{kL?u1LHdd+({S27Cb3@n2%dG^A%GzdbHB5o+GG z$&~v&SqUK(ItDhXe)OX_U%!;TJMwDAiw3ud?u{ydlU-~^^Fx6*t^>FCJdykrL5smM z1;}24O~@NLRl6U|u}#ILAL&c2)$ta=P17B)-`kvnKT=^Xa=jzy*<^d=iP2n>6`ml- zWaJ&S>U*8iEGIA}V8ht!X`n1!t*BqS7+l;USf zgV6@Z*`p|}p0yVw1dzq}j&^&rVkR1VT$24v1QyAd`gXFV=N$Zy#g)zbHfnk!irC1{ zO$p{2O@kKlp4Fe}Q~a93H=$VYpdi?LE6&Z z4%7$iT&o0g>oUr4D3#$r#lhZ1GPM5HAae+q_26xlH5eWDMt|JZuj{*?-gNz|g1$Ep zO*;Lr!Oj)OAn%{P0Ux0g5g^rJ)ctMDHYa~d_=39FzQNvPt=|L$ux%&zI3md9I{#3Q z;Op7Vao$3$ymt@59}P%``&VFC`(=(f9z(hS`@x*)1nboLZ*`aW6wh9$xD@XqsoE^5e>WiCow9lBRQLm~lOahE9LyF}Y zz)wVke|vLil@)SiEmbk$ELUh^W@5I%x^KHoebfH6t_M2zFH38N;95gA>v^W1x6PMI zo#aGwNK@}5L)oeZc1L4==+IpVP!}*sP;)RMCLo^rcgN!oxlO_223WVmt5ZeSRodF{ z#cl|nt|FcNo6(KvaW%@jor}`|RLi6%f&wq*JGkYnpj6rFB89;tB0{QA?E{+=&T~jK z&7mc3Ad%~U1GxZ5$PGnjrG<_raCAD`?YU^cS~uE31T|%s=U&_tBcjyAksH`_{@fYt zP0nImx7!J^mpZfNEk0tM$HJzcdlDuVg>lL+?*oHl+p7|!RC6wLE< zZo(Yx%z{mgE3Zi1zkf($js!GmB9uqqp6)y2Mv!%Ljc>G^UO^X)Q@WeGKGT(ZWAC~|5W_g`6%0WTJ z4aFj*ANs7htIg++Q^uNNOOr=(G6F+z`_>aL_{8XJLttD3Wzk_De(4Nws_`c5aJTI!WW*1O; z%rCf2cr|lqK8IZI8Wt}<2`TouL0cP1J%@J~Mm-sj4D^^#^pA_8Ed9D#S=fgI%P0!Y-( zlck`jkT!YH>WWY=c-<&Kl8sB=qUg8wJ~b$gLYBII5=*~8r&L{uB!R#0TXw*eAztpJ zwNKBgHb1m1%(&zk+YGnPapxj#r{>XNW8!IT<0sR>(*^S%6QVj|gw8IjIXZwDTqbDQ)t8l)~K(Rt%tvfJF!-qB|?)UJX+|9Q{$csDE~HhEE`CBTh3s?_B( z5ROr6K1pgZTZ+N7*a%-aR+U@eBb7{b5NYfPJpUc6v;8>?t&`MT6G2F6OR`~U%g+d? z@;LPaMAvRlr2z>YA}54F&E{0+ zH(w!7*!7oodS7;{%LgD_R~+kqP+TmMSS-s5HsD&bt_|k2L%L+nZd`6S8kCQSV8}-z zBK6KJmS<#Pe26KJd`E>E$FR7$A~$-?B|T2V!hlOiLEpPLUo1bg8{_~@`W}+fA@}QN z_(${6pe)(4*ppM<&-NK82nZ0GnhPp1%Guw6%oYe`&x|O$S|5P;7ztsGJR3KLKFmDE z&2Z*R%}Z+JWHe1Q4@F2fz?TBb>Mtt=8CYg`idl&k!e-*v_M{uOv{(vb91Hdt>^ z69c%xq_jZDc}yRVf`SV}uTwAg=8Cy@S$qXad%k_~#EQW2TM-%FM00&X(?=4FX3xe` z>V}=8szv5_4q=MMn-q;S)I0);dz(M#LTtftDv{s5g{`!^1sQwMTFe*IJI^cW?QcQc z{%;pR-mB(m$mpf$r4`isI;$e8(iT&@(|07tH#p6F4x&hn<+{L;Ck7~>4>eNw3h^2TgFv<@2++hi2F0|CCw~iRkvV%^2dyeEf_4^_&cUz}f+c z1VA1i6rm4bsBre-yqZuunxzcigtwyBv%_VSUxbvcTGEVXC)C&uop z2wIk2R{4a?dAf!mG`@qsLS^Nej~~A)MwW;(At9IDJ7WQaJiY( zOVZN(h>VG)#8r=IruvFQtOU#9B_Fh|2zTiSYMAprWgGB1)&#M726{A~;|?+%vC@o3 zT#hn?0jT>2_xEV_*hz!vElK>XmLj0|&(uQc&{oqVkBKP$#bKK4E_?k(>%VzLa8M;b z)B@yT+kesVvN49HG+CaF?%fQo_4w7Z2pwLgSp&z~CHPf!Q$1IC98wzKvSb ze!0i|f^tB3QaHE`C=OWFVO=Vgoj76`#GeHjO0_(e4zgk9{1JbBq$YnaDZDxRxhg_Q zVX{*Ber$1|jb5a_&9Z6E(+q%vpZCjcsJ*f)q6Y#PkQj3K{-et6p+_+w0AGic{lgSa ztSANpa%FlW?q6XgX=b0=_@k*tk;-pu+96JLNTAy70kbvzyrthOxkHS?a@^*(pfG{d zoAxvM7n=U6isk{2&)WPji!RfzFRVPf1hJyjlK%g|sedrE8Cu60RSJQF!F;85?bj6n zcx2?@7$V-)vkr!a4^SE?YL_!zv8#(Bs3J3;J%gt4hMefe3)uMN`PK{r5SZz#UH(6_f@oBdG8m2Hl>X^Ul0}0x>kS*up-Jrc1KOLI~_FZ-MO1qb5b1o32*;1gl%Hl>Y`v)_$8I z8e)8Wf*swt@{=B&$x5rVsWbUoU2}bY9$+4^ujes?M#3HR966#k6#6lGXkvTANhLcFs(%@s1&N7f74MIj2V zt)tH<%pe7Lx~k^f2J*x{)T~cGlXdYYdeD;niN}uBA;!BhQD|orL`f>zrf?%PbVk6O z0E1wWr|EJ8z9ObYRa>zhHlO$}ZPD-`v-r;KznmTVHXZF&20>-sMV0WoUmE=NfEj!( zdsUFO8RH~|poV74Ix1x;Ag5)ND*e*o#ojW@+Pcl>l_L|BbmKJ-xWa>(e>I2K)6&*@ z=9Yrh8nGGVhe%hcb+(lOvVIy)(tnLqRSYJaXSq*YzKMb;|v z7}^7cMDgS0C4XkTp8s0v5l*#_U}pC<3bT5VJ=WzAhCa}{vMAKE*Y+AZDy|H#wl(Dj zFL5{G;%`2Bh39dT1lKiJja9M_*5&Ma0M6tBjAI^WaTsJ>PTjet_sPVepr#w9}e!6q~g$99nqR%fG2UGp4SQvKA?lSx~IBwO_K zcg;eouXQ&--~G6r_}*Z95I=m)=Tpt*NOEUS-{A(5HaYM1;HFbAj<<+~`5sIov-OeT z`<0O)7s>2J77dnl(bD)**h78aQG6CG^om0TN5xV zm6&V5Aql?!>cUoPwJC2k5MP>$7%?_jW6>~U_@Q!#TYq#uG|u=Wjrry;DAIRgJ> zrH~ptaO+XRrM2Q_Bzw^fF-75CR%j@7s1tdzyZOQyA;9YZWh7GUiFxIsHaaPBusae=yJY>T#;Bq$$B6ounZWEh#e@ zg$hb4!u$eonaUP{dRNUC!GC&2*VQ-OP2=<29Q>+Qi={~6_p#m?ce^(;h3*;KLzChI z5vJ5NRsWYCp(b*(j0Kqu1YD-+St^ur`GA*qa#!i41H;GcEo@fsmu;81!;0~TuAkA@ zb4j+e)L47&Yq^<`$z;J|(vNA;1}qkF=RX)z!pxki4hI637O*xBLn`d4PGLn#sl}?= zxbyl8NOF~b4I)E!b)rFoCB}A3nU&Pkg5TquxDaGwMM>7WbJ=+0-x-q3Q31CC%hbGY?vM8%<%I zU+YeW+x`;l|A$f+W~q|V$*em2OQmjmd^xuDdjFMxGw@R`1``Y<$rQYqSuaYfBf065 zc5^+&>p8b{#OMCA7Q%Jj`Mo?^RK3ae07E#VVEd2QDbkGb4IMG9DEBKsv^`P%&Dz1E z!hrxzZFBM}a+EehZ~#CNF;nM`igJ~swEW=;Gn-#@Ko9!Ork0YPo|}@#Lz=8wTAA6h zy7Zf~qScd*<1Xb?DNg_0n#}|3quXH(_xqX(aroLN9H0#3MGcmW{;v#h|1ASb`ma__ zmm&ARpCU#@vz`fZD!%MvhPXXq55V-z z2^*cXt5_usEZ1IaoUD1q$mO`mHv|mP8vwGzJp)n3YNn(YVWsqx`A}@e{fGTu1oi(G zqX60PceJQUXP*qu?Dpw&no<(R{)+RNVJop`m)l$IAxGXdGJSzDF2^mUAfSi~GDK_> zupHv+id>DrlkV)nokLRkp~VSCn%xsO@8`GSqu`J4lW{bxz6`3Fib-%T=V%x}Cy&U;)z#b+%)bXJf8R1|blNhTD>1<|lZjc6UR zl?-r^f4#Uda~DIpj12CZ3GEr4Wbv=hH126DMrk*kBrskIPnhi?IxWGl2%-XXpU5`dDVA@#CKoHGVW?KK{QFMfSHu zoqxGHbRSgJPh$}1xh7jBgO$hED*r9ycF1)6{AvgP!YbeB$GE0pFLEt3*y&2;bLG5M59~@`S2bqliPfHZl_qQcVvSBXb7XXHX zR>)eghX~i09Z{@CF(D-6H~!YxTVOyC#jK}x@qySIo`cX_94och(-1C60AH2Wz^RI1 zARy<%8^Y_u1zC|Q&gk&3KdXO0_h(OOWJ2us2tS*;y)QVA9}QELc=un#rraz({B*5k z1GrP^&O}(!F49VACpk3Uq%r1e6uk&dyWZFFr@Sh^Oj&A~A`cF&GSeheAEYx}adco) z47*8a^A|yl$`;{757?cR-%9EQNNEdGL(+>B6lb{54Ko-ZhJL~#uu|$IDDZlltJclY z9~S@V*Zed{V8<#x)mNlJAzV#Z=vgS&WoUl=bOkbFE9f$)Pr^C|(GZS_?HHJ|E@5cL z#|2WD3KU?Bsr-U>RqXVS2G(9b9>N@F>#RgTeF||4DWc#lH={A@O@6Bmus+gJT4-;Y zznrdU58Oa9dW#HNBsmn#Ow~cyKtjBVS|8X%myT7yU21c07w>^yV^F*|3SC|+@C}Q- zCHZ-!I{MK(CuHDR7hDa{aYtCq<^7(FQH@Ey!m8wH??YDt^NC4-5k*6}Day_BQNV&I zTH=EI0P7yP?;XyD3YT{W#Xa(kN*j~7_Ty)v)110EbZ^y~m`Qv)i+z0$U;p|WSMuEt z_Vr(@c{Lvt0KbPXt1kH;wJ!+ak9xzc?*K1}F&M(G)D$Eq65XPRs6)#Ee8FKDzRHN3 zMxeY;2L@yDN?Pw~C>(NgbNv}nrEAMjwOzY62LcZ5xgYrd(?9m~`vc!WA!N-~&m5`{ z%rt(BV)uk1WgAPhW#C<2N1v=8%`U?BlUgGAp93dp>J=D95@?Z*K+q=H!9oI;UCD*g1u}$ z+N8@HUI$L2gEg1JQ z85o^K7%hp^+yrQTz^6RDLYB$}CXEqLJ6??dXCoyg6&}uZ+F?6t!{_i$*{BE8!M_kx zk61q~rzCjXiFul;BA#KMN;2AC*z49`mx!0|d=g^LRZUE?)`Tk%#z4sP zh!8l=Y*U@wOIHB|v5e&1$O{b-m(!JFkol@F>>`XOeFf66XI#?s62wnOrIWU%AbLiT zxJHK6tgnF>Y}^I*xJ$z@d_n%Y;dt z@z#)965&t-%6SZBs1w^)ig>jkF#W?d0yYR3gsrNDn!P0+yi^#LDl-W~rAw|PZ2gtCZwhuo;Y*s=_a}pcNDL~NTlJh;J{Q|oubK}LJKoME^~M^pnFeS zEKV^R(?0JMu$w(jQLB${b{R(czRI@h*C&wmn%&d7IM`!jc(U^sc0>|f z=NdL@6B)&bC8%y2LpsS9i&Y&kz%21Id6P39zjx}YJ>?U#Win_r4I_F*wF1h)|I{Hu z*{!E89tu4F1H*n;=2r-(bv|I}*UbErCbm8VvA#T@TuL?5vq2!1ztQwBU_E^u5@iBr za)oKwrGNSem@1=WB&0qFqL$&QgqT673IIppM`*D=`=$w*ynhKe92wUpC#j0(hYP--Z*fBtEE zdVg-0{7=|}2J=r_tha0BlhDodv7`<~prlS}y7q1UxLIC6-_!1bw+m`H>g*k07^J_= zbL;d8yA7G>gp&8w976tL(~}M5DK2Y-xZIKOPKNgs{w(U?i(^$=<9cIRjaWO&;%Kl< z)(^0+z%`Z_bHE@aNAO!BM4#7j@Gk`X!Rr)zc!rwawW!fgj#>Al&%Uu=;!@EJ;riEy z^`!t7eE0>q=uhhI(gDJWTVMuMsT%2u1Ga|37mC?i!fWV`iwhzK=D8@x;QO#DH$LGW+#AU$CE@pS#-z-|)n6Ab*V?<*2bTRD$!HFNm|4S zqqXEX-LGBi^12P#Q=IT$5&c@LRe1rSY8)|Md^c=-Is$$5wmCScSGn_a1ShGAuY~!r zWkkiZS{c|Ck=QgmK8-T(quIs^b?vK3AQ5#(r@a3X!k?m`gc8tvvk zsZK7q2#sSm95Rzf~F)>pl+fCf4 z&XrNv7=w8MDlL0^wg%fJ+^p|6#%loftn;#RAxB*n$%LlG&BBC-1h#d-x>oauj=qbR zCN?Om_9NNnD~5{e@R}+aqQDaF)~2*O%SLKH^6(^k;($)*I}=^;Z<}&xK(j_H>EcIe zR@nEAevG{fSe_LM*QrSy!TNtpU1d~UTeCz1!Citw(7}Qe+%32}1b26bpur^s_u%gC z5+F#zV1v86>pQvkervtI^KZ`Hy}P=)x|(QtZ05=}h*`Cv^J?3V0n$Yuwv^%Z))5c< zCcPls(d@U|=7M9n+d^6`2zUWUtPxGw8Q4L0R@JvKM z9!~4if${EelZ>=9>Ld#He%dt;11i)KXIYCI^CR*5tz~iNAhNI%N4S567L5Wm3z*ss z0bLenTgj8!e?q`q9V#FTfgdh3UX#Q^#s>tkCHh|hby1!Pi=8^>o6&@0DI@SOXrXGQ z>Ct65?-4mp)TVpZd;215>&tuTt`roj5-o4g3*Nx6-XJRi1x+~IF>eh%%JZtyeT`XN zz~u!h%4!Chm50i@XUDS4GUw` z@8rgW4e*8;4nNEpAU3%neBz-eyp%yb+a=hdRje%qD>9$J+6UJYA|lTD$|z47$a+aoL$9`U_v&)^gbs&5DY_NNI4t5!3N`3bj@~l@MC#&oAvI&C&Re zq9al4lnJ4W@okdB=Ut9o0UZXsVBSXOH*vclTI0YSsh{)SUq!A1GrYKSVHu!#t>~b) zmw$zrSPkcXqZX*Q4uAif-rg*4LxX$yz1oYxXe&ypGm8jOJD1XGUG@~fJ*f4RZ44gy z1IrJ~- zCI<4nU&cS>`&7DguHgO2yZhu%r$EDn9cs)n5z;{)@Lps0^D8~iG?K}+^T&6nRl3bb z8=u{c3CQxJQ!4pR8SXO4zwu?;O7+l*;)+szbVWkP@xIw~dj*|}U7Yj6n8iZXyEzU9 zv4uA+z1bRo{_?Gpz~l`hQv*FsM`If2U0!O6M$~9*7{n{`%-bi+makF$olq+|{6sEA zGsU~!5g!eZGUGyOhXGc)8tw*^{0(411Wd&%O#0+x(GEsX_Lt*bs=heZJq%b+vuDKg zH@(M|KvDk&RLD?(!2r{dY$Pm8tRE)eX+XYV31`D#8_5H>s1m$FpNTHoHo`q@-5ETf}RhswZ7_Svg^=3 zWTWMkuPAR3zGr(s)9?N6{%VlPUEAL|ez3V3wB5NmcDUl_TQL&ujIq$RN!nQ`< zcaBL9KwEH2MLe@BRA(NK^y3m|&c5%nX> z3jW&B#dd5{*$m>nFQpL2(}=p8H`Ulz9M7awZHI`%OZ5Z;wEkd< zjJ!rB&ZHw;G&%5ZC85f`fPtwR{LsI_NZWYm3WItJOqVI+HP!22=kWvcVpUs1$YGUp z*PRzjc21i24*&jt`!y*}1#LsevSYS8^)g$0FZ{>+GY9HV6k0jQ>k$9n&? zQ1L*#zOG~*axT_H3U~8m8n=NDW2)YxBWkj0Oq`>2v8gUB0DW+UHBbY49rJ~R=ik|I z<**jm!>X2;k*F`p!KR3=2Pk;Cmo(-%#Zd+YuP+xYPpl3hJc=rf1h?CsmeJ@Jj0vuU zGT)WoUA>S;JIpGMs9FZ5g_t9H?USkeKsu`{eE`Aaa>cD(8;_CvkziQnkm zlNusO{t90gttYVt>0c!$T7(8h>(OTe9y+JeXN~%{GrUI$3KcNi&l-3ZA6Gy1oso^s zS<80+{OntguHhF*fwCC+OllfdIh}jii;+ZPlDL5sO48zZ0K>m~n`s~o)3meayC+B* zdB{dD^cE)bskbex4G?JXk&DIjW!#iAzIDr1Q4=ylx!l(v~>nbT4 zC^wcax(uzxHli?s2T=~|=paj5hCe!7e7jAX8ZNR?Do&xej&ks*;U7}TsvRn%6SkDw zU{u3VAE;2$lAj{I(&>@0BAPS`%(4zZSGpw5T2&Usr>y@acr5L8=kVGlj!-n%;G;5% z9YppLHy+;+L0}pXowA zese=b3Jx#QN!k7BlPIDB9Mp2RTTT{*Yv!cmS9|_#z~8|w*8r9s38K@ zIL$a-`@5A|-}M`Bmn5g2WxTX-jTj0+L4&KOgWnRXP=v_<#@!hLBTsC2HeX=(OhRvd zYu`uth-;V3RIV=nVM}pcZ0JQE0mo!Y_2>=|72_5r&wtoXRQ~GD9K0BFP-E>Rht6Wy zd7ILrm<9+HqY{!2ahfR>a|SFTOX`3G2&s#ICGw-19|KNqdv9XotY(P(WHi z8t*_*d?V0;nhMR9r@r_7U<$TnO}dc1VCIqs7QcXi@$SUuQZ68H-{0my5YD~kdd6mv zaoK0ON51A_#BK{ZeNddA53wqO;_ik z+a}RC4w#hVHQ}U$cxl3BjSy-aEaSd>6TV{mU?P#?U*53dtn_vdjsdz=XZDSv?lnH~ z??(!sH%ff#hXf3up#dfBkD__7SV(>-VStJlR=P|AT2~H;OeSVpp|}4DAw3Z>fW}k! z8l~o?YBGcQ!|hGD2YxL3ee};3Sp}Z=QaMQ}CR1_5({xL8Kgammc=o}nBflk1nPHHT zk>6VOJ`4`bjKu^U?-bM0OSzJ`Ab$wfloZzAHHi5!9_09ohhvcX>sMAYg$S*VBlV z7LRwrl>b?BZwVGevyr7y|Gry23N5+f4{||H_^wpjk!L&L$z^M>}ewxQmkUYTt|9a5;GooLJl#JDbub|gQEpMwm2^AJQ)!#we0 z+Fm-{44eUXbrHf`tItG84y?G{?!-xk7#r^g2C z;r`T6rewK+5Ad4^t8UOW&&=n{(oTji{;eoV0uo~pxqth`G)!2*I9A!q<3gb$!XT>B zR#$|oJj}blW>Qt6YPx~1l(kCg96d_;Qc`5@Wzi|UD2ayZ#j*SIjm_)&vvJSsLOa?W zUD>ryqP)WwJa1Hrcnn$5x7!Y4I5V_(NBT#Kn4Szb!4CuBs6Nm>nl%)`^YB=R@1bGx z!dI|#ycgtOR=L;h&AIvNzc$$3w7-~IoOHVEuCR00*I#D&Ox%rKsUO&BPQ%{F!#XtW zvdau&;RGT`uAashM}Tg|`-!~yU?$q0NL7SeZip+d3M>TsSPG}sGf%voh#%0asa^MQ z@g}++Z1ubhE8j(4et1AI(Hw>2zp=ZbDpemQ_#*j*m&||Z227ll^+-iu^}U>Y2CG@t z$?P}o&CA~D9QU#-_ReHZ+^pUHsmsZ0LrO~rvz$wKF+MceXvfJx|nzZI*(!>8r10>6oqxIK2fZL>SIAr=`S zLD&yLunm_EZl)I=g3avSu})7bkPOv8v&p%YRb6!Z82{$?3y|w3RL!8J6l#!+(>8e+ z7a>x>!zA5t(Jq32tkK4-)v_YEjPvT=y<#`9FAf1%PO4`{om&h-r~mT#Y)pIFI12aZ`s@to~ca8K7-(7q3&ZmFP6TL}$JmlGvE zt2(6V#D#-{77@lc(}(-Qv(`S>RU7>jBU8->S8MpZ3HBE8_b_3irN0w0e}>jTVp=rZ zs{4qrIN7~#7!hlAP^V{hV7C8Sx)JEV-Dy2{pJeCc1>ZUtQxyEtVkq<)3Uf(jt^F$Q zlv|dM^=C-2e>47 z1alC+nMAW?)P|S@9j{t&-3A*R976tEdM4M1LrWg+t0n?&es?N`_B#$ueMyBy*-xEr zWn*l+p9!K@!LL6J3eUz1vKa2Sa)`YvYS&yyHD z0;iYfw-45Hl?q%2B$ESy4y&1=X5D6GjK``h*stR;uVDTIb$Ih^XDH6VjzYE_3m$&0 zvX4)tsbbCQgm<(LZ`SPq}O#R3ROACo&8ya%nl#(O%n@yY#t&LLqJ8fY<@10=Y~ z+q`y7vP!EGO(Z-$7bm8Jm`7xVEY9;QQnPTp)SxNBmQEU$QzdCWT9eq5g{EQSqSq}J znh;zne))-kpmUZusqR0C^5~N+YF96Z)|6^<;frZ zWwpr=kJG^7Zhsv!vRlPxVop9n1ieep?PuW`;j2Ajhw8F!!}_{050`GSBtdyrR{E5V z?q4BB@0`kq$||umFB|g^oR49-7aVbqV!G@2f(wFcH{^%OLQ3OM#Q#*uH_T>6LQ$=J z40*rHsFP;E8A!dO=;KZh<&J04D`s_*&H6Uzk|`**_p}Jctl^bi?ki&o{zrJ%JH^ja2o8$v4MvPe}A8cfC<%U)Upd;TDU4#Fm3_%dB@wWSzAtcL#hIn__WP5xkd?^5Hp910 zJ$%(US2`FNf8(_076W4=@KpbDi0kt%)Ong_G!z2(Rp#^95{HhA(C#zu8u(x$G7{=G z;lN^;G%NG#(c{y}32EJ{+WzwRZ7f}j(CKFD-BmzNS%0IVpo5P$_ez~U%B<!|HV2={KqhKLKgrqz7OA+q`;q7m znU@p^om<}!+j?zxy<4VR;cIVR~LUsSySUBWX-;Rgn zyHBVzr?SE*VO4ArKd|+XoO~cC%$b`1G#mo~5p?}|{2u<06lrPKI_Pl%r-j=;-lzq9 z0LD?FW|gzyL6j^l>Sph$tfDb$p?24_UTu69Z|f2;(^tq z>>Rn-H+H(eVU_U=+RB-y#_Ckj1I`e?J$1|n@${eGF z<{qW;8vbSw3Fuc3wnnTVt;tdYdLBLn+JPT7oAQ6a$k7E(bL0}81u6a(T~{HkASyZ5|0{(Vx_;gQT4 z5A5x!e>>&Iv&l9buH2c@+hABs(N=VR-R!YPU|Yg_;JXUuuEcC+Xrc_#sBS{Bwd&S% zA$fM<(yHB4dY2DNkGP}JX~Y|~|MHQ;8gY>9`N)n-azog3jt<4olHA?=$cY@SMaM7e zT$T%o?UM5jL4Y6X*Jqn!DR<`OknQTqsf&e0qxsEJp6?!rR!OOrUQdeMPJUBuIX0h{ ztFd22w}X%>Pil%0sOs##Jo7j#P9ZoqCp+}-jPlECm&hv-{V@h0L7&V@XSIgnG8xf> zuL}(t@v+|Mb9yq!f5|oq7k|hA=j+MqK^d5@r3k}HoOG&KqqK7R@PIU3}y3MTb&>y5*3luw_!jkiH9JHym=^9h4ZW-$ZPs-*) z-L?e^NU_dg6^z-LlvNd_A}pXnd_E z%@rTZY4z=z|7^oFk>5Q+x{usQUyqzzWh$NpY$rM9r>cE-J;I*+A%6;KNzOie;%~{o zQ4pg#WQ5ORQd`ZdMeyQrx4$g6GPEv3?+}QB*-2e>fp;KP{`ifcj$}~Gg!E(nrOUR@ zY8!s%o4HbpgA~%CZZ_y=1-DE-+i71R`sst!`;ROeACm(xZl@_(%u1bl?FyXMN$}f` zGYph(0)!MK%F!y9NU3n)g2I9WF%aI3rkr(uLPo-;WFv?1>mxR_FYofkc3NYngdKmbIoak! zP@QahV7w{}|8YixiHL+6T=_ffn5cOZKT$_z0D5S>m3MN%FS)6DHP*_bd+Tg_%uVee zdx?Zw$c;TX!m?);MmACQ160;A%8MNIvOCThxcY_E{RLRSxAp6K%l~(PY<&`Pv_?X? zKgSnE+~jB5ZHph}aIMvRBb%NAUc?^A`;rVW$Di6K%iiUE!$_3nGKdMN+bMCW$s!x& zRhkg-Q~Ztm>Q6fMT7}AXPMpQzm#FV1N-`<*?%TLwuwwSJxLJ@L6Y1mKhX>mq+xwAh znZ2NUDGW^NHZ$ysF}-lI)#qvd3jYv0R!@aGj)lt7=6N#mR*J%cCHWPI%`|o4SDNz) zGb*WAaeI4aN-C;^BvrYvHD4Ng_(WOBS45YM*~L1ydDX^@;Ff0$v&b#6XLVI2*ErY9M&;5^(@E<2}sD6J_{(w7NfI!QLh)56G z=z2`{+>Q9|ACleSE;rXnKl4&Cbq#E3YSD4$^lECzSCY~BZ&d!S%TZ7?WQ@QjDB9|h zQB;xu&eff*moLxi@B|pV7!)rYa#nP{Tq)tz1G{BmAlEq z+xa+lh;&9hqQlDT4~=c<2-r^(catTrJejGhIW%ap!o^=EbbOGz3sxQrU*M)9-p4)2!0Ql%z4$b*$E=NEf@}j3#YjrfKg{XdU_SxSQ8iLj_aG~08GL0 zw93B2&$aq2F3%ub-lpj(DiV7hSI@=%T!m`}0lUpG0a(8#p6hEnNihK3rwZ7OL07yR z)pfLlPKT4s^U!io9neg!63%Nx?9UHNpM}Va)Nta!9TNXusPyjL$R7+VA5b^!jO3{i z^byEzyOB}#jYiUTBCoSb?)*-6VUBH+M0UYrG%5n=D!zi3`*w)GZEkh* z2EFqYW6{+fxrz8Yk8}Rbw(otwh~i6|%h55LS8gCFqR-8#l|#RQG|@PLZ=^Ib2(JS0 zeT3}$hFlM#+Uvrb&F$I=$9JDI^oBHrXc=@6A4jnE3_#a66!O4wa&i)gL!;EY0xv^C z01m=QlHy@8ruyPwlT^~TMo?+i@Y;bV+KvZhx4A-l^AR+{;&tTPH%Z;zzk{~5HPl4$ zVB!{FA08eiAjO8xAN^ksJbaqcbq^k#dKWZvVnC}5Dmlej{Z&KK@^&!8i^4Qwqet)s>7jjb;4$R-cZR&`-7^bNDU=Y@U*pk=P{Hn zmhbcRoV2|B;975EK!}{&w>}*^DoBvX1I>Y*L+LyWjl(s{p-6Az$KcTPYE*%EEeBy{ zwt45ZIG@+q;CiRjp*%>*#Es4;QaXrcEZ(`eqZ8)VE?FfZj*(g1U7@Z|o}cxy?8DV7 z(_qjN>>k$-qK%2;{KlWx9)+ipXY}QVEAqStLNF3 zfy;@M%lU#!&O(M`(;Id8iKY^VqD~2K$6>mTQ7<60Yo);vd!`|X>XFAQcM&zkQbw`O7(mV1aLcjDz5)6y9}4q~VFoiBql zLKTC9Z0zjQ9j$=C+Z(~#Y3KtL+LG%Z`R_sjg1~l?aA1S27w-Fm!S6^$MYulzlmMmE zsnaYw1rD6W!VG*1{j*_|^Y>@QZap>mCRmB=QhY{`LD=YltO|=G$#)IEYnlxxlVNC0 zb0YIs_4a%SZuW%NoRG7CLiZNR?oq*_1^xZ#g)>|@ui&27BHdX3>mL&Vf#KQ`3%q3Y z<4b;Bl@5YwnkLvXdv_Tw8S2zo((UxZUm3&tahMKuhNQ-**z_JU8{T;&LKWh z{3K7_9;u~*vsbpDK-bE&OGf*!nc3`vx*rOnF<3u?syaA>T_(Ei&cl>f%YI3E%Ti&R zz(a?rYM&jvMBS=Wi?lNNdS&6uziq-aIo@pj>$e`i%O<-{C&eyWZ{_6D522W*%GVEh zNcYg%llFjhPFr}H2ft9vPDzY3l-1yV1d`FlxBF4>cF~x!n!5O~pzN({?e6O%!)k$d#P|Iv?_+(a=y@T$>duA0_F~7g{9-f7gXph3g!k9`V% zn}jf*yS)0@H-`UYO{S~_rggRQMMJfM_3#Y{i@0Plu2)_KbM5`)3-z7k6+*U1f|-S+&395 zi@GAZs~J0zVj-NsJ}n8=*IjD_xv|i{DI)LNNH6TQJ8dTI8BN5s#25e4r{p3^0kZ$# zbXexE*=OAtwGntZcz)sM){V6hm})4!-vvY;A`pa=Kurmq(>fF5pR10W;(2nj+c0Qk zUDLt#^x{F2CxbjZC=9}9mf9g(Smf*KZ5a8g&MzP{xpZgf^9Vah8WEt_S-)Z&j)eAk ziE#7i{*OO}&m@5qCznRYHu#9Dnj{i*(yCHku8HY)6K-LJskq#SQl(^|5+=p$HP975 z><_bOlsK?8N!yN!beKZ;J(D#B$LCtcp`P_+`+U#i9sMwt)#tti?3#dE|4If($IdBZ z4W#BEP7`8s@-8G`*|%%)`cBn>J)VQ(?QMsXX;7a@%zD+YcE{KIq@!ofL6yC9Q*p2 zsVBt%k(C4fRCJmr^6wli_@ zb>pZ

HX6QF`}Nv#UzA2)Jb786i)*6+-8SsolYy4c}|xIT+tQhu_Y9ZwoRl?Ty4m z>hgEP!<&?wdzwg2m$)(1jFkJr~aXla&x*y261X%=a{%dk6C} zIFq*8XLNn}0|f7ruUS5x-fr!sPcu=HLrs5=%4=$L`#{lgU?-N{8DAT%456miI699F z)QRs2vmdTq#wzB@)Ty`6%O69b;~LMB`!UA#|9JrzNp;aYPU7+)@^3#{G`1c1;h*v( zp~|A+ZhFU8gpPF%_SdrNdxv#^#lQjKcg|88)dW&e*UX>8V`^gwv{P_+R68i zGd83xVaaMFup3LwBj(B@Ds?7zl-{$Vwy6y;yQ&lF)O)lWL8fFe;?Gv6&nm-+qL zI@zf6Da`Yrym*OtUCX!P!&a*^Lgl>XTxRv$77MKy)B8U;S>QMGH|j-Wrxf4~8yoen z*LWx-Qwu*|x*iJz`NABBVKl`TO7 zMu{CwQwK~14b2Q><#L+sdYuQp(Z#?x%7^Zxj-5yGE0Ko51c6cq)$40hc(ZFi-bK5Y z-{v5NVdyXS+OVAyZU@t_b}1Y1t9{q{1zNY)hjg1kei6V;rN#po(a zL_gBIapbl781#ARbS4Q%-P*5)R0!4eXb{?bt14~oGe8|05sMBWugTHIT~KpGqT;Qy zY~p3rUZ`5+PsmwP)K$lY5er6De1xQSuZI+`RBKKErR6IB;!8ofLFnf9b?aE~cW{tT zA~nAN{4FCMs3GFk1Us=@DT0cJ$NXDk?gq8m1p4r>)Tays;i_b$i5(^z&be9Lc6jI_ zCIUI80K90Il`(Z=*46q&f-MW<%(RP%Hs2s221k<;!RAIoL4MxjR_#Q@e&L;p?*6CA z%B-XNDlWX~XSM;X;4Md;n9qsx-Zm`Q%_RnK^!5Oy3dhL8tnv>8JC{hV`X2mb-m&e$@TcXnb&8KgOS8xSLL=eLfa*s7DA*Oy`r*N*2|_?u6OMf*xYdlS z-(S!pa+m@&>r{Mo>|xhir$9$#mLWm?dw}Uq;>gjObA;~j=1eNo)=iXhkDKx=u-54+ z7O6Q-fz;+j2cuHp*dn@<*ecPg_z5hCXoJb_ZNDS7AdQw~*LC>RHWG-*tSmDgQxe6x z;Sle=Tf`*^{`g%zv`IEojZI~wsUUAc`Q~PH?AM>~gSwIW9Uc$^*S(d5`4N*N^vE9} zR~LN}Ld*ld?+Ht2N*!KA*o_9D#Konu8(nF+ zgmH|^6}Eo>$*qCde(QgL#DM|;B&%&JU2Hm37d@{rFG5S=pssGueYtV3)8PP=;YM_W z@=7#r>$TFu?16`hYn>0EZx_5amMo9ewn!6-^!e zx|eZxo|%y!vw4X}h%l%a&=hDKJzqC|03A#6K!61-055o>EVz@iv10l4J?zyV+Ep{^ zM~0{)a4LG%lg|rYh)m?QWqbZ~#yI+IkZQS$wIM7pLM;{^=%S>7>(s-U!yl_k7ys^r z>_^?9AD#w2WjP&kr4`tgzcQOsj!?YN*zjSdWYH+(!6%W|IxIbTe|N9#;(A`a7W-$y z&mST+i3eD9BwCV9u-O>RY(L$9z=RaM$q268DaSzgWPEWiS}$Cc=;82g{*$W32UxkC z48o5pKQLG`9CSfVpH;m`{kVq<V5yrg&V#j)_frVGHj{OC5R6#Hu=KCZ%$DM1B-N)+leGO$Dqw9K%KCf35uhvm* zdI`sfG%?v*x#UI;YiD8Qxce1hXRC)f?j@tp#}-{)LuOOY6MtTg9nqSc=evEn`_Dho z7^y(h(inf@u?cSa^*V49JNW$P)PsJ`q52A)+yCBHP|$ma>Tz01R)xoaz+S71RMFtF zu+z|-2!GiLQ*zZkNDQ}A0uFyKg)|cvb*NJu3D;tVUX6#tw5~w2Zc1~rm3tdPPwR&H zy-TQ$36c0M@G$dWHN0pqUtPHt{z%tNBb6BgZbJib8$1SruJQ*LWwq+TTWRzxc zm~&WW5X^81yl^bn0>wnNVCc;o)E2G(~?(vC-lI-ZZk7L|&5|LZ z?Ld&Ei;Nlh|LzU(-+Oa`e6-mQ0x|U3+;+yMAZsor@VtJzbUM#6X?V6yoHtBu{KhC( zUu$-ZN4%&Rkq;Ys0|4MSq6H2llAnp>#!4^`=!)lbMkmB{P28TIA+V1?r^`{GJfzPr z{A`__jB;}qC0uNt?TWIF+>!_~L1B2jRaiLy!ZxVI#^%v_ddl3zAjT^zn->arFhUay zcyW2`x|A%=&N^RfM<@SMp{f3`wvT=W!S629KP$V3WVeFN9?Miv=~ICYcg9f|Iz?i# zT+Bn&!R78ncl~)^x8YRX_)xCUkk_N9@yIAFAici#`Eg0#_{`2sq%ey{g?bG2W9bUT zId#k0?OGQ5-n7(dlJ}OU!Ru7_Hh*sIeK`Ma*6%*7qvh;4KxnX+@EbNx@5WLf@snOR z&(gzHrKQXle3{@tZb~Csv;lD{yOmA@?l1FO;mTs}%xP(9mMcpx*U8wNQ_2M+vF6v- zOm77I9-TJw9hwNM+w@|yLtH2_{zu}`-Y_LO zx`)tPml2PZpY-|?A~moC!Xh;%`qr8Oa!Rv=)C+S|xxsbT3a2y>=$rQ^q98cF)&!3^ zTZE^d?&KX@xn&NT-IZR%NgmG5rbv`e{l^jqr!qSwBv=cML4oN7WL0-(-{17kVmsN; zVm3egMU2#ogK0MPqh(_?4Ro-Yw_TEB@xoOUYAV8r1n|zO@g00ZJZMqGmQjCbD2LB3 zgVJx=RzIx+ubSp%%&9h^>!dZ|43>bHPx|Kp4egXB+2$ya7p^!RqlY5hR+7~LJanPK zny8ap1niI0ej8-)WhMqf#|@V3F#KPU2?m@m3UCYW=HYL9*PWq!R%ii2Yi6=xns01` zj78d@{Z^eZt$85Niv}fAaK|;%WwSNH8gEgpB)fJr zGK_{bFkHHv^bA){R_Z;&YKi*2g_pEoRYD`qP)yC3o-?`dek_8uDgN7CA*xpsS{g?R zgc}y307CDp(;z%M8~k*VaiKu@@{5|_huWj&B6l-b(p%kq2fl`V>tRXpSoGINil$0P z*#eWR!rn$%x|G*2vY1BQe3vm1{E=LQ8dfR%wr-OZ^efa?t8Vw5U&EM}A8N)XhoIekwP)$)(bxKYd@kK6Yc$i!#g2i2b_AQ#(9A8)a?0sbQ zGF@nX!n&)S2&;HvrNeX7 z->L`KnU~0QCAsZl7IK9+7v6R`Pf%Oz8|98H_c zAZI2smA6bvqA3eg$;I^~-3tfK1*k~$JhVT+_gd2-nhP(bb32f(_}%KG5DOyGrxxY< zNE0uEvw1*A-!b4uRK{}4OmYr22+Kh0ImEO9qVFCwO|oy4$GS5lUy z%c4IvLnDEzOekTMU-vDC6=}>Cd*ZZ(mIGXTRPRp(gj>=cbar!-;i2O_oYl7#mQ$4; z+&$GAxU=fO@=p7q8A$QsPk^H%*!GeoX8qVzyUT98@fBZixFc@2m6%~u(?I70ka4B) zI4%wr8}I(evRo`HrLY>Pfu_fRdq?*-t99W^c6P7V6AZS*H6Q>fZ2+gHN_bgGA?PuR zgtWqdsTf=?H$_P|M1lBYan`WmDA6|2gN2%9f#S!U4MJu5Hn`p?Z+VKjM59l-kyb=o zbXFnk7O2tlZsmeQnF;4;g3xLM&e*^;=M(f%kkr#MQF2zYs)|` zY7`VX=7>ON|DdZ}C^p0hp&ZQ;WC8o6htL~_MiK>hqEj)lp$IW%#n@BS{-yR=Xx=&q zJzx$FFmW1E$Lpfg{G6g#BOdw1qzkO>x_^Hv7`U}8=E_OJjlO_p>~-$kfMzY`PVB_C zO1z#R01NkzLgxATc|V)8XDwp!$Zgv+1wdDRRF4#+-S*&+OOenfO4C%+JoQ{3QR7N= z4R~!VLXbWEMh|w5P|zWQ;i>P(g;v{SK&N9DP}eyaW0zobYYV8)$*EDa5In$hD{H>ye~H8wgce6P$0njfY9ML+E3? z)(pNr03%CCNa%3d2XOlQ)YvgGni2kTM~Fa#HjnmM;wKU%N<$Gj!3TJBTCgSs8lCvo zA6ys)x<`o#{H^no6?Y@1$}lrg;qDPr@dv3EYn8D-{IOQo9-oXr7d){Z+KM|Ra$4{I zyoKhJ$~tX-us+;>X-_)o@BHbWTN-&BnC{|Rq3xrWEzw)Yrb`IKW2EkZ76lsX$HZt1 zKp?T;ZbYi_jN6eI_D8MzCqQXrkN8^~ir znDAy9Kn6M2ZzkkjH5YxU z9mf3wJ!%b1JO~m5*d2b}8Bg0=sHZh}y>a`s*`ldMsRP&K2u}W{sjVMGJt#NDQA3C& zhD~x%+6DA(pz#)*M1|qa(sPY!@wU`$XCt60e@^t!f#Vy8`oxei2?-=fhlhIK zq!maa%65?gCw6_S?|0u6bu7NLC|%yrUcgj2yOaFw+YfE5Ot)F>maRm&BQQ8~6Q$s$ z#a4a6nmE_z5$d7<{RVl5@$uO(0`?v8MI1B?LwrV;|Az&%nvEzQuUkNx40n$}LCrt7 zQkGuMwt5sl5{3*nJ{UqxXz@>bS|>4j9a?3L6S3qm z1qipDL>a07h1*|9d3OgFzm5ze6)CK({1)w)MVwrlRlM02NiJ}kPB*z97v{<5-slBd zE#X)~?GN1-36g-XsBZT6IB@S^{(1j2MgG2jE#CNdiWAJh=0thua^cF7ydHrB=yZ6l z;vTjkF0j9 zOB$`GXtJw|%6l#SI-N`I9pUfVvb0{eP-t9x!A4aFW5jNuQCqxJxjQ9orw3!WPUU`% zJ_84~2$C`HLc!&lRKRcyNV94N%{US47E&c<iwPUrMeBJmZGROzO|y5 z=@`GOI`9Av;tGl@(T+0}AX}uFOHjoAl?>%hQvt#_$$&_+!2e)>aG>V@$F5<2v8$?T z$oTjt4u_SF-fKhP>&NnTBR>^q0TZ1R#ewR%b;rVpUU;-pGw4f)K|5f5gvv(}jp~Cj zFqaRp1-?bnu#@gU(r~TzD4llHfi=+^UU;IJ4`tPYq(#UQxx;JUIdB(4UOD3Qr2 zQU=KOt_o7Hy1Hg7Dwi$w{mJX0;3pd~;5DP8zGQ`wbWx+S$Hj62l?Xb?Ex)$B?J@+y za*!QwvnbV|F9MT_6~NQp1gB+7@J*XWSfU7&RC`I*su8f>YNilEXxx!!<7;$)(@agR z5>9AD6e|b|9*7`L9Op*hF&&1220%omq4+oT9Sv>d{WQ)ctf;Q_=Z4uC;!Toky)g*L z(g05f`2&x04Sk+tP6|%*NPezB7B^SJ1jAQ|qe2w9i~1_axKT&KsfUo9WmVHfCIZ&L zpQ7mTzmF>z@|PUnQ5h5dFCyTet3`Vhh%F@sf?9#(K8vpid|vnKQ<%6NejtHd;nbj5 zOUk@~KPg9ry+yJu0GF5P`7Ig^=8IWHphZjmHjXn<-+4;YPXq+Xk$AR_%&L!N)$HFC zq=e&ZZ7a)IasWEMu@1}RakUV2=}5A+#*qS^sE>YrY7VC`r3mCyb3p!vn=lxPpd6pB z)D#y3ZzGVtLeQ{eC-Rbbn_P{tB?DVkhvw=#R;X+Pq8S=sp|Q@1rD&!nb>)@))-Z+4jgM?ey$ zRiq#?IT4#FqWe+FwW8W(N~CIgq;Lwc%;oRZ z`P91L$~tNN5-%m>_btjxzq7T=f%9>u`)pzo%45_G6JsIfCxqmbd#ldMiL1J_SEK9l}B@6}l zOWR0g0V!*FfMUkYPb|i8lpqF}MKFewO>WuF*+RJ{3l1)>l@}vnjx^Q^(}>CtojO>n zwuLTh$G(B)_P8v&tr+e9$a>4DDz~U@Sh@shB&C%u=|&onknZl2KCsqR3x{&><$!eur5 zT;${@}%gmZnf?GJ*yH^ve~WV;V6Aswwewz$H%URyMn8aqQ^X zQ+9RrF(EmbZcF?gcdAt8UY5Rp2XFqL2!}NdrvNVbtfNbKzY%6mIU}M;;o7@h{K*AN z%F%>tEA(hW3fYdY*Yzb1yf_+1bk%!LM)ZczTkZyZe-t!Pz7fh%av=d2?T7e{ry~_3 zVUT>BQMwOLzv6j^oJ%~_r39%5r!3T1(=jPzDHCa}JQV*kr5tIi-*Eh$Qds^>Dca?= zQSbjP=0GTe9eKrT*OB$g6af*5O{dwS+S5JDn>_ul@^X}whlgM!ot(o=L>K8ja1eJ{ zJrL2o9j^BjSNo+EzhTYC8$W7ByMi$YLq8YbtL1XS=nShLPMJJjK`4#X9~OuvUF zi_my@auT1`NpmR-!u#_F8U9y%H3OZf9&&>5lgHuK9%b_QF4@SA*>(?;8J|)-D$Po@^&fWW+e$D;VAGJrFFSsr3X)h~^{OVSAdNTcdMQxJ*ucXtfh|`#Hco0)dhNEMy%)&Qe77u3(rL0NrVPFybePwzBTz7< zaImo(^IUmV!7hhdCgMOmj~GscOg`vPt7z~t@rTxQ1ak;ab#(>qc&XX2K3B3KtdUI~u zhZVSGkL3L~Wlff!rB{MxWnVilJWW-q*~j*fT$hs8p&jRMQEBD&lLaWzaA9Sl3gA`-t0JvY%Q< zv_RSO6+cc410)eEySbp%9t(h=kLATt(tkk^i0+$_U3=qi>|p#o-T42G<&Vn)8zC^Z z&^PKj`ypyHQ*W)oxz&C`aI-M0N*c8GVwmybL-;dyl24ClBQNIOXsnLhWJ;E5JM!tZ zl3BKSCvNHU$0f5?2st=zFz6e;$G(h@#tf4>u3RbDoC2F@<{6D!ypoNk_zZ;_oQv0p z!@Ny1_CV^p>F6!oresMEO;4A7dQ2Z$j9=r{YdiFZ0Lo^r=Hd}}F%7Z%KOT^$tf3?! zFTU)Sv%@w~V2F@}m>4)7Chs=GA`LFu9NOQD;5lAREYEtRJ()f9-&oYPu!n-RD!7OL zmJz*N0d?2vm8AkrF^L5p=-@=@MSlTr`Z#w!!_BHKl?jh%6(OG1x4={+arcIG=M(<34{RttaE+s`ugN0e?)I|-zOE8#9_N49=per8LFUV0=E%fP zcM6y`PjpFB@|B6wE9-p8i)}8XA|=c>w6N_ezo2ug0Xy4M>)MC{0aZ)&a)tZi7I2iV z_+aPn{KZUdpCt?)<#t;wd9$)_5%p?`$i-HhZn*XSp$CD+{%xJMV}G9F+3rdi(YF7V zAD1#P)b&EuIg%k5HBau;GBzNTB{xpf#{dE;A`)2rN+Ch2e z`>)wu{s;a)SariaI*-QL^3s`JDgBHxYJ|khi)Le_OFYibwESR6r(Q0{`Xkq$Vz^+nsY^JY% z&~aX;K0M*`#_#PZ=Rsdrl=xo@cmPTEqe#ko-K=^rVz$lmDACZtg4i0I7<8^E!Cqqe zQdGY@*ZOTeosix}5-m;(_nD=1KW{IS@;i{b;=ma<@;}w=5X@UH|(Ht30>Me}^N1E3l5j<1X`Kns3L>P6v0<`9NWQ~Yb~MtuNqtN%Hnygw)87IPdW9`GqvCXxT5(4m{%NiNBDR~=lfA%Mz=`vg10;xa`LX-h&T)GPp(0&SHZ@-rukKV5s zRFUfc^rj9Bdtz0p-5A8WP6i9?Kf%NVS4m&4{p?I@shU{3#Md=-w`E7^0AP+hNvPdE z%2Cj1J*HexzgM$XXWic(dLVDY1iSD>VxEF#0hiip#U57!DT`{}cE zhZ4@Rkp#eE-*4h+p3Zh3shEC?B4f@T%JtO@y{>oii?PN&3NZ~-4RuVrp{o_oNbs z>2M0yuAkA1!&wKkS3n}40-1&30&IDnlQMuQYEVS-^k0n8zvha%SC*A!v?bX#&H)=$2Iuurft9doBT_gGMqY~PVNgE{4hc!&SB($v-xpid zYjJ!3_EHh97(V);JH)l)7JFxu?6~P=~~W+)m-LU&%Wwr z#45#_;4kNJ&?5gQL_yyeN(SDtMEQ4oIqXvK{BL|A4qWMG%!TA3Hi-KOtIMS>6<`1V zD%u@94IszJD=x9J z(H8&0RjSm#K-3x(;Mg-@3!^T1ZWVT8m0H6b!a{he3;OI}rUS}%b+OjgJoMU=iNss8 zmds$oQBHeZwIl32T|#mwyNZqp7AywNsf6JtxMoGD`&xAYZi>9Ue9BT%S^^rPY*@Q| zgvJ2L^@$k7n&np#_DM;E91{_`+$(|r7?72|BtpEo_Lwjv$?a$&8L&tsaxzPF@9@B7 ziAc_68yOM+AOusKdKpy_i33AjS<9(N!`P1>!5w3$bA<+I_KARlW?zagUgaN55d8~0 z$oY4WClf_1rYkUUmc;9W@}fRAUEj%y8D20fY^bWrk3hIWUWxD~IK9GXglVKRkoi*J z-HeuVqm+K#h-JW!TO>m;r6XEKtOAF@Q((?VSijGVq`2JNXwI!}LEM45;IOB=G`>ka zykbPmes_<2CYBbuHf}Pk!;N%5fF(FhKmW~(BAHe0po4F5UzcY(DM_`mQ&xRIZCiaH z{ndD7G$IZ>n+HR<;TT^NSaIQ3`?ew?uAZ}u3R4l3FC(FJ{`5e%!XW#5OT&(sILUD< zoJ#%udti1q-YvCRauDp6tfVp(wP|Z8(HFb7KT;0zyAoXu4hS#9=^@F6Gn|t-(2berr z94g{H5=b|D$fORS!AV~@uWGlO0y5kGJHTd$T9M5Fj zjZ8x5gUm+L62B* zf8V6*&UA){?DWE$TrpCS3kVl=*OD0h!K%ROgxF1+QvsM|#FuzQDn&_7abQCOlg~58 ze&5TKmo-Y+9eXWh&f1~L zxx#*ASC>>-#F?KKFgb^~y=HKVej!e^>U)NbNC$S=W+@cPes}4I8+iTrKsl?ro|0ct?SYo7MN-jI_ zAiIUi@RiM235!)Y5?nHgIxo0TKm&N@mR!hf*JMmzX;%M^GV^V4m%LK2=&DyPKq~)G z92fb9f)B-QfEfLWNrsD%lasRmi{&`UJKJus=>V$Oh07BCKQ=S4gSVkI@^d6VEms{0 z%;#1t`+3b%_5a8SK(Q1guF=4B zI@=wKX-&WEI&WfT$Ha5|X(*P1Fba%6!BK+3i=x4G69na{LQ{+!sx?hr@7hN}D?{TI zzmaICe)IfNA8yVtvvro0=tAsJi`nw{`vG#$3kvC%m)w+F%Lj4L9>lU<>EE5~q_vN8 z6|tMF*`3RFqV6;`=G%8vpD+OgTtl!h$r2oEV1=%k9;*y-VI|k2u3k@#rR2Fsim!j} zJ0d#dgV`Ttp=cGmmoJwB5uC#G4+dw=J!ja@!*_!iACwP&aX6_4WLsW<^JSDK;_TX)K$K~QX| zYT>tpAc{yn#1GDX2B=5jkQoExUzLWMh8Xqpkowg+KnneGvP^mJd&_x?6#Ju^JA%ig zv{=?IFBgBjC!B!Gb^C2I;!siq8{j1*75&LvdZDIgSC!6kIiats<8P-dluLPnC6=_Ig03c5igZv#8?yWrr)cq zXL_zoEVgW{uyCHHsj)955BEiP#PaxeBu}+XqI#XobA4Q&*ilCN;YBgUELlI#sYC(z zu#hT*qA*@3DVED(6I1DhF$MyHwwCqrcP*Dl<`B}sHq8PJE?Hv{GLH7&)!l0g9^b%W z`ah^m60C+(0tddnGzTmfzkd;^lMhVz&OUn45UwngRWt{!m0Wl~y z#dLoQpa1(aD22?|mxrn|o}TyR790ywrvf-`$DrYQq^feBcYX+M`gzuyNbZp-<#UvX<43t1Qa1p zX=0;s#4MRR=p%1Q-H-sr?gYL^u4jC}whOvXKl#cJMVma1m`k3^Qs3&d84=Mrzt8I4 zbyd`k+ZgnJ8ueD@i}ANNss1nB<(WN>=JPH0j_3Y&=J;!FA`fV2Z0s450{04Q*~_KW zObg-eeUynLn<5|PQqObpIW(@;Xg2ZT#YFC7X_KA!Wt|7&k?;FtZc4C_{?&k|_X@Vu z(>)$*xVl!qTkCl7belqZQgM832h`_F9P^A+n-vYV)0qsS+f>C@o6ZlwJ7WH$Tb{*K zVZ4_1l@0L;pJ){@ezv#afAKEZG8fZ3i3^Qj$r7z$z5oMsyu4{qVynIsMdh1`BED&Z z)7cl7(fejUP5wu!c!ryDJH!kH9T%eGVD~LW+LMraxuj_Pr4j$x?y&};`O?7rEY|qs zIU*vPPJ_iMNOijP* z;0HiDdcCxwt!u3_2z0q%ZGYAP{uQ9~i{&4y z6p#u=4&7Lh7?yfECJ~I0pXwd2G%4ySZHUJ+5D743XKL>N$_tY(!KSpwP}5?v!)qrq z6nKTgruTmAFt*v~`A$j6FJ64)>3v74cQvykzjyA$@rZ~l1)4^-^eY?9crvsxgkGq2 z*CTz#!ZQt*HpcvwmQQ4{04m!5Qcz-ga>oBd$a~nv!rB6;tu#{6%@P$66XLumJnbo# zFMeO19?h#(;eoz`Y)GbOvySb|@fL6l;@m#B{@nF9EH!d@g7kVL355H0VH85^B)Ixs zzo6&aW9D|iv&)!)5N+CTBHN8;6VhZUrz3>)^>68y0%Hm`O&PvXKT_vO=cUYUYHnQT z{feh}SK7lCQ`%$MmMS&(3hUym3?mLT>-JJGW5HkG&XkMd7*HL7BSg*o`JqG5r9xbu zsWMn5n8s7Hysc>E5{Qe;my6b17FcVU$2I~+?LYC(Bn$m%o?5w9k6;1SXYsOe-~VZt zY6kB}gZ~!JSn6@V4!TOc(v$hZ*zfinu0EPC0;NNk{uO5a5XUd^0<|B{Fo9)G)M9OH z{x^54FRNZY-c3tUTisk3CpL0BRQ$wAa4=?{aod%Wl& z5s;zIu}i)}qDE}qC8g%hRUN?tf|Gih2R!(z^^OPaA}x7qMy2p^{gfZ8jMYPdElst? zAE#Q`Du7V!?Cdz#4^QT+%tN(iX72ThN$5YPqjeI3N}1xCy(c0*SN;7?4YAO z`jJB5T+$6~=n+?Plw@E0VmlsSix>wy%RAS^$@rmiD?Z3TdLlI-Ilfm_OCibX8Sb$x zhRtYN_V0s+c9Y>^^VdYgbiX~uc@Dh2t&WHlBUmKeBUsA!dA@(a;JC{?W(JI6wIgDcXYVGQU|0D~T(YN2VQpakAIp-&Ch%(LN5)hitTgWJ*aQ_?= zPMgNE5{2`w1U&kUXlFe`e~KdwZ0iHvFci{>!kjYUPadKcP-2vFcw@_ zt$09kTM^r2MPDFtBaX=@O?FxLuk-~l&`-U;ysXmI(~1vQ)JH*;OC4%<7W`o}o(Z;d z{5b{C39k&k6_eDm+0p?lf&_0DcPlgMS5*J|Ywc!cSs7)9BtmxCDT%WV(eAE4hO*QG z2QKFo5rBDmQ$zP$*y9J19Wy$yoMfek6WGK-n7|GLqy*J7Y$BU_5XNRLwy0UCJ-bG` z2_ixl%ABPd*^JiS6>@v$K}Lv(Vt;9VaJ=*-Tl3R?iCvkqgA%CsLVF#0l8Z}(`+3E* zRr$6h!vQVgRf;gHU5ah%a$<8;fkRonREBm$dcIq=$kkQA3-i&Jt`=>n2wRqxz*_t} zd-jgr$42z!Hydx;l~P~VRocW^aZUF}aT9 zesd|3@p5zH3knL#O-zQ1YqUNG_7geeXo8o1D$Qp0g8=sxF$oBll`Q-_OeCg8&5mUT zY+^;H*%?y)-SD-zZcS79G*(~ii&#PN5lI=Z&-bc0eEiY25$xfHr5mDTYqUnj zTa;aE9p(@bB>SpYE@dq?Q|L^}en>yWARaE&J-dY;_IKU|&Br&BrfY54F3r?0b`WO^ z2)srvj1eNz7WXIG=-!wRs;sjOARcG#{P{_~fvTT{Y2GCaj2%Sx-t+{E)1l^*NQ|}` zXC2ajrziHF89<;L6qD>~{{iMqzDD}HKO(qHTmpRWuiu8A*uZ~0Q8&Qg3D8+ykvm-D z>Xrs#%b6o2M_clqP%0+~PwMtNn-t!x2q7^osl3n$`h3_^)uxK4mME65#8m@NgNR;1 z)x)|*J{Jk7Vw_S|dosVGN4Jimh-oNj-8;>)tSDO2dEZ3rQ|9vpn3>TZT@1rF3ko3W z2?{-Bz=LoRo~&+*`dZabHD@$EVtrDb-#y`7TMpn}&GLuY-ms5+Kc2)nH(Z=?bKA>9 zCR_`|$7L}(q9qCx!}G&+4CizB8m|AA&yj3CAxk%!NE;IPmXeJrhIyl-d9^Kc z{eF~Fq=E1w$-kAU<)BHy`9I2%Jg_i3JBaiJhqP;Tt?&9yvM>2l($@cO7&Ur^8*=%9 zU)gNv%$^E0{k#CoZBp2760m8ceA+C3#l|+$hMy1=w2}a%V?;qF{9Xg)fMJ;ZyCIXy zYNwk(mN!L>!JU}Bm>116iIjjM)AY}?v=%k=j;i*MuZr8$&wnQSQ(t>EE50vy+X-I{ zh+X#a=(Z{|f6X^^at|CPSj!#QfL$afW_ZXG2;32uDMIirI0V ze|#fCh4?>31vqjMWbOdt5L-``H@JAHv$()IO8}U*0RKYEcECd*L-C6K+LX;jrL=Jw z1;+F0i(TnS$0h_U5;LUR`i%soGmZQb5+eWPkbzv`^bAICD~b(fioMOrkg^yalDOG$ zI~1>cv1PecM8D@_ubGJ4@wDP+IMZ5(l!*!TJp@ZiULHM-H&S;0TPMWi97P_G!GzR+ z0GtvN^(yT*6z`B!G?=|MhL$SzAk@RezL(E|c{Oju6;MQh5=Nx%ae*k*@EL&C4Xb|S z{;9WE%tQ7^B#^8kv*+L4Kb*yC5SjlHwBFnzF)E+4RXNrbW^!qdL4;S)R7uC%8dp&I zp%OHJ+5UJuXRYU!${C!hj@|}TXL2z*QC3rRoyI%RI@RU3W4SJ+K6|*qjofDKDHsM4 z!6w-rDrGsh#dU+2t?Kfvqw;+qxlhLoNl$LFjZ!{`1m`HYy$->qiil)~{Pb7R%v24$ z{jTOD^r}k&gC^ z2H6&i_~InZ!+b_$hpK+uS};Z|KUJB6^Z{v1z07BdhG+ohcv( zxXswqml!zR@wCv$O2~8#!92-@SyVXEBD17-F}Wp8?52xr?54x10DC>r*ZvdLsTCM) z?52BKKj?m@bQG<+AM^x;{H7U$uQC|kR)Y96j4VvG9e2m-@K$EJoi#pfCNKWYNPfIm zucc;Pq6qAu0>6P@o?M@~VLW1JMDKT#rb$%WPt)cYWJwnGWs^p*MIRQE2|rio3EqFZ z0O;+|Gqer%$(P?ZqBUCPA>%4*&NbWj5<>Z5f4pKB9Z|7=0S+XJ})1uTHCk zK1es=k3x9TJ+un6EdSGw>-9!Nn-8~)jT8uv%hfY7iXQ`lb>AYw>)-rjA7vanuCMi; z6>-T{t0tV8`C+uC2a<88AA1KasiRcA9?h;NV5-qS z{qgwwZ2%1KBh&}qVU*z_4N<*9fe~LC5|CobAAA1boPW}+jxmvKqJ*;iIJEbm!UEGz zErV2iCxX{jH!4UFn%cG@dU;`#K^lm<4A=ahp}<@PT~y!?qvw^bb3aeeF=4f(TCqdf0a^xXYR0{@#56mM+a!90PvCTpD4>F#gi$A3M(6n^+&M zbM)6Ikcyhj6KUpb5g41>V_~~t4N&Tu_nK=f(vK&B*d!~HL6lYPhS)?95gbT-d9!o9 z>~ySv24Y*1on(h(>VCC2R#SpD<-r*$^~!)C50inBcF{k# z|A0Ny?Z4aKkN4bnwr2jTaTrg90KuGyD$BFO%`ZQH|0lBon4O-Q^W+dCO%&csg3~cA z$02|rD!^Ar2*jp3Cl2(0JbH@2Hx$^wW~O$28UqosQh_FTosp{lGvU(#=FyyQ666j0 zC|RiFA3qDAWe8TzK&4wh-L`*oLCAUKG9^5mfm{X7i^sywi@y*!WWWlYu~;8xf)F$X zyUmi!EL{=~)t!UApDAp?v5)P^AROiFebqUZROWjMm8L9cPRx9C>;gEBK}*x*L1IBc z*mwwe$m_&PjFN$rESVrV!>h)5I~TxMUJO?MmLfJQ3q-}+P)I}iYI54zq&WOV~uTb*RlmFu?7m8DiF zG7RVu3*q`yi=!?;w(K@Q45jK;4j-e76}Wsq-X3N3?(@RJ^|fNB?fIte14^w(CMS`V zP*aZ|;}%6rV`;Vx)|fy;Z5t~=9il+_X3Z*xtSWa}o!tgtFqN~i=tU$+YaNizcDspW z#jn-JA9XN}cy!cs_*qCGFF%e{87R;TXqB1NkxUA69$Eu>R58hqg4+PO8}_3=SThCL zbOQE>^!A{vP;*1rHW4Bmd;6g})3lEMLEi!~On2ip09dN)xR&+U#y`0#So*s0eMNEW zW9az(ssV$v!WK^wu{S%9B1-LsKhZjDAmVoMSY7;v{Q_4B6Do%$O`b_6XO{ROZ#6$_ z9)~jturk0SSfn9L-vs%o?ViLbM~Can0Ym!ZXN7~rVC3w{4&BHsxi&cPdTTAc2iRe*#Z_x%AI z9Ka2VoY7~HXB0J}CmN{*ZN9Xei_O;7KP?$Tj*pDx(glqXqaHG(x0gR5x#YK=E;8v} zllPZ&`UioL+bhn|CV@>T!4$0x7kS%lnZLV;6<-Yr_knFiOjl}5k5ohK-kAKfh~ z11eps{MtXwVA=FC3=KAGb9^I8uKo3anu#_A_!JM61Jer_1IZnG8Sq#{Xe2F?t-ep| zZ4)asYxr~CXMRq3%p{GY&gie#dylaY8kmQVe`p=Gg#hrK--zZNZS!ANT)kGx(#xM8 zkX@JN%Mr9r`0=enbi^GeffATxQRD>W?UJtiNu`YzVZWAj*|UO7nDCeY!HAqsR}T8Y zS-7&+g7iloBwmrFZUnyqcc-IO7#~^bdU|nGwQ^yE`9!byj<#W+w>cbA7ZRr5uLt^k z{OBjompOwF6^{u$^E7(KALzr_=_LhSRDZXRA`LyIpEqdU5Hp}lA-#w@(@s)bYa49! z#wHN|Nag-baA8JyZDq=&=7!x8KR9(mkqHHsj1MZc{N#h*!;J~m_Qr?2w{|0opY+g2 z4~ei-=Xh0c+x~JjKiINyTdw@wI<{i|&BMp&)6YM#i{jJPyZ)ZsRz=~>lbuv;~~ z0OLp=E#qT~_I@B(uBo@R$VIMw%FXD+g<;0E{Fsi)=UQsi0smoUA6J0iMersJDI|c( z$ne`gWxA9j*UjiJNI}EXfqXWI z*Fr4gb=enKL2C~qp(`CPRv!a{gOTsfr&QQW&mwM@&%w}4jfck?IF-u>qpilP=GNm9 zIA99L5BwF&#m0~0LW66!f&2E80Xi~z<2*wAbNtFjT+UvBFSsvzBwRR6egvE3uK3hN zFqGAn)-Prr@znTa@A&#yX`lM9$+323PRK{u7QY+brz&ZQaTPTC%dEF!4O=_{Upm}LN0#wcNfa-%c(PTCA( zd`Lj%VLsAZ&-#*gYZR@Tz7Ym7w5b*rb=Ot*PpW`z!*3l7!1TJvrUOGFF-m=6+H?I6Q{o2p;_{V4)Q}|YvM*A1 z;rbVpVS_4%>Ui*;cuS79qC0Z|5?R|ZrqrXi7z$v>OVCt9&%Gec_W=zoe4ld5uv-dT zIQ1{XNKK|>nigGk1h}13ix2Na+^0?dP-X^c7BKaMFJ|x=E4q_G%q@;WuE!^}Yt5CS zyH}vk^z+OuJR8C=NAFJLyfRMr-n!voF#2$^kqbI6>e~JibwN zWE*{BfYqex(!(yg*l^)Qkfbh#7Yz;?ZGE7z5<<*|q>nkLJUu`-KZ7>x^(59B z9>)E{caNP8!&-xk3y@Zk<%k%G^`qFq&nN9Vk z=*GFQpre)h^)6FA*0;YvS?@SH@eQd#M@L3RL_`QoCPw5*^3z+4gMJ*_IgTT$06nsw zO-eClI7(1ceTEJRPPOm&yeRw>)x-UVcSZIfE{=~$@YR}^p zu}&Ivv?sYJNt!6-GxK=v6wg9=jL3!W z645tm$8|+>39F`1U~Rh>VQ{uF4qo;p0pE|ph6&p7?gj94ylv0{V$;&zD{IUi`aDTU zQW`|gTY^^|`L4&0yb_A2-6q~@LK1dsg#t3hK0yhmi&r%lq3rJiJv&Y5BAHu!=6P&6 z&BKbaaOh^>kxyb1Mvh4=R88%)o8vj-!wIr*-L(1=^AK}uIMSTQG~@8oYR)TLW{1DI z{bdm3MJY9m7pAF|FRga1gqQe21v&+4S7PEbYw;Qqb*5$=&&82l|qQAI`aNIL(}a4L`G;k-OWoU%rn zf&qfARvgJHNCmHFv}crk!$0jzAh+l)kAEd|fcI_Xi93`BWcdb>2djUlRB?T{M1=;_ ztC26TvHZSB(N=0B!tB95+^21HZ2ba8&{@aH{GyAkDckxcSYNMSbkbfYT(Ww>oo$Pt zp**XOhisjATO40Pi<^0M0{9D|{_lld&g`vIAl1OF7lz%Icx*o{7arTV$xVHfRh4Ir z(^=PR5#!#nZd5Qh5rCzu3P0(;{}n$_u1sqyAqPD(_{f$SDfs$TfXN}>Qob_ZoM-;r zM;YAae2i=i?Kjk$dY5{G^i+(ekR-m=kghVNjt`_(?p1WHeZ#E7kLy<=JsQvE79$n| zafTnavU(D~;yPu%?kXz<5S~lNXQ#*e)cgBa=x8^`uFqpc{$66ECiIre5#zWwa`qN0 zXu#hPL#>vKPSR`(u8A7unPSUnbvgONn? zR|Ixh-|@Q6QByo2yo}@`DWaS-g_f!r*QO&!LQApxK!PACewu)zOUIUG#uZPx43p1Z z59rRDkWi&B6Z7o-tu!!6zFkT6kwWKM>`TAuR_v`1UO;=ORk z{GLP3DTxHK+7_s%in^5zC2*i0sMryil&zzp?j3^>?a1TGX=5AneERgo`&up_iIXw4||$h$E-4HAkDKE}cZ znXgz!4Z;Or&t4M@VYxs#f;Ppnked+^lg~_~*X5lR<&A}5q|E?{q<~6zMG0?C_oVY^ zn#c==P{R3I*X%ah8eS^uCz>sn0zd26_0N?HPURAXd%>4Ihk#Y6mH=B{?SJo$Zd)(u zkbt;;kfoah$;LRrM==GlEH9YaS<|l1LLtqmOO6|g2}T6sHKU&4(J2Pq$mDaAX2qG0 zQGSzXUwfGUn4V~nj5EYQKQH3_TjogS=)>dePR$de*r*w6$h8~75k8H*#(pggn;?ie5X@?ZUt z(K+YfGPR?qWJsY4R*&C|-JE_uI;xIOZWsPW#s)=zceED7vOGV!d1j9{jh>nXnx?B3F&Cy%ZQ5e60S}$D(+>_W=kRYuW3yUUucH*%jVI5xckY+npxs2xPhba%kT*y5Uj121&PNW3 zY}on|(5nGdHVh9k3nUD^0+OiH^V7*zJzyF0aU>s1vwf3Wc*BZ^_Qx6U=PIL^DSZ7V z%3KVbI?s5X)2DDS>yjrr=f?9cFVS}np5Ey!7Tx7jrvqR?398Du7MU3n=5x~b&hsla zEXHS6-yv|*|H($e-YZWwwxpf$>`{0xAw+JE*1ZgSH3WJ0xCupD zLoEDuSy^Z!q{mOLa22Egda9saKmw7*wM78MHsLFK2d;o(V|;M==Y(KB-3hel1r}FU zN#TQNG_}vGrYsGsrdk~<#0+Cg5)56SvFAZ z?q=5LHl!yp%k}alnm_GZJsr(Py$_T?1A@GGw&9->3Ry))koAApMxx}XsYuWxyF|lB zhFUcd6;E2VGxB2B9xbPbNV!G0DsR)jt)GDk53aTm5ewPzHP1n1;WCJvQ}?$ym)#0F z6vaxu@_hz!md2Imc%GN~I3ldv-&*0(H~Lr{5p(+^%;$V=YQk{0i1KH>2V=ybXJPRC zBB`;4e>^1|*9rR*i!tvbjf5$~XZwZgiekm}x>^l7aIOo>3ka-# zqon&;%Glur!T_WSRx^{D^fOG^>BSHvQp zQ@(bT{V^1JQZH<@VG}&{>@SJ{O6Zyu4X{EDmPl~>?&q+s`LDvq4`U;XN7zd$xHwQw z8)x8m9DUw@?yQaySQL%>1X;d!&kafv{W|^QMX50x+B2$zhzQJu24`cd8oUJq#Elio z<9CxaTt}XW=|)CwS20W!ju|tllozKY{*qaVoA`SuhX%3wr2PzLh1OnXg?mPhngTFK zxco{(3Q&^XOdRyRp=uJffheM^;E$$2X|WZ8twa7C7lfvYud+i0 zD#tDvpwY(Q^g7083X~BVWBv?Oq0GT)!D9rxX|e|4NZkjkQwl5N_1J3oa>jKKC1xXn z+A)4L$|6^l9{6n^L&QVR!F1eszg~TnGEho4l0e&$xs;*X>X3nY%5AZfs7w6j1ett( z=^?ZULGu!A8Q+4ar1vB;=aCGvxC_JJua6UNELw|QT->4A{enD=rc!tm@LmD~)j`Jd z>KbNCJMWUz!XNZSB_)F*eWNZbzsGzYNW02V`fSr1yLkg>K`r(&X^ zEBfjzrsm_3mmz_Szwbg;g9J6QOVGYUq)w>~!t@I^+cbqA-GP>0%iH6JCGD~#0ez3m zKSc+g4_r*;8x{};1&~p~``y_=@gqfE}1^VvLZvp3#N{ z#+-va5I|$s6}V1S9t9Y6JQ=#>mbs^?RNZ4}<#l*nWwg?(w?Z&D4Rd>p+I0#CJFBDS z;O{OqT=I*NJL|fs;@S7O)t_>!u~h14+R&(A9IVdvm9`e6MLT&F~8P`*nB zmf}~{oZ?KuPZ5zyMLQ=92s54%ki_}V=V@vy_IpFezMnHh+@yS4tq40a`*yTe>k?kt zS!o$HD%9*B^#r+m8s}5DR?f!Aq)5L4Q>dB z%gh~9%k;MW6Xg%S?;msK&*D{RuS*;TV~5%I0hRugP3kLr3hn z*p?N7-K$+M7X3=2r$`yVwZu7ocF{eENSReLD(ozLm08XOK^EBM62%kz~vzLD1x|5}tn z&td)o!EbmKvN!>GvjsU$g?w;i2wZN)UH#{$<*r4|8DB%tbvoQ^{Y2F_}hy62pIsN6{!1U-Rhs*pq>9h8!1IXBLz*ybuR#7*ke>~kZ|u5?IsnQeNW@JBZHfIc3bjf+k3)gP_;#si(!|i-*Z}yvn&* zpvSL{nG4+f#ibwjXe<5TOxLT&4kiFNt;xg5n=m(%6gVVgB#njG*+AkX zJtNd;ps&~FCb9*#-J^=s-!}tRs)_TDl%nKgSTZ&}srP0B-*UX6$*{_gQjgx|0(j7U9naumpdD_FoFw$Cz>A<8 zlDWYGkB&SNV7BLDHV$HtammblDDpkJQ_1nLs;TzqLvQ%fRka8VX2ep&jO8}v`Em0_ zxEEAe*RkhdC8oz6e8%!a`9sx_fZ5SJA|{%C{n+mw+fl>plHQW0zTT2(rRtnDuftjN z@G^SiH-3U}6`F`yoH5TZTjtb_((M;V2ZXoX3E-kTdCO=rxWY8FlLkt4;rd@0RpQ;4 z*Zzb6)e<+j;0DE+n> zQZtR{X!!uqK11y@O8_gm5EvY5h7~Zv%6p6M7$KE@J$?lq5;S1yh2a>V7of4CJFKG} z4^l@%u82?LJ z+j&})5=GCw$@I4#w$Vq&`GP@E#aj*mq;=xY=4sd2*r2gR(romY9#MO1S)vY5Y?1$5 zM0tKr-=rto)7&tOVs14;+gG~r;e_?rOfG`3DaM<2{5{^b5<+}<^d95!-B{RK!rY{<#%ES006)9U zjdHd!KIqx8_=YijePv*u4tx)o4F#qTt5v&+@$&S;R!14Nx0Q41qZ*7Z5G_U!7SE7o zq7>`gu%3A^l8VSl@NnkAL(Ar(_0>Yz=TEKMKvOTfXbH~6_#b@Bhkkd2q%zp>@PFuf z@2DobcWYPaMIm$%A{~{gqVy_7st8h~1*M805ULPrLp zTe2}`dj9cnz=6kAgBIZw9HL+>JHgUKfM7HC{}dPy6eN|=4e@<9_(S0=1!?^}?{DJf zwQ)=AW73=|{FFe!zB9QhwkCvKPkMACf4ba;y9>*w-A>dqv|aB|Q7yNyFnPv7qWMMe z$$q!UP%Un<+) zHx`CS-+2kl{-MVDH3guR9p2`>`9Fd$jL0w$Iq5F;WkSMe^NxhTE|K2x|K9fWgnmS4 z9mk1&|Is>M?y=ok)3&C!v0UN<{x5Ik#ikw>a8HXCB$Cdd`?Y-A_xLGhIC#z;k`O39 z38D1Bef|{iVC3U;kcWPgiZWS-Y^WnlATRb1hQZztDwtgH8vS5h#n5F+5Ru>&T{@An zpEwQ#1x7aq~!c4A`DB> zP0pkGHpvNJQ%)l^ zTf;OpQa9XEneR%YZ)CxS(-v&6AW#TxWGB46q|ZUcS3Tq$djB!q4S~r@F0B;IJ!eZe#);fH>v>z40ZuLL5;0SZ zjJs~^S4BDkB{`Xmp7_3RUkY76K0Vz<-T>Vl7;IM0!y!VoI9cdi!N*_*ovZ2#neQZ) zuIhU$AC+dj3OZ3BfFBu-92>eooLS3e2eQ+b?Bg~xV&g!Aj*aFvO&EIL#r^8>D#o3& z#nU4W!MCYid}e1(o8SqhR2sf#z#Y~bqUQN`G;)5PeYi2eC|a$G5ACxTaA=hC5U;M4 zply_E1K%MUp4QO<%?TG?px!Utl>Hnpc!8$uiC|RM$M}}rP}aY{z#2BKZ=;@?1d%{j zZXI83s4(WOVCxBwQ-~{8_)lpYFdYg@JO)=|Hx8||0O zRGF!mOgSyuxxM|l)nOU^(G|Dlq2n?98@oR;$)_Z92&DuN6pCN8nC)57bWW^Xe$v+V zEy>p&iJ@SuZTfwcR%@VwleNq))D@&OKi zxSHU?rOnw|u%}V`8)eWlPyOA1d;q3yTP?z4W7WwB^Ac{8UqJ-9)>o?S1n_3uD?CU3 zoRNXb%{-X+*rLfsqhA4k;tY1V0K&@h;v)|0N(LvbZmKtZRq-=T!*EAOM_GCn{uG-A zx8Aw~=aEdkjT0{it6eM173K7tgD~86Qr-J*LiX?eekoJyjK#CtoCv&8S;<1<^=bGp z|M`3dB$sljSurX1SdF!fO(g{na?kaUfi#4$^HW^+%o{>xFHT~5aYU8kUv!=I1LaeKv z;f_oi2nEtFM6DTYlxj5mqYrC@Q+>Sm-}&r`2&+t^%0P{^%l z>6wGrDcPPpr9Y<%-g~zG^|^-82put2od6$#VUVCidny{CG5Km^_8s<6R8hvJc;5B_ zkq&#P3!$th$P6W!$^yu`?k!G~=vBq>flM>anyJayp}X?7;x6SeQs zwZ=XyA)yrAcHiR4t`6iY9BgW5mRgv@*Iu9zRiw1bdx`xNb_%L*E=AKKqD@0WgI()H zzhN#Zf{F&hq5Aht*8<;a+uSi%UhN4Md}Hh*}VEd_l|4B z%GwnMdi(po=JF@!l99fzj@4rqK2ZZ3`Jc0Sm+jFaVr|La@qqX3cctyg%}BkTLEjwf zby8R4zc2bLyX`_AEwsArmY}JY3=N|96qolE8 z*V`B&bH9JbltS*toR^4n_^8a0upvff%b1HAV~VHShGNy~iu{B5)0o9Wbs(bN+mM|m z&>ZGRGeq?NcEdd_E&=1QdOrg!R3#*sq`dMB>@@Mn=DF3tPpWoO+o|DSRLKlrf?pH7 zbBE6F!=>L#L3M`(>pqh{vvZs(?}5Hl9y1J&e8$Bh*m*`y{gNyUqnoAJed6)0j4RW( zGYYD6JN-Yc_+88(Z z(?-Wd$WVhJBqMoZ7sJIy;Y*CdokuoUUIdOhU@FU34Qm!!&a!T0dU{ltUlCR;!7p}~ z;wQ?Vy?(HesnOb_OfGdkN)8w}J_WB~?-XjKL$;zY$v3`H>0G71MSG1KZ(agSd~Tqu z*QRi>yx(f19yG4e${k1vvnZbTkMZ2lankWDAJ4&bd9KD~l3;8!6xSHL%D(Lqo>yH$ zPnh!0A-e?mebwv^DgQl9v=84?-2V4Y{i!p*RCA0_a&S2RSnpICeO_qoN^;lS|8Z?(<<3aRhKP3}!*wm>c$$|U4s6f)P`D#0sjt*1jL#lv zX3;h+R=uI9?lH=&S~nWjdS%u@TmFVHgCbe$!m_e{?antojS0V3rg40FOZX_SJV)iZ zRg4c+=6XU2X?7Z(j73@LEY)o5ERD7rnt#FbK<~Xjc_%ab(K^PaW4dbM0@&^nlCohP zcMj{GI1lU-8cj-g7FQOvzAaW8hHwkVJV3E6XAJwZQ>q|ZKy+mwTeh8d0DrH$h1;cI z5{{4Ze5h%t&&W?<4!67rk+=8$tDHf=AM*M0VpN~I?kRQ2XfN2ZvW^nn0_U&?iw@+m zkT1$^W@j!R*0DF2{ovps9zEH4zk;*(z6`>5@hn+9$|W2r4+nieb4be_RZ<*v&5I960XW z^Cf`A;NeE1Neyr#Tpt&k*xSToGlXvt`)KWrSTEERzqEN#&MAq+aQB0=*J{4MIlR7k zLrd{gP0B$%$_Pt6n+|2~3Q z?*X#qGE`~Wp8n

{CR*TO4C$5=9K|&nEh=hLlGhcSveB4vhwGuPH$XZKS*eKq{~> z6?8>!wV9v{e+T;Dk(_s@G=cid+d3dS8!noi)aPE~@5rX`75*lWkR=i{wZc9J#AHb; zVF?Y{-^CtvW7wNM$ZRoIY?7?RTnW#vQzr6Zb|m#!+?& zto>0P9gR|Oj$$569AY-Bi=QdLUeb?{&?NB$jj+nRbT67w5H2Z9ZYm%})DAplo@#u% zN;^IwOEIlurM0N;CU2NOA?59kX*~nL*vm)6Bfs*YF_uIs_%N0S?_|heiB>1h#tSYB z?>VOw28l~^@;!|ChIP!SukbkUX2tB5ISu#mbhn6;oCv-L8xtcd`xyP*mPD?a?dj2E z{r<|rf5YNvRbG*-)+Mdi?el<8`*&h?}b?3jkSI>0%of3BufW(To%mX z^Q~I(S8wSM$f)7+4jhgQAb^VOrg}J`fz&2sDJvPiBjEJpF+{7@@9A|#{D@WYEekC+ z8jUQVe0bJBKQ;?B8BsZw&OETc!jHU!kJs^a6(5-k)YxeJsS3-Y|M9_vDMq$cFQNEN zA9CW;c|rjLvXNSw>1imq#AWXrG(PflxButXYjyhb|9$c>+|k2_ho(!Zk&}15k9M1m z?A@e;6W-K>hwTRc=N@%3p~G6^x8>w^!eCm0&T?t~t{ku0mAs1DqDs3TP0P-iaw&bW)hCJt=x8Rk*byZ7@NTfU!rWlKf&% z$x^{@uLs$CdyWM1Pq*DyF5c@#5Rh-qp!Wk&+Y}CpsQ34DYnQW2M!f;{C>0G{KZ9BQ6_wV~OW|+`b8fZfWHYKw-pOB9u`3ExQT#o?^7T z-q*;6o=4%hQw9@7gR8u-dZEOK{4udLV@|~D!k||&;h;_qp>yN!;lj;h+;Oeb@VT#F z#9uz10s?1>@jG8>N#y6Tp-UT2hTE47cdkkHYo;Cp8urMB^thy>%Tz= z)(1>WVxK@+_HnZy^8Pkc(@-?$a?4i;1t5f+8IB?esZk5@eWn3d3rOd792YQntKv(o z^Y5*W8=o~&ucg9nb(e*iZWu3OZV;JfN!5G#(vy`2jIWJa=oqz``zPU!T>BqfHcqVk zxdeVa=9pjn%NjMMm38nq!AG~&Qa~te*o3){8+%S9@~$pKU23DzLYr_KlaWx)3EbD! zYb%eymCw6vY;4DSz%9yX|nO_K&!ipvI_OXo|i*%{Eo8ff;t|HRhY|&JbwzP zh3Br0S@3)#z#{w_`2QI+Kl4!qw=8HW1rd=JEalH?4!#S7HXC82|AU~3isFbi(U=ot z4L?|0R^N0cTZh{)JEI^2u>=SiLS_et%`2AXatq#2C#V`Bxd^ z>TvK$Aj3kl*fOEBZTAz3O=;MaWEAvWBc2RM=B_A1CzF06X-1gONN);zUlBdyl!g{_XM1ED>ebstLMepeRFM;SwEB|K#}|$Q%_qrPwguZi95XMVFha4rc+>_<9||_ zY@B#vSH|{Au)k5C;P{~Zf{{e#tCCluzpDdRl_x(=J*O(-xqA#;U;BIIw4>5)Sp#}b zJt-3=;wowNylBMf%;ucJpJ(F*{eGPH4h$wZyK(KgebsAmzqSL!Vp(B$zTMd-&t4$f z5!J_g(TJuAl#U#uyCa~25b=T%Tu{n839;{(WL!PB3|w5{KJZK%sbgQT_V0;$x_^eV zrrHpMCtEmTTHKC2>xz669^Ir4QLFdWAyjvl@ch`#b|c!F5e-Gp*t)?{N64ym+s^2jwpQ0*!p!lDs zSW|)OJnPGp;g&bsKtl(NulUC0~_V z3FTkn@#jMpINp5mBJ(wf^9uK=e@`G(yhs1yg0{pXTZOMP!E+B|sGWNfQ-7`DbL3DX z%!czYhi|GPYZ!{I*m+1sOG7HUKUEw()A*$lnw2+oat(t|IX%_(ZAzxz=C;bK4OhS^ zI9m7xiB1F=NEB?FHsxnJOXC(W`7$C9_YGT7G7$2=xwcbf?J@=MuQa3hF!_`4nz%l^ zo)}oC7GR((hMc2f=VJHKDy2-@Oy$yO;8f3&p;i)DWTqQUD3Pr# zsv$g#L5!=k`zZmh3>d%s))nyF@B@03vv248ox~5_dPitPPYDnpj6+ZAKnm4_8G3&5 z2uaMXV48%1Vh|Nb@YKIl<=d~fG%S-NqAl1LZ)boK)UP~!cf6az+F8lDd)NOD+T-N`S=q8mGX zc|%P0JPZ@ac60l%(lGi2y2b>6Nt4hF4M#dXDxp zb7{R?8o18hT~yHfC_E81p%3Hzp?aY5Ro z>dJlnVlTV>t?B(3cu>;^-@Bxqq(_ouPDQoVQ+J5De%!3xJ|xSN|xotQD)9Y z>dEe#`1qQj4yQM9lqCIHDP{BmhM(+fU!tPo;_O7t|2o6h{=+D-A*qLkx+`0Il$Y{YFcf=>RhZ~yW674QmhI&m1ULA-Q#W}|er#M7l=*izI zy!*jbRo^hCg%VGMx$sxWmM!YvFakt~*4xK4g|(oUUAL>sO=qdulRod!a_RQQjdPhQ zj=vs+%uOFn1ZEd-Y4LX*xH~;f#?;V9H#ft1oF;B`j;oEo_tBOQ1{Ul41>5Ja-{)4Z zwa&~bldsA3gRLFFL(x6(nw$0>YQl{VQGGR0Bz1*6jDoNtgX2@}yVsydZv05RV4i0z zsjb^FV@b!W&KzICmi+a-&`wqSwhgnz+J3^b*d=VPj9Y-u2fa+S0 zV~*wfpCfrvjLI>5*Lu6VrurHM_NPy5J6bkefA5oT)4wO1h6Yy7aIh+=A!jGAkOoxzuViz zv8uA1zx1I=P4temGMTl21RL%5MCG(lZSpLN@;xxYZ8BH zdaJpV$Ov5h3hB4}<`YueIq<8?57l=p`rC$qs;bSwwic zS9p1#Pzd0`IxMLZkFKKlK8%$jU-6$V7-`g(e8IpIOh8V0hLE@TFKCi%s){j9vJx2L|C3y;1_ zcxaLyW_%pJ2`5ZX@q!sTeu*=8IPzrhKBvlvQ)uydoqT6ZJNcK~Sgs#f50U@6@Qus?Rs=~7>Rb5=AO+0Xv^KWEhQQg9FTnVm5aDVvC8xiCwbT=}Vu`Akc zErjs~$N!w$zn0bQ{KL9~wd8j%OsXFZGQD5f+1%_OjD1M^FG?fm%O~`s4j*nYl&>xu zviE-b+s8;Lqm|~6sOhCKK;9_l8SI0;y)WrtA2XSK3-+-l+#q^iM9@||)0k>u9-WJ} zM{W(tH$Ti9p!_~)kfMEkXI+rfU}QrRR|l?-LnJ{@Y?ucAkzpCr{Wbnq06~&JE7R+T zl8k5iw*+r>7kxe^A^ZN=;_%A9k%oaA7Ht)zwm2xR!|%dS2;B(XUj#MZ3 zgLsGNcGaGETUBOj&=Vons5-tYsn?Yq?_4$=)&(mPDhiJI~K=Hdy=t<&t zhYFR}j&#`Lz4lJg>JBmTn(HuxQlUv$=^^JqDV$M>l1>&!F%$b+^rCXS&6^3q?qXP_ zmf7&X{5-3f64^#2{ipL}sNqeZYJ8{<(o;EJ6C~K8by%%&huxe7+d04c?}Ujzj~N!P z<{CWaA2=?qZf0CLR`Qq5m9>0tSx33H(=G`wu33%@L^Se04B^JtztOkfVrcsDLFzGs zcf=}k!g-N(z42MxxjnS#tA2?0mfS!YG8MMKT{kc|m|E-_A$1YCQ-aqN?80&-`6l=` zULI#)dx<#8?~J&}pKW`E7TNV>yMvnb(_>ck3qD~az9l2aKO>ge4>dQ8>k5K;oOszA2~61510gV8W{`!dvfP>=Ea+LuD$Ijs<(Wt%VlHQK@VNg_ZDu3p7KFtf*Wl3( z8qbKRoVAIA)8RmCJr@juRRS56ag^45a;Pyt@U1++{9_iF%* z3=N&ckJo5hTp48sa+(#5R#Q8NkLUY`H*2EAHznWXG`M2b@AXg-93y<}Ir^ z5TeSPigr^ZH;K(-@D{~aU9|{J`B$m9<__>oP3s4aW-Rr*HN}{~%Z$WxWOxNNTILQ$LYjI=*_C;_PEY&X4yrI`zq3L#X-rpZQd{ zc28Fk#i1QM5QC@7rNrsGtH98a{PV2x z5#QOf1KOcOA0L+G8WfwUP@k;u^l;2FDB`))`ZS+<^93TS2&fUXp6v(!9V}ZK1~wUe38kX>PW z+jFfBm0Df!o@CnDSZGwsi=`A2Fmp9;ny`EY7FCye@Lpg2*o=RD@+rVx1*L+C7XNwJhO zbo)o50i>=qul2g^aG;XRWdEA5cYRzgxyMheHh@E6{RhPzq1&jBfTr8C-aQ$Vs9)%* zqBCu9*MNAgQSRG?;HtpW5Wagy!<2zzwLyd+;XxFGQ{&>iDWQIqw$k99L%gS}+;Jd! z{{q}q|L78!(^uN9FvYpYs-{&8!rBDVRN;%`ax?;i<-)eU&lGmcm4RD_CIMNT_R~O{ zFuD1GyTy(G2t@_ns$8+%7i6#e=vC^Q5GV*=V>h??-FS;eVM^t=(QMDQ{`pQM^C+_* zVcW7edyn-?jE@`0nlZ7xd?)b)KcZ}Jba*GI7=NO_*Y}{AoAwFA@2J|YTpAY+>#&{t z)oXlf3Mcz3bRoaxq?5acv;nh2XY_=@sgSLc>qThkf5Qg7n*cVbDgJ*^G%j%+j~$PF z_O=xwJkDLaJ8@#~8(v-9x%7RzW3M020{LIPf`Q?u2+h$>jz*?H_=ZMNL*Z=+38N+4 z@j=9MMVU754+6QiR8!c3xJ#^!@E-NzUiZ?GCO)WnSr92rkIo@LQ;!MV7f}!5}&ju72J9^>!;KQ@c zNtm~d)cxg|MyZ@V-f6ib>&f_*VW8Q|%{T@1dMTE~V#zAy04|BV1|OZK4tnf$^Og22 z9v=B_gI^e%d?t3c5xwu(G&kz zD~OWw*M!NjKsyd_j}3Wf!DZ2_I;QsX-FoHn@R93gdp2GSpj(6CIB4*vkE|LmL&_vu z1^r8_+p~WnTc~WzmjMxY_Bud#xUHlz;|bfI3h8@IRk^&dn4d3*9?z!{`w2lu`QA5( zhW+p@#wHA_8ob@~`_?&Ivo?P8{v$T~CepbsX$vPz=2y7D_|VIbjRDbY>)j9K{0Kfz zNV6Kp<7h-nd$LfgUOJ|rE%SO}AnoPdfB>FF>~JOD0;b9GqVW+(EV+vyNe4bRwY)4= zx%q7BLnKD1vxP~vQ+qxrghk&i%o1m(8O=$d(vlz`@IoPcm)SZSr7|nicu&>cB{+o+ zT8SXfthZ-MjCy)mevJ`kXo{cdlK#rVxrD^DyB$$prXQ$!d%ANkL@bq=#@l73(!9+w z&@3Movv%fya*sh%FY?I`oH(`q~MQ_#{{AL-Sy;K+T$JtsK}kccCpPr zW6kxGIvAh-@Hd)HQc+re!IbKb!LrI$NZ0=~KVnf{v4SUv32^6KzH-U+jxt*Uhn9*z z2EMqvguR~T3N7ctn&fCmv}=FRzodg1N)qanBA8`KjWRO+Z?2T4|`UWWDwID(5$n#L!}mhY@N&RX&R}^ihZj- z!2y_7Yqpo5C7-p_!=Yg1b(Lz9KK`-25pcg|Dxjj|1+VG>aDrA<@zqJU9@iaJXkB=~ zPvJ;7FWB0lW_N9sR;@;E|K>Ufnz12X#sxfsU82GszEu|{n_fQwXTOemuJ6kWQ&bI_ zJJ5Otiz??d1%|2pJ=x|8BUL^kEEjfy&Hl6sS#9;&SGa~^{>~gU!*qeW!ZOTdXH|v> z6G4tLHx-Bq0P(JHuyyQt%v=rpX${@L7=9ip;aAw+Pp{h}G`TQR6eRv@WAX3N-Y z%OvYqL@?LsB%wXw?VJl{G4gapHJ)t!X{yHu3QmPV7R7guvJ}6o)w5~c*Y)K#%4aBW zm~xflr#$}g-sK&=(a>?pTSpq}q^qTc`hg?PHi@2ZR2c7TJfN4+#x$NmcA4K5XgYhT z;ZCw#ONV0y*S*OvYmEWp<(B3S4iAHNDn7Pfx9q#xOM5kjf_ zdsP47ZU3*MD(w4nH)QWIYnXNUu)}3m=;er*u^(^0N*kuR=s4ggoAT zjv@igGM3HWT*VB7JulE+ot_QL_=EvD{3g?D;xU>;7r?(iQ<3$Vr^=r(&5QeUdjDAX z7iO0e%6JlYuXCE+S$3(lvM>{zTETY09hG?zGMh>`??u)gG4wtTWN3^_OM&_2Icl}4 z`8H8QVh;mIfTmMuBZxgb0o^ZsIx+XFB01Qg5Ey##g5Ms9*R9y96{a~*M$_(!l7ofB zTw;X9un-FLA|_$x*SYg{rbsVz!8QmROUi??kUlW_5h{8>A~@L55CtdXOr6A7LxoX2 zXg!PKb)aIWd|$A&Wy5Y~^cO9ARZTwggazxLcl#Qe!BC(gk#Bi1q%Jl@Jp^{)1h2Is ziwA_mX%i=KhN#ZdD5t()<#yEUp2_9y!C)?TJo-=%1)zzx_KTnIg+|}RbYnA~RGXY# zQIHJ#lZlygdMVSk=hM5q$F<|CNsgKzU6o6Slg##1uUDEzBTR_#@)D66ah2WrY;gaDfk@>e2gwha@o{s(Gs!i~g9pP;y{tGWyQ%@4n+X>&ncEG3kkQEU zJ#xG%j1nu^7V^%qR{ri9?3Bf1RK>KP;Cp~;T{sS+N}6|M#UI6E4)VxF-+BGcEQfP{Ilnc+(d z=H~84HZgZaM;u!`<@ldckf;vk>+{a}N3RcloL3oe_TTYy{k3M@Ka+o0$n&=9$E*KP z(7Yeg4CN|H&h3sVEMnU@mNonP5|-LbkI*e!joyzJs=}?p{R93QZi;`0TSlJO{?KO3 zy?^)wvKMitQFLOp8cllhZ3+5U$O4bq8`z}j)~OB5&tO7@bLqlJh1P(wE~L8czL7ku z#82G*gSh?CD)+@Z)0cIYsErio9QyA*cNAwfoC*KJHfc{42udE8gygP!6N<~h z2kHiVs36S-&e}@dhMY_p~GqiQJMXyC&x3Gpp{HKYjB9oQSwuODB7Mh|*;&TmvbO z+v7t&!mTnLC3e*0`g@vd%4}B+?Bj?tIG*H;I9))(n4I<{fWnL$gme%{9|?vj@l!&! zZa({3{fN?OTyydHB*%x9!`v+x?$~-2v)?4%Fguf%gxb@IF2j<+Zmu@HydPSrO=EISSWLW^a1cz!t3$`hO z5|Pg&THPdXHjGMwkEQIpXjMGlDF4>?j?d@p=_%k27Q-7@j7soDhg>Z3t!@PcL7H}( zW4SPGV%yc>8xSvW*}ttMfGhh=2iqL*imM4cTuH&#sU-hpO1V&s<;nydwn;Kv{-yJ zo+*9kHY)w(JQrrVJSY3B{U1D>DparV1HKzZJ(~kUt(h2WHU%4?D?~FqcG1pPCWAt!v8PTq zLbbwq;2PJ3ke`3+?gYY_YLaaLv2A;16Js2R#Kf+p-nmjR)o5(t_fLfkeC9M5=ly7M ze{v+m6*tS;0uC!KN4E_6S&y~R zecei!WkwgY)Yu=u3D9U7C}Ou)W?VZ-TBAcd6bqLATZLlHZd|72@a*yO~qG%&b;iW9IIt2 z@5GQd9-a4n2CL<2Z8<7i&TyDHa1;x5PS_X0HrGV=sQaaacy8$acnqRdVJ zfAG~$fKY5CwNaz@m(JBczjI}#izmYvcUaP6VxHl+1jw$|sb|8IwSBZoe?~=!ZcYM# zpL(D!x^jEP)$Dx^Fw+K6U8#DXiORRwDF!dwlfB-djneQRnTazM1iW(EAPAxV0GbDg z-vV~UNd$C;tt?E4d4Schv5`6~H~-^W_`SXBZw4w7ryH*p@P*NTSJSr|ys)K24#cW; z>zz-UU;t0vtyNmpG6tif7|px$wzg8T(J8aB7k2AV*Yg+s3EEy*CThng)Sm7SQeb!4 zt^G(9=2DZYXN1qzkN{L(<ku!FV^_0lhu&uer>bTZ!b8Uh6It`-aP&6#eGT}97ime`PjG_b!$~l2K5o5=R1cA4!4N3WoBrB>$IhE9-1@=yxg$|8FoJMay zPphi;?S#|^J zVn=jstb)ivCkN0FYQ6iu~uT&Ydz>_ihXtX$83h4aRpIGh2hG1N7Qb<9-=kYKR3;6TaO|d|Q*H zppASj`{6?AXY%)u<3XJzNq)-o3r+dbLzjlHa1RIKSz?bO`8J~s$0*#(bO2D_x)9oc zxWbzqjVwsa`;NBzUs)ErZ-^-raa>m*{Xnw8h%m5i0cRgC8bh>thc!_fVTLUm$d=K< z{UKmpkbN;U85Tr9)UI>$Nyki-36ldfN3Qh-K`4n+d8?ncg~q|U~hTG{^mRJp5SSihpR>XI(4zj@s*FUMs#!5w^JCxFS6X0cpGAU{vp z=y*DuoQd(V!eEEr6-v+=aDG4|e%AXRWi@Mzk*$@^TdfY@{SGx3x`Niq6JqLuuPBPjpTg%}ATeT(sczJ*Jhl%`^kYYm>4 z8BzWKLA!z8|7HH%F%HWW%$oBqsN^pPnG*=?Few{G9eA!Ry!&q4u<6IkrAp5Bl7_PP zai@ZYr5-+Er8HLG%e4S9#x9oYzgPsxP5Z5zc%i5HzeN5Um`Toi>s$8krHL^B*>KXa zM0?n>uXuD+a}~eEMM+r0b#fT4!u3eODDz9THsp&Dl1+3)9>)hOX3Y!KaHDoWc+3-bj3gC?G^%gS=^{lE8y6YykCB zdvdyx+XnR$$y7v*t?EI`;MC0NTF?l9z6agYeWD_9tQ4Tz5NRhOKT^sQU|IZVgxSk7 z{wZc|tQHipe}jQsKPZ`t{`e?jkLrzI5zj5vx6*YcKu1IEUqvBU8TA&vGBPxkTm-DM z#ZlXGIG_sBGb|;AqJ858WwdDOOnbPKf0gj~TFLG8N)o;s+Jj5@Au- zV=X#H9F$esoQarB{Mgn$;~lBN)D?a7d*BJ_-&Ay;Un^T*l`zU4Vv35es5L7yYHAT zbkuS8?B{9W|M2s^6gYPJ6yPKIvg0Yny#k`Vv#lq4`BW|vK#ZURo^P&5zIe4@{*%h_mtWzL1zdODsl`$@j8XGMPj_Ws z=_SuX_B%ssB2uzWM{=~oY%+dAJ;Hnyd3|Fg?LpD) zRG0y#{aHK-Uc);?9%);=Z;Db9(-hb zAMj?Y+ z@QPv%Z1_hj%Z;c^8bozJT%$06j^Xt#6aBeSx&T6Y(GCY?Uz336{aZ$-MOf{qLl}G; z5}%le)Qw7bY$=X${F$yc>T;}d`Qe1T~ zn(p`hv|KXt6q9eYe$zfq2Ljbcf-N-hhN&Xk9{I`!)C zCcj^@ZOMG_z#WXF1IJ9DP_ZZS+u&FDT(y49YX-D`!nh>a^I{yr-e`yJQ5-^&P@sqt zL-yabi2RZ5J~hSXq`-5dJV@^%vhY)eleJ@LbhyKV?Ge$;sU(_50yDgypBesP}We5le=u>Fh zNjxBC(Jy{#XdIc!3_BvfvXQwcdHEMzrQ#VyuUDs_au#~VXD&LIBxYVt@6o^K@nuJV z$NS#`kPf3$CrjvGs{0uZ@mFL{-EQ>TTGxH=w-7J-oqZ%KUkuSaL{(UR@)CJaK-MR8 zCM;wIs#P1TCd((uUwg3i%WF+f^*2xZ?i1BtW4E`MrEjvT%le!R=6Fqrd(OzwW zc*W?wqVSD{n%+w5fiw*S2nrE!r97ewn8bp#Kr|wQR9_)$I}NW9@;0W(f4UxtgH!Wo z`M!$|+esZ1^&UYf{QgxBq%9FCg*Q&6!6HL892xnbdhmr;lx8va@Ru$oMKTmhNrEQT z7sBZoelI))k)-`35};JRJrAx*_pMHAc^+5h5_BF{#j^U$@#~TK(`PKHv|Q(iU>RC> z{$0%T@(@D`7P zAHdllvC&B+Z7Y$_5!OaXiGmNC23z($!ag!ZfwIEfi$Y*I)f>;(bvZonix(t9DblF6 zxu=wBFrA#lg?n0I4>=(eGDZ}XK`4?32avv!=V#%O7eheTgLBHyuIGPw04k167N=o> zjJa#a%yd9M&V|+XGc>(uU8MzGIRlX-%OMC#bJijvQn!e?CNj7^?N3n;V+_ zKZLzyRFqNM_CGX;(jn4~AfYIo3ZgVfcZcMV!qA8eT~d;Q2$Ir63?-cc(nCwb&w9zH4TAQAxVdO)&HqOrpVuXeVxy~$m)l81?vrsTCFExyK@d(i($QB7I6+4 z&X#zo_vDeF+s%QZwi!)jYT!9ydAAGed$7p!=l$h|wSTu{F@Aa3KEUdJ&Bq{ihXBuM zGWWCqPTM*7Vl<%}#kb0OtEqe9`hlZ}p5ixJl1O=&nX>6e;gsVx*R@Z9<6q9O0W^kK zbE8IMb(dx3BoRVQx!RDu!kX27I_&#*$rr&_ zrdN?%6@0s=tvl6{wFJ{ofes29gM1xsXoopEeIBm$CT#dvERj&S+eR4UTA~ZkpJ9&J zM4xaIsacSx`>i39ErJ6*gaDUk*1rjT4z9>#zL_9}?_pO8y)+lmudvFCMebM6-yv%yaPR zNnyOLB#~SB${o6rlkeaB+K@f_{mKJ#iyA^Np=6r23Q_&YO&)lNHv?d5RAwCyD82psgYjWB z$_=sGP5MSe2{2WjVKZ4{&rD1KAItHE5n}NBQaE;Om`U(Ktj|BNnQGOsMby1Wn|Z-7 zvBe;x_{MzR9(DvEu3evN8-{Q|wM7Bsvz@N^1hI$bdj9jS1e{BTrzBMaQ9YE_K(EF` zvUUxrx^`!>VEAWL4tMoUmE)ds>c|7*Sg0M1wvb6ol}cPbN7j6%!(A<}Q<7?~lsJ%A z*N~IFo%{ASg(L1!DtGWv@iP~XfFdPXX}J6Be@bsal7kYOz&~tEnaWakJs)@= zUxu4hIwdOj~3Tsr@tqG^S0k@@_O9Fi3Z;tK{M?BBk9BWzXuai7V6i~W=Sz{hbM zo6hWNk3P52R=UXJdoGU>+pZ z0Xa*rf!FFQ66^6#+`ZgVys|}qpobi1QoeH{HR^H_-&H`BUz?vL-dz53g6*rUeHvd? z`duhV2magE<^XJw6sQ<^L^@z6_yokZIWo*vxasSvCxMlyFWDhF+5E{{8;f>5fPZc! zO1TYGGj~?Q*N+bnQ#oC?9}4((tG~mQr^vKP?FC~_4>2Zf=laXfgg3I5Z|thT+{M4a z!Ptzb$(0p`9rr2^c0db1YiL|Ywx0ZsDD%MUpo|?v>{BYdbc=U1}8lAbt zj(od-mk#R%XZ~~ger*}47YV}^M(UEzx&ORMG*Qg~Jlue}i;-;-wIT+14dy@mOCTTb zu_Nb((0kYBSxA>WmBjd=UaEhaPuMAo0m%xYkDYgU23*BSAD_Pt`~Cp1XyF2eZf7XH z4}x1V`MdqndB8%6&4r>cb!R*@nH*!$DP z?l?dPE=BdfnrX>{%_wZxhubP3(2S^JAiG(EAC;x02#~KO1XNDt5iCp#_0O!>yCZ1< z7g=+5^ZhluDhHrV_#~6b9e?XNfv}T%XFkkS#h4okV1{L9_h8F&8^CXgaMBX$m=AwU zCLenYWf7~5K3y^6H+|SL(8mN(1+w7FsaI7Qf_Xt0xb`1TL=9LfAU~%zGaen5L{OfSpI_K=n}+yHp<5)tC9q{mu4ya1|SplxR|kCY}E&t_{OcIrCh^ zR)E-hvF8F{YH(gArdAFCFoh24>z=g_n)t5ACp9YB)B@;&@T*|L)4VArFe57Xgge;E zYNly1GWWW<)#uvja%L6)ERg9x;W39^0nQIuuC9HBpI*4!At8gj#6;QhneD*XSh{sz{CQ9@?Umk;+o{kKQ4e?|&#-0Lrl0}OBr2<9|#(jPOX3#|2Gm#Yhc$h+Gd07-<1v2TG= zQU?H?tZh<3v({N$1L9S3-zkV(nyx|QSVURstvGjuPJpasj?xtN`p}Na`P4u7JrH5e zcm67Mx1F#dh&d4)LCT(Hj`g=JomUQjtphr>^lyH4UcCaqRYp&0frK7~DBAd9Xyamt zi}PdH7uXcgi$^H}ZTSyC9Mi|b>`IC*gZ2>Sb^rp0w>bc#D8N-fB(3VfOzEfoD0H57 zSY`w--VET+2c}=p+8S;=19BrfctBnh8h0_sLxy@M-5vOuVvQS_US-7fVyb@gx&P=p z$zNAEOS!$wqj3&)O+YL)Oe>`3p72n8&0E3WqfbWV+Jb4rS@>UhC?HA?JrD^W6C?iR zn2Kira#3nmg&|iUhijI@beKv&IP0%1))#=Lt`kI2FRY7HPBk()_0L8$=q z1W^08WGnp$G|iu58K@sw(CHTam4nrv`4vyh&hq#!dZ%BJ-5_Oy8)J>6v}xFgO1hHF zCRNv%Y-$1M6F_b}h_$!N>Gu8%n4yZuI;mlb?z>Ju$U9AU0H}t!ilB`G85z|}oi(E@ z$}GS&nwwxd)~Dq>SjROj>9-|2;e`JRSN>^{)75)>WExkb9Yv6K&G2)vl1{TV@zNpxyTYMZqNig`t1XAQPpBllV`wp+N0&{Wd1 zZfVAOQkl#A)-CD7B*1omSUU1P3L=37h6(|*Dd71;@vkp3w#2Z5YFUwVspg3|YxsZD zoD}0#V7T%ku!3Z@J}^5J5|E@;u=o$>u(-J3N)d}T(oX=oj-qYzp@ID9zse`~B#`1~T6%u)q^aaj?Gw+n>eh_bCusSTSpHAzH4fT# zvhZ6)6@(Wh3fR%_Z@awPY|sO&>yOs(*`O4n;My`e8_#<-wSC~=?{K4B7r$w;twQO4 zE!My30Mb}W(^&FntCjdt`Y{`{nJNk`T?$~ckievyKBGjl#PYN`&f09PtGCcNk?a+6%ah>TJ_q_U`r)f7F0G}aMS>YjF6}(mMPNHlBlu!;7Il%GmO>x+2ee_SAF%&f^z5?P;!0 z*Sc-tP!@lt=T${p^P_CKQSsmU8BKqvj8Y_)AH`sKTow5 zz}52uc->T6J^-7TdD4^VZfw=u54dt-ew>;BW835zlo-byb&AfoL;!ISJg%Q;gxCN{0R#QOqwdmJ! zBOiwWWpVMm#=tS4&a3+-KOhI!%JTvhOoQMT1to#I?uz?F_>mSS`*nfk(_oF7E0YP0 ze`5rMdFRwbl(eo~%xXdz`)MP$*qxZS7Am<`9Wg@gi4X;>+yLQvB+Ze$iz2lxVHgFdmngg7r3@ zsy8&L{2q%V!yUvd#BUMl5>PZB3nSet%&6btuyfT`YtU)dvU5R`QUVjddC=}q3DEul zvZW$l-iZ)I(S-{yz%K+HE#964$YH%FPzK~aAG?YlX8^ni!HhdT`jO1M4-UAINP@U3 zOE!aF(xx&KHQt<>8>ddp%{|_u+K{y+Y$nt4PMa~cT#m?2uUljVO6Ys=?ym~b`7+{LrU|7V08$YgIg^{iHB=Ra;zrRoB zLtllr`>MnLwOIdmm2ZK2cRc>!WUd|k`8Ti<>AkLc3qh%PXcTsd5te!T(sAOV{0ju`$Nh|SVa97+$sa}>ph z=*nap61A>e`);|Qh9{@LY2$)CW(pvw-M6hCjqADfIL4GLuJ_1O5wCEM58Q%{4YWGW zXM%5MPA(eT8F3SmVe-s3;Ipn;>!8nl;5;>dtm-qM*{DqDq{vlh)?I&}C@qH2t!n)w zBl~qUXy60woO91>F9akC0*s-eibo57Aj#|T6bJIIh0^snKwPUz*j$;ANIzQ9KvEOI z?sIKaUYY;QniPsAXL%O*4`eSPAt6`J-|i!lr%3G4!> zFp~-!fO|p9KQ8!gp*z)5(H%jHo^*O1Mqw;@bcDJ3esi6^cb`c^`uy|HJ@kB0j$wE5 z&WrA6q*20PK7e&4jPZPm8CpEW@BmG-Qu0(;4Se}GRE?ke3RFW6)1Iar;lckUtARDRgXgKT6#}E9YIzGxA%OkEMIFFoAM{C^ zMn6LF=5*MSwLal#nKwlLenmQPi&}Pz&zhp2G?QK=6CGNK7aZh|md;ZFplds&KLEt6 zs`ey6z2KN_J@FfD+vb5=#IR-scEhK^ek4=m9M4~L$C1;szV_E@S^(^QJ0$6H?pFmT zJ7$YzKqDOBX-8qzU!Y0+kp-pA!i^&%3@?l!P*Nlw%ZYR-sX%&$T zg9~P-dUU!H5-BG+^O!kv!&Zn_BCSR`}fv(F@b_7gk9^YR)js{cN5 z?`g58PEEL%XQQN@XSTj$YyXlU3LPCFhE!R!sil@bnqOx_3fJR&P+LMTK{j@Z^F-V$ zjT_Z_0vx@A^Tr_x`ZjE|5lno%*DMPH|!7D7u; z+wjZgU=W9Q@&#jTt5m*czEr-oWsgO%j zhS@t3jEP@C)-Mc=#)tIIE=&3x2a-9hJnxWSQ>E-8o`K6EH+^YB&?z_B`y(8D)I@BB zpBeQ%rmLM^zItE5*;(;@Pr`IdTsOGg>8U3zr80>SQ2`}JK&y%Nv-abPeiQlXeb__< z8mW^kcC_C&_B#&l=3z9fD*H=9$m8#u>#ME2$8E;K*OwJ7qdyBpBb_8H{>yW;CE>o6 z;p;dsQSqltpOi)*=x$5oLe|K5ZDF=`av8g<+36L-O8h|Ywa8ROM5B&LbFH!u0YaZg zU=d4HGqYhM1Ekx8heUZGFU%THQyYrb=DJ!Yc4;=+%O!1LuobUtC4<}6o<81XM?=28 z?6+i(e#-MATNJ@&b)&p_o%rN$SMi#MjriJMu(%nCHmS*c>-v@AI&bn{crp3!C^qXk+hh~+we5CcAO9RmtXLsmBee87= zWn~9o+@6=v7TLK>6|EY~I_y2*E1Y}%p%UaAPXhHZ5s8d4c2eAzW8^aW5oJ8Vi>XJO zor2Z`;b#=4kb`=&1=8_TG@OxUM$FZrY|K7op<$yiC;0ZpYt&6vx%Bh(G0TU5rjPX4|JbGYg+G3jLSo zlG%5qhJN$+_X>iYlOe%k0GWN8fVv1?XzgySZV>Ff4PIaycjP4L5SZ6bMFmU$#H8;H z^v-A9>yduailrRMSJPN_4?&f>i;)wkNcN6~ob_u*wP3l5B+{>#C9@z(^cU?AI4}*R zUN}B%OmN7)>_&mqy|5V{!5fOwAK(p+=r<(j_{OV1)WJ&^ty0|TP;O4Wg5M#nTn5`R zW1@oZD^KB+#dzP`vQwd2N5p)@em_Re)KI|4I^x19yNLx`qU=2>&d9??uWB6I;z0+zGO#sv;adVk)sV5$97AdWyFA0{hl~+FV?&p*5IYCq zDa(5xl6ZEBnxt(`w?+sD@GHvIBaSgSc}rin4jUN6ek{zIx%^qHpSy#0|gO^39kQwO+6rq8)#Fda9NIec%rg0jqcsWp?0 zo!@HwMsRDgm{|5}gWCM~MR;uv|lR6n&S7=U1vR+k@ZkZktH#F$B^^iPqW%_C7a2y1yKs#Dxj+6I$ z)h<~?*?407tCG5JpWCiB>30!^x7j+b!^1Z!-YXT2O9hYLeb+QOf1`Y@tM&n{!Uz+8 zIEnRNp3uqPp;)wVIf6;yoi;SGFp@dw^11WzncGX?JgQ+_zY~T?xO8JVvOXA^SB#;# z&9uG_ZmV=Icjk3Id50Bt%pB$9xztjcCO@}{4LdgA6#h-#j}cF;Q}%g)fI%5&%{n>) zS6T6qbTBO*r01ZAf=^%$i(b_;ha%oay#bJGez~W0Drt|cf}8?pS-|qT!C2aXVUb}u z?C&hrD5p8mY)%PcyG$dHi)rzSDv-Y=j&*&-{zvNtG@^ zrqSouEgvwI{sSRw&tkmwuF?+9-Znsy8xJLqfZ5}akl*@CdGh)XVA|dpe;d+U#9vy< zDn0JtZPIhMZSiWN{Hw?vg4>D2lc4x$ykdqe>S{&sq$ltv4a?%NvST+%x9}X#OL-A& zcT&-mYk?3*BN7jUL`6L&^fpc zvivAT*ddI&`Qw!>({Z}(#dITd+swd+Y0ezh_7X=A4Y+5uO(YMP*D=Ycps#pif=5Q( zv_d%|zK=$NUEA#Gx^X(z5RKo~+KBo{p3iiXz~YpTd5}XDCsplt{#d+L%K%8Mu-clR z+R^MW4(dMD^u0>815+3+)#LuxC#@Gx!A=@qTaJSbx%xM)Wh@7k9XW( zPDWF79%gF<&ceCGI_R7$2QNucVI6XZvFNX_BJWL!__RS#u1|v@OFoY#f}k_U0e}`v z#l(cn{3Lo+e9u9Rru@X|^bmSsb{Zh7<9*|{7lykk!mG%OLUt#uZX<5h1E`pZhq-!DcDC}T$ zxjQ5)-Qk~Z>P=bitAC3_{&ly82_Seo)7zo$pyZ930Sw%Ak{BNi5Iq?>PVM2`<1fC6G~^ z6UTcVcRa7(25grW>Z~P-L)*SwAGLOVo0^GQ!g1s{jwQJL~7p8 zs_lp1iDXYS)9%|)ynK{G-mKuQ#Vf$A{^iXf{4`gr!L9vPFT_`!;GP@<$R@bKS3eb( zUT)igqO@%c*PW6|eUUvowQP3AZq!fK%tIA)!<(KMtF4FF5^_HjSmA&` zaoDuQ^fkWE9)uE94-5^uUf~uuZgriniz+B6keZZ=hHwQrgrXUkFm##+ebboI6Z_QQ z^C|twFE59rtLw)$g=K9_O%nG@7yF5+M3hSK~ay ztd34Vb=O$7O9mhF;*pc~t2gQ&5<@0i=T0Af_T{3d{@gP2=l*6a8cK38#RvC|W_&31 z;U~*|3%nbVj>~pwd-}NTr(I6;>1O_qN3zA=krDsBA8b*ZZ68d%9!zA9s8u$3aTjBD z;yE*NNxDXJ%73;li?ihzpq?XMvwqujNN&Zq&pAM0=x6x+Y~t(Z20pdn-WQ%Xmtb_k zn$l?q7LG$Om^Y#?VmwjrRf(VH7J3U5jsUe7N9Og~KW>@He&Sn&1Jm8?N?7JY;Lj{2Gn=slMy3%B5NI(wH0+I6cv3+ zUmJLgOL=s4|6pa3Ak0OgCX*@n0~_Of5Y#O5?%vgW?>y;H1thcxknAp2OJmXZR@%t< zknm1gbkOlq#X;Rql!nH>ZVYYM zT=--iHO#jEoIBBKEZX1sS0xUTzXvYkBgczesMT(+?1JaViur`(@LPpt;C^ML@8p$@$)S=UD5UW(2k&v*bR#qXDE9soC!A)i_}nxy=!^Mqm5N5-exPi`Ct=_r zq=xbsGz5veSQKY{x}WHJ85DlDHI!JKhXv6?rHc4YzClb&PJN}gMz=g!keXedOgEE31{+zJH%`#;aP`<7P(D_DV)Y z{^)5p))+rn5AHZ#fR~z(%l9d{*2RL#??zN>Ep0(M;I_ za(BVwno(}n3&ne455OT=kLg<+)|IfoO+F5l28QD#hF{7SW4oUo9|C<(Q!BjiTFAZ7L(n}Bez3%w7fO$RArawA z1xlCVTApDu*r<)Jl?a!VSXmkrd)*6{>Kdlwu9xC3!oaT$u!gcph9u0s8g&lfgZA28|s2Eb}}LdU=YVCLpLU6j8)9aO zwFs-;Xjm2T0j*t(NBxeV$;PIU+!)f zv80tWL34G{r4NM$OdR0}`T@g(zCmg4-;lBSd}_}VqaD*GYxGd@#03v|S_CaI{5>Q? zDGnX!-8|6^{{nTy`tcR=O%nCR-0U`D$$y<+7BZ!A0*|Di19a0^5?CZ;M8>D8^Rg;B z=ZEsg@C7qHmzynnM~j}?KV$@FGu@fZ3IPF9p<==pxxo_|Ed+T#iY?<8{+}<|zt=Bc znep-{1^0{Cfl73I^tZSy5q-x+Ogb?zXaxPdJ9Ki^Z_YqeEj3yNq^55}O_EuzpAxZY%7=+*d4zT1`~F*) zdS!ZQDq&=LS3LxV6$nG3Frsr_w@Ed#x`o-$SBjAzBVfSi{jlg-uXdfZiRPQGT*3!*h3IokUp4rsS>a3xDP0N{q<`yIn*d;$qE*E8{s#RX4tUE zc^zH7eUR~n6B`34rqy03t8JV(WY^qcI?UOYc(lXN^r9PUi%n^)A`@6b2M1P*(2CCef) zf~=9|5R>%!LF^IRhSIlTfLa`C;=wB;U!wp>UfUpJTuBYJYVEcT}zqGo5u zRrLM4wxJ>2hR8sXoL*7c8+QZO3v{jHfJVEfoe@ncRjMZ3{~R~Utv`{~>+Wic*jAF{ zVLYFERn$7TqL9m{gWvs-wexZL$;M%r9pjftzP#4q^>$J4P-7;GJO>x|@AyOC-yNR# z_k6$ZPVr%0_w$U9byxUzs)|g+8s7B+)7S&Xof2v1>z%m!?Q%;#zwt34D&2rM&xD4E4 z-bQi~S4REcL*C+hxJh``1f?Eq@|TtDQN|C5>Bq0&cdL>MhxStK$7m88`d}1UZqXd) z9D}4#$Kkcdnk5sa5nA#jK8Hw)vhnwj5ZBgrrHXu*bssgEMgbLY&c78%P^19^CwoX8PCmZGQDy_z}x-h&k<)_7M6 z0_%rU6>Ah}7oZA+!A46Qqg<~v=&P%9D)X;spW;CQ93r+kU7cuZNzlsc9I)V9GKmF8 z2Fe0$L=!D6hB2`t&>A0U{FDA;se(>4F~!G7SdZS-6b_u|$;H9YES8u^9Uz2g>gaXP z*vg{0c;QOrg#GF+T?;~*j!mwum^<`y%dU^gLsluHGf@n zSv%M_`@*{SzEPb+l6{sXxuw_L%Bkbu{v_O@QnwrlZ=g21AqzYi?Q5%*%KJJUv{RH` z0<;O%m`DNHAs)B!htzr-tt{$gS%fwiTQdSG!pe;IKcxPzAcF6bj2hMHuBF;Qe^7BW33X}ag|1>M|NmQL%bbZ*= zc}RRRNG55Gh4SKOjOJ)Z~ERe3hv>GgqL5;+2{Gua)bH5F0SdWiB2(TnmBI+s~l6^8%8W0#4Q58?@u|-@4w+ zd3$U+s}lUThe5)9^+nZ0ys6T5G@pRA<%)A;(H$l)w!P?;&8K#G6^6qxrc_oEcY;MS zg3{!nJ*QN`cCi3M9}n#EWXoD&khs9_IiGoL1h@8sl5X)|rn)b|V98{?QLp8V<)r=3E?z_{bS; zyqYQ>g3;3Ti9pQv>ED}5$^q(t=ab|-*Cg~XY9vgSIn!-FGxO`_N`=kn` z$*u7L{q%je$lCQR7N20VQr#oifNj~$w|x$B(tNxuL$IMvsXpRRa9*S36s0K%Ts~<{ zX7(8NnJp~g&mh!2xQfC=FV#9`6B|RHsXU8N3kIP-icvck16SCtQ9Pp_Y0B_0Z0ve< zvIVl#E^9qsy{ghTWYl!1>k_I&7P8a=9HcZLX?}EA;pSBgqi25|o6kym|7!_LygO90 zkHp#HnV#L-PyL#al%L)W9JhlFtlyK*ch$}ULbY-`#te%JXM3r}Hpaf~(0=TG(CIvP zm6i^@OfQd56;XM)YF`b@ZmWb7{_WZ1m`$j}NL+64 zonLU-hwq^vJhsp9T*!v_7fI{Nz~psw5~b3_Z+b!PW?BndB7~?_>ZLugY$2)ajb``# z{s-3OSW%X{wHo7<{?M{_BcG?p={NiPj zjJLb@viE$(Tw}(>GY%Gngrm&BaEryGes_PD9j8B>--)O0T^yQ)^mFbMfDivdc9N3P z%97gyt>>fJ`5brQ8rpBPpe*xe2TtU&2e zZkd6|#>?(7H~ve1EVWyd&eN+fVu5_j&6aFi40MdewPM6fMGJ5IGn`W;@ha%<%Dx@- zdzQOzeC9BKbGqfAVS!MK^zPo%$554VER=3kPLjG@vgVpwdxlTOz&#d_$kuG;_b;<_ zF|eNXU>l9%k*FPB6!t3y%?4y_?&HJB#6*_)=Ia6BYm?#5ERu}>^1u${(y`t>=5Exy zc@S`UwCbvl4sK&mP;lkB$z@HDu(== zh!&H_$aG*`)=$gVbX-Z)SuabYq2xUCj4JtMDLOgxxW_!os-j z;V!^6Iof5=wsx?FXVm~3F5)V{$#2ZgE!jrWmOKXOU5Gz@YAuK(3NZzV8tCx={g=a1 zEaXh>8U;DgmXKJ?6YpXPZQPY0;ul2LWjDV!)nlS|_-ge4m%v@KLYl8cpR-pSQn?xn z=%X6P{_<%SDN|i&(n8dhL6I^l{K~dKPg{?svFN%yS~u0pWW4F)8w%WuxDrsY2W2W- z-*Nvop%HA?6jhCLZ-uFky-<7YgGp_IA@KF5sdnR0*apbRO#0%H|BVP{g)6OkX4Y_h zd!D$;mNxU##Xv4f(-P;t9(6y3=yE{q&vO(HvJmL8k?q3fTN^jWYv1KkpL)M{4A#3c zAQ!;lO|3_W6O!=hSDx|QO(rlF$Q;u+JV}>d{@u5-Vqw(i%2;8z9(cH(DeiY-Bi}El zsvM60;BV>Li@2yK&_tdEQ)Nus&hqE1ti2b@cb30@%6n~D`;IrA6U=k#`?q@?YB&4+ z9fHf_+zI_c)((#G`0@tN4KWQQC&0*68>?dD^^9L^HQByDsffb}N+QNW0WzsI}| z69yg%d0KnJY?2O>(K>8H3*-Q%>``(-NoUuIhmxK!zpK~Md)?h z-ek>knqL90(7}3wEKOfDsx*uqi||>ETo`&LuF%1FLPR|30=c-?#I&Ih1FKIS(0qC} z>zzPUkWFccyN=;L417tPXUL=H+Z-v?0Y5P0MQx#tI9GCsu#OpiPz(sDs_Smd(uq|% z1lg~OM{oUj_c4JXc-3Te%&~jNVB}1=5a$G~c4bY`qY#-xN8l1a@n7%ID;i$;wx}ou zJvD6mGOb^8)hLbqHNlorR+1zA$#O&Cu)t?`i7xtXgKi@(0)o%tHJhcVEr-b18U%5v4%t0xOEaA_kDSlQ8Cwj zwso=Cw`9_6e0(?u3yA)?zua@V^gyFdU6Q)|WM-q%`bDFR;Czj*;Q8Rjd86qM*kOm? ze0?~)BKcq_J%^V)QNk*G(QxAxSe9ivxc6y4==S>jG{np;xIPizS9+{XatvbL82a%e zZmDr&Wkz0Zu0V(aPwK%`iJJBMnR5OYuCoTSrbB+q!5C@p?ceeN!*)U@1{=aW_qp#G zO8cLY?JffS<=+!UDm^yiD)EdT(5sC^3yF9g_faWNO8CDvI|G{2m#O#KfSXTR*+Oy! zcg4=H*W-)04-My?ouZ2cz5CM-*E_|EZVfIATp$H95=|E7a`W~jVhal4LCg>3{f^y} zSaj}}SO`##4_D&^+?GPiv4Bp_AXl3A|Uij!CTWdX$p&3MSCcDC+8 zf#wZ6q-%g^{7@;xdCiu-g@wM86LgTEZ96+s zB-Oh9dZwK9>Oy>fa|7;&`HS8OO<^fIa1?13 zh+a+r#!+_r$+e!Ch%QD-u3@0()qa+1&%~zGn}BL)lkwUnQ}ef^xILTGuo5@8UG|z( zJNxqvpjHt+w0`-j!ai&mCfjiLQ0)U06zlKg~DbD=oQ<0k1!Hif?7F`?=1h_eihqb+P2}Mb5(Q==(?z z(?tc$20_5!hzm6+YJ#YwH|dd?4L;e%Z+1hE8;#qisa&y*Y|1x_uRf$zju@%{ee>zR z9M$X5GQ5g60+uIw=$}3etNHPa25I%N1$ac2Wl~=4RnT?`!vO7W0=&lz`YUF&nWjQt zo@V}}qh!6I&PBtJ_5r87w<(JW1=Q72chjPick_%LhYE=`T(Vw|7lR7t$M>rOxi+`| z+igOX;EeI|qA%|GbrPv~gJcHgQ`5WlF2jb8z+OXp&%FjHF-sWB^id75KH4VNW&|z7 zzVF^+>4e=!xPJ?IhXpen*|jo1FKyodZrDF;B@f)%vh}D}g8Vi;qVGqObrhQ{FPHVh z1jPpj8$Lv9@+>?9A-2B##BK@|S$_!cW==y3(aGsfz(HErrTZ=h8?SztN|#8W7Hd*{ z<^{@6YOxUMTZ)qW`l+*6Ge|sV+JMiPqsrRB;G^o}vtxoh=&Nv?P+2sk0)9t<$utnt z{y6nQhFRR>qd?34Gi*P(=~?bx?Y{IZ1 zNP+|ttdWSOGlVB~2mj}7C=J#MFEKeTbKqIh&6{^J*)L!Do5{t!ylJ8B$J4i~`u2@E zWxI#Kj4EY?1RMLKn~U8VO=Z*4I6MPa9~^vD+P(gMI$vq&Lx#W4wqnIEqo|Q`qxxuz zaKSymgj|e;gHPsvwW7Q3xZDtEMe_#Kes}Fugx2RpTF1YhOv+Kh?I3kGG8zuP+DK=! zE4tKq8hTV4?7}PUvry>a%f^eRcRg#bshOB@HyA1G&|6sbAuY5*y2E!lc|CY1_t~ST z`#w+Uhb;XdtB_M^CUgKALWXr=k1RGfN2e~CU$zCNeMmT@l^!JSFZ{f3gP%n&1TK3) zO-maCC3;KN+2vl;>7A~_lsZw99|3AE8tS@53DXrxU4Z=bL=o=cGj~Fy%i_IiWYwB1 zb*dQ_p(~g7<(U6IFIHYX#$4yl_!S8I9L#U*)CMw+tKb8)h~#$B1TxAHY%5+{Qi<3&kgvk16vn+w=2K`_zftp2Zpo@nCq>o$Nao;Y?=KYx>bDA zr$9bV#{Y6PQabOr_Yp<~_d~oBu?>wOpr(*nlgJ#}{|?e7igacDd8k^2v^3q$6M7=*quBrSvF^C!?{5y^{#_W$hK)Bc&I2 zTQv|hiWK_YkAJAgWQqfamd8(n@-%2)l(`4G8!f5I)=Gpx?z;*}x4bAySZYQOOMLLG z`~T})cjQ+LGYNLHB9F9rbb-)M0!C=6|dq@`DJ29dC^?NUKovmRsTFUaR0MIy- z?vI9)*c(SQPGE~(hS$*d+;ZW&s{dRwgVFL(q5eO{-ZCo6H*EVII;A^DN*V#_MjAyr zq+2D24yhqT8YDy-Bm|^|AqEDd1O+Lnp+#yKQikTeefEC8?dQMt;sa~(fwfr7eck7E zoWJ8Z3?cK8`*qyj8wU)cKMb*{1)^pUaARbk(|`g(wUS@aq;JuW1Svje_1(@FC8L)| zJ;hDb_f%vF8^e*4vmX|!c2-=4;2pY2oWsKR+->~Od2g$#W(3>wl)o1qn{A?KyyZVH zx%_FQD$^xQCeen?G0O55)riGq9kNr7ecBrU49r5)w+-hS_XVyhiv+WT700ja*kS_{ z4o9SKSEI?dfhg^-?$5%G2I!6Fs(*_1sZ?u+{@qodS}x#edKdbzgo_`X@sSxz50#oXSw=dZ`8+~c53IL_tlI<~des%7N65v<;CBU!2Wm$_H5SecL?hIewp&nW@&mn=|9AO#h*IRiayw ziNu6wSSEn`QQaDF^Ur20IN}XEA9>>-VT+)owx9WT4vhYoTsH(k6(n!4KGXGFz5V34lZf~03s0)%SaT0r2&t)5dmcrn7m_z=IE8BN7q$By zQ>LO7MEGviGyzg9{*K&$gkG42OyDeQRqAxNgViI+Tq2C$nw&Cxb zMe)xG<;U*6nQj}eK3z^Yw5P&Vsx;<~c;LR!{kBxKAj0)itNouFf4Rdyd0ted5RRbQ z1=#xA9(2?PPVxU>q#v?M-(!tumrvGn`90Bda7C|h+8|XJz$AH>v+{4&F`I63)>Wl@ zm|4pz7eHgnRj57Q!=LrH`elgp9fatU z3LFZq4!&F?ZczD5Ri9v@7-Erc`K0X>KG1B*%eS$mLRbGtHx5rR=ywVkD5v6>5<6gWZ}m&RaI$$PR$j&Bjl5F1C^+; z783RyKKOAWLOY+fip+uhcYc;UZhX(X^l`P_w;W6&K9gO(h zw}>@WXwTI$R^2rjd)ypK3-IMR_Ki>B z)ZTD9=gz{LIsjLD6G=w1@(KG%>=e-;mW72gOI8LvdsjStAP0uW5uZd7>CiR49~N{i zQm(Lqs?cN)Gxk(X6X%*Bq*j40dfVY7%S5!eGF}Yo5vHXG48snn=3`OHQke&a)+zq8 zo1e?>x{qux?Jfx%jHKCk=fm8d4YGxt4Y=;S!Gs?ca2xM?i$zA`6OSLIA?z~7Cs8Q( z-Kok{vs2abQcExCmW`tb>eRdB+~3n2EwMdMgDcv(d+o24dX_F6;9*ySnv~-j-}9O)aKuv9 zJ5vYsJAaM!RE~e3mVDswAB)t(Hk7id9+p4zd8TxrZ4&xmk^-1myqIK$V`F34xNBN; z;yAw0;aKUJ^qF4NFdTkn`$&LZ>Y&$Jtxz{7Y2~ihx4M+WZ$rS#aa|U7=DKNg#&lL| zCk3${szCU9T_Uaq9wp)$3^w9~<M+M63 zyxN`GkR3PhXrylSjgSoLn{zKev*}h-8lwGU{WxKGAZP7U6 zBE2c-%7+b2f@H6%*w34gvlaZEzX4eKZe;*7DdE{BFGwChj<7GVF?0BtO#yv<>h#uX znnG+Qp+J(mJbN~6HpFNy*%%*!5bfZ zG^b2mnWd9deb_~K1Cm6KSTmIZzp`ML!8XU62C`uTb|Ap6HmKM7^6z*hU{og9W*pi` zH=-@%{}@v{Q&(!3HgPXDJfeHtI2MMJ_7Dk+f%%I>>Rd}>K0O%v@Ud3!tUmxz$G%!r zOU?aY4sI7Mu%5kGMYDA7MUVwy`La7xIx}h(+Kz1}8?OLzFR}6pSMh~z22Ryh*i?J* zp6&}v?aQ@Pb|K|!h25{$;vDPq^I7`fj&1chClkytCR}{{5$$zax~+Jfx2d~)4z4J$ z@L95>vI;j@&Gl*$1yeZMTlAjJ!TrZ^@fgssik|X}hgHsp)|aY8JzNRnx3}$weIz?k zTSF6Tk(>_KA_@k7@H9pRLuxHUsxy^$BTwRR^Y#~81V%#1cC|NLl)$BiD7&LXk#4Pq z`W*i>adeyS6GYz--DJi=_V72G`g7qmLlEXe@hjRhG!YR|UlThC&SAE&kZH^~rOwp_ zSV0U;*(jt9ZSo2_${27cpx$|adPk&y1@^cb6Lq1D#&r$NUKpomh5_q}7@VltL3eiV z0`MhXwO1OL?zS71Ims#uN3aP94iH2N9N;W z7+IurQm6r{s_gunxM?JbQG9zupq{2jTbVz-Q9NjQwKF}Pd@hcw?}E7*5Kus9@dWYU zE|l(2O2dhc%u!!BxqYWtK*c8#43n)Y(Jzb{!<`B=Pt*sWo7+5{{5Bq^4-%{GE%Ept z>)G2LmR)fa-sbF8PN|>~e?a2IYvk%mu!;=`kM}!vJ}jS3_uIQ}X@6)I`;Tejd@*o! zb2hwU{_Y~H+AP)OjooO$>A^3Wf5!t$gwxpQD0Kj=0N>wbU{f-kv(5;asFY^rm<3*J ztsy?^)=Xr8P4J@eyx+Z3{R^GvUrAms?4^^NTRmU^L3^`Xjnp2PUi><3i+^ug0!wl} zH24h5K>+_k6Fdy`GhqDl(0KawLp2?3u>o)N=Xh#;+U~rsEOL>BnoTC%Mw?li6CY3# z`J$?>COh(!F4yT;DbjL-cj&V!GtvF#aYbr1+%Ak6sL8I!s<{Fq zkx^81VDQAWo>F%&1HMH3`t#_6v9Nqu^6lRSNSy80hrI;qAE9AA1z1e*n+A}lkC89j zxCFHC&EgHhy6@mYj58OEhwc8Ay)Jz*IUaWf+dHD^oD0*$u8AkTrEbQ(v7CSbte6De z;~<>}0X0=6Aar>ME#R>4>&3|b27q?+Sl|Gnn5ar#3Q^`~2MU51-V) zR)!l%A0$2lu%~LatkQ#W5$_(KO&xLw$bbPnU7VmtD>_feMHtwK6I>qre2A7Dl9&ZJ zat=SYv^?rSHH~(m)*k5PYr)9#iO~{6BTH_;Mju}kpX&V0x}_|2By6T#ePAx}s@QXh z*u+&s1Ai&}YCkhSAJro8;nB;Cc)@fM6dx``AZy{x7i!1*6^>yqzmi7=0NX24WX>T& zS3=a)Zp+CV*1};*6uI65-cZ4B?{_d$IA*DwYLdz~$p?fJL@+NvuoTWG>bNaq-seTG zG2UVEE_iJ0L#S>ME{#*i5x3_XmB^ced zn|F$)%nmHb$TstiL$J){j|bBI_*XM$&%FYZl(_~3>bp2tRL?|-$m%#3O+Eg~vhgmK zoNAJTq;byM=))cVbx%*%7qwBp^|LzS?p>!3U*kG3l5Frl|YS_~+Ew)8-$lzJxB_kVbtNzj<(@;jeA2Di(VU^2Wz-(OMz zgBZ3;jvZZ{t)m{xN7TeP4*eJr$50f8BJv71!0OWgTB4E!_H5HU-I_%NyUBIs7j2!& zZRXvtbM~=~h``_v;kM2(lq9`CF}jSF?xINcU(OQD8{2!-aciX zEw$}wD*Wn-o`26N^&h9mL)OEUDBZfEro#63_x)u&(2W>2M-`a)o1`DMG{Z}ly2n@2b( z&k&2g_N4}Mvl+!!on<3LBhC}t$P8PPhLKd_B!?`RYz{qt?{}&yTr4DkwCC}Q_D~ha zfT&0SZF)_M(HrgKmnrz%yP+;3^s^EMGmqTv2FD>a0JcwUFzD~@%qG_nvGYj`5w-kO zac#^bl^v@VK4{qq3z>I>CcQ-MjmzgAz9(qFD^6prQYq}P{@dldrAJIgmYqk{eiD%7 zWv<4M%&t`Uyv3#%HU^RwkH}F^_jN(?W_3=k|pr7*#Zp=l!@M;(j+;~ZfJg{5srp3V|KSIEj z{#GBH9a?n6oI22o)&2#rHvzQNAI~aP6o^O@w{qWJR=Em?R3i%4;GIfvCBI8fnu)Nr zs+uwJ5ZN}8Bn{$4+1ABs7Jj})AnUyvsV_S;_~4n#mL*mqhe6qDs=)+5KK zBxUPPasvA4@Z=oOQ2!B{e?i6OAjUss8p)-)!tMQgY&bBSRYu&J&re!jjgj5=woZ># zTxdpPaU2lNtMp>sXZb-PqD8SSHR3qy$w^^+auLi|0fRDPHs^ z;zBRZJyQ<R9vt$_#J(g6BAtiL~<_=}-QZP>=ddNP@ z>mc#`%PVen&TFmGQA3h@h+#^ldj=Y7FfXMIcK$cWTaekJN35TKlKj*X4|!gbQ<|Iaj2 z>{0*C*ve3+*>#+bqp!#DF)1Xn$Dfz$h~OxXy*oE<1%l&GW`={&M{W_$sw>(3|`!D;LM&KJ&A`xf14xtq0)^m`$asJ%kG_(RyW3__5)=(GA4jtZl?o z>khxhXH*9zet#?bFxC{{U$wE#d}S{9w(pW({Bmqo7%>$jp}=2V{I}1kh6#J^h2`mA zJ7Oz$j1|CpjDAOa<*+sz*1Q88u?mSQFYgU}DUgoLHOB|DM_xe<;U{mf3pBE>LB!t z@O`iZh_t#F@pLFYrP|sTS6eLA!dqm!-ioYsN)8xi5W$bx@n96C3zHaKkEMJRBaLANU(J>qf*>pDx(Npba|)o!suV& z_G^cZ(~{9D=Y7+xcuPDyyy;dC%XZ$SA84*E+q-AGRC5-fJ&?Qe+Oj(mCVEkaV@4Bs zcB|7dtLB@TsK7r!?@mCp(eZZHNb()&YIR9M!Mu+*$-R9u%JL@2hK9c~|Ner>HCZ|?O;VQIuUg8yU1X2A!p-0#AtKMP<&Y7yZy3cYBU6LyskOEYo=gU*=aXLQU?5*dqk#2);Q3 z@hIzI)cEMFaLH6zbrZJcVi)`)Jc+~gD8K!i*~#}LL-FJ11n?7ua2UN-Q?eCB40CX5 z6ivd_*SARzW_tzdw6(1eTV^iZs%G~(iM`+>m~aLuXTfAvQ9Wt1sQmQ6MIiKO!hthPTHk;v*@SO z@3#ojBh3kPJy>gpjfAQ>UDt?yUh z6+oh1Y1p(HX;Y~g@w|&PUav0}ZjB}fJpsLtPRCJ(njc;=w`|i2L?c%u*J?#EKiXSF z+(j(8Eq6_-a{oDBu`POS^u@^G5xl^f;}NENRz3yqy=R~#LiWlBGjTFyTkJ(|K4NN2UGTV!Yo?+Y4a z+Vi-Ys8E58?k9Ux*qF%21M#feT*|@0R^9d4Cb#^pBLv8cJaY0?8c9O&l1x5eRT=HxE(YqFVn2SI&Wm zU=q#RoH&V)ZG>Jb$&dMm$mZIJph*fF%ZlT-;e996+c&d&&KB~w zm51lH^ANlvR#rJvE0GNwCe1Kml)zv_KP9Lik8=1ETj-_e<*d-%@du$OxhpkT#P#K9 z{dBPk&Xv>AGMii^k^T$L&IKEgLFLmt9-jLfx0Ag&lj%5`{jT|PrD}jX*u_@8E0F1~ z&0|r6-pV`5^`iT3TNCpGmeZk$Dc+9c9rs(2k|}SOud6RHxX}m(IY@s3S8viZYbvuu zaz-2V;;puC6;c~u`{aK4@e{AR| z(*4op!bD5x#+^0fX3jB{h@PlwdLPs(LLr!H*6nbu^nF5kX%4E$^YHnTIOSXPy8n4@huP@r7!Y5Kb^8%S5Y z5yE$SWX;U*xY|jSof2ORd)N){_9vWxT5sN7C3~06XAhP!5+h16L_t)N_}J$ubYqT@ z0vRIyO-CI`g~kiY?g004i`kcC5|xValg#oXIaXu1%6Ob?zwgb@V8LD^RLWhFU+7$K zPBicR{yzU-&-nGe+{43ldcJ*qeCxyUlWipux?7Jc$U;0$;Bs&!h#goOYYCXvN|!P5 z@5f@zDTi$t14bk<@1AE)EVIt5GoKEJFtjoP77+F_6J$LgT+rm;IN(cr=skI}f12I( zxwh36U>=|Sv)h6!{ppr(5%_=x=@J<0WEwG=ki!}pR2(|S5iUE`< z0~PlymWi3b?AfhkQlyjjU>G}oo_*)Ov8IgIYu{U!R*XNr9L1_CcXxpa^o{xFdvdP* zPo-(KDBOUVmbM=erY2W8fD4ukUx>~;48@kHn~4W?eWhEl-FFl?h37jjOO^_83+vIe zPX*ZFffS4j{vd}&{jSgq4MC_V9H;=CF` zFZbhvXdcs$jHdhVM(FKcvhCETBqN%Y;x~>F4GL{`CsBQm7AzXxA^C*uv)IQlxxyj~ zJ|f%Hi#e*i3~x5ts@JI;(!qM|W=L_OQaWmu{OWi|0@K8~D5mFB{T=ctI(*+K`60ly z<5Go3#obkzdV=lgxs6elYK@pKqYi}THHG8 zUI~g0^AEoOv(L40Zfykd{3v=t5R9twZ=zGUnVX8(m7++&T3JFwEz@26Uo70F2RCpWK~)vlwJ?M8>rTkB z<)f!ZC-H+sGFI7h?#yecwVup(xw?U5jC`&8r{mYp&KFpelH*py@^2M(S033#{#_)j zqe!F|shFNNru4nP@#uHJp#o<&!ZuYIIU@AEENo?;Hioq77-DNz03?Pa+?8ZM+koZW z8~Qcz>?FLzw#QmEp0@q^cyv>^%Z5EqCQ9-BZTIY>D}rG1eG=re2MMW(@eI-}(zKDG zY2>1&yX$EWZ_E795}h&uJXt_G%B#VM)X;lMoU(C&-m+?R;WIHm#E-dtsQedFek~-|;E18@INUWGX#c8U(lK?*n)mMz z(f7L_$Pl3q{c@CREEYjetXjo;Lrar$_WwNjvJ|9e@ZnmTc0ynAZd<}Bc!e}3d`R@>BLySp@L)xTe^g^{0paJ};`{mtcJM?fZZ#HMeReZ)Qn z?+_W+;UKoDSXXaHr=M={h(hGJt)r$AiasnTH{H9RFhoT z40l(ee%gkNmq_FZ$o7PpM&IU()hjb(N&tF}4{ZQ~pM5_8JSAz%w}&3G7C{+>AxAG? z(bt3*ejedxznjm(j}1qRtzf#8p}Id7o;7OLBoeYq-v?(S+K!)X8abG9%m@cs^_mm) z;SJwoSZpY)E#J}INo?yWhGj^19FGXVtvG&*Nf*|pxyv#IOuJo=kB?UZ>>kyiK%iV$dNtRwK@EFL6=7P11UvXgC-5&3<&y`+A1EO;Oa=6*DH7 z`nF!(-S;R=cOA6O(tWur(9V0~v+X*fH(sLn_fj7qP~j~2@+3#hkw*-vA)2BP0Ypc| z{>f&j%q?fB`W6gYmD$&&8nS)-hF`{i?w};B{OWyD8V?d6h2t$b0>F8Wu28q5^9p>l z(Xka(AtULL6&>@ZR>gxqSnCuWNL$>+TpfsKqNxpOo&3vev5M!%Bz>fw7zb0E~=we zY$hF?!Qjs8Q1jXvfT;XIFD&1Damo8T(uzFWhzwP60t=w6tNTA&1-N3Mj-*I0ht=g| zxJU@2xlr0ZMuLmbwCgc^fxDVy08cjh$HfyU;;-pD@f5iih`8TfP9q}UW}){8=J1s&QJTsd>ZaJ(iD(WGBwH z7}^=7>YWW@9RJ{H1peNn>XP zUCs-OYV@W?xMCp?$Y=G~9&4;$C?X-0LYxbNAk(qrg1ve^00-01v9AwTjOnb?v6xZ4 zhdC$T6&P0vg;?mnq9i6~R}@4F1G`GF7&K-=td^VlieFA{F3`a`XKi7HJMX+(!NA9A zQFq5?!{wTQ2rA?KoGKqkPof&=k=Q*?x-+wa;Bh~AF!gV}4s45>i#`NL&;@|>#N%s; ziAjt1-3nz%#((B#>_q;rUx<}P9M(%k6}(vP)7A6FM$-O zYj*Hd#nZw+!&0h{Zz-UL60?~Q{9e2C&qhew;CWzsakb>$nkL7}oi~FJ%zm5N>&j>FQu{*=mfSCbdv}Lr7V{F8#DywnS-Lu9K>26eN z6gX0?b663J_}=fLk{rii@a(&;DwaKp{p zze@%oYpMZPFD=IY)qp~&q3U^IEWZg5x3EiRl0J&|3Qqb2cAItvHw$%!RE>n85Q-_7 zIe~akwJ6Dz4|7&ip`)u5K|U)$r)mtKL^sXu|4Y1kYq_K@;>cp@ zlzjL0N$FY@O|lM>L&X@7p(Zuaa<$)<=#E$HUD+lcsPblgvGY}|=}#0_`~4Jb;xu!G zK45qU231Is&AeL$NfJEdQ_TRsZsS%Oony2X5z2y%9sxAEdU;kTABNAiL~mt{It%+v z>4UX`uD5)# zqpI-brld_2<*vAnF-B9Gkb_j|RY0zwYFiDOj4XI&{3WAIT@~l?mlOdgZqyuBcy^qr zZwhPLMi284EeKP!%+mOge+|~xuSl_|Oy^gfVZutq6j1q!mHtIn^#VmV7B4DLVG zrCIDA#20u&?N%#F$Y9@GR^v{J%1v&8@C&?$M7kIx(kfVX{fo_1$fF=c@S_*2!P9MC zwh4(fsF5^_58n`B#52C1VYBbRX`J0fC;e+^!0Aoj7^;HZscY<7{A9}pd{R20n`NS? zMX?CvozWgNCP3#iEPLqo)RZ`pCz<3x#Ed zyZohwBUMStyYcrNFxcI3CU~ogH6NQov4-v9OVYV%F;8WDK-ovt*xu~@KtM{6{)}Xb zD5zak3gFQmSP$K<^K& zT>-k*^VqhJ8ux)QT@LHwe((Kmtp0~IbtS<{W7L$2VGc7~u*aN|uN zwEH_65B95uz-*SPEyG$-m;CsEDlPiCj|j6{2+^dnmR)?nHAd}R%{L~skBFn@QJ$es z#;A4aHFaBu-;Qd$08Z3F9J8nXmx=Fky0Q;F79If+2#RwHv0%Tids;T_?~3q`b4WXct~UO%~`+_ptozX;+WQm<6&-rRK|g} z|JeLu46TK2DW&1$2UI7C;pa6QSz%#3_rtio%Q#g)?lwPC2qC2CJ;a}CAhWbK!_3Kd zP(ffj^o15h-bw3Ei0Q>y($3Sh{F!~@2Xkv6J&MZ55OU@9xkg97_miK%b@3f)fu@6G znFf49<5ql#Kf`t8bi259}*oRFom~tpE27H=x{1J)zb&`&F zHPs*};Qk>%=fX@f^_I#yjr<~Mv!Ht}fUg_MAPHbsOfo)PHEt&iFk8T8f??% z(1pW0H(`@{kD-X*=YZhmu3ZPjJHc^|b#2Wz&3xJJN&q4z;fkjud7k@YY;_igBFAv7 z4vQn5m1@V)xvljYl>_^oCGMlj(^J33J0L^Y!2CUCF4BgssfN&j#u#rkTYN~N?)e|!yBYWaG7y-l`zI@5| z<%I#nx~KV;HC(Y;+dJM#%MODbm~chIM*Z-%hSiKLo4O5|2FCRsPnQP*O_6O6<~K`n z@L}{^(gAUw=&g!Y3yb;N&=TO@eMc%57NS>;ju}P;>x`q#*%rbE300ZEXP7Eq%jU)# z-+JOsr}9Vcj;jTMY#{mxV8ixXvr91vzNlzMp}z0@r`&2SdpJy8G%;k$Z4hM6>#@{& zlO*R6zkWZ3UCc=<%L;1p;8i@%=F-dAI@Zmtt*Lw&-|b0V5pKXpOFv~sgfvr!c<~8; zP6};ME_{8g-Ep?ywgX(YPcKvW5pkZRNNI7krg;aJR=7TjgGbrK&DR|AEdU{#74*u0 ze}ANnCgP){$$`bEX&Qd zUux1In97cF!@vFSoIVzE+R3godPCF!UvT?m4?y#UxQyUC-C2IfiU_y>E1(}& zdR}iaCvcC7!;!#`E1ty^S?`R2E;Nj|7F(Ot(PHu;BEZ|^jq&fMXlW3)An};+Xtq$< zy%|YEnxU~5QK4+jVvN@5dVtNs;FFo9dBa4AL@GJrp07)d!J+SgrWUay*$SZ{qEAiZ@ zdAMDe!AE`l7Io8d9^UQ^3SJ2MKZa5u;3Pv?OQ=`rBYO8cR=ByH2U-$1Un5iQWVq*_ zQ^kOLY0W7W!7QIfk~egoKmXv}fe?4y5O#ZQ;xYAIE>!j*P;~e~F-*4e3*OHP+GqM& z@#5e1Q8I46JpXlTceNx~I4gWIFi`%;r~k*J;E@(_^yx=lHiB-imF_8@>o{rsWsSLO z^ipSzXV;NN$Hv=qzj)dl*_z+4@#SY5GMY^Fg>qV_E|WHHVn5ZwoK}}(*TWeFdP`%e z%qw9rCQUXF+pHwVES;a>&V4pg1s(yNVz3yca5>t@JPA@eW7irdp}G7a{@x7v&pK8v zENu>}xS)h;!3HeTi(RXL-!B2$e=O4ES;^191(%t~JY3WI-KeVqQq^)%hM8QT-y52* z(%_H=mX>M{p|Iq36HCED{%$0;tb}DIT)B{LOWZSxzULXmEbYt^yKeI@Z(Bj|W{Fog z9de)t@fslWOG5FYZ7UB90FR<{_{YRSe%nE^?GQd|)enujTH?vN!`1FUZgo?#UaWi6 zk2h7J#`38|26~l`O_Eu8T~@1$G`N8ik|H#ha6&QPkwA4<7R@XUbUJJdrB(tkObj~Y zR9N<)qE~gFefaE5&jbtUi;85&Kmkkh*apf6y}32{-gW3jugebAq)@_;ARdGyhMsrF z4G3FPp_a32kD+-*i^+g(nnG*=7gDmxEhJpS{re(#P=|F66>v>3m*mO^nuIBDiRywd z@dXh3tfGD{SgBF$^9O{Ca#IWUh@fvR1Px)&dlLntQp95i}M2E=4q@pB@ zn+7dB_xRHj=}DQoOno9b|JvWKFZ0hVUIC9Jh6zv{2Z@F~P>5{#@N7C?6lv$I8XwZF zNv9c?s7_InU$r#g9)(A*&JRUzYr5txth$2;$h? za}1kbIh9`x?2U<-v0E4;jHvogtxZOzI9)moQ57Bahlg)XXLj&C zbr@m2U(gy5J~6ZfBtlP60DYlUSq)ZsPyp2)s@uFl|1byb%h8y)QFlXz_Ur z1xgy|1D)nePKg|@=-E^R&66_r^jlLyaE#yO1ma$;A?LOu40%?msMbTW!yBgee zoGQa%fNioHWm7f+8-f!C(n&Sq#OvJe9`tw2)8rm=DwD>W*HSA-Kw^t1JE~}Oxydk0 zf6r6@2jY+V;raPk>ZmzW+gl%b5~0T(Z}|lJVi@#NAbBHU(Wb-SDt#k@MF1@qS5p99 zg{=?ruRMm@)r91~s|F$8G>1l^E%@tnPsKXuV>{j!tZnHDgx^*XCtNidl|LWKvVOwb zEB>?+CQ{+RboE>M^3RX@0Z)~~*FetPH5+u%MrSCC&i)cu$+kf$<%%It5D}UI2LtZ? zgmn9TKqaN8j13C}E@pWn0BOYUGg|swbE5DykLMACc}{^pO%Z1tI!r-u`^J_z(;bj<%3v^Q9qq*Z5?dp<3b?4Om zgdHl}y&s6Ms=k2B7ZaijnLXdQ3IB`%0?$)^IWfQK-^bYQ=0766ti105@tFMMLqc=l z&&L4Z-i-0D2OgJW(ezjpzB|Z%)elRGjY9C>IhQY>yLpIk&!PT(AG~P= z0yo$SlR_hGyOLO|==*{vn1Dus0q59|FPxT#a@0C$K!>BCtaP03{0u_Dfu~8gS-nLI z1jN~Dz7UT#eX9CF-#6f)ewHP0_bdO%xuMw6ImvI^PX^o?$TZ8hwJ2l<;35d6fOTo6 zK#cj!&QCzxG%i-;cRuZ(IGqW#HRwFs-!4;UmvQN1-ylqTdckZhfL=|DqJ9gG|(VJI3hy#VFO1Sm_~>JDYap-!=f=uO(cjA)9G)3G}_ z;AnY|q49uZ%7;W8*{o!;ngNFUAIicr+H+XssLN6cu;yPd4~4dndTA#=K1EA-YI#pW z%IR@1YglV3Lwuq0kwiG>!X~jcg|g+TI*=>DtXb!KK)`ETiU2{ddoRPrpFpBQg)2!^ zrKV@q`NSk|XWV)n3?kbQ(nS$~$CXY2d#pYK8ppKYCY z^!R%I4O^Nz*cVBFM+CjTD+M#r} zmv;jGYVg0;wH;PGwp;E*XKg&-ckVw_IaZ3Wxs(chts;FW+wBh^_1PrCFy+aKysp3l zn=Ju#E(X>d&!%1~DK1`ukjv69DWQ9~i5+blS8C6szCvu@ri9|n7bh3A3ip1B^{x+b zsC*g{RVwmaSJE=)JYJpxGv8fx?VORfmZ}q;Y;ZxG0~B|1`A8UC_|?C9MR4}3O=I&) zjRM@eJ&msO#;zYyH07>-G6dhlRh;z3<(ea-C92?NZjFMK5JJ;0g{f+>oz5?-{dp$C z^M@ck9U;k&co?#jR{W*G9%p46;q1e1Z5!Q;F$+{&mVTqVe?GG|%raWf>`)!R+R!G+ zgwT(aOINk+-uHk?IGp}4gN{+icy;E?nXFMUb5f`jwOr}r@$n^KYPHbk?XJp8s{A|CzMw#|<Z|L!Mwk?NB#gszj<`s5dfiIlivQE6Pw52gul0dgk|tPX z%%+zE?uWcQXCw?V`ST}utE6eQN7cDE?|g~u4IRb*N4xUBqE0CS?4tJlt>ZrSRxO}D zRM4{3>hdgE3b$ZSlO+bH{zUH4t7+bsuU=(r6-C%oJk@;~_mUNp4P#6a`4bj4u~6kC z+ne%>J&HQ0V~XfI(BGS>&~y#&+N<8N`I`uKM?GRah*khfD_u$SJ<58x?Is+!7t2 zF17C&6pr7tX8!oocm5ATcv>m(%(dU=2n|27Yc)XFFxA?vpm{{R|Lj9^rlC!>oee}& zuE96(Uj`*5+KHXI#uBC!-BZybv)6@g5;FK}1O=T;K!NM>S6Sitz_ElFm_19*w&g6$ zjB~vLi-Rz*D>RSLfhB$OHQ?8~t2+?K!Z`}-4TQ~?m10B!E$B2)Rx9{BRUJmA){HE)juX_FC+Kb9e?8M2X zn*)x&g9o%7^RP6a-*8CV?j!g+`l*S&TX^P3ySb8mSjhc0Ny4WDDYKw*8JjdNf1yqx z3Ac-Zy8lW5a}3=;tWpb!*~jj@o873)N>8zSin(O&IQe__STGd55??4Wn3ag0*H?=x zl3qrk&<=5o;r_MPLHS1d_1dL7LhZf3%PsrrGm%Kb(ogN*LPfru+{IiURk3ad{(5O) zXvmRc<@46q*!cZW-R1Y0ZIFy=Mrb!Js9qc#FV$9>(1} zFwOGFafn4WXkNcuSLElQ-eT2j4Q&jz9)y!4Z;gr|YHx3IMuR(tBOru1J|I~)zyGP^ zB#BG!%@&iT8$;G>WFogZ1vB{JcFtz1P6Z(cR?hY7XU1YnXIv~iP}t61ypEpMyEOR; zyZ^)2d;e4Y|9`;Y$f1lwl6~y5j>z8GE7{p9QV|XzJ11lw+eud0TS$&QBH5Y8IOZ|K zk&%-Q&h_eh-LBt0@9Ph6+-~PQp3mpwJ|2(zv;HVb=rt)0XNrj{e$Y0xRFJx8{rEQe zW!Oje7{G*2b=OT!+u%b?Nixf9fqmy>g)BEYQ14_==HqSU1~9pWS9APA z&c!ah_>I~qoE)sNJes}ueiG==qUZr60_NBJrHO8vvdZFCuEWVxXZ9*T48FwLr`)3* zG`*W);Og-68m(rJa?tD;oIBM)?X_`=d-Y;rIp>cN>la5Pb64|r9z0M&iy_|aeQ?Km zx<>^za#;&KZ3+Gubk8m9fGj7$0}{cL1s^$$uyu)%93rt>Y)b>Fv$0UYpp7u#DmSpR zrboSMzzBG_xtg1tzJt|EogLt>8%)!!DHI&)$?(Coq)9ayPt1G&+TgCu_YrX2tzS`43f z0u;i{aH+@!DK`-e8ziE}PfY~-Xs=U+U)}Ka(+`e*tpo42rO&d9%5%!(%|H;U@!P?n zGdA@WQ3c*lUfp{xI$uJ2m0i~EVriem(I{W9q+cV?D;4{Mr>$7RZ((l3Ce@*!O#e2b z`w5sPZX<555wL&An*6SXF>t*yfc^@n4_2G5wils1ovm9fg6XM*MnV4TUIxOym+ds1 zCWjQ0*cN%2z9puxIGo{{z69kg=0Q2O?l|WJbq~ufiH<50vafO~m^%clr%UB#zr^wh z2YeC|miMmN2zXq=hk`~2X&yA^rzqk;NDe|MYT4shYa)4v&x37t@XDbk%HhLa+!vEm zzA=^@1qCdk#+Uki`&0C&p8lpH2f?K8fGvsK2K8a7O@DRc5(gnw4gHsspa6-G@oP>) z21Q@jy_4kSMSG&<`c+_$N&P|H!@w28R62%6BSu*pMp0gmvjz$x6fxtf$$;ZQ$j!rN zLmfXOi3z%}4^EFw8nu4SmozN(uy!0N#BrcBmeCvm-yU<4q92t=wef=e0=U&n2*-B| zS)6-HSbqF&yIeiF2;8`xFMhJQK@{y#Pu1!oh1xzy`vn@}AD9=t>d{U&FzStf35PLd zFRgot!7e>ZOTO-yei!vePylY+7ZPam-U{^4jcT%6Os+q`sQEJW_)kIAS@%uSZC$&E zv9H~pM%tRJ4(LTEV2@M_4l#VfVNkr{V=odsVT6g03dR29Z0Y}U%+>+sW%Pg;G=jS= zi{gjW_YA9pE@U6%YdZ;U@M0v6!YDSv_^Z(yq>0KLlSizOLa&&h2-_FDacfSqpKg*y`IcS8)O89Hc+nsg3c)@XRR7@DDTHu}{2EFL&wKp=L8xE7>a?v_ zGbUJ_##{S#a}12tdE3iCIN>l46EBPTne+sK_P3C`4{9v-rd3fzYk94&uX}GZqx3CY z()FIiR5+}JGU}{uSNvLO9&5X^l*QP)Mqhk=_LLMWQk|48V)b4)#x*Y_BZ~$nhU^5= zgNmA)ZT_B!r>w(54#be!a%caYLq8%7y1=&)4EjGx_V@Rv07;d4mX@t!M|SfB^f`5R zxSVtI)6-aHku_lOIt?KIXIuO|y(J9%2eLqY zFQ50FGZ`!K#N}CwvMhKdw-USK;tP~|4Fp-9^X9HP{wbLTV?r z;{1F)QGB7foP>D&X*7u4Hs?kE8^Z$wTl2bk__M%(F@b7tYTl7NljAGuh^FrwUrUiQ z)>M6M`F?5!GG<^e4!nlro|;7kE;sob?uXtk;FKLop~ZQR!LymT(>+t6cY2I$1G~tR zhFNP{`sc_5Wj~3$zr9%fFh50#A>D5I9!!Ea`ddL1!$YmR*iT5}qIFxs{2dQvanF79 z%POmJuls%&d`4~4W3|6@FRPwMXLS$V&CJMa=IvFPRlsjDk4W)O=!LQuXUlQbVql!xMv zMrO2$sxxq79(p&|V4(j1-`hc#fFDr`bKGf=GO^GXC^x;{uykG@D%+#H8D{sm@{bqo z2+7dyfVxqRDi^5qES$=epNJQ%$jiV7CIDXotuDe4(VAX{1IuxFBAe`0cp^XZUfXA~ZZXHzSmrJxHOBj@I*bg2ZHvi~ zYdrR@?{)4oLI^G!4O09LXcysNFRp{X(^!MkOn>nWbqqA4sMFg<5rPGVKnshDweih6 z;!S|gVQ);I*rvtspd+*`_}3fmYsC@qT(df%q}^f9WIp=0tp9oIb~Yh9i?~D0Wjqy~ z|11gR{Ruxir)0_9X{ns+fO6dLI!0C9@!Ne+R#SuW4B4^W7=&p93prAp{vsdmpt&jg z&X;F6JxEbU2G*hS?6NoZ2Vrrp4RHjc29E$KY(@GGEnI4P0+^F;A@)Z(JKkAM401U! z{9X6N8WA!UuRrn=ITzkvyK_KhAfwY05)AX){C=5HHWc;8&5N4LP2n=CVqW0E*wxkI zl%8QV2Eya)z5-wDz7{8?+bvxg^mD2|kNDYXq-fM*(fu2=Tc4}l%kMxI4?=^&AMo(v z-*&&y!xXszJt03n5#gR|eTT`U|DdN0qTUIgWkDJ!@Z0~3rha-eS-BzTQyctM;;I%y zuvistyxB_cWs4?52AVrt>cPz)aM~ApuM-CHY~+jronIy{6vpIK+X=MQ5x;9{dbC89 z>8l$y)Zi{I>Ee~QBDv$r)52rR)0AFfNAJJH=4nfI*+oO&ML=XzdP1%%;8*?tjn!V~ z;$LO2wTySdWGw1)m+AssEcg*0z7I+tcUP4zYKUOR3nSi@rxh~^lJSkIWBmmeLzwMX zskK&Vs%shT+>|g|?~v@&(a?-=C5(BI=7Z^16IYNj88n-2esw`w3AWIW^_x10iiarh{I^ni9Wv^i%JtCQs*Avu}{`pjW* zE0ZDo)Xmikro|XB3J3nw*P@csDKOj-gh~2Ug87xbcBU%MeWY5>W2a$&_0){j4=`(j zV_jG4HL-G0N3J3x21bkwwa%PLjef<_ZBN)XNOFn`oo`B+z z_(BKYfmS3lGw2(v~WPLe4Jz#db)m?UK6|y zo1J9;izT!Z+UlN72nNj0FAA!jZyOsWz7YSLAA$^;hwmp4C}#~<+g0=Ep|En*Ka@DJS=l7Bd zpzKOW13#>*hyt<^YBK3|FCEIMh==_L%Zu+(wbfsN%}hqn>Y2eEHX=L2zHJNr z17x2c4(JN8655K4>S$sNaZXwe#N|}%@L&D=VJXha|o6Tvwn7PTe5F#p}oORB>x8w;yWuI z!H5J>=2XvsQ-M!h4kpAvf4<*wW{%FvjUS*Q+hXgfQ!(*O5wqK=wRC3rsrAiiY15yL$e2C(UDkuHnzh%GyTfu><`hHX6F@ zkdvaS75L;N+Me(micd8;jZI@1v6g4O=wUX%N(u0*4)39t?l*iih_uac-)NeizfJOw z`iJLqWqgohOayr^?9Owl;204Z;SSXllW+ORuhNzuFBh`Zz0P$U}@%H~9ovFfZD4XktE%7u@D`g<#ry=Bedni1a7lgSuqUWV>6&oG+G;u&JSDmt&}<&TBGJzEn}O};oMUYTg_27^EA zEU&!f&Y@7{x)JWLv-64nP>Yg($5ts%s?)g*_t?BsUL zxsjwDwz99&iQ0VDrQ=e1>a`_o@zs%H5?ONqX*9Pyzp8#%!nYviAsRlUVr_|H-X|e)w27d2yX0XacKHd+X7%$Q!7?)H=uMNB5IrGLio5=;L} z4v$j1pOQz`r@UHg|Hk?3)@0EAp;bopG=Tx<$5Yyx$8O_SEcF#QKBy3cH!B&zih_Kn z1y+U@G=zm{C_~4nui!?L7K2Tl{4Iow57zptLo`!!m!-5?Cau8k|b>yKP6NAi|KOJkW&zE0IFtCFm=E+tU9n({K z4t+C83%qgwV$?${w|(X{SrkUZb)ftNslP`rzXO>|h@3R9Q*1cUd>lCLW<{gEm0&(k zjU3-5SHoy4!VpKJq#Das8VKk)l@f1sso(Nv%?7ChUO7H2!s(tAsbar~;Ug#F#tRzL z{6GcM{GVPh`~G9LNz0ely4ue7JXu1g!$`y>7=O%HF!-hBGO)E5)f>l>)hks(o9=pX z6ossa=%4`jwErsD)@pF@ag>mA|A6v_^rFv!;`-RW>Oph+B75zDkB!`~z+EgmnYX=t zPCPq{*81XUPmcG3VvP?zDuFNsk@dpzrDXY%SJxIrMesH^sw37Bpderm=sK{zHRJSX zQ;1V3I8@+!Qp48zR1fiesk0vk%cxJ=XP!O1#Uq1N6*|ImGlPO3ze{<#49!FNI8Y1M z5$Hsrti%FLfJk7UMO*7Y`_qX*I@BvonI0wraFx(EM+>QaUS_j_^n@;>>3P9k19+Xm{r>BfpK{`n;73`eAc)pFF@^{p^A7EVz46<>fq5+VjgR9Sw@AXC#5yTI)l~uk*4y(8d@GA7&bNg{y*^)qK|qGm%Y}`o95Fa*PdMe{yErxRou%!@&DTe zFt9N>v?{`6nS>8RX(W|-5tfE_EeZLiY^59SB z$G;l7F>@}@9pCZd*sfjM&`=s6w4Rp&9CHkl(EFQz0fBNZSmp)k)gNW36Xee!szX$j zI6FF&#~1ZGL-p+NI`RIW|8A@SI>T3MIYp#HtY%N?xdV=ZnnH~N~2M}6m)&^-$#{WhylWZQozN5lL=?m2m- zMnT0!L=P44Mtf?hOK<=n8kx6efa0E0AqV2~t+&vL`*GDC)fv)TXjdH>H*!0T9mYa~ z?AwdMo@e54`_}z$Y(4ihK((dwyn0_d&||E1*0zeDD4dVRqWJFCeazw$o}HvKueS8% zsHq~G#rBj2%g#EbLf=I#S8?8Soia>s5@(b!q+ZioC4L25N*40b`>jsRc^NYgd*g(vnQnQ4el&#ZisXgFV}#>o$RljC1Tdk`=s+vRi!iXfS{eo?oSSF zf&GAy**=RpkrK4lvUFaJ^mtnR-qbr1;XLT-`AiT^F; zxFsf=)pRg{8DMFDg>h`|_c=MOp_1i}pt!Z@(foKTrGIEB4G7;Xd7x2fJp=DBoPxXC zBQ}}yP~NtFJe=S8-%8(e(!e@PEMhqKN9MvGd*x*zgQ(zN9UM|<&wps`^Req*NH>1! z!{#{rxSp;$;*Q~REzg@P4K|R`G&wym>F39vx-@?i(CrJktcX;8@LY>Q8p=N1DK=ah zoB+<}87nRIx~a|vTBvxeE^}3v@}YNT2B_XyYN>Dc^X}S_;>Ec2sy0*l=+PN?#Q&ST zzmBgC_G!{(>u|cCT+wDvn4^IXgD?%3r$G{Ua*xTYaG_UHB=3E^5-0-}u`&;Je)NVA!T)Rer?tNFThJfVWn5roo<8*V5s4Y%Bu1W|2f-IoM(R$d3A( z%l(fNl?9nKM3Q!s-q29__nl9(P0~R_oc+e!aefK&quvZ?_INLYu|a&yfU5;r|EGU3 z-lyLG0%{~epi|MNcjZX+!s|oF2;|NLyrztE%g+45yMN?+igoTQ0;jK642FnX(3461 zvztc&$u9AOddCnqMMV(H9N%d&(CVV7G+4TaHPFxIkXWv_Ot`t5w-@_CWBrW@(?K%; zOoZYWOimjF)77L%ZWCADk=Hgl2>ok8)OZk|5RBXzGQy6wim7F#oB;2UO;X3Y`o$?q z_rxyW(Nmi{C0*JWP<|)ni@0mQ7jHy<7z4e>s~3xCLdJ-w{vAs+R2T%7#jKZ=GwIuuePhXO%=wgz1U?4b9ve`v zejF2U)dDbQ-!@RR4Pw4^Gza)LW^s55Ep6xcCgbOx1aVS7{sH_*ngjKoKYD~w_OZ_jdI;{1g;m*I=3O+F(LoMMCWXDk z_QgR{Fqz%(G9%cGSz_r~Gvl$Av9_{C&5YT?w|j2SsQ@NgqjY7)y3veTvF+*uQ|M}y zzPe~*nSMAJVQ~vyP}elhUwu5NFd^Xam~`s~^dPjC*{W1eLNJ0u%{00Ox2S-ZZrf#7 z!`4r`K)ydg8Du*jeN9vr?fAP`Q`lT*e#Ki_93B=J zZbW@BY(2a+{lKOP=wiYG6Z}T%c)@}!@oIMMZl(K0eiv5S>uxE~n;FYz`^_ltr7G~q zTLDZ$bWqr8tN=cQSME_$0%6>Jh@>LQQ#ViuuKpo(6HBMgLn)vD(S(`HNl<>3{)}KO zX1Nb4;ltlT{CfOM(@VEF^R2L83AlQN|2dURmJMX7EiKOgPy+o|WA$K8j0~Dy_+x|u zSpOY*ToCb?cFL;b@08TI^-YfNSsAY5Zr6DQdZgdo>pkWGj0!Ss-z;7J@s-cyUE7qX zmx0hXlktfO7a%XkRhjEL2(-#D)9hPe=R0L__aj%|&@+woOj0Y4$L~){|6P}_)k+x9 z8-M#;X3MvT<=x#%0cQu?{Rm3@K1Sg;`R3MurQCgUbN(z@Pkc(-)fkG~5{q76X%7M7 z*-IYBkfW$&7qYU2Kl?H_>@LiVlj)h4&4w;3n-Yx1xcHe92uhIL|ci)@* zs;-UV1yg2ZxPMTK9MQo@nDo;?dY;JP2ba+V0PE1a?UH|ER2Q|=(10AYj}B3Z__NvX z6-pTR$=af~gWe_UYui_N{P(gO&DJ;H1imeoC8UF&g}F*}5aWRq%@ec?b=NSEHPsVu z4J92;96iE_eQj!ijBUT43%AhigwCogr7}Jmzw%muv#hNiVD+f^c|<8@Idrf-BX*yl z3Kkej@V_I%}>-wzoJ4XHmuor+6@i^yX{c9I=Bd(zLC z0=^k3J~Mr8RV}mFCMFz4o_|Perqgi{3$uu=iya$OnQC%KaTnLzyA<7merR@gQvBeK z>TLEkkJD)4H1HtJ=*AW0kop44O)J6GU|xV9o)#GCVaA+|_T;{>S0U|adcDNfNN-EW z(s0K@ZJ>yo-k6aG5Id{sks1DIN;K+8VJ~JxqU|+dfhYD7pQHE^xpsrW-Q}@y$P7$QUU*iiU$qNAO(3XICzbZJK4%1pjXKu zDNl^#Hy`v*q^>!dNrO~&(s6lWV>3Pu78Fois*IhwA;Nc>dtxPyS&jnGg znoLc#U6nrd7@%jsW^5ZV42A9T2Da&UA({b?BSn91RC6w$)-z+)7z#1>Ki~i$%&~dZ zlH!!fs@y2@jU~vcq?GXVt2f66>7x3pNePQ4qFXXE@mjtp74Ulz#$TXObc38xqk>;^ zYKBi$y|JV4YF9U^;-Hk*qVQ4k>w>XpZA~l&=kW8oI>+-Il8y#Qc18qY`v_m8vNC-= z+s%bRZs?oc@bh#uoIKC|or8p2P?9Og->+=~Fd7xr)tS003|10`uUX#ulu_buxsOI= z1Qt41M$DK4ZAzrAOxqK4Euz*=I@c8NJSMCca02}1+aX)xCWOZd~%0RQ( zd;Zkbm#H%@QEOug++GSU5~gCUT-J|7TuzS$N4MC*eotAs03qJ}s9OF9gT?b`U~vW{7Oq$7 zq363{LXNJhJu%*ZHCHf_%{O|W1ee=D!E`u+)Zd`&3i)%sAC`x$xdLmwkMAH|%v@QA z+96uznn2sgCw89%^a42e_4fgNNg96Cq^KeSJ3n+OD~oRj?Y-jg+eFG{U!wj($v<-+ zSNUa7Z`_-Ej=OL7%EtEu61>1pVk@}CW+0E>19@pM1h$LfRylm*ynCCNQqy)~(*%fC zLq|WecGP7mL9;+_kA=V44cJtfA>{O_i;7wQfQO`b2A`WW6~xD-^})}7WAefUmJp=?|g7ajz0+EAltXau2^NCuY~ua`ged_>_C-F&Lc`{d0a&~&0mz%MVCECmX znhQO8)#uZ#(jN=G{Gofrk#sQVI>c`Gc1q09Y2}At^~@Yu9+=lV2G*5L(-#t+goimHpnShL<9vn6^NWI5DRIxFoc0~r5W7u>!I z%uBIZGz$0BVwg}y+|6?M08y{3a1d(H2CWF<6_1*FoHApM826jpX856bA3jONh}Zxm z`L(~}Zk6D>si4p8Puk37rvM!b+4rK?dOwuV0;g9+jJ{KJ=C8i(;8y*Sxyz*WB}WWv zee9FR_r51CtUVi|0RHoZ>z+Cm{r6{u%eE>1@G)MH9pfHvD<(=0E`^|hebUau@1^^H zZ;o&XE;a45Z7)cvOo=vm{}K4_E~}BB+(Mf6u*^1;chC|aZXv~HCX|>;x-eqb`I7-E zzCy16rs8m_wrLn(_-?oEwscO>#^_jHn&@KCt%a<+R)05;r6%xYIAw z-)J^g7{{CA>sRSJEwbzZ|JA*g1VmK%|picfhCYz za;wO%)ffj5{7A<`A@0|D;ED24GVL{lop-;0-$Z{N93oT}%}#TVD{$-Mn$I50U-^_d zQ^{YFgWy3B<|J^R5$Ol|`Goe&hDT3&q*)|VvXFx(A!8@hN=vf^^YA3EM&4Lt;(;fi z0VlZa_U36XOmJfA%Qy$Ua$HJW<<0Jz+G}toe)Z_!HQ_O|-lC^Ed@u?3I$LGwkjTH$ zf;ONzTZ13>(4vMUPcsW2ugXr)UX)1W?p3Xa!jl4 z#r&7G=CjD|1Y3_LEt%E1vZv(|Wo*?w=_8YLrHXiU8TLNqz*&1FailQfX+Mx`L*?vv z?!bytrK4tYZSL`G`)wHj#GTVRT${U^dE5MA+lMg1I0^Lr{syab`bt$Y|gh0 zq`FIg=Sh~_4&h!5^jGGaN8bQqqW;TcmKpvJ4{6k6GSEvgLE9^TH5U270JlUKy@=Oj zxZwYw@yfpa;;9@ylXsb|Ow;Qc*)J)kijTW9G|6B$*C#B)$x1qVY5`0WZW1J%GLYEJ zW>^#RoC9M|V6^63KSM&jW<|E}}6k(CRe`Y)5kT9D3{ zNcMCba!$G53R4?l9DGZE6GaML39q^UgB)Ka`+s5V*PTl$t&;$qD|{HhNuNgA#Ieb7a2;z;h^2pKwygXqgI*Aq4$~tf{}8LGgN> zDN&hAP10!=ETd-Ne&8m=5VdEp^zNTK~CLSk5$cbjD zHbr+|_v>6ZW$BC;?R4RN@JEQpfr7}8Xhhb+9y(EzVZDks`gjJVO17GaB+8*)^~uMx zKe9RnQdA!|J~O)uoJ3p2&N-Ut9`v{OY* z0uLYfQI#I7)My*+8RmZCP+rEloSmM!Vr=)Y*Mzm&v6w~Tqku>Dy-v7egZl1f3i>xX zSRgv+Syj%8D^yVGHoONp2R)#VLb;Si%v#JkpL5wCaYjoKt{Yb{hL%hf_G8)_uXf z=(q<+7r#)@&LBaiYJG!wmC;0wMca#m0m{Qfs}@zhGl3fR-2b1`+?3Es3V2dbs1eCzZ-@AJ2v@>_**tW-fm!$ zlP~cqWKy=m6T2}4tXXzOxS#Y8$F=&=@H||KGer#bi$L|u<5yZu`t?!Jpa8(;ARK18 zsJRK(;q3EusFUAj9HoG#?g@IBHi69cdY&XjltA97(sL55=>RhWxyfl~j64tccRNJE0cdn2$d=BAM zp@Xbe1Gb3#rr90zQk>ha+E+3ScX>|K_)QV$&F$PN3f^eFYE%a8mP?KMq0YK3qzC-6 zj$TW)#jQ}ETCYLnafz`SgGBugPDT63^0bCh{q3v;<8NQ5_SpFG(!Ago=ArzE2jjvB zFAUx1rXChB-z+W)@VpI%<5c`#VV^&d=OjcyDYdJ+O3(O8qu#A-yZ9GH%W(xzf0+un{cy1J@xS$X|q7FSAJ&7P4*euzty{nO1# z`FHi+3PaYMB2o{4_6!zwPvxXNaTe%OS%<9=$K=taP==ic54-PNvs`+BLK}d(9-sC6 z&P&34X@BAnM=hsB7n@?5V15+Vhx<^v&P_VycEJkZjP(0Dv&qV2((S&TlGL4=4iiOE z21Q5qM1oWF7#23(d{IqBZw4*2=L&V)bQEvit`4e>ld=eX!#>ZVbv5NDZ zrJPZ->r3N?TR!brUxNMNThv8Tu+o(LLvF)J0cuY{>{b`HpNUw^HpAK`A*}rHSLp%Ozovy88vh0Z? z{_TAGK}*C{KPjLwJ?jPkYQ6xB{6YZ!GxiXH$r_AgsoGquZ^SvTeOl~aUqStElH}i* z%3?M+jYBv!i2E$(EMS|itn7s$mvc?-f?@e_0x8(c+xkz@4g^4$l*Q8~?~{{xf=)g+ z)y}I|;!nSUvAVwYI708Xq$d zejbyvr(Z?MP`|`L;i*~4J9{odv-dBDf8}7X6h}cL8FwnCvoJ*lSn0VIkp0f=%=R-a zsK>Y&lWcG(U;d$0Cwxh);4S=G^OxCV_h?&;A4ei0$TzyLt(;liy^3=7Qs25LL)j^^ zbZsOWI>#gn@9SQVLaU4Q@+TdLLqK{^LGU%#+-7lz%3eK zbu4bBcX}rmt~g>pi64Ay(nv;2@_I&ztHd=^0?OTgDrIm>z5a@g>q(W3P|KFo?xZw zl-1Z#yKtR}HI~HO96KiwsXu4-r6CW`2PeDB#wm_W`fj@nTq`Rrm3v$ZgM3t;HLO#z zh@_^TqkqI5kWY|f(Lk40yV`a+aB{>rpLnP-P=vRc}S%*+}5Q>IgM zpH+o+)lKi4Mg``cp4p9yQ+J0J*Y3>bh0R=0)qusDQL%Q7%Ql+<4OE+yT9T<(ihi36WvD4W>|zmVa{p`N|Hy&qRNQq(~bXi9cy4Loh_#U8f%}9 zjJb0y+%9Z_)>QGcwg6&6uv((+18AGl(0>c}j&uv~O1;UCf0SvtoJ(rpm7{4~3_GA& z`!VuH@=*`?G!M8ZevGjAK0)r50_=EKR-3`4*0YFTLTKF zx;B%mM9mm9!lB2{c$iCQ)!_;`9+S#-d0bM$eDAHnFZa6d)$ZAyTjF@s)abF|Jiyrj z6CZkia3S=|aRH&~o(mOaUtCbn`Zp51#;hdTE;(yN0bfR~UuXNh@;rX_{L-_gGOl~# z_2TM|=H8Qo?Uhpki`Uchu=iUWr!keN=Nd9CLg{a4snomoWK4d)Q36cIxJ3q8Xx9dt zkXNv|BrqIPF8lp;CE3RB`MnG4j_0*lin^vkE+p|xZ%Yp8|J1h__8xfTuv%L93%@MT zZPOJr05On+SuT_uC!voWnc4mn1<(q1GPiMV^w`JR_1K@(@ zxIGRDH`^}tgtW+4D60=JH_`it9EqeT3iN3nr+)BF;66XlYuc}0G8B~bvS)0Cwf^d6 zHLgD*Ict6)+?6yUCdWQ)-}vUXmJ7p9gf`TNZk{<#nDeFD2{kb0 zzS;c&Eaq`xY%z*sdIv=-ew>iZF zX7#`OV~ZD$`{`slbrtSvI!r1Xi7{R{zGeMt;chhIGMl==0L3%=@28|A)PgXvao@Ol z`Xc+&6o!3Rp{Xz_fmh#Za`%*ur(h|Sd+tSwC zp)1#WEBd@(E>rZI7_`ZP9be;8NY>egYS?4T(|+SnUe7S%JkF9H;P7kM$rF_hqwjM} zwU5NXzA4nJQ#20-9BB{EUq>d#Ew0>42wd=!aF_dU3gsT(ukpMu{UqBjW)58>IxcYl z-DOC9x&0x`vuSmpA}K0}$U69yyGE5_n2W%Ldfa|og6rS-KWb%oE}X~94*8OcT2F%R+kT67n!_c= ztGv-3P*t3E9n1!j&fU~g5nnIw5o8y7gU&ftLi-Jxb(ar+Kp8-1w<@LF*4hI&n;&In z{i||C{H)m?Uzeo?pD-&gqu>3OC;<7Z0NwrF{Vwvs@*j-S6R4F8r9`R zfBV9wxh~6zV(n~|D1iD+O&FC#7V3sYL$@^j{td*zvFwl@&1mRfw9KuVw2s))UD z%?64JHCd#%{zBXnLVB-T)H&}QzMlX)UFt%~pWPv*%h~0>9rT zIV@jSp9z7E4Lfr?uIfutm*#xv@hCVq9wBLiN}zZ?h@F%Q3cX(h3le(U3r{&-DdufyLu;qV(HXnpw#ht)`WmjI zu!YdfBvmqAOaH*(8b2kNtmDhnJ3{Z*+xA0hf(b24zUs_eSHP`!@Ex9jW#SS^rQF9M zZEwEOz~27cx+W8a5;!ONrNRq{{n@NvUAnR%lA)D~_FI(+5tRR11Ni4Ec*#m$vnQIq zUUBlh+&9RDSZykAV-u=<{yOAi_O2+a)RRKHKlr$Hv%J6YYim6|1t;<=n>%mR80cP- zTlp;=#+7p^2Y&zidq-V2kDU68-a2|xAzW-$;^V^l^U&I93+k5xmo63o#tz? zqeio(fMgmTc7T8uPF^k;1DrfYU539*S=WT?w8I? z#z$HVUrkaaIec$btJj(U!)YkA5VhZAl9N;pZ*+5pg0)4syhxVMv?j(N?*F)Hk(MO% zs&~vh3-7GmrLI9hw%@xHLo|CgulM9hNiP#YhzHM$2f#UY0Dh(M^DUkz^xN059xf{;XN+eJ1wVV? zMi?NgWr4cPuS}bM!nMolo}x{bOku646U}Zi&h_fC+rp|I_KKvV2bXd0>@>1KpP7L! z$8%%V;r$LUeK{)d{up7QOa9#BfQfM8)6bucY6F6}KC6CtdI1dhfRt2L9sX&+Ivc$3 zUK|>;XkfG&Tb7fF%O<^GOae}bk=-o)cqr^U@gM*P1#c~JwM&I2j=Mcr(STsnq_}qe zyfunBq;AxQj}~0IZJe5ItYm~qKK0Died%grU^j_Mmy7_NGzxn8lPz04JZ>J^>;C87~;bv1p zCBZ4K4;koQU3yOP($ILb{pp3fd)m>cs-w!mci(Z16~@UFCz-_%b1nsZ{Q`2ZDBU38 zO$z5q^+N@`;_+dz4M|gkx)@Hby_~q28i0Dtcc=~dx@2*q1}OOwbq{Vkcik8! z@z2yKiU7co{usQq>PgmBD!3}v01J~6f0h^ItHb$y<8Of^CvxRcp;r$r;PaIi$LKO~ zlpMuv^t~7n-3-t>d_oSAJ?97+vzsl9aJ~)H;;Z;ZYpfOvdp4>-(6EZVb3LTeuJz~O@j83sO~2+I>z|X#AqV$0O951 zoRuOqj{q;$mmQaVG(Ax0g%Lj@v!jm5Yj9j9zuz)gkM0bK$VfI&DAbxXG>u?Kp0tZk zFx}w=r!tcBF3P%^xL^BS#|Nj>48zJL93yBg*fsgEj{?=r6>)wsW}q(N$Y+sekn|-! zd{I$t6IqOiNU|42EkrjI^T4Xj@_Dj@8e&q~uvHNx*81(0u}?ygu3bMxhM_5(ma)3; zHp>X`#R5$Guq1}An*;Cx>+9RCpK0Y7V)5Xu_b;ayb1N$YGqcs2hAS(+|LX{-{U1Krdad?EJL=lCwf^NU z3YXQwTXj#z<_(M7Jv`0`Q=Br8i;igEVKX;or9B-z+9@HI-wnvS8`E9WS_|3P|9`DtVc=`-;jm_0B-u;d=Bh{5F8#FU{IT-u$bj2)$X|EgcNEY7Fk?zg~IL$7QfIS_moAjI$ zwGPeFyV+b_KYm$Gd~Cnc*UsD4hsL#qZoZJmtEJellLt}ytU6t~Q#X|{pP4=VgMX0! zo(BWrqWl^oA(957sfk7PE42a(TUli9y#_sI2HSV?EnoiEdR^R08h%SVdljPj*5Tx2 zURE4l$a37%x{)yO7MyQsqUZ&_N6G=FNP%Z=<+=!m?yJ9yRw)*@xKQXSebz!RtQ)nd zqN`DqZ4>INz+yyN(z-77SWk=Fgc}|Cp#5clcnEQpsxVdHLA=Az>M=0pIdl$O6|6pf z>*bcMgDw)li)EzJ2=cZ;LoZ6e$Ok9i4;ArzqS6ggIudE|Uul3ZpfLEjwSBwkI zsQ3Fe`SAYze1v|0pX8S;fZDxO8^%{L6x2RSx_d&n`JsS8O{P-^!uflSbf7@kmH?F zK%Kdz<@=DIUIhUV>&T}4;z zmAzY*BX^uyC`)Q;gvOA6>3oUD!N};xHyQ znn(_q?FtTM$KXr4ke=XZ=xWEOFUf0ceagTOatuiK5cL@(v5X7#N(>mGYkYNK_?>+p z!Pw2jK%fpYZ)aAI51(;m&{VC4H1(WY+)Bn>@UJdxvUw|bWR+wA4kv!~e>1557O|F~ zu?)slc+6M$D3W+{aRG&{bGWljnX~QU7_}qH-cQ$_#^BH>1W8vdxv@?8?BMx*;OzqD zCN@dRRY;;dW+lV z!bcVite|ZIUlmDaa=-hE!je3L53@E@L4Aqbp4j((J6$Y$)foQ`I;Cgma4k3$Ig(3{ZakBRPZYOfPeK!q5Uk ziBb+NAfdp}F?1s>k|Lqf-Hn7u3rKfMD&2z8-Cfc!NQs2ZeLd&g`#k6Te&-(jgXa-u z_Uw80+V5WRU7rP~Of)9x-mtN$swW-nwuZ7-#HB6ll_4I;!!NPbCE>=6j^g(lLQ_D- zR6Ed0j^`t%N0}x03KZ0ONtWvlHtpv5+M8ROUe&*lf~sj&Mf9qMdwZw+9>G3spNdiC zejEGl6qh`7OC?rK4AdLfgkfy=;O-yi8P)4?qfrhb8PJ4QQeX+*CfR$BZh7sY1d0;EtuG5Z_ zw7VEPEZ;T*!ZX}{p9XltUC8!J;#>yg*s8%QGleQ`voXoVGFD&Vyjq#^n@L6+X{ZLw zV>@i{wC-8Xk#pxS!O)1S-(7vb;Xfwy)(YJ(fz-?@7Z6n}c>Q5%2l6;-;lH(y;vNy0 ziuCYTCqt+ZAiorhRD-32xB>mLNlxMC&AJK~rXkt@Hd_e~dRTQhsNDJwsR9HXvG zTwhBPD@GqSfAs81ONf0TpXq`b07QCrI{Y8X{jsxqI0Ha?t7C9q1xImOC7G!CwJY?@ zS|0jzB|!!fBFGSZ_JP`Nf|j@T)3RfaIs+Mtl-&en9&r(wDAD}G@j)w%$-(c2#h{JI zWNh-hm7m_wN};`Pr+1>J(iH`-0HtDSdkQ?N_C?C)v}#@5DBX%EGIR+tEmzlQJ?m zK|{ft6C`!a4Y2gMfMdp{y4uaFouilld3f1m&_R6Gg0zHCaG zWz9#asSGIwrjllkd12p?iPBwpDUh7CmdbjfC!pC3sD0Mzz@CD;>O&t&8LK4PwT@Gl z!8c{4W?;bpN^gtw0TpWJ8h8a+iOrsSjJo)%<`J=VBSJ{U$3mGTlmM`mVwq&uDsk$T+f0EW68$6e^Y-%J_d;^=d7Zb(`W2=`mNP&zrYdUZ{ znB}vwwrZN+>IY ztTibjdM8C!XBE0U|CxPJ^*!X9VnCtPYHvetH574jEQqWIIhLNKkYfk5&`Y1lE@6=SRWL-gm*5;+@p1nl2^kT`>T_JgK0mw6rd=a~)rdA%xs zm3dmbZwN4v%cL4z^__#~k_LZu)>fenW99=4pIt7g!h5_!q4j>*@| z$=a$6d3eW!zIM$ABQm8!7S8RGY_q>DEypPrMefL}kn@3y`pjoFo#6>+)(jQwR{hX7V}5 zPC;QB&<_7_#2K@@UuG9E*pDhm;YST|Pnpfb1yHvB*pXorqLnPt&~5%MZx*##+)`eOo@x76(xHtq>c+s8j)8OhObm?5ZEY#) z8_jQUnke8($vHb$o0^&R6trHmxOXugoXEK8t(s#BBEzYu&k9Jk7RyY|YBC_8rpKCJp3Wm(Q)%sH`+D~1F1amF!<(B>u@i(?n;n+mkp?{6j=t9d-zv#AM# zVScvXW+Kgr^c1s?b3*f!g3cF(THWug&muvjZrW>Q`1LvAL0}YB2*rSh#8>ybH;Sga zkV!sW-oQTBM7&2{b(u+q?d7eB`cidQUsoPH4_a668S2KG5niZYWU9O0hnQv;C-n?A z{aWT(#mgu~5^qa&wiW&(3}&Ty4N8h~oS{*@<&v*EST*^qw13C42EPQ z0}RdixZ&}}PG>Ea4-M}Pgk_J$j!Y_fSZHBji1wj*KMKg@Q;0uDUi#<)MoJn=tFu`h z^6ZNYL$W-7V=x?tE*Yix5C!zxsi>s7s@n!94TWB%*pX`IBo19dpu^%N@nHJ`SHY`T z7%S}~J`1>N6!s=F)RthRK;iZCE)+z-UFRUy+POnQ@NH>VDqDo2Am-@ zA}X=Xm&H{+ReQ`w)w?ZB)f3gRU9sv zFo>-M%X8kaACn}C2O?sCBMBfpI%*Q{h)wRjZa*$y^qP`WBkk--KN?eKh0M*8npWQuLK^P&0x` zYJKQPYF(qwnQBTmv7s}@Dyv9g#mhO9*Z%dC?R-`xUca1@Ml^ zHrc0!fjL7EL}RJhGXQZ}PQ859dfQQ3tba_jGA8NclNI##D9dQ8{6bT-4PP5R8y-c_ zmDBf>1k5?_Fn+0W~6#7JoKyM=mQ)|;x0(8()?lOw1?)_efg0dSiX22roy zqiB8JQi2F7o~|zNJ)3|<JUyiYY#xHJV`>+4NGFxdtDee1I$A#}N z1<;Mt0-&p1!oL0QA08d`ab{Gr&)Jv1amBis7>~KSNG=pn=Vp`G>7CVK-&1J%uGOm<;D@H~ED@yqLedJMpnc(fE-Co>XT~i9dj4CW^l$>&gTmEMx>ih*T;Y zQ;&Aig0+?!4C*tx`Ln@2b|*w*@T9#hj&ua}=aZ_yP-fjt3V9;PVBdqQaLTl>-rq&) zHW7~-9Ha8EwM)~pH=Zv6vKKjl1mQy7SwNVZs4vpi2^+Y)wXA;>Y+@XO@0y zQ*V~(5IxiL4U;3=qo?=fk9aZE`A}@eJ6E+8m?idr*)si(A%yohR7FnT-MYiocBLE}8L6lY;IW=UNrubz`mqqbr!7ik zD+4qexO3#}hu$NTj4EhWL0ybCudtv8Ncxl{LpxHA!y>y!7uiQs6jfQ}Q}*Sh)R}8O zoK$C`L{ghMv8t4{t%{HlAYO>Y--c0osrB4ee1oU&Jp!OWIstVDNi&K}{x_WP1M`Rw z59~^run-OhXfq<1rpGYCnI;COQKU0fq(co2r0S)*V4nt%AWOX%-Dr(3hA$!6)Mo4^WV3G zjgrVT!e3&m$0oO!;?4D`JZf>(Zo;M;iG_ZhAg5E+%-CgB0<2`Fn|U;*7HxV3lURS|c-IAGj8R6C!L(_i_`s@Mi?*|s3n7;sASnw3bZBr}&1!uXc{ z&E2L%2Or8{hESq(2J99W7tAjLA5Cg*v)9OrbK?S*Qz$ut3&+I3k~l~eLbRX=0%eyW z)KenJnPy6zi6z$U?9a3dhxI{`o`WdKho~qZ3j?LAp4lX{@j=tCQ-gNbM!H|lTbZQ{ z<5CZz{=wh7Aw^yGYz*(KIOJhBeZZ5aVE|)Gc-$(~yc`hD8NHzAnvxhnQs3}{Mfkr` zm;aykt?3ZE2}~t*09`#P)c+XN?2kpW6`Q7D*}fIa{yKtz=;vqh78n=-eOn?WCg#~< z9Vg&K??igC8^%QTh`;%oY3m>AzKpLfsoX=C}$^ zR)(i2SE#}BD{IXy>$pxR4nh<&a1NuKC;IYMK@^{F-3v;cRA>-NS@2l7Z5$gHb4%S7 zzZBa9WUvLM?S}bVJq5Zq1VJJP0U5!g!*0klct`al_}v=9;0q)2x;88yuL#ubt-5q-{nmn28)JNM#)3v zsJFxGhk3`3#iTgn!*KQ2!u?Ez?U$0yjZzGu0!XzGH6A>9aX@Io#hubBR||t>Gr&NP zx6AdW1*wT{>X+kDgB5U1SSaiUP+d@r0p+XS-3^L zf4>RTVi}4L?90#fC9*VBJR-|_9Y>I5KaiZZ3`q6z5kp3Hq((}+oB6Va|s+XbAT>Od}nB} zyTN()8<_4EJvyp>ygXbfxjuOBI;F|7^R-3!yiJ%h+NDXscAdUP{j3wwB|zJ6V$_vN+s=)C>h0bp%3#9yA`*ReSl%{3UE9c zA}^h{>dIe>e`UWf5U}e%g+U?_Ov}>AtMnRu6wiy{iBnn(%vmSz9doalcK7gVbnmK*PMb#xIJ$4#Z>XKy;Wx@Kz>9gFH0ZD9LD`zL8%q_x z$jsTNQGaokS<&VzSM(}^6OR`~rT$Q83OCpO>ZJEvp)=i zyvsHQ^V2ar%`^M@j>J1V%)3Kia&V%Qi^9b+bv&T9`G&QbopX~wIZ&Qxi$UOLiG6V z`GBp@^GV~x{kg>-aQD?PW(J{4tHg(%wetLh)QN8l4NFPtN;zj$=f`GJ6A}m#GgE*z z&`OR=)#%f&fF@>-8D{s?0S3w)lGDY)Z@-+|h;y^cIg@Q~^#otN^;$D`YKo6#XczV# zVP0XF2arPJ&R%ZAMc(X|7sk30V@pjq7-2=l(kVG z5VuozHPw*qz8X{EC4K%N6Icx zgpR+|F8mlJzf7vLa=fkT`}B@(f*eD?Xum#}M!M&Yz_I7g*8lgU{d?&Kz*57rayv5Tp*B(Nj?}6X6 zqx?V2;u2f&3r2=)-csdIJq-_27=+<$Fd+esiMAbtwG1gbNhfGB>rtwwu{~yQ$IG8{ zmKUr|PZGT7HgC{5=04$QKM!>sa_GDSptZ*!@4VPd5eD<}9&wiAe1d_NW8ng9%oRi@ zw9x|}M24gBpLR7f+HBmZ65u<(cDY)Ibf>Yy`sEy);@8pv3~h@Panc=n5N;!7bb=eI zDJSS)F!cm$wGzHkZU?+gEYF!^8IXr_1(2+|hid0rW*K3STRNcqFMq5m%OmiXCO?9@G7sc9L;~&&4Ho$%sEDMZs!fmm1)*oB zo=qn7B=JT)o$+*LtGLDL4eZhXXt-zCflI;(d9!1u(Yw&uSaswC$1H`<)Iqlqt8cI@ zWa=(a5K;Sz;f@ezY38s_Nc-A1-t?#j0-1|OnH227I7pBnS;O%^3eV4goE5J?a1)ih z@l-cZib|RIv4yl91(B4+25L?2UQ$FcsmVP)6=p7!j`d@$pD*J4^p8>dwM!hoN$fmu zc(TFH%*O(|nGmt7l{$l4Kx_KHGJCDGkXbw|A24>IH&-6n`*J@E=pk+waXdNbVa|9b zOXDu3k|t{g6n8zAhLhpXCM+5tSd_S^uiS4;#*H7v* zn9bHjx-M6T``75Vv@yc1ZY^Ut`jbJ0Ui=kU?e=1`BnLKfTucbJ3Wh8lG!t9(MbquH z1vVEkdXOO54r|$xFE0E>VXNr_CTH+;7a}Ul2e1;fGdTt6^~~M%SM^>k8-5YstG$Dq z`1BJu1Y~e^b?Q~(4KugHH$)%NegoY`NzpzGAM-L?+t$_ z2yg;IfHPc_1rA zX_cnOr2Qxnkz+AwsJ9)v`i)fpjnK#W?N>C5(+7A~qeXgZgsiyC&z<`kuITqDhzWUj zBhT!yV{2BM7!v3mWTwA>u{b0;E-;)`Y%p7U)68PSci(*m_?egWor5StA}S=2-mBW~ z?~Nx3DMs!bHMgwu4_)KQnD z+Z@VQ_8WVmw+xw-Vg$Jewt)A@u89i;mrMbWq=Zk=)G4A0}YXxNVAZ-02BJe->3;o0Q-v>fZ9y3 z#IoNHn8p+C@{oddKJiSmmN}Bul}sYS|Ne9BYY%APB>FN4p~_fGH|3^2bE+bjtRUjM zHFGP`4QPi!T3Wl92l1G@wcFc}CJ5VvTnhU9v6jS5fPfuRWD7{aB1$b`&C{U6UUz@F z>8ngE#SoyuaC4>UgHA3v$UoOA9gRAg1*=13V;4^Ab8msHAyoRMwA}~EBOKhgV_(q? zU(Md%g1xQR3^1eD9l^<`a{5I-n98WW2>+r`&zAWX*>)wygUlOnu-?BV4P`z|3Z$!> z|MZg~(?PTTgX~0f>G!Prr>)n!t)@fiB0ayl`}(-I(OehZEkNVe|A$HNuP3$JF^tRV z2_+|45V~#b+3DQUb7pxW!3glueVinzGAO4aRTI{GofqYN{TL#Maj{+!+( z^&3?b-HeS|~T7?o%>nV!f9 z<|stq;wTdb{V;q5CJ2BHOUe%9WBS>WcLlvy+@f~4ysFXh3vqRIC#vb{I^>iyGtD$< zr0iw_(=K~v{e`8eiOc8WprnHmXfiF&ughOBB0(HKd+rNT2rMr_sVAU0=7@wZ8Kat; zKRtre`W-XUU81v*x$EVvOomDsVyh}<)+J4T zFp(!j9KHvc;i8}mZ8rq1oGjf^Mouk_oPuCBVoxkyi1|D}4giAXVVjWYH{zU(UjE+K zCdWq^7>4BGs{TS_ywl_cDTM$xS9TidXSI0K!L9DAJx;AJ{1wEUtU($;pYKUpHPNPU zulAVFk&E2O>$I*{XhsYkoAD{24B)Wk%-8t5PJG;U()(q}?oR@AI?W?ne39yLlqdqeq+8q(7 z?L9uhP2!L>Lp7nc&NxXq(L0E|+VL>HP9As5hAy`~WG0?JVKt~{E?E><;|tA}?I2Dn~$MjjDll-nC9 zMs+`dP!_y0#528mqD^r~mV7`r^@AJ-nR+-&U%Wh28^SlURu?s>$bc}ti1X`rpLu)U z`XUPeE$H~AiJ@bL@At;E==dkEtlH*ZQJMs?Q({sa|0r`}ysoZ9 zCvQDVCxhynWLe&o38**bNmAyJjJM0nZjA(@9{M*e-)=&>7%T@KzDd$ea-?eO)FXYW zOxiwUWM4dXr?I%<46``J*WIDPAK7sABd^%g_jLc{nPpp%5C8Bs_M#g-e~I#ivM}Pt zO$b!{KY#cK+%7CFjnB%WeF;?I`T0y1T@uTF+zNbb7pvi%xj8wKK39jb?%$W&Gqf*2 zfuyfktK=K-Qxh5*1a&J{qr?LWUC}%P2MEiP{Zl5)ANy=))U2RZvj+QYHZmtQ&r8TL zhWconq5Y@;;ohb_L2T`(>J-l^+H=3tG_&s#@0zBcCt4o{!mbA(h%3)+wi&~XUxrxi z+4Id8vd~UT^9rs@gMeh#b3Kw<&~=sN!lEO`yO2Ict)udd$iCX!tz>r~4<98QsWXc4 z9k|;f4`5380e8+e!UUsdd3Y$_FB#SZ&l`ow*`*vO$Y=Qx~F=2HCu#vnaE zTfg)aoTd`x1Q32Hw;yPnjP;B5_h_|R2Kx-lX7D7?6gcKvVXiUWjBtTGT292i|+@ z`A!I#V~#WOy#Cb%YQh{hB=6>oB{HRb%~oTRUU>!TT%o(0mx%LONhnhz0m5$hm5kbo z00(}=UAFa!2~l+PYmYaXR~8*BAfYgDKr9&+YWdYcx%(4$GLA&ScU|*ABCR29OzYVb zjFEimJC9D)4}MPhwct9c;5~H`d@Ie!gw4T#EfFZ?R2*J$jmj@dB&VP3JI=SIq$E7} zn2p&Ukp$x^rBvK}t-6x4IeMAzi;s8zFmT!M(TwFaMRGIKQ--lC&Z zj=}qZ_)AN?8vO75ufMkkXABr0`znsZLhl#F@@ie?;55y3@mMMLdN z`YiGiMMgb6*610SPyJnkaaVs+(AR7D5fLOygCDZg>;a%Ct)`f>Mm&!5txqmYhFO~% zMRMkwi_IJdAdoBBnO$|bh5e*+{#T77?;TI~Mbl%qQ(ZpSdDH>~v1M@f3;}J>5G<^9 zL(=Jfv{Rn+9v8bxd@1C5FR#htE-7TCfB|J$Bp=@j3_VYaFkQ!AxP!SHQ8gKv}4-qZX7_GF|7(v3u&Wy(d3oqvw*`)j_)H-rvBSXH2)T zhfNlO>-}&=wQf+`bY0mQ8I=*9HL{!A#-BUey^x?KbV< zDQU-8DRLyiuE!vhVZ!hk%YeX$tM-iwC1^4$Dm96fQQ=2GA;PZSJYd+0X9WQjd#?m_ zH7ylDy5Ago_Vca|&rb>Tr@+qJ1$?0TTHQc61f3`59#DP46k2#!=gQ;mdIN(4G_{`N za3`+^LaNCr`?a4`l&MYp5_NyZQl?9Gpq|IggSizo^?`m!#B4T6Hi{<9aMlgk(zp3<(emv)w@&Witg_9XGR;BPIlt9!yMk`4$FuQ%ai&+1621dg!DF$Oy%d9<;M>O7T)cUn4dRRzV#9&df?28zTiWuxKML< z3{QkcnOf95HJumT9FgF$=iz#+uOTX!h8WIHh#F|EXn6R;t-nMTub+nsvEAWp(3<2t z_|Wr?w&+t%=L_rwmml*OCXq8%b5x$BilvFKNSx>L^3w)JWKzxU@^u}H?*^Z5wxkX4 znZ5IVb7LV-KT>wCj>h%z!f<2wz4snPuCZ7P%4KtdrD+jK)>wM47KWW=URo^H(4QnG zv#z0CNg8AH2j%UtWG26cPlKsbruZ`H9 z)1o%rkci7W)e)Pc7VGmU%Vw|C=J==-c-b%CT!(STBHhokKc!Ij3^|<*)S{SQHD7f+ zYKmw?qL?psLSlyU)GmAvX4}U5Mv1N07!?wnM;xO{u2IN^Mu*EE*1Anrl~aRXWtEj5 zYT&-e^b@@p>gwwHh>W^M6nH)T^djbie*I`4mzIVG{psnc|G`Ssf3qUIn_#QEPQ`6- zOiN1})L^LN-!ggHRqwd@4b)v?GJG!N%M2QsYHbz(hnc9}7u9lRb~;u?N;-Dv_~8E@ zQ=As(h4|*@*LcmD17>{*J&r>n%m9HF+XVd9BQOTy_Gnf``Tu?Kzwf<|rO*b^vA}59 zyqug4dXt8Lq;eLWa>LV&t_h(Pr%3$)^-KiDh5kL>!1iR`IE~z0s&&Op4qn$h%%v|G zeIpAL%`APJzZ5xaalEZ}MhsX|QbaJkw{X+l+82PvbYa^gq^V@xwK>jENhV03-SyRG zl}H+ALuam(1&fvcE}?C=LmYC>On2w%Ti<~2PTB2d65Y8kvGt1jrRu?5!Oe~K&M?=<=BAa2wDlIv{=;KO|nP9hB;OyXS45h5NzJ{wQ6=|U{Klx_MrHRzw_%Kq}Zc^rO1syj3^2|GKlYzh1N`qse}ME*$NrTx3-j-HFrAHB4+baZlZWB-|=VmJ>EPu+~(WWc{$$bWwD zzy7L%kZ@+Uwl;zhU6Xh#SkXqYw-j!bU;i!=T=pMK+} zs<8q(RwRuG>*bIAWB8|M9hjhmfW7x$EbZSf2$XJkagl>rB{dohF9ma+aonpHG75O? zjNROx6ql5Yc}_1Zu=kig`o*pKM{8kRq(zZzvxa z7#JNHS=o2_r+M&;LbOBv{cQebd0>9xhuEKLY83SJQe4HyVitUR(Xl){J@Z;xd}av> z-HOp<|G>t$<{+4um`zsb*Y@{>Ts~A3uzn|$%Av)?!deGiCvj4AE2w1IKeCo?Xm|fk ztMjh5fZekASEGWB?ROSb?~@nIOaElONfSZiqOV955@{pP6sO@64Ixkxcu4st*BPe2 z4A8ox+C%tUr3)fl}x3(_5g^%xk zscmS8#2_3WAOCsTOaCXPg3W(B6ue(YCvk7FSx>!4^MWfSIXSno^5LKEL2YT`AHa@U z!C~06g+X|7c2*GOu=X{g$`|DWXhhpfhmilh*5G>XVVIEfXNK?y3-4hNrV2PRc?Xe^ zku8p6*uP(*ZPxfN>5_jxbY>fd$zA^C6u_X+1alWRQClK5b2S4g4X`97l?xHEKUrHp zaU@zL0o8EF_Wpi+nvkmkYkabkld`gM)$=Zbe9rf|pZ`k~{O`BXHzw$1+f5Owv<71^ zI00-vixrZ$g^Jk`>X%<#JCL53RE)^fbTf6cmBG8RBG8TN{EXK*64bW z;J7(>-g-pC#>$$TmnXfmv%~Z&{obG4S$6E1&!;@rJUu-dZxIl9lpO5sfxw-0DkVAL zluUEkk28CWoJ!d3yXsAdPMQ9p(dR$1A{7jtMn6ovt+lxD#l5n-$9VQE-EBATHI$U} zrF=Gi@j_0!^eJP3O4<>s^N)@Hc6Xqd&HRxo*vSXMzO3KJ}wr;{vaob*|PnF*%t6rGs(#gw!7$ zqhu(O{(m2`qn4s0;(+xf*L}=jTHH==t5w1aC-~Roq@185(kc zy>s!{d3Vx1N)R7y=9Vz-HQ~Qo?B8r;_gQb^Qxuk$4|ymF8tUulFoOtJDT>j;+ur}$ zzwM@m-9O#N>(nNry8Bw?^ELqL|6v<~lO?fm~oq z5q`EiS;R&`NqJ20(@u@1#@WQ!SRA07O%0gGSI1Pv z@ag+5TuT$5GQq?5MSl_*!ks1NN+uP>xb9D_*=y0*zsRXsU&)Sm<^FOl7cDRpgQC15?y(+rhXF)h8%1yATrsHk?@RJ((db#v${A3B0fhdH0m(ygU=#EI}%$JDRGE!pOd>_${c`>JiI$}IKY$yT1sm5D~p>hIoG z1ZT+}Y`MUHlEg9i%zm5OdMHPKN=cu-VT*ZHo!P4E^XjmSf%lLNw`ehMhxFz9!e=g1 zO|urIMy+5>A&3$eP0KW(TWeNzJ*nUtT!AO#38wETFYOSsAujJ_diH{!Ep6mWBS-kLfXZU`{gmZ+l`>?GlX?fpq z;eZ99&J?dbf7I=c>ar4GD;XlVzn<1kmswg}SnK?f(%%Uh0OqrFm(0Vx?#f4e=nXt0 z@Z?v6m`7YKChj-SwVbko$K+^Oj}%nc0y#X+3KwTSkn2AcIB|dX76NB}tP|J?Ps^h@ z*ceD*j8`x#{C556!w03`%Qe6BbI7$$(0&Y!g`iq^BxbIxTXn!&W{NoR@h z$qmTPPB?c8p=g=tyRN*5*>M~l>+y{~Ivs1Lj$nL;Z{0CX-~y6g36hV_e6%>8#qy3T z;gq+!P}1?Ivftvi-U&(w2^ZD4+>c{c-SB?4Q>-$5tD$a(+q&g}1Gsa*632%qQ#w1- zRI~hE1~~Yp)|yk#9SzoYgegLHDN;noCKAE!G_K`7tv^-wD(Ckryz>_$?03dlteM=| zIR$p|w@>szz4Hk*^N z{73k7|6}pyl`zM;+Sad{zT+2JzGBc2pSg+63?TRXy7Dn(0Nls!)2Xs+Mine)ZP;JG7>^Jz0Ol3GY==;rx@vYnv)%@MDvEU|sJwZ}AYaLO6mbJL%+ ze5pU4$F$w-8=WGMLC51k8-X85QHBg?Vo*rO*;4#a!W8qPJ9e!Xo_a@J6Mm3<;K4)o ztOb4j8gAX!A)Pjihdal!#nIbQqw<)Br1}kfr?xC?{3rK zy>_45;AHOizP40jeGk0J<=!XZ@H}BFVA_vqIDWKCQ#y}`TFt$vJW2YbRvJSRZcWLa zAeco_v#CaI%|o#MtdS6GE(WQx*1JLB-WB_`2IBxo{PF#c6cspwh_6{olcKdxW@R7# z{_Mi=>o-Tklf0F71SxhEr6e4)Set6C+HdS=(kj4z8*UHgkR))-7MAtY42?V2&5|2T zSs7qcy)R`Pvj2kz70eidO_fRUx4y-HJm$EljDjn-*En$2iJc(&^P>b zABZovJ>c*np#GmvJ@95UtU~lZai{xDz)Q{TR~)zg literal 0 HcmV?d00001 diff --git a/docs/images/aggregation/init-agg-group.png b/docs/images/aggregation/init-agg-group.png new file mode 100644 index 0000000000000000000000000000000000000000..3713dc15ba0d32374d00fd1554aa6b2aececf420 GIT binary patch literal 158795 zcmeFYWmr`08a7O$Afc3igo<=`hjcT5bQwc8jO37lNGk%;-7s_wtsn~0F?34JfOOZl zc%Hqt&wjVx@AuC;$1$umYu$Zcah}(8^G-`ckpPbh4-E~CKv_vn8x0Mc6AcY>;T8_? zMEvIxI~v*@X&YHtEoE6*dM#HcOB;JI8k*9(czs+0o$uu7#vfv(tpl}k-^^j%WkJt< zGl-$Y6%-YVb?eDjX|2k~5qL0Db_-d3c~UWYa#9dGdw2^MIq648rG@@nZUF4O=eYm4 zujTr3?T3GVs#yIUG%BXX7cPXwx20js=J%-Ywck%k{Y($R#9ELh`h+f;n8;))uc(M- z9lX{y-9$Y|J@rg6Xy?-=Rw{CyDFPGPA#~sPLj`wcYVjKQy& z+#jzf0;@cI33dyGG{sAZn1qhe|02tiSCAo5<@?7_M&=99V>4Rn@jHVL zU$44het9SuBy8}Jgfi>r$Z$x zXG*!1RTmr&qshXaVvwF0$iNwd^)}3tQ-p-g?cMTYNs2mg_xP4&$qqY=JWaf=oH-}J zfno97g^#~cQ}YN@w9<1c@pzOP)2oE(Abp9~a64JQE|I!u)C|)JQT%-K&C@N+bRTc& z8m7|eLU!jOgghhkgy_^DlvXt#A^vZZhG?J^x_h)#Y_M-K#b}_HS~`lp7Xup0-FbpT zvgIs0Lvj4|@xlrS=$Gg>CxxB{*;IM(TWFGNCf?%X-KS(f zktgOBew04q92!v_!G>0~9kFzH$f$5NE4}V8-+1bewlKoa&Fy?3Mlp`n?niWTLU-Cf z$-7t}OBUd!=&7w781w~>b9>n|zC+ zi2+3xjGu5-q$v$>^nwp9NG9-J2T3fT72xV{>oHB^)XX}nNgc2cUa^*Q` z>-aTvn4ee`x_@vhM!u`8WM;=!?Pi#;FC}H`F0L>ty*sy7V!$*W0quMzOklMT?Tr0) z>9#OYM$o`lie=PpnkUU%ka-u~67)m}Bi=&JncnlR@SDmvcRuSry`7a>k>>o#8Anpp z^o@tyqxTv$553#1c#=NJr^=Y9%}CFviYsr@OEMnCDKTJv!2JG}RDS8hp7N7iNbYPd z<1m>v#YS*=ad=U9gi3`>C`X_lG3S&KZ)RtsY=jR6_IUJ^vWqNwuK>!sB&0!(@NlakPM{A*xO>dv6`{X zu?gGdv0Pi#(bt7(x=m$Ap(=SsI+sN(qrS*{wgQLzhv&IySX1_~ng6HOwpQsjUH|ec z$fYLEonZQ4``{IvL!4s*7g?V#_rDZ{QWFf^KO!8VT$P9vDs`U-eC`pc_1GuJAcs=H zIBF{LEJ{66HnQt0DpHgBIC)V0LcK~&FDaFKgC{TPEE!K-KS?l2(rTU8Ao9UiBL)1N zfgj)4N7ZcWx=Lp^s~e-dg1p|Jn(d#Hlj7&g!@F1`EF&aUb$WRc_&Ds?w+-HxebP(P zvnw?!n=_s^U^3*YfYtx18>(NNAe|bWn5oX2I-As}EURjwpkk)REixS}tSmYkO9`wX5qp8dPxtEWS~vuyLI0EGEm z^xWiJQ*)EzbR>D?oZwkyOd}D(Dnhe>>xw`#QA5*tHezLqSK}#((7)N`I_>Kj>v2*d(h8L4}l8J*^f5w+ux0%)OV0jr&+Fr2j-n;5bal_#=mHl0C22 z)K9|7;0C!FkuJL)Bn$L|^Y9RYs)ZO;xLR%->>=8?2W4ljBe#ptw#m`6bIpk%=2W214y|aEa zmNswpzL;CWZ;QTwY?WiG4ky-dLCoPUlrBlulTj;vy&kYZBT&QAZVwL|N{>e`9_HQT zJ&fF6fvu#BYmF-*++k4$(-r-$vsDoDaVuDx*w{Ke#P6JhMLQc~UqsMAQ8v`q=}7MC$?8irntI!2HkoS8+>mEd{(9 z5{~JkzQ+d>BM74xMpFZCGDJKCPc}F7^z24NGTgfRebS^{1HH1Yvd_EVg7tM@+L|u? z8fIHsetqc93++u$q!v@~NNrg1SQ#B_FR?0jftfbbdbKPoe2y{}>Ym?h?rIx@ime@$ zh+%t3&ZqgrIF}!^u3qlpDi9T4Rh++cD)3p`eH%zLP1_{_KmT-m(iPXtWDXC{(9TGZ z;)CkVZVf|AnHtvi_FV>=%)-ole{r1Zpayyp>$r$6SFZJq=O zP}8ed>sPqjuJceo^RteHFa0q;V#+c;NJ^ZGAL4KHPL;nlr)~@JZxV_Xk;m{(sT@aF~F+0AoaY97UcK~ZS?c?@h7tr4>ytk==8!7 zKGH4{mp&O!x|vB@F^bAI*hk>BRohaofoS5vp#X;v!3DL>rilNy^EJNMB6v{7z%@pi ztgo4gA?`XrDZ;@9%9iTtXsp2fEi^22Dl}~14juSPqEr9pUICpM4fC((7-(p3ZP2j( zRYn82-h5(#?@gV*ub3Y~(QtuZ_kgcwCdR)@V{>L={`($t0eFWdqa&-V3|w_yx`M%u zZq`ojEm7|(fCsmol?>g`(8!N&8F4{YQ#b%84&Wj7ymVB67{|GwVbPn^fqs#c((Nuw#t$>@5bZ_eU2 zla5Vx@8Ysi$$%oUE51Ix+eA;rCT|bAE0dG0%^!Z3Sb>bHZkvS^mX4&|SNamWj6S z-rce;#i~#s$OLIEwp@P~OPOr6dTr%lwLQBhmO-@nYx5)De9~;o{tvf)n+kaTeQb0r zVrjJh{lok)5C@A>b=_Kv?9T;uux^>SKB5mm!?^W7f26K)VcB`o`r!Ln}&Yftyr)@qx{ZReUB3GQ>Z z!#%ds5u)TAMnA`%BS(w8IJOask$n9Xb1gn=EzxYShR?L#>kp0V><#BogCbeWm5ZU& ztF_T8B24aPX0Io~b=$4yt1A`m8`{+3o)u)S8rEO(_9$s2ylZN$haPc7c3S!b3{V=$StKEyr1-Ymqj%eCV_C7lb$;mSz`I<@98o45!Uvy@ckhe1<{k^_ zL9e3qdaQIaVxC4TYOl#?RG|J;*+yd@rAG&8=w4DcJ!gy!we$RSInNta^H`|bo@t;HYfSsu^K$g7|Lzs!#|Bo z_`GfXA!EpQ;C9*_Wty0${qM9-A{h@`I7WP96oD5TBL!`UN{7j+ zJoR*O!?#b?wDIPrm9L`7_b4S6-;mh%r;3PBhf*;A^x!XQgOD|R5^=L^u&p#hG5%b9 zOGj%rXV15CULQZ6GJ5>F-f`BxGwkkUG19k}vMln0a!(oGoj_QQyNd};`qL$*59G5I!4Eb%2kEpP zp-M=;vvy3PpU}JMqavVhYqI_mtkadT1BK%Fr~RYY&Ip(({?!~O192u)X-Q0Ho}@?Z zTm8z@du=&+dufJB^wQmr-$kTppU!dq-Ka__g>)TQz<~0olu^nu3K5-B(~@VN-PJd zO3104*yXHk2U8LX@8IUa`Jvn^*wDP*Ax%!$em%_b(fT&>XElm=ZZ%e1${e!PmQI+oK_pMe{wbR5aQSZXxBrDWqb=9@Z zn;>l*R~~GI`e=lH?th%1@P|Q2(}M!T@l05ov@!jeEjFk`)Gkd=WmF_S7$&VZcJ}2wX4#hBA?K^m3eaB{`e1$9Sx%q?}Re3V}(k{ zakHz3pVu20BrBtH5hjoIApN$d`}SO#BwMf*=h*>N&woPX2P4alM9m$p8};CrT&7;m>;)gR%TIsya*d zZN6>h!itWj#It)ziCCOEZ$p^=7l{GLNH-%OIr?~g=nLw&%COo}&~5d{2WjTU_LXe; zh*I-5KiYo49hl!6n*AWGt&8y|)D`&XcHzcJn#Y_C67?Z}>Fzhvrq?%~+h@C2cJSA1 zlj4u>ul~3&#D%(nA9@-_trdtQ4q~ll3qg-l_-#G_zBpTZYm)2Fb;5|5liF9_+$OQ# zS?DOTwZ+QeG==Ay?PL9EXy{nQikS=;iAb~>tAX@7`$=BStAi2E`iHWix&t5!%|D4D zu{6DU<{V}s@??MK3sF4)ReZ?o{&of;WgnXVpy7ZT1|shj9=C(%TcHjBFiD4>)u!^i z#CiNqA0hH>?$IBWOf}OCGf^jBtO*RX^`n z#JGfsbEhH@6IX8yWKsVAKM9P)4p3)`fSqL=qk8r`&#A1%KWoAmn-*7K?@oaDmr#Og zV8`^gK^A{h(ZANl3?uPI)j&B4USy!Ul^;obBdlHce;zcmI(n3G&TyV;VzypIKW1W2 z-HiYBCE#1ZbJ0<>67Y2~LtPoY=s%%->G)fb)I$BCgfz%4hf3Tj*nT^hd%6k`uA_xu zBE9d=!9stWIkQus#X^-%frjAg&57(Stu$Ot0JYfeuMHf(E}Q^OW*1ey_~T%UwWQ06 zrt2J1ruG3iU>rftpI{G;vNy5WQCxU^!Jre#Ce19w({z!`=_yC|=Jy|dF{%inX(x!KYWf!Vk z2k4IO{7H)fvu|VJZcqx^e}Zh!NensMZBY9d7yri<0IU`Xqgq8rFs_f-beXXbHO~vV zY%z@mUXX@f=pT0(a*N%l=G)P{zrT}J1P@#|uoo#DGokv&$=sv370r1ys-tO$8ZRT6 z#tH20WH!9}N8>*Bgm`mpud0``8XyNuIMNHeJSuu`{yc))Sm%wKqeXF1G-850Xc&p_ zKvj?a*nhFA#l~>nc%6f>QvtP*qrKZ&zxthkcJPajrGMn3fk+bJ&5AGffZ3mmW||y$ z0!&UUapL748^1?#!wRCP1bV0hY+nHyTLlmuhAKy3Jls|H{;m@KxZ*_@-uW4-+#ya~ zkfk5TfbG9#9~;mPfcjE^$GpSlc$Sxq!&#t~^x}p|9c@id?<{^Hp|>FWTMBez0{l*V zjy5;8oZk}CsCn=R&HL^n(f`{fW)@?IQ#FYH3-!|O9Bhn`L^bFgf?lrurF+Fs7kqi0q4efzw^nZt_X>?5PS-~s>kPli-YtM27pv6 zCabLq%8px(=c;SRjo2Khxc;pjf5~H@3MsSgf-68PG?D<}1MU26>;nxjn9@D7`%*>t z9Oi&IPvTJ9#|lPK{Fj(ZKfxpAaBc)RpWSp_nbmj|ST#l5%k{}wrRLw|NYxhriK?YA z^L*%V$<=mIRY*Uv74Eu|H5yc0Fm8 zQM*+?FZqWl6vnlK|HTL5*SjVE)*mf?bp5X@0K;@_`USTes{0=PUq&rZ`!2K5g6q$3 z?*(l~nVqC?@7%Y*#3f{phvw7%J^YU$0d~&mxU`<_Q}s^fe4BAWA=7K3Du}ox_=&fADlqpvv3Q>jHg#HFgsUF9g)s{2#uK z6_jZm*eer^Guio$M56k9yKVjmz)LCz5iOcGO27E!?Q}PtzkP~*)w{#3X+*y$irsE7 zQ)Uyw{TEo2hTMMO(s}o2HN`1^i=Q&l=D~k;5Qs+(Om*)KiADv0;ym5QXd9ZBqs2Cn zCFL*()$HqOhx`a_yuZbrGccM)Y)WjqIqrxYYLf4=^IyP()d0{*ng=ryM-yLKPnIH# zjJ3yCo+O%R+li9wQ5aOd2q|khnyULsW%sk~UV43lE^~IXh$m^HDvpQf@`RdE^F4)3O5vbm&e{5LTJ$h%#Cvk%Z zL{b-ngyK8WYg7G_=k`tO*%7JRJUp%K$gz^G5DK%36sOinGw9*7^Y5B|FK}txV*&W& zRX3s`)@6Rz?NNaA5cX%NAS-u)B=v^i7xiuUuE5K$ckBQ}?1P?9E+4R8$6Q)U`rB3q zmSSLWR^KBN3_g3*wDTE1jSWH*1W0Jlv8?z-&xm(aR7ZNQk~;0+zgd6`qebB;Vqrt z#Lw4dNc&9yY`OE6lJ8#KD+37w1p+vkxx-#ye5X9%-u7*v@{J#6JQMLGrfZSNPyzB6Um>zhKJ zg^GCM2fo7k$l}#`SMk*kG){L_V~5)KmV;r{5E$dy%67}Kna|JnifZW94)wF+sHY#zUao$LC51i z36KHL43}$2Ym72NVBVXdHq}q|(3pz-#wZoV^g0*SK#8`^p>3?IJYm!uK$!bW*v0iHn)^+n5S*1!dJEyJeJN{)(BrG~ z?X_kPsuLwldJu6TexR{qW0zyYVraG_@}rJM<^1*4vHYxp^Gb;BQ@l$+JC`jr>$MH>|Ry{@V@a!q5o*0PTRccCI zj!K^XB)P#TEl|Wa6xBxsjh!p8WEe zB1~w<@1MRWd`lPoKf;sAjuqrDmgu`s#V!}2^ZH`}3Q+$tz!e#;mDEx4 z(}yUeYfCdAvR6;3vOII51bZN$RmDvDz_Akj=`IR$dw|!QPkDcQcs!DEHHxC+we55U z*pDL3RW zb6XA)^L~KfU15~D`;h$Wqe-iRInEkb5n24VxEQIciH~Dc>~9~qhDFgyNeX}WF$3me zhl}#WSaYBWt`_+ZIt>iJOYdyB^mgxP+Ns_S*2rkbVS4!yY+Cr)%Anb^@^E9M+Z{c< z2#D&;+I?s`|J7ub9Cb|0EP}4TQ1ze{fArOPwAe3h0<>jTcl3-Xr{Ieq!WF>r@qzV`O>FmLFJ`y>*3^kK*p99q_-t0b z9($%Qe{@dyOh4)ZmO{6pH3vO56+0M|9Vwp5Q6y>t{K!ath`H|P*0;w+_shxd>m5Hz zt?wEHr~PA`tiU+sVw#h~Y&+*~pt2sp@su5x)99fApZO z8`*(V%VBe)yI03;*A7&N`bl6`+k9B-9fYw{3%nR-nuRa$2HJeSG$91yV<_6JAFsPw zFOQ6M+@@*~X44wtyK+_}Ck#h6Z)sldoO#W7t);aD1_7v+dfIotkHaw|xmdOWcIFGK z{7R*)TegnUs>kRVCLjmrnR{>KbtqG2B%)faW{Td-xr3#dYHUkZ_XSyG$AkX^h_%uI zxS@K`AdfxIk5S5LLF<39Taz5;Vw3;gd92hd=dBehl0@?{YvNOoWk0ak>)_O)YY~0W zbsm+LEah_=vDXI*GOzo(Ymg+j`~uH?{DIK^6i@eq*vMeMJnx@y_>m0jd>%u_}g zLCwhSI^FXtwHN-?WiNrae*T?a3uWHH%PG5ZFPVLg`kCQ#*zj?@LT!2IXD?#{#U!ZQ ze|UBwQQ!iED|mmj`>wOS&SiMf?ZT3zeM}RdNbj75ZfK8r0GkOptCd{Hw@JFB0xOi2nN zh}R|k=689B*aC3MQI$b_OEG;ohC$v38{@F!y_Mb;X%kNx+t;Rvm$f%mYwq*_0L@-W zVN&Ol19*Fv1O8Vp;W9ebUYnKO_Vu%`+1v%lYA*nZM1hlWJya&wpyx*f8a?NYY|C2n zO~`23m>&B>zVDaa>VlJ<vt?>;WKV~e%24FtFFpt`7N69H!9Q_UB*f1ef-N|006yZ$u#bRGOrP)9+Lo0wZ^l+ z>6JX+eAzQxIu0D>IG%L!KcD_KtDhgQ(En^l1;}gYJB<}TV{|fAv*X`Qm+&Dcb$&O~ zuuO1hUR>JX%xB(R+OI51G0K&O5uFsyxtXt#h6F<_QLm)~v7?SkL7~UfPHm~l{b!I9 zz``}-D`P)#wed!AWpw65qiSqMny*yJ9uQJs`^UFx+=TxXYV0;*HHYiIp3G8axA!CYP;^iINQ4l_-l5u=17ym>eMj`2M>)Jzd%glvK6lWXzO?u|aSalTs8 z5?H``A>3!XMW{@hs*_2a!yy#UkH@P20oN_E+6Z`<_WtA{!hD|kxk(40S_EB4ab;x7gddoeqPf>)zjOscus(*V zJ>=Vn@(cmn(ZcZ;ISNr#?;k6UC&O;TZ|7xN0`bwbZN9s|@rf_?Et7=7N}s*&Ok11P z1Fy%Gk`j6>wT7hedi-vYw-k8ou~3n&04A@QnS79UaWHKt zaZ&`kmYOsU_x{idkfVrbv}|F1c@w^TpVL6=_4BgfE=A50M2?%LDVxXj?tyAh^k?8eS zEh2v`ixh~UV}c&X#sERJGT5ZqdNZOiNzTO(+us;(Hq;Rgrp^+dcVNyGvl(Wv8_rek zFVWgjpMT#-3eo5G+WTojzPrkCysy_*WZRKBpnLLFrvJVG{=xSAJnC3CsSK9AZHL_) zET8mqvCSp>?2?ePVEc~#wWe7;})$U;m^dw96l%XNHadq>YSNn{cy z?{)=x)1?huV)(@hQPAhdjKkw zKpv?bx&#mt(`u{K{0K{I^@BhVvCeg}fZeJ9_{S5mk+B~f&Cd<(e0m;$GIf$(>K+5KM%hvG~P+lyr%qUXBxi*#S@CAQ_hmY4Zn__83N31cF}#G1{b zU_ZwsC=rjDt%8Ex_KG&0+*fMaO~7ytoM>v>N4_bfgWm9kNJ@OqQpWi?bWT%rBX=RU zGoxhs4kY6)C;YAsYqz$HC)AI>*aIe?k@b#*iE((lfGx)yr57vmE_-#D!ra$o&d_@j z=?oABge@}NhEm`Q!b7LQE?%t{Yrg4pHN`IKGp%o1>{_1Z9P6}EVB11e#M7f8bG^w} z+rtI;?gG-|SueI!8*)#!&BZ0$D4#)~K|ghihv@g>?*ghpJ}-9cr2T0yH%qfq3EUJ?e&S zIi|*Ce0swtB9-(jR3Hh)+^T9Q9W70Y9A$EBsxie?fP-z*D1uAjDs_^BAKw+_BVn5g z0@GtW-`pd|=u(rVSG9jG@41!9_63zUEa6${l_t-lvV}}u?E`@e0dWzO9oUd{TEK+g zwz7ndPrt2v5xm?y&ZDK%seRiMTwINDXVK4r3%;KHt#nHbwKqTF@dbdy+ihwRI4LA0 zT!!FRT`Wme<24-Eg7E|eZdx+6J7EZyx-7;Gq!pjZ$0x;>ZOR{?B8vyQxcpChONHpF z#`Xbnxa&AfI=q&{TRDbss`xf=n6^9+H_(bYSCKK2K%Sdlw_fb2a`}kD85dS^a3g@$Al^A9=}oy@HVRDEyJZC^!f%dy8Ur zUH5tf4$78%XWGWK2tE+~eg}*=OZpv&*E$A}9BQt4oSYBabW}47hvOEx{(?4?k=2;p z7{up5&`*5)sNLLL7&VfSz5$vHf)i^qTEOHxVD^UTNy*UwA#}@M05@3cx?n3r7>?E_ zM%y$<34$w`+c#Nzlt6q(_UL?3cVHd-{7$X%)7qC)bq**9ekQ)GDFf>uHQ}3(`bAM9 z(pU(1*m{zXM)YSqlq^0ZCm<<#2R8!a3YX5;*;!j=fJ~fC&kYg8N^WEjEEZCoI7VtP z85=@s80wVG{Hdh%$-WJqIP}cjN*^Eg3mc%ji(sBL?Q1~Hhe${9DFpV_U%0!L3C+Wi zeC6yD!X2Wsxx++NS1d}e56AT1dVEJ6SI0)$BAnDmJ@VRieUE3&B`*r43xmRY81z*! z38={H#co3`7m=8v%>)qtYzvpvA2NqoMcZ89A*aTbg#Lln)gcdUUAMKJqvdAKynQ{W z+v;9RR)@E_rKo-j&|avD-=0W9zJW_gQ{|D?vqI3|?8%wT`P2mj@#qW3PN+ zV1P}pGkBXBq!9Id8nIAqOeoyu5!e_#22~UqI(ZuH zKoM?Geda1Fh%K%`KcEy4iPV?!JgZb@XFl`HX!)Wcij$&7p_!X`V~(-p}vi zJw-yXk))+}jysA2e|^S0whmT~ZiOAj4|`EVa+D75P=J+Cp&v`uJ&|HclPR3C93+Go ziV)4LeG{Qr_p24756w33QFTebV_x)L=%!8QvsNo3M9>BWAfUubq9%iXxN_ghIo`QT zWw%!0<5)Wv>;p^iFj?XEgeM6$3v`I1FC(LB2Ix9u;A&?2yoIFXA(ntb`-g4JVfpx9 zoCQB!`=xeU@M&TnD#XxkfpLrJWKEq}l_nUj4K=T`tDuN{EUv}By4hGF+0=&)l_`lGcG zR7015c1^;w3O&qPEL9+A^$0Vz;}lq&^a8w-9uC%u-1o)tAl>jMtOlv&9?o>8=pgL6 zgT{BPXOe@JLsncS*;S4Zaez&v2{u*5kCn-BK0#jS1ULfBrWgF|{r;*E0-zZ){OwAk zjqzpM1BLeD&$MPKe@@SaNM83u;*$$(%vt9|1;q`gufex%@U1+v{O}K3DC*}m!I`WP zygHx+|ABOgEg$55!2 z_pwO)OXv!^GrXKwWPeNeJd8=G`-q!t05-0yQA zg%8j#IbfV;ADDq%EcPm#_cy;+l%>3B{}6TZt7*GA^Bvp1W(OG#)})0&kp5<2x}W?#ogbOw}c;^!l#;NLm~tJ22`n z!-hU1b#^w*z20*2@3qUFvFIYb)p&U{9elHRBX10#VXPzLWLR3x$})`ch4wTw^J);4 z5{rcp-gXlx4Xch~N9O{Xj3`kK?+8Z80dK(QDUOG?4axdFihXSO zEJ?|iie<;?KEEn435DCw9kuH6TZMw;q0@aCQc^$!>6dL#sl_#rzB9I)QzOalPh{ZZ zbSo=ilV0DJP3>?dHNiiY-{2}&1ZT<5Hr(!~@PiH5pFHpag5D!%T7+D2sMfcML~Y|9 zururEo^Yap@_E78QXRU_lPdWzY0j1dZKr@-HPqnjG~a+YpKHRxvDr^Go_S`#(h5Bu z3sKS?B2bwXb{s7Pjk5P|fGz<=l`)(>`k>29! zw)@S)kiCGA=i* zZq(I(TBT`hf!qZL8V2XqJ$h5c%zG$Sw8hep9Wn0QmgZ*~6jR*9sb~7wmW_CgDmEu} zp%2LYv_y;-=G(aH?|@_gx(G##lIlYD8hIrzE+?SJvn}Yi52oTN4V0jl4(lMwTsXH! z{6ae`?3hno$J`x{;cbizH|i!U)gK@&8O*?3m=)kIe3B=M6Ws2XW-3E=EpZL{itOep zzSh!rHtA*kRQ|(U>gBk(Ig2>_EEk_P&KFgi+)vj&ksh?M_&^to#wKtFCrIC$5_3o6B2ZHzf|Dldd{ zC`QbVS7V7P7ZH~F$GMbZRUpq7xm?B=372*y1M9GA<*tt7X|R$!4Ky;{Y2=+ZK@IXt z^(q@G+U`+5bKR&{oiu=lxnQe5bHYye2&eA3dbTjI}J_lzK&b2|0^_vk>?14ZMELod`Xy zC-EC4xDBl<2hyH9#7TBCP#Mq9=Nd&L9{X!V?YIjTi?Wb}aLgb zfiFZ9*TNwMYiFXY&PDc9YJn)>X%Gd*qD(?ik_(7pq0!QWHLe!OU%h{(ok31@C}dCt zR)Y7CD7vy!NA0t{>a{8t8 zEd&~fykYd|GwzPHj}3M@eXNM7vAm!%?EQn+u%FIs7ZS6n1*lj`gWAS*Y^#)!-qVY^DQ7ZJ)SGGSr5%ZK( z%<86J<-j&?5!7UbF?gj`_}|_l2L5T{fIxoIqCttdsK72hZYMAM2ZV7C4oF&2y)8#{ zc!2*T#xAEvMaK>19LMf&29{DNz{ex1v)DpOn0X!XI8jlbUbUC)F=)-C^$DwW z7Cs2O^;tH?6xEN1zrxIWzG({@>BgE)gGPEPDjhLi*}bM(o6*tg5;*fpMVHa7hjSJu za-IY~(a7W!ZbLQtSARK7mr_^%LN*wM_lc!f&#n%7WlppJSOnQQ2`6n76@j*5FaON)l%W`*7^exPaw7VLV#<0eM@N0GskYf-)8cN zr#Y5wSzEmW$WBtF3v5ABp_;~-t0Aq5wOlR=cl0gv`#u8*W_ri=MN2f^{U%jJRHMkKDe-4TW5JpGs z)Okkzj-#If?CWuKC9e&?uu>hpYUffV<4sm-E>jSQzIoaD5tuC5!prh?;m%rdd-n2| zb1cMEWHqB*ftUC6lo=WZx+;Nv0}x#IrQv6{ICad^VfPCB z)suP!r!(1Mr7eZ~Vg?kZ_cL63SUQwz)7j8q(w+^>pYG9~ii&GOo5wSidnWJ_3L|qD z#3$2B3=92DO!tx1FiT_-21bv?ofz-36qd!+RHHW331o9(=zJ4ix1CHr##K$A@$ZJ;(?z3tFWy71!9MzW;qQ}+$0RhF) zF%qL+7H8%qw}E>Y-PjZM8T2g_a{?8y-?-l`Q7bIe0P-|fK3oPxXwv!kT9D>)zlURp zm-(#~{H#SK*JA#PwFIX2g;G_8+4v9vT>FWNtefyAC6_0pBPa7&s+uPFqUf}dmsG&D zw&WEO)k{_}C`wxTY*3JuZD>1Uw6?`0f8##N;BCa&^oUYJBRTeq+SWopv0#cVJn>(R zYw7)JFKD>R9?x`6m&XKQ_Y9A@4CtAlD?DeVONv^hwnvL}(hU6?v1TqIWapsC zfvH62OnJkvNeVd8I_1E@os_DR?C`Uk2xNALxdQs=4ngDs{j)1y@6GYDl$JAONr&>> zzI|Wf7yO2P%(3U~0qBc{I_0Lp>+8OU=Z@-qC)b)wqXI}DdL3u23_9?DWzWLMvD}I0 z`JoPDL$quP`+5>!in+Vu$c0Z71b@`3xWM*qXQ7_xbIGm%0nQ(`))|Pm&x9 zi;5aiUo+~;GSiKy?17|JNAb^tX5D_{XH)UDg&SuQ?gZq8$Gl`v@&iiuwEB*5d{|~q zn;De0dX{;p89A>XT#6lmkqzHf_`QYF^z6U^K!)u6I_TNKcduL%(?V9^EPEPL`i%^Q zCB4wk04H1q7lsI~2=tD~OIKLG*;B*3&HP0vr;c@B(@+Q=ri-TC*Sg)bEx9JI=zIyU zxMg`ECoq4M2s+9*`=Fy^4-RFO&JV~-P6$0`+E2VKi4&}kCsLOqz z0X5MMy2R?Jw!?AbYC@G&TUOZSg#9IF_oN4uj!jI2m2ZBY2^;f}I)A<LhWJ!yoJu0FW0JdZ_##LxB#?zUFUqx(%&AQEI-cq?DvgHo5)9uP`(c36Zr2Wm zHk`HY*XQA1`kueuM`U#u4n-c$Ep!C8NUD4ZBxxzgMuSDCRSoiD2aVO|VQ??H?E^_i7G10f8xgn_5|n_aB|j#V2!UhxfF?CXok-P2d4A}9ETXxieZF7Piw>;^H0S|x z_@RjjCQr!P?bctwVJ#00>;e{U^7s|n!fX=qlx_M6sfQ7bj62@!5Xtw~*WI}BB?w+6!~b~OZ!8VG6DEfo9J*AASe$JN8VT*{`) z<1l>Sl(BX!H@mJE=LKbI2=C`eivSs-;4InOxU&JH9+vrgog|tDz|oTy!_nhv$7Fao z@E<{9k1vdulZB99Gn0$UfkT%t`A`3dhjsxT`TBZ+Ek`!(vI!?rS6{(o>tV9m{eWRr?Q1!r!^NGfSbg+eWJEK6YzGZs94mM~B>$5u{OE3r0 z1R~JVmv?pb|FQR;;cz~G_-GIbHUd;tktkNB`~N z6&&X>CXx5fMn$jUN0YZuuyGRWhsMW4TCQ<$o6*Wlcbn45V0foO2z~eMqqQcgu2sCo zNiS>PQ&$Heowal;=&xPV@t;tyU#OpR#26}i^j9)xBr;dn?WAXgfOx2*O;od~*grNm zkz5KwL8I5_Vt~CyOdS7`FW;!tVjM$=N}0vh$$m>ya2iL|+3I|dryo5YR0rv}mw*z0sNE){ zUnef*kxhjA5K8hV$i2IBY~H}@M`MXr23b_C_pWHP5+X|E*$E#^ zBJ>pDbC^uLGEF?xJ1VtZT!1yWmE@~<<-=E%+?Ak|LtW7$OeWRjMb`WST#q~#tR}3N zAHrcZy}6ojLnPW`HHrErIR~JZa*jCdL9&y)2<`}h-_HQ$B=!H}UkPtmd z^dGAndME#)m88!tN3vljSCgfn@KfTBu{dVXOwsVz?XYr~;-!Q*wg!D~x-2^9)I@R_ zLY0QOyq$QLjf>efH{lDp&>BaKL?)9lVuv&`6@6DoF!T-`%2U<*9@HZynp<^5yEI2O zoGLD9zj}z*NFd&akDo+SV{*_q0;h9-E(aOPw%C*BEEytdQCX3uuxX-Nr|Q?fx>Cx{ zAYLA`Sg~mFVxLky>*VhW+@Drc#xIk~07>Pzk#-LDBe}M9o!AT1t^3=FNce(khU&M% zhGdY|b8205&J&Q*;p3}$W>xUdV z69+nkGbHR9a~7(|rlZdwNo;pE*ysY;xry}T>tW&YAsnh^tEbvI@cYndXsZ5th@3oO3i`w+}RWM zNrQPX#@(UH#=-C?pE%fnZbn@toI+r`Pm!2#YwvU#)oouRoT?%5yeGF8TDF;EFg~lZ zclF_NW^`?{6WeysmkUO|rJM{-8Cg*Cja7_6-iCYc_EIY&StrvRksmKj!!e2qBBdS= z*whvEQu7$^u0E8I&gPZ)#q$H3fxb$OEutaAJ~8cXgqGCgZJ3&OO9pF;D%M~fQdTZ_ zL<%pY4Mm;_`?Md;PsR*G>M3ijI;T# zvH;{k`c7R+VPl`hux4oR$JspORQ8)hmM8S&jhZfBKc@Ps-k~+1C%-8pV90%(i{?|? zHTj!Kao6W(uYa?-fe^2J(uzX=%7;7IJDv<3n<5G(h@3R!ky!I%=EDv>;upY+QxGnw!k@j{Dmh_ox*g|+p zZRKfn+HCJ_<758CPy{Dbx+PD3>&Hv>+uC*r`l~~gq-40 zYN1*0*|g$5k^{5uvxocEPYRx$)|h1Nb$ng4QmLCV5mCv}%1L3zhhUjP3Tb#xG}Wk{ z>W_cm5Rc7Xo#aQL)x_+vqSxzk7-%AWim>iUj2mMgQT?l%l~5npL;{CAR&sXu#vKRm z)?BU9LgO#x>sKHBJ?SAK)$j%|&AJNS{dKwOD5XsM`kSu^)LKm(s0mMqxuWJh+C&sP z7V*`UD-)m~b%~<(E%iz!|im8fop8xJEUJWNUy$B+j9!#I*hp0Gfb4M&XphDGe z!eFOrZ87;sdQ>gg9u>ZR`R=~;=?vfDRDvR;oP5Yol+5np22+x(^1v=~#TM*MuUl>~ zhJDOLC78S|k-f~-bcCQ)x?h?EN!tZeB3k@7M>=?2UTH1_>JYZN=V;V+7aPQ;-oi9` z^C&fB$cmecu!on>`fRnv>YaKL!bHuo#oVgSwkN@BRDyg)CDvh<3FN}s(gwQ;%sKwN z$PvzoWrpuM)4U;BLuN5Eu-==^xd4f1)buV6id#D&*(QnTx?TQyfAb?q_4b{;h1ja@ z4;A>dY>2fRD;*v1;;p70eLcbt;KmaW2mHJg1-O5I?K~^qRmxc++^r~0C=Dx;^)9O1_h~6*9GClBoMwYJpVpuIOhDboUF7@(>}cdl zMuH6^gteTv1)#vDP@7y6w%*{3-pEz|(RR9@(A`OhF})#;v+wr5ol3COjxrqdeO^Te^6X-1lStDPvm0pVtg?xP}I zKz~e)wPwkfOVKdTVfH7;h^#d1CRy2$>VKsA6*K*s;c?kZxmR zNm&1!lJ^y4nao^tr`UyNP`KJ{YI&dvN?+Bcn|4lG&hwHez<(%lows!silcV8W zkUz$8XQ%@*0ZY0c>4Hb@?Ketr*56;0ji?lULAqx>ZeskY9Qa>5oj75-A#`1C1C}k*g)eT3DVmetuBM=8)7K71HVoq^zIG^Ledc%h5vgPKFxG8@ zF^&1b+Z5kJr%QEc|JE*ykzAv@Ewm@*wrum#omwwb(t*xKDGkL6DZQ4(vXDr|BalIE zM2;lTQ0#kGeh#z9(9fYHQz8PD-_n4+|$y4e+~7$l2DRNIS8^aqU`9wY&2QI#o{Uj2`VgDWz5D zrW2Vc?V`@Lh5`V_rPI!fmb5m?WAMr2SpSsYv$fO~^6{FPG9!JCdI0-$ltid4L+)fu)DyNck|tx5?LLcf#F?@?K1U4N8E&UJr-biP_e${nfP%Vk}g$-s;x{n%Hih811Y83P!R ze(${@&x6>`@8MC{(p8fL0VXEm2_bR_8a%hqg%rMuv>r=|T z5dB-A-IpTG&8X4i*H(GkR3CRnMg1Yfkm%J8jW7tb0^Xl?&z)VNFTSeN*Di_pS|89x zi>_d25Ll1|#SJ(o+^Imp@Zl%%Hlk>S4?uYz_5#! zN=G;gZ1WRaPpibIlWGvPfxQNEP~NBMaQe`(&u?g80y)0+?wd?j)^*&1vTxxlu3BMR zXE=7rehJh<-j5FNBhxNg2W595r~KX%l(2s*(K2F}4_KD$d+D|4sM>H${&7&-L0vR-c(<(V~I!1B`Z4D2+w{nQ%JIX z)@J0n1hMuX&o^x-E(5iSrx>_Rq=$>;IV(QzV3wls@^PGy(PzsP4}*4iw;5Bb3VqA{ z7v1s86Z#dN0Vo64XB=?Sswi(_#QLCFh`VAl)(Y%Ix#pOp^0H8$Y}GR8s10bt!v|df z;w&wA&#O~pZqOK1&rRC6f_Md>hc1NwdZ6Uq>MgO=x?=(FtLMP(63f)zlG{Iq@R~8$ z(w*@{Ic08xk~#}FfGnHlmD+AmwppM5T8i7>9RVnbc}M}<2RaT=tX}Lm)kMfD5}(D? z0o1m6J5EZqUS0rgXkW8|>JXaXuc3^<<50EW5#QV&8IBQTY3G%K9mQTe_K=?>whJI? z3W3|>wiW$fSSx-w30~lQEWE4WdU|T^sZWON;sC*53#tSJA+4c$_VEA^AZY&Ojb3lV z89)vohDSo>n5aFf%_$vj#bZ^t5_Z4{o=jkfFjnATIA=jYn)zBn>UV2iMsTj=;J2|d zAtfG(_`ApT`!^L9V5{1urK$Bgn0&VCeSW+$el!%koeLezHyx<)+|+rc94*)Z5JG35 zMhFEwc#5_h6h!(h+NNH7bNUeg6~@j%Msq3D*uS4RaGQU$KQt6i(j|JnF(+u}n1=B? z>!?`eND0qUTTs(qK#Of)+BX3ZHCxnn0AC!?Nj%1akoCuTzq>~_o>rRF`xkHa8+h&S zZT|;^i&xT_%L&QCSDOW(1ML=_F}e0ZIiGf39UhE} zFMy?&9+lMXrTP9JLrD@ER0r5GYyi{408kLWCD&9FkK>?&Np`XF=+=|&K4F|)aa3lf-AWB^j>h@QE zCh>q2gqlMIz~*{vhuOEzeI5taHNcwJXf(3#3j%<>IE048XtIznJSq`t z!Yms=nYk*STZitFvXA%1M8c4LEm8bdr`>pF|HF0Tcl{*Lzd+(`P)=Uf#sweU2e=bN zW(}`lzcxfIVEWpdimHDp1)Pl2N5mc!N3ZiB%tu7e*0qIqy$pl1)y5;!R~S1yeAp}Er(je*lj_i&$d%K_@B80(k{IL3|#iO zS=WD!J-)e0Ay5Q)Ot%Wp|J~jGbTgUSz~^o-dSd>bXdiytchq3&3ZDhO{5Kgw4d^>- zbFgUcKl+FFkD=LkHId-g_&4bS65n@CbDYHgJhY!&_^yFmLor}VOUx)) z{;5rmrBg^0K5*BAqg_kGN|ypae?wEP56bc|0Eb9sAZR~63Mzk99q#BJ{f|fbX-qXj z@DvXB%B=F@ZheaTn-LsXTD)xe$#?#H)$q7R0PA$Sn||`&-JdLh-SKD?p+pY=JId+Fb_gaaU9j^;6w zt!D~|I>HDMm(rR%V@oNii!i!Yk-hk(DD22KP*q0R{y5t zdMrJ2B2;lwZ5^=m6bo|88R+R>kXD zpzlUTJ=cG`%Qd{atZjZB^`D#$Kd~wW1bBLynYIYe-7>sC+AOL^Myajpf%p0AQV#v5 zQBs#66JP`A6#g*A)YAX~qom{g+!VrOA1mfo3bLfL2C7M!1t^RIscy@ea+YpY|{?CSC+H_7}=NJ z;Ap4RY3BYefM-_w&QxjlftU%YZUXo^c(M3x3cyTcQQ9TBI|vYkrb~dyqszL$_TVvo zym%vNK^%m{TjqNw@K7&+7PSVD_Pf?$*~CB;zFh0hH@$`D0Kqd*OnFtm_&ec`!CN^o z;wMs*r90193KVih<>-~Pj-UaSzoB%{#%v@-lMg!j&{nKkonhnwIyl?Ch-JVbwj@*G z+1Q;Uem4ng^|u3V7_EhQueZRLGT7{g3uh*VM+upRSmSS$F!`>--3HbPK7dRc`Z3zf z{qMj@X~4kii6ad0!PF-LJev!?zGr-zMUj5wMe`CQV3&zLE*PJLNvj^ur4wmk^jgm= zbs6%l-dF(~rWinqLJQG9#OIQ*hY#*>JmyqjkpRmPc#%T;+U+KK{)Z+t$|8fDjCbpJ zY0mRqd<^K5v<)_&jKOd=_BCKu3IU4~-w)gTn}3J5Mhk|w-~2`bF9n69$N6sH9*xks z*42(!8V1tgAMk)w96L&|C^T{ih|}^t-x_UUrQSf5h5XF@jhYz1Pr$YWhZ%0 z$!g(a!_ZV|+&*^c1#4V>a5R6Rc05+mO6#2-n4Eqxl+WY}lnAvw@LB?+gD33m_dIf- zu%TH8Iu6n}&q+UG(s2rc2*NWUCjgnb3!&?%qntjV3&Li)u~sGEWK#51t%23lypiy= zsd-dlj9$A8-(vJI~eXzTN$zbnud4oIr4TLb!LIfrY+u1%Gxo z1d>Kew=vf!tk=Ya(|YV#_&TqbmsfkT-LerwzQuuy;36wD712D~YhLI9<=#uY*DJ5h z#>eF8=llRvf81y58*%i`>XF@{(rB%ApMuL^jFAK@KC2ZlNxM*?gu&evUzJF)7NdjF zP~%7QiHf`G=V-1@;!N7(ou3LASmq>awRzxGDUrmhY>gP&zwBf`b}vX;H}w7?4nMyj zf_+%J#r^O0l|)(4`%O32Sn$qAT9f8$4M^kqo{{CfitIHD z`QwTGc4$}gZfU;1vY&d{K^NB?N{_d9cTTPPas#O4(}*xY9CZrVL7Jmsng?qDyEbY3 z%=$e*`JIeY#J|Z53m7->@nx6gyiWm^TY`vwPt?#KpPNp86ep_T3FrWMU$c4QInd7D zsl(L+6LNN3RCggsdB4Ms5glV&22-!vOH9h%4rl0ZaUCKd*Z{>-g|*W|oPYf3o1n+O z7cxu0UUO4u34Wl|Y`-4d)v)`ORo)heVA}Nw_)5LtJnD4@LOg@hkE?d4iA8)TiuUue z*WIj_WA6VOYowpXkF836vhY5oXcgZ0PhSRIhB{5I6{PLk;f!{2F1YnA4?GNjp@%wN z8qU6Gu^)7Ydo!2#&aI1l>6A)SN5U$^w`Zl;mwLM*MxvmC`4-Xk=FprH*$R!CWaQB9 zXw_21g1slqV`k`;?Ryp}wv3&tnsmS;-YaDMV^R}*j_)n1*v8gDu;-7JzQkLgr;l*~ zstCZz6sYKFSGzQv`m|)!5q&p{t*gDvm}39^=Kj}bB_*ne_2)5VwG*HwPZP5O!4;%3 z*sEA-(ydn!FMW-w%Qf~9O3&Jmg!tkhEs~tuTN4=6EBlckkrfZz*>du5S5G^$hFp|g zM`^;4XZO_+bZ=)xn^DX*m3`^rv-6hCxCdP}uR>>bK@hc;uWI#YxXf4a0zlx`c9u-} zr_cXHuummx2zZDm#a6$VVt%B_*B}+*T|U}YYzKbJbkE}|w@Fm=`kptAF3pJ^N!r|R z6)|R0IPW;p>ng8k{N61{cu4fZXyLr(j$I>F>E=akUG8}yoB1C`7S>;d7|o91w?8*& zeM}kl%wwYzS_M#Jn^I+j_P#dn<0CEes`pI?1omvI6m5~8kLu2u1>#QZ6;238Z?x`? zKanS9GEKu3HsHpy6D*H&vd^0ovi~%bKsCeg%HtYW2hofpK=A`9JE;r=Grg^D#^2ZL zf!unzh-ST1$dcI-W>j!PbnD6o8P`c4vEeO!55ft~j5+O)N3Bicd=0MSj%1!lzx_LT zBQ;c=1#P+R;6SgBOAM_dwvv78`a8cm$Q?#4SJK~+YUbSp<#T&@P!w#?=NcA!c<+vFy4HOY`Vcir+=iXn z8J6_4ITNG7)rCVZ-G}ymdl@vM)eGQRg9Ca?#ZPU^zwMOndq78gYq8^Ybl4!o9*|{O z$+_q;Wq5}}?c9NKN3&=#H@jKA)t^QJ#4aGi z>r*;EK1JiX$i(Ei%eOqMzBGw-b6OrzWgWLhkMDc|B*OBq3(iz6Eyd)WWudHsM2!h9 zyXRdG8AWGB?Tbaz=I9|N`1|dca5r-{q_*oS=yu6-N~GVifJ4>(QTRlLR9OJkARBLf zm36^o?dQZ(gPY&=Z{5>lim(182imbU~>k4iTs_eRG(=FWIJS?|G9>?%{! zF1QbJLW@wABsa~?=C!6+$LJ(o$Q!M;p3k!QY8rPwcK5yW&1aES11xIpQ&mG+ zghYrF22CP6?Falu8X@@2Q;`7_8=%LIl=dd%#MT9r8+#xLprwtnCDZ+^@v=;tZDM&w z3&BcDa90T*95$r%!AuC|{RW~_#AJPRy=*BX!#-7($1P_f-gV-GQ+dN z1udofpyz%TA7vp~=Ta!f=QJ0vOJli=JZvS`1t_Q}UCu%x^%5xKN6~fXtM-O*l#DJ) z@d2nk=TSyd72mxDUU1ok@>2XOc&3QYL^Q7;0$Tps7H+F<8NP&=$S!8Ei+T_(I9Asw zij~%f<_Hun5A{GMhRFEFODoV3k94cyblv??_S+L^r4GPXu`{gtUY)L1?<0{O(tz*g z-cSuyA#A{i&xxVczh>dJrOhy!tY=m=VR?4ZL)N1H`rX`GaYSeSaxwKlb+-?Qufx8z zlo&DTpJ6_~zdO21h1kLI&QN&P&Z1XaEZb1c^wBqbh6nW!$C18{TXv;bK}D!7yYRO$ zD#J}trOqguW@1L4tNr}C zD_QPqj8%3JZDZH$P4RMtt?lADn!(6`WeR1bnt{qtnY7fBk$!U#%9^ctJ-@GV@q5t} zmFE6s3M=n7izZ@yA6dKOUJz&^agMJ#=n;D3@vKPIA?O%X7Kv$lTtr!79PfVj`tziP zqEw?$0r}M;-y3C+Hh5M>L!eQ2yRmVM9OR16`huE%iM3`&9VY{Xe3tN71Z~X75ShmI z$X2<>%LSN7FId`XN(e*%rm%h=v%24>GE2bT7KXgZ1A^QXhPm@=QzL;b4 zY{|zOF(xcB<76Fz5u6s1gBVz@Dqa}bW%L-bD|f%&G*vLdc_X{4Wml}~=GHnI;%csl zEBPjqZI>|I7E@}BDfKiPQ$4F-t-&? z>FJQn1?Tl;Vhdtc&n{{b`1Z~jaK6igdeSSG2GJde+QzY#mtO2eb9rC0e%rU6R9RfY zkhk>KBZpO@^PCwzT(Tba^u>CcbL;D7$i=%v(;7&uKaRC+4^B%CCzGt$-5(O1u4Jvg zS7B~Y%VYlnYcYR0<7kopVvxu9W*>Ixld;>qHvfTbSTmEc!i_|;v;Kzs_MO?xmo zZ7$VYYh&MRb0e>UF8`i&q_Fayb6ilHu-+kKSPP6{t5g%}A~1#-_&BBiO4VkCTiWM% zAD1`S#aA-|M}w&`FvZ6!=%^rR$n-WHEb2!S)MHTfS&8C}BvvlmMnlxbs3!Z#!UjyV zyZ5}e5#R7yw)XgD_}9k*?^*Uld8H8*BiHPY#7zQ*+;JTz zDTsn;IQM%x=Y6&Hj6;W&<^uGGm;&R^(q*qLCMuNkS!0}cP;S_J7}QjN^$z=*Jn^;4->?QW4wuhlLtRJZXS z*Ps*#LpyDTOqgL4QSI$g!OIs)TKm=;nXn_2!rue?7AMaTPKpg2XY9|LWfzDqlR}sa z5WiB8HS+e4c#)}q^roy4h2$nA@1#%=VX39#*V?_RtUs$~nEYT0#P(F4AHH5boZaOu zMi8N!@!_CGBb^lFhnkKbhQL;2x~yL%r9mcFy&tXAL*>5pj}h`Gh*FA2V-S-0XLf@W zb8h;@n`$$zY&@IabUR1>{bAE9V%HK$0o@0=iIZ`>E1ŨOziD&FemM)7zD`*rI z)!)ylchaxaNLI7Rqr;KicmbIiMka zzB_7r&C_))(k*osCLvPE!YrQ2!kx|1%QshTN$fJn)9e(_;<~o6N6^Ne z6lVFD_{&V&sjUE+m-kOr5nSqtp1DPFR3p^rwoK;JOV3HtPb8jAx#O*QyN;me)$>r5 zP{vE?Bu}2}+$Fe1D*Lk5Rr>6kwGO^&zIHo7nA`gsxC+jpF*eK2?GEcNOKi6Gg7GaO zQWrwSf>)KF^?8y8#Lmp`y}d!-A1Y>-bE!Ye;H~ZB#cHi%`K&};5n(SfFm~Qr>PCx~ zRyYPxg~rO|+fC3bvtI)fP2)-^_-8K%3(a0F5nPfHUVUNF^hwl`CeM*7s3gLsy5LQy zorl6)q>GQS%Y1F;Q!8&=AyPN{)`org5=uO5sN+SYOTYGMvpr8UTjN|NB#(+4W(PKa zsO;>`@{wDq^}QK*D{kM?4-*;ENxToiqsCM5k?l1nd*R_6395B4*vdnsRzP(=R@84@-AV68y}87D}dHn_v`0%*xB8(JeN7=ws`(VL!4cT zeMg7)(aW*=Xc{wWD~e+<&ere?Ms6n%{2X3|%9k^YHtug@L*O%6eKi(bjcq~wMX>Vf zk{sRhm#c#BXl09>%fZrrafQRYYNciZ`u)h#Jh76lj} zo0&zq2(z)}EOuKxta0u$Vtt5Co?Ni$_}FeIH$m@8jv<{Xor zmt|hhx`9`jCYYT*)7jxEH~nGB-SczRDZ}qIJVW)Ocl(YE?8{uHjh5hJ7&p>1q8}d^ zwhjF3Rw_9X;I!NqcUrTm+Iw_c_Lkb*{WW@yuJaT9?k}e(R)MfjD6`X+e0eu$i5ckQ!vDN>%t#AZsT@ZG%&6{ZLo9m zE$@K`?mop_P9AG{PagjXt=}Fcr@$B8)$t)CJI=6{?*Yztez-R|^k3|%v5c=-AFEaC z)%h`4u|U8NqlZtALTbxsid!7&3U1LV(pj!MxKN@?C>P(Ln}gghz+jNgumo6Koo7&g zD6F>*HDBPn)U9$%Y{}0(`fHWKGg~e_Ooe;gGGg5M)G0dc#sTJ%44x-rLs@z7Q=)O= z?s^yoiT-zG%dT{@HZ)sssW<@|qTz}~c%j3BE|fxj!Yieynb2Z%jwMKFw#&0ZK!>l5OAxjpmct`=f?0y$SxIK{ihSC$wI^HXgj%6Q2e5Kbj-Gsn%& z6{fl0t`Zk*fXsUnl2)^gc%3?%oSG%rDx$IXq_?(S<_jap>a zKpXdb)H+DQBy5*WeYGT>#j=kqil?vCpdSWa+L)k-g%(?_NRM*#u2%7! z=4)FC+p=%oeJ`BY$#>l7wV=Ac=xmW!`Es-Rjy~$mSNAx~wO0axjAd23s2bQuYD0%c z?_P(sF*b=5t&N`9k6fioVV=Pi)H#Z`@~%^Jl}SeBRj!qXhgqzfw=bolDJhu-??)9& zFnDB|_C(1rdcLyW(uQxhaUNaRTcD~cZlqUQ&fr{L8Ze4gZ<1gxD;4ImPt?n(kP?j5 z{xP2Wp7%++C}O&>0xM3r;aZvv=A;63&o>M8&Orxv$_>wDxOEurtn@dx5erkj z%H6pzHVtu}FgK%ij)nI~low;F4CXQQ-f_dq2+D9F7Vm3C!KOQv5Spk;>i2UY$0OWR zrw&d{n6H3!xV2W$a}BJ6^c8r3{ALA~aS@hAccfB^%&>0i>rvZ5KJ8!vI~SlUh9GDZ@OL*X6X-EXJFic|oYWL?g>7 zrFg~FVz4^!Ih3aHd8d7*%#@T|NZZ#W*7iFe_IJ@2x2Gm2)(~%#);1?_F9$Lnz9p%c z+)Hc{k4{PcR1$ITDU`Dy^V?dc4>m3?dVxUtGSN7MGk4nQ?T?G&^FPG8=eM(>G2sgQ zfsHuW^aq~4ZS~$!ddA$NsMw}Ap0V(vc1_x-Iep*xP=cZ$>gpUt+Iq*%c&nu@l;(|G zxIpJVa5czOx8k*q2)8-aD^$q!{d%k=7jI z0;AkcRGbpnt|OLol^?opg>)3t78S;=s5j6IU4)l(xyN!%oH5u@dtZ29=O$@^o!hs( zONRd>1n;ttCoQvBlj~Pi(f3%t>x@O;rwc-oOKxxYI zdjxBqGo4^IdQXua{kS6kg|p`J$fBlTgFJ;>wqir_g0o;$O{2B3FD&rei0Eh6)@)_o z;~xh00z0<}b9njm3wG~a7T0_5bZbjVzN9?`2g9W6zte)}HEtY{%C>q%DCKEgw$2 zz+9xbU*(#8Il@rulTN*uAxr1hpyaGqk}34lCsy%+F7KxfOD7f#`m5YqPJGmd!7XJ2gVye&5?v z3#rOWv@@J~rR^_od`*q$&q2aTQG(+UAJH6E_)62sOK&FXQ+5-t^^VWKGyQfpXA&{7 zuN_t9JC>d0>9A1C{Zc;QJQF>AmOeCHw|cN5vEk#0Tl~Y#)4jd?*Lv!@kzY3#<|Y_k zEyU0vFKF)4yI;-q4vcg47z=4i6q*{>TP(b^@p3!lMed%s)vSj^<7n}?{KOJ#<4X&J z;88j+E0y(r)`sa-LIKbzMR3REhGj4{gPPJS)~u*CHUIn<QpBU;E_N$TN(-55fS zj<6Wq3@`Yg*c)i+S+_j8@qS&v10oX;(L>f> z28Ff-J79WV$$yY|C?6)O%o!x!HGfopnA}dlYk6jq*L=V-{aygTF0uotv;%j?SfAF{m(Vb12LAI3bqj+oZEe%z^wPlKsIVWh|}J;4AqhGsB>Z4Y%U0BTdUIz5S;Y zYb=rskSM#9nq#n|u~=EE`?ffHSuxndn$EQJ`dQzm+(P-0oo{?}TG#=j*YR|S#p`p# zK0E5K&!J_ET&q3u z1}&sFw}JG?I+Y+$BsrW%KT07aUgDNJ0G|ukFu#kVnRUECI-WSS$XR0_j~dbOaD(5F zw_oZgXT{QPPj$gl)Sk9=Jl2|HH?-rpB7odD9${&s{kb7ppeZJby3N>{>FZ9r@ml0i zlA3lQBEcv)YFxL0CFSYHf-H#(yqi{UMBR`L zP2!2UH3}s+=*OGka7{r53GK0GW6*0pA*ZI@>}and!mEUDZFk0Xd)QgLl7WjzG6VNt(KuPAtxVdf=kfv zhRl3kV6r^-5|wctI#J_>jj!gf4yN&R5$ya~7CPG$n>X7$KhR_jUHOj7Vjf(WJrgd&hDh&bt<_3?aKyvNy_d z7#Ccl$-NRYyyGOJjeU0IF9kh)Rm*#G_<9)dn$0d4{}ScSdImsCDPic@pIMdm4Lw91 zQ4I@mSr+M^;kWh0)dsg4`HONT`zT`=w{A)DrJn3&%o-Sd; z^UGY5zg*|HeUH%!U6~5!c&>>o&m_+#3lcS$&ZDwU*o+^g>QHAx#>G^k{$b8B{^@Ge zyVJTvKRm|2LyW(O{b*U!$rBS-I{I;L>%ryn^{1C0Kc^S}gOJq(%a|!ZGC#?COhl}I(o`ojE@=u(*K>#<+*o`{qiQ9X^p9W2(%n<+t~8 zed@ls=(VAH%ZOqUz723_b{x}v*{X`$Qc2p5S`v@FVJmyro=x&fg;o&%aT}9TM;A9$ zYHmprN9Qb6g?7O>W zaq@;JR*lk&jw>zIS1o4+yexZeRZyD7HuRwjzbMLx`9QL71HK5GL1Hi%Z&o5 zY;3)_UO`~=&3o6R9TwyaULF*MugTQyNNCvKafqav9K~NUMxu3ekyti?D54%o&#|$* z)qU1^Up{BWaxw6Ov+->4vS?5_@4oX%2uuTq4*or?fA(!&`neE zhV+-wNIMw^i?cQ^Dd9Rk$UDls=X(5y3n0inGq-jhiC%Jc(v^-k+sN&I>fmAoD0^y| z%>%WG`Mo&VPXFw<|QrW}RPMXw(b2Dksz$bj-P&f7!zudzdUSm`T>Cy3U`aXm_F7vtMnZE%S8f-<&jo_ zj2`<*gUvC{_$;kjHLcE-xxx@Ck}Y|Y$<@ZwaF?vxvPR5oY?dB-JBvo4e^E@6OyIq7 zeX8d(yg6$8&I=R!}$>$F>_TXk~;A(A3;rw42!tTowNkxxn_x#9`_ioo6> zje7E=4&W2b2eNh5c3RceT3U}@{K`WAO?PtaX?)bt**Y^`w~3_&3k&pT_vS(z-Rt}%n-GMs!*|}xBbDKKo(uS=HY+Er?`QfL9V$ZB9$V` zQ@9t+X=u1dXg1z%?}oc1dfN*{wv=+eu}vbr-RSrpv&us1N2j%EHarbdUv8|U44c&G zyGwuH)r$jI4>OgQ_~O%>x^;R9bKf{OAPzQY6}#)UyYY3DX7#aIZ3Z!EaqsBqQ-Gpb zYgOIv6iGXoN-`Z?hfdONT^C_UdGsT((3zDCJYy$&~3 z=Ci4mW;`wg>KEyD{CUK^Obgjg(Yeyo{;o9w1NI{a`4!#WL42LwlU!+VJ;@BX zPe*`3hqtiYr94eAEBM*+R4Rnzc+BHm##)Dkx!IfIdLkZrM5C7zZyuxp(kD(D5cN$Z z;EQk{;|Kxj^D-d*eh2O#i4=PETv(11Ui`#BJ1HoXL)9QkD1%Mg(D7p^o8kSJ)NCjm zsUtBk^lW}f&B|YDvA$pY;`c+TPq10sjV$zr?`=jjG~G0{#8vfXLev4iGJ{pw>|mlN zz)`}^c<5=o#w474{^A7>L3^0f`g}n`I#(FDmUEK!no)InvCUTnqYG${RiQwdpYHLO zk2HKd*|YL_6iEF^5*t5otIrouaeGC?z39BKgY;`ADNX?6%^Typo)1-C0%CfS@Y1DA z)1#kGsU<%=#pc~leOfaux7f}*qz}N3PG-r+9*pWZfd=qv(oE$22a53bf1sxrkRvs0 zFZ8itxRKf#3zo3sO6Xh<-qGsWx4B!gdpS#vA6%@U-B4fhUR z4F&#p3z%-E2Wn?QJJ7hE1$@&+IcVpgv=mr$Ng#jU(Aid?QU>3K?l=i2-<$knhex<} z?YEBd&voa4OT!uEMLg;1XLQNNf7R8VbOn8wZho=i{?*6Nj(pC{4;7`+8HBT`%L>|! z&Tn|{(HP(%H<*VOCPZHT8+9C~m)$4UNr2=`@mw_lU|=?)MEgE;=BzdSwC zbvW?FeW6*(Uypwh3{2dhxHUb#6!cfuj~`C}DZwhWQ~X*f65s{fR^11R58c^%)E6)Q z`XAPQ**KERpb59fAV2k?$&2Z`aw;yKNK7$phEHwg~+|J_6iKNio~;=@gR@lE8zoqjg~QrO1rHP4R^ zDF4X56u!xc@CUz}$l=H0g)BbY1V;jzG>57EZX%B#ix-NY{!lM)lD^*(6&@QGx4a=i z8Y?K6C*P9%xVV@fcoSI`xuHK19N`UV2igMl+>`yGX~>+K{$ z)9s_=|7hamS>|#V>s7`>!#+-Mug<-*#jeHmwpXo(pjPL#i1t5Z@R#cw#hqzjr?g!uYh@y_Mq=Oj{$9zD8_%i z`bi7$mno*s>Y&ma=%5TRLv**PUS<9=4wFz|I*ebbF&-!cUQO=`m4@P`XmrJ zqHnj7S`Q89$qO*z58@A_ej$4v!`yhWk2bYAUS!GCVk=8h*t}w_4J~yu&~@$ zP*4cJI>%BT1E4<|c}9kbNl8TYD{;6myXn5~rA-Hw8@Zr}>bb>Y?0IkI3-Zf#ovHEQw{yQI!PxNK$s#7vaaxe(myaCS4 zBim3Zpe-hJ<_8h#ko5t}v|oi)?Ck7J7Sq?R^=5!GrU)pQUO@}!y+{d)4Rm;79N^4(dU*x5W_Cyr zv0k07-*%D65DpXs&?y?jwuu2g7KM6WW-J6qr)a~wI&_VkE}+1L(An`G87-|O_>Xp! zxok2QkkjTR{F(xv2m@n5r__P+?-}K)#y|+0l%$9V3L@?5>XIF^xgwUhz@Rmf+qaSz z=W};pkO>Ls>+9d_jd=TyiFD?Aeg{^@1V<#h_fl=UNML5m!WjdqyhtQVUqYw?T*xDS zZ+{#*ZGnUJF}{hsKXjAdd#3>QO*^6Xns3=iDgx7Kz3O(y_t>g7dNfQwh zzp~w1CIfwRLqN@;p#?4A2X9QMVo@oQ0%tRjfZcX*VdwV0$_cGe1XxTuoc$|B$rTI( zYIOwFenM^WK*ozV9Lk+90V9cOZ_717`Oi8HLdcRcxf@XQ84~ixYI{GtJv{>k2j%n7 z#A^}?zKcss2T{sr6B#qjkpH_q2>)YgCJ^Ei5(NK|?mP+c8ZGS|_2WdWpi6~$buwbf zf!<0L54OBqbbnNsb&8K=@F5BrTv4cfGg#Nh2YNf&FQE8doo*(xT%Ia8DmA!opuSL+ zHx3<>=pbKwvViufd{NM`1Et$_>#Z?Ed-~#oo$5NMoab zprBwAni8Wyzli1PATcOL5)7&|-BvnF`47_^HWTQ^Ne(=Sfd+=fC4sD0 zUOS~igMoG=2H~Km#3?B$uez|v{~D#Jv|kb#<4y1^A^u8Z%>frAnVN25eHYe`)d{{ zS)~s;0K4x>Koev15x-p>wBI7;QjNAaQB`$h$5)`=rwQzf-JWyVRE!qGZ-YwhgMl2my1f1-NXqpaK0-SKOx!BVNJ(?652h*b9t@ z57z=2PpFF1<+7qyM~XVq<7rAgj^hjNUyDwmGlMq|yz2dDlDy|<2dYEa>TW?)Z~Hku zG5z~(&RnnV98wkU?rBEA!ToZ$aztcq8a()PJLKbC%&*|is*tVrDw#3lO8hR*A(uZm zhPQ`q#g$%u+o2G29aYTNC6+aLX}hbdH&I!U^(6`a-xI)94IDq=ti$3n)xt7i(i@#6XdCQJAD??U{;cwSG8uVG;zzVYnRc-pyu1`JK0o#pIcj>b@QE953Qqr!f{q(KKfB7pA2-+@YbIn72sEd@LTh%yJntfPMmYjR*O3|H@izi z;U&Q#Q!dBjqnHRby)&yPKarj{_z@CzW6`7hSzRJ?mzX=$bjC2Re=oQFGf>yz)=-{~ zpMqwJWI%Vxnka3IDv#>8D})Vk5rI>|FB8HlKPbUx+byyYOAz`t3wl*I?%JdLkwFkt zfm~>gw~CO`omGqh#?WlG5gp1_?pkl6;9v-=g&OcugB6KJ@V2)JZ4E0LqV!5}wdWQh zzSad?ozQt6dldH*T1_X|2zDwRF>3)MuguA4yf;u_)St%%m1C5+;%eR-b)R8)JOT0(E~q<+dE*K9=@c;j!^y8VHIl) z3-g+&3xS64NYy$e%~$;Ub7B+n*P?#~@D!N5B{`M5Pzu16C&vD08w7>eOGk-2elgzl z75b!3^`u>%CQe1%Uq#vYTN0r$`9g%$QzT4XGv>R$d=#na79I=o^182%SFqyW(WUz{ z&|v6Va5Sg~QS(e>V?edaT*D#Sl4R*SM}}iDdGOZQCNMAD7FxRdMp|v(D=_g=Q(HJz z_XXS~o}_WXGjlg0(LB zsO5neqSR`3^QW7G)i$~BM8DYV0|b}=h~Ao6lO++Oox+q39|jdM3FN%vn*%oB&iU;w zh~9m9C|wM}ohXlN8Wy}l^Z^bOk%0=n(Sr!(qcRRV>JNChBvdO0yGjb7-}MW%A?#nq z_BY~g+AtTcc3AQ;dTFsJHZoApCJ6Lvd8vbK`Um;o%+(3qZ`R`CrlDOwfB}UM(gD@` zZL^YvXSyo-ii1ngq?`-Kt?_St20g8Izcc0Kndzzf$s9IJ=*hcC^gc1+35xw=jNJipGMjW zX&g`9l*G!0W`Bt3nxatqcmN!^0C#tWDa2PT*H+%;IfMlbgcqfQUJ-`@=JhKmFecc* z$kgOjJ8aGJ(cvcP=H|UTt+nGwZ6Db>PHGu)6#}k;IC6wTRW*rG42A4jI!|v(Vvqtz z6y3Z~F1d9r7?2r6Q^E>jJAh0%3|EZMH*yf;rV!u^z`ostA zHyBt8O+w_^zVDBBSwmmw1b!%^QC#MeK}v6iQ+UrII7|k>xk2yveBf*=)m@^9?Chog zL?9Xqz+ug7Du%We;i1c2O!7w1`&P4U`az2Pd3UFt{O7{>86GLk7VL@ejFS&*Y9#7I_r|!~zxJFoC zL2N=hCs;sMW-NB*-f$kaHlfb zX*XR-KG(W(pe7uz^2HGk&PhMKHoAuDB9T>5@p$<3&+}meWBYn~AT}piFgLDb6YVyP z-N8D;4wtfgAqb1B6b%~JXn|@uS||H!^_r^z;QwcT;vu{M>bml_b2BO@_sN-DUiZ{} z#wdu>ucAi(g>kyHlfy?=LLwU;HIjSI=}&63xHz!L z|3o)oKSDwEl@J4&$%}0GIx80;K!-;|kiU43oPnbQ(n=hJ5+XoYW}$x1rs%- zP0^okgq5_gWOr4PGdb*h{0mLOMc$TnqShViTWg*Ygx4yA$Qn4q zZVL);q`)iA1R<~X9~K|AUo;v!;Roa<2zzFd6Hve1?q}~VS&+^vQD*56KzCceo0`b# z)>4n0+7zFI3M+GVDRi83|- zViL7}k|wDp;dXvBsRPBy{snbWUk2s-jnWrxaIC!PPN)xDYK#7GJ-CR9z!*v}XY9eu zVs60wG)+F_m!MaRD@hNZmS=oKqMX<^ zPQ>c{(Naj+x)NPd=Buozf#BTT5S`=q%%;;p(v>#JHmCD=Spi2qgoPbvV~Zb$cE<-= z$G?}Fk9D7$i4J+K&9pNS8XYRCgcpldj_l-p(6`G2(2mgi)b_qR_p7P~j(sCHUCN8L zg4lIW6QAB;E zl~Y~>DNfiBd-!W>0WPJ~Ao60x3Z-{ox0Q*MTw9dOoPxKcID!i5Nod+*R-0Rdg}mw+ zw6b3qPrqEa-yCp(J+dzp@;a$3NJBY!j$O-Lwp9N^EH_R|&*mF!J%!>AX%?O>D-cBH z*S;R9obsD)tt!XP>|H4BeuiwEMlPjd*U8aPvRt^{YIrjoBxS<2n2}{^?AcnXdn-Nt z#&zXKcD9DLXPi*A6{p?h)&sSD+A|*{@7OM%MwQBx?@KD=J$+nTfGcpZJ)&y%Z3}7C zO&*yFfFcd|*V!)0Nodd7GJi}N%Z~JCIr@JMsPj>Y2^mhJvUf@=SGV`0k6)iC%E56Y zWVhtDWn&cnTDV%+QC6X%6}oSWeZo3Npd^NGUS)4&?}tTsBr@$o%l;mBx;=bcnEU~| z?vN7PqUq9d&6DfHbHgP9T=M5fzd&}z1qK`8(_}XeuGVdi)1|W6^Zg=Cg!0jjiISBk zdRt)Iu25JmW^^eTnzZ@V4yf~Zl=Vh`PVk@%9 zTk($ae7Lrv-8w|t;iIDNn5lRqq_A&9==r&E(5w69>~N{|$vl7}=&G;8sj4#Mud2a* zZlCP!LGB~cYa=?bVIEoMs7mt4$r4w2k|nOFgOxB)$-YILhn+4{XznXAcCb4mnqYM> zl3t6WGT>M)Md14~@5|8-g^IcQmFBR8{0$DlLdPSwBNd3U`EvK;fmZuZak`)v^GEw; z;w^ZsXYCs?x06DYarLrLHp(UZEXuMnalB*$|k1 z%}R^kUERd)Z8sg@vCX@|x@OM)(WC!WMj&;m#`|NNloy{8_9ych7fhB;M#e}w*v*S9 z1g80^LPE_0TRZ$XdaD=rh9HD5-F`;Fwg>vQ!rO@`{FIXd&++!D&uhev(*4Y+gd_F0T?`;lKvte~ykg# zNWf$(f<<@G5fx2YtE#9UO{dTMR?o`n{YlW~K%E-U>>uW#XWZg{_`N?O3J zEtP6J3-p1r%>T4N*HNY8Xn7Im82M#;|Ls`YSo>u68QaPZJ3q}>5)$G9U6JqBGUVa@ z-=uDC(hLGL0a)G$bLu_=?dp=O<%2}{+rILvw9-~htFPyd!?acnB8($yzeS!{#72bP zEA>QqU0F3i9}@K;&-<*1UyfFsklrkZ(wb%_doskM;2X+&eXqQMcom5gk2J0E-cM^$ zfx~Zev>KS&a&6fp#G`C-KOU)EERH$(rXQJ<9odXJu?Vcp4~#UVGbM>f2j8};(1$O; z#Whl5mmm?2xN`si=X=ZPapOGLLpA^(3DD>y7Cq{_Ecm0i+?9uGujf1zb)r6^+{!{{ z|H&r45}9jSe7dzTQSX0-gERk;UYz!P@nG{|_rwFstdW*1vc~w(m9~T925dsq0v=Jj zq~B1=Ra>_1HAW4B&cykyhTH9dma-)^TSWRYv3ye_g-JZK^>?&Y%v z!4Ls`8JhRX^#+W65(%LVOzz% zD~}q%W+Svr@7R<8AJZ<%T>Jf^vnTj)3dJV=f)xkFq0};4$*c=*{@nQBltgscH?O11 z#2^!$W4V8J+;n?wU3FFCxE|-rlFosN`FQ@Px!{X3epJu)3>)JC&-s(@o{0NUAJ0fjROJoj;W)IlOIK&3f0m(m-A%Dhnj202|23BIJR~O zyUmug7DjN3KO66$*(^nqQj*@>o{3euZ)y_LfYU%V%VqMGpNRP#sxiBYWgT|$g|ZJQ zT~?}tjnunN#{CZvsrMYsd@EQT=O0O)9H+H)JR}Swd0Tln&Y@k=)k!<9+rd}QEzC4( znW&RYkmnJqgQcYZkW-nJ)XhzXIx7mCa`v+9KYlD1ksVmfRUc+3kBacXMFN>x!`s;? z1-*AE(VT=je53=VhSFYHxvW9M_$}euK#7!{ko4O2zRpS;ULa>sj)qd&2ug1?WV?#Q9D?)x}?-w;dPoXiS*eni`Kb_@!bMPL#Yvl-N|0vkg#3CGP)oGxP=&%Q7Vzp>Ym-Pxx4+83`jv*WgV zJN&nyX~y==7M2OAEWU&BkC2ahdVHcv&HiG~7P-9gSo3?ucD~F!VtqtJk9WyS3v-k^ z15u8{GvZ7K)x9OUt$Fom+n>B+XZsT?Fh8*s{nGkgikYmiaCjmkH{&?`AlUeQ`}5A> zkiImOM_Kke5`IbYJZpQCrW(bLeQ!O67TB3)jm*2q7Vu+mhcg)}@2yk#t?-H2rJ?Bn z#vsK?@qTAtRB8YIet8@1^qkiG>y?w77e{YxvJa3}?$e1BxoV#g%r5vllwrI&^_Zmn zOh_Q}2gP?+-gz?Dneavt^orRdwA7jeOv7Zp?l2d#lAJGj1$x(ov3>oH3oCSI z;lh6KE38(bJ95kKINC*d+7EFmTG6_Pvxw)YkX7?JniJh!_)(iD);ha5)XGgW+1{^N zc^IPR`+Y4Y&S>Ll-%Rt#QQS+bPwng9-`%VOw7k$GItB#16K;4t%~s+RuiVQfU31tw&coH`dmVcXO>Ts8XkG5? z;O4?reb-&zWN8&A$txPkAvGb#y8Vtqm0n0mYN0vYrcq?voVrG%)k^z}@_dJ)cM8Y}&z;kQ55fnv9wn{!*0r0s;1!K3av z1Wg0kvSyZFY2hFyjE6w@$7g{x1BkRx4RSHaN4~2zpGVyeQ>k~;iImaJv?U)R= z6Dq?z9!p{0=YQZe2(FpM%yMDD0Ct!P7vs{*&`{DRr_(y=Nhv}%&lJw{lo zC~BkhwWygj$5xga!f|7l-p`ZoDicJm=hBu~`0iEYaP0iAOF|R`hk=2i37}TpAbPXZ zHDkEoQ(hPfXo6-+P}cUoGh{+>RrDGc2uN8xqn+lBO0veN+Lg7%1q}!USDbHux#RV& zb=;Tp{2->pPUdummf1)!v0`_ZRJp?TXWBuiOCnuy?^GMEFZ;{gRSESKmG*%y)3x|D zWv1b>4^!Wh`Dun-mU;&W0BOCiJ419~&HEv(YJcX0-dgi!YQ=py+*eUd;V7Z!V+guV zXKOTZR+FmRJ6WH!CR{kRqF;boFBMfSR(6hsE4pK^-I_c#)Ya`KQ?Ci?8 z{Y)MnXauz%>lre_g(&SB~=`VfK) zYHS?Bb*4&Gw|C*f&HHC&6SY!nwRp1ZR3XZe-0zSk0C!%SW(2OsZm7a^?cT*feTjcy zyOe&LLh7f;nj97mO-i%^s`t^t=yNT~?^L2Uyn*#mzrqVgw5p5XEh(bkI4k zTm_-pe}F|CsH)QRy@Zr&P-5MT1X81g$_rumTivvDx@L-tsogBRC11PBI(ib#?=Q1? z%K8bd2_AKA-<+Rjku@pVtJR7B{DSw$r)j0N1U+(2mw7NlDnDP(!mUEThz_KPnUu-w zXOl6hYfBhN-w(wHzpP43I<9y*$Ad#R)!IDYu0Gp;W`Q0)w8JEP+TAfXnB(Nb=jnZ; z6w`k>Dq@P{(%ol@^xP!vFB>9SwmPTc=F#*W&UOHe^CbGUclNgN(NPFb zp_fHRL?t!EzX4IjMgIf9I&6QAI|kVXhKL3|Yxb>p2u@ zT&a`~*jIYUsTKJGd#*@3J7aFl#sKpX={aWzUqnBSQJ}UXi4#5>Ng5U#H9Gpj=VNN+ zy>FxH5z&2U2O16H-z%(~=skRBzHfa-vfFaVTj%kX$k)LJE=oab(E?f1PW=y&{{C6t zW$jJ_hxtAkfC%Yx6TgmZay>8c%cd1TGaHc9ad$S~AO zY`w?x7^xYmsu${?*f?uLIB#dLn=9$CN!DU@6OMel1}!$8AmIO^Bx%k^GH&@>lvv(@-dHNL?2zUD{ti9;cx7jUF}?rUhcz_2tC1bGzugA0=PUp0Q?|VIH6-d2cKc7e8$FFIvjcJq zLbFT{5W6TH%ZePT7G7uh0|M(%wD}3Y-H#|8*suMv#U}1UwR!S6=5r^6j)D1+@aa7K zb{*|x`Wqk0yVPhLCN{nno;ll}jfv-ueK(f=bi`Cz5%!W$Q|zbqYtwRMQ{cLyIaSl6 zUZ-VD5Wt=;Q@aYoBN;=teiO88M*vpRzQ}ibQH&4Wp`ObQ^|v{QbqLZ8cu~Kb9i{h9 zSK2ZIF;GBklbGkU~4to=OIY$>Sjy&>}cvT`j`*(!EHs6(0ik zQYwd{tVR8Smt!AZh3$_$4ozY*k`R6D<3N-YGJjMKCtTPZQ(QN{*M{5gePp!sf8M63HAHv)u1EEO? zzsquQzn|%dneC|47Y?v?-@+h(AxK?b&CIaAvOsqHdwDxsPQ$Ie+0Z9%HJ!RH+B?;U z7=FASTrd{H*UaQH+SuF*heBwH2bJ2vsT8(OTU!$I-*^sN`wQL6>;I^q*Z68 zzsfD!X9+MIb_zxXVuIPbBh1&f$0*{KRaJ`;>$^)WFRSzg2L zY~EI`W?jg96SB|CM@bLNU(bIS0d$G(&Gi@iOB)?aj|x&l3OP@f+e-IX696H3ZMqwr z`c$1{2V!G11PSn{8xGH<|31nMdK8wZ$r8NUm*0<4xO!C5u3$Iy(y&gVjoBwV{TAWc zk@?l17m;&H&llq6Zp|)(Y?_6Gm>9wDkjF+V3AdKdHYekHtq0drAc7v$j6$&Ca` zSbb*;i&eD7U0BbQn7N1ZOW#lf>xiXej^HwR59B&3b@+X*JzbabyO3rB+)f*rSr+K5 z3BZA~HWK(+D2L>uTRx^VD5?CDZ*p<#o+P_|@sN`0X&vy+b^qvZwDfB}jDRNw87Tup z--`DA>pTu<6%>Y7-{wvjBzbiB#dP za#LbJ1Oov5Xm+Kd1n1gPICGsNLXaqX`*PmbuOWn6MJ9wg1#my1ckg}#U7@V6aR6NL z*EzbEAV-I#d2ZW6FD za03Yc{bZT6V0@Mj0xY8rf#vi+L`@i76+s!lK?EfHO8My!qi>Ckh@@m>REEQ%Jl89G z6V@K!l|aP;I%YIchq}~ z{y}m7o6+;9foW=L>W<}sH;?sFj%E*`IhcJ6B#KJ~?m7+Zg-sJYTwt(J@F4RqU=;z1 zi#>PPqzAI#D)$dPXYxG)=i$M(BYwaG=8*MN0?`lUKwgy(WytD~S>klMWb){`f|XTMMBB6Jw<%<>&Rk z*h1<)UZ1`jFjD|ZjWga6-4Db74Ut~ISHN;@SgTO2`#XI6)zhs$XAbi(l@LtzIbK@w`&3l!e7 zAw+99${kI{I?eWEOterHWWB($C6AgYT5u#LjY<1NqjX$Sg67BQ6%|apYw{bS@%!r~ zs)5CYJH1g10j?$yT{s+$8v$x|79?f0O@Y_kI2RJ^zoH)$@RjU~@Xlj~xl{`3Ald|B zL7+DrOof{f%v?Th0PMwU%HRgV5~+8#Oi0`P5)Mv?5a)4!?Scp8bTa zS5<}2h;^6sWFk_QDx5Ycp^EVK+#i^_I)u#TM-5K|+0aGAU%!4mxH&HbU}W-(Ou<;- zH7*25@DvMmA7}a6uoErwkp0%NzkhKJ%o&yLu2XMm|d& z@l~xkB^o+9@dB2wKM9|gz*1VKe=>_jg2jqmJlO4*)Nc3w9(L}xlan=W^&7EGk{OB& z&)ycM5BQK>W1)b~N8oVzFCN=9>t^rY&@*H*#bR_iq9B+NXqy)RaE-RgoB)a6{>`Wd zU=FXj>|n8? z9Uqadt1#Sv=^DU7BVpPNyZn|Zu0{xm-x?SH2mS+IhV|x>fD5@9$Io}Qw$*%U04^n2 z<>sceb;$SnSRTJp-J`YHJPBj?=cVn+Yrl)$oq zdzi-J8`aY&Ktdi0t+;%Id%5#o$X(Ubk0i4>xfL!={&) z6LRUipM5Sl# zP-J(%e?NjI0e!-O9Y=>N2XPsm!>Mli%lG60j907B zCyjrS)(Dyfu%Qb*v|jGKSZ-o zjIFgZ2cX{xMp7H>0FYgQs=Wu;a6vIKtogKkfkv@^KfW0ULYai7&n`JaF0S{pxf_u2 z7(XWw1OixacZ$>#y-U~Bo)h(~&?!~_5Aom=SOSh;@lYW$ zNYM#iD-KU@Z-AIq$kh}KS=Pda*r`$&_nxsY=G?gPAWo3?mGnI=0R8_uWMUoybM%kV z10y^HlHUF4`~6Gk3|Gt@O-v4LOeMwIUj^*oD7B#3-d1Ey`2$UXKKwsaB4;W!s%ohU zTMGrK1+MCZ=I>9_g`SF( z4p!=)n_^@A=h<8wkiK}tSahj*Vu)aK!?L>XJ_ROWEmfhm1?RC*;O2k03wME>=HBQm zA-jCu5mjfElaoTeZU7Pv#rNaYm5$b{tY-Ht!KLEc40Ndg ze{mJ5?jd1c2kaOTr3=)h<-Rnx^pAgGZ_r!rN>(4Aaj#59_>yN?Yy&r(EImOuBv>?@ zajwAecSIRlKn{%WfE)FzuFlhZT`0{e0cdj%&l803cmIq6fd>N7T8Ny?w}mHwKq(R( zVl551=(?ouzbg+B=h2UxaadH4+iGpG72F~#=R-7=2T7G33~1fRo>dX8OS81x)wdh~Pt+M-KXXTGlmz-tnBbb;!et55&S#rhb#8mKoR-CFr@mYsjC8L<%x~u3 zXZnW%+oDaodNmmre8ffkV3z0vOeGbEy}s>QRL}8M`*J@(qrq1&miyy_$594+5JdU!5R<^_O1T|?$8&u~0JpCufbiA?3e{|~ z`sPc=)8Ytb)1RkLc*6wfU?jK}ih8TN&lql`34Plpk^mizx}4}E`b&h3OXjfjhH?`C zNp3Edp86kC{h$0b?lq>omB9FwlpZ;*8H}t6ju~>?D+0KGfaNx9?f)v{ZGhnjm3#U7 zCIut#FNoX#6W!ET$>xYpGR;=clhQOZUsp*gvwWKSnN(@eeF{ltXMHglZqXS2U6Z4Q z#XBz#MiGvVdx|9tR0azdz+F7o_=3=EGKo`5i#rv%F@Yj_M4V^#5@J#L=VkZQZ-1X@ z@I!t31ouw~1ER^d5VF%&5&~uWZ@?hq()+X9?SDU@k|CF5DXI{>y=(A&V{g&}A(?CC zq+U1`T~}FY*?uV32;r$Z(dP7^Ozvbw$J}IWRa?Josw3lLl88Ur59m%FkPl=FD}!jL}>2>)SD%YF76A^ z|7>}Ej;*qsI&<%_2tu89_v1g3=3Lv8Lf=nKZYsaGZyv#mm(TgJy*Ds%-`XMh+B<-B@infH^R6!T5#VCr*3xtY zfWX2+1M3^e-c43;C4_r4Hw7kXf?k$ppzFDvDiPTqvkKnu5eN;owGwm-)_k(Ln%B|1 zI+T~tDihLpjF~eO*=`DUA%+WXnTe1L-jU_0a+&3~;^JbM3`C}X^riFU*9~Y`z|<e&6&ujkR!CI64T*q1-dfOrzjSDAtg=Us^k*-D_&gHh(JbYHj5!< zd!ty4Y=)xQVkhy^;QV6%TMAzlTVMv?E)fIcHXYpPN*0adMG^cQOHPEUv()6e=M?m9 z2-HWvX%~+7lO+?r;IrFGYl|*E1~{wjKjWALg$TO~Iz*ny_R7aB;9-G|LqQmSE!RKy zCKp3ka_MBECt?x)3|<=X?~a7?UWV5~VqA0AzH9+ouiG-BBxZ1Z%?dab&y+okT zGh3P5|nuBgj<$ z1Mvsn3qz$OX;Kqbcln zXm~bpdg5I{*p`ynZr%I$;(%YUWn2}udzurK4%&1OAq)?xQY`mI^yuGu>Q&u(xKxyRBcn1Dy<3Ukz>**yNC z3smOgd&fu61@UU$HAHG>DCri+pvv5}?lQi)!Z|z+8v-hN22MC$2s@sv(HyOxsA?`` zMr+oE1>wt`?P0SDrfCOV#RkS^udi7 zb!MholhL-*zn-N4>{Q+lkijQ0nD9bA6_VnCy(=- z6vMsYBE5~N^tt^rzO=+>J#U;dE(5=4`{V5ui=>|Hk}Cv*dL>`7Gv78&9)l3*LrU= zJN}FnVRqamA3fWPtK`Wgq{pyXe$W6?k z@JEtrI=eW<<{LThKF2H(*1DUrV_L{BY8UaMy}%Zu@}K{qv5O^l%DPcP&zhI|ZG2kc z{6cIE5YMxplHMe#WX-TbPZmPCLvVYI5_fr!XbRiuIgeS}rgMpY^4l^NdVSq?1aaYP zjd-0ta!iZk&AT78obOZ-)`=dDRd47?8{*E6J5UB>o*9k^t`m0pWyl`y)kuAQX@c?& zqy6Vz^oBv`=Tal(FrMCi&mZ16B-EHD5-w;&&{{ZO{!@Je_K@|tCGm9kjmgF!=rxya zE@#esA7X0LN_bRNcZHgh7#S~H=SuvCtCX?GQpD7%kq8s())gP^BfJ7f?jiehx-WE_SLU9c^|APts{5NoOStN?LJ!-97#IR&6YUOgbf< z)ZYI_B<%?+v(0g7Y4}q+p8}VcbohKYbd0xnV4FfM=>_iclQLCXklayB+Gmw$OFxAD zYHz5qQ!Hb37OdePVaDg;PIO?+YXKNZBDuS*#;?H~J`U`#1=o_%(oMOet3Tf z?#qZ}ZlD>es(C_fkg(!qgtYH&hyD3k=8E*Pg|6|CK*Ku6jsfd94NPw4EO~nkVvoa# zTS&Oi^tt2tDV+E2C{n1x(5T45=ezHB!n7swN^z|1c3szHsqQoWe>rBtO3UHqq*4J? zN{(FmjJ&3yq2F%hVb}fqW?;38oFT;-kn-?WB9il6%ji z$(mcT&G?<__gVFpv|WOZh>ANcvShT9M}^r-Mt%6ft%SF!!uruI5<*{S`xKMmVwP;X z^V)y1^!d(R%%=HVlIHA`%7$)|@J7M+tk*At@80r(!DxT*AoYFt?H@9t-}#NhVPzx3 zzF1n!w|s+7;3VEGjpV?1Djk;G_>lP>!)Uur{Q8flaqnpDE6RvyH14(6c-z=vGRm9a zcA~S@+rvJ5@Jey;jPF+hioNu!e-zsdVhLVI?8EMvD)4qTdS8k?HxPSKQKjZ4PXdv; zV)Hz?SmD<8vxkQ88J+p&54VVSbt>08v$Sy^8=R^(p%`lh;w!TjAVACoe2 za)P}Cydnbbsv1=n%+jb|G_ zIeQ$jV878J2IuOhc#TTaMSl;1+X-I99+~UoUkUc>mu~s-FjXlht8#IdIqBXVx;~!D zt=t@gQH-NUl*pp@clbyKoXniw)FY*u^3;A+H?}X8R*kcju|9qs_476&sPDJ41dqHt zph;ej&-_!sv>@ySN$V}!J>5rpk!ptxO?~T6=%Vw6NuQ@u*)dbz8<5}-`?cPK zhQEE>ap%S01+9Qtb6@u%(r1L7AOyh@+^_?KmcpM-=(9>hF3s<^8mlG}4`ZJZhHPKk znM-q{MR@ZZ#hZemZbNX}RAGMuN0AtqoFEdI)7ba=(-=kmCvC%28LeOVQ2bD$r5DL* zl$t@SEOi#kG#o*XfRA%`Fb(jEq9o8hRnDALFGLUJ>Hf0m9Aca}c$^h-5VA1i)ab1D z%8s}1D9A=KXlLzBO17Fpd2+-10(Dg%mjj&8h90Cu32!YSK9N`Ftpf>fE3;YxqQc$v z*H4~uX6woj&Qtoju~R&EmFd1{Yrwy#v%s1X=4hhX-<6|X-9r)-i z%R9zz#x_TI;d&%I12^PdsPpv}8RvJgxB)~s(*{&ro>3H4!+sHKDqBZ zh$)m-AFGi!OIJtPZjKvE5o*)QBt%%x&r;qdRV-e;vHDqTZtNF&1IqX5nBoK8I7*&h z2}nrI7=L^MXs;f)-*AGj7vojJg?)%~5Bk&QJo+T+d#QUNCkmYad8HEQ!be8^CbqBU zvHZ=S>&A~44~qiv4Ao{6lq#(ET1gs9PFM$DE>Jv0#Jz>7#WTi;K+>@P2yw8!u;3jo zt)&}xN$~vzRI2c2-}FL;g7-~1rX+mO83l-@80Dhsk30~ozMqiT$4`=(bxURl)jGmk zp1i;iV?skD?CEXr_8RIwJ}NGjajX@RtCJnWrcrS%aHw2Hc^{z0jUh~-_iVY7GchU6 zCtsO^^#EP!$9Q0GaL9LKUaLg?I(O@M!f3=e>2uT^j>ZXe!}PbeRI*yVifTqjk&d%p zB3Vt476#C0j5NGPF8EqczT`KZ{$gj=UboL8^ZT@%7k~BhB-?%oV|4;r!GZnt zwn0f6l&NI{Isx0{&$9Rs;=(dRX6~W~As{1?H1y7Hewm!wJaToIR$@+C3onm(#+usW zDK$_X2*Sq<&*pXRMi~6!haOCVuca&MH^R~- zDpXFE2zJ~`_y=QK5Tre~-6!%2S}*i8gDf@o(A#Av=_-rb32*Z%nZYgz{KuB4J%u@a z_~0bewfL0g#*G^n7ca4H`(BqWUX6r{xUWUna1rvW>*4E1#8f0yk#}9cc9UU93O#j= zaVH@oBK|?tP5ks>H<`dAqA@Dd*)h}d9lP~NbxY$cs{_jo$BvV&u~40;)Am(@lMyHD z^3E^W{x|M`A7*zhU;LN7#X6`+MJtB}jQD|>4hyL-^0Y*$-pbm)dvWgb7B$|m%~*O2 zbw*KWuNjVvnVEO{8?I)4&x+4oJoxa!=pYorvv*2I1~W;^hd%m`d`fd_afRKAzThn! z*CbF_B^U+X*d2awu;w3%i%vG+$j}~g$ec9Nyg7VRP$IJwqq3O zIascEtm@PyQC`m0S714&(-i#hB6X%nA1lCff*!G_;SDBVGb+P=mAV39Q{ut`)DXMqadOxRAA#g;fE;oy} zbY8HBsb#QfMTRw7u^`6I-C?sM>3BmtSmQ}%s^$CekZ(0-Jc=Xo-jxyFSq+} zQ6r^LxE*tvs}_>Oxbwi;_jo^G<+KROn?q?N<9wPwA} z_kJRTOz64_X^6_-$x^Dc6(4q@mlBJvNC=y4i&W{SS|TGRriquYqd|U!vXj-foTn+I zhXhBQskMNk4qHMHnI@Eln%0!Tikk3YFJxNh+ub9r=0U9Kl^HE>^r2T3^AmVK0>%p^ zTqtvALr|&eLG=JX`>nl=RKek1$6TG-r0J!lSS!=TH0m4bUzJymh3j{pc^|EhIsMp* zB!l-;qFqi$y}h6E=%J#_^HgH4Z^^sgOnuKlSqKF)ZM$cmIS{PL@rv+6{NSk=ho>f6 z#8+Cr860R1w`TA=B=uv?_KaJ2n~#o2=@>UBZ}i{gHK{88^#+FTO*}(ye&9Y>;C8(a(E$$-bTa) zsTJge`=O6k7|%3%FW6Fiq{*W@m_PYXSZ%&j$lLn<=9Amky1bSj(X44KJBqD7{gms( zN86h8soB1YK9zLiargKc1B(o>Uu3=|oIE!gv{$MZKj}eZ-D50ns5wmQM6^`Os+#u|Kes2(HF-zZXTa{Bu z*q`A;EXY1;vHgj9I8FYlO_-8oxb?OjjP2B0{Jbu}%G`D_5#1cC?*m zHT!K8r&akUJGH7^kW!iH#Po4=)jMEO#(?&MmnbI{7}cEO|BrG5GII(cxow5220 z;XrSO1%*7QiQ?sHPb266RcBb2#PsuQq`2G=JgRK2_Cg`(e{e1PuE;eqhj>Vej+B1OS*!WK<|6crIKt%O&~>ZZ?^ zHD)aGv|mOl=fw~tRvb8n<@MMLKIBn~I9E1!Cd-@#t7m{irxz)5p;gvpG`jhCTWsvz zGCrr#sok-Mv<+RAra;K3uG=kysPMwbpsoqyY_#V&{S%IB zJ`e+<@q3T}#9a!2^+ltJ8zcaNmjZCKtB_r;IG^xsPA+V6;Kvr;_Mu?RY4RPMnIt3n zLS!bh;=nk=5euQ=50NRpdlzmdGV*!Syf4#AikCC9jx9dpIJIsV5hL7Yvwv*N6#k(8 zhvbHa9a5Hm=NP z<5jk%UMcP`9-}`xndHOQUtiv%w*su8sux<*%}5=2PQGFIr%+Y_sGyP7_{!}>o|T?JH> zTh|su1yNdBasX+iyQEW4I+c>{ZUyNYU}#VTk!~0oDQTpoLrQw+=0BtNe&7GCg?G8U zi<$R5XP>?I^Xz?|y(=A|PbWGs_qvW0LA`Tx-Jcn*!d3X1mSSi>pVbyP%P0gzw3Y3M zoyE?G=eL8Cu@<+(5p+Yc%CtAsmv45ZFwesMV55!)uKrsiGrL7Wvr$Doz2-g}IuRGD zH)rLJv$){`x(w)6rOVM*ope4$w?D8Y1od8P9BI8jN~_+WvXnS&=Z;@}oZFkxJXr7x z_dTiXFE@Fez{f3jw2a=Q8%7%%U!uRL=qdQn@m2WxQyyW<4J`^D)xSToPthzVidlh7 zQ{r-W92hyq-u9}4z`rf9S0RzD8%UC61Bno|uP;6&6-QqoPku`g2A@0}EnZDsMoY`~ zTmHZlsw$C0AI?3?vV&ENXNOs8Ms$21HCY+=oBg@>ulz((-Oai4r0|KflQ`+6~y_#o~CPKU?__gHH{p74-6u@Y^cEU*YUJ%tfa|k`2G!fz9C% z+LG0z#Ypwuz`zX64`r+G>;p^$`^gfp;uNJRpZf)gRV>u_^km^-NKma(Wf&Ia2UWWJ z1x0AZmHr^VxDwDVI}VXILJe1Fq}o|>S6Y6ioAGn3;yUl~~Bs3v!((_ESevgwm%Fi)u>`j{#8Vl;{!q* zyI}9DU@0~7!y?_!rH7+SJ|EUdvBO7;GyAxAqN}7H7w#FMxF4PSY&Uz!2ljmI_#s(1 zlzeBLH})kmR>j=|OCNs5 zJe#DQ`AAlsVyUG}Q*GW$pA&f{>3S=pnQexatoqWpJ&NmuQJp52bnwzUwIns#0Xz4) zLA~gXS3n3hb=BfD9mFL+J3o(t-DwWmsCFGGqyrB;Cv2sIzNKcwn=?12FRF1TL@2g+ z5?{RkVZ{cVh%N5>5pOX6S%}@UobrAmZtN2Ge~&5^iaCybl6{_^_ILVIY(w)I`2lfM z$U+#};h=<9eVB2HO2%D_GP3VMW#xgU>eH($@kzMJUhI;J=-eT8A*iwV#gm>wQtH36 zZ|d#OL?^$}Vex;YE0`=Y#l%)&2_!~W#k{km>uJ-tdYZAwN@P3pvg-8R8r$4BS~KP% zcZF3We=Sz~3ka(`jHN8hnAl*6=m2#AFB%>PS5apod{Ml#y1Dq+kKZz=FReyFe`n@c z>GynvhPzVR==^xQj??(*ZQZ+aJp`dS6S$J%`)qrRB-!lS&vxcH zc~{0RlXF~;TK7{{)gBYIer~j5YiDSm7V_OsNfUB;PGwE#PWSn6TUEixf-vr#;#`S3 z1z}uqK6YR_L3f-M#9FQ(R2!KSW2k5p`H{5Uv2iQ>O_>KL_|+l#qs*p8TY5j~-?aV5 z)IOBo*atx~X_5zv?|xLbq1KshAlf z07+o5#OiXfeU`K|d&G;s^#2oXF+K2NB#DRoycjWJ4Gar!7dhr`S03DrZTRYZ6#L<# zN@;zA9|!6Sq{K`o!5wVMLX)|3G z?*bAF5s~X)kIg(iqmNq-g(~zw7$K%hub?~@mU3}ik=Y6<8c@3XkthY63(Wt87%~1< zIAAwPPXY=+7z@cod!ko27a0T(351++Z}x9pTR=V&5`n5=+YupY+oNyW%`If_)pr|e zM|$h%8nd{E11(^CyRzKD!J8OeT<$UPwr&qjyB1L6{pcGOZb!8Onvi6-w_edp`THN-I@1v)dh|iGmo3 zCmC~z0|$bZ7}f6->yKh8yGIn}9-OIr4}TYrUY1)h_P{%o%(x?(@Nq#5V|ePGDE4@< z)r|#4y5wG_a9FU}QQpl=G@a*Zp+v)$dg+x?YPa;YxH$@a|T*XSJ_Qg}fhNj#B z@+$b5wSKreS7Kd2& z7sk-_gs-h2hP$I^-#AC^0KJj|S6oMA`Na|~8uCj8A& zQcgW&uDu*J@kO8Ccqj?_JCB%z$qxr+FA3NnV=7I_yR` zbfxe;`S0O@bE4c&-0?nvy8_cpWcY}gc(X;q6zbEc@7t*N(F^CP`DkZYFxFT0q7;}$ z`0jbP9uh$c%mHk(+(W@6pMQV*rcb*~HIdcI^32ASi1kxUCi%@tO2e2l)RGjR*J*4> z`38*4p0dDj;E8Y{JdxrQROo&)nO>fe8}M!4@;Cda9^eh^d(w{6vU``<91pI!xlPU* z5|)P>dmVodXq)1O%GGML}(=E?zXJho&&#j?6OqU-f_9wwotn#1F=}bp| zxIN~ITPe*QoQ0#iw@^2z^v1|9|rK+ZMg63eWbSXXUkK z6@19Fxp?Acc~f~Fs_iv zWgQN{XVp2bIi#!`OxC2+G=#_ znUdcFu~k7Tmno zjefpnYb4g#6F1>G#g=saL9Di7BTG0Q;vO5^y;9q#PJVFvy#;8?R6EM(a zsIYArWU{}0@(ZC{TLq`uTegUa7zKsjy0_6ChatMPmank`Zjq=?dM&(jzH*NegK!eT zBYC}Q;%k`(qKxam^XbAL(~CGCmiOsjCN|8kKV;PSIgdpvEh0&@?V)wCCaP8{W}sc+ zYP|educ0GxGr0Rx_{AtwxEV8};V3zQS;wzx<(S4bx9*CM44qZ4d@XX!zD*5RY8iiy zy7ymSEQUJ3AhtMs*2hl^5&=}2r|x@3$=p^jlm0ZF>Y{T9%66Kr3q4A^1i6T7LYjz| z!UBzmt-En|Ca-Zk%wO_poVhiW(tP*DcD-9ZjZd9Mw3?wgPgp-idabfsW`RVysw2pz zEvCODJh*a9s<)FVx=B^~yi!szJ?7BV7i3~GyU24=wcl>3?QrNuTZ>KV;dLfIWUa%I z>8KB>f+hdYLbud{h%J!FYYuRztPvV$il_6Z*k-b%R{exM z;U8H`cy=VCtH+zE_YlrD;--}7S4;t2So_wQ(+8gEt`w>Kk!C)ri_m{yF!)O;#Qyg% z9q82$9sR2M9O#mocs#pnPQYe-D^eecmZ${K4$*|ubu?pCye2}GHLL#|<6K-~9?yhJ zF7sdrKoiPo=t|bVD!bF44n=$U!p3A03zEyfbhF`;Du+0a8y;UVr7GLR-}_=BBIkXT zT}jA!@cF(z?e^f^|L$bW4|M0PpQ_^}hV0kZ*X71CUo|zYiTM=aESkk>pxI5TXuCBC zvrL=z0vW*kb@A7McKYSlW~N{zqN^rP->P&fm~i<;5Gn)wwe(1c>8cgN$N{i5&a zjT{iH6eHYC%{_*(Nzo^dyIaN|do1)&Cm)=U^ZQ-I!}`{OHr|_v)Q&XM6e(22wk8`y z4)a+{{u}Xuhe}71^bxF6&#|48?2&r(XZyYEIiPKk!=wwxc<9fH1_a7dx$VrPT&y@* zj1(4n58Z4BhWLBY18?^l^}X~Zn^Pr3WzN9Badk_Z$iQMN@%TcN*<`$!WJ-)3l3+-h zJhYYj_MyP*i<9!T87vRd)1T|X4weLkwEana-;G*t{~HSe0_9q40(%2Jyx>;Ie=Ak` zs9R%O{-)!;3nJKoHOF)1T{T%b40*=W6pdEuX!il5Vma=%F%gS_|Jf{Pec5!%r7kj; zw>!QFpL??#M=k@BqiA>j`PBVP(kiOzrZ{m$2G$+v^@&1yJ;Fq!qr8!Rw#O|YK2)FA zz`ByBr}MwthVu~k@y{sXN^(e3VZ$IVHq^}k(|9r~f9LGAnyP_o82m09@jAnmWvn^U zkHQiiJpA}T((WIkUC9!|UQ?*w%=pb{(#fGDV)C>~2-|~A&O5G$k9F1b+~jvw<4tO` zZBP7T(YM{+SUQjcgi7l3l!UizrcUvp$4OiNNy)@ecjQ50SwF{LoC+R89GCCN!qDS% zLb=MEy?I&f@PvBi$!yv=i;&NtWyz6`5w_AqLVuZaByBF#-6nzOlSXhhB3_R@B&)F`tGVTZu={5Q+ zkECQOAXqvIQ>8Y&2XlNlR1{X7&YBOTIVdUZ3>h=FUC!W+>+ZSVJs;Z#SpixIcLgJM!i1OV@1{HlQ%j}GC z46y>-m<#6EBgucz*7oq*B*!9F-BiHN!iZa|A80O zcsb*7QKW-G^TpJpfkMRVi9V5Atc=R-_K^_<&sRINkDL;s3w5(6p6#64;4kmxTmdp@ z?`GOdHpB?kk)Eshbjk!Kx;O}Vb6>ietNyFp)8qgaNS=(=!4)G zgSApG`?$Yj+Q4+R$zorMA9BOJ^Eu?2s$CDwq? zO7jUHr+FnWs+pUTFX{HC=&Cfa`yF#n&x>MryufA3{qK~ZQBt^m1zeB8BPM<#E-wDT z^i}rl(mz!qE!@5ReLPA#c7)P0vv!3Ebl9{5e(zf=_Ofz|5}g5$ zPh$=jo28+u!)EPv`CbcZtt;>0F52qxkus!YjfIv{bHm2qel+qh{{3m$25G%>y>927 zI1t^&;8T{2#Z6Jx$dx3^8plw+UE@7PTUKESGXr zKJ=-%&n-QP>N5N5WQrv|p6rV_B967I(l+jF$|gFdKd!FKtN+p)#hJQ-sF3L-w}>;3 zaSP{$O`Ta0|4(ewMCJx#yEqdvsE`;cbTo2d8uRnxc%p#8c23nd-3QxfavEP_$P4wQ z4AAa`snkXExCmK%qk;MjkD`4GOof1aHQ-eG<4qWf<9R1WRndOQE0~DgvEw@7M1~%F zZ`36{@2(;DDncPm!jj-CU6FkC(+&6k=ah<(PN`syr1%g0f&(wmRyQE#(K0~07n*3u z(S6gVC>PIIkwz&jzf)|$M$r`7&TDMu;rhEFJ6}h^@gaGd(NFu&f0k|@{egj?WR$+h z*Xkpks9}!M^IWX?r64aN)38-(-}6p64+IStrqGfRY(QV!Q~k1Q@xTf7-*^FnA|uFI zpV58yMShskJyhPr8+y%3Vpx_>?4fe9&i#Hvri`X0%jM@T1#(>#xzIWdVT_-R`tNq< z6C$<+6kxET?Wdfor~WM5-`dj#*^4#DASou_Ra%=GlkEAO{7G8qVPbn@Qo;ukiJto=ahNkG-Q1 zd6ju%TR3y&6B_2C?Y(WruT@Z9-qpwdCL)_nzw_<)f8yW?(HT|5w5q1pKwGLAmSwd7w7#4CEdsik}k8RmR_ z32i+7zScqp8vSk>6IpbLYWN_GCo#hA>A-I?ZD;qgLV;>E3ZW*ezQ`(PrV+XOrk-P8 zfBiY~e*s1N9pr?H?gVDd(NO>nLyqu-xOzVkl~K^8p}6kOvEIQXj0f@;A4cAdAvbQ} z9Bsf)SYoHxFpn^WPsNw49rCGmi%M&$S3^;RSSorcxbbRDIvRX}Gdh>rm zSxYjS8Mx7gq}M@71XLuSzY(qibrNtbr04u-vtxeV5KIQb-3)RBlMsNZVWQ^V^Pxu( ztofQ3N65x@)_ft-$AQ(T{#+tM1V)gMuaNQW`O3g~l_O%4y@S(1jj`I%C+WEGuW^CL zEcMhYwTrs{7|P}$ojh)$lV;9*YgFomd3vAyiB1A&*V`j6DIQPd);$~{)1v?Ny8dWe$XKiY)Dh>83t5m%RNh~n_ZzKNQ!k>c zy^B>{+ws1tpey{Nn4EfJ@$$LzM@y&k3SYcZkWUZjirrUEeO*DxEZ6b~L$A(R4mmLt zT+ejwSaDE-EI6X^WX>|p+flGnvrDwIJ2)nfJJ&%F%H+A5LKpM=$qhc`Y4t8WS~7J6 z$=Uk9UDfElEdDoqVIjj;oBk3hh!bh@3z^47%acJBmIGWza|z^)BGvpildqqzTz_#{ zN`6vCpj$Qc)~Vq+nTO|fW%kz>Qu_}cn~fg$qv@rK#nCI73}y(I`&_to!&p-b*b_Ht zQ4*>4PF8MnShm??_)@Th`o#78{P2s;xcq@bQ?X)Z&Xh3BvK@A>L>Pknj`x8i&8GW_uuA62F4 zk7Xt|UlWIRy1b6)PUjwn7B&wB+=DF8pgAw@gORn0g=;h?fxY(FsJXV8uy4%w(9wa#m4Wq~XfL(ZKGb8)KAz#s%OtokPM4~G?~>mW-eL~@6^;vCRic+J z^>4#Zq<>MY$6xEX5~c^=nel(% zZ>qwbSL)lVV#T)5x0)Sae6f=Aww=R1V%z)STj(Whu1)fy%5dwhncaVbi~TJST(JCm zsDU-j1h@P;SX+5jql~wISl06(!|JpKx__jGYr9~w8@-HJyjn`9Ngz10$&s3F5v*C! zigRmk8+jR4GH>P($pBf{Kw9Sc3NkxZa1l= z{mSgTG}pv-rakK_oi%l3De-nk8*eOu;Z^RRRVcs*EZc@#2(aV}{G?jD>O$daiH^1Q z_3pUr_|=k{c%3uLpMD%svj2@~LlD5DrnLM_17XD%2j2L)QSW?}IW}f(X(dtbeqS1M z6Hqp&tNle;nV4&?9HugLsOG*{{&buD=Jfqc3a>)L%bWxn1z&8Z^hc0|9^?}2&n`~M zwv?eYT|32Z2{!A7JL6^1_hhiuM7Uc8g0sH$Gn8N8-xaupAt~nHYKms>9*TSbJb0ON zs9w5|LC4oaxpa;D>^jAG`;yhuUMWGVwN1jWDz-8oCRC2)2mf|7+Y|FBHLig4Bx@R^ zKV$B5(Lh=Oc>f-^U@Kab@A&TIhms!(NodS%GhW4cSiV-tC;UpM-{b0fxttd8UvFm( zm`RJ<_X~jUTw)I>sPKi6HD7+^S&yTxAjqiU>q!+JFHc4@sE0IN^Jo7ktmU~#U!M`r zNE4y%mQPI7l#VE${Ez@y$Va~VM*3o%EZ|D2Gv^sc+!PanRjRUGbmWq`styZhrs*nj z1QQ>aJ8=aFy6toUMt}|=ZnXl9(@?y}jaX4Ce>P=70Qo@!jSKSNfyfmXmfC>|?j#Mj zLY6iE%2={fme8+$b>`)sq}co`X8c@*a!tdk$&xPWbxC)O&inM5TLbY6=txP6>q|~9 z>C@}03p$&rN{lEJ-oFscNDf2c&<&Dj1+PEk<&h?_sEm@0P$sMkHaA_u><3kV0ZTrydSQ zJ8J+Lwp~}ERWo$+gsmV5qO{D=Sl~@lV#XTmi;+yVG76>77?8b)JA=5WhzK%~Hy4wS-NW+1V|2toE3r+HRhWRi`pWIbEO| z40_v{3a=`wyN68ycvdOoOt5mJZ|~lQCt8gc#v;d3ju&bpM@mB(U(%CR1Zr^G&FQ)B z%;YIIoi1U5VRSxl8;`$++Tw!I@`OgW+S*l{DoCN;XqgJ+RK@ZO^CH44L!W#sJ-T~e z#Wb`vhaRh8H|oEX$Pj`q+r=j~(6&YEl91z2$&ij}g!;-`|D@RV_!)q<1icm13(p>` zm1_}37sB*qYJ{D{rk)h%-!Ye8VaK0Kr!t_}v5;D4*#JEVi6B|Z|Bdxn0Fp;@iZZHg z6*eH`wmX-w)Dd35CVF)k_o_gHwPzOBjLLi}!?B>D;~nI(UH<5ln0T^o=onP(Gt8$pB7jsl#y!>#m}Q2$gs z;u@EC)}oI{gMUo-%IGu2r~!%}xT5xsM8htIG1`TVZEk%~ya7^;3fI04b;O-EQB zF2b}D=pCt22=no2?Jsv54_$B9%)>D7xxP*J_GVEq>^7bNv3{cS_nb?*$f?R^B3?7o(u5QoZGH8+ z%z*VCGI0!eg2y*roP%Z{l&h1QcayQaKFJU!%$oZ6FSL{IA+?j+LCMrMs1&Hrs*Rr? z)$3_|`2|L!psA(5!iWLqK~nlYesBRidu~)Ow(Ypk)bBlUE#ECIxG@`9(|ZGT9@S0h z{PGkAJX->|MVx2aqE62CYv&bcPl1$Ni!#0j!y8wCwMLEa`(14FlJMDoG$L=La^idA zOA~$y24SxBC37O@oUlVE(lb6;bOI^2y;*!5NQ(mb`9A&#=&bspN*6jMfMoP1frCawNSa><9T_x+n4 zyt>Ma1d$Odu~>a~Aq!{#<)q(AeidN*l8=qaKfXjv^tQk%VRS|u>A<_?V>Z45g3uH1O)l$!F_nuwIK@G&0B5< z6NY~JBG;$Ja^PepQOBTTXe7_g&Al!zd-o?Yv>?!CsoOUutboPgZ*LvF*vjz`(#R(VNT9g$G6ivAlK$av4Ik zvumLP{x}Ob75l*b^?2e~?IREvob5xOWmbx6B=mm+FTx0J^MbO{mq#iYLbwny|EF3X z#ZVH?1iu(djSglQO7uVDq)ow}Fz{G;l)!~0K{qpo%g&kAw-pjz>-<}-!wZ;r?8Cv$ zzx#+qWi;4KqLtC6*YUX24NDGbz_@tudY{y^n>YTjt7#x|`Y_&^n``mehn032{pD`a z;Lj_^K9!SRc+Q)3*&eyE%j-jeA2=(>FxnrY=))yx(7X}GNS7S2jze5w+OQ?msO^H} zL7S}h^quR1rwPY>fpkF}7$w*XN;fQ19qrrai*3P7%+;&`LvQxbm9H;$MS-V`C@1Es z7j5goE+b=p_0?$T0*rN3%TuFMzlYd9nfI%KQ^du6pTaJ*pQPaK;jo~>qeiOVk@DJ- z$GvXbKA5s9VXU#`^7r3bTO+ca?qZv1_A?A4WF^(+^f;MyGX-)4vDG7FEn^L?Hmq`4 z8utlVo^+WF`~#7_k!CoU#tFE4IBtNI)0%7wDK|v*{J}GWqqaDPl~)tT`F`UWK6KBo z?y8b142on_4c_YBBQ$1{!XPE9eGmX%;K_;9{X{k=1vn0#y&$US6}N&0#hFr3{fqIX zeba5~vZ>q9R_EP0{aT`6UYw;3Y*GS;ec|>bAQzH#c{E|z`4ndkOlkhwDI<~el`zD3 zd#VbPHY??V)&h(=c8OrplH7Ph&w|ct{nJQq1lx$;V{@D}!J3|f2*|W} z<}z1sBwZHWZ?3O;9!(4JqM7LhyQgwls2$Lyeh0TK;+uRfn0mF_`wM}xWO*?;zhn00 z>96Q~73bNF#js*?>geKi3#ZsZB((&vGA=ObXRV#eNaMkALH#~e9n+$)*1=#*$SCBd z8ZSsui1^Zg?!k7JI|$3xdGG|7J_Bm0C5fnS!}+jGxPn`qlO8h$oG?K9!Z%?an^hy+ zvwppPxcJp3F)PlbY$~rEn|(!8p%{7~)m!}xQ9rilA0EUS`kr;1LmjYC@Fl=p+?9yB zJuE66iDfQb?I}x41Qtymn;6aN5}94Yrxdfnj9H10Ak0g*d7oSnzoA<`YkkR|S?=0I z+9)3yZ`Re?g4t8W~uK6La+Lit8=Tv5eu~`_$&z zk~tZ_bWogoKp~`+F0JVuNVh3N>63j<6ZIreh43$+SVLx?-m7V`Ad^MGJ+(WauiYD4 zv;5946A?R_(<^ClLM<-t_|?^Pth&APvFb*fG-N>@ne7eFF6hX%G-5o^7JL<@C-Tvz z_4(b=OFem0W|E?Z-_A>cvJ)X8lz0J1)x46^2E{jHT!#IEk|wq7{y4O|ck@e$#=A8Pu3uM6- zO@RIUg{>%@Ez}#@H8ZYClv%2R9%go87ZPdTJdi2c93ZaJ`tJUwR$^&o;GKm`ziW%i z;3IvIPsV$}4J@Lj(Q)xwsSIoqutiSALPO8FA(H_{!COC{*e84tgzqRn#4U42ENzem zYC#6XznzBQhN5wUw|p>>RVfoq!DLmvk$q(tb9|doFQ46@;j1QNtg_1a`yOD@@+& zD#5&3Cf%C3*T6rcB8K&ht|ow_6dhV?PccYlJ6AG&&*eT~J+@3c*`1%qX*=uzkw^iQ zg$`dH3D_@)1&CSb<5-PY3!^K@x4%?`LPDiRf-s4gCY(b&aSUC;Md9z;Bvjdhu_z_Gf-iy#JsAURppn z^xG0ADgZW81PyDVgmsYePKaUI1l|14b<^jVk~!YomuS#q$v-3Q$v2Lb4u z-cxV4T3);mGE`a~Usk`&dDd>c+<0wPGbsyPoH-+myu`gFx{Zk7KUcS7s{;){O;CJd-=JhGiO6L}G_*KUZu9RDB@4w7cw5Ll+q zakBIJ5Ag;1$Utt-p`d1e|69mGlw}N8o{}X=w_%ZWdnBsUU-jC~53?n3Y+Y&`4ugI? z;N6?9vC|Ftw#pYLdb#)h0i}qZ(io88F}}JUH&SDEn40sI7>DnsREt?_@ zcKI9f9b{TAd9gofywMe^6y0HeBZz*1q$K_8@!@<94Kvzb=#~(NP`u3=X)i$rPv-=N z?^tqbN{4-vN<*Aeab>qL`279Z^k^7Sn?9B!&#RaCohy0@gnyMOKFuv8R+a?68ZoB74mJfHcyaRNoO1X3zYKU2@ly$&ET_|^mv2jePx)M8bnr7#|y_-GYd z>3KUvLqo$TUE#xLPgSW%{cajpyR#l0Pylh0gTd-H1l1hFo8&Al1`dgI{vq5Yv+=L0 zx5x4qtU#XcTd*2fP&c`uZr7&y*Q0;Oe|mEjx-+k+TkzT5BK-a)z~p%30sqpUs^Oji@KZNu*lDXWYh3<&N%Ik9sR zQbB)TY9d+K3sMmD@(9{#-ga#*i7f1L2QXBsPe{EmdU47oeetnIIu4}&q?0_8AT$Jr z2U7{P1R4WC5>P4%oXD|x4=(UU zczgkzh7V)!rA~yKXSJ(Yk986c@#z8CBeUxl9B#X`$)rt0pP&5;oj29eqb1tSAX986)Nq&Wx@{#6f7bShtThG9@FUm$hRZ&pM|EGL z#6Vwt5|4tKO5%m&d>0To)I=jG@4rJst5=IpCvQ?r*xo%l+d(}e&Ju`|cR>8?jbu_) ztJ(+)h0cSwx%e};g%L3#kn;B+hj=NfUV@_vGKM#!)FT2=CT-o?ow-I`Q(|7y>zf4- zAJv+?JwBfUssiTxdzfTx6(Tn-mmJe#vD_6atxHY|dxH6g{qlE0^0I7p1E69Va|@mP zE8>f6my_IdX`adM9IdpVh*kx(!JUG|gF!iQIi<8s0<~DO6j!=R!CaRP+*(&r#DJkZ`Og zvKZKC-_zkby}_6%iVY&W6KaOzHKZ3)mKsy&LXrdyzbT`(gl5J%o2sPPX?Pu)*p@JaLad4b21U`kQ|gSul8Q{T7oMdr3hT|uY{J5U%dGhMI8Mt z3bG)J8Zv#Cw+^5r1yJ^BQ+$+cKbfmSiR=-2de@BD8Ck3e@o)M3QD*p@NQ=|63yRe& z^<#J+ZwaCk?gL|$td*fH;DMe|#VR0{Iuc$~fgDJDt7Gpz3kleJ<3+5f>QyujPs5NL z9BkE?CtWOQzis+TLV80e-AiFvQsMV!@(`0&$~z-K)s#*;Dr(f-LHPV)is&MEvd(IXSl^BFTu z2k=!&+J8$sX)3q;E@HH>f}R0Ll8J;Q?X;)?HFn))W+_$vgnh!s7i`cRsb1o3HZi9* z;n`oe5qtZvrOoR0w+~!tP-O6b#3{7slVdzARFG^neZ|64^!*n<(i+#%NUsiyk8^n> z>A5mkv|pMG70~(wg>whCc7zkjOc6d@P%F}vYR$r)%lq1I}JRyeH-H0;k~y3w00NY-Ul_O0ahyb-w{QWiE8A;{wu}=DRNQ5 zMJF;CY?Z_7*xKVjsoxm#?%kkmduxN&NpZPaWTYU!-VqqzSZwwjcGSh;*{1dgA=Q6L zublSPSLpdZomHaG0gp9C*Xrh}!~;xis}Bnw0>$h#GX>p>gL-RgMgJk0$Q8tRlb+t3 z5~Y$nhNa!}TXa({H)}n{Ympo#!rW52W#(MJjS`qT55U_J|cR3`h0H%+whWJ zF^S5!Q;VKgf`&qi>y2Vaxqa@3xA!oWv?#BUa1lx+H09XLODef|iw3BN|7Lxo{9$5y zrq&77N_7SwzO#A9dXXlBNu8;Ysc$omL509=GbyFiEQl)1M{6G6CkEtY`4^GxivQ49 z4l=j=O12aBMqanlyvNa1&VhYR}A9Z^e;s(TRiQ0LbU4dGns#Qx0m=Mn>8*ia44p8>w^mP@9q}?0jR6XAeQC zsE1ea2D0htOmH$r>qM7z{;LH*yNmvvKHya4-?k9B--*(`md7tI=S%Tj(cRWTik{CG zm8+ayo`fqw2c4#hf)uX24we5ljEeiGWeQZuA^_gtt$(e@2GYGP404;P7w>H6O7gq_04^E zK$OIE9`jp?;j^?m8kC?pyL82ZOftva5&Tnh|rlYMu#5W5E=1W*QMXd?RDFRmM170Yvb zS%1xukX4BP1l>1(i_28vfL%!bXeY|6f{GE|^2w8%ZW%!{ z+8l;LTy9k$TtKXeXMEl<{8OxXmo(ea-;828lmfOyi!EER${ ziZj&h4;>=j`hQ)g1Z-yyKwN>Kqu$%pNF*A}kPYkMh75kB#&|P01Hq7#{0J^SB3b33 zity*~z7+0)t(n@F*4if0T!baa3aLMLfLTR~ENEdIWz#bhNU*i@NnZO4Taf?7L0(?@ z92go2)@LTUrPi)F*2~J52-}X5vB|YU&@RY|0i_9v>Aj5Uh_*^59?AaQOtSzs}58DhuJ!!Ilbm1nk~`&#)B(_AEn_QD*#| zX>k~IslZsYX~5tb7%0QQzp=q3!LMr4?2Vl-I_@nQAV34%!4!*qPL2cXe-_AA7L@GB z=D66BfoW}^VT~3-_Jy>Dd8}zB>rWy)WvU?oXvl9V0WzxAfFL*U+U7m>xBfFD(!{PJ)9ii%65CVhI2;$lU;*tbjjI9vhv!m$^}g z8)FK?Al<5dW`*3BDBvq<>P_{Yf^+c-Bkk<+rAMh%fjd1S?hW&}R2A8Dtk^s1kM-dU zN-3`>M15HR5u6C7z4UnPHs-QC#eaZq8xNc_8(JOvm2B~BBLwuUB>q;)oiaFB z`JdL`R9r20_gI#nf-*(t2UE{;B*)*_R^MjhEk;Mq;%U0vS2#LZ1N_*^d?vgO*H&N` zoOh&N{9^HSoQgqFI=dlaVsH?LhXnL1axZE!Q|VCZftD{uJgm8a z{hQ8P*gNg8o>BqBDN{X7V$i;GlkkXH15`n?`s^oi+BsVG!veLFtBCC8KNcoSWOY7pHslQ-O(Za}d7S3IQ@(az+MB z;HOU-qV>2j+UZbcrGzZ9OgQRSN*rnl>IaMb&X_a`lP&?IbEOphgpK|UZ=UFAskn6o zY=$YA>nWY|C}gI>tREUd!nvXQtKc^J!aN(GgE^!>tk`aj7p;IiYZ%1S$r`&%hsdRz zfOp_|?NQI?l@*tVTXWjv?Emx^q6fD$XCA_2?p@>U#^>d94L@3}e8UYkNBJWFZjM*y zE-pRULIRJPK$;DZ6?VN}eARb}oAKjFyi{aW~zkC0FWvC-{w+~_Y;;Z;2PK0Kkv42nHGUI6MGAaXkkLRS)Zr~tN$eeH$yrbJU*|r zv_swNvW6c^a)~|a26ta=Z_eD|bLY_-S{X>6PBI1bq#6(+|CM&HjN!)@C9=r|kBVn6 z0Fa^u_wKoz1Kt;CmgN6TE6|+AlH)S%A;<%xG%yRfuGvtif6sYtvQ~Wq$eNj1b91&8 z58Q8P=FZ36#dBcSUSbG%X585oh=GqB&8i0W?zC2p_va&!O2FmxXn=c|!Tn*Z>zOM{ zC*`Y4;)c1{2U(Ct+rz#=Qm`=up9~s3Z7UtwfRYJl)UQmQ)}avbpklll?o#RZ4eQO} zGqKg$0b&T>iOFEP;3YT*Ja7b2fkFaNI|CPyR=dyj+1e^oqL4J6pw_DonhV!a5p`YG z{Y!kP|5imY<~-IO(`iknJZz;yQ*3;S@kyEHx)BS^CkTb1Pn3dLNT)U8ypy=#iECB$ z=)vM2U^M(_GD3FXYU`R=&r6k8zWseEWiS1YqnEvfCUVwT0@~T+8-yB{FX%@gjfR&_ zysdCP;D^YGM=<|KAQ5q`jir3tzz)7=Vqn!xAq=20%LhDcp%{!r+>MLe+w5?W;` z;RWXYzQ}uKn*XyivW#%N{}j zBUhra{yz2Xf*$gW<<<$m7@U_LAbExx(177q(!$O7!YMSE4*mL1VGsNUB`Qw!US>&2 ziAjHodWZS@0@^qIbqdK1PWjeC6qen?U#uDOZY;kRe5o&-9~VHlJ$kncM8?(MV4L8J z+AsKns6V3NlhiPzL3h%y*dOBL<{82VRQ z0jPKG%Z3p!_9c3NTNJBnUxDfchf#~ay}6*zdF88uFF|p`J;3%@fzv~_P<->jNGh_5 zeep1KOe;+VM8$(-C*&saQrnkGOC`t&J+B{tR^?b-f99=wf(XB=bJ;{|-m5_JnE+lJ zqLxk-fXEg>DLR+KTYGh>Oj6?zuVb92w^9j*vEvuO4_j%qWDuY3l;73)5g6T)_2FoL zKfaG-L>8!5$cHV9sMScn?oMM&I5k&d4fT0m!7%0I8wGBE09S70_dk~;s*b-3p}W(_ zL>VS*^6vH?tevklCgMHSA0l1vd0#N>C5P=nfya`pcGi2c+ZaJ&?x*vKmd5x1P(Nsd zF*uRn#t{{`BBl08;?7YJa3JxjDWsFZi)FX0_?%|sojd2JLAQytMH;d^uif2rJY}U; z`Bt?%>6vl-c}F#Fq4;9PB`6q>v2B$84C2N`Bi6dARTwh<@q07q6!RH=$D<%*A(=@* z!Kcw(Y8bn4%-|Wz?tC+Q^W}c{f=93^gFI*$T29b3P#q6If7!ARy(JWd)3XSYhT#Se zCl{1=ZYH9a3;bVdT8LI=3Jzla1|p0yXG?vr5ZMFkBkBcs&9nDXXhw{B(zk*WL2d+Y zf7~8p&i@G`nR12}7ERZ2qnk)cZn4CJS!S{aZia~}b-{Jj@S{$+`yYoAV2=?&UIX&j zY4#IJAq|_o4pPLlUS8#SRgRGF@e3%uJX?kCfa~cn&JB(OP8%~$Bp_GG=)Z;63eR}r z9&D8~UvuQYW(3K2J58X~38;B7f+{j+aj@GhqIST*2DA zQl|r4{wUOvGC4arMHsJ8cp;XgP~L;mkQ|7@HV@ZI1!k}-a>$k**v`OJXKyQRSi4!N5whw8I*9Y01Xyv-tdC^N85hGo zex&Na+I=)Lfg~Qx1wbuPevd^xKOWDMMIc#uXSP1KUDGQ>4F~_`^cS}H5?RQ59JYMT zl49ufqDO@$iqoFh8}W+u#|9dRkkv?qW85O<;$m^~z^Wy?<|{LV$NP(W3euNG1Y`S` zXHMLAMT}|A^}?)$v;?2We7KiVyC6e$*%Ew`S<9kR+0{iFje~WX7H;$HAcC7ktF(pq z{vbsLf4df|c6qky;QJQOTA zY$|;l-f%qtDJ*C$yzlp&^Y2{OzWC2Pdp~Qfd)?7|=W>indfZl0_s8E^ z(znfT``y?y7r-;dG$LS~pxTituGCgo=;9a9HGEq4dTm=ab20Bg1T zKAEdtsGwjEga?$??*9Z<_CJK&6fm&K8+*|R*%?3K1;hM64v7Z;k_`^0<})v5#|&6HdtvLW#@> zIH~feB!v)S04kVbo>b%oJ9dY}AMre*b3c?S_$$=hBKyVKQk0E%m?S)MD9P)W-4yE* z7&UYHKX?ohH}ukg&7Ws)iMMWsxaYtF^uE>YAi35)IMt?1Om~pv5S+Bx{wmtqpb*f- zF@J_nv|S&hL~a5g-WuW_F#F=N{=5CPkD(KAePVoszFEqdLRx;>-L*V?5;b~2nswM^rz$I=T$Sb7ZQy6}O~8aB2I z1u$!STmjupWwwTG*Wt<1U<-%kRd*Wxl77)RH^q^(Z_6VYmgh~~rOhuR9q{!<( z@}1n(;Voy;HTFMyM&Gyzns2NP*zyLO3!9dp}iZ}JycTceZPTRmum2Wn3&`bh=EDp)I6*4 zI+6T&p897UvvxMASy)GTZx5=^EMh${ma}|93<9ye5b722~$wAaErWC4+(2a)3Ub0>x zdt&~GdJ0H9y?>vnHtL4(7O?N+J^m1M?P=PihEgSt@Acqk!9S+Gop}AJR;x^VUvCw_ zGjNP9tx=C)B4T99EA>!TRXS`vQq%XPAz! ze=p*)A&iD1L@6TPa|hM|MZ3x&49V(>r)50pxUU45H5egU2b6HI)hqyDA)3M_7`kcA zC43v$&lsZaOE3PV|6ZQs>yf-uimLHgHpk9A?#!c)O=nhH#7FC^7t6T7CVFSFx(2+l zC9cPNhyG5<;wIzwF}s6RXUKeHmiYI@&&ql0K>LR@VQ)9+36Pa0v25c z*o%*z)S%>O_|c_+MsQ`sF^!2-sWmNQ}Qs`~e?i&C^!dm{BD(M3& z?eb-^@362zqA>eCX_Ih&NB@2ucjPJ%2s>l0rK^WX;+r0*kNb7B zLAeG8GQ%q!{Rq(y1t~*juL0P`5J;`d&GpyQhXjeBoBib9as!s^-(;)KCqRV zQ-X{fgpqcHWiA&UdSBmlc@tDlc|)`ws)H7i?}P{HKDb}Y$RS2}-MPiQ?#9XkXTjs? zZf?lpw0D&*uv+Y2diMLu`C1X|o}#Z4&t*HcJ%e<#Ijl@aAH4EA=V5I<3QLfpRl+K$ zD3LDO<;|bIb94`n_BB_0^ZkBr^mdWl4)rFb5|6BVU=T!>T>BGY>f*F%d#bd-(66og z#vXHKA(Z7qd*FwikLnC5egT-y>)%~I{ugi@r&rCJe#>S$foq8k-#w-WV4f_NJL8k7 z9?}mc-@x^6-r3o1?^GDvbY;2$XqZ|$X|mEykOKh=Rp|>37O6*sP?>6_*gRx-3Vs8| zq)`BrS>KWK-&5a;_N~p#2Xd_Jks1fSeGSZhCKsl9NFMribfkvZ;L@~;_r0VOONhEo z;T(URJ42{U2`Z?rgqZDk%|1(fm_&p}<^#pzFuX5MMl4g;1@9;8gOvB76)Ye=P)40v zz6=31%MH~y?J+k3CUP~%h!d%N;4{CL?uG=q%pZ9_MWxrj^Ej_a5$B&We4Yl$SvK)87Yk*^AC&E$YOCfAIwfei z&?X`pQPddPf_70HyS%%(zgTzMuZP!fwwslG;SLe@queWwzWrcxu*iuv3B4wAO=aVA z7$eRU5*C5niM`?UDq2lsGOt59C9)zDe zP&c|7wT2D4A-GBjY8X2W;zU}ltL6)F4)<|-0wd=&KY3L&_GgVEp3wOwrz?79zpD-k zrK>eAUunPehFn*V_T7)#dpY^9o(cwCzZ1b7%ZB%Sx}30%37;T$jnT2=Bs@Cp=(R>& zU9N^TxX*|8b1l_`;Jgh;P7B>mJhs^yytlLk8p6>34RGm~N}fzL1%&iuf zMO)Y)t|72=UJ*)WJWgRBUl-yc1&B51;e2QaO1e1vc}S*Wa*6#oFj8mLN0K=X0RaeMD%-4;o`0TS=vW;Yk(qrY6OI< zP35Q|th$oR`KbZ4)eSKiPQDR+pZY1Xomzmr@4NDY)2Dv!pM$Nuq9#y>d{@W!lOg^$ zpZc^}p|Al60EtN3gaf2%mIF0cAJ3$2l2z7*O`#B%x`nFiK*UlCNqlIXZoXSBb$dmH zj&8!{AP5gRQckHTI(aQ(;#J!^1Zpc9Ve7I0O*3t3cS=@DHT!;v5Jf&IM!W~* za7)@KJxIu;MFX-r#;@nz_+(JhsL9?~B>$-6K1hh#g@1mquS}0;?qH0-xum!GN%|eN zsn!YQw|N*2oHQRnc?XdF)%G^Jemznhgt6}E_5M6_h>1+wGQ8DLWqRNyKeQ%HPR(a9yjznfsQ5^2G@B3@=d|+dEjH6} zn{q0+7bPhpH+TB;xX2C!_o(jhlIqvbyUA*9qXX2fwXFWm3*u4FtbI9eM zs6by|E#8T%Z>2t=bBJoGL59JgI#e>TD^Y_`6Aaosms6Rv5~d%ok?DigUj|21T%iV84OY z5|I44)~)bfKUN5zgyBDJ2!c$`{}>X{6?BpGt8JUEN#1_&XYeApkV>RK`?kooD_~oS zJjiRHDm|CjFSJA4(I!zI^c~!X56t=|FloG=*Q~D{nw#`F(2u{!!3E_FDsne+1`JCZ z_##QvxbyxZz^ozAb*(gRUxNI=NupBENv#m3as4=t6zFCGR1JWaYCB>(XaDI9p`A}S zxca@lX-y9Ddn&juG>L1v2I3NUA#{btABGW4Nscav(wf5VZ1gi&V&30kB9)v zI)n#Q?UqaS_$m7<_Nq`N^ZwhSdN-O7jff)9qFdm^;rm!l>;MfyC1=c2&?ow_JT*-t z<@JHeK`6QkYY!PCa`u%Q`N3`fw#Ine$@#Lb5&%D9)uyIQ8}YSFJ>?Mm;Xd%T3hzOLX9_gH$; zeW>p6qy~r5G#c4*aW&=Dzl&8~b&2zx()FUjv)>hB)`zTxnEUtc1eZ1Tv@Jvj<`wg} zE?h=<9xq)IUFlX?VSh?A_;4HJ+U9hcHTxI-L;Q8}?MXb~@vEJReDY-izbVzc{pLOp zgbv!eEHENU>F0dmOQZ*=MOC_-q*l0SzECD4)|T3uT^K7kVimL<|#~SK0ZU9y`b*&fIBrTQko2%_hI$uKqj3pMa`kHPTZ3 z+i)VEJW2Q%{#4fv$F5p^6DNbk7i9Poj=C;{b%dHbUteG9bfHn+b3&&kww`?zj+-7d zI}Ty(bit4P0KJ+uG2nwTh#LDZ<|nMLU$sw)50{k zXZkr|`W`uiV;$Qv33@MeIVTj{)J(~0dKyZptps|ro{YMV22o8gGHPrHdJAtlZs?4a zK!lJ-a~#Spl)AjP@x5a5bt-8O`&Cu6(X;spYuAsambbpn9&E>zg>3B71dQ4R0{zi= zjMEZ<8|q|Ht*wONhzd;V3QWhC(fEwVR}=gX{7SX*e68~C-^M($k+{;DU5?+$Q<=mO z4Q}Xq8S9-5@_I5${E$(T*0huTMC93{zpCg-LfoFuCX5RX`9Pizvg0r$wJWn?Lha3% z*QG^x>_WVUN6UvHV+n3QS}+N1WaaF0LGG_k{#J$0mh#5NHz9%O)foGUpmvAB^LWfw zk~xyUU;QzUFf3i?4T72K@E}{;I?6&+Yv|Q6L-2h@+7hdg{vrW>3DzzA0^x74RkVf% zT=N?%@>dIqtjoHWi9(%YFyLsT5_0oARmT!(j#(gHYJc8{;t~Bd_f%fHM5@6Pg)0f0 z5PGN%p2tr=tuwp~OyQ7{mKEs)hv@I%HSDm(H`l0a`AU}=HrGB0@jk)KnY!{RA=~-V z(P5GesxJ~{JX2RDpj6C*1hmUa+gdzy%(kDEYGaUB+6*ceyTkQhij@ALRT_`Dx6K}x zScIPcOGS8QfSEsQ)qbA)t52&z#+LXm)u4&T;Exgom8v&S0aG7;idWfn6u)##nqo)$d?SAho&+omei`!AY7!1e5JWo$?z~j{(O?B)0bD!<%?Ub4GvFbE4iK>+Pr<3ZB$&ip7&S z33>xRU^+>D_rR-CY!mq^iTm%BW^jT{o5z0J_GY`!^=PTkrjQgESti0A9U1v;8Ew^o zLD+WBCa#xMY){d?-EGAD7Qn<-F8*F26K`JKh?iKA8C>D1vYF13KIJn#Ju?S=Fdbf1 zZ$dzdfWcmA=5ZYk*kKE@#fo+j9m$Mq<~J5pw7+qqScT>TD^rnBvVxo3*Ht>cd{jEQ zmcbveu=;aIM36N7^&cWa{?seH?bChRg@sY5@EylDPOLy`AH!UdvS3Y&(i3e{mddV~ zgH5=8y#h1de1y{JZKE1(|HF;=Yq$Gy$@nj zVwXS|uKye>HN|Z1?Hb(pI#FgKZ9yuEaQtx0Aou9M(}*KAY;d)1y6MP1veL`z2Nduu zXsOqS>K{Yen6T^R!YK4FVGm5_&EAn8#;h-z`oug>h!Dl?z7Y!}e5KvUYj(KNL5;d+&bnh0^`J+X-6jCjgdyqKMMl! zLIdeTM-gQHATp{5l2EKUtayMW@%<}}?0#{%*Ael(kc&MleaI6CeGUH&mWU6AaUk-h z=2+?o&V67eY|EtpJWp3jS@z~Da#m$rR6OhtH-Tc$T@yzsV^sS563sLTl}BM=ll?!3 zpCQV~fkW`6Brb7gn6l}-Ra^#}&XgcH^b^aGH}UVfu;d4&lgdAZhcVdQW}GpL=D_C`B<6HYsn= zgL%h{J?QeoJI*)p#--}aUIIbF6emQy%vs43ypGm@`W#XWhH4_3Q4K9Hv*$zA;%@>F zd!XxVg*!U)G4aS=2qws)$R6$kn>bnSpdVVnsb3x!3`{Pi9YrI2O7+6@HX)@m)S}c7 zsZ$2Pv8F`~*>sFZAsV*DW%;}0=Y{Ma^XlNeU{OYG_$&|+*gmM<7FaUX+HylioSmqm zO#Rx508Tg)3=cMmwLM>^j{^rJuHt=s zO9-o;Y`qk16NW2Py20?q=R?v&nrL=cpl#SDoM&K0izp~jjS|5)#xvHyZXYQ;N5rE% zw%!BtCo#w!t4dOq*++QHsH-~ED^Fk!sc6T z^U@&9IlUd+(SMPGjeM&ESY%CKyMiT=wk1-opA#%d_?0(^o`MetELVi;QN!QWRAo!W zlPp%){T#-kL`w*ynYROLI&BZe;TKFYm=a=^*l+w77z1Qz6DA$WWQg~BI|t5Tz)QgOMM#;m{CBBNUP`%;KidRE%UrWboEG34GP&4KE`S%px>kQNfIx`)^LE z)C=T_CJz(F<6(}Lls#hWY^6Qz!RfdXuO)o7f?tJFx|4rQ#;=Pf6Ql;%>aG?gtAn~I zWUB@;*!}DBL3_xbbn#Jwa`M^?Inlqzo|0{t0h-6))Zr`30`@O92ww@nHNXzpBenBL zFSx@Dk3AN#XvDkS4U(4VfUsd*#ay6|1PV=dLyEvVd@vE;%p zh9Nyt!dbzGFK}Qi(T25i!shp}W(A)hE+IXV>b$;=y&5mOmKGiFhnv$my4HAIEHkz? zr3w`PA!{@r*d~0RmQPP9vjqZaLOK{w^4#ra7{ULFmU9Yx=tAz4mGcF3itwSO`9q*5 z@-8fhoOED-lWOrRCJC5ep1Sx4w(sh+C2n3TpMcXQx9-mZu6|RcHH(p=vxgqy$aJpQ ztsnR6DYtL=Mt*L8clKIeqblA(8DGo84;uzjsV=~`F2%{;T}3r<(Z?ZKlh>CAjV{|u z!t$Ba9aq^Aa#!``GV{6fCG3QlpJBPSyw97L(d`wvsyjdDx^Nc!w0gL(sl$TESKkV^-f<>3`502B+wON(Bl>x zLKH_|e}%u@e#;S6nL}NhA7txn58LSg5w5w}pa;8s76oyvT}Fat*YF&={nQ?@nNioN z1u($-$X+tR&Nk&|xP8A_5leQ3IvoOia+1SFr)E zux8Y+n9>t4;s{DR(Ed@0++mkGrib{k{z4FCnkpF@pEysbHMdJcOP*;wEUJFA?%&=Y z;OC^<_7p`;iK{bH0UG25iE4}qP0jmbnClq$zQcLVlvJEf$-{M?h|oCkc5k1X>1F}^l$y6|eXfR)7F z5m?1KvC>6bo$efkL>`65y^)KpV)$p*=%}yW;toUmN?wV_j&y^SD5-OG;I28OC9O|j zPeGjlaRar%*2p*I)F|tbJny<%XVRQM@)A-?#(&|Qr#TjsPzj!iG2LfY5*64a|5pZK z#}S-2sIKg*Wc>4ID`FH2;mnExChdJdIP$yLK3PJzd*nK7gnkGhR!= zttWN;i#yu-Hx-OSr-oC(r~>m>zP6yvWiwQtYI!ci#+Ff;aUOzJ; z<~5DBhw)dH891*)7TuI6&GR6^j<98eL8@ncSZRx$UTv!;r_B_kVljCuuO>$K&P z4oGJdXI<=U4{yOMm(@W@p>dB%1nh?Q?NWx+Wja_W&S0}29H`KRdP%WNgwziEx&Ckh z;%RSPK}a9Scwdfv;Ozoj3VP6&nv#fi)cuAcaW%Y)V*{OT!k< zVhxz=I&%gFTSiB>`LIy>E$b!Dm;BpU>~(CJBw){I$e@80XpCiqL=;aL|1L=D90~dN zF6q`P-ejJM)-PZyAayJU;dE>C)&R()R}gf8bXiTfvx%mRJB%!~9m^BCpS)v;)t^Y* z!H8CvV6Q|WJS%t%4Q2D}jQz1z#bH#b>7K^bVyc_EK4*6<>Z!NSlVQ6gDW?(KB+>Mv z$-375U9)`S_P2}BLZ5$*^sPPj-M!&hmwBvHdD7ni>yA*Csz#Xq$>9+eMKX&{$Fn$8 zH}=me(IzjRxVSE(ORkW5?2@+Q z@o%lyS*eRvSzXZ|hs(7S!4A}JQSsuSS2I=1qe%=ZCQC<=Zy$_ETjkrDvYxZ+D&D>S z&7Zk*aDQt(r~)3RvmuO#HzMP|cRM}Ab>ufd0D-&5G)(el6A4MroQ*SoQrP%VU;8XCAX!2ED0k54R zdb4=$UbQ?qf}i@}c!>9dgiOp|dwQFO=b~SqZ8ZkUI~hb4JI<4S-8g1BF7d#kFe#*Tn+=ylQ`U^O}R;}cXQ0|Povo+yB z;egDPunMbmXJ0k;6sb5owQ-XcniQV6o6}e5n*)(x3_HRkzA+I^9X^a{%pch7jXxHd zH9V9-c&NVgvRU-`wQ@?kXeDij#mn)K%8Q{~+R5z|ZvV9r2dYt);BvfX$?acb{d63^wn!B^8^bJOz6C*bF);%M|j35ZP3G`p$K# zR^^^e4=lRRuA7UK)NX?Qmdc|HXRv&?7*ADV)Me(16BhA#QuysJ2yja%^1VM-mW1ob zM!-Zkwx)3B3-p5{t#Gp*eZELd_YtT#1{?~r#4e9Rl>txvyk9phsSFthpBh#ku5;OC zR$HH?*i9Z)-e^gAAzaNoQw(nw*gnEti zQG;Cx9pCkV-F_pS-M5@nTw(ll_PQCsCT{v!l_O+-cJtFhLfLh~r!4xE7R3|!wfUTL zY6zLji?A>&8gMH4_QD!lnI_@29on19hu3o`Cy9%OO_Bj?^{W#-+I9usD&zJGPT+G+ zS*rg_7>Le;qOw2>9D?6)27lE*A;{aW=kc*GWE{g6J1drtF6;oHkb)}!&53JkA*SN2 z-A%UNWN7t$s?K`L6QbyOeltg8POtgXFP`M)yRRNow)Z|~{PRcf`roH&)hqHQ3oDJ~ zox%d?d^7adeEB$S46cOFOn%{&yco6Ldu#m~`)mD2)HFmRm_S;PW@1SfVel@~S#5{s zsXk6${6~CJR1`V|kO7(lx{tW$40Ll^+h-!Qpim`4*Tv#dPN;#2g?NSza~N;k3?0IF zVB5^;(7hl3VQQB^s*Q!Q?lL)q!DasB`0Wy=tLcRytkwkaIBno@D50aP}zGvRC`;^|2d^)yziVm`&+7Tb_JCc(QCsLc@)SzzCO#waauXCW(Y#!l)TvrBioByLe)@hhWVLKn-)y`zV=L~KJ>{;YaH}nGssYp=I zbHuO=U?{9cEt>u}>qs+Q0N-nmqM``*MKB8!gaa5U-?v9nqO-HJf!*rrzekOx&m>@m z7Z`;wkUR6;(6?SBT7|@|M7vW|4>sJQ^A>|9sn^Nrv7(-(n0so<2}rO;qvA`ycnVE} zQomDhAA^#qO>{4OMJ;^}k#MreT*sOUnI5pM1DB~}w@*j+C2@SyFq4YMp?3X>iw53v z<(ZH)z>w$xnig80)i|fwh(u|@HaKv4ykq* zh@w(&*=PJb71E+MhaxRBPl#w+NvPQi9k+_}huh*t0|T}vcce-1S@Q%7aVzf+aZ z;a#O@CSTFlq$y+4Qex>b`z1U3-bDBP#1TsJ`c8#*6qkotx7LH;`==w}!z&alLdiQ3 z^1BhMG%g?O1muRtCB5lllnyZ*!}Z$TurTU5`Y3ChMftB zR{rI#Ko6twS(3>_e0A@!`|qC7*lk5auJ@)SUrb*dnw|>1_h)}HQW4TzsJS8JC;MyREuh5$b7XLt^ij>BKMKVGDc*T`h7 zIuct)r1Gn^nZBCyj}-Ez;tQEQ+wJ!%g)S?n^b^v^Y-mLd3EKK|+YafTk$nC(!nZvq ze$vVY_u(n4^`zHsuTgJlX=Hq}k*2f*M$Q?xPGM87;*15t-I&G+Zs{uD;lnFai@xwL zF)ACbMq%UkK}O&IS)Jp*F=?*$cxdOkt>XZ4oEZLXx>Pq{;Aj=82L_Pnd9Gh zB9d&SKdx>dp~1jr(dd<*hr=D#ET&(3hCF5m%W@>V`}Mrz%1z3a-DTi$`uj0a?cCY$ zZGH2~eRz#E#{&y)(PL3(N77C3d6P*~%1V9Px3~^}s^j{-w-Xn4{g7mb4cMJ}?je$j z{UL^r6p(5WX|Fdm6gyCy0K~X%CT7nx9{xv47tRuEntB=yzZNYzD-11;y(x^aCq3>nosC`T(Jxk z;Wcht*7m4195B!BoITshLyp_F!|ZptdUn-YHpdJcEU6JZ7m39YO#C&aOAaiH3Zt#R zxfHs{O-T@MS6rD77Z)OBBrmoWX3)+=5hFGt$~OIRwyhgo5SBN|J1)l9E`!tuwUXU` z`!a~V0C1#!+0H3YB@r(J{>TN!?K#ic5zfxYsmTt2QXo;-$9-hRmjWFQ6t_E(%>f7v zct*U4W0)3Rq!?t_s((M#(CO?2I^wePJ3*g%3z`pm*?xJQ@2u|;II{pVd9L`{G)Ml{ zYC1j+r?=8@>*}8wCLSnO1@CP5U1)HyVapa^i*vdE8^k_IRpOxkl(|DpWTNk3Ol9Ez zE;lleLR=a;K@xIH&7<_FSQ*XIj51tko)toR-wfdYmqAyv-+vj4GED#PUb{D zg#P-m#)V28!!;*>P?=U8BP-dJm!s20jb`NL<2#F4spTqo!1s0rk~&?WP_S(p!vTz7 zvppQ5sCaM~1~G7O&>(?t1_zXM3Q~gQD3~(qc=KObI6IQ_Cw29RcaM0H-M9z*9O$ET zQHw^b8WVvjcj1Rj=*{Np`!NK=QY=2gdo)REv5E(O`7dcsQsTXoXYvCXw_7xpXcg$ z_Ozkdpf|@x{(|aA-1+PkPnF^MTYsH)6=k%CXR*jQ&lWc~>)`}wJ4MRn4)l9URhAuo zU!5&}-``FKR)-B&9GDCo;~Q@O2TzUF-+)R{4seVjTm9Yax@)mFnG3MP2m*_DO4Ra| z5dF!rLdb#67U;tcae&unYB(RTv}|3Iejx%N2X%cd#orqHCX3Ulged=a)31M!D$H}+ z@-p|>c76J6q;aZln$ht6bD>&}1PG)=uk_b+p5@d0KxHDe7o==MX&N3apWn{I5+@U^+Z8h-g^`6IP56Y2su)m1LNUo6T`8W?BOtuCaIl|}K87dR!kvzbaI(EWuy%^SFu27yn_#TCgEW);H$Q2C@CZ zf2w~4XZed|_HI4}rxeC%Tl6oe4pV6T25K(wAyFh}!RutOU*Ff1@|0>!UXUTdl2>XU z3-DNyvTA6?u*4;vA37|D`|_Pvrx6ugGE&MbkZN?YJG+E z7S~j(v)WrT+wJDmY^LCm>ku!05eCOmMGaNz5bc5BD{fw zdzo&`bxwNA6QJ)lye>=UFLZ{OGxczNo4d>V;{^6_CO(3k z)pKlyKXfSf0YnL$%3~`$(pT(wsbcAT!l68YjalkqI8O3+VbtgBJAwXU84Y-ipb5FC)SEBxO@1&qu z%x5`^_g*;cLO6+8+SkuWXW8k(tRIfy>Cl=#zDy$ntNX)tT1@VZ_;wV%@4S=XihU79 zCW}Js?_1u9DU_o5*qYwcc0c88_)70=lHI|Yv#$P6a%e_C9qSXa|L>0P~_4w61;7jQIE^yOi{#(j?rCbcF{!@+o%%2WpUG^N2StB|_2@A&iZJTd7aDn!x zG0F6~uY2Ctzro%56S{oY07lhf%}^AXKdb4FCN+8!r^~@jc*rR^tvxcYTW?0ewk41b zBz9{!{`F$zt%t-7;R=2lz-&KB(GSdy18|MPC39uFO*qu}&mRr6U2II3pI3aC!{d$S zO81HY^Fx1r^P#Jp><*su55aHKnf!4497j5-_QAW`Yg43S?A>o7^Zs>qpEcsxm zTZPO2;oHBZh^P7jnePupyA7CO@#phxo9?OXDNJ9L>0?Sb)G9RVWf-rk69YN|;FQ$Jbhc={yHgY31)H<(g@MirIF7iE*lb+x8uJK%9>yFybbQ+VxUADzyts&$5Fq>w~ zzl3dHWGP&<{O2s9{*j>8m#5EbyAU2(3x$&kOD}t@<;~ie9xGd+f7Qax5)4or2^Q1__+$rNinDv4 zmU5uhE!MOUOUWf6TQ3qVOij-QiC4AgK4BglZN6;FdH>jWPDU@RiW}`kO_>pb(caBJ zQ~mxnS)QNCpzueWu$r@DC*V+5Je`-ZbXVgD>QwlVfX+}Dz~agZMEX8wE0G_PeZcTc zAnDhCw{m0*#3Nvi``GmVfa;6q6|gt{)%{66K)E`(ETF zH0{3+063PMG^NhtKRcP0G?krzb^&&ob5vLJD&T{COpZfmYBpTY>M4kksV@-orVPtX z2RWtv%_B2f@-zNnW>=YmSbV9=*#(C1sGjoMruh{0#)0DfpNpHG6)qPgkp{}U>HMsv zI@#JiWRTk!X$>XvdiX6ctM;doGy#MuHGC~`6Xc&7GW_V87z+XuX@OAdL?-Qg++>3#DKbqlPH*+KC+ zJtOzCIpRPYN)Vwu+y8lMY$J$uz>{H>%u*Mmz2cR(vmP0S`d9dk?@&dA>%f6$Wx86y zkC!o&DNQ-qZhgt9zS?09)kbS^9h|0%+bB65LmgiqM8o(){pYxDUA0|3rf%h&3y#cm zsYsdCM-%72grI<0Ut3{@O3ueurOMX#<&FB#B>1V?E>OsGv*HuaWgSH9`%zl@uFaAr z3LD4A>*uhClE{3}`$6=|3NN-@-pg(~i@uwR76y(eNv0|_nu)cJaH^7gqLfnYxc(nC zCd6o18$p($Z4U`5b9NZn@WQRWrfpkSn`@{km%k={y%sL}*zCyv8=Z}xg(OOb zxAOWa96_f|rNUuJph}HV%MU4jiE3u|2--gD;5VS!r1|US|7I1e4Zqy>3v%nz-%jux zfP+WI4wN)W`tHGzC7+jmNehgpaB2|*uxj8#7H)!z-R3KrSixufyi^-szw4j9J1NcS zDX-@}pr>6p;2H9fW42EXpSp4fXp!0+vy6tY zZhuAY9^YeIYH|esgVy9RjT(&w56uRpfv5T^O(>lQVCkej0oP(HScL>`(u`mAI)(M5h zOrF{6)18&#pUJziIpvjVI~ZaODvb7W5}mBh;)i=Lm!j-FQnVe5F=nOQ)Q5%dC$o`-E;$=sH_Ma1cSslAN|o+m9q@PbGc6qz3#zw4pj)&cxGdJantrXt_K-V--YqM}u&aG0?9p zIbqbs=FdBjnx#T!Y{FHbVoxCAP=WL7fC4{r<8JW*6S;lnxX z(B0FIW*FNZFDK}JEo#6ht8&jE8IlrblIT+8# zIlJibte&7WL*4jeLkwA`bkRq%23rN9URUG-jE*$Zow>;K4;Y+ko8X&i{tH z9NqanVsB_I&7#yBmgp(tFL4g+fg`9D+x<%l@Eg4<2$@Lu-C7szP&QN*k0`X}Pz#8z zBYXJB4N{=e8g@N22z@8Ne&h^D$muTY*NqNFh`n@iM4Yl@{f2}C?K=HBoTd^myoAK}i z=hecrCmMD>Vb@3MN4()ycnYUvtIb;bekJaXw>0Q?IciP56M+H$Wf%c(mM7oqOL9VO z^`?tMAL$oM&>;E$%RBhWlwpy%^Lc7&|E3lp_6@*N!mRwJ;!a;Ak0@r!KxxqyPxM3Zb~+N@he%(v~~G1+()Y|b}4D@X~^XA31CaW?Y#rWY0_fiBT>&Zj)-T%hOeWI769SeL4H`8rwk&8pbs z)QF`?q@+DVEVf$hT4h@GU(gS9B(XU7vt$?cewmQ`baba=6Bamf3lm!?(vx4|8eR1QK%4l zQe7>jphYs zjYhvI#kvG~8`v+&)Jj`qw;w_+UnSRn3Ex#E_m`~(=ndY`ad%Eh49t}Nyci_ax4E9$ zoZlL#2v2v0WfN>fLHtg6)tEYBvY#@TQY=TRqUEpOxEVMDL`Y;XzUe75t^W3i;V-jJyhqmji+NN=;!wpcQK{}sy0RyzMDODm0tlJ9 zL?6?qOQht@>vMiKga81*|D2JO3ddO!(cqq|!X+nNAtZDhXes%%phLg13*%{UlpXBB zB+z>bjlT12=xDpsvHBoKLou5-Gg{VZYOg<*Z ziH}9Pl|i#}c1MA4F>PGesP|>k92)$`d>?y72654{<;${%Sh~l|4^Qj;-OHM9at5e4 z#1;4_L8-(H?w=z8kbO<~5jMpwzI`st=nezGEvsSdD`s^d@wuR8&~{R_L|HxzZrbPK(0Z8x;eV z(&!WaudLv_lodX;ri+4ik_JFo`9bacOIyp!@1(MiNne5Ml5%U-aFs{buLMb_2@>!x z?)YC=yZdZkZiD_%K}8T}GK5N2e>^t|zDZsWio%z3#6n zq^6{pHMhe&H;(}daqJ=Zv(X%62H;y3FXAX}3`8C$F%3;cR2zC$3N9V51eBtQ)$rz` z-U^U=35)n%7`h({RV+Sgk7TDQV`cJIBApTl%7CfJA>LZ$p<887%o9vILV1zLj5(VT zkH{SVhq14Kin@u~f5k;W6qW{Q*ac~k?nXjdq(P)aiKR5>$XR1pN}l%+!?6%>^2 zlvu5)ln{QKkI!)vQ^ZZMr-3zgRd3GjMBrD zUpWdQ=25-am5ndzv8G>EyR@^EXKKnzoqickt#P*ko-nPllu(%uthn-oMl8jHM(frHS4Ydm#Xiidnr+%#G!-o7dBKgnng8ooBA4N_LCSFS+Ir0PO4 z(nOf8^!Wzb=L{<}GnvlRsfBcjJWQ?jUeXv&6B-0Pixp+$O1Wi4U|yF1S?c{~2y1WMJN;tImZ)3(l8}nOr z)NY`NG$q=YnGkPB54I0C>IvbBj-8e6LH#=({g5B$rG@Tlzi#EQZrGrP0qEQtkT$1I zZ#R}O!&B=%P2X?xc*kc##woKyL(gp5Xu7f*{#4$J4psS3Q@Ypw8jGgvEIf%u1u25MZv^&c|G|x@n7{#WAMh83l$N5Y)|iO|@03>5c^yf|@!K%5r1Bq_ zYZ=+49;+XaPCHH_BQ%TOJEzMyaa6p`NhG@=cokDG<~!lq#+uj|gD_pB*W0@V7vJm0 zT)Zh-=_51PmG<6u*+AMMql?5bBxlWI%ow^~2Le<53rUR_^L#<%NfK(;STtbeiLZEH zk;-S>?tQ=A!9fz>{!{8n6Y`#2MWfNaa#CJ4q3%;hPA&DFY zOgxq!VgO^hGGG==;}`Z%JUmbDFV{_e0b~nm5Xpd`c(~rHDuG7y2Nx9sa|-`cG96B( z$O0kwXzLsCkk-*aWwYs208Fk-r$kNg&gFCiD1L&vdR*Nw7_?a3Efl$KrG#_VEZ0>7Suk|d8q41L!qCdH!JN<t~f1D6!8a}m|$ zUi`IHl3Y(dX)UZMU1nvz-rIM&s3#f|-}G%b6avml&)L1w4$Pso+U?gK9#iDV@sl3K zR|Dm{J;^s=u^eo|!u&Dyb-{m4${0K4OOLBxPI?)YhWiH!b4LqT(jbT)VBj3GBK?-M z=Jzv&2)X6iJ8SJC)b4NcPHzwI=xpmHH(O#VKDB~NtL!9Ey)Y>!S+*^3kEA@7t2W&3 z?seH6^W8~YjTe_0_;w%!GhGlr!YCcuJ;Umqo7KFiVwLpK4vx9~npBa=n zk!-h&was*N=&l%? z8>T}Zr!+N}y|28nRTXFYjXdn2Ge^9$r9V)4EU-alplb}i!KO1>!}69j+@#u7O-UP1 zKTzJ3#@)6I^SNKCB%%ECGcU>9JLPx{*;E9C7?1IT0#ocGUm}($50YztiXnlJ@djG2 zI}PhOs7wVCB3E!1It!EI;nEe~;_TmfaW$yxkGx9MXqgZ5K7)CF>7X$k1!ZrG=CVff zL)1r0`dRr|fM+ucm3y0uS_n6|&ljEYA}UwMeVd(o+@}sa@RCLIkXyvj0a8;%3nUR; zeGlGB?+0*G*Lu=jXb2JAd>vS-`1^{&-5YL?$tSOH{?wC1mr2mtl);rPWlIb2TtVmEofIMCqJifiXpkz^rujVNM22}T*rsg5m zBIg8Y<25tqjb+fGRP#{3N9M=Y|KR{=&_sOglML4sFs9@EE)%Ksd&R(5^;jR<^&yQc z0FxkE@d@;i#}Wqdh}>Xsu6;q*n@Ms4TB22I4&Zh0TUQR1?ze|8$Iu^^i9KGmtS1z5 zw21olJpnW$y?a@TW+J9*Ie?mA(v>)E=vnuigAW&jj@uWW!jG_|Fzh2ZD-(KnXh$(2^;wJ$&8~)9_WsZR=ja{@ju> z4$4QZ%u9KSVP=Tf#v+mEY+TZmL zpveBD7=$ItU=EB#(ZWYlco;bi)=a%U&lAaq9Tnf=XpNNy4t~%B9glzzA?wBVek;LV zeVg4ir5{rW^@NN#>(QlFJ&#!(zS;TuV?!#L@NTFR=*9b;zt977)LMtE(OsZrKzu6B z>PQ`o%Ue;4%X%zFJK`}_xW7Jb0GMwWStOL`d~FWJ5m*_3Fc5-oQtSaMt;t}q0$Zv?X3BZg zer{=LmXOixZYRIX>-3BUCqN~K}VVek8sfNC3kzWKRetW;4Y?2nVKkmYL^ zUN>u3c{>qn^(WdHjygO*g&KwS>R75hH4yVWNw^+L=&{3Gq{}g># zx=SEm@#S}|oVdZI60XG+nUQTw#zeTO19clAul)k?m8Tk4v&R;Rxr`#8_5WxS*MZ&$ zPaG~YkNJ4s=n*u|8~58s`$76%t32|bx-9m|w#5P{o>~A&RLzomQb4!%P3Hp|@M(vf z=$iy#bzaPRuRp}Z-38u5Ih%G|&Lk5aq)L*imp6)(aItix~ zNCZ5}3}b7tls}2HfMsJhLV(snD~F_^SDpJ-%f|Rr53mv{5CsLhINHwj64EQ(>8uLN zVBBv?V9}EANf*g+3nLQ<{&HUSSN@aHi*1k4D+k5He&Knz&G-FVkb4QWr+#oiY8MB3 zj?vSMPq+yPy8$y?6%fw77tl0ZWHh&hlz{7xM8=sd7*f`|>k~W}eOqV)_!1`?yz`#} z9@9*iGU2W5x!drqnbw$apK6%g_F){W1x6SRU)i;`TR*K0pJ@{?4db9()ruzTxK;#c zzshyJVlL_|9F4BPu%#6TM;L2wS|ZJ3TI*|nPG+Z8kAApYcGS@Zz;i?U_YM-;3y2nSFfUnD9 z`_mQ)-WW4_VD2JFARwmrv?CVwup&3^rko>-Zf@wE!sBe>>*`ITRq`i)8>J+m{g*=r zi!KnmLVcFO_UNbBpEH#ku8!9A_+0xbP5k3D`MINhtPU^O?NU7V(RAzhjEEX7EDz5) zzt)at+GQ%~OQr*#ss7%*zkMb6jt<(bT@FtR09o7}9I7|^QWK&cFvVqo*gGE$f|o}O zBmXwpKb}Q_&KZ4AmAc0;2%IExkcoO9j*zaTFF?%lak{KxOp_ds)3yKg8 zY=O(gj%#`%hdaNpi$h2c8sw@bBs$IkukGKD^yl?dI`EL5i{7uk!lue0s+Es8dR1u> z$n~6*-iB#;Cteg#{a^lSIvog@ZTf^%OAumPL81n78H#C7nH87$67O&{;8@kpx+>5; zhPv;}g^d|k-6c$9VbmaE#-UA};r#nKe*{$0=r>Zo)q@GvCygG_QJhuhwWEJCGL3F) zd)~>CH41(0hX^^E%CFG0DHVrul~Ekedy;y0y5c{N^3M_7f*6mT>!^MyVNGivDdG2qB*Qjz^!nUZ)?aH2a{S&QKXZHj-|u`QAU2zrvg(JOzwyEZy# zEhBURY^VbQ!Os;h>rjpwGsGu`H}@#Nc~ic451V3yte-c?6zy)XI4D1jjr2Y~=_smJ z`S)~OaDxi^oT2~`Z9iPARs{vfOmiU+$T#^Kkj%hq=V4$OUMfu?NF+Q?cEcX;+FB_Z zzp~6N`uHC01ep=`y>``K^7C6+45e>gJssXcuKrMf{paG@!NAb_zIF}O3&1baab{;i6?9h4}aK)sIaGOkBbfCui*FpV?@at zBAxL8zKW6TLa@;+>%^p^G%(TtY*BFs>7KXtEZk0Wu?4eC9x)cYzybWsI)47&lmv2* z-TB(y`glJ|9IAW+E#vV9792p7DN8XzlT)suAy4>;QD7C^VrpX%l~^wtU87h z_1Q_nyX*`s2^Y^3T$5YJ-Ex zd#}f75|F%O^cI`M$3J-(Asc%AsOo&WEkO{?!Owb-rtvT2N9(sS42?04|GyB`COIAsbk1`N=*9^H-W2#Zr%&(0V9 zAe{1#R0M6|Q~AKP9xtaaHYhed`E%GPl+Gw*@THES2y@Wf=$J@>Xxi@Bhx2g9y1+QE zd3~MRokIT^kV2C+c(!!;4jk|Uc2G0mm0IUCxvoLToM5Q@t=^(Dq*t-2AG^-`+z?NP zl`-~ydxB`A-7fDAy2$W*hS4wlqQODC^hzLOl<9&SA?n79V$q>#uFrjzF;qNyFNe|# zgamL<{!O)m#q;V%albQvw%2o1|JraA^zH*o^CMy67ON~1?^5bqxWnqQXPj|2Rb4xqz_2VEus zwBCt=wwyuAIRuVqNd;vL<`jWMvZNlxx;P8C%!bb|1HFhtwadn`y`DaIFu8;r_ze4x z>$0bd5plEgINW?`xBMf}90uVac>g(Y>kE+gQ3(N}^#0%PkB1(dU6BP?nfi&->cI^fdRDW`K}&3n0DNls4ue_zR6Mf z5gz>BoWG_T+Jz1*uUK#zM*VOQRC$}#bzXa#eOJkuaKefA=(-Fu}_pTvj;+#G!g{AN&r)Sjtrr7yo za0B4?D|&*^X@zJga1!`66u>i5!4P#0-X0VPy|P{d*3Vj5&Isb#|Bki^3yfBxWQqXH zQwZAfUkU_U#`;nJF{ro6xG?zdQ#=Yc zwZ1%vb{;WJOB)RdIY2;lp;Qaz+GB_pbzk5=+ko&;(ywJ6i&PZ66WEErt)Q@RM3F+BeWr{u@gD+t{i8&0qOv$zN?7x;xg*HO^+h}M8ng~&SBqyHnP)_{5ucq!z zd}@Nf$F#8%FFLlDnzeFL4!dsL(yXxxaI1gsI9d>QpEmiUi3VwurXCm5Y$JL%2*tcD zJH@@<{yirAZ*2H(hsr6>FDYd9D*x|e`UGT+(b)J?7OgJ88%A}_tXL}$(2EYzXAS;9 zsI7X4B_t%4(6szH(}G~yxL^pE<2%*m529qFo+~&O>&^y=yy%2Yc|kbOMHSZV$LZeq zt}=#oA05&Bwf>)30H*_B5l;Xl-3mY;0tQ$wRC^csr^_CE&Lo6FUr@RqZCil*IS}w* z6Eahc>5bjnB;q4I!{Mx}714h?W0LP=<7q zZA*5(*@ze>Ie>r&u~V8>P7dZDsHhx$1`=L z&*RR&8kkhvE`AtNEynxrGaqbKkQefyPi<0+_GMpqv7J%U;XJG2F|8NM;;Jfu$fkrb zbaGT;MIsiJwij&YUmagH6yUb{0|fkfEF=`2%St1tfPLM2-ZZ=}(J|Y&uTbHf3~P@R z+85bj1^MnXA{76l;8PLb*D%}dCs{UM1V603yzrmt1_Whs#c1@b3W_Rqj~z?nH)ePr z&fzg&~^$7#c_V^;JqKb z97_|IA{?}af!gRAx9ex*Ym&4Ay+li+km`b`ih#jX1 zlK|6`SXsUmrIfd8GU5cc{!|uzc~-v&u*aMU6jVW^1bx=@Szmj8uf8@8b=O;Qd||YS z+C3ONl^OQ$tP3aqYg`4jut%g3?@!UbKWoY>mZ{EW> z%%t}Bkj{;&4Vl zjI)bwQxsq_qUjM{JMuW>{OU6&+KId4HD3SMkm|hml<;(}XBWqN?JF-MIK&Y>L+ zJtM*`ztQlhy111~ORDeS8%dd>VEBK%nju6?obvNYot;F8d78MDvgO%Hfxu zVEh^r9pd1C+A-T#EzBS@(tNdllF=rU;6FChpaR~=e$sb612U#M^`)wCZupHEA)}XF zC%;eG=-}+HA=w{rR)48YZDrBaym^(J>VFQYDF+9rdEHLrGU4>`@j=U)Ctk2 zah6t&D$^?u^q%(ulU(>>zWOnMXMTVfTVj8WapSmOK;F6)-BBH73$x6KhOc~cFuupF z@~1rWbIWh?Vi+?TdR`M8L0h#er04vT9?kc1wajcqE+Na0_#!f>n;-yD!H8B%iz#0- zQ2IdS0b4;S@}F#B{5w1c4}(6=NGUSVNslgu+gtni`~c4g*>Luws?+sS*S;Hsn-&;q zJ!!E#wkb-#Qpko#f3N?f!PxZRq^5_on*wt%Ri275QS>-isLWdDQzuM17PWaYy^JilN7;>3wjs85Si989uD)Ft(@u#H}NT#V)^^7qes+RY#bY} zvaDE9K>=(3;PHQtX!%R0qgAJ97%^MaoL)K0pApreJY-aPbKx3VS&3;oq*(vwh&C6N zw1hOX)9efq70Y3F6WM93nV&FRFaav7J+?x|i^^ef>_Kz-s1G{aK)v$c2fivADu9UM zfx`yd066`Dr#e;kN@FeW;j>NjrjM@ZfLXo4xOsf;Eu>!_C#(AOjw6?mh3m(`jIkOQ zZKd^>>qb@u=3 zBk82UQGw2tikT3(h7KMF`-NGQf`){|oHJj&UT(pl8UIZVuc#T|4j2PuJ{wfUkDmd4 z3uQp46~HH6=K^GRN>_(UWhU zkhjIk-2=U1V=*8J5xsOxwi}2FDzpURs4Svo|Nik64m`D-Kf2UmtT@${8RY163<-n6 z^X5i(Q+3vn^6`@Ha74*%jW8%*%G@YQtj>Y1CG{YkSF=eO{u@{+g_i??QFDlD{0en^~s+v z4MJdEU;$PX$D+jqK0C^yO5Gce{)!OW^GDU98r)R;a$$tgiFC<3u7h!EpDWv{*Egj1l$SLQG)uUGAu{nd64s1y(f+ZP6u zI{}LDJ6%a!Kwrf$7I_gXMbxUW@V(PN7jqqrL5Ud(0rg(V>x74H{gc)Iv@r!9gMD2S_D)}<*YYS zRO}WzeZQYE0}X|zK&}H2(W%`QBZ4cz(2SY)--jpnOiuxNd(E<}1&_SSI**;M6uurE z-H;djj|SK&_-&qv7a!-Vzm(R@zsrF>BfhW{djYYetRB~+?Rm$8b)(vSxAif-hLiPL zNpC}LEGEFpU4#}Weg0f~`l9Iq&Oerl1Vb*IIQG$C(NYvH;xgi`+@}_|2_ccN%B{`o zlDZl|)DaG1^?{mAD&P?CK`YR-lgGhwHq3tFu6W*L(~k&zN(d75_$a8(V^@-`r(Bz@ zkBCt>c)Zcq1XM$reNVSSK04$M6b8Z#(qqgV$50O2K;$gBriq6e2p1in?n^&axp?mT zmsA0ODUh;v%o@RjD8COUQOVi-P6oHM>`BvmYd8L|<~81w5NOJ?Rg7hji+S0b+i|tI zCnI+d)G$Gg5}z_r7!AlCzSu^AUh`q$TDOfvVBg9BOL4YWFOT5Q0FW?to#qq&l2W#H zgA`tC=i*T`Wdy*8TY%xBjtll2t_x`A(uYTwqh{;ER#m@%X}gWs++ zIGR5(nx+WYukYr8%l}*RkQR%)J8vN0DJ(&Jj|P~2Q;j^#tB7;zk)DOq0LVCy0Qssi zuWQB24StaKgiwztg~}G6R-E|GbJsaCZEPeI!h!CL$O2+PHG^sMe0x;VL%U1;n2j@+ zzS8MS&sD(l-fV5w1uEn@a!ml&E`GAgb9Pf!;1x6idT8QvBnN=?S0?R4n7E`0pm6%t z*)qCKTEdd7KyU9-BPKFd;^@@Y z>wz9cmlpVm9m8*cYfk^|-Avf+Z=YD9T#6<}&+9@p94FiZT%x6ze_3$0&)o`zzPVYTyXD82^Ht7p}*BsRe@Ha1`LwEKks;ck!n$U-UG zf_gk-72IcBE%|14kI?nccb?4tN`I_C9RtKdyOk|_jkrUG10Vm;L;mHop+)Grq)t_Q zD*c5G2E=*J`Zi)iH7rgrc~1X+{pCWpwa|e$KQ|)52~ z-`t~oSjQfNuEf4tH+#OIbL5k}Lgr5`y5sImd z&8d4~VnUHBnHuN{8A3+Eamy z0Wppz{!FbxbE9~wJg;k|P?ewi%fPAZDX!oiBYmo+f&(a)#QL0+^Tgh)?9%`#I`M=} z>*8WYz~q_Dcm^1Hr@(I~lw0*K%S6&-c0|8fe*{QgRXku%oMh7E-K#6_%cFD(^(*`2 z)VfdRmxUsC{sp=<^m|5Ab-M) z2DZwyojiQjpPHu&>)Z$B$vHgrl57y~y!KJ5<4MP!nF9XJQ-QChS(!+PliracwUR1w zxu%kJzGF7@aa11tZDGNyrgIdwtKLH;rkXjb)ZW5#^Sq=KnnH}z<6*4RdCf8eG|%2% z+ZXfNx8V-YetRtJDUJenu9ZwB&U4ceHvNNgU@4`2!Wpv-!-RUxULaJcAzy0FcY+Z!h-R>tnrlQYTm7!`6d#QQ5Z5h_-P9idHUmAhvUdYNfm3?SW*IP} zbOC4)4tRcRhKBNWxXy&7-w`N9U%GJfRut7S_xa5Q2w!@IpyZ(dq%kJs zr*_>YyZX&SU^_!B`pdzFc*)Vw``9sgB&>H7=;$cw3BK}mxAR!_C;&z6+c_FBIiQ2X zaj}QG{#kTsoS?82L`EAi*UifxdJj7FsQ@P$3Z=h(-$os%i9P~PK~Qc7rJx@GsmjMy z(I0pD+`}2NsHC%IxIsUYo6>utrNo;zoWWm}nRgPF->~lv|CalgNv*4` zH~dbZN$LF{YwcIJzJb^TQmvwGP;U98A%_sIr10Sx_g8Ch`;f8jUV9{k8Ph#~7Fnx1 zy0@I(Q*D-=o*hi?9c_}e3{ZPer(D|nhixb{%>t*mi!zW7KyzRjQr*;DKPgU9D7~)? zN_^hE8uXDDZpkKBzC=;=+#Gyc~2r z=~pnnRpwJ?){ZAe_G-(&(q9H;r-%Ij1n-{oX@IHQfYy$UHp{`zjmqG1(skFs%v<+( zea|2F$3b~^<~q$hqi%SJ>d;TL#ccLUd^bEGm9v_y_q*{XuS5nrNHWaJZuk?v>9>x6 zH`k=A>G^B!<9)!{Fnr(e_-MO^`Hg;C-la!Kbvt9WN4)XQZ?jt-@G?k(??|h!Wp1R%<}Ln*p@JphR4f1 z1*QZ6#F)zI6ca2H_eBhw=`G)>h&U~L=sKfapJBUOfDmFTS#K5e|BSW&=<-`jM|zuI zy4XL{Kx{RT1HRiqIedXZ9UyNjtl?yD?c#Vnwv{`%UdxZ23epo|V_}SXIw$Zna>S3G zG;#?{y7@}V1V``%N?To#Q%i5k);ExAd4CPZ1=_DkhMfp)jCmsbXrREevYRluE!M%1 zOxL}|)-VasWGYV;6&7ypda5ahKG<9S8V`!JZ}_3o+_d)G*Gm2KY>9E;)CC(4u72$PDxvd|# zVoHcNtxZ+ZW;w0085NlzD<44~m{D{~^Dz8&VihtSz z8TK9aKA?Uprp?>8<#dpD1$II~KX~XzT{sk$~O@q+lwtL&yMUW0Zhv3~#qYG*GW$M&?9p{Kk zdNLgT6`aZba`zVfd>ohCH{iYp6LD*?*P564AD&3@uT9i@d6BTF2ts-3GZ>Op(OT{* zrbiU(8&HPnhta<0x}SuZuSJTdd2fsSBXpBI1Rx-VczZY_9GOd3^7^iJShb^GnZ&T+ zcKsJ&)`bs>agw>vA-+Q~AtB$QxwY-rW11)Ok*(?!2B@f65Ha2abea}nhrn^7ZeNs! z$B}5i(Ub#%#fiDGbh9D(^qu69sh0L%M5F> zts1vf6Cn4z=Z}*Fgt6a7QVOVI*3$A=Mv6V~`QfijApCD1a3&qB#-!SDF~Hj*546)q z(I}Pdy^Rh6$JKn6xLOyl&aWuN7*7j6I4Z0u`Mu8+Mgsa#)~IW^)|vhQY6T_Wafwdxr*Sm zbPVWQgo<9L0(}EuR8$!PI(UyaGZf0qH9Z%q;=&c#{qd7Q`+q3GS_ z2s?@Y2%l6gyht+#;8HlCV82jhcJa@H_CkP;XaWCgfJr#P?a(b!NGJf>Zpz|8LZpBy z;wA)7kD*ISSynMowEC$*1@E+5hggi{QzF6L`71()OCfQE8d{y-l}7y@L{bT9Uc#fg zUT9F22z;8736wf2Us`l0_T}po4<(>L%@8aIsN!`)g~_Qv zPmE*>(U8CbkC3p7ze4y^O`qwwz)5=0UZZh)DwM>)wUNYtUwJ%O^MzKDpF-X7a{f*G z3DK{X09l9w;j`aJIjGkBOzk9XFbt~f5&&_1{#7;cqpT~}c(1#{i+;{3sMOpH#Jdy% za^x66$iP4$=O9-`QfmI!#L>ysWI>U6V+Fd)I8^E1(^o0SOzeuba>phry(A&V9`ht< zutz%;$W1E)$Q9!uEnb3%+d4a%l)7%WPkFlqmn-mHjF3Fxrot3R@B7NF)b#4yOY@vI z>JO9v2V&NpH<~n~2;e6rXNZP?hyGHar?rLxbGN;e^PK6N4)p_C!}bGn^4Gmmbj8^m zI0M&M+cPs^})X;!_Uys?URj0o$?eO zt0+MBD-N+DKcENx)qqu}$i}S%v|?8uEu^k(oJwL$%TEE7$!GWGj)7EZ&?_9r_VZ6UKtE$_9Qk z0tNm+JS1Obp3;YeZin!CGU4(m1qHea94eF3@>P$V9P_;GP-4$~HOYV=k{NLM@D62b z4Lg9DCD&x?Z$fCW#cG;rDu9;s@uN&G`m}w}R|e1{hVfdrB=ArYpWB_IW1nZJg9L@6@oYbJLq6&nrGVX2VVC9HN+9W8 z0-RirMt})$Vqj!wSfp^a`7`)a7$GwK;&JC2+Qy_gf$VsZ2BW76Uh{SebT!y%r!L8> zp85Ia6>+8{csD2#t?Mfh)6y)U$RgWJ$@lQ-74l6J{}TI#Bl)gGj#P6e+2G*-Ed0E) zcQqt;i>CE~2q7zI_SJCE&;m18)Q_F-Fl*Q*RH%HsyQJNA-&@u6OzdEcakf30%*HHC z0trBHox8=x93bI~C1%mQPT!S&SPuwVMCvmz7sRtCFYMT_U9m17Z*45gVJCGUVr#DU_kY%ARO_lzRvOqJWtxs}eBHriCs_9sgev*s&(dz0PWb}s zrB?;PQz$JRUNl(dX>6N~VI$SzLpjZ?-s={Nm(JUSr{&u+6zOharx};)zjd=?Tk6)1 z$&?AkxDgeJLOEG>|hm$B&62O?4?SEm;P!yavCkwmeg*&`y)T;Xe zJ2hz$Bd6zqjzUSFW^~oKtiV79!I`R}LE+0dVR&qGjcJoiOElowWeq;6E&#JZ!g5PQ zLBwfT=dN*;GV^AE0?7RsK?zQ+jPWQ>;yU+WvGV#|Tlk=s3X#S0F&rFOTqF9Q*=p+&qA`*s&>gH5Px)JUy-cEkiZTs0r%)w z4?Mj&obMy2LIgwHakf9n;J+cm>wzhN4@EPU=f4Ijc8O%h?#;t0agR+vol118m1w^q zwuGJda#1gIy2@@m&p5>!oMihYbzyC;@D&+pW0{XP7;Q%?96VU=MKzCF^^e!tj=tSK zUMcpF-9p(v0hH^DiHvL5p-*Vgvf^#O+?}sFCe^y%gI3L3 z-0v;+q{o8>+B}LTF>Ppz0j0v?1ph5d*cq?7+L z-o&TmzkRm;go1}@LPiBB{4%X}xCaYVSl-TckA6lCJgBU|h3*l}_cf8#oK>P{b#vaO z+Ri&utBQ1?kRJO{Cf~->v@8xy>2Tj`Ln^^bp4TD;@}<)5*3~*DAw9q~FGa!#g3>-H z&qqLH?;Ja5?Df%*#PRv`&4|^-45g1Lde>pEszF8M)iaP2q1iJn5I;813Cq|pT{k$RzNKo=6imWGjhp+XgDjj z?je}+hhbNBp9|O<9|3nmBx~a&P-5bHrk&w)Z0$uPZPBDp%;kE6|Hdc9jH&gaS>chZ zGlv(_Vy|2+I!IBbs@)#B(!yR~8esI{umhAWE6M|~%4l_CTl9(GMKe}(CuT-yhl|aJ zfezro6*3Ta6)sR(d#fiQAKeW{_%GH!7PC zvVy4~TQnV41N~~`ZKOpFy~?L9=e~pHMaoiyvRRVC#;${llhv%v5+WQ^yLgixi_}PU zXgENj*;r{(dFs0(9J+sH-diGWNoPd5ye7>S)w2>Fm3`-? z+wk!jH{-w^de)YQWs31kjwfqX?IU(YewB0uT+!4buOX7U1U;z&y#+ISJnTmxu3D^A zHRNx3gg59E8jOWU6~mFpC*FcZ=b9q{*JY?m9%_4jmf~{b5^BV%BSy-udY-g1g)iUy z!!r$!i*y7DoP&jAGlz@L!-J!2h-beFE{?AI0r19@wNCm9ACn5LW9iw+EvjfD?r@e6VM?UZHu9wE=TS<`CPeIRd{G+j|O$l?4#GWq6;8z&K1zZ11Uey_7*( zt)kI~Ry)KICqPx3Wc;Uax;=?ovE*-ClcJ3*lnI>bua+J-6~UlV5ZB!m#_#8K>T%iC zfVug=4DvBjiyoQ9h&bvn9Uj_sP)uMeewraF_5%5hEb&|yf%(M&`;L6e)qWBYmW6M` zLYq@~7cRqzgSu_+4j}<Uktu?0Xp>*fh4(LJ4K>XBwhkdE9HGUMV*)$nipVp9sLAdLBxhtk(^=FOx5-g0l6OAv&6w z8tjNR$;&;3pp@MW>PL{lkTfBu8s1xmyt+?U{GsU17b5rl>XXqnmurKMo?f_k2?cxB z=MCG}Z2a=hCl*Lp?XOJ=T{|o*31VH=p*_F1+TBv!{Y^#5C;!B5t z*j8Qm8Wl*1n$KpbB!~yksba`Nmzs(AeIYGR+c;HeJZ~`AQbvSR* zrnak(7C+7YM+aiYRu^!__mmG9uX0B;~h_))Mi|#t;VIElA1VMmr zlQDOT)*E*98ZY8**v^3@`Eoj)15rVWOEYdd>Qq(gN_*uM%xXruDNi=UMmsJCPcP(r z-84J=S-#u2OH$PW8X^D@;4KgVc2E=Ip)W3&VAKX*NsOQT7;Da=rC5y_ac0}*34VzN z+feMRxWHcbmZs%LPK$=q5t+iF{EyE(phj&`)Rg&zG(&W}9?B4ZEH>2jnU{PZA}>9@ zp>w+3&DB+g6XxVB<)ypYL_&eXdCGxp9e`B_D!E1$c|r5|H-kZNq7T|tata_H3&u+} zWenR|ua|uA$Ihi)-)aJf`kncQSF2>6g5*?oD(YJy-TEOY&p#z6C6l^Bs`2e&(Tl0o zHS}2=X!CK6 zBT5SY{@ve!KGhXe^UcDp25Qdg{3!DUGN4QZlu*+9J)WSFs7()Gw#_DzC1-skSlNdw<= zyMSjw1)nn@&Ly%a2x?!rJ){w<4vK`cv45i*YOdKHcbgl^D)<-Ipjb_Et4<6tmuW|`vc$Xq$Xt39i{*&}+ zJ6kLc7P$vu)zWnXn3MH5#WfjD*5RG*pQlxJ+cuhkzgS#tnHOOfY<~75Txkgcn-(FY zNq<9CSDg;!LT5iOb8m=BI*>@$psL2O(cSo_b`evMXZzZMcng7pCu~Xp`6v-SR^#~< zF84Y$d(~#}!`BJn=8XVV!M||HbZfq092kizNTIf?BnADU>fo}tuFAb;j0?LFTn(X> zD#d;!!C1b&H;+;j@twWuWn1W17eO9VHBJJ`*+eETqxaak2SFGsMfdjQJ!7bf>C1=W zFVNYP(yn>})px1LASgz@iV>a9jMPw4CGmdr>e2)Iw%ULjINa~kUj!ecFJ;$iE@Bey zNJ4lDgI6!NVd*NgTqV650vemN?wh>P%)T%7&5ZrEpFihR-em{q!0i%0o**roTdN}2 ztAZ2fq-hdRwR_Zg5n2^H2}TnYj)owX<~TM+A*67mOAdGMZn|N{Hd8;cUmLTPiR=QQ zckSq$D|L^H?FFn-QZogmeT2t*36x5XLag)?ih)ni68cp}-U|h>F5K^F47Kj9t*AH; zuFaGM*76h)=_8!?FMK9;Ffv`cq%3X!-WfM4;=M&laqNPwivqMj#^z3c%BO^L8S**7!;?Qlpig&Qw4>EWWF|NBoAv=m!Dpuv{33eW&)4ork1E2X1G7kGTxowp83Kx3Hh;9WOwM`|?m6NmvCIA0=&Tkc&cA@vivVpnrJKmG=} zr~efKMT*T}YWz9u^CwtLiCOd|R&&@w=U<6q=UB2%GfmIuh@Ul*qj2Mi+Mh?)^1!_4}~ygPf~7`DF9OC`ep;9GIuMs7?!BvGWo-a5lpjGEf^06n+j0P z4C5iVL=*0{`;8zXfhLLT0_pIR*efmZ+QengQcD+|8xhun3@lMwpxSP$tc3lHT#F9V zAAjt%3BaR9YjC9tS=7NrE}w{x7^V=;*TOaz?$+lM} zDuk*O&`l*-2l=YK+bl<|=swo8#(`BvFLWVxYsd)c{&M_tDYcpG=wVyVJKRgC)SB+V zH9IO$$x%k(%RE+L-G_cKoewedTe*DLLimZe89^;|s>+ibmVvbpB~DmIM^Dn@L}lVV z_VH}`ann&MT$$9VvfE1Qw7oPWsIn10G^nuX^U?iD3BP=#vM(-sBz4!wpm~`9TqvEnm!cR>Cph7`Qdn9e zo5@^v{WInuLEYR%qEnhk&`mUdKrm=&MSgWWQTbQ!4Qfkkp@HAr?@eEDgAJ!C+;0ALZRXLv zCs+dgT_33~&ODMP=EGzVy)M%c2(1NOmwP24A>vwNfqns4tAYFCpSZ5h}E zT8^kWqi>)v=3fV$yaLgckW`QlvmDr;1~-ookVYWPPWJ{)<^ft5^|j6LCC8)Y%X`y! zINh?luew3#3?WsoX#!0!m}6X(f-kF%^rau1t^Ys5-a0DEwSE5<1Zfx=q+6sxIt7#t zQRzo``J&p-?iTFTJy(VYrD8-?zyjP&g(dj z<8#&@y!eTiW-GL*4iz*0wce9UwA$YHw5CY}Yfy#pk{k7LLv+xV1!j?_Fd%l=b$0bb zc9lB1oN9jGUhU1NbYp0i<}HON3NgdoFqUz$n^=eI39a@b?_S^0GNzRD@%5+ESL5+0 z;tOV~v_+qQ9L9zpdSG6W?EpoXqb4DV$K;fefKSi)h?4Mz=QYt(C?X#ovv`5{M5)Q9)H7Y!mfsZ4#h6=6meP=#xioeZOr z2$aqJT;s&}Oou`dWYq!^XQ7c}fiqVRSz{gW-?0Kr@w?1rK?F zP)*E4kIIA{(y_z=3w;5<>v+TM|s}+K8+={Xz%|hzb>w^pIc*uKf=G)-paoDPMftQ-5(PnmsLC5kw}+8_55UY$K%|=Dw{k{Ga)G>YTRMTJ~P(I5o-!S z#;m^d4$xymQ2`?l@8SnOOYE>L4OZHvAt@+}b0g8$gO=>ts!fv;Iy-oFp`B;B?BD2ki6L=8U-CVgDo-4Pz-zYd|>4c0L>iS+ubxdy=n9{G^=XP7XaHWBGX+LaK9CgzC# zWT_EXnJ>2^@1c4q8WQ4Hq{+mf*6T9vBb#?V(bdxGM{<;NMPz&_W0m4h1TO0palenW znyt)qorO@JSn11%6{RL5?7N%lYP7cDw6VzYapbk8J6s$tm!qKb<)!bF;@Ou)yLw?J zCAef1&GaX_*GaMy{`A6A;?NdNQ3yR84kLS2Mi7kpiRIiL-sY(u`XLVsg#{g3TLg=} zb-0=nNYHr%T2!*xw=HXW2f zTg9!MEU=HeC_eBR6nAHJw<_XvTyi5lZdYY_bc`&02Vb`hEc78RtHz*PTplu#&^+Q| z))%Q)g7b6YI;ZqHBq&fSG&0S+)~2T^W}}%Z%M0@tM;_rWG)FFc=HEV~7GC5Gl1iOD z#iJ3cmGv$t(M7gx?1)1NxxErfgYckSYFvH!|=3~`eP=n z@0Sk`>0A0l{Ng5m(-{trRCnaq(WWcK^O~S_>J^*UY&8PX zN#iS&`q@V-*ll>2)E9?`2UTJ&m`^q&(h@|c(bx@Komebee&DU-iS}`zzY@u*-e=HO z(uS{?2o58Eg^R;$9KPp5d^{CDASKF|1$#GfzQV%S!g49=LZa|!e&#tG!XH%KCi{iG^iwB;G|d#t0wicCnazHC1V!N$wE7V$&dD~LVYQ2X&eL}q z2?b>u6I>i4EV7bTt&aS+)XF3<7En0RPVaA1#@;$){I={jXf$BS;VnItFEFoTY`4X5 z3K*~rb%VsCYW ztrvgo7Z7iR5PnxC3lR`OK^`P6n@XN=Guj|XDDf2D+P8?)iOT$|6Z8>XTkrrwqX^3? z;x>_16un)wI5dc?PlKz=3l1+|$nMlGK~j(i4=3g7(k7QPE28;Dru3rnC$GM{D074! zjoJ>N!5aMw>GB&&m={A67XqueMybBL(7;Pkcl`|H$EPtJRS9d9EqG=oSDp% zb!iJm*!)hg2%qu^zUp#Q8;{aCg?1(l6bE`ohKQ0zW}+a|m3=i-IrVVeYeQ{L)xn!Q z1ai0i2Sb2t-w?fAsb>1c&zSgWtA9{b3`9NHQA?n+vHt)F8i2Oa0VbFv2s{84>1 z98Mh;Nd;D&Zz9S~duqoksiV!HYQ{Qtbw$-Hv2|5wqJ0Ftw+17t6>~K&#rss|a~mxt z2+c4AIDNLb$U;vcdm4+ceUnzpV`e(E%jfg~*|cVVaA#C;)V$gyBWEz=V!q0>cjlBw zPU+3Fi5A@SML1Bj%~YG=hBv~uN{IV%C0Ips9jtdB=j7r z+?#=%e+m+SmIq>5ErSHcX6fPDQO!&SPzRgAnv9O>CEX8K?F4>inqTcQ=XEFBIQQ7Y zpH$oT-#QIAk+9jp=J9Tu)cXZmYo3an7dGp0z2Ehzug~w`H>_-RBkClW*mWbLD0w^? z9z~{f5m7E}I6Vt}Q%s6jw)R*@}nsam#2$$hiE$}6X2 zbY9+~`NKSF$Gmjd^y-Sc{XCHp$YC%7#mtFXd`>Cap0IUeKro}Mq|BkfU?IYk#q)E7 zC;NkpInD8|JGX(e*mHSllbFyPsW+VVjf7H(bkB(sB#px(!uN~Z`e9yS?};NM%4o)L zhG^IbUF??6eoeRisyJ`-7%#)ZqhKZPi^HjAg*G7CB9oGaY9+Vc(!IWoQW|6>z6O+guvM^H6wK)?W6zS4fcI5%(Hnp13{PE!(}P(ZJmsKdAhL&Ypwfp|lv zgezO#QSNWatDBM7ms^Sp)z|!>bK!^)YT<<>#N|!UeMj^XbSBzo@A!P6El1J+9QacC z+|w$vOyCQ(SUeLBK{dbvYFUI%mdlvH3g&$W%(AXv@ixGJiZbKOku|XHVKfE1qSqgH z!d^19CfTJ(Foi2wQ`1yN4sKwF%d?Q~Wa*~Ue2*tiP+798dy>sn#TUCJWR}o|iO2_W z?vE|ZeHl0s=G(7xiR!w6Y@m^dAYoi`n|bSTdFKN0+kL7GSEkD-0_(Vq=jfy!uTQv) zdx)W)Vdw6Ct8q{06?+i26p7uOnDC)PmlZIxz zUqvY=36O~{3SK#caV8XDevH8Psz|qNTCy5<<*9&4-%m@ z^X-@~Nz-Q0ow8T&r|A=iMMIslOA=J~N;OgG}=>J4c7nyXd&J7(U|70VgdgQH0r z2O<0yvqV7ymqHFTCc&w4dCJM-cdt3B=S(%92^Jsi$#}e4Fvq19n_h=0O}+iN-esxQ zvmWTp^s=Nu!|UgXe9HYzY2>%k#y+v#Yd?;1G%SIyXVmM?d43Y7dqF9Uwws=nDUm)J z-Jq1eOp$=TRr-|jW;h8SbMF4s{Afh|hHImYPmSrS;BW=D^DmsXK>98VOxl8|=}tHr z+!WKgV29PL_S5dBpC>ffJNK9WyG3+(A$0vNg9qM;Ds0%iWS;+qG9X28F6Bs?D1dyQ zRawx==tgKhO(pA_mR=)OHYo>>_8V3WL7iC21}EABixz2e0GKo{59v9GQ&!Gp z_G^2q$@X8sX-4i6e?uO?VyM0qa7Wxgt(x*o;fQQKZ5TJm~@w zParm}WC5R=d{g{}EGepF3>XU);mD8GLyubA#%GTc>u*?4WcVx#>wW6;2Qj6a{a8MUO zW9rx}vuQkL2i!Jrl4~V6GffJ^gqKZTE?zLiKBSI-MjS%KEqaMWlY}BM9Q<8@BgXNg zosJfk#N35U8UH3G7M;1D8_Asf65TzdzAwc7m65m95fZLOmJLPgVi>nsybSjdRuD^mS+MQb^@@Uu)Ks_h%z2%- zqA8E{Bh32E59Gg8hH&p)&ngWlCR9pxUXS`^^7!8<`7Rn^t5B&Ge;(rS0U6m+p$t zQZ=yuXG>g2i#h8?Ci^VUJ+{-N)Qg}%&A#uj0z|4=lg`3bl@y{Jd2^dhSsTWVpYM7z4bJ%~A#jvjhE_RFKBIGo2_n;@l!?S^Mss=kJ3 zY=vO&27hhfhLK3d>Z`L$=HQTSwBXdLZmH3(5aH`>7mCv(L4R)_zL3fpJV^vJRmv2< z475-BFpv_P^q0_Zkye&9LMgEJ3}H^oAy2TJ(c4O&h|Yb`b}}9xJ}SfP zQ`qc*Q?R3ixPRK&cR)XchEd?u$Sm2@2Uzc{M20VrY-!XlCre&fYO0*%FWRI!Zl3y{ z$bdq(ZZ)~+1gEMblxdE!wP%syt?^0Lc~tdc8Q2C`qE>d59!>1Ez5aD>Yrl4#=N7|j zX35I8Sr5&?>{bPX1&z&I`fPr+7_tigVpE7^t6sbkEZ#jF#l<_dgr)Lykk55&$&zol z*EBueg1vkR#)J>3oLM!Px4BVFy)rly4RMF^41$iV;a)}6%CFi8G*i+b7RJYr|6hUymtlbBeT1PiYwIjNFp!LvhFqieuZa32nGUX@OHL8!8fIw zD|Nz)X;VB4VsyIT2x4N5tYvY$2W@f8;)E!x(@Z*Cs7#P#k)cI4H>2)Bsc%7tzPy9v z1ZEa~dl)k|b--lUu`(x}DU<-og*wpL1xSRDacv;;)PX2nSSqW&DaH2|Dk}P~ucPiQ z2+8qKt8IR5K6ckqs~<9xTtey-c4PuiZOyU;Ul*2^#{ii$@8C-~ObDpFQ`*+~~Qvwe9aH z&rct47fILmP3tkF!DLnpkJ`!VkD3oo?CC) z)zRvH{Xu4TT(G^SyhSD6HwOzJpQl{CpX}k*BBlTb^rm&z@Vcd*4QlJ(8Nvyblu6^< zdm3;$7!d>|47wF2V;62V?YAhq z>~Hd(Rq~FDat@m z`jWR9zV)sYjX|CPi>%j%Ef4XfgCwQzNN3Pqqxl|3lapR`I&v6JFAL&i(tTP!zmFVS zEpe0z?=OLjHcX6}omB<#M-L`MChA?h^VlTWFKHZ`AU(%|S8l1nYD_JEx?2{WPM{f1 zZfnOu)ce^tn!aDw&lAzoUbl>@>MgBIuL@FiVCu z`8NJJ#h*F#nnmJ`vK4Gc!1a0u14Rnh!j<|lRv_#cbB)57#)Bq06XZjE=~%c6HOm9{ zsV5K3?CS0a8(`MC?|srQR3_WkKUhMzHyc@c)ANLurL!;qI=bNSPmbw&6Duvq* zH%5h8xbmQHtIHu213p=z4M<&q#VqCXu3x%j=pCq^(LQ9DZsM$2?$8&TH(!3b_iHn= z`qvNl4PG*zq?;$Rn|lWh9QmYNF@JZIbL=nExG*}BTQ)Q~)>a~mHSBiH6|6AsvtuaT znbf_&aXl|PupKIM4XQ?=zP_vQ9hvR90ZqwiSs3=mB!_TI>|v?P! zgmtL+4>r7&IJ`&56P+gbFpocAffSKhRrR2`rcFi!RvXH(J|?lyEX1T_Gx5S!_GsAU z>29~PgC(?ATK0O01l*|9ly>iGX6jy?yck+M&4X?`Qq*|Uk^8$lw&E~4WT^L}Np>W* zQoOQ)3E@;jJe=B~MaJdGBV(@{`h5dNLT8K~`4~Rm#D81s`j~g&+@EOwx8Wm|bQdO= z5aIM_UatOh9T@Q^VmMm+wvlo(a)#R+J*k{}H3ogi_4yR%aBD^ydHdLW-=gsW7td7= zG&!lNwp5+|jw7SCMaqI!Qx_W6t%T2|Y%W~{m%xU*cF{`R+e5-z5>NIere`1*bz|R# z@B%O{ks>`8#kAIPYPKZIicj0u(``8RsLR_S)1QQ_63{2=&|0=$FB&+Jo_r@)79Ea7 zlMKnw9|nPT^Ljs8&irI^`R)kUCW~NgG%RH`)MIK%--aZXhWxtQ61*2RK~Gq_qWmtI z?+B`v#mVyU6|dG;-&tCEr)^$l>c+CL!i@VnjihW#E3xJbDNz~T11ZJ#n1~?HyNVoN zU*O5?DK8{sOkU>=l4U5YxZK1fEff>tA~+0fN%&g4-nMb7VSYi@vqaaEB6(A$x7cjCVBbS3lMQ(y)FxDRp=Jz7>|q=kH0fFQXkXAPqnj4fTjrGtCEXsx z^CRT5mU#cdRg_QSwP&uYdI6m^uGN*VbUaSBo+Z+D!;OL?QZr(3gldx#-arnQV(fJF zq^nNi$tfp^EG=I$4#7Uhb3!U;<9_KK-pPtRPO_(6dEC#3b(Rx*O6xD7pEux{@x54= z37xglF!p**4h;9#`_kcjq@TG-ZmQ2gFN@O^HPGsr38n}p#(N;g_ ziJU2OcmUj*SrPMQ#3hIkB8Gg(E7Nb_X}R1)W5U;Om9ezPD&j4ztj)Oe__3HuSj)(3 zYUm_mG!bYrFr%Id?Gl#f&`SLXsMdu;ye)#(qq2LLI_Cf>Vtp`q9Vq+M zphI8+)v231ANUAAE0=hQCqQUSP(|&gxk-@-mFYxwD8h|Tkl2(^Rwhv$c5c%`>wIgt zqJEdjpEX>4P3hXZSxihlsF$3}?>LH;3Lu#No|}`VcN0qF_1moH#cgvL3r5{@_lyD* z$WPf&@1qTlB?ol~8Kf5#wS2GPwOWnskX!ctI(n%HtFtmJX|YXdw+r%QqrU zI?I&FT&Q)c$KHy)#2Vdd&S0+}9|PQ@hky(Qay2tt&?EX01IpbNFLMOa^Y0fr73hS>Y-(Rnz7;JWnn5Rd3tvXA zW~*UOWYbdZ6h_!VdTaKIN}F@2c%P;0FER9+kpG(nFn#abvcvolR)+>W^ns;5TV>=g z@*~w~c{DSaaVQ!^M>bFFRh7+z;@kI@qQ{lBG>la+Q2FDUra?vsDwkTZ9CjWgt%BG) zu`Qp|2*AqN?tAa(<|sMT$HSeq(AMMb#D zu0V^6^7V9=(@OK29(947QRks%m<*m>447;X%}0jid%Q#k(~h%@u=A9u@y@cg#F6uE zP!86D@x2MlUsSF#IZ|%ksjn;b{I5U!|8W(n%H8Xr?+(}Do zDGfbIz;t`tU~&=wRgiOGavFF_2|2I9o!LVB5+xshet&eIp14D$hOhnQ9Q&3B@O~pF zx<_D;WPB(7UKBb68KKQXBtRyM_44&Ue39BLvi0r85uPhKIz)L4nT#y+dW1!Aw6sCx zFaeny%#jGg*@}^nt{VsPu3P7TQ1g(4m4H>&MaQ6>z@)?=KGk+2an{I6x%`y`sgTDIiu>6C zTfVO@?XSgMhM7-KGzF}NDkFfmu|OHh(`4UxJgG!? zweGqrrx%nCd&2fVDbOHMU}IYLMfol1i~hdsPKQh3J_E6*84n7sCXLxca%4z&kry{KAN3yN$(7K{d{jmEfvcgm2%@!nQo0006Qd_Pb!yL_+cY&gjG)RmiKC$UtE>6Vu^L8?+XJ5O+f@&WX!il6HLNq?Y5Z(2a zXUj>*^!|3=LCWLop+*YT$};8(Ys|K606+5FBAsm81oiLh@(=v4m%#ZCdPdH%I5knq zIyIOc@0#1EP>v@j+gBy5MsS$gQgV*g({s?AAeLUI5LtJ6?a&>TrErIu_%H?9^oT+T z`T^5z#daKu7V5vv4o}}>Z3wI6Y_4ayvrKl>j30*PG^hh=y7@aqv zO$9Rn{`>kk3JV`SzD&A6BR7n8Rha!5!{aglg>7aehsji0_O{{+#?Z}6TrkxBQe~Ev zuLx|V3T1@NgJ>Y3ig@}{=06#CJsdaju_hxOw{ght^wmONry{N{R=RysohsKP3jfl z^u^qm^Xo+ig?RD&xC{xlDzv_l_fqtP(|p$q!V=zfYreTTYc?mc^attnO7h5X4-k-W z?ulp9N^=i96!@l&dld7Zbm(6Me$D?)S!g>!d-Nzc%Z*_5fUZhd=`=^(rq z`PINDM#v1xS>%1zZcF7D{o?P6`}rwP&&-8I|9J1Tun&j`(u=fRFfxT!7sJl`7)SW{ zsx*e(?LO=IjoL^U?3fv*jdB)6#UYm$k(kC+m7tetO>HZxg%F7U_vO!m3_!_o041UT zNL*sfUMJ&|^Oxp4noiYpO*_&4>%erG2r?e>fP=R>OEvqK4mgZaRQO!Ql+z$C;`~KS z+W{`7*Y91xaev=(%hlEQ9wUcI?wFKm=$I>Ll}O0)A8(zJ5%>!+SpkquT^flw393`5 z!8B9tnAuhZ;7p7iod$?Y_mq20zJb~bu%t3VruPMIX8fa@or zQ6cD~`t!>h)-cP#!heu;gYLtCKoht(Ms2esL=8f}-)PI03!gf#Z_z3E*h;YSJL#G$ zhgH(TJRw4l%<3p9Pv4vStyuR=ZA^HcbmaZLr2e(A5Nk^kY_lLrSgb(1hN0i=YXA=9 zE$n5~k>lH6Y|Xv%DOQyCC-<1NR+*`7ocS?i(f)pfWtjBZnPZymueOpNS^ZPKgiR1qGb4->3}=(77;QAvLazaBtt=dvuG3Z=2UUa+V*H4)gpVuhh(RRj1VCh zgUjZ(TRb$S1g|J17;gIA1AmVHuC;$HI$C+~*Y9C4*3-e%nf(20Kqw~`F|87$H6fqc zm47-F7ygRw&~KzqrvAhk?A4gQwz_;rp0*I3^5#}==KS_fkH+UFBF{!!4IE%_|98pr z@7gB|+7s)KLZ9XjCPw@#6RH<9B(Og=`}f*F7`tpPDHYN=^yWm(lnBtxkh@7Zq8smC z^$tXhzn`r?M_)~bZvEpnFC+4WH8hVwm;Mc8xQ*!($k@)x?QJbIG(GdF_4-jW=jx5m z%47M)wS8$fC!ysi(=SH+#!F%26aPo!p@nCCFv5RuU*)~P_ZE9g21Q|v{($08zb8=d zhnXmVSxuwLuESn`wNS?0ckG4N^DLa_@qEBaFPs_byv7tTlT}3Luue7ulwqEghb0_Ag z2`v};8-~x#pbGouN3sFo=@+kq_7gV$MO>q zfx)&o0#{rftho6sCbenfR7wzmhmA15x=9Oyb;cEv;E1%8~z=Cqave z*nR^7i-JFk?$^j{45m|{@@)|D!vNC*F(dA>cTFrM>|nhFFSNI(makY$yeIb}$l7_V z`1Kn>Qyc;~1UY}IC}A&sR`JrA_~TZz2|h2I>Oba$&{85kN_P&)QqmXbTT7(`_VVDO zJM|`~+>FrmUSd`tVcmZXz%}cenSQwKI*bF+Q-Th(T56QBUh_4r^}WVxRLk&XPDW)o^%HUrefKu4G;uT0^&JAIRKuw3_Bg{eYYpE#ahAMMHl`{QzzAZ6Ri+B|N!sfvekZW_cyXxK?n*)l z7_|SpD@rjU{yv7>NFQa1?$ISC@k8^eBoH=z0YY!LgLv;%5N>6~vJQ5qIj3u#YnHNq zmw*Y(#c@~rXul~cm@$HQV8^pxdltDQS**k3qE|8_G5&v^dYaW{&!cQ2HqHK$mrZY8 z1A0IKAm$mk@6WMzN70Zn+ShL9rLFkp>h1XKvbnb@2XZm)RHi!bWi+eTP6|Oaw_E24m#18upi;;#%krf=LadK?WRU z4vhkUBn%SPUx3Ke_!6+EGK26F-T*A*qCnu0w?+x&97|!vX58qQRq(|t;f7Z%WjRJP z!pF0v8a6+a6_R<(T7)>MBp6dO92g5-He)qn^HArgaolkz=vkVRnPyTYSbpg?5#>uv zgS^atiXreHwAbiK7}+wqHFNbIrEfqsNfA5n0m*;`U@q_iXSM|r#?&qV?Z|`i-|H|m z>S&N=Rs@yFO`8h zRo^e+j!ZTA0P*Lu=7 zOCx8=E@e8D_RYliF{$M{#N;?M`iHu&cR0-q*ED^sV@iz65a3%ndsWt?AXDpg4JV=! zRvpz_R?z%IhfA%uQ+jaeamInJBrT-2+l2_S$Ib{UiPqL z7D$xp*JfY7`d(EyuU7SDdZ9i)&G4*J9vlmNTUTBs3v7~yphR-%uzO#gML()0j*H1u z!}h$VrS_X`hhp-5oKB@uGFLNh&5bada?hjy2cyMTN)5}dUH{oiFwg9dZR-M27C!C9 zO-L9LJB~172$7%2SEEq?sCR?kKrT>BN(Fe{#=&270NVQ(size@O;Y3lx8RK9yY^mg zZ8?;N#15ge09Ux=CsvdP+OIF48;@c+5t*8S%{v~IqeEH^mAs#4lh>lR2Hl}dKvuGB z|M+FJ*el?;MWmnOAE58=fVA;5{peTdD@Ej$$62A&F+e0$__@~Oazyc=- z+Qb3}ixS5OB!H_c7fYz6cY-iVDl*;L8q>)Gd;>OuAPY%?z6c}I)@^)$ag5y7@nocF9a{QcZoBcF9PkimG zS2(r4%V}{*C%JQRSCjtt{08baNvPX6+B`;rA|bq5<#2G!1>4H z*_H@mP~0N!5OfgpGL9mnrwMoR}%Bj(kscvad;9JEvm+M zTZu@DA(7>-Bx-Z3Uc120Q@~rFdIih10o>sJm(-Gu3k@@!&ZKQDgLL*5Z&!_V6-{Ts zlkw7ztLY+}z-Hd<7HpVo&kP(tYnNx+j}%Tl>HB#&C+xh`-fC0#%ljnW4o@pQq0lA6 zQ{JyWM1y)@Xagy*3IJAXcx=7{H#xto}?4L)0d6;uuA6t#>^uvxw$ zRFx6@BO1UjrH4=;@W1o=I%3jf4dBDRC{otGYeQ;`?gQmGTQkMMlD2mX9kQRjB+bO* z!3Ud^w>LKA85Yq4V?RO3mhS{E1Y(NE|vsp83qa>ed@*ne&(_Uf#4y5%nPU{s|*UTw7}Lv(}IqDZ+9; z=Q^fx=HAa#((RB>EQjCrRY$;K?g796b8FS!U^gLhp9n;}J(?BzD%=GFiDZ2>N>cOq znNiViKry$OLR$sl?j04*J^(rF$@IImE*L|Qv6B7>=Djr$q&UyU4!TmLs;jnV6Zc6pX;R>#g zkb~By>6w}4zPiCI^vX>;PqY%(0p2|B5YE&wDD1LTY3O@OlIC9hd+V8)phL@ zsZmZ1sK{XXn$_}St&_rT;p$q%XvyZ4-N@(R6reWKFJB9Qnu*TyA(kN%esqTLzM zxxAL6rx-8}d%eEkft}U3k5;D)cwwh`97~=hN989GG3{;Ee;ga7WC^*yghUTMq#luL zjp|8usFC;2<`ccL!DLf5r%CH89 zK_OfcRj2z@DDAC5+MWpq#%FON?WpLt??s6F(!<|i@2k^J7r7c)Y*p5a z`{AZ7-5)4~F6h!RI*w|}wPMy)Ei}J6aJ$6&t|kjkQP_QtO6VSW-iqZ`c}HagIiETyGy)r;S5v+gp< z1czyuFTI1H-A1~r41_Xn{2*riy;^!J@a%MpnHs|RUltYOXb6KNhJ`BQ7ABkLf+Gv^ z(QszKnlOMgWk|l<6M^JM`*%No{B^&y=~B`DX~;1Uxw<}KlA6_sEqd+vDNrgVuv_a< zNGH7kDNeI5?U+_}PvG8Am>f^+ca#{rE>|;6DxKetA;HkR`3V)RGfrSRM{CJ&L|}r!VDCt0%6Z% zGWgv-EA@9C1Bw<`@6XZ99&5p)@n2Yqg-$RlzH=*@qW|b|e;yXg<JOpTFPf`uu#|q2&JcZ^`G6CU(ADRsNzXmw48I@wu)^QVG3&P0Ln$V|&Sctg{xYmP68_N&k&?%s z9x08wzB-ofSpcx3sh`b>luh?)OME$M-S;Erj^@>3<`l^`)kYjvYIVQF&#CdB=41T% z>i{SxWI&oVah6gMv~?s;9oTO9fp40{36%@6eN==E@~WRY9(U;VXG%6#ba-EG=ifL- zg}^UB0Pm9nelJHfc6V7CG+9^B6Ur;!J$%Mddde|~Zc+Ub7NmBadAVdinP@+48HK=S4!76C}(T*;y)?Rar4ByD#KY>sOQdJZEm$DIN6^d~^M7Y+?Z z{M--@GrRkqaDNs47{>l1j|`Ert2Za{m{B0|o0-f>P5WhE$f%U8*vp|!*~ut$s?PMG znD)8F5M6Hm(3yAA4vpUnU>eu>%QSxRo)(*`)TiocT9N1tH4)n%n-+-Yih-L)=LdJd z9W)iG1zYU$4TfD@8_FGDd zmEat_5pDKXyfv>YWvCZIQ80jB_eY#*M+s%+K|k-@V+$$R=ZZ-Hbi69|u|dV}*-RfV zzELY@RTxc=`HUDaN)sx`m}L8>?f}j8*EY5=mc1lX#HB zio~<>KPL&vK0?@Xy`1JU7_F8jl$lrjF6v~h|YaI*MAW|DA6Az_bm?By;aElo9;g6 zqdOhy@c#W?&?m_Sd-%Qp2Eos*q&zqhWgpaPyzSj?}sX+g*4yldTS=COg&TI9r zEBwCUgl~vZ)hkBI{TB7;2}O~HwX|A4UmxCDj;Xc5VV$n5Hj?>1e1cKyy!Ih3<><@D z!Gw7$ioF##eV4?j>jt^Z9IBTvo~SMs6GJY)k{2Q;FpxTKsq`=<*p5v<@HSdpf*Bro%IqQSscY$qlqV~gllN)N#9d6X;jfPw%;uz;128aV~PsWWT zJ~HS2kodWNk;~($ZUXY&_l0SleBei4w^{|=d;M^#)QA)wi=fFD=~ULa*@pw4|2zaO&}kQ% z*7{QK@ydlv(m)(*EP*S&^Bq%nAPKybW3&5-sx-4EJ^m0H&6^J! z?`}4~y_eBh_~A=Osa2r<*l&IGE{va_CMM~#7Gt|O&=%|aoA!|iwOH_BQXKSxUtd|) zCLY8Q5a#u$ygq(S#XsNCxU&58&wBLXU-c+d#xrmU<+C_E2^Ev*&~}YQE#pOYCP~RK zdH{`n>@rC@Kt)Xq0p?5`9G!=tWKo2KK8VFU&7H(N^r^dzRqbYm_X~myP(h0dqV9?6 zFVJ`Vf&Vx>7JQTg}8}JfcRE?kV zvvgpm^O%KH%stV3ou;;cS;R{xpYvfKoEC_U9PMjV%ny==)~3fXFrQ>)F))r%tpksk zbcjC*>5a;0W$jC>+8VRZL=>VrqrWb0FkcYikUyYA|HtWj7(>cC;9mU5&y()}dDL*F zRz{X6&wf{l18y&`+P)%*M5&FgtQ&FCoco|p*`tl=(tq}^n)@;s)u7Sq^qSU6 zQz`{!iAl=%_wOVmKY*mZkAB-JiA<3q&Z>pGANyj);r%Gz#>Eu;i#+Xn^diSm^%81H z`X_Rcu_sT$z2=&+Ua(4dFn)EH2xLf_mX;1_SH0Ka4?Q&P3FlLQTXd5Iai<3rbSV;;nHIJ zt9n+w8lh+XA+rRh$o*#L=5#tM{0S7}udCEQFzBqY zOcf*!Tkbl8n$qhXYT+z2*%PJhXqs;4>baVDE^R!JfSKeK#Z2Fq=TqG*zW4J>;p&OW zj~?Y~1`uIhxvmbO@G=%CLSCFJDrZznHB?kd-{w~sTBQ2k`|I>`?qO)AV%MmJ*kHQD z3iSx|FK^|4-<=c$!L~&1==iY$3QJ_$rbpJJ@_-*qD4u6#I>1M8PRQ9^A`^52%sqbfag>6rhq%)LCCV@c&v}iYs?>R3S*GXcZk!k-TPxn(ZXmE(liQuHvokmcBdkoL9domo_(F|eN;(b zE*mVi&oADfaY3EUMQlp(r`T14EvhdAjFBF?HtT5qdl(i|Vy?q_b|fE_GmLE23Zs$V zz&x)h&Y$lXOL)N08S3}rE3Vc*3XEu}H;DcLp7>WL7&<*S*sQh}8tYT=O~-vNS%cvh zlnZ`_2aHx)H=KX7sk`&pz(4&!aM-QrtQY0ryiuL2@nO-p$Jy^m_0^QaN>h>GyugOy zpPviRhy%f72}#;=OyHY?0mZHltmlar;JC^LprrDTltmwJ7NBOkb}A~Ct374eYMK_t zxKJ{lDx-7$nW__$yRfS9LB^_j;A``|k}B7Yg2RruRE1^AZNjX4 zR!vi$$>sN5Q2%)Bz?Xc1K=o&S(0+$PpWFhh0Uv>{ni9lzZ4@w1~KHWepHM%mKpGL?UOH{C2UlfpPQqLIJO^H^EzJt13!` zD#$;opa+vx9=MdW`6NDUzv|9ncKK~Ozn7R9kE)1~+%NfezwYlljZP0L#m7NCCGxcT z7*a)I0Ox*Rg7+;$j*!wJdQsuam!>893D_PBY)|gt;TU%BzC~G)Z%OQ2`?nRKA-%qj z!;RW)25YWkO441X|6^(m)_q?D|M!1h=)8!l-0R(!2Q8uTVm&PZyXg-hktpIoQO51I z4%$sR_v0~)RIb5at&mzgXH?2XlXfqn)OLk%o4rW*molfB_;yDq>GgNF_mnvI>+Fu1 zyy{+Tkp6KQa7j2CxYCh-cOD~z6=lKix6dyCES+#=R!w^V`tXr)g~9z|0*%A^oO=s& zti1E=7FK=~ysIlx*@^y(gBwoyD@qZ>P2~h0gq} zEW$8x8obz5mnI^G1m!Sn z8V2$(QnjBvPIpFBKvplr`Sb#xRp^4D=gp>pK*WuiT;F@vRi;g>;`K?Zl-Zr_|Kqa8 zpzSaUz&!jgFc;(-VCA%X@KU~(hfM!nVOp#wj8!oVj@eG2=6iZxQM#oxBvwcF)A8;x*q-h4+`Uw#U|%tZ0Hp1a% z?ac5bG(~&NatG_FHgC3a68B?juPwS$VWQvlU}ISWPQkO`uX>ZtaF1jn)E*Z@?>&C+ zsf)Q>CDJSJnm0ng8+QT#(Ut>s%gl%gW(f8TA~U_r8~h={MJDIcd41gJ-v$cxOLG=R z2q|)RB71_kkGFivET&@55X&b{+<_EUG&$_}zvCnmB74jkzl(2UihE_8W{7A1`+@%Y zWU@lAS%><83={5?7U0=H&?LcZ!k3m=2tgXb%!h%(uoqaxh~c2O(g&c`G%@PN__`ol znOJ^v)J>ycNjD89fx$l4aLSXQgikyQ;_Vx%d#sooKPc&%F{`Fpt+DV7h#t6|1ZbPPJ^w0kZ7<7H09ky-l(ot&kH|}Ya z=m9%Zx7P;I5v@=wBh$Z&u9obI@$qEm|Hsx_fJNDL-@}R^rL>4j!+^AaNT)O?okOQ| zhtegDOnlDfY<%FW{Wsi8A%{>MB%{Ro`p^fU976B!KQU1Q?kox9 z3jM%D6b1c_p2|l7k9X_Q;RDkk@Yn3nYy^G~{KW-(av4FwOAXMtVD5W$w#&YVpb2w3 z346r82B=}L#>$Z^D#0DY6qO2U+Q*2n10Bub)?;S(?piGF$4fyybC5P5os_pH4a7g>a_R-5 z=LpSzuUA#;07?fk*+)`E*Mn(-y1?}a*TUmzk+*vARLzh&Ow#$sZsivYW+MdZ2r^Rt zHrp8Ydr;o$7zc}+Xrug5$+`nC>7zs2TfBG;lmEK>Sl2?7Xh*9e#|+*^Hg)yuJzbx-Rk|jb)T-gmy(OpZPFaSi#cdZfJaiEo2*+TmRmADd=pFLp1j+Kya zw5e#v%i}W}8W<>#WCQg18%2RJ!o2`!Z$vX{u*IL)T~~m{TLQ2MAYaC2hovo`8%O&|0GZa&3*XlJWD=?Km!a z?P=cit9ljsQ^Tr!L+>WcZR8^hQNpsz0hdDTvorXhuZDUa6Is0 z1Hi9maJJLVM70>F%eB6?`+Vt_G(i?%RTLxaceidAiMdyf@6i9b_1u)P2e~ef^X^lC zZ&oCc2jfJeLsW%pPJnIUbc3G-SBIW9uvafZ=Dp_6kKZ@&rNmVNH*g(1*Ya(UBGvv_ zI{eK~)_~->Jz07Xtvx@j0b=QToSIIsQv_h6v>Xj`F7^Q90!B1_?_+L&cl)+bprtM_ zy4_3ySFCCEYp<&$2mzH?AD)QwiHxxYMT5LE6HH23@lln$bG{|JQGYWmr2}{%>slc)JcV@~tVL+oqc+3nz zb1sOGco!^+=u-8s1qh+%pMaDyw4~27e6HNUs^HT24&L|p)-r~^>D&}P2=-i|H(ag5N7`QZ(3o-K>tl%ds|Vn;C8yi{@T_K3EW zq@;WXGvmW3$-65Gz~m`z3Ed83WfJ67%1Q%g;OKir`Mz$46K7NJ)l`uNdjkV}0GMuu z_Km2f`4fGNUT2IZh0xH@`!H9l=7HX{ifX;<3%&KEWW7KRF+RmgM)fiwjbRbd7btFM z8@7*H00&^4ea)(Zsz|}_?)~VF!RH70XNSo^KV2M9w6coAHwO}^f*yo)$faV;EPX)3 znM*-IV)HkOUHdhaK1(r7z*xvCVJdv}rJKxaZcm*&t0oH5{-%1GdXt1O4cBgHgOA21 z9sld2j`8r4A!hr1#>!yS6ds%Z@<4z8V+&EZp=nB>i38a8O^9mXM?AaVDYyka;rdUN&&r-6o!E1E#zCltgvPFR&t)XSGdCgGz8n`Hcg_ zAC{^5S=fT@9Znjv$g%q3!rX4__0wZVwOknvM!l$M;x{DpL~HW+RRB?{W15)Z?F7ecLZ?RRoe3zt=1bQ1ECj7S2khVk z$m>2-=W0IedU7GJfETOYu%3*TJU6;W41_J%q@mRPKgyzzDjMZ+3pz{R6RA&@J z)Gse4&OrBkn?!6twsI-OdkNh_=^2Oi4_x0{2PwI}AFB|e?e+JD&w>7RRpH7b?Mjny zk0O_6WVD$Rlk3s6A%w6HvZ9f3IK(Fs2aK?@O)d0=lwW+&kf|f64-WL_vH5F;44f}p ze%|>|#4*efAO_p|bzcFOc^cqO6%tlm@{aXsiNRJ&0vImla{F_(=z)-jj9dFHH=dA0 zA--W0LL*7~Ol^;p6(WCSxZe)anL!8c^G-5&Uz^h-B+L*ccmGZjgIDjAWJ8~iEmoR- zz$0!l+Zn320S4d9`%OfGjCZPWkRMvWH8X{(0v;4;*xtX_ieo73vor?$ZW|sKJpg;( znhezf6_#G+Jr2A>R#>(TJa3T?$LL^dErLwD;@-H^yj9dLsvYg&K)OH?>hkORCAQ%% zv;_&*MbO&Uz$?T$EF?rxD(;z%RTBoLe9%b_PB@s}@1Pi%V$u`Phmu=%;H0ja;f_+Jpgz2;N71B9noS%M(Y+oc?c00_4@HcWm!S zlCCos@@y|1BUyNJo0St^pEPin>f+#u7ajb@{@C?(g5)ti#evmgbeS@3d1k9DO+? zO;N}4f9?Qcc0hLC7dYINLi}-+@_fFd9gZ-Tt@&m@?w}2#;W0xG(+hoEGKr(Rbh`J7 zb-v8WgG33(qwV_1N*MPG)38^$Dnz1RK}URNf9w>bdvekGZlL#tE|F*l{Jkc)}@ zQZA#YB(oMLyo$F`02Px9O$0y*_|!H&&vJ{U6i{Kswshh?*(o(9$Szth#T_cigJl59 zW3`MwDbG2@%A5iV9UdO0f1gL1r1Z0m=QHF{#v$3k2%B%080W|C5)HeXJ zLDhO1IO1f=5FrKm;>-ZKq70ktnjU1VPx!uE$lB^hzSUbMA8_vA%=|dc3Q&lT_{Hm; z9>m%jFn>?b8wWioKIH5(!JZ?TZLowQiMoA;P*32uL`6&4vZv6RqJ43z&B3Hxtzqrg zk5OH*aWR!1^2xcDd{%gUuC5orp zMQm%ZD@FG;_2BwztV=!@N%ARV&r8=PV$wN@rD(g&`oYi5B$xk}j7|%(F7Z|ok>~8I zN<76Iiiy5TwmuN54mz61tp2#x3iAEXl&{^01K@qZSp=u=I` zcO7c2_Gra6IYUDs5nu}O#F7Z7;jNX0t=<#k^q0Eea0oe zy67Dvc4ySvb3RKbE}jn7IQ>h%+`$mBG5?$TUrDv^BQb~N0`IkQ@Uh@5&!M*ssYih^ zhuTh#`BK_KksCg2&Gz$R zA}10TsHyuav>+V37zRm6BTcdP6XGx2209r+t?#o7a}8)5MM&VmtB=He()S%%CBC7t z7v0|GzIpoWX2RFrYQY_FA>%$~<^i9i&opI~hHkxfP&2wy=~BJLkZZ(5$X~B4d;lv^ z=~1M#5zm?HpKMs!)hm8UE1L!GqLo-^*AC^;&VA3f6qT)mLN{MpYQ50R2`kuWDg6Jgel zVo-LThcvI!%6@;s~C;DSX4^ymG}T10RUn zqOU6%da1ZO&ajL*@M7-&>^VeV*2k9o7BBZgck(3q zv#ULeQ^aRF_jgoxGqBm|(EPdKw{AdR1*yLc7`#psM%z-n=cV_5m3zz2u5ASerH#Bo zKB<$AG;y-*LkB-j$((V+gpt zt0ADxq6Qjq*||eey8$-C@$mo8pbS|3G9a2Df91EI!GY;@m#C^}Mbgz*w+Gj7=%GSa znpD6`C|ohg?rKT=KJGmONonJQg(ha0{(e!#+{tZDWq4-BbMrAt;G3|ccwYo7gbof@ zF2f>!ogihlv=L!Bgo;h7T+Ui8^K3>q**}10Vp7%@=@73ip6)fz!5vbbb0E;cZc&*q zg_coTGr*o8qQ$|kk?9rW&>P&I^9=nH+C(1kvB0tQ2{O~XpCO3W0>^r2&1CW_g=ZXc zmty-|lYHfKHIYb4rJj3GKYHz(AgHbGT^qED88Na0B0TI*T!!o`#B((=@yx|5x`{jQ zE$UmP%k)0`(yQjwOm%R~VIs`~m%CgM^p6I))}dS_Wvv~2<`YF`rMYiON`990_!BjO zf*0ZC*IZN+q*mvCZqM6A^*Rw2tO*y-ARZg-oNbvAiX3)njMRJ2jvE1 zQKXa4)LXU)DU7yDQY9uPt#` zaVQh%q=l({KfWgl+Q*2#{~6^W3M{JZ{YxM9DPUn!{o!WdXSzueC~V|3_NN~-P5JzM z?`-1p?Cx?O!Jh4W7=yWHMcdkRSj(H!xI_;$<(}x`N9jzdm~eTjt$U)3B(tNtlLX&e zRU7)Ks5pMUQ5dycHERKtHofw#k_HEd)3MiSob%}vvLaayw6@z9Lrkf-S9W&}{F#S3 zUhz=z83-qO-C|v>*sj^H*g(ts+sd;pzlHlKDM=d8hNYfzKq4F6cQzf3dW)Ub}sD6@I1B2E#>#42^{7=aKCwyng zAPM6$-ysQO$J_D(RQkSop1Sgvt==k+qt9Eb2`BV$yc`KFs{+#32Y&9B$OS<_{od%& zo4_>J9NC5yT}?jpDHmxN)i+_tKkQwZrsxqqnx2WUrO>FalJ3rMX5ijMr~{5wBvB`U z1NpOJ?S_dO9AK9Mf3&kwjM-H3p?EZVrt!T;5h|m=AW^cUxIa;>rjh?T3FDDFlWkmJ zf5krbb!#-9YGJQNdqyg^=QF>{T4vknyJ|LbdBa_1TNwH{NS`>xpdyJ*%PV2k?=M~L zQC7Pukv`!!4jO1eSNIskD996inTFag@Vtl*bI8x{Gcq;NNj?tXZdDxDh-Ab#h^4h7 zWIhd279XPij~2j9upnV8wN7YcnQzxcVjE-upZbcmxFY53rB{$Jrq0vL%O+;i7aok! zbd}2LjUp9mee0HOhW~%NRt+(Y_Lf^;_^z;Y_omF2bD4ms$0&q|d1JqbGsQ!g&;PQ( z*`mv-&(s)R;T5h#gvpHQxnmwO0u$dR0 z=Aqi%MfY67i*OBGAP8a$GDW-HRi))vHKKLCg*O+OGO$#vL_YTuxr9;}=1tPaHkupec1E zhMihV6?;}bPs89JpDy`vJP+t!yYR879f}AyZ6o*?W06}*5dnc`w5||-F(=0qk@=_^ zTc+cmT?o`>D3sgj@yq2r=v>plGnY0#_-na# zvl6;wlr(oABnia5yw*_(JfCri7TM$ZCNuYcxd$Na6RJnhio;<9lte|7dT}*CMKF$% zwC%~9#5}rl{v#9VgqE0{elX}+_%O%<=fl$eeSKWjRND`MTpL4)N{iSA^mxav_9=lx z->ULPt;$a(-i=%0fAa{#*2u#pyp>^Qj#;E1^#;wC3GcfrRBt*Lzmq&m2uOo8Srv%x z0<3TLAcG8fbCg4}qebx*r~$-5NKA2b3GBqL z=^A(i>AbvJBr4XzqA|#dju14JzlaO}k{Eo-sKIt)j1Tam^l8QO)+-KROSwhQ;o`be zTx)dov!A&aPvq#CByzo(m*CO*Ogq)ooUX z!+Wg-3IAAA3g24P2UcN`a5!9kF5>Lph55ZyXfHwS)o!c6MUD48z&nA1cu&9hXo&Y9 zZz`4DbSM^Rg5_Dy*S)&|?E^*7oQe&09Ql%FjL+weZLY?f1S0>&AxVQ@P39Gi*q^{_{hwk@1yFwCje=bn_1U z>gTapovVV^zL6@@##Ff>I~LK<-z`QVu-Lt*KFA~JJ^;S$2SARN$sL&oLBeIJX)7m4I7RA&juD*z- zr?2qR;FJoR*Vaw%R!58S8AJv~;EG!IcL8if3~mDZkfuY!{q=5yPICFYSsZGn!%dLLxb z<4oK8pmqQ6IMWsfdgHd*kH7@FJNd00_(GFgziFDsMvW8THyQyH*}7C4`0bhi8oEu( zbxmkLT;nb9)l>vcpWmNT4**3YHCE8^3;-dqIez=9T=lwnyY>}XbZBrcH8g5P(Rq7d zjUm@|ys9c$U|*`P#r%*!^ZRdya^(0{39^Iep?TB7q_0P8@SQ62m0g!@`p@OyBh$#l=VO*)l8oIk$?5|z#cTowB00sO|gD^E=!U1{goYb%EdM8^Pd z^@z9}>twyWLZ$fvi3a}X1X~tMmyDu3^#+V@ql3$O{O8Mx3`HoIgg?I3Z11E`M*yih zH>797&Fe5|tm~FXVLu0jCMo|rW$8HjxS5=yG;!qS61A8A#KpVX7y zoq!zkxxuyZ&Y-eWYaarl#6d#3anmb{E8Da&1LLwCot*(o5siOS^}K75DaGL({$Tu< zQt~?vTAH)X?Q&AYjW#ku3he$6w@BDb3oEUSO=W}H6VPpQU>WgCoF1FJd|0ol7nbvpC<0Pynl7X}DNw1@7!qk{Vk2)d4tChpU>YGie;htZ5$a0Ibpzv}Vb z1DU^o^(8>-8oBKs_zx&sg8bU+I*$0;uu96n%^JJn(qVtXfVM>)X%pbMvsp|q(kYXd zT5930zKfFhy+$U;`&Q4CBW_$ZXKHO&fN$}=N-!_|&zm9sjmML(q`6!cMcDxumXDoE*)XtLx?F$(5nqHf@xF~5^J_F&2Ld$( z>6KLv8F3hFb+hRY5J}zyMmZC7)L_Sm(`V(Za+vn?_^%in+f`P@QCH;5=K@eez! zl>w>SbS+|wKq+nVZRxL z5gOl6oYSI(<$m&hcIPvnM$<2O?!!?tr%+yc0$3+|8v4eb@61^MX8hXs@)$*Us}o@N zY@{!Qro;dZpd0m!1yqLSs19~7neLkL^@75ZBV~?^FN+r~lVd;c9Nw`L^;=r{Ya4{2 zeOqK=%@Vpw7GGuyBaDBEV1fGR;ryHo(LQ&|dB}MX`Uyv zXP>0+i30o{#A@L$U6V$tbgx=bD~$LBRzoJlbKCPBx!J2Jp4xi)X+niARc)zyl7HYY zd-Q5sZm-_>qwUttQ&;nOT}_&Ba|<$wwjrOFExkSE7DnrulLK96s&o|W-4YUQ1YBK? z;(BdW;cSkMjfpCuG;v1XyiK+&-^b^^TxQc|W}p@O&x4G!jQryO_VF$NgLP%G(tgKx zsimEF9BYf1o9h?igISWYz<^+fC0-sgepuVI=l!4>8M~_+C0Z@ri#{ggrkO^ zY!gV;;vD`PC4(FhpJBkS%{Z%+rdtlNEczlsUUmM8@ZF1}OR2fS2A6y)ZQgM-eER*s z!(SOzM>9*=`z)HKI}?nBj7&E5tbK+g;9%3k3rZ?`E#A_w(A>Gov>%n`#ZQHbeM3*{ z$;agN%jq>v;7OOe!pKdWxV`RpwDY{=%qQDNQiYx6|E(2`2#|okW;{qh^R^V>15XQw z-Xgvy6p8My3HNv7n(Ih0ZY_M3Wwp~cG^%gs{_9HY5jgJ#G=A3gyOQ7HpI`Q@YjUTc z)8K27FLV6d?njRQ^KerXBA&{3l1t2U)UZ2!lUX!dHk-*&=D;OaRm6D5jnghQBFGRU z9=BYUnEF47hR|KAuPt6)3BdCg2*gML*l+i2*AJ$`2eQPKm6a^(`Fb|95$YDyKCFLk zrmqBfETFSAZNCUrnyyV@f6NWf<~#aIGBj%DdDXCv_iIk(K&mZX5FVB-)BX5JdDd;c zevVsf=$QNj+4BOTVBHg=RAJN`n{|kIuqPL9(x2{<`^lba7j2HjD;7!W&_TCNwoWar z=yaZ!vS??$eS#5h){IOHR_8x2XB;K_l$r$`0Ym{*DS{ z9bQ~Z^I}$~)V8KGIMKcI897RivzkvyH+>oQJ%zNCB@)8W414d?*Z=X;hx&oLQjw3V zh?yiJQeu6Y5_|BA`JijQt3&*>Yn(D`VSRBgrRUyduJt7hW(#f#r%&B@oHlZFEv^rc z-AHalbJD3Pio>{!XqM|Py~K>?g1U{G)*~*#lMZoFcFtET zOkV3r9nW1xFq8UFJi6wJNoV63w1wVdy`eF-Zs3+mVQJxJBrCZRcRsjO(12_-3FHzq za@Rs!VR5KT!|ZM~j8Ze-@PV(i>#mKD%G%LS8{>s&EBm>SOtj(ZiEHC5g=-8LWWAcS zfivcLT+K$Uy^dOZedPv3vZYo)d6N%f)7C^gH1*Vh(0!lsS*F6l$Gb<$4Q|>OW2nC> zBm{9(b6d_5?vXight}>l*k@;qG#;jmtSn1kD2G7(seW$GiOKvJpjXp&QVBVEBFLla zyep7u$zQfu)9g~=6x|S1O?XQ>Uw%iS!0E^M#13q z_Jht_)*9&BzRjf8u7PvL#b8=YN&w`kfwlEa{i2}~ znk-Y|9Nia9<~`1h!F9z0u#cG(PDzaSFlv|2M|o%&o{138C70>;#WQ`(T3~6prjXNQ zmEjM5!wTcIM;x7r#8N0D*|6+vlj$)G{AX<#5A|SbsEjpYm9>`~F|ZGC44>TBH%u&z zJe(R>smOA=(>sS0suo`aFtz=yJYL!8x?TuxoyszKd^8GuhOD&dPIF9X)PhMT%+A@uA)G(speYJM&b8$|oac9ICvzx-v zTa)l2rwNZWrkEfrlz`a}88-jl$Qda_grKL2hkpL_PXdUGR%842a0T*qyHf+{6Do@D z%ys~aCd_BDmwkS-C-H+z!7;u#AF~q~W8rDa4M~kwfJB4_6!!h1w)KU=jWmU=C6q*&o9t%J`C8%omx+EmC`R?tD!yzR;jGTJXn!`|?STK8b_-i|R5@Qq{2@j4_YnkYlqvuR}Q0oiSplFie z+bWT!sqR8Yo1uzUG=o>l3t`$;wFfi#IgSU4vjSa1!D+8$cHvGH-`o33>g()HYFD17 zI_3M_3B!eXaISxdws9ejsZQIisFBd#9weF+-Y?WtOxgLjhVbyX4xH!j+V!tC9d*C^_e;+Q~lDxC{O!{%4SF^?T#RI){gAb-Cu6<#d%v+Gxl- zW|f>$w`JDEDXI5F?tiZGgAiL^ZllIuq|7dwYBUp;c+FI2x1EVew)Q8r==5*Tfej3c=q-< zs^}LLOdUzeiy2PaU2lzW@%Ft+x7z#m`qsMN%h=ye&^%(ZkMnr2x_zlgWj43u%bQBm3__ci%@3Bi@$9`f}j8 zTZvWTf>m9MfB{+ixJozNu|9=zLw7QMQH?TrzIAXYv2#8a)Yq8@vEKtksU-!DrZ*S+ z@#W;4MpydsX-*iV=aP!o1F2eiP0<(oug9$`i)QV6_Rj8CF*J?WCRCZf>lAyGF>T=o z>&b{TJ>^E9ga@h&7T_;7FfGDbGJP63Buj1dPpYK-QC zad1B73a53I*89um^G~51Ynwao(1AnrAua=qfm+5?urP^cnsR8l>Hhnf%|WLE9k0B- zU!@E}-$aPK6(PqAYTFNB7ER3i9U2}?98mUC$c(TuG3OO4oeE{keE91NKm4W!XgM>} zveHA{DP631bS@ zmB=dE$*XC`8kB~+pP0d6^eXub^t1l0^?kk~4x@J)ja0wy5s8l;dOVK2xgeeeCKpc6 z#_KJk(5E=J$UNPPb_KtO8Yt$9KQfbVXdG)St+Ed%;d2iNxiLlBG3d0g97)eG_JF-Q z-M>-rJU_uUv+lGLP42N%5>QYfE62NX+)tM=?>IbpGG~Q*=d}5K`>qUc&5yIEL9=ncW?7li?J(Mm4Q1v6_1yNV)fmb>SHk49 z6DiKIw*%?{>yX24;lm%ws0dAL4OR_yIJ6@T!0@r}#5 zNm|ZGe?{I{=aW#jT$CjkM=8SjPlHO;xF73adq1nA_}r+sh*|N|#jQ(X19^imx(BB2 zgDircgPkKI+HRvNWa#f>gG}e44R+YB)#X*y6J6S@m7c3ST|wm;TucYHScm+A?xmr7 zBjFBiEjPU(e%y~63I-CU6!ak57Iq~r=9R;S3(aMM$K8U8#S73EoHkw9q%tP_7Er~Z z>gex3NF6)FuiDR2D-F^KZMQH8lT@}F*X^-T>Cgx2Zx=ZXx@^wAx~9&B@Fb!zU$~uK zQ`g3`7}3Hn+N4Ft=`Z>m%qPyOY`U(0DATHM(rMidW&BJ2)#p0}fJWRWe5rm{`_HLx z?k$hV7DOiof2OyWKcK0w_=3K^Q8bISbsSV>=3Fb_I31Djqdbmrlj@SVLh>c~NE>m{ z=bECoP56)9`=o0Jf&b2NR>uIW{&Gt(>O*2Y2 zo|V@bLhxjcWvO_#k9Jghujh6@wn5a3lPAYt6*J`OX-}<{+to`bw|UZZ~4-*r*lM;C87<`iPfWvbI%&YWX$ngy>>-0>ON z#_QsAACTw+14F?%ANH2;Ht}|jrEkROePSXcwh1aZDR`>IR% zLN!G>ceYNRv6m!0n!_DQ4?A(+>d-KOHtE)yXatjt^zD(|aZJ&(1~(vv`I6>Ou=sAW0UCaq?hp;i9_?RW>6*f+IB(i*%oM*Cp3JSIGfsf3A)y;FlAyBd z5u~SNV&g1T#pd~QRk2i6Bi?fV%~F*~z_&}q#Qclpv0lG+|$GdaUaW5(?cAh)jb z(^p1TG%PbU?yXiJGSRik6Eo!&--7EX$cA%`sJ3?UM9fR^lGxtL7d!n&TYd3s<_wh$ zbOWC;9Itx1`Rn^%=bZ~s{uL?QXHarVG`{dZm z(CZMP-s?vCljZG)*IfR6Xz}UVQ%-ZQ)%Ay5Z`h~JAsB5OSGc#394f_K3O3OW%rzf? z0$VKg><_T$^A(X{^Gr12|2|-te1tIKbD$LtZ8#sv-;v)rw1Pz{_Vf|(8|)fe@Qwv= zaod>Fj_-aD{Gve^Y*Vie_@enn#$nCOE?9APkC4JDpO>g9h&PRE8W)hB4n2e$qEGKdktG>a>?2Dt<3*R9-{}rffG~dO~>j!&W-iji4)c2?B1JFshQ z;Iw=bqkFp(PUy#=u{c}<3R1SRv-a)8{r=I3t@?c?x8qblT{m@73%d{fOyuv~LO2Q- z1aA7YW)tz+j=$B;_M8xw+O+V}XVR9`7V9aHQL?TWI`8o7zOb4vZNTvt@pcX7I93J& zTbe}^PJ?B4XHLk~hA>s<9{YAO$WM%d^%n?C4IZPZOS&uP36aJPUxb!=+TywoZz@?8LKCW=)H~XQ zykUMwj4CNm`DXfN+Vn*I)ucg~aKpw<$o`PnNVgkkksE;bedwvd*U9g z?3nzl+ukM)kIlioDpR~Ku&l;^+*lfA8J*nR7iDf8DlS*Xq&iG^?A~3k)A&ZGRy)Da z>f$TOk@L+)!NNs&!+HX#!|n@BQGv@ZD0I;5G>>M}NgiKO^<^zk0UdAKTK-iP5dqZm zPJ2|-?@Cn^AI;yE+kHUuVZ2p`UGlZHL~UCX`c5%?+`xp+*(6X0G?}-KM%+yBKV0U_ zGSH{+T-VyW=$Hm`ED+=xKb~O(C8~Y(sgXzCrI`2+(+o0sjlL6Z%VqI}*1qjxnSVRK zQbRBEz^*MWf$dJ1eNNCaM)Ut7aPIv$X|(tuETMdLGXH0E>V-GQ3V!i^RCx@ix}E4` zfq5B$XptD8m|mW3-ne`fOLsa&*T-Web8@!P;?(YRx+hbavPEAqAAjOxBPuk|@(y&z zwXIon)ghVO?mt}OISye>?AC>~%p30-r`oIp>UU%xGAJJNoF0ae=M-*XMZlAb87UUb zjCb$(q+g2UAuCzly{x=RcEOE5Z0=xjA8y8;>vp+5;{VEBy6uhl|Bw;|7qYb+0kz98^D>pR@%4Xy9M zd5-$bc!K7w;TruVf!6Xe9`@}*vsv^Sv2U$9IA{#BCyFz|?q8;l@z*l%^J+65I`Yol zxgM{K-pX2QzPHnfKKi1dT#qU0@QDHG+{&?*OZc{8cfY&W@QZv;IlF$QhTu?J^xOB* zT{f7`+wAY_?yQ(#4d2t~XC*D;#N3};D=+U8dw1F(C(Ti4Q@K0kA@W#|M_l8Lwbo%Y zKdCmV#beXp(j3X$(z@BU3)`RRgaYLfeEpAaI^U0ZxW!DN62=66_+`Rx<0ZA;z-+2S z25Lnsvw};Jr+yF9Jmu?rcN{hd9m4l{=8c;k4!Y6=t4F-h^>7nStWDE<;W8cgjN41o z)HtPxg;BRjTjv{x8W{#-j;V&1yO6r~8pgsGvSOFxjFS@Z)0dy0;C zm)e^;9{Qd9zCgZu0CfD8)$&K5s}APQvRB83=9@9W=0sD16(^X355N7|E+J#*tzrKh z)^M>=_tS}YeL6LM-lRW{nY@3&Yp9Vp(xbyhlvsiQ%@v*Oq@ptB=4$8BBQ}Fq82+L*`(Y~BXl`80T{3K3?eswQYN|r9Df%<% z*&D@{j4CT7mEcJnT{s}1~ z10-ZVFSRr#_ZMCV`h3~jw2sonWxV*&1hk&)%v)rj(s(BC{19@uK=J(axsM*{y$@Q`E&a&bSFybp|L>c0DZ4;jO z_mVVx=r7@?HTu$=7F(uKZ{+!QE8v|Ja>H2Iqc3w%yFx94y2L$-dpw37iIcCENBGFX zJg6}fpRX#(aPd>0TldhRnaZy=EB0zGiQGCc4=}L}iC>8L(3V}BrT$kTHA3qH$Y=dP zYjb=JV3TWS)WG*JJd)&h2wp>k-~>W~B_223cKz)MTRGildt1u)+Z~l$Hgl2-hWM0U z7`pPTy1aXX$C}%&0l)X6BY@1E?#_E=In}J^-M55~x3JC%T{F=*aE4G;c4sqHXMGzu z4aMe0ZqFK-o#Y_EVfTo zb}SY&Q>^Y(%Pq6XQ%_?fvci8}$PwPZ=JELNieTSM?1WrR<-SUy0eS{7ASI3(am(!= z$tIzW}XL(a!gxMJ-k^+ump%B*|~=0LC;Q z%iVkl`tj9cPLUDj?b5gjEjQdQ3UwKq5q+r*eUvkHjt?OB9jcN@da-0Cz zH^I<)`DgG(ND;B@TId$bcz5_bE~)_G_esOQJQ00a0Q-q1+M3ZcAI54={E@phNk_*WaEY!x(R*-~G_YV-_%kxV=3OsNZKeVnj#w0dyg9A>;Tzhx`)l?dt|e zJ-XZS$d|eb?%yjT6omGTCEk{RdD3c%iK@$kxoTQ>I7&7MK~Er<_zWO5(CSG(&g0`H>4!|Q<+)Q zocy~G>p@xzLZ{m@&8;FfF1oYo(H+DgMD>4e*ztA?AW9w#uErI*Me#2bOVvR2I=dvMf zws_`pYU1=})JIg+B*uJNJsL@o9l(+af$vV|r*T02?|^GvF2nenQ=vssX53E*q`|{^ zgqM#n9`666e)CfbxS;Uj-<3Qq92J0;ai;+7l-^{NwEi-e6>Y7qL7x7F1NwF8Ma$HR zGi=gduXM;7m1`_{_0rf~wkH@F@WlnJQ&&9c(XLE1Iw^U1Luli#l^weasFxo6hjyVn zFVyk+XglX0Qpvv;9mxnjVeiVa*gwg?E;<2=ju!C!A-5oOiNWhUI-hwZKTC3RSY>~( za5|37d}am?s16M~1||9frYkKSuT?+lan*qxdShz`mUlp$5#Ji!{divC_F}i@92ID~ ztkq!2MGKLm?+X459kU|LEpo#q<|67TnSS^uB1C$BKwep0%?3=!0w+husvif7i`}Uu z&ke)+85rTekD*5}At2DX22_&RJhV^&A+EP<-P@nHSRgNZt?|^n zwCi5E;nX+RxM1RPwm@u4x)bxblL96LZOI?oCmbaFfy|?je;85gM1{-jJQ4nXdTMMm zK2uZb!J(l_F$WXHNgW*>Kkxhgebo8FN`^o&a&}+0T?Q5D>N=#<=9cjQ-l z4G3!gQR|@$5CSt_)D#rL_kkcponp1QN=xAD+IL1>C{S+7hmSuZgGk@ z|Aup-nXYaU&3)&+Te`W86s)i3rOI zH%|QHtbVbLtUuQsEfhs5wri&ZuTroL{JkFJILMJFR=v@XXx{C|=jQDeKHs7}5P?Ui z6@AX`Mo)_GxVGxegt~Y~1iCB2-0Qz@CfUk~#KSH3?{2JpOqZ)RM4)g!Riis8>5-#ec%B zH*#C5(AR)|gmMZ(-Ur6V%F3#b?SBUWAp3;LE&)Bz7(n-6K*l#ri=5%;GcqDh22@Xg zWU7~&H1L5g07ph(((}mS16j1pVY{b)qL>iUXL20JWyQ-{K#cAMbN+s;sHgv=4LCwb zz)J|`ae10gx=3>FKb9|b$mZ=j{Q!88VI#{9SNyQ5mA;z!;PEZs)t{vRCRpK!&x%kh z1gc2{@j}@mAD;IRvmhQ}(Z@bV>;3(JA2?RV%crSw(JWhgC<%QD)R;yveFSXO_m?s8 z#~d>7K8P_JMMN;HeSW)iO zs?;day0pT|qL51%WY3Uo@-+MB&-3uS^O*PhzH`p`p7VXbbI#|`Xmk*7wsp&5qKaF` zuMZa`)n$|l8SVwAnw43d8zmgUbThr{z(lP#w)~ip!2)t)s@E=+GRA4$b|P{GjgsRb}w}xL#c0 zw`awFHf01AKq zMFpp2B>V+$`1MRQ8X>c00rjTi|Jj@;9{|%0$r{@*@phY+kIi;knhe*aZ6IoKd7r_S zt)YtWg#Le+DwE88L^_|F!PeS zEdcI3coK+A=XPoN7%gS?v0ZcNhBXoQU`wD)MCV8z$?82LMY`mLq)r&AWJmn?Qlb|W z#IifFW!k8@R0SM6^HFzqcYqO*vH-QzcwXGIOJsW_OhkZwD7roavNNB%=Z7obne=z# zU{sMtCIfcX)+g#*WJ$cdYyAI9mn*m}t!0ZduqqaL2>?d!uT6D>NU}mj``hO9lHQkW zZTzQHv)jf6j?X1_roDfj{0wl0Avi7IdFnT8SQOYK3VxRQSp{um%3PjuMo%i5J5n&T2sOA zo!CBTO#p=!Tha~`6T8$A((47k=Nz-k?i>-bn1uJCJyceP7P#Ftye4CR$PCB(BOtn? zT%j?gZ9mkD7(9U|eHmV7ZcRaK8dd z8>j^v?zri>6bO_lahwUE+}wZvT)#<>NQP<*(|nuq9UBhA0AJn2{1$gTAi?!6$#tH%(SKp(5=1uS2J%K$`1 zH|Ex(Md+KhCa#a(NTDgqWXk>$VT{QMGi+Gvn00-(cU)*F(ti{j&EdQrNZu}H{JglT z`Jrwwb7+45F4Od(&dN%oUEP9+6hSH87+}>C|~i zvOJa7U<(lJbKA zrXei|M-u58E~7XiII^J!9E_J1<7?vh4=mDvfqa*{Dx?yc8Z4xNDti@xj`npF$sH|u zT}0ccB@6zq0Epj#2i%2VAIP)eKgK$ZEb^8wi-#q4(n&w4%fz7OP>N4^cvr=M+Oo`U zM;raRM{7Lfog`-)uSXVHmcrBEy1@fu#jpo*MCwWGK;kv*7VPsC)ws1-82d%jo^M}_ zF85;*o4CqXKK4T^LvNmemI4vcXNfBGr`$YmMhqC-@z}6u5dln1=N|}S(Ghx}ElWJo zSopEX#Ariocr5u?zk1TVpd#r(+PSk_P;+5!0S&co6ayjSa61%C3B=8(CMcPmzkK}y za{k*g{`=eF=cPgq){>8FFDEwUrdmM}hDLX;%3ESwO~wXgSj?H;x=+UX1_~fHBT`OP zH<{H=O`VmGligXK&#DJf&MnXXBoNcVzjtc^ar)@2B+pPgfI?|%;$@QSY|K3-o5dt< z>{LHJT_|Na186tq9^M>%VB+aq4e+l;41W46-h_EgZ3~)#L&iOKm5^eYRH%i+z#La4$=r8p;GN z>xDBu7Bc8zXV$-mUlIbyd?$2f0~OF?Wq^NkVV17ez(!T4q~s#*Nl}>{pZlD$;YweE z$hYYfu@rpQsLlX`@qp^5KR4YW_*%Q|hTh7400uwI=go%=M9J0kPMujRn#c6ph0)8> zsmzADVLVirjSXmd9T5mm@#dK@3a!-^sF*UIq--cL{nL|K$%ISX8n%E@ zW6&B*Ou4vtCid(8&s_Qz^{CLS$H#1pa3)2}7CY9*Fd3U*_0u%5uM(PqD$4P!+`^Zz zP%xWKk1o7~%L9NVi*_l^ESq>g)@s&gqBD_@1LT7yhkVA&SW=|4VtbKG&qR4#_TPis zsW_dvUX9Y3(ue15ebB+b(@iUb0$YD`rEeo1Ytv{^O%^Yw8*9!4>*E6NQI0%hJCDkZaE(&m*Xc z6+txKQ74t)7P*7?$(ce>W}w($|Dcm2rc!^-7v7)L&)EL5+@E4qQf_0-1b)!qI$;5Uo~B>aS+Ik6jHTZSB(&wAFn;JtG2dBOzU{wO2fj0 zx;R34%$ESw-BR|a) zrLm+jM?6%Zs!H1mJpQFplR0BFHRcfZ^P)A5SeVT2?C6{r(Z7NUhvY8Mg%_ZoaKXp< zy2D*h<5Sd}DDnpxq4<=fDeS4nRBGq2GX$lR(BN?BR)2FTVb<;YOLmf>P}W4<(Y5BnjzosNMl z?>v8lH{u)c-?8z}LpcjFnreO8$ui<-=2)p@CgpXflPnrW2r*C(PjnhC#>` z`g4JcLLN?Em@QJPUvC!{99FEL9hwcqZJUf!o4@Umr1s`|A!i?Y7ple8bLinAoctL) zjRP~^;W3Hw;wr~TSaTKjc< Date: Fri, 19 Apr 2024 13:59:30 +0800 Subject: [PATCH 13/22] ci: fix poetry in integration test (#16394) Signed-off-by: xxchan --- ci/scripts/integration-tests.sh | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/ci/scripts/integration-tests.sh b/ci/scripts/integration-tests.sh index 90d23b2d80355..9502c297f87ca 100755 --- a/ci/scripts/integration-tests.sh +++ b/ci/scripts/integration-tests.sh @@ -24,17 +24,17 @@ shift $((OPTIND -1)) echo "export INTEGRATION_TEST_CASE=${case}" > env_vars.sh -echo "--- clean up docker" +echo "~~~ clean up docker" if [ $(docker ps -aq |wc -l) -gt 0 ]; then docker rm -f $(docker ps -aq) fi docker network prune -f docker volume prune -f -echo "--- ghcr login" +echo "~~~ ghcr login" echo "$GHCR_TOKEN" | docker login ghcr.io -u "$GHCR_USERNAME" --password-stdin -echo "--- case: ${case}, format: ${format}" +echo "+++ set RW_IMAGE" if [[ -n "${RW_IMAGE_TAG+x}" ]]; then export RW_IMAGE="ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_TAG}" @@ -47,18 +47,24 @@ if [ "${BUILDKITE_SOURCE}" == "schedule" ]; then echo Docker image: "$RW_IMAGE" fi +if [ -z "${RW_IMAGE+x}" ]; then + echo "RW_IMAGE is not set. The image defined in docker-compose.yml will be used." +fi + +echo "--- case: ${case}, format: ${format}" + if [ "${case}" == "client-library" ]; then cd integration_tests/client-library python3 client_test.py exit 0 fi -echo "--- install postgresql" +echo "~~~ install postgresql" sudo yum install -y postgresql15 -echo "--- install poetry" +echo "~~~ install poetry" curl -sSL https://install.python-poetry.org | POETRY_VERSION=1.8.0 python3 - - +export PATH=$PATH:$HOME/.local/bin echo "--- download rwctest-key" From be57d0a4d78b31ac63d06c590a235476abb25bd0 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 19 Apr 2024 15:48:55 +0800 Subject: [PATCH 14/22] fix(frontend): add `stream_job_status` to `TableCatalog` (#16398) --- src/common/src/catalog/mod.rs | 24 +++++++++++++++++++ src/frontend/src/catalog/table_catalog.rs | 16 ++++++++++--- .../optimizer/plan_node/stream_materialize.rs | 3 ++- src/frontend/src/optimizer/plan_node/utils.rs | 3 ++- .../src/scheduler/distributed/query.rs | 4 +++- 5 files changed, 44 insertions(+), 6 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 0fc23af8c1051..1909ca1635f7c 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -29,6 +29,7 @@ use parse_display::Display; pub use physical_table::*; use risingwave_pb::catalog::{ CreateType as PbCreateType, HandleConflictBehavior as PbHandleConflictBehavior, + StreamJobStatus as PbStreamJobStatus, }; use risingwave_pb::plan_common::ColumnDescVersion; pub use schema::{test_utils as schema_test_utils, Field, FieldDisplay, Schema}; @@ -478,6 +479,29 @@ impl ConflictBehavior { } } +#[derive(Clone, Copy, Debug, Default, Display, Hash, PartialOrd, PartialEq, Eq, Ord)] +pub enum StreamJobStatus { + #[default] + Creating, + Created, +} + +impl StreamJobStatus { + pub fn from_proto(stream_job_status: PbStreamJobStatus) -> Self { + match stream_job_status { + PbStreamJobStatus::Creating => StreamJobStatus::Creating, + PbStreamJobStatus::Created | PbStreamJobStatus::Unspecified => StreamJobStatus::Created, + } + } + + pub fn to_proto(self) -> PbStreamJobStatus { + match self { + StreamJobStatus::Creating => PbStreamJobStatus::Creating, + StreamJobStatus::Created => PbStreamJobStatus::Created, + } + } +} + #[derive(Clone, Copy, Debug, Display, Hash, PartialOrd, PartialEq, Eq, Ord)] pub enum CreateType { Foreground, diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 10ce56a1060cf..f5f95861d29cd 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -17,7 +17,8 @@ use std::collections::{HashMap, HashSet}; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ - ColumnCatalog, ConflictBehavior, CreateType, Field, Schema, TableDesc, TableId, TableVersionId, + ColumnCatalog, ConflictBehavior, CreateType, Field, Schema, StreamJobStatus, TableDesc, + TableId, TableVersionId, }; use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; @@ -154,6 +155,10 @@ pub struct TableCatalog { /// Indicate whether to create table in background or foreground. pub create_type: CreateType, + /// Indicate the stream job status, whether it is created or creating. + /// If it is creating, we should hide it. + pub stream_job_status: StreamJobStatus, + /// description of table, set by `comment on`. pub description: Option, @@ -413,7 +418,7 @@ impl TableCatalog { initialized_at_epoch: self.initialized_at_epoch.map(|epoch| epoch.0), created_at_epoch: self.created_at_epoch.map(|epoch| epoch.0), cleaned_by_watermark: self.cleaned_by_watermark, - stream_job_status: PbStreamJobStatus::Creating.into(), + stream_job_status: self.stream_job_status.to_proto().into(), create_type: self.create_type.to_proto().into(), description: self.description.clone(), incoming_sinks: self.incoming_sinks.clone(), @@ -481,6 +486,9 @@ impl From for TableCatalog { let id = tb.id; let tb_conflict_behavior = tb.handle_pk_conflict_behavior(); let table_type = tb.get_table_type().unwrap(); + let stream_job_status = tb + .get_stream_job_status() + .unwrap_or(PbStreamJobStatus::Created); let create_type = tb.get_create_type().unwrap_or(PbCreateType::Foreground); let associated_source_id = tb.optional_associated_source_id.map(|id| match id { OptionalAssociatedSourceId::AssociatedSourceId(id) => id, @@ -543,6 +551,7 @@ impl From for TableCatalog { initialized_at_epoch: tb.initialized_at_epoch.map(Epoch::from), cleaned_by_watermark: tb.cleaned_by_watermark, create_type: CreateType::from_proto(create_type), + stream_job_status: StreamJobStatus::from_proto(stream_job_status), description: tb.description, incoming_sinks: tb.incoming_sinks.clone(), created_at_cluster_version: tb.created_at_cluster_version.clone(), @@ -639,7 +648,7 @@ mod tests { cardinality: None, created_at_epoch: None, cleaned_by_watermark: false, - stream_job_status: PbStreamJobStatus::Creating.into(), + stream_job_status: PbStreamJobStatus::Created.into(), create_type: PbCreateType::Foreground.into(), description: Some("description".to_string()), incoming_sinks: vec![], @@ -700,6 +709,7 @@ mod tests { created_at_epoch: None, initialized_at_epoch: None, cleaned_by_watermark: false, + stream_job_status: StreamJobStatus::Created, create_type: CreateType::Foreground, description: Some("description".to_string()), incoming_sinks: vec![], diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 821176a5af302..3da52ef4f55a2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -19,7 +19,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ - ColumnCatalog, ConflictBehavior, CreateType, TableId, OBJECT_ID_PLACEHOLDER, + ColumnCatalog, ConflictBehavior, CreateType, StreamJobStatus, TableId, OBJECT_ID_PLACEHOLDER, }; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; @@ -262,6 +262,7 @@ impl StreamMaterialize { initialized_at_epoch: None, cleaned_by_watermark: false, create_type: CreateType::Foreground, // Will be updated in the handler itself. + stream_job_status: StreamJobStatus::Creating, description: None, incoming_sinks: vec![], initialized_at_cluster_version: None, diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index dc957735e676e..f16aed4521180 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -21,7 +21,7 @@ use itertools::Itertools; use pretty_xmlish::{Pretty, Str, StrAssocArr, XmlNode}; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, ConflictBehavior, CreateType, Field, FieldDisplay, Schema, - OBJECT_ID_PLACEHOLDER, + StreamJobStatus, OBJECT_ID_PLACEHOLDER, }; use risingwave_common::constants::log_store::v2::{ KV_LOG_STORE_PREDEFINED_COLUMNS, PK_ORDERING, VNODE_COLUMN_INDEX, @@ -176,6 +176,7 @@ impl TableCatalogBuilder { // NOTE(kwannoel): This may not match the create type of the materialized table. // It should be ignored for internal tables. create_type: CreateType::Foreground, + stream_job_status: StreamJobStatus::Creating, description: None, incoming_sinks: vec![], initialized_at_cluster_version: None, diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 004c16071ecf4..c6e866630067b 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -474,7 +474,8 @@ pub(crate) mod tests { WorkerNodeManager, WorkerNodeSelector, }; use risingwave_common::catalog::{ - ColumnCatalog, ColumnDesc, ConflictBehavior, CreateType, DEFAULT_SUPER_USER_ID, + ColumnCatalog, ColumnDesc, ConflictBehavior, CreateType, StreamJobStatus, + DEFAULT_SUPER_USER_ID, }; use risingwave_common::hash::ParallelUnitMapping; use risingwave_common::types::DataType; @@ -590,6 +591,7 @@ pub(crate) mod tests { cleaned_by_watermark: false, created_at_epoch: None, initialized_at_epoch: None, + stream_job_status: StreamJobStatus::Creating, create_type: CreateType::Foreground, description: None, incoming_sinks: vec![], From 47ad484a89d0c236e6a0e0a419ed5a5f81ae3817 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Fri, 19 Apr 2024 16:29:52 +0800 Subject: [PATCH 15/22] refactor(storage): pass sync result in commit epoch in test (#16403) --- .../src/hummock/mock_hummock_meta_client.rs | 55 +++++++------------ src/meta/src/stream/stream_manager.rs | 9 +-- src/rpc_client/src/hummock_meta_client.rs | 8 +-- src/rpc_client/src/meta_client.rs | 10 +--- src/storage/hummock_sdk/src/lib.rs | 14 +++++ .../src/bin/replay/replay_impl.rs | 4 +- .../hummock_test/src/compactor_tests.rs | 40 ++++---------- .../hummock_test/src/failpoint_tests.rs | 16 ++---- .../hummock_test/src/hummock_storage_tests.rs | 26 ++++----- .../hummock_test/src/snapshot_tests.rs | 32 +++-------- .../hummock_test/src/state_store_tests.rs | 34 ++++-------- src/storage/hummock_test/src/test_utils.rs | 8 +-- .../event_handler/hummock_event_handler.rs | 3 +- src/storage/src/hummock/event_handler/mod.rs | 4 +- .../src/hummock/hummock_meta_client.rs | 8 +-- .../src/hummock/store/hummock_storage.rs | 2 +- src/storage/src/memory.rs | 2 +- src/storage/src/monitor/monitored_store.rs | 2 +- src/storage/src/monitor/traced_store.rs | 2 +- src/storage/src/panic_store.rs | 2 +- src/storage/src/store.rs | 17 +----- src/storage/src/store_impl.rs | 4 +- .../common/log_store_impl/kv_log_store/mod.rs | 3 +- .../src/common/table/test_state_table.rs | 24 ++------ src/stream/src/task/barrier_manager.rs | 3 +- .../src/task/barrier_manager/managed_state.rs | 2 +- .../src/delete_range_runner.rs | 2 +- 27 files changed, 114 insertions(+), 222 deletions(-) diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 74c760e25776a..9a888e27bc78e 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; use std::sync::atomic::{AtomicU32, Ordering}; use std::sync::Arc; use std::time::SystemTime; @@ -22,13 +21,11 @@ use async_trait::async_trait; use fail::fail_point; use futures::stream::BoxStream; use futures::{Stream, StreamExt}; -use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; -use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, - SstObjectIdRange, + SstObjectIdRange, SyncResult, }; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; @@ -93,31 +90,6 @@ impl MockHummockMetaClient { .await .unwrap_or(None) } - - pub async fn commit_epoch_with_watermark( - &self, - epoch: HummockEpoch, - sstables: Vec, - new_table_watermarks: HashMap, - ) -> Result<()> { - let sst_to_worker = sstables - .iter() - .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), self.context_id)) - .collect(); - self.hummock_manager - .commit_epoch( - epoch, - CommitEpochInfo::new( - sstables.into_iter().map(Into::into).collect(), - new_table_watermarks, - sst_to_worker, - None, - ), - ) - .await - .map_err(mock_err)?; - Ok(()) - } } fn mock_err(error: super::error::Error) -> RpcError { @@ -183,17 +155,28 @@ impl HummockMetaClient for MockHummockMetaClient { }) } - async fn commit_epoch( - &self, - epoch: HummockEpoch, - sstables: Vec, - ) -> Result<()> { - let sst_to_worker = sstables + async fn commit_epoch(&self, epoch: HummockEpoch, sync_result: SyncResult) -> Result<()> { + let sst_to_worker = sync_result + .uncommitted_ssts .iter() .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), self.context_id)) .collect(); + let new_table_watermark = sync_result.table_watermarks; + self.hummock_manager - .commit_epoch(epoch, CommitEpochInfo::for_test(sstables, sst_to_worker)) + .commit_epoch( + epoch, + CommitEpochInfo::new( + sync_result + .uncommitted_ssts + .into_iter() + .map(|sst| sst.into()) + .collect(), + new_table_watermark, + sst_to_worker, + None, + ), + ) .await .map_err(mock_err)?; Ok(()) diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 94b97abcdc007..6a801932785c7 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -872,14 +872,7 @@ mod tests { let _ = tx.send(Ok(StreamingControlStreamResponse { response: Some( streaming_control_stream_response::Response::CompleteBarrier( - BarrierCompleteResponse { - request_id: "".to_string(), - status: None, - create_mview_progress: vec![], - synced_sstables: vec![], - worker_id: 0, - table_watermarks: Default::default(), - }, + BarrierCompleteResponse::default(), ), ), })); diff --git a/src/rpc_client/src/hummock_meta_client.rs b/src/rpc_client/src/hummock_meta_client.rs index 573ebfb2698d8..6e1dfec3b7be3 100644 --- a/src/rpc_client/src/hummock_meta_client.rs +++ b/src/rpc_client/src/hummock_meta_client.rs @@ -16,7 +16,7 @@ use async_trait::async_trait; use futures::stream::BoxStream; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ - HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, SstObjectIdRange, + HummockEpoch, HummockSstableObjectId, HummockVersionId, SstObjectIdRange, SyncResult, }; use risingwave_pb::hummock::{ HummockSnapshot, SubscribeCompactionEventRequest, SubscribeCompactionEventResponse, VacuumTask, @@ -37,11 +37,7 @@ pub trait HummockMetaClient: Send + Sync + 'static { async fn get_snapshot(&self) -> Result; async fn get_new_sst_ids(&self, number: u32) -> Result; // We keep `commit_epoch` only for test/benchmark. - async fn commit_epoch( - &self, - epoch: HummockEpoch, - sstables: Vec, - ) -> Result<()>; + async fn commit_epoch(&self, epoch: HummockEpoch, sync_result: SyncResult) -> Result<()>; async fn update_current_epoch(&self, epoch: HummockEpoch) -> Result<()>; async fn report_vacuum_task(&self, vacuum_task: VacuumTask) -> Result<()>; async fn trigger_manual_compaction( diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index d438c13ba7a00..fbb8dff1f5a98 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -37,8 +37,8 @@ use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ - CompactionGroupId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, - SstObjectIdRange, + CompactionGroupId, HummockEpoch, HummockSstableObjectId, HummockVersionId, SstObjectIdRange, + SyncResult, }; use risingwave_pb::backup_service::backup_service_client::BackupServiceClient; use risingwave_pb::backup_service::*; @@ -1392,11 +1392,7 @@ impl HummockMetaClient for MetaClient { Ok(SstObjectIdRange::new(resp.start_id, resp.end_id)) } - async fn commit_epoch( - &self, - _epoch: HummockEpoch, - _sstables: Vec, - ) -> Result<()> { + async fn commit_epoch(&self, _epoch: HummockEpoch, _sync_result: SyncResult) -> Result<()> { panic!("Only meta service can commit_epoch in production.") } diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index aa095b6c66322..1fbd627e13f02 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -27,6 +27,7 @@ mod key_cmp; use std::cmp::Ordering; +use std::collections::HashMap; pub use key_cmp::*; use risingwave_common::util::epoch::EPOCH_SPILL_TIME_MASK; @@ -48,6 +49,9 @@ pub mod table_watermark; pub mod version; pub use compact::*; +use risingwave_common::catalog::TableId; + +use crate::table_watermark::TableWatermarks; pub type HummockSstableObjectId = u64; pub type HummockSstableId = u64; @@ -89,6 +93,16 @@ macro_rules! info_in_release { } } +#[derive(Default, Debug)] +pub struct SyncResult { + /// The size of all synced shared buffers. + pub sync_size: usize, + /// The `sst_info` of sync. + pub uncommitted_ssts: Vec, + /// The collected table watermarks written by state tables. + pub table_watermarks: HashMap, +} + #[derive(Debug, Clone)] pub struct LocalSstableInfo { pub compaction_group_id: CompactionGroupId, diff --git a/src/storage/hummock_test/src/bin/replay/replay_impl.rs b/src/storage/hummock_test/src/bin/replay/replay_impl.rs index 43899fa7e256c..3b7f4bc74740e 100644 --- a/src/storage/hummock_test/src/bin/replay/replay_impl.rs +++ b/src/storage/hummock_test/src/bin/replay/replay_impl.rs @@ -20,7 +20,7 @@ use futures_async_stream::try_stream; use risingwave_common::util::addr::HostAddr; use risingwave_common_service::observer_manager::{Channel, NotificationClient, ObserverError}; use risingwave_hummock_sdk::key::TableKey; -use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use risingwave_hummock_trace::{ GlobalReplay, LocalReplay, LocalReplayRead, ReplayItem, ReplayRead, ReplayStateStore, ReplayWrite, Result, TraceError, TracedBytes, TracedInitOptions, TracedNewLocalOptions, @@ -33,7 +33,7 @@ use risingwave_pb::meta::{SubscribeResponse, SubscribeType}; use risingwave_storage::hummock::store::LocalHummockStorage; use risingwave_storage::hummock::HummockStorage; use risingwave_storage::store::{ - to_owned_item, LocalStateStore, StateStoreIterExt, StateStoreRead, SyncResult, + to_owned_item, LocalStateStore, StateStoreIterExt, StateStoreRead, }; use risingwave_storage::{StateStore, StateStoreIter, StateStoreReadIter}; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 3718b06f00fe5..7b8da8c92f077 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -186,13 +186,9 @@ pub(crate) mod tests { } else { local.seal_current_epoch(u64::MAX, SealCurrentEpochOptions::for_test()); } - let ssts = storage - .seal_and_sync_epoch(epoch) - .await - .unwrap() - .uncommitted_ssts; + let res = storage.seal_and_sync_epoch(epoch).await.unwrap(); - hummock_meta_client.commit_epoch(epoch, ssts).await.unwrap(); + hummock_meta_client.commit_epoch(epoch, res).await.unwrap(); } } @@ -537,12 +533,8 @@ pub(crate) mod tests { storage: &HummockStorage, epoch: u64, ) { - let ssts = storage - .seal_and_sync_epoch(epoch) - .await - .unwrap() - .uncommitted_ssts; - hummock_meta_client.commit_epoch(epoch, ssts).await.unwrap(); + let res = storage.seal_and_sync_epoch(epoch).await.unwrap(); + hummock_meta_client.commit_epoch(epoch, res).await.unwrap(); } async fn prepare_data( @@ -766,12 +758,8 @@ pub(crate) mod tests { storage.seal_current_epoch(next_epoch, SealCurrentEpochOptions::for_test()); other.seal_current_epoch(next_epoch, SealCurrentEpochOptions::for_test()); - let ssts = global_storage - .seal_and_sync_epoch(epoch) - .await - .unwrap() - .uncommitted_ssts; - hummock_meta_client.commit_epoch(epoch, ssts).await.unwrap(); + let res = global_storage.seal_and_sync_epoch(epoch).await.unwrap(); + hummock_meta_client.commit_epoch(epoch, res).await.unwrap(); } // Mimic dropping table @@ -959,12 +947,8 @@ pub(crate) mod tests { local.flush().await.unwrap(); local.seal_current_epoch(next_epoch, SealCurrentEpochOptions::for_test()); - let ssts = storage - .seal_and_sync_epoch(epoch) - .await - .unwrap() - .uncommitted_ssts; - hummock_meta_client.commit_epoch(epoch, ssts).await.unwrap(); + let res = storage.seal_and_sync_epoch(epoch).await.unwrap(); + hummock_meta_client.commit_epoch(epoch, res).await.unwrap(); } let manual_compcation_option = ManualCompactionOption { @@ -1158,12 +1142,8 @@ pub(crate) mod tests { .unwrap(); local.flush().await.unwrap(); local.seal_current_epoch(next_epoch, SealCurrentEpochOptions::for_test()); - let ssts = storage - .seal_and_sync_epoch(epoch) - .await - .unwrap() - .uncommitted_ssts; - hummock_meta_client.commit_epoch(epoch, ssts).await.unwrap(); + let res = storage.seal_and_sync_epoch(epoch).await.unwrap(); + hummock_meta_client.commit_epoch(epoch, res).await.unwrap(); } let manual_compcation_option = ManualCompactionOption { diff --git a/src/storage/hummock_test/src/failpoint_tests.rs b/src/storage/hummock_test/src/failpoint_tests.rs index 98fe2759abfcb..d2ab797cb634a 100644 --- a/src/storage/hummock_test/src/failpoint_tests.rs +++ b/src/storage/hummock_test/src/failpoint_tests.rs @@ -140,12 +140,8 @@ async fn test_failpoints_state_store_read_upload() { ); // sync epoch1 test the read_error - let ssts = hummock_storage - .seal_and_sync_epoch(1) - .await - .unwrap() - .uncommitted_ssts; - meta_client.commit_epoch(1, ssts).await.unwrap(); + let res = hummock_storage.seal_and_sync_epoch(1).await.unwrap(); + meta_client.commit_epoch(1, res).await.unwrap(); hummock_storage .try_wait_epoch(HummockReadEpoch::Committed(1)) .await @@ -216,12 +212,8 @@ async fn test_failpoints_state_store_read_upload() { assert!(result.is_err()); fail::remove(mem_upload_err); - let ssts = hummock_storage - .seal_and_sync_epoch(3) - .await - .unwrap() - .uncommitted_ssts; - meta_client.commit_epoch(3, ssts).await.unwrap(); + let res = hummock_storage.seal_and_sync_epoch(3).await.unwrap(); + meta_client.commit_epoch(3, res).await.unwrap(); hummock_storage .try_wait_epoch(HummockReadEpoch::Committed(3)) .await diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 940eeeaabbc29..f7711b7fcdf13 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -534,7 +534,7 @@ async fn test_state_store_sync() { let res = test_env.storage.seal_and_sync_epoch(epoch1).await.unwrap(); test_env .meta_client - .commit_epoch(epoch1, res.uncommitted_ssts) + .commit_epoch(epoch1, res) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch1).await; @@ -576,7 +576,7 @@ async fn test_state_store_sync() { let res = test_env.storage.seal_and_sync_epoch(epoch2).await.unwrap(); test_env .meta_client - .commit_epoch(epoch2, res.uncommitted_ssts) + .commit_epoch(epoch2, res) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch2).await; @@ -780,7 +780,7 @@ async fn test_delete_get() { let res = test_env.storage.seal_and_sync_epoch(epoch1).await.unwrap(); test_env .meta_client - .commit_epoch(epoch1, res.uncommitted_ssts) + .commit_epoch(epoch1, res) .await .unwrap(); let epoch2 = epoch1.next_epoch(); @@ -802,7 +802,7 @@ async fn test_delete_get() { let res = test_env.storage.seal_and_sync_epoch(epoch2).await.unwrap(); test_env .meta_client - .commit_epoch(epoch2, res.uncommitted_ssts) + .commit_epoch(epoch2, res) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch2).await; @@ -960,13 +960,13 @@ async fn test_multiple_epoch_sync() { test_env .meta_client - .commit_epoch(epoch2, sync_result2.uncommitted_ssts) + .commit_epoch(epoch2, sync_result2) .await .unwrap(); test_env .meta_client - .commit_epoch(epoch3, sync_result3.uncommitted_ssts) + .commit_epoch(epoch3, sync_result3) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch3).await; @@ -1114,12 +1114,12 @@ async fn test_iter_with_min_epoch() { let sync_result2 = test_env.storage.seal_and_sync_epoch(epoch2).await.unwrap(); test_env .meta_client - .commit_epoch(epoch1, sync_result1.uncommitted_ssts) + .commit_epoch(epoch1, sync_result1) .await .unwrap(); test_env .meta_client - .commit_epoch(epoch2, sync_result2.uncommitted_ssts) + .commit_epoch(epoch2, sync_result2) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch2).await; @@ -1390,7 +1390,7 @@ async fn test_hummock_version_reader() { let sync_result1 = test_env.storage.seal_and_sync_epoch(epoch1).await.unwrap(); test_env .meta_client - .commit_epoch(epoch1, sync_result1.uncommitted_ssts) + .commit_epoch(epoch1, sync_result1) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch1).await; @@ -1398,7 +1398,7 @@ async fn test_hummock_version_reader() { let sync_result2 = test_env.storage.seal_and_sync_epoch(epoch2).await.unwrap(); test_env .meta_client - .commit_epoch(epoch2, sync_result2.uncommitted_ssts) + .commit_epoch(epoch2, sync_result2) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch2).await; @@ -1406,7 +1406,7 @@ async fn test_hummock_version_reader() { let sync_result3 = test_env.storage.seal_and_sync_epoch(epoch3).await.unwrap(); test_env .meta_client - .commit_epoch(epoch3, sync_result3.uncommitted_ssts) + .commit_epoch(epoch3, sync_result3) .await .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch3).await; @@ -1777,12 +1777,12 @@ async fn test_get_with_min_epoch() { let sync_result2 = test_env.storage.seal_and_sync_epoch(epoch2).await.unwrap(); test_env .meta_client - .commit_epoch(epoch1, sync_result1.uncommitted_ssts) + .commit_epoch(epoch1, sync_result1) .await .unwrap(); test_env .meta_client - .commit_epoch(epoch2, sync_result2.uncommitted_ssts) + .commit_epoch(epoch2, sync_result2) .await .unwrap(); diff --git a/src/storage/hummock_test/src/snapshot_tests.rs b/src/storage/hummock_test/src/snapshot_tests.rs index c019f12a0268b..981ed3687bd40 100644 --- a/src/storage/hummock_test/src/snapshot_tests.rs +++ b/src/storage/hummock_test/src/snapshot_tests.rs @@ -137,14 +137,10 @@ async fn test_snapshot_inner( let epoch2 = epoch1.next_epoch(); local.seal_current_epoch(epoch2, SealCurrentEpochOptions::for_test()); if enable_sync { - let ssts = hummock_storage - .seal_and_sync_epoch(epoch1) - .await - .unwrap() - .uncommitted_ssts; + let res = hummock_storage.seal_and_sync_epoch(epoch1).await.unwrap(); if enable_commit { mock_hummock_meta_client - .commit_epoch(epoch1, ssts) + .commit_epoch(epoch1, res) .await .unwrap(); hummock_storage @@ -181,14 +177,10 @@ async fn test_snapshot_inner( let epoch3 = epoch2.next_epoch(); local.seal_current_epoch(epoch3, SealCurrentEpochOptions::for_test()); if enable_sync { - let ssts = hummock_storage - .seal_and_sync_epoch(epoch2) - .await - .unwrap() - .uncommitted_ssts; + let res = hummock_storage.seal_and_sync_epoch(epoch2).await.unwrap(); if enable_commit { mock_hummock_meta_client - .commit_epoch(epoch2, ssts) + .commit_epoch(epoch2, res) .await .unwrap(); hummock_storage @@ -225,14 +217,10 @@ async fn test_snapshot_inner( .unwrap(); local.seal_current_epoch(u64::MAX, SealCurrentEpochOptions::for_test()); if enable_sync { - let ssts = hummock_storage - .seal_and_sync_epoch(epoch3) - .await - .unwrap() - .uncommitted_ssts; + let res = hummock_storage.seal_and_sync_epoch(epoch3).await.unwrap(); if enable_commit { mock_hummock_meta_client - .commit_epoch(epoch3, ssts) + .commit_epoch(epoch3, res) .await .unwrap(); hummock_storage @@ -287,14 +275,10 @@ async fn test_snapshot_range_scan_inner( .unwrap(); local.seal_current_epoch(u64::MAX, SealCurrentEpochOptions::for_test()); if enable_sync { - let ssts = hummock_storage - .seal_and_sync_epoch(epoch) - .await - .unwrap() - .uncommitted_ssts; + let res = hummock_storage.seal_and_sync_epoch(epoch).await.unwrap(); if enable_commit { mock_hummock_meta_client - .commit_epoch(epoch, ssts) + .commit_epoch(epoch, res) .await .unwrap(); hummock_storage diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index ff6385e35ab1e..40dd23d78d987 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -24,7 +24,9 @@ use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_hummock_sdk::key::prefixed_range_with_vnode; -use risingwave_hummock_sdk::{HummockReadEpoch, HummockSstableObjectId, LocalSstableInfo}; +use risingwave_hummock_sdk::{ + HummockReadEpoch, HummockSstableObjectId, LocalSstableInfo, SyncResult, +}; use risingwave_meta::hummock::test_utils::setup_compute_env; use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_rpc_client::HummockMetaClient; @@ -372,12 +374,8 @@ async fn test_basic_inner( .unwrap(); let len = count_stream(iter).await; assert_eq!(len, 4); - let ssts = hummock_storage - .seal_and_sync_epoch(epoch1) - .await - .unwrap() - .uncommitted_ssts; - meta_client.commit_epoch(epoch1, ssts).await.unwrap(); + let res = hummock_storage.seal_and_sync_epoch(epoch1).await.unwrap(); + meta_client.commit_epoch(epoch1, res).await.unwrap(); hummock_storage .try_wait_epoch(HummockReadEpoch::Committed(epoch1)) .await @@ -1062,12 +1060,8 @@ async fn test_delete_get_inner( ) .await .unwrap(); - let ssts = hummock_storage - .seal_and_sync_epoch(epoch1) - .await - .unwrap() - .uncommitted_ssts; - meta_client.commit_epoch(epoch1, ssts).await.unwrap(); + let res = hummock_storage.seal_and_sync_epoch(epoch1).await.unwrap(); + meta_client.commit_epoch(epoch1, res).await.unwrap(); let epoch2 = epoch1.next_epoch(); local.seal_current_epoch(epoch2, SealCurrentEpochOptions::for_test()); @@ -1086,12 +1080,8 @@ async fn test_delete_get_inner( .await .unwrap(); local.seal_current_epoch(u64::MAX, SealCurrentEpochOptions::for_test()); - let ssts = hummock_storage - .seal_and_sync_epoch(epoch2) - .await - .unwrap() - .uncommitted_ssts; - meta_client.commit_epoch(epoch2, ssts).await.unwrap(); + let res = hummock_storage.seal_and_sync_epoch(epoch2).await.unwrap(); + meta_client.commit_epoch(epoch2, res).await.unwrap(); hummock_storage .try_wait_epoch(HummockReadEpoch::Committed(epoch2)) .await @@ -1242,12 +1232,12 @@ async fn test_multiple_epoch_sync_inner( test_get().await; meta_client - .commit_epoch(epoch2, sync_result2.uncommitted_ssts) + .commit_epoch(epoch2, sync_result2) .await .unwrap(); meta_client - .commit_epoch(epoch3, sync_result3.uncommitted_ssts) + .commit_epoch(epoch3, sync_result3) .await .unwrap(); hummock_storage @@ -1340,7 +1330,7 @@ async fn test_gc_watermark_and_clear_shared_buffer() { min_object_id_epoch1, ); meta_client - .commit_epoch(epoch1, sync_result1.uncommitted_ssts) + .commit_epoch(epoch1, sync_result1) .await .unwrap(); hummock_storage diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index f5d1a10a18839..82b01bccaa12f 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -21,6 +21,8 @@ use risingwave_common_service::observer_manager::ObserverManager; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::TableKey; pub use risingwave_hummock_sdk::key::{gen_key_from_bytes, gen_key_from_str}; +#[cfg(test)] +use risingwave_hummock_sdk::SyncResult; use risingwave_meta::hummock::test_utils::{ register_table_ids_to_compaction_group, setup_compute_env, }; @@ -28,6 +30,7 @@ use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_meta::manager::MetaSrvEnv; use risingwave_pb::catalog::{PbTable, Table}; use risingwave_pb::common::WorkerNode; +use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::error::StorageResult; use risingwave_storage::filter_key_extractor::{ FilterKeyExtractorImpl, FilterKeyExtractorManager, FullKeyFilterKeyExtractor, @@ -252,10 +255,7 @@ impl HummockTestEnv { // On completion of this function call, the provided epoch should be committed and visible. pub async fn commit_epoch(&self, epoch: u64) { let res = self.storage.seal_and_sync_epoch(epoch).await.unwrap(); - self.meta_client - .commit_epoch_with_watermark(epoch, res.uncommitted_ssts, res.table_watermarks) - .await - .unwrap(); + self.meta_client.commit_epoch(epoch, res).await.unwrap(); self.storage.try_wait_epoch_for_test(epoch).await; } diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index fcfcac53a478e..0b96aa19cb837 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -28,7 +28,7 @@ use prometheus::core::{AtomicU64, GenericGauge}; use prometheus::{Histogram, IntGauge}; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::SstDeltaInfo; -use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; +use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo, SyncResult}; use thiserror_ext::AsReport; use tokio::spawn; use tokio::sync::mpsc::error::SendError; @@ -60,7 +60,6 @@ use crate::hummock::{ }; use crate::monitor::HummockStateStoreMetrics; use crate::opts::StorageOpts; -use crate::store::SyncResult; #[derive(Clone)] pub struct BufferTracker { diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index 04532cc9b9c10..9e98fac4ee3b1 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -18,14 +18,14 @@ use std::sync::Arc; use parking_lot::{RwLock, RwLockReadGuard}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::TableId; -use risingwave_hummock_sdk::HummockEpoch; +use risingwave_hummock_sdk::{HummockEpoch, SyncResult}; use thiserror_ext::AsReport; use tokio::sync::oneshot; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; use crate::hummock::HummockResult; use crate::mem_table::ImmutableMemtable; -use crate::store::{SealCurrentEpochOptions, SyncResult}; +use crate::store::SealCurrentEpochOptions; pub mod hummock_event_handler; pub mod refiller; diff --git a/src/storage/src/hummock/hummock_meta_client.rs b/src/storage/src/hummock/hummock_meta_client.rs index 50c60ce5f11be..5cf380285cf1e 100644 --- a/src/storage/src/hummock/hummock_meta_client.rs +++ b/src/storage/src/hummock/hummock_meta_client.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use async_trait::async_trait; use futures::stream::BoxStream; use risingwave_hummock_sdk::version::HummockVersion; -use risingwave_hummock_sdk::{HummockSstableObjectId, LocalSstableInfo, SstObjectIdRange}; +use risingwave_hummock_sdk::{HummockSstableObjectId, SstObjectIdRange, SyncResult}; use risingwave_pb::hummock::{HummockSnapshot, SubscribeCompactionEventRequest, VacuumTask}; use risingwave_rpc_client::error::Result; use risingwave_rpc_client::{CompactionEventItem, HummockMetaClient, MetaClient}; @@ -78,11 +78,7 @@ impl HummockMetaClient for MonitoredHummockMetaClient { res } - async fn commit_epoch( - &self, - _epoch: HummockEpoch, - _sstables: Vec, - ) -> Result<()> { + async fn commit_epoch(&self, _epoch: HummockEpoch, _sync_result: SyncResult) -> Result<()> { panic!("Only meta service can commit_epoch in production.") } diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 2bdbd70c17d56..8c18b647ca819 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -28,7 +28,7 @@ use risingwave_hummock_sdk::key::{ is_empty_key_range, vnode, vnode_range, TableKey, TableKeyRange, }; use risingwave_hummock_sdk::table_watermark::TableWatermarksIndex; -use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use risingwave_pb::hummock::SstableInfo; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; diff --git a/src/storage/src/memory.rs b/src/storage/src/memory.rs index db8eafcf4bc1d..5e645aa7ea57f 100644 --- a/src/storage/src/memory.rs +++ b/src/storage/src/memory.rs @@ -22,7 +22,7 @@ use bytes::Bytes; use parking_lot::RwLock; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange, UserKey}; -use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch}; +use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch, SyncResult}; use crate::error::StorageResult; use crate::mem_table::MemtableLocalStateStore; diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index 9486256181f9f..79f5bf3bdcf1b 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -20,7 +20,7 @@ use futures::{Future, TryFutureExt}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; -use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use thiserror_ext::AsReport; use tokio::time::Instant; use tracing::{error, Instrument}; diff --git a/src/storage/src/monitor/traced_store.rs b/src/storage/src/monitor/traced_store.rs index 51c15b944c2ba..47c0de67729b6 100644 --- a/src/storage/src/monitor/traced_store.rs +++ b/src/storage/src/monitor/traced_store.rs @@ -17,7 +17,7 @@ use bytes::Bytes; use futures::{Future, TryFutureExt}; use risingwave_common::buffer::Bitmap; use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; -use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use risingwave_hummock_trace::{ init_collector, should_use_trace, ConcurrentId, MayTraceSpan, OperationResult, StorageType, TraceResult, TraceSpan, TracedBytes, TracedSealCurrentEpochOptions, LOCAL_ID, diff --git a/src/storage/src/panic_store.rs b/src/storage/src/panic_store.rs index 546dd5a399c75..7c9645697caa1 100644 --- a/src/storage/src/panic_store.rs +++ b/src/storage/src/panic_store.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use bytes::Bytes; use risingwave_common::buffer::Bitmap; use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; -use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use crate::error::StorageResult; use crate::storage_value::StorageValue; diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index 8b15e581796a8..95705d154af6c 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::cmp::min; -use std::collections::HashMap; use std::default::Default; use std::fmt::{Debug, Formatter}; use std::future::Future; @@ -30,10 +29,8 @@ use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{Epoch, EpochPair}; use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange}; -use risingwave_hummock_sdk::table_watermark::{ - TableWatermarks, VnodeWatermark, WatermarkDirection, -}; -use risingwave_hummock_sdk::{HummockReadEpoch, LocalSstableInfo}; +use risingwave_hummock_sdk::table_watermark::{VnodeWatermark, WatermarkDirection}; +use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use risingwave_hummock_trace::{ TracedInitOptions, TracedNewLocalOptions, TracedOpConsistencyLevel, TracedPrefetchOptions, TracedReadOptions, TracedSealCurrentEpochOptions, TracedWriteOptions, @@ -311,16 +308,6 @@ pub trait StateStoreWrite: StaticSendSync { ) -> StorageResult; } -#[derive(Default, Debug)] -pub struct SyncResult { - /// The size of all synced shared buffers. - pub sync_size: usize, - /// The `sst_info` of sync. - pub uncommitted_ssts: Vec, - /// The collected table watermarks written by state tables. - pub table_watermarks: HashMap, -} - pub trait StateStore: StateStoreRead + StaticSendSync + Clone { type Local: LocalStateStore; diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 1d67e2fa9da2f..7fdb838654b70 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -212,7 +212,7 @@ pub mod verify { use bytes::Bytes; use risingwave_common::buffer::Bitmap; use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; - use risingwave_hummock_sdk::HummockReadEpoch; + use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use tracing::log::warn; use crate::error::StorageResult; @@ -733,7 +733,7 @@ pub mod boxed_state_store { use dyn_clone::{clone_trait_object, DynClone}; use risingwave_common::buffer::Bitmap; use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; - use risingwave_hummock_sdk::HummockReadEpoch; + use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use crate::error::StorageResult; use crate::hummock::HummockStorage; diff --git a/src/stream/src/common/log_store_impl/kv_log_store/mod.rs b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs index 5d7c1d021a6b1..3d28ccdcb7e6c 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/mod.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs @@ -411,10 +411,9 @@ mod tests { use risingwave_connector::sink::log_store::{ ChunkId, LogReader, LogStoreFactory, LogStoreReadItem, LogWriter, TruncateOffset, }; - use risingwave_hummock_sdk::HummockReadEpoch; + use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use risingwave_hummock_test::test_utils::prepare_hummock_test_env; use risingwave_storage::hummock::HummockStorage; - use risingwave_storage::store::SyncResult; use risingwave_storage::StateStore; use crate::common::log_store_impl::kv_log_store::reader::KvLogStoreReader; diff --git a/src/stream/src/common/table/test_state_table.rs b/src/stream/src/common/table/test_state_table.rs index b04c9f1dd6378..4ef3381871cbf 100644 --- a/src/stream/src/common/table/test_state_table.rs +++ b/src/stream/src/common/table/test_state_table.rs @@ -1356,11 +1356,7 @@ async fn test_state_table_may_exist() { // test may_exist with data only in uncommitted ssts (e1) check_may_exist(&state_table, vec![1, 4], vec![2, 3, 6, 12]).await; - test_env - .meta_client - .commit_epoch(e1, e1_res.uncommitted_ssts) - .await - .unwrap(); + test_env.meta_client.commit_epoch(e1, e1_res).await.unwrap(); test_env.storage.try_wait_epoch_for_test(e1).await; // test may_exist with data only in committed ssts (e1) @@ -1434,11 +1430,7 @@ async fn test_state_table_may_exist() { // (e2), committed ssts (e1) check_may_exist(&state_table, vec![1, 3, 4, 6], vec![12]).await; - test_env - .meta_client - .commit_epoch(e2, e2_res.uncommitted_ssts) - .await - .unwrap(); + test_env.meta_client.commit_epoch(e2, e2_res).await.unwrap(); test_env.storage.try_wait_epoch_for_test(e2).await; epoch.inc_for_test(); @@ -1451,18 +1443,10 @@ async fn test_state_table_may_exist() { // test may_exist with data in uncommitted ssts (e3, e4), committed ssts (e1, e2, e3, e4) check_may_exist(&state_table, vec![1, 3, 4, 6], vec![12]).await; - test_env - .meta_client - .commit_epoch(e3, e3_res.uncommitted_ssts) - .await - .unwrap(); + test_env.meta_client.commit_epoch(e3, e3_res).await.unwrap(); test_env.storage.try_wait_epoch_for_test(e3).await; - test_env - .meta_client - .commit_epoch(e4, e4_res.uncommitted_ssts) - .await - .unwrap(); + test_env.meta_client.commit_epoch(e4, e4_res).await.unwrap(); test_env.storage.try_wait_epoch_for_test(e4).await; // test may_exist with data in committed ssts (e1, e2, e3, e4) diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 9d006a426d6b7..6e7b6f70e2421 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -48,7 +48,7 @@ mod tests; pub use progress::CreateMviewProgress; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; -use risingwave_hummock_sdk::LocalSstableInfo; +use risingwave_hummock_sdk::{LocalSstableInfo, SyncResult}; use risingwave_pb::common::ActorInfo; use risingwave_pb::stream_plan; use risingwave_pb::stream_plan::barrier::BarrierKind; @@ -58,7 +58,6 @@ use risingwave_pb::stream_service::{ streaming_control_stream_response, BarrierCompleteResponse, StreamingControlStreamRequest, StreamingControlStreamResponse, }; -use risingwave_storage::store::SyncResult; use crate::executor::exchange::permit::Receiver; use crate::executor::monitor::StreamingMetrics; diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index ed192b39dfe5f..4bb39ddabff1e 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -25,9 +25,9 @@ use futures::stream::FuturesOrdered; use futures::{FutureExt, StreamExt}; use prometheus::HistogramTimer; use risingwave_common::must_match; +use risingwave_hummock_sdk::SyncResult; use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; -use risingwave_storage::store::SyncResult; use risingwave_storage::{dispatch_state_store, StateStore, StateStoreImpl}; use rw_futures_util::pending_on_none; use thiserror_ext::AsReport; diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 66abdf4286e37..341e89a58af89 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -377,7 +377,7 @@ async fn run_compare_result( // let checkpoint = epoch % 10 == 0; let ret = hummock.seal_and_sync_epoch(epoch).await.unwrap(); meta_client - .commit_epoch(epoch, ret.uncommitted_ssts) + .commit_epoch(epoch, ret) .await .map_err(|e| format!("{:?}", e))?; if (epoch / test_epoch(1)) % 200 == 0 { From 048017fcf8b0a63f237ee5d1268b85928e37d3d5 Mon Sep 17 00:00:00 2001 From: CAJan93 Date: Fri, 19 Apr 2024 10:42:06 +0200 Subject: [PATCH 16/22] feat(chore): dev-compactor (#16381) --- risedev.yml | 28 +++++++++++++++++++--------- 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/risedev.yml b/risedev.yml index aad64a0b135a2..721043ccf7fa2 100644 --- a/risedev.yml +++ b/risedev.yml @@ -92,6 +92,17 @@ profile: - use: compute-node - use: frontend + # You can use this in combination with the virtual compactor + # provided in https://github.com/risingwavelabs/risingwave-extensions + dev-compactor: + steps: + - use: minio + - use: meta-node + - use: compute-node + - use: frontend + - use: compactor + user-managed: true + full: steps: - use: minio @@ -962,14 +973,14 @@ profile: full-with-batch-query-limit: config-path: src/config/full-with-batch-query-limit.toml steps: - - use: minio - - use: etcd - - use: meta-node - - use: compute-node - - use: frontend - - use: compactor - - use: prometheus - - use: grafana + - use: minio + - use: etcd + - use: meta-node + - use: compute-node + - use: frontend + - use: compactor + - use: prometheus + - use: grafana compose: risingwave: "ghcr.io/risingwavelabs/risingwave:latest" @@ -1395,4 +1406,3 @@ template: # address of redis address: "127.0.0.1" - From 46287069c7509499a3831a0b962599f97846b509 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 19 Apr 2024 16:47:19 +0800 Subject: [PATCH 17/22] test: add test for root cause identification of recovery (#16397) Signed-off-by: Bugen Zhao --- ci/scripts/run-e2e-test.sh | 4 ++-- e2e_test/error_ui/simple/recovery.slt | 30 +++++++++++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 e2e_test/error_ui/simple/recovery.slt diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index da42fb5acac39..f6fcde450aaab 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -141,9 +141,9 @@ sqllogictest -p 4566 -d dev './e2e_test/generated/**/*.slt' --junit "generated-$ echo "--- Kill cluster" cluster_stop -echo "--- e2e, $mode, error ui" +echo "--- e2e, ci-3cn-1fe-with-recovery, error ui" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ -cluster_start +risedev ci-start ci-3cn-1fe-with-recovery sqllogictest -p 4566 -d dev './e2e_test/error_ui/simple/**/*.slt' sqllogictest -p 4566 -d dev -e postgres-extended './e2e_test/error_ui/extended/**/*.slt' diff --git a/e2e_test/error_ui/simple/recovery.slt b/e2e_test/error_ui/simple/recovery.slt new file mode 100644 index 0000000000000..aa159c4486492 --- /dev/null +++ b/e2e_test/error_ui/simple/recovery.slt @@ -0,0 +1,30 @@ +# TODO: the test triggers a recovery caused by a known issue: https://github.com/risingwavelabs/risingwave/issues/12474. +# We should consider using a mechanism designed for testing recovery instead of depending on a bug. + +statement ok +create table t (v int); + +statement ok +create materialized view mv as select generate_series(1, 10), coalesce(pg_sleep(2), v) / 0 bomb from t; + +# The bomb will be triggered after 2 seconds of sleep, so the insertion should return successfully. +statement ok +insert into t values (1); + +# Wait for recovery to complete. +sleep 10s + +# Check that there's a log entry for barrier collection failure on the mata node. +# The message should contain the root cause of the failure: "Division by zero". +query T +select info::varchar like +'%Actor % exited unexpectedly: Executor error: Chunk operation error: Division by zero%' +from rw_catalog.rw_event_logs +where event_type = 'COLLECT_BARRIER_FAIL' +order by timestamp desc +limit 1; +---- +t + +statement ok +drop table t cascade; From efa1fdadafa71f8750c562e911550f356fac7d1c Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 19 Apr 2024 17:01:03 +0800 Subject: [PATCH 18/22] feat(source): use fragment id only as Kafka consumer group id (#16111) Signed-off-by: xxchan --- .../source_inline/kafka/consumer_group.slt | 19 ++++++++----------- src/connector/src/source/base.rs | 1 - src/connector/src/source/kafka/mod.rs | 8 +++++--- .../src/source/kafka/source/reader.rs | 7 +------ src/connector/with_options_source.yaml | 2 +- 5 files changed, 15 insertions(+), 22 deletions(-) diff --git a/e2e_test/source_inline/kafka/consumer_group.slt b/e2e_test/source_inline/kafka/consumer_group.slt index ed97dec558f35..a43c6e58b17bf 100644 --- a/e2e_test/source_inline/kafka/consumer_group.slt +++ b/e2e_test/source_inline/kafka/consumer_group.slt @@ -41,7 +41,7 @@ b c -# There are 4 consumer groups, 1 for batch query (not listed below), 3 for MV. +# There are 2 consumer groups, 1 for batch query (not listed below), 1 for MV. # All of them are "Empty" state with 0 members, because we manually `assign` partitions to them. # At the begginning, the MV's consumer group will not occur. They will be created after committing offset to Kafka. # (enable.auto.commit defaults to true, and auto.commit.interval.ms defaults to 5s) @@ -50,29 +50,23 @@ sleep 5s system ok ./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-members ---- -0,0,0 +0 -# The lag for batch query's group is 0, and each MV parition's group is 2 (1 of 3 consumed). +# The lag for MV's group is 0. system ok ./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-lags ---- -2,2,2 +0 # We try to interfere by creating consumers that subscribing to the topic with the RW's group id. system ok -./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-groups | tr ',' '\\n' | xargs -P4 -I {} sh -c "timeout 40s rpk topic consume test_consumer_group -g {}" & +./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-groups | tr ',' '\\n' | xargs -P4 -I {} sh -c "rpk topic consume test_consumer_group -g {}" & # Wait a while for them to subscribe to the topic. sleep 15s -# The lag is changed to 0 -system ok -./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-lags ----- -0,0,0 - system ok cat <, diff --git a/src/connector/src/source/kafka/mod.rs b/src/connector/src/source/kafka/mod.rs index 55f84ea4f75a7..91d4ccce5ca88 100644 --- a/src/connector/src/source/kafka/mod.rs +++ b/src/connector/src/source/kafka/mod.rs @@ -82,9 +82,11 @@ pub struct RdKafkaPropertiesConsumer { #[serde_as(as = "Option")] pub fetch_max_bytes: Option, - /// Automatically and periodically commit offsets in the background. - /// Note: setting this to false does not prevent the consumer from fetching previously committed start offsets. - /// To circumvent this behaviour set specific start offsets per partition in the call to assign(). + /// Whether to automatically and periodically commit offsets in the background. + /// + /// Note that RisingWave does NOT rely on committed offsets. Committing offset is only for exposing the + /// progress for monitoring. Setting this to false can avoid creating consumer groups. + /// /// default: true #[serde(rename = "properties.enable.auto.commit")] #[serde_as(as = "Option")] diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index ea1d2d847f51c..73c24dd5f810d 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -66,8 +66,6 @@ impl SplitReader for KafkaSplitReader { // disable partition eof config.set("enable.partition.eof", "false"); - // change to `RdKafkaPropertiesConsumer::enable_auto_commit` to enable auto commit - // config.set("enable.auto.commit", "false"); config.set("auto.offset.reset", "smallest"); config.set("isolation.level", KAFKA_ISOLATION_LEVEL); config.set("bootstrap.servers", bootstrap_servers); @@ -77,10 +75,7 @@ impl SplitReader for KafkaSplitReader { config.set( "group.id", - format!( - "rw-consumer-{}-{}", - source_ctx.fragment_id, source_ctx.actor_id - ), + format!("rw-consumer-{}", source_ctx.fragment_id), ); let client_ctx = PrivateLinkConsumerContext::new( diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 83fdac056d96f..2ea0e5f3488ee 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -208,7 +208,7 @@ KafkaProperties: required: false - name: properties.enable.auto.commit field_type: bool - comments: 'Automatically and periodically commit offsets in the background. Note: setting this to false does not prevent the consumer from fetching previously committed start offsets. To circumvent this behaviour set specific start offsets per partition in the call to assign(). default: true' + comments: 'Whether to automatically and periodically commit offsets in the background. Note that RisingWave does NOT rely on committed offsets. Committing offset is only for exposing the progress for monitoring. Setting this to false can avoid creating consumer groups. default: true' required: false - name: broker.rewrite.endpoints field_type: HashMap From edc65a063aed8ffabdc71161982cb2732b34056d Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Fri, 19 Apr 2024 20:53:05 +0800 Subject: [PATCH 19/22] refactor(stream): add prelude for streaming executor modules (#16404) Signed-off-by: Richard Chien --- src/common/src/array/mod.rs | 1 + src/compute/tests/integration_tests.rs | 2 +- .../src/common/log_store_impl/in_mem.rs | 3 +- src/stream/src/common/mod.rs | 1 - src/stream/src/executor/agg_common.rs | 6 +-- .../executor/backfill/arrangement_backfill.rs | 20 +++------- .../src/executor/backfill/cdc/cdc_backfill.rs | 21 +++------- .../executor/backfill/no_shuffle_backfill.rs | 20 ++-------- src/stream/src/executor/barrier_recv.rs | 6 +-- src/stream/src/executor/batch_query.rs | 11 +---- src/stream/src/executor/chain.rs | 8 +--- .../src/executor/dedup/append_only_dedup.rs | 19 ++------- src/stream/src/executor/dedup/cache.rs | 2 +- src/stream/src/executor/dispatch.rs | 17 +++----- src/stream/src/executor/dml.rs | 8 +--- src/stream/src/executor/dynamic_filter.rs | 17 +++----- src/stream/src/executor/exchange/input.rs | 6 +-- src/stream/src/executor/expand.rs | 9 +---- src/stream/src/executor/filter.rs | 12 +----- src/stream/src/executor/hash_agg.rs | 18 +-------- src/stream/src/executor/hash_join.rs | 19 ++------- src/stream/src/executor/hop_window.rs | 6 +-- src/stream/src/executor/lookup.rs | 6 +-- src/stream/src/executor/lookup/cache.rs | 6 +-- src/stream/src/executor/lookup/impl_.rs | 12 ++---- src/stream/src/executor/lookup_union.rs | 4 +- src/stream/src/executor/merge.rs | 6 +-- src/stream/src/executor/mod.rs | 3 +- src/stream/src/executor/mview/materialize.rs | 22 ++++------ src/stream/src/executor/no_op.rs | 2 +- src/stream/src/executor/now.rs | 13 ++---- src/stream/src/executor/over_window/eowc.rs | 15 ++----- .../src/executor/over_window/general.rs | 17 ++------ src/stream/src/executor/prelude.rs | 40 +++++++++++++++++++ src/stream/src/executor/project.rs | 14 ++----- src/stream/src/executor/project_set.rs | 14 ++----- src/stream/src/executor/rearranged_chain.rs | 11 ++--- src/stream/src/executor/receiver.rs | 9 +---- src/stream/src/executor/row_id_gen.rs | 9 +---- src/stream/src/executor/simple_agg.rs | 17 +++----- src/stream/src/executor/sink.rs | 15 ++----- src/stream/src/executor/sort.rs | 11 +---- .../src/executor/source/fetch_executor.rs | 23 ++++------- .../src/executor/source/fs_source_executor.rs | 20 +++++----- .../src/executor/source/list_executor.rs | 11 ++--- src/stream/src/executor/source/mod.rs | 26 +++++++----- .../source/source_backfill_executor.rs | 17 ++++---- .../src/executor/source/source_executor.rs | 21 ++++++---- .../src/executor/stateless_simple_agg.rs | 8 +--- src/stream/src/executor/stream_reader.rs | 3 +- src/stream/src/executor/subscription.rs | 7 +--- src/stream/src/executor/subtask.rs | 6 +-- src/stream/src/executor/temporal_join.rs | 20 ++-------- src/stream/src/executor/top_n/group_top_n.rs | 11 +---- .../executor/top_n/group_top_n_appendonly.rs | 12 +----- .../src/executor/top_n/top_n_appendonly.rs | 9 +---- src/stream/src/executor/top_n/top_n_plain.rs | 9 +---- src/stream/src/executor/top_n/utils.rs | 14 ++----- src/stream/src/executor/troublemaker.rs | 6 +-- src/stream/src/executor/union.rs | 6 +-- src/stream/src/executor/utils.rs | 6 +-- src/stream/src/executor/values.rs | 8 +--- src/stream/src/executor/watermark_filter.rs | 12 +----- src/stream/src/executor/wrapper.rs | 6 +-- src/stream/src/from_proto/source/fs_fetch.rs | 5 +-- .../src/from_proto/source/trad_source.rs | 8 ++-- src/stream/src/from_proto/source_backfill.rs | 8 ++-- 67 files changed, 246 insertions(+), 514 deletions(-) create mode 100644 src/stream/src/executor/prelude.rs diff --git a/src/common/src/array/mod.rs b/src/common/src/array/mod.rs index 0fcb1222eef74..93f6255038e9e 100644 --- a/src/common/src/array/mod.rs +++ b/src/common/src/array/mod.rs @@ -63,6 +63,7 @@ pub use primitive_array::{PrimitiveArray, PrimitiveArrayBuilder, PrimitiveArrayI use risingwave_common_estimate_size::EstimateSize; use risingwave_pb::data::PbArray; pub use stream_chunk::{Op, StreamChunk, StreamChunkTestExt}; +pub use stream_chunk_builder::StreamChunkBuilder; pub use struct_array::{StructArray, StructArrayBuilder, StructRef, StructValue}; pub use utf8_array::*; diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 49f54c18a5ff6..f34a2940e9ce5 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -53,7 +53,7 @@ use risingwave_stream::error::StreamResult; use risingwave_stream::executor::dml::DmlExecutor; use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::executor::row_id_gen::RowIdGenExecutor; -use risingwave_stream::executor::source_executor::SourceExecutor; +use risingwave_stream::executor::source::SourceExecutor; use risingwave_stream::executor::{ ActorContext, Barrier, Execute, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices, }; diff --git a/src/stream/src/common/log_store_impl/in_mem.rs b/src/stream/src/common/log_store_impl/in_mem.rs index 4b1f003ce6562..ccb2eb878ede9 100644 --- a/src/stream/src/common/log_store_impl/in_mem.rs +++ b/src/stream/src/common/log_store_impl/in_mem.rs @@ -319,7 +319,7 @@ mod tests { use std::task::Poll; use futures::FutureExt; - use risingwave_common::array::Op; + use risingwave_common::array::{Op, StreamChunkBuilder}; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::epoch::{test_epoch, EpochPair}; use risingwave_connector::sink::log_store::{ @@ -327,7 +327,6 @@ mod tests { }; use crate::common::log_store_impl::in_mem::BoundedInMemLogStoreFactory; - use crate::common::StreamChunkBuilder; #[tokio::test] async fn test_in_memory_log_store() { diff --git a/src/stream/src/common/mod.rs b/src/stream/src/common/mod.rs index e7d6fda232ac0..23dc006bf8d63 100644 --- a/src/stream/src/common/mod.rs +++ b/src/stream/src/common/mod.rs @@ -13,7 +13,6 @@ // limitations under the License. pub use column_mapping::*; -pub use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; pub mod cache; mod column_mapping; diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs index 91c414877a76b..2cb3cad8fb2d8 100644 --- a/src/stream/src/executor/agg_common.rs +++ b/src/stream/src/executor/agg_common.rs @@ -16,13 +16,9 @@ use std::collections::HashMap; use risingwave_expr::aggregate::AggCall; use risingwave_pb::stream_plan::PbAggNodeVersion; -use risingwave_storage::StateStore; use super::aggregation::AggStateStorage; -use super::{Executor, ExecutorInfo}; -use crate::common::table::state_table::StateTable; -use crate::executor::ActorContextRef; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// Arguments needed to construct an `XxxAggExecutor`. pub struct AggExecutorArgs { diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index b4adf038e5b78..f43d7767b45ce 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -12,24 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::pin; -use std::sync::Arc; +use std::collections::HashMap; use either::Either; use futures::stream::{select_all, select_with_strategy}; -use futures::{stream, StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::{stream, TryStreamExt}; use itertools::Itertools; -use risingwave_common::array::{DataChunk, Op, StreamChunk}; +use risingwave_common::array::{DataChunk, Op}; use risingwave_common::bail; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; -use risingwave_common::row::OwnedRow; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_storage::row_serde::value_serde::ValueRowSerde; use risingwave_storage::store::PrefetchOptions; -use risingwave_storage::StateStore; -use crate::common::table::state_table::{ReplicatedStateTable, StateTable}; +use crate::common::table::state_table::ReplicatedStateTable; #[cfg(debug_assertions)] use crate::executor::backfill::utils::METADATA_STATE_LEN; use crate::executor::backfill::utils::{ @@ -37,12 +33,8 @@ use crate::executor::backfill::utils::{ mapping_message, mark_chunk_ref_by_vnode, owned_row_iter, persist_state_per_vnode, update_pos_by_vnode, BackfillProgressPerVnode, BackfillRateLimiter, BackfillState, }; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ - expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, HashMap, Message, - StreamExecutorError, StreamExecutorResult, -}; -use crate::task::{ActorId, CreateMviewProgress}; +use crate::executor::prelude::*; +use crate::task::CreateMviewProgress; type Builders = HashMap; diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 96a53ace0b25b..dc8e6bc67ddaa 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -12,19 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::{pin, Pin}; -use std::sync::Arc; +use std::pin::Pin; use either::Either; +use futures::stream; use futures::stream::select_with_strategy; -use futures::{pin_mut, stream, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::array::{DataChunk, StreamChunk}; +use risingwave_common::array::DataChunk; use risingwave_common::bail; -use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; -use risingwave_common::row::{OwnedRow, Row, RowExt}; -use risingwave_common::types::{DataType, ScalarRefImpl}; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; +use risingwave_common::row::RowExt; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::parser::{ DebeziumParser, DebeziumProps, EncodingProperties, JsonProperties, ProtocolProperties, @@ -32,10 +29,8 @@ use risingwave_connector::parser::{ }; use risingwave_connector::source::cdc::external::CdcOffset; use risingwave_connector::source::{SourceColumnDesc, SourceContext}; -use risingwave_storage::StateStore; use rw_futures_util::pausable; -use crate::common::table::state_table::StateTable; use crate::executor::backfill::cdc::state::CdcBackfillState; use crate::executor::backfill::cdc::upstream_table::external::ExternalStorageTable; use crate::executor::backfill::cdc::upstream_table::snapshot::{ @@ -44,11 +39,7 @@ use crate::executor::backfill::cdc::upstream_table::snapshot::{ use crate::executor::backfill::utils::{ get_cdc_chunk_last_offset, get_new_pos, mapping_chunk, mapping_message, mark_cdc_chunk, }; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, - StreamExecutorError, StreamExecutorResult, -}; +use crate::executor::prelude::*; use crate::task::CreateMviewProgress; /// `split_id`, `is_finished`, `row_count`, `cdc_offset` all occupy 1 column each. diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index ca14f904e2dd0..fdf955fbbdfc7 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -12,37 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::pin; -use std::sync::Arc; - use either::Either; +use futures::stream; use futures::stream::select_with_strategy; -use futures::{stream, StreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::array::{DataChunk, Op, StreamChunk}; +use risingwave_common::array::{DataChunk, Op}; use risingwave_common::hash::VnodeBitmapExt; -use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; use risingwave_common::{bail, row}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::StateStore; -use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils; use crate::executor::backfill::utils::{ compute_bounds, construct_initial_finished_state, create_builder, create_limiter, get_new_pos, mapping_chunk, mapping_message, mark_chunk, owned_row_iter, BackfillRateLimiter, METADATA_STATE_LEN, }; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ - expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, Message, Mutation, - StreamExecutorError, StreamExecutorResult, -}; -use crate::task::{ActorId, CreateMviewProgress}; +use crate::executor::prelude::*; +use crate::task::CreateMviewProgress; /// Schema: | vnode | pk ... | `backfill_finished` | `row_count` | /// We can decode that into `BackfillState` on recovery. diff --git a/src/stream/src/executor/barrier_recv.rs b/src/stream/src/executor/barrier_recv.rs index 79f6157ab6bcd..a7c02ad6e24f5 100644 --- a/src/stream/src/executor/barrier_recv.rs +++ b/src/stream/src/executor/barrier_recv.rs @@ -12,14 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; use tokio::sync::mpsc::UnboundedReceiver; use tokio_stream::wrappers::UnboundedReceiverStream; -use super::{ - ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Execute, Message, - StreamExecutorError, -}; +use crate::executor::prelude::*; /// The executor only for receiving barrier from the meta service. It always resides in the leaves /// of the streaming graph. diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index 7c92bcd732423..d7c7f38d99504 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -12,20 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use await_tree::InstrumentAwait; -use futures::{pin_mut, StreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; +use risingwave_common::array::Op; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::collect_data_chunk; -use risingwave_storage::StateStore; -use super::error::StreamExecutorError; -use super::{Execute, Message}; -use crate::executor::BoxedMessageStream; +use crate::executor::prelude::*; pub struct BatchQueryExecutor { /// The [`StorageTable`] that needs to be queried diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs index c8b7b25852e74..ebcbe1e4e49bb 100644 --- a/src/stream/src/executor/chain.rs +++ b/src/stream/src/executor/chain.rs @@ -12,12 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; -use futures_async_stream::try_stream; - -use super::error::StreamExecutorError; -use super::{expect_first_barrier, Execute, Executor, Message}; -use crate::task::{ActorId, CreateMviewProgress}; +use crate::executor::prelude::*; +use crate::task::CreateMviewProgress; /// [`ChainExecutor`] is an executor that enables synchronization between the existing stream and /// newly appended executors. Currently, [`ChainExecutor`] is mainly used to implement MV on MV diff --git a/src/stream/src/executor/dedup/append_only_dedup.rs b/src/stream/src/executor/dedup/append_only_dedup.rs index 7b171a1ac844d..f73e196815400 100644 --- a/src/stream/src/executor/dedup/append_only_dedup.rs +++ b/src/stream/src/executor/dedup/append_only_dedup.rs @@ -12,26 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - -use futures::{stream, StreamExt}; -use futures_async_stream::try_stream; +use futures::stream; use itertools::Itertools; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::buffer::BitmapBuilder; -use risingwave_common::row::{OwnedRow, Row, RowExt}; -use risingwave_storage::StateStore; +use risingwave_common::row::RowExt; use super::cache::DedupCache; use crate::common::metrics::MetricsInfo; -use crate::common::table::state_table::StateTable; -use crate::executor::error::StreamExecutorError; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, - StreamExecutorResult, -}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// [`AppendOnlyDedupExecutor`] drops any message that has duplicate pk columns with previous /// messages. It only accepts append-only input, and its output will be append-only as well. diff --git a/src/stream/src/executor/dedup/cache.rs b/src/stream/src/executor/dedup/cache.rs index 5a9d876c356e3..245e7b7dbae95 100644 --- a/src/stream/src/executor/dedup/cache.rs +++ b/src/stream/src/executor/dedup/cache.rs @@ -18,7 +18,7 @@ use risingwave_common_estimate_size::EstimateSize; use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// [`DedupCache`] is used for key deduplication. Currently, the cache behaves like a set that only /// accepts a key without a value. This could be refined in the future to support k-v pairs. diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index 28c5edd57ead8..6c644466f9683 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -13,17 +13,13 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; -use std::fmt::{Debug, Formatter}; use std::future::Future; use std::iter::repeat_with; use std::ops::{Deref, DerefMut}; -use std::sync::Arc; -use await_tree::InstrumentAwait; -use futures::{Stream, StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::TryStreamExt; use itertools::Itertools; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::hash::{ActorMapping, ExpandedActorMapping, VirtualNode}; use risingwave_common::metrics::LabelGuardedIntCounter; @@ -35,11 +31,10 @@ use tokio::time::Instant; use tracing::{event, Instrument}; use super::exchange::output::{new_output, BoxedOutput}; -use super::{AddMutation, Executor, TroublemakerExecutor, UpdateMutation, Watermark}; -use crate::error::StreamResult; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{Barrier, Message, Mutation, StreamConsumer}; -use crate::task::{ActorId, DispatcherId, SharedContext}; +use super::{AddMutation, TroublemakerExecutor, UpdateMutation}; +use crate::executor::prelude::*; +use crate::executor::StreamConsumer; +use crate::task::{DispatcherId, SharedContext}; /// [`DispatchExecutor`] consumes messages and send them into downstream actors. Usually, /// data chunks will be dispatched with some specified policy, while control message diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index 69230ae60735b..b8839d76000c1 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -16,17 +16,13 @@ use std::collections::BTreeMap; use std::mem; use either::Either; -use futures::{StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::array::StreamChunk; +use futures::TryStreamExt; use risingwave_common::catalog::{ColumnDesc, TableId, TableVersionId}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; use risingwave_dml::dml_manager::DmlManagerRef; -use super::error::StreamExecutorError; -use super::{expect_first_barrier, BoxedMessageStream, Execute, Executor, Message, Mutation}; -use crate::common::StreamChunkBuilder; +use crate::executor::prelude::*; use crate::executor::stream_reader::StreamReaderWithPause; /// [`DmlExecutor`] accepts both stream data and batch data for data manipulation on a specific diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index fe94c6449ac49..ddbe8352b2e8a 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -15,14 +15,12 @@ use std::ops::Bound::{self, *}; use std::sync::Arc; -use futures::{pin_mut, stream, StreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::array::{Array, ArrayImpl, Op, StreamChunk}; +use futures::stream; +use risingwave_common::array::{Array, ArrayImpl, Op}; use risingwave_common::bail; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; -use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; -use risingwave_common::row::{self, once, OwnedRow, OwnedRow as RowData, Row}; +use risingwave_common::row::{self, once, OwnedRow as RowData}; use risingwave_common::types::{DataType, Datum, DefaultOrd, ScalarImpl, ToDatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_expr::expr::{ @@ -33,16 +31,11 @@ use risingwave_pb::expr::expr_node::Type::{ GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, }; use risingwave_storage::store::PrefetchOptions; -use risingwave_storage::StateStore; use super::barrier_align::*; -use super::error::StreamExecutorError; -use super::monitor::StreamingMetrics; -use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, Message}; -use crate::common::table::state_table::{StateTable, WatermarkCacheParameterizedStateTable}; -use crate::common::StreamChunkBuilder; +use crate::common::table::state_table::WatermarkCacheParameterizedStateTable; use crate::consistency::consistency_panic; -use crate::executor::expect_first_barrier_from_aligned_stream; +use crate::executor::prelude::*; use crate::task::ActorEvalErrorReport; pub struct DynamicFilterExecutor { diff --git a/src/stream/src/executor/exchange/input.rs b/src/stream/src/executor/exchange/input.rs index 7b7cc151f46cd..98ac3d278f0a3 100644 --- a/src/stream/src/executor/exchange/input.rs +++ b/src/stream/src/executor/exchange/input.rs @@ -29,13 +29,13 @@ use super::permit::Receiver; use crate::error::StreamResult; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; -use crate::executor::*; +use crate::executor::prelude::*; use crate::task::{ FragmentId, LocalBarrierManager, SharedContext, UpDownActorIds, UpDownFragmentIds, }; -/// `Input` provides an interface for [`MergeExecutor`] and [`ReceiverExecutor`] to receive data -/// from upstream actors. +/// `Input` provides an interface for [`MergeExecutor`](crate::executor::MergeExecutor) and +/// [`ReceiverExecutor`](crate::executor::ReceiverExecutor) to receive data from upstream actors. pub trait Input: MessageStream { /// The upstream actor id. fn actor_id(&self) -> ActorId; diff --git a/src/stream/src/executor/expand.rs b/src/stream/src/executor/expand.rs index 2375ebca3d0df..46e1d5765ee0e 100644 --- a/src/stream/src/executor/expand.rs +++ b/src/stream/src/executor/expand.rs @@ -12,14 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::Debug; +use risingwave_common::array::{Array, I64Array}; -use futures::StreamExt; -use futures_async_stream::try_stream; -use risingwave_common::array::{Array, I64Array, StreamChunk}; - -use super::error::StreamExecutorError; -use super::{BoxedMessageStream, Execute, Executor, Message}; +use crate::executor::prelude::*; pub struct ExpandExecutor { input: Executor, diff --git a/src/stream/src/executor/filter.rs b/src/stream/src/executor/filter.rs index ffc8847c42abf..4d1ecb098bd8f 100644 --- a/src/stream/src/executor/filter.rs +++ b/src/stream/src/executor/filter.rs @@ -12,20 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::{Debug, Formatter}; -use std::sync::Arc; - -use futures::StreamExt; -use futures_async_stream::try_stream; -use risingwave_common::array::{Array, ArrayImpl, Op, StreamChunk}; +use risingwave_common::array::{Array, ArrayImpl, Op}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; -use super::{ - ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError, - StreamExecutorResult, -}; +use crate::executor::prelude::*; /// `FilterExecutor` filters data with the `expr`. The `expr` takes a chunk of data, /// and returns a boolean array on whether each item should be retained. And then, diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 685b2e65e0831..02ad790e53983 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -14,23 +14,17 @@ use std::collections::HashMap; use std::marker::PhantomData; -use std::sync::Arc; -use futures::{stream, StreamExt}; -use futures_async_stream::try_stream; +use futures::stream; use itertools::Itertools; -use risingwave_common::array::StreamChunk; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; -use risingwave_common::catalog::Schema; use risingwave_common::hash::{HashKey, PrecomputedBuildHasher}; -use risingwave_common::types::ScalarImpl; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common_estimate_size::collections::EstimatedHashMap; use risingwave_common_estimate_size::EstimateSize; use risingwave_expr::aggregate::{build_retractable, AggCall, BoxedAggregateFunction}; use risingwave_pb::stream_plan::PbAggNodeVersion; -use risingwave_storage::StateStore; use super::agg_common::{AggExecutorArgs, HashAggExecutorExtraArgs}; use super::aggregation::{ @@ -38,18 +32,10 @@ use super::aggregation::{ OnlyOutputIfHasInput, }; use super::sort_buffer::SortBuffer; -use super::{ - expect_first_barrier, ActorContextRef, Executor, ExecutorInfo, StreamExecutorResult, Watermark, -}; use crate::cache::{cache_may_stale, new_with_hasher, ManagedLruCache}; use crate::common::metrics::MetricsInfo; -use crate::common::table::state_table::StateTable; -use crate::common::StreamChunkBuilder; -use crate::error::StreamResult; use crate::executor::aggregation::AggGroup as GenericAggGroup; -use crate::executor::error::StreamExecutorError; -use crate::executor::{BoxedMessageStream, Execute, Message}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; type AggGroup = GenericAggGroup; type BoxedAggGroup = Box>; diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index b3f6b8102b4f4..ea0253c9cbbc1 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -13,40 +13,27 @@ // limitations under the License. use std::collections::{BTreeMap, HashSet}; -use std::sync::Arc; use std::time::Duration; -use await_tree::InstrumentAwait; -use futures::{pin_mut, Stream, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; use multimap::MultiMap; -use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::array::{Op, RowRef}; use risingwave_common::hash::{HashKey, NullBitmap}; -use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{DataType, DefaultOrd, ToOwnedDatum}; +use risingwave_common::types::{DefaultOrd, ToOwnedDatum}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; -use risingwave_storage::StateStore; use tokio::time::Instant; use self::builder::JoinChunkBuilder; use super::barrier_align::*; -use super::error::{StreamExecutorError, StreamExecutorResult}; use super::join::hash_join::*; use super::join::row::JoinRow; use super::join::{JoinTypePrimitive, SideTypePrimitive, *}; -use super::monitor::StreamingMetrics; use super::watermark::*; -use super::{ - ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message, Watermark, -}; -use crate::common::table::state_table::StateTable; -use crate::executor::expect_first_barrier_from_aligned_stream; use crate::executor::join::builder::JoinStreamChunkBuilder; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// Evict the cache every n rows. const EVICT_EVERY_N_ROWS: u32 = 16; diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index 801f3daa9f53a..28408ad046a4e 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -14,17 +14,13 @@ use std::num::NonZeroUsize; -use futures::StreamExt; -use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{DataChunk, Op}; use risingwave_common::types::Interval; use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; -use super::error::StreamExecutorError; -use super::{ActorContextRef, Execute, Executor, Message}; -use crate::common::StreamChunkBuilder; +use crate::executor::prelude::*; pub struct HopWindowExecutor { _ctx: ActorContextRef, diff --git a/src/stream/src/executor/lookup.rs b/src/stream/src/executor/lookup.rs index 2c1de3170a801..8d62d27e37f06 100644 --- a/src/stream/src/executor/lookup.rs +++ b/src/stream/src/executor/lookup.rs @@ -14,10 +14,6 @@ use async_trait::async_trait; use futures::StreamExt; -use risingwave_common::types::DataType; -use risingwave_storage::StateStore; - -use crate::executor::{Barrier, BoxedMessageStream, Execute}; mod cache; mod sides; @@ -27,7 +23,7 @@ mod impl_; pub use impl_::LookupExecutorParams; -use super::{ActorContextRef, Executor}; +use crate::executor::prelude::*; #[cfg(test)] mod tests; diff --git a/src/stream/src/executor/lookup/cache.rs b/src/stream/src/executor/lookup/cache.rs index 610551352f371..287f11c2af608 100644 --- a/src/stream/src/executor/lookup/cache.rs +++ b/src/stream/src/executor/lookup/cache.rs @@ -12,14 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::array::Op; +use risingwave_common::row::RowExt; use risingwave_common_estimate_size::collections::{EstimatedHashSet, EstimatedVec}; use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::consistency::consistency_panic; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; pub type LookupEntryState = EstimatedHashSet; diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index 20873a8c7338f..8ce6688540e3f 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -12,12 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::{pin_mut, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::array::RowRef; -use risingwave_common::catalog::{ColumnDesc, Schema}; -use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::catalog::ColumnDesc; +use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::sort_util::ColumnOrder; @@ -26,18 +23,15 @@ use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableIter; -use risingwave_storage::StateStore; use super::sides::{stream_lookup_arrange_prev_epoch, stream_lookup_arrange_this_epoch}; use crate::cache::cache_may_stale; use crate::common::metrics::MetricsInfo; -use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; use crate::executor::join::builder::JoinStreamChunkBuilder; use crate::executor::lookup::cache::LookupCache; use crate::executor::lookup::sides::{ArrangeJoinSide, ArrangeMessage, StreamJoinSide}; use crate::executor::lookup::LookupExecutor; -use crate::executor::{ActorContextRef, Barrier, Executor, ExecutorInfo, Message}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// Parameters for [`LookupExecutor`]. pub struct LookupExecutorParams { diff --git a/src/stream/src/executor/lookup_union.rs b/src/stream/src/executor/lookup_union.rs index 1a8cff3ba3aba..2189a821fab11 100644 --- a/src/stream/src/executor/lookup_union.rs +++ b/src/stream/src/executor/lookup_union.rs @@ -16,11 +16,9 @@ use async_trait::async_trait; use futures::channel::mpsc; use futures::future::{join_all, select, Either}; use futures::{FutureExt, SinkExt, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; -use super::error::StreamExecutorError; -use super::{Barrier, BoxedMessageStream, Execute, Executor, Message}; +use crate::executor::prelude::*; /// Merges data from multiple inputs with order. If `order = [2, 1, 0]`, then /// it will first pipe data from the third input; after the third input gets a barrier, it will then diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs index 44e5f9c93f910..0f27386ffc836 100644 --- a/src/stream/src/executor/merge.rs +++ b/src/stream/src/executor/merge.rs @@ -18,16 +18,14 @@ use std::task::{Context, Poll}; use anyhow::Context as _; use futures::stream::{FusedStream, FuturesUnordered, StreamFuture}; -use futures::{pin_mut, Stream, StreamExt}; -use futures_async_stream::try_stream; +use futures::StreamExt; use tokio::time::Instant; -use super::error::StreamExecutorError; use super::exchange::input::BoxedInput; use super::watermark::*; use super::*; use crate::executor::exchange::input::new_input; -use crate::executor::monitor::StreamingMetrics; +use crate::executor::prelude::*; use crate::executor::utils::ActorInputMetrics; use crate::task::{FragmentId, SharedContext}; diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index ab53b02ef7ce9..0748fac0a6569 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod prelude; + use std::collections::{HashMap, HashSet}; use std::fmt::Debug; use std::sync::Arc; @@ -137,7 +139,6 @@ use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; pub use simple_agg::SimpleAggExecutor; pub use sink::SinkExecutor; pub use sort::*; -pub use source::*; pub use stateless_simple_agg::StatelessSimpleAggExecutor; pub use subscription::SubscriptionExecutor; pub use temporal_join::*; diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 3215fdd0fc9c9..29a3e131fd2f9 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -17,17 +17,15 @@ use std::collections::hash_map::Entry; use std::collections::{HashMap, HashSet}; use std::marker::PhantomData; use std::ops::{Deref, Index}; -use std::sync::Arc; use bytes::Bytes; -use futures::{stream, StreamExt}; -use futures_async_stream::try_stream; +use futures::stream; use itertools::Itertools; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Schema, TableId}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, TableId}; use risingwave_common::row::{CompactedRow, OwnedRow, RowDeserializer}; -use risingwave_common::types::{DataType, DefaultOrd, ScalarImpl}; +use risingwave_common::types::DefaultOrd; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; use risingwave_common::util::sort_util::ColumnOrder; @@ -35,18 +33,12 @@ use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerializer}; use risingwave_pb::catalog::Table; use risingwave_storage::mem_table::KeyOp; use risingwave_storage::row_serde::value_serde::{ValueRowSerde, ValueRowSerdeNew}; -use risingwave_storage::StateStore; use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTableInner; -use crate::executor::error::StreamExecutorError; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ - expect_first_barrier, ActorContext, ActorContextRef, AddMutation, BoxedMessageStream, Execute, - Executor, Message, Mutation, StreamExecutorResult, UpdateMutation, -}; -use crate::task::{ActorId, AtomicU64Ref}; +use crate::executor::prelude::*; +use crate::executor::{AddMutation, UpdateMutation}; /// `MaterializeExecutor` materializes changes in stream into a materialized view on storage. pub struct MaterializeExecutor { @@ -790,9 +782,9 @@ mod tests { use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; + use super::*; use crate::executor::test_utils::prelude::StateTable; use crate::executor::test_utils::*; - use crate::executor::*; #[tokio::test] async fn test_materialize_executor() { diff --git a/src/stream/src/executor/no_op.rs b/src/stream/src/executor/no_op.rs index ac12bf99d5d7f..d3957e0b02765 100644 --- a/src/stream/src/executor/no_op.rs +++ b/src/stream/src/executor/no_op.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use super::{ActorContextRef, BoxedMessageStream, Execute, Executor}; +use crate::executor::prelude::*; /// No-op executor directly forwards the input stream. Currently used to break the multiple edges in /// the fragment graph. diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs index 755e48f325965..43316fef71094 100644 --- a/src/stream/src/executor/now.rs +++ b/src/stream/src/executor/now.rs @@ -15,19 +15,12 @@ use std::ops::Bound; use std::ops::Bound::Unbounded; -use futures::{pin_mut, StreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::row::{self, OwnedRow}; -use risingwave_common::types::{DataType, Datum}; -use risingwave_storage::StateStore; +use risingwave_common::array::Op; +use risingwave_common::row; use tokio::sync::mpsc::UnboundedReceiver; use tokio_stream::wrappers::UnboundedReceiverStream; -use super::{ - Barrier, BoxedMessageStream, Execute, Message, Mutation, StreamExecutorError, Watermark, -}; -use crate::common::table::state_table::StateTable; +use crate::executor::prelude::*; pub struct NowExecutor { data_types: Vec, diff --git a/src/stream/src/executor/over_window/eowc.rs b/src/stream/src/executor/over_window/eowc.rs index 16fe77cb64ebc..1691f0f0c1b60 100644 --- a/src/stream/src/executor/over_window/eowc.rs +++ b/src/stream/src/executor/over_window/eowc.rs @@ -15,13 +15,10 @@ use std::marker::PhantomData; use std::ops::Bound; -use futures::StreamExt; -use futures_async_stream::{for_await, try_stream}; use itertools::Itertools; use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{ArrayRef, Op, StreamChunk}; -use risingwave_common::catalog::Schema; -use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::array::{ArrayRef, Op}; +use risingwave_common::row::RowExt; use risingwave_common::types::{ToDatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; use risingwave_common::util::memcmp_encoding::{self, MemcmpEncoded}; @@ -33,16 +30,10 @@ use risingwave_expr::window_function::{ create_window_state, StateEvictHint, StateKey, WindowFuncCall, WindowStates, }; use risingwave_storage::store::PrefetchOptions; -use risingwave_storage::StateStore; use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; -use crate::common::table::state_table::StateTable; -use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, - StreamExecutorError, StreamExecutorResult, -}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; struct Partition { states: WindowStates, diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index a7245c57f368c..7cb29c39f7e37 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -15,18 +15,14 @@ use std::collections::{btree_map, BTreeMap, HashSet}; use std::marker::PhantomData; use std::ops::RangeInclusive; -use std::sync::Arc; use delta_btree_map::{Change, PositionType}; -use futures::StreamExt; -use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{Op, RowRef, StreamChunk}; -use risingwave_common::catalog::Schema; -use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::array::Op; +use risingwave_common::row::RowExt; use risingwave_common::session_config::OverWindowCachePolicy as CachePolicy; -use risingwave_common::types::{DataType, DefaultOrdered}; +use risingwave_common::types::DefaultOrdered; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::memcmp_encoding::{self, MemcmpEncoded}; use risingwave_common::util::sort_util::OrderType; @@ -43,14 +39,9 @@ use super::over_partition::{ use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; -use crate::common::StreamChunkBuilder; use crate::executor::monitor::StreamingMetrics; use crate::executor::over_window::over_partition::AffectedRange; -use crate::executor::{ - expect_first_barrier, ActorContextRef, Execute, Executor, Message, StreamExecutorError, - StreamExecutorResult, -}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// [`OverWindowExecutor`] consumes retractable input stream and produces window function outputs. /// One [`OverWindowExecutor`] can handle one combination of partition key and order key. diff --git a/src/stream/src/executor/prelude.rs b/src/stream/src/executor/prelude.rs new file mode 100644 index 0000000000000..446efa2705810 --- /dev/null +++ b/src/stream/src/executor/prelude.rs @@ -0,0 +1,40 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![allow(unused_imports)] + +pub use std::fmt::{Debug, Formatter}; +pub use std::pin::pin; +pub use std::sync::Arc; + +pub use await_tree::InstrumentAwait; +pub use futures::{pin_mut, Stream, StreamExt}; +pub use futures_async_stream::{for_await, try_stream}; +pub use risingwave_common::array::{RowRef, StreamChunk, StreamChunkBuilder}; +pub use risingwave_common::catalog::Schema; +pub use risingwave_common::row::{OwnedRow, Row}; +pub use risingwave_common::types::{DataType, Datum, DatumRef, ScalarImpl, ScalarRefImpl}; +pub use risingwave_storage::StateStore; + +pub use crate::common::table::state_table::StateTable; +pub use crate::error::StreamResult; +pub use crate::executor::actor::{ActorContext, ActorContextRef}; +pub use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; +pub use crate::executor::monitor::streaming_stats::StreamingMetrics; +pub use crate::executor::{ + expect_first_barrier, expect_first_barrier_from_aligned_stream, Barrier, BoxedMessageStream, + Execute, Executor, ExecutorInfo, Message, MessageStream, MessageStreamItem, Mutation, + PkDataTypes, PkIndices, PkIndicesRef, Watermark, +}; +pub use crate::task::{ActorId, AtomicU64Ref}; diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index 12b8dff2d7742..e78238585c9fa 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -12,21 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::{Debug, Formatter}; - -use futures::StreamExt; -use futures_async_stream::try_stream; use multimap::MultiMap; -use risingwave_common::array::StreamChunk; -use risingwave_common::row::{Row, RowExt}; -use risingwave_common::types::{ScalarImpl, ToOwnedDatum}; +use risingwave_common::row::RowExt; +use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; -use super::{ - ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError, - StreamExecutorResult, Watermark, -}; +use crate::executor::prelude::*; /// `ProjectExecutor` project data with the `expr`. The `expr` takes a chunk of data, /// and returns a new data chunk. And then, `ProjectExecutor` will insert, delete diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs index dc2a3ea3e15e7..43a2d65cbfb11 100644 --- a/src/stream/src/executor/project_set.rs +++ b/src/stream/src/executor/project_set.rs @@ -12,23 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::{Debug, Formatter}; - use either::Either; -use futures::StreamExt; -use futures_async_stream::try_stream; use multimap::MultiMap; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::bail; -use risingwave_common::row::{Row, RowExt}; -use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum}; +use risingwave_common::row::RowExt; +use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_expr::table_function::ProjectSetSelectItem; -use super::error::StreamExecutorError; -use super::{ActorContextRef, Execute, Executor, Message, StreamExecutorResult, Watermark}; -use crate::common::StreamChunkBuilder; +use crate::executor::prelude::*; const PROJ_ROW_ID_OFFSET: usize = 1; diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs index 0f96c07a1e95c..19ebfeabc2988 100644 --- a/src/stream/src/executor/rearranged_chain.rs +++ b/src/stream/src/executor/rearranged_chain.rs @@ -12,17 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::pin; - use futures::channel::{mpsc, oneshot}; +use futures::stream; use futures::stream::select_with_strategy; -use futures::{stream, StreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::array::StreamChunk; -use super::error::StreamExecutorError; -use super::{expect_first_barrier, Barrier, Execute, Executor, Message, MessageStream}; -use crate::task::{ActorId, CreateMviewProgress}; +use crate::executor::prelude::*; +use crate::task::CreateMviewProgress; /// `ChainExecutor` is an executor that enables synchronization between the existing stream and /// newly appended executors. Currently, `ChainExecutor` is mainly used to implement MV on MV diff --git a/src/stream/src/executor/receiver.rs b/src/stream/src/executor/receiver.rs index 436ee8768cb80..3966b25febf5d 100644 --- a/src/stream/src/executor/receiver.rs +++ b/src/stream/src/executor/receiver.rs @@ -11,21 +11,17 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; use anyhow::Context; -use futures::StreamExt; -use futures_async_stream::try_stream; use itertools::Itertools; use tokio::time::Instant; use super::exchange::input::BoxedInput; -use super::ActorContextRef; use crate::executor::exchange::input::new_input; -use crate::executor::monitor::StreamingMetrics; +use crate::executor::prelude::*; use crate::executor::utils::ActorInputMetrics; -use crate::executor::{expect_first_barrier, BoxedMessageStream, Execute, Message}; use crate::task::{FragmentId, SharedContext}; + /// `ReceiverExecutor` is used along with a channel. After creating a mpsc channel, /// there should be a `ReceiverExecutor` running in the background, so as to push /// messages down to the executors. @@ -80,7 +76,6 @@ impl ReceiverExecutor { pub fn for_test(input: super::exchange::permit::Receiver) -> Self { use super::exchange::input::LocalInput; use crate::executor::exchange::input::Input; - use crate::executor::ActorContext; Self::new( ActorContext::for_test(114), diff --git a/src/stream/src/executor/row_id_gen.rs b/src/stream/src/executor/row_id_gen.rs index 12de3e0b88a36..c779092a307fb 100644 --- a/src/stream/src/executor/row_id_gen.rs +++ b/src/stream/src/executor/row_id_gen.rs @@ -12,20 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; -use futures_async_stream::try_stream; use risingwave_common::array::stream_chunk::Ops; -use risingwave_common::array::{ - Array, ArrayBuilder, ArrayRef, Op, SerialArrayBuilder, StreamChunk, -}; +use risingwave_common::array::{Array, ArrayBuilder, ArrayRef, Op, SerialArrayBuilder}; use risingwave_common::buffer::Bitmap; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::types::Serial; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::row_id::RowIdGenerator; -use super::{expect_first_barrier, ActorContextRef, Execute, Executor}; -use crate::executor::{Message, StreamExecutorError}; +use crate::executor::prelude::*; /// [`RowIdGenExecutor`] generates row id for data, where the user has not specified a pk. pub struct RowIdGenExecutor { diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index 890754f33462f..9c35627857828 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -12,26 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; -use futures_async_stream::try_stream; -use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; +use std::collections::HashMap; + +use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{build_retractable, AggCall, BoxedAggregateFunction}; use risingwave_pb::stream_plan::PbAggNodeVersion; -use risingwave_storage::StateStore; use super::agg_common::{AggExecutorArgs, SimpleAggExecutorExtraArgs}; use super::aggregation::{ agg_call_filter_res, iter_table_storage, AggStateStorage, AlwaysOutput, DistinctDeduplicater, }; -use super::*; -use crate::common::table::state_table::StateTable; -use crate::error::StreamResult; use crate::executor::aggregation::AggGroup; -use crate::executor::error::StreamExecutorError; -use crate::executor::{BoxedMessageStream, Message}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// `SimpleAggExecutor` is the aggregation operator for streaming system. /// To create an aggregation operator, states and expressions should be passed along the @@ -306,9 +299,9 @@ mod tests { use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::StateStore; + use super::*; use crate::executor::test_utils::agg_executor::new_boxed_simple_agg_executor; use crate::executor::test_utils::*; - use crate::executor::*; #[tokio::test] async fn test_simple_aggregation_in_memory() { diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index f552f8ba8e184..858fe39bd1b8e 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -16,14 +16,12 @@ use std::mem; use anyhow::anyhow; use futures::stream::select; -use futures::{pin_mut, FutureExt, StreamExt, TryFutureExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::{FutureExt, TryFutureExt, TryStreamExt}; use itertools::Itertools; use risingwave_common::array::stream_chunk::StreamChunkMut; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::{ColumnCatalog, Field, Schema}; +use risingwave_common::array::Op; +use risingwave_common::catalog::{ColumnCatalog, Field}; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; -use risingwave_common::types::DataType; use risingwave_connector::dispatch_sink; use risingwave_connector::sink::catalog::SinkType; use risingwave_connector::sink::log_store::{ @@ -34,13 +32,8 @@ use risingwave_connector::sink::{ }; use thiserror_ext::AsReport; -use super::error::{StreamExecutorError, StreamExecutorResult}; -use super::{Execute, Executor, ExecutorInfo, Message, PkIndices}; use crate::common::compact_chunk::{merge_chunk_row, StreamChunkCompactor}; -use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedMessageStream, MessageStream, Mutation, -}; -use crate::task::ActorId; +use crate::executor::prelude::*; pub struct SinkExecutor { actor_context: ActorContextRef, diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index dc38d4dcfbeba..37d08c746bba8 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -12,19 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; -use futures_async_stream::try_stream; use risingwave_common::array::Op; -use risingwave_common::catalog::Schema; -use risingwave_storage::StateStore; use super::sort_buffer::SortBuffer; -use super::{ - expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, - StreamExecutorError, Watermark, -}; -use crate::common::table::state_table::StateTable; -use crate::common::StreamChunkBuilder; +use crate::executor::prelude::*; pub struct SortExecutor { input: Executor, diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index c555fb036d94c..6399b2bc930ce 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -12,20 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::{Debug, Formatter}; use std::marker::PhantomData; use std::ops::Bound; -use std::sync::Arc; use either::Either; -use futures::stream::{self, StreamExt}; -use futures::{pin_mut, TryStreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::array::StreamChunk; +use futures::{stream, TryStreamExt}; use risingwave_common::catalog::{ColumnId, TableId}; use risingwave_common::hash::VnodeBitmapExt; -use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{ScalarRef, ScalarRefImpl}; +use risingwave_common::types::ScalarRef; use risingwave_connector::source::filesystem::opendal_source::{ OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource, }; @@ -35,17 +29,14 @@ use risingwave_connector::source::{ BoxChunkSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, }; use risingwave_storage::store::PrefetchOptions; -use risingwave_storage::StateStore; use thiserror_ext::AsReport; -use super::{get_split_offset_col_idx, SourceStateTableHandler}; -use crate::executor::source_executor::apply_rate_limit; -use crate::executor::stream_reader::StreamReaderWithPause; -use crate::executor::{ - expect_first_barrier, get_split_offset_mapping_from_chunk, prune_additional_cols, - ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError, - StreamExecutorResult, StreamSourceCore, +use super::{get_split_offset_col_idx, SourceStateTableHandler, StreamSourceCore}; +use crate::executor::prelude::*; +use crate::executor::source::{ + apply_rate_limit, get_split_offset_mapping_from_chunk, prune_additional_cols, }; +use crate::executor::stream_reader::StreamReaderWithPause; const SPLIT_BATCH_SIZE: usize = 1000; diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 36b7783950232..77e93e086c8fd 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -15,32 +15,32 @@ #![deprecated = "will be replaced by new fs source (list + fetch)"] #![expect(deprecated)] -use std::fmt::{Debug, Formatter}; -use std::sync::Arc; +use std::collections::{HashMap, HashSet}; use anyhow::anyhow; use either::Either; -use futures::{StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::TryStreamExt; +use itertools::Itertools; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_common::util::epoch::EpochPair; use risingwave_connector::error::ConnectorError; use risingwave_connector::source::reader::desc::{FsSourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, }; -use risingwave_storage::StateStore; use tokio::sync::mpsc::UnboundedReceiver; use tokio::time::Instant; use super::executor_core::StreamSourceCore; -use crate::error::StreamResult; -use crate::executor::error::StreamExecutorError; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::source_executor::apply_rate_limit; +use crate::executor::prelude::*; +use crate::executor::source::{ + apply_rate_limit, barrier_to_message_stream, get_split_offset_col_idx, + get_split_offset_mapping_from_chunk, prune_additional_cols, +}; use crate::executor::stream_reader::StreamReaderWithPause; -use crate::executor::*; +use crate::executor::{AddMutation, UpdateMutation}; /// A constant to multiply when calculating the maximum time to wait for a barrier. This is due to /// some latencies in network and cost in meta. diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 6d5fde8ee86bb..a5c2be3cb10b6 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -12,25 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::Formatter; -use std::sync::Arc; - use anyhow::anyhow; use either::Either; -use futures::{StreamExt, TryStreamExt}; +use futures::TryStreamExt; use futures_async_stream::try_stream; use risingwave_common::array::Op; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::SourceCtrlOpts; -use risingwave_storage::StateStore; use thiserror_ext::AsReport; use tokio::sync::mpsc::UnboundedReceiver; -use crate::executor::error::StreamExecutorError; -use crate::executor::monitor::StreamingMetrics; +use super::{barrier_to_message_stream, StreamSourceCore}; +use crate::executor::prelude::*; use crate::executor::stream_reader::StreamReaderWithPause; -use crate::executor::*; const CHUNK_SIZE: usize = 1024; diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index eae70cfe657a5..5777926a6a34f 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -12,14 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod executor_core; use std::collections::HashMap; use await_tree::InstrumentAwait; -pub use executor_core::StreamSourceCore; -mod fs_source_executor; -#[expect(deprecated)] -pub use fs_source_executor::*; use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::bail; @@ -28,18 +23,27 @@ use risingwave_connector::source::{SourceColumnDesc, SplitId}; use risingwave_pb::plan_common::additional_column::ColumnType; use risingwave_pb::plan_common::AdditionalColumn; pub use state_table_handler::*; -pub mod fetch_executor; + +mod executor_core; +pub use executor_core::StreamSourceCore; + +mod fs_source_executor; +#[expect(deprecated)] +pub use fs_source_executor::*; +mod source_executor; +pub use source_executor::*; +mod source_backfill_executor; +pub use source_backfill_executor::*; +mod list_executor; +pub use list_executor::*; +mod fetch_executor; pub use fetch_executor::*; -pub mod source_backfill_executor; -pub mod source_backfill_state_table; +mod source_backfill_state_table; pub use source_backfill_state_table::BackfillStateTableHandler; -pub mod source_executor; -pub mod list_executor; pub mod state_table_handler; use futures_async_stream::try_stream; -pub use list_executor::*; use tokio::sync::mpsc::UnboundedReceiver; use crate::executor::error::StreamExecutorError; diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 7b3c594cab5af..f399fdb8d5a55 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -14,35 +14,32 @@ use std::cmp::Ordering; use std::collections::hash_map::Entry; -use std::fmt::Formatter; +use std::collections::{HashMap, HashSet}; use std::time::Instant; use anyhow::anyhow; use either::Either; use futures::stream::{select_with_strategy, PollNext}; -use futures::StreamExt; -use futures_async_stream::try_stream; +use itertools::Itertools; use prometheus::IntCounter; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; -use risingwave_common::row::Row; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::types::JsonbVal; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ - BoxChunkSourceStream, SourceContext, SourceCtrlOpts, SplitId, SplitMetaData, + BoxChunkSourceStream, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, }; -use risingwave_storage::StateStore; use serde::{Deserialize, Serialize}; -use source_backfill_executor::source_executor::WAIT_BARRIER_MULTIPLE_TIMES; use thiserror_ext::AsReport; use super::executor_core::StreamSourceCore; use super::source_backfill_state_table::BackfillStateTableHandler; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::source_executor::apply_rate_limit; -use crate::executor::*; +use crate::executor::prelude::*; +use crate::executor::source::source_executor::WAIT_BARRIER_MULTIPLE_TIMES; +use crate::executor::source::{apply_rate_limit, get_split_offset_col_idx}; +use crate::executor::{AddMutation, UpdateMutation}; #[derive(Clone, Debug, Deserialize, Serialize)] pub enum BackfillState { diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index cf54166afd396..a752fa234cad5 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -12,37 +12,42 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::Formatter; +use std::collections::HashMap; use std::num::NonZeroU32; use std::str::FromStr; use std::time::Duration; use anyhow::anyhow; use either::Either; -use futures::{StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::TryStreamExt; use governor::clock::MonotonicClock; use governor::{Quota, RateLimiter}; +use itertools::Itertools; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_common::util::epoch::{Epoch, EpochPair}; use risingwave_connector::error::ConnectorError; use risingwave_connector::source::cdc::jni_source; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ - BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitMetaData, + BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, + SplitMetaData, }; use risingwave_hummock_sdk::HummockReadEpoch; -use risingwave_storage::StateStore; use thiserror_ext::AsReport; use tokio::sync::mpsc; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::time::Instant; use super::executor_core::StreamSourceCore; -use crate::executor::monitor::StreamingMetrics; +use crate::executor::prelude::*; +use crate::executor::source::{ + barrier_to_message_stream, get_split_offset_col_idx, get_split_offset_mapping_from_chunk, + prune_additional_cols, +}; use crate::executor::stream_reader::StreamReaderWithPause; -use crate::executor::*; +use crate::executor::{AddMutation, UpdateMutation}; /// A constant to multiply when calculating the maximum time to wait for a barrier. This is due to /// some latencies in network and cost in meta. @@ -769,6 +774,7 @@ impl WaitEpochWorker { #[cfg(test)] mod tests { + use std::collections::HashSet; use std::time::Duration; use futures::StreamExt; @@ -788,6 +794,7 @@ mod tests { use tracing_test::traced_test; use super::*; + use crate::executor::source::{default_source_internal_table, SourceStateTableHandler}; use crate::executor::ActorContext; const MOCK_SOURCE_NAME: &str = "mock_source"; diff --git a/src/stream/src/executor/stateless_simple_agg.rs b/src/stream/src/executor/stateless_simple_agg.rs index c6bca07df8cad..1360f4166008e 100644 --- a/src/stream/src/executor/stateless_simple_agg.rs +++ b/src/stream/src/executor/stateless_simple_agg.rs @@ -12,19 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; -use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{ build_retractable, AggCall, AggregateState, BoxedAggregateFunction, }; use super::aggregation::agg_call_filter_res; -use super::error::StreamExecutorError; -use super::*; -use crate::error::StreamResult; +use crate::executor::prelude::*; pub struct StatelessSimpleAggExecutor { _ctx: ActorContextRef, diff --git a/src/stream/src/executor/stream_reader.rs b/src/stream/src/executor/stream_reader.rs index 6bbf76d0bf85c..7fbf43a5794b0 100644 --- a/src/stream/src/executor/stream_reader.rs +++ b/src/stream/src/executor/stream_reader.rs @@ -137,7 +137,8 @@ mod tests { use tokio::sync::mpsc; use super::*; - use crate::executor::{barrier_to_message_stream, Barrier, StreamExecutorError}; + use crate::executor::source::barrier_to_message_stream; + use crate::executor::{Barrier, StreamExecutorError}; const TEST_TRANSACTION_ID1: TxnId = 0; const TEST_TRANSACTION_ID2: TxnId = 1; diff --git a/src/stream/src/executor/subscription.rs b/src/stream/src/executor/subscription.rs index cd752dfdf28cd..4bacb2dc75f05 100644 --- a/src/stream/src/executor/subscription.rs +++ b/src/stream/src/executor/subscription.rs @@ -16,19 +16,14 @@ use core::str::FromStr; use core::time::Duration; use std::collections::HashMap; -use futures::prelude::stream::StreamExt; -use futures_async_stream::try_stream; use risingwave_common::types::{Interval, Timestamptz}; use risingwave_common::util::epoch::Epoch; use risingwave_storage::store::LocalStateStore; use tokio::time::Instant; -use super::{ - expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, - StreamExecutorError, StreamExecutorResult, -}; use crate::common::log_store_impl::kv_log_store::ReaderTruncationOffsetType; use crate::common::log_store_impl::subscription_log_store::SubscriptionLogStoreWriter; +use crate::executor::prelude::*; const EXECUTE_GC_INTERVAL: u64 = 3600; const MAX_RETENTION_DAYS: i32 = 365; diff --git a/src/stream/src/executor/subtask.rs b/src/stream/src/executor/subtask.rs index fea9644b151f3..45e02932d1908 100644 --- a/src/stream/src/executor/subtask.rs +++ b/src/stream/src/executor/subtask.rs @@ -12,16 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use await_tree::InstrumentAwait; -use futures::{Future, StreamExt}; +use futures::Future; use thiserror_ext::AsReport; use tokio::sync::mpsc; use tokio::sync::mpsc::error::SendError; use tokio_stream::wrappers::ReceiverStream; use super::actor::spawn_blocking_drop_stream; -use super::{Execute, Executor, Message, MessageStreamItem}; -use crate::task::ActorId; +use crate::executor::prelude::*; /// Handle used to drive the subtask. pub type SubtaskHandle = impl Future + Send + 'static; diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index f365706317a90..b744fb245a3b4 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -15,21 +15,17 @@ use std::alloc::Global; use std::collections::hash_map::Entry; use std::collections::HashMap; -use std::pin::pin; -use std::sync::Arc; use either::Either; use futures::stream::{self, PollNext}; -use futures::{pin_mut, StreamExt, TryStreamExt}; -use futures_async_stream::{for_await, try_stream}; +use futures::TryStreamExt; use itertools::Itertools; use local_stats_alloc::{SharedStatsAlloc, StatsAlloc}; use lru::DefaultHasher; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::hash::{HashKey, NullBitmap}; -use risingwave_common::row::{OwnedRow, Row, RowExt}; -use risingwave_common::types::DataType; +use risingwave_common::row::RowExt; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common_estimate_size::{EstimateSize, KvSize}; use risingwave_expr::expr::NonStrictExpression; @@ -37,20 +33,12 @@ use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch}; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableIter; -use risingwave_storage::StateStore; use super::join::{JoinType, JoinTypePrimitive}; -use super::{ - Barrier, Execute, ExecutorInfo, Message, MessageStream, StreamExecutorError, - StreamExecutorResult, -}; use crate::cache::{cache_may_stale, new_with_hasher_in, ManagedLruCache}; use crate::common::metrics::MetricsInfo; -use crate::common::table::state_table::StateTable; use crate::executor::join::builder::JoinStreamChunkBuilder; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, Executor, Watermark}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; pub struct TemporalJoinExecutor< K: HashKey, diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs index 4f8e3da165994..3a8a618e3c836 100644 --- a/src/stream/src/executor/top_n/group_top_n.rs +++ b/src/stream/src/executor/top_n/group_top_n.rs @@ -13,28 +13,21 @@ // limitations under the License. use std::ops::{Deref, DerefMut}; -use std::sync::Arc; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::Schema; use risingwave_common::hash::HashKey; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_storage::StateStore; use super::top_n_cache::TopNCacheTrait; use super::utils::*; use super::{ManagedTopNState, TopNCache}; use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; -use crate::common::table::state_table::StateTable; -use crate::error::StreamResult; -use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; pub type GroupTopNExecutor = TopNExecutorWrapper>; diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs index 346d73fed0196..cbd3f8215c87b 100644 --- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs @@ -12,28 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::Schema; use risingwave_common::hash::HashKey; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_storage::StateStore; use super::group_top_n::GroupTopNCache; use super::top_n_cache::AppendOnlyTopNCacheTrait; use super::utils::*; use super::{ManagedTopNState, TopNCache}; use crate::common::metrics::MetricsInfo; -use crate::common::table::state_table::StateTable; -use crate::error::StreamResult; -use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; -use crate::task::AtomicU64Ref; +use crate::executor::prelude::*; /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need /// to keep all the rows seen. As long as a record diff --git a/src/stream/src/executor/top_n/top_n_appendonly.rs b/src/stream/src/executor/top_n/top_n_appendonly.rs index c88d9af3c0f7f..c99b911b951ef 100644 --- a/src/stream/src/executor/top_n/top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/top_n_appendonly.rs @@ -12,20 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; +use risingwave_common::array::Op; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_storage::StateStore; use super::top_n_cache::AppendOnlyTopNCacheTrait; use super::utils::*; use super::{ManagedTopNState, TopNCache, NO_GROUP_KEY}; -use crate::common::table::state_table::StateTable; -use crate::error::StreamResult; -use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; +use crate::executor::prelude::*; /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need /// to keep all the rows seen. As long as a record diff --git a/src/stream/src/executor/top_n/top_n_plain.rs b/src/stream/src/executor/top_n/top_n_plain.rs index 5a70efc3f3f8e..30aee860a8fc8 100644 --- a/src/stream/src/executor/top_n/top_n_plain.rs +++ b/src/stream/src/executor/top_n/top_n_plain.rs @@ -12,19 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; +use risingwave_common::array::Op; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_storage::StateStore; use super::utils::*; use super::{ManagedTopNState, TopNCache, TopNCacheTrait}; -use crate::common::table::state_table::StateTable; -use crate::error::StreamResult; -use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; +use crate::executor::prelude::*; /// `TopNExecutor` works with input with modification, it keeps all the data /// records/rows that have been seen, and returns topN records overall. diff --git a/src/stream/src/executor/top_n/utils.rs b/src/stream/src/executor/top_n/utils.rs index bbd956cde2168..4700a8900221e 100644 --- a/src/stream/src/executor/top_n/utils.rs +++ b/src/stream/src/executor/top_n/utils.rs @@ -13,26 +13,18 @@ // limitations under the License. use std::future::Future; -use std::sync::Arc; -use futures::StreamExt; -use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::Op; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::Schema; -use risingwave_common::row::{CompactedRow, Row, RowDeserializer}; +use risingwave_common::row::{CompactedRow, RowDeserializer}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::row_serde::OrderedRowSerde; use risingwave_common::util::sort_util::ColumnOrder; use super::CacheKey; -use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; -use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, - Watermark, -}; +use crate::executor::prelude::*; pub trait TopNExecutorBase: Send + 'static { /// Apply the chunk to the dirty state and get the diffs. diff --git a/src/stream/src/executor/troublemaker.rs b/src/stream/src/executor/troublemaker.rs index 72a257f331450..7b93392fef419 100644 --- a/src/stream/src/executor/troublemaker.rs +++ b/src/stream/src/executor/troublemaker.rs @@ -12,20 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; -use futures_async_stream::try_stream; use rand::Rng; use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; use risingwave_common::array::stream_record::{Record, RecordType}; use risingwave_common::array::Op; use risingwave_common::field_generator::{FieldGeneratorImpl, VarcharProperty}; -use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use smallvec::SmallVec; -use super::{BoxedMessageStream, Execute, Executor, Message, StreamExecutorError}; use crate::consistency::insane; +use crate::executor::prelude::*; /// [`TroublemakerExecutor`] is used to make some trouble in the stream graph. Specifically, /// it is attached to `StreamScan` and `Source` executors in **insane mode**. It randomly diff --git a/src/stream/src/executor/union.rs b/src/stream/src/executor/union.rs index 5881474d90197..ac8f3581dda18 100644 --- a/src/stream/src/executor/union.rs +++ b/src/stream/src/executor/union.rs @@ -17,14 +17,10 @@ use std::pin::Pin; use std::task::{Context, Poll}; use futures::stream::{FusedStream, FuturesUnordered}; -use futures::{Stream, StreamExt}; -use futures_async_stream::try_stream; use pin_project::pin_project; use super::watermark::BufferedWatermarks; -use super::{ - Barrier, BoxedMessageStream, Execute, Executor, Message, MessageStreamItem, StreamExecutorError, -}; +use crate::executor::prelude::*; /// `UnionExecutor` merges data from multiple inputs. pub struct UnionExecutor { diff --git a/src/stream/src/executor/utils.rs b/src/stream/src/executor/utils.rs index 18b64633a3e12..565866e0d9e95 100644 --- a/src/stream/src/executor/utils.rs +++ b/src/stream/src/executor/utils.rs @@ -12,12 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::StreamExt; use risingwave_common::metrics::LabelGuardedIntCounter; -use crate::executor::monitor::StreamingMetrics; -use crate::executor::{BoxedMessageStream, Execute}; -use crate::task::{ActorId, FragmentId}; +use crate::executor::prelude::*; +use crate::task::FragmentId; #[derive(Default)] pub struct DummyExecutor; diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 79fdf890cc7e7..dfa5579d66b49 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -14,17 +14,13 @@ use std::vec; -use await_tree::InstrumentAwait; -use futures::StreamExt; -use futures_async_stream::try_stream; -use risingwave_common::array::{DataChunk, Op, StreamChunk}; -use risingwave_common::catalog::Schema; +use risingwave_common::array::{DataChunk, Op}; use risingwave_common::ensure; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; use tokio::sync::mpsc::UnboundedReceiver; -use super::{ActorContextRef, Barrier, BoxedMessageStream, Execute, Message, StreamExecutorError}; +use crate::executor::prelude::*; use crate::task::CreateMviewProgress; const DEFAULT_CHUNK_SIZE: usize = 1024; diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index fc2f922371155..813ccbef28920 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -14,14 +14,10 @@ use std::cmp; use std::ops::Deref; -use std::sync::Arc; use futures::future::{try_join, try_join_all}; -use futures::StreamExt; -use futures_async_stream::try_stream; use risingwave_common::hash::VnodeBitmapExt; -use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{DataType, DefaultOrd, ScalarImpl}; +use risingwave_common::types::{DefaultOrd, ScalarImpl}; use risingwave_common::{bail, row}; use risingwave_expr::expr::{ build_func_non_strict, ExpressionBoxExt, InputRefExpression, LiteralExpression, @@ -32,13 +28,9 @@ use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_pb::expr::expr_node::Type; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableDistribution; -use risingwave_storage::StateStore; -use super::error::StreamExecutorError; use super::filter::FilterExecutor; -use super::{ActorContextRef, Execute, Executor, Message, StreamExecutorResult}; -use crate::common::table::state_table::StateTable; -use crate::executor::{expect_first_barrier, Watermark}; +use crate::executor::prelude::*; use crate::task::ActorEvalErrorReport; /// The executor will generate a `Watermark` after each chunk. diff --git a/src/stream/src/executor/wrapper.rs b/src/stream/src/executor/wrapper.rs index 74923928eaf6d..3da1a9f0127d3 100644 --- a/src/stream/src/executor/wrapper.rs +++ b/src/stream/src/executor/wrapper.rs @@ -12,11 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - -use futures::StreamExt; - -use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, MessageStream}; +use crate::executor::prelude::*; mod epoch_check; mod epoch_provide; diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index e07670fb97a6d..04baf944b552f 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -24,9 +24,8 @@ use risingwave_pb::stream_plan::StreamFsFetchNode; use risingwave_storage::StateStore; use crate::error::StreamResult; -use crate::executor::{ - Execute, Executor, FsFetchExecutor, SourceStateTableHandler, StreamSourceCore, -}; +use crate::executor::source::{FsFetchExecutor, SourceStateTableHandler, StreamSourceCore}; +use crate::executor::{Execute, Executor}; use crate::from_proto::ExecutorBuilder; use crate::task::ExecutorParams; diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 3610c8a95caf0..ac305e53d386a 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -35,9 +35,9 @@ use risingwave_storage::panic_store::PanicStateStore; use tokio::sync::mpsc::unbounded_channel; use super::*; -use crate::executor::source::{FsListExecutor, StreamSourceCore}; -use crate::executor::source_executor::SourceExecutor; -use crate::executor::state_table_handler::SourceStateTableHandler; +use crate::executor::source::{ + FsListExecutor, SourceExecutor, SourceStateTableHandler, StreamSourceCore, +}; use crate::executor::TroublemakerExecutor; const FS_CONNECTORS: &[&str] = &["s3"]; @@ -213,7 +213,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { if is_fs_connector { #[expect(deprecated)] - crate::executor::FsSourceExecutor::new( + crate::executor::source::FsSourceExecutor::new( params.actor_context.clone(), stream_source_core, params.executor_stats, diff --git a/src/stream/src/from_proto/source_backfill.rs b/src/stream/src/from_proto/source_backfill.rs index eadb949217070..12b159b6fdd7c 100644 --- a/src/stream/src/from_proto/source_backfill.rs +++ b/src/stream/src/from_proto/source_backfill.rs @@ -17,12 +17,10 @@ use risingwave_connector::source::SourceCtrlOpts; use risingwave_pb::stream_plan::SourceBackfillNode; use super::*; -use crate::executor::source::StreamSourceCore; -use crate::executor::source_backfill_executor::{ - SourceBackfillExecutor, SourceBackfillExecutorInner, +use crate::executor::source::{ + BackfillStateTableHandler, SourceBackfillExecutor, SourceBackfillExecutorInner, + SourceStateTableHandler, StreamSourceCore, }; -use crate::executor::state_table_handler::SourceStateTableHandler; -use crate::executor::BackfillStateTableHandler; pub struct SourceBackfillExecutorBuilder; From 8b089f1d62820cf058019b4a19c1b2196e4c644e Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Sat, 20 Apr 2024 01:51:34 +0800 Subject: [PATCH 20/22] feat(meta): collect non-checkpoint epoch for checkpoint barrier commit (#16400) --- src/meta/src/barrier/command.rs | 36 +++++++++++++++++++++++++- src/meta/src/barrier/mod.rs | 44 +++++++++++++++++--------------- src/meta/src/barrier/recovery.rs | 3 +-- src/meta/src/barrier/rpc.rs | 2 +- 4 files changed, 61 insertions(+), 24 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 73157f91c2d0c..6cece710bd10b 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -27,7 +27,7 @@ use risingwave_pb::catalog::CreateType; use risingwave_pb::meta::table_fragments::PbActorStatus; use risingwave_pb::meta::PausedReason; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; -use risingwave_pb::stream_plan::barrier::BarrierKind; +use risingwave_pb::stream_plan::barrier::BarrierKind as PbBarrierKind; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::throttle_mutation::RateLimit; use risingwave_pb::stream_plan::update_mutation::*; @@ -304,6 +304,40 @@ impl Command { } } +#[derive(Debug)] +pub enum BarrierKind { + Initial, + Barrier, + /// Hold a list of previous non-checkpoint prev-epoch + current prev-epoch + Checkpoint(Vec), +} + +impl BarrierKind { + pub fn to_protobuf(&self) -> PbBarrierKind { + match self { + BarrierKind::Initial => PbBarrierKind::Initial, + BarrierKind::Barrier => PbBarrierKind::Barrier, + BarrierKind::Checkpoint(_) => PbBarrierKind::Checkpoint, + } + } + + pub fn is_checkpoint(&self) -> bool { + matches!(self, BarrierKind::Checkpoint(_)) + } + + pub fn is_initial(&self) -> bool { + matches!(self, BarrierKind::Initial) + } + + pub fn as_str_name(&self) -> &'static str { + match self { + BarrierKind::Initial => "Initial", + BarrierKind::Barrier => "Barrier", + BarrierKind::Checkpoint(_) => "Checkpoint", + } + } +} + /// [`CommandContext`] is used for generating barrier and doing post stuffs according to the given /// [`Command`]. pub struct CommandContext { diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 0b570d9b2aaa1..05dbdddfc3320 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -38,7 +38,6 @@ use risingwave_pb::catalog::table::TableType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::PausedReason; -use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; use risingwave_pb::stream_service::BarrierCompleteResponse; use thiserror_ext::AsReport; @@ -77,7 +76,7 @@ mod schedule; mod state; mod trace; -pub use self::command::{Command, ReplaceTablePlan, Reschedule}; +pub use self::command::{BarrierKind, Command, ReplaceTablePlan, Reschedule}; pub use self::rpc::StreamRpcManager; pub use self::schedule::BarrierScheduler; pub use self::trace::TracedEpoch; @@ -191,6 +190,9 @@ pub struct GlobalBarrierManager { checkpoint_control: CheckpointControl, + /// The `prev_epoch` of pending non checkpoint barriers + pending_non_checkpoint_barriers: Vec, + active_streaming_nodes: ActiveStreamingWorkerNodes, control_stream_manager: ControlStreamManager, @@ -477,6 +479,7 @@ impl GlobalBarrierManager { env, state: initial_invalid_state, checkpoint_control, + pending_non_checkpoint_barriers: Vec::new(), active_streaming_nodes, control_stream_manager, } @@ -720,8 +723,11 @@ impl GlobalBarrierManager { let info = self.state.apply_command(&command); let (prev_epoch, curr_epoch) = self.state.next_epoch_pair(); + self.pending_non_checkpoint_barriers + .push(prev_epoch.value().0); let kind = if checkpoint { - BarrierKind::Checkpoint + let epochs = take(&mut self.pending_non_checkpoint_barriers); + BarrierKind::Checkpoint(epochs) } else { BarrierKind::Barrier }; @@ -782,6 +788,7 @@ impl GlobalBarrierManager { async fn failure_recovery(&mut self, err: MetaError) { self.context.tracker.lock().await.abort_all(&err); self.checkpoint_control.clear_on_err(&err).await; + self.pending_non_checkpoint_barriers.clear(); if self.enable_recovery { self.context @@ -844,8 +851,11 @@ impl GlobalBarrierManagerContext { assert!(state.node_to_collect.is_empty()); let resps = state.resps; let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); - let (commit_info, create_mview_progress) = collect_commit_epoch_info(resps, &command_ctx); - if let Err(e) = self.update_snapshot(&command_ctx, commit_info).await { + let create_mview_progress = resps + .iter() + .flat_map(|resp| resp.create_mview_progress.iter().cloned()) + .collect(); + if let Err(e) = self.update_snapshot(&command_ctx, resps).await { for notifier in notifiers { notifier.notify_collection_failed(e.clone()); } @@ -872,7 +882,7 @@ impl GlobalBarrierManagerContext { async fn update_snapshot( &self, command_ctx: &CommandContext, - commit_info: CommitEpochInfo, + resps: Vec, ) -> MetaResult<()> { { { @@ -881,17 +891,13 @@ impl GlobalBarrierManagerContext { // because the storage engine will query from new to old in the order in which // the L0 layer files are generated. // See https://github.com/risingwave-labs/risingwave/issues/1251 - let kind = command_ctx.kind; // hummock_manager commit epoch. let mut new_snapshot = None; - match kind { - BarrierKind::Unspecified => unreachable!(), - BarrierKind::Initial => assert!( - commit_info.sstables.is_empty(), - "no sstables should be produced in the first epoch" - ), - BarrierKind::Checkpoint => { + match &command_ctx.kind { + BarrierKind::Initial => {} + BarrierKind::Checkpoint(epochs) => { + let commit_info = collect_commit_epoch_info(resps, command_ctx, epochs); new_snapshot = self .hummock_manager .commit_epoch(command_ctx.prev_epoch.value().0, commit_info) @@ -1138,11 +1144,11 @@ pub type BarrierManagerRef = GlobalBarrierManagerContext; fn collect_commit_epoch_info( resps: Vec, command_ctx: &CommandContext, -) -> (CommitEpochInfo, Vec) { + _epochs: &Vec, +) -> CommitEpochInfo { let mut sst_to_worker: HashMap = HashMap::new(); let mut synced_ssts: Vec = vec![]; let mut table_watermarks = Vec::with_capacity(resps.len()); - let mut progresses = Vec::new(); for resp in resps { let ssts_iter = resp.synced_sstables.into_iter().map(|grouped| { let sst_info = grouped.sst.expect("field not None"); @@ -1155,7 +1161,6 @@ fn collect_commit_epoch_info( }); synced_ssts.extend(ssts_iter); table_watermarks.push(resp.table_watermarks); - progresses.extend(resp.create_mview_progress); } let new_table_fragment_info = if let Command::CreateStreamingJob { table_fragments, .. @@ -1174,7 +1179,7 @@ fn collect_commit_epoch_info( None }; - let info = CommitEpochInfo::new( + CommitEpochInfo::new( synced_ssts, merge_multiple_new_table_watermarks( table_watermarks @@ -1194,6 +1199,5 @@ fn collect_commit_epoch_info( ), sst_to_worker, new_table_fragment_info, - ); - (info, progresses) + ) } diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index f2ec59c3d6f48..a3f19e555bf75 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -25,7 +25,6 @@ use risingwave_pb::common::ActorInfo; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::State; use risingwave_pb::meta::{PausedReason, Recovery}; -use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::AddMutation; use thiserror_ext::AsReport; @@ -41,7 +40,7 @@ use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::rpc::ControlStreamManager; use crate::barrier::schedule::ScheduledBarriers; use crate::barrier::state::BarrierManagerState; -use crate::barrier::{Command, GlobalBarrierManager, GlobalBarrierManagerContext}; +use crate::barrier::{BarrierKind, Command, GlobalBarrierManager, GlobalBarrierManagerContext}; use crate::controller::catalog::ReleaseContext; use crate::manager::{ActiveStreamingWorkerNodes, MetadataManager, WorkerId}; use crate::model::{MetadataModel, MigrationPlan, TableFragments, TableParallelism}; diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index b7ea512ffbbfa..d23859609e05e 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -280,7 +280,7 @@ impl ControlStreamManager { command_context.curr_epoch.span(), ) .to_protobuf(), - kind: command_context.kind as i32, + kind: command_context.kind.to_protobuf() as i32, passed_actors: vec![], }; From faaed4ec432b771289fb7614808b8c105b7d6284 Mon Sep 17 00:00:00 2001 From: Zihao Xu Date: Sat, 20 Apr 2024 00:01:39 -0400 Subject: [PATCH 21/22] chore(snowflake-sink): update spec and related demo sql(s) (#16415) --- integration_tests/snowflake-sink/create_sink.sql | 5 ++--- integration_tests/snowflake-sink/create_source.sql | 3 +++ integration_tests/snowflake-sink/snowflake_prep.sql | 10 +++++++--- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/integration_tests/snowflake-sink/create_sink.sql b/integration_tests/snowflake-sink/create_sink.sql index 457def4f82e2d..0986005d8c717 100644 --- a/integration_tests/snowflake-sink/create_sink.sql +++ b/integration_tests/snowflake-sink/create_sink.sql @@ -12,9 +12,8 @@ CREATE SINK snowflake_sink FROM ss_mv WITH ( snowflake.aws_access_key_id = 'EXAMPLE_AWS_ID', snowflake.aws_secret_access_key = 'EXAMPLE_SECRET_KEY', snowflake.aws_region = 'EXAMPLE_REGION', - snowflake.max_batch_row_num = '1030', snowflake.s3_path = 'EXAMPLE_S3_PATH', - -- depends on your mv setup, note that snowflake sink only supports - -- append-only at present. + -- depends on your mv setup, note that snowflake sink *only* supports + -- `append-only` mode at present. force_append_only = 'true' ); \ No newline at end of file diff --git a/integration_tests/snowflake-sink/create_source.sql b/integration_tests/snowflake-sink/create_source.sql index ed7c02341638a..0a5bc60f49922 100644 --- a/integration_tests/snowflake-sink/create_source.sql +++ b/integration_tests/snowflake-sink/create_source.sql @@ -1,3 +1,6 @@ +-- please note that this will create a source that generates 1,000 rows in 10 seconds +-- you may want to change the configuration for better testing / demo purpose + CREATE table user_behaviors ( user_id int, target_id VARCHAR, diff --git a/integration_tests/snowflake-sink/snowflake_prep.sql b/integration_tests/snowflake-sink/snowflake_prep.sql index 96026d0884415..b684a93b2d299 100644 --- a/integration_tests/snowflake-sink/snowflake_prep.sql +++ b/integration_tests/snowflake-sink/snowflake_prep.sql @@ -19,11 +19,15 @@ SELECT TRIM( CREATE OR REPLACE TABLE example_snowflake_sink_table (user_id INT, target_id VARCHAR, event_timestamp TIMESTAMP_TZ); -- snowflake stage, we only supports json as sink format at present -CREATE OR REPLACE STAGE example_snowflake_stage URL = '' - credentials = ( AWS_KEY_ID = '' AWS_SECRET_KEY = '' ) file_format = ( type = JSON ); +CREATE OR REPLACE STAGE example_snowflake_stage + url = '' + credentials = ( AWS_KEY_ID = '' AWS_SECRET_KEY = '' ) + file_format = ( type = JSON ); -- snowflake pipe -CREATE OR REPLACE PIPE example_snowflake_pipe AS COPY INTO example_snowflake_sink_table FROM @example_snowflake_stage MATCH_BY_COLUMN_NAME = CASE_INSENSITIVE; +CREATE OR REPLACE PIPE example_snowflake_pipe + AS COPY INTO example_snowflake_sink_table + FROM @example_snowflake_stage MATCH_BY_COLUMN_NAME = CASE_INSENSITIVE; -- select from table after sinking from rw SELECT * FROM example_snowflake_sink_table WHERE event_timestamp IS NOT NULL; From ce45021bbd2aa3ad2c358a4e3cda884d7a8f7a5e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 21 Apr 2024 11:05:53 +0800 Subject: [PATCH 22/22] chore(deps): Bump rustls from 0.23.4 to 0.23.5 (#16417) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 6 +++--- src/object_store/Cargo.toml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index bf24db465e5f1..52409c1b0db38 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11091,7 +11091,7 @@ dependencies = [ "prometheus", "reqwest 0.11.20", "risingwave_common", - "rustls 0.23.4", + "rustls 0.23.5", "spin 0.9.8", "thiserror", "thiserror-ext", @@ -11806,9 +11806,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.4" +version = "0.23.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c4d6d8ad9f2492485e13453acbb291dd08f64441b6609c491f1c2cd2c6b4fe1" +checksum = "afabcee0551bd1aa3e18e5adbf2c0544722014b899adb31bd186ec638d3da97e" dependencies = [ "aws-lc-rs", "log", diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index bda496ea24f80..e4d573e8f4d2d 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -32,7 +32,7 @@ opendal = "0.45.1" prometheus = { version = "0.13", features = ["process"] } reqwest = "0.11" # required by opendal risingwave_common = { workspace = true } -rustls = "0.23.4" +rustls = "0.23.5" spin = "0.9" thiserror = "1" thiserror-ext = { workspace = true }