From fbfa70c14ce53f9341a4cdd2513df0b1ca5d0970 Mon Sep 17 00:00:00 2001 From: Sandy Spicer Date: Tue, 8 Oct 2024 14:52:04 -0700 Subject: [PATCH] fix: udf fix for oom (#25403) Co-authored-by: github-actions <41898282+github-actions[bot]@users.noreply.github.com> --- docker/clickhouse/user_defined_function.xml | 16 +- funnel-udf/src/main.rs | 315 +-- .../insights/funnels/funnel_udf.py | 15 +- .../test_funnel_correlation_actors_udf.ambr | 28 +- .../test_funnel_correlation_udf.ambr | 250 +- .../test_funnel_persons_udf.ambr | 9 +- .../__snapshots__/test_funnel_strict.ambr | 1127 -------- .../test_funnel_strict_persons_udf.ambr | 9 +- .../__snapshots__/test_funnel_strict_udf.ambr | 2391 ++--------------- .../test/__snapshots__/test_funnel_udf.ambr | 48 +- .../insights/funnels/test/test_funnel.py | 85 + .../funnels/test/test_funnel_strict_udf.py | 70 + .../insights/funnels/test/test_funnel_udf.py | 39 + posthog/user_scripts/aggregate_funnel_aarch64 | Bin 567912 -> 574720 bytes posthog/user_scripts/aggregate_funnel_x86_64 | Bin 572616 -> 580240 bytes .../latest_user_defined_function.xml | 34 +- .../user_scripts/v0/aggregate_funnel_aarch64 | Bin 567912 -> 574720 bytes .../user_scripts/v0/aggregate_funnel_x86_64 | Bin 572616 -> 580240 bytes .../user_scripts/v0/user_defined_function.xml | 34 +- 19 files changed, 803 insertions(+), 3667 deletions(-) diff --git a/docker/clickhouse/user_defined_function.xml b/docker/clickhouse/user_defined_function.xml index 9b58b380066e6..6af0f89bdafdb 100644 --- a/docker/clickhouse/user_defined_function.xml +++ b/docker/clickhouse/user_defined_function.xml @@ -1,6 +1,6 @@ - executable + executable_pool aggregate_funnel Array(Tuple(Int8, Nullable(String), Array(Float64), Array(Array(UUID)))) result @@ -34,7 +34,7 @@ - executable + executable_pool aggregate_funnel_cohort Array(Tuple(Int8, UInt64, Array(Float64), Array(Array(UUID)))) result @@ -68,7 +68,7 @@ - executable + executable_pool aggregate_funnel_array Array(Tuple(Int8, Array(String), Array(Float64), Array(Array(UUID)))) result @@ -102,7 +102,7 @@ - executable + executable_pool aggregate_funnel_test String result @@ -136,7 +136,7 @@ - executable + executable_pool aggregate_funnel_trends Array(Tuple(DateTime, Int8, Nullable(String))) result @@ -178,7 +178,7 @@ - executable + executable_pool aggregate_funnel_array_trends Array(Tuple(DateTime, Int8, Array(String))) @@ -217,7 +217,7 @@ - executable + executable_pool aggregate_funnel_cohort_trends Array(Tuple(DateTime, Int8, UInt64)) @@ -256,7 +256,7 @@ - executable + executable_pool aggregate_funnel_array_trends_test String result diff --git a/funnel-udf/src/main.rs b/funnel-udf/src/main.rs index 78775a9dd35b3..6169b0de2f824 100644 --- a/funnel-udf/src/main.rs +++ b/funnel-udf/src/main.rs @@ -2,7 +2,7 @@ use serde::{Deserialize, Serialize}; use serde_json::json; use std::io::{self, BufRead, Write}; use std::iter::repeat; -use itertools::Itertools; +use itertools::{Itertools}; use uuid::Uuid; #[derive(Clone, PartialEq, Deserialize, Serialize)] @@ -25,13 +25,13 @@ struct Event { timestamp: f64, uuid: Uuid, breakdown: PropVal, - steps: Vec, + steps: Vec, } #[derive(Deserialize)] struct Args { - num_steps: i32, - conversion_window_limit: u64, + num_steps: usize, + conversion_window_limit: u64, // In seconds breakdown_attribution_type: String, funnel_order_type: String, prop_vals: Vec, @@ -39,47 +39,56 @@ struct Args { } #[derive(Serialize)] -struct Result(i32, PropVal, Vec, Vec>); +struct Result(i8, PropVal, Vec, Vec>); + +struct Vars { + max_step: (usize, EnteredTimestamp), + event_uuids: Vec>, + entered_timestamp: Vec +} + +struct AggregateFunnelRow { + breakdown_step: Option, + results: Vec, +} const MAX_REPLAY_EVENTS: usize = 10; +const DEFAULT_ENTERED_TIMESTAMP: EnteredTimestamp = EnteredTimestamp { + timestamp: 0.0, + timings: vec![], + uuids: vec![], +}; + #[inline(always)] fn parse_args(line: &str) -> Args { serde_json::from_str(line).expect("Invalid JSON input") } -#[inline(always)] -fn calculate_funnel_from_user_events( - num_steps: i32, - conversion_window_limit_seconds: u64, - breakdown_attribution_type: &str, - funnel_order_type: &str, - prop_vals: Vec, - events: Vec, -) -> Vec { - let default_entered_timestamp = EnteredTimestamp { - timestamp: 0.0, - timings: vec![], - uuids: vec![], - }; - let breakdown_step = if breakdown_attribution_type.starts_with("step_") { - breakdown_attribution_type[5..].parse::().ok() - } else { - None - }; +impl AggregateFunnelRow { + #[inline(always)] + fn calculate_funnel_from_user_events(&mut self, args: &Args) -> &Vec { + if args.breakdown_attribution_type.starts_with("step_") { + self.breakdown_step = args.breakdown_attribution_type[5..].parse::().ok() + } + + args.prop_vals.iter().for_each(|prop_val| self.loop_prop_val(args, prop_val)); - let mut results: Vec = Vec::with_capacity(prop_vals.len()); + &self.results + } - for prop_val in prop_vals { - let mut max_step = (0, default_entered_timestamp.clone()); - let mut entered_timestamp = vec![default_entered_timestamp.clone(); (num_steps + 1) as usize]; - let mut event_uuids: Vec> = repeat(Vec::new()).take(num_steps as usize).collect(); - let mut add_max_step = true; + #[inline(always)] + fn loop_prop_val(&mut self, args: &Args, prop_val: &PropVal) { + let mut vars = Vars { + max_step: (0, DEFAULT_ENTERED_TIMESTAMP.clone()), + event_uuids: repeat(Vec::new()).take(args.num_steps).collect(), + entered_timestamp: vec![DEFAULT_ENTERED_TIMESTAMP.clone(); args.num_steps + 1] + }; - let filtered_events = events.iter() + let filtered_events = args.value.iter() .filter(|e| { - if breakdown_attribution_type == "all_events" { - e.breakdown == prop_val + if args.breakdown_attribution_type == "all_events" { + e.breakdown == *prop_val } else { true } @@ -88,149 +97,152 @@ fn calculate_funnel_from_user_events( for (timestamp, events_with_same_timestamp) in &filtered_events { let events_with_same_timestamp: Vec<_> = events_with_same_timestamp.collect(); - entered_timestamp[0] = EnteredTimestamp { + vars.entered_timestamp[0] = EnteredTimestamp { timestamp, timings: vec![], uuids: vec![], }; if events_with_same_timestamp.len() == 1 { - if !process_event( + if !self.process_event( + args, + &mut vars, &events_with_same_timestamp[0], - &mut entered_timestamp, - &prop_val, - &mut event_uuids, - conversion_window_limit_seconds, - funnel_order_type, - &mut max_step, - breakdown_step, - &mut results, + prop_val, + false ) { - add_max_step = false; - break; + return; } - } else { - // Handle permutations for events with the same timestamp - let mut entered_timestamps: Vec<_> = vec![]; - for perm in events_with_same_timestamp.iter().permutations(events_with_same_timestamp.len()) { - entered_timestamps.push(entered_timestamp.clone()); - for event in perm { - if !process_event( - &event, - &mut entered_timestamps.last_mut().unwrap(), - &prop_val, - &mut event_uuids, - conversion_window_limit_seconds, - funnel_order_type, - &mut max_step, - breakdown_step, - &mut results, - ) { - add_max_step = false; - break; - } + } else if events_with_same_timestamp.iter().map(|x| &x.steps).all_equal() { + // Deal with the most common case where they are all the same event (order doesn't matter) + for event in events_with_same_timestamp { + if !self.process_event( + args, + &mut vars, + event, + prop_val, + false + ) { + return; } } - for i in 0..entered_timestamp.len() { - entered_timestamp[i] = entered_timestamps.iter().max_by_key(|x| x[i].timestamp as i32).unwrap()[i].clone(); + } else { + // Handle permutations for different events with the same timestamp + // We ignore strict steps and exclusions in this case + // The behavior here is mostly dictated by how it was handled in the old style + + let sorted_events = events_with_same_timestamp + .iter() + .flat_map(|&event| { + event.steps + .iter() + .filter(|&&step| step > 0) + .map(|&step| Event { steps: vec![step], ..event.clone() }) + }).sorted_by_key(|event| event.steps[0]); + + // Run exclusions, if they exist, then run matching events. + for event in sorted_events { + if !self.process_event( + args, + &mut vars, + &event, + &prop_val, + true + ) { + return; + } } } - if entered_timestamp[num_steps as usize].timestamp > 0.0 { + // If we hit the goal, we can terminate early + if vars.entered_timestamp[args.num_steps].timestamp > 0.0 { break; } } - if add_max_step { - let final_index = max_step.0; - let final_value = &max_step.1; + // Find the furthest step we have made it to and print it + let final_index = vars.max_step.0; + let final_value = &vars.max_step.1; - for i in 0..final_index { - //if event_uuids[i].len() >= MAX_REPLAY_EVENTS && !event_uuids[i].contains(&final_value.uuids[i]) { - // Always put the actual event uuids first, we use it to extract timestamps - // This might create duplicates, but that's fine (we can remove it in clickhouse) - event_uuids[i].insert(0, final_value.uuids[i].clone()); - } - results.push(Result( - final_index as i32 - 1, - prop_val, - final_value.timings.windows(2).map(|w| w[1] - w[0]).collect(), - event_uuids - )) + for i in 0..final_index { + //if event_uuids[i].len() >= MAX_REPLAY_EVENTS && !event_uuids[i].contains(&final_value.uuids[i]) { + // Always put the actual event uuids first, we use it to extract timestamps + // This might create duplicates, but that's fine (we can remove it in clickhouse) + vars.event_uuids[i].insert(0, final_value.uuids[i].clone()); } + self.results.push(Result( + final_index as i8 - 1, + prop_val.clone(), + final_value.timings.windows(2).map(|w| w[1] - w[0]).collect(), + vars.event_uuids, + )) } - results -} - -#[inline(always)] -fn process_event( - event: &Event, - entered_timestamp: &mut Vec, - prop_val: &PropVal, - event_uuids: &mut Vec>, - conversion_window_limit_seconds: u64, - funnel_order_type: &str, - max_step: &mut (usize, EnteredTimestamp), - breakdown_step: Option, - results: &mut Vec -) -> bool { - for step in event.steps.iter().rev() { - let mut exclusion = false; - let step = (if *step < 0 { - exclusion = true; - -*step - } else { - *step - }) as usize; + #[inline(always)] + fn process_event( + &mut self, + args: &Args, + vars: &mut Vars, + event: &Event, + prop_val: &PropVal, + processing_multiple_events: bool + ) -> bool { + for step in event.steps.iter().rev() { + let mut exclusion = false; + let step = (if *step < 0 { + exclusion = true; + -*step + } else { + *step + }) as usize; - let in_match_window = (event.timestamp - entered_timestamp[step - 1].timestamp) <= conversion_window_limit_seconds as f64; - let already_reached_this_step = entered_timestamp[step].timestamp == entered_timestamp[step - 1].timestamp - && entered_timestamp[step].timestamp != 0.0; + let in_match_window = (event.timestamp - vars.entered_timestamp[step - 1].timestamp) <= args.conversion_window_limit as f64; + let already_reached_this_step = vars.entered_timestamp[step].timestamp == vars.entered_timestamp[step - 1].timestamp + && vars.entered_timestamp[step].timestamp != 0.0; - if in_match_window && !already_reached_this_step { - if exclusion { - results.push(Result ( -1, prop_val.clone(), vec![], vec![] )); - return false; - } - let is_unmatched_step_attribution = breakdown_step.map( |breakdown_step| step == breakdown_step - 1 ).unwrap_or(false) && *prop_val != event.breakdown; - if !is_unmatched_step_attribution { - entered_timestamp[step] = EnteredTimestamp { - timestamp: entered_timestamp[step - 1].timestamp, - timings: { - let mut timings = entered_timestamp[step - 1].timings.clone(); - timings.push(event.timestamp); - timings - }, - uuids: { - let mut uuids = entered_timestamp[step - 1].uuids.clone(); - uuids.push(event.uuid); - uuids - }, - }; - if event_uuids[step - 1].len() < MAX_REPLAY_EVENTS - 1 { - event_uuids[step - 1].push(event.uuid); + if in_match_window && !already_reached_this_step { + if exclusion { + self.results.push(Result(-1, prop_val.clone(), vec![], vec![])); + return false; + } + let is_unmatched_step_attribution = self.breakdown_step.map(|breakdown_step| step == breakdown_step - 1).unwrap_or(false) && *prop_val != event.breakdown; + let already_used_event = processing_multiple_events && vars.entered_timestamp[step-1].uuids.contains(&event.uuid); + if !is_unmatched_step_attribution && !already_used_event { + vars.entered_timestamp[step] = EnteredTimestamp { + timestamp: vars.entered_timestamp[step - 1].timestamp, + timings: { + let mut timings = vars.entered_timestamp[step - 1].timings.clone(); + timings.push(event.timestamp); + timings + }, + uuids: { + let mut uuids = vars.entered_timestamp[step - 1].uuids.clone(); + uuids.push(event.uuid); + uuids + }, + }; + if vars.event_uuids[step - 1].len() < MAX_REPLAY_EVENTS - 1 { + vars.event_uuids[step - 1].push(event.uuid); + } + if step > vars.max_step.0 { + vars.max_step = (step, vars.entered_timestamp[step].clone()); + } } - } - if step > max_step.0 { - *max_step = (step, entered_timestamp[step].clone()); } } - } - if funnel_order_type == "strict" { - for i in 1..entered_timestamp.len() { - if !event.steps.contains(&(i as i32)) { - entered_timestamp[i] = EnteredTimestamp { - timestamp: 0.0, - timings: vec![], - uuids: vec![], - }; + // If a strict funnel, clear all of the steps that we didn't match to + // If we are processing multiple events, skip this step, because ordering makes it complicated + if !processing_multiple_events && args.funnel_order_type == "strict" { + for i in 1..vars.entered_timestamp.len() { + if !event.steps.contains(&(i as i8)) { + vars.entered_timestamp[i] = DEFAULT_ENTERED_TIMESTAMP; + } } } - } - true + true + } } fn main() { @@ -240,14 +252,11 @@ fn main() { for line in stdin.lock().lines() { if let Ok(line) = line { let args = parse_args(&line); - let result = calculate_funnel_from_user_events( - args.num_steps, - args.conversion_window_limit, - &args.breakdown_attribution_type, - &args.funnel_order_type, - args.prop_vals, - args.value - ); + let mut aggregate_funnel_row = AggregateFunnelRow { + results: Vec::with_capacity(args.prop_vals.len()), + breakdown_step: Option::None, + }; + let result = aggregate_funnel_row.calculate_funnel_from_user_events(&args); let output = json!({ "result": result }); writeln!(stdout, "{}", output).unwrap(); stdout.flush().unwrap(); diff --git a/posthog/hogql_queries/insights/funnels/funnel_udf.py b/posthog/hogql_queries/insights/funnels/funnel_udf.py index b2568acc74538..a49cbd09d4984 100644 --- a/posthog/hogql_queries/insights/funnels/funnel_udf.py +++ b/posthog/hogql_queries/insights/funnels/funnel_udf.py @@ -37,6 +37,18 @@ def conversion_window_limit(self) -> int: self.context.funnelWindowInterval * DATERANGE_MAP[self.context.funnelWindowIntervalUnit].total_seconds() ) + # This is used to reduce the number of events we look at in strict funnels + # We remove a non-matching event if there was already one before it (that don't have the same timestamp) + def _array_filter(self): + if self.context.funnelsFilter.funnelOrderType == "strict": + return f""" + arrayFilter( + (x, i) -> not (isNotNull(events_array[i-1]) and empty(x.4) and empty(events_array[i-1].4) and x.1 > events_array[i-1].1), + events_array, + arrayEnumerate(events_array)) + """ + return "events_array" + # This is the function that calls the UDF # This is used by both the query itself and the actors query def _inner_aggregation_query(self): @@ -93,13 +105,14 @@ def matched_event_arrays_selects(): inner_select = parse_select( f""" SELECT + arraySort(t -> t.1, groupArray(tuple(toFloat(timestamp), uuid, {prop_selector}, arrayFilter((x) -> x != 0, [{steps}{exclusions}])))) as events_array, arrayJoin({fn}( {self.context.max_steps}, {self.conversion_window_limit()}, '{breakdown_attribution_string}', '{self.context.funnelsFilter.funnelOrderType}', {prop_vals}, - arraySort(t -> t.1, groupArray(tuple(toFloat(timestamp), uuid, {prop_selector}, arrayFilter((x) -> x != 0, [{steps}{exclusions}])))) + {self._array_filter()} )) as af_tuple, af_tuple.1 as step_reached, af_tuple.1 + 1 as steps, -- Backward compatibility diff --git a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_actors_udf.ambr b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_actors_udf.ambr index c55def5ee7da4..a53fbc8987b1e 100644 --- a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_actors_udf.ambr +++ b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_actors_udf.ambr @@ -25,7 +25,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -76,7 +77,7 @@ JOIN (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, '$pageview'), 1, 0) AS step_0, if(equals(e.event, 'insight analyzed'), 1, 0) AS step_1 FROM events AS e @@ -147,7 +148,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -199,7 +201,7 @@ JOIN (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[3][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, '$pageview'), 1, 0) AS step_0, if(equals(e.event, 'insight analyzed'), 1, 0) AS step_1, if(equals(e.event, 'insight updated'), 1, 0) AS step_2 FROM events AS e @@ -262,7 +264,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -316,7 +319,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, '$pageview'), 1, 0) AS step_0, if(equals(e.event, 'insight analyzed'), 1, 0) AS step_1 FROM events AS e @@ -385,7 +388,8 @@ rowNumberInBlock() AS row_number, breakdown AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -441,7 +445,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -495,7 +500,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, '$pageview'), 1, 0) AS step_0, if(equals(e.event, 'insight analyzed'), 1, 0) AS step_1 FROM events AS e @@ -567,7 +572,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -621,7 +627,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, '$pageview'), 1, 0) AS step_0, if(equals(e.event, 'insight analyzed'), 1, 0) AS step_1 FROM events AS e diff --git a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_udf.ambr b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_udf.ambr index e1ddaaabb7ece..ed6e57f0db7b0 100644 --- a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_udf.ambr +++ b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_correlation_udf.ambr @@ -21,7 +21,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -68,7 +69,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -126,7 +128,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -186,7 +189,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -244,7 +248,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -298,7 +303,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -370,7 +375,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -424,7 +430,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -496,7 +502,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -550,7 +557,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -622,7 +629,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -676,7 +684,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -748,7 +756,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -808,7 +817,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -866,7 +876,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -920,7 +931,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -992,7 +1003,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1046,7 +1058,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -1118,7 +1130,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1172,7 +1185,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -1244,7 +1257,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1298,7 +1312,7 @@ FROM (SELECT aggregation_target AS actor_id, matched_events_array[plus(step_reached, 1)] AS matching_events, (matched_events_array[1][1]).1 AS timestamp, nullIf((matched_events_array[2][1]).1, 0) AS final_timestamp, (matched_events_array[1][1]).1 AS first_timestamp, steps AS steps, final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, af_tuple.3 AS timings, af_tuple.4 AS matched_event_uuids_array_array, groupArray(tuple(timestamp, uuid, `$session_id`, `$window_id`)) AS user_events, mapFromArrays(arrayMap(x -> x.2, user_events), user_events) AS user_events_map, arrayMap(matched_event_uuids_array -> arrayMap(event_uuid -> user_events_map[event_uuid], arrayDistinct(matched_event_uuids_array)), matched_event_uuids_array_array) AS matched_events_array, aggregation_target AS aggregation_target FROM (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, e.uuid AS uuid, e.`$session_id` AS `$session_id`, e.`$window_id` AS `$window_id`, if(equals(e.event, 'user signed up'), 1, 0) AS step_0, if(equals(e.event, 'paid'), 1, 0) AS step_1 FROM events AS e @@ -1372,7 +1386,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1413,7 +1428,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1466,7 +1482,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1507,7 +1524,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1555,7 +1573,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1595,7 +1614,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1645,7 +1665,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1707,7 +1728,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1769,7 +1791,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1831,7 +1854,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1891,7 +1915,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1939,7 +1964,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1997,7 +2023,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2059,7 +2086,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2119,7 +2147,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2159,7 +2188,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2209,7 +2239,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2271,7 +2302,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2333,7 +2365,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2395,7 +2428,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2455,7 +2489,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2503,7 +2538,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2561,7 +2597,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2623,7 +2660,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2686,7 +2724,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2738,7 +2777,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2787,7 +2827,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2856,7 +2897,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2925,7 +2967,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2994,7 +3037,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3065,7 +3109,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3117,7 +3162,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3168,7 +3214,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3220,7 +3267,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3269,7 +3317,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3338,7 +3387,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3407,7 +3457,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3476,7 +3527,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3547,7 +3599,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3599,7 +3652,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3650,7 +3704,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3702,7 +3757,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3751,7 +3807,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3820,7 +3877,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3889,7 +3947,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -3958,7 +4017,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4029,7 +4089,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4081,7 +4142,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4132,7 +4194,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4184,7 +4247,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4233,7 +4297,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4302,7 +4367,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4371,7 +4437,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4440,7 +4507,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4511,7 +4579,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4563,7 +4632,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4614,7 +4684,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4666,7 +4737,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4715,7 +4787,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4784,7 +4857,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4853,7 +4927,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4922,7 +4997,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -4993,7 +5069,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -5045,7 +5122,8 @@ final_timestamp, first_timestamp FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, diff --git a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_persons_udf.ambr b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_persons_udf.ambr index 90e66603e5963..42c71c0354c39 100644 --- a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_persons_udf.ambr +++ b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_persons_udf.ambr @@ -8,7 +8,8 @@ (SELECT aggregation_target AS actor_id, matched_events_array[1] AS matching_events FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -81,7 +82,8 @@ (SELECT aggregation_target AS actor_id, matched_events_array[2] AS matching_events FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -154,7 +156,8 @@ (SELECT aggregation_target AS actor_id, matched_events_array[2] AS matching_events FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, diff --git a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict.ambr b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict.ambr index 5133b5e227437..fbc88b8a5d2ec 100644 --- a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict.ambr +++ b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict.ambr @@ -1,1131 +1,4 @@ # serializer version: 1 -# name: BaseTestFunnelStrictStepsBreakdown.test_funnel_breakdown_correct_breakdown_props_are_chosen - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, ['Other']) AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_2 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestFunnelStrictStepsBreakdown.test_funnel_breakdown_correct_breakdown_props_are_chosen_for_step - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, ['Other']) AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_2 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop_basic AS prop_basic, - prop_0 AS prop_0, - prop_1 AS prop_1, - prop, - prop_vals AS prop_vals, - prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - if(ifNull(equals(step_0, 1), 0), prop_basic, []) AS prop_0, - if(ifNull(equals(step_1, 1), 0), prop_basic, []) AS prop_1, - prop_1 AS prop, - groupUniqArray(prop) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))) ARRAY - JOIN prop_vals AS prop - WHERE ifNull(notEquals(prop, []), isNotNull(prop) - or isNotNull([])))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestFunnelStrictStepsBreakdown.test_funnel_step_multiple_breakdown_snapshot - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, ['Other']) AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_2 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['', '']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'buy'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), ''), ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_aggregate_by_groups_breakdown_group_person_on_events - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - sum(step_3) AS step_3, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(avgArrayOrNull(step_2_conversion_time_array) AS inter_2_conversion), NULL, inter_2_conversion) AS step_2_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(isNaN(medianArrayOrNull(step_2_conversion_time_array) AS inter_2_median), NULL, inter_2_median) AS step_2_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, 'Other') AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - countIf(ifNull(equals(steps, 3), 0)) AS step_3, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - groupArray(step_2_conversion_time) AS step_2_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_3 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time, - min(step_2_conversion_time) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - step_2_conversion_time AS step_2_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0), ifNull(lessOrEquals(latest_1, latest_2), 0), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 3, if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1)) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time, - if(and(isNotNull(latest_2), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_1, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_1, latest_2), NULL) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - step_2 AS step_2, - min(latest_2) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) AS latest_2, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - e.`$group_0` AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_aggregate_by_groups_breakdown_group_person_on_events_poe_v2 - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - sum(step_3) AS step_3, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(avgArrayOrNull(step_2_conversion_time_array) AS inter_2_conversion), NULL, inter_2_conversion) AS step_2_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(isNaN(medianArrayOrNull(step_2_conversion_time_array) AS inter_2_median), NULL, inter_2_median) AS step_2_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, 'Other') AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - countIf(ifNull(equals(steps, 3), 0)) AS step_3, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - groupArray(step_2_conversion_time) AS step_2_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_3 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time, - min(step_2_conversion_time) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - step_2_conversion_time AS step_2_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0), ifNull(lessOrEquals(latest_1, latest_2), 0), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 3, if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1)) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time, - if(and(isNotNull(latest_2), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_1, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_1, latest_2), NULL) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - step_2 AS step_2, - min(latest_2) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) AS latest_2, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - e.`$group_0` AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - sum(step_3) AS step_3, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(avgArrayOrNull(step_2_conversion_time_array) AS inter_2_conversion), NULL, inter_2_conversion) AS step_2_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(isNaN(medianArrayOrNull(step_2_conversion_time_array) AS inter_2_median), NULL, inter_2_median) AS step_2_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, 'Other') AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - countIf(ifNull(equals(steps, 3), 0)) AS step_3, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - groupArray(step_2_conversion_time) AS step_2_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_3 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time, - min(step_2_conversion_time) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - step_2_conversion_time AS step_2_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0), ifNull(lessOrEquals(latest_1, latest_2), 0), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 3, if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1)) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time, - if(and(isNotNull(latest_2), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_1, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_1, latest_2), NULL) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - step_2 AS step_2, - min(latest_2) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) AS latest_2, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.1 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.2 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [1, 2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('finance')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.3 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.4 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('finance')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.5 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.6 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [1, 2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('technology')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.7 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.8 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('technology')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- # name: TestFunnelStrictStepsBreakdown.test_funnel_breakdown_correct_breakdown_props_are_chosen ''' SELECT sum(step_1) AS step_1, diff --git a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_persons_udf.ambr b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_persons_udf.ambr index 1dd99c28faa3a..a42f48b846b8f 100644 --- a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_persons_udf.ambr +++ b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_persons_udf.ambr @@ -8,7 +8,8 @@ (SELECT aggregation_target AS actor_id, matched_events_array[1] AS matching_events FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -81,7 +82,8 @@ (SELECT aggregation_target AS actor_id, matched_events_array[2] AS matching_events FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -154,7 +156,8 @@ (SELECT aggregation_target AS actor_id, matched_events_array[2] AS matching_events FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'strict', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'strict', [[]], arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, diff --git a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_udf.ambr b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_udf.ambr index ca74dfd1d449e..4b930a59783d8 100644 --- a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_udf.ambr +++ b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_strict_udf.ambr @@ -1,403 +1,62 @@ # serializer version: 1 -# name: BaseTestFunnelStrictStepsBreakdown.test_funnel_breakdown_correct_breakdown_props_are_chosen - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, ['Other']) AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_2 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestFunnelStrictStepsBreakdown.test_funnel_breakdown_correct_breakdown_props_are_chosen_for_step - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, ['Other']) AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_2 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop_basic AS prop_basic, - prop_0 AS prop_0, - prop_1 AS prop_1, - prop, - prop_vals AS prop_vals, - prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - if(ifNull(equals(step_0, 1), 0), prop_basic, []) AS prop_0, - if(ifNull(equals(step_1, 1), 0), prop_basic, []) AS prop_1, - prop_1 AS prop, - groupUniqArray(prop) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))) ARRAY - JOIN prop_vals AS prop - WHERE ifNull(notEquals(prop, []), isNotNull(prop) - or isNotNull([])))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestFunnelStrictStepsBreakdown.test_funnel_step_multiple_breakdown_snapshot - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, ['Other']) AS final_prop - FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_2 DESC) AS row_number - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - prop AS prop - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['', '']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'buy'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), ''), ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_aggregate_by_groups_breakdown_group_person_on_events +# name: TestFunnelStrictStepsBreakdownUDF.test_funnel_breakdown_correct_breakdown_props_are_chosen ''' SELECT sum(step_1) AS step_1, sum(step_2) AS step_2, - sum(step_3) AS step_3, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(avgArrayOrNull(step_2_conversion_time_array) AS inter_2_conversion), NULL, inter_2_conversion) AS step_2_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(isNaN(medianArrayOrNull(step_2_conversion_time_array) AS inter_2_median), NULL, inter_2_median) AS step_2_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, 'Other') AS final_prop + arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, + arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, + groupArray(row_number) AS row_number, + final_prop AS final_prop FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - countIf(ifNull(equals(steps, 3), 0)) AS step_3, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - groupArray(step_2_conversion_time) AS step_2_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_3 DESC) AS row_number + (SELECT countIf(ifNull(ifNull(equals(step_reached, 0), 0), 0)) AS step_1, + countIf(ifNull(ifNull(equals(step_reached, 1), 0), 0)) AS step_2, + groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, + rowNumberInBlock() AS row_number, + if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time, - min(step_2_conversion_time) AS step_2_conversion_time + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, + af_tuple.1 AS step_reached, + plus(af_tuple.1, 1) AS steps, + af_tuple.2 AS breakdown, + af_tuple.3 AS timings, + aggregation_target AS aggregation_target FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - step_2_conversion_time AS step_2_conversion_time, - prop AS prop + (SELECT timestamp AS timestamp, + aggregation_target AS aggregation_target, + uuid AS uuid, + `$session_id` AS `$session_id`, + `$window_id` AS `$window_id`, + step_0 AS step_0, + step_1 AS step_1, + prop_basic AS prop_basic, + prop, + prop_vals AS prop_vals, + if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['']) AS prop FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0), ifNull(lessOrEquals(latest_1, latest_2), 0), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 3, if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1)) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time, - if(and(isNotNull(latest_2), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_1, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_1, latest_2), NULL) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - step_2 AS step_2, - min(latest_2) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) AS latest_2, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - e.`$group_0` AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) + (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, + if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, + e.uuid AS uuid, + e.`$session_id` AS `$session_id`, + e.`$window_id` AS `$window_id`, + if(equals(e.event, 'sign up'), 1, 0) AS step_0, + if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, + [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, + prop_basic AS prop, + argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals + FROM events AS e + LEFT OUTER JOIN + (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, + person_distinct_id_overrides.distinct_id AS distinct_id + FROM person_distinct_id_overrides + WHERE equals(person_distinct_id_overrides.team_id, 2) + GROUP BY person_distinct_id_overrides.distinct_id + HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) + WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) + GROUP BY aggregation_target + HAVING ifNull(greaterOrEquals(step_reached, 0), 0)) + GROUP BY breakdown + ORDER BY step_2 DESC, step_1 DESC) GROUP BY final_prop LIMIT 100 SETTINGS readonly=2, max_execution_time=60, @@ -409,110 +68,71 @@ allow_experimental_analyzer=1 ''' # --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_aggregate_by_groups_breakdown_group_person_on_events_poe_v2 +# name: TestFunnelStrictStepsBreakdownUDF.test_funnel_breakdown_correct_breakdown_props_are_chosen_for_step ''' SELECT sum(step_1) AS step_1, sum(step_2) AS step_2, - sum(step_3) AS step_3, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(avgArrayOrNull(step_2_conversion_time_array) AS inter_2_conversion), NULL, inter_2_conversion) AS step_2_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(isNaN(medianArrayOrNull(step_2_conversion_time_array) AS inter_2_median), NULL, inter_2_median) AS step_2_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, 'Other') AS final_prop + arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, + arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, + groupArray(row_number) AS row_number, + final_prop AS final_prop FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - countIf(ifNull(equals(steps, 3), 0)) AS step_3, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - groupArray(step_2_conversion_time) AS step_2_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_3 DESC) AS row_number + (SELECT countIf(ifNull(ifNull(equals(step_reached, 0), 0), 0)) AS step_1, + countIf(ifNull(ifNull(equals(step_reached, 1), 0), 0)) AS step_2, + groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, + rowNumberInBlock() AS row_number, + if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time, - min(step_2_conversion_time) AS step_2_conversion_time + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'step_1', 'strict', groupUniqArray(prop), arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, + af_tuple.1 AS step_reached, + plus(af_tuple.1, 1) AS steps, + af_tuple.2 AS breakdown, + af_tuple.3 AS timings, + aggregation_target AS aggregation_target FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - step_2_conversion_time AS step_2_conversion_time, - prop AS prop + (SELECT timestamp AS timestamp, + aggregation_target AS aggregation_target, + uuid AS uuid, + `$session_id` AS `$session_id`, + `$window_id` AS `$window_id`, + step_0 AS step_0, + step_1 AS step_1, + prop_basic AS prop_basic, + prop_0 AS prop_0, + prop_1 AS prop_1, + prop, + prop_vals AS prop_vals, + prop FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0), ifNull(lessOrEquals(latest_1, latest_2), 0), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 3, if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1)) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time, - if(and(isNotNull(latest_2), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_1, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_1, latest_2), NULL) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - step_2 AS step_2, - min(latest_2) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) AS latest_2, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - e.`$group_0` AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) + (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, + if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, + e.uuid AS uuid, + e.`$session_id` AS `$session_id`, + e.`$window_id` AS `$window_id`, + if(equals(e.event, 'sign up'), 1, 0) AS step_0, + if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, + [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, + if(ifNull(equals(step_0, 1), 0), prop_basic, []) AS prop_0, + if(ifNull(equals(step_1, 1), 0), prop_basic, []) AS prop_1, + prop_1 AS prop, + groupUniqArray(prop) OVER (PARTITION BY aggregation_target) AS prop_vals + FROM events AS e + LEFT OUTER JOIN + (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, + person_distinct_id_overrides.distinct_id AS distinct_id + FROM person_distinct_id_overrides + WHERE equals(person_distinct_id_overrides.team_id, 2) + GROUP BY person_distinct_id_overrides.distinct_id + HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) + WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))) ARRAY + JOIN prop_vals AS prop + WHERE ifNull(notEquals(prop, []), isNotNull(prop) + or isNotNull([]))) + GROUP BY aggregation_target + HAVING ifNull(greaterOrEquals(step_reached, 0), 0)) + GROUP BY breakdown + ORDER BY step_2 DESC, step_1 DESC) GROUP BY final_prop LIMIT 100 SETTINGS readonly=2, max_execution_time=60, @@ -524,117 +144,64 @@ allow_experimental_analyzer=1 ''' # --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group +# name: TestFunnelStrictStepsBreakdownUDF.test_funnel_step_multiple_breakdown_snapshot ''' SELECT sum(step_1) AS step_1, sum(step_2) AS step_2, - sum(step_3) AS step_3, - if(isNaN(avgArrayOrNull(step_1_conversion_time_array) AS inter_1_conversion), NULL, inter_1_conversion) AS step_1_average_conversion_time, - if(isNaN(avgArrayOrNull(step_2_conversion_time_array) AS inter_2_conversion), NULL, inter_2_conversion) AS step_2_average_conversion_time, - if(isNaN(medianArrayOrNull(step_1_conversion_time_array) AS inter_1_median), NULL, inter_1_median) AS step_1_median_conversion_time, - if(isNaN(medianArrayOrNull(step_2_conversion_time_array) AS inter_2_median), NULL, inter_2_median) AS step_2_median_conversion_time, - if(ifNull(less(row_number, 26), 0), prop, 'Other') AS final_prop + arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, + arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, + groupArray(row_number) AS row_number, + final_prop AS final_prop FROM - (SELECT countIf(ifNull(equals(steps, 1), 0)) AS step_1, - countIf(ifNull(equals(steps, 2), 0)) AS step_2, - countIf(ifNull(equals(steps, 3), 0)) AS step_3, - groupArray(step_1_conversion_time) AS step_1_conversion_time_array, - groupArray(step_2_conversion_time) AS step_2_conversion_time_array, - prop AS prop, - row_number() OVER ( - ORDER BY step_3 DESC) AS row_number + (SELECT countIf(ifNull(ifNull(equals(step_reached, 0), 0), 0)) AS step_1, + countIf(ifNull(ifNull(equals(step_reached, 1), 0), 0)) AS step_2, + groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, + rowNumberInBlock() AS row_number, + if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - prop AS prop, - min(step_1_conversion_time) AS step_1_conversion_time, - min(step_2_conversion_time) AS step_2_conversion_time + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, + af_tuple.1 AS step_reached, + plus(af_tuple.1, 1) AS steps, + af_tuple.2 AS breakdown, + af_tuple.3 AS timings, + aggregation_target AS aggregation_target FROM - (SELECT aggregation_target AS aggregation_target, - steps AS steps, - prop AS prop, - max(steps) OVER (PARTITION BY aggregation_target, - prop) AS max_steps, - step_1_conversion_time AS step_1_conversion_time, - step_2_conversion_time AS step_2_conversion_time, - prop AS prop + (SELECT timestamp AS timestamp, + aggregation_target AS aggregation_target, + uuid AS uuid, + `$session_id` AS `$session_id`, + `$window_id` AS `$window_id`, + step_0 AS step_0, + step_1 AS step_1, + prop_basic AS prop_basic, + prop, + prop_vals AS prop_vals, + if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['', '']) AS prop FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop AS prop, - if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0), ifNull(lessOrEquals(latest_1, latest_2), 0), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 3, if(and(ifNull(lessOrEquals(latest_0, latest_1), 0), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), 2, 1)) AS steps, - if(and(isNotNull(latest_1), ifNull(lessOrEquals(latest_1, plus(toTimeZone(latest_0, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_0, latest_1), NULL) AS step_1_conversion_time, - if(and(isNotNull(latest_2), ifNull(lessOrEquals(latest_2, plus(toTimeZone(latest_1, 'UTC'), toIntervalDay(14))), 0)), dateDiff('second', latest_1, latest_2), NULL) AS step_2_conversion_time - FROM - (SELECT aggregation_target AS aggregation_target, - timestamp AS timestamp, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - min(latest_1) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) AS latest_1, - step_2 AS step_2, - min(latest_2) OVER (PARTITION BY aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) AS latest_2, - prop AS prop - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))))) - WHERE ifNull(equals(step_0, 1), 0))) - GROUP BY aggregation_target, - steps, - prop - HAVING ifNull(equals(steps, max(max_steps)), isNull(steps) - and isNull(max(max_steps)))) - GROUP BY prop) + (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, + if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, + e.uuid AS uuid, + e.`$session_id` AS `$session_id`, + e.`$window_id` AS `$window_id`, + if(equals(e.event, 'sign up'), 1, 0) AS step_0, + if(equals(e.event, 'buy'), 1, 0) AS step_1, + [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), ''), ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, + prop_basic AS prop, + argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals + FROM events AS e + LEFT OUTER JOIN + (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, + person_distinct_id_overrides.distinct_id AS distinct_id + FROM person_distinct_id_overrides + WHERE equals(person_distinct_id_overrides.team_id, 2) + GROUP BY person_distinct_id_overrides.distinct_id + HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) + WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) + GROUP BY aggregation_target + HAVING ifNull(greaterOrEquals(step_reached, 0), 0)) + GROUP BY breakdown + ORDER BY step_2 DESC, step_1 DESC) GROUP BY final_prop LIMIT 100 SETTINGS readonly=2, max_execution_time=60, @@ -646,1635 +213,6 @@ allow_experimental_analyzer=1 ''' # --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.1 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.2 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [1, 2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('finance')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.3 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.4 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('finance')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.5 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.6 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [1, 2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('technology')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.7 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: BaseTestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.8 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('technology')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: TestFunnelStrictStepsBreakdown.test_funnel_breakdown_correct_breakdown_props_are_chosen - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(af, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(af, 1), 0), 0)) AS step_2, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, - af_tuple.1 AS af, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - exclusion_0 AS exclusion_0, - exclusion_1 AS exclusion_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - 0 AS exclusion_0, - 0 AS exclusion_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(af, 0), 0)) - GROUP BY breakdown - ORDER BY step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestFunnelStrictStepsBreakdown.test_funnel_breakdown_correct_breakdown_props_are_chosen_for_step - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(af, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(af, 1), 0), 0)) AS step_2, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'step_1', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, - af_tuple.1 AS af, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - exclusion_0 AS exclusion_0, - exclusion_1 AS exclusion_1, - prop_basic AS prop_basic, - prop_0 AS prop_0, - prop_1 AS prop_1, - prop, - prop_vals AS prop_vals, - prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - 0 AS exclusion_0, - 0 AS exclusion_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - if(ifNull(equals(step_0, 1), 0), prop_basic, []) AS prop_0, - if(ifNull(equals(step_1, 1), 0), prop_basic, []) AS prop_1, - prop_1 AS prop, - groupUniqArray(prop) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))) ARRAY - JOIN prop_vals AS prop - WHERE ifNull(notEquals(prop, []), isNotNull(prop) - or isNotNull([]))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(af, 0), 0)) - GROUP BY breakdown - ORDER BY step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestFunnelStrictStepsBreakdown.test_funnel_step_multiple_breakdown_snapshot - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(af, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(af, 1), 0), 0)) AS step_2, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, - af_tuple.1 AS af, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - exclusion_0 AS exclusion_0, - exclusion_1 AS exclusion_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['', '']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'buy'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - 0 AS exclusion_0, - 0 AS exclusion_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), ''), ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(af, 0), 0)) - GROUP BY breakdown - ORDER BY step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestFunnelStrictStepsBreakdownUDF.test_funnel_breakdown_correct_breakdown_props_are_chosen - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(step_reached, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(step_reached, 1), 0), 0)) AS step_2, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, - af_tuple.1 AS step_reached, - plus(af_tuple.1, 1) AS steps, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings, - aggregation_target AS aggregation_target - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - uuid AS uuid, - `$session_id` AS `$session_id`, - `$window_id` AS `$window_id`, - step_0 AS step_0, - step_1 AS step_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - e.uuid AS uuid, - e.`$session_id` AS `$session_id`, - e.`$window_id` AS `$window_id`, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(step_reached, 0), 0)) - GROUP BY breakdown - ORDER BY step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestFunnelStrictStepsBreakdownUDF.test_funnel_breakdown_correct_breakdown_props_are_chosen_for_step - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(step_reached, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(step_reached, 1), 0), 0)) AS step_2, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'step_1', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, - af_tuple.1 AS step_reached, - plus(af_tuple.1, 1) AS steps, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings, - aggregation_target AS aggregation_target - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - uuid AS uuid, - `$session_id` AS `$session_id`, - `$window_id` AS `$window_id`, - step_0 AS step_0, - step_1 AS step_1, - prop_basic AS prop_basic, - prop_0 AS prop_0, - prop_1 AS prop_1, - prop, - prop_vals AS prop_vals, - prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - e.uuid AS uuid, - e.`$session_id` AS `$session_id`, - e.`$window_id` AS `$window_id`, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(and(equals(e.event, 'buy'), ifNull(equals(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', ''), 'xyz'), 0)), 1, 0) AS step_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - if(ifNull(equals(step_0, 1), 0), prop_basic, []) AS prop_0, - if(ifNull(equals(step_1, 1), 0), prop_basic, []) AS prop_1, - prop_1 AS prop, - groupUniqArray(prop) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC'))))) ARRAY - JOIN prop_vals AS prop - WHERE ifNull(notEquals(prop, []), isNotNull(prop) - or isNotNull([]))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(step_reached, 0), 0)) - GROUP BY breakdown - ORDER BY step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestFunnelStrictStepsBreakdownUDF.test_funnel_step_multiple_breakdown_snapshot - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(step_reached, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(step_reached, 1), 0), 0)) AS step_2, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, - af_tuple.1 AS step_reached, - plus(af_tuple.1, 1) AS steps, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings, - aggregation_target AS aggregation_target - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - uuid AS uuid, - `$session_id` AS `$session_id`, - `$window_id` AS `$window_id`, - step_0 AS step_0, - step_1 AS step_1, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - if(notEmpty(arrayFilter(x -> notEmpty(x), prop_vals)), prop_vals, ['', '']) AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - e.uuid AS uuid, - e.`$session_id` AS `$session_id`, - e.`$window_id` AS `$window_id`, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(equals(e.event, 'buy'), 1, 0) AS step_1, - [ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$browser'), ''), 'null'), '^"|"$', '')), ''), ifNull(toString(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(e.properties, '$version'), ''), 'null'), '^"|"$', '')), '')] AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, notEmpty(arrayFilter(x -> notEmpty(x), prop))) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(step_reached, 0), 0)) - GROUP BY breakdown - ORDER BY step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_aggregate_by_groups_breakdown_group_person_on_events - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - sum(step_3) AS step_3, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_2_conversion_times)])[1] AS step_2_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_2_conversion_times)])[1] AS step_2_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(af, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(af, 1), 0), 0)) AS step_2, - countIf(ifNull(ifNull(equals(af, 2), 0), 0)) AS step_3, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - groupArrayIf(timings[2], ifNull(greater(timings[2], 0), 0)) AS step_2_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, - af_tuple.1 AS af, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - exclusion_0 AS exclusion_0, - exclusion_1 AS exclusion_1, - exclusion_2 AS exclusion_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - e.`$group_0` AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - 0 AS exclusion_0, - 0 AS exclusion_1, - 0 AS exclusion_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(af, 0), 0)) - GROUP BY breakdown - ORDER BY step_3 DESC, step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_aggregate_by_groups_breakdown_group_person_on_events_poe_v2 - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - sum(step_3) AS step_3, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_2_conversion_times)])[1] AS step_2_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_2_conversion_times)])[1] AS step_2_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(af, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(af, 1), 0), 0)) AS step_2, - countIf(ifNull(ifNull(equals(af, 2), 0), 0)) AS step_3, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - groupArrayIf(timings[2], ifNull(greater(timings[2], 0), 0)) AS step_2_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, - af_tuple.1 AS af, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - exclusion_0 AS exclusion_0, - exclusion_1 AS exclusion_1, - exclusion_2 AS exclusion_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - e.`$group_0` AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - 0 AS exclusion_0, - 0 AS exclusion_1, - 0 AS exclusion_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(af, 0), 0)) - GROUP BY breakdown - ORDER BY step_3 DESC, step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group - ''' - SELECT sum(step_1) AS step_1, - sum(step_2) AS step_2, - sum(step_3) AS step_3, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_1_conversion_times)])[1] AS step_1_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [avgArrayOrNull(step_2_conversion_times)])[1] AS step_2_average_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_1_conversion_times)])[1] AS step_1_median_conversion_time, - arrayMap(x -> if(isNaN(x), NULL, x), [medianArrayOrNull(step_2_conversion_times)])[1] AS step_2_median_conversion_time, - groupArray(row_number) AS row_number, - final_prop AS final_prop - FROM - (SELECT countIf(ifNull(ifNull(equals(af, 0), 0), 0)) AS step_1, - countIf(ifNull(ifNull(equals(af, 1), 0), 0)) AS step_2, - countIf(ifNull(ifNull(equals(af, 2), 0), 0)) AS step_3, - groupArrayIf(timings[1], ifNull(greater(timings[1], 0), 0)) AS step_1_conversion_times, - groupArrayIf(timings[2], ifNull(greater(timings[2], 0), 0)) AS step_2_conversion_times, - rowNumberInBlock() AS row_number, - if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop - FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, - af_tuple.1 AS af, - af_tuple.2 AS breakdown, - af_tuple.3 AS timings - FROM - (SELECT timestamp AS timestamp, - aggregation_target AS aggregation_target, - step_0 AS step_0, - latest_0 AS latest_0, - step_1 AS step_1, - latest_1 AS latest_1, - step_2 AS step_2, - latest_2 AS latest_2, - exclusion_0 AS exclusion_0, - exclusion_1 AS exclusion_1, - exclusion_2 AS exclusion_2, - prop_basic AS prop_basic, - prop, - prop_vals AS prop_vals, - prop_vals AS prop - FROM - (SELECT toTimeZone(e.timestamp, 'UTC') AS timestamp, - if(not(empty(e__override.distinct_id)), e__override.person_id, e.person_id) AS aggregation_target, - if(equals(e.event, 'sign up'), 1, 0) AS step_0, - if(ifNull(equals(step_0, 1), 0), timestamp, NULL) AS latest_0, - if(equals(e.event, 'play movie'), 1, 0) AS step_1, - if(ifNull(equals(step_1, 1), 0), timestamp, NULL) AS latest_1, - if(equals(e.event, 'buy'), 1, 0) AS step_2, - if(ifNull(equals(step_2, 1), 0), timestamp, NULL) AS latest_2, - 0 AS exclusion_0, - 0 AS exclusion_1, - 0 AS exclusion_2, - ifNull(toString(e__group_0.properties___industry), '') AS prop_basic, - prop_basic AS prop, - argMinIf(prop, timestamp, isNotNull(prop)) OVER (PARTITION BY aggregation_target) AS prop_vals - FROM events AS e - LEFT OUTER JOIN - (SELECT argMax(person_distinct_id_overrides.person_id, person_distinct_id_overrides.version) AS person_id, - person_distinct_id_overrides.distinct_id AS distinct_id - FROM person_distinct_id_overrides - WHERE equals(person_distinct_id_overrides.team_id, 2) - GROUP BY person_distinct_id_overrides.distinct_id - HAVING ifNull(equals(argMax(person_distinct_id_overrides.is_deleted, person_distinct_id_overrides.version), 0), 0) SETTINGS optimize_aggregation_in_order=1) AS e__override ON equals(e.distinct_id, e__override.distinct_id) - LEFT JOIN - (SELECT argMax(replaceRegexpAll(nullIf(nullIf(JSONExtractRaw(groups.group_properties, 'industry'), ''), 'null'), '^"|"$', ''), toTimeZone(groups._timestamp, 'UTC')) AS properties___industry, - groups.group_type_index AS index, - groups.group_key AS key - FROM groups - WHERE and(equals(groups.team_id, 2), equals(index, 0)) - GROUP BY groups.group_type_index, - groups.group_key) AS e__group_0 ON equals(e.`$group_0`, e__group_0.key) - WHERE and(equals(e.team_id, 2), and(greaterOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-01 00:00:00.000000', 6, 'UTC')), lessOrEquals(toTimeZone(e.timestamp, 'UTC'), toDateTime64('2020-01-08 23:59:59.999999', 6, 'UTC')))))) - GROUP BY aggregation_target - HAVING ifNull(greaterOrEquals(af, 0), 0)) - GROUP BY breakdown - ORDER BY step_3 DESC, step_2 DESC, step_1 DESC) - GROUP BY final_prop - LIMIT 100 SETTINGS readonly=2, - max_execution_time=60, - allow_experimental_object_type=1, - format_csv_allow_double_quotes=0, - max_ast_elements=4000000, - max_expanded_ast_elements=4000000, - max_bytes_before_external_group_by=23622320128, - allow_experimental_analyzer=1 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.1 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.2 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [1, 2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('finance')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.3 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.4 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('finance')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.5 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.6 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [1, 2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('technology')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.7 - ''' - - SELECT replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS value, - count(*) as count - FROM events e - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND event IN ['buy', 'play movie', 'sign up'] - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - GROUP BY value - ORDER BY count DESC, value DESC - LIMIT 26 - OFFSET 0 - ''' -# --- -# name: TestStrictFunnelGroupBreakdown.test_funnel_breakdown_group.8 - ''' - - SELECT aggregation_target AS actor_id - FROM - (SELECT aggregation_target, - steps, - avg(step_1_conversion_time) step_1_average_conversion_time_inner, - avg(step_2_conversion_time) step_2_average_conversion_time_inner, - median(step_1_conversion_time) step_1_median_conversion_time_inner, - median(step_2_conversion_time) step_2_median_conversion_time_inner, - prop - FROM - (SELECT aggregation_target, - steps, - max(steps) over (PARTITION BY aggregation_target, - prop) as max_steps, - step_1_conversion_time, - step_2_conversion_time, - prop - FROM - (SELECT *, - if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY - AND latest_1 <= latest_2 - AND latest_2 <= latest_0 + INTERVAL 7 DAY, 3, if(latest_0 <= latest_1 - AND latest_1 <= latest_0 + INTERVAL 7 DAY, 2, 1)) AS steps, - if(isNotNull(latest_1) - AND latest_1 <= latest_0 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_0), toDateTime(latest_1)), NULL) step_1_conversion_time, - if(isNotNull(latest_2) - AND latest_2 <= latest_1 + INTERVAL 7 DAY, dateDiff('second', toDateTime(latest_1), toDateTime(latest_2)), NULL) step_2_conversion_time - FROM - (SELECT aggregation_target, timestamp, step_0, - latest_0, - step_1, - min(latest_1) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING) latest_1, - step_2, - min(latest_2) over (PARTITION by aggregation_target, - prop - ORDER BY timestamp DESC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING) latest_2 , - if(has(['technology', 'finance'], prop), prop, 'Other') as prop - FROM - (SELECT *, - prop_vals as prop - FROM - (SELECT e.timestamp as timestamp, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as aggregation_target, - if(notEmpty(pdi.distinct_id), pdi.person_id, e.person_id) as person_id, - if(event = 'sign up', 1, 0) as step_0, - if(step_0 = 1, timestamp, null) as latest_0, - if(event = 'play movie', 1, 0) as step_1, - if(step_1 = 1, timestamp, null) as latest_1, - if(event = 'buy', 1, 0) as step_2, - if(step_2 = 1, timestamp, null) as latest_2, - replaceRegexpAll(JSONExtractRaw(group_properties_0, 'industry'), '^"|"$', '') AS prop_basic, - prop_basic as prop, - argMinIf(prop, timestamp, isNotNull(prop)) over (PARTITION by aggregation_target) as prop_vals - FROM events e - LEFT OUTER JOIN - (SELECT distinct_id, - argMax(person_id, version) as person_id - FROM person_distinct_id2 - WHERE team_id = 2 - AND distinct_id IN - (SELECT distinct_id - FROM events - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') ) - GROUP BY distinct_id - HAVING argMax(is_deleted, version) = 0) AS pdi ON e.distinct_id = pdi.distinct_id - LEFT JOIN - (SELECT group_key, - argMax(group_properties, _timestamp) AS group_properties_0 - FROM groups - WHERE team_id = 2 - AND group_type_index = 0 - GROUP BY group_key) groups_0 ON "$group_0" == groups_0.group_key - WHERE team_id = 2 - AND toTimeZone(timestamp, 'UTC') >= toDateTime('2020-01-01 00:00:00', 'UTC') - AND toTimeZone(timestamp, 'UTC') <= toDateTime('2020-01-08 23:59:59', 'UTC') - AND (1=1) ))) - WHERE step_0 = 1 )) - GROUP BY aggregation_target, - steps, - prop - HAVING steps = max_steps) - WHERE steps IN [2, 3] - AND arrayFlatten(array(prop)) = arrayFlatten(array('technology')) - ORDER BY aggregation_target - LIMIT 100 - OFFSET 0 SETTINGS max_ast_elements=1000000, - max_expanded_ast_elements=1000000 - ''' -# --- # name: TestStrictFunnelGroupBreakdownUDF.test_funnel_aggregate_by_groups_breakdown_group_person_on_events ''' SELECT sum(step_1) AS step_1, @@ -2295,7 +233,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2371,7 +310,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -2447,7 +387,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'strict', groupUniqArray(prop), arrayFilter((x, i) -> not(and(isNotNull(events_array[minus(i, 1)]), empty(x.4), empty((events_array[minus(i, 1)]).4), ifNull(greater(x.1, (events_array[minus(i, 1)]).1), 0))), events_array, arrayEnumerate(events_array)))) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, diff --git a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_udf.ambr b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_udf.ambr index fc27315723526..22e1d6badcb15 100644 --- a/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_udf.ambr +++ b/posthog/hogql_queries/insights/funnels/test/__snapshots__/test_funnel_udf.ambr @@ -19,7 +19,8 @@ rowNumberInBlock() AS row_number, breakdown AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 15, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 15, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -67,7 +68,8 @@ FROM (SELECT aggregation_target AS actor_id FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 15, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 15, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -143,7 +145,8 @@ rowNumberInBlock() AS row_number, breakdown AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -217,7 +220,8 @@ rowNumberInBlock() AS row_number, breakdown AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -278,7 +282,8 @@ rowNumberInBlock() AS row_number, breakdown AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -337,7 +342,8 @@ FROM (SELECT aggregation_target AS actor_id FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -404,7 +410,8 @@ FROM (SELECT aggregation_target AS actor_id FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -471,7 +478,8 @@ FROM (SELECT aggregation_target AS actor_id FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(3, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -553,7 +561,8 @@ rowNumberInBlock() AS row_number, breakdown AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -609,7 +618,8 @@ rowNumberInBlock() AS row_number, breakdown AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, [], arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', [[]], events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -662,7 +672,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -730,7 +741,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'step_1', 'ordered', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'step_1', 'ordered', groupUniqArray(prop), events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -805,7 +817,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, ['Other']) AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1)])))) AS events_array, + arrayJoin(aggregate_funnel_array_v0(2, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -878,7 +891,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -954,7 +968,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, @@ -1030,7 +1045,8 @@ rowNumberInBlock() AS row_number, if(ifNull(less(row_number, 25), 0), breakdown, 'Other') AS final_prop FROM - (SELECT arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))))) AS af_tuple, + (SELECT arraySort(t -> t.1, groupArray(tuple(accurateCastOrNull(timestamp, 'Float64'), uuid, prop, arrayFilter(x -> ifNull(notEquals(x, 0), 1), [multiply(1, step_0), multiply(2, step_1), multiply(3, step_2)])))) AS events_array, + arrayJoin(aggregate_funnel_v0(3, 1209600, 'first_touch', 'ordered', groupUniqArray(prop), events_array)) AS af_tuple, af_tuple.1 AS step_reached, plus(af_tuple.1, 1) AS steps, af_tuple.2 AS breakdown, diff --git a/posthog/hogql_queries/insights/funnels/test/test_funnel.py b/posthog/hogql_queries/insights/funnels/test/test_funnel.py index 536bf006ac226..c5dee72089d1d 100644 --- a/posthog/hogql_queries/insights/funnels/test/test_funnel.py +++ b/posthog/hogql_queries/insights/funnels/test/test_funnel.py @@ -2827,6 +2827,28 @@ def test_funnel_with_denormalised_properties(self): self.assertEqual(results[0]["name"], "user signed up") self.assertEqual(results[0]["count"], 1) + def test_same_event_same_timestamp(self): + _create_person(distinct_ids=["test"], team_id=self.team.pk) + with freeze_time("2024-01-10T12:01:00"): + for _ in range(20): + _create_event(team=self.team, event="step one", distinct_id="test") + with freeze_time("2024-01-11T12:01:00"): + _create_event(team=self.team, event="step two", distinct_id="test") + filters = { + "insight": INSIGHT_FUNNELS, + "funnel_viz_type": "steps", + "date_from": "2024-01-10 00:00:00", + "date_to": "2024-01-12 00:00:00", + "events": [ + {"id": "step one", "order": 0}, + {"id": "step two", "order": 1}, + ], + } + + query = cast(FunnelsQuery, filter_to_query(filters)) + results = FunnelsQueryRunner(query=query, team=self.team).calculate().results + self.assertEqual(results[-1]["count"], 1) + def test_funnel_with_elements_chain(self): person1 = _create_person(distinct_ids=["test"], team_id=self.team.pk) _create_event(team=self.team, event="user signed up", distinct_id="test") @@ -3960,6 +3982,69 @@ def test_first_time_for_user_funnel_with_actions(self): self.assertEqual(result[2]["name"], "watched movie") self.assertEqual(result[2]["count"], 0) + def test_multiple_events_same_timestamp_exclusions(self): + _create_person(distinct_ids=["test"], team_id=self.team.pk) + with freeze_time("2024-01-10T12:00:00"): + _create_event(team=self.team, event="step zero", distinct_id="test") + with freeze_time("2024-01-10T12:01:00"): + for _ in range(30): + _create_event(team=self.team, event="step one", distinct_id="test") + _create_event(team=self.team, event="exclusion", distinct_id="test") + _create_event(team=self.team, event="step two", distinct_id="test") + with freeze_time("2024-01-10T12:02:00"): + _create_event(team=self.team, event="step three", distinct_id="test") + filters = { + "insight": INSIGHT_FUNNELS, + "funnel_viz_type": "steps", + "date_from": "2024-01-10 00:00:00", + "date_to": "2024-01-12 00:00:00", + "events": [ + {"id": "step zero", "order": 0}, + {"id": "step one", "order": 1}, + {"id": "step two", "order": 2}, + {"id": "step three", "order": 3}, + ], + "exclusions": [ + { + "id": "exclusion", + "type": "events", + "funnel_from_step": 0, + "funnel_to_step": 1, + } + ], + } + + query = cast(FunnelsQuery, filter_to_query(filters)) + results = FunnelsQueryRunner(query=query, team=self.team).calculate().results + self.assertEqual(4, len(results)) + self.assertEqual(1, results[-1]["count"]) + + filters = { + "insight": INSIGHT_FUNNELS, + "funnel_viz_type": "steps", + "date_from": "2024-01-10 00:00:00", + "date_to": "2024-01-12 00:00:00", + "events": [ + {"id": "step zero", "order": 0}, + {"id": "step one", "order": 1}, + {"id": "step two", "order": 2}, + {"id": "step three", "order": 3}, + ], + "exclusions": [ + { + "id": "exclusion", + "type": "events", + "funnel_from_step": 1, + "funnel_to_step": 2, + } + ], + } + + query = cast(FunnelsQuery, filter_to_query(filters)) + results = FunnelsQueryRunner(query=query, team=self.team).calculate().results + self.assertEqual(4, len(results)) + self.assertEqual(1, results[-1]["count"]) + def test_first_time_for_user_funnel_filters(self): _create_person( distinct_ids=[f"user_1"], diff --git a/posthog/hogql_queries/insights/funnels/test/test_funnel_strict_udf.py b/posthog/hogql_queries/insights/funnels/test/test_funnel_strict_udf.py index fdde235e5bc7e..934f55babcc0b 100644 --- a/posthog/hogql_queries/insights/funnels/test/test_funnel_strict_udf.py +++ b/posthog/hogql_queries/insights/funnels/test/test_funnel_strict_udf.py @@ -1,11 +1,22 @@ +from typing import cast from unittest.mock import Mock, patch +from freezegun import freeze_time + +from hogql_parser import parse_expr +from posthog.constants import INSIGHT_FUNNELS +from posthog.hogql.constants import HogQLGlobalSettings, MAX_BYTES_BEFORE_EXTERNAL_GROUP_BY +from posthog.hogql.query import execute_hogql_query +from posthog.hogql_queries.insights.funnels.funnels_query_runner import FunnelsQueryRunner from posthog.hogql_queries.insights.funnels.test.test_funnel_strict import ( BaseTestFunnelStrictStepsBreakdown, BaseTestFunnelStrictSteps, BaseTestStrictFunnelGroupBreakdown, BaseTestFunnelStrictStepsConversionTime, ) +from posthog.hogql_queries.legacy_compatibility.filter_to_query import filter_to_query +from posthog.schema import FunnelsQuery +from posthog.test.base import _create_person, _create_event @patch("posthoganalytics.feature_enabled", new=Mock(return_value=True)) @@ -17,6 +28,65 @@ class TestFunnelStrictStepsBreakdownUDF(BaseTestFunnelStrictStepsBreakdown): class TestFunnelStrictStepsUDF(BaseTestFunnelStrictSteps): __test__ = True + def test_redundant_event_filtering_strict_funnel(self): + filters = { + "insight": INSIGHT_FUNNELS, + "funnel_order_type": "strict", + "events": [ + {"id": "$pageview", "order": 1}, + {"id": "insight viewed", "order": 2}, + ], + } + + _create_person( + distinct_ids=["many_other_events"], + team_id=self.team.pk, + properties={"test": "okay"}, + ) + for _ in range(10): + _create_event(team=self.team, event="user signed up", distinct_id="many_other_events") + + query = cast(FunnelsQuery, filter_to_query(filters)) + runner = FunnelsQueryRunner(query=query, team=self.team) + inner_aggregation_query = runner.funnel_class._inner_aggregation_query() + inner_aggregation_query.select.append(parse_expr(f"{runner.funnel_class._array_filter()} AS filtered_array")) + inner_aggregation_query.having = None + response = execute_hogql_query( + query_type="FunnelsQuery", + query=inner_aggregation_query, + team=self.team, + settings=HogQLGlobalSettings( + # Make sure funnel queries never OOM + max_bytes_before_external_group_by=MAX_BYTES_BEFORE_EXTERNAL_GROUP_BY, + allow_experimental_analyzer=True, + ), + ) + # Make sure the events have been condensed down to one + self.assertEqual(1, len(response.results[0][-1])) + + def test_multiple_events_same_timestamp_doesnt_blow_up(self): + _create_person(distinct_ids=["test"], team_id=self.team.pk) + with freeze_time("2024-01-10T12:01:00"): + for _ in range(30): + _create_event(team=self.team, event="step one", distinct_id="test") + _create_event(team=self.team, event="step two", distinct_id="test") + _create_event(team=self.team, event="step three", distinct_id="test") + filters = { + "insight": INSIGHT_FUNNELS, + "funnel_viz_type": "steps", + "date_from": "2024-01-10 00:00:00", + "date_to": "2024-01-12 00:00:00", + "events": [ + {"id": "step one", "order": 0}, + {"id": "step two", "order": 1}, + {"id": "step three", "order": 2}, + ], + } + + query = cast(FunnelsQuery, filter_to_query(filters)) + results = FunnelsQueryRunner(query=query, team=self.team).calculate().results + self.assertEqual(1, results[-1]["count"]) + @patch("posthoganalytics.feature_enabled", new=Mock(return_value=True)) class TestStrictFunnelGroupBreakdownUDF(BaseTestStrictFunnelGroupBreakdown): diff --git a/posthog/hogql_queries/insights/funnels/test/test_funnel_udf.py b/posthog/hogql_queries/insights/funnels/test/test_funnel_udf.py index 5fba037657116..2802aee785edd 100644 --- a/posthog/hogql_queries/insights/funnels/test/test_funnel_udf.py +++ b/posthog/hogql_queries/insights/funnels/test/test_funnel_udf.py @@ -1,6 +1,8 @@ from typing import cast from unittest.mock import patch, Mock +from freezegun import freeze_time + from posthog.constants import FunnelOrderType, INSIGHT_FUNNELS from posthog.hogql_queries.insights.funnels import Funnel from posthog.hogql_queries.insights.funnels.funnels_query_runner import FunnelsQueryRunner @@ -99,6 +101,43 @@ def test_assert_trends_flag_is_off(self): self.assertFalse(results.isUdf) + # Old style funnels fails on this (not sure why) + def test_events_same_timestamp_no_exclusions(self): + _create_person(distinct_ids=["test"], team_id=self.team.pk) + with freeze_time("2024-01-10T12:01:00"): + _create_event(team=self.team, event="step one, ten", distinct_id="test") + _create_event(team=self.team, event="step two, three, seven", distinct_id="test") + _create_event(team=self.team, event="step two, three, seven", distinct_id="test") + _create_event(team=self.team, event="step four, five, eight", distinct_id="test") + _create_event(team=self.team, event="step four, five, eight", distinct_id="test") + _create_event(team=self.team, event="step six, nine", distinct_id="test") + _create_event(team=self.team, event="step two, three, seven", distinct_id="test") + _create_event(team=self.team, event="step four, five, eight", distinct_id="test") + _create_event(team=self.team, event="step six, nine", distinct_id="test") + _create_event(team=self.team, event="step one, ten", distinct_id="test") + filters = { + "insight": INSIGHT_FUNNELS, + "funnel_viz_type": "steps", + "date_from": "2024-01-10 00:00:00", + "date_to": "2024-01-12 00:00:00", + "events": [ + {"id": "step one, ten", "order": 0}, + {"id": "step two, three, seven", "order": 1}, + {"id": "step two, three, seven", "order": 2}, + {"id": "step four, five, eight", "order": 3}, + {"id": "step four, five, eight", "order": 4}, + {"id": "step six, nine", "order": 5}, + {"id": "step two, three, seven", "order": 6}, + {"id": "step four, five, eight", "order": 7}, + {"id": "step six, nine", "order": 8}, + {"id": "step one, ten", "order": 9}, + ], + } + + query = cast(FunnelsQuery, filter_to_query(filters)) + results = FunnelsQueryRunner(query=query, team=self.team).calculate().results + self.assertEqual(1, results[-1]["count"]) + maxDiff = None diff --git a/posthog/user_scripts/aggregate_funnel_aarch64 b/posthog/user_scripts/aggregate_funnel_aarch64 index c86bfbb060d3203d0ce30d60e292202325ea2d03..c6169148db0b6d1e25a98baba5934b638d7765f8 100755 GIT binary patch delta 195186 zcmd444OmrW);E6deNYaNiSj9aaKKd1KKvF^;vBF-Ej6WdrY2B(3T37=o3hOI0sA;c zHJmnuA5EgpOq^2|6{OTOg7&;aInz*1=}gmjz?${}oj^d&R`L9QYk$CI>v^94^Iq?J zU2oS#d*8qNd);fTd#!cf`zZZ2)%pDR^2Cv`Pcr2{yZT`4WB26W37xCJ9G??d!* z6H`zBafnjX{QPc&0<-T%h-bOQsh8JQdyLjb7=5j{n!hnNA*Z(M-%CeK_~6;0i%0(E z={#}6{L^9$Uzn75tJ<9L{}z_SSi}31Eni0p(G_ZYmN2zM&1UtPXu_K{+-Aa;o>ixu zX2KJV&I+ zQ~&)-pi|#KujJX6D?2XLi-_Li2BIFLc(Vi(7#4cSci z!F+WFQ%(4%8lGpuzt`|W6K=UvP#fB0%9wJeif=dJUmjHP>Ja!r6JDgX(;Nbq1XIQ{ zT7`fKpKzDjv9)qANo}{Oc%lj4s+CVO;X5@vL&F7K{Pc$MOckbT9jr9r`?Q9NOt{`b zxe2#TSC?}8U|e8%Gu4c$!5OSc!>djBs1@pr8%+324L=bA?=a!NRVWSqvu+tD^2)6G`xs#;(vV!>>8;zg{a^%;g__Awwv(Sl^T?Lac&FKXojCj7up)bi0g22&ciD*tJhXHfsl9s*w( z0$*>!zt%c-neZ+R-@Zc_JR-e9jj6(UtwMtduhFES*@Q3EmeOa!$L>@q(V^plw(VMf zs%EgAg9j9&;n61Cx=1Z=GvS$9`Kcyc@7Qj_1-;{?ri^T@p+Xa`H?-b_->H>%neg=* zUUg+$UFnnU1{3D3~V2Tb@K8gAV+n4&+d(I}_j;6Gjb^o*sZ3NO^F3TnLx zUw=%+%T4$g&GM-+;Tal!(1;WN-*TTiBA+n>c*Snj!jVk)<68NE2|uIZ(XS1j@l~3E zV>jU$n3y7eg{F)#+Kh`#_{tyE0Xa?hRdp&QTqfM9;pHaWr?p@6TDU6z`Vw^1stp}9 zRk&TV4Vz53&WtBa_&r*A$%I>Ws+11&<7)q`SgTq~ILgzHP-GvWFY2$Ct|S&b3_6Rw-*(XS7tkdCK? z!0jenw9DzCHx*3V~OJzz>GNn}I9*uZy2PQYl0QRyCLsdilf<_|y=1 z-TftQECt3%)om*JZIf8w$Xb&Weh;MP4?ri3j7o)H3HDul== z3V}OK_B|LYmP5ETL;@aVl)rbJo@+&&l=`qy~n;EaBJeF)qY z0PpSsntf4uLnAa6xbAgegPEJ3`>r-wmdOUfvc0&$u!!u%)Jqi`vWzO?ZRm zYOOcnv(~9x?KI&lG`!q|>jSM;aPXfletJejhzch{;2j}w>l+~`WWsMO&VTf!X-3B{~a{pMcP0cO!!k8-t>ma z{?|KhHdT0Bt8l`EKceA26Mjk?kYvL3_B;Aj57__rX&Kv16?7M@+Jyf?D}T_0>*bqF_@i3+ z6N7N9zg|Hal)+euRv}=*br&q!Gngk_AFJDLq6xpK;WiUKlTJ>E|I$nu*Hvn3X2J`! zwOknjUmpT@g}@I2SNLDofV%tLV5*=G=!6N^2h?H0zu%!wfO!W~XsQ;piXOmK`D@oI z*ak8b`O^c9Q%!h}R^D#Hb^m^;3BOwN`wLCDWv6EUuQz3^*W!0h6Rz`0xd|Vqbx<7w zZwP^(5JF^hgutzD4koEn>(~|o&j^7p)$o4%UvH==LqDb@u3%QI!@6$(u_i&KMBMJ8Nr2*#=j*9Wr6 zgqQ14G{TAhb@BVTmf(eH#>!2&-q3avzFljl%7kC?suVqFz!mw^Gnx$#!r#&G<}2d@ldhb> z*gJz~c3f*H(S++AOf}(?-c|>cXTmG4QYT!f;NU-9{H6|5Gm1gTzkAFv{M8_3yMgFWcgGsGd zu!X=gO!z#lgQX$xA`||AR^C}-vj3mdGRjRAYBjvdglE)fOlQJZXn2ze4|`WF@9W1^ z{likUjE?>cQdf-W6|CoF?3@mERr$ zuL*%Sneex@b_Aa(XELg4l*;lzI{uap6NeF)rT!V5IYZ8zci zfND(mHm!UUaE1SM4X9`MLR1Kt@S|Eo(Y1pq(W&8SCOrCjb&2f*xGH~kEhBFrgQSmz zY5RSV2@hOUHRE~{ZtYX?O(y)K57drbCj9dcH2Z(MDdVqNg=!OiuTDA>{x7Zk2@`%n z!zB|QaX=kVKrm&@)iSJggGp-FT&F}6{*+ce&4k~q4K%}q>vzBMG`!#be^F~_rKtjJ zG-WB)oA9X)bq3`ooMo$cjR`NhNyQHi#s%imG8zVFux1T!GU1YjH=A&sxg--lRU1%; z34cMs!G8f$hRd#wlzlLG2{xTmnagUzf6yqGXu|a+NHgL35@Zlg{I82&rnY8zrV4sP zD@`~%s1CHqgwN3MO(wih!(9ekkw1Mz+YK4C=&xuM8cg_Gn$Gu`@K3e!9VT4AHOCGN zrfB_#+WntsQ^wH`)tRN4@CzDlH{mlrQp+zj;qx@S(1fqh@bw2w_Wx!r!)dDE)$r{m z{Dg*An{d5+LkRqYj_dZnUZF$JPP#Gq?NW<+WT+lNL zO&LzDLXipIuHoxL;G0bNQ`$gXCcIk31-9LkaZamHZNig2R%hH00zYBG@7KypCj4Cu zXZ1u;@V_p8-)k9(O2z;_;tv`#n(%xLw}-%2n(%}}YCG$VILV)$;WB0bzg??P6#_qK z!mDo6C}hI_p&6e|Cj1O=MgE&j88uo%Cro&QhWku-Q0qW4;UnKu2h?H0uhH;8y~+Nc zre&}X2XlM2hFeW|zJ^De@InnwG~vY>ZtKUD`7`F!GSd1pK!5!AiiS@$;lI)F3={sI zhTBc}e`i+P5537-)_QdG`vd12kifkwTx<0g$&K|sWIVSXyp%@@Y5RJV8VUc#F_@- zSpV}{h2}vS_=ZE<jr&@lh3Abyw9k{~(x&~aUWh^ySNYpwgGU4S~`AsJLZ4Gys@FHy~4<72D zm@0qHh3ZIs0~w0=F%6eYxYeqLVp&6fLuz>~Z4ZGjHQ@(`2v(v(r8ebv&Sb< z23)1FTKHcdt%3ih8Zt&2I><2KF$Uajz+(+~o&k?D;7bj7 zya8WHIPpI%L4qNp&`=@KfEO9?Q3iay0mp}3{r_z;;G_FdY<31*7-PtA88XHi@Nxq_ z&VX+>;K>HO%7EJpc(nmfQE^rNYYZ8wh6)D__;>@}V8E|2;7ta6f&p(f;70K~@dsgm z8z&hm_zV@&47g;#(+zls0iSHZ0|tDG0cVPU-Bv0Ry_lF|F?(8i8kQZ8yZS9 z;5Qg>n*lfKl{5oB4dqq(<7KKLW4fWC30z-Jlor3QTV04}hVhKxA_ z8LZHNXBhAz1Ae0cUvI$Y8t_d9e4YV!Dmdm(;+JK}a2YDxWWdV}c(wuGZousZyvl$( z40yEx&k<)Wh!GYIV-<^E1z%Iaj?ITWH5S$};-9z3(XF%HHa_|5WSiwH{ZGkj zMIJv#$y3t*LORuuZbLf9kp4H)s}1S1NS7*U;cKOVa{%@l3Y z7#P4%q*D#)NThQN>8p`mZAgzqx|GsFvO0ix0DBDul8`=ZNRLCh)sRj_n$H{Fg;+=6tfA-xdk z97B3B(yI;WWk{Fa=oV;wl>yuZV6UOTPmw-sNZ*Tet0BD_X@2g&0PaUR)sS9?bdDkY zAkwQ1>4%XnohuA9@F;-2h5}C@eb|tG3h7oux)f=C-a!3lkxn(FpGP{!kbVj2)rR!T zNS6u*;8g&74e6aoA2y_4N4nLJ-itJ!H86nRA)RVSdy&pDq~At*wITf;(n6^L_yE9O zL;6Fc4;#{dK)ThCK8!Sf)4%}!7t*PQ^k+!t7}B33z1ooe@>cn(=+Xf|%)k8+-ZH)N z58>QIvE%kc{)Xvd~T1=CoCSa^F8KUK>+xOkX2cFB|*p16^#2p-|S=-WKh)3=p# zOBUu{%UDhN7qi{flWy`z;~AdWo@xuLv8JSmk1e?|Ue7|&{;WNt#XlOdjJ5WphzU#A z3N9Wk3`ue?WX@&2aOPd{8goPI|N(%~K{m*HvE zMll8>bS{H26l<*K472Z7>K@VBdN;*gYHi3;>RZ?~G%ck;-*X%aT9rmy6Pzdzp*Gwy zMjU^~b>YfX_e>Fgl|NQ|r;+;zvy3Cd6v0%2%+%PnQh(_#tp zYEEav9?wI>hCRbr& zw!gw_fucX;>9a65DwVT;B)SO;=F<9w0q5LJVa{cqT=vCY3oFm@C5bbZPYVa-%dNj0 zB^ECq6|Ug+(PG*1^n@R7L|qHjwRrkKHB7wRx+hhcxU_uK#LL&Y+}e_=5T6!s4_G3_ z7cK@Dj}qpRtekThl@W{bZA4U5CZam{sMp2GcaDpTwy@Z43-kIc zG2YMYtiorB^^O&X+&P}_`85@YKhF;osMQLDUlQ-VbBncO-9+a|?muo8Pu-aw?garC zI);cbcWnvFyqU#{>sMYAd1*y?^E>;aq$Z$0}Md+U_N{crLBD8SQiK zBuFW|tbd#r9Bc&jCfd8A20i22H`t7eElu8xa-0*iiWhFzxqY1bF;yXRVNq3FIR zDXe25H|za(?sw8B}nt&`?%VIo(Te&YJ3v(+oT4i-ZoyaOHOczhyJ7LP+V6gJi6wG9++EwWCGpPLP zm(1lY{1=y&ge&4gd>6zt*s7cRitz3ic%p8mw1&AopkDf(X>Q*P))ySjrm&i-P6Rj!lY$mK!r<7ZUqH+g#uM1Z6`*%_o10Jh11=?0b4&^b&b0f zQ)@xBYq6jtq0(@>+y#~{VyxB|W0(I7>JrUkE{J!nnlQA3=w2k2teWD;RA_1YE0y-4 zmAY3+0AjqosNp2`fLvZ$V)3NWq-Y{r8L>bI8{%9>t=ul2S#`tMvBp-$GxrXym2c3> zePZV7i7Cz2lq|P@yos+ylh#vssz!9KPK<<X88D%aF^9!wYVXTzb9AH+!}d-#pN4 z)n4|MAqAGQe1C?#ShlrDBOff_7Q;t8>3n6Bk^__k&2f|Q(^RS zE;Sg&I!hSKr%4{lw0zYCjr~44E!t_9ck|Zv7UV&~nyp!m`s5*Ony)LEZymx$(6V6H zY|in8xuFRsr(l48nT_@s+t7G6nE%L5CPPYxEThK6N7jv>-262^#2ta|n@a7OpYlw$ zhOwDS)4wH-WVQ6pJa+j;v2oq?xy{x_wPh}axsy3CC$(;68}^(Ds?$f$`t--6_lGh4 z`-8!RDr&|y#GUv%ae8r%FbpsWVHBQ6k&Ov&C;-G{R1zJ@hi?cN9$K#cDGSOs81-7^3nFcJ%`S;MyTZcwi# z$7XRiMmd)~ggix44mD^>Xpw*Rs)PRro}^KT=dMJuzluNk`J`npV(paKH-xz##yet} zv6X0n!0{Nra z#e+u<5Tjju!Y1DLP$K`_Eb*y_#__@|@%4vtB66sk3&9Vr75xuQ$x^0VHAIoA$sEg! z+`@a38(Ajf*#1{%X{6C*4CRcZnJC_|ey$LbA8f-1@y>%pE8+*|aQ96hDowS2G0-wC z#?SR0^~Estc=4RNS)RpHGfSuGc4>{Z#L~}{x?!2pU>P)s+c5~m$r<@u#Q};uM=e4J zcApc!-jK>~Oc&4N@tjv2_wYFD-RFV{WEP0?AHJ4bEN*ysQp9{^1hL-d#XS!v^Nw`! z@WT`NZ{88x9-hoEpDo63yfI9vk-ssGixRhNymg4&6HJ&Xe!4L`Lh4Z&_UQTl10}tpJ?VO*8J05w1pYj`J&=JG6K@Sr*{IZz; zafbNMU*5zYn<0*SbYVmlIw6*PUfl5LipdK9tr`$kjV|RWRjGsMaQ;@5_g~YlgFm;kIxXFeJnL|DdvSQvX*hIdKqo%KSy12 zy^nxV8#>pJYRVB?9!p48%q8nA68!QUHr`zVr^JV$E8MgvOcb6N8%FP9#wPoiTD(;3 z4Z9MT*CMF+<_Z_0`Icel#rC&WFQlWjAGDjKzhv8NT&CAagOKLIY#&GKfwIK*-{T#2 zR4lPNhpzuYgB%b8rANveu8%|&gUd#;yE`~cF&bY%0}7R%e8Q%ZtLh1zTvt>IM6L{e z$8?2M?&*p`$fr?2A>0H~jbBO4F{f%lzTO3f9Re~TUHFuutuPi}d1{%=w4Nz~FQcbk!N7)>sXc@X7P1@fpql77k!9?8(|V7NlLi(TU1pPbEC z?GZJCp1Fy8s)M38{ltjmc@E`>V+Hs6p;a(EO? zcF-={u~$>NYxBn*DxZY%q}A*e=a&h1CzG%RJ7giB^ImHLlYcs0Y}=A5DD&#RRaU%( zj$37T3*}G$<5rofEW@9YfBIIX`CGbz3H@CbHmDAqiV!~k^Hd~(=DUKIXheg=s>~z+d!ZX-H;xgeg?qhMRbB^_$v%yL- zfW&eq+->Jff!tXshV@-~z~TuMgn7=b7~<((GSpL#?(bsuO!5}8PdPG40>}d`Km#n2 z)gFYAP9%+IEYmvLUT;md_qjMW6BgKpRp^x~%TZinhJ7z#A*&!ADp^>4!g0GNj%Rh4 zcWnIe9KlluL-Uu`zya#S2ZQ2PbYQcx+FmqH);1chUx3Q~@hqJx|B}iN9VkB+<NF1NU&cV}XNr0OZ4|6}x+twMZ>^aFiJ@7Jv+8}!Rf_1Z*Kn3GbFhMEkLqTZy z{WMxWrXkBUj`g_~Gmo>hWL`;W>AcdB&Ma3XlV@@86H$)>3g-||kYen8fr9y-OAq8K z!G#CU1`{ZcaLPC45f{)v$sUz{g3+sPdYspXcbv(_Mo(ZBxzWtV9QGR{^`U%)NoC!DQ(Y5_Oy>akRQ=*@65V!gjySm|BBrSc=rR(`&K z8zcVCxolz&s3ya3EgGlxLFGDt6@f>TK4CMS?i9zBOypZT#al|oaeo*6&y5#dB{TU? zI>kSfjN^abDgG0W+D`FO$?Dm=Xc;im?j;TcG!9o+b_Nrw(MS&{Kn+p{FKcDWm1msy z>rV02((B_(`>W<)Wl%jlH?XBs{CnwS{^3q>=+=n~i4b?xAtC21iQe->MU>VjIa=u}&WmwXww)Cpd^$dEBwlGTUD{$^TGCiO zOx*GGoUrdGUbSgkm`$vDHY}{CBbZRR-o?j?(a*%D{RMe?2QF<{h{E4>5T)b1lhNw? zSZ1;Mncwg)=821+oz8nZD#uk}srPhiZvQp0VCj$}`5>qCDfhlf;|1T_YSx z`T}Y}4RGxVb6fjPx{^mgkEL742}5jkNIzBrOfipqWW%anWzO|vWWifW)5_m$my;o7 z4wMbXW208Vi%Ngcwa(pbS?SWR1 zEMpYqd;Xgox%+sItm?;6UMHVa*8XqByeVH{8C7=3lGr|W8}y+iu1vCw_4c9-dm76R z;Qf%r+RkGq=tIb66qk7PIG1;zoBFj%7!ra-G zi1t;%VE#sBHp!D!Egs`_V^(mx-u0mYT;OTH2qE3P)wxqyB1f1zfX0$HFoe^`ct=Mt zd53s=*<7J8R&6zzQ(H|(kR2!2u`aY`W$NBCr^M<_p1?LZFSv>T)umv5+7`Q*jW)PAhQP6N8MP@5wF@s_#IPbF*caImtkWU~ z^QAzrR=!ja=!TpRh~g+Ow)Ihd_-r=s=}LGRntzuhyth1#`!^9<-$-BhSj5 z^+Z(%>R|$cw-2)jKFmB9u!Fv08GE#PsQqQw3!SGa*uhv`b&fqV`9*aZZbc*2H>qic z26B-mvd#io^?JVm*^@!lJs3dsrNU!ax=tDl?hV8$X|%OxDDu!xM8q(4365Y1su6NH z`6@q7z*5jQ{vKx7fSE7gtYz6lz+qD?`Prb*3~(cVttJ0lKjq>u@YrB05d*ZlA;|+PWNa0mnKee)mofU$HZmb$ls7 z2JtHlb)-y=@ocsSzCtzZl?aQqEWNWJ19stt6i_?@*MK|yB(OtRexxO$%nBx5hyn$( z*aj=NZD%)l#Sh`NqEUYrQ-W<(s5lz~?6O37iI<9$-8j*^8Wi+*+2sJJVJyE2ZNt0( z`BmzZRyy@{5Q3lRt^WFfU@g2=r7;>UHCPR^?B7_5u7X^-JKXV}!mz;nz$Ea`l?H{Q zfqdNUiVZ-)zm0M|pw4^BGtv72B<+2%^Th-q?@M*DXJDwmx`fuyI>iqgI{Ey*7>8^{ zkkWrQF9!?ow}_W}m+*361>TqQvSf+L3`B8qcLFE-c}p#gB%-(|BdX!)FQ>4Evu&xL z?#wP>8y0d=&(GgsKkoyB5IL-1APMbdb5^0f1ng=}WU<-5;5OK2vHa{a+=iALEctC( z!EVyn!=1|#X%SJ0w6kC7*dhl$;^e(qEdRns4*C2c2i7Hda^De$d@(9ljzama;kmLL zlPePg1UGYX@G(wq--tf}e=Bn2pRW}!l~0<8R`*~On>ZG``(4&ahI+(4el`u0dZS(s zw{zkm_gI0{GvNiCRl%v=9m7$thuh%%7nlEOH9u$fHt4(8d8J&}L!3{Z{MDT35 zyA@Av6?4B(&MG9ZYFDpACV_SSj?Zto;E=QJtn+pVX!d;8X}=23dst^fG@eClVrNM_ z%U^WcOJ5dEw0qvX_N}w)BJFbVi+?)Xe&*k2lewt&qi4##ED}e&e1p&ugP~&Lw3?2C ze7^r~D7u0i5HO-W73nq*;NK59x{{E}ZpVMMH{gFS^>f%*N$C z;lzNV77r9`ANKayb;xT!`uW)@FPu?^Z{rp|k#aTIWFuG)gQt#D4hOi_W9Z3>(%K?e|}LMUNKpC3><+i!1>1D zu5)TG@49;TzYKbXmM0n347YdiAWMIH`5;}}FCU~GbS6tF!@)tO7!`pma8mVUJlXs~s6)t%} zEv)P|=veP?gQ3G@F1FqFmfBf2HW50f_T%ujO6#!XB}0c~grL*EWwD9?222y7K@-gj zL5e;M_&he23zkH|Tdp;*>RCersQmUURzU%ZDvvf|Xu)T+vMIkZl+`+m&Se#TjXJvg zu~I%?ChA~!^Fw=H#B(rH2$Ahu{}RmaJHo*kjrDd%Ib_;9EKZFWV7P)J<1*+I0 z=Yw-dozu#b8mIk0RqGM$ehk<@7s{6QVeBKccUa%&@@XH@_ml!Is;%%nZty-LKueTB z6X(&IbYZR0PzyBeec059m1z%20{Pl4%QJl^8?sJgw&!8#7vn&z>o_mBV<7FLu}@>) zZ$F8@WV`I-w$5umh4%!z+}4Zd5BS60B46O;GA=CO57Gu`kpemrJZHpSFjuMv!|S=N*O{5Zg4_i$fp8utm%%44U;MZPqt}8a?oJ z=HfXIe>5m!9tQIEV7F|+Y^c$~`K)4-VzJ=(`!vOkV!h{wu$kT1MX3&LAce44da#=Y zu$!L8Hc!@UwH6@xAS#+7B=O#^_F!cJ`7jpoiS5Zl9&mV$r&dlbo0As7U!;D65y@V9cBoP)I@nYLq?W*zdc$B#Uj! z+22wCc3V@D#g*9iw>|*2T#odL5`}%K1NJSS@4Fi;UC@}-2KT)6fyOKe+kqcug#8(A zzi$olmN#aR3O`p2n|T|*-?ySMtLq_Nc9gLFB*7)f&wgZLw)G7pyQZ`>FMST^61Pxtt&HwMwG}YRJ+@A_ozq2ut!k%!4}uaKf<+dzo@JNl)A5yq7;Yo3_Arb~k1(!OH3%{TjP`HwMq%rL~S7 zwASaa8TXvZk#|SJ)qEdRgar7&`N55NPU2+zP;9;uD*8cy6O3)h-=5P4pVhmfYs;}6 z|9 z;hdb06M~aa%@;6`F(9Q6!4$pBS6Z3w^TArzrOn5xpH~{k1*%a><8idCF1{95IK3A^ zoxh-+M<#%Q(H2cuiIsq11=9JnHnbno&QIJaVQXR*$T9Xp7LF(OKo;f90+nRUOb%c# zuwp>$JN8i93^1bwD)@~h9%9CBbU+a~V#*klk%rS|e3e^@!4pjDkAkWOr5Hrsiv96r zT+NlO;m&1goOKs&h{!>3a9aY_YR!nu-WgPxlk-a8mKJlev@Zv)E}rjTP2a#f#8+B@ zzc=xoi{~0nUXbGOEK0YhlSk=4#N>1b>SlAQYx_pLa?!2*0AJsBgsYd9paJ68o*6kZ zIeZ?hs~?@_x%oP2JWSC7%m{{AG8fm5(Y{P340ic^+GvRa&l9&Nljj!K?vG;mG(+MQ zIz?J_JF7SkQa4-*)=E(C{tJ#iScG1pQr=F!wrhBfEIXnQFo^*ZCgjL;2dinEy*?t` zUJsR9+yE}kw#2ko!P7j4b+%m`~!LdaAv9l4Ls5DgM_gv3Kh|~5Y3s&s0B;<~Z z!TcuNOp3sMoR2jn5?Jw9F_d*O>`XjomnY+|fJL^qAT*d#QV?MHo41O&$KvmI#O2_6 z#Ztst`@wj&OTn3c0uvohLX?2|Vel^yn&Qw$1Nz$G4+@peKyX&WID4NT%oMPU_S#Yk z$2mbRA`vl{l*}tehVKH`*9$V78_mmfu-1sJP@06@D;xn8-g=aFt!iA0lGfajS$#G* z>-=b}FUtPCD_D09WRpNOB4sic-!5(BWP3Q5-y1`+3^JbI2vZOw!cUgODh+f5XVJRQ zn%oJCzYMb=uGa_I3r6eS>`=LLF5f3*gCQ->Wh4nbxjZ_8aeZWj`^L3t0pgLKY;7{& zg*!)bweqd#6UNC70fG(zZ)UiKKmtAiWOP2s*W#`oEOT5lf*bDI%-2;<$NF(e3PV-n z*46?ni31`kquGu(2g&JITPzMqIM=?3QCv!tNw)%a&s#-x7AO zyWm<%#W}L8gs;2!5Y87sRdPcQ{501?BAcFUueT0CVEsDIT5N|~5ai?A?I&2i3`O6Y z!?~+d?ec*nCP(Op+>mSKkbB}CjWFJ;C)(xo(f0aC4sF2g=MWu4?0thBQjBHYfJ+1U zli>nI3?u1|Xy<@Wf$L*Mb0pmnknAG-k#yU|1-~6FWI%>Vj;ke{!a;ufWf@{gfy@B* zYF}S)W)o(Ss<+G{+DihEXS8ZV>pM`FVQFkQC$C17pCo2aq`mLJQ24}=tgjmOoeekJ zNlF?zg89=SZS=l17!(q8M%#w%Pv-^h3XI0S57y;L!P>WDpaW3W%p-Yzl}B9P8+hbf zl}8p~@Y^&d^B7G(CsX^xD~d>hUnBwUkH4MoBQuAr8{M)=Tfx_MW6$jY zqq-nYMIfOQqzIQGUN?z6OxslJ0!dBgeNjwj}@Jybv>-qF4 zyV6J5`|S90WUd{@Y-`1nZ$t<#aH$piLF&H+6tmfi#^Fpk-}Se^90@_;)7VvAxI85t z0#9PB&^=#}bhbbuT#yF;fgDJKwb%jOg7C&_aZ_ zZ1)>?d9-~_=&VAaR8R(DrEGK@F*i0j8~zd?93&*ZAA*%+_>fRYkXB-H+UV*qU>C&N zf5f2-!VdjaLl`=3MW?j+mGlJjOVFwBkX`;9?XVtfGayU^2qZrSidLtl?;7>3-BSr; zR=V`$G0D!?oriv*ErPZOAGB{dDmG;EwPccYV*q68lWjp(R3doFhk>hD2=O{C8wry=#x7R?hwunl{xuF6_ouRm zfyt1yYlf^HhAc@?9t_!>>F^=3)%#b#rJtC#s}^R%?uo2(HSJfBQ1Urefq}`dd
5}l|vFa?o?)aefPVmNIbWV_(uE7K#
zg|yyutA)r-ZWcPPe>2LX;7c6x&A_`AqZcY<32YHf
zO~_zGias^|Gnju11}$U2#Hh1DdLlmsne@IY(3e1+HC+tW{Q$A)!YMr|s$YT{@tEbP
zVKHT~5X(c@C>!RhN^BcQZ(aLnfTvB8IvE4wd%dD4;r-xo6{mpGX+fx#i^|
ze(X^?>W-VXV<&!HA~uh{&MS#_?}obPw%PYHt|7}gle)_*-^nbZwah$y5P_@
z&i-=y5%%7~Em6DdxXrPr*smPWr;M>ZnEG_&N~u85bw$<|sq!+qV
zBX+qAo8~Lv3EDKRy}?@AG~Gy(+9uzW{1!5%iK=9iJQIfR`>;fk0DG~AHoGE70T}6N
zRHCEmEhs=+xZ=3(18=;+%cGDEzKJzPxW^B_vkRoGzTdh_M!p}oA88roq<47v&gWR|
z40vZWXPw(@qPpBhl#9~?8r83
zIFI5`f6kKm9v8?7tPeI{F=@G;V$5JS6b|fDg(}PWdNPzrj4xs^%5f|-rwngR$;U)#
zAcf;uev`Vp3xl@z*RbX8#uBW)#3|eR+CHv-d*3!i-P$K>TYF08I&AIh#*tXkR$gS-
z-hCiDVNKZH^Ii+g@qdJYf~vH|L-p7C4>{y?^p(vCSG9BGv|?}fm*v!7z^Eaj#T@qh
z?&Uf4kctfrFas%{_IVClxRecfzrmqc?zHdA`|k4iciZL6aMsxkO8f*tE~7u&2qxD3
z*;&})lkj)yd%9i}=j{=QgwS|4C=&dtODyf>5*nxdyoK~U_VSi+hkP6r=z|
z>k6NvZ_%*KVD*eo-!=LxT&%!GQ4PBwZ#ZA~p}J4h=yt(#7=Z6;zE;Mlb(?@xt`f+^
zAjlq|(9SHBlZ$z|?~hFG!#3QrQMC&;&s29`vIQa@pLMZf9d5VI;RM+a8*wK(&I@in
zCPngfKl@R!1+6N5qvG}_cA)6&gmKqbxx-hK6Ks!k!*#|h}Sy5aD>TZ1Dw}7
zZ^3a3ZT$BccIxuNc|C
z_vZrZ!Je-u8VpLe1J2;`33W#M{VatC;`$qYRc%-30w*dxJtgIKc$W;=|Kn1KL_*YJ!22}
z=Wz0`+wGnYC$L&l+wWVF6eoX1fL&k2a|P!W%pMS5;kqdTc;l8@w^`
zFK>(r>3vl{{Qr>mwHdtgf5Q7(i*~N)ePw`I$pbq8g-0Hk)yL@`Sj|N!GF{uIqXFMP
zIYsG_7e-MX%GW)y(TZhGicgiMuTH6+7%4s76B|QH4|+^-{{(|8cH!h;PpkriAy2I7
z@0{+5Y1(&;Hyi_L_?tQqvo|)NeMu#Z@z!Dx{odF&gS|1q?2bKw!_fbm-qt@xV|t9_Jocz~sDn_R*ny&5S&=laz+;Q9QG|WBax9-+=2HnFe@8@9f)PKDiCI{#WkVB&|az!n%K9mi>+k
zG;6=laspQ#b)V%p0vo!|f~$}P@G*!vi(sjdm96AEzl{1
zyTU}Fn~`n=g~-pz!ad5W(!=xilpdMK((nT=rSk)OOB?6Wrn#@QVV*C$B#S;2ZOO&q
zY#A@N&MnCjPR%J%tTcp-YWCv0qAGk>w5N3IJWpv@U|*?o9%<3PwnEJR7gtLsg)O)_
zL$*gNZrr%gn20|XhARoM$XcKZ_-JQV>x{-MQfM6hu^kGiv`oRR8@lp9;iDTGv*eF(
zhacA+df&(q<{Uw_so1nahgTazsb
z?Gy&IK4%XE-~x_CI6!)zmp{bvP$%c%2KHSD*7bnY`Yo#`affENWmJ0=?$eRVb{PhC
z@ql4qEzCX2Ffj7u$>=8ISrU#R7J(>?z2xvbN6C?S3rgk(Xr1gOjq~W<7p;|}1Z#)2
zYRv^Hv0T2ncxE?douVKc){zWqM@i|toRY1m9~Q_han7S)KO)TUHq>+ZMdyWMYayP`
zV@W7x67h_FK>o;R!E!SJ4;56G--DdJ|5_tS@QhOE-0(DkuhaGODwYD0Ijef
zFoJK;KuQWBJRccbMrSC9ak@{fNpky_TjU%#N@s9OO3KZZn?B&`dTz~yQyuHwOX5q1
zJrmU!`-+=c#TgmK7-ks+z9BwauK7ENx2$cA6b0-#bCmryIAL8M11Ii2Zfg=!=v}y
zFGTF3yV~v8`yai7;q0yBOWM#jMliV$WF@XH1Zj!$Pa~{Y2onE%0+SayIGnk1xQ-mr
z&Tyi$9C0nNlB-?B-4j@ZXFlRui%RYeyudeP7IO`mySb?1b_(8
znE7oJ5O*xbEFZ!eZ!AGaI7hWamHCD`_P35~$ZDO4(}8C#`&)SX{+3dH|C_iKaGJj-
zaNDz7?X@WPB6Z^6zIcH;L}!P%tBRx0*%agfcYlT@(1RrhTDi=l0gxX9NaJFTF7f}s
zzeiUHJUe!rJqNN_w|u%Qw(pi>C;8a+LhJz~I>#4VcS%V&wuaa>fTR)+s~*Y+r-`*a
zIP@b5o`*}()0!jx@oy=@XowfxI-@eQECK8T$_9u%C>D7VjPkF^>4KW~AqqGg*tb+3
zDnKa*l@JNc80G1PgwZ}lw+o10`4>2x93bM%6v9=>LL7s980>b4Wel9n+Y!q69F)n?`#>nep`FXy
z6L2P5h6)Wh$fK_yY8U5vcJndqHfTN>GpNAnWEDPE(Xvg0o@~{8x3Dk*%^pu;INj$RZg7c>fRtRKl#q&
z6>?DFR8HU`9QXma80GfCA2L`)@3FA@zo27dBc$&yk`%yirACOWyU_^UF{N_GHuj^9
zF0}CtwL#Z`PwF4F#CtQb)c6QZxsN&<@4iX!_=C92LnHbYKSN>cvQS2%6`&c@@waq5tUeC|moT_WZGznvVU#!FPKT!lzm+rVIr#u=h;lki
zA)Da?AD9gaif$e_wF*Q9nmKh~ZgjU+h61spe&+DZ!3BU7&**KccdGWFP21C4E#14J
zyE#PyT}y?0cOiq67(P_4#aFs=&kA5zYy6%}tfJ1rA5Hk%jh(h4N4*ARLnx^lCp>i!
zy9Tvp6DDHqWqIxod3LiY@DXZ|EsHw+A$
zbNTWJS6rXZ4Wy*y`6_|cz9Q(PRF`K723Yu%x;zK8^5!K`B1NXbbf>a9MO9<)Np}hk
z&nq_U0Z$HYJOhn8pHv&~L*tnwyNc>nziXFHA`}JhnnbbZF~l`N6&c}0QZPX&37OInc;28u3k)ARb|*iBZXPxW&vN_e4BmeVM?*j0?km2ol#UE@GpOA8
zSen<-Io-yl!&ue5Pw;+<&U?|hj4S*U)9l3n{wJL}(dTZhPk}lup&e1{bP`!Wf5d2U
zvy^OxBAjcqAy8I;HD?jn#$i3PVR*uNW+QOt?gApX=}ac`j2IOWVD(IXglxU%CWy2xAUqH+qu=EZs$$fc20MCu$6nx
zW68v(zuAO%bfE9&B*PRsCK=ZE^s8w^5Di*K+ShSZpzP~b;NO_`bqg@v7`Z@u`f`Vy
zW@U9b#Dth#ObSke5sXtKd$APR6Wp$$?29xU1<|J)#>id=ttC1zGuXnA3>Nc0V=#rk
zgaHOKuJ!M5JKVh1|2u0;6i_G$MfQ6PT(Us#hv&$#U}Cz+Ss_mNB;lrak)y0ZP^Wv2
z{1z|@D*Obyf`@7%u%NUA75H09Lk%kQTqB-qoG5GuMAY&D_P;|vn1S16v8-+nIvJoK
z#h>E5Xa1>*D2XMBC`rhb^g-<3=!Pne!ZmmZTCfE7wj#uZM^Xh6dk4He
zx-1}J{B$7TTaH`rP!*&aq?5pDZV*b`?NDrLQXvBF2uRQnu5fn8ndOM@L!~reWsQrO
zY+Otj{Uo>yLZ}NR1ohehNrm*1bdm)3OEjdNR$p5^ar7rCj_W}H^O~gtqcx+TP#)H8
z0;RED^yM2Z2+_xnUnEJyq7;cg`y^MHHh!@wpM@pfyAR73fg1P%)Jw-ybX-Mo@`wqM
zyJohqFDSO@!gV`}##FQ-CQiSKOVJ%{Y)+VPF8n5j6=88_s$K4-B}RJ`Mrg;g3+XGg
zD_pT*$ND1H2VKR6`wpj}GaczHHwp?)_{-J7`n*0p9Cy+yFy{zKr`hW}N*?ZUYDJ
z9iM}HS0Rp(QgwrSHJ&-P8{CcXBi7^hPlBgLWYUKt-{3ptE_|5fy!}!geNxd@kfYqH
zrHCTkJeaa}U54;3zREnp)UPsyhJnvAeTQJ~!>p&z@!AU1PbL(rsEh0W_URxLTs}Ia
znQNe#Ybr|*sI_=yG9
zj1S>(ouvpNANsh8Ha3c~oEU=Zlz4{f$oOFv4M_h|>?I78Ixj+fqAs1sW-nl!MM?C9
zSfuv|q_T}Zl=mY7vk2SoRS4xR+H!Q)3ytiKpe3rkJ$!f4?g=0jDi&MVoP`mQWwb;H
z^L?=pS2PQ_tKMA5T~%g*FB(hBbTdR5&N3SHMsfDcVzeVreUSb(w2yk7R#120maBFx
zbWAC0As?206Q}(nPHsEI$zR`piT-sucx|p;9oeo2zT%sK#kyWP9{b4
z!?%b6U&qzDUVLd;HxxzJBu=(k+2eU{zp$+6?YAHH;Sk@-SbgjwRx7q`K1TI>{s=2g
zxx*FXJuaXD+YGLj)@3&|P~kMhv9Y~(BOc+p%eqVX5FYV}AYu>_4*n@%Dvi!nG*iWD
z&fRttpXWjxt#L7Bew5kGN8(E`D2^3S7|<~6R(za-P8mMYkT5ph(SHa_^OF_Sw!YYnK0r_EX
z26o|QAtp9!mc2fWmJZaQAZfy?`JPQ+{1ZZqtc?mc(y>1Iv_3J|)rbOE8scpF8Gs9#
z+fIx~0S^}%p$|>S*Q9}y;Y!Eel%`K-Bv6aW5aq~MRR)1H`4#F9oFsZX|AD(!G`pU=
z@x3ZWMzhO9x(#IP#%y~qvjApB9x1JDSmlL#)htPb4Vn~L=;#V`M1
z9qC$v9Xg=liCUkdwtCz%*dG>R&K1`|v!4zMv5aZ^tUds6v){O3Iw}BEwW*QXv1+-@dF#|tF
zOlg{({~`Ri76#^rcTQv{D%ucIB#9yW6YmD4m>bk
zl@gc0t0F?mA#Q{&9x)0pe2Mj8JahxpayNpdqz`-uh6WKZ^%djN21Ja$a8Y!^a_9tf
zM*e1-#nG=5XvVtP??*jFCnU3cx~=%gXx#TDS(X0A>U8Z?4}VkFPVZp=g@426R57t-
zJ-BVuJr_(i1^jRU_x$mt1Ad8F?K~SIiO=sc;h(lm8R$IQqIDi@P?V7$oW?VuV;Vo*JvqvrIc2y#b6DT}W7c8zm#@b&ZCz1DI^LJyeLeoJjj?As@pS~g
z*Q`%lw>Bd^!Ja97G!*Z@$hdAa-cJ$!V20FA&xbOkF1|hw_1EE#+M0qkCa+tkvoB=CnuwUV|={}h5Y*{vnWG4iZm);JIS7@q(36OIAh9qls&-LSEKA&
z{3&^FB8{@wP0`BkLg&a^sCAA|GwQC(@B)*z()-$sZS*dPe0>oLO}Rm>n2ro-v)1uk
z74sMK_0*BHMr~yB414BmDz{PXbb;2%9K!C_+M8p~%)%sS5bN-_7G-WkIxX9tITPhF
zFqr3o3)_$pg%>NHbCEFvg-A`#LVp}uFT(HP
z;WzQ53pw>;e~$Msytn7nC#}c374IM2QlIb$-mk)YS5AH6CcH=E{hOTn*eCHG%LG!V
z^>I!l5|MEvr+)O)cproJqdE0sTzJ0*?;qvVLvCej8s6yw7-ZRcE#CLx9kOhliuX5j
z>a7`g?!t2(o@IF2@!X2%d^|-wZ^Lslo=fp8#xq~Q%No3_#B(E_g?QeL=Y1@*8oy$0
zwP64=@t23cdyqFbM~M%Rypy(EB|(Yc`<||JR$(Xofqe2^Ne57Dnk*pFB|YH|vI7S6
z2_(r635pP)Q2ZC5OduNcp-_7epEj?#+N~>-t`!JMLz(zD&aYqi?(GLjk*wudy?;rr
zylxshsCXYxK%{I^@DmO`9GEm?Fqi1o3CaxW6gq?sQ%H&Vi_s_af#64l_Fth>XtMrJ
z7opQ!e8s=-=K6)xZa>(43VQ>zAH^b-&T$Gu1EJ1o5RAhw20{(fiOsv{+>d+-Cp6?9
zt%EvmgXU{P9(_?mh6)`ywUxm|OwJV$Vu!T?|B5t|gbwK=U_ab9e*$&Vkd=qoQpZ*d
zggR`u#AW88Q#zTWj#Ca|*839OB0By<9~=$TqtEhaCnT3A2un)REGVLEJIYF@a_bko
zJO5x|$#o6$djA4P;;5y5!8FT3-;d5LQmY9OFrSa%);YG|bN`zEcpSdpYZ@9ae)M^g
z4Ht~EaWyxIi#VD(&9(D`Z}5@wlXj8r{3WhoU(|^`pHC5baHm-U1I(l^jjoO>!v{8B
zBxE_{e}YMTXyFail8)xq`!OQlj}2Mr({c~e5>T9&qM%ZIT-gH5k~S2I4>DZdC6cot
zcm|zW5%{I#1=H*YBZhz%j?J$}NKcL!!44`Od`&7`BD#~|0e
zhr`?#;M46!ypK4PI-r5HErQEy8I@@qkV1O=TwB*9i%f$dawouQ=>|o7B8~|w@d0@Z
zuDx9kj|qojbm~e!BPJgg@ZAL5?Ow2X5Nv*!KFmNzG1$qLVQuKg0ca2-@TIA7U{7FB
zqW5klZvz@cvu*Ygkh!!g=3uTKaTJ`
z8jL`v5`LljE*SrJrr=xa_{cIDjnNs@Hh3Y_SoQt*mkSWCC5OKSrZfF|!&nTI226o1
z3A0r0V`C!KC@(==X%a0E*^J4sAHZmnQGPogd9(q}y+G$c*6_qK9v18?f5#91p^^_D
zaP#e(k5OmzqwnSp%TSKYZN#HS(9*2p*?#F?VUJ&rN%X|v(^>EaQLO?zWFABU4FZFt
zlYJO2LM(M?9OZDWfORbUA|e7d9!8(Pc5NUMGu#I1#7&6V^(xj?jJK8@ha<8Ult~1|
zq@f9AG#NH2au-czH3q4#y)!={D4HXWe%CJhc41JvOR_$~7X@^;x#d%q|Hu#XA5UtT|2T25
zWjf1G!;A|O*oF(R_ekT>5w8n_wxRI`7@-BD*@hAv)m0O9VgD&S0I?kj>UN>8M?k@f
zVPsapqbcNvx>td;M7B6kq7ih*?fEm-;ZUCi@Jd{UV+esozW7;+OhpVs&^@?5PEZhb)9u+e4KVM%4B__DnU
z1ERq9DU_kRPA=5f%TasO2Dx3Xt4Ei88YwXMXIN9*KWR_M;^gPChN6B#K)k$SP
z#fa%6&~yhhWg>WQPKyqjrPv68lc&XusGc|q^sDIS}9LzaZHZMh>e@#prR#K$ltTtLD<
zCS-M3-1M>>m#DCXdTx|{r3V`11J@_wu&^t~E~bg3^`aPJi2W
zkczgeTk$1{;YOFdJ>Q1#QGt2qz70yOLTrbXQYoQuhqGlwY2zAmwXr}dO{E*#5*(&N
zJTT6e?!2>5S~d}JDc4-&>v^re3$;{)lo|{fD;)&IMIb+&}bM*KLn!*Qp1qyMbVJLXo9q)
zebCP3P`@_BpkF@vaWa@_I~s6FQB3J-oBeDrATAq+DiqGFXRdx8;pL-~tz^fhoXYgm
z3&CKH#t|tU`oJ`V2=xTg$SDM!oT&hPY{`goz*oa{T#F&;cW;OL$Qx+uiun9XwRTO=
zHRehKyk!%_vlc*-7+Cr`S9w}MP|lDDTfr!UZ^j5b+ANdI@Bt4xmWfF6Tw!%>UNoJFd1WCFE$g~^@7DhNr6=w>
z0~DlQg?A~usB;rq(%=7Msnvoye}gUu`g2m7{spz^8mt3KfzCd%2?Z`f%b{yb`vbVn
zOLWAj@^r7JIg31CBn6&AHw-5J>N1lUBR}kfdd@?pO9CDei0D+->PgYvEI;zP$ZIerpzl*C<>(nGj}L~x0;($G#}jW&plj$XK?!@o_T(c(
z+}WE{y%yvLHLg{lRiqssg>E42_*$$diaH(6hbnP0`wu6FhF?XkR>wk;
zps`!pk24y~rYorTI&P?3rF-Cx(5!BkA!u$!WsgQawtoc)2#%8L0;qZgNpwP5|--T
z8lWWr^gl)577)-|60emX`@bl!m%ZVMeu&fnNT9<16oFa*umac0S7G|n03)_f)3Pqv
z8z7JM3gizA$Tv)?8$t)Rj5jP`o#fl@Iym2YWY8JTRJ2B)Qj_)n;9PW2rNo1^PSF43
zNM{`WlRMCd#{hL}I=bYM-2f_Deg)sAF+hE&b2swsvk%BT3gEv1tJ@`~HJAjC1}dE|
z(ZIbs<9?@~{tnktGf)Tt!cw`~g3RP@ur3MflAMH|e;|#)2er!o3h5i8EVUweU9)~D_b3rMXWrKLgrMy(q4|AD}(8Uplul#g9Dq}+$*J!*
z+J73rU)B8gvz@|!5{PvqqJYF-K;ri8q-sLKf_kuzunZ=k6fl9$r=p@Hh>3Lcnv5?U
zy|#vK;5G<>>h6b8nq&g~gn5ukIcRt8jVJcS1&I`2L-ejxGG#;m7kEmd!sM2Pe4u
zS;NZ`ohxEJ=>hk;o_>PQF6RDhU=70g4$PDM@va^E4dLP{Vzy)~9pkg^)2)P10uvA2NpD2Vv5NrGTa
zr(!zkBSCs{s!jQc^t7_3g4#s+))c4!*$N+lOJai`T;wU}j$0Pk|PNsw&<`v9gol_N5wXtMP{rzmnR6og7reYa_NVOU$r
zlQVsx9Z)ZkRp1Z~k>$x@zVdDSzc&Vx2Cb(Fcx$`4`tDFn#QwM)jUhNe19VRX^k3nS
zC%EeY>h?{CL$L4nCz%I&v72p@s(k@;Fo0J41-%KjA9OxS!Zwt(1NQ>+r~i)mf35J}
z5dRBsx=Ke^g~!q+ir!pJrvwXxh2
zK*pFG%a7wb-|VMsI+HLV2Lh-EfKn@|FBK#l)QwGT%hBHzf7GR5fYMSr%$*4MaO~wv
z-ARs8V3B4~16Rz|&Lfn2rLaDBytXQlXPo@{uPa{v*WeeR>h$O$gJ#WSCZd
z2yfKW9gXC32;+$>-+SLn)to3r2z^hK3bc73Z6(r({4J4YskmUSZi_z^C)F+>(pI05
zs)=jUwA80w4at;gg8};lNdC1`ECMjvtXG_&3Ic3@B5pq7?jKSscScEY0RuMyhhP(h
zv^S~p&(&^gPyJ7rv_WNh>ox(ySCNVOxq73dc=;nlsI}Bp3VAo0tG@*>=pB43)~Ue0
zLUTLCAdMAql++NJY$n6iwT)4pc*aaX`UT
zF)dl41a-ko6!t_}z2feu*bQNY{J1JN1?dKaDX)<{L54zv#jzAo!hv(L3Y@oH4>ee-
z=@t^*K6()aZsTG)k&7_BISAwii7Ez}t4AWO3TZSZNNGC^;olHm?1t66CO*J681cU#
zJRk8tY4Hb8rXS)R2w%sF3{kX7I^I!bZeC96nRAulcuDdeX?=Z)}sgcYXbXpI-kvHQod=Sr<>$sIe*084)eR9Wa>tnJWF
z8s!*}ixvvf0ML;$UfPDcXndWp_92C1CZ-N5ZG{S!paQHFRnom65>fFI?_=^BJXiEG
z(%#aLVK~;h@fGSoSY1cdir1nZJZEgN(ZHFdXND&C5;D&~I2wzPyI+881c$4Pa`k`5eOI5hg*6cm;hGhDL)Kg4+XsX&J=Y(tJ;;
z^h=yDISiHn+mom;2oXmSp-6bp)Z9Qu7-q%NK%_WLYg0-NR3Y*BlIQL&C`Y52(rHJh
z>>~IA_>V54(K-8A}`dev4aq%nKe_BrDH8fFprVsGni(?YH58ZJNiG-ikoOn
z7~X|Thpv(;QI=_JEt;VWNkzf<=4k*4#zDZKx3qsnIl9zEZX9T9LS<<$Oj}P19>AVqt+isWmEyhtwkIi
z^xiz<6mY}t`zg{GzG}Lkvf$AFy?{gUpo9he(gcRJIH1u;@(T*rBH?2zLFVmRC4{-|
z2cB9$j`8S`0(y)f6~e9qmj?Er>nh>zo#HAD`K0lVC*@lv#Y5avoviB)*l}q41u9f5
zdKJ&?xbGH&0_d~Di0_3lu?&@W)LVf;G(JRHMi+aORv|nF;VuX-){v|na9(mG8m2eq
z4KySRc^tD@_4ml%8oz7{`j!TS+P6(?8&N4OqaHOC;9LYE`jbFls#d{Vts98HlbB72
zM3Z(NJGmOkX^rTbrc00*jBc~w>2^j
z(JHJv4t$=HN=Il2=q-Z+-nR&fbpl98fDu(&QGO+Uui~fYqx5IAvTzeZ@fKyoljWZ-
zf+@F1>d1ee*HFS=9)cFoG#ZaMDnoPn8N`i1B?O$f9Ir$27_Bl|$Xt-`H1ZKwC5;kw
zLy%-q|1|#_mOeGe{7nu-0qU0J$a@_Fi2&Sq4QK8MchUe7LI+@5_imkV3@X+eNB1-8
z!n{TDKVjY$fC(T0Xhy9>1?2pwq27A5pfBPh>*U*rie{kv!~7wDW)S$pUF4cH9!Skyh1^I#$h>2UhA1Cic{{0?!Sr@TFNrqo0s{FHkLW3S|yxdA?G<7UEoF9vWnO4Gs%WA~GMXr%Y5?2Nx+p
zZwWCRy&vngx_Wj}bzdN>=c95zqdeJF(*JZ^p(E*O8!D$Wbv@qL5b@;sWHS<8)+(fX
z*SbVR7CuMN$2#CwkSF_Z{KDV%0m@PX3h*T(U9x=zFDa#2^d=z%%VGNeI{p(fQD<=ukW
z07gQo+Lc~%c%81KMXHxJ&h*Am_aZOxe}DM`4CnQP?_;FW{zyzj0U{Ok-w~<=lp2jn
zd;m<3M@yCX5s0Vqggi~LzFMQyw?KStgON@%O0VNFq!Zjbr1PdF_fh>ycV-n*zNUCx
zg0RrUmJC3&e_O9rpaXnVdCI5j7S?L{|B=tqA0!46JCQ*j7<#Lsv{wBag0#aW#`AEw
zCUsi-j|x#=0m`VYhtB)ofD`r^$XJBwm5l@CqQ_MJj>FfjwA7L*Lo&X59+O~V451Zd
z|Lat3H~!mfnFp8<*GeR8RuaxQEsBJ*9~_GY4rDWBlupA4CaTbZ3X%P10H!d_5_N$2
z%V#lWHX>YtvbP3G*6U=-1T7kkatGXQkfQ+yTyM~&K?mG#kPp*JxZwCZMF5l(N(3C!
zNdCxIS~eI!3p9jia85x*Ifz?P6Xzwrgg9DMO7UHQw0DrU6hBZzL2!>E%rv#xfjHOV
z2-fv!yt+Uu`vhi)!-7i3;197!VvWm?2$33(fj5%nI{)cvN!?|5ke5_CM*}<-U+T}g
z*WTVo#nc7FImsy=vEaYJ2NWKVldPPfXF~63DSGHGWDW{q%~q~KX5&9m~y|IIAYWrLYzv%R%tx9m?PX;6gs?&Wjr0N~d;!igKxlqibI`P*D-?Pu;+;
z1D-@26j6&L90i>E0e2zlA~{e;auf>oN8nYog5Is@i~k)SxFrzRBPK$Z;^==gY$JOk
ztaG>Dq4^|ay|po+^8vDpZAbl!tyUZQ!UD84eZk6G(ihTk2h`z!y-0RDX_+gn{|rDR
z{G6D#5Adto&sfob7+yby@lGl3zTg_#sXV8%wAKv4*A_|uVTMo+^($r&^5Yy0ailAT
zIV0(cbB(=JT`}2Btj3lv6FTH*j1PYZkh!F%RrYq;o41F&^6vf=Vi}~sZDM;8-_yX0
z?)*L|hsu}iu%gflk;VewM_iFJf?YV_HG%3P`-^DpdfXo&x%^a
z^WOX`G-xtnK7^2$j^@D*p*o#QyB$c~b0Kf%U!(b&3wc5JVjHeN57GwsV<6G%bl$*w
z?HhXIFLb5tDqwH2x;*dQd-0YjT2vOM%2NC{;z8I@T(DvYD~g2mt>h%sa}h%63@vYJ
zjC!bULVQ}C@^k(#<;UseqW~LaQQm9dGg_nx#}bpu^M>3HwbZF{kd`X7fVpO%4q@-u0
zYTswf;F~Z&HL&8&NTgmkcrwrbLAd_`ly|I?c4tONoo-E2vdQaD=Jk6pvF|5Hsx&}s
z!?e7J5)M>H%4@qGUr#Cx#>7wt?y00iZyTUm7+f>B(3p<2qTcG1PD5B}
zU1I}m^vsFIB=emk5SSF84i;!hLuQi(=m7!I?6LItx<^AxQTgr@;6<2+H6%9~6^
z(JM^C{Lsrny%&4wzr!>PoOx2rVl2p#3PdCzk^v=D-4;0DT4^KV9&SYuv2BzwPC!es
zozFzYg>cRCIgl$r{=z$M3Uec!m3w1D;(0g4hYh003Y7=x027;K8v1!k0y%0lI@%f(gB{Q24jqMm_;
z#OD)bG1At+sK_zTpp3MXH>71rDDx-YH>5qz4P3oddLxX>5)`dj4a6`5>t25i+O-{r
zenddA-oTnSUyc~sIh{bsx+O-T7Ob@LlSGBz46JM1&6%}S=!W)13t8
zaGp9_%yMBtK^`Y?5NO$N?Gc8%r{E=y2uU_&X6Hwr5L;YW3U6^j+;w3gyuk?(its3Nwy1Dr0iwAZYZT@U5@eq>!xY`$_6ItDyd*7CmC8Q0_DB=v
zJud0uV>cG<(SWG6&}{WQA+EZy_{d=Zmk!Iu3CWUdhMWi``lSFKXY7g5B7Y^Dz!t@YjZaZ+;$s8$^(E*$SUe}M?Vlwvh_Z0
zU*N$SvLKAITq&%H+#D?%;H1r{YQi4;rP$)hp7jK&)oY@yj*Eu%StNJ)QpDG1DdC?V
zgB}mhT~~5;tRg^Cys;M%^BW_{qXFd7aj~{Oo8|unYQHfw*_z4yU_bA#+$b$LCfYY(
zv3$>QF|q+mZoUe67@AiJ4O@-k%g0>Z1o70>qwp2w{;s|P{BBc6;N+)t{Sx2+fudAJ
z+E2+x_bMrTbDm^%dP&tPX#<44A?qAkE-mpPR-8M=96+o%cYY=`lrYjh0RLO)sD@rF
zDD)FRrCJTB_5SSrcv@bP!+f@M5i9Y{#4d;*~%aD0016G_U?b?D1l8
zJm9$fvKNbFhILf8$jn87j?0R
zdb4vLJgt}gucpkKbHsRodo#BxB+3R4k>7?#h?!yRid#5JB%51`3==!tyehdc1Y^f&
zV>sg~{rvA}8_E6sbYC$joON}nO8HN?JujTi;2v+s%AQEdXx&oyv|veYNq}f7@fO2d
zu#PU}rU6YwUJEuLtURWonS6N+OLVxg;>su&E5`V;R7j9|enTW1>{4Y~BH4#WvK#NKW7*>@K
zD89QjnocJUM9PbRVq*;NQ4)jpnNi;I7!5|B%z62jP#&|qZ1ETYf(>jN~E`Ejd=bcF{?dG@+wa*i%~kC3?+IU7GJbyHXe6a%;>If$1G41f85@8N(l(5M~#awL(h)pf&uC=5GVNqf{5GCH}$bNGv
zF}>s~ES=bbhF^A)>o=9W8(?A-H%Jg0Y=!-z|4WY
z5f?_9gRofnBL3_1)nkC)AiA^A5(h=7rQFz#i$k5+5ts5MNiO1@MAkR71k*>Z7?wHk
z!U3%=N30x%{3iU)9kAa`WZhU)i7_keZ>-vc+Dv6(z_ONAryUR@yRbPtz07{T3yZHe
zQ9-@1IW{0Zw(|WNc&UXFKCXvTTWHb3`DHo~!Y(oz>&a%@Nnc_ClEkM+q|}
z&zJkfoMhIle#w&mgv+hb&f*qr?HiN9T5BfSZ;!J^4S`RtKG*nuyGJiJh7os}+?%D)
zAAvu=i=sZPkGS_F3*$XM5g{p9m)m|KCSoG+h)=|-6c)#(iZ4@G0B`(>_$7sn9(Lzr
zl~ckQnDxQXgGDz!h7Ft%u1FvXNiH~K*5a=sUjJWa7uk9BN&QHqM
z&ldZMSo;*4%fH!YkL=6tGJeBdg!f~;dGCEo!V(*DEZezz#Nt7yr2Z%NErS?mA*=S_MjQ^>L!XzXBw_nV
zM-H8Ev5-);9~CDCV=5otEoME-BD=p1;tWR(3PytDNEZd8sfZ_6ZH_}+o_W+M1|d%o
zVci9J0Ea4{@Ub}mEc5pmgS_t$g-l5jO<#*wY`2enXhQ
zq2ya9$z2fF5^=}(isYdzyvZRg4Oc&|m60R^SUb?Y2gI90SwQ&D
zyVa^%0_+YgEA@UUZrYIA=cB!1=TH_B5{}V8MgIbwXf%*le?l<;%Ni<~k05(nY(G%twqE#sXb?>ft~k4`Ua&%U&^cID0Jo!Y*wFxh3DZ_c{I(F#+_u
zU4#@y*KToQI2*>#?h&yg*m>S}ukd(*4dO5M5~E&VF>W}t&P_>j74N*j68T$uMBGT`
zFB~thzWiAw@POyzJBszE5|&XY5seZ?l=x^AO5EQqE{|fP`A@q=zf?BMbqi>bEz(A_
z2vL>F`mpy!Vj3&p8N0>VG!`!2dyzHm-TAJ17|g&B2-THd(y|@Og!i#2%m7X4YOie;
zF$TR^@piRYZ`eh@aS*G5?}|C&FmXcd#Frw)k#Vd!uWuJuDN#b?coyz)d#75cn_b*}
zk;O&+P_1^DUfIt(Yw31T%NuPMDdW+w|LhW5#^4Jk|rUiO&!u5STrdQ#DK&*=szr|N}fLdlEo)aOjfRTIc660QBaV}la590VM5Ti?8ff$)k6AYWeTDGi`mPsop72=>K#e5M-z>kj*jDHtF8-BOW14GY$!QgLD$o64Ul6(gs!ATi=q7Rb}8
z#f(?6NDQhLTV7@F2Y@oII`iS@w~3w8S?~78x2Y=Z
zbm$@VW$Saq-NcBVhUJU~$c_GZXly8!!cm8c%C?EP87#QNURx0OjdwH8U@jDL^O@DyM7YCp}u>Rua
zYpk94;&s+tjC~!u(v4IHb`u|N6X$25LkG-cBZP4#Qs!tWZzw5qF>{J%vM7&dwCMg?
zfudRLIrc!L%|cp?mKLR?wGnSu7zRP)RMPGdQmE+eR<)wSH`oBSNliPerG2HP0f`WC
zKGUT;o2zCh)-r6;GIV~Eq07}Yi7UBK~@jA|xLU3^eVlmeu|
z!6->o-G0*bTI*2#=)C#Iz;INh|;+aJf#6pO8_7P{tOnp
zcq|O~B|J9wFr2pu4(=s{!IR#V04ag~JJ2|Sb+93>Z#)G+7I@M-0*qQk9Lrz*F9>A-
z5b|Bi0zxFMl7vdS6!oE~(AaNpLR5o(q(=jIZ}d|=G4y~dw?e=r504C$hGs=yNrK@&
zc0RH9493_tWXh5b|8IEddP*|x0qOw5LBhjz0OG2Z;@W`#LT{-6T!Z`s&;)=4jDR{_
zrH&BpeM1?)))ReQ87x&-uagp?)0ZoHF!>@lRu?e;CR(0w3xHAK9xhVHe2r8|x1ED0
z<3AO?`%vLlC`^}GNPD}8OC5>4kE$eG%Qurr9q0cGtR~1~(Sc3yg7;M%FNA9MdT7Rf
zb07TM&0PKQ#Lk@qapANjD$T}gWOuYdweEK~)J7Z)-U@9X55t7=M}>-nT_=7p;z=)j
z3w8M7t4!3wwibmfv;g$N=%)i$a$?mLqT#~;9tQwoMTDfXzyPrra1rvApN*Mf#2xf(
z%o2ii?&yWFyEiRyzgzjH@TM;yg*hqaI{~j(|SK3e~RSYhp6R6
zjQtoAZbF-$$IhUCjS)ybi>&oZ3i>nLS5Y|}q%4Qwd
zbTKoV?d3f_67h@J5dPbBv2+pZ-tg8RkS<2`vXWLxDi-G!VF&N|k*K#A-^-gt`^9Xa
z_t%?&Z5Lho2)3QtELJRLGrSp=B9kMI*0x-_54NpDgygVT|MkdEhwhay=2s<_rnc3a
z#mF2;A*I*FmK>Jc@MWsEB}p$$ZLe(>4{}(z_oRpFnTYzAu$|m|S6p6#O?mKUG3jj<
z>EBp`?;PU>m!GZ*quO;x*xqJIo+Z-e#>$oKZ%V|iw^?kQ3&=|Qwruom4Aw5zcu{^O
zgHmZxWBC%J6TBcftZ{b!eM6Jj10|y0QtV|e77N=_)-1S0Du=U!?Mb`u{%WpX0|1wN
zpeqZ--KP?9aVdm6d$Abu4ph^{#bVAoEY5XJF_iow>mAmTFE1A7-+{=tuvqx#0{L;p
zB0rZs7L>F}BM!#peFKzVM{)cd3j5tn;%YAT6upWCG?VT9I|Eb?5b>`~U;$_(jnxjt
zVw{X>qL3Cx)da7|Wk&I_%zRw~5hp8gq6(VL2K5o9H|fxEyWbS;av|>{-c`iB%RISr
zvxt8eCpx!5$zAZ+goVFE%5E&T2BGjjS1J#Z$!UiY4;n<#e~r>V)Dw%}#fHUSeDW^q
z(){5;Ukh_R44Y|Hs#Li_bvhB0T`^n#^WwZ^T#%bjGv{RC8flxx(qD
z^hrpuShXDDbhFK3-*V<332W44l|A%K
z_$m+avtcnLb4pMIdLt!Wi@h+tXjeRV=rJlk-Go|uhh62yiTJH^bDl_
z&vk`p4Cb2f3&1G4lMFH!=zrH0#7l26xTE)@=C=6Y#
z;#>jx;eL_0TYz(rJ=eu~q105Z7Ca`=N+7A-%lHsUq5WSbj`Oj4Bd;egGcx^+pl23Jk@o
zNDN%Xl6$*-ub$SbTCf1yDF6(^u0qw2`Psfin6AIj8uEo+Sp+<^LWuya5i~_UwzrGK
zIly#HKzeoFmkWb=o1(lGZ|#J!4(rHoY!o4dAkUSJc^ekQiB}5Q>5oQMf
z1SqPe<>=D9MsN9wJB2K;p?#xDr~CLWu!@j1tbM&d@c(ryDOCK$OKVtaTs9(Tg)o8%
zsw;$_Z64x`SQE*?JiQMlp%dmH4(4blo@7nMkgaT>f0|aX?q+zzMzL=zHnPud%)7iW
zG;eF>6Omo?Y&HH?d~NZFOV>_jQ?YMuzcDW)E0RZV%@^^fjl!(?)w(b?X3s`9DGE_yRq%1lq_aZL34k^Px%AJU-5HnL)SkUB8i#7w@8+fJo*3h0n
z3QqxyM>8jXD&DcWn8dwxtfg0Z4{R!Z;q8T5+j)bCU(a&b4RLTiYlnlMR49iI(FV)k
zmTD9HLrlbh8$=ww{!eHK(UGc-Q1S*b=|dK8>4-FPN$xPfDG{`^sqi+I4rO>1;=sUg
zD%Tb;al!!MtS2T9iJD|IMbD6{V;T3+07IW_KS0`{aXe0oYpR`22O&&5j<2-zk!re^
zjUrEKkGZgilFU+hDZ)
zY!xXR*#Q@*n%W7kBId^jZxWG3?8`0`qv+uDtTQmvnxgyZkiHKpQ%Ht9N;;;tq6>_W
z61_KxI4kRfx*kfDHi;Ql7UD^V$`FF8=ur90K^&XSUoZM?;BowB5t`4tyDYtqHL`32
zcknjr#iYY5w*DFn2@jmy&j72Wt3O?yNJ~T}uJD~boK=>}2
zQ@8q|TJ;~`8@WLY+{%XdcgKI4Vs)8L;(CSYbZZZ&HQG0~o_A=EqEWZpI6YV~NT|U1
zcm#YQq)MSm+7Y7sH9t97wA;pf+z>1=rnM4nw!u28-1u^^oJ^9=Lxs>?aC-GQ}<_!xj$f_R&9r3_>T|8z#ZWJ%X*6$JJ@G#
z^>Nm^#2R4lS;~H6E+vVZf<*^AtEh<&-o?%_ZVMK-ceAtH8Z1ukVM#$nQkf0X40Eo)sO<|}Jzh5JU9favla-rCA5L)EiPuVNH(^|nlV~_E#l12B=U;|{t
zzKe!D@vV)k33cK_zBXu`G{Y2M+v|m*;dbWVcM#HF8zxy_1-I21{{t|7XHTZ=tHDap
z72r%@BLwpnXhJmIgt7k_Xr`aLr2=VDvW9*kmG?>H7;
zgVpI~K4ttWcVNuDq%%&~Q$|aLQ+wd;wdl6l`J_PWkd>WpIgRHGuU1`w$Um>ZjvX#Y?|RmTeQpIsWv=lZibf_bb_0)rl&qzSm#r
zFEWpMunH0yIdet7k%^SE9=irMXQfI=y+}iij*NdqLvSt$fy)>9LT&35jQ}+Jk3b@6
zHT6{-u5V*${sgU8mrZ}$9{g9=5lL=(QKilt#JPbwqAJV~*v&c5^ORA#^X_ef0cusQ
znDX^-F<*<8j5$9=!D_jO1RqlHXIXFXF%KA!_F(KVKY(==etwX1JP&Z$Jqzz8B(YQ1W_!qgMsP*hdZ`Vw-tK_W
z3GL412!nMI54{2#z`+E=Ve%`egjR^UCYj=*%)5`cN%6Q-c+CioLaS}(cG;HvA3?^S
zpf0QcYyTM)-ZFT>abF|4wz-UECRIk2db$Pzp&;8IpwbxNPp;}o2^We7Cd1)flSLS)v5FzcjX6l<$dmI65GXkabv~FeS0;H<0Uh
zgGVeOu#}mrCz+%jOWE){arDKDgeQRYg{;;YEoopvXIYCGz6-^oKA%BN+8KZ*W^nVx-y>R-Qf^GdP^Zvx+!yuInnelAJwQwliN!d!L_R4FO`0wfJ
zZmUF#E(gPpuIux{>oGI}CBV%D2?x;T?8XR2@&Bj1^HCL1auBQw7{#1NcA(z*oAd@AXpng20Dtg0te
z1TFjuMldPQpmXcW@`
zHqHzkM1$AGFLa=jS)^s(-FHhckMS!=M}Z1M@V{n;aV897)k^2Vig`6E?}_gyL-3VB
z9SZU-K>7%D;Lj5*a4HL)-6f_%i$+sWlwP?tb$Bm-J}5?M`MhZkw_s>jLQ{9UgG#n1
z4I}Oxh1F{ezKigEa^CPevG}!weOO=oVlcQ;@aq9GPT0C6|FV5`kSQyv`1#hQ1rdwP
z#g{j(4jR_3w18!?;vR;;D~3fjOguI`Ik^JL&YZ7(gFi2X7gCFAxA2f$RjF{
z=phANk=)+u>bUA0+~md50&-((!t*uLP$|_l1>d%GC+G%tU02$`$u{@8^8Bo}cVZ{D
z?Ys#6{)g*RIgPMetu}bt79q`w_q)%)d4W4|Kjv4^^1fd|3iHglkGSW#lmIz%n*-y7
z8uYLIJdf+yo8N9joZR8RFNDD`M7Wwt_FaEWC?x
zpuX_3=v@p`=))_;;$k+?<0FNFKDL`H?Y|T=f5ykJ6c4tr1@(I=2|+fJK}G5!-qdmn
zOD!GZ(sJ8lKAwXFeb7mR>7C;-UEr)RnH(MaUpZoy2Jd>1bY_1D);rI@9AbZh8*^sh
zceE+E3p$Kgo)?xm^#H=(26=s1HrQS$3%s(fnKZbaa>dmZliekDRx&#k?2(y)msI?0
zx0^^E4bGltjPR2ZYGBvW1{e#By)y@n#+Me`Z0!2*tb;P&jwDDs`e5E=Lm=P_U(7HTS$i6mI2iu3pks+OY({`r_Nl6|d0~FmN8?R`kwtF@CFA
zgN&E9W#?~hA6!tnVR?Qgo`eg}G8bpzkw=s3r(MZ+-P|mx7{}t>#(;T@0=3J)8widq
zJh&jdaap%8*CedQl76D=kH>+o(74T>TUzZ|_QIl8z(0a?2H
zUC9K%qJhB$moZe_GlB~UAkMF^5pYQXe19_sT$VL=vkC5MOF^wrl#6JE8!9OGGo)8#
zaj`($`NfL`_AijGQTHeR;T${?l@r;Hpa7ArS|ghc2-162M6n!D$`j&GOHl4u
zB)*{7l7}d!KZTvKHaov%o8W>(`}_H+8pYB;F^ftvk1L}=vNVlk`>_Z((85?$Or#Ho
z_(SAN#KGu&P$>xuMc7*hHz#4CFr$ozLN$*o^FXz0Ks?&B$=|3}mhTjPc(*0{wt=eaS5<^LuK$AxNO$kq+*gM=6aNMOPkAz
zGwYd)M|)&;qo=%~tim(iNyP`f5w2I(?Eq)R4m>bh<{=f+L_CPAGw$xjoXgnV)6455
zuUx^zb-0;}e}m97dxB)0>5eNs^ftLC;Nxv_?M-q^HTlwwfv84fEE96(Q+SUp13;E=
zyaSG3rmMMlE`~%J-tou+5mP~|%m!uMW}?T^AVa1uV#V*n_rObd#Jn%Qv(SSt;Z5vEUFV{kaYd#$E9*^62B|m|uUIc(L+`xgFYO@Y`l6DItRZ*wA#%mgz;*=_CWAM~
zp_@`x9(as|+R>Q79-xpXmWSUkTz?0pQqagobT{=XA}S1KAW_!s50LA7kn1~;s|;j1
z0FoWV^X(^FNY%xfP@92av>!mrK&>tA4RTHZ&>y^E&I$Y+pw>lD>j0=#0cw?jS`JX_
zJ5cKYs8t4PC3;CaPJ&v0z&V2D9LRMNXC67DnAa6)!JJ~wFLdP?BNE>7VsqatW_V~K3>0g`hu&%1K-xih
zpFuq*(11pBhurxiiCN@E98wi;lyMAXJAn-QKsF*%rF-VP$HCEy!#xWwVK7|=!%GKQ
zPk?csfPfT-7A?j11Q>Q3G@=p0v<09>Twq=qC$=rh{{syB04Co-P=Y4h0cI|~3IWQD
zNtXo5y>t>-rt9Br|wfHo(DTs!T}_kDJIMsm^xgm-`=wz7svtXDG3w^!z|&Tf(f&PsE`JCnK^*>HH2v5jSbMiR_#{g
zU+ofw_Y#5O(H6x8R})qT%>|}spD!*TY-!}{?12iI(BEJoGMOnHz@|tv6+FyxkdE%^
zh)p42uMfvHK^&=Ar(wvFy3!rh`y^Tm9?s{5e9AV7j8(JZHA8(@w{Qs@@Nfz_7WguV0
zJacg+hWtfCS-05}yI5!PrgzfkhT?VkRbA8Oe$aLF+?8F^kdI_G+OcNi6+qgt_MvYk
z7B)|Gpz%3#^%3~npe?LBmTiwWz|#lUE}?xfB4~or5>9gaC5(#I*t2>1!Z~CJY*e=9
z%P^8i0WQ{XCw!WQzfi=pA-MTn`6Gjll<)G530OIW`?HGvfX?QT9_{~6s
z^1cjM#;uF;`{BzlK&*gA^8T0jnliD;U=1=NvZZ2+1i?QYfN^4mr823QGMOZo;s~Qb
z*%b6aMPa^m(lYB6M`XJyX<^Y|Zzvzu6}%N?=e|v4-HXHP8;f=E_~9|KQ0b1QckZZ)
zr+7v4q-$0zxR7XobT$1RBrix?_BqA^8UsI~zXN}sb%sfJx<(4fsS>R}a|vLV#kEwI
z0CD7&p^*jpq#Y!b^}#otgfo1N=*>l#-xKkpb1PjaB7uy~u&77>DU_`aN@vED3CukO
zE~KVR@LrZr=USooFEv_}Pv>4F6F<~0S6gp1Y()!~5E1k_55R==yXWw=dHzN5@LY)7CCA*&yPC*;~
z`r^x@EWmC)#tK=;7(`ZHoO!13AZhyiY`k&6G2AJyE}F!vM+ouYI8H+Q2@(G#l%$D5
zEdCPu-Z&w4ehEQ6SlIaqHi>cP3NiI0+sEHoA=;Ot>NzXKjB@Cghp!O)6iZGShe~J?
zLtx7#uW?F4HLr1BA&oR!64{Jw#}h9IlR>AFcV!bbZz>Erp%pwT#L`o2n8)rFs$LEC
z{&|YE@F^#GUfKw;8X8)X^PgEEVo$TBK|e3YhOJ>HPK$hO2PPT|sHN1x3(Mi!fQ=h*
z34nkM&TnchN_%Mt6yo%NEJ~GhsE!$|lLsO#3>!SE?|m2zD2LXN%?4=*J)?YmZ318M
z#t$q^>^Q@k@?TNi8MYwk%{u6?j9PSFTP~)41!JMK<>HI4*y7QnAA#0B#F4GUqtH5r
zG*YlDsA{&O+Ym=|d{qAZE5y#PS%~jG;6tSOSoxy!QHJiqHW7($!j2{IEPIYeKQ8vU
zu}F{lNF&Q&j}>C=S(e~c*hM)(Bx{{fNV21~5jd_`@_Z+;?mz6Pn+@bkIcl;`JI|&w
z{vc0WzJNX5A9?mj2YcYcKYG_5`7>L{_*RqsqhHugL&N28`q$X}F3OdsYz-CDF0<$A
z=Sn|I{nz{`onkNdvVU`#wd1^@EIfW^b`QTmc@wf`Yz?y?`<>n8d_ph#=T$I$X;cye
zP#yrKriY5}|74pRl^9XN3#OVybijjB;yT-E95QsNG6Jy8?}ql^eq|r@GU6QADUN+d
zNy8O2GC^TBQzF``=2zl=#s4F8Kw_eFu3TZe2pB__idFY;r)Gejx6ECpvYqv?R1f)K
zngBIWhdOZ-Zli|_$yX_vqBOO2SK|~M3BO1^j{--{W76Z`x>AE}v9{1^8dU>Oi6g{}w#|CJFg($OPENzmsK=R&U>3o)s<(9lc_ABIZeV^c2CWK
zn`vF;OaQ3ARD5=yE%yF+2^8j|_TaS>I9trk9Jp(Vee?si#lVl}2;7R@?FstQQI(Hv
z#}fNl&YLjAD83AQY#B>%8H!J`lwpyV%w6TdsN*PlA!ZF3Z*c|ri_nMuf;uoEN|FmH
z-e?<&FnTN3$h!XRy59lhF)4X!uloh+Wy|Q4#%SxPhf~#X+a-`P6yTB?(MrvLhW1bj
zMC)O*8g8zK9o?1mFg={9hMVf)Fg2_pkei&U4ohD>y^O*JG}7W@+w!+aoU5zhlH@R=
zozdo|haGCzpodfGrkl~`%+V-hhQ9*P`Y%0PM(IAb+j_W=(v7z3dN>UC(mH{}f9MgJ
zbWzR2c0~{4sUIrvs~$E}7eMKnV-Kpw$FRmN+A8l#nXq%j;kmVRu~_QH{rRtp#RfNS
zYF*Poa8sW@#$Q`378rSm$7IBjA^-?%F#-YaCE_z9kMH`t
z60hF(&|wY+%p$D%Bs``o24lXq{#w4Lm9L*I4_`5$B@44Z=g!A6eqynGp9c?T{PjiR
zq9@Plk!uQUp=k7Ca86Am3S&LaI$%nW1$JqYn#%ACkvqSHoTWeTRbXyRR%)YjTNHj=s0yZfQdvgNVRbIUH8YAMO`%K0I7})sT;BS(9z=@VU76Y)*1%QP?zK*C6L;l>igQmoRA918ssTAWww(!%#wZgi(8HORX3&C
zuVWxI=FNQQIFcQ$i{&GtoCBzXOYN2`!`XbVY>=Rdgco1Luneg!C5tffpA)<-4JyBHp
z$}J%c2+_lcNfj`QnkUI`evJ6rZc3jNI-EO{CTB}EC|kTYNXz9OAwoVdE;i~kun~x&mAMxmk;ARKddP{+Q9q+Giq

?PS(?aegW88+YQv z?8p6is)1izDB^;6ivP)lir?a@`5M*WSjqOqLi^Gne!}3EMN{K&lo%7j6V*Uvlvo$S zhjEN~X9$nspB)mhp*)r^d&@o~l)E#}a#OB4;u7|XiD7(<`%h{YHO1xOFdoIKS>?DN>=l6&cCf>3^quPO9zDLvx=PjC7nKEm;>ChgGJFOebd?d9S&EBGa zI1gzMR$F-c9x*GN2e&>PD<1^G=~AxLNJ@A^nXpslT`->`!4+B=CA7HH;>-*{t?Z+#a#CIgIEvfS!9qPzyeY zFO3zGTkzKW@^1U87JN2qjC5KRh-VTLSh!onM)CdJ6fdqs@yGlwMaftwVo7qLAxS1; zJ$H-8qJg;vx#ez=AI)2PmFwKD*=|sgiB-|Ce7zCPJ0Qhai@VEi5g)^6H<({bu8X_G zz8IcXuiT`tgtX4whacJ{9*gC}V>CY%Hr&=NyXKbg5>!TYTAQm6thE3U%X_r01N9v$ z@|!x9wT>*W+mQ=*iP)Arl!Ku?)sheE^%m^1>a;W-Egb_8IP8kOVepTYepNxGqA!(+ z*DUfM`G@Wjms=7uRJ)+>F5%ybcc_b0S3pd?Sc?=!MY*ZYXzsX6%xJ}#6pO4XJ5tWMC2RxFQzo@O((2&-t6kVy^U!C^-ZiZ72NX*^>z45K85P6K zGfv|Bdi;Ho*g$oTv;n{{a3CYG4>I)O4YaE9I zM;5Tg5EmeQ``BQexVvn>NN&ShM=L|OF6=3UF&t$bdGPp|&`;g3Vb_DJ7+vh54R>XH zcd1y_mX}7A&o2xDO1?0E@w55O1tBm)Qz2-fziTLdeW#e!jt9GG++LJ);J*Clonl)% zKAt>F1h?noOz^Zp)hDBATXv!!z|^wf4~XLE%9wShSPLf;l>PJeJcYhK9e5|+uvDaU z;0evZM~Sa0EdW2YVoy%ImiX+%onlo7-rB8Pax|5rvuNqzEJzuUk63^FchdSZ-miA) zf!a`AdTy86vGl}Xbx>Pvw0)K-R>kvBzu%&Oyh>EE18PqFv(J7yoQIvf3avZf^sj7)+LR(9LBArGIuE+Dlg^bBch|mO-mGRfq3!CV<6nl9b(dBJk~26 zrTrzI6CmNN{L(?O@iG2>!@3~n?6CLi0u1n!Dc&tuhh55;%m(s$eCL}z<+VMfQ)$ph z9GoNeb>%TGC8a9I^jp!52XfCHBB&eBaw$2B*wx*@N6&8;C%W;1pl#mrIV|BTFdUF3 zlZ*i$4JEhd#EN;{k?+IpVq16qfWL@u4=l0$wu^>l-a%)gJ+_M!Gw-jj1~J>kT63*b z6@p64{2WgzwcC>T3p}kvasrrak=U8SA9GVg43xQ@!ehJwaWBme3$-5E<|!WIgm?powVSAN|h;52@`KJJcf5v!i( zz3r(3p$mAF+&O8BSURY7-eY#vh@Abm*gXdGLyT}a`7Flx>0Q(ABz zQ8fgU=AB~UF%;~$rVTT3wM+6ADFbjjKXWJw>bRyBi_eDg?F4Z4b3BQM7mIz*@%R{J z`f3emOod&gPPC<{1V^P4@>ekimetnU_IcEbD`F|n^Cvw^bXIbCvmH&X#~xFlEm6tw z;byRNgA_j#e&@En&wjMo=8YQpee!7{w1JRR5)SIHU9pXS3ii*v(yKg$y+fW1sSIqFmviO92cE}LNk5@AlHyuP@~n+Z~=h(XjM-=csg!Pgkwwg8i$v>HmZ zQq$rIdE9`Az0tq77(R?Q7Q069pIX*!-mjaW#l{kr>LDe#<4DjQTSmI?F%3#XpaP0UT>$=qQRpQZ6G$@OVL^j8+d6E-Xw zZt4b_BnJg^U$p~1M;gwUm7V9`h6%9lIT~ZG)F!5mMsKd6*;c`$PWGuYpmhhE! zqG69ko?*!(Zn? zk=VpxSygj^4bZIsr36?7ow(wENphn|QSdsi;Gawub7u1JIM-M3f)!9*XK21ku|JaSm3hhhgmo-Ln>BKci89+bc`@5c&Nh+;i zEe#LWA>S{PMaV3acOVbdtH76hYw}U~@AUGHy5*aoe30$*WHD(L)F#I!i^ce+9InYr zeWjlKhAU(bw0)wdr4sz6iU^K(h-`MmV?5<1lV>>OHQ~2qlDJB6H%%5Evw2X5^?KPb zzZ%@sF%V??0BO2MI=YU6b*IFb*xEjHHlM;=d_RN%+JJGmlI zK_0MwKZmz=<^3{*$9z7L|1wcboR6iyX@)pGpD!cY!>*z*hCI1hubY$}r^+!x3MVP;B7``G`xm2`>clGY2MfX&`KDI>@>B8vnkki3( zTMVVAXtkBhiI*ZIIraC0-4wa1I9PnM2v?m>2Mf2w{A-Vj2?W~5c42~udxrU}K1eV1?(B~D&~#8fSDq?YLNHec!X3OIYC z#a+b4xA}*415b;>rI?=!B(Z%d%AQPDG5Io`FzG_^8Sn6!9;>zZ57Ncecla3Z_960e z^k*q#X2{FAXO~K1OfJ96ZQaEv8E~3w`NI`BLE?rC?2+jCE)uK9tCie_VJVs`3g6{n zyvxht;JZB0rRpr&;iPOIYuWaUR~xx3k0+?fC-ab;ttBtelE*IN&(N5#FXN->8@!yq z$=`ZO(5+)`g^`4)~?{Ay}ubp$obj6R=z&AGvkE6 zfYt>NGXzE>zQ+VaTzs#*$74jR_jsnq%Ub4)aVo0i@9_Ya@(eYP)uV%!+)hb;UQPMy zeZB;4HIXMLpD*_)e^DjMi5FF*9xveQJyvOvE49e_EBP{ySG34UFRGOtUdi9{NYWyE zXpxy8V3Uz4i3=a_XqQ~*H-+aa9_fKwYUsDz(v%3%Wfh;wL6VPFfh6T)wc5w3Fdtvd z&D=g#ELhEFc)zbD3gzo-`_EX>un>czeTaCfkQaLl)6$Kkg;OYT1wY>|)Fr0`-=sW|woVhU0E+?Zc#|N=`p%YcZUeE(2~!w)KJ7`2-kGmq&w(4A#T1nB=(-YzFAzmqPKrTjs#NdbrReA4GZ| zJuIo=$MtZdDB-rAhge30E9E-t@o>gE8gOu*N3AzKOsSVRhl0n)5%OH%(Mk_5X(7+W zWs_(<{8gwTD$>AmkKqEc8 zLq&*=(3J`XM=A&!_3#fUkG(d|X4LdTtlb~-*r-#T~<-l zp4Ey$cqzionohm*+=o1DU}ZjJRG42Jfdohv7SLwK68r6_b~U8i6u8)Y5>m73fkhEb zz73Yg0K%BeNO>&2^Wkyy;vr}%&>G8F@x_NcE+j}BDyjGy*Z-ZZr>^=xl)VdFRK@#0 zes<0|fJlh2EVv+d5fBj(5s+|2L_$=&q-17@W@u(+-f!UD%!*a>@uOyFehM}->Z`7r zC0fRI^(^sG(~7#<)Qw8M(CF9<+CZS|8oyDfP-(vX{Tf8t_j)8 z`_KPJ_M*Kw+wCahE9UD_Ja!89nlMcADKOu33xXg!Y%wqUik)XEzh+_1j+=h?OoS`r z%o~H-fuK&#hfOYg9Bbi+yDPZ;8smGu0Kxq03XXK)I2Zq1x<1RibqD)hqbWD%7h@%( zDW}Nqn0N4bgyWZ&^Ei0iT>t4D;^+5rsJb_ZCohYpe#bf>$olVC`?i(s(F`!f6gsNf zH8vW6R%zHYv0W}_wc_Gn7l z&pJEhm%ZQ3_zp_`kM}##-2LoRwj!JKKd|5kd9>}=9LgpZXAt>)74C>s-SY>n8)!VC zWv}7Esn==p59}IyW0ZN|kL;<&=VgfgJ}7AMLY;>-Nh*%U@t${uqfqy1Hf5Kv0G6Fi z%Su=Z`{`9GD`ETCq%pMeAWLL5*>wCM3(YO&R|Pif^xqqWS7b4HN-}h|O(mhv-Au#f zA69?uObft`>QDHaHoKWtz{7H-i*}}Ipf%^DX0!FltkW%Me{7D%+serDi5A5B@W_kd zSPR;3pz&ge1v3X50apm74v0k8Yro3Cw;%!WMw4imjF}31V+*txT*8Tj5bCMg5I4}8$ zw=2Fww(%zx=GfK8s5WzKQ=R&W{p|R7B+rkBsd^-BJ<9rq7h?m<_!(>UAQU1#VUZAr z1$ucXLLGdn44V$2n}NIM5JNfNKyy3B-fiQUrFfLUL5&u>UTcoASR@G3sbj2z>qc7w z|4|XM&yjTPa~yUJDrMpJY9CPUMI%bta8}|)-;}cX_8GQGNGR)8Cb6qtgAhy*wuWhHS+{3 zVxNzufRpTHYr=C+vLOuRd+H?S7>lqIc@n!2$bO$Pd>PFcO_R%5KZZ=2%h*7kN&8dG z*fNvZ0k)4(c__u!kHDFeu&WkFo4FmOs;U+Qma_r&ss+=nndK}j!uB!w_V3ugaJL@c z=FM-if^CAw$}Vg%>k9TA zD@~*W6)c!lCDPprESWDRlG{1dF!qA`ox^xpl1RDdumr43q>VtI@O?wwjtz9QrM(31-%OJH(=cA=FRAQ;*2b5;{1c1bbS_gNm7e!c`YpJZ z!3|@6KHnxspOT+sh*Y=9AC91P=dqDIbQGOA&jPxoq19UOty#e+#I;5(BYuR!tEOl= zx@td+CI@czj+-+a1psYT49eA^yS%951=gOuGn`T`FdwHk;b_1%9EP@hnsR}KkBl6_ zH{qaviCJ0{%zUhAwnxD3_xut~t*IBzN%Ez`L&Rd$qsG0tFmu{nAl8BEbtD1?%8kS6 z=>?V@JR0zUqJp)VYIq2 zlxu6@ZEg=??;_A}dGlWDJy1}df--J{$7lLsFPpQku-zJ;iSWQZ^DP0sl{DUF=Lfz& zdzoT>#p0l?6TSB<`074hf>uz8b1f*CV7 z^zU{>fA!>AR}Mu@i#0|*wov#@=IuRkC3i57sZzuuH44ERMfNb- ze2WF!{Q)j#y-de%VQu~HP>Q(C;-fLu&370h8{bD9yjiIB30_8i#a-2xc*kWdK33UY zhN${d;Pd@_+H{*`TI;a(HjB-8rk39i*caEJr84!o5ndWR&jJ=?ycs+g!X*HQ?U2qw z9MRz8vhae#K(2p24Sv@qF{6Wvz@IfMro7)+x^5o&0X{uY8!BSqN3FLFrJKL807nnx zgg2_!hLZCg^lw9kQv4k@)yh-x9oEqoN~;>9ThsMYB(&>2L|X8kU_9U}>>U$ksC#_~ zJ-x$x{6=u5@RhmJLFm5I*Rb~j?9*Ba(MT$cPkSi(F7tE!1yr6m8cP^&EWZq)@poAl zb|Z%9E=%;l#V623<9!-*I5kGi7>zLkK`5^h%|tCPsfro2YM&{;$5Gt#}Aot1;vQ%N0!Z?IR-@{-R3HN&}&9Oaa7mRp;{d+7ptgX%72FOpp ztD1Z0#`&Z*3vH;C>h5kDG=x&_v&qgE23zIwAWqH)4WUE#F=TZgOc4(ti~k)$*$>$0 zghKe?y+X@2^!ecb-G(;X{B2SZ_e%iriA?guL(ov@k9vq!|0j1t*Y?L?>u0n*4_N~1 zF_>Q;?L*q%Sq8hC!S9TYpsL@Q3v0-rFMh}LV|fN=L1*ePm|q+n#_nYBJEQfaeZ)5T zL?chUkH}#LqZ{5<%t1a2d43^BzhzL_BXo<)P(F_^eJFSMv=$%Q zQ`fd>6DHI;;P;@Lrl=EI-&?(-Xyc!lhFw}tyZ>Z~(Q}dB6EvkK(1+s>v#8&W{X>nV z!4CWgkTDcXv(Ot*$+k->Dh+PMU-hY`kjFTfv~xMSCa8#};!}t1B_eeRE~q8yh9&r5 zG}4Ts#@7eo1J*!XhAOE;ERVrKo+&k1lW0cyfH)Q+V12JjQ_@q4#m<5juhDnx0Zjhj2=H`JuufZcY4BnG|ex2XpT`= zCVuAg*%IFWrs576>&h7wgOAnLkGL;nP{3a-JHCf4)|PS^HRylKW#{J9RdhGKJBZ63 z+SDL=^cR*%!XR>e%3jo9OX(@gbbU0CGpjW{1oB+Xv?FJGjV@M|jY@kgW8F=g5mhO~ zj9-hxn--r5T(yO}tfI~|(+;as_#?k~?1pj+LKXq|_%gvUAJ(B1r#MMA< zD(ww>Q=lPP`JOF|m92c0tw6ze3;idb+uws<{#GLQe*^3?d}%LW;@32#j=A_055ROe z5bXvo|AqN^yMcY(LfTx1_FOiAj@L1-*Q(m8%6>&jLfvzQ1JyFgf37LqTqcSCEE6-B zP|HMcwPtJzMmi+%IfRv$@i)sUsh)N4D{j_T3PGV?L>Jj>i|}qv8=MTq#4+uYSXy4s zk{qzA;qE1L<4@)m)(dPu!Sy~*Fp@TxJ4VL*B~ws8ydW0i5H!-5x15|C*a+A6(L%c$ z_6nVoiZCC?*O5*%vw?M%zCk<`*}y!_B@L{vrmKs$T7HH2^2C*+RmJE&c$orGC)u*O z1?(%h(ogB9iR-a~Vw%`{eEavSCQKR(uJlV28{Gd8qCE&Lo#r;U3ic{Ymu+ouXC8y2bWXpwFRv--W?Aw=M?L^ zVGj82p-!{MNUGX3SGiX03tK_uI)yLb(5RQue2uuixnZ+m3u$}}+w2p*@PD;z93il_ z><#JWM@+<$q_n^FfzmDb=05@jTLC*#-P>xu;cTzAcqS0^gY1PU-cGTl6t=&=IY1I$ z5<8VAmAs&~&;o=&i!Dz=)8KbjY-+aSs~)$n@2}qfY2Ils-j)nq()i3?Yf`(J5=S+L zA)xX2Tha!ArrHgyj0W(<7dd{Q@mr33%uk)fcjed7PdP#IxC)7*X-}#xfM09StSTf| znrV5QX8jnoqnT!KGFq$A*HqL@t8D$8wk?&8x{8z8=l!rC5_OdBCML60{b;qDn8s%J zqib%W4|}(t8ICV%+`vjJqud;g;Amv2=7H|wY<{%FmV+(*=!{N`Vm(sKb{^sz8ci`x z@)W1p`DWnd$rQTdDehp4Q)pc~@ust5inUfWI8EFwg_h~X;jC*4UDJyJ&Q*P_ZZ}Xf zZBxkEOZ>?BoR#+N7|`zbC6kwkFPDAksFxVeYLkh1i^KJI(OXW(Kz$o~SSZ>2BtLtj z20P(nQfR8T_@W!O9K4JLw=m42i{+YXa~IGdZ!v&*&Z9ftVrM6HO4-U!Fyb;C@@+5r zIJeas`83smw>%7_{o9Kjyzb9Klfca#r8wwW;vmFi1tb+|AoBiB;q*a!F;S-ml8RDM zsb~ttg^Rsw{igS&>+MCTF7ap-M8kfVN5MW~1lvEChWm)|t-nPM1wHZ3>Pkhx^>6+=7V zH7{>Itu7A5Hmv$gnTYLlOY;M4z4(}(PwBW#OX`fwP>D0vIP?5Ox39(Jk?z6v-^`^Hh zJ7MSf6}4vW!lzw)I`exU8t5-}VtW&5u0JY%c3(Q{FZO2d^&wq=nCS2_*xQxlVqAAk zitYFW5%DqcPoO(rHuSJgLDf|>XsLXOfw$AX4-jMQ=J#!?>Xu9&UzfU4NTArJO|jsI z2NLk(>Aus4QUW2={rb@OKru|$4jtIDu>-L1mC#H*Arl@ZDByk(C`P2@B5FS4eFmQW z@D#4KCFxAw$qH=|adWVZM}Dht1^n_Z8oQ`4{a}5K5FDrR}Si9V?H!+DuhM|@DCee&AwA`C>D7cfjBlz>) zJfPmR1=z?Jsr_vA(ce(FiMt`}oZeL52~2(5+Z-G&W{YgJjw#g$%Cy5H!C$}h%DNc3lZ?HxlYM{if z;`j(I#F;ou-7Hmn@x-6QIQTBMK+ra=HL8OSjbK{Ri_UaKOZPI+)2?DLYil5%C}hsj zK=DyxKyDu5=Bq(|mxfjOL1&r^8ea==>3HW2VJ**i6((xdNt(`7)5}_dkAa;Kwy=fU z@2Z>L^Z*zHx0yH$UfIb|cNca)*zqK>wnbdIxJuR<*>&LiY6*LquEM+wX8z%=6p@GU zNFY4tRllC53gCi&F?C9H!?nsBKi3$0gYoG+KA00>p342VsIkM`-||?)-YvEO-Qj5h z4}e_Hhj}~?0C_6J)U@^;UfQ0f-SC?Q`s=-EbvH50xrp0Y@47K|z3YzlqT}7f2;BzI z)Z0iMn_r&Snm*}8&e7<*7O7HTtTC0 zu2}R0bJnKAf|1ueZHpE&rI%n@*If)(4HVg3^maUg0DM_^s22_IE_URdOKx|uD_fOl z{;s#(O-vMjggRss$0h3eRwU|!Hm+^ga5 z@mV|S>YkL;TTE)R31d?wj^|?rTJG?2qIpek4E!v^aR}0fSWvUW9muq$)etQu^NJJ{ovNk@z$)}5UsBcGDF^w!SUwJDVVBs1pDpOeMjDh#`8XT5|#~m z^ax<&Cfqn7tL*hj8a#lgJ6 zI(O6BSnC~a1=x?ChU-;BFkj~K#;f2bxbe+7zQW~$-pg@h86YM)PEhIYCi^%_8Hjn` zH*Yn|k=8UPj%EzRsPkp4IT#nAYTU~UxAu<9am1VUd@j~L(b!F$YGjCUY<^$z9W0)6 z^itz;HFeQ|$17x@q1 zp#*nESs#M7jvR?S=>8Clk3)LUzM*3O)(fLlEqmmzm|TX5UeG+wf}SE)Z3(gFm|@}q z4Lch{yI;l}dcJ|!a4{-k3I-Bhw|p<7`GsdeqyONI82*U5#|Kqygu1x`*CN)La270k zxVVg6i=q3&#qmD*@KYcO-Jhbq)}jCR$EA+kAD^nUkDHM_>Td_aNN#9?ZUtA)ic9@4XV~@wTfD za^QCv?7UAo0pu%__VjKhWWo?fA7_dk>=`DkNz>x!`%KZ_YAJ}LThA@pcxwGhvx^~~ zy1mjIW>P%8{@i6=Pg?%mvaKh5^V|}WKo?#S+gqb9jHia@mhyP=9Vxc&sB+-MZ&TnO z<~>ZGV~V)VQCQ1ASa2a#-3>iy>`3t^Yc$}!%d5>yf%g&5E#N)(xdps``D!z};Qj1# z7x3Qj+ydSMM>WR=-uuJSNli2)0Ze(TEpRcI@&r>D_W$(Iezu`rT)08?gDlag=KLW` z-#@oF^`z^g#CA58dy+Cr{59<7X!IVxKtLC_H8OvGW>zrG1Fh=4q4*@BQEM#^wdc{D z>_(%1Er46WPk;O8g)%hSyl%93NE5J#2RvXKoe%e}TfpsK+U#HTuA380bF#!C?2TwD z%Mt@TM!`L&#r|(5?8;()^{z(>8Y9-QsohAQE$)uG-3>8))5j`A|I??glH0$x*$?!o zyUOjy`qVi*!ExMQ`qW+QMr;>k(4+^-0GYK_owsVhS|$&k@RSanB@9&4Av};!Y%=o z!x!JjA}Q%zaW|XOh3enM!fbLE3Y#h>v8FCGd8!y6upa(-w<3qgLp$# z6%!BBoi22Isu&iq6#g1){;YAknO=rJt160@>1-FX?me+19(|0XUem;YfQQY}i#wDN zr?(`my|GxRAmR~E%SQP zXWU*AO~=~D0g*Ig7Sfv%L0e~u5m7P7EH8k{KF&scvamKD zadx%QQejrzAi?V~#pBgnz#lZ5)wgMNHX5Hm^Ej$D z;R?iiNPJ=)>7C5xgEmJ!`DLCxnTH2 z(EZt{@E0N|VvZOPCUHM&u-~Pzr2{ih$I_Xn<7G;Q`$?49i7w_jbHpcND_rWb1=Een z&gK~l#Q%yt%9F5{a3kEhw77vDTrJs3!FUxkK7#X^QlU7ymQP4Z!AZg5(X~=I4Ot{6 z+9klxPn~JqB5@r%*@+?+ivjjdpnZbW+pNPSOOqFiNp>}z6q?`J`T^v`VljtRM^nra ztgLQE)8r+nqzhrRYzc(m?M}1<=r|Su_`qD$iLNaX_c6y#<_$~5i5m7znAvW*I8eiS zhSBg9Sd}M*n|G`bUvp?(F5vDgK9DzbH0zA$I@sio=<;guGW)6{?Op?~pL8VWwc?i6 zl_wdjl{PL&q1E-adoPe*=O7B+CeC2*2GZtjqE7!-AbOCF zhB_!jgcXXf*5;_y^=j=Kfm8-}_IjZC&NeYdWbT3FQz));eJcQuY%wjaV#df0-SIaA z=xCuhP(KuOJ`2Ma^FLSUKGPRs+oTeTjROMk8ip7)Tnk3{RR@KOUsL6RmqY<%Bmv>L zI#JcF-s;S{ic3CbfQ3Ll`-_GC+2rtzvB(e*3&U|xX->QlE zgcqrFt?3V7b9!&6>A9J<+T5yu@1nN&GQ(`QU0lsr&Ihz%2O7o556owFh)-oU-j|N= z6k~!jeNh?MZ7H;1Y`~&y80`FtQ{JU;Uih?pkgqv-mx#B|w`Eu#66XVewPS~B8UpNM zd~SscbA4#TZZSsp5&ZFyxdd83&HhIF(w*I6cH&VVlqTj?Ji#oC)Q4c_3G(mTTK&^E32s^__a})uC4TOv3DAUS<{_GgjZ;JZB$ov!Au6JKu{D&THCRmqUDaAiupi zV86Ip!?r9p@BTsD%5;mok;z{Sb)JiL)-Oz_40ZFpX>y6UAk2QCb(_xcJWqPl1^DM{ zh^LrN@$rDK35I%;?w}YIRgEpPhkUJq{N?e1kgNmY0fgZ_T0Q*oH46{(3^z;0d()hQ zVt{K`xbu55AO6S(VZEudH|;nme%Obf%`skrVgR?+tb=ig1qVR zA?yjBhTAlpw5+@(8J%rzT5tZcZ2%2BEcR?&&HFc=y^&tD;V_27R$lbuVX-q`HG8P* zojzXllsi1uL$Oe7-wzI+szW5=&TGyA0?42U}&l-Xz7-JNr` zu#K>*3bSjzmgOlzCwRdT^y8=W^u-Y@+K%bz=n=6Sn~-2`I3jk?j6R5(F<|uJli*cq z2CcJws|0$OmV?IokLFc?)vu>%A?Vgx=C3idGtGyc*CcOVW>p&9Om%u%a8%U!JprS< zkKt~(JwqL@9)ZT2do;qS-!ACPQSseY#c^VS@d?&ylzt3byMG-re|$`=bM&smb}e_( zwu1HxTDbcdAWu~Vsl?2pR?%lxlbxTD^9YS3>BhQQhjAsh&f3)9;ok+ zZLJgo^EhrtgHc;8T5lG&e`&LSWvCm)?L%$$5r(=<4`S!UFs}ii^G%ZHn_}Q86*Q$! zD|KdT9(PV0$8;Ik9pbspgLa>_MBPn(9(3)37_VEcvqn*f>*ZJ6=JKsIeW)`>Tol7J zEJ;TrFNtCL8~spB&pQNn({~6o5JlNl2gNMWCn^;7mig(x?Je@-&+RSp(_Tl9E{Xes zpMh7dScU=UItDi$Kfg?Z$IZj;Q^mr=RN_wASHuX{ojiOXGd8NPe?ZCFowi;PQ=B)0 z&dZh;$VQ|EzYZVUXt)qU(vsn-=){(~n+IMMkF)H-JRWS_4FWdJ{jumtYwY*o(}jcI z-kGI(q@u`*DQ*1+48p5{bcgq|q+^6LlFFrIS5C^Uc zu8TqJOD)ykL~Hs?OLwkgaQ*-xtsMVTM+MKu4XhMDbW^_&@#bjVkI|WAEj_9fQ$qS{ z@dhToX<3V0XF7_Ati>!C@jQT9yemQDN-R|k}JFHX65;e8|iOhS{~4kCf^cYX5g;9O{^gE?0##Yk+SliBPkWy_9RosCNb9R00+!5ru#EtTDOh^TqnoTvnIA`|S7 zsp+QZ#kPucIZzrI_5nzRHZKM6vYdNS8~Q(+%|ekz2T8-@-}+aSZ$9_0HI0Kg2Wsi1 z=v26x6<5J|I)cwJ4;Q{BnjZy88ys2qTjrHrq<8Iu@EMw~mL(HkXnn6(Xe)*oE6qWr z?FP-ahqP2zihAKy*s`?_vCy>@QUG1;A$fP)V6k|lb^W_y+!4}RF?h(sVo`sBMWTeq5(YO|%%3Gm?KNU6m=5%j zUXVn&-eNx0PwJ{+d+I1+ixg#!eo2}q_H18op&i(u;}VCw;G^~57cU3E?VnPJW>U*J zb2oiaXE8rWmlleSjhuphq`uZXBSRV~vn5Z>XI_@#HLUwnv-1dPyv(lDm}iWV9MMQ> z%!^)=zGl6u|Go`cR28xo@-Nu@Ht56r9!Cl1NT~sxLCYC!>yxp(2<7&PHanIV9crj-jHF|}HFRf;l@M{hc<9~f_zz7RB+c+C z`_rQC=5n!CixLB28fd&9&HvM!^tSYohUGjqpP4KzW%jrtDBk$fpXSl;N}&v=xdyi} zfAF4k$-&X`yH&y(f2YawBs~87J8hmPEeH#EWFd`TbN8+_oq#`{)HJo$bf%YhI3)G) z@02oMN@EWnnpe-4>@;@qa5(&kHZPENu%8~%z+7o0+wqW0xza#+!^7t#z6M5=a4|1& zEz+`;IKJt}8(kimqw*wIjpI>x;^Kegp?Tm!X_x3${qI)aaNpYMW0y+aA$=ZLJf2{U z(L7N9_~7qWp8zxGcM-6~MZVjYN_mc}xEqpL`G8`UNdu#v-G?-_knbnl-a@|r;Px3d zdB^hk!F}4jOp15D3VQRL=+b?%ER!OfPqv^PzfbYYrId(0ptbCUcfxM%Vzs7kVa~DG z;;uK;eSM#HESEmAKMndw6bJgh*Qj8Hl)zf4Sx@XR_`bzM(<0X=;Ip}}OM_Vz2>SDL zY%68Hk0#djK5c$qN^B})7x#w}2*E44s=wKIdkVB)`Qw|u(%pM#y)JmuiLS1c5?s!}h4*WoNNLMI%Rlo2 zl)pdc1m*90pKg92r3HKl|1Hb^eQs}2{!8y!`oT=o2YQ?{(KgS<8}& z+S}f`dmZ+wDhu7MBIk9|NV^o+PgT+6b<&LF>vvF#|0=7?pv^#8mD;#1L0O&S{{B@~ z<)Hn_`wN#0TzvKO9ZDoA&1E;Jyqp?=OOdMeqs*my;v}>GddUkHIsax(+#o>>*8ey2 z=#A2SNxX9l4E2*b)2sq%C;R%A8N<&TY|Q@}0>=CwL%^%IEtLJQAs`#HyV%^}&GDbZ z6PD;Vt?b=Kn(8FUJY}17gt1OH%`sncu5R5h4=q7{Qp2dyyzFb~p!hnj z9BO`{fT?2=7{bX~j7f|6m?RceL1*$&rOJ?P*2dS-(2)Uc4mM)`FNTcQtqjeqG=K3e zdU@$Ec2+oNq2>>F;>LQfRCHi`f3Bb%-VDNY>m~efqu7ff3;WW6UD9yfYQboB6HaGh z!Z91adFYTf1qM^6-O>P7sG~W%B|UTUqjkHbt?aJe9B-BmX#BH+p}WE_+VC)igS&AU z*_V`bu+cQ8j+La}EBV;_V^-mx7frqPN_uw#?w$_hXIqimQ$msQrVtm}jwbJw-eOSG zyaNARO|xOI6y4?@H)owd2@p)+RFMn)&iR>37)|hcfwOf}iSfGe!TZ7Mx1s z_Q+q&cS4azxTQes`Idl-9KV3I+Tm?P^<`l*dC`Z*8f*Vy#a8w66Ye0hMQu0_TFV$S1KJ!I>;l1zXQO3Ea`C0sd-e&!VEGN?tB(l$Y+7L z?5KGb8+DmB{452sF;3<07S;lzhTIyJ(>`c))lK&a1YZ zJ#vlHd##&?CtQYoi9R|h#jto+I(iZfx{E74J}H&4HJ9m7nKYfB2gH>Jr=+0B^3dK< z#w?s439~5qX*c|gI9y+zgE9TyS3;2qhwbluBNY`b)|@U7Hzwqb=}RAf)}C;vdUO%$402MD1?vt2UMnGR$Fnj93p@WDFY0j`a)lF^BTh?c z&i~9deZ$?5y;G;9u5L#Wf#77lYYB3Tmf4Nm&PW5>tO>*UC6vuAsR$2X71NY6k{{dJ zhE|`E;@#FDRDtaxFEvRdb^VMKZDrcytQ6%u9ZGw*G;tw_lWRwgI4gBvZ?>TYXCe5S z!}Q>+G+I^tt&rNWV@@=_Lh^52-K^)^^&+jVkcMbqzrdx}&D1P7uU^3LlGjEPD{en; zq3_#hI*b2BLwwLi)0?)PlWw!&=V|46DNM`&wO3n>JHI^;n&Hu^QhmNNT|O@bGwlUp z7o-68{do$$APqplPrD%Xhsvv}ystHly1s{k?!wxVI%PNS zlK7_h=V+Vv(3xX-ozG8)pijkA-+DUaDp*(PYjsw7WgFu`=&hc=U|I9cn>hM@Nh)M> z&Qbj(=?q(U9(y@bwomvuo?l;6Dbs$SLzQMh;H-MBI+N>78hAyDXS#E=@(OCO;Rczm zND;m6qa>2}inqiUezm4oVdr}~D$Nhd`>F=9)-)e<&N!ceJK&8Xs9n@uL3--=iQDQj z9cQYkyPKw;qmQmiQ64!q`*;;SOw({+;VWAR^TVrBy2jDb=EcDpMAuNkGvIEjQ_&hE z5e#3G^uC`u@t2;t(wCijJT4J>(fn*mE3U(845P!>q>tEinTB4MA{~nF_b|k1tV$qp zbmY2}9CjIT=0OFJk4_@Yyr(fh17%I?S~E6=u~T9_R})=9QI%45yH0jDq->U0K@)FCF)rLIFFT&3 z>b3ZW6zHaU{fqa{NI3^y{yax_Zb<$;iUk8dd!{kfb`#?GQgt6#$--k)TF^E4aiy7^ zi+dK$0v!q+SPkh+v(HlEO^iC%&eF)6sNa|hnsZYc9&;Ohc#+-&?u6EHL7?U|ujc@C z2}$VwcV3wbU041#6|Pl*9|-6!w+gbm1qm-ZOJTR9;rczcICcZI$S&84uW=| zh(&d0Xy9!rz+oOD=PiZGZ)0mK|F-00jknm7KEI9DKmIHozbyqjU$Ob)d)gPzlI1oQ zj6=_o?l&pGG0nzk>RC$qO^SDnQ|)f1+_SXoH)*Q_gMAvWtH?W&SHI%-&_3|g--CsC z+KgAN5VTPJrupR=lv%V!6_uxp-Eh_pSEJxEC?U_^cw@1*@p^!!t~6LvcRfr~SAB+7 z-;vTnZXk@q;@4{(6lJFXrL13J{|k+~dEIjfcAf@Fcco74@pMM>LRM$`nJ4!pUiKW? zyKVyal73hE*d2=6xT67|ma)IR)A1$s+K^(ZzAKgLkDtcuQj9ho3}}V+i4Qnw3w4>A5Fh8R5u4=bv{@av5^NZ#_?BLCAg`jMn}t}h46zEE`w>|N(^+Pv(M z6r*9S%IU{Hq$uykQ&_Dr?Sm>56%-umdSm-A(1;x!E*VbMlDGF$_}~TE%;0_4bL@m~ zk^6jea~XTYrjJ4P^1-Qxsey#W*j7S9Ps~eeeaRXkrayea0#C${K07 zhWVW`-~3C8(y(J?qS4_G1WY(xX%-z*X!Z_WBRTzjhV z1U+uRY#`<&g*Bq9zkPy+G)mvG5huy634S|q>J@}$Oi<6#KX7h9lL4ZWWy(D~6q_oot-wI$mDr>W2p zNB6`9HF-lmg&(e3QiqPBN1jTGf&iNY_|z;iBOPb(&G$h6Nr zi>hv6`wa<9|GZsMDklBBKT3o@zcgFI@W+eu&*NqvN&ZmdTYYb_pW17kJ#HaG;y>A2 zcASpO@@zMJ!qPui$K*HWZfvCCc5)hfxq`OZ$pNh2aXM-zk7V5|D8OEhXTcRT+Ftfy z?T(ul*vm}g^DCNiAu{KAGO4Jh0ZMdF@X=9pgVLqeboyu7>>x+6pU=`6j{DD&T`PG3 z+l20-H6r`)EX6y?dbaRqy4FfIdAw?i%i%OGdAlXlrD7Tcbwc{gQI24H?C6xEJb`sV zpw@B{^EpckT7#3eNTjtqnpw_RBmLuyHPTz9JW?G^a*{{oj{n(OqNTrE&`bBO%ZB}} zXBf!(KtH&TRdw?!Xgwf|2HpcfHSbrHTQ}JGZiEIsQU%;G8W$&Jbx>P-0Ni2EPb%qU z>;|*y52`w+;E#XFQT<|Fftf%Kx^-WZuGtf+>L@(%y8={C&_rqTJhB9%1_}DYu5C^n zk?woWOeFHlGa%kXlmlM(S%?on7m|O(r<6I#o>-=MIDFm%RhY1io z^ZOAwt>yuam?DA|7>ap`wELMXFUY#lzacDl&#?KcgaE(#zCuf3zh-l*gd9HkLZP9c zU9!=t&~yZU!I%NRP^{H*spIh!V1m^jG$_?-%Z}4jXE}@=K2DpRWqtPjQVX>Oy%3YL zRyY`kSIv#BUJ&ZS!6y&f+$9LG^N-#t4QcB?wSEiR-ke4s@ic$orMV}pLA+eB^*@uo z4dm;WQi^ERjYJPjx zUxB#SY_YoVP**Kgoiw~17gLRk?8h_$>Dq$B#8QfHE3XUep~0jIqgUZy%C5hz7>Z_M zwpRts&MK%UqpFJq7d<^`E61=rJ#})Gcd?LCy6K8m(!riw-Q*#Rj+w{1$?KSNG9+|K zL86gg3HhLu9Ch-*6jf?%*Nm1QYnI|@%sU-SCz|cH>q$#tFVr}z*ORskcjy>x)*->) zj#8OUUgU7(C@*ujFpBh$-RW%)*~8JSlC-9#qx6x7tarJGqTn}NSNWo8^Q`ajki*@2 zgF${?wm{UJzKP-OB@6alJ>=Pr^QWKuYmRg%BHa(#&0`dxM}%KBM_6Es@Y+vm8JMpAM341|@T|>#8aNjL z=lm-|Cqy_Mw2j9o(Muj_uNKe^Kl;o|)-g>xGJDC>dwi%y0tIMbAy(ngcvBaj53%?x zv=_P;V*dlHm;BH%9dI5_D01MPCYEYrz2!=6ej2DD5MjQ~#i!z8ES_mZdwGmwm>R9S zDdZ{=m$(Qq6>h>V z)!Qb2*)v|1`#cBr(4&I%FQS8ce&!#E@&v1Ro_MPn?EgFw2Y>p}Pww!~WNtTDlfj9; zn6)poNG7CZGMA9dcSzdOf^gkWs^@uV(SmvuRL|FzwHP68;u!~fC=1t+yf+$mjoF)%HA%0 z5k>PMXO%jsqr8fx)>C~)`E6X1j*SMs*>`g>Zd|CR86jX@dx*XZ!4&4!A-WSH2RpuO z<7VR_(uK;y*@HTo94eqHhdv5kY&@NKZ{U;h@RW2?Huqe1i&7QvLB!_xs{3{wiTwsXR zLcYJZ2R-Q|cVJ@=nf2lFQahL8wTE1cf4$0zx}%T4IZ*m0N_J;)2k2;&Jlb{80gJ~? zbx%{krW&R~2PmeS97Z|ath~NL<7x_R@1pG1D^0M5~obj@(?c@|Yy<61u znCR&|7W1S$d9tJV+Dh5c->zzfq7-5AzBzbYY@(`1|O5l2TL8t+i1D%%CE8}#6PzX%LeCHy!14EY9D|x_KKKC|*dju8& zZv&42!{4u0oN%LJEHD9`Uo!Bj9~c1E0?UC~e?f3qiG%|LApsbI_|^m01FM0NL4uI6 zs$R(f)&TQ^as|QfBP5Kuai+0enbZ*;fliQ=bZGa609OIifkA8Ql`LQ*a0QOvjQXTr zxefbvpn$E~Mqml(``00Xg@O=C^~yzH0dT(oCAq#{nFRs94crdQ_)op!or_gnJ`7V9 z34-^h^-6RyIt8F#iXc1$1}zf=m(S~!OxU&CkT5U-SPaYu9)WxMf9sWs<$_@N5(%Ii zn*}_P4*!MqO6B{gQeeVLBxphvzbOc{z;yIf#b4DcxzZ$X`gOfx0H*IiolF*lO~6&_ z!ANnvG7om?TSNrA=l|3zRj}^?u6;)kjsUl9Mvj0bz-c>C6_BTGKqptwfqua7UG+*b zFcLTgm=0V5%-L0+tE`9N;%;z06&&rUSE^y(zZWHe!99N;iV_&|J#xg|fe!D1z8{PN zSNs4Of%|^oPT0r&Sg$k!4cL1t2YnAvf>8S%tXDFo3Bsg9@DI#PY*3sq`S&*DHYgb| zB;$&jSa@6o90m-3sXMs&>XK$ zhCiU^I}OTuVCK{Y1Q zuy{j*BH(04>Zc9LBw)hk2E}D2ssflFn2Vf$)}ZWxgRljh0sTG)Bj_M@ZUcw#==t9U zB@d|mqCqJK<^Z$MUr2?h5@10Q7z1j*f(!&hD7QB#4zo}Kzzoaff>M% zP;ht~bl^5%K5*7=4a!xxPq~Alg+1dgqj?;!)|2`}D9D?v{v{SG_O4_F9X3#MMA(T zpkD%tz6d!7W^G5tUIe4x!2TkN(qU(#5}Sy+{jpJ*1}r^}k^oi%tANqJpzeABPc$kS zzzpClVDULH3Jkh{92y|yKtEt1FdDcYxGfK(;6*S3T>A&AU?CS)Fbv%G7~ygkp~LtK zMS}pFfTMsePaBmLz){Z-5O}d3403<0NjcFQ47P4kDq*j2YEra|Av&&2N*3^;wn^C! zT;$wC)+ObL54niKV1h@@Y0o)G6 z>Q5*JHUd|LG%2GnK?w?NQc@8vJgiAs0($~*Junrx9XJkH0-Oh&wFL8o&WMN8yEG}| z`hpHz1zg*uDOcGE!*&p^0*?SsAV76wlj6A)33hE#q!d&QFbnpKZb$&QC%Q?w4J_&2 zq$I%o1n?rzI|d0ZL*2(VDNg;+?t#g`YG5PiC*t4^dsTdslD-@n?wN}s10f`#NqGo7 z@nVy*77p_go0RBOBmm3+Rsq)nqkA#6Vxt4|HGwF!_D7o&iltIqb!`gHT8PkudNGuo_qcbjfH^ywg#nz+~XGAx%mi zux2Q70`wlyq!0B@6=3H0CPf&863J;&rU9!b zf^lGO?OTW_19dnVbqN%vzyUZ7xE_;6&&8-apms@<5LL9fL#P`NA8?PW zML7bD)maqJd{lKOi;{tG6~MJabCF<#MX3Ox5GV~pPu0buBmn0D^MDtDIS5!8iPb6W z>$_T%N}wUiqPV<_Is}%$J-mlS83wx{&Z6uAI`y(BPDW&)uSL-gNBk)kB@6aF{VYm4 z>^p~ofm|3$vMoyF2qZWb9)Kr+D}d>5Ta=Jj(Sc00C=Y>C-h;nU5Jq4YaLF`_vJJRx z84^Uq+m^#0?8z%FO6qH1;3Fge6!46u%W8z*gs7` zVBZfcg}w9wWZ@Iw#awuRVcI1m2;6?zqLjm9&J}oAha$XcQAPm^fx;Uo;@e0F_O-vE zD1kM=5P}fig-pP`@*y$+`>ZFBq46l<8ZZp>tg|R5Ko8g8=1thEMBHHu+?gvWO30fi zdVCt51>A;PxQ+mi0KIcCS~)7pJYZvM+|C7@%ov#QM40acUP2jV1iCjmH-70 zMcEGA54;Kt_k{n6ND!C-3~vX2z#KjN0doyric$-M+m%D`EmYu`kP0$1f==VTK~ZUWqaMU^H+)a2&7#xEA;jcmybI2BSc4V9KI4yawED7S%Yfu5fsJzzL+J5ZBMOApJwVbiu^cLd_Y|6K43$1fi=1Camu z;b*{a|5hqHjBUcot@QY?92T|*e!M~Ff6d`aTF}r>s^LnvQpyo|ptAvXNRzM!dlpU_ z8eJiKQFr`v6;AVZ@zL!Ya_KXiPnr+#he(mg0`e9`=$4VQ3#WYgK;@q&$=T?e@cRHHH=(lLPE;!(1NIKp#KE7Gp-Bon+^P92C4w zWyj<&y9Aim$B^Zi9B(%d=Aa&wSSqL3mBL)nlU9}@hHBN^Ks_wT11}yw8C_#fEV*dQ z9qfWOmCA1R!$5ONXrR`9b}!nmhsE>725MuG{q0M*3oN_{969UNAY@LWvY%z0os%6} z?j-uLOx6X2z+5m1J2CKs>`h6v6+;lW!Y$}UI(h}lLDazaum zTnZ2>8ot0zMl(BHoNvQz58NuOZk3v5N7y;L5~QW7ueAcgVZPeeS~K>;U^xNHAU`{Q zyLqsb^rev}iFmL-iHA8cRJhm#hO(Fl|vae$-tS+t4c2num2|30t9p+JK6n0V$ z2*6d~LL{O{h95AT)f@%SKt9|IX*Bty9A#Gub7neyaZ*mSb8y+f~9mZlYRuuQkHFexj|dw-8NU-=3|l zZmBT0tkF{x8wbfLjgR40QTu*)3oate(i8Et==`5rSw|z|Cb4jX#Btfje#f zoUF+`Zza6w%HcL{5bZdHqH}_WDH(MClpJAafO%F1`IJNSSHV1KC`~MvhXmY)xn=^o zqJPDuc4;r5&wrV&m7^&9V6Mz0OSzobY8=d&uV4>|zU^wKrB_bNzIFxN?<=(6w47#F z4YS`UDmyI)+j%>qBYBPLPb1wNm_tUZExpwen6q*qd*pvic577t>nvC^>0lX3Vn3`q zVdXhOfj*ZN-f%9$t@<_EeMTN=7v2UP@;Guli~evJ%uB}e5(;qSC71_$&3HW1A*TeK zfcfg1&n5kNe5&lh?Z}(-=qv<)+g)5Bk9d#|akh$u*$MlA<#Z#|&YNaeAgA*{TMq%L zK<)}*E}TinE9CKZ+P3IWW>d^L*~cyx=9)P)@*Mcef_chZT67K_Pyx)sd^&UvLV6MA zJ>#jKn_XN{x^Gg{dH9coc~lOKJTHf}%7Hl(+k$K9^m!=Fe12Z`cg#c-+dJvR#w)mT0hhzNhT|ksbmNt8Nv|8>9C{po=k>h7}qciI}F3HqaB7h zh9S-lLnp&13>}7HnEE`Puh-+bwX3~vpWFAJ@8@;9ZP#_(pTAzeuGjUte!i33ac1IJ z4^Ywlhds=$?459|aFctOW#hbuKF9vsiuKa*-8gi7mWg`)@$=|}fpSkX2bYCoys^#EZ5W8EuwFOWduSWRg=Va8p6tD|E&3sM&A#Xh)4eOVN1rulJJvI1;7V}7duNE7 zbo&N(FK_S{(Q$5SGNuIwc`08+&l!}5^+gBcvT~Hngc7}VUtq{<0IWaMd+`e#Bs;O* zd8qf}7w}#&7KO|7?)?&Ti0xRPG1vS2OAPZlDHxvTd5gb-PB+$9WqZGW6}{Fi+z&(V z0&i_Q4s@HbJ_R4uZ;w99oq@4)`(khU*Qkk=SYLXqcfr><6jWoqGRJ%LYxwWLdi`vtfld`+w1Qxvf~Q zKiAv(KNLD^B6h<(Z{By&vj-MoJ?ng2QD%X@gR>^E2C&9RFZQ+QQQr3NqD$O0lTgZ) z-j&}+UpA-{>oK?hpBU*~9N{Kd4Gi8;Enk) z`m#_N*6VJ^)o|IXkYqfr221pC_d-BkY!1fiEE%h1Xp%c6D5vxw!<+gOvcCfBttWY7I-`dVDuZ$nwhNy1 z*8LQHt0`*oG!*?Ld$hrB>MX%_`pKd{%3X!^+6He4cYNwS`!hOT`T@}S z-246K=rJcP!Ft`c-a<-caWq4yDN`zn zf@}loT@E;?<77R3k>0f(vNhl1heL46D5b7h=Dpd4)7NyY7h(Pk-|kYop7I;k3$ecA zJ8!{n=ohtEANGT{_BT|}4y@86Rc!>=5$SCd#T{7h zj`W`089j4g+HCM>+(+={HlcUN?}|=ySAkWY>do1OimAkUhwELuD|*h5cC5Dy#H|6l zL$n3F&pty3`Dho$MY+q?6zlEWg<*CL*308>pYLAmc3`!8=uuA$dn4WP1D(U+ z7I$_kWDHL89v|Sw4MIn8vcU5X^J1gi>4Vl_z3pfeIKq2kp}UWFPLzA-pjN%ENm3)wq`v=}ir~sots4?kKkgF!fsR%4m1qz%Hye ze~7zQcKH7!+QsR^Tnu?zJlAz+N3X$ps(0J~cfw%Fkvtbvz7==LysLkU9_npy-6VHA zSkqST3D=$Drp-fJZ}on3-AMzMVm+<(Pw&%scidhw%ohXH08ING7h(|CsS}Nl#d7wb zTx@q^yZJNkwE^yOH}?qasoT6sLHDEq^;mEC{!j18M0fu|vMY1~toXruG3d^7GXN6* z#5)r19qtya7w_ebNpi=Vo)$b36&vHdI}oKx$9hK0rr!s;cSX9***GR8cy|qUUvx{4 zMnewwmd7IhTd+QBg!e$KJ858W0dg<`cOGojd>8Ac3|a)X1uQwi*5{l!)MqJR%_MJm zoO_bnf%VQQ-g9xN{EUTYzVw~DI$f(gH6HC=23Ro1J1gED@0uHGhkG02-39KjMJRiw z_es1v$(0*wOXqsAL)_&9tFT@+SFYlpE+gP=L)-&g=NRny^Sn2QAjl%Dx6i{J90r+_ zfFM+@xU4lE^U553TCElq+-AP`~P$-rF zv>1TnG1kjZ^fnH4Z*Z3!i<&&m8#BzE?UrJ_=rnJ+thZo2b(weXFn7M2oP$&@KTy>h?~tUGx-O}m?3lyn?+sq=Bm!<#$D9qC03cZa)6z=A8iF~i;Q0}8Re zXr(pq@`oc^H6V?cd<=JQb+eC0?d01=mTpym?UujV?N)48VY}iA@3Il@)`7(*U@W~F z_fEY1gP0XvnTX2SF6<`niA3yGNlVaoZ^HexDBDD_Bi*Uv3jmkgwR2Z}1iVGoM(jQ! z$WBlOc}KZ8nCg{`MDCgaith358i{sHJ`r=3`@L^Qy7LFD#QK)|ckXuSvlIl=)8(Vk z%8gjBxZm3->z!DyoaenL>!~MU9&x|--6(hJpru&PdJy-3Fs^QkPW2Y-Z>%7Iu*oUiy*Wvz{^R)}F4?u0nNpWwayC!4QwA==dL;~2dV`bQ5nlROH`OfxOUv?39qY~>(2VuA z7xCI4=BnlltqY(9w+wSp9F(Weh2&wUqoH2%J{jw#%+J7jQr4bV_mW|@6tX3+;>M`; z(Q{CW**9M1wI8F)IcuP^;YV+NihI1WjythaVQ*B#IR)(9=R&_AG^Cg!St z{28dHXnar$-DVcntE0V$RIKM>y=Z_p3F|4vSl=)Jcf@@06R|C~atjCGIe2uN2CT0c z;N6hw9vup-4~Vvtv`@^wj*KOe*`=trE#SfTWDGz;JulJA3ErTxm^)+djgQVAHYhTq z_q6;IH#=*uK}+={`@d!zBRa24)4ogFKe339<&CC9IOSB}Hgv9>90 z+TPNPa<`{x5uSM!;WIdMvX_Kz8t)z$w*+HQ{V{khb~R)dPWJvi9_?EbJ18>g94~T$ zJ2tis3R!R9L0d8Y%9}pHof+E=*7zo#4i197?OG1 z!`|XFcTAjvU3tj}Jgx_M&`bALr@05kWdU{##bsoHXQz8FrMaWy@&QYsnS(AFJR#lt zHqAY3NIhWc1VGUof!uocx8UYT&O|ppwiB`y`{Thok$sCd2;A{Q7h!!0rh+Bl-NIi= z_dcEI&W&9Im@ySkTnY?%4~g;&tk;4U;O?2>h&pYOJ8{Sk!1lv%nMsrnrI(mqP9a2vl7)Lg?o6K_wfGiA*TLj z48$q*NIY;ZUapwt{V~-YH#7|!DtYU&!ltm~BHq2Ogw*I(`LJvkM1oeh6e zuxBi}6;*-82)?wxY5MV%1N!tKWCuH{1c!3wJ!ndlIC!@=HQk*T+XUABFdj*jLO$lL zO?T%fBn?J8;mKgrvKQ_TOK7(%q56rElazVa10q!g}yPqhJn~8*o!8t{TE7 z2+ck?0(m{aJ!rbzSZW7M6pK%%;6KuCC)wh4SS%(t(8*y{8X! z$H#6LRsdV;|y5;_h8jqy#s58)vxV;~K!W zT#qLsWem7(itLuj-oBIFXs>;Sn;0821Y`6@Tz8XjH+r!Lxr<}-z|wB%-c=ziWlV(fC22AlL%yf^8EdcCz zlsUtJXQn$Rwh^rPF1-9cUT(uUqX_9rH=)F42oe$=rEq-pz-)$HcV*mIInY>b_~-w*s@UgS6l(S0&mZ_~8U^TDm*IyEwxg zHKY`hw61P@2-`J5>-OmkH)UiSWEH>R$y;gOS0Iy=726T%eRc@0)khrWPK}itO`E%M zolWv}vp453cV65Eu;M6OT#%BUJi)v9FqEnVu+7D3zYHk{nk{yYp5j*L+VGZc@z zN_h{T;2l24Jtz_1Z*peg8cCzH>(HLq4fbO6V{!`&m}3IMaS@T4RP5sXapRv(UBH2^jM<_i3Df_GoGJ29>cupQ8h z4zB~wMxRQ|bW`K9lQ7&Z{&QD8mci>7^0G{KRBRbY8J-|*6-OU>pJd`1_0yTy$y*^v z#}lPdQaa4SGu@eS$)mB0fGt8>1@QwAra2bOMb70m$&PinASG1}$xBDLY2Mv)QCfN7 zD{(!Zp+%1jUO}7h#AetS?63FX(NBpGn&8D9;ZFCCp68B;TM0?~{eSLSBlU3T1n-i0 z*v0AqyB@~{8fh_9n+dUP9>y)qb)C$A{JBdWG(s6-k8l^ntpMBJi6>=+-8SCacfNb% z&>R%99)k7=oF~dab?kU?b7Wk6GUh&bs?@+Z1Jk@ivylB2kfe>lGmBC`|Hj>dOq|!O z&B9Ee3X&cP8;X#%5sk!k~0>~I13LoilY(Z zw8P>S$ne+_$O`3IQ;~h;-Fl=uDXt03nN9carCRr29qCStO+vpZ#uKTgXg7Of=DQ1q ztODDSiIWuxxiD47f?MXh6Ji@7tHN~;qrb1{&rgt=z{g&koFniQq(uG58*>!STl2xH zz-Dbw*{;b%yiyuDYo-=W~1|_qOx*N zL`_J;Ka*k`{@A0@I0cYoorxsT?$9msSz`lI=F=&W!YlkGOhlJ@q_&2h)LQ8%Hows|+?xKmG`h4oe2@FYBRBuQQtf#re48)E;- z`U>z|AIBr5`{V|zBAvW)bhDz!Af8n$iNucMWu9hQLy(zw#Rca)MrbOCVT^?3UaTkOrtN4Fq>|K)pN=Y{xvW`<_5>3%0xN z4Fu~@oR0ef!Od81c`y*uAYT5CUvS5=>hFO9>$>QS+E&jx~0w$^6@@uhIlx+xGJ>crfuyv`Q7=%0c3Hf)zY zhuE;&{#+nY3^qJ(w=3!ctr4boy%b0kg{p=?YoyU=v>F>;4P;lrL-lKcxg?><8|0>Ll!j;eU`Z zEUW$#7}f;6Em)SIt94-6hQ1Rjem4-)ij^`f8>KU0S&wC-EO($e-U|dbAaLb-ftYe^ zH(*%o zy#vdlErDT$Se9ei0{hx6=U3Knzyv1_UbxIc)=i9Rr>A0odsqgOvgB_JKG6Rt}1(8x*J+6k~&XXPxSHdv~1X zemihiAee-^f9dJog3~eCUboCW!y9tCo8g^vx|@86$)ZfQY6I~_5l&+u3*jPyWf6gr z2>LQb*%gR)y7t1!#9lKRW1Q-k8Qv$$+=-9Y>6Az7Vi|s4=Wtvl*YrumiQ=u z*7!wU=ydmvO%s>9-w)jspYH}E!u|Ik?evn%+@P0ynHvn%tTt8XG;^8rBgPP<#H+~H z*yY{lV9Z83*KRsjqA2@gz5E&raHug!mnF|xKf}VvH^iX@qz(h;TE{4}N!j>64}X5X z0&Ks2_`P75{qCXv7#Pm(`sHXxa&#aK{5s&G9zic}0z0A~-UgP|51$K`+z*dJjp5|l z!J;>1Vs*nNTLUWampmGRDbVwC7d+{eEcS4 zKzgvB-vKTi=I_3O${r<+Kpt|Eu_5U`{7GSsWdP~TlJ4gAG`s06$F3(?dy`yitd-j- z)XVf$fM34T=wlbcWi*q9Ewv>c%1Eqo&E%}Z~FO>ILVXXe*Tv*9G{Ce`X4Z`_RaP1A%oFR;1b`lgPr5} z6-6ke`<)JCzmQbJzJ!^wNjbi%9(`Benh2h1w{@@@%I)!b3=A%`3c9x+LIo7Sr2m`4 z!>O-L>DNDza_!7IDCM6TvVq4jko4G@_{*n(@3lWvZ0qn0g1bmb{0OTTxEykl6&%F+4J3eqfTn z<(B#6k$2f*Xji(BavlF+?e6#87jD+YE6))T%Cq zpuY?W@2>)#13yex+<_6xBfJYQcSna>Z^aIclzbbPrFMD3`8L2* zBXyj^u#|z{&u53>QvLnup9?N02=o(jG{7Z*xPhR89-a>-1CBrNWnfaJ{qgL6_+P-J z?fdGZ%dZ9OFF-B0>>hm$^_U^!z3gSe%`UG3>A5Pvs(g*qq{;Z-A72PABb`6MsbI2q__@@Oj1#+Y z6W@_D!UOsWumuJ#Q@Py?443nyqp;S^lIg(G;F}yDbb;-}DaOiiPUg1GSZ+9yqTry8W^A^EWY1 z5Wa;ak|vUL7XDF4KzNfY{c=meBZO7PJgwyGLgg-u+xljkweS}s-ZZ`@wR`;v3|`~Yzm(f;3~==Yb`}~sl0c_EE4VF39!dV` z?nVC>`Q?=FWzF}nznXF!G9-nZaFg@m7tq*9-CWM#m9zalTSD2bB)Nm^?sW!Bu5-3g zI9kb6t3N)rheJaRVtWj{8?JY!hV%r$A8;Mw%l_$id^b8sj^9#_ ze4FGyEC1QbLuSwr4gVQMEYC3Fto2D%`f?N*4lIppAjNNonQE~IIetxyB0Uh`j3?k4fSq!Cp@#p8+U^0KdjQY7)mn8Ey z*+t*2_I|$!lg(;3vgblc^oC{>6Ca9q_X({SWNXG>)2E=O$6rbSqT;-75a$d+f@k-C4jOMlZJ1)K`G zpUeE@VDRBj+Ki4DCy4LJ6)2e-^buhDEEve{?dMm3OGozeZ3ry$#ta6~`NmS{ORf0% zGhiVZpw6R#9)&LilVi*p@>y7yRLgC#kupowL5UG){qy9-o z+j0DQQ!uVdEBX0!^lCXBIZDn+k!&1vt`?L?YNxLN2Ul=8DEj%I;L6m_g2e^#ve!ejbFAw9VaW{R5-&5r;>?YWqKTStxu#3(rdM}vlb$OV%tlh?LMzvCcrE2>e*Uz-4i!-7fgW*63Hvk?#Isih$eUHpk8( zETc?sCHW1UIvYR-+5@>Q-9u6?w`G@?u^?2{slZlGx`S}vmMK8w~sE%{`2zvS1;L@^zdc3jdj^8`y zSKBr7c3;}bR3TBBZ@ISKR)UT#RovDt{0q0W4o1CT^|caLBfMOB@bj<0q+-vt`XOf_ zHY7bo2igJT$SX(bFUV!+mn1u(gc|aq4S0k=q$S+Wu$$&OiR|!xE;B?qB^vRP*kinv zONx?;QE7)@X$8If7I0~?V~l}4-x2-S@F&wmKbMDnM8AS(m90GD|BuTW@&;MTc)yE+ zgr{wZQn@^U%fq=`!euU(7jjw1H%t?NN1S; zs4;F^ib>e3ED`0I+}0iKd~VBhKvE*@Pt*HPcLFBkuY`)yu9BYMw$8@#7J4__?cytL zM_IY-m6EC!*bL}!8eclVaGj+kn=RY%WHr6iODK0kffu}6%iXCd*>kPj8eu)wZtDRj z`#|sWa`)oEXt~mUNhqDs{~s+w&i|vpjEg?=|1<;AXaE1sK-u%QZxo`V-S&CJxIyf2 z!Jyob86MFvK<<$b^F~y-lSVf5^dr-a>Z0wDGVjMtr&hRY10n5hwv*Zak+`j*y}B=t z=_zbqKG;duv3oZ`B>JxI4ZC3{5~ubJHq&;WS79VRo3-P0-IJB#A|4JO&HwF>Cpg*X z>CA6;ef;S7ZGB5F(n!o`b$7Jdj30@)abJNIN8;W~ol#g&`^~)0-ZJxgRBb}-j z`pM8}IsW+CNDL!!i9NT$yX--CdBm_a-n$RFQx_~+v%5>jc@~=ym<%Y_nf5ZHwr3Ms zE=}SssC1`}N54_il~Bnsqj086(d#6Y4$tWtY$q0LF^misOoA9TF8$z`%EuaKTF890Lo9HA9*s?)PFH0 z579s$nN{xovcmV0K~@7=f8=`zIPfg0spOU{V+^uggkJ ztSPc?y6CNt0hDW$_0(S>m&cJ*f2)i5;%LfSR!~QaW!Q^_Cmv^ko&?=)_U;6y$d}QF z!|=!HXVtaVkIr>>8vVh}st^T?>&W-EO&gO}QIKn9254iS{<(%HIF-H({Vfa+fs6l~ z_11By)fU|Om;qFv8QMmJv`4I>>K}zH-%nm-;c+Te64+KF&<>LWA*xO{!{_CmR9ICG$*g z>JeTLhUbGzhL+x7RpkpSl5RKpz1&hfJVyN$mw-tBFQ?>Z3X-PUEasSz59b!>eG;He zE|ek7ina!~kTLIvLE}a%kPnAQIs&s& zF<5eOuYvIA$z$ZHF{A>Fl#=GeS^c!*?e;ReaEi%ioMZJn$=``Xd0 z!VAu|yplYRJo!V*tL(ygg}kxF0(+3b=_F1_#ZT8-%jI^Oq~pm8@s5}Q8sH8JO4rfA zNF8S@`F4&INpv(4-9!SUUTcAlAE%SIo?;znO_Y+$uhvM)vEv_}gr=aZuy2N9aRn@Z znN%Uqs8CRo&H$Idtqh#U3|)-Fn((aaEJ$V_x*AO?yt%>x`4EPr&?gjR$PIct=i>3a+A{i2HG-k>aUT@?}rhMr^cagtIUQ%s&3;Evr%1VM0-!rt^_)gj$0&6i9Tmodtt1d{5cHvYS{a$XhaUqZT zi%znN@+~S!9poi)*9B3H-Id1bfd8f`1M_|G=lxN*Sc{8Zy#?n*=&q=)xwt-uNHw2tIc4wBvn7e|?9 zt2d)Zjk6ieW!KOEHY5R z^&=+Qj3y4@un?qVGz4-^3W4gH<$g5CpKBX2kq)Qj6TqeI^`RE6p;IS|MlVTpT4c zz-9_AroM93Uq$^L)R&KdnEw9-4OE~8-_l^k2wQ?-cCT{MrisR2N4e@x1(yuXSZwt> z?82!u`U%b&-|n|bcnG_nU+@AQ6^yhV^69A4IKpI^Tm%2k09C#K<`+TW-_Payc9Kzf zjZ?;-o{7=95J<`N{Z84Ew2}t;z9&Xvlm1D4<;2cQ)Ytbw3n&# zQXUU38LgUW2O_hxK$t?_xX5be(BM4!QLcV2qJC|Nfiqa*W*X?+R1H2Luk~fLm5w^( ziD;w}cGA7g-CCRilB=J6z@-3FxEqIxDLIRR!m-v-H67haN6OXFedOv$`QOQP@~iwY zaveX&L(WqwI7U_Nauc}erQ|xb`-%GPKKNZ%!0+w5^^0GZ!{d21y=eD zwnHFouZ}cZ-;k>#~vG#oN(FxZ)Fh&*__rdw%bMztH**%hLl^W zuUz$Cq<-6Et1knZi9dR#4WI%w*biK)u5*8DU=HUH7E)ii>aQo) z6DZ~K>)BEOz22ex)|rSe0am8l0JV1E{F?^K)!=<_(bxB&)7VK5JJ<$LuKGuSi@v@u zT}Z*>)X$#5%K~cf3=Q79$!p2%n<9a@7xQqQM#(45Q>HV~~K~5bZl@bPo}Q zJ$a1bI5pIw(oX`p&XS3pL~s-=j&TF zu=`&f4OE~OFVbLHmNi(yjK&>mGn6;if==!Qi@>D-dX}$#j;DST2MC?I6@#PvD4C8N zwAe_4MVU50J^B0OK?cqxpKYoHx2~v9>|~OwzVi8o$2s{6`ILYPj-i2;O!*RWtqJ9) zfy+g+4Y~GkTtvasVsJ5Noo@|vce;=I z%2od%a=j#^{BinOahxSfDHs}>BWsvUQb6vE0++z`MK|Mv#3ApD4Wsb0aj8@ zedVhEAbChx91g2W5%|uGo;Peg?T-v$h=ZkAOf*un2 z%kB0myKox7Wdz*p8yQ=|@Q)0~n6LY1o9GeW#mBAb%Vp+HMuDNex*PjFK6`<9?E48M+k^u~(%*+{M#>S_PmA&`=_ zEwQS4RqR`E8Dewpk2ZoT?j4EqWwW;?tQGFWfSb0bpGpcYr@qz@amcxff(#m{qat$6 zkn$TDAm&0_m!+)I$fHC97tzU8e~|K%l-T7EaP&W1Q$E9jm6Ts@1h|Itoo!s@|0J(i zX(O(n{$ew88H^u-W=_aYG(5pc_chu1yYrsE*;utl|Z-?P6^rN>C)z3%Z61X|n60Om>3rzgp z8mXYe8M6UNMl%H;R0>0+%Dl*GUdw8g?pn#6_PODmlz@-F< zTWz0c;C}x#dD4~a6XXZuB9`c9onk$d(a|mD;ug+?sL$~aH%4esxW+mvXTjsm1qDn> z$u&bGz$HKuZ+Ny+U@utoa@D`v_(^b9_-Z2bWEjD-46Jul)zJ&&S~BJK3ddka?`tZ5 zi~4#jQ2s7B3J`MIc#x>2WVku6$3R4`2BXRKj;-=kaH)wI*VrmkKY7$wuKJfyzuoAg z|7+VkLjx;Nb;o&*ys(IELeb0QdP`U>n#lEzt@3{xj%QzXZvxr>w)H0f|AvnA7Oy(` z0bEMF;(D8*R0_^C*9{Wzn*@D3^qrA!1Azo6W`GPk^gFIO3GL;oKiKdD zr^y$1Y#5$KKX?qmj6dqfnFfIb?%-#A%r62%`ZReF??P#l*|T5Vz__$;iQ~+5KG7=8#H=`(3sf=qBJKtq)ckVio!{p9%k*M-6eUSa@sqyb(d z*Qa!pzfC_Yd;vZS!`ta6q>uHeqwi@@>2nlmPDJqhk&h2D9CN+Lv`Vd~A>_FAtFFz` zo{nQSTy1!7`|nEweRNB$#xrn(FYxRzd@lXyEoAkRO|Ca!El2!gA&@qc-xQOSY5wDq z2?MnG0>2uDZ)N~}6ohV^_vk;3Z|%oWFdB2k2$}!KFSA5vv8&A_vNuEOXq^~9zknGn zW6i!nedVg(Yf3(?CecN&$aLG{4 zo0jWANPa_4^!2#~*^+d#;R#N6Z;jgc5AG>229MH_K1E|7{yhaQffHG!@f7?)eSHQ+ z^&`yO5|5nnCcE;1OH6#^^mIjb8E*^{@HPFu<3VZ|J{epxq#sI1qPukrtX!j9PyLwC zKWx>lqU2i|==a#v;71zh_t_S)B~Cg~G%!D)AJspdT+fh{uP{8pNoK)%+P}ajP=gR1 z>8Y9W_0-Sy)zDpG`2F;wr*iZYa;j*s!sn40nr@nI4ucbb|Uf0L}AJjLzr~aW5ak4D|J|o!2 z;An7ZW93GOe>v307x4_z0(jI{uKM>SEhhUm5e%3Tu zu7}+jW~5AT3VnSmyNCPc|Hm2u?szbO20n=f)jmfTgyEOck7h{y*dZEs*O}3t@i#;R z9ZJ<;J-I%&to$YhuJ>i={xH0Xe)PFU^%Hs$0;xLZZCiD@9PqxTqgG#lKf~~-)5K-3 z9}TbBQS#-}-O^cBum?Y%0rWW^HOXfHd~eer-3vBRU%BeP zNNyh`+im}Ql>&VxO$|1a>ocdy-!mMOOkV-M3d6snAAP3P_>uTOQ=nC(0pu4(rRwrO zvnAVL7tRsOEbso@f++H9!R0DfhA%^RhT#?9h~FDP13zpGaMOtals^G(`pR}2QGPK_ z(s$H%$Sp;|sK4V!t6xvSAvoHKpRVsX|F2*PvT>!cixAZu&l4rv3=jcakLjAlPf$sJV z;e18Gu#uJ*L|K!RGi4d)oaCEKPBa{oRr;~oc#V!+{U|>eTmo+xZT#bn!+;e%N&^a1 z-+78$pN>;O9UYY(U}v{U^z$F`ql0qf|<&tIY=&8Rwh4P2^p77LJ1!GUL6 zUXb3mhK>Z60wg0Y%HNw#*-g+(@j#;Cy#Z8q9`)0G1y~)1uLGBiRxzXa7>9uKLDg0oCp^m1b7eCk)rXY-IUD0qtc%88x26^JhZ+G((YXFR{sK)D+1 zWPqlbHo!^>W}jpItek3j7CYl=@|tOu>zf|WkUOF2R?rk>Jq6RX zbjwAy0F9AG5^^%mvx4pmtss$AcO`lKk(TG%h0{bnjE))@VJ}SjBtX|325_xlA^EWL zEN`ZtV#9I$fBX5CBvSAu1X6XiYy$ds{zFGA4zr4S<6*@4R$sa5k0I9#DIW(e8J&_1 zKT`jRwkn(pC|Gff4Ny&sN66zBTdwnf4)XG2Enmi(JNyD0Af1QLTKZW>p0&{GOJ6bV z-$+3hyQjW?5PzX{w4II`>F5OVsuQgveOBxaa5+X4`;HO!7>*H;O-VOS<%K3bt3oN3 zkI_JLs-maKwT6`clU!>^c>}mqb-~#-qw<{|Nz=@Ji`#q0hOEG1d>H})G2q*)4u=J2 zBMr2MRR4DBFImc(V1UsVi3XmO))@>9QpvSOIsV}*L=@)yA+qbshkC6=EAleEtz^1pZYmMuvsM#52BkJbNff-M`sAwrHuW>ea|c?< zZ`4=L@ejhw>|O5smG-!rySFtsg1q@^@?n;*1D6t%-(xehiUoKDTuQ9mYU8ax>c`(_ z^-V=WI4$%SYoG!(mDQT{qz0A-u*9^0g}G3Rn*7;cav{vwYPP<#@x_sTs?~H(7@YSv%U8W%HHXpDOW-oY z`Q10edCl+yr^Z)+tzr0QhRbIG7=Zqrb_z5@%D)Gfj0VrKq>O^2ue4l04XyVD9tW2u zQ%-fKmih%9wqPN3)I|e*8b%Fv(IERJYaqXdA?cjeHgFgBjx82DuYgNn)hBk|G(653 zx|q9*6@;As&_Erj!4`788KwMF!}0i?ud06v!+)or=3Mn)mw`31#Qa~M`%`fc0;%eZ zGcC!aAjcT=a@9YX`fI2!KZYUc4(cnn)N$^ie(igxQR)9vIP}I}B`OJyb4B0%J~a%V zY&cF-#L;ePI=DQt^?5m~n&MpL{Pr|TY4^X443Osw{6HA~h~XFyTKWdAp?;aqPh%MV z)>S5c?_)Juh<9nw?sN2I82&%Q2RnL&`%j|aQtB&L{Q~Ou z{IDhkZ%|*k>NkTU{$MBX$G-dVhjdi)v*mhuK4qPl;)O+DiD!o4hZ&Bi>gh)VXMsz> z@bOAp|A~~`M+5z;r5Zd$1O3XS>PKHA8YsZ85vppIG2jwFKY^+G=TTod$3Hv~Ndx^> zrW(9W1LYc}h1|YGt`t9tVR(X5;;W(W!|-21bfhnxtE)e0Q15dzq)0UI`i+k#8jgXB zFR81aeaZD@bmim8L;8Nd3ifA!CSQQLVR$wj>GuHC&#~ar1j(FoRa0;!{p_&*d-uPM zeFf^rxt#&@!vZQOXMic@EEylmq2L4Z7B|w6MdW*5D=T=U<8Qkk3lM)|zXF{18{Ydm zJ^eTbgG#46F98mod>Zh%=WC;aZh0FYZ4F&ZSj9za8wDQXcI-=bNE+t6q zQv%0%CJcYhaLfU5d2jdhGP!&hrMLc}(L{mfRE_=(E+w9Yi%60>19my^I?L<+w7Xx9 z#m*vd*)5e5J13Cqryam;{8J%t&BxBIYBDV<7+__jZObb1?Zy!XB415(h2gsl@2zRN zaol1PKfx*YIT}?gDtq#=hU0b}n?@a_k?V(1luskqrc^$Y{%b`Ctc8B4SHzK~|$o~9A z@2i+s+$eUvT)nL|JOPi-_YH7s7=9ul;PSN#j9-_4(ZnWgRj8U-_WGl;kIoC!DC0QGy@jJ9#; zJ)69ojx?i>k++g3GxC6qvJ&Tv>wEg`gkp^#!I|Q#(v&cKqT#rg!wk{Clg{B{horFJzT| zN&_oUb;tRV+qqcoGey}U$-z-WvJ0@3?E5Idi!xXDnBypwv|3N{`WGjeg zhW;cE?rV8wv~@MTM3%S~L~be0BfuqqlW)mp3N}(-x$55zj`#^qmapv}F$R0`8gL1a zooX|b#98+j44_;C?4bUN@m61cn@-Ycw^%>3CPmu*U%(Ro0|F_T8pxKUdU6<8!2hq1 zYt5(Vj<^KVf0%T3KerDYO& z{m}LUtw9z`_AeSJSA#cbkT%omH?jy(rJ^y|QLg%XflG#V9BlPhQLv2qO#QBHwvoXNPMFIUg-QWRyQ#+o!O*#dq++{P=M1A>zV@YM; z(ixS@mZbX($0J+5j6QRhDgT~==NKR-$9j(EaXtEOafd4xJJB( zj4oSh^+Ofh@6Vxu3e;dF4N~*0K`Tq}0(s4USTY&}?va%QXNIo;d-u;nP7(x?qwZ5} zfa;+2RA~%)IsH42QQx`1>hlL5oWt(5ei|>aJT1jqTti-Uw&j^D@C)GRe;7)AIsK0j z?8&znj@xW>q#62@0W#S3RTRv)&t|Co92;04C%PG2n#SC z^k9YcpTp;Gn(4oz0`VnqLzxXcg@U>FTZ0|)L!y${=?ck{E39HW#m(eZ_gE0aHooKm z>nC`Z)o(D)@Z%rK|IYb;ev*}ZLqWmctfLtWJn=#6sBW_5g#)erh2$0cFmRmZmE@Ii zmUr)E`M2anzeO5QrR_hh(gvv9Y6XRM;Z&2Sequq#FzaY5`HpfMpx$C<%-^k_+WRcm znme65_kPP~v0$O04_Sk<2l_fXnY^^p^6mjP!d>LUezF;=X5gU@TR$y)V7QXJfxJsT zc8`Q%!=!cCP0;iqr?$ch)X}U*tfM6~P)8qUmml(@NuiJ ze54742Nu@2(A4_J>i&HM^n%3YB(8>nBs)+*|u;sf%;UDnSq z>7)`r>2Vui=_^*SXoU52J^7B?E!StcJ_MHmM7h<*>wDBsshVVgOHC}j-Ih#`at|`VW?z6!Vfc&mk0#r#{t@L>8g%#^{WlD6 z1D6b?-EC_~A7Y7pQnK93wE%k?jtdlPwI&#U;vNEPl1@iDa;c+(!6k6pZMG)1P_Ul7 zuGI1h^7rXS2QKx~8loWhcUuA#Y@?$F?q+&DAf?6zPFigPlyN^^OTM|w`q8HO7kT&9 zY%=L|NP(xUpHOX)6+q9VObS+9Yk3>_8uG05mT#b=XUKQ_Yy)ej`<1-8*y^LYOq%z! z4P0}T;UTA+0scur?HX&KeCRWxfTz`a+RPeE2*am>OVbp*Y%|nK{}o4_T&l7HC%QEYjYaC!dUFF2l#bUaW;r+`b~ zSqIq~D&zHlTgj^@Szb?r56DYnE!XP`dvCISN{)@P<6i~aW&s8H8P*^kY?GL-1%x z|IST@$2p_yFQq7UO6g~%&wthPX8*@2l~3>_1L$E_Jv|F9C2RcEw!NNa|4d%~yXEY| zPAb|$^t*nsJfDtkAkPcY(J~5tr=XMuE65Lc(K?#K0P5&=aH+Wsz5+ZFhSwO58yFl{ zddA;P6zG7%xXz2@x_>Kwm0UBbd^5S0SowS8c`Pt_$k|GPR;e0%1};@MC15+tX7=`d zUJ?yl=l7L(S{OdlaC`xoBcu91%y2yaqjND8WYIvYTKPh9ZDZxf8y@Fm`#RkzhU4Sv3 z|1rcC_TBwb!tjZP<20L C5%gZ#%=b2;G1mZXClY(@(&uz`m$%%u$`K28>Wfv<&u zWXR7q7>@m((`wO@bX$Y`#|aFl;j$&^t_B-;L!J$+^Mg;wYtIk$R)rWOy)6IZoOgX~ zxu4;<7tA46v?NU;*AdWC{N6ITo-Zh$1ujk2ex8k6Y3d)+n~g!7Gw67Bb6VU%M>^)K zqkE{o=7hfbmDJZER`nkv*CAPX=xGQfM=czy%PHCTFInm3YH&Nb9wU^OgNvVZmP{ug zZPYK(V+Resvic#X>qJ`>o$M52Ul9%b>Xxrh92ACU7>);wa5URJ9RY4KbiCzFkyd&o z_0tX4`iJal3N%CNteB2=EVGXCc_#E0^_8psyVPH{!s=HBtYl=P&5(1h<>|sD{%H_M ziI@3Gd`TF7x#2jxGJsf1x{6!_Sn4=Mk%1VNh@2jDs!tlk0;|eGJ(2a8< zxt^qQ{KLyHbX4REkROI$K}UL>Km)8L*OOG`*OKeBUHOI(1C;p!SA^jY(~-`S)zK5= zoK`D#{z0xMqbjH;59x`43SNOg+H4ljY;-<9;5E^}$K-t(-6srBF&q!4^87;mPbBC0 zMQ{73Lm+{hZ?K&uFKA89r-7b$sH1%Ho)ZiTu24=ts=wCoIH#r0h>7^u(m+eB1{=ur z98dXeh9@|6zM6PA41a=t^qf%r{6n~0|JT!L71To@H4%S@tqDEHOMKn(D$jDA11@P2 zOMHIIm!Z5c{1U^h|K9qCi_2-S!{_LRF#Hxe(hO+<%EQD$jsFk@J=gna z@i@5DM0&j~af@9zUm1OTNXVDbKf>^cH^d>Xn3U52{SSP@#K$#VUN|Io;*9|=6kP2p zGRxF3d@=*{Ty~??bZ{v^@JSn3FIKOlpBW)vfI=1c_;nh9SIab48>ye|(|;fge}sPY zI$%%x*U(^v&rxF-{uUkSbwkb2yX1QLPWgZ7r>L+0kn_L(1x_a&>4ige^au6J`Z{vr z-V_a7%<}Q!hT|Yq*|+US8;<$E-tE!=sSrq;weiBCP8z>tfLdRG-^1|0TjFuBqZc66 zf6#Dz##j9G)IX$2y9-ns%l+s`Go+$P)DN!f+ezny;YZSsc1H1I;xB?gO1!z&R#6%K z-An`J8u$+CuXx_-Z>C_2(NA#Fd>PsihX3@IiI4mJbff`(XJFm$l?M<>G8*-=4LmGp zC0Qm1y$`;W%Yu01f;LdCOZiu<}32 z^-fDq`$xTPOSYle8t6r(v%sYjDOX45Q@@0_-;ybKiu%e`zm8mQQYjC;M1kJNQ^9NG zdPh_F+vIvjQ+W%xG)?<^wg8;+IV0Z@jb5($$>7k(`@dVPK?ZB;A}er~er$ox0iPr< zZMD3So_=Hi^`ilPGaPS1_?mL?zePj&Zu{RbBfvnx0P1KTa4A_D@9S&eGpMf_RsD0x zRbTl<w7}V zSJ6)>kB;;j?FTeat_B~|ppE)Dtckhri3X0AT2vZjgG)x!zq18PBL5rp$$Q(skOqrr z&_RPX8Ym}rz9QG3303|b{S;qriGJ22twpx+swcUXBK|ZZz{#br6CDwTFQ6m+9a8Ja zagGC*5_hmF&HX(HZ=}9*)xTA^T>mff1%AjF?8zUeBmD(ebyN#3fvY-fRk!gdxA*&^ z(aTjo3EcFd)ja-JP;!|uz!b`tq4i<-O$-nPN4uv|1}Of?W~hO};0LmTCLqU#ZT~nU zz{BCb!25;alfb0}*_}2(Ej#1s44|BGofY)6-RHmH0}~$?lYN2^19bn~xA)dCfN~A6 ziTa7ZSp8fI{zZM|s{aP{3qmv)MoHj9S;1W@ay1BoONlp-Z)O`GPJQL7Ki_a%eD)RW zR3Goz|ITE9O15PTJ>AFv$~C~P)L;6WEis?Sa1yqN22N_pRez-6xW;@o+|_LZ#M9Hw44_;C+(G@|pH{zNl9hZzedVhEgVD$QziEO!Eb1N2v;Qk9m>~4c zsdGgbzSi(wKN{d#a4Eo&N!EXTq?LR{Kk?W4=Kp`}E>JI6Mr@UhUPme#2rdEQr`iB| zEchGsm8*Us^)sp80D)=$k7%F*HTc|c{DOhxbhq?N82%>%IMZx~^s(MUJ`!Ub@5$9w z=0_%ef|KbBu*4Yb$xovr{i2yVI*a^*B(gQ*^r|PJP24y}+WnuVz;L-#& z)2*Kpw($=Ppj`cTQ9qNvADE~6-@I0tqm6QczGuAd2ba!R@5|6X!ti>-2Rj--1HVG9 zKU}Ch^dze=u=t21x|8|+{5TDi z6Far!`h&#EpC{KZ9V`Er;Uk<`rM!7W!5bk8a(sej@_Zlvki5jlKQTPcNnT?6ga-bC z`c*#t9ps?~pWr77w)pt(A%_Mf0^(Q zs;c)|eu(;CK)<*Ck(u>0$o3iBM1#l^`*yBU z>gW3O@237fKK%#CNB43Y{}Bo%_X

s9}HtUw}>2Kg_59BK3=X`ma(y&!@k6n~6Wd zsq-1UM@P**zLk!y^*Q>CeB4QWYp9)m+I@b$r~bhqpTRE_obKa)l3(rPQQM`W;+$)I zd@u65d_2K$?5>{eLp|+3k_Hd@43cRuiw34$@NWWnfsaq6pK70Fwy?B@V05;SG{NAb+3m2Qkic5~mUQ{2pB8--LmFBY~bo zK6I9&&5UO;(8?ZxWWU&n`q4zc8}%EA{$HP}`-iW?ArAk=KqZ8?W1#IM(5I-cBl@pU zf0*dMLw(DvsPoQ#vJJDH+&CofFU2@U0x{(A_d5H;VC0#ECn3+Vx!wLJqhXe9kon+e zjN%^SzobADQC~&$KGc6t^kLMmB>L!CXlNjYb1_gS36zKYXToP-oUJ6z6{zQPV)Py1 zza^SSjYMKtfDRGDs}zUTk|ekh^=A?NEvV1A&)GC9P+y?*cKt)C?nH+w;&3-QRFMSt zp?)RNuSdR)@C_KJzGobne+wEqdK!Q~jeOe`F<5E((u(~0LMML(`OAdwLH@~Hr~Yl= zkl$m(7dZ{@C zwX0(wa{KT6+B^aI=cO?=QrA?)VfDM;?i1MlqtKAvWzc@(kcSD+LOzS|Ao3c*Pe}VkT`RZFC@Gid2|UeT!V(22)|zO6yq>S{72N^ zL-fm$uO)mX@~1TK9)EYC;Vt5@2Kk4CuSI^C@JEn$5dKfaVG4KL0OWrH4e<}yokF+$ zGsrUu-+_EO;jbZ|PxxEN8wlSoc@%neUb(aF-$#e-#NiXgJ;s$p|0U`V5dBfqFC_Y7 zs6Ph!?)rzq{Emjywa%It@VTrJkFk^lic=g;wM0J@`7FXmAg}4>I{ztXXy`V`jK&1y z+X?q7?lJyC3KT;9VWK}1_4gBf?&m5$bPD2d5eABXkg0;$|6GPVlkhp1z*XUF%3{># z5dD1A4~i0pYte8O;WuEQStL*`>IKpN8TC~}|5xNIk$1KK-_fv+INXZ~UL*WzOeyH(Wn4b%sSejn=NiT;4%QIBye zarh7&GKs@y$a4tqKt7A`|3h9y_%Fy;O5Rofa1ZXo@T!po2!A$%cnzCPMr74nW3qal60li!3qO!zY7`Gntwd_LiSQ`}=bVfQ(^{p-8;nD4A*i8&CB0os@tH_TL-iAE>A!p6~ z2YDvp?Sg{+bT~{LrXV*Sc2=o{Je}}t-LtaGqg}@=d z#~8K1*@^z396ZKY!e=A*5?+M-bi%JfelFpQ6i+dhJAzA^s6(RsREdCgG1F&mp`C`7FYpLS9Aqb5UjR7|Y2Wzl07e ziNov2*Ac!K`F6tphkP&LA0Uq&CWb?3FdCgT@wMXcppg{xd(@{A{m-cXGts-gl%otr zjpp6$&(ZJ~;^0Ak8{x^wR}!9v{C2`K6i+d{j)Fq|EHq4a7!1QiUPO2{Cb)wn$VGh( z(dQvwMR%hJAv$NMjP@~gzx`S z<@Xr#NuYL2u#oUW$QKdbf&4ncJCQFT{5bNPqQnsMl`OBvxS8-cgnT*SX~=IS zJOlZkkax9zmNGzRBvoW0-$8gb@&knDB0oZS9&-Meqpw7b0%d@+94Tob@_!Isj(i>A zmB`ldzK!rkNbHe~a)A&QA-~5sObl7*@D<@E@(#kY6^Ey2q^fgK{{zwIA)iGD5W)4KIKe>CcOnl*iQzaJ z@(GXmR^B%((}c$%Uq*Nm@&>}wkZ(oa)&3c1*i9U=6o)@1Mef+Nx&23`WclDH+vX2C z`%o_OjsI})sF8<;Elz_`fP5?Ag~*>Eyd3#cgjXVen(%7HVgJ9K7;4esS;AK!f1dC< zDTKEoA4B*q#Z!!fJ(^hMZ$rbe z9tPQ#`!P@k3Dl1I_)X5M21T- z))5}3xW_oQ(U~9#`7ea0VVv!o$n*aUH0&h_vM|tZB#?=`@So1a*%-%Ya@J5T>gU;d zyZ!Uf!Al$pkWVMP5P1>d<(Oczoj_I?{8Xa8hUlZ!=dbPxDNS1 z!t0SQAbhQkA2rsI1dZBYY$d!2`EJ6SF~MSzpcVB8iGCOA-&T6N{-KK6(BT+y*pEDZ zv$HMRk$VY0gb4yo&Kl}KeSS~9%-@L)bBV)obg1g-0Q#7t@DGfFm4wGB?lDS9oFvq* zBlT{5iTp6(*@}CN|B?8)s5iEd=l^+VNZsPB>H-WjezUVm z3sLVS`f}9g6MZG}!Y+>OUyX+OT?Sdjup>f#1K}%>uOPe*`6|Nek>5}F+9(<}5qx{X=1@(eND! zREs>k)!FtdFu^*KpbqsjiM}57+q?BT|5|i-lsGgZ@4wBNxCsO8?hYh7gVBuo(L~>h z`h!Hj>pPVnj!4ANhJj|0K>Lvw65ft{A>oHGG2hm+)4`4i^>tC=(1{M|#NjwPY$p1c z@8x~N6i#@Y;_%3o#7RQ^`>5|~|1>mokU$y8eNeN4|{kO2uLSUr!9x=PwaA|!d#Bcf#kFWSLmGPgG$Ox@@FwK*2yaF{pYT@Xi+gaDe-|2V?_rSn+mJs*_j}?B{zseJ?VpQ=Wwt@y5d7pJUqN^Q@>PTvDh}_Eka1s*`qe}q ztwe`q#Gx8_1L3vEw-UYr6Ko<0>X2_Hyk5tN8oNoLwdn8~acI;I#zCTQLVk?!W{mR* ziPMVu_$TC)Y}Y@$+=Yg8WssGxf7+0T3Ez(iBF)Y|)Q&ut@I%PY>&a#Q4m8Z|X#n1- zIJ6~6d>r|F!ecsRf#G8{gvTLoAUp{;?S+|`9Z?7kRK!5L>~X7bM?zc zp7|u%|L3A1hdAURpG9~9@+!g$755nPNR5`Ger1;)+rJVG4P6FVm9QhiK(~`XwW!}p z^ed3>CcF;$LBi{!n4pm)Sc`lQ;f)yR7zxyb`lCeOjQaSeoK4V*`sj$KoxOJ#I%E=u zHstw)@5ex=k~rx>gXjxT ze>%|@qW&1sm!rNK^xgFjg{eftEhJDi@;eEy#RRB4bjK^D2o7_TcVFs96o1&`mXj*Lc>-P zC=CO>P6B11elO8yq5e&xH<7>XzApf55LgemeoJ~`% zIPCw25r~-c|q5TJ>nSk~pkIUO{*x^6Lq2LLMc& z8TqOnT;*>?!$Um`vis~pzLoGceH)zmV_@>`B|1Dq9I6#hF^m?cz83X+h<*j? z*4!hPNE|xQVOdWH(08K#0MQ>ueFM?Q`~*(0&)7hC9B{~= zV(cJ>By>1HcpCB}gl8!3Fd-$3*$P(QrYSwnSEbl5-~ z>e1nB;;hlqYZ>Wycd z1!_lq!V9VLmXa|n<5S-N_RRgO52KMoD+9R@fT zAm2)O8uAwj&p_Trcoy=5gqxDf&;NW+4B6=5dePZgaux6XYq;`$3w?TW)U zEJVAjLf!NsbV%LlZ2J!6MK3vdC-VOGcROR%9k=5U9`lP7z;{8vqT_V6e;gWW(13A_ zB;@M|PecAY;Tej<{@^+1xX(g;|6Pt7h#Dq3q!Wj1bjWc!7`dqT5`7-(7Z7~`>T`&` zP;vPFpL}8{$3Uw{pi1PM39rTk^GTdq)IUS?E0Dh?c~|{Ieb=F(ojBAZ|B~>vHh-9CmWaB0I&BVpG5fjK6v!;J`CIX;Lr5IclNp zXZPVy+y`IK2fwZlep5H^QU7qq%exKmTldfHeel(N@O%5<>w9s`|9CHhPQI-V{!AZy zXCM6aKKMR4|DC7_-syvX)Cd2f4}P={{uA;j9eys?i3=3l2T$mO59@=EcJj-*8hL`# z(8Yay@QC6G#`JfctKU?Y7w4zO6_%uadD33-B%UopUzs_Sx>n6rY7j210*VB=0KDCK`c zzKZB|d4?MXSy9%belO9t0GDfYk=E-DD~q55B^z`v{Hk*3%{wqbYOV7I>>8ze?VnVf zB*V#rcH#u%(|4RrY}YiFv@1)hTh-jeRp*>YpI{V~J8Swh)bHp+UqkeBP=BNk{VJkg ziu&|vCoazhqF<-!|o@UnD{F&z|)UjahWs1_qg!p5A9-psgg()k+_4eEpiU`D-!3))$@UpEl>v}b<>(xAXm9Apm46ow5C+M!y-rl8swCmM&`&HUo@G8OG zy)4-$phLDIrvCuDB*WmW|9o`m0v8%W%+e^S^T|`}BTVwn?kr(3f_7`B`6viG7g3m8CD2=nKqP z65g#ZkLpW{zUr%a~;eT`%g~AAssaK|VuH z7wGSlTiTF;y2u=ET>fNYOQir;-On{ad{hT%&exYBeOat8EA(ZJzHHQ&R&_~x8zRV> zqlrHNuB-T%)+dO4e}e>meSs#Ezm@1qwZ7Ep%R~C|w7Rq$_&dA6og67H4+g@aKqOER zG((|at}(V|vR^!Y4?Ec%_KMf;VHy1|SyWmszP*Qy9}_7pzUG>F*TV0<(ELCsh%@Thc>^q2hLG4-&n`_38|R&K&b;!P(($fAVd26^7WKH?X1N+zC8Uyt$smioE;b z*Le+Wn7I62Hdy?nfz5E40>0RFoY-$7{XQJp-K1l4}!VXMb}0ed5w)mJ02f_5hpc3JBMS?EIG153qwSvHWAUNX+~POA^;T$VwARiz9(h z*s>}Dr4fIi%$F-_KW5j9_x=I5le`{EU$>5pm9Hj=nGdlMV%LxCWHCapL9+e^iB;=a zy0o4v-hPOk8xQ>f1nzyjR{*(Q_ zkXYQv$`Z=0l9ICG5;G7gudvLLK(2s44{;s{yneCmGnNQFu57YDC?0r%-Rv@3QlDfO zy5m&e^{f8v4~cC7o+Lh4&4-FnPqQmqI-A)=1E#26>kk|A<}I$ie#tx-Z`YQpF&VMZ z;_@yS@QSME*(i6wCssetBJO}+SLW}}vnt(X5_S9hy_F>o3(7bFQ~j&T5PbVzLk~`>adHY2+#pMb5jCpe|4tnQ}y=cbR>z9btFTx}(iL54^-yGAIKKeoIVxnVm5_ z)NAAhpmbmli{&qaYedcjYE*Gi_yS81iLbC!x9JtDUxB%#@)dT9+l2dng_XHYy#I;2 z*m-dQd2^<^Y10(GSJ{;=OZ@HMY?>JKDjUnxy_;ci->d9Q$N`se;&%`Yx4hEP@~MAe ze8X!M$_k2o0y_Ndudyt+kzcELL>zgIwG2~rX{mUYYCbC>y+U5C|Icog4Rc4`zge!B z`8pd5Lwf1!tPI-cqt{tUoGM^QwMxhm`ENiig@kJlyFj$R!N$14^7XY)!0YxfQ+&S% z?!RRZ%a#*Q&E#-E{IZ9Ip>|ctroYKfg_7w~zR6BYRI^i9l`pKy7Y<8nqWI!XWq=TqP@&3Hoe74p)%wD#ZHI9 zUi2@v?PR!d(87(woaAbz{CGq<#K2A*8$NXH~w-4r*1^d`F5bVGE z*m)3au-oSqZ@$egiS_BAFb6$t-a_{PW^_Y>$eV zBJTkT6@YmH#%1ZXi^~I5OBVaY@(xHB|2_+fi#Nj5{$x8lJu49DHNe!(gMZNL^!NJ2 z=&fvm`0#x;6nfs~b~Z5;mX%Oxaap(`T;ePBmj%=^?+g0GU*BiLVuQNy!GOql4@TC= z_t+RQ=mR#{6>K^E1D5TEC^sF1)#tkjJRtse5T*uQ?NC=9o+*}fLMfkp1n%wIgDi4M z#AlYktQ#r!Tb5N`7SXx%Q1n?kl^%>fE7<$4;7(<&SZXl&EK4uPwbNlG{^lb#(*xa7 znsxhIVYy5$PK8Ctw8|=LtieW|o+&%c_UFPmgyW4EQ^qVpC;L3Sv(V>-eyZQB&Bm zZ=v>|{2JEN=0mJh4E~l`{d_RchknMchhZt#GOW0WE^Q>N9l!gAB_;UekcScQ&v{a%$C zMyIT(@p68J#nr1*pJ3~!8mvsDeZlUE5Bhz9vWgPFue8Kp0#ku_{tI>|^y<<;uq;#z zTd$zMBvhhW-S4wR!ad9q)4zm0bV$8U4Vz`fu%oH)L!X3xS%FH}@=4_ysZ^75y14l$ zOOExcuHuL7+ZA84D@6Ggb}`Obe!pMre2k@u%3MCVrKE$6a#cX9!j1TK{rFGNrDcOl zg{~CbI{0;Mz%D^nz5fK`TXq6)3+TeEDdowc=rq`sE-T=JE22}s@2LfX24DOHjhDw80X({Xki-TS-EKgGD3F|fq==KYUgfC&E_34)^ zIY%zsP$Ipr@Wbq+emg<=B5LUPVg8W{*j}iMf6Z)ec|s+jU|D$xHlKlVv$$BiP|SlK zZ$+@I)E9(puD>J__N#)VM4(pSE`z%Ig1S6GQ`~oyoePsQ>^^UF$NM6>xN>yKT&fHC z^-ARr>MmsVK0T-&4Z%%aHrQF%Tb$Lf-kkXZ8!1wLVB=uvh1W~tvGba`0?d$j;|Ep~ zYw8*>BjUmzncw9VxBSQ+_j`4L^t9%OX^kFnRL_RodaCoo#$)y|_VaT^YLQw&upa2*RuL&*6tds8$G0} zF{Gj(A7^nbFZ{|Lak)cbiPFWNe`Awk^_=C0IZLvZ1HZAE?uokQ z^pxe-Q|*~Q%lu`3I7Q^kq6TAlgzNbQcFwDQ zh7-pfF?>AK$uCv>H1Tx|Pj-95qW=JXwHr1cRRee}*kI$8C>Ab+&2Bv7DX?e*NEg%P zYriOB+z&;$oAKA7D-?3@gbj$S?lrMInZuStF3=&h5f<-%0lj7ha5tPHu8HMAw=XOn zisff}RKo>)sy7B;nwvb3Pi#pW$lr786)50Wl^K93WM3>#^r+zz(DOzB`t;5pp-A0F zy5cxKIUZI>=nVlqbp-sD_EbYApl6VPAGU)Kj;q-s0DBWy#HWY{UxY-9#ls?g3?I!@U^sM0=VTaHkVelO0lkX}z%E9fSH&I=Um~N!DI;L&v&kRh zpQnG%kB#af;vGE z&hZI+7=u|sCK5X{cw#?)*eCW5;iKhX0jXZR0(#g7^so<@YKnn#t(cw2gR!PAsO1%F z68ST+dfE(FJ~3-3{}K95ei9Es-+@b#SewMtp``FSS^Ou7kA{-k^NmNOCiCPR81S+* zdiN5L2MzgK@43|<(XFZXEP;^jIB*tI-S%(E(4ql-^qg`xj}@;Z!@%3Ll=OCX@gS zc{;%b^b{D-(?CFNSw#9UJ`&UFElL2kD7C}*nK%R@I(Wn{-WbO3gqyf_7*tl}aGvin zTV5CrbvDwcd$+6ss2sgt3CQ_h{uU9@1$=@ScQT*DRm%op>C0cuhl{xFEKY1Wnctu{ zIzg|R5QDH^X&b?ZpQLIhsOAkZA&rkrQe7@6@1bULQ13!uKRkLQPfU?3G#p*^DJAIB zfqc-&tNA4H5jf|bpf+FyS59hD48okztG?tyF2kx07KB-2+DLxYSUq0^^_BznIjTJR zpb~_+0Y(_~y@dJPw`LSK0nN^=gN5TWjarWxzaTn`nX?2M)SG)p@8KVXN`tB-#(f@Ah(zD8OYVA z@Yf@9D8VhluA)~Fr+|E^qGJpn zCl7p34Z)xoF_zyB!);Oqbp02`^2fx#$M7Mdc08Q&{+Yp((_rJ!t3-16Ly6!K#RnPu zG!EMcy^f6@3+qT+Kj@cLU&*(#@Ko^2d2ZS3Z@~7~d(_nV$RdP8WLCIk=vd`o`G0Y1So>#9D zfAaD|J@VmsvT7EXu9Pa>$H%}(hu0IuA|Iaw6{cQy2BF)$?G|RO#^pGgw@ZH{-4)2>TB^=*97BX@F->x*RZ*U1st{F=~Mj zJA&M7`Nib(PLdM{#DkO91!wcq#oDQSm^*B>Y@f>i;F4RIqjRA5=YHsI z;fPp#4qvR7@1TD05`-r&J`?<5VjDb-2jO<%*TliP#1TDOBVyGA*tPw28ecjvsHeAJ zB-E0S%g^qI3rjE(Zn<(gpX%xtgsE=dIXtzW>GifecRqjI1#LZbI&2$MsZBk-nchIl zZ5Q&4m}I!svdN}bWr7F3`Zr5^sP1O!v4C<)A9wD$yM>7+@GT2s$q zCd^?_p6PP>+EEOBYEl6o)r=JZ!uiQ-!I{ z&0^hbsK#wG;QrVg@Ppljy4n0We5^k5!`ld|^7VsD6E+ydb9ka$wji*aFls~%h~0sZ zcyJC+liO$$9!^3Zor!I)N1dskWSF6VIFbuf*pe&wggo6V^l8A9+X0BFhmk3ViVW3z z4+0aT5{7lJ(1Xc@!PI^JHLv7FF03nkHZa4mkG+B?j@7+D?+r}-IK$MBGvFNql@i90 zs_*`D`Ir>FaF}{DnR=IC!eElIJZc6r<#7{A3Xd|u?V@;jEQ8yS$45ymC7q+YirzH9 z8#BH8psZi%rfS>LPdqGm$l0s2>Pgzt#F=w>YJ#Oln57ROme&&BoWai(Z=A-j z6sxk~c+szjj}dRp<(#W&%7Wqc!CXFkqRw}M#~eYq87zZG)v8}x@Jv*m+Qp_KwZ~Jv zq!VHe($7mQpKMEUZ4u8F#l_IRFBS1LF=9547mpP4>(bOvxAefY^uVA$Fe z<0+gCa;~yBkd}VmLA?n9PqZw2=K!7+$R}C@Eq#WyV8vH^?=|z)OtE=Bui&cdS*E4; zq`3=t@>n=M(4M4bS7>1U9KnL;2(X3Cfd>J-g!<@jXM_$YH-El)0=|@KxafB^KNs%* zf~)xgz4Tb}YzWO}!TTcVa9GY*C|7Vx6fS_NrfLto$pPoxj1w*6rhJGFC4yyK-Ba5_ zo|5P_)O%G`#fuiG_o^&-K>FiCekSx6>l&VYf~|#`@gYnoD8BlfP3~vG`n~2Fz9J1~ zm|piLceGNAhmczU8=+g4z-pu3${T$> z+?`)syo4_h#}C8NezODt4W=yNBcU3=cn8$L;G^&!)8J}2pnKH;9UcR3tLB&JIDZE& zSJ#cz{H`=O{PZfDd;lkl5s{Kg0il_A;|88`@(KE@TpmFSZ_lfo{u@;pFSwEW#;S=W zq$Vp6picF=kbc`E1m~YSZsa3$Uk>R855Z>Oxf}TexS0=b`=PFeOgC3zq((yZEy6wS!vhA&*+^ z)m7&;_2B}z&TGo23(BtZntJ&Hdqn(n7cXv^xSAj6C;qaEKQdnF^aD?yTByKJtyJo& z7FPJ(FOJ{Czi4@S4gc<>yVKn2CM*fOz8(K6&8usrn-_)29k|Bew?HA3jSx zPh7V!CRxmCpkDkv+6IsDrET4bY|3wxIQ$roPq05R>_tB={YHt*4Lp8`{eRGf z(N;8GT(E&piN*i9s^K|t>joZ*ZPg#`+t4arf?wn9&ob}O1u4=6Nq?LVi$zn>4srJ5 z{49=&KZn8mB&> KBqj%AhW|fq7f+}0pIXsu-et)9ss(gq! z_38c!u0NMfR+4PPP3lvx$0JEKc2%hLc2%hL^k+O3z<=6PT*S>Ec6A@3pBtb4%m=Yb zQS;M2Oo5qwnApnQm40Imfhc%5 z3LgK=wKO}O5H$AmvIUY+|g6JDO7;=2eZ`PZkw zo~dS3nkx9Tjt`sgKWcbG6kIal({zrm@8zhXe|ko;A%oQKwAMj}37_OpS?w_44{G>g z6aIvT7n<&$Nb?nkwiG6`AmVYvtW0oaL*8?uvpRFybWtl?7^rdSiwG z5181$3QQS}4^=}{WWqOSxW|NF)_l4u6P~PUH#=a$ zztRXg^?ZM|ggyn=c~l=QV5(rdQyoFG3E!;Y)=hm!eBC0oe5wimRx3ZH2iGV(uVobW zWRNw${y-bC+k~I{K;@9fgzHXhWfc6d85cC?v%!=xW3f5{DGHwaLf;WrZc{mzWx^l+ zPnA%I3D?^fmPW}~W5U1G2;F7EAJ#Z@*o2pB_$d>9orVWgyx0HNeYg0{eMk1M_tg<3 zoA4NI3Q|qDbU-biVZ#6Wpo&lFgJb@suT|C+^vPiCUq{sni%s~_V=BJXgu69XZ!qEY zT03qN-a#BC`71MJ=mw|KgvV%7eZYio(@bow2~W_^n6&C#)u1=R~{-*VQtk7gzO9&v5i+DC(!Q~GA~;000e!YKHLD0ta5amZiQH8U8ije?(wf;UIOttHpyL@MDV z|GN6=oXCn&VR00^C<^X2;Wultwab7j`ln|cFl3Pa=>*kB!2?n7a1=a#%T=6E^*`h4 z3`PGPQShZv@HJ6zPZYdzOBCrn9Hl~o3AZfKmT41ykA^32?K|RWTAUzD$94aI?k05+ zTB>Jg{{P&8D!##lSKqGUyG-~KT04~{{D6j6nQ-C23u;3LOc^J%5!6S)15xmB6g>W= zzMRn77g)x%GngX^zBCHHCJOF}f>$aypiNFmR$o^^snj~8Ki%;QSehy@a8DE^{0I~p%aw)Q)2{=bG-MZs&M;HRSC&1PKCIoct{)A6cZlS@GRhp{OcC*-{tDuI7}5{HG&FE zxK7Yw6MkOnV5tc&);Ltyi>vx)(K^`Bo1v)RUj&sPj|o4g;k!(@zO$(^;T>A}!zO(1 zRxP96lyODFPnmGN1IdKjbeS;W`i?4odtaGI{*}7wCJUyFSG0^&6aGi-m@UJEf1q{j zFyY^8Q@Jz>zDC1){C~ZICrX9GCS2du)|zmgpi?GXZ@;-OF0f*4_sVwkWl{Z9b?)O$ z_)}W>R1>a;Xs4L)jaqq!30HT`kiW&IjMudag(h6zX0I{fsaoXIZNha~DKp{wwDzkA zC;8XaPtT~0QsGn-yg3SPE$z$En3q(JrW$ZX|MZM3Lk8)ejxUaa7e&F{QSe*S4S3(g2%tomlJyVj8{zlzpk$iQ-vuuHRe-b!u9fnCj1Vqe31#) z+gYRIy8oY{0UPuT&Hqn5r5Xyi3BRlbtvn_?S?jpWgzN3>GT}mo)RJ~mi1+| zUOqJno@K)SsI|XX#e4mKJ)_7};euAdZNg*TQVA_H;d3;+(uCJ)_~AY{=Kr*oQQs$n zu~&Ah9Ry6cPs5u{xL)4s?aQGyt$eBpA6|j>N&d1-88ayZ{}r0>LHgYd6RvwcZWBI2 zE59oWegL>4|GEX#E7Y4R%+MMNnD9j!9yZ~6`S{oSa$>bsKD8HD^-r&m)tjMkK*txG z@Xxe{ilX4|DEO|caLvy@a8-sn1@%$zKomS21&;RPnV zT=N5pOt|OYs%2bb!gnb+O?dJI6|Xhng&JOO!Z&Dmg9+DVOM1iP z|JQ02noSi(XQ&;AO?a|4GWKTQDX^Yah0bcipVBy&tmC@>uTMdSo}t$7nSv}6UZ+i2 zfeF{!S!%*PTE|5uT=+!G*kH<#G~8ptb(z>@!pH2>rpAO%(+CZi@VnLW0&6y9=ob#z z&b};l{!;BA-h|(y;Ta};tA;yF_$L~^SivFxy88W5%P2HeIIZDpOt|GgRD#?l{1y%0 zWx@+Jyvm4^{^=EJjTr{~R1~~93U1ximvh^-j;EOLz-i6@&oX6%kwO1CqTq!lJU*b7 zFEZg78onk9zG0Wi|L^xJm8Bk2g((`o%Y?tE;Z-JlpN7{)!B6$ziv45kyjDT#$xs-; zvVW}-#NO)5(kC@M*@VBP;Tb0U9~$m3B z_Jvwg#*Z}Tv)+V1tKkhMyijZClnGy>;Q0L|@m~Lbx0cats_+{P51a5B4QIRi zPR*Y*+-k!Aq2cj;aLm7?WhD2>V608UQ%$(_=PK(mO!ycLpJKvq(C{o1K8tXYKZhyf zPOUJ&9U2rU)0Kb zO!&}WsGKY7#Z~>AsAcTx%~13&OT#No_-z_qWx^XY&*y*%e?TjL*n~gx3(fzpHD#1( zBda&zTeOB6O!zJhKV`!AYk0tfAJ%Y5FlGEz%V;*?-)MN)gny^utfH?LWW1-cI^KkD z)9_Rc@A3a%(K4o(D%5MZ!-ThJ_+k@YxnJ$L$b=`qui_i};sTqYWqA5#Fg8uCaClGO zk;Q9s95CT!x2WYM6YjdKnsQdzw;eSN`Ey(|1Lp!Jyx_dL$}KhFVeM$9R_mC9>t@sX zq({90A7a282q*cc3h{=FQ-%uT4S2HwpJKqn20Yt<*Zx8sF^~4u*r}RXZ@_i`Q^{#C zWT?zkE1WXm>U(c09x&i)S1K+UaH|R_|8F+naVoCrf7p;Qz)*p`-8+JT2Ha}E2O02q z13uV*CmV3%6r{c_^mbsJf(%22p@xR07;x=9Jta5GfF~HrI}EtpfEVaE@qf4>W3iqA zJkfwJHQ-4GywHFr8}K3nKEi;nG2nO!P__S*zrm0(%22^=z(*T!j{zTJz{?EySOdPx zfT#500;@D+r1oa8Dg&Nozz-PkbOV0afR8iawFdk;175G-kUvt7>kSzVh6+aYJ7vHp z7|I6>c!mL&40xsiZ#Li)RryzQ!-kAWh6+s6ZzA*t18z0o+S{N?ZoC24-eXnpWCO0h z*Q)fNYRE7emJ9=av!UZD2K*KSo@KzN8gPdJpJu=d_Gtb;jcmFhW3i#a3duQ~8vyQc^8n z>VHb!ncHO{o}Z~?D!_LroNh=rA)RMPpGSIyA>E4f7DKud>3xRuMWl}!(pQi^V@UJ! z1Ua71>LsAxJP=?=$040(NDoGOg&{o*=`DtIBGUT|>5)huHKfNPea4VZ7Xa|ty#$O$ zI^B?-h;*JIJsIg0hV(5+Z!x52Aid9!o`v*LL;6;v&lu7|9sqt;F9EkBoo+}kKswKm zz6Go+tK`luoOBGPAi(!znETLAELdI|U`(&>iu4y5x8>DQ26VMxD@^cF+< zO{Di3(z}s9YDmA0^cg)ZKt=We;B$Hj_!ZLWhO{5)JVW|Dq*oZyA0oZQkp2kieTMXJ zkv?ij{{iVUTKXQ+(W3zPTYCxkFQn5A=|3Z#XGs4Q=@o|b=SXkqO^eqpe1_jKS^Vo$ z_vpjbU5`TQBeLQFNR!z9oCyirx=K4Gq)@fmMt2dc+JXV#0z)K>dAtY>S?zm zNzA>oH>2An6hE~PgJn}@dY8w#mpm1T?2K&XPIPaI^>ycS-kvdIEYzyP7;d_jH! zR+_+kR?fXdDVv%kTJFwFS~`=JM=V5x#n%nKDSRmz3to&73-2B=P9F}~r%h>twQg)5 zQ4xQB_YKKvy%evpACe<6aCfT4Y+~>j@wz3~@vrU_7c3b&e58fh+bzr=uq60j%wgpL zi{0N}+`MEQzg{Vj7-J|iag4Fg!xBIT#f159IjPriY#4StjN^E%h>a0I7@w=r* z`R%uhKl$4?cYT-~%*C9Tv`?(&)_oqxkB>DZ`$*7}?o&lKI+(FqumJ`*raT504Z)=q(k} zMyz_7dOpUTxMG1uzr(4z1Cf#@#3&k`$+70l#);lQKJzvXU_NO=EDi90wPc2Oz9O+4 z%5mOcg!f4aO#T=eb7d;1Jf<>CAC=OMIw~s->wuM;NyA$44pq>GMI4S;_!Cza^(WOj zK%yD}?ab%wF2e7ifKJ}wvZa;GyB%$0)=l#Urn2tHC?-e1E+TF@>ZN(6dIQIp_e?+c zl75rLS&xofQvN!)w`iUZ%_|e>CM}mZ?^ZOk(mmb#614NB%ImymEEp%Mr9%oxo24NR zxeXdn#8`Czg0LSfB}TVh6o2*T_5I3-*#+W>M<+Q)C|phbf=UO_%8g~DFA4syQR6wZ zMml_Dk;RumgQ9_KWr|f|urt)kjbhHSo5psdH1(^rGLCtNYOQ<%{K3oOUkk5KE3+<7 z@dn44=4X@}R#cgr2C-pTa(oK$H6Ig^uMSvjC>%HIf30eDW?pkzW5gSmKgCabQ+#{* z3jU5?i2RDD`73W6d~QWQ?ye_9R*s#8-OW2>4~ay7CO-}g+Z=n4$E+cJ#$xra0ERu6 zeLKyhhCKr8I$9)9j_N%iiho}D;)ra(mvS4=_umoTTbJ*t?u))9(TCvYGU^ZYhd#}rpoi_uI2 zbvhDBa)NzlU^6jU+!S$S(a0NGh_o2iQi2QnqzXr}Enn{4 z!^#h$@uJ-hc@J+5H6l+Tv9KuV`m(%0j5lCOnV5!dH){i{YmK;`+0A597^*4$yl5O( zCe{`u4J)($2gw?%rWTmbAukq#MU(T(taWNLTpII&7jhXjWo2vk{t!_|fNt~=j6<^? zjQBC|w33=hjrIQ59`V8AJYgJQGRtXr9!ECDvbF?F@Q@qUn0D0?npVkydghHpUS(xHSf>X%PSM#ApZj(U6up!yDMjyzyv~3}0h^)!;Jb ze*N%Gy>jGESLLV(3ppv~%jkuaX+&B1zF5UEdjrZu14+c>e@yc7#FJV4^qa&-H;(0R z1Zf-dY)ero^-|=cJaOnxCdJTnh>o8mr_ZH+G(jgYtunSpOlY`sx*+JY1ftt|^^{@= zL;6hrZuEi~Y+&vo-c{PLh%mir;_yhlOK@1njT|0^HfWeuJT>ocvcdhc^vD&ti)f%B zt=7_qh>ha%r&jZ|Z_|z=ZXmS=GZoh*UU@2=j~_3luSw^p|5Cht%~LE-rq0AYXF5_~_H=+(@zH>6A1Py{^lP z_mYN_mUj^+XHCFiw1H-D>_9H;Vn9xs@-uQM~<`1-6gS z7l~Y@_|`M`PgLesVi?WVy)j;E_h}C;08WSs%8QGh8$>G9 zEEYcJ7!4I}F5ee(l>+Rf0>67Mi*LL^QGmbQASSNQv&Dc0Qh@W~^7Z3JQ!9+Uq_`h< zwMTa1GkVolq)8yf%JrkuLN}Ndqt||tVw@G7&(Hq<)(a;(*Y#p31_+PVPR!!aWh=VZ z{T@{xV4;xrd!2$FJ?Trqq4r3;qA2g80hMF1V(xQeR1Vc|&^dIqiBLH7#FMGw+>Ocn zJe5niHz;Oe6%hk2xxl5A2NJ*)g;Q?TQLAo)AjX1INEg1ytDA9H3^&LXUSvKYo%0Dt zS@sg0bjZcERD8kziFs9^NoU>$FtUevpyGk6m=~)s?>e1=9_ID6YopsE{T1d7YZv)V zGx)nI#95ot`Ghv{(M^;2M-#=@HjRBO?JZWWM<}lhNeMIF{rN1gfTddRpDc!c|_dY5=8Rf|WIxOzbC*e2wc8ypN3;Dbc&m=Lq-wk5i3+aL~ zzV>-Cobra1>f%DgVbjnW{X7KPCU%Jf-;u+agIlUAj4SZN(Fj-B7JM-iEZmJb5`t zX@)n(5Ny3x`839kr(YIMItP}B@4hf7hO|)p-3zH?oSF-_II|rbo33>HuX(Z?jo~+x z#vy+xcE2#bUzzodY2Fn(m)CF!ad7y`tUsRS-NNm>QVkPVxH)!EtV7PtWG!htb6Gj@ z)}!OaU7LT&oe* z9;FNWIpi9opSdqj+_NQ(f4@~+uw^X&>(-qw{)S5tw{Mx5=hd4)(RvOW7c|knD9_h< zzteZ6e|~s-E1E~1gyx=!bI9iWl2(!5x?)BVnxQp_jMbIH%)v8df1GA@7woEJY zi1Hlnf2UR4y>)WpqMoX~y)I}K|G9M{pW76PhzY(xtcU-0kPa4t^f#fF;R)3F4I0 zCZ6Ax7<2J9v=;Dl_QB&cI*akNw(a~w6_+GBJt@P7Azi;^S=bthRM$f01|!`ee(M>< z^&_IhLEC3@t>TI83EU6jBiko&XTb8VkD6Ho2_2FkG=Z;m91_c?vfiF>!C@O91N z;2o1k9MXy$RGta`50q!3ze~J#$8~~b)QN~ii+C6oy|jPw{>^|68s6Z4Nm_cC>OR8#&EtsEy#U!QQ5=BfTjt`PmuViysE_lf2SSgbgfq zuaL<(J{*8Q>DX{@@Dh`QTSs`UNaw(+OYaZ&W#IRG%DjhyL7wxZ z(rulX^h zxw)*R-tsteQN_w3%oU`LRuyNZWU{qv1asj!tvAaNtxIP1vmrYeFP)I z?&yPng}IhlSm+Xj`@ya5-Q8%zL(}Gr@m>beDNi!rNE?$kig&&;Td1_Ft)_5ltC{)i z1kNDsF0=+ktf9ct%o3|V<$AUj-jR#EpSoqtb;L3%v^9qIl)V((q*1W^V#GBUzip_W zi}ICMBCd?h4lx&P>>bWrdYe)|hw%C(kY}^86S+3l0*`^9J$OyF~Y^H*n|0{jW|E+S>ADDU=7HN;savjR>V@y3A zJ+pw_dNo+Cudp zLglV1z=s8F(3C4v1$nmc@V|}=NF}!Ez7dtR;+b`{WeTfkaXO} zTusQgLarBZw$cSCvurwBYvs1@4nn@9?_t!?hh(w&bGcC=77JDH?-V|y5mmm+yChT& zevgRZdR~sKz) zr@M1Y*xCi$fE{z+a$I!xLkDQjim8y$W-ez9Er(o;N~AD_%}+vK#BRkZNK|7^+rSp- zGuUlVhnSr*nu1uz>#`-TM_X5Z9&u4RHW&x#!_R5g%f4EMKS!nRzAB(*)B zFH8NMA4)V{bW?CSa_B51Mj;#Nh?y7Ik{w1qP0~;C%zctun>r0&oA`lSTZa{-NgGBo z@`FR%OA_&)-8sP`cTVNyJXC6%nkRS9%#&wgQZLWWlP^ukmzyC47boS* z7jDXzVP7ZGq$4yghc@94MnAL-&!@yr@1*f)b}zW}OODz1e85_klg1VDGl+}S9d&zj zATQ2)eT?9MwI+NSI7>LydmtY5I=Hng%(;qJ@-z2rhnPIXE9IKc@--LMb261xX48^@ zz&^=jf|py)yythYatSioe!(e|_NEr{uEsW}oax&sd7>R11}HT1*z))a0)==cw%pL;}`c#}|)z+4h$CWT85 zl|0M4g6kla>+_Jdg|;Bw^i-aFe!Wv(2w@U}Y;DyF-gPeh`0havvKt{}d$>VOg~{rW zNtpB@zn#GBxpo*Bup{?6*5Zt3u0QVwMets>r#rI!KfK$z>b1WLt^R>TQe0@&VE2+# zZo%_z6dZ6@Z^6)@_gX{8|Jpj~l^+z+sZ8S_5WNZf0MXnFeTiPN`5Ht&b~VxN(M0ql zETaahzXjDaISgUWqG8N_{5L8OkN;(V>khHFd{S|v6s(W6uwx=hf$4Rc7q?U*1D++#7R*t>_4*r1nRBJ36}R8Xettbw`EZlewb;bc9K$ z`SavgmTVv9$lkI4y{Ezd)l~>0r*Z|)yOp&(!TXQttXv?!xb_+T&;>|9^Vz(bCQUjr zp++=mmPY6(46KAa5V|(7td~$(kt#^h<5v^wx#wzP-LRD=VsCOy2DPVoh~^``4U=dd z#f)AhnoGpT-b@iRoy$N?2y9Eqd{xg#Fs)=FpxN3|?hP*_-O7V*<;gbWQOoWp&=b}O zD%Xy4t{~2^ehc$MZE@zdU?KFj{;jno7#mB)L@AA-U=W9XRa!%J1-(RJH|M==8Y?IM z(otb*5q~HH{FPveaF>PwphPQHX? zdJP((*(A%{eRgm*v0uVSVVSd~Z-{L}yp8bvKfHxTkzZ4fd(0g$an%@c0*Fh(A1)i( zrLXgAti#!~z&B3Wnvv%tvw8+AZ1|h}8rwjJJgIn9mhb^vTR)O_{bn5Ol3+AUBY4*W z%|^%BnPQ6zh_xY3ZUg~NEO0x~Njuj6Bi&e{uv*YE)ddCbp2^GP0$iTWW3{lz7bjV; zQdqD$s`Y8wsEsLTgVbaea2f95(%JYi2Y)2C43xHE*=fX*L%a4xd93_7WsjNQ?>vXo z!+va98V5@&HTCWe_Nydn-BaU#PUqG9pyVi-UKk z2rHewuMvYM*RyK{^CtIaCr+Vx30p-Ewu%85+Y_ikp^GZSaWlFjhnnzPhI`zFt;Z#c zePTPtdwD>94UKi3w*CDh>O?L&S|v-x$RfsO1O_$$Wnqu4klUTj!_m_p53p882T-U-3K}R)*?$CXOdZ_K zHi%k#e>Dno`wgs|eBfy|q%otv!x_E^X3@XTYAyc=objoBs<4IL9O z(ouCe=P<%HjBYhX*@2OTFv?JhL%xU+DjXQ)klh$r2qO(8fkPNw2%`-_D!MSrQjC~J zDq*ByZKVDCj`Z3C75 zsc@+!5IW$I+AJZKyZJ9}-nAV#`vvFPF_N{c9>az$-^saz{cNpX-;F)IaEPsS;7M4+ zK*#_zgCX7`1aG>-C%p@6J`*Km-Olph=1S-Am3dCc4EJE&{7k8*CAtsO1_mMOHgYG z#3_NG%^a?*Smf?Y0$g_JF>VJ<+rIMR%$_;h=8(r?&agOEQ$w`7BNMH&4H1sD5~Mr; z>5i1*c@wzw4wLiY&QnB*qVmf6hz|cN3o6@Hn-e_ikjwi~blwduv4;YeYI6#Oe(sz= z1V7?%(ZqtC+ejv_JHxzbkbK1&l6`>nlPvjiVL!Hi1*wrIIPxVu+7;`#CY!DtK0kM$rm9jT?5d`kUW{RrhcrWCbPez23oVa7Ez#O7JJBz zJ!Pk5r2oNu&U=wKg#Zwuk1cf&QVD*=#8+PZT%3C})wUddOZ`YRAl|$$MaY0hKt7ri zrlvj-k&yuo`5}l*rjvQC_}kPU<_60R&!GwYEn{(^Mwphgl4W6rzv;`FcQpQOrq_;jbj$Oca%x)P zSc-5-;FA?ZNJ+d*{!2G_5oqJOI}><072~gik(N?m=_5{r8m-5bjaeJSdfeUFQfp*- zI04}alud4r9O?xB+QGj)d}63pVlJJJVHoC%{h&m({Z!edTTV&?&5`LeIW#c^@c&*V z2Mok@LmCuk3VFe)e76LL(~9egZE!e)TZekHiZj9_8J#mV;xX35CRW{*n+Go;PhOAP zL$Eky&?n`vxHE|JKVc?c4F!Y?&iwV5CF@N7Q2i~mOeHH~RF4Q*CfHvN5v52OUSO9` zk_4b^{WZ!45v8Rv67~h8C$>TJ!W7%~24=c1WEK&tQbf!qHL8^$j&RS?zQ? z8mNVfN};$)jGj8`0&z4QL`-TTtJaBGziDb%Av!}PdL4AaNO%?GL~qxJY1P}^fp+nd znSC#Z$Z7oyhx{po!D`Xw5w@`Yj5E6)a`E*<93oAGiNZ-SjfyUvQJ&NVnIR&bR})w` zlvNA7J(Pyv*PedtgW8!m%(y6=+z#SO!NDD^n@_^0@}7l&rf`BSbSZ>+e7Sr#`M0+2 z7fw=`)m~D%I~Q@pEhsD{?HS2!-h7g@n3k32u=kU;A|5uKKXhqZo_qxI(f$pS|F$IG zrz~kuPHQXz=$v&mL-ZjR%2|61f`>gqHk7kI-68)CM%@;RV@(@0v=JJro@?G+Lb^E~ zOB(buj-xi>R&%(sMk|m58j7i03m4bb1dXx{BF(jhIN%{-pv@sah*`2hYm4yrAZf1n zG6h2~PIAiRM0ucUMPR8LY@Q9advJ8Bq;YgQ6KPtYg?{y#1p(&P;A$JCI6@0era3e;58-QzV_j(?j1aEI0oHv2zLi?_8u3R=C z^l3NeYQjK@Cni?x7~|-sAndBl85Nl$9NiB5IkSt$veaT>EbZV@Q;WvdLsYF07_#vc zN+v7=nCFx5OlcVw4rMP-i>NImt#o1y*fX+#XEP*pk_-;f0$Y@xSdn-FN5HTNo+~e$ z^u+Rqc0x_bw5kqxlD}NSJ%<>+`Z5exhefda@miz(90HKi!Tg%-Bh-a+`A{6GK(7i% z;t9Q~!?Wpor|cO)dWA4MqJ9$gyW595WS3z@gjd(%hRpO?5eKtAD`Fy6#CJdh31p%G z%8g6n6CkDwwz&g;qaio}ESu_CV3L0tMp$2lDSE`gBCM94Vo6sKaR2!gW73WAux)ViGQ z{h6{0AlV8d^2j#+ZVox$L8_BqC*;^l_%O{aD=C>nqnMsHqryWD6f~oYG=tOz=C?}I zfJ$h<3n01*8bA&O2`p(sB{bnV;G_w`MNYXAnm~RnX+x!^4V9WUR6!e7v~n)1MF9KZ z?~+HdZoIXEE^F*bXDvn8qZXk<+Q}`4K+?`F2BRuM=Yeq;6$oS-tLGHsI8Tus3?LV0 zzem4|xM(mmkZBmmo8LKnZ-M<85wN|LJ2bV;D2xgIRsn1+Y~`wH84W|lXbNa;rfEo0 z<{zh75EQxxw+Cg#r}KQ|PW~KTJk4oA3wP+vK69Gr{}$!k7dRYfGrz3J4;^$S`hP}K zsBtTBzf+zBZV6;u+D|9@Hi3nKhn(_wfD+dCCX9mSi$>Q@J}!vsf?rHF;1c#gU5E?P zIZ@;3h|3A0J3%lx;&iQ$!V>D~YNG#Xty2$9i2e<=mOh4-jZw9%@JvK-eE{%`u3B&c z>pCK?5~x)G>h%pOr*=f9+fb$nmWwBA*|-vq-=?-CuAiES!qI; zijMRX2C7H;WPWMs9lmi|m^NKB)$Pb9KcI4efFrcC_h-ZDo*uk}a6%W>YzR#uoNzkR z!}i=u4ekNP(MAlFzo956!ma0MG+<{Cl1j(kE2!jVsyGJ>@HtbjL12KvC z99Qdd{_(flwMrhUFo?C-;GmIc)3nekMTD?A=Cv_MHTj$V}(bS}3YG_IpU`m23dAWC5gdNJf`~?mkvASdtZBvp%4m<-%=x_i| zG#M8vsO4N_tsFS3PiTgteh(IWTmmAN>(RBJe}@GC9!=p9uA0JqWJ=x7uM3mMN*5RB z??)I3zI78cEOTRFc>czz;h=Djlja6TcnP7s6lX-5wkr4seOwRn@H}fc?4I-Gk7~H; zdUzEXpKyo1fLB4CxAk;x_ixrZulxk%5De=Cf%VwfHGvR{KNImzjgakCoNFNn8Hj-h zwixF90ThvI6M$`9UHnA&-QpX<3Nff$h{p60WMCS8HweKotw9>4G2izQmNjjJ_-KsR zicvO&m|U|%n!8E{@L>$ z=c3UD5JMnV=VPR)?{U>KRv0q;6zE!JNenq??+rUb63*d{M;knoCbWlzK8K7^<2Gz&d0S>JXD5P|-MfSN zoLsPvt1f!k5f0@laY=a^{OTZF=XUtk(rmca<5~5tI4drSB`NdYjYZ%R7K)3jIc3L6 zL+dH;no59+QrxKwQEhj_VR1^+Mj|5#n(2u+Awf-x<=O~ z;D5svUu(GLYmM>GMWr^_?GD|&g+%`Y$2jC;w{L}GuxZX^nhs+Bic-#10JhuEK}QOV z(l}d1;*aw*$8Xa~&#Q3!;G^|&Cc*y<+E0CztEO{Sp##1ZxhRw%o$Tv><9;yMmrA1m z>&X9ZAB704=6|0Z6WO-`%(LR%^XEew!sR5Sh5ox z>6t=BmBF|E6*^MZci43DsrMo6ffN7R=WyhG7w5~s?~l^)dgUUgj91~hp*5TNiRa%L5el4vD7u??Bnq z7rAO`SD>4tx|1Im$H~7!rqNN)qq&EnAhtnj6+bDFSnvPYAhBq63VMZYbg|H{Qsx`|H7tiSTT# z@NAcHOz~_xK%8PJ;WO-eMpez>smXnW{5%FiLXKTYb)^6)ZvxOrqk?53V^oP{mSQjz ze<7npu^0}kTO$k+afhxyP!M=7USsV(7E`0RWgq9)7~?k5pJHm@?Jkj!PXVS*)N4ci z8Q2hh^s@Xh2`CLlAanUNvGyf(BS;=BU2dbBu=+kK9r>MD3A?ZYc0PqI4jQ#G_pr8u z;0W|vx+k_qdY9tsc5ma%x|}$0G6Gv6VC8IrjY!4GY%W~-CJ}XaYO84eAaBPMCX`F> zJ7pWr8v|IIi?EF1CGHxA~V?ya&&rSW@4`lA6hs0}fi#g9phFAs_2Hi^UPp zJ>bkQaaUoTQ+^W!Ux#}bIv0t9w5X9CpcSwSiyD=$x*|jljYL6xQ!o?1Nz&Epi;#Bg zwYH+MHqAY^!_XkGulV#a*?AYG{(DB!or_dVnU!NR$w^hT1g_b$sHa?nfOAS5rgdK? z--36)9&F=2Y{SytMfN{qoV}v>-Zxq=K^3i=KdrEK&pjz&9WPozcPlw%CqtVQa!9%$ zd~R&VB3g9eTCuiD+|~*wVMm2&$X?D>FGpY@g78@0QY_U7zCwO*u^iSB0R@5fYs#{S z@D)RBj52L(!&q__snWjtIM*Gp)#q+WtZ*ZLAoBL$c*pv|j@`N3z|{*7QOc})WhCw_ zv@C}MTlEN6P4=GZ^2=0z@FLp2@1(;wf0U#988S|AWa}SAK&W-IU|5-w{U^@CGCj&6 zK1Q2(iaaFy14*2`75IJY^L)RDM_UEU=6QrWbhwXIOZFe7y6F5OOl*f`jGy*@6v?2| zI@MbCw)d3YUN`1nk2=xC8*(!BjvqrI32dZb(B%*UK@<$a6wn4f0?F>cZs%eG+$jVA zx)DANfK!U>+=Mb1u}53jC~Vq>wmi^z5xRLiB!8u|IZ?hDq2ew>%pD25nSnFLqR-VGZ)Ne|v_@UMJ%YS4!{EpaY(g0GB;%8I|OB zAe=c0#}CUUv6f2~UQVMsnLOU6niy*xMW=yJ9!(PHY$0xxc3MXI6>4LPD=iy}?ys$$rZhV0V-%Px%z#{!;T>Q4<1MZ+`)I7Lp-lZY+md z*;1hU7hunE#7oDTuu$FD?^PpCJ~kl0KQDsi2`3<$04b&OPg*W%*(6tn&V(O<5VS*I zPhoMVGL%n8lN64kG9=l5Mj3sTbHikkz$tDwe)q0!c4c+jf=}aQV0t7=Y>luK9Z~d- zDoFif)ujy?OMtg)OB>#9t#&=4FKu>Zv-rUo7=?$q>P{F1d@jyTl1+i24dlr{dO1Z7 z@@fhOA(#(euLz;du7}{gE`$za5O`sqB49!e1y)Iqs_@s9NBebqs1SCz9zoczV5ci- z_qhn`HC?G#$l-k(^%>x6ai6Rt=S$e&dg$U2Jv;(E+%}n&pM-H34!zuW{)Ll;XqT4U zxgeTOAp+YT**^_8@p2I30cLQb1^EF2wrVs&zV1a@{JP&vMH%4Y1)yNoudhs( zPCMoLudY;60QwG;IfyszY5&;<8Mk9Rbo-|q;)amXiGv`32%!s_M2G|Bi5G(~S&9XR z{C2Fws~H2PKzuNjKfr~fHv{N}qR{|DmfiisJvjC! zi2?}>f};z|eMVuy#4Eyr$tPC_3-*+ppixR7If|6P6Ab9m1PLSQG6+OkOXJl7aSj;W zQP=?Hc*F<(lCKQS-5GJQnB+(OKm*4HdB2*)%G)Xs`~SK&N1`F(ni(Cv(B@VWIXMmX zNLr@NG4Ma3RXPuVC*4qs*6BQel&~0h=LLB62&THHan+tzUt7{Hf^|4@wOZN6g7;n# zmK42*i*}6FAe>t*HfKa{3-koWBXHx#BRd1m6Tc z4lfD0bKJMVvZHf`cY&z$S?CyBwd$fJZd@YA^|2%9h-NH$!4KXK-6qiu4gRuD9 z@w;*Bqx*xOaB?@ww4KH6PR8~Z-QuX}yuj#4VXXskY_jls@x=DP|IBz?Tf`O&KH&2w zizaY~o=eAPAv6x?qaOhxD#w6z{p_XfFgvuLBscDFFpy-)?YO-cfEl8rmcS=W4fVrQ z4{QT5aH8qlubxjRg{HJ*<4yF-h-t9I6D!8a57{c*gqWHiu*hX}YB zfB}ssKQc+VYi%j;Ln^(Dh2-ZfSyB3U8gZ|ER(th#QsVNo55 zvTf*u=3U>b5=V$LWSH%w^Y|klmAK*oaba+R%2Ju$Re;_r6_WiMwGnid;vyV`hDJei z+lCQ!VgzA~fY>QvUfaRq%6{Db`{CBl#9e0zQzs>GhuXAhb76a+Pu;s{H!uWSNo@bR zuryKU6s46&mayyue$C{FP|7O-NadUso!Gd9QrcB_;q;T{2~O31TAvEx2)Uns?y~L5 zh0h}(FdgRyTy5dl)YKJY{{u!VG{7lSNS{txlp8J=RR5(3W<7)zp$T=G5Q8FNJq>}U zt6HG}xOfo4=;&e@#frOeNBN(aAG$h2oPb)|>2QiJS<)sp2&yR^B+5Y)ZF2QnoT(+X zb4UxqALBYjeo{8$O6D}ee35LNfk_K=!FGUyd*k^1F2pgWjZ*|)I7#NXC=Q_+jfG2x z_OkFHVqq*-P3}KQd;2#C<*u++1WrR0&th7jS`zMd&<2Or3$mBVka9YtB;6;SPq9F~ zX>+VCs6o_YDa>Vuzw8<%sOndBf%OoOgkY9FhqFoeE?}OA+3}_%=6@;-`y_PS#c)9v z0ybMcU6XL3jINZDKh!iyy?RKUN65BN=eSx*HYSMjwRn+C;vLz+eva&$2jPoHHy^co zo8lbVp}}I}S7U{9c!S7|3uw{+OHBsePYxzJvZdqw@cYTEA4cML2-hK3{Ui&E5uQ(F zNfry7HplC$@RzZwC~Fe-%oA6w%F0AqI!k5NWM!hArsI4~(^%C1hOhCUOz>UGE6S2i zB8{@=#yhf+7P6#cl(9N1gm=oMPxzWD6fVY}QurX!DEz}jrEr$?4qsD%yaih4l1Oz| zW%+?grS!WvYdig};A@IdCNx>CM+D)%uG;Y|6~hadHO1YFP^W3$liKNg zt&?oRR%#8-aAfCT6hy=-{1v0jG#VA&+?|1Pv&53G6ND`MnhKQc`*e^u8^fbH{_SZ9 zSOexalvk7R3d`OOzPHWJlSdb^Y)F5$G$6lb%oF$>gWqj=HTE_5ZN=~7x7Un#2EPa5 zw2`f0rZzu(1gsHSxaejm)Mv1Z}<4xV%H6!CQ6S&HXeJS*_L1JA8^F2-{m zo-RCBiO;-FU#E+HY?o*?Lso3knp&`vATen<-_TzG7^F+L(LE4XeTD8g13|rJl zF*%%ltO4{O2>Dm60d#@%fG${uE@8FDU2xqRP@IjdL2%{VnguuCd6=w)K;BO=$7+J# zKx1JR$V*yvGdoNsBP{@8aKZR5Am$3=6QSyD>N5_;OJc2TC z+g8Dhgx<&Y8rs};68B%VI9n=dGs-s|!wnyFMk5Z=t7AuCA@FSl8kjQ3l^VtQ{xiHO5V(ZnMSQ+*{>`@? zw#7mSEVtF*U)7UsLqRgRyX5v_Aw)d^cMk5)ruzj5Y_FvWdJgLQ)lqyZ3a6&zP@Kn0 zH#%kkMlV@7u~O0t8M*WEzBqh6>j>sO3pby5Xv3a%0eCuKxFhS&Fa8+;R=gxz$L&0e_1p2)mkhKe;dcvu(;IU1 zW}Nx0xGBP7HjS=T(ajc8MtZ?;{=IlR9y<#<4!(?Aug?3Ki$c2e@>D8)JuiN`*NN3~ z6L#}-e~`|sD4Iua(9NKa*d~yK%&iIC?~q>sMpH&pcWZ0JUfgwKH#LwY)AT4aF~ z@_x8vRDlYTX17C>r~=*7l@3~f^M{)j;fmH|Y~4nc(mw+Q^fb3co&$6+pI83JIBi!-Txq)Czji(KjG`*D6=-r0-%j&H*=w z*l0pV??8~8V5grLEv2l+)~*$JBs z8iT9&uHxb_o!oE7bR^*P{!Pfx8;RD^udxbkhiWEcdZd0yrNl8dA+CZrw)I=Qct9)* zyo>9zU|Hk)xU>Ktkvh}aMrnmY>C524^HVl&)sMd+}W{Ss{MK!+cJop0i`l=^;&rIma??_RK( zSU43NISP(dVjn>qJNS8`@9-a(d?WC{JGi;^>$;qy*qBtN$9ebiao$S2&NLo*kD_zx zOE7k}9i3S*=fpOe%h}ji&L}Pl{~evzqr>UrCRRL*w`hZy&+Yw3mlh&Tou-43p=k8r z66QOZIHI&Raa8FRNd93=9nE(f^6JmnygEVLpi!mPK_g1HwBbOc5EL;d^KvLe>tQ__ z&El5#lt&zQ$_nBoEu`sv^irP+AWcV{x4IQsj5 z$=QuD8OK z{k@=m{{KLI7uZgMs}oNGX(XIJA+*>)0I6sPyayT2cKDx;l9D;jk}Y$DHW&xm!_ymF z!2#IC4neRqftTNSpH-8+4#Txz52-f$Z~s7;1~>)QQ*fY>asW{enCszlIJxpeCLcKy zvysfS1T($-eT4Y_HMiH4-^kXU*^f`WK(6BO8Y3An8|NEP5yB|q(8v$@;+e*&!b0t> z9awp~5aa`c=@u~VR%YYRn^D4r>lr>rRdT(T+^_<83)baBz1T3R!5TQ9b_n5s1B~HQVQ}iKFR}1~ zmst5KU<+DeyzS!lvnfIyTBA=T(c#BMyV^@5dZCZ6;t1(L8a@$BA2_DZf3|`smqm}9-2@x%dV%3tR;&EXXpYJQc*gFusqEB{p zAdFL!&cug49W&*pXjchI_k7}SxW5FLgb-~Jg{b>EV;r=IqouI8rDF}ma*Rft}033Y`dCyW~FeWSUTY5;#>+h78QCn>Y>3rspGhXaQ znGya6ft?RUl4zwE_qIAYMe?_ik)ap5qeja|pX3L1gzwIl&s({ zzk!A%jV|4%t@|foz)rz{y$!tJtZo4jkmLr9R!rbNT;n3D$S{xgs9U zMtn@1)^76uXwCi-Yqsw1rGY>R1F#NY(<1m*1QFkeYd}@>0dlA{eP&@U+^pRw7r`=o z3BKfB8k|5*eVPBr}{> zhx==mIBzjH6Sr|=n9MMJxavU|u?JwpmLsgrVWU@{j>Qb0>JS3wyAP>u)nM<$aR@|f zwwbLMz;y3k= z5*RnaJ9HE)rzu`fS|kiW5E_Dpd9Dr&amZ!(-HC`m*?Ej~J=D*zTcJ;K(04p&b3xzo zppC`~IGnU?HI_4e>m!=Oxlbcnpclr-<)Ujo4m5b4I)(-Q;h@9EO8zvb>85-+gy1uU z+=2+zP-vnZgq|la_JTX7=`1I|hdLLrcIa~S7KBTN9Cc!iCpn^bL;^T$hsuWP@KSCe z`hkU~&r%59Jp7>VQrrm^bz=Lp8G=KjA)%{BLGxbi{7tw)1C=tm&`)Wj5mA>uyF&G@ z#Sy)3uL~}~8-Xq_(;C=`2LAsm|3A*YJg&;(`~R7_z`g9qE+Sk&5l}HVaK#HQ8I_v* zQi@B8nwn;oT3%c-(-t04r_>Z&lKZ%!f>+H2w8fL)|F80O5l^-21uOp-1#7Xct+=IL{^S2o z#O+Z!fKnh(0qsvXy3f6Nn!^t1=K|*JeUb@qukxAk6WgtYml_Mysas>dTIo}~;M+ua z&pO0>{>*~oMOfj0(Rg0=4APv`JcS{+q;tM99AuhN4_)_B2_0P7L#$te@wM8#(!+ft5E4kgvs^SJAmr*U`t|Q5iw-?Eiw@ z{{+k+xI!J^XbjuGA@w+nx{ceYm+gk3TfHo@!UQKOAO|^MLwdRt&&J?i$`}U|<+i+Txm3YOAuFu~A8;xY) zmow;^9i}4**modY(}AMFa#u%bejh0>v0PeqZJ53zP>uwVTp;s!LvKJJ@uoRRy1gm& zrDY9rc3cgq?_0d!+ZLGFUq^klaxJ{nY-PTSH;{9hobs!ob@BBCzAnJ`5@6Sz>aK_< zMhFeQdPGDWL3rN}k<{g*0v%X^P9>3obXd2?nHJ_U21cG9>S8OLv?K!O?Ci z7E+8H^ahJ7y8 zgg}VTmH_4)&UouqVMCr^STI;5sc*%3X|FHx`XgL?&zu|w)W@S-+%;4CZ_(8ISP4}+ z!Lu498>;^^d}N%c+OIMmVFTJPzUS zjj;OgLjz?L;_&uVV0rJDxeyGpTU_>LK&_A8Uk@@;A}a$o_~F25O!d+#>;uGL*l8^zV>& z3*{vR5`>RZg!Ao)&|2psw?XCPY0`!%C1rWZ7ZJ`i!nxk^d4yLZTsB2-navw26G@3h z{0ak3nU~taD8$K#qw4{*IUzR;g{bgd#FYosHpET|)ZI5iQmeKg!vd-Rg{TEpu&yLb z%fV5hst#`9_9Q~zT;ds6G~K&hmcH(GjA-woY>rYr|=CIU_#RyG_x z*aLf_wU)WfR39_6p(VMD4nk^jK<`WN{Q$0@h}jf3L(2l?696y_0B&B^Dxxmeda$MZ zE7JNSO%3O^kZ&T~hw|!OP8RsDAns|z<)hIRm+kDKNE8Mk5|5u$(Jd);<2Men+Ye+1 zguNLU4tR5cWNR&V1u~;^-Q{lhF39zhFJPCF3a<1ly1Xp_Yr~f)w?nuk!gUFe?BTGe zDEoj!0;So{+f*VUR!O#qb-RiPrzbL6DI?OzsG$6NnCd<)sk>MjI+f5)Bd&>5Ro28- z)d~3nDgOyAAF*QE-_Z=9;ZPBH8X9h}+Lp*`$1z7>`Iof-njsR;80uZ>i7t->@kA=` zBfc6yh*k(-G{W}~{?QDVGx!?*05n?gl!;Ehz}#1R!{#pELjJ4BNA$!nwfzzPQ^NQg z-SrnDNLyKkc}vCtbemnJjVLeGZ}){KZP3L@%1g-ivkvIS5~+%|Oa_dfk#cn=~S;Kt;_d{C91b z)MskN)?GtJfV_kZA$22JM=BY2{((d< z#A!?|K$-S~y+))XqU>?N#{8CQks1WRxEHF;jN*7%9TFU_nsvTW20no)CclP-oQ8-t zCm`TTOR7a*WQ;*g(;xhUl!HCWE4V0Ao-qr7Xa&I zRu31|tMd>R!J>=E*_KMt*jmp;UMMXSdY~*JNoCgj z+CdUvP=itcgW#6gag^Bv&LZff`Z-icQzjXS3}32%5FtiGzCI{UEo2Il8vzjRi$hr$ zsB}fS7m*f^a@3n-bgOr7-y7&5LWmF~n~6Y(=VbGY*`(T2*e?V_Ty8?Q<^pk-bY&QN zA&JH+pW?AdkT_Y_9cUaXQ!l-#@i}1?;mD8fk4F5jsN4c1RK!6W^&+KZ=MiUwy%8oK zQ8geuwIN-7?eabdpfrZ}^hjfTNLct7#G_>lky|4lOuQ3ljQ>RZQRrtW&4MlTQRt6l zz|eupbOk6G;oo$a#2uf)ETiErgZ?k0su`AJ_7&- zLR6R}AhLp^|BFENbS0mzcI`*`GIVbveCwm!myb1O63RVFXQ9g3v!F9O*7XliU^#ww z|9HkhfQ<@i%xC;759yS)(dZE{nhn$=;02^;9`rCiP>w)6^%A*-$VI$PNAk5G zX&JHYTp-s8Us}-Y1ayNAoOU$la3)VeVjXOYZr;EP7C?fQPY1B3@Dm8rqC(}D8s*U& zc2wRQ;kWf3D8pICEW|ZH+&silrA9;1ID6Ht%}{#-Q}SR!EhcY!!~^KE;)h17fY>Gg zp^9j4N0rjJlUVjD;wVfrCs_Y)^a|0CCt!FWo+Ka2Hvr*U+QEHtBAyY*@Pb~LjxtEJ zN`t*KoiYy4TSD|e0EXae1`xM8$YlXJ;_XyP%@iO%Nb?FvS3k)`4>czFVSZmZ3Hend z?HD`4l2*5ldPQ}yKc`np<5g!!Fz*)LP7M^;R9t?c?~7*`*o}Q`|Dl1oBjE z#DRCHaZx~sdV*xa0HjlX^@;$9JxotiH6kScQKHf1zhm!C+pv1bMAUl?g>Qqg7-RPu zcs>>U4e=x$n~{&g<$5LP4Y?^Ak*9~(8}XnY@_-bAZl^rw@FgO=y$X_#^>WXXYT6p4k(@!BkPi3}0v)xlU4df)01i-=BU2kavA-JEP{~GI3q7v&eC;~L zD;IFc1RF2x^9nQddmb6M=kXTuX6iTUGO#wxMVM~X(Q(=w{BNkksXLTq^X>KITpGV4 zKJps%>n*RQFKlgA(iezLL1FSCu>uY%%DQ^V2NC}#fYEZ@2fyC<^}_F2{EY4=Zbz-8 zORMUvcmXY(Es;ERrs=nHp=qYMb)QK5Zq4nqS6!pZ$1fo zj;A#TCLuWQv%^DW84f01?;Elz=ntD?A51FyIt%`jvy(;YMZ_SbIpQxNo-QJ)tpMT0AT? zt^<)kLad#p99{mnGYFTZpN0fg1F!YLI5Z^e&H~FKjO!V+>?p>MuE`Yh5qJFnNIjpu zQ+s8|6LGXM_UBR?y2aH$Mv`3sgY;u`-6kJ!Dgfse-30&riL>bBW4QkF3|vbFtU^6>%gvr7 z)EYx*1zb`-{Y>eI&jgGz`>@oi9}v;ngbQuEx;{L$&ZkrB^T>o#>zfx-)e9r^rHe(& zqqQC@LQBco@ibbR52UEaa?x|20iN1pRG#Pr!*mF7WI2k*rv(tJqPivYh6 zm+1(JHAuJPvTQMkGbkj0Of^I0e7s3PXZ2VC`!9l{*SqM>!l9|cK8Howm!JeSg&@W0 zPV6g28|Uhw%KSQ*K@d=zItOue&(~*B_#3?p&e>;B7)sNg7!!4WrYlX0fxvIryJbO{ zN&*Q4#(tHg!c_NXx+c}>RcobS#2Zh=l9j@C04L4GMtp0I;PrIuLT~o=aA>ZbnQRM$ z?|(Ld9TT6sv8LWn5RbvMgknRE6%X846Ym&`!h`J;bzihJv&gVeMA42R6BWSPh=^iu z8Is?;ASRgEC_d(bIB8}HK6Go1I96L!{@oejW?`*-mYh~!pK3|YK|{Y2JuEEA{-+){ z7tF|JL(n#A`+cQHVcRr~DoK|DVdAH_A8BPD)YU zx9@R1G738OKTZn|cNS@{Q|DKDveXz>vXQo#78aNEI9^BeJY_M+)8gOr?9=lUM?;kl z>5Oo1uEp&bP}=c~74gMjm|GAROr`0MF)34 zxNN<&H!DVJd;1kNpWN`Y;2vxOPdY6od$8a(PasbPn5P9#M$uBBlG&-$_Tc1gb<(Pg zKgHy@2Wu@~ye=)Hbg{>SMLpTlSFSKgRrLYj0xB&7#-x1<(gEhfkmr8_w#5Ku^<+^E zbzn8$g1w`LfM=ms0&pUo(KE$xPnMMSh9~IRcpZE_S6>HjHVZcPbbeUTm{c^`qiyXp z4NVr8FU;q1i3RF?kWCRY8V?hdsLt2gSU!QvQ?&I!b)~}U#Twa9;h6~-1Gq10Ho#j(2g%gB1{l0^-M5qu(h|O+Tb6U7A?e$M;^MgOSXn} z@~0v#N{8)Bje@}v#KMXFjfA?h)C`1Lp;v0aTJ-TKjgK)(n*Zx+@L+LlrYh@6C`u5P2s;0)ITs-njmmaneH=&mep7TBj6rE>3@ zi7}PNWANSUncA293zo%IHEl}~e;xUYs~*iyZg|IPV_qPB^=8)4H(rD2G6*A&5&jJS z{WO{^5aB);qDhqcnp6c2Tv)c=B)7u&pF^BV16d(*1B|t%){T^F0HqvI%0LISt5)Ns zNLh^=ZBo;lV904TMo$z-#gXzDl(#}7(;sm}1`_PSNEf|}delV{hkRK5q|I938eKdr zn{QGHao9|0St<&d;4POJ=uMd3s6%4q*zIR6UFsoxd|8TD7#h0^CS%lTVz4i3)W`=h zwBnq{vo&=~yo_3~;K<1;De>#4IB$Yn=Sm}OhebIdC>mt#Qg@w&O zRTENeKji3Qg%LgYOIZwD@MkSrnRiNz@?+_I%_*_Xk45l>r^HD=mJsWEim(b+wyNKt zN1lPD=Wyvo-&l7MQIBFxy-Z4F{UVs8M8 zj6Dr-nGh_-r%AGnNktLpcj`~7emdqJ^~Z`*Q60egx^6_?4AHrgHxw&^n6H={$RZQs zz+oPq<1~>R)FDxbiz>srX5ctAHE?C@DOJroC5{KO#Szm1L_$TObt<}vxj+@QFBQ{* zSQ>w(RO}97Ls+Ts3ub}AUMP}$9{*A;`mkcm*_HM42=QWdXLTrQgs92Pi+_tG=@?-mT9z zw?W-O>Oi_89&$#JhFp-EkB;ald~3essPR=FWvPs^28fAatli)OX-j)K22C5C>n}IL zcR_BTybfB661=Ff1z6C;dZil6jkyjcP@N{}n7o1SgV~214=@DQMrn?o(DhnAY9Q{1 zvF1GH8FoOyMUX)&J!ZAK4MbUxoPgH+YHx<^re^BU?&H z`Zi=AHRV(nNJ_%D#>7rNm~C=AI06si5=m{{PDg(3ek~3YByCi?w{hAOpbx$Mbog@}+YEiJj z)&PA)4#I-eyU)Fk!LJe>Q^dJv0r87&kGmD(xq@$w(TrR&!CkIXkWb6h@sxUDvuMVSU)_k}`gaR){!rj;!tn{s9M@%QjCBSgRbk<;$ zHM}Aj!N`$8{KOX<3NK9d=zS(KHja9lR$7S*Fz7%s(n8FWyDH6J{6kh$M zn9zmg^gwNn7+`#ZXi>_JAL*}Q4s6u$b3q#BkF+F3Z%G%uB{8C_ja_lcj>=CGty9@T zAMa4vD_ru&t=NLC&(;^i(pZ$JNM+sm*9}FBG(7dZ+^aUjEpN&&fxChf)7V%aS2$U~ z)8x{Eyk5+`M0D)R%AU_o%ZOFmGT{pdqUb&?%oNrL!uIEn#?1}Q>U#!p;CIl!#>!Js z#+aENeS-SY1;nN}$WiCU)JEd9ZtSQ_!Mk%~Mc3}^naJ!kYZJMAXjb3J2HfIU*^2y8 z_>KSExve|v%$j6dMuuyLR<)cwJ1RFESk`j#%Fl%VlWYzzKjK{aBujF$bMU9MEUQdx zcf=8R4x@0H79M7VZBhEL1t zJs!Nj3Kz*0FZAxa4PE8tG(6pdox^&8Z$F%yxqX^#_qjb`9PVoE>u zm+*^3&`m<2)LWFAQO%QBqKNL_) zb>Vr%IVp1^d_JOlZrvTUqFo8fnJ_n+zgL{|{@iep^gZ(z8~QUVpII!v=#NEcY_aek zz*_MkT82>rn19%M=<8{P&!@!&U86qzZce-VWU+XE02`H%2>1`<@g#ggF;q4dEx&~B zF`kNVR4l>o8Hi$5sb zhO;Qo0_s1i{fh&l)o>Qg2NydB4riS4?+%EeBUl(u{z}Xkf$7f=i0BL!8~fQIn24Z8 z+Aq_a;^pgIKudwjLBx~0v9cpvt~wzmXRzQ7MTkpA-kuOw%CYUHd=(v~s_lrU!_~0E zI0eM5*Q1ZY+H^=9M0uWfNSw`J5n&%8e*?fFv4Fu` zi8n{F5VvQM?aD#1aumDGBM&*3j%Gs{-+5469Rr!7|Vw8=!4?hv3PO*!-HbyIM&ayHT8i7PQ0X? zkpp79)s=W;Q#o}==0D?ea63Zu&t&1Qr+`SDxHg6bh((#~8P_~D9Q7g#6>#%^k^g=` zoE^`4^R20($pjYf4IW!*0T?!9{6C5cX8Hsc&YwCUmQG-!T%V{d?)nOAA#4-b6a3zO zF>NB7%m#|`iEI^j?iX`jV$myJVQnbAu^93S^K@N~#>9zNC$gBJms~UpOj4ec5o8|J zRCbXEc%9-Qu2gv9l^DE3hOhcK)p+QJGNB`??w#hSIU>olueijd0!XTRtvga5;HKf; z3@$woUej5oYYo=bYVp@m79!ehVD(+`-dmiDNZH3?>{t`kxd8f11~%hS{h=bf=L;^EZkbu3aH|%jhl-32em~C_k5iY^W_A#;~?(#?|v#=H>;Dn_O z9cmc)vH*|rG3cHsUm0#pR6y+RiHfl>KyGUVOAN^_mmh+B(p73d0|ZPhiAOJ!X@>iQ z5vQ?#EINdjOX+N%!pESOfoi3#Jz9!?96&kE57l~8Yvm;=6+)C{;Lc$gO>Q7x!w_lN7w_nGii)bQIo`(beSf1CBN0melh#6Da zEdJv@G4>TmaJPe2Jf#^2B*DqIC3)`{uUnYC5PokzPu+Ed$wZSK@|%RQ&og|J>qu?m)#X% zZ(>pZ_O9srCi^HPJFPfQE|1ZOwRDfR45sZ9d=5(s9&pQOW}4&nAaLOkNAh!u#Lzjc z$H2KonzB922H+!xLcsESD>NLD5bXCt3}jBwc)yyHW}q;vJ@7H+jCei-ljnSbul*w~h3QG_p=dmUp9(tOkNc=LF z&0`M)n~&)KP;~(IQz1nM!>uP9%+6?OMS9wgLXMNZZEv>ztmaM0> zdyBoynrdmDdYXHoIQJHd2&PUYm6GAwfYeTncL|?`Y?^=Z&KePg3xRa2od$6KPO)?$ z>ow}DU8;N!@@o&S7NU%V+>-*hX3VZKtT)pB6pcJ{+o|OFG}1^{LG@5GgG*|JdRB-* zZVOlhyS4faQfA^CS|ob)F-3@sx0y>AAxM0M?(x$7yZV20n@JMkv7KVh+iZgWSRlzl z8a_mF)J}2lZA_&;J4MTqD)q)K(1=y3zYz}IET&V2<6$yh9rX5^gSm07W z822#aaNIEBB6*|>R@wrLgky9R%6B@jpKvrsYAo`-R_lzK{PzYaCy)nk zk3P6PsGDOZZj-(~fjRDtOLM$7nLBRw?(VodqN}5NN~)v6+~XhO0A#sJYPX-M0jQ7u z|42xgT4@rqp*#Bykwj6{6WaZ#K&6kGF+f>?@1>Z%lf=(I^H5RtKFj2T3!Uj7fJ^YZ z&pH>&nALnhl{mhXopT9uk(P=49M*~7^ANx1uz~D%(Rmqb%@&HO%h;#<{7#X$9J?Ib zO|ft}OYups1fLs0`?_#tae?@DIe5(GouZn)(+fnx3f9+uL;+NC9SuI%Kp9pb-e19H z_?MF=syIQTHe8zlUkes2Avh8FwKH3zOt=zXAO;HXKKGmA6T#AaO8!Qkro>0X6;>c_ z3l{Bvcl)FD+}th_K4iOj%6)O+L+tG~ZWm)eVzI%ikdNTkj5fC~>*}hCZtqozbsw=* z&ur;RYxO2-n?iB*BbJc30YgT6hefK?0}JVT@2sp~(CQp(Egxf2WqiHPvbEl#l@Cny z6Cw&lkCkj_hj}Qop02N;(UGj-yRqYbzR$?_8v_te-#bRBb1)M@5f6N+)WviH{A1+# z5`*$F>)^Tvd1S4QH$H}rX-k2)hq8Qafk<4#B9a8s_h1iLjJ8WhTgzm_hW~KF*jY}{ z{==tD8<{M`jLz+65%bnSVwoU{*07F|823_r+)L@YRC{ofB?ibVt%dJe){uAKB@)-N zmAqYnIKGzk3ckBlCyGd%qvz8t^Kj+HR*{&?hQ?Ze6CD82mVnM}Gtj4xS8{Tv*p|y; zpSXp=Kk!H&Sd>b{?a|9;K?K(DwkSuirw}fA?3u8$$U(M@`G8zer}s*?iwSwGp>JuL zwYgkgV^+KLV1ZbX$0AY=pb$L)cgx%poDDC=PYJ?2!Tl4Ctx6Yzwu6+n(X>T}akK|d z3X2X)U|Dc=dre%=V~I(v|DxF*tV~B9Dr*4}#E$6y=YTUq<*Gr-OKLi~y$@9QPBCyD zOX>a=>OYERRXm2Y9r1?|4~|}%W=nX8_cU}P;Y(I$>UIX?wntoB#{&42?c(Y>(ot*| zZtGdPH38K%0xIgJ7#f7xZrbjg!d0e*3Bqp$3$=$MR~6{Ii!WX&0sWU9ag$%&r|rkp zlAytEc!N4B zKe85>mC>5Qkdl3N{Zd+8?sNTN6(wB#-fd2nt*#|vsYC|wgB zgB9nN8bP-SNlSA3`&7r{#uaC2k40WW5LiKikhBlLO zgxoalMRmELB04}Chz=!c6R3_Tj0Rx0oj#>8jH!qF4{1#(ZsS~o|gxtmxM zUugR1P!IkUAjCA=EY5DiG2J;uRBr;q58N!0@>yDs9GrL)-DzS#h;kNCNes~Bh9Koz zgf+P#NcqO7F-DT_f$pJuT1J>I-7F5~17@eqIp-Ed@HU%8u!H@{tCXCCx1)KblGE$$ zMA2Vi-T0?l#7YI@P_QNE>f6EMXN86O>{qmY;eejEMexn6mE|VlaxkviPPflymXY`# zqG-vrVAwU9hA&{mS)e5$MQK3Gd8MO=ZA|6*;-8M2BEz7SP%O>l&rJ z3zIiuN!!7K(_Mjv8)X;{6@0bCYbE z-UNiU3hsmu!gmnH{-a1Ep8(~YLtNj1M*YJ@#7?L*slqr*D%fUMM>&oiZ{sAmdPJ%N z=bG~0uz4q##2QI4V@wWuaC3FOn7orU4nU8!mdm?HaF9TiTa+iaicfa3k_H87xy@yD za{yvgbxY8!-H4Ie=WP@7cCoKJsu?K3DER~EDIAner{<)rQ2Px))N}&~0F2ls78kO% z%~b$(6Q2bGibKI#g%Ct-P$RWPWNd{>=I7na>XY#RYo4xrf8l}faAMZY%g?EPD_Hb& zGB1yGq?6Wmgd&DJK~ByGVvdsq_ZXkAk<)g3dtvOVeIxGP?9EhLx36|%7Xyv2p{YH( zTT_8h+~-K6l_(G8Y>GJNWQpz#Q3ui3eNoTJB6%8BpTI2-#PB^V*srBt(`)z!qb9N7 zJCtgpvAo2bG7<(o~;i0@f2#)G51;mkk&{@Ldgl3(A zEi5&7g^p@ng^~P;xVndhbzb~Q9p#X@s|uLx7qAb3{Q{Fmfv#zO#-QuaZD%R%D5W*; zD5r^$ds%=P!E8&3j~KC+g{Htx;NJ_W05|?$(<&{?JIm_`U~o`@<-PyTC;A*`)|B62 z*;Roa&jf3^)i2dSeRk93u9NzWXr21)@93%wOm{Lly_P}z?rAA_JNHi|Efv&K=` z#6oa`5JDVk!#!Xc2WpD6MFtUA+5W1v^rTf?y<}29r zynC#(bHu0Y3UkSBe}akL2Uz*T_~e7^9OK8C2qz#bY7S1eH#koQF%;V$OYj6umxUt<1R3RhiD(iMKyvQv)2d!yFkWS2Xc(p!+a{ zErsXjEZ(CAv_!JVsZAo~bM`(zc~_kL9IV@Yy}0)|>)~=|T}_C{IKix9%ops5unX&I z1WjU9V z;faI1jCaoyD|WL4udhK|hM|sVc7pZyUO~%YjAlrDOT2f2#qiyEV&@4K?bp4A+6E~u z4|l52K4IzNY7HbBAlsf2(otk3%io?sfBt%uz09|)7Xy!hseavAEIr1OVlgbG231=C zLt7LdrRpE^RI2tW%&ifx4V4_zvHFXQV=OS?FyO1JB-Sv3G6}sw<9Z%Vpey5=Bp0ZZ z+JS4{gI9R8oR z&fUM0A>59&;=&&+%{_N5E^IWw`>Ep8?<~aGt%BvUh$j$Pabf1^+{4sE@KsVT`v7qEL@|F8|uA%m4ppie56epTPDq5#2S(KH>SqUkHx{i zG5xlDEYAMT#t`(CjS#Y3LuP{RW1}bClB{mT*3M*l~ zPzdmqY#~qoSPZ|0>biX_W?X}&Eb(J;@)}z_s>Q!ln};~kyQ=MctlDizqbaJQqU&9k zBkp0pJeGeQ2IM-62wd{9)-Ugib52f)_4tvpKwJO) z$KsP4EZIA&gDQuTN-AxQsQ!wD*@N}fo|as4OUAIdoIc#eY4jZ}wxUahhWftML57#U{!fzE1 zI&m9Ur9xQf2~q)IV_-iO8Y7*;+Y0h|H)+RgxT4!XyZv37KE)nz>jGfs@mAYmcm|_& z)51cRxX&GN_Z{qhN5FG%gSnA+MJ_Jn-^ixUr+>%(J-r2@`Lj;vpyS z{PyL$trxAtn$h3Tf@iu*MaHnqpkaZEuIeT#`IClkujV*gmt?q+^$ZPG9uw%nk6hz*aXvc#xzJF05p5eTch3UJ9+(R^AsKE)R&`C|S8HVF|X zznnK-4D~ocshC^GR%?fL{;SnPJ6^%XEF8?yccmTnI)_(j2X<#yX$N+rRuNR(Ct9r? z*p;lJ1H0bh+FsVcX}QVFFb{pVey9kz#g3VeL#j(JF*`5bVy`kjcBL442P@N{mChA+ zShWie`N+BAANDrm4zn||n(a3EoWgor&$d({8N=ZU|3ZAQSAq1K)I0B@bc*dwbw0^? zOU`EqG0TNJJ)Y|zZ${RM4mW2G zjvwESNWp?qW>3DuGVshYwHG1Fbwi-3FN`{lSbZRcbUrWHo}q{@&`a+6Rzy26G!2#MmU$; zppeouT-mS1sW=jT8;m?E94*gUBMxpjbl7%f+arkpbQ?62FB8Jevh1*Fc$UgtI)<2F@|U8Se5(OuW9cw217q+)i>904!c6l7jeR z|3*2W`4Riz(lj!GXJ_@b<~Y9%;@eET%~CP1KHuw^ot8y1Xn@ix$Jrx{*Jp@PZ;b>f zmzIhN4ftexE^T#eo#bx;3w9mZ@$igAqd>@Cz$R@B;xRu^H^p0&GfQzofif1>=@FAj zh*2<^q&&4IlPi!{En7^Yk44#Ogfq18wxu<8ErX;*D9EsmbLGQwqA zc&-u7Ae%Rf@}?2C;i3rodZrOBrt|>iRU@2B=@wUdDP0sF#e3}OpNByjF){N`o{3NU0@mRDX#-& zcu)tb1dC#2UCx$4IW!l%EXtNb*JHC6RJK8PN%H#9l4?bc-XOP(+5B8i=cn z_$|+ZsDC`=G)%|5!{TaV9*+pCpBxQU^s>WZO#+V=-D3HOrVq1CJ&emvky0`l)nef* z%mc(IKQEc=!07Jq_`~94EPp3Jjihcd64DNf2~Buo&(f%Z&T31N4vUpdc#OAKZQ+E& z;&>B2rMZrQ7wVD%q~!5fxT(G~9=7BRPpJ_7QyPV9BOJ;K(_t|-j$btckIJY;v0_*} zZy0d=b7-mD;a?Ah)P#3q#gcd)>#hUdRwDMr^B8v>#`+TRdpu8$(|aNZW)A4`WRfQq zovSILsv%!cQ?sKMpmRjaWK#fNSR#fd@QwzMS4*h7`gyPRlS$8U3sd5S5>cJNTec{T z%GGA>JS02|=+^t@WIES?wnKxO@2Ad@3rO!?B8E5RVf@WwVpdZgWB&{lBAWWP6^csV znUhBXp<6hMO)bay;(SSd=@8=YnAI`KEvq4rm7sx#M`4A&}dnv+F%GhVOZQ7?HtINE5*OWuI*0&6&K6-sg*rd0b6 zoaxQDnFUeL07+89K9~(dMkp+g6pP`{@KIq{$FtL;kd6cBMYQCm2Fk47;*}eU#kptr zP-mw^o?+q{?~64pd3x}(@584a)sjIsBZz@VDm~wKR=4CQO*GEgQTx5c>9)MR7AW%; z0ZDu)2hmSS;_zQ9jR zX~%=Y?jL~bM09Ogh)QCZSsIDmIw01x<1Gy0`2z-}?Re*=rST&pWDERnN8z=CchKeW zd4X~?{!FNL#mqxu?o98va$*MFbkI~`SK%8@lk^*b1o_s*;gWzU##lQn1tRo-c zt9KNcCTF-v$>NErJW#Ce$Xm8NT1&mLl={0l`A|BWDuLNkS7V<*b>KdZHbM;#i0Y1f zzA>j@ce}U~@5fj8iSwO!bN=Igr%z`-o7F>wns{r01$kEA8T-Y`6n==8hl*icc!!{J zZ)ojV0KM|-5TIubHK zr137z>p-1DMXqR5`QHX|#;0Os8js{)XrHF>q1~6qKGITpgRn0^U{8>6(-fQ_{h@)% zz*u79Ra%0){wsfvPsPx#V21h-bo^8-?8;l$MXD(vW?ZO6N~MZA)Ex7v_`NG{5?F_p zgQ*?o&B}Me&$20s$YVQWPF}8q){&}=YEw^T{rmQb#oc*Bf2`qHnt@9J3gm)s z|0e3y=OLoBI}ZrJVnsYZ;30vo-6w8$=g<18Y=;tvuovwUeS7f6E<}uCb`M@Z; zA?GJRS%IDL(S@uYPD@Aiu5 zr}#L!j~p zm9rN*SN7)DS*I}z^)B*ww2OAZs<+r(O0uFTX~w7B~RzsoU$5oX&%Nfk9*``C!CSTrfJzo0ei+AA}6qRXQ~i=ESzQ zL}Gs)?^0T%G0Y(Ga(^Dm@9h?g`}4UjrDqZCI{)j${0ACgMZ9DlKmT)+tr8Jpj z3|v|%y|d6y+@O5Rc8lnNyqfpNcMz7?&b!5&LA&!V>JlfE&K=9N>$#d zWH3L+pV;FJ8^T}UBjJ-cR2K6w7tmZXdB%PPd%IB8*3=F0o1xRHK;NrUealEWe-myj zj@H9@E%25Q(P0?kJ3I2BZyR4zQxK2)g8B0!?qWBWEq)lr8}SYE=#u2(2EP(Vca_>? z4)1?@La#ojC#F1edOR)DC4pl7^ZY&62jE1KIM9(rdg7Irgb7H)RZ2191-_>#c8Z>I zDrEDH^J=f^(Y`;AR;y(J>b#~Ko$nTkwBbC7@7N_~4d>Tz>kIP(n@}D98YZ!0Q{}h z<|J~=M)Sj}2^%C|i!PZwh?`*?mdQVLe^--O&h8Ll%j)`Wu6qT&S1R(5*2op+R~;1 zM=?-3M=CgQ%wASoYwpXa_3C!9?`8fJdQM;Be%$UvQ{7kx71}bbJC?z0$OiXAu%Qi% z440;%`&4;~L;HHx3TT`Zi&d$dsF^z9=372eC9gaX*jB$3Ayb-B1zqapBcH_!tXZQZdW z=!UCJOitL2BjOlad&(P#%Q;!b@}lp|q5)6BZEhdH1DhxuNvvJ(i3VC)5+M)OW5Uhe zJ;aHryq<`g&VOxMw|QsDiVLdG3=b*Uo#DwoY#PZRItNX71x-kP723nY+r)%d`Pqmi zh@6NeA`KR^;18wPAA_c$46Cn%8)=`(5~sM)m&NM8fO5t9Pys zvci%PjG}t0nDsiJVlcz9t>XIYJdJ;|RV2;i9n%77O$=6OO*{a5D_H9#Qy-A*fHqb} zg=+(_8);c?+TPP3lj%rb2CR3@1kG*ODlW{_Mw2!N4+WPMTSeF`o{G&*nKt9*Zxv%_ z@mCV`&Km>3wtvjYl#BR%INN#f4ZhEn$Io)kn#V72{?hAW_**WS}Zcz+Rwu(l$IUqG`+{i^MEYyma+rmuaEL!N}n0x7a7`< zr)G$I_@<{l%u54Cc!a7F_eFd=8fh5>KU|MzN?XK(sBot$RwfxP3cqHris4HEH-3hg zg>UOyIovZ@1#V5e|$8j>aVecx9^wpH z&Ra1)vbpo=6}-7C-#t~#TE&O+UQ@-nRgeG{OciOX`P(EYpIFV`r|+0>^V zJ)v@tvJ?nZP>AWqHFQn1Cj=x51f;SCQkB&OhB)_4jzuIW^;yfU(QR>dUmXL}GOz%$ zBOu$8Xs%rES{^^+@swWn$(O~NPq3=K zypF%&5v4~)>XClyAr9~q(PTZJ>T!Dt;R0ps6ydW2CHAl9QIvR|5|8SMU+9U`H}KWw zY*g&GGD?JOa_;{f(Zob5p3NIh|cp6I7=D?jpz zXr=J>5YN$$DGIVR(z8WR)*9)unI~(>T{a`Ra*|fX)k#|NcboZhL=#Q6@KN+#vW36N zya$Ll4>P12C7+=}{7OcVaw_$dD_YGRPmSEb)AfX9o~+j*A9nU^$NPQRq#Vk_VwE|n9tJPwaldh$nV@(?X$;tu|< zOJzFp+~2{Md$iN@B)y~|Rk)LH@Nm;3UG&H|cJXB%KTOmrJ3CRUtW6>R(8HleZqg$w z3$fS8jupL&cpMzIWV(ylMLgDHX)KLfLD2+vvAKv(?zc1FCPt2_#D+8E*2-PC%Cb*OlJuFan`BviCxZ7)Wd8Cq z8yBw}pZCtn$ggeJvCdUmy#3`Yws>W0wrC~oPgPp;ZXnJd!rt<^W}@a0Z^iQxMC)SS zGV(5L4{rAE>8Qs2vzo423$4~PhuYtTlK7y6H*{A6ze@52agER@F6L&)NBNRCQOvFQ z_p&7ZF6P~WOVd*$)Yoiu2nA7#@DxLBA-B>f9EGl{jX+tOT1T`?B}8UwWdrwA+TNG> z$Xv@;S;7Ncup_Vv^I;wrhi6cm$d@)V2hFees2I7TwX~PyDKaX)gfU3Kh^I4)7<3rT zoAN(3uRkNS>x<&UJTe4h39S=WN|J=>0%jvjsXObPw-58PyhrzO5Z&Q*jbsrrd;1mP zU^u|0b&$x9z8Q$4^cpxMqZM8PY2Lz(AG+Z~2dC0Fr~eoHFUAUXAH{x#!IR!G9v<5t z#mH+MfgsQq@#~kV-62!YOaZ21thjKDH%1Z5ao(_Ar3v!@(x=o!HFFz2P9z=Y5fLp= zY7zFfWZjktY9@$jV>Pz9XsmPEasH!=xgag8nQR*C?E4KL?!qsR5j#)w=3WISKdLWJ zLC637Q9W__H2;|UjTei*xffNG0?hLSenv_VDCCDj#}5IMtC4Hy-s8}GEz5muM-!7IO>Mn zQMflb#urxwAW1#BiC1;C78u1lC|aqAIh zU^OF8!)k3EOs67_kXl(M?j^(#Qp@YajYAwERXMUoe1Lem-}Yd(xE+D(_BVlvrYZCM z4^QxT&PB#7SJhQr{=<)X3`G&@q`@PF|1Z3IGfI)ktc+ZGjGrg9SGVaXRf3mPUj@;T zDVWM9ph%nl3!mz{bS!RuB0&WU?U(61Z-)x;ugiJfIMAe5z|?Pm@+7`=u_jzBJI|Y$ zqrk}EwM~3^o)6?wiij-ZZ<=4$&whZSp(sMJ;{pcP`g$>8NP^Y$^+#>3J3C zc}YcgVRcR6RO%))J_kKwRs#^b{`i9@BzGG-qbDskT5l*MlAX8SFmfYXe8!e!3aYu?7RZ_rH#bjSFj7LY$QDY!p^>|k(_~!ppD;}E9ji- zHHCEhR_zO^vFQ32@8zl2y#VR-f}b_L8y#(zwZMeaudia;+HV9r@l$-(qw(}a&@)H_ z`>!O8c^YYKZBM8g^9jRYune8@PoqTIRUX~?TTEQ_rV!R4+RjxEI*98V%<(F?wH52h zp$n<)FFcPPfcF6`%a!q;Ga;_>hWz{I#qO&-!s{C(+Mthn3YCbmtGwCJ4__c{986~* zsjb9+T5XK*3WTi>okIjFUF@3Mu>Ocw&G#|)-SM9Wzc;?v464?d5|&ES3u5fwd{pcg zs4){pv*Xhw)qyEVYv4x(v|_aZCd97ad0t%o8=Kvg&x;O~{J2LL!GytL=nKN<8owTU z8)3o&m{&?!t>o)^7+%$|lO^YK7A5Bc5qlluIwxE7y3PkCno#duEL7$F$YLu%fq~H8 zKZijS3$g9hZY|>i8k45dVfP?2?i?aI+`|-nbEuenkB4|pWwO$wi$kT+?0Y;sSnZJ92=(&KEdV2_GgHgl zwoc~wl7@1=ucSU(b18;999^YTm!KA@s4+QI5RwL>Y^dmVv8fR6I0Li;%#U_dTwpI$SfB0Z+{4)RH z?fSfmF?%rk0F>g|^n)`|*ABvRrj!dvseK}Z+t>zKdLxX`74w=g=Qrp5v z+ukk!I~_00qN)Uvio}L$ZuM`CSgIToC4nYN>mlN7HE+SA>I-uXPYJQsmtl)W5+rOm znaoBqc?AHAN}3{Se@(C9wm_B7NsZ(a;C)2lYRb_X-jiF0EcZ7h@UI6iPcbF(lEKR- zn}+kM!OKsY()fpig+DW;d%Q)c0*U!Rota`2X6oVB@THsSs?{F&Hp!_Om^)L|^dQAK zSnOt|$v#~WCYmgP75lq`MJsNaz&{=&)^JlR9y3II%T2>O`Ke_dwF>kc{NMA(P0#0| z*O87d&c<9qMV}m(2(iv&8tXNL(!kONQ5sIrf<=OhsRLg-NQ`wcP3FA^iL)-I6pzt^ zz`XIsM82=6wzFW5zXwt?gO&FNi4<4#Nv}a-f~#qF#7=;rPRryrIUp6}g(l`xQ$aq@ zBCne%mb(lQW8F*xTq-RG8jC}2rssI*AmMK|E#yl^h?Qnj!+=t0PaLf`(7BuB5>H$T z7%YyPO}%*VVgctqNf9=x-s^5MIS^=z{}(`|xljYs)#I%T#bk>q0%iqtl}@|OJA9y6 zX)$$78rfJ*2iWxTSa~zD$K$A8cc$8z0oZXrhq>9u@z03fIJD~JsG0AguBHUR!ZZh~N5#}2 zQ$XMI)$Wd?=~7igjLUm(QJ>TPEI>I0P*ep?TXidhyVlQY;FTPF4fbhSRJ%gaQSyR! z#3&C_GNc*jDi2cxP^7WMi{+lkB>-lojZ;?$phP|BvNL^)-^~8N9&G)+URbX+H%Bo(@UnQ-(ZDGzK#FFngdnxyFR8H zWOCYBrXFbVj#%t#>Nw=X2C5B@1*&XGwTWfzqd*3_O>Dvcq3cWFqO8{cXWorrL_~H0 zVL(JcL_|bH#9;{$3CV~^&2SBu5S7e~1eFZU49tAYOv$_klNoifvGN+4x5yRK%!Le< z3=I{Plx3Lr|2^*vM%~|^kK^}wp0l6xoc+A#6o;`h$m+7y7&fqRGrq{inOOCxUT?^R zu~(gM^T@JY(12n*C(Gh^n^}k=(wHKJUv9)L}8ia#$%4nSLIP7EH1Hoi$!Z~2c^~~~cfMP~z!`Iv5v#FmgD~c= zqZN+)(~j>3l-Z8Zf2kT0U&)5xnz+97yCXkDa6|H`+>v)ThB@(%sfVRorb89jZ(>KI zm1EHf>QQLAtKzLi{{f;ePZTQUaq3Y4Z_sXMehFLj^>pFAIUJcB?ZQ7XK@V@MR{d2k zTM^wRoXdPvFWvO*d62Q{@&)iwy%ccGl*BKU|d}nv@%#ZbIs0MEBpsT*?H?EHcLUhUPHIs`7PabTb)2Z z_2PGUdlvt_YWtz#?5^7GPa3Cr@SHk4Y?9?&m=Cbq|1|P720TLj()LyTnX&jdiOCOf zy~A1GhgGCt|4GJ;p8R-za8-OwZ>55I!v!jAReUz%eGDGmJht)}?>b4j-o)td!#4{0 zPtw^?-r1nIs9_J6x)|ypWB&J)9fM5a^|obt#EX5Av>h_aj^e$IbNu;5;&V7qV+X}! z$49h=$cc)B(s&(vtk`mV*`||)Gcf1^+@J7ks^c9Q}#-u z+`+sn9SG(Vxz@*t8_cJ0S01Od!BC3}j~ih)M(u>C*p{tMvqnLsEh|bh9t`1MX5W=q zQ{aQbs7%I8aqGG#aZamozi_i}%vk~ECJYQH$uQYv@?G2^{d zdN7R7_Y-@m5P)#3aYr>b*%aq;psbhaoGc1&&FN(zravu#sI znGyUDyE-YW9tRHisuOs+@p6n^rCR}Y&E2^q%G^)sK zzxYaD*3ddTQfCAo>#8^sDw9#Dws2Un+N(`>D2^tM;Bjn;d9!c?Kakr~NSjBX@A!BH zeK&%S?llxC)WI9oEJx}UiEjIFDg{LHK|nzMkMiigw?Vn=#S{he!RG%wpGcpr4?qWmrr&8!(6hD}|;7@-&&_QgAtQQn_I)SxNwiOq+f0lKhB!$`{@l*uw6OGc(?e(Fnuqxneb zhq$h$;fW+a5Qfo~XueM$e0E+m$rV2q@XzDvi)d)|r{n2dG#~6bAJKj|^Kndh1=Le_ zD21D0$9Qrb&4(uaKEC@N6<31}-<(>ilB=N@L4LrioVz2*?zf78jI}A_ku@V$YVeUO zn}*|)`|-4HG(Sy9Kt-3l#c z@nwEIJD#@3@K?13ic5FHa`-t0nV0BF#xWRzP99I%Saf?%<0&$hkN10b8Psz=#%fp- zD^L~vSTEGv^Txt=KHR$^#Kq%r?3+*0`dee-H{N(NmOsfkZcMfe;8}y%_PYMd-N|$` z4%O`SWV#-Qs(boX+8odC4d?}X4Oo7$1)A6tb)2<4Cg|G*W(K!c$I-k5MC#!<@YeO zbPKcJqr?U?d#^{KKeRd+0ZYF#2T1j+pc|P3=u-tILv0ILZo3=2VfQlRzlwUeh$7sa_G+=H?w0Qzb;nrhxa02fa(fHV3 z?$yk%cug7Z?cBur%z@z9u?+Us3LCSs_f}ZRlXEJcB?K`lFO?6mQ01w-mo@}0*vv3E ziM~ta2eIbmPAWf)YfUuzJ7w#V?ah@`zSU z*8R|U$D^8kpT@7{k{_c<6QMw`%&L$3twiS7#qbA^9Sws=!L^`ieH!mh%@cWtNxxXF zTvE_E_)7&buS2-3I3EqESbJcxW2m#*afKab&b1FZvt#XVCR?>nGrO1U@Zv z`l9XuqpRUx@wlRvX*Vv3d6dFl7?48F;x;FqU3Q@(k170e*N1UvfG19uK?Gn3*^0>F{%G&bB$5_i}B`McaYv z4Z3>+HX;bqQ+PH%gOgGSc49pqpyhmcC$7B}4>U6$i#WG8n6GL*-OlD$xfbcMLwhX_ z+lc!1GoAl2z#C?I9N~P?Dvt;6g3s((gW7>9Z}d-c^%y;^n88P_Xa)AZP7T6Q0Oq3$Hn>w?SO|An!m=#dhm03$h*{z-wi%*H)jP$c3iMUO3-#(?Mb1|4=Ef)fpV*xrJ*8!o1q7*FG0S!_7Wrs;s2G-~O zXDsE;#?s7bxM5Xj;aK`&HpYG1k@uEf79;VobbU5DojzlYo9FPAYL}{topGaZvDye| zV3lT!f7x(lsFA~u;(myuw{rNOw5t{WoD3^tDR(Z0M<2)17jyYY$LUu2bJp&cN^@K3wVGL<+T~jzc25J{aA}1 zuf@>iT(qXSN%SC>4+wpHL3jSS8p@Cqj9N-@%R*Tt(#=j}u=QlNl2>l?YUSE7l(G>0 z->NY*XCZ2SzcI7{yvIF+r(EyE^b_&D4W8+5#28jf!qoWClaH7P9li31$=1<_dEIW->*&-YlWo5A@DX#wmwXm=`u4>;XthSi9aJ0gWX z|E`bbga8RyHs zmQ@)VqfuTR-M;oQ3{9(1Ev#e`MwHB<|H{y7>}6i8~QZvCDWrw<9pm>9PD<172Q@()yI|Pn(wUt=z>?wC-j8 zi||zD)29iRVbZ@7+LIaowUz%Sp*@lD-zT(d?&GZVv4nPgG%a3^A>lXMj2^G>(Kg&i zBPnwg7O3AEY5ZappDBd*8~Jx1T^QfP$G(hbKCZU-*n1?!zK;I!ZY1q}o!@C+8;Q9{ z0Xv!PZuoB`O)KH!9eGKl)3-0O&nzt6yvU4M7>sl<2&yJ+a zYx!uu*I}PEsw$i*xl+iM*l^7S{0&mhl(-qvA}M+uAMBS8d+eTFX3yfcli@$GXE}S~ zX&4=8T(gcJq~_Xu=+ifNKR-LBC`@~u4cFkO7w+i3vRc*73X8*rV?31a{u`*FyL9Ba z9@qe7WZ3LW^BMa4(c28YpnVK8eCgVHbd}}4)aOm`MQ$|mP5u*Z<_Nm}Ccl-789}Qy z@S%RyBiNk9y88ibsshjUXEkQjs%$c3tp8@CqzXJw&PF4(h)FqKC|?jxhu`I6rCs4TJ|02RR(>-#BAm8w<^62( zA#*~gg`Ws+ba^XQ7N}WSVfGc%kQC>#ty>+;!uO1hLQI+ zK0teH7|SMSL*H;p+s1#zEgojnzR%B7b8f?oOSkiBYVN}@`fdkiBU0L>7yxqJu(@8KtNUk{-* zd-!$SmLW#RPcU>14;g}7#jEH!tbf6i;DL;vY2}}fYxie7-YBnNHD>TxCrZO9;fJB~mESMXEXSOUsdPiZ?5 zM6odEz7H~H9^fN+?$sdL@D0D&@k}7QEyX!lgAa2WaA5pYAVt>jX&zrg&PHL_D!*Ie zy2GOa%O(vt&A2a+w%72%nHv!icBrXh^|Oj*u$(A|mh6H%c7&+u8Xh%g`i}^(5nu(& zNsRus!JPH(%#6*PZ-lb>lat{xG8t~JJg$3GCclLIOGI3F2MGZ1BgwU!Dq zItR?i4uEGjDY7<&MTL#ZjRTFFkMon%es4c%c~tGMmc5^$9uD^5vRr0y@<7s_G^e!%OD>DSi8IesuX~ens#DEI6}ax&BdaQdcr|)!e zA7s6-hv6MR^1i^onSlDJDtj7^8!)O@>D6U6{xXHa14WDB$y_*mSy}dQHOv9clwuFC z+kCno-M+ww$7evs64_m=a#*cHLes2nHa&9VWf-r>*xbm|5a({GCF}iY#zm|Mj)GY} zKCx`LCY0G*%?kY33oU+hV%-=0Wbtvga0GtZX=_symSZ-5-dtbfiH3)r|3#=n?V4$;)T@{l5H6MIY6^CagcO znJjBi*ZR`!OK8SN`jTrs=4?ZKDYBj)&IS1x=hyQC)zhj_GJ14BY!uuI54#8Z77cVW z+<}ZWAKh|`U3bF`$StMJUL)ye_zgTONmjdziZvR(5?48Rg4-yVmOhhBVB`*%l{XP?j#!Rij*R zo!lA7$@wO~jGLE=J1EiOGn=IUgI{2ws{Y`e1;tkFAAC-HU4NAKTh>~Np7}WAQDGDX zNA0a;{6|*)V}1J(#y@H0pT=ffe_Gtk2Yc>=oUJ1~S`tH?svwgSY?Yy{@oY0ci*p@| zyiIMb)41ZiKc(H_qg^|^EM8QgHz;Xj`t5A^)62O14v(uj4PJD#g%9?q!wPZ3 zqXxmn;DIpGkd^Ox;XoGe9bN`cdZy=1#`j3iTE_QC&ud<|h?4&*;6-oCiwaCncEL|e z<#RH0Ax)N|cQG9IqS$-9tK&HMG4N5{@-QTL(VTmHcyuJ>tX>rWyI*U-XCFw2t6?Z) z?5sAsM}qmy^z|!6E!+%&Ui9NVKGe~RxgWrlneZh#^C#Pj9Pjf<_7dbQZ&`-$27&v8wUU$_pl=U@ zc}}RMEVIM)?IDbhw(_x9CS`n*wZ}?<+g|KinZp%+o;dL0O%2ib(Awc4@6Yw|q5(m0{`tHyWPa9h$Y8e z4WWpErG-+JZICUwdk>zoGc4g`ZsV8Rb%AAx?X-%mM_ayX$I+HERqkpF`2_A=9ev%2)sQPb z4EvFni7)2L$J0&|A2~>!j7b0=_qi>WpnTM}e`MqE4r|=X`>~7p^zD_{Jq~)pmt+$^ z+kQ2K?33k&Yq&jWJT30xBRx)xW22mZdf;qW(2ovx@z;arApo7yEV&!@!hLsOEO%Id zHP~!m0S5M?AI+Ez`M^&1D5mE+L_f%xZavq}=qmGH34F4fOrL%x^fNZ|f`HvBH{(4) z;6(l#7aZq)P8dS|TA`X>18&EDVVKdbmoSB!UZq(XX}!9RSKe(ugl@;TCbEnj$B|`$ zJ^9?Ocf}=ekjLvrVrFLf`f(g&W1g_IjI3{GDs4G<=W00N%%bLE_&37ZJVz*?M0=sZ z;|DZfti>2>RTo#~at>F+K4*&UBg6)FIm>PliGQigGPt&=3|@v%d~`CzKt?UzxD4$` zA3<%$+4>p3_Y`b-n>SSWkYw%v`q^9PvM@n|jF)_bmA$mZ_7=TbWlwVk3!&Z*ds{ND zyAo)eEfqzDTiaWD)8~T)d_)&?BiMLnun;M5aYK!R!h{*3cE+%>VimrVsR}c`JwkX* zbaB-}C9Gqm8jO`3!QZ17@N^cZ)l$l6VRG<&TPTCoN;$0D0ndh0tl)Yct+`=KCr1mJ z(Pv=Z)5}wj>^mEdgJy3L*P^juXRw%i_6{GrobJE2#%VFaRxS7aTw_O)u*k+ACwkc3 zpn`$Tz(z^9dx0%YDKd>Bi-7E{G~qRM1ufbpI6;GCnpiIQQtCv(YY->OZlf@>>aM@9 zirt0bZ0LkdmVh`>Hoi4cxXbk_!3}dx8dbY$;34B1lZ0u!vc7gVwl8ahnP>r__l+?6d9@T1=Y`~GN)k}2`;p@8Ku`@7!x<|-Vd?}dBH9?pNRCb z;gZ5*DmkH(_7(_fuJ6N+5_L49-BI7tPL8h%t28gc>^_pE#8^q_v~Jry68cO#OXw<1 z)IV%H8*5ewL-;;Dl=~T2Q7%m+uT?@AclV(&ZIy6R&HeO{yo-gU-jQu)Wl@*4FG^(4 zA%6al6_}Hu?V<7WVqt^Y?m8^VtJ%3^p^{41s-2D5YlP=HT;J0WU6#;hJXwM*HlY;9 zbd8mK~`$p~^K~hUQVG#Ex*>4v1azpOZ!Og;C&hO<`0-t{ z+g06P4L|_Gu(Za`hFy0p@x2M`dDr&)LXozWnIV{acWL+gLR$E9;Ct%&6vp?^^`{xX z)T;ZK?>>2#0=5g$_KA>p50&EXQtoyk)P6({S@>NtY!{M3b&&OJF}=WBT3Ba8KhQa5 ztKS{^c9*;4y+ha_je`9AJ=%~aOe2SKA%xx$o>N%-o8XJbTOO+zHpCOSQ_^{=qA*1G>+A{islW~bbb2_$X){7 zWX*vF)b?kYJ>)zd+>AM!R$M<;uB%Av={v2cj)pb!qUWXplkn;WATtJMUK^4qg#U< z8^i4VBgdj3`vpx16+Mp|v4-8Ey?cd}KHiYBsLlfN2vxqO_NMV_VS%yiQ^AwtPXA%t z`T^jSF+s=l`{moEGwOf-fa5b`ygDh?nnvCClBkbh4 z8NVCneJiX}bMkfLgCoKrKDNj34G*kfu^jQT4Bu9>;TvDsgfo>3epljU9lo)dw!2{o zWOFI*Yr#$1{+lHlzg{=G{(#0__#6{g^jJa0u496m+H)jZjKFeTc|TQkGCKP4_~CZ3 zF@Y!IXy^$c)AexQGNY5~bQT60FXLB$fyOp04!n0lc$}ktbn}GZ!FBqPbW+&Ko$GHj zoD{xM`|AC0l7(Hi;bt&gH`6RQDlYf6%G8E`U8i+*g11c^0`t|o(WiBShf7gl89NVp zg-6;>sVe2?RioU!=yIJfpB>tqa9U6fZO%U}MBuii9+zu<=0mDb==hu{MQe=$Zd(D`6UT_t-H~LdrTMT?OYw_o=HJE9B2j8E&)Id^$a5z55 zYHtwmZ__+{>(t#tWn=VM4u@<6Siwesxb>%d1iQ6?v{wY&1srZ1dPSIpE7LVD1C4Z5 z=+ALa|4Lsp3f{psznbZ7Tq6HAK3nZMa%5*v&trsa>6@GkUJYb!6mS8m3q}5hdhG2& zGkz0Ja(%B*_;uk$_B|l3KDaLUhqVN$UCOfYJxQ=xu1dj?dwgWyl7rs;#>c8k13tRH z@u^T*xmta?oPRH-s4|Y^>%y?Hk2zt<$3_)-2h*mxGw`W1bBvGMnWJEMfMY`~$+(JO z9yEy^9y8$d?mQ@U${~-k?4U!m@ONRDPY^u8>4lLwtMsFMMAf2|`0A%}fa)}sb8tbt zJ$>`L5QwvuO~0e>@8v`uO~MHD)2D~glTCtOIHJwI0B!n6ncbA#D3mIjhGLc@##i2Q zR1PCV5@N@aG99x$-x4)_)CAqaXPDnN2`Tn}CA$>KCZ8KHL-NMn5QaJJg9j=*%f(B+ zm@W7gxzg$zLRz11N8|e(4)d^;Rz>NCY+UrVwF? z^y@c;aQj1XT}{{2qXOY4J23hEO<^FH=}5P4LivXrqA7m})0D%&e+d1!FZ$5AKLp=i zb=^mQKfO#`voOi=`(Ky_b%{xQB&{I8xe~7BM!&4nc&bNd{?qEH2 z+!BHVc0$UIk+P>6yH7W>*J9!@qw`foC}-{q`%=nnA%H9Rg%;ly{I~(X(B|91<0ys3 z+rk7KiB-<_V~+TXG5U_+s?K^A%}zCPq{5Y4T)L>>1~o{u+?$9*AGpM#uK zF4UWq6(vQc@sH-OZ8uQEJOMkx>-!P)H1pho40_B*?m{`4=q% zA3Z)Fd%IVg5sSxIPO*G|v+W}7d??Ha8x1>b{WKe9sv52=0tITZrz-Qg5Nur$ax&B* zw1xGQ*)I5wELG{*w`poa+i=w=HiGqoVYVxtC4*dzA6vk$UOl?9;R5_sJ^;nr7#C=7 zJ9?s|i*&RdCA|G2-E0>!BU9m)wcjbgY8*^1AD}+XO4JWc#3pnJ)t6quX;{nK8lS*F z=;*#zF4EEtD0$>X+S(yxdg!cvcmti0Ty}1fx$(G2+D=O^*5bQ9mWFl;{e@( z-)s`>Sm)T{TNg`CCNu^oFHp2e2(V8;94vLp&7i+Xxh6~`zrH|gOoE^Gb9lkhT6KXw zHwn?&4;0?X@cRXNU=ns}=7G;=MOEGe zQy#2#!l!t6Zb6n&UVPb=)m~-4`l;J%1Jv!m2CLh%E)ZuHQUb@rjb`<8Z5kO*U*g!- zW8m2@^63`5hG8tYkMNk}lX2~HnUw=(;Vl>Fw5n`4wx6-4 zUacLE&qJ6|wk*_ogrCQVnvXCX^0P$+#U2mdnBiV^c`xGpJMm()<3Jc+NnlS^xEnY) z?00_O25~pe4E}746vR+9xB6$ABZ`sXFaB(qgH&wnj*Pn@53(jcuxGqTf2Nb7=wUwtb{c$e6mCFVd*RQ<2k=YHEjmY`He$HftaF&R;enkdWE74- zw4)F+29zm>wp*ycM)dNU1{*9JyW=+rd=B0N8V@BgR@;cqJQsY9j%vl-oa!7c?j`zh z4}YS~y~IfS8$VfRO;{+u{u5p7C3IT5;N6Y$xp^fy~S`fH}of3V=r#hEIn&x zXT@8Lx;~;#@T)$92B1V$Z9?r~D~>Txw(b)^9wX zcpa_3n8JN@ksQ5I+@ID_q_;Skd;KD9^cJJJWf$qBH;Q^?o$=Dq7kS-;@nTf zSGIQIkllUkO7c!Pl<903RY&$ZF`VP{d$2614Iw*#wkkk z6%B5z(PMsTPUGgcYwSHz3_-^DQy)Julsg|nWBtUr+)6m|6XUtO3v}BLF?snE*$)t> zap_i1@(1cJ>Q~9BSEvR9>s1Av+I+C zo*TEPc3VPGg@R_62Pl?sR^?6bpy63enHmAU58k;uaCqtm-T^#rue2_Jdl*`wUv7}I zu2GNSBd$3lpMhe($Sw$3Cs%=yJKHU;g`$r&`S#pojN&%k(5ohJJL(PPFW#HTOh6&Re^_SU!KNvbm2-gpZXau$c~CLsqr z+Z9)22;1X*%Vt?{cNhu$>CQJWceJ*IQ(>Pe_Nr5~4&iXaPtooGD0A>BIvRlRZBEmJ z05MYA*J|(SQxrK!3=icPZ*Bh31Xcy%TjX`96gqX$n7|-)<%m}~bkH6KJ?#BjN1F$U z-i{-gn@jrkTPH1HBCPKQiO=I12O1hEy67|*6R|6`{v3eI&Q`0+wBXqkrfIO6)#Fl{ z6)5Vs7LL{gB8Iz9k|9vsJRkrI-%aSdD*lxB{dq}W`4R?uO`LvI6OJz{MIEN1nL%hF z%T@GVkhqU4JxP;-QA?b)v?^Gf#K|X&=Yqw}oV`C<-Qx1tvTJC*=ANSEL&UTsKghZ- zAWcKO6&@EOvzsSUx_O=D;?gAW6>59s;?kb(^e4$a6ahYdf?`9(Rhp0!Ed5TwR30k2 z&@Z8)o7P7WIU8o5Aakhb;b6;#hscU1A5?9Y^uVEFh?6fO!#<}i=hdgL4p5avsc}AL zsQ9wB@wnon;p%a^IaG}0-aSFVVWLO)F{}Is1zik%5KP7DscmdDnFCEPg2ug<&o&NR z3>uhybdr{ai7EUQEaU$=OdP=>J{@6Vy!K72M{gXbq+z08@G2``1k@whWAS5moOOHp zBs`6VEc+yF90m`!c6;b;^)SimLE>?m5e^SWTlrDIcEqF&zn&i2!NdEI^*c#>!^O!q zN(Nz`*Jrrs%EgVQLBqutNB((?WrM5XKCl8)a-4}%W}?lQiB{O)<9aNJVD^HSb8M75 z95mK>+nM2FgL731q=8UjHtEG)avf&U^nHXlUAs~7*2PeIjKW7Cck7PP;t^ti-^b{j zvY`8#aX8S?uD@4vQ_lDuGAEU4QU*p*jc4eq5n^!ROso6p3OWU^C}KJ8Z1@-WWSp*H z`)@4c*al;kV#n2xX|=n=YIiGmW{>Gju2uF9WIg8=wpf|yt~2bb`y7^-!TiX|!Kj>y z!J*M*K~Jc#o`dh~28)$J=p2L8uDG{ORmpBR`(rRB-?F;$26nXP%Xn!;yzG&LK$;Lq z%ZjVU&~enFYWiceaill|Q^}f|4*JiiVqHvvM#E7$a_hmj}j(Yz_u5bqJHq{wjZf79FE|W5lG;G|)`5QWeBQl0fzek43v& z3`th=MBw0KG$&RZ)~{4~RvPm)6-IEEYT*G?k!1x{#fn}IpTKJO@`56*j}_N*BfDtc zSn&lqr5Ag1?p<_!ETUXcOMZHcT_*fMnR+om`yb{RV#xnMYxH6!chN+b^o_(nW%um;9*0(57(>0{#R!M0V0D>=0dw7yWg(>`+q{_;Swv ztI(;NLxswl_tj-hg}6Y#EZ1rJ_9LGJG01c5KfKZ5GDA!gN_}I-(98sJAa|wKxGq6_ zO>!vRgelISQ@F|zXu0?8)R0e#=)&#zFGZ$^(;SWe#U2iN7%Z3utDspUzoXqLVldhw zl}e2t)uo8#3dbarH$jZT^vv>SET142X(%g0%%&eQ#7Ticx+OUaUtBOtH)FOgMz>&g zt}bxy65Zn2FDzU-eSvOn?yT9%6p7LEDG`Gum2pp|__pum7?|mZLyDZ6@J@dIaixML35yY^DzY4y1zfhbLtYO& z3#Kj2H+N0Sg;RS_!zS6 z3#4=$QUFX{qf%vm4Fo^y08#*458MT805-#X+kZM_)9c8&Z%{PYA*lt{JP!Lc9kON} z5^|_Ro&_vF+#xrhjZFTwLoU?Mg@PUFklPlZcK#P-`~p1s4l!Q{od8yEK}3E)4uH=9 zYTi|;a*iR#z-x}9?DJGAFW~X*h$yfLcpMlAy)pq~fH5baGr(eCKClE>4%~I3qfo8} zQ3F9OFzghR1LFyyu0zfMZUZh@f~-7^ECmWbcF3(j6L1sE!_L4SR&UM&4}gz2+aYfQ z?{Ka|ZUJAy5d7Jnks!=7yvBFRdw>loopK9M^JHPC><_I^f3j2716zRkz&7A|V8c_L zvIe!FAhT1B0xo#EQ!WSY0-gsR0IE`IP6W2>MH!(TD6T?Gfo-25LO}m&UjsTnXcghvOsb2wMUIlCfFVrA`gAnn<&@rHJ1PLpIg$58uASn6{MSzCK|9b=s ztOYIuE~rHo19d+jr+_uU7T~O-ow9onQg#f=3#0CSEbLN6mhC*U5~a;g*4D~#^y z5DxgP(@5ZQ6xH=kxiAMrbrX~U7}AWg2IkyC7QsOsQ2&ZbwdYQ!yb0I{Y=peL1p%W) z%)bjQhkU_3oH0J-|)C8ekRhEbut63D^WQ z0R^>5cEBf0I-ox=1{ei&=L$`7K8QfhB$ooifEBHP)*mHk~OR0Pp>XH1{l$|OD+Mn zI(5l~jUYC;cFFz;P!_K)xfEyu)&dj#y5y)tG)M!x|Ak2)GSc0jvhr0gnS)VZUuCBK8^r4C|8HATI}&fX@o=lFtHbhIh$!YoOc_ zU2@ns^s>O?K>rbi2oMHgBfDf(G8!CU6tE5G|2iy2b;%3Bw~XqN^TC%ycga=Y!$x<> z4ZyQ8UGf3Qi^p`yf#VSXFcH`SECI&E!W`HHOfNwZ#dXQe;ERCHDJc5F1f&E8sfk^( z`&y*nF<6Ab`lK#-8?XYn0P^#xUGfp|`_oWFsR$UjAJ_(L0=iG^k{upLmI9-I1y3Nt zK+_}?1<*gUOO8lG1g0P-fkn@v$bf|z)1ed-k(D#LTV zz?X-a<+H%#VP;v#L=gg;U>*@^M*W8%Im#^W0XoN;<%lw*Aki#qo<;zVAwcjoNr(ve z1Czi5Yp0uK|11PJ!z^b2&jJg884Jv^*HpA0i_CHZaM@zmdlt$F%mA)mVwSf6_q+yk z_`7F~SuSjVAa$)-j+urCyomsSnoVXoY9l;&&n!2B_j=zfJ3I$_+s$$|P}qSK0xN*6 zz`Am?oSKd5`2hj|R()ia*TcQ;Q<#Hq0XBf&RArV!rXvBLndLlS)aT|x+5c^L@NctR z0m1QVvs}+C>_dPvpc7x3wLYqMMqzUTlv0%rfmEbC?>rQew41q=_F<^8~nqlow{ zXg%;a+?_pUmUF=8A2-W(o6r!QKv4pZoP>XnN1Q5zMG*C;%(4lnIfE3yg66DQ-ZUE) zfi2+c&Y=k20sahgV8MCl2(bEsSv~@Lc^6^6898{#EEfZ-fUS`4xrVX_zpc?MyAvV? zOb7a3hkOonr0@^K1cIV_P@?CN#rF|opso!bKpvrz+ib0PO}A0P(qzTn%*SFU!RUFrDEG zsCItv7pNP62;@Qz3>mLZ0P%*Ayai|i9%uG( zH+(Cw_(kLh zxE|O7+y%5-g0f#O%iCbD9#{>udj$~y#sC|E3xFozHlXu%Bmfu!GyxNVA$Uq+*i;Zx zLF5D11J?t+Rw95Ma0t``cL6hi)xZUG;)3WCyzQ$FISjo4`*XlA1i$@|>4EG|ho2t5 z#;?frA{I6^UsLo&F*vvZcDx{Gf8FkyddM;$*W=gn6>YdEruAlFK$}z=tZ|&B>6b)j zdjGEIq>X{>IDQoeXxb$)TC!84RrrRsUlRRnBS1T2&0{LF@1!oEl>o)39GJLc^ZA17+8X?taaXQ-KkeSxdhF?V2vF<#y4CYa=iy-yMcKsc;bMLct0aXOd&adf^8O@@T zXf^W(n{IJOeMU*XHp{?N$92*j2dO{hT@?kJa!4v5`Be=``ejIJA(^_y5>qOIL~4U1 zFOJq-M&3ptrKUJKdRa`eDF(eKsgnxrrT$8q_9z01{Z$N>>Oq$$QSPr|sEvb&!iUAT zMo439GQhNqv&NyyNfM+2W;~8%MX>E2(9W-65silZCOS#AVAEft*ap#6Y6LwCPXYfX zy85alr~<5|TT-0Yog4n(yq~dg^MWSZ&XOE?tdQfy#ano%~v?Z1O*1 z#$s*P3#GbRsT;O|pqt@LPjL-!mI&4X39>qSR2+eDmJPGxxV-&0Y+|mr+OxBG|EStT zfv

ZZdga7bB%c&>>T3@^vv(a);%JDYW6bm?X^t9XO3HUKc0%9RS_Dszd&#Uo5rf zA&knVwBM21Hqdy!BlmYP);6X$G8Aig*>uQ3a;6`C7k#8D$KH0K2pU;8o{z`8O-uvW~p0#o51b?%aYr(MPYTS zikan5zzs1?68gaDLRx+UiWCA`zldd@pO$5B7Wk?~bn=FnjrsQu8ukXW1HB^9RyZ0J?Pz?fnBy+yT(VC3O1_FcJiKB9`?wF|7sC0Yp-88c9^ z`X@M{_N>RUE?R!LO;3KC+a$arZP#eIJ+OB6C+36V(n@aYmU6|Q`ap_=!)+=&N?b`D z&0?m6y?<5tTa~9$NwQqh3xDr!sC6G1$L$(dF z)-#<>kH(Bq2)GJD(-~Uzrx;n-40_wyPMJjr3BjlgnTz_svQXoRwghIHM`jke4)P+% zm8{(^q}uDO3F>JdZes9Dg{&BM3(z0H|0*uaQad+mNV~=ET^g&EdeMVc(KTf~Y}Q?c zQo0I8`;|hPKh$h@;aF53n|xD3eIG91(aJO>`ny@ zKl9L}ap_1OGt%ZV76l^`wv<&UK_9Zcy}+J0XgB!N7C6~h6I6coPCwGQ;h8$0-kN;9%=2A4^ZC*ob8JBk?c1= z6a~L@$kVY2z}%=7{C#k`0A{sqtmb=bS!uF8-o$p+x`Zp;OC9)%4hncECOLR{gZ4DZ z^%#vpeM>3qAx3!bJrw&(IWUUy!3_{-rP=<|vH|qqA?p5F&}0n(bzD;zr6?{+& z0!*?Um0cGDXmuOhr$g2fNW0t6z?FmE5=0N$SVIPSPY`*xi@T+K9a`ov9MuGCVwAKW!fm7Q`VK}9&7jYY!tD?6knLAh z#M13fF;-%GR?cH7+$7GCk_Vv7^t9asJFTEIk|?E1Tp=w(f#{RzWEVyrM?l+Q7;8pk zLIxt=Q)!hMiJt{J>T&wYjEL<5U7SW8X87L(dfP;bmBkIV1p)A964u%2Zl~CfZp#=7 z<4R3c6=X*&vV&8kKGelaecnmO@LU*#9DDMe$x^J8406j;?`)UirBaZltaq+UDN+N- zy2*4|EqQChP&gW>SnXt*rj|k-VnDCQHUciqV_I8hDYdUL+mK3NH1#>=%uibdI^#L4 zq4RX|DQO@L<)o*ih#)A;3|hxYTcxw0x6PtpUYcPW6O6iutp|7dp)bY-pYf8XZ2@Hd z&ts#;nkpWi)<9Mc*%96{8;cO6QYn8hk~)u$3esY2Gw9|8@Q5e-VHnX)6Qwy)K9sp& zAsrQ^Y0`Pn6-y{ok|HJNAxQA6G*6O}Y_dT&tTkC?J}k*4ihwRlQmAwQbjw=2Is>`| z^nrDB+W}dV4}0#R2z?#p+DMBFOF?hJhJ`XtU}pI3f~;*5mh?fZa_rMl$1zN}tW?4p zn5AMvgEeRDk5$IA|A(vzGI#7tblXGLs!UcHivx-w4C4y=_`cXr3LJ`t{ok}gBZUS` z1zn$uU3QD5QZ;L0vXv%OnRge!Jgb_1)JPkHQTbKwNcsBzkNs4bN5Q=IARV@qQoKt* zm%y%``HjCE{I)@sa|T;TBE=?1zBEBA#o20xq0r7VRgBY1E@aS39+DoikPG<2K$<62 zffg=PbT6o5E3?}`D|$)ZneO4}fv)_OY6k=mVwIt`D1S_*hpZm!? z;Fkkg%@w-WOG29hIwnsU=-AXU`l^WyN-%}ClZ1p8NUE=4zejOvDHmZl${8}d9x`{x zDj;K7$NpHXLM-;!T(7@&83Y*+G+n35cG6OtI?#4aCd)(+pBAZ1zgrm{V%XfxB!j89 zF#V~ewmt9YHG{bU^H9fzEr#s-d5>9`vk-1de<{

zyjf1Z^=_qJ%%Qvlh%1ln{Bs&-iEW` ze~rH!rQ?tVYAMqPQEvsEt~IXk!HuQTrt#?09msotbkH^{1qF%yt@YOY8UV{|yD`;; z0{oHAXs6{L?%R4IGQjGZh`}i~RU3 zKpJbi0Kz>G2GSKf$(!5;NyDTgkQL}DZ4g3sNJCqxr>%pKp9?_yr{OfKB|o!=Ne)_8 zFuNeno=BmAP{Ah9>FKmM5Gtsfh$=FPE(c1%%3vk~e3Qb5YD+*@J&9c#p7sVxZZsrF zS}3`vBafeQ&tP8)0ie#7ldcgfi$bvE-ggU(q9!AkIg62+hOzN$YA~v6 zBg~dfrQBe|J?se-=`{K(SXwSsfp(rlCkIObih9~MKrrihwo6o&A@!p*gVE->PeS`Q zk9NY6ln6R|9&->Z<%2#ykIaLmjkb1AA{TQX>Bim=$K z&$a{|h-xa#1#Be#<=#Pyf>&*Ve9L>}JrtwKM$i#ED0e8vSz(z_vmN-1rh9c-%7SG7 zGjw&R6l_xhx*DczO2^32qzd%$y%as14S+y;0yn>x_`Fj)AjI7AYrqLla1 zh7st`lR@WyVYYORY<=1`kC}dftx~!Yi8|sODTPTjkflD2iNM!-RJq~tHLE$ZSabU6}@|0d9_2grM*nOu}TCgrpaHB`_yl+;d8q` z9=H3>&ZC|0`@iq|ak*@d^Ss`FJ$^l(&&Tume4OWbn~EPR*Kzga$K7GwA58hHwz_b9;Us;+pz>u!-x z$L3OOrXgs@LX5&@8Dte*BfUljPdVVG`Eq(k<@fDzfXTrJDP`9AShI^f)GhMH$l$Qv z#rQl2tXbGc2V)7oJT!}>9}~<X+7kkp9eJS#b( z4~}SN0t6LPusj%f=un!ZjtUO#RU?uMu!316`$kS56&#ynaLg#1io5n#fxSNjUjf06 zQP-<`;@a~fpNtCj?==ky)3La>UW`tTbU!wDWUm^q7A*A5!gL zOCt|mm>rpaZ1C7#8Ht#)eubsWVvOHXIW{&nzNQ``ui<4Q)=wMpXbm&YN?ry(@uE~xh9Tyzjs|s+$C@h48e(2Q5Nyh~TrN{xa z3A`8_$6OOSAo4bNA31=^6Sf85yM+IIXe4}G@c5)+z*Tr;XqBHmG}0|MI5?>iF#S~A zGZf85henRe4IbaC1u%O&?pL{{y(2d`vR5`9j}=b9Voqts10x^i21h3?0L#bn=RDEt zc4*|y6NBkPA4u(*pVmF0IeNeGlh14 zkTZUFy)?7EX-e8}l|{Ng(4%`|L4a=Ujqj4^Mm~H@gAap6%<@<>bRlYe5>8axj}NBx zS`Eq2tMHgW3VKci70QepIVQMwuQmwEXQBEcI509f4?FstF&L5s7(vN%+(VU6Qsk*I z=;*m%tMCBAj`_cawU7A`eI$Alq|G1%V@IC+lN zgw@w=a2YxW7E^-*U}OM`ZTIp%*Ys3@(I$qz;GY9Bk%8XFYUE@Hun_o$eh8=Lh#nDub-`EG&GL z@*gnNb^7}JU~a`pUf_`Tbx!-0nFAyKtx%B4F)nvGmasLuK#VFOqi(N{G9; zreGMp=^%mb92&Xgl;D27aAV!%%!D6-^CL@6!GWw6tPqN}L?=f+JSDh)uWev+p|{O; zeI#LQa8R$bB(Q5>i*bx2=aFNBLwXg0tjL3s01-nLaPQ^Zxn~Vv`?YM;|;gTYgPYs^lYdKh`0&7-< zB}UePCAER&W5Mh;DgH|bNBR~7`zK}f#z4X1*^tD0;^4^10*e5*$O>91*lh<#t|0%~DlGDqaIX!CJaJlZaPNA+RZnUehQN!x!-)&Py_4j@S_|4JQ|i8GNaXWkbW!f< zII$H$vH;6@dq@>;8xpzZbet}#0kVE^9o9g^gKu)8;i%ogjZGf=FxkX0hi4yi|EuqzgB z3S{+?Z<A7T9ar{+y(2FbqL~^Y8~Os;T0A|Q75T9c zZJmjcIs!|Rh}jlK0h;fT2gvn9I65B>G9!%BFBsa zoC~;i8*a^@T|?&`vI@uA(DQ?&2!kT0O$Z*IB+tG|Q?N8x>bqf3&20-OL>+zt->1PG|r4&I@d>rni;2axzObQ;kvL8+;51tj= zulFhlvrfRWR^ZSgsj)o_WhCI{!p~;~dnfIHLKPH>Md8AOB0o;X$yUDXwh_=COHKtW z#HgEob}%z(0VGAI;l4mR)c0iDzI-;?wgHm4SYT_1*?AHPs{{6mB%FgILNc_IFZ<)K zE>h_eBE!xJ9+@-&tP!k0`s7f0T6RvbUs5$B4L9Iklk@~6NfU!vk?+qzBeX%%R)*EU z7=EGO*$AVHV3WTec5ZyB(C){H2Suiw8yp_Fw*sS1*fRapEj&GQm$im;z-2W(n} zn=#Un_eFY53=WPQelCu2x%=Z3zZQ3zq@?FXCZCHLUL{!l&sbb7tnZ-6%t^r$?L@H^ zf{g)OGIfLC{ee=@6O-ioeCfFDVBy^sW<_qBh`y+VB($I06F@$GptjK46LHGk0$Gm( zurO0%95_%~JU!C)JoH6&26nZ9xWgdh>+{UW8RrF0>|F+!brdeP#MOP7aAo&RJOJ6d zS3P9KSdLpOvdbe|&kGJo+6C4OX5F8Y>D(VR2{X3B{usWvfnh7xCvxSa;Mm?ZU=6rW zVGk|u90>3Bi1Edw;6Zz|K~{b|^dm2xk3sP^+PPOD#$@&wEC7~>QzBQMk5gwgSUFgZ zR0yZe^Mm+_uk&%pX@ejOtAlMT{EQ7ZPKHfJD~v*{a>$m8qYVRGF|U~%9GJ8ilEwmD zor&bm0g;*K<1*~^$-%z88X;M5n!B_Lg(E*qMs-7Ik!mnIelM|paAKQ+@mmB*?gTW1 zbo@~R+}L_>3dYtdNHUB6{3}&<+28s{-kgHYH0Y(wYp}Ao3*htkO{(JHAxWbE%gV4= zR?2c)|Hyg8nB~s^%*Kf@EdlWI$Y;eUUrjL%h;CC1ev2r7 zk5~N2QN8d2oRW(m*;VnU+h0G=aC_d;3vf!Vgk-^8_;n9y&}XeAJ@Wnq!F_sd5y{=S zJ0ad~jO@H1I58;?J7DwQamOVQ?1YTSo6}I98>R*i?NtTIhB8W-y^3(4Ui&(Wa$MwZ7Y0Z7ss_u#!sNM9h{VXt7Y2tVHG@ri6U(-R z{hkr|>q4|<51cP1V8yb9KU=tuojT<}kcTDA6;d!%{hVpRV~;8W%z77flNS9l1OJh* zSg@eSei?Unc_4T{`Xn1@74+6ZZ&MI@@7P)m?9~i77i*YzK>;J}&uPKoNg1fz2&`wW z5r#4ycTsR)c=2u6_*5>GqX`ERQw^Tc311JM(+S@MUf2oW4vzQSxo7eWA^y88#v%Vx zEE7jVwI>mzLr~d?!3glx;6vNBaX$^83&1ya!l!{Zdwdl17l5~cFUIIK$491K92`Di zJy`xB_=+l!IUb7*5>EUK^ED&eO^_8F5_#@o++o;-&!Mf6%@+rU9h-eSI&&*l{zFHS zlqC-=9}JVK_9UMtf#-JQk&7+~J{fM@Gq7<_BTFH-bqzJX9Wbl9C9i%LZyehrv>3m= zQQkdyQ)2+@>tq#YbI;JOcLJt4d42b)H<0gJ9@q|xs{MMbe=}g#XN1-_;9L6rLuJ^i z>YpjH>H(n!z_tOQQh^7Ctig~Tbsq%G^5H$2!|=EMh)@$EHjfM~##YP7?WoSC<3bgi z0%pr`_XTOxtA5LPU6 zo6YwJLX{|F+x>ymO7uy?1A)-?*U>N!1d7AhZy$8xrU&s}Ra9cjgMmp1SQQABL%FUh zAPXdxSG!_us)jA__Ub@t1$=CKB#UMN){R?o2 z?JdHgU%DP`Cwj=sgs(>ppx3%SFlsfn*S{F(u?5==F9yOH0@_{-B-ddySN$uHxlKm( zzXCm)F&10?6$tHuV#P~=&^C;kI^<<;AHkuU#uphp?v)c!k+8UbwlcObJIimec=#dhh-7&^#TBd^Bj zY(!p$?aB?PDTY$b28>!bXhhx$yX_kSU85c@?V@_LNRX5@9q7jF#oSPZ`D^+2fQ^+2HFjX-Gm8-c(kktAC|v~G4N&}unl=_Lm*Js5J=mCd|Lx*h`jbKH1%77KpXP1w*!Idw*x)aBVYbb zAhdLIAkcuk>777ebz>m3zA+G}N8X5h$Gd2`_Ym+s>^R6<-VcO!ybr&d0zFDU2uupJ zd;su4Ag~Mh(huSALo_Y&YUJyYZ~F)Z_$Ux)`Zy4(MxbWoRmfYAFGt>ryb^io=0LI( zq+)ZR7H4Jb*yj1~uLw>^=o`uk<(lkFgKzr;x@U!20wx?=JBlkt?10(RNq*Vza90b-T6ym^`7}}I*7AH=?Q8ckfYhqw~ zBF>)SNcUO6pCkQe2a6;7%n1(a7D`VoFxlCW>t_d#OZ_X*CpWbkKl=jB1)2Cz708JE zb2e_FT{I_HVT-UB$IZpT(E6ZR8XSdRsR;hm=boj(u;mu$Jd@iHt1^k`h3$VQ& zeQ z`1AQ6Sa5-FfcH-56Um*>H549^H^4j6trUmu9&!9yMp7Y&8CybxNX^AC5^*o3gVm21UTC7#!n0y zBf)+B*>O1D8E*A2W?&tgH^YYnkc`+PxpCyx6gn~($y&7aO8nA-KfvD)R=ow5c%x$UT85nw)^}t_>sL$xII9qqbneg z)244Q4}3rzE@wqwe?Iha4DXbC`tAU=AdZ1>&j4ObuDr3gARkjKWI1rXku{_Sz}g?H2sYv0d-K<06W zu|Rqz$^etS!soJg;92W#EhpCna)IHu=wslA@6N+UZWGG6#y1dduROdKdJ-r9ft~1g z!ExZUPWTP*Bd22D|GnUc%J}yMcS9h%qtBlOKd=-24{$k;`}!|{4+c+^2^~_Q{cp^p z$c2m-9{JCT;K7Nw3W}c1^X4Z+DmLKTStT>Nh7$66M9zE>&yTZl<_Me=nfGvT|L~0Y z*t3yVYc9{9k=|vbVRp-yW0A}8+2gk;e}# zr%-9y&Uk(&d^VVjfmnU?+4X>(1y~C%yGg8p{9ttFfo}yrr4#+zzz&GR4@Lal0^AQK z!`=_P5KLy2J}(0s*a??Ofbh_G86s}NA-Np)Wp@)U@$F1Rayg@@K(_U)b#NShD!3f+ z`~VZdWH<4-w2&MyI&d4`URfU&E5H^G6gV60-3cE7elqwx)=UpDh2BNOvy{u>SJDUE z)`itUoU*^AielDc_#mpzsZ?Vh6riI%};)Bl0;OBo=Ma?~p(y%l!$ zIwRU{x6%G`Mv~;uGJl~#Jo%rL@9E5Uv)_XS!nwqzPG_ys|7rLc9Z8Fs*Yh6gGjt%k znWSKMx4V7Ame9FeKb*y64%Qj(-wE&4K|iwi@!-(*iMtbbATcB%pH^8Y4Tn>-1j*LkHmt%C7GS?Cp;Ikxos(B)EpFaYAtPBEQ@FWCs zB=Y%AaM_W4{ulV+o$y2yXmlrh5Ac!X65m`P|B%jxptAte!DVVP<`!48Vo^pKhR&A2 zKJO2n+X>$jgGDY3I_wTMzR^LGy$7lit6;ZPUO>6(Wi57lScW`sMcSp;uGCYWMfuI# zeWqA#bfKIC^o2h;C3YyOHPM2^+i&v?4%$fm#dF&x>^4XxAGo~kLzjOO_IBtE zXyMLwXyN_gz~3F8CqMG?-5pXYmR?F&jpq%2cN$N zb`bu)zsJ?80G}n@i}si9Ix6CLmZdm@VPMIb%;zt-0TVgIrBQpF~~x{IR)b$Ud1&?+u{aB6Z6HNaiDnP z(kO0=jifqm%b=9h1%c$~_&K?r3KuVP zeq=r`DS$5PgZ%MjUC*&VnnjYYD7&T<-za@h+c9d%!1 zguOw=%;nVAyM(^pP#jdHmwY}PqgpOb-l4sVgTb{4HImke72wPYE@woa{{b!o)92k$ zQ`zl(zIPm63O-l{w{LI*1Trvveh;`D)_s0Q9KJk`pZX5^62HC4U+f_0$k))(H*7kY zKS=Tm(uAQVP2c-LH|DfU9{`iR&gUOX|I481L?DC1jHe@e9Kbeyu*+UNTuZJ65g(GO z_I80Y>{j$Z)=0jTCo+n$*G49;4fZ;v{44B5NR`Y_sB(T}pi4TN+Zcj2P4c$CX2S&B z^Uy0P)`P{3+}6|EUEG#7k+gx^GOdx+d1SAOguHu+D! zkFR!enKw!r&+P$j)7*$$j@~zt4Qlk}!pY4}kmcpUM_t8qm(S+54)N=`tqrk)+uD-zQ3knB;q!06q;-FA`eAb_HY6SW zm>#VkPcsLp1qy`xEFsJ_)B}*38Y^o9m#E-t(g&#TVDvCG$!{HCzoAQ zQVwgb$Iq-;-7MExT!!6p15ZBY&x)k14~7HxN0zJ)4jWLyLF|km@8$GmtV=36BC>UT z@XEjlxmubTE@t%qqdaW>j{-9;#?1d|0c6bn|6PE}``p+_#z4D!YeHuCis5dgPS}Nq zIDG?nVP;+MxP7YHN01#i`9JWezxnxzb;0WcVI62z%72M?{F2KN70Y|HS2LFPG{vWN zB<^z}My>7#9i@r5;xBajhuhm#n1~mZ&3Q)mVWl{y;>1xdw=levh}%JJg*qpU3KMaG z6ygpSrQ#>m=#J<(<0oR)94m0*L_9;t@6=IWQ`jj_!^=rxn_N64;tX7KexK?xfLM( z$#z!R)2KiV3M}7!?8)Lkxkc%Z$qT`y0=i_mnU0>Nf91q@u?Ied;i$`*v4i^!2Jrbi z3^4ab=c0|C4zQDX)Rs1T-u68WYOI+tf7$@(dvaofw{{_=m=Ruq z@WQuFxx4{h{1@CLatuP3nR!7QKPn(Mr8G!o)9GM*le~tGws2GrwtIa~qg?&Sf*}c< z_n=c${kzCB!Vft?4lU%axYVrBtJ#P+d?dI8C|l$l%%z_dRzJ##O%3(srywPT3s{4L z_K;2N_Y}yMq(dxEF%@2ca^vumz@w>W!wdyAyYt$vg{ivGWf2BWCBoRT-mr_0?4 zB>BMvNq>=-yy!rV9V5nE(%l8loZ<8*kUv0P!L?o#oBc7lOE6*d{Uy#|Ia}f$E9hxv z$TbzxPzLUW-9c*Bu+VXNgNdZO$g`F?E-xREl!{rR=(kZou zDB_q{FJ6$2!L(2uHJ;}@R6A^LAm1_8@zvxXkZ+MCjYvHhxF2r8i=XXNoT9|Fn3UNPxvpIN*-;#ymwn;ct%1!!Ai*BqP2U z?2sGDNLsUjxP%nGRK5m+RAv7gb18WXzB?sJ?zl@@P2R+p8m)EM$lKb*Pu2ZSf0WD2 zIk-_EJoCD+BYFh9kAfat@3@7Il5vYl3@Yb3eJ!Z`4ru@roors}prSTCLjC&5Zo82D z6Y?$BIX}}G-+Y(q0^|{7aM1mMK_-EVZ*f3gH6m#ua^Vdhb+Owzs&6CTl-~isOwoj4{i|^NStOn!QIp>9SZqd!{-6$viTt{#lOW zC%)#zI(m=>n`SsJxAG-@P2O~c+s<;Cksmyj3M@av0S#~o`L5F(M+4h*ANllgvWfAZ z?znlI0{O8kNg5!4$(sa7XMk)cHp9ucdx5VeUw@`kl-*2H4Y~XQRmK)Ad%Q4(6-rd!DiUvdR4XV-j2Udsc~DC!@B$1LKf@p7j)f&6*$9^WR|SKv%`56?r}K=Nkxj>B zNY>L-kXwj<$7i^})u%c|d7qc0MH!Bdc+~L`^s`@o$D1E^U;=l`XTuasf6NK=WRpI? z8B`XFTRh;h|FNP8TxvG=9Lz*Iq}${0yDjf&ChY2C4ou3D#4sE3auKZ}`%29!AWunQV!#9Ou|JSAk35d{>^ZeQULR{*(sQEp+4* zO!`2!#Mh4ZaG99_E{<}krv?3t`ir?#J(==BgPfn$mpG8wRdV!yF$KBJ&ht>#>`w9- zUpT&;juH=YjzVk+-D^gXr*Ut}rT%R4xeTl=`?&Bhnsfn&t?rk<(x9N53tUV`duF*B zXK-6>S=z6Gpz;r$~u-Oyb&i`||yTF{<8F{O{7$}!5Neit(PqTVYr=RNfPV*A=l{;-? zUZH;J7^mOrGINZbK;W{!2#4_Y_J`H!9R=zI`2=ujq57WAgWgKH&FV+Fn%qS{*%F{b zlAnT>{k|i&@mDf{enD0bB!4h~a`lyXm@ANekrqd8n+n0D<&-;$=epF_FWAndVC`YH z{ymKf)Lq{D!XTXMjr!P3!Fxb*xMy*KhLbp#D4ki?JeCdCyRmKYt_BG>OG!cJraAp`3Qi$! z*vs)%^iyijYPe7#S3fs9j`=?wsX%_5K$=LW)ykKU>j6i3CArRSl|KM31ubEdZK7Z^ z^&Ri9|25N~OfQ{iFa!^)#ZeWx9!h7C*LwPIf=dO|58aq|$<>eL68|F#)R7u|YI*xz zbj9XN>gzS03cdxGg65s+h^;6-9(kl}N4b+?5f!+d|F_CF*O7FTK1c)QYVZiS1Q?p( z^z{hXxnOQ;Xtq5WZX1qFJEq>g5j>%Ac5*D^qDrYq1|mzn3N zuUz%lQ(wPup2h1AlbItbI7#rE=xQ(oTxy`-OfRKiDfN}B{xa(8H`w)n^CtC`tNy!T z3iM)81s~Coyv$va?)Pa&%NEYxzMq{1eXewQ9qxebEqFyfts98g9aMd z@LQNBjS022e*D;D5+1B53vtYU%Bc(Lj9T?r$2!O{gr&}5xoDugN_cj zMT_1KP)A3DOW;)bz73Mj^=49Ex$0j{{YqZ`uBTu<`2+-(RIKx#{>REDZm;R}vTPz9 zM6L#o;-<9aNhW!OJ6kG0ocdC;j_GJ}Z3&g-%F=qdf0urWb5ur21O~#Z$hlNU3ficzT=oB= zzWg@5Bn_Nb6l>d^fbz8U``^}t7he=flCGQPI3+k=w~(cm8<^qRv-8OD^89z z=y$xRM7bLDC%5V$q3#v`S*zxq?i)#Sp8!h{wuiDxW`#yjrZ2N z%#0i(AEKPt9B+AwDfRrHHO9u@T`-A`^!A!Mx)59fkGeKsQ8fjRk?%Oi@yY}z*g-yH zrQ^C6l%L@IRIlXxzlM(fpkV3a&Y+3BI?oxDp69rpjz^v-3Mpo}SF_XO@Utw(0}mFo zef~cQ0y}yqy8s!i!AkOJZ0cesHoM5@Omd)+{FIZVKzIbhM>(=3sgPVN;3)e494kmM ztImx*PhTE~&!QuJ7^RM`CD#HeF9ny1O+UvGJz{kSnDGer{x_^lQ(g)P9{Ll>&>j*sU6c?cK zB3JThR@;~-!KFo&6PvZ*h@XPr&j~8C`9^E7JAa1(b}<0YA?Dn%VurIHxjHMhJjK*| z{^vw_hxUindQ$dj*;nhhHA1BYkSEj%I<|!Sb@}<2sj_E#z~rVjq(aDRBA? z=Q$6RbW{c|EvNd#<~Hhwx6F2qmb=U(+Q(%$*^sM2cW^1t+^e19777+oU%BevVR?$# z?zO~F$LUBP604)tuj6WcYv+6h65ndTL!yS){&JMV2d;)WSmOO!@~N$15j-OxDva5~b( zC+cS;^_O~nPWO0w|DQ!ix>Q0PO#+ve*z$m@@pKC6=xCD{U{f6a3H_@dx(S=D6zCEu z4g4(~wRi#kh{LzXj-Bgl1` zo$_NW#~lza@EIL>d;7!cY&z11#OmmLaJ!pLawM06m*{Ak7vTLkd^7z|qaSf&b z)fDKM%(54oSbo>RK#UHxJjt|j)r&AmM^ay(emjaUu!BnlR`YFSTPRpL-p0q|@G&>I z)ZiI9(rLJIcU6q*1+T^pR)2T?KK@e{l7m2x^zkd45XuG&(YCwcrN|u(k%6JGP$mlP#!*=0$uf@ zg0sM-&zAED7^2`la(zQw7I{y5fzs0`S3jxX;%C8ij`W~lw9^k8U23HUC(=M)IhRhs zzZgKd>c2vLeN~-anw?;;6L4A1L$&IUC3lB#ZGTA5gg{EFFAmhel{8SUjvk~w-a2TJ z>VHjr<*NT5>SuP;m-r*>r4d%6QJ@CLfJ=e!{>7L83#qSM^>3j*-Ub+>-$;Gss{awV zz5kDQI>s36Z!b77*YgTA5L^m`Hw?z;&!E0?4KSNr_j~2@$gxnVL;FK|gB9S>==rg` z-{0f#<#eQLJv6|>gY5d=5paL~WIM*3uX1I$~`F#vYqsjGsh`M+D z4P2(%%4JK^b>upqa1?h^!*rC!l?+`tpZ|nBb&6AMqQ%h@~%tl2`76^{R9O*Ut@VRfCm02^+$Pr-ipIF(GLq0?f*|G&?S=^ zU@N%PtcGvBNv2@vB>8|T6kAC34+j_h2~W7DlpjWubSd?9^{bJN2ql) z$S-!P`v2eLl^)+{d5S6UT4GBa{w26w|D&(POlO3V=gWtlMme!L9$X5lZ?T+1!Nb&7 zuKFvfuP@`&JEFZNiw2&MY~cNWHQ0*+`C>rFBtMWU_X(zXHOr2}j{=v1>f25Ay5Ktc zR}QDH|I28g@8i@y{s#?|YoXSY>zZ!m8|Y`(tF8j_V}z0t?HIug3UWtrPiYF`OG)t( zf6LZWaykrzJRIx$MFLr$C`wmRt?pktzsHK=PuZ7Cu@Y^hh|1bmCDEPO82HQMG zkH+Co(vcQO{j8;ao9Abv$6fow>K!`LmQY6@f=f$OZ+BI#4mimkQ^~m$JC6o)z-4gZ z=MP9+HlJLVZc`XG3+X6lvop}k=2xu&zFaan_FS+z4*!Ay)Q?7Qq5m!PuU)zCg`$D` z|0+;}0T4)y7i@6_D(LSd#nvFoRsRy|Z~VjQZ%K5Lhsmo02^PrLZX|tX{Un*-kvONc zw?715SpoKYIc74J52V2~&(TqF_-Jq`XpX$63#mt<&CEX-K)Iv%u?OnsartZ;1>yZKkx#hYuIp+k z7zi!_M)h<7Mv-4ZedWYv4)y0!-~JRYgwIl6x$3WT`e9RafD15#l3(d)*WQlvIifiV zQx_>vZkprzeW01-75yCF*u^<|i#%_jA2)}1l0(V+Yw1_(Mqx67TQ>N6Z~q@z6YIpZ8) z51E_6?bu=$(7m~x`a1@4{f$0qd7lOlsIJX7lWT!g_yrxU&t_n@#2^eNX`vlQIKGMo z7m-&T>G)z@f3%8%p*c=4iB0!AdEF?-*V57GD_nq0hdZugrIft1IN9-Z9$20uZ#mWR z3`Y3Vo-D9#fHl^@U9WO}*3!S~7lPyX-_xi-4bBlkhXN^|Os)k|ei6AANcjwM?K0)F z$z?Z=wm+ovAdnjGnBrIGFAemy+j$fu+lxIcQ1tZsf=d8> zJ-6Y3Yb^EKIsKfe`U_lv!g|rUPz@;10^LG`2J+%Poa7(WSFZZ&!DZ!C+ga|wRlJvz z$S(3@R}=CB4~aA{KI&JA7zNqzlk4ZGJt_&xQNtNu^aPcC=< zU0l4+=Wn8c`M(O(;CKk6N%aRz)Zh+l5ap`>ck1UbKo%=dPkrU8|0eaj$f7bC|Jx|p zX$`Op$D0H8x<(Xs=X+Tm{cb4Tm~?Qdz^b=gjq52mhkW`gj#n|hSwcbW`%cj8GV?Zh zvS=a|lbr_4tZ#!<9unBIqO=d-*AB|iFR4}wWz<8wB&7Pkk6A{uX&<~ z%W1DA2E^fmEk{?fpe_tv8%q7sDV)Jjkk`(=`~RoXk*?WNN8`Y4L07nfuBPBoI;!>p ztc%0{P5&kI(|-KjNQ3pBqb+gxmvp2}ss;Uy`VF3+KjZLl*ZHny`h$AgTm?-rxa@w) zHNYj*pVaCU^>FcTa((G7c6XZ&T;Tk7dDQ9a-ZUQ^6P&PF>CJE!T0x50=GEYyIQ)Lg zai5<7Tv&XUoLmc}{0VUB%4U0|g#E9QlCIY|fxc#2Z zXM!VsbVZUnDq;X#(Wrb1xHMhrg^uJ>@EG-#tNv5epFn;20TD_6p+30OKbm&XAaA{E zB3&$YiB-e1Tay((>G2bwqYB}!z z&+r^gp}^-CTaNiZkL~nht{~Twjq*9is z*Mf__zF$1kVY8C@%8AW0qA&A*eF1rhl4cqxSA%aDKwnheg@Uvj)8 zKtsv5P&$tK!lUgE;kgvJLu1reF}WV;RB{Qq&I6QRX*s@rAJf$M2IcPD#lMA)^cjyj zsvy@#Gs+|2wvYM9CY^$f)Yo4~Q2lqPufINZF=RyEys$8*cJ+#lR6ie)96UQ zey@%uP`@?Sk(nNc&!ivyy1x4Po8_4Q>-X|ia2*ZwO$W-$7+{wdxH1lZkbd;d2zoT{_B`8dP1Ie-(#sw;T%&zl!bSpTQ+S7kjxK{qDyhiyWUOfsqRE zXUG2x2;|T>$_rc?hcB`m_lCcY4SWX!wEf_Cp14J7roO&yeC$)~Y}ls`v5ZPtH>{(mDzpz8QJEjrSsR!I}NJ{eX1Ik=1o zeG8{vD-O6#G@{(rBfK@LS#_spW#^HH%l*~bA9pT@p)KByLOo_uU21k6H<+#79 zqbneg8jRwBB8!4|tfMGb{SV3YfTR3Vay{TE-$srjV2AdH^m_{Q;!+KM0+#|c@h3v` zd|$9wG*Zk^ug1l3_$8J{{qXpQ;|~N9K!1aT&lAjZG*GSudXf72lOD?{$hloU;2zO0 zv7`Ms%dr;JE9m%e9Kl3zDUkbZ5WG2<2bQ-PK)Dv^1NtfS0(={Xx6*%De{-abo({aj z)j+uh7y>Q@(%&Z0E;x_+%2j_V_0!v2OJs9wucJPBwEzE2gT*G%`pBWD{qA&u>&b_* zWv?eM_|t)Aw$M8Aq14xcCR8{-GsrWnf5e{*fjnyYz`M-8$nq3Z>^14Nad@facr?QT zS^xOAi2ncd{NEv5=Koq!HTXLO($xBM0+|$SWdP->|1G)JOnEE$V%AvsZ{*UV(e{V* zPY5J%Yl7>7Qa0(4cZq3~tD|Gd_0^5a^T5SVNf+lQ%Vp*@>MMu7i~lwa3iooFWenV7 ziF1^j>Uc8w1oFymj^}oDp>F|~f+}~^m^QhJuYL4$Re-SSnH+pr|iflKPOV zzN2{KIJvf*^8V!7((V191%cFHll+bfQVwfyE)A5cqhjj!*vIJ?Q~w_F>RC>+gcbOn zepH`+!saIk0{BhBX;s}UqrN%z@3M)7M07+0{AiVYN->MQt|^Mcr+`bvX6)+MK|MsnlPP0exxzIg~t0 z11nH%%_A>bCLfZ_+@rX|lTWc6Hype%F*^>Q2QG1!9v*g0x{HoJvj!q?oY;ItgN(ns z8njYScAs<9I@;-1k^cu=25H)~*eP9LxyN^3xD>RROB<3|!AVv>$~C}+ z)R*Uil4@OMUUWQcwybkax|}=yuhtRXfb19O0kPbj_qH5+0lR?y@e5mQ(c@eb=#q|$ zsb68awm+<{puzTBXHdl>+Iku&*8m%+zm{*I-NFi_KIj7HYQ<&4btr_R}&qjc>%V^ z;Xl*Ac7gie8D@ayGUvIG0Zx9{1une71s+OXNxq=Waa7-?kH}YZhi{?yplatQ=Vk|V zmEWu6;mU>1a~+G+^$}-KdAs8!=}vGwd1$J0G=hOk$@5bkFWtlGzf7Lpg@KbC|DAk9 zoAnNI=b@-$G5RS>S!1gM1RzU&X1l&yFKZ6(T%P^#SHug zdGbA7-1%PxB~OV478-b?x-1UA4O{{*n8!YG@y%wd-+?pm7t|khiBrUZ)9rtUJnbCS zeB&H(1*Vxz-tvUw1$~{TX7UBsJFZK!hOHJk0+TbYIRab?T2tx*>)5(>wT+MGau>%Q zC~miocIQhOK=&FAa32G-T;~GlS#Br&m`h^)_j*PwaexYYg1xK&-_2m3X@GPZWP1A9 zarjZ-QsY|s$>N~Ci2jxH_=l4a9p!leO5^ZFbfgC^E$AKK61Zf6Yl$WbJ|)j*3spIe z_kXUDPq-bfQxzvLhm-4ph}et-mjH!aXRr4It|ec^y+FUj`T_Z*FWgwsuG!~V@q_vQ z;>*})2`)sD6=3(f!g1S$5Jkv$kT$l^p6yZ~TasS4 zJjLwr0(=^WZv&UE*+xHIIOrDHHji@W-IzOSZTu9IF+FzoSYZuz=a18o9uL&fYH$g> zaX;5Wm0g|WTk=tT9M^NfA^&vxnZI%VuQwKEP>_-1464{Q50RUJj%TE}06(h!QI6~B zc-T4@ppAhGS<-?R9N&JR)1RyR-|Z-x)S#Gx#WZ;NMfrePuh&8!#^Ik@j%PwVThfi$ zW_gl%!@F_{e@{nMUVy*i@Zi71B^Isnw5tJnf=dM^G`qp2m)Qm6L%(#KhXpg=@vtfX z+zCqP=?faPkxwE&^CcG`lYAO^d7b0yId4!u+rg#f>b#1Xe|O~C{$Sm#05=?X{uVV! zy~y=|<0yWDm0b64<>};FP~``bYlSVB_(Ler8mqx!R$GqS_LsWf z-%)*29~HDS`S2zK==?y1?~!XuDE}B-TB!U-*FueKko1>DBg%RFGv+`D#GvR`HYq(_ zWDRhL^_ukBIJ}er)Q=Wu5qais&cE_I!VpN{x+0YqkFTA{_93^{=YHNiQc7w2s)-FaxIX`KBuDv zCp!HSUI`6&RWvXSCs+N!;8L;e1x~+OzkggpLB;7#pxK7pdg9|3`t&+tuJp259gC{}zY;Nk2MERzF=g$|o$@;;dSE56dzC*Q-$#^r3-X z7bx!sE?uy3zUzV-3MO0q6tl%E=&U&WTFZHT(cb=WR!V_hU})e);1alawhKHx} z*9#BTe~sK;U?@cXwsQIrk&XX>6(pH5?~u8L23lkF^(DDp<0=2na=i1+Yl%PO@UE}h z_;}LEi$ZnO<8}FjC6*rV11>F5b-il|y~dkKp2O>VodZ7chWNpo&b$J>)Xu{}(P&Sv zT0t~`3f`hY#$~bkU&P@p^rHn*KR=S|jR@ENkp4=6-tSX`Kgiqf_fgR0P1h22)vhJD z*wu^%ml|h#1sxZMhtIVF9IJ0~jx=yFx!yQbehD4rc>(6f;S1@f{k9t&-2#rT44cY` z3#@mmH_*{^FTh7}_-7h`cgwV(Ur~R7=jVT7c*p(EPId4g!E@Bj@)T3)IodZ4A3#5P z-%txQm|Sn)SuXLj>8REVkQaxaN=JI%Py>vkeuL-d;yC;YaKy*ShIca6(HsWQr(MeD zgUjw$f*XgDbk2C}Te6j6TD-t##^FVlW72z1myYdk%oG}=Umm-g&5pz8(UBHNiXrI+ z>W}jL{Czjh_{-@?2cvUi%){VP<4O0s8dtJhUsGSX>i>uOJ05oWwjm)r^li~dF_XLk zoe+*A7;8D6_tTLEIFnrWd*wyoQqZN3yTGX-C%KRM%2j^_^}{vKI0OBr#J3boTN_)0 zgWnNT+#B|4a9kXIlI6JNO8*-8G;;cn_WuM5^ihi%oChv7OaG@Ukls}KC-s%9ejW8Q z`1mb@f{>juVKz+eFybdspiimP(ca|x98dXv7dRKuGmzfgL zm-)X6)ZiK#wA4AxQ8cJm{)z))FDdx03y?`(@31)q+@3}0hu9Qaj&DMsE2HfX>9h_4 zRhIm6I?@0tn*}a4t9{iKNCQ7d{T3ET^-qrY?~&^Y2jw4= z>$(Bu&8q*pE2#2s-bH-*(n-~-u8sv4xu~F1?0LS zLiu=bX}OBSH16R;OwnQcMl~dh3Nd3AGp)c*9PDvvTtU$Gqe?)^VVt~|Y|KkS2hw=g6f+r_7 zqsjFrSCyX#E(I-SjnWdh|AlFw0yVhVa$H<`gJ@YC{t*4>Pr9m~8gL2hzT}EuMn+&s zTR*h*j|v<%=354+^#cD9hj;l%ETg}MtA6$bm%uq6yOwKa;2G4<`8ezZYA~AyTNXG{ zMZxRV0M7%x0(}~XZ({)UqyE3A|FkVGZk_IbQ<_8tzw1MR8eD8S)`riBoe|B8!*2kW z8aL6;S`Nmw^sij~hhLzhG%vtAarlP}u=q1q10B5GK6VvQt^s<1OMz;rUrfP1PCwi( zP=gE_gg$oxQYk260OhJbh5G5#Z(twaLw)6{f4}9p`D`n!?GNcQF#=V`iX}SIH<+sA zC2*-(%NMSoRlJLK+-A{;a@9Y{a@hL5{{G;0{ZCP{JNxOA%L;3N2|~w`!m2tBe}Vzjj|Nx+E*038;{2~? z3#D!m4XlR0(!2kk6&0utn;&X<)R7901eX9+d%FO7%zu>n%2oeK>Nirqj`I%_`czc# zOT`qZL6YTo^Wl}TyWfFv_z-X@P(@!?AYHzDG5sr7|5t?RXpR@)hB&;Oj`T%j>gZ1D z*LZ#&j>A{dPgvjirjDMWL4)V$l{mZsTxy)Vud8td`*{D)#AB4J|AFA5-?+ck51Z+f z%!EKzG={vjTyI-PDW=sc(AGHoTg%%6Xk61uuD`J81la$6gFs3&jX%BEgC@s*E+3+t z*yMqWehKy0QgA=@m8*WW<@oaY%-AthD?E(dz!$tkNBYuPb@VD7<$I1c$KhXqOF`QP zxvt5eqw|{OgX3W`n@w$Y25AR5gM12Gz{Nm0vH1~P0u)m}je?Rd!?48f(GgCzS_ipaL~`gj|1WS@|k*eQ~k!XD#n*3TJgwWb<|8(_?wq{F{RESb^C{ zUgPn1$m=}*A$gO>KP7MV_%_SKeNAX~Y)QYTL59bFBG2*o|Bx4Y+C` z@Q(XmFA6F>gT2UCdpwNz@qe7na7ktZJ)8~70Nl#X2D zkDy?1M*;XTmiIO0Z?S`P4E2Y5`lnDo+tV*3FY)*};E11OMtP1VGeEV+FCu@;<1@$~ z_xNnf`0g&Toc=J_gkK1K5s85<5$xl z%`)>Hm}Z6FvPp@);g~Elfwbo};(Pi#`4U9o^wM+CpCK@h|D8 z)bsNl^~2A22LGi&m1odKgHJvEzsP^`c<>u(r@m&j=clLTNoK#}?cNdX|5O?@dj|W` z;ACeYUTqqnd`#?aHkf`|JwMsxp}F1E1#-kcih_(70q*sZ=Xm@?@s z>gm7njg8;eRC1d$msEPbKkAF^nd|qteugMcnisk5L zM7C2xi+sRt=2c)1@{Jx(wmivfq@VWw?@NPb&tQKVeBlKiNc~n%e<=0!7nNB~b2z#F zqB42djHKY2g4h}#&j4m#H}zrjeDXAppFy7O@w3SbJ$}CBnE#h}f@w6U@c8AHCz%hu zg3hA;YES=K@;Z-~($7!gr@j4Aw?!2E?ghAmyvu2^E%bNtJv_eL^1i0o3;ZzkGdt+p z_$w(mq=P`(`Wf;(kJpk<_V`QWGd=z)`CN~`)ne-8)F~Axxz@gM{^z@G;Z}E8eSPIO7*e1;*&+z!EVUrk=+@de~{9>3A@B=en(tL=}bzLf^ep21z@Z62>A z&%Q3U(me@}Hr#=0jsh@Ryw+`)(me@%_uIDK59fT-ok;g-p zCz+4D07=v@@$~nmevd+JQO4ho2IZbXCV7>|52m9d+Z{;(jTuh;T2KFI@D~^3E zUhMG(^0^*=Uw9a!y22A|ra_IzzaX#kcnf)x$A2Vm_4u#kp&PvU{~r`&K+rLD`ChjA znjDYsNnYsj6!PgF-^cPKb69lDx&F_fL8)hO5DhMi8rX6TBVXz9Bj~5n^D~NkwZ~8R z-p221HhO}wG-&qtndEIAFCtG~7~67F$g@0tvE?`kg}oYFK|!m>=g?8E=V(58k;fO3 zmw5aZ@^bR_{jY+8D$gK7UhDA(EKf3t<6@5=k5IqS(_cmX@v-`0^DGToJcD)Q=Em42 z{Wp1r$2XGadi)*oVvm1lc{s`3A6p{xsTCxd6&~M4{*=eRC;z9%e+D(Hz0ck-7Lp7o>zlj~_xo|MF@)jt1qP!9?;Zk5479_4sAvjUJyx-cshBB<4|IZi;P*g_b9o z!1&nFek=9UJpCoqPw@1YktZtG{$D{s7Zu1*Kw3%O)#Gc(gC1W;-p%7LTi(}Xdj$=@ zK|#JJc#pi;S;H ztsd`Zd6JoF{cHQf^#BUydVzjG} z|Bk%X<3EKd2rY^oL^~}{G9j-(2|q}&l1zV(cPAg<@f7j{J>HKzJkS#ipx_{n4nj4Y3z~lGU~TbzrFuw(ID9~m`9%O z@rC3=Jbo+rAs$~sesnAkn`IQ_#|q2}^0Pd?lKeuCuOYwKoyxWUmr)Q} z7~4cE$oKX5O7bH-zJ`3Z$JdcB@_6`V3SRI8Z;*fE@%PC0yfL<*o5}a__*U|v9{ir$3VV4|UX+_+w~L;u(yk!HSLs&>u(r3QvC`d5y=XlGl0sGH}H2YnnX4 zEE=?We4gb=rb}6D-!G(o=(gAvx|KY`<4eeMZu9Q{FQcH)Ggv`B-Qz3COFh1Zywc_W#QiY-|^xosr(K9BVqf8o#If=Gd0lth_vyZzUggODz8`Ou^xv;3x7C9^XlR zq{kC}lH%g`f<4}ye5A)y$it&NK|cycdwc+SuE&RvkMa0$@)aH*N&c|o(fw}>1=XIx zSn@|aK92lRk543j%;Qrn$HT6eE=K&zC}@ij;JSd09`_v0qki(@*wMa_`t3hpCOv|5 zD|yqcv3MB5*Qd6|MnFTfjg zbhGE^J@T(SzL|c^?XfMimHOXA^cFi-`t57ZEt@x(RUQ2iJ(LJiH@H39POljwx^%4LyVJ5lc(RE{HqS!#!sQ3wSz!h z^dtY%;{(Wh>U%at8EFW4n#YHeAL#LsJ8b=v%y3UIh6bZNK9>AckB=iS^7usZOFTZ6 z{HjWC{(l(-H+lxM$d`D0p5=W_zSji{$>)0fR`MF<+W$)^s8fM!QM1hQB=dk5cm?&F zJpGmAtsY-P9;%2PTj6ySWO#y?El)D_Uf?&VpX2GjM_%ah&Ghr5=VzH|N6QX_$Z5Ozq=vKBs}thBm@#dl41xD_IvLlLU<{lATPN{ zk+7THAcznGcoCE!zL0`EtPV!3fNfQ*3TR!#tMa;tv_7D{7SUp)6)7qLRY4TFXJ*cq zWGC6?_tW|IKj$%X=FFKh=UWorcie>Vc3q)Q6*$}%1ifp2AkD;%#fqRj#CI!r5h~cE zC@4evPZfF{>324#=lc62LwSAtjE5k;O2J2=po7f?ft>M3|C~ZU1?d|U`Y_<2AFfDB z$1D`opa`l({Bs3gjQGD5e7V5kUs$b+ce<5G-}wop{=Xg>3KfQWWU!xzx6l^E9R=Tt zasrB+9Y`N(q{sE&g&hkUJHVdsvlsCP75ot5^$LCj@joc|F~na}@Dp{|@wU=&8u3pQ z{5;~P6#Nq6|5k9{_k0@g9${m=9&d&sT6A@#iJp zwEy~J$9{!j2;v_r_$b7`Q1J1HpH=WFh+m4sL4O!KGM;5!h15AY`Y!(et{$Hxl8Uc^r- z_#wo9Q1By&|ES={5cmD2(b7`?3GARv9o%g_rxEX<;O7y~Rq#s!hc_Y0zV}_=($YL# z6#CQ)LO*;LLt)4i4DjMX!SfK`rr<@Wz*H2JA#N+Uj&csxDT4YV!;K2V5M*dj=tm)b zQNhQfoZA#RQ;mSCBLhRrM<3FU_b_zr=?_y4yk9lMa>pn~s3yg|VaA%0Q8k074@ zNc`z{4DnKqH`*UeeF8fw6o%7?Pgd~rh}S6iC4r}TzECXcyT}(b$Frgtz0jYE9UGc; zaMDbHr+G4-YTU0J_T(Y`HiflelR(>#3@1^rR+1O*?0f-Wk8Mj`#3 z3jKJbPycoN5==q*r=O~epII0gN)?7#h>un9S`_q(qIfal?Kj2CUygXGi@WQ;5<7;t z9m3Ea&w9j%DtJBOlN5Xl;!_oTtH7ZGqD-J}2Qq{ehF!?;tirGt>2Fi$4`j;7yy!pFyd`Xj^S<_v%jL41*dk3xKd zf{#c10|lP~IOtFFR4F433kJ9%DflcDL?4TvaV^rXQ0NyUeL$gKj`Xi@R`UOq$Z)H| zupSvsDGKV5evv}I1?ls*#4qJmq~Fj;kL$k!JGM1;h!ybcLP6scL3@$@1BLz&(oafvlaY2;!70#65=Zr+;@pjFwIl9QRzs6Y0_}?VnwEuMM_@~0qAMrB^J_PX}6?_!pd<){wDEL;y zUk1F%{xFyw*zvy#!!E@CqTqWG|5U*bA$~@|k09>-ZR65N{l~DQeNzW_PtOU&a}@kE z;w1`x9`Sw(ehG0?!F@jp{hd95+W3f;`Xg7L<{7LoWC}b7{uop|eIC+JQ0R-0K3$^dFfx3q2%3fT(-rz! zq)+-?;~H~QdKM#nO+0;_XE`$H3d2fdSQgLVS&#J775aLl-=NTMLHt<--zsp(|92}L zJCNb1g6~57tb*?qc$()SWk83Jo^FWul_LVjKX}||e=yN8WXM$*P9Q!)!A}c3&C{SL zK96|Y-^VY(C6u$MnH-_tCw}c!n#Zp&qzW7^49yt$T4o~sQiVPb={G3!MM%Fzp)dPM z#2@&2t-_!q!y$#CKPuR*&<_!KmWL|acofPxqsSSLa*l3LzW+Z38G_Hmn=XtDrxeAr zkp6asz82{{kH@e5Vx)iErFYkVIWpub3@ef06PJP8AAZ&&eW^lUk9dWGZ$UY^&&E%z zZYy^5QaW}Z9#HUIh+n7RdlA1$!4DyRtAZbqc%5g7(s2wK9#ZfV0&n_@$I=c^BSQM8 z6#DZ5hi}Izmbip?jlf;|gEZe|E;7yYe~O?~fx|9P1Z5(9+H>(H%|kq2!HZDNrsi_E z{xZ?g^jB>8F!+y-f^0=lf5e9?_z;12@|3E(0CB&9j|Uv|cWQC^O+khVg&~ai2nC;o zxcWQaY6YI=nXC+SvA}aYJ0DlR|Faw!Hf@bJ=}N>;Js*#+M|{;E;_-S{j)HG-@fTcr zT>q`u;d#;RkV~)w@lplfh4_OCzE|LoA3PDi?++pUDuwHP}* zX{0}=(4R;83Wfd>;&o${4&Tpw=4qbXm*Opviny)dnF5DzN-J{mkbba2UxfIL5^vgn zW!Mo_7<9yI6}-QTZ;K!35P|1;DmKM0!6<=u_0+`Spnp7c@LxRYGElkno;wjAfeh)0 zzbWu0LHHY%J?9Z$fb=e2aYg=->O~43;&>hW_3FE@qYWxpiww&UcL%f|@f8Z*4u;SF z3ietBABy+}1-}dNO@P;Novwoa#g1nchAgjCunqC{IM5M@??yZq@h1>Jig+R72N1uA zc)+cHUcwIlX1Qh+$k5ZrjhN?|j0^~S$_3udGsy+8wGr>F-~$uj*ZS(Xf^Hs_VN3#s zNeS?23GnI!_#77(_HQ=YJKT{%`{PF$Hhb{~M*l)&Khh_=^efR}mQQ~ie=qaG)za`j%_$oYVrV1_adz9~Pnks*y@JDtyy^9xO_syd1+#0Y-q4)yfH;Z-PjZ^Ua9T&m6Uc$8yg zAkuGB=qCcs57g;MFAfPl1Su%nqb9&rf}th85*1uj6u5R&6YR%?oX#EvH&AhgXXC#3 zg>?<;j?}f&AKS3Jr%Cr-eOe>W@TA`uZ|A?YVJo+gg0is*=t~v)3vJle{~*3XA@7nb z#cxX>ezHP;O)~2b%bsZTH41qpicjy``fOJy^mWO?t9m6M-=vT~h2m=x(C<{}_aObI z1oR&$^q(XB2MOp;DfFbRwB}m9_318CHc=;}&ox@7Kd8`WA$@5A`UZs_HtFf#k%C^| zs(4$zi!wYa?qQ$pYnxNo2fBHTS%yyGN8cTcc%gz%Lfjpen~wh%;_mcZd>Z1e)4B2& zO1#eFx=Sf4UI88OSG`+o?#EG(yZ(KU{zb&yb#w83h&SK_T=_=<_j=A<*&uF8%?_T= zpo1$`@9p2BpbB&hR}dkbKHalmd;Egt0nQa{dn*3sbDGW@sqOmD6LSZ1?~=SfbNI$+BLHR;yHDQlrGPalQrj}+pvq?J{s7u+(R7(4sPY=m33p7li0+Z|p}U8# z>GtA|;vsGiUq??s*YjFGy$JF#ZtDBk-ltQj~IG1V27=MW!llua8Jpi za-AK1k6f3`9}VVxg^X+iPbNF}XEK0o{3Gd{Yk964I7{BgX`A7G!oO8(r|;oTa`7)E=rWv`GF z_V*a+>(yAhoh0Pd*~S?81-r15WU;m7l(5P0LU)AM^Xz)uA5k*Ik#3>(V!8G zI8h^H+tm?UOi45OoJJ&RL7Sc4Nv5^s$~B8WPxl2(mVTbBWuLr8GFjbD(uw(Yk?CHC zeYA_*+f@s+n1yDDaR%9czfQ_vYaQN8u3{g2M2ccxzD^pvY{DC)D%~^;)7Go3h~)&T zL%PlOyh-M>;G3i~JMbpCmp%495IKH5WNCJ_88)l^!D@@0`wrB<^c~4zbKfE*?8R3| zlzsmlNt4)w-K3ChzKWWx1AmIVLWZz&2f!YS_z0TlUVDoaCfQ~s`yfUN*z&>;;GL34ODmyoUTI}#1(rIvT=7MPLg1K{N&)4*t zXzlF_7lanfoI7W}W(=Ksd(G_Vg6IT5qE0AsM`%t}6c)~`)`G#1rbi=2bvSIWvs~7+ z%Oo?oa>^LnVcT9I2772C?ZU2qnRI18O`yHlguO7H>F<&0>A)EYhD@_M6bkvx>Hr^# z$=d8AGuU?rxc&BmYmVwbzl}xqlfFI{eH(mf+}mL7bqB~V+1Udm6kGW=sqwOte+Tks zUWV@T?+~Nhn7<$rtO?DTS#{gYIWsgZ9Gx+9PNe`~&~_*othUXFp;=YIprx~hLm+bB zAy|%A-XX(S|96Pat{fzm6#MbZq#v984jI~Pv93Qv7O`&+k#TJL%Ot?oy-G$W83COQ z{u8++w&hP`lb5Y{mxMDykt)jyYla1;uC5A2hI8*Nyr$94<>k%oTwWe$XWSfmI1mbk zonRzTt%oAfAPc3@EMjtZ)-`6mPnLM~*!K6y^*&bfI%!M5(IFm~Z2DhFFShdtnHqce z2)UsxD}SG~?PYl?D{B|dUr-sEJ$r7IptN16HrfO_EafB8-KPgw*+(Sk(}R4&7;M8w zU~RE^vbanAqKn2w7`phsP;9`!0I_0Ij*<3W3c{z1(RG7u{)GG=Ye=G5Y|71)p4`+%(w!PsxBh zv&Amw+M(ZKc@1PTq!JMS3T5D#J&_ZpA=lbr1$B-uieEkd>pkr}ukOmoU^pOb+; zJILmK10KHVa~Of2Z9fD1tMDth^jsMLVL9m}=?Ug8J4wPF#ELp%MIDnpeiBx|VxOEO zw}Vlk8O7F?(KPn-7bKhcAD|Hade9E+*cYTTb;OXJAiMAdxt54A2Q)V96bZl@ilMAM z1qXzI6F5b#&Vnhy2_q&D5EBU4vRD5XvA`a;{)fm4~znbmpHD1LRL_iRrC8L%lfa# zUe=?Ic4pJs!%0>1H5mXS9`iL!ao^YE7LeBC47m=Z%{)UYz_CG^=6AReBm6MT`G(w> z;+G;dKezKtFeWs^S;1LSLWD&%jdkn;2j#4@WKIRw4@ajaCa!5>=9(r2Yhv-V)($ei z*aMmtWIYa%GB)5GDFk!ea*pIlb7&BTjzd7(wg-HF$2oG=YsH>EPu}%Xv78!&24L*X zp8l2`Y%7(rj8dAFtNFn!Fd$v()`gKY*!n`xf$vBq%Qy#&W4|Xxny|bkcfMvAZ2k9S zDVQkr0vXSGULwV8Tq!MQZxHaSN*|@{mlsHJTW&JL;qo9IxIq5o4Y1umkS>`gUF4XKMOP5qt`JL)WxXK?1IFXuE`ybRS2KIv}>b7SLMCvgGpc$-?{tMk^H3LNX?3GAL*PX|F)b6jighIog3NX*h@y z5BX?!w$Mu_QJ516SuLrH-ZFvFox*zfo2w%@QpIjo4T`TsNjpmnsklRpyDt!f7@AaKBiO|cu< zfn>TS*Yev|m92%VLQcdCI8`XrWP`U8n?2o@j$wlcjDQmAu#frEB>wb5J2;`ACN$79 z8m$gmh7k(tnh_0}c*D`;s_?xHV}#V_>QtInE>}PouC42)Fl^%NkDV=`J#zh4AgTu; zvPB}{pb;|U0CkHkT}`iHhaaFlyW3UaNW?a5(~dY*0o?YwF4s?Ym`>?0*~C)XyD`LF zm83(qRMCOvvxYlpd8}Uw?d}C%1&4xICB|&D;!xaUfx?guCF7}X@FP9Q&Lz>#NxEUO zwP)cF`q4*Ae2__9`3GeBSw5P16}N*Shh(&}DT2}k4c4WBOl((f1f`K6(}Mh`GS%Sy zJX7@Z94M4-1f`Sm=)_?jG_z9pQ3>ONs5xLgwQ@sYr#cjF!MbTO?Ey&b zL>7=mX?BW(_w}^Gy`nhRYP2RX2kzb`RfCTq_S0Fciqc)cqwk z7N?NwZ?Vm9lm2r|I&D&>>SBm_Zk%*wXm6 z$==GKIz*uJ8T2`wANnv0c@pT5v9*{d&j&b#v=t<9?`G2V?c`wLU?^F{u3RCVQ*_7_ zIGw`=<->-0DvMV89D~8BH^T=BoVNfr*fX7I4k*4S2O{I~PB3aXzBj5b}c6}E**;gh_$+rM! zmJgjS9+6ACUcWgEXfu$*v4*jHQQ7`b7O<^X-6;Q3}XxE zxDp}AfEW$Z4H=IO8IKL_JsdO}=StaE1$02Ra7sf)VFRMDpbAK;Te{QUu#Uny&vvK5 z6o`VnA5u23QYZFvcY2ji)5XS|UP!YkR1_e}IvU&E8FI!eg*1h2ETq>`$me(jHyZIU z7#+4p5xcJkEP&m(RZ3u6{Iv%-Lx7PY`VJ^RTtsL2biZ7#1;sSm2VrhiF&*#I;g(uV zO}_uLOX!d^vDya2J3d6+ip?mY$w?i+CK3233~#V`-;t~gojVB!h5Zf4=AOKYR*&Y1 z1E3I*xB!SgEga7gC!OKm?OGmjI=>!p873QBN<;17Y6*9Ke_nwR4am(0r5->OpaP!4rRTD4JxAxs2n$hF229nnKB-7 zgflsMB|G;PXfNqaN9tULDJNn|r+~z%g--6~FcA^6Sb1N_Ybwu@cC23?S`KTI)(4h% zejn;5;#hV3Y}F+=i1+rPGt#-=1E-8rhNVg3F@5RHKG+Li_oY4R#QudMj+hVJ^jd0% zsd5=|)xhstQpnYomqEjlo-f0UA;XLTc?now^TRDP60Mvyf9@Ou9_`^1*vN94kI@H? zGd8!JzLp|S2Ezuw>_;zvONRU~$ANxY1TG2B7ED_}f8bE_oB26E&8`q}r`(Xqr@Oi9f5)Q9bbY^{Gv@3f_qgPS6>TriVuF+{7rF~>9 zf^xyCqfj678?-NbREHe`Q3;+8d3!w|=nDKjw>0@Y7Jn#b=|pa~*Wevr;6k*Sy|LzAfso0~EunQ*fd zF@oK*oNDaTB&czI8KlMOa*LWWMwt+!#Cdg7e^_3_&u;Hem*uw-qQr*f)gYZcaX+nM zw_i=O^Nm*AP~L7#2viV;i`hP)lOY*1mP=RDoOYs8Wd-8GR}QNTLYd>{0m4b=4WR!6 zH*N6MY&Pr~dN2r?3+!CeXvI=)m)NVODFTcsGb_{7&v|0KP zn#&Fhq&@i_;5`9uzo5ki-$?tgIiHe_%p62Vf}brNM0fivKdY#qQ+yDIybUCa2T5KG z(^;&7_NSIOui&tCnftJDgXz3%sZ`dWOc{kt$bW?Hvx9jO%b(YQ5@<@;fg!Yy&xSJ! zG_Y+WDS;|S+E9AE4>FypLuswgHrU}+uvbnFg#*=Ouf(7XvFwjf@qgpbB$W*xMoYl> zdq&ZoZ0#@_gy6$Ncpg<*_CR4Acw@?vbOvhr+kES~N z_w}?FZ~|!=yY2>CkhpskjAhJd8fIJgii{pjD_FNNbXPi0 znLwk=oJ@#OurR||*^P7_+dP&U3}lUA$EQ(}A}_ZlM5R7s=^R#G$uYUy0ijrneLR*< zZpSN5Tw;)2J(W)M1^uygQ|WlG53YGI8@BcbxfqLEMx@;Go3UR<=yTrUmM0>_RIV4Uk=*GmsIkFJJH-}M z(~M+Z)Z|^_gs^zI4L(7p)*>>>dyA!s;AF9JOW0o8BIdx;8bmxU+mOWy$-WC9^GyrMH9Og5orGYYi<*7S{yU zcOFy@x7E-)vxR$D5O?@+p?Jm{$c4C>*o1j>M1hPnmW->G3@?@pFBVkCUz$Q-Uw1JsN*gf#%gK0mrkmlQ}1DJf$pi$;3mdsTwnX6c`ux-J`NNk;-7t&&GQ%)|< zD~ksVn6cw9=MFjujzaNCnn7-UOKxpT9=C8A^e(1ZIie0#4pkMwAh)&!Tbq{!Vo(-% z*gIUBRPJyKZbdBy&dmWMk?Dvf(-BLiBT$SKf`bff+>JYk)?U`EEU~TG$~$R(I=^ay z0GW$ekc%`eYWPlCoW_fhykATQBDsGF?ey3Z2pnQbZ79BKOK4WM7${VQxJFynv1~|8 zc%MvxY`H;g*r0*|XWU&d-!)6&Sb1*=E%rgE{^l)Zu6{3kuPwX#E;=ut zdrY}4H?1uBeGp z)XJ8d)s~ypmbn1r?b}Dg<}Y3bw{~$d<*?8)I-pxi_u(;@tA}v?+A=zZ=L9MD&`CTe z2;D;~eUJ_8y$9~u7M{}mmeXM}>$IU-A__yc+@&_Wkm7}*6zC8`wwk@_nR_qIAtDql(zGIFEbtH=mM(*)Eu)DI7s^!+(FN?uU-1k$rq$HRKwf+iViM;u+n10GRe1PsDaup+qD{!d=b%X$QnLb z4OxQ?S;NRxbP)UeSG0@Q5k--}Dtd3WoUP0lY#Cu3c)QP)El z0dI(uJ2J#LP_E%%2jRaCRLps*%f4C5YpiP^hvAiPHvM5yLRr0r&SalGOpC!Y#4}X0 z3%CK+(3OxP@=_%my_OegWGUO?rSKc2h{J&U;Ij0uwRBCsrnfYJd_Cc41}C{)6v|o3 zda!3J?#horm;!t)1$5p39`n$8s-=nO;mCBtffseh*VC>%&jw1FAvn^39Js&HN8s)k zc!ZYmS86~Y&uj+@uJazDPVA*e=wFg$?iYkNMRz?8#USvvcOIwxe!-m}QXZ5==Af)3 z2W9vP@^b_FAUnMDxJY@2)k6X1<$5|f*7XTG-3voqu#x^PHsMKH*kPm?8@#O+=?k<% zFtp+*f>tamv?8==mUK*L#gL&D3kj{kR&AvZcEOKu&-gurKO=8h)^q+N{&qiG~T=e8;Id%xN_*sr!1Xw{Y$8Pg9 ziGP-NF=*x3ZE0`#jPiKk6?7;MUya$OGZs5gpuFN~HoncWr7OT^IeQytFQO?I$fKxe zae$T+8?l}KyIri}4O-~))mSY27A;S$87jZZRx^~QO7Y)Vr09#{SG~hO3jz&$*zNmi?{-JmOL4P0dfD22$@y&e PewynWvY8#*Pka79f90Vm diff --git a/posthog/user_scripts/aggregate_funnel_x86_64 b/posthog/user_scripts/aggregate_funnel_x86_64 index 7b49f9a90dd5be28235cee8890ba43a995cb3e7c..615934388b0275142af8e9f143b57d409ac25128 100755 GIT binary patch delta 174440 zcmc${3w#q*-ankCEd-%ViWH)92@oL2trc)vZl+LRf+OC={6%?v2!c1qVA{@69={*T5f1@C9GN9}5| z^3)7mvgkIy8wU(m`!vd7GHF+RIVw@VqB~2k_orSXV;F3*jAPefCas_u&s4DOFT6n8 zliF2%BID9NC)TH)ZXPl8`!{*_jMw*bc{A_ls#loj4Lz|mue?U|re=(~*F3rJsHR7| z?tS3*U1vO>QV!#hv z$NRtEfOkH?>*?gUV$vFJDc}uv846U00@eT@nU>!k;Q4I^ym%>ZI6n<8r{Idoq!p-0 zNl4-~Y4Cyn*`oyz}qk7^>0k#tbJ40MSKL!$pq?`smH|}Z#Uq>1s*ZrGX-wC zJC#C91#UOs$6r%~u$+*=F z0slzgEe2exr`>>8i26GOu4wA?rKqUWP(W+IwmOxhohyWyG~m@YbKGgb+XbGViYumz z7V{CysR<^pz$?<=H3s}^LF$Gy_(lVMAFCho*K9~QAR2Br;HNC%9qlmS0|ee_z^etR zP4%gi)ACyhC;8V{FkMt+Hx#%~;Q0ppa)G-Hcv#?N2K;XVuh8L4|F*o$M^>#%AoY7o z6sR@eM=$01*Qddo(%@|;;#~hbPE4ToOgmxD@b zMV&$m$_)i-1uv@%_=OMi4%HZNSu{{@z~>8my@vDgn@sB`@QOBS32gi()2*UFvjNu{ zZZqIo!}|@mBI=14ajl{*L&6_L0qZ@f6lxN)T;H28i4-jYEM8Dxv$pueVmEkk&MEtERqM&aQU8t@i@yA1f6#&!(w1%Yr1p!*I1wlV}+r>22r5efKPsy^S;)AKPmDz7;vrr z<^(SE&w487&Hh9Ji3omTWL*ZlR*>2jPNiIz~%JEt$ZyLc{aZ;Bp$gA`M=1 z07BGuaSDap}-+c z3)0|i23)JBBMsh_iYq3qfc1fsQ^J`BccsD0)8N%7#UX$7CrvP!8q?s-Y4G+mc*KD3 zyq?RJNyUIk{x$W>yMZU3y z4S1HIbjvz}|8EfqZH59)QBk`A-zZqH-+;d+@D2m6H5^IcLjPV81v(Q6q;~K_aHj#^CK|{$;97?Y)8LBBknp6a zNH*ZF2)xXIuWRBwE;r!YMg9r{u63x2<4OPjSy75QaME-UIuI2AA;9-&9^ygGr=|%%1`Lh}lXkAV}rvW!j z;&{FRuMl{l0k?{+9G3yVN>H>MIFovf5BU!|T)FjcM?91Fppb_9tDsGc%iT#?P>6c0S6_x{+k|3Wr1etECyV&bXEhdSvs2m@3==;OatD~%+CfC zK4kF!J4J%aP{8^!FCZIm%|KP8!E4gs4H_=SZ_+B-s3kD_XEH?uNt+G$5slOa{O`|l zHR~|oB?9j<;EI-D{c9=-7o1U4CAm(SV!8$%l;wTx+1kh;QXAXfxo}mTW%%w;K{_TX})~23!-; z&NR5CF_qM1FL27))8K`T2LE3xP?n}Zl>xt5G*FucUvI!45&4@ExX{1NBB3Rb!1eD3 zfwvp*<(q}j8SwiB-eth2zsU1jA4#RqC5lL}8xphvg$8_`$X{l_UlMqg0Y4z{1_S=A z*z#2x4GG#_aFYRlRTOATgLkCCyA1e8q8{tt7$uVa|6!5fiw7Yg2IWe}$-MeM&+SZ%Tu=rNKMW;9V!fN&c*VPZbKSL(Vj~D-B+r2Cq(o*8^wr zuUSB?;l?xtnhkhJP_E5@YxQ&(@E1h>t|ZR&PfM^qno0=`cN*}cq9T_8AGn3fM7aU) z=*jVF18(Xi{Qnw5LcZ_=)*JB7Hf|Xk4Y+9!$2S`AZ$*ci4ft;YZ&M5jrdN4S_Z#qg zG}0OHJ|e&6zfvhTQs6cNK1twCfhYX`YeYh!p+LJ>QppDV36a0tfZrrKT4lhsLuxgt zxMF%qR8*guVA6cIMg!hx=M-u&;I2U&?=av^134Zs;Oz{D{B;@_e@*>52lJlh8wyyrap5X7;HSRE^UDTYV?ntA*H}=c!f%gzX8|scc#HD8&e0YXa($s1Pw1V;DfhuN>mtd*Y(_4tuo*h0tYX$By;A;df8}M}kFEijzDI%fVkkBgd3IpCQ@G1lTslclZ_)h|_ zG2s1Q;;d0>4GHIogn9#BCGZ9Vt_Xa+0S^hh(SWZN_(qN={eP2iG@1+r{wfMI8}Jtd z-eSPt7I>Qh|5)JdDLD8a6$$%O5@32ohdK=SF_AxFz-?l3>NMb1QBRivA3^z<{5_c} z?NdcX76X31Hv1Xy#R9h(@Z|!x8*oM7PT;J4%>v#d67mfN8uxHvaT)Nqz{?DH*6Tcf zg#mX7ygG?<{kv8q)Fu;{`q@MS>kW8=$iLBmKP~WP1KuTAu>R>(N?RSm|KDgx=p4pJ zU}{P&ka#xWE%W)&3ab$>=eXT~YxU$S#)R9%G-|+IVlT%cI!fK+;blfBlU0ZFCOK}? z;k;&!+Xc?X&t8Y(xXGz2z}-%skgvl9ZCFB~4(GJu`CU3(lP^kvgzz8H?c!I{Z{! zJ&iiNuMTe_oaCR#nWsx=))nZd!&`KCe;wYY!)-deU5B5h!}sfO<#b&_hc4j^9Ujr) z19W(&4j-t)yL9*<9d3GtO9&0v&T+2)7G1&+T>+~OAF9J`I((Q8x9f0+4tMJC;W|A3 z8R7p^k4ESc3UviW>Ts71cj|CihmX?XWjg#!9bT^CH2%@LgbFPI*ccsNrNhVS@M;~- zUp(ZAH9CB}E`O~K&j-%wkDGd3LV>P8gAPAShp*S+6Lff^4xgyQH|p?7Nn9~C=@QOP zCYYLa_#bq5iw-Z;;cYs+NQbxU@X0!SKf^J8;{6m|LWi!vR2?4C;nQ?@rw(`N@Gc#G zjt=kqC)Wb{+ea^rUAlz9x&o%BpFd^pk?iO5=Q_;~l=ZsevhF6iuElGPx6A$=mhCKO zeA3HHx|`y|x8Scd*o8ZKuA;PPM;yQ5&(SkY8PRt3`~*GIR1s}q&-b9($_6TcQ)f@# zR%y=4Sl8k-4_1FE8=iSZc7{6e{2`gs%^7O(`O7me$`*+qoIg~(=7N#t-rLoN3r3$_ zoReXagR}O_;nGeySlWU6h#bzoZc_Jkxxq}Edhmkjr@z}Pk*7r@-cK#=KDOY46CLI`CsA?in3~2kh2Z&7X_+k!3E~&lhnoqvvVFhot5iE z!peO0@PZ2aufth~kaZRjpdBf9_tnOneV|NTv9SE~f*u-@La7G}#0ZZq9Gkf{vzt10 z(YTzS9IQg3~XGhQ5#f!wvkV3yzieZCw#j!HirueOw*d-_k^5?#_W=AQB0*C~gx zhx8y}k;9Y57pdo6G{Ah*cnp7V=A_T2sP|no*gU;JZMx_p^U{TC&x=Qy<%R0m7Y`rx z2qtDZoOe%gcatahqbK-HMEW%zmwLK`yX4?!k-tsO0Dd$w`G^|0IN#i|K;4Y84=qqX zzj*k>Bbe+_cDJj$DRv5O_Ti>9_yh8Lg2yA}UAlT=MZ6xrI{A{j`+qzy!&E6tH|(jD zuEu3x-9W1m{E0da$g#mFv;siV<2X%6qDE=^xEmAFBN>8)@EJ zqMmcv1oKBnRPSX|&7KnVxywE<`?~)H(Ulu(tQ+@by zUsf|-!d=(ioH6*cSwBL^L$^Dua>(njMb5@!C6gMp;i`d-`;LLm!4ijE4wpDAbmxRf zM6*cbN*wvPpLVAD?p0^?x(LPttyFD?eNoT5;!Ja9wz~3)Gt8I1|NQzZEE(&Xa?Dxk z&(};g|MjSvUv);$TT5Z5f}h3noad?ws|L^gb}l_Hc33{%n#qN^wLJr3gdca(2WHc;81TCX0Evn`o5 zZ=~wJ>a46>Au zwZJaf4%Pg4Qa5DFX+ud*IQ#RnP(k=Yhy68H=mM}r>L~}e%YG2V!|tGt{nYmzy8>_4#Gj*)q)QBkJ8tCz#JDQD0elw)xAM>S0`-oT-jpHqGpfsMjrf z%luz6)S~N)%-79Oz1KUckt?o2|v_+cyk1 zM_uYKxNLK&18zKDSv`h^M+z4Sd;!S`=e;l*2I@@QJViHCaPtaoJV7SLt}o;9=p$em z;70+M2EP?4BPeAxPq`;0r593S)hKWYRX6}Q7tjqg;a2EwF7C=xihP8w#HIr-PJy=* zJQDC;z;f^%>h(LHvtHX$GCqimI;j}#$S}pOh^+k3>oxoObA zb>Bhhf|#hoIa`P$;Zp@ePn?pcihn0FP* zK0Zw?S$?@$nyx;+eBh{Z=UIR>XQAlo?9--oH~G6VeS=_Ha;B**x8|AenWB1D%+8!^ z%2fZjqNMMsW|-zs_6BOBXuI#!P|ibB)q^We&szmVs%s*H793cGhhWYPQ`G_9A<8^F zAZamK@ZH(d@I)?O?X-o2dllI~Z6dqNkweoaP->p92z6O!HA>dfCMh>(1VAY_Tf#+- zXPYQX&XFn9+;D1cclN*VK)-^P)$HEiWtQrCP+EA>o->iS^>w7ad?)_AgkB(DJkLD2 zyD9q3G0HP-GqOZ(HD{{dc+czqOd%=I(RkitxQXVWt1vrfPEb9*Gt2`gsMq@J%7HV{ zlklp>$eACr@~4hN`A^~L#zvsF47{h(;!}#?$_WKAOBlmq&0H&-)T-pv!!XFl&K=FspC@VQ#=B){E&3x>jZa zqr%hN03%trk69xBB<%}sZzd9~&OQtxYrOuphWEv{ z^ft4Kx!4~$n@C}cPedpDJLF4>7Z)!f&t?fsQTCt5sRwVhPk3<@8{PWIV;?g9Z^d)u zV#osHtDx^)=)Xqan^~nj6si;y*s5N7+d%W>BglvCGwHk$thV+@Kh#FPk$OYf)tSM4 ze38@p@ej~OXjO+C@~i=8E9CrT zT7bp#M#DXD@1}(*uRZRY8e4(dCyZ1N+&*eV1SMIkac>a@12=}`JlopB$=^(cp9ob zVdU-cyx4g4_f?IV8Se?24Bo4wNgwr%JIZ>zG@gmeN8?n75;NC1)!bTp&lOHa+KrLB z-c_g8)|pQor*5wu>;7si;JTJO9%c?3adz6M8B9%|8woXapcj!sG@&_tq3jD#J47!j z{h(QP&Um%x&gsfAXupu)ANDZ`jsbxLm!KI)@YR?jgt6a?1SY|lx=t#=&muz_3H}o$ z86@~FVCs=Ougv)!H(O11V&BFwYUN#j==~uY*TnCQu}r4>(qwglOg}xGWV+-{J0sQl z$SZF%nSLM7ksIHG6_c2g#HNzSYj33r`D|S)lUXZHKb!79i)??B2#6W(0+z=>yX>)? zcIc9z-8OtO3!INpCqo0XPzz``9F|t29TFJrhNq()GNhv&B^hZ4Og%erWzH+O*-BsK zisx0GsXi7c?7a_-YqWcPG^5>nr1B@w?)qVjcA-IncDG^75GEDRk#XAKfpxW8Gx{ph<};d8wQc@@5=BkX09Tvq*D$p$d7z{d4_3s7U>z}=z5DMw8~1x=KSbW47u6~ zBp-8h}+v7HXu)MpvV^LaP4?5Xv6{ zOSN{p5Slsyo-^|6U3U!6I|x7dV2hh+6Qti@99Y#aW}bAm1)$sqy?K_VC3xM0btv^#jI#l3sB zmF{WO{XM~NAo=(Tx{N?ZUo)=#0~!DH3`E7D6=ia0ML9Jv3XGQhD`gXn=_tBy(~oHwa;IXcYwd>c zM05K9byv7x#6trp>lEIwC7u_?Ga9mcLMxr6;Yx_m*xLqlR|noZA+rS6`MCDRHF$4H z=I%4PtL^t*G3AOgPzB~h76({NZbN+dJYeqN?xeQ|*K#t?7tbre4MZk#`K#*n_f?&{ z^>jwGc;05*prXV`;=>;W0-19AS7C|Hea8DKcWBx@yijrcLzKW@a5nPv7S-M`Nx22I zar>v6yq~&5ij~dL@!@Au-F}?C^R~%Jv@b((o_^^A4~#k0iZ8pBhP@7#8*v;O zeSC>}&I9u@ud#Jk8y@J_<4*`i!TmjNoBGTH;~dgu$Q}GP_MIRllSs3TH*wa>uw%zs z)eLn&zriTx4*p0PDK6sM8rv0lqg9=tIy3*=zq@*=dcJj5e~eJ`5HZqq%9j}fx}63z zcIvvzpULj4{&B4>5A^So%FGY?byuHL<;;65aNp(-4ENtY}s0mZiwx8i{1p^Kng2;HhZ7;2;0}d-a!Wb6D`cQ|t zJjY?zMesIg}G8qK3wrsKsVXP4rXTMLV$` zj(9?oI$84YpKy^wV-L}#ZjaX?2R#&sT;|C41Um{2kZE>C???5LLOntxGe7>GCp>NF zgW0CiqHmU$7Q_pVc!J->^E&&Y3i7FY$wfP5X?6r0iuFMRlJF~Svj6uUzUi|HzQTg_ zCbULBPjL8hYB)G{1unCwqAxw84hQ%9J2Kr@X07&Qbj3SrS6YX8E29Mpk+StmyX=3% zoSLIDX+&#RO2fR9quI#IRzVb%Y>n8#W3MCnBZ3N*?3ajVCenvVb?r!I70Y4yxa|Mc zEZubmXvQYOKJj=iT$`C#Kq4*+lZZ9RCF&0w`YA(Cg=_@g$7-AAzW1W9<1X1BPiPC{ zi5z;0@yQ*YIv$~TPB4@#%|K+HXYm9l%_Hf}hX0 zD+YOjc=iN;qDckm3OWPzjAD<2E$c8t(K99_)fo7TRTA6HnacN9$t(cD)A4b^eMnRHyIRc;2*A>ETLx&hozR z@9H7dU1wI$U)Nt*o(*~OWrL})xt?$Zx>bY!tGl~H-N^CECTg|2{dZU(q?6vt?QZgR z15_N}2MdP3Bn&hVBfoZM=Ruyh@J#AQq7@w59~X*i+Y0uS27hsndaE?-P$akim_*WS zy=An!=xr%*2!#sv6xUAeb&3=if)UB_R(H``zJpO(z-eaMjX->N+27IS8P(zInw={ES!5-l*iu&F~6{t@xYV#eWLNR5b#;DTZH_>Y#N20*tV(G@LtV-!d z>~Q>;Gf1T^i}!OzuY($P93)h&ZCM}7D+3)@9P z6142peQKzb}WZ@3Y9oXZ`47xD{f-CUDTB7UCoYmff4=8}+sjB&C|ExAg3yXUw27 zuJogOyYAkJd(f5P`3zSOAPP9ja9_*%`pbQA8Yrcph0i@=gwp1|X4(In{g!d^Yjn_s z)gt|U--M0i#)x?-^&ZiA( z^~~` zZzM(66QI@ic62j}AC_)3S4!UOSJvR9!R8s(O6i8SN=5SK5Kt){z|`?!>$G?r4b;7k zE|(VlTdMox-Dx^>Hw5*>4xLeVVux<05gI!*hZQ%=>RFAaneWI@OB)9(Bhjd*h>L+# zmz%ETs~B00r1;?^q@Wj=PiT2Kifg2JFO*I9bGw5sIp)nFwc(M|lyXSXjWmuE`9Qvt zOY)bUbnJ)kI&ti?Pd4^k6xVt%Nc7+o>Olq`z!5zs0uPZkLGH+EL_U|W%!381D_}H8 zm6PW($Ryo0`rk05ndI|BB1ye4ob^PU6Ne-Dr4z+Ufm}UwlA%lkcTON?JCwy!^t)8& zPS?VpZl|V=-1@Q>s6lYqX@qzSRwSC-ljJU<@+9nacT%U4as(rxc?L4{k2+Y#r9tKt;tAi7%s3Zb%^` zaXIjwAB8b|KFWZjNs!S_WV|WS#)1<^NNz`hikHzwF{sBt!6r}9Z&IKc^()B+N^o!m zv_ynsEt?yNruOJ5Xs5Cvb=DHx;^vsGn@RDe@TqUbLQaX%{897oxThd(zVH-~<~O^v zdS4mJyr$^P=|0K7jX+2Wv&7-P7CF9+W>x=hX5Xtco0Bt$wh}r@gCs1{t0^8o&S!F( z+t^saOC~YFO=A5Pa5OR!1{`2MBke{-02EauQXmG64q#O;o*M)uIS*#%!|4w=Y5^XW zl-YI~dm(aqf{!{}RD!zBt{WU>41dZ|PS2W28S!t*v|X~ItH7mEO2E-hHvp&tX;i?` zBA)Zba}%yGV0;CDn>B1Sm68=CMG|f{IPC0Z)CNZ*zky%CZj?1_KCFZ!emVhFrdy4O zkv)hqwOEUrymk_EHojpHLW;=n^$8hgP=M!iGOpRYHkmRd@{r|;=beW0#Kfx4IB*%LJA-cP?$H9FlR)q{Qh|4a8s;U+B7X&l1I(6%+- zP&cu~5Q`(gNRtu$Y7r{}&9Ur^-9aa1`Xu!3b{vU--o>t=IfUtMIN+#Jdp~u$QVorP z>ca-Gn^CX|{04RcHO?O6MiF8!&zH25ZB+Mk+8rC|ZX=qFnV$L=P3|G6jl zd8~Ug7JZ1jksYqZ9sP)^WJF2(EKF}PchWQRG~!A>_!fwsqkY5#vX|RJnJ=&KUR_<# zE^VHlQ}pq+n9rh)h!7>~hTWIL4e^!LCj$zVg}*N#(eXTEmSY zDy*OiQM!yZ8gNvJ=L&ibXT3#Z!qr823JHU)DyQqxZH0v8GYqSDrYXg7=3BP|j15bP zvw3P};F!yjk zV!~0EojN1hI9Yp~H@TUVmu4;AckV5+au8J}5l}dftTekHaL|8dnKZ83os8nQ9pe-y zX@QIIyP)_Rwdev4i)u9l+hTom6nllKU<-|}9pj^GirqwqYRTx}N4mlIrNCuDz=(I1 z{1+z$=V-eG*0v2frMoII0&0>*LZV~C6?^cU66}0JnXE}ZD!_(PCs#MPbj$_vIBaxS z&1K>{{Id!U!t;^gCONS&3x@zB{0$=vj;4OpW5{xFCi;Y8a`0SuNlUR0@~FcJP4fh= zw0nXp_z{3Iv^_H)?oT27C5looouYUrmPIs15O{iGOALD=pzoZOjJ^X%hC{T+rR44T zRZzDOX*dKyu97X*o3BZVUu85!hv8OviBcD$P@WHEpgrV1P=BR>gEo+w@rQohp+wT^ zW4E!Uo+IX=DU9+mt)q+jXr?Cm7wn1k)1_;=i=82& z76~J$1)~mrkTl}v1m8b*z8!jhF4_A`QXe?Jhy<=-mPC=m#deYuW^7O+6MsCnkp$sc z6MqXNhn|@lfX!tj-(+g&cY-}?Kmt*YGLa3UAC+eI7qvH1zwz9mt*tf^0OQ4{Ja`TC z4DD+ccnz*(yx@Ppm!N7pfDpB@?2O=+HPwk5x({WbjR8>DP;nOSmNAtsW(y3N-JfrB zFWprLA}EZ=az-1B3R`n@GR;}*a8bVS3?3i{mM|SQ5Q3fHgZ)6=705?-B@R0W$`gRh zaVPDKO-izuSV$dphi7NNw*~3(kl3mqzMIg5RQGpC2@SLw$uv+;(I&PIB+X#`rV6-&&H5OXiqGOTG9a@A&7uck_!Fa++D=fDZXFTuE$V-vF9zb#W$T|my zvv4xWg*TYmNS&I|8J`(vJ>t+FjFost#KeCfbS_i}cNBPNp+vG%(T`dGX%8o{g(KCi z14uS>G~7{irOt)+NC-_vXKh0+RzkT>BoXizab){ahozVvL@F!G!~YV z5ui;*DUg>>@h9YILLTBQagTQ~QlCay#ivH1C7s*Xr}X)&udrKpq{sp-4nVJv%}0X8XoJz zSGWlEgGVdT{e*iu+Y0x`-n(GjYMEk{uiRT%vT8xGbORBLD{rr4Qq`C+MI zeTw1C445LrJ|eaUIdZ|TU*@n#flHkr`@rD$vVUip9Asr+jcdq8G8cx(LS;(Hu!nzzCKYliQ}QiyE6XG*or_kbIzu#+A~IgX$*<*fm=n4J{sb$9%2 z@0DaBk%L>v8aZJhjplJ(C(L6o>QSK56p7j%9Y@R{CrF-Bmavf3u#mSqU?QuKpG>3$ zxyVG8ILJnxYei(r0s_ypfX-wKXIi~Ox!HuNAU!6UNp&CM^?fPy8f;heP24g=!ct+O zOwq?FnjMCHg)~Zc{{v8@l?e_G$Rs5Gh*bB@@5o1+%2(6A%i5c%=2wy1D6-P(7m$IA zVixXB5L*{a92Z*!ts{qCkC>Sa_>e>BI;pq!W&}RT?&`I8g*SGUROfIYYf@jN)h^J8 zH~u0z%#L~{bmoW@n3-rCZ|5k;!SgPmywI6nugEY(<9L9F4gZ2$jG$HDJhxmr+iMq6 zPGV2LREg&cH?ewQX;$PXhRM4T8Me_KJs@_N59eUnO}H06Dp}Ozu>y9Y6$eYlAB6p7 zMi!U7RpOwg)BQxfflL$ zEGGsD+g6KbQqIK=8Go;GluH5H|0UWg^+>|87f(@#RCg*sVB}tFs#`o|HjDwyXY@{5 zpo86jV`FJChrHxWyQj>~m+Brwk~=gTOhViR_sld=d~?5Xl5@x3kphn+3o5`72*x-v zSesZmZnKV~5WTN%@an#cF|&pB6>L^>l&Z!n8Z0o@jta^0NPe37iMNmfm||jqJ$9M_ z(1co{W1yUowA)Y^&48^3Knn|ueSlQwLmQNZn1s5VIMC{K;wV1Wh*#loU+^hM6=Vqu zeux^4J7vz2RzCxF&kIjw6BWFe>!2xga4zpOj0BcOkpg!j2VXO4B#8AY-eN&j%mUDo znjM;N`;Lk}#bzoJWgA;IKgy;mlJi;;X)(P;Mnk`e_-w9&$42uRGZ^I%tK2NM#H0~H z{=7~*2<3Fjbq7h2Xbj}bR?w)DtL*MLc_^*^_cH41&+gzsck9<#7|(|9sZ&43^Ug)@ zm~DQCWuea%#Cnp$iBTM6!GhfwBs+d+VguoQQ)nbs-AL%L7aIvkPC6}F*|wM+y}=3? z|Br$bNg|_-G!r8wzXAFAq#cDbki}xAR6I8eZu`zet#DAu^2xy}8!fD>71WD$r-jjm zv_lv%)J6m#VP?Jro*8rTOs)jco7DrH$Vg5xiGe3P6(_={VoD^@^U}$4*vlS~Zde~F zHgE(Y#!x%XqkjvEDX74X4!MJK{@bNKUWHg>uf*T;tn|0LJY}(?O!D_xN|eaDgxa@I z`yp?u6;4u*6-62(hX=HJrcB_kF77{0eVvKO`%gqAg#4Xo9|f`a&a!}vh`~T@pfNI5 zdd?N|uug+1z7eJv!htOnkkq>%DdJ2hoZz@8Jn4C0jG`=As$VlNF{vffjLW)7|91XhohF3z2twR2`SxTr|s2wCt9P9hTsUKF8CIP0t-gNjjfhY+Wd2+X*X`eJ zft`MXNJEY^O}r#*WW}7g!c9?<2fpB?%K+&|ohih9!5%kTN&%|%K;fCnA^3M;vO@^| z(QBNqBm7l6*>p!InXr7S8`+uWhBvW2rO1RGob&`@fg#nsn}=KqTBYpt^AyD(8LaGl zOz+}=Lui5wcSH`&wo)ufs~vWLI!H#}#jZH*xJRnH5mQg>MVLp@f0-C(XF^4Xb!A%1 zA_xo9ipdjar4LGVafRtVR(5OI#zV;b(hBeLYYW<>7w2Y?6H{~NPgi0A*Q1r{LbMdK z^qXYA$XCPh_7=JiUxf3{Uv^=`+&$`TchNq)A8^PMzO9qLuN7NBD@>h(BU!Vf_rh7o zon@D-O7`3Q9x+5~_H($0C71=y%*rQ{&@jMWH0_S>^k8L!7;p#96SFS-Cfza-E}9gG zA!k8L;xKGU;g}5XJG4?Q+U7eHeG9269YFA`=ndbs#nR@tWaZ@y@sp)zUKcAF4;C>t zcc`yPE_hwu8b8i|a(Loo`RHDG$RT&gYqH1%Z_7A&d~}~Y()u+*Ml1n>M_;?O)Bw1j3u`0)^bY_YNO^WYkC_X2o6sYQ8do$?hOq<}wv zJC*?C7SHA{M!Jv(3Io9_4c*{!XM7B5FGDrOQ|1kj)*>cGL)X*`GnR+c%R^rI?${U} z384cCz@fS&uG6aBke|M|C|_HSE#6KIIX&1GWFCA5^Wc5?w~aG+?w640Dfmdz zC0CQ{UW&(+gQBwul_>m{>eb#w0Zel0KNtWOA56IQQs8V-MlidcNP+|GQ?S(|)sa&H zkbHR@%_43{bqIA}f?mUj7krC3^a`0nL}U?^W)!990m72x*7IO4bK}W?ldsL(9vCma zmXPCbxTLi^rNC3HD9krX$t?Ducja+x8#d}zEZh)cWI44cEoNv1q1)2nPP}B)!M^f@ zB}!>9f}m|_@MoIQOlXZXjS6Ba3CBvbIg(5!b(2Rtprep@-mE9Mop&&vxA!xxR(EiJ zT?=(LW-d7Bs{IAQOvoMlFRsP&!o7;z{#M)EP)U(}Zpc&QB%LYPo2vJS+brkFyElQzT|`LxX_?0ap;zT%Zg3Qzq6)sXWw>~pcc*!!t)b%r5> zjLk2;@mB#sAUaWvvD5L&F0YhrhQUkc8yzeYz2=Kob(c~-=qSfvQm6vV-(AEQT)EG^ zA$9e!>oB!~9aq*Gcg!{Y$j#bMtmBffT!DIC$Gyu9j(muiOfxGwN4A36^p+6;Ojsc# zojvAtPPl@+<{#ay-;#?r7UIL!RUdyM#(n&9sw7H%V(Hr@>mEU!`B1~Fc-M$y(F>)CiW@d2E$79jzku!RL~_ZY)!L_{dH

7+ zAFf6?jl^CrQAk*UHpl^yzKm!XMsu_pp)kLlqBk@@tv!Gd($qwNLi3WJ5R92@h8&#r z-!4R?+y!>H2&M2B;3v?25SAg!FEg3W4QJg<4OJ@%vgn}_Sjh5x8j}+1 zkb|r=DMC3@2nZ$?LD7$hZR`-AZvFAbzc|X*AJk!fp3ej>;C&-X1!H+VSo#ug645U@ z5}@#z^E*%Q7&C~`FYx*eLMnVl1-8SZtxAE}BX(VR)C+wUB|{2zm4MrRj)f zdG=;E7Fv^7w_xwlAzqJ|_Bfm(knshsIBf4=fy$ z%OF~TkhSqC^R>O7=i$ZS>=q3I?<1ibG|^DOc;2;WBl=fBFz<=&JgS!|p})m~*YT2d zdtV^dQ4$Mf&c^zA9;9~Fy{j zSoxhhyoCEZELM5+BJ{oJ2kGvQaFdKx64S`Mz>@1imF}e*X$p|G0 zl1;jcwt5hq%c)&VzRnCht#n4=kt_Nm5-A>!C}=}ga(n!J(S0ze79GGi;ICtX#Qq_k za060Ffw95_)>94&X-4pp4=^x_+#nvx+>b=&`Gnbx0?$XK{K!P!1dnBI0;0t-N!|^y z%mfQqB=bd4T<`r*Aqr&ru7{7~314o5Gnv?$p7$XcfzO$Tqj1*3wb>M#fj)2_clH3^ zq!{*cxQD~y1H%pR0n~v)=NKFga)uHzkrYWG9pFUtW9+r)WQgKV#8V0b>LQz1KTvqY z;+>T+YqSN2V>TDltoySrqPY#q!Dil@_QBYY(e20b>_m(SK|qWW;Y`iHz?+?j7P5(U zD6+%j5)HK(whn7d;t_X0xD}#j%FJVTq(K`o5*FX=q%>_@ zc9?n=-aH|kIsYQ;Ke_S7al(%{!gz!D?06${_o?N=z?${xC;Z_5c4`Zz@!pnVsQ~X zgIo%9DP2g@O-#b}QQwB=3Cs*P>iD@b)32JsgPLQ%BiYae zHBT1P3`Urpl#603$|$IbSR#V>U$OXeIT+);OfTs>q}6AZQ{bLQ!Qfzi&X|m;t${{J zVjf2~NlD}wB1By6k{ZXX7tLX0xT81x7LT)IMiky;VMbVX@FR=SpT395rucqD(v!qF zby0XjobwsCsz}L31ofYT5Y&Xh|)#Utwr}9Thvl#klodj*^SP zgPC$LrVY*84c}{|4B$DcW71^8km(u2W6w7Bi~ty7M~p7f3iw;ha*Zp|rr%`Dj7@-I zsRJ_rO!(aWnOu1N{{~U({VesECd&WI1R_>+h1J)emFb{0nU^ z#JYZ6nT@r7?ra3XvOf{`aB8jDEqI_9dpCS;NCd%PIB_V6V($4;z{G0B;TOcdyr5c?pvAEF)OY0gXp11vN5>ys*>qx}ho(33w7mx1Hd4;s7Y=Sgm%+@(w^!Gdq{as|E zzn9^d|0tZNrwr04oUrFNYnTD&8&4$3!hpJi=M|z4L#{3|g_( z1v41WoBs|vMkZs(LYe^(aRv-C4Kd`#SUcLOz15|?)J3QI=}Bo_NiuIXbt04a9 z2Az~cUn~HSpg63!LA~fZYPGYzc z%!2ncYf`QH;C%z0BkEJ6>tg152`$mDmQK0V#c{{RPX+0Rty%9m`09ywG8}70A{vD| z)Y}uf7B0@MR`$sj(S5A)*g6D)K@H(On`k+S9O5t(M3n+x615TzADIB01h^TX9Q+72 zfx7)2EvXb5sJ`DD@f`W&$#ne=OrSc#YsJESZbCLC>x?eRI@cmuFTfV%#ZFBmHHoBm z7HUzJIS#mDdlf;ku=4-BCQY)5(4-}Da0dDE-05PzIkn5Yd#)NzO<8g;b}BGA7ufJ; zt_6Q)bRj(EfB?{fyXZIHRrqQE7>HvNQsBDzH1HcU_3+M5)&k8MpVZ4-bTs6Us+Tox_Kg4If`Vuzi0CK?&khsxL4)Yyt(BJCn!XFv* zx0Znu)^f6E8ux_M=nWg^9*&bi_5{btkD&)EjUNv#dN7HHq`85BFc68PTO=@wg>C?F zQ-N=su;&fJN{D5@|95O!-ANy^_WzzI`L{rx5SW$2S=gfJz!pU(Z8;R|!2%(sU+Q{- zzsSKQ@V4+B#NgGK9Uwe6UL_Q@kmyUT zVk2z^G?lXg2{yWCZg%i37S6!*QZl|1GepUF7kxmeWPB4{JmXtv=DBcuJ6#r!@1RL! z@%VhYTsYp2EQ)8ml`dK1u>~&$=%oU;f6Mr47%6o0zwqjiJG5zh6&BKQ)>E7}Y(j5u zzt59#jD5;#*UTDzaF^_P1QHV+OCRJ7w$T|=C%z>lzx39d{3jp%oIcyq8tVpEk%9-d zq~&uLeJlkilz(nG>rrABUT5uI94hHVXpw;(2i3FOP&Vt4E*2d(dp=z$QfWuXq5790Q8k3HoVdJ$17m1vsrz z;9a$ti2JX=qc4*1WIDlMMNrkcHz`#Zych3 z5MyMKitlMmE*w|V9~5;bj?%l?QF?@HaZnUR;Oel0qS1-e|L{y$5sblule0JsSxBEG z`$YPM)0q>*b#A5R zJ?xBafW2KnVvdOc?UV7RDFXjdU*HVj&O034XvI4U11S*E0)zycqL-{Yh=l-(jiKdg zISL^Ps8xe3rs%J4QRh0)vTwC^*q^`li36C>$mn8{bKEJ`XeTBm79hryfwCbUO~bc3 zC@RcyQ{>Vj)y)yTpj>jO8k@0Y_?|q50RfI2%fi=f`P_tTrsz@V9jyqHo)-GAs~`W? z4b_jAnl>YvXK3bD@&VwLTBs06N+w$TgCSU&-MOtB{PY9-9}s~o z--KuWrcZwKh1ZEgI4-N})L9sbHfX8tTs#rgPdZ-7&=LCAY_-fG7rn~lMR7q9+#wNk zXOxV7lB`(4bVZabx1pa9VWdd))v#k&aL_+g%L46Rv0-4jF^gsge5&m7WCf;^H)fz} zbUxUjbq}o}7+r0HUyYYae`8Ia&YRqhe@mBbXDrv6{L#&u{1>tX4*1wk1rMT+Y(@7V z;!cX3(PHTD7$q$qOssd{39guTVg-ri0t|n0s>h7j&8^@RN^mK*mbTkqFRcg~c#6LE zF7Xsyg>9(7(`bV!Kgw_u9T#u`ebE04}A_c?a zf?Cq)+iK*27fvcc5gxJ_U$^OBFknxj)w_wu1M?EMpcuQe$jy-suLk=!-l4)a# zO$@?)7p@lkFHXRBCybmn!syB*%U$n8(vtg{7@%m3jif#l~=mxl=FgVj5RL;c+=l@S5OuV7|KaJ3rlNjB) z4~PI*R}cY~zJW*#LI#Y?7h1{^F^dtoB^In^MA!m1n){vlIU2Rs_jZLyi3+Pg6R3+6 zp!on3OzK`CzLWC%76=M0!62{b!tIV43MzgLl&k|^PT+J4=>9)Bt0@x*zLX1(bsg$V z%jTz`B8<+q~O^s8kP0OjL;XY&+Be3A!u@w`n)qUC^9+eZI80njT8lJP-J zOYG!AP^RA}SFlmOjJkl0ayD;Kzu8#YxYBK5)k$^Z@fe+eKe|u$b8|k4#+WeYkES!{ ze9=`-${MbQ0Ct&ipwOd8Gk&L!_D{iglbW&Apda=VcT1}m7326*c)&604`#DMCXSe6 zzNJypY&HrR=un?UoX^1gf;R_HKON9u#sLcr7eygS)tUvVAmLpgR!r5Kz5{ z-|n!Z1@2|y9}CG_MWqmOTH1!#5I#$V4q%Js6uim~R&0-*p0v*K?e4HLyAXRd_@Y*D z7yeDLBDVh%KoNk2ChsTMym8^*8=^kGD~D?Au}0krCZ5;lzi(?W*(%+NhZ@wicOKfq z@pvyhU25!e-q_YnXzT;n6%Y=lq#Yt)k^+xFt7w!ooOs?}kSBUCAVUM_A^VUueUV2B zd;vKqkv7lFF0Sp`D&0Ha9IBBL@UI%evaHC)lhRG9`xAIt?C%;i`v>ii0{XP(bGUi!bfn{(TKIQl@|7F2O_A=`A&j#rL;qd5=;0B(T?=)h z76ty--Glk6#-O3?7&v-}WBeLr5Y^`&>Wd7&l&`;CTL00XXpbh)D17Vhj5&1&r~?1? zZU}Dv2EP;mcl?_T??`vO1-xJn?e*cUJ36WV@+p+_=oiIOLkqra+}ppc-Tzw;e3B9H zEF86|qY}lzcDzyXvVvQD*1KiJw&=^aV@@GS2aKpp*P?wZipd}p(nENjS=u}`pFR*Q zt*!up+|4HZC4q5EfA5m~Zw*H-9FK%&%u3T2&JX$CARnAapce`muWbOs2QtSj!9lJw z{z=|UJXi8hz|=v*N$6>E2)CRjhqG#fd+~P$eGR4P9p4u(Q|mbL%l?6fS{efQMG}Os zVqpt;6RJh)g``FJ(Y-A0F&pjKq+kyZiyP?F&CiJmNtz7vx!aYs%u7|ZAaS>8`PD0Y zH1Rj$iI&NM=X#?u%}^p+6B6j3SZ)0>8}%-Se)1!6TBT2;Qucwc7jffW`gXaT$-d%3 z$1|`SfiBTL+7PHOAeH2bM_<8gHp9X9`X;@$9McuvN{gO@atsi(ah34k7 zuOpSv4%I_I@czA2_bA01<zDYTdap}xJQlBU6&%1Cn_h|wIGkWE zeRF1Z1I53n7-t|l#u-RA2$Xonu(t%W#;`Z~AUM=>|4utwAz&6m^t2~IFp@T${IVf7 zyUd1N5WH)86=s~Xr1}pFAcw)D_!q6RUk5+gE1tJLBl^FaCE{hy2>!)^Qv3@uJu4fu zFF?UR!|ST}*F;7mx8P(Z`D{dWvR-!R6J3dswBJii^i08rKQJ0v?%-8oCv==_U?;^p zBP&|rT=fQ&I-8Y*u9(laYbb?&OlEMkhS z!D|@5VnaA-=$~Q%W$@||4)a>klO~YmQAa)$f!tP7f_g6a%+2Q1!s|KghU+y8Dd1V! z%rkmuzq7c$*pBif_Qk>CA<~NhF86pCP8mt?k!K`HDDl7@EOyd8Rg8Ot7+?_jeW9ii zWQj`=XvFJ*FlMAiNx5go056(A87eIF5g;z*8^LZS<%|9)hf&2)1g_5uIr%rQq|hR}2#`T4Nv-_{5p56}7daAw$zM}OG+W3d zjVAK4g4YrmLGi>sEo3mb=rvL_LDKlk_!XPjKxC$L`q4d6MiVHFI*G4R5pghkhtWf1 zldKuE5>SXJ7&D1w7Lkx_0TjPGahyPyxu0;6UR>+=g|3@7Iga8=K4QXbknEDRC@zD1 zxQc`B;>zOS`Ll}aZ<^x{-pW43MP`~THG~0g-a-~Gfc8Qb=m0N%nM@e+VGbY+_T&WT zBN?Djq2oy*4E>Pj+;Fd(-Qiw%5ereHY<2y?LF^leE)0;_(3sop-vN7s84A;dtec@| zL`(!&mto(MGZdSkXnvgXD0^85Uxq+^v;ju%=x;?qOj0|jzJ%pwoeqyiF8cZjHl4J2 z`LU-&w--Diy1g8^n3hrt{GDk}MwA9nNO}wD?f_GW0kv!t{If^I(@w!(_WEVYu118F zVAC&c+Uo_J7;?hss3vSuA#D8=YgGz-h_ob&T8Q@VDt7S(9D^y38K1;cl1#CjkC_Ouw z4x5M0C9{xIM_IZhv*1mFWV$Sd(mk2JhMm?HSrm+;1{GXy2}Utij4T&cGJ?+gzm{f6 zn|(b>f}O1q=s<}hzu5ncguK2}5|9lDa?R>FE7)3$^2MkY&kK6Z3cgp|8pHD!^t>vk zB={>!{+23O)jQaQI{kRS1h_;h4t`wR`hAxFOHg)Mp2=1JlsK!84OfPsii!P7>Nl|? zP?)Kf9rz4;G#Vh&Uo&%5<;A;$fkR20HdZ=s8& z9{v%o&UzUhJO2b~y_`u*d}A6SM1O5RtRE9*5dcRZYE|yUW!FIr+P2^)iOR zUtsz_yuAx}Ojq|mo_%r=mpBtL#3cklf|4L42nh*im=JfZTUwe@)FlbpVnPx!O{Yb% zwWaj#ZM98X3@wqkBxqe*mugEWb&gS@>Q-d_pS90SZuP$3_xF3g|6iUbbI#e6P%Aqkz;2pY+;Hgo@&t(<@;!M+v4^!e)-rB0eX%Wqi*3Rz+~DBTg#}>LeftuVS?0 zoklrH5fMhp0}7|5gAzRDY!8<=4}GbVfIaQ{gX$^_u{w%(%L&(-ACv;H8(mIVL;}<* zs#B5J*Y)mZFy(@RIFK$VD49!8RF)Hv2~-<2ZgOgivyT2N&N}3#TY3nKjgYk@M@Sno z0E*t!BGTS*S{Iy{Nh_l-^$j~()EgmHl`?7(#XF)Qed9em-njZ8#8M6+ae5s>V(?WV zoCT1)0HF|Gt3qy3D=`NonP(mK&^q`&!AV~gaKd>PjuGGn=rn?tgcY=M_be#gNYJXD zq(c^Mqa^O6ZvU>ca1!$t99WcV!0bZ2_?!LVZt3MuKEF`YMY)i_G_{2WgZ!FeE_{^;Eo z?`~=^xWeH{XAAo21B`o{o*)GWA=5Gp^(@eviU(jSL2?MO-|!Giqv52KWA;cBVTes3 zjnuII2HI|g$ez?iDHw6%i@e$wKKwqR^ z_bc-1?L}K`C@*0jkABwp-mIV1+#9X8`4^Qi8=^&Xc&EEq0AAJ zB!!X60Cbdx6vox`gu=-2B-H4+p{Q&`fFPF)AO^B$mX04f9x=tbDkL3G5@7T2bX(^W zWc!5AiB@o#Uu}8GKGayQHGzbQQ_&yMULH!P^Zn$^GK9EwCi3eW|$1niA3O3G*L7CZ3jv{w3lGS zAe$C}px7pmYwFxYTdyM1;$otA91Jpdk*To0)|80?6ZA&L2p`=T;ggbJ6{Mhf)*&ju zf#-y7(w4N^bOIo{g@hX6Urw2*>lLvv_AuHn7*6cd%6YA9@88OO#EE z)IXcHx-IE1d8lVJ6&;Q#p1c*UbQHFCrCbQn|K}?OOiZGVVkp>#kkwAp? zGr=-JsDvW&pP#Q=m*5(K0OHrAO@M6aUx5jZSPJlg7hNz_K7XUo#Tv#EArTv<5wMBM z0e$9)Cr$*$lUR3wD3>}%6bJBhIVZ>zn9-}ki#c~kGSZ4`66+J2YaWY^nhpU#$HU!d zO$!G0cc~PN>o45HnnVN!*8-T%mMkOZ7H5H2q(d?qazbdwWj^#y4WKTjs!^4P_Jz29 zqd7*zLo7Pzs|*h@Hl`ugosek3&H0E7sA~1?tpuswJizf0!4YR07s7-aFCD{bvTdVr z8*jaAvgH$Y``db5Mp2l823T;VZi{t9wF+8bM8auOhMkv zN1iZ~>wg&-QX2u9=-W9j#al06Ezx9KvleARs8-A(>E_tJFuvtcZ>3oC2rURHb%od# zu@vieSKvJa@)bVJXfa#ulwQd!@=USXMRew#U3a!|Hk%5chKG8YYuk1YL>L9425S7GoT|F}Z=V4wU;fo7YJHa@q z;8mfUc>g4o&S~t5+|@e@dSD99zKf3$4|9xRAkHi!;#Ao6fuIlSF_b~O6yW=J>rJqR zKL5Uca+fUUc~z(Zqu!eg=<(jb5FGn$gCI}(NdtRYS6vn~V6_Pnh~8h(@p2|Duf&J_ z=G>e4n`c@V;7FuZm_}>J@qB_WYf@W@{vrLMi9vZP<5|9J0rq^9^AhS<$Z8^X#r8N> zqT5hhJg_>qa1SRpw*Vn#00Dv!&2-pLGDae8Q@|QA_=sLsC=L(tKz%xc5;!21g%=ke z!GIxFlHCR~A1$09bB%u?NL>m|`LGWE67-U}@4gUwS|CXT^!k4qo)#LP)w>xX72}ag zs6O_HE4us}DmnlSUa{8I;L1bIl>2HC(TLglr9Rdci*bV;Y|ih~Ef;C|;(x6RbVg0+ z&RzvKe{tKj*hfs5gSe!=ttuuejt0NPVZ3r1-GNw zt&l_g4e{3Bpp1dpHMiaXt1~l<4gHJ-;&-?#`H1s)71AgdOkT}}4%Qs|mx5NVhQeg) zdP7+(XnK&X*A-=PvEX~e^l)i6X=4U06ENmC6adHM4)z8~oKxqTTr1GD@+>XKb^*Gw zvJX$WUo6*xfc1a0%4A)H29^Aid-9oeK~ICMTdufd@sE`o^zhZfWDy*Xrbz)SpP*SDU^ybTX$Hf6x&D; zfk~1qnP~nVp2#WQ4QLYmQ7Ixf(s>ZIBAqXu6b%($ zJ+U4;!pY=OLSP}Wcv6-0jl_gG`V%^tNE6w13F-w4qI%WNiHL(HO6jnPqk&@m4kw?a z&(i+Yn0S}#wrm8YjkO-gq-&#-%IJQ_IcDLr1NM?u2b&q6#Abup_9p zcjKI=0V#f~SsBRjOrJYB2iHlDP>N~1!`ig0zb~A@aBfd#k?-z92>NC*e#DhT5d}`; zE|d%T3aG*VveySv(_)fV!gJ-@bzWdpF~e8-JPQ_-ZKtSW@4kp48wXPLqf|amO6Xee zP896d8o%iSkg5(|=r7{nGCz99(sFtyQ9ImY9>6cA0th%A5ggr?^I87P7(Pru;Bt)l zCXHku_RyjzB8rXlBttjc{{G#D{ptO98i?tp#fnU5FFZhKPikMh=TXE*S zh3WH&W&E#s)))oWDtXq>nZH8oK$YscC>LMT=A0#L|1VYyzKHIsRI#gGv3RG3lvn)s zPlO_Dm(`Ee&WEZ+RQN4c{?rNm2$_2o+k%i}mCYf0z$G$&k?R+dd87r9v*b|xgaRvXQpLi-|PQspAe~n!~L^U2sZVNZiH>9jHDeAe)AZKYVqp}sYbVhW^c{` ztn&z0|ChvKiLdORCU(}ju|wbq*RyK)c@RlsQ-=v~DAq!pU`BVj2WrcxYyOWQLH)^; z1O7Ju84@?1usiV|J@z~xssxEnj@VVI4MZ>gXLu}7PX8Ovy?9>2{z_+5yyr*43ue8dVEt}NT;7T9#qx(si-CqaizY>-w5Cd9=>S*}Y0*0Kblpgc z@8uMZgxg*5I^)DdUdaWI)_y=Az#Z5<8vpBFuW0kX8-I`MbsSWz)z13g==F+Dw|J0t z@;ElLKEmDMYcXa2Z)mxqRsX|SqAFd4W~t=hxDwKXc%Y_!1mf5gfD`#LLEsjum( zUqRu~Y$yKT)TuYK52`fhTBd`HjQ@XD{5XcauamoPEn+42{{f=)Mw`nkZh0I6{_pXY z^fqo@B6PFr%0N?O&}v)IF|HW(o26uzI4P_TZ-bC9t=6tCq69G?_o4aeen&Xo7vtU0 z5-+{!1>V5=3#Ps?W(ui+mlyirs3e`e00W`o$Pm(5GKfCmPiLSHosW=J9UnTkf=dM1 z*^?!~U{ zkhGCl?IFgn$$G>cQ?8|4Lby9n*S~PPi*{5`Es!XZ>S}~$*Tc2<@N`e>l-yClO75ai zCHiz~KVtS7E*!#2(jObkhe+^%@xpeEd+G7o{hr+ylXK%_aw&7_fw(HPB?76N43&G0I&9rApnLVG2JEB2uYkd#HzG=?xJuY*z#Z>PIfCacQoeBmZI<8Pllp>U#j)wO z0J)MXeasCwK0RnR6n^ww5nmyT$1%*{PpDAk_r3Vq}-9`2Z zm;9wZqVXjR5v~HM6UgSEZp@~FbrWWJ!97RdT#$B^sdZ2K;N1cZw#e(_L^F%=P=`Sq zp)N$JrKm-{I2qHMjsytlFtqF@l$c7m`?4LHR;x0oXqAA>rK&=maNE71QDhM8)c2&x zZUO{!&f`XUB2dg+*SbDjgE-+Of%5ItCl+|7Y*Fn>c>2dTb~zF7kdxI0lNuQjH$H{s!mqw3He0#fTsVZS1bc7#jau{}pMQNC2FR% zS}5-vs0^p2-%&rK=ATA55c(#O9xnZIYDE-KG2umI<@p@wU+oVccbeB@9#0ct*A_Oi z2IA2A1;CnndjRZAq24IKZ^1Ai$mX4nmblGr`GC+lBy zer%sa!+~Hd-Q#BL0?IeJIr23_KGAfnx@pNsqvr`k9&M-#UzE*JOOGIb2Rca8S8Yw% z&l{-a#>Ie*>=4P28jd;&x{cr%PNCK^HJ5O=l~U>eT8wXrnDgo!idm^oYQ@t7KLoaNn>3}J5$nedj2EGFS=fBN}>HlXpAqCs^1X>MIu17G~W_T z&3~>L0$b>*Y9p`u5v4L9U^}6T20{s*V(Wq!%Qz35!a*_f@vXL{{1@PvQ;xplh}N-0 zN7;%^39h(14ji?PJ54dLkbuq{_f^#(JCbtisX6?bo{>7 z1rgP9gD+PDmcG@ODR-m`Sm;0KY~c=Wi@EwFnl7lgVMSY9@JlV}i(b0PU}gqXQ)W)ay-S{HREg+&I@JkmR3g+ZdOER|0G=UYC}-u`>D!{trrO(6?Jg$OfC{R@ zqyXcmt#lrRaI*ia4{%D9HXIW}hj7v-8^3#$5Nb2%+eNmjOf?*YpAUde9Sxud6T8@s zfZe`mzej#75%tx^2>gIjL2SlSVFrR1PY(gNOO}$OBb8h#o)|}XLeB^6lpOL%$aTsh z-Z6Kl9O&g&sQh?Qsle2_kYM1&LeSESI*sC6Q@oDJbLx{6QXe7e?&hGkn(jC(Wvek1 zcegg;-lw=f@TLnwjCK#TImP~jFzPUU0uVC=N3nu+Uo#8VZ5#{^g_ROv|INqF0=Tje z{eMWAY%%7{(Em!c->rBD@mA)_cyGV-W(2(D3hD{s#$jSv@4 zaiz@=*Ms6L?#8}Ak)a5xLU81t=S$>Mi3QQO+J}}|e{XFn z@_Dbq7u^p6s?>hULv)OI)iQ>_?kpgP8QX=xHLf1%m3oK$y3APY@C^D7#Q5)q97CCDmr$w8NFu68Cg~Z|R#ieVhdMD*jC(U~jZCpeH@mC-dNpXRq zuX|-{52d6frX6_E7683e7BC^Q_Ci*@U^%-BO3=DfF>6qaJ_EG?Abr}7BOU)oYvppZ zjFJUYbCT%|)Bh@V^RHRN9~wt8?VyfCf`mK|)vw2ZSsK-*dJEja7IhuM2`2@VX*G?` zzad8)q7~GvOLNeyo#<*!MTvSi$|TnsHJS1Xw81L}S*iO1wInKJD++N>vPI8CU_&R| zE738PCrz&TOPVzgC%SgcX0|E5BUY_}p~mN4z0 z0>WGXIlXra%8uEDhdL=5>yM7~%5}z}PRAtmZwzay=rz~Kc6w?FAZ=|e5kdq;Dn%No zRZA(PxgJ7PB@p;2CVL>nI^ch}5hL#F2R?!Ekkd>>t3a3DAb~Uh!s9((LXxcEJrgWAu8hmD< zAnV;y-hgYUp2im@)fjGYuoI{2Yb`1Y6i}|F23f1+okdl8xPcI z%nR+;_UfQ(O)JWoMGdb>UC~>e0}0`3nW*#65vcP;v|Salqpgc*oVpa!9^zYDiB566 zK)Ku$_!S`6mQ+Jh+mR&y+9I*50+L@EDVbVZS1)07E10P7M}2~<0Hf1*2I{f)6$ODV zdQ*j2Iusg?78GTu((TBly79WhM-5aEojoF^9tjN<(>(7hljnB%^oSEv9&7a;He2n_&0I}dmz z)}>pV=%{o+q`^F(7l(PP-m1}nj#?FkDxJpXQJ@y@!mWFd%>eP0YfFL1R<474;6s;r z;rb&uqCi{X?i$xLABXor{)tNTxxNoo(%l6KDiOTRDh7g!w;e?h>XimMJ-R6?$%P6x zSRBS5trUUkE>}u-H{b#}7q4;}-<%G+j9K8{9i3iFKV$M7psz?c8wvIBB7_sp>FnW6 zgjEzq(^zdaJ`hoUK-m-|1gh7#%AvGQV;IWOGW8-Vcz~sfeMfsskJsX=QJF4YbmF-% zjrtkWX0oDFZ)vm)Ac(<5la*6{EmFG~36U2pM)4tvMy-UioDQ{ijk+5b+^2V@f^z%e zsF-_TK(LIbdE;pLRBS38goRRorMH^|q|-PWg*zHR)7yrwR9Do)F0I^Js?XhZNz`>TlvN)TA_knv+O%V$b$Ul^ubIXL;LK+jS%I;mF}yM$)wZ82GYb@+=T zCHyewp)XG&C30~^Y2}#lrFz_zwx==SZqPff4`Q9_1p!-3EhF4zPU`Pv0eC-%cg<5L zDvgFJ>Ja*=1%r_~RVNv}SF>q?P!4&y>^XvD$Ohg)^o?T-anf}`I_?vJ#!E${$2$h; zNAcCK=8r~4ZJ07|usQYgQ~2Oljzid*p&AG7Jay+6GLd*0&O!7f5D=kac)3;L8&eSG zZuKX>5FNV78Y{!3+$oNFL`~(vv~^Dr&3sHNH&SuA4IXn+oez#ROS;y;@&Z!)oAb7Y zQ)VG!NWc-hgRdLDgFVtm(nXgOXdTLz@Qf?lNX{oYPX? z`q58s{7^ecw%|HbU_FR5P|2pZjWka*@szyUX2L6ME! zeweVgt7|_;L}t+#ib=h0p+c7jjEa69AaYNTyKwj&;(8!AxFhjlX#~)lRa5=oPW73ds+*oFwQ8!*Ch>?~tYK>}q$V*bXQd+fZrGI2V;d;Y z>K%@b16snKl=Do}8d z0oZSF51BA7>KA$cfj6caT4HKt_5KMT(CtW$qFP@VNQzrg1Wn7Sirtw00>Mt>CMwQ0 z0wK`7;Ti5UuBlo9FR$QLsvuGQqJ^ko{y%C+sak_54>b&+@~lCjNUy-pG8omMMl?V@ z8N#RbW*x#$w?&Dzm|vcQMaSl+4FOhc-gC&H1HgRq`y+!oyoO8ZR^M#R_xEO<{Vtn8 zYaW<+=6dHp&As}tmKGlvU`@L*66g>8ybV>u^{CQ{-ek~;oc$w#kn7zLALi_Pa&`&6 zfy+Voh6nQiJUr4ynxiB8Q|c^ID3Mxw+LihuQWO6PL@cqNRb9>D41-63q4P?kYG77I3G!qb+o=ZGI^TL^Gek!y~N=n)x)|PoQQZdaN?`T6y`alM>p%q>!Riu+vyJIQiP*=t-BI6zO z0@DzL`Nzj%ZQXnOXzm}!LYp)QB3dd|J2t}aiO9^EejWFgpa+hl7*|6`bkEJW3U(-z@OPIM%CQJ1kIYyB-p6+go{(To2*I?&i4C z*Fs-RR-AQOA3z%?7kRZlh9@;+@nF%6r01IG?X;jdYArNcIK8koHqW?}{o!iX9@{wX24yH5+*Sr>_hd9P!@U2AY*kqs9PRPz)Px4{?S(LQ$N&aeo zHpH`0C!aUo9oft;vPLJ$yzrRX4aLVog!!HkD(`sj-YB3M)GMbN7SR?7)C_YzV1Eq(j`5uKGY5Zyj(8hC*%A9KFN3Lai zto^B5@0RgTOl(Z;6IhxNiumG@+;1Qo?HPrb=LWK7(v=Z>*+ACZ^Uj?z{;z>7%X21P z!Ur*<)OG|PFo<>V3`4-&L9BO+O}EQnyA%rk!UA!t^uhzUJ49Hp&#Q<2D&r>yu`|uu z!o)6sJw-8Sp;3rGf@c-dd}n4&rC&q2W@eo%{U9B|JF`QP#pYc%jF2?)O8zJiwtD7X z8BMuX+!u6=3*QdtbslpXjkE>~LQc4aB*dmhDs(^YqF}fP!PP4kZbFf>M|tb^`|-|_ zM zGH53+8yx#+hqNdJJ5OzqZ?)kahOl?que@1_P_pq+0ZisKc!bE?+lp}UhJ zS~rh+VbNly@%~Z1dML}0_z8RSVN8~!!H0SH2o@;CAGQx1!NQr;@_YM&k?gi-t=iCm zDNFqW{N*RvGt!kq_VZ7&ex7wNbO!Pk-#4HRdCAwDC9wwf{^MDFCVdiNA3K4qWS%oU zym{$FHnUAoL#%sb5L)YBBrqR`u{KwEb~5WBW&L3jT9CZ@PC02T%3|(c5>@|R2oy4N07EkcjliA=}(K|&0 zg5q}ag_GInT4%o1ze~QgpPS6yX43A<{N)rjS$gs^zmURAo-GYN{E6q-tJ3@T__xoo z;5zThZs_@0e&snfzUKEZH>0XccX{Fz7ArNl#9yDnp7$JH*M~ox!rDqLLV4?{XhXzr zeCSkGUwW>sJ!L9O_w2l*7J$xOcno73yY>#5t(yAU&(w<}514bGJq9ZL0x7_T6!LpJ zq|W}C`^`YVd|~HpXRxW#^sRj53|1yh-(p|+0&B^n!{_<7ne5e|s}E?KR>E?{W`pG( ztT%BjJ~x7!hSv1q6KAn->9gZ}#VjEB=?DDqEEee52@xtH>V)9pLOQy%P<`q&#}R+A zekHzyTkx#UQH%mo@ z*N^a|RQ98kd5X75V-pQ8og=LI?Zv}v^ZVsIU!BI<8=n4IgnoqwLTCQMkEOBJ_3WFe zmu`6P7S9y#;?2B9I$I~Dm+%kMS++E%gm;+Bo^ILD3w?7Mimy8Id*IP#yzU9X-vT6& zPd{p(@A(&V*odB)~gU4)Qbe*#arHjeXRMth?8QnGXW} z7db*!=^;iD;#9YSb^*JO$U^r#sj2(9VBtU@C+8i>ZaDkRilgV=>u{bKe~hk+P5ucwV7vMKX3cKC9DbK&9j-ml-8Jc z&SsxW`x^0!*(@UP#ql&bgNTAj{JbvqCOBKa+=#bd3X(H4;%k<&=F*KoZePlp)jtZa z9U=6J@B&V&-uZ^BOW9oMoxMDH8EeQk*yk@}^O%&_j^E5-qol2c++=0Z?0I{pl|7Ky zH{9|n>sEha11jgJ+bh;-^adYkC3vkzQm1bc{9%++A&-lA4i$i5<9X+4BcVmzP_LVzZ`1Vz- zHQUY4tzr#>u53pAur%bLMyK&zVyBSA#}TTo+hVV`8bmJj*~UAqVXxNuGM9MBIPV>~ zeE%Bui4?Pi&*m&ldgg1sjd%zc4-m13_D;iG~J!p45Z-(Sl* zw*0{dRR~pZf;RCFA_bf*4VCX(goB1u$0mMjEvWqKFZrN1(VuI-&_`}{HijNg@IUT>6+Z(4f*oCgll*`h;U;k(~reLH^*gEzkG6eeUB%HVctz<=H=YoB1)R z=a>BJbu6-WR<>T_tJ$JPF0W@brN5T(#_L)4&XXX)%4>^q*VbBO(QC7!=F|!9+TPCQ znd@1M*ERGMznRAZ_?h)AKq~l*UtbUD@aAUz>^p3Qbm}dB?;SRV2fxSKw%PwX38D&) z667H!mI8hLcqjD5y~k&~$6CZKLRyEvk~@!hkXq0W)B=693PajUj9Bl9z3yK40%26l z`FHuz_gHk07m5+os95WSD7b*zj0?DNcd3&5#84-1{XW|+@iKn?eb%Vff-*hp=8atX zfK8Vs{EN@}00VaXExzId)>{g?#gBi$dRgROi2tXB)F8m#L-QSd#Z8(;u=jg{Y2Wyn z;$b5aiPSuJB0GZ1Q%Fd0xN3^)80iqMg3EtRuY2^>oy>jVWA_RbZn#4F8%$5RLxOqJ z57;p2`o}tOf85l8`|PFwZo>Jqm4NGvQBV!I-6FmmxL6wQRe@VyHN_%;?*h*AD?aT* zu-_Ly;+H>UA4?fe^VdFNTct)Hy1s+?^^e(%&T;3fcJrI}%KJ%6I9oAd79*Vc>dUw6 zwoS~FNv~S$EB}Q9?;q>v_#QOf(|b@vuCWA{~C4ugGH! zr5`WzPx3IwdFL`O$z#20<;~Ft%l0|E&6li8v*F*P5jcDWClAw2Q$^~<4FWO5gztXH z=Y7c{q`yw`PrhUwq>eZEnJ?KBlJh#R@fB;<`uue_0?PfS-SoN-QG|oPmh*vMLDs#0 zgTMV1vTnY~4P>$2E7+|vh1G3xPl$s)6d$_*}kb3wv3L&=dIQzg8u|7q9Uzwy?LQ zXC8|D^>5g!M3>-`AKS*P()$lY0{`%;30l~=&CV7{H~-e77rj~~|IAl;pY4#O;jZXi zxmBV+&t0dFT9M?D{hW`ao9L3ywer_`nT)_%h4Q^MWW64;eHSj8TCk=Sc@tg zO+-Y@@b?9w&7gQeXbwb@FHx~>gsVduBP1D?h~abFE5vDh<}wis1jCbfA&F;Q2AMPh zT_7fJaiil##f^*`0mCw1u#+|Rdrbx8XP~$`jXi(o@9%_3VAy3oeHWDdsLOomF1AqG zafy59v$c~tEk$KEqwn5Iz&=3fjzJ*(9UBFtKkRWsIuxyQL_iNOr6Bku0W2J5utz;W zSui(^S*mB*Se~U>&YImUVEQI>9}y~;S+d%mRMILfTzt{h!ufb1BHomR7N#N-UDQa^ zTjOvg7l$W-agi-{sGc5o%@hY}zZ91_E37Cf%Xw)5dqHYuA`v4elJGpuc!Ibds#QR^)7$_UYtw_KJYsKXfF$C|L1=4N5a?` zdLFUp3~@&e?XAoZjgJqBN+5UUposJQ_Fnc?vxVm<-8h`Rm5&7UmqDW@ifjZ2SMmM( zSgSF;C=D3?^b8lREC`8JUq2@}ojMwUYD+g25w=biQWueVcG_M&O_z#{!RjbiV6zM! zyPpL?Ku*}tTKJiEdbH8K3UQ_+@34$@o}Q0VwGCJp$}AfzXKSF3xU#|Fa@IY(;cyR= z6&42AgTo1GdWnLTCthMHM89||;t!lvf=kOY=^#i`?x;XoG%P>X3kUd0wEuoy3SWGX z)wje^$+REe6nm$Lr6s#-p#o@89i8jS5`Dh`Ph_oK+ZaeP}*G= zScJP@43N>pGeK8%G0fhD+oigB4Rt$KVFWG3W)vNZ%_yrSXO5#%TvdxpKUCUnSgH2b zm1<9$Dr$bdqGn)MueG95xXDnh9jMms?phr)U|GokkaM602r#kuUIogBJCMFcdHNVv z`c!yR;&^Ucn32|1oM-uOvhSko2Ueqz7HVZ4BLfO~0V+MGhlHogR9Coq;18OPU}RwH zScn{#gWI9$7{=)#v8LR?!6sO{V&U@#gf6kc?nmTE%aUS*;%tMvs+tXsGfsqK8LA@u z7=_1+YZx4sQE0u#c70serCjfW)oNBsPXz|zMLS8>WcsR7%qu8{I5M~tygP9^Cm?ki zn?j+%xs`Ec{llrW)+gu?p=)XN$QIbrY;9$>z6=&M&EV*XkhB&S+Q9`Y_M!s8MI5zp zykD9O{(_R7#$!JbtGkB;B>3AcC#vY@YqqwBB08qhz9<2lv2q))gE?vw$rM9y)3Oc` zm_W-96cvvs;ii_P!!Kv|Mu-RoLRwTt$SNGTDTvWwF7Hm#JRjN#Yi`n2*ROOZYh5MT z$*Rfbx|0#ULd|2*E|O9jR70mD0OC*Z(x1-g{IBNAR48^32^?dIS>X6vSFGbD5sMte zI}^{Kwcwd25vp!n0_7p27)*XqroFTnhC6B7LVM^TrZB0)FZMa#F>faAzs+C#o;@wC zY|DT7p3Ranz0 zi;!Cn@uyF+PF;{dcW1~$QM5XliV!)w8%Bi|jUQ}$RFF;d7G&2Q_2r-V-jggwDxJrr zQ!Kg0U`)K!XEXTJQ*4a%X&T>qicM@eFir1JU#;ccilf3(vb;RvX}6pDiiO9YX2CW3 zAw->I;nPmDcce+Fy!IJrlwDJKk29=&jT))w+cRan&lwiPt@x3O=I~8tV71Je!>^u! zO5Ah~ANCU)Bpsd2H~hq+f>zH)juh8`PA1C5I1*AnGMR2x?KK_R3klchK`7 zjU=b>^nTvwEK8`N&O#|W9#XxH_y=cMiy9wO$S;rdka=gBUyYY2WROVHfXhF_G9NdK zKk+kb*1ZK{h~#~>`+uTnTg)ox=h|X{!kBIuULKD52<$^IGieA2q+Sc;OMYg3du`f- zJ&JHHru>=rywb$EW_nK9fehzyp6}6B|Hja^PSbVZ;zD2WvXkJ>2#yo8CHg2HFEZNB7oPh;h^y z*BE zBp9C4C?OK$?`R6|7SgNE0&HS}y6@wq-4u&6FL7!4Tpm-xx(D5R9zc4zSeqaaK^;`@ zug~)pC9Fy0taX)ud4DE#rr@C4(~)(HyM4sl%d^^^=NC#~K$-JAe^|nLG|0z^ctQ%0 zroL00BB;LS&&?ND3rq@LxWF1(M!yG?^b#67DPi}pOC9HzQ+oQ;=(CwRX9^OP^R;m( zmUz^w$%1oH$-$rtb=?O@Tu)6v;_?DQ^U)7Tn~bz-d&E>IQOid{q32!~SwgKsfjy_u zkK(D;d`}wMMEjNAJ;CiOOkyDKYE!pXOsAi%dDR~3>pFT;F7Uu zY~myy_dAO~TcJdj%<9F7%&YKWT4oK0v>gg8z?(bL%= zDqMDSm8humQ^@n^X};nL7K`%i-ftA!!dovSQAChp()2P0&J29xYV2UFa)l<^Qedrl z;wd}tpfa;m$If3;St}lt&FYW&aT{TC9Es1#A-*Zg6QeK7S*t-kIM($uJaEdZsuY&s zPNlA%x$&NXQ*W$lC!P%IwTu*j@Dxo8CLRqZ z=B{5;5pa`KSGp2Cr6V5n%$B zqJK_%Vm>tWhge3wwbj+i_fP@!C-_{F*44;G_2*Qo21)g43e0lAH5iLnCm-@P?{kf{ zFn0K7^PWuM>DPdPZaw*?Yiw%6_OI(0&@FEIbW`m%ornIxCQRG*+G8RMrx6bN`3ggu z)f1f*JeKDFR##%!_AqA^xb%5ePg?9L1hz=2-!f5ch1a-^``3}8q)9OR){#P_@ss%M zI#RgT14tiktnYb($i1RYO2UZiIELq4XMqtP z;A~eohX(6p3Z^hYTY(^&-s_?25`5AqZcN!OkLLHUV=?;rpFI2qs?Yk254*uSw0Z~9 z*JQos&??o}_cOuq)Hy?tF=jN4@F4#A9S}pqJ$Mt6mYWs=uog&lB~$lHAA*y3Wl@;3}? zl4p^9|3fTKNWbj0Uwg#HNz$S{s|QL~q}7GI3zPnC`?`hbF)bvPa4ijo_&W*+;;=qL zvJX}ToOrJunrt5-Nok(`A@xudcEgcM3KLHHBL0)N6zR8U1QDf>p#4VjCO*>B(wQB6 zk&pD8G~hD7;v9xt^!ikmiNdT90}${Co7FtU@IEstZvy1-3?@TH2r2 z^_4D3(mMW^uXI`pdz&AwDcz8w-s1VSB;Itxn*@1GFtDSx?@QteU+w5Cs4t&gTWTwf zdy~IaTT0=jb)^*P_kMhgEIlE$jpfT^scXoaZwrCqG%6?uldMGRZDIxuT<#cmRF9rz zJYpqWJ?^dLcVwwquQPB#B+iEEncM2>t&}&s@nGPtIe4(XEcSMfD)N{c?m>Or_$2Rm zkF^Zh_m->f)U}VVyL}H=-L2O0+EpKSWKjYz;U1jRCAz|^rQZ~dtGU?WrA>hIE)G-f7d(=)CHx= zaaCDdwh@O#d`mxy-1nRkeMXT7K(~C*H8m^&yE`(=;^O3$1&a0JjxGQ|6;_MI`>2G9 zK}7EFkbKIHF8ZHmq}djdf#cTa%{0*@_BQzPR$xKQ>Lvqvn)OZ)_UNV|Ld6ynR6caR zR{+0u-pkiA0d;{`aqPq;Oz7|iceyHcI$Qs?${QP>;o_to@9t7v=wlx~h}OW28ojJb zpPSu)gq^)B_f?8ep4`^ZYbpkOJd|;>$$G$qYgN6VjvILAWnOI};zPy&NlaiYjV{_P zO${Q%;fl1dLLBcLyFC`L0(XVfkU&QpzI2&aE82LZd#c-VXg)@hVH^~p#L~+YC9@!p zQqPLpMGPJXT_!{vG}B3km=Bn7_BfOfkU?VDmATU^o6XGDA8;)52LjhPxF+_%J)g~3blj0X~JU034z4ljk_T-~BlomtC<{sp*Fq97Fadl%rE0(|s8hJa)E z#{Xcv_bfXK_8Q|rT|%{%=vkE9go+<+@yQBu&iD})S(5>m~iKHW||R`ShcX z!IcWNRFvvQlwM$9MS*@*3tU*QT7iky2j-sRLITmh@{*r0a|Rk0c8L?(hAH!bGzac^ zXO-hu2idy5t)mU9#wq14j*IJR+2#t-U`U5-KZEWSlu^lKi-X7n+22);U&o2mAxhXD zD84DQ$}>ihzc_{eA|>J&lu~)uF(vcQJ|^o~qNCI{+B$Ij<=?~-O7$>#i4{Q=7!>wb zx>h4uziR*&{dLXIhYu^Ifihq5OrsOXwuEG$6mdE~?f#_&c!vT*8`d$EFkA|AMWHp8 zC8WL)jngfZu?j{C9MRHXTeo$f3qGMYiV>#g2aL=jFTH^t?;EwM4k`5Vk}LQ!IS&rZ z-TXaCL6h}{>G%yC<7J|@ckND`t%X&Muc>C1u=IwYa_~4RXx>ikBV^TSk=hbgN}Le) z%6OSQCx;kf@9f7OasmqbS)$T{;_#g|*<{Rogz@$XYSNy9aB(yHPRzTZMYn62ttSMF za3oq!s+ri#nrOXBnk_X;G|<&lG@8mxUxyRVcjKxEFEfK#tW?Q7z|8Wlg1F4mJ_#V= z%+f^=7UhMi5h(F-Wx;h8kS*CtlVyT&$)`jg(RVS!GRb|j}KVQEj_o)o}ZFo+&qdJ9#r@7U$78izon+dNhcW*A>Lc>%ou@DYE&r zZ0H+{;T|g+B-t>~ARC4S%Z8`IW&bhwPYR~{l4O4mG{G8&CYacxL4X)<3Ok$_ZNVh( zb+SVd%Z~n4&Yp@6(iTz(T_f5_S^)px52oO&=64& z;q2iruLEp3ISb2Rh!G~RJ*cn<4LIjSp(3Oh*@bbqmj_kuPCV#eo-Gvtc-zih( zz9c33x}0?j@;>K)V)M#?95%82i8x6$8Cxvnr?J46h>dMHhVrcNh;4v`-f0c#2N9U4bZ#|egs@N2c+5)r|OAPvyi$DOn; zLP(DY`<2K5T!3ioMVrvP)OR~V?@4l=9#d)3Q(fJ!^)IO75R2vNI2l!3dO_y3K(mEJ zleHK8I0gio!%CGMhrieVl)D#9)*nrU*L-x>jz_svsj#k-d7y%d|DBz{WNYdZmmBXR z3WWyp7`xSXG=Z3~5 zFYO8oGx@@NiaZ2PFqy~R5g&-(fvW;2{*YMGFh^gImpq3I+9_Y?ZglouKcUTGFl-LV z0F;T=N3O=i(XD(-c7WpHWs3{nJuyP9P~Uv?!`~p>rXCN+@G!GKac4tpJS|#@PN{Elh~fTtxs&!eejp~@WR-r zvRHXtQG9rjA}8$0I%tv;iX4q9q%I%@Si<0NvMyf#LER@zg;#x`a6-+plz1iDLVV(I z=bulk97d`u?gFH5+^kls^BQRD&_%4N zi^-Y>b|X~Sa~v?DaOij89m(}618tEf7d7lSo}$rLZ;$Q9$d*| zvdx!FnWbew7xbPIuf8A|Zx8(9swn)9@CI%S!h1ahxxHM)(3$T&s<~@r6N>Zb;8SMQP^5=tuIB;~3L!-AVBjTFY4|8}YEq z1}EU04jYcjRVTnj7K0oNqy?r){*bHMTm)6dks#ZPD9mKVm0MK|#Kdz9W?NHFatRbQ zV50Vu7?%~D3L?REv7&I0MG2l5<@_&UFPj*A?Vk*UWG%1(EYHdC7eA zMxq5Y4QD}+r!7Dftgx$<$h&^^a?uU-sT=B3H`J$Ys81{UC=)uZ=0mit4`fU@++=M% zmq=E|`x&&Xehj1-9D%IkKs}_485sUe#Mp)sUwabpRv~i5HV?iccgrvV-)$i^8;QK4@MDe=+%QUT z!zg6?2azLDd=#pwO5}w^HO_ zNeSbTY9vx6B2|KZRZV3vo0&#^OAO3<98Cu{td7BzM**=qixnOlF9625200YcMh+#p z6G{@uzkeX+_^<_x!XybQFAhak=y>7G!X$I_MLDYh6^}_USrKeqw~4j_r`>_k?!-8{ zqcP*U14gedpw%u2itkRS?}DFkT@XH^OE5IVv@SR;`Hq}5$cymXs|zd}G*{_`PdL(d zfuv6BvK`|w!6Uwhk8Fcg3QPVnj2;S21CxQoD3X&&cnPH!;AudhE;+47s__MY(4|xd z0dIML)Y7E@pmsS1P`i{=z@-BW$e|!5!P`ho;dXYJg~|b25gB1~cXoln*(o3+14w+2 z8Vfcm?3_lkJUYia?1B;v&mTE+GY~~HoWOYN9->1qr~D*AG&dZUX#&J@aRgI6O-6BT zVVw{h?SYvdX~osL2J;w{CgwDu_#J`CIr1bYLcdeU83uI)o16t?_aVqkGU2?sPxj!MSEX>a+142|XIgw%OEY){?~BT0#X)%pC~FF93`oXz8o> z9$x$E-A8YF|Cy{eX{iZN1e8*f18B^~2jt<`98iGY5dp{WJ1(FEvEu^52^9gs_%#O@ z@aq*|jy@!3ZAN#PGIs{J`mQm7oVzN()pwThP-OPynLyMDJsK}~!^D2n7Ey2867$@u z21b2-aM(OC5d?`gpcylUv|H!ar>z@lIAjVXbG7L31WaoO`T`4qVrqQ?sGxZ@6iqn4 zpEAql+W_8&bbi& zLL6AVnPF%aC7GcGnsZH5FWlxarT;mhhrer1k9rGH??Kc{ym|oPvl6ei;FFj&aUgIa ze{dxK8m7BHgI?Yyx-qi@rn_g{s)?DwfLol0`wivnKg9SbOcH!|RD=^1E76ByZ0ZF1L1?vvi z)t3;WuEq&9Aq0=S0kIGQxxY zj?owS2j>ofwHZ8Z!zP3)xf|}`fj*Bo5e0G*B)&)9hUwadjfjZrfwyI)loBpjviSi1 z;f^LKdJ#&D&pBZh6m7YPX-Pms*C3iKsumNtC-GW|(oO6h?8G!6Vi7^aJ}`GdL!x+p zbFP9p&=`7eByVdU7X_1_AbELP^SG$9Abw0kkHU6z4^e;vd&EVn#R+v59VRlFn_;*> z3M4E@%#FkAStLFlCvU48*8{@xAik)S`(%Bh@mtOIB>Ptx$@*|xN)&?uN4gQIM*3iT zCxb%$N6nJ`ErXL$76~$xxwJ&2uBI5gb4;XOQy?C%<%!4Ajr6Fhf1-d=Jq`7w?7zk! z9`6LxLnstWiJM?mc{~l6+>sbC!^nK}R23?zu;(PvVeHmhShMk+Gfqs#tp~(h+}0}t z2Jb%RsTXK2jt*>VvL069E)AO9&%@VxbxeT9NvqbtblON*UGNwzX6b1IbZQzzQ&zY+ z<54r}08P7LkBmBo8F_4YY1DB{$YaBA@?)vqO|fWWd~bwnaL;Z82gM-iR{3c8ik=#gfhP)`bFQpZ%DgAKgW{yFxW^(7mVy3OVpMyz)pnPpdv(htl} zn}u9!MXoh3*bg+6R!Gv=>3nQs>8dny8XwR^YA5xZ##5U}nNt4q_IpjFPduf1&+!kM zOS7bnDZD|j^ry7>SzfD!baTK^A25u5SZHn7LiqI$?FT3RNc4re%oj0X!c8cjYE#=# zQu60t6)ch#^+nQfN;=M$_iZVSthN3b-2imhi@)1a>fm?!+4A;fP2y);N>>`cOi8BQ znuUwlP18=!QgX*1gUQHgob(L;xs~*x9UHejHKKFz~gOO6Ji zNMf4yS26HDWC}}E_cKvAg z?;%ZK(r4H~-c$OsCQgyse}AU&&lC3Py(Db!eeMUlJw}?#q#MV1r#{jd$#~peGgkVP zu^;%>IO&?S-q-$qUn$)ap1qCwOQWQi{`T4ZrMD#GM>jmNWuT64_Ca~o9&6%{*i`-& zvj>}`;Y^C|%hLx+iPF_L`@VrF)N|bM=?5>im!}SsT1uyPaz0GzEPcI`e>Y6( z6_ky`S!I8mUF+-mGg&j&`hn4fdcU-jhYXjdN?%0r*N01~QjaLR*9hrfo^`!0czS@J zcZE9c?fjE}J6hW7H@t+P>I$zUznOp9bH_* z)w9e%n9GUjlOvSMyC0<@p6#OaNg|KlvZqk#yS+hLJ8a-r#2!X3~`gZk}9`vd^(RI~dKxVYO<*thwa9B^aR8)thgd){( zSO5@@5OGIPKITlg9z##JO%!(&HRPyY9TW+>xQkQ$^)zRZM$hD-2h;@Q;;tHXA_90^ zinKu**V=wDMXK#xYj-eEXY-ySi#`7Vkwp@HW6%Hiy#kTl^ax9&*&WedL>SlMn|i3% zJ^@C8_{5&}BK4_a3h{e`GKj9j;Q#KJfwQ)d8g`ubmqc!y>MBSR;-@K zr%=6k(R2w6t{IP=AvI*1_?Q{eV)ijVJ43q9KH@*VAhoDvDFfTH9#uOPa_0+D=UVT( zzHb%qo-?Ic0msF+;CsIph_;P>$al|_f-UPQ#u z0B4He~I79qw4t^!ZtGex$K zn=Kt?Z9gIz6g?mqhq~oX#M09|xNANaNHlcDZ2eOxTsYscF%H&{+W5WGgycC!hSNBx z5yy6~FsW4&{&}ht)X{`Ix~@R+>cwv$4`}MI32ElrAvy48TdJx_Bg zD#zLSA_7EVFlV{XRzIRAf;_&7~Km_I^&pD!BoL7@|#2@*XcrnQR@;e^DAV4wVaC>^!ZP_PPk`2jI&a@FC;BE|4&vq9`$Mw|6q~SuFhFFQtAlbQpC?Ll0xeY zaVJwVig;ZMLV8mDf^(aTcwdXuCc2k97xwzY53LI@bf2=pdM?Jl7q&z=jPcfm1r!~C zo-H6|I^!~b*CK_~*;Zb~uKoPDMGB5-ze~r|1hOIMc$c&4()7U=Ik(KyLo1~Sx`g1q zu**bFY3f3|n!KNP%#dQ58NUZ9*!((|(n%}cUH-xVw2l1#`1%g8s*dJw&h9ya0)oOt zP?{)KR4k}~sHj9xRO}_T*rL%`BG?fWP!X?U9eb}yRIJAiDuONc-bB=wLgxM!vWSv^_k56V2;@@1k3t>z6duv_tWThA`3111E7(~XK{R{ z%}x7aLhX^?(&1ubTpBe>V!`Z45p_b65YfgTp=Q$|xkMeIG1GxcXWo!Hok3{3K`*8= zH@g{d>Ut<%w7c{=a!z5IG@ZVruu^t|Ut5~54c$xLGgvLVnjjxD=KAlYo-AItdORhYL663)2*7bgW+?T+5IOF$YO)%NV6D>Hy(tsKX!SyulJ}rVgJyG{ne;vC-*t+sNGQe*6}btdcU5HL)A? z2h!>k9;{OPnQWZh9+5^zuU)(8`b<{AuI<5s@)CBF?JQQ$t`S56sjF^5&P)Ak}- z5T&+uJ3X1hLhULFGSNq~^LDB?mksguSPzJ(H%Bl##<`V z8gHZ4^Vo8`1vv#}FG!=0^H^3mtAXnkA*4(nI zTS*m=_1_@tw~{s=G}(N2L7Gup$#ntqvWr;*h(1Oww^GvutfJlav4H3!>$;UjEnx8t zDtv>?*rHQ#VjI}?&F!s{0nIMX0y(*k#WDerEkt;FipU4oS6el6W><{!ZrNd2*h|Kx4Qk43MzT?CoNgXs#cwBDu5h-Uin|tATn$hb?NxL z>`!`y40aO*1S^V_HsDXHw1`y=bSr2NR#?psrG%;yJek&Sg@;$d!h~5+tIE$uLgN^) zVifXpEP3;^Xt8!piKZ`R4aW~eCF)M$wNfK*U}^1LKCunYL*;>L&aDqs_$rC zr%+9DcJ5}B+Obdp+7x6#*+LFwUoKkzq}EQMHq8E7Y#KATu!reSTuSMBK7V$BwaA~LpPjzpsm@;}=`=glGCBfITwW;%|-b!2o8S*Ff zSjpVj`eW*tm8`F1ef~Tu>pn+)u!?PDc5?<748+Hkv|ugsDZOLk_xkG>nvDZ1>B3s} z%-?uMwEY22gtXVNg@asP>50Bv`JK|2tND1#1k2U&LsbQq^fJR+Xi;a`p#N>Z0GVyYgC_g}$4n z=&f!784CN{u>hIXF=_&J-pm@NzJqFQ!HTiuCt97NqlwCg?p6@bx98G?eaNiy7<}zS!0bPbyFI}hMQLz_o*vgJD zmoqdtjg?_8%V=5}hV{>-bS#bi!0MkS`)#Z`EBb^&wn0JhX*iTFgABNw*L$b|~cpZXm)ERBpXd1hREwq~@7DSzqkC;Iod$CkjnMcj{ zvIbs7VXO=v1IKbMq8r~{1Ht+smfd-Rf9?Mom(LSXs&LlMq#P zzE(U%jo-)0NzTWG?D;Q3yqZkven?mqJLrVi2?_p1h;hj(B&-mLl}n}82Z7znG)g|m zR2DXinjFGtJ5HsEhaea~99PhdV^iqFA#`KnOnP&OH7?V2ilrO#P;&wA_CgoDW}3TD zX$lQUXB+K`M-_A-I*CeWFh8G)#)2+1NYZg~LU;-nLTd+HDTp+Nco}J62J>fc6KQb< z)2yGRfUuNAN=Fjz zHln_}=!IsUPqw7iSOU#B!aTh;qGX*wg@A2>wV^SfS1W+@#rN1`I&p-(vzs@qAY0}n zI(d|NuxZJZa}>hC@JZyE$udg?wbaLa5E3ThxFQUaYqmB&o<3!=du-Gcx_Jx|#O$dk zMo8{jN*wVl*3IE8p7q%$jivW+Blp`LLxEwf+uIh z(wS`LR`GWM)YrZxv0qKehvAXpghNsjB(n@_uSjD+-y|x29246tmRcNVes&iMb~Bn? zEKNSnI@;X^(>7!59~0=|aaP-|NOM5+-Vd2TJ|~!J{cJMEd&_vbcY>9#G!6a@U|bGz zOKc=WfCIX9v3c+IIjkUSAGjDe$^5K`{0@$GcM$bD34z%$fl^O0R~EfQ{rx0cEZL`R z6jXK>*Hg4#2dQmOvmkCaX`K#(G+%K5(OEXAWQSFdKLt@)lK$CWEqRVDV#Us{6oh+5 zE817xb%C|z#U8B{fGo|k8#!KKu5oYiEe7^;SDl3a(m%7cXZVbiKEW^^*N4u4womK% z&T5xLgi*P?)FSYrLUBI?OG%4fddx);W1fkE7=OfIk$D!QFEYgzqei206zQfoIMEGE;^#am`@fT+#_wlw5OPxbQ^mdx4l5j6D%&O^Ko zsh@7Jo{YU8N?mWW4gMX6a;uQ`QBcPwsvy!(2UQqh2ZRW46CJ#=5ZHGp`QKrN=Jp~w zvV+L=TcPN8zXIG`2aD$%3JSX|z>#`k;w7E}@S(jl;|{pRUn|tzci0ui;uC1*eO8aP zRVn*E`xy6bCeUF_+b^c=(zN-)APXZ`VY7MU5QUK|GfRw|9%o*myG5Bt9?@np*RPI6 zV(ik|=`mxGLyrM<=y!+?YYWUVkC6vrjecSFh{zE)(HvnCb?K>v7uW0B&t~v69V`^F z$ANzB0IYc-N4$cd6=|Ekc^Rd9%Hy8%s-sc3l(0c<~t4O`3ACSMGyOurhQKUyX^1r()D&J&|xghdb*n|aJ zW+CSL8MFg6_?wkt*@M)Mf3q;jW;)zG_Q?6O53T-(ePpW#s!JcS3NU(}pk_X1_hm*a zXv0(RA^(;1{wa%M@0U~fGnUEf&7$JZahtLDEUNb$@B=d``8jyvvYE8;IpVJxY3K_U z%H|nq+Y2_z)7=Q2zA85vko<$V{q!~Yc*4h`qsk;_UZ&}I zQeU&GUcUd%_4)@r*HwU#>-0jp@tR#=jz7?rH|z{+97Yr0LVER>MwQ+{niw{fuDoMg z%LF&obM5rebCsA6%&1~9YN1Jtdr|s(wyaLSCfJnq1lGmFSkNJ{p|Hm`2DAng?Q|%s z=Pf@Ed8>V&S#$QSAJ{S$9ZIb~GS{N+Qv_L6OYo*)A6YsZHHZ8^u^>lREE@o4d+O!* z&7pChSbnkMc<2bwex95Te_=kfDv!mpgj6b*&kj2FTfy;qw2oQeBMTLbIoskXWT)`$C7u3LIJ~DIX;qt&df{u5%#rQA9T%vV+=w94{4~vTNQ4G>d30v;Yh6Pd$Ti(;T z6r9OG_i8c}*8}i_uf)?TTV9dfildXZyoFOLJjZ~~!U-b+f(X^JRMCz%E_!<=x@Yud z_B7m%A2f`fC|e0X5-&^_MS&M_UtFF^P3(E7ht&u@<$GJbZ%#;<>-TM|Vxby)Ze;a_ zQ|scqwo|!SOQTBbfe%^>FPc}Jj|*NfL2m-~@KD$5iguIlWWfHbXCG=#XEO9hK&Y1I zds2i0AL8Q|DNlLMN; z&ZUeUtIpqyl_}DWm1CQO)%zv6BJr#-vi{TnOAMl7&ODHf^`$Up9$K_M)N`n=o-fUE z=58h2#HT~EySb72Di@;AUT5x6e0`|7SK;VOm7nNuXI{PdFHpFP{`CgzeIN2R@LHbl z`eIgc#fbd-k%Sz;_c_pP124n&_9df%d$H+#X^VjedA{w6jj^pjmw)H+7I}*4d5VOJ zu)R5~^+DK950~y(4)xs>lKj8(?#-iE7haPc>P2ZTyj<}>xD*g=7484E7u|Q^rHYrY z@EumEmuSBSl`F+PJnV4Y4dgZub8+-P`r!S_Q zM%XFfLSU{fJWda%+|s+ZB1@%G^mGpm|foH~1f|KicDcrPI5@t(Bbi-)qr zp7hm=S7FiRsfsuEEcFS`#1zOtVz-D3cG&g0Q7>&p2QjQOX^h`HuRE}3P)GZ77R=iJn z74s93(hyY?`;o?%<8I8REG;O<2ePoP3(E89!jNW0c{Dq<3+*b8$%*Ph@5Fb-F68OU zC$SM-Xq7K7&vLp@wl81b?#;L6qs&E}xsVl&0do+@NyIEC3>XVw?r#W; z6M;bp3=@Iga43*)&mr{i3)rI=;&Ldx0&i+-5K)je14?!#`-;4O-{qY^A_BX#l_x+O z|5E=R1v{zL7Pw-zAAK#Xm0dO0;<}J76^?+cF9D2M%|v;q&k3<+onBy3x?GWmuwOfo zb0uEIu?Y0O5R68@Mh}ev^*gDpEAb{01G&tp%-a?1jl~e~MIGo_Wl(tywlWhMO3+sxl8a#E{c_Hlc>(Cim6&;Ett^t1|Q_gq_w zsmd3z?e^s82h^$Aj@tV{u94bNq95PlS`z!jT;~E^VS!NCpe?nih6&yN3+=4Ne`7tO zs8V$f&v|XAd3DTWbr%|5o%dzKThrz0Jc;GBqV6>S&uB$SH8A7xt>~Ee?%ax=*8uUq z`h`0B^I&G@OmqCXzh^gRj9&sA&cg$HbDP(~9^WQQF3*MIjE2s1&mTfod1uP=M_Jvw zQ&3G_C9ZyVP`yxZ!uZ#8PIy7BjEf<6LAZj=EG{YNsJTr9NPF=0vu+n4eA;U_2}tY1g&S`RpI% zxoWpcZ0L{VUyJ(&WrI+mq?k-EMPw`c5nDcV(6;H4v7smsf;zK)b%m;c0~I#SPid@)e8ZA$~|^C0%R1FfwO&XU!EuGQzhaTOa242JP`mRA15 zXEAQo0KvcuNJ%rKuQ`o4by|Cx>j@*2_udKbuV7JuGaSi-4xE%8dpgDAQa#CKP_ho`y%KR6`ec*>Mp1$X7c zlSjk${HxZ>7W@Rxec(I{F4%Ap&mQl&91iAX+4i>dcQ9|&Y%rLK1xKuwdQ$UOw#4~kHO5KL=86#Wze#%~=(vqCUk7u(R$5aj#44ZRKFRatBsD%+4( zjf-kixMexe9E<5W5+sKDv3BH@K3i3TbuOjtFn`w3KObv#%%8D(Vw0x4{%p#bh32Dx zwt(e2ArLMCWfAC%fO%SlPo3+?nmYj33k;2Q_$}Qa@3~%KZW|;oQBZ)52#Dz~RVd+C zZILLT0>D-Piygo;0WbjYBCv2pBLMz}2sRq44Mgap4SsJygg8uq_PiE=t+mbtKmofU zU_n~b!mum>tDw~=4BIYXCABhzVaov%m{s>VFiS#=?kcH0To!c~qb-Celc8uUP)Sn7 zjOdX9BlisWHQKxXZ@&!-Pv%)oZGJ=Anj)>(!k7$qBI#u#?(Hsr2W^WK*)yVpBa>lr zV1Fc#{2Fu5e(g}DS(p(ywZ8bwbtnW7@dXxBT1-c#4_%LEpM|)BCgY`XTsafN83EY0 zEu8KK9nKBoY)rl%ThNxqd~^jB3IF|aTQ@ZNI(j9ZxPWUI>?jjjQp-?YkKJfViJ|;> z$*89ocCkx0ufw5FKdU{Pa0lk{A^Drmk>K zZ&_Gz?^DPmu`40uMK-4CVceH>Y)sq2xVLmwy%5IxO41do+nisLE|YCIH;JDJzKm(z z>Gue}sl+8wl${zbeKOsTObOWDg!;7r=ltA+rnlhrJcFNL2)|uoGe4J>VvUs?#$z=? z>23>B7P@C4gjQtcC z`r}A;4`#b*JsRDbS9YpY4<#6536WA@SYe{hN-e>x=H7{%9 za2F$XGhRI^#ShE2wi{gH+xgoOfNw4>(65M*hk%PlF`^zD8KZ<*{uO`+93mxm0 zkAL1a({Wa4LL(`rS_DWFo^>3pvaGG5y-bD}ohg3M*4@%4{$;qyHbyrdNW{UGnD`mD zs9js$jnxREEp2(_h}&T4K#iF3;VA*v5zY=1H-5z6JT)0?07wqgl5lt%1VKBc$J3;n zF?Er0>wi<;2_o-y+}Yn15yJjKXiAdr*31x{2^kDvv>mve$?y^F1Ip^h#U{haKzb$C*Hfh0zQZKf2Y2;p!EMz-#tP7f}LGO zKv19cRl3@VC&=s?^dW~UF;55YtAfc|E@?PjrIgNmc==7$F?3IFh@tBTos4f9h{!71 zqhEMR17Dq@3{z{9R~H`Uwh$;&AR(JyczoIzqkZ0&V!QAk*pnbS)&=5YMlE{Og?HgY zKYymsuG~<=1)5-VtfDWrO~Tin_DK$}S8E-N%rpPdUjH|#MDJ=ZtpN3_&i)UzA+uQf#<1}rrJwJQC2Iequ7d`3*ged!rs{P0#S?!uM{zvX-uQ-XN zJ8UI#bt^|Zf8?S5ou7(rjkxhsT-fdk^j%uRqQ$y~O1>XwV`7y>cq6ep_YR&~3GFe) zPSdJi1jE>S^~?9y&Kg`W3m=7V5&5G;^iJ!2{_H8=G_?=+=@XB1 z`hFMY$KlC6&P@xumKAm#GZCQRIYpGQM21klvSFrkee@d)c^Ly|;hY)_Z$iFT7Q}ZiS~~ zM=lNR$Lq7(ZnUu<53GKwqA0dUT4Zu8{Fn8}K*Q^qv%;zb?ZS(kxUy|FW=MZR4*j`T z(IIf2g>(Htiy{7{D9w_0PXh@&M=*)KYYM0JHb&55!$#Ed|U#BgIEp}y5*#(H5X1-0f$7E<$UhlaUoh7(|`Mp=u*#y&A zhp8*Ym>QKQDkc8RAA7WY{>AkFzA0O-9FXMb0IJ_yDqqGCNd!-F*li73ej+%I+1R~e zh%k2drKbaV9ahd+^%=y+OU(2@P|T0+7ym;r+dpxOLGi{+av97`Zq{XGyr*J{2@p6F zp5j5kY@9H?^j516;a8Znos?mGi{IE10;Rud_cPE@qoJMtDK3bfGXpAr1EgoDcZcyn zE^Q&Vk$j!BnbJq{!}ysL&1W_VvlZpM*VgU(44u$B6xC9PJl96<|14M@Ab}RhPk`X_ zWHE>gLJ1=B{ko6Z#_)!vZ@XY{MZ4?aRL>h9k+5wuFBbc1R)n)i{)O>x!-$02Dc zHisrdBPa1vgFhT5c}FCiVUX=QxzOw}-22BSNN5#PE0LasY}oa~c$1?z!07u7msP*4(|$mUI_eM#J}WW!vsR39{(c63bDIvt^pNqh=B zlT7hb_z#AjPYT7)4@Vock%#H#6fmlbS@d-Z*3F1R6gHKY^?&{uoJweazG^KuioS;o z&Jz5^J}gIPVauP1qB^FU4|DeXK?|nx>g-HO%9;xN+Iqli6r5y0v&&!h6<9y;t}+^MK7iSRXcqm$7Jj?m2|wx z%FB5-#)=qz{+>8^s9rN;sv_eXj~3zfkul(-EL&084&2%4EfcIon}6!FF!w!?rCg9{ zM>20--Uksv@)P2OkYF9AsW5f`qDv?TE2YVh#>jIzKf>G-=)rWT(?>~EKLv7SxAzpC z!h?O^0ReQ$A__7cES#wo9)Q<9z@kgE z(#T7dDjfSQ&zO01#>neP^N7!cHtNuOsx*@a+BO9R=mQw^g@(-Jt!#63{K1Gm_)M8I zc^x}*( z&thJA*oAe%%eZ;8VKz3#d%)-QS=jnX{V%UE!p-1&{FUszh)`FW||DM z-{bZG_KLnAC}KX2vG9!T%lTOJ(ll~ez;D@Z#~|t@ZF);z7I0VFSpcNOB4^!q1X%!Hmm7PboS8`vM)q-KA?1y>)KRayjV=|n{CD&EFe3OCyBW@n*4WD$>QKX0X z*8`dXbY6>@QE0~^6c_OEr1XAD6IVeBc6&;Ht>P8y4m64b)7X1fG2RerweV!+jwmhJ z%nd+>;&*YR>4 z?F$mf2p4`ZHbLKvoCjW_}{x7g-7X?+DMBDy^JePil9{&ar}0god{8Zmq}-a|`QEP&aOa zZm#6h9~st%#*PCVhT=`*X#1x>suy;^WQQ%iOvQHd+HBYrYP6fjGVKx_+Kp|F;}yEH zn^$&RcL^n>Kn0l+vk;s5%jBS9TQKAj1*!bFYsstJN@uoM?~Yrz!3#|uL#6k?wqW!n zs<($nvriXj^Bx}N>2u{<>LlU(&5|LzRdS|Qd^9=k1)EB^NLBaZ{P^H<+O`)0$b{we zdM~fU=3F56eS8jke~!}jL3MTM99`JQ?|7BR9t2wErg%9id_Sg5{ELq+ZDtKu&(hib z;O^Vc(%bz!u2tpVFj!^a<|tXOAN7Lkh?WSAza1il<{3Uu4laZsX%!6*xCdA2BIU<3 z;0gxBiQ}D=7;qhM$&3R$$nAH;AVXt)<#a&?A&r;}bI*{|L5L%1XQ}Q%?p3S}k_ifq z8&16rVkm52Lfs@QF#t~-nV>@a0czl{#YZ=)38y+CcA0eKARo<+wxQrdybWt|lx7@4 zRd@api)l7`8snVX&_X)KkQYbj?jc@2EO4p5!;MJ3DpFh$56@K7$;e92mxUELy4ILM zkP;rFa-EkhK#Tx)#m+d_-DDVbn!?k0IonXc^{oDr@8U~1Tw)70e%Z^gD>)7 zQgU#hBBY<5hz-TnK}ag%4=l;D>BKSUHx2u#W)=@Fu0c%;?yavg@*s6w7N07yAAcr( z9E)}RRH}QNSF2GpRnVf?Wr4Lx*u@K9ASOetOtewT75lIF*aKEEc%};k?0l5w9OwI4 z=OfhS1dI?8j?&E&Vl#S#q?0&`ZZ?yuoaBMQf&GAa^|aN43hhRH0fRg`5q>bVy@L>6 zswUP&ppd>!JUk3VAnD#db0JzijKBSz2 zRlubzYIcf;Gs>cwr+8D=Ba1Gc!v48l7P*{8JkO$zr@{J;AEPCw!OgDVitlOI9uj2V zzmR_9G3xafzE5V-M)5r>lb-#B1G`@`sq7h0dACfp;TgV4V)c&D{j)%W4~MDxIZS`n zVfy79GA=$$JH+>(!}Q`DDyn{%0?wnN28U_zc}PcZGw9%X#9z;#PvW~`2DQ9^?|11m z^#Uq7mrnaHprVX)_1XnkFxqd&G^X2N+rF>D)994C{%ztJ6Wp;{j)ydf^ z+8(nKr4zPn26xdJga8W)HW{AnpuJc5p%MXz&RL0?!c)fLp)JoHH0>I9FKM+y6!u+D z`*ORw`x@`bTBfGywb*i38r;_S$3^=S@Ill*N~{?8V? zTA_pg+kzV_=yWcMPhUZgb0Hvju282v<&spU%yN!%wP%qjyY68?91?w>2(F5*u;sM& z86W1}wF$ONNg1FjXz7oO-Gtb52xY;r#%hh{oJnl_GOGOo2vB($CB48N@z@f&{{p(t zMvbY)OYZ8}5jV4Pvh@Q35PzE`)cK{*H8rByFL`PES0RY)i&&H4`C@g?OFoz}w-x03 z25I`Opk8mV5bW7PiEn`O)3?ykH$cI@TjOuL}b9Fdt+J)YMb~u!9sd zB?+SSg|OI%oyV)M|%HRyn_l)MG;^;3IF%Du+CgeNCQIyb0!p@fl1IhHbwF7qWJt^ zb4*Sbd=+lNYZ#X1j?r!g)5T9bpwv>R<-knRtQ?YVV|H|FP;FK)VdLV;VuC3!4+a-u zv+1WiY>KM|)1*A!-0fDXjw)i|D@-ZJQt7_P_D3q^=V2^|rBc0oUZr+Zkvt`73|go^ z=~klP1MaB7()C#EtUJ_#{OlEkKc$1zdHLK=acD3YYB967rMueM8uQP}kJK;j! zFz9e!V5o*Kpjs*x>kO?6pcs5~Tu4b9g+Hm3*ff)2PZB*>WPj^J6;Vra z4e~7_|IA#IXl4=F!@8ybey&d2i^#pKPgTIjg6iaIEe~V=L{YM}T-9w)6qI8rLmkCC zb&=v#BHZzLU^3iKq`$0XAGRxjURldt#pWgGtF4ylOQmh(3rtH;pWDc?#2!pm-xif| zUD0I{RkD?vv&-Z0zLV_4>Q_;x*vf;rfN5cE}MaN}>_ za2>qa4DRv`+*=1P7^ONIm5Rag$AOoR+8db(gnFl0r_Nc+)~r zfAW-N*}V~LZiOG*goEbFaWH<+T)FQcn&cz9GTTA4#7FjGPX?+7eB^)dG}i!H<0}th z3;R>?3UXuCsXs+kke9RF{pe){c{6k9udb*lSCpiDHM^4BOR|675AA8Jw`WNowRRPG zO%Y-Kcg&a+%A7q(+Au1%B^2w#9{A1aX1yF6<*L+jXo_i*XeSzqq zX8$Fh=G2rgx;N^o520Y^zI_FuAx6}py|#39yp3juE3CRuxWp293X$4^ zynNLySgs(mzg|&jBe^M?_DY@7Nd6*856Qa;-pX1SDo>Rjs0c4q*wd!8uem(dZA4R4 zkaF-G23P2aLRuL|SAv$jTT_Y*M;V5u>g;fNwZuk*Qa}s2Dr*)>eOkyhm{X`aw}rfm zv&;u-i`Mc5&ZY&DXFJUAkU%x4o&2-peG12omi+>jFNY-YN|Vm!b=GqG1|4oM$FQOG zsD1}I(qFqN6z2EDNfQ)aev!BY_C0a_bs%UDaA1gz*n5}^uj|sz4su0yqAuO;AbYY+ zb?Hk7Iib8b2i3VW%J8wQlh3sx$eueG=!oqHt4nJ;%3;+K%qjPX+NwhJRZ!V+ZPB%S zFulm+U-#se)P?2L0CMRhw_wczs81(3s?6&;Vp4J=jr;GRNx~*Z5Cl#{uGgWzI?3l- zH>?1Tnv7x~r1lYE<1MuR5c&^Bs_!4fF%b3{4vG3@C-YVciVZ*o!hOArtd@7J_tmDS zo#omrsW!n}*-&MuCC&1JG+5BP2^e}OWP%DKbnMV3UZ&n%WIyIwn`U;AOR;*FX+sxz zvF$~1+$O|}$+ao6t6ZKns7=3hmFw9KMud)5rEAm9t{B_%wQzJR``OwSMC_|Y&fO3( zq85d8lWW?->d;(N$67SD8<4}M7On3lHEY6O+;;hxsZQxF zAD5*JwM~E7R+09rKmH;oNNi@NdT+2CCP{m!!msjl|F<5fAVC*h&FjfP5RS=k+YA;f zvl$$(gR?y7-ml;}Tg}k}buK4l)!_)43=2J|_)yt}#d}bNp>lN=<)L;ODmz19PuFcb zbgB`D-WEjus<|Dcq!F0ZH*U0Jgj|>HcT;mm$n_;Q$(4LZ%C&q92L=8M{dSC5q!Q!m zzaQAQ1m&0v)m&-pNV%qeUa9}tkB4UPfBSK|l->`vs}#K&Dd*NYZVjBic^ETS8AJL6 zU!e1IW}#aAXj=~=&fZK?nDErSR6Is@XX9MRFGluv3wObkmd-WgI_bLjo6&H}jxAwL z7mA6I>$!e3aI3Jy_c2Ao#bM-c;<*v5Tc!6fHO@>xxz;H8QjLFcz znNyUbQkr==)6~)OO50~>uHL|<&QxoR+_9>M6Z#*I(>u$=Jkt1Xonpg{2d}+A{}XkC zB=|eoMcZ*>MRwX<(59NZ3bm>HE*dvhj+guFDu|BXStxqsPO3UiuEN@spw8pu2CPI0 zf)=a{rHzwYvRC`*%Q(4{?P0V73S!zthh!k->*?!16dN4^YXTPE-6a!{ZgLPDLvfSUP8bG;2 zTP1%+?}VtT_1j6)C(9vD9dRTUasIK;mtap1HREX;^~PkGOAa;CFg8FqwCugM{danr zAeY6Z#P5W>&`h!`O@>K2_T1IBZOX@UrvBq(U)IK&Iwy+$T3h;?D0{H0Md)lIP<}-b zvQC2Rw^|KKlBX&xS5l8om#0gtmzCOPh8&|XElnC-=T=@xSw;s}Dk}LZj zekcyG%ziqK5k}f*$O1?|<$ABZ0Yoc)23mdDx!CcyW&_HX^VR1h_mgaoyb}Om>4-Pf zW0BmTk}WO?XhRnlLV{QGbR^J+;@K;4lyz70m`8^f$(JM}Em$m%mS(EOm&kFFl%mdD zDj$*H?yBu_xrdZYTbIipq-1r&3b~XlB~tntNK=Wb-&%Q;BqgY8*2!hL6i>%C%9o{h zb;%}qjVw)|MqA{`Qk;5Z3-Cc2uNL1Xk1r-)99e)&uYN0pOn?7I`}WIK-P6zN4TF2) zN>9N67kw+?6#uLVZri!OZ#ytvDe^F}Uh`bwaMs#vi7tCqbq4 z57W?-m`=CDs_~?}T9*2${%2)pN$N*o=j82p!29Dl`A__;J1@7B`luhz%c~SNva7n} zsyx`b-1`~Kypa)y!pUv*DSYr921S2cs(e>oZrf_BWx77Y!ObGshu+?mOIKR(2j=Wk z`$BUzq4oDU8-rLeXB%6ob??bnia5MX{8rlwEg?cJ^H_#5`{pY3;uC>!ZC0wip331e zQ5Clhhxrl`M&+B&g+zDruZ052ed?N(J8$L-po~ zKsrG@($)Z?9qK~f_KIt<=s0~qwWZ!_6MLnY+#X)w zp!8td#?w0orEdAL<3-JdSolCQ;5!TNJRXj+l`<@7JPmPFzOdkNG`NIviS-|=hL%)X zNX%^v8J(1J?r%qn0SJj12)u~**2NEFNSyfk&6`d;DSqtF(Uj+;)MlecQw?V{MjB1) zoRtJNVU)R%Ka8T@2Efn7s8FYA z>m<0+|4A9W2q;y-g}QqwT_pqU_EIWK2KAnoVlP=w#R-Pi!byGWt^CHMlIqXpl)1%# zF9kRi4~+`Y?;gZ*O&e)NRjVsiS@oXOrMl9)u3s0?@DJJ$3AoT)v9NU^#hoANqS zCz>_7hr64TzWSO+&#No#S$=o5X${3&vL5>-&!ja|=of$G1#8)jeyoYPi0P`7Vkxw|V=gJoQ-}Tt>Hj z)=5WFnDKQmQ+Z(S{0MK`+DD$A~Y=A+z(i(#j_@U6C9C-D8Q1&2J42t$Lk3KX|y0f;e zsbh%JxP*g4}}B9G=bQkt{lEpdxnadB;_ckjPLxwxg^A*DSL`(GAv@vLr7 zsx5hzN6i{5aL3Y;);3mx9J5-8&YQINPtlIUZ}ZgWjg=8aSy(9j)Iwq6!;r!OeI<>&UWepB%k~Qe`24|`8SF)g|$>7ZCVPj%ac4>)KaO){%%A^S}H4e z(?*O!pTh&em`J6rcw=I;evn%65!#Th1pJyt-Vh{iHdizb~ffg7`KV~B`WFm zW8aHT=o{XzzpMU9$}>rBw9!&si5?pYRTqU1v*=qxrqNTC2J985T~n2Xvcm>*2Cw4l z3#Ip7PZOsprQ962pp)^gx*!LuGK(PhkN-#7|ypwc?2HrP7Ub+=$ zW`Qj&Uj2npTOKUtd*m!lBN(dIDeCzaWv6(YeM2uu!3FNe)ODfKz}^c6H5PMiccRq` zl>@Bx0qVU-ajlcNFHhV*G=}8np-=F4^f%h8Tk)S3sxwlZA#w}WBmQ zBmTwX2{i1y-|jWOghz?#hg%oqfMdns52{+LUA zwc_t|A2Feagl$&3_ydcwC-lQ=rHbRF9CN=qfdQHf%X27xwc^D>p3o{}W4Rh#Tn$;Y zmqyLkC{?TGf%W3}Q|v;ZiXJ0=lJWmN)Zai#T?=+kAFi?*En9;Eu0E!-Ym^b?`vL8- zPQcm`fe4{r760O4gK%7^Zc%lU$JB2v#Mx%=X!csH@1<|k>9xufkA1hqkS298kCeM6 z25gsD^-PB5k7(FBXn)E)qLg(?g|4;V;$lJ&T!~d z8opjR$;SOd4K_gR8Tv>d)D(TQ1Y@V&%4YOB-5dA)paK4eqBkgwSk6P*wLvKx{9Kf( zZ_UJjLwX&mT{n~w7zQi+X9sAD_OHIk%= zwxop2_=oyzRI0|c#;0x_20e(ap!}31$F~@}WThHnz$B8a@cG6ePO-;#c(zg+KjsS- z9%alBSNw>7!eHyW-4Sk_<$^JViP{*?roCI0%1%daB6CXY#_*I~0=Z0vSGUx6TNP)y{=@6~P@cvQOqS%B zb{L*h_z(s+hpvgcD?C8OH<-gq9B1iOL!*6-LU$_OMN`bVqp#C1JC)LLm9B|u!d3!@ zMdDV0Ut<*QBJR4NTg{ME97?pY^etl0m2Z6;DDoM1eX9U?W-{D_hmZ&pEVKej=(Q{p zgb-5?{5bRiFk63ReY{}U-|{&K1T`7fAR~Hl?V4J0m%?Q>>w;QpuMz{z+j({7KBb<- z?w+OW{n$QCI!nF>uzhHR)iU=VXu!pRlgZ$JmPQ?bByA0g*#pWfHu0?Le^8mkq*3bj zbmfC&JNPfuAXs1NQ?&Ciq`Xrn>EU4|+&1U5C1T-8s(VE7_ULpHNA{(0#+_PV8i^}- z?{NYP%hZ!J@`&=3g&n7#jw&uJ-~^353i&7%HkwD37OdkP`gl}n#yo*PnMyD#e}~3r z;``}sIxN0lwxq9_5Pz!QrjTP`CJwjhmt%@Iv%0O$IHs(Y>>p)=5N&|hFkd@H!P$sw zdyINyE3Qs8GSN=`Z7H$0hpgzwl;sjLskGsu za?Y>}hUJBWjo>9*3q_Gcv2b1sOY@Y_oHj>E*@S~N?3dSt;;(;6`j zmGqUyp@iZus?1hG@MBqM8k!-&_!0l~QhlNzwmj=hhi^kt%4>sO;1Et{LAy3vD*n3- z<=s|(t5*cl8eE6H3{Q@MMz$jkRl&`1`xEbYqZeu6#xQ{OyXIHM!7wX1ECaD~*U*|f z*k|+`L{IL3wJh399(S=LDmIid?_y{>?xUQ$pw>3~$o8Hxmo;ofEAJ^g%f|l!8r>#| z>nI5@zzR3P?%=jr=sf{B|EE7F=DyOHdHq4^eWfyMkf=VmuY8r5T|7N_2u7J4NB#d+ zda+7zl=Zg~Cf}cko#dpy6%W<#A7z%rYR0MuA1S{}ENMIq(v*J12#%x0zWQ+_UDcGj z-Lhwkbqz!agXrd@mjUv4dS$~Kj^feA}k)bPtMYwnjr`EoY~rKd`3!l zzh^rjvSt53Kj#1o-B;7}95itMD$2!ov4yKdeIK;rP08hnQq}%uJAGKi)&2Slsr3^j z&>?mXs?x3FJH~5!deDL=N`g-cs6t-|CW}D97R>x$kGPyWcOC`jDt`51ME{Js$)Q%X znUK5m*YLjEwL@BTD=5oRkQ($BjhoQ&T&0vp#U{X#*AaOZrw0X24#uGTOAF`%%CA&J zjDvprvg1lEv^yFP&kJxY{sXSqJ0$6+`M8xlSE4I+if{4RbM@8;$yW1iN?o5SO*$4` z0hn0QtPs%J_r`lh%VFHAMIdAm5OtaM{&GvYYY1pn!_l+~_#^5wdwLisaybe3p>TTn z6xg+PzPXWEvkEm*`tg4^a?l)0BiriZt)EG2|Ad+wd19HR@(cttC-_*_dJe>&q6igk zDpvSjHRuZ``3i!n4cSyoPy;b;rGldF?U8yZ5CE@7T z@T6-FN$|3YtA)=XldSc`XTN?+T)ZR`~ETi zx;P&45VdNH`(VvE40|#ywQn9?r#KcG|4z!mop6&D7Y7IwA4{^714t_6&dFs0YqKH} zV$GK^AZ{OBq_+|8K4P=*Ev^Vm!x3Q~jFkvE>;{`|gLfkYk!1(xnVKbIbdvGHDB$%h z@U4JyMk#@XRwzC(r=;t@!Zf|GN z;x~#%Y2P4ys_$XPYBJm-v?KYN*s0qG37u(ewsA2Pd8-V!o+$7!*+@UVQ`~9MTctwD zuxWsZw~~zWT?KK(wx{!Nv54KDPS4&d<%*x4F5>mQpw>HxJl`qR9W!)RQE*rAmzmW4 zozl8wx2gXwX-+#j@D3%tPd1lyKlyt}F@aR{y;9b{&qB+FeYCx0vmONq99S4bnj@e! z834wyTWN}LctS|ih1Bi6vaIMj(ZLjDJuUd47(7Sg>wj#PdjqVU8bGx_D1X_m1^Mf= zyksufe#9=}ONwZe54HTLtYL?`)60(#s$y4@>nDhp*H%&IPfC(g+f}e19gR`hqc3Pi z`o3fEF7>HS?9mE!UvLbVuN{$q*&@9kdZn;mL5jINS!y!a9t&uKh9#H6oqnhHAh0 z6w-Z5Bm;pAC{jMnhkcWYM;JyZ+Djb$#wyxAi^z{=eVn@p$j`{=DAr&-rtGuG8nf zHb6=nF+C%eY<_u6^{iz~?>}8#Sj$Od&}^qU6Oz^4wT(x|d_Kj_ww#}%79E&)u_~{n zui1`T2S&_z(sBNvkMh-V?fU4gYE)h0D_jHA+jWhTy7a$IroH|7Pd?El+E`|O@?eL} zLNA%_QXToK=7NXJ2-V5rqGeXB8fG=_-EYY`j#vEfCr9|whx0$)6Ip?}P7-#6FG*N( z9N|lrYzL6lwCU<)t1&xzhpTU`#);AGbOssNzROz>WwvQ{bkOnoH2#F60M(zcv_8>rLpT7HP%&U)cCENP$Y8w@EYr$ek6q$>@0C^mliR@j^ zu$I(VM?nku9Q~#ZHP#V7%3sJ{FCKC>)V3*B>egAgpt`R zV5ObT_xJkni8A)pn_w$`_~1?nSeuD2Ju_ex;*&|-KcsiS>Wae`{Q_3`X#p!7hsoeX zGU$!N`DX>JXV9l&|C8UFyh(icMfmm%SgZMI$aH=_^C+_Qx_}jnoQ~XyRJ&*N+7C{W$EX=d`z;9`yPD#h;x#{U_Dt3&`BB1J)vB1TqVG zna$`!`TX2%-vq3_`vTVEdjnQSK3w-IQih}>#mH{IN^c%S zZu|osiA4Mp_;W`k1d@PEL5BTF27l3o;6CJ0Aj#gN5Kj;9cN+qf*zeNzKD}zK-+D6szhUS>q%(5C>DAV>6t4HcEMx`` z1)2?TBncdZxX!A!V$ZI%@&?sf{XhduJdcIvd@2FPft`@}3u~<~Bo;Y*0fmDDkT@h2 z3BV2UqZ~vzhNPhH10%t&z(jBmG7?Ebl94oI6yiY)WCHRDz8PR9G8{=jMj$`Hmm(h` z>yTn(EPOgriQZv&t>wR+KTDC(8b$Zl(rKBrY-ynTd?o@h89+kY{x~=yyNy zvys=3bCEZZOOQDTK7Kp1Gl^X85Y|KBEy#Qw4`%d#gN*D|!J)A5(<@ty1Gk|roU(N|u|96)Lj(f4WHpK=*zD*i*1!Ej_;4iK}7UNkUx-W+bW8yx*x zxKVJgBQK6({3Y@%vJ@#rmLu!5z6#ueEWME@`2gvSWFy@Wg=|KukXMjp z$S;U-7ypC+9E~KPw+6jnH1ZAlT%-ay14%@hAk7eupFb^dIs#sd4E&IUt68}=(@h#h z7imvCa}@=BM5Ty-s8Mv0sl+qZ)mpbAbC77b2Q-Q<@+$GKkl)v`bZ_ENzjwW(NJQi) z#zzk_7k)v&xkxYM5b|0Tr`-Qi%BpLv6*Z?!=RYaKnK*VL!KX+7$)>RXE15!B<30X5 ztA|-*qmr*0txQ2Kg8X!~a6^_!<5l{N|w- z!CykWmNc&re~ZR^0j>hKB7Px*5;O+m|C<=hTMDly{^PZg!s;1pJy?NrXxVzC$xYN4 zYz^Lk3}*Nqxx3DK;odrHQ8v8;juO1D&YFrmj9mIaomGk7><1kbU8D{^a4*WwlbIx7 zCmaNWas>{p$*dRlV)z9bMHji0xO#{=F{REbA%4|V2Spc2Bp!kNYV3u?7i$z<#Jwh zS&g5d$8zheUHq$)m(k16m!bRL!r^Qp5qWji>tJ8-TmEPHmGXbfaa_&+s$Tvmt0CB& z1b2W-sYE|)tx22A|2$uvQ)k_VJc_;jBX!o9q)j2sJn}k&!uO;9NV@y5{RQ_SX=>5$ znZfaYJ&C4Ju>AA%$8cOu=Fw#K3i1DYp|JXetfJtF$d}Z13jfUhzm@+a-AO+F|3M*A zsbD5S{7)-7A*4KiXu|p#Hlzo+Al9!;XkNj1ecvFb(f?m}aYzRLqx4dt4a~Wrvl;)P zv;RAB68`Hn|A~)XuK!;Y9?T?&|7k@}{}*{m9`*C{*N+7CPz+7fP(^}i8nXQh&i`L` z|G&uWq`ag;Pje`L#5!^gi_i-iMHkscJW9rY;kJP%8H-A{ex1gjCL- zLQQ7XYeBHUQ2dh&g_F#KB9#edEEue0Lj?!@MxNZ;{*yAewO&ht3H}vIMP`xa4l3em zAnjERaKXwu`Refxl0j^!&`tExbN}E%au(PA;AW(R0+wkM+i2pK(84%m0P;F%rh%^^ zh1jOzvk&fl;{PDN%}-45G<_us>x-ym+E&jluAP@YDA1esTl6WeqxqQ9+e z6G$t3LurCO%eAkhlYE80cRuHTQKXRi4iXP$xEfs&gvKKmaQU6jrTJ4QWH0eFeEVVx z@<)h^9>gH;AmcBF_Ow6LEtL!A}A zkvkgjpGX|K;2Gcy#Ms25q)~K{cZsWiQr^!T|F`hxNhBFj$oI(GNTXkvJQ}YCmmo8d zwa7}1`^jWA`gG(6WUt0Af2D%IbEA5gMeGm$S0{Q$LkV()!zMfwj^bx;N@p|^4pWsFC1Eo;(=J;K}<`t1qNEVXTgWp@+&3+A$JY*3v zmHiqycUabfahCNM@+7i~AIy1XHY`_SyGh%h1s_6|YpI`RvGy6SaVEH~*GkLEc;B-A zLS}woSwC#DtXn_f=Re3f$hDg-YbSUZG1NQng|$+1^UXwc_$)J{=`CZj#@up8_5+EN z?z{Ew+a^r9U!@H+uUxTZAUm;Fj2dLV7_C~2F#D)qE-^c(FZ`I{@s}w+nOD-)*@svvJ6T7g*p=4YVDO~w-s4ensGfN2i)7Q@4fv~ z?v;F2%$jPt&G~cPo)xpFnSHyOlgC?D={U9Vb+dzOoci%~v#&d6ylV4?+1{Nue#O8y z%$6R1X;Z^Ei{GE6Osldy{BwtFuosvM#`6yQBiQ}U{;C15s~(gq$(Y-{+9^yyXBptgol0*(YLz-%ye zMimd2s0cWhcs{s9?BHfFWhMnA;0JC4)4^Wt@dJm0o>^5^CYTEPI*<=oNq#9$R#~%d zGK@-aaz}nwlRc-(TGf%7&!q)m_Pi=9p%WSAw$x~E-9}MQJ zDl0CNih&D>r!S;uz!ESOzVK-(Fpm76sj?Pf2$sLJwhzJeOR!&qNQf$vsX2h-`QW%R(+jCC=N*HB|{&=hbvJwdzz z%pzW~lAa)*`91|-OAmn+VCDxDcpU|-CO_s^@yGN8bE*s+308s=B)+D~Z>3#NM(e7q zVhCRe6={wGmgk2-~lkxj}psaQvwbGE9OxU z==lc~0t-P8hf~RX3IL1w`T8h9enFoH#yv}g!Tjf_D2G=um;si7If5@xAXr&IfndtZ z)m9vbUpklu`qN*jwkD%^_z*-PnEQIQRSH&sMqBtdXen3>`VxpQAt9IwP6o@CR$GgO zf0qJ4-*PJ4jtYalz_=CF)<`gYB|QS>zF%#XfcbnJ;BY&R|I82R0glr`Fb6E=YXn7L z_PT0o7wB7GZ3V!T5?aV{oVlUe$^`Q_(gR@bCJF{izyUXrZZi!43%{(k5)!HC*YpUO zvYQ5s#vhzT+_$INTAAp_;Ttw3a@^;BOV@JTmz9$MM;~8NSp?>TyTB5@z><$WrGhR6 z3xA+upywwl)|rYO05fO+m`mJqh>C#ehw%doe`lL6_Ut2+$GYOHq{sazB`Bp}?ok}U z)IY1Oq%LIeSGAQ6mVh~6?lC4Gm{LWBz)G-!3@d7A`564_=z6}RPzlZ=p5Ca&DhKn! zYOIu-8H&y|Rvwt|tFaRCi|bOu1|^gVlwvTuYmK!F2VYW+HRKjr1ZIJ$-D@m^6)&X+ zU$+qKS!1n%_p#Y{5G!3upBig3n44TP4a_NE3OF202S2AQ=7JT&h?8+@DqB3k zV)m(*!KbkCIj$$=u+MlTn0+z%-bMp2VT0-fl-v3T4A0tjFDFcla}1=(3U98Afrv3i|G z+>erulK*&(l>?T6i$LEK6b!~qXY(qU4hF!?8T3SN{AQ97=$lnzWrHbmDHu$Ds>Ui2 z`-e4FC78RW#`5)nUq{QqijDLXn7^4J1pO5#Q>anyCpFd*sUcVndbTj;chFVf0ZF*M z#)?m-B|De{U_2EZm~xc4(2tD40bn*Cj>?qyu^MY3m|Mf70(}80oI=51 z5}0n$VlcB&z{&$lJnSh1E5K5TH)a>!8GZ_gV9(zf)GV4Uc3^63z$ySMo8kx-H^Z?% z88&CLBAD9}Kf(BbwGgam!@g3mvI9FFg=gPFOzMY<7hEh0#@G6RzCdW z*b}@E%m&LO{T)>3EbJ3#$au=hW>@7v@?ndwe*%irKVlW#l0~01vu@~tX zF!v?4X^J11OFXlX&8(p3Rq_WbUiDKT4wVq;gDLnmhF}scd7UvMo(mR&C4%sUZ{Sb7 z>`gKtp1FiECm#1!z>3czAJBI$>E8}mL%?)!3YZNRf|cNAFm5Ro#^2+AmzF{Jz_1}? zSRAlcih-{~9R|~vGxqRtE7)`nrhvY?$Y@o-$|7F;5wm(I6#`4aO7Jikx0(tK!yg;~ zW`iTa!ZpkxFm7GI+6DTvH&F0-bj21N1-G)77cAVt)?G07Q+Bz=R9QZ#xBJPu(H=#jx+JwX(osa3PpdT5D~? zUi>xa7h*R%T!vE+m<;CcWp4@P?Ck?l=kpi^o%t*eCJM5PSCBykaLF zcMq@d!AdaUGW>a+I|3}@-SHIA!;2{6a?*oEl8%=aNyOtGqe8?hAM@8)kq=Ol>2+2* zh7xcxm_36BN#e*8@FCcJvw2=fyaJp;JbNCGX~5in@N5W7na?Ah2kB~_30C6gg~n9ak_gfuF`gQ>m5OL(>v`w)J99!d>DNqvd2!=aJ~Pm_t4yvwtH;@Qi13NH8o zLm+-Ud^$$FWHT+f0{#;!4yNwleKP3TNf(1Y@Gw{c4!M#ZF6GH8==p+{lTYQ>RE)U4 zWH&8C$=yT7pl>ge$#LAvG|D#BB)L0~F43QRdfMFoE& zAsJ>KVUmKrKN!NPw6KyczLqWq2Z6q$veg5BaERFdVv>WN02PFf0}H2e{O9(wtX&vV z`dd~phRVSfyIXKL*Rqlxrea_^m^#$5a>3kTJd*)S&a%T_%ex$ z9n1$k>6Wz@^nn2|_Zo{E3mO6*l=Ro~WGx3fm;lBZPH;VVrJ+WxG2_~lE~&Em2$|}Q z74RV`NYY(u#u~Hh`9tpF{#y(>h*LH)mM05s>T!9(*W+0KjYUWf9AC8g%514(ntNl_ zPhXiWqBAg5AW1xiNO7rsKbl?CZ`;k&qgdLE;V$aUV~HJV=sL5#x?!!^C7jg>Wz^t3HF%vl%+8mOgBqppRatl1 zdALTXjCH2ZTnL*&Ti2Ow!dIea-c@bQSMz={+p5j$%%nD+MuxEx&eWy!0^{L|?@|%# z%^oKI3f{<^s0OY#JDFMNd+%0nY@pgX>rIcj5F-5^H3MTf|K`h>HL=QSu0CCFHaGVY zEx$+YSZ}rrKZc%-shPUV^fvD<|MmF5J=IpUF7k2rc~Ry?k=`)k2}la|UD)H*Bb&@t z-gGCbVtzMUMEJ00L&V!vY9Aq;vxs=nJ^W`jv#YroJvCc>P(sy@p{GBfel9WFnDLEm zJz@hj9fCgM0q%%Z(@)G6>f#M%v^fDT=K*!o2J%>lp8cR2{0U=Puz_Z-bO_qrDZE?~ z-&tk#wVN3pAeu$=HYXZSH zY=JyAdov@j7`=>#P_L=AOb=DM*^CS?gB!^Msk@w(*KRhOm`1o^ESasMKVd2)qwk$9 zL)s=hO?bNCUT4TA5RIi+J%V{eGl|AK)vUF`{vk zsw~cuC$%7vXj}xX=TX_~^=B;7#^hre1vE`9I)oJyVEoP*d z1Gi+3n!QDO1-)cWwKYe5x6h1kF|^U~oD)gCOLCsCe%xZV4-cScQR#5I(p}=4P`L&C zP=TZ6_^_Z#x2Ir>f4bV5r@lML!EnP?Gu#{nH}`2ZZYxJh9(wW9>Zz^fKy$O`&!{hj zuS8$@ta5KdPmJUUU8H(%VCD2zf+WElE8T{)R(D$#Kgjq6BkA+do}qJE`% z5iP=&@SIw`jd3kU54@-jZDVx`i{ixevTC{AY;X2LPb^dex0{_hrK877K_Pi163@bR zuuwfrn$z;pkMYPbss5pH{=LF$({c_{ihL{j%bBRafmWPdDRP$MRoOGj^E0 z%@OE1%hl=~=3sL!`rehwy^~H@g?{*b)m!uf=$Rj=>vx*>nVB($5w=QwxfB1z=-Crg zn^N?H=tXy`n@aJIXLh7aR5MD=9ub46!wBMOlhkdWn*AeW6`4gm@gudm)a)IROuU46 z(i-*Zr}#_!81c-t67OyHYHAn->s0&C%z+U@uw@ZnwO-0yF8bn6ZJ(#gOpkleXKIG1!vMla0PIs&IrBmh1uF$0heS|tI@m6p%KGrc5(~D z801!~zu?pKRljdHdvzj}u@}n_CeIXfBQ-k9eSdVAOu|a|#doTR zFU=lpaMMDnhtfaEBvmx*e~! z6kGrqz>(>0hY|cOhC3Fac6@DKY|SGWX13Rk^SEA;vMMAkeDJDH2L2eF53Ykt_1X7JdU1HcY4P0HbB4XUwP za^)rK*s$pMq0x!cJhK};?rv1)ijE&A;%Fn8XyYWE(qN4rRQZi~^lSzl)D8*!<)6qK$voygN0{Js%4FxvoC4Jgd)f`^$X>H=n@seiQ9RGo>7@vHS(1CB z8vG5%XCeCH8`ZdP%>Hf4(Wi`|$GK*aU>`VedAF+%zA<}*``R(OJmq~`eOkr!Pvd=N zjF}EMa-M3lkJTy42l<&;=@G`h#d|P6jb4Bysca#^| zkvgoAdUmxhdVR_kPdy_+~iY1~vP8PSs(Z zxb@qlc6`ruLo#{-56WM$>#Ab*bBZ4+_RXsOe)=E}y>g4XO!SrLWjoZv`^`??a`ed3 znguU-TSUmX8l5GsHsTkajJ~S0#+qo4Yy5D}@oN$B%)lBb5RK7%nqi!Yg)8hJyk&^&i9RK{{w?I zSA2O&|A*5Ncl|&oZicJ)PR;&-j;chDuc)zH&Y*sQJ)&0^)~N$E*6c{N`Wq?`^`jXR zF$yw`r}U>ssplC9bq=I=7G#DypnfUg93yY9By!J>W=C%sk;z1!{Siwgk>QO3>Rh?m zDE-m&c;mV{%$^_3^Zgk_21W9<0hjb5O8}A$w=jywoHpZdd2oertSEMnv&AB~BDkr- zWJagCM@EN9dU@kDwoky4#X^EizRhsiaHE7^*=l>(w}}c)X?Pi>?o-J)fH$`$>iYh#@!fKA~QQ zu{|jSj@8c&fVh#b|apHuxz~AcaQ@h6@BppxoKZ}h+`wuj;bENQi)vj zg=5vlzp}_KK`$Do?iRfaeb+ek40?C~y>vWZ)o>QBy}z2VUU~PIaVJN`QML!g9x^@Q z@)B_KMBWh42Q5j$J;%ptD# zkD>b}snv(5czkcJF7HAgMHO~&T$$sh>ZNHfVUEcCObnrs~<=ayPV?GbAOmI;Rhiz7VVUY zH?q->@g|7#-=CbruCBy)Fa{XbK@##^@@NvNq_L%zlC^KvX`rOA< zvp*?gGJ56XYVe=%@(Oj<6XMg}zQfu~JdURme{#(dKrf#sdXFfM4kMZC$6o*77tHz; z^_SVylvk`1{-L`5WnK`z1U)vNA15nUHSK)zFP_uahm(zB=TDrtvfVM-JOOgZCa1pn=({$l>yJ_QRpPu^Jt}$ydeTny z#xeRWj&sE1GJZVo^u^I*^u=(viZT^lMZZl(Px(R(u3~VPpcj0>ulVDfQ{LoQqFFBer?E z>x9?LsXwM60kQ@>(j*zPwS-1?ZcDa78}#Zh($X7=*nGuPW5(av@oZJ$wXQDcs}w z!Cft*It4B{sm?l||2uPxdx254W~{jcGOY*SXEY!7CZEG1)UQq!+K%Q(BK#-a_I31r zF8YD9)WvlyjEjXIs3z7i+@WwhV$L5W?OOzg}#`VU&ov-{=%XQlg_2kY<0+D_(!7W-@l-}w`+JV zYCNy6w%C`23vTtsn8i>lr!08T+uq!Tnm%>G^WHY*G1SVN7OeJm@+J=9@R?et-ee_p zo;eSPNg1u8U0xnYqfeQt`is60-S@D%$>r@EUWz{Ai8>Y!`?=Lpmp94`z#V=<#=Voh zBqt4Job%Kn7X=PS_e@vOZc5BTpEbQs&H2;pt}@(Sk68$_%eJ<+pFeLSo-srH18=hu zeew+Tf#?atC}F1h-t8S|jzZ6xr8+h84h&y}Udm5DrY>0PjZ+Ua@-_)CflA^9T$a5m zw7ilzZZGvRwGLyOfNiVozP*t*COqLh=FE*%Tu|E=^tEv22)K-!R5OpaNB9)*q6Yq$yaxKs972M!UgIX4{fbPU%WtlfF3SyyI1lauilez?-00y3+mM94Q3bb z1R~{6)!Ems_Hrig!6B>)XPLOOu{Y)dc_&^uCcdMxbJ-CE*FwK!Y(o6noV#He#vD7maOOOs)_^+D&ZZ@|kMY&C>S0$UU3z^Et)DE&S zlh9L-sY9a6EAy0NDmn~(tfTi9Jx~0r)n(}6i*21Bvf3ANT}p@!udZ_*Dui6fm1A4N zMeh}AjLX|ftqJo+c8wg)ObPYB)agYzNB6Sr>PVP(aEIxJ(Y7rnK2mFBVw16$!Zl=L zH%G(gT~wbPEIV=$&VOfoF~-}%b3t@QsyvKnr3%`3Ju$QJFCJHA%>g|#yQs^Y9cOs9s{Rq)-Z4dR<#4Zn zo@agPMFuQ1*yG1m>xx9S7oQW~?KJi^x|?PArRa4}hM zIdEfy^LJPMqP?fZEP*VB^huj8@2)0Ad#{U*GdYOaB5bGrOMMwl>NL1ScJpqN)SHrI zlsc>aF?5)`u?pjb!9mGlQj#-;zKQk5sDc=8Voao$i_>SSt#711&rMQ0W4uG7Wq30% zWU#OIN{@<;@wQe^#ClJUDTbWzB0Ep1m#2x&j9b6xMva<#>;Y{hQ&-Z5F=!)S~U}Nqt(v}Xob{{qK_G~U!0SB;7?pI}Th_Z{e zo4P#C+pc*Mk@yGC5xY3}tz@5PXc;blp2t&?JTb8p|~qa?c~RFCby8Pe`1-IPDhdqqs4ko&5w zTbYTT&%3H`s!V5!y1$LI^HN2zUQcsbEQ@g+<>70?&2aU1 zQ6GQ9;r~$!Z%oVqEO{+?x+vu=bu8_QPu#Id>I4eNkFT-Lm(HG}qFZ`9*;6A8J|m;X zdKMqg{aw`jmdrC*nMxsV6!P{iYJE#@mp;Xi0mzm@p4;W51w|%VnYND?d!H_g;&dpTt-yTUFyMWc|9RrR&VrDt4V2qh@06pM^PxG*czb zA3nR7e~;cRWi=8Qeiv!1^c0zGuCz<7us_V^UMR z*{}Mw)bV93a(1$Zc5Z9$fS7#D+uE=NQ3}1>K67+d)valE1(p$Q1J*lY>84K|=e407 zNsMb=a+TFp3Txt160qa{?sYdI%-O<+DGdcrHLAQYQljQcXda13qYyW(>gdTi6S6cF#(?9MDr>B)XCCC@GXfUhpXW&PsjUgrz8Es$Og7X?^0q$pmw3uMv%P2V`yr+>N}?$b6~{3mWVrZ&d>~l3Z#%1) z_3{9;oi{$Z7|TH(|4kRm3iW3@Z;$AMaFv62Vj$d1)w{j7O-vj!e-^oZD%HJTSK`k0 zRAM-mg@f6LDV4akvmBZ&R6%>rF|)7~o)@tGA^G&|tai4iGfE&MQv=ook`cFa=T^#z zyaOk4d9fCD0UJA|lfP91J9s-qC$-{P+&NXw>3EYG*MaI~z!kyWBRRaRUV@8R2simo z9s_|Me6l}1d5`fGWSRI&b-MFTJO`I! zyH`9b+4O+5UMAbiYGNYw8U;*_FdEp*@Eodur2UX4R;hy3+78xST)vG@@{OIyy)4>tqp~izN`jrvoI5 z6>4KwW=JtyuSm9X3a8YcUA^t2%i+r5>|EzauAQ0>z#i9u`O%#HtpM|BEr!BU7Pwl=LGB2L*97uVMIyy6?V-oJ3L~4hntK_~v z(XrepmLXV*+VK4yNwXc_P&S5MZR@NT#PG6CvR#rCVJY>=Qzb0Zv?aVd{aArz z3C~`xmo)xF6~nxZ?nr~-I*9iq?R`gg@)?08`!u#|%6MFrsOEI{c8SS>Tr;@N+9Z>q zU!rrJ@U>)Kj3t3B!avGHY_2U)J?MM+IpzWO1fLJGePQn5@JSY_3TWrempCzi=r^6utAl#H%&EbVs6!(C?$;-EH`;FEhVzaf{h=BPtG zm=*H=E|xvT@+i(TN#}T0PjZaw%;J1MAAfRVxx9nAwkM~9RLFz}ahHai*THGxbK*W3 z%h-q6A$#TtOS|uT96l zuC259-%N(*mP9qam$zd-d7-y*9Y6IJ^7h0NS3ws|v+wUr-C5Pk8`Gx@%Ph7Rua~kX)ytCYK9~%7FF0rmALjBp^^mhmGg*}rx^PYR z8QXsCaZop%#t}0DF86ajm?D`yt7f0Z!>qD9IY)k}-aXAbY>fPB!}y9H;{(CK$(21l zEEHdoSC9m``r+LNfAHr_e>_;0tL^Bcpr#q z`~{I;V}2m=S=2*;viR6q5 zn~h)IsB${zz?c@(;A+Q|MjOWVtf-wkXhYU_cz<|z)Eh(&WrwY#_J3wCj5LhdkGP_K zr!$^?#3dR2>3`G}^*Ql3A9ICm$7|VRt}i$x=FM}ps3h&`f4Eu{QZUUfyijs2I`U11+l#XnwmjWmp{Z@8jfhu!st zYbD(|_ia~HE`9gh+pe8{DsgbBD{2pktCqU9)1Zy-xuW*s{rP*Ycj7{>56)Smb|soC3oasm${I@mW#I!JJI!rUE3Mqxkp_s z-p1>dqpkqc2cL0>z0bN=dWBf}vb&pf-Y*$bw};}iYgSj8|Q! zyJLP~hCN^Gj{FfUDR#H`;4`-||2=o(%~ass_uMUBBz_pIN&c#)qiT zp5^WqTedU)N0+-}$}mh{;qJVSjDB9>ZZRL6z0%#df(#a}bVq)Ked9_p0&Bs4g6rOQ zM_SC{>i69(UZ+I`AGljQ&MbcK19z;nwBiGI)KTnzet;kR6Cb)`HiEx@=x)3QK5vyf z>dmcwxADv>cdPd?EC-i@pRIDoJSEd$mAjk8us`#WJ90U=>?3!#S>U%Hxm&HlUITsz zE?Mnvu?*a}+8y;3V_d%4-R(Va*2nHXQjyA!-Eof)e{zjGzKF4ZV~snel8L^0jl0z@ zKgw>1JZ9k`@Nw|nweA*W;G*@+yY+6@SKv{wyu=;#TZ!BC7dUT&+qGQenPrW+^$M+_c!=&aYy-|-Qsr5 z*yfIUdYjwzBDf6veY-nq+79?n-BE>~x?QWl9pJJucjVeKx9g8GchqBFxLsd<;g0$h zJPOYF(#^jRbVscR3&GW3=^l5~w|m^K-@xsA-L3$*8JxZsKk%u&?sieDQI!n6Nx_=KhJ~X>L5n+^@XBo!R-YC@d z-mq69jGw$s{s=dYc+cZ>6Sl$ye=vLyH;)lbK8$cJi{SZ(e?{8?-i@w~mMbi}Nk5}? zZ^Pg;6?a|BH7;XPPs4b|uyK@{U$_4)hLEYX^E0&w93_tF2K?^@_PtcNyzk z=aAABHplIH!Y%pgvmTkna(0q4RMJba#@yIGYT!U`_r#9IOs@QAwuybKt+A?2?2fkF z_jB>~YTL#TZ?+w$_6+nsvts($-fk_IC10H5Rz+8MWBtju(`3TjSx)w88uyIjR)}z$ zLmTp(@HF(pa{owR;kH8%*PQO;lW_`rVyu4Rq$9KMacHD4?&aR!$r#(15enVOexw({4l zuGdb?q4CAqT?S3UH;9xQ%KxDG4cb4ExP)78aPn{FM2)n*PF$`P30E2GwF6HT?66zM zxka%P(NSo?dm@)&pHTM_c>Uq@w!n?1-Qw!!yWvUva>_@C#gW! z2KcAoWy1Ur($HyeG6ycyf;==y)gX zmle_hq4vSHKZgv@hU}28GhlVH!x$Zx*(>2m9p@329bVFLRzW+g`oO7>(~n%;YI_fD z|LZ2l-a*H&q!rC5-)P7n$7U#hKIx_KQ2w4%@Y$MgIoc^WI5!_B4_S=*!AtoDQ{E1t zj$)M74b&OPaUtQbo=aS5+94;TtJ7a_3JmtdaGk!1_Pan8eb=+^oUC&SU2c!oZe8Qkp-_0o0T-tILn^~hA-U1Ux4hPygC zx-6Kk0@d_-Z-ycF4Yh})ScvGS0K5V}Ust#t?DxZXt2@^V2OR0yfn4x?h z?k{D94CR+Hx8%4#ndcz?4TJA*299&aSqey*;7rosN>)ekKj|_Z_@t3p1DWI zFUKa~Uh?e=t`lAs8M$zfz<*)3b=yEZ$VbysS-nE}abzT29LnDbFK5zF{z3RoA-t4t zh(Xr<&;$>`cRdB)kpGlZ_zT~V|I|jbzo7y!G??Hq_|6SfNO)PELmTh}d}xbtGlX|4 zq*uUTSLb19s6xUw)Pk9(C_wmz{AY#cZ}&en5<^1?o(xSO)h7twkbgG3T&RTh#5#DH zoT0pbJqDSnp$6e)#)k4ApCY3*r^rY6!Q^v99|gg+KM{_j%l|in9A@%dFtp$oIjGNQ zfPW3XUjzJNcv<5^^ATQ7bPiAd2gm*`4#zIX4JUZ{$?>HP@Q;v@oc%%zcp6?7vQU0L ze8*7!YKUbRI)@sFzXva;&d`iLgzpZ&;ZAByNVETPdkA7OT1-^#TfDs+aaX}DMhlh3 z+0$({Qd#-(a`GW>n7i~D=iDR~KhHzzhtTs{smTZuV!aL;VGhop=#)Yk<#!ljSkg|86+x?}qr{CwRXRY{5N{GBZLGJPg-R z0mblL8}NS*uAzcwz+dz?e`erJ$PRxqIK0fH&^<e$XUmk_J~DL*Gtu%D2P4lT5pO`z#?0n;&gM@Cqw5<56TRb0n1i7+!`j zlyA?$B{zJbd}sJhHm}b>!5;8okg*C)@C3Y6DwHqdXgsX}ehW<$K0eE7kPMN8n-RGz zKF&MsH{@$^_Shab5SMTjEt3|sUF2ls(Onp(F-h}_b|Ft@v^{nlGvwe{#Ej@lUEfK;4j>9SK zk7|1p$Ng{VpVW4*^AN?<<+sr8r@E0a&Ttmk42`k+v0kQ5F9($bGr~Ezp4RQY+@Xy& zG)pcEL-@|Nq2u2TM)dIn>ZLopz4@L?^c{);C*q( zeQhUgr`~o5ePqzfvHR^MLp(xnf6mt##5n2yW-s_Rcj8y;^l}VH_=|Q*hn&oFK-6G} z{F{MC1ssHbPr!K$UJl@sGct@AczZ!+@1BGV`>$d24s)(q=7;bd;fKfp8EW8DAND1E z>6=bQV{}FZRP=oGP+oR@$_Y7?e+yoguTZ`kUKV!Y?FC^B5eXKCeqd;VMzl;W>q7Z( z_<;@Z*T4r0z-$Eb9}6$nIQ0$&XF;MT>`;0QH6YY;u;W4dVQmlk4OztH8X-;JrYCD$ zqy2(>hUN$8qi3|8idP2GewodohE~*6&VQ$>$f*w7@)^$I9C}?fS3bEzIIJ=D11J8p zKje7+nIV_UhS0-qJ^l?G=Z5ed8}MJD`2#wCIWbH4Q^zlLqQ=d#NoW|6m)~(FTevo) z>UglLE`#rYg!04TtN_SxzOVj9Lmcj%b8N4_nd}xl?qY5@d8~fmkTEuh|4}2IaWjY`{8AF zhw}R=Sgs`R(**=CK}O;)y%NeVgY(N`^P)}=JX4N`lM~N!%@07Aa7f1`jfA_GJ1vn( zDIxgUW}fDQXUM^_H;^C~^%7ikxGYXLtT3;&3F#IIp^jy=NGhUq2Aym#=LTBT_ch;9 z^KukO*t0^7yNlNi@92l*O&r;emV|fc zaW0?z#vj@s6Iw#Bf}sgsq=RI##>inoxW@i#7@>Z>SZrmKLixTNeR79#k52ED!{lPu zfW0`KY~YLwFK5b7z81b~1AGK6mBT*NUp87d#OJ_wm&qNPK#n?@n4$br@Uq;8^0QCD zFFHj&CH4HJ{Nr8zQ9VO_{#~8XOx<3Hz;c7~2ipGde}$K@(T zLTSiV>Kh^H@6yFF5$$l&J6|s4jQ4c{IUPzk)d{~Kkrt?_qr7b!4{0A%{YCAZc9m*O z6cZs?};$A*7#-wc4KIn2p<+Ude{qcb~Ue_%6LHxt$?Bs$Dq}>=638`G+-5+c;m_ z;V&KUq~p^-S*BaBb^Lm3K6njqh3117HrMO8UUvP)y_%3EP{PwXE(?N$%{m^;cq^O~ z_^0NB=loE=P1-M%Pp2;=y~DE(1~X1O$#8G&@r6g=fTT(jkgmxcQ4|<5s?eUP=1m2FV#1jVe*4OLTioXnm;TkD}GOH7_2c( zqexHdgapb$pPbO2n@ZSs^ zdvNTB=y>ojyFCsMH=C&QqQq87%C%QVz zQ}~hYH@)^_`omMbi<`&ll7pUyJDbZoUVM$~T=_yNVa-*}d1%c@2bqjc(Dq=|ey4Xl zV&u&4cyW?@$n7h}<#?}gxr;7T>mKpm=MT2?gd+!}?9fhj+J7?N)Ht}UA-BUosSy_;c}{=Yv-7y65yx||qmPp)jd;S_FyD%fZ>WHHV@-p6Yh&}9_5JmH zYQ%!E>#h0*r}#$lpft2!?FuCrSw-sZY2FK37tC^egEL_)Q8y2;)~HXXd0Y9*wfP&z z%Q%bivdvFAL}Oz;&*NGJIuU*W`qXTnD`AM5ab z`|d$p-qVI0{TL*8^>aw`<@213WL=lg-#(xGeFTTYOTndc96x!`F5yYd8%G?X)Mgpr zBiazF3)o>BScQ;$Ki!_%&5cnZ1)K+O4?(`;n56ylHJ<@*Pw)%0 z;jqqVY)D3p7;?#|NM|I^?j(%Se8wLR5#-lvK5?YuC#PEp0nNwihMld;H)h%EPIDtW zq~I6e?IC!|$spK6UUC$^T=RE2X5)U%rzwXRq$~QTU4!OEaY+8*5wu@2n!C_RAm1;L z&^yAJ?UVJv6}+FnS@T7jA7Xp+LkP_$>Z8MM54JDh?TS9=IO_TueOS+hAL7q)s(4J- z^cHO>zuKPOhWyM`!dA@}yyrNk=`)Wz(#a^Xg)=9v*7-cD`J68tLVlAkp)nV6QgG2y zhxd=y38rd8qTGoSg8Wgo_S$g)-!+5kZ%4f0@^A zaf!nR56e>9-%kran2l)pPEaM-wca?qP(6b)FJlqe~~t<8RujqkJKe(X@2hgPDULyzee*p^BrQp z<8E|r=@c;JR)+|-_y*0-eb@2N(D~%Iqw{!IkGa#BJKpJ>DrG<7$VvQuDWHevRhu)jS^;oxw@; zYi%Af(&77Sb4Pk!>RWV;L*QnIbj@enUwW=;UNHGUQ5a6TDO?XOc6PdNnbcb$0yILJH1+7yq=o z9DlhDmGGAKPd@H{yJPrKCpdh!lc3C@jW#}~1;cebxUM}8Z}*%%iJVtR;<7>Wv9eW+ z;L%;(Lam+6!S+NK-LZOVIExwg@9@%r+4|hKHsI7NIJbLua}s3fj^&(hhubxubFC9UU-QpvzCc$fMe}8v zPrl9}g2zb5Bq#rz1=0~*2wbQW^jF`_p+i2#E;9tMgRZYJyt|XG^d6_a!9`#iyv#t4 z{2z*X;VJDu_ev*+U@g8GOe}VdRF6}GO<|1w0tj(WmKKSBPAHl}%p1R%oP9iw4yLvgiG1e*fem$?e zr#XD^MQx(azfo_S_Z!I%I0@!B{b*d|7>qQ%9RI(%&IQh@asU5o@6vjTYELtj5_T85 zNbT$Fy)m& z)`5q>=`<;`%P$b*Loh+~or}T}xEj0*0zV60kH>&)i~bkfJ|09X6_fhV8I21NYA-#; z4v8~}o7puqS`^D*Fc$`ulO$p%1l|W;jWvdyrjL)JO8KHLr7`fsZ+bLU&DSd_@s~kP z_k-75XwEK@ow1d4Zos!v4 z4fb~Lg~Z+m`*vBRk5wTJ*8JcCG-o4;D>_pO3-qgGQ4i ztaB#6LXsWXCy#OjjNPg;HO@zCE39vUmaE*XxeGZ_0X2M<7h9C7aK=}J%U z<#$$^48g!IR-Oc}pCu)J4365SM8E1<;e+Mii~z5{P5@PaNq2+S%r-pj^hFhY2SMH~ zVqn+9snaBYGf#L}4$ds_`Z)s3+g&Kc2Jpbm!j~ZMzrd@(M;K1+e+ku_+9vtyozM1Ej$#9}4piK&$A$}@u6$k%-{`KJbdzLGcxs6?-w1N|Muxzkl0?sI7)+7H1ShXscJ0Y|~{ z=D*IH$3E*7ZxF*gb&H)0KNe<70pL>XN=k4{7bl&vZmZI<`3<_}eStHot*L|8Q zyst|J>aO_ zdHjK{D0=iy24CahrxE82<<&}t@=$XxnWwhN(RUt$tuQFZ(`ggwIvJ-iBT?1lj?-fE z*Kr;s&KXLMZ9EE|hJM9Q5_l%`JDe~2!!AiTHs}Mvgi9Lp0pKeS0R933h2BCUYC+i%DW75 z^5u)4iVjk9y97JwX(ZsP`_<+Mf}3Da-%||Qfxi#^=HnXIP#yH`-NT)spBN>9?emeC zUYvV{rycviB;Rks>30%vhPrqHWQ`V{=iw)TPw?;*ajv-vpA^7mr~pWFA-7GHYJniAP zfN%8hdx>+2%R5Pic$ZU7A47keNB>*s2YS)^hZFBq%Bhc8bkgb4V=#a?2dL@VxGisi zeqem#&Ui2ML%E`lPVdx#=O=}KA@iT(bTdn>Y$xV%luMighf-p|`=oN31s)tFyjrmH z0(j*J;c+=Q?~D^awIc-#LDPJe9?zEa^O7fUJ$Q|WH@{T$w|V%n#5tq2$k0RR6C=PY z@WU~!IZfIGp0>Xzn;Dcuzmb41^i(ulfD(f^4O|Wo7$YV476ImfJA8{bC0dJ`^fq|@ zt@8g-&E?pZ79{K^PxRAHYd9JMLH<~Ags*7MN^tv=x6SBY@BoH|LV#WqB|z{p@zYrb zk24KCuUPo>lQ=`v|29BSvp@t-z|nWs;7;MUfp?!I0m>H%$Jy5@09NUf33OnL|==gh8$mk1gKpujyix31}_Jv3NYy&;@pSoJ$+&Wc;M2;qoam6 zH$gr5Nwet{{D;!1g(3*GX`Iu6#LblKadaVgUk|?=ehRSuwiD4F@Y?&NV9#Lv;p7!b z;L7_&K;ODeS^!=PehLcmw$*=J^l5aN)T~(i)T|M{4}36q^~1t3lRLAEsr+0OHIItm zEv!s-!@yZB99Qn0E+wL0|FrPs2r$WVWav-yl?TCVHj4fi=!|>7>(YBf@FE;_DU|>r z{N7|QEKCH?s}ucx&BWk-@TwPuf8`VYz2#pBuY&)+WhlUI;b}~4&J+kLpAkV{IC|3Z zmxSA21b+q(0xU#;=;aci8hj1>{2jdD4e@iX@U-)e71W5pR`t=7;fVIZ;`EdpoU!2L zw7(W7J8JF&577QvoE|oRX)_vd-iM3R_3#s%!v1OfAH)~<_fVyG8UbCJrwzF{;ku&p zEqEU7D8{J`+W0E^-p+lfcB}9mMoKq=f>+bdWSpi!KSaaL^f`VqgvZ~BMoho@LQoAy zoxooQul-gW`Hm7naH=>e|4R73P}L=te=U5H9Gnbz-2nlMkHPpmXbt`=f^cgIkhoF; zRO1<7o6!~E_Ss-NT(hT%zI|5sOBCQ7aQp0V4*0dxsQg?NK|E9(Z7Gt~WMF=*o!-KL zg7(6{0T0{?4o@dtC4mdFqz&zx=E}hB#vsQc@H61`?eMsXU8DUU1c7XEbR!H3$|XQ0 zcpFspo#4SD(f=0uFB9h(GwGQbKZ1VkAklxH+n-bG=@KA+xClN$n+*f6A1WLtM`s>* z6()i*=>HeIwnX&pRqtD_7XP7+A$~c>4)T z^{AonOwpGIwy6GTp<>cq5Y)xQ;5hIfh;xb8cmn@v9Z{8Yx=#+yxLFdwKEP?e*t`TD zLes2deQtl>wPH{ycU?IbpahqLn>PrM-s@~Y=}qFCp&CzyJ|WJdrrRnR9rPcQ)ODi& zvq!%RadZ8@f@U^O<~jo%OGv;~T7hdjTa1(*ON66iE^FMzk3v6xjO6?&=0Tt&kW+6p%PDq7C=93A203#gXbZzy9we{;IuLm16}j+ zI68qiH$fhL(6uGI9=yQAcY#+3Pn+AwId*i8II8j(EG5n*xWg0p5%7&3{sMT7hkpS7 z1^8`=Dwqhb>( z>+|}*jyK?;lxP0ZP4^WNpw^6Y$2kjp8hFqQD|(m&{3-Bi{M5f4JUW*@(3RDK#+`8} zaV|g&`7wFhie5Aqg1T!YLtWwM4dN}#!$Sgs;H_@r8amGDh3o$T@C%4@fc)#l&*R{C zK|gr-_W~2&d4U9+(=nc$egIzX;d{X=Jp4c4cYAoNnvm<$Y)N#&1Q&vL%fJc7~^zBoo=@$`TCj|Df(;ORkz67wle)3x+I2BmVdmT}V zn7{ObAMgV3UMSe{w~C*-g`)qA=%<}=5LEGUn3A0VSAjcl6f{!0K1iIa)Z8Eo#)qs0 z2mn7FA)5$Zxk&umdoPz8PV0XMcOHxqxzlN(II3DL05?WBW5H`45^g)uO7McE!i!Nu z+rTSirg7FGL%na~5A*{D-{C9$UnFOefZMWGi=$s)aI@w22*-Cx=Xv5hL5;%%g{F6! z-!6VC?-zZ$mP~<{KPa3hS`&XQ1a+%KU`yt|LmW8|3-6D%jDm;E$%P)0MC1j;xnzC0 zWPeF3tp20$4?iszQG7ED)`_47js`D6O{^Dw37jn>&L#f&@C0Hc??6AaLG<}`C7ik~ z7C(9Gq)n;?J7wUZ^=T1QA@I`>)bhUMlxoah+B^47RO#cw?*@MuJoJQcbCM+4_u$pw z=Ia3QlPbke;4#s!w(UO!f*J^Dt!UD3;Pog$0=!@ef1uxlJXJk|IM-rxZcvL@{U zubL`>o1wtzYiWjIgV2X!@EQ#2%*;c#{CNWW4E;I;*bM#TU2-l+qG>|l%ZRi8Dt;1{ z62B3K(>CH%fV5NV3Gf{Z3bu$Q9vpL`ROjp{&z!Ew5!+K*1YW&DO7@mu=PlDlblUa! z-wl1Imi<%xUyiEiwM+t(ZD1 z;5PBY@3S%EZ!H8h--rNDkvpepF{n5woL^tVsRF$23*q)6^9#f|aLg0<1L#+ycTyiW zsrx+?-{feoIKl-@XATK?m|f?YSoRU;0Bbz~ekIO+3YJR~@a99D#@x$)vz^Tz{mH~x zzp_tr>Hm`@zVj>u1!Kk0QZzw}`^3N*FI=I?hFIQDB7QAmXBl|yr2^)`PmJ1#16TDD z{dqF~(F_JbeN+U`_2is8&XeGwusE`(*F)gWd!j$Bo#^*mApt7(2p@_rFa^AxH_M@9 zS45AeA;{Y-f|uHeqo2WpHwfqV{BWB7fCQ*X3*dMCa9Rl-K#4aAcD@2HnAo`Hx;-d< z(!l}|oQx{n1wnqHaOFvyx5@8Z}9zUT&=?THkF7S$w_<0!omMRpW$nbQ- zuhv6XiJ*Lv7(5Dt-@yZ7&~ROE#Y3WB5fS}T=+9g&Jk(nv}zk$BPn<`WK%>1RDflL)&ao`(>Q~3|4 zD@kC+C93W>i0+5M7LTJRpCswEvIW{bc{JS{vQ+^iQ!HuV`+ zps!}05?6pXFKb+LcN6EDaCmoVO7^~>j}7PR{||_OUxCKSzflaT9uYoEurnDvi1A$n zzRT*bXsmz6v+#4T@B-*xW2RKPA&E2k*YS5N2{@+}kBPyPEyUtg7*u*nwjF$phkp-0 zp=$BNZ*b$(HzWQFUJyn{-hcA`s1%>+P~UOWav ziE~bChKNBQbfOCAmw5E=hJGDRLiVg!4<5qF>>5*_)EUkCo!0*|JdW~2;N0WkV~BHx z3MwQ+zoIiPhW-YR{zKq39=-{FLRYaL8h>_})x)69W6=B+&N6i(4?h+>Fu8FX_XQ7m zcpiA(UwN8bk0F8S6w{<+_HnEZH4=EjUkwQL0l&k;2Y|2j@X^HCf89*+&u{kQw5$f} z|7wf^nxakGNdhkM7LTJJh?^$3RUEyIKGE-0;nmB8n{ya>C&b1Z#HsjL}p!1s&pu9pH?2-d*Um`qIErI`qe7y;t4)MN#lzxXn z`?p|#Up9JyPXV|47ES- zh;vN@JT=i~iv*~>S@i#a{xIUK-@~In0s6tsQs8dQSdaSOOV-gX3}~q&2PgX-{y@Ls zdK~p6&ZEF?z-LZ|qxGXU`pbS)aFPxXxqn9~jsj{cP3sd{+Wjy(%S` z%=x1f0k?ZUDZ$R)z=PNm>LR}gRuSi~_w{tC#*9DIoU&>4f`$GHsp6(0TR#7&8@lN!EdJ3oNi9oB}z&vg3A zCHMcFAUO&^@J}P4#UXfW@L3;A0JD1=vCiO|!R?l8Mc}DzqHniovs)d!PMrJ1Z(IQB z|6;Kd26ivEN8o6}b}%rR*H}N~~pyz;{ zpZS+b7+%PTPJdCzmvC}r5f%oeNg60`kjDy)?YS0vQzqG*5?<;9W}e* z)4dI!);D~5x#80%4WE8#_|%bB2SMW{N}t5vaQf8Eu&Wx_qJ~ea8$NAn__U+p)A#n1 zuce;;zS}X}`xDwInI;q4;I_~q;$8j6Ig`;5Y_UcdvrZ>+4*hK%@4i$;BA!Lo?fmrR z37cfm>d2%y67*V#Y0oT!?P)W?oa#6ak5>E}mAulJ`JR!(+j7_^hhOB-S%?4OcJ!Ss zhl}JeLkyZj{1BzD!9|*H&H`UiM<<19T0s-fB69g$d*GNISep|PNS&P zaK6h?o>COI(1^qDQC|ztpsK|D2&uCQX4M$9{UMks&Tlmb9r(~~-k#09O|(|jHk-ro zAHi9{a5*1@&|*7@o8IF4Ui7KqKcs^6mc!|C7%hh~Iou$JWpY?AhZ;HTHHS|A&!x7! znQEiEZYA&C&Ecqlq-DCVZ=CSI%i%6Ltk+Mm^4sQrnYACgJ$+fF;Ye9YaanOPToNwJ zjmL&8zfg5><5R9IpL$JFS5Hvwb)S6INmp%iyC0t_E}S}b(pA%nOT&rDWyQJiWTd1p zUYbl4MPm^?d9T~kpO~bd8m~I)2Y0x~>EPFHOkeP$8_~b`Ra<@K0rw<*@gHtm{oO&g zzYcA8^Yk^Bsx1BJm+rZG-VQfk|5oR=*Za27r`g-xQTn=_qUdxBosTea5i4!4&t znF;N1hxv43m)l0q-|0T3Z`e&*hjzKu%_5<&UiFFlsV|~$-0jZP=LS`_uIQ#Z>&@d; z>s~Xbl=SO&5&i8kbW9KXkBX8ii5Eq}MMcGtqQcTxqEzqO?Jo7jGuM9Vw)Sgxk6W&1 zeD2n-igW^{L<~H7u2~u z8vffwkKXHc(*1_3Tz&hOZm>a+(1-TA=LF(qC555lXe^p4OT?qax%&IP?rD0?S8lsN zI8+kO4JC4u(cEaFxHPFp{o&^5x{wO!H@} zR8o{mhh+1;{kM2138rA*k-G$t`{q^%dQEr~7cR$l@yQ!3Z_b0b3D-w^VawBEY zl1MBSk41`y=&e7xmo@Jf38(bpuicVBakR8J84AUtMMWi{NLiWwVw*dxBgw)$dE_RU4=y@A;3LlX>9Z?l482=$qf&SuK2VeOYsL zt}i$9XmhpAr+c}oSWnDW39Vce%%oj4|0tj7QrVeKt<_}L9}ekJJydpPS&q8VM|RmA z)FaLG^3EzxU(}u2Vs2*&WLp2|<5W!V|H3WSU5FhME57RL(rEdNtHabs^Jl6)dUF@m z&Oaro_jFOGw9ZW=i;5yeC8?sqa9l5Gu5y}%!wLP#aq3ZjIH_;yq8{^86`a~x4c0q4 ztM1uN3LA-K1|6?D`+Z3rzC^{^BvPShsH`MWoQ#rFy=tF3oCjZj-7%=T`=cp+Mo`^s z3gf52TmHFw^-0B{RB=2JD@>JyQ_)bkY)I28jYW03t9rLtBoWp%K`LuuH#Mt${0NFE zUHET$=N{LVKOWZi{p=3nDvJ4Hdb*;7}ph)lSTI?+eHY1Any zn&3Nh-5`dOA4dmqIgj_94{;` zEgZu3opgs@YK%V>)d5Xj7L%($JYHOuOqG_zXnYjKBDs2$R$Z3gt2&%i77M3n>?BGH z!>Lp$P4!L7oyyJpaH`7qDVK9|R5s10A-yC-*`+b?mD^p{hg4dh9HaU4vxsV)=^9b* zH17~@T7i*BXn8^{ZQpcEMj{dYWS;7fc`2dBv?aeIhpJODXAV`n{p}-7G89Qfb>HEt ztWP3#1X`?$LL3R%daC?9ya_MUpZ7;z%{fpG@gLMylIySDy}y zq>hlGRBqFZrxHhCNxiUx>b3k<)n?fRstuoXXOQbuW>vm=+vg7@bk+kZsBassX8OXL zE>!be~ef2{jro@vQl-+ zti4oS?DM5$R?fLhz1zO&xQxal`l{`8ilo!Gt2gwe^J(ZP)gse>qB`XB=f*N87O07S zUoZxR$=TDf)FsfTlQ8n5(RrS^Xo}%8;l{c!M zx_*i}I&<)E)VuxEv`cPOC+q5|YH()iRCN}ic&7SF)r&qP_3mkE8aF%ZWXi8n#Yg)> z`q>-Q>6x}SsONk%nEUOgoDRI5)+mqNMblcV8&yc^iaN&yH_{w8^eWXmletm-y_tS< zwhCkh%~gYZ&7#!6x7?&Q`N&`GbQ%?-=c%@Q#+iPTnIp{Pz~}SKnuUtqS|2?;UFfBE zP-iY!sB-)?!SEcRf4@VO^1NuLlrWz;+N9M9nc79Fr{5n*=|l6>u{x_#{Xwn*OVmBt zO-E=<4;n{9KNic?TbHP@l(P@#(2O>4sfzpK37N`XU8*MeI>eiHn^>25mH zQjyVaTcRr-%FR~If+*#nL+565zk;4N{4>?I=l@>%K}J$sp2tbe^*^(LpL_+`tf z_|)L$KYuP)kNd)z$F-{E_92#|Prp~S%dELqwWjfutAD#sos`*iAD#B8F6?=uJDr_q zpnc9YSO0(-Nu?j~pn8`)PzNs7Q&&=*)4*w=m#w6kAQCsDGV|+7)xyWsd(KHT_Z+oK zE%t?J9X(v{S*7-q_jevrGyKt5X5eZS_4#Rt%vht&?-OlW;CQU*sEfzK`qcw;CjE1b z`ajCZkE^IP*7Q&(>d+(V3jJ~~b%tJZhYB={Q;R+Eh&t0BOJoi{qDq>zipQKu+-`+-`S@ycaTF8!tF Nd8>TM?Bg@={{eoZjspMy delta 167416 zcmb@v34B!589qFjEDTGy!xD|kGT?wg1rr6E256ZR&z?WdaIusYcw=x<%_U(@}|9HMq_9Jm;J{nYl3E|Nr}b-~5z& z?{l{Iyyt!2^Pcydd(-~#;l2B&w-=9Z4%x)dBQLhLhi*CYPorGRNzNlE+(Tfdnkq?uE=@vzgcQaxBrgVuVQVyfE|HH%Gv>!DGbra5NwRJ1#8P zd!sDS8#1qvzvi4je*gL+huUXO_uVhavtIMY%UbisM=r82pT2)-S$UnJrfyDTkNt#+ zKRxWNYo6^}`q1^G5BHz!OU)RhG<&w|%v!Rvvu{S>Rs z2AZ=J=*WV1Wx;#1;Df`sXx~=+J2_^yix1r6;w-ps!RxA}?$%iFW`);V@Xl&O7HF^} z*p^GY$%4C1m3XrSuQ@?>pu>V+q3|vX{&R)*8kU4^mC$d&|Df=g1)p@MYo8;W87-Ed>sqFALOU!RsyfT$R7cg1@TpwoKfxb&Xf7%S^BxyFlVw zEx1?V-4@)er`LiVvN7W`S& z0S|G?zc~aSsDxTe0TXYq;AfaD&4PDY@T980D-9R=cgjM^>fW>j+Jzew-fzJ-D?Dbw z`xL8(EV!B9acL$;?_XG;^xu;3v}&Nlg1@Zratr>0!gUKi{zTcaS_^J!P5q@tCPfQX zfhJ3VOHYy&wP(S%X2GLb@O~4Q{kPf71_sRp@*lsAs$oZcCZTsLyx4-9`AaSMUsZmO z1vl!HP}Eow2Gm&AS@7;=$%zIlt{P~z;5!xGZowTBMLmYC(~=NZ1-dM_*>Kc~tNeWy ze6PX>EVx%CgJ1lsS64GJ|ZW^2t3+`rq%3rx9 z;kU}f>K6Ps)j+KUHyfz8;3g-UEx5^vHsC`3P4zQ5(QYYVa-ze6n`+sW1@Fy*4-Df{ z|ICVph7*J|nz-|-%po{hX-!EM++)E_?Wo-kHEvpQxbi2mDbho349h;2<~kJl)?)PkoXAQs&7 zft>vsEVxnEBoo>#37)@8yvu^usfxNSc&Ean7QFpKSx>J8|6F#!uninI!8Viyclt9q zp(?VKWWhaI@LGX`|EBtx6*Xij(3S=7%z}4k!TZv2j=$M(EIlC&w>2KP$HiIj(k!@s z09?pl-2oCPe@$8Nb_-roBZYKp7Cf2-?{BpD|7JylSqeC=%jATa-<1U~&w|&aaHW4{ zMfIr!_TOgvwdz@u1#eKR<~9rdipt+%!M`#&WWh@el@PTiD7-%lK9~h}tj)rqEVxmg zC7~t@UT?u~SRu#0ISbyA1@Dshu>U_&Rn%=M;8(1UTJR=?_ge7H>m@;b7Q8^!-=BeF z{9P(xAS1zMGaHUs@PkzTK?^=k;X@YuFooNiGKWUgPx*6L5?WLboEChe!iz2VI>iZ> z1#eb(i3N8nf=Yo4`8O?~+2e9c0TcIF@X@Lw-GUb@yvBlmbBi3B+F@Mk-waitZa6`x zpV@G|1s|*OH(2nX;!u+XKV0Q+w&4Bil>gsuNidgGofiDPb+QLt7TnAqwc!6%`TH!m znSa2rBz&a`*sjkMma9yeu;Atzuf&44k5{{67JN|QdQ>9nak{c3vLoLXu-{T95-Zgu34?7i!FGm2tY#qN-PN*Re^E~ zZU&8Y3*N5s*IICM8DDR~|Dy6Y5f}O|WZh;n3$$4Z*d|D!>#*Ra5N);KkE#ZuS@8Ze zTlWNJP<81z z$KOn7N>50`+q2+XEx3NG)S7M!zEa_R7CbjnAm{%9OTs;>z)%+4xgm21#-JkpN-X#l z3a_=`UA2#4KgJ%%dKWJwq}U-r1&f;V3v@eT{#SEo2+!Q0g-nXMMwtlx-Q z6Yi8l&}+dv?~-_*1$X{V;{6uf6zah&xT9If(6ImC^(#39E}4*0OA{~8g4blh>n-@} zs{ZB-9OG{$bYvu8`)!LPsLO&Mr0}Q(U#{>z3%*C;F$?}VamwG2CE>RnvSH^0kX2H8HctUZmHxoB(X2S!S z3AUoUoDhp{%ukT#c2uD zzuOgFV!?MSyxf9&yHivQPJ63p{}O&0tRRZ*J-H(j?*3tsx8 z9D=9?@B2>T1Fe?X$)gftmI8GOAGF|I3LmoIQH9$!4zrpZHD#;V#HIbi8BkTgWhO8; zes$kVN=qzw^A8d)wcwpuIy#n^nUd z7JR$J4ck^r!Y8Ugw*@x|>b2n0I%Nk2EqJ*i)VV2h2+mabT>=OHP4zQRK$cnxEct_M zz+=JBQh2Qezh2=D7W^LyZ%)TK{*S4I_VfhcuPMCKg6~s!mjy4mTN2u9!B;B0--0v0 zkiWqLBv2?EZJDey8*pX8%PqK}8m_b8eaob?THj{z{|8khxFeX!(GLt&pxBad%pYaLr51dZ z!gUM&ONG~3@I4A|$buVfmW12?BpdF`f_Gc+=T-i`EO^X6V0VRDl``KIYeQXlgC^G=TksPV-T++4zi9zisDvgs9;SLKvsPb>M;A8J)e#l?9C81u8WydcvS>2f<8`x^Wn-y+r&&;2Cw%|2o z@oU~+>+^fY{=tV8@;lZKBPILCiNT0*Cp z0Bm9!-j#+IrQzLa_(5rSGz~vE4ew3Ei-Ak~i<`c*ghSE_^rzv6rr`r=_+e>yEDfKO zh7YFUlZSD`Hk6hyWjMiR`=t~S4vi}fcckId(r{-QK0OUDPQwpR!(9SL|Ji^$Eukc> zz>GA!G!37bhL@+|N2KANG`u7Y*VFJ>DgGlvOGqRgp(A;%V1b)0smER{)*J0G<>Q+)IUm_nE%8FKVO>2zqMve?MWjLlZpE6 z$v(a1X~#~HH97CGg(Gaq8BgF}ZK?-%e6D9&d?1P6j0gD4i6h=8p8v{c&KL2hc>eiG zk&A1tw$J)%%kNg$U6HS9X5>Ebl`S%P#kAaCd|``JuDI6z7nSzfis_M`oHp^$mw$j! z4=(A~!_|X&uzCRZF+H6Bhw&q}?44LJt(djz;dvt_f#C4^&KLz_wKlaX^4W?~`<%{* z>$IZ|op#_lzL+p#%T0gFEsm@|?ZiX>@qMbDp|p0UqaBgwQ{<@}&XZ2dv1UEzoqkj9 zJ0Isq-aCED(LdznlI{*Yyaaidbb{t(1v!MaqATb|brXTh5UC&2t`FTQRM6e{4;0dtus!u@0t)PJ@e$kjiW@lK_ujAk(M*-T!-We zf{=9yT3^zKl+}466L_0{d1)kfW$mHk_NBOx-oxom#f9+7*||^sCqMG!$~lGO?4m-B z#uKHp1o$?)3#f zifQ|jNo}+@xJwVd8{2VI4)AYc&wUk{_|sB*&AiCTKb>WtJuh<2PiM?s2`2$%&vK2h zB}U<9HEybdpCP|5_+zYgD6O8+s-6jv=YG2J;O|e$v8~p%tDar0T~w$&oDU&AxA5SH zBgT1?M=xHjo%QS&nzLx{BavU8bKl_al)%oqvS=-FcCh&Ogrn z?JJSI3l`X~m={@e!E5%{3hq9nF6W>LFWDjJyY=8ZFdjAMRGwdXLHrndPCQY7`@PQ7KF+hOyM7CcWD?Kpz%yc_!>5?Xyk zq5Y^4w(WE|l0|X7U@WPE?oGNi#No4+| z$4=RcMA;HZcEoB_nV}fU9QS!7bZP1ITke&-Db|AnCYwKfOJd~R9pfW!UwV`M{{KV< zR~~>B?d^j&MP(Ocq+28hWJ^bcy>t~OPEWGUfk-vjpfR7!AM@oKnt^Kq&9=!8s#X0uc zU6E~m%&$P(JS9RDu(;`1!HN$?$w8)IBXV|}RMSNGU zFdiAh?on%E?*BkH!bKmBhLt%IH*fJ~K5i1Y@dZT(2aqTJdv-8f^eW)$;KwRu7gILL zl*cks?nO$X0R=*=a1w5A<_(*;-r(Ib+_hyCxq(Ga1bj^fyo&G~K=Jz9eu9-3WFH@*!>$m#CGw%O;K5pl2ArhMHo{apV$xGmiBt~jeZK3=P zQzGtbryX++(i`7QTn((7^VOqVD-UY_C|tyZ9s2GiodASW70RD8CE~yKLi8=&f)1Pv1^II9b$2M`vM<%Vw*5%7%Y^+TuZ9XyMsYB9qrlKI-2{_09UOI_F39 z-xvHkSyT?Bn@jLb`CgFMGL+jm1*T=;zC7a_*>i?2%{IR84$^wkwqX z0^6wA**GCo_|Hj^$^Ju&?gOH=cKt}kPwv1&u<*B&B0m2#V*?(LbS_j)-<^LYp6KZ{ zFWgAnZ|E%xFBW%&dT8M)rWXD25Y**d(y2MCyR>nID*;IAyYN!+&#*|pY@ObgH3XFqJjmKDe3Oq_V^p(AYZkA5J!@T0><*kTm}_Q*kv zryl%nF;(W9WYO!mi5G!57@D6>h^%Wo%w9bqaz~@f7!8RP&f{}$iSmobqWp&UAmKAn zTRx;9=7NKpfVo+?X*!KT zO)Ix4|8S(3qkHh79Nm$(p?>PYtC7E4cmBxPBq=mw`N0wA+8u{2!YH&1cWbEMzNi#GxGTZxzaM6sZVJt~!X#&%UH6r5xjt01~uKBhiCkk9XCW&Ay!X zlJ7lA6`^)~elWD-?Zh?0naot2*O^4c=|EnkI6HtvR$jk(?6J5pSuyiqvZDAHG>a)h zKd18C%Jb-uoCo8R|LoYv{G|S@fw;4tsl@s zjj`Bsku^6SVSnI-$elMHF(J6uZrhb>t8V?EadHCQLt%#_@;Z{2j*ld7oH}ymcp=BJ zm60RXO|$2{5LveFxLoa9d!%{Y%6UJIBXLRpT5ouROAkNngqT0p4$dTt&OtfvZZ3;t z?a9Uki8ZKw)riRG^|NLcBU!YX^jDxS@GJU}Q_Q62C1-r$q@3oBi~Mx`!jbpkfdl<^ zq;q|VJ@8!Qx%EdKe=1rA2#MH*IjA++=t{U@5$nK&$0?YJgYa~k`GlVLC5sNjif2P- z?ux%Eo}9WTjVFsD-)*QF8P|lgjM5@!7zz8sj!0#*YxLa?!P%{`>3bqqG`HFpkBxlU zJli`F>WanchTma;9p`Xh7e0Ep(9-we96rTJO{^bHnC@FB|7O&lA^hN3{z5Hs*-a-J zv)Qr~;QQfagZjgOK!E*)f~4DF4wQAb%WAYu?kocQ9w^Hqz`K}a5#R@aMP}Z-s_-aO zyxr#F@b!<0{OabPj2(d_Q~JIu6e3;4*=awKekY$IUG=?9u&N_=&Pzh1^Ku~4+N&^P zn(&Im+!6cipP4d#9lB_?@-xv&$%C|wknt~HAOkU_-7NB?84mJpUZLb2StNOPGu95| z9T4ztE^2}DpE+uncd{C7(~NAqL-(@rj!9PD0gJ2-tSU_4W;?&vmMrQR6?r{SKDGde z*{g3y3*O~|ko|dgSFYgQ<3Cb()_24X1$!V&JMkQQ@#Qq$q4wB)&u8+k5nVJ}xlFV& z@><#z@a}=g^Q}kOSH2V(YMpLBc4TDI#$tQ-xXAGvr`q4x_25|>|7W*f*!W;|@I-J>oSZlLKlk zjkQKI1Rog}21WjPH|IqrZ9dHY&)i7G=E+Ac0cdMH>P$?~kD+4F`JsOdW~Qg`5If{) zI8GPChV06poEK@{e2jg^eUX1`o?`#ieUXyxIY=_%>M( zx$BnWa-YWa6&RQHa+l&-bL&O(-{*k9Hi@;{xwPKN87~6!26qonx|rb~llhWG z|G*8V;n;i8$fvi~pK_9TM26#KhSVgGoZ$xI4Lw&3%46x=Okw=`T;wT=U{jP99~=H z%F}C5Cid!?sG4jZM>b1Vou8ahB+3u3dD8~s+waMwZ(>KHWz#`MeP>m!tUTlN8ZTln zp$jlNmpZ+{cfBp2jOys58 z=eU1&I&ud;PW)F578*3sxtHL&Xp5eED6ZixmrQ$LpQt8T^cUPDD!rleoz;ohg`mCfKXr^hrOacLMH&%jMhVd>W_+X$5%+8_s;{@lg~azNDtoT9)M-igPYyW zytejgtwTY0N89p~-y&Z)egZYh*wes=x7i4NZAA|2C#rIj{9ERlSJP4nRuwM18u?2yn{`4!-nu@cq^X9N@V~H3K9>^{<*kJ z4)GH7xm|kev%cWPSkZHBuLTgS1b&5E*MqSoWnU-GKy1jU0v5VgwB^HEFV7(Q4wVOR?ceo+hLgQ8= zS+w_Suxl0cYADbDddtvAt@Q;mO4S(bX?ZVK53l%9Z~4|PsJJ#Cd~M7t`!rGR3)g}4 zdi-BILJ#F~w#e6mdl}%r(E+|rz7cgx=oRQ<@|ChrWk+`BzlLt&XJABTzm9u7$PNaJ z_TV0hR`zUl@O$s97xa(_oK13gTfW!y;28JXx!#JGw7_>LTZXu&vRKUvMvu}0H^Ir* zlRe&w7aBi{{|h-%>3Vqa2)$)sNT1c;IKdnCxd+RliDei9Lsj6b81QO~pDue=O>we9 z?vbNHy`s1AGf^sWiWta%wEkI}<(b2EB07CpM3Izj@TYy50jXxN|> zWG`VDwD4Z zd4|MX>n%%`eT|}($=4{2n8_=4Lj3w6eu;6(q9^_by;xQ1ZTWFzqdA(1wy_l$9>+G4GtKjdB6V>HZ-sk zfdjx`Uc8$zgq$8!nA8H*Fs;7u^*O$Z_q3K-w6pt-ml1;$tUXK*rw!J(vH!e_TC!WE zZ<9r3Pr@V`8l{1RFe;= zF+|Sc>93;Ke`Wj)6goh?NzoV!_+YM`srnm#pSaJ=P6MZO(j+5zuX~VFC-D6mYdr31 zJ#?F(E!Zct1Hu@5UCQHU*!kfDMe=>#EADN%@8;Yo2CS5R)z_Yf_^jD8q9zpE2jQ;mRNH>(cJ)anDEM4ZPvRwJYFm$9=P) zFdm>|F|3~E7;4+b@(xeK$&$L0p;}(yO8DqMyj(VZ4`(b-*&fzJZejz+^@kb9 zkb!zdPvf4rXt^bMbmPjecjRb+PyI+p?{TlzvSU@m|A*OzS?2Y4a25KST(1TGeUZ(^ zVy%TuHk+q8IbCa=WU0U$3HCy%tR9xM=;7PM=*4nwv)h!;V)DMt-4E)tz%^@81Y{48 z*6_(rtPX~Xv`xR`Y!@zk#~&%Zd#bUPPbsDHR_xMRk6*`f@B~?o;w<`#eynUHaGT(R zwr#FmZ~5^AZPU2xfo@M5^KZ14BdM6_T=Fi`9 zH8!8O8H=2&wW~0*`wI!I*51VZ8$An?Nief@&VIz{TI>JV+JT6>nS;0=aX}2SmAIwk zzaY*~j3%oVqr6GoDIf;-%SMp0Ir~#~J0%0l6%VBE$^#SRgL_r1wKN=%s*4m=P<8at zrfYX&0FO(Nl}1%r8dbl#N>Vk7R9%mUOnwFKQ1V)bI%8fQUx0mUy-^HW=4d{UL(_Hh zVaE4xT}m-l=u0{JaxdlTK+JU9P}wNChrV;2GE6JAGjNxn zQck0CBNVm@`OXTUtZI-G#5Cr`!IvQHei}rlsw3QRU`U}r`;5C+Yvpwad-u;c&4 z4G0amaXgx)jiggU=K#kHaDB+#MQ#+QD0t%kZQwBgzc>tE#~p6cI`Ec!FGV%tK9Kj$*^_OwaUJWP&Rxf6ZiUzml&$Tz`~4DFatg zWbp;>Q7fF=lnUOaR8kd#O7mIq)!;h7XK4ALP}$quMoa-+lwXYgpeV2sUcuE; zU%u3g0os-^Ig2n3HXkXDz^wDoc|A z?R)B9cu1=gmIigk>DC|&6`pr|6vG%U(r}5L_Lw+V<=rkRur;euB=4eJYKSEp63wM1 zhi5t!6b0#*kKMGg@wE)YdCZ|#tcJe*g!_c*uDD6 z|LxfQ|1osN|HrXgh_N$=&VLw(?XG`|;Y$`>@jjTp-{7#n9L)G1!78#`j;!&ue_Co- z6IdMYUox@QwB_U-i|Ju$+cl@h0ZG-1A)GZO_@}m9h&FUh$RzzbA*CL9bEmroHz=hu zTOIDdG_vN5 zHu2$`eoR)F-L!4~QFLZp+NR$$0k=4cn}|^X1rO&sFwtSbi-6LPIgQV7K>Jin{k)=O zeyoO+45;UBVK(7#sOLJBpB!NRHkE%!JktQ9Fef-pdsr6-+z#;&a5t-*hEo9zs_J6( z90Tv@0=W3RD&Q_rcr&k<{_D^O6@974ZL;XwcQeEei+&E676>ZGU2^j_cQXQs6vj4O zyK)Rry^hqje3!rGbUi%T2kG@;W^>?dfEK_;CM5Da|I14bKF^o zOEE-A=dRA52XMns$}8ZoFZeC$$wOM=hSlvM`=eZSmi21emgTj4e!<9|#yi-UKO%ZMq1F^(vuIgKx>WH!0{P6N(-z+@>A@8%g}5z_3QT- zsyTZFl$FZEo;$g$S10l*)f$X<~M zH-uLRm>5$O>K8Z+J+6W`c?(%!ka;*L;bl1xts&-G6<_Vuu0kO2qWohn6eoG}elEQf&cBNej4CMl8&G0kw z^js*RAPZ8)VGS=rP%GdbV&3o)abGHidr%L?Y2Qo1wZ!q1pRMGd9yTUlfU%QNye>e= zqDS7wSVB@o|K~&Y(SQ~>WC3&QE>g&pkOOAKsf`P`#|R~Ndh*aEkb$R`eYsj7HXqaS zX5p4{+GQ@G@noBsr@Z7yUVa(X7k6J8Hln=DGQgA~?%@a+wr*9yiGMVXfGaUnxdGIOgK z575Lhg5C&gg!X7F_yeoh9!|M2wTAt>nSWpoyI^SnJ9s92069lSQ7HC!Ay+$yk=jJH zi%Vg>rSE_n)0=?nF^EFw?ddN3a|-QbuHLe<+?(9xt9Z!Gv2FY#t0D9BpsIxph*~7O zQVvW;EwFZ!+M`?F)B@k40IqFEN_NenB>}rAa>=4s-csz^yC%~2z+{8M1%Y@L|LQ^5 z&YqITKqJD60sISWccmCfT!)D3yZJAqdwdjYj4 z6ZK%F#SjuNh7~CsDIur07d=NNb}FTy9B4cXMTJ~B z_zanD777Z-R6Tc!&cjJk&;58t7dcCh^XOtz*MlUOz$BKEI&KAFHMk*Zda#csu9mf2 zz5^Borh~L!zOx*ALf9I^$`y^FQ7ote?(dN%guPo92EC{Vvdic{Vu5lKpdPs-P;SVp z6I)M-BUAfbPq^8>H@pb1oX`LMi&NTXDt97cf-F`h_;G%3Z^@Sx~epZrDQL zNzma7-6NcLj)WKibkfDmX(wsMm26LFp~aKtFUivaKd|6bn{WQbZkHDL40TTN1^?r1 zdAf%FEOnQM8?fC%TkQ=ku|=lFrW*I-A+)%7{t{c`y|4@Z4lDy3n{a@p)Hi>9abt%b zT3kZzdi2l|tcU~dZkA5&e$^u5`@#^4LSaxoYABk*w==4&3{+Y!HbBm`&Cytzy;DBv^0oAXy z04O-PxE%lZ+%lxM=%DggAMS?p9oPEAgejFLS>i<%4iN)4K>vx_FhU4 zlJ#)-IE)-V*mTKKJ>+qa&n_}LiW~R?=@K%AL-|VeQ9rgl5M4!v?qd9dn4XkrGLSM4dzaThpQ=gt4PW9-PKqH5 z&^Wd6iWB2!Vy4=N;oI&4fb$1$D7$;~6|CYJJ%}K|Nl>TJc=qldfk8`F3NPq!%&oTS zic9OXjT><1HMp~bp*a!w>ULSEZn%)lc?(n;wJ;;pw!s@Zwc#;gzF<5$0m@z0>rn;H zt>gDN|9O)Ci12{ecKsPrTklukpifAH%k2$n zLYv!=nAQo6Z=o#qAxh$ zJ!)yaw_<6@+J9Fz8%?My2BGqXmezQ8FRfuWMEmdIMsXoA+nmBwfk%WC?v0&yyp?Jk z?Z&x}ceK{iKnr|RILbGXzZ|50=}7}8TJuKg75%!lv>&rjH*)c(VUypZ{c0C|Tyex- z*B0Rf_r$kJPR~cVa8q&c{9bnykHXL56}r)js~8&C!zh~8!&oB$i@AsJ09!{8J+x6! zAihQ~o_)cX5Ah*aT)qTwCw3|b-XjGXt#qP{H*|F=cp_#AnwG>tp_)naSHSR%LGEFe z>htdvHVPY}8rndVtK}J-IAh^cJ$lRaw18OtqIDdR8E5EEU3kS67hH1PC0FTXUuchb z9r{h5`x!_b`LfpfplJ70JycUXe_Z3AaPq|;YWWuP2FzUJRXFVeTXlAEV;C7=B&Fi{ zLKiyq<#%B&?J__B<_2q0H@W3 z8KuOKDXq`?pNh{yW;BB4R=_kKCZ4W=X>5XNd}g>z>$6(xj}QeYzo-0*O3ity)5~Ln zkRu0}WCJvAsnb83W|Q3SgikBKf206T8Luin; z@sFs3FFf848RS94&wGagLfkc7X=&m%TynjV_^TM1q3y!mLsbLPFT&6l3soKnb;fpV4GWb%9Dp=UCE-$ z2tXVM*`|^1h#x4D2uB>DG1Hy@0XxyBiJf!=JXt=38GSLP`%|140I_Z%YR+kxYPzA{ zTA&Cv!53c8hVxe*rZ!XSvC&>?aObg{0uSeccbKcGCy99&o@%@_i$FM%&3SH}8|Q*{ ze~)N;HCD5Xh0_hAIe?cBx_m+8s0=OrKG^fkzcjs9U-c{w-avI85^VBTVCV9Ui$Gz; zi`vF_absXaR-x7ByqE7qJJN{;cJ3^A^d(P!p`D8**BK!0@;$=k7zcRH^Nq*dP=@5& zxGKgN&FRx_UJm)eUEw=tV5_**gQpv(#VZ7;RvRL^qzYg=b28FeqnQ7()&u?Ab}04H zuxM=$y2-;c;2a9Rz%6Mf7jKV_k^vCR>QYs4BaNhg6-TrNk^l#*1m{YxC~;Qv)XiQF zN6M&sE1uCdb+4lN|9W}Y{VDp7dbj02|1wX*e4H%0;UzVZ{~Vco!319Oic?MHIUzLB zDVmUDG=Fh%pv#Vu$`gfD8|H0r7?Ev01ZIc#rL{ zjD{AV=6ORC$9m_VQ3%#u%G}`#X%-8IyrFetVHQ7mk@USMBnGZxZ+sEk;VPNkOG$^{ z?TKH26yYc>Df^)8O}z7n*g6icC5xI-2c?S>5M!DyEGv9L13?r4;8Z8BbmN&J*76N$ z#R8Zhdc#Z*3oNnG;unjwae0Y!j=MlwNeN!BVGEtYi+IT0fiiv^62aDI`Y4RRc~x!G z!56Xl4-LKN!#s?Dk4lLXlSSYCQ;oo<7c#?tD7q}37)gfO%04wag(YFK=sxC7>r}Dm zl-Z|yQb9uMo8}ao37xpb8WsP~LI{*g#8jti&}nZH(M1`@1jjMW=ZR1$vZ2ooZ}Mr* z%aT>nMEQbs#bgXQg;k$JaETMR@V|wTL4D)}6+cQ&P>#SOJ{!FNc*2a<&g3ZPy_#j= zNhQl#7L=sM*F5C^;PdD+4Vn-K)2iJ?6cvjG(~Vl%dVF{}o{CWz<8poT*YWJbiaML^ zlyIIwBfu3joZueJo=L~rGCe%0$2Whzd?ju6*Ccc?Hq;rbLnH~oW64v&ADrX75figg zY6S8;6d#&2Y0BpNun`CU+NHI;0w$q3skh5Q=ax8@LwTXMIBofmo9kHrymZ~&rEPM8 z$mQV$U!gkc?V&<1@Jisb?0svr(HXZRhmwEJMZVm7Q*nWIDvfvv7-7$3Q6tKDjnH{G z*?0*>*M}WN|J+oAQcPdD0PcG5V{gyLl+wRJ+{7;8PgCtZg$zP6vB-wXaZHCY+ZoHe zJbt1=chuQr(crTJHQc3OjAW2tiwy5IKwOeV|70HUM^-48`y$wb1~5I4q0aaazOR5h z4A#wGJPFI-Z}BqCY#~Bi3rRC{dTAqenb(xCG|f=5==aZw1*H*q5&6yW$}lA+S$H+` zK$eEVI8IKorQu<{dA9Jy2jX=g?7$_(oF+>ckc2gR;@`rPVpK2n82~y1lq&vszZLfm zb>C&;kkn3M;XDLPH>>?ita*W9i5SC0l#TKRzYGThbyDSE|B=9%q8zmxCCai-E0;M zR^fKE)?%oI5I~niaR1uN5Zr%cS>8>buR{EAWDnkZn8!TIjl$vInRxW}d|;O^7@nCv zB;W|Gi#K>OoB&!pJRHEfz{g(cT8)2~JNd65O`m_Ka6q)cE)?PnncRs(-m6y|X$|1K zx*sQiSL9VzU{LQ+7UaA1*4)`4Rrh#Vy6$U$=)$W~Qt6Xra3tk5!z8-Hlub79parR!k zXf2MQhYS0b<(minsnm?|CI_hZ{o+j9b2G4ZdJEqKQERGs_*_fuVS;-;2e@l#lZbS6 zf)uLuR+5M*gFZ%l;8Pk|xwZ~@CRK_^Xv9u>%pm{#bKz45{tXPT@?b{-5jlA%f|Jq& z7s3IjF%6YaNch^})YxSB<#1T>thLTUYUt!*RBdbgjc|AU&;29My%{1}Je`LLjHN*tQXsui?_`=t~_ci=u==^GMem~vG zN%wPFIQYKL$Pseu|3w6H3s`0)_}w#;*dNN$a+m@o zmO|j=>BJI1!c^(F$x#7eDnkeBR12Z_BIpt-MSPrAP_&2S_+Rn060?6|MBF{*oDOof+is|AP zfHE84qKZa^yUs6qpTHe|&ED|2ZaXJ%#;Hz$8@@o=k=(VTU`_B*j%Sdoy5eIkz#?pv zm4GJ0=XUY6IP9e2Iazf3_5z?+14RIPxPC1VU^!%SqJd=5Rmeu4M%cHKD!Aj=EMN`F zuyL;!j(r_6n`Tc~I6g#0Hkk;EH|YQtZ(vEv;w1+J1BP8Yvsr{yum?ZfWJZgIn#EyM z|1X=o%GzYgmo!<)lu`G0s8t2-k)m2YWs~9mYpvl-!(JQzjV(I0^wKC%CXqJG+CX`?)Y0`W^sd#a5npDtFVwK+DOWu~fIoi#?uOTO3 z&iOMCfbY>Zy$OuNC&gs5h?`+G`cIVO`fmz!OdM+Y=aamFdqT1pX8rn9+WHMiu@wSM z@Xu#^6z8%iEvp_v_!y_fH{7>?j~!h6OZiM*MFeGQ~k(#ecNG!G$dLkT?uST`VTK zh{!gFDbO(9o?k&!fGb! z=;4@sbWA0=WByHYEAAAH;P7xa8^NzxuRvV38FHfqR*yl+Zttyb{%7YFA zO$QY(2+u7S+K47%<3GV@<0LfFDD4~&R)pQD3bj-LD*aj9KpAjqk~fHBol)WXauKX( zg_BQt6$q~)ruvW}y2+i5&Ek%OO6g$}f<_2iJ)eO$;L0)Mx|9pMPF$s_;9YFnfGoBZ z52u93Tu1)12qK)lPT@bYVp!L_UbWCfMUX$9_K1v87Wr6NIqAlEA$VR3ya;sU`f{Ie8>AL9 zPmYiYyb`=yyZKxnAoRNv(;b%|qUI3Zr-Jjf2%@!2#K4h3OxU2*iKevx53lLryf$-# zNDnXbuph-F4Ev6vA1)cpMi3a$AuMGL1hQ)qN4JZPacEv3tlg<uG3`zWt%zxN@JrxYpcz?D37tFBacamvlYue0*_X1C8ee$X^u0GD zC^q476{OnH4dtRST!#!eWDtK7)wO4a;agV0*bXyU2oeXti0{BCi`fej-$`+WCXik8 zE8*ssfKag%pc+9+;A!JV1R$sk^RGwPI}drNE?q*_vAUo{h0ZQ94^NnCjgu3wjDid^ z(?o}XlM|!0)*jX>k4=n*ofXF>zIufHi(@UYEBz;QK<$sbP!1>w9a!84nl6Qj(^Shk>nL#*Mh8` z>iItrg5HQAb~1L&yQn<06!)K}0@|g9;FgGJ?;;h6Dv%*!td)qdUX8ui#B5qu4i7e^ zOB<(xc_riq#XVVcCOS*iTY{WIi=>H$fRS5_^wxMQKB1;js1OXFBsN`Xt3J~nU#yFpMuMEU{QmsOaVRJr1 zrFchP$?#uz+lK}b5w!eQ6WySk4h=ChS z1g*)UcOH;%sgT4J_tN9!k24F#N3@F887$gso)eS(sJs;XDw^OAP zC&5MAo#)0aHpg+|oFgNmmJsPGx8Wq{VonB8`T1LT7+djJ4Sz9k9O{DX!j$4w^Wk*> zEDYBPdS1l>yv)(txHs`~hKoifiP>BDh1d%|tBRb#$*ap>!bePS>|H4AXz1rUH&phL ze^6zyJ%qVv1uQfoGHxjDa=`IQJh!tx(1@wPUnL?!GP>1xDXA5P-_#K-Am}f}zRk*7sgS;AdC$U5GRL6IG&t7 z4&WB!^(jLw8^@jNfX_4T3b|` zJv`=6e6tB>Uh=eCqqx|$+wo@PEB;bfZ{yLau!*c_NK~|u6+vFa$1$9O;5GLb7{c0Y zMR&+f;UUw#)Wdlm{Qpc0^&~r4`fGu*-VSe=)?j33V+)!0=+X2On7) zmU8kBwGN{FY?~>mqtj6SjNO#sos=yk4$A2Q!#8?3hw)<^6<=yCWRc(?}@Uuetm zb&lP|-ej+T5#D2!4M$HTzt(aOylQN!k#~=kjmYcKgO!h>LR+P_?DLA(*FCHUhZ0>v zD|diw>0LqATHatMLQU|>6T&O6I;Qm0(8b5l6+VWpu-INFz5d0ixk>uN1s*K@!bh8% z#D^d}FV>j7_i>~#DLu)Cg4n4>&@wSW$e5TIakwDLRYXm7j)5R6f^o3;n4ZK@gUeAG ziwReC$iU19>nnqu{*zO!V`juVH=j+T;qDp_RkPmu%|eL%JF@t zMB(aod6cW{Q|uw*4K>6+ak^n2{wR`gb=XbOUy6+MLLq8V@qn|@ppv3T#SqfUE1vss zr7i0h*N5DC0iW@?3-K(sOoqj2y%;;WxLzJM=E}nd=8o8r-BmyA4F87he=z^ zs2S`=CzM2dd=Q6w?m;5?(+q<}&nrG3apE)r)nb%t9yD+kORh21yhGa|ty}BSaMh$? z`oA=4pTPGvKlFrXXo@4Cr4u@qK6{qpnX@O(<#f@Kq)nfkAY#w`Am$(%ZY0~{%W;!Y zp`1S5PXmW-WEQ*Ll;<6?0qye+((4HB{I#wsDX zI`_r!feTc}z>4(e@J=QM4*HQeak$5byNW<{6j|vgjlgQc7rY!FCEAH2*t?v*3dRXk z)l?&!|7?;_%gAGG2;Atv($2pb zXFi?2;D7a=_w(RCB#Zjc;rJR(2?%Sj4`OoV(BcNGW^pqM8`ESymtPzf5>-!03}1tD z$@}c^rhVF?A#v<{Z7$l!+ri!RGw_35gJs$3p*$1)1bW z0jMQ0WKj+bOJViWXp3pcnDBk+z4Gz0Opdqre3=h`wD zc4=-mz}(iO3i)k=e~E7!)UweRX~~DRjveB=cB(_HJ6Xg(lA2n-F@n;849TKH6kMt@ zQ<;-Rxj+MN;)T##|BM$xTNmyYf+nqDj(-yDOzSLQq9N3dEMxE(FCqrUcd<8Mo>+dj z>~ti?TM680K>t6(w!+R`tcFa)iA6B+=lC11i2-vV%s5RQD&uTl&nr}+feqqkYACgH z&4{u+@$;Dht+Hd7mkW@Pe=Xv!_?+S(!jnUYm%7&Y_@dgN9-4yn0ERYvZL;Mfykkix z499F!V;mX$YWksZs5zAJbG_xg@AYuri>tMX(Xu@hgUEufmo98aDqcd3inYaXUE=6G ztX?nbBROnFpY@{NvI`&Z`!V<0b3u!%J<|2{IYt8piEcl;C=8FF!aSIQBSr9)AskJt z0d}{a$#|wr@_9f!E6jRyO1XKgH<@+QDSfzabPuWby zDuuiitCSL~Qb?{`rQj)whGj$Y#?;#qD4;F!@}TQbp{NJa$FY%_LoVa~-%}G0sD%+E zJ(v^!QcUR{SS5r9L3FIefj{v>&T&~KY&PR(s5Vq=Gjanv7>lhf>T6=-<3C+`YBN1_gy zciXPLWKAR6WZNPd{AU%yFDG%xR*!7_V>~_o$w#$EpjyljX#aJ#&sxh)Kt|zzqqSx) z;^Bc(K;Fw=tkrYX-U0@>U33Z?Rp=4xb?^$Bow)Lss887{u6xlcwivM5R5|fx4PN6G z{Bz0ohA|EDBo8FRvWR4jTq}t>utmX^VHSP`HfVupA>?3N{)PB^(A%5QY_e#q*^%z! z>4oejQ5-PRMMC>n99TcD{84obQTM7vIH1s@_ry=ac5EL64DSHrFrE-QbvOa!58xS* zMz{=EZ_E2)ZJR7m*){?wPpni1m*!}X1l+ITel;Gz?11|XCbTUT5lmLg0Hz?t04;d7 zg%*{<)uUoKFxFb&A%G%w{yc7Ya$gsU&LIrlpbJM}6gg{p0lC5pcm*BDev5~ez1W8@ zTFbFMiE_jD`#<}g8tNtUSSyy!G&1B(FR$PYr$7F<*Mkiy9QDEXu@I8Oi)%{|mP5`? zh*Q~{iE7i4h7ZibVnE3{+^F}Sv1jI+Ew;|8W^?D89tG=%Gw;Gy8r-aPZUbKgJ`$zq zU%Q0!wJhW>$6?=#%dwFAz+b7g6(2`_A>8;jhWk2%jzHn~H*P?uEFt z)(P-@#NGg)*!er*XgsSbpV;PwwV|g z^zznDHDHyXROsdrNWd6)5%#}?S{_CAE=1U6Q;Zqwi7PpPOyH0A92bYWHwX4f0$q8a ztL)839rR@>>Ks&A$*d>QXL!2!P#rSAnfdywX-j6;h41Ne&M4?_R+-@{fEQ z){LdDVbz$Ht{OA&-N2M?96_~Gx`BXZO5Lz3NU2-7&>7GG(FeWPa;?wyq|U77Xr(Cx zLeXy?%YnovU{fqdDBm=Jb09+aBP`I1x9?PJo-6XJR~x7nXh^698^jME-_7Mq2H1xH z$DkL9`5MDe7U|z|l0By=r*F%d_ME9k@f3zQoJE5!enMi-%S!TW*NqqF03U$X!)inF z5|i5LR*(l`Wlt#-QwOePd)hEF6HtapEcrza$S+ItgcbSDgT-{pwbqk_oFZDy$Jnh= z)+W0YpMfMnI1wiEyY0ur`x7FhIx9p7_bCxlcxp4BJb=H6W3}Xy=FFiRK_bybv}Qw~ z3#Vl$g{DW3$)9q;MD(-=7wwVT+|Tg3?IHJWTp1c<3CU63VCA;72FBA<89cQT7WJ$p zm2GR6dV_{;N&^H!bWLhi12-qM0R9q*kOl)?O(tOrhlmLpteZN+s zkcYh7lJ|zkyhNs#lDFV-C$8+*R$ifGUAXc`tP2cDoRw*Ra9VJ8G1)8%q|JU&6cWag zs=5E<$9g%IY!^rLP%}1tkJuJwsj%>_rq0oI#q{y=D-{U*go1drC9i#5ZKkdAtuK4x4?;G;ZAKZbo^$ zY3igMWlL~iFl5@cC3v5+C!U8dT#cHHJ@IX0#X4j2oGjlKAI{<75Numgke&m| zY#Yu|D|3uU&%sSgGY8CEC%TNU_Fd_y3dYfCba5OkXfrZZwtT1|rEw&_w%rqh(jdxI z+42#%&^RiE6azq}C8L)Fdn!@B5>?~*%rQ%XuT=IV@ccfXFDa}Fz9*7DK>77!gF~pX zfh-(Hs%{{294dM=2<9i3i64ACD zeNZLR#(CNhvE!YScg4J$;6Z-u&mw%gZSHtJaBo%&9ZvJL%Y-o#+ct|Y6^Sv%zMSBG z-ofAnSAS?_!NKT#?N-#@rjO<=N4ZTOv&&OBn~xmeHhnB_ zIl65+SctFsfW|f*(uX+{Y--cT?edHl`8kqpI>?r%SK9Q6yd^K&^dgbp05{t7gLZii zo?X?}P+Zkke+X%FwdsfQTHK}|#(d;pn?8xROAsbK>b`Tc9oF-Tqf2nDL_Jq>}j-xgmV3NRrfCl{K z2K;R-gpE)xxblC&<@mmn^k2ksQW}_X1?KXe>qgpPt@Y=8LkSBrKE>|B;w9b$WOxAr zy+cZ(0)42WuY-;g;E(sT$tK-2xcsH)rZtz-8w01BvobGzegnJnFo#7^0uIqmT-bp@ zz`>QT2AAWbuX3&(?tr0y}zQ=((>(Yrl6q6fv6y_!G9x~n9l_# zXKFcRh?{Y~arkso-ZSDC z*v(ZGUZBTI2PBqHu`&=F{{%>S_(zBbQWHmP@r=rG8(cdeoX15XDCY|y9855dZ}_3r z7>LE}n!rUYrMaf&K0+sYh$9X3b?b18Pxm4}7ed8sVJlO^t@Rw2N5a?giD%uyV-P(+ zw(!wjefLhz`v!a(#$_)@d<(O115QoU;M=dSVl$xtN1#wnTioaU<}JD%Y$sVX=T=yn zj@GD8TQY#Zu+jPfb_u(a$ca~`v2uxl+4w67&HHlvf7Z73;Jv~k#GLs{h3@&45|<(z zXH~=j)+Wdjy7hJw+KBV5-pm4KjyDWttOBs!T?B22WDmO4KsW zYzxC9-=D%6Y39e5+&;wx3-TwyviNo5pe>)m?4S#xQI2c~hM&FQ$JTEF9$c0iw9^IY zz=LRuy@B(y2;ULGi+0;kp{4&)VZnyG*}YW%VR_J29ou5%7p7NUKU_IpnUNF%}E1T%;)_@)+q%Wk16Ox}rZ z#n(AO+P3^(f)Cp{!pWiq-~tzJ_Q3sH9vho8-S7pw8bp1~O}KzS&;^YqvxG)>ikoqy zR!7(ny+%F4MB&O`H;UQfd$lww!{_Imf*mto*!>JO!JG4muOfjHLcGWE5jH|&Lb*1) z@G2Z$*l8!>{dgP&NKwN@D*Ha~Pn3P&q$LI22%u&Z!U5?3KhnionEDtFvEsXr?^kN=QEl7Aik2OH zX2F(kN3NZ?BoJ-X@F}e1ZaqhqSo3)NzxTn!2l$~W3>3uhj?t(?4HQNXqz!+ijixX5 z+M|$z$8{ueJ>H0d%6RDEG3V&v1--(s;M`sHtZ$Zu^TaNS9-ipI0vLbP>=y`nTiCfD zH#w<5!ziQZcN8sUauW_9*@+N^-t$qup7R+sSIlBXczNb8 z3v89iXRzMPU$m=go^`5O6!gu@Srj~DUf%NHx_ORatwqeGYMyg-RZvKK{80qSFhJvU zBRn}5RcbAl;KhS71dp9>$xUFpx-#~{YWCGZ-J-7#4xWUZIuYgzVYJ3r# zrVGMf{GxlI1))4Qoz@206ucq9)dB8Y9LVPXhqW((kNJ52&&*~eNNkXmxPyea6Ql$o zAz?$}tfQ__q?Dp0N*x=LU>jSvQTI*1jykIAj1Us4j=Hb9kI!27t;qi0&wO?xQQzbB zd;PuI?C0~$%rnnC^UO2PJTsqXkcoUd#;&wu+MD2z?nwUu@-P=21`NWvRR{s?4unI^ zCs1EvVv+xn=pF+cDNP43y_7_Sm^@(*u^LiV^`Xy?D6({JP-$!3VpQ8gIflW+V&J7@ ztF?m>cFg}m=f`A{SW(AGv4UQg0nd8_AgvYrDpwH2(t3tkkVgV{H6UX1h+z>ahaC|w z0;k2woDk=|SgXdqQ%SPi@{Ut-3?d?p9Hi1Yk3@2)4WnO&{*?+@^eW2yjJwm|y>9e> z#UW`dI~`Q0l9oFY1geT22U}x8yzmzSh=sQlDVm$5B;)Akd5Vxgr$A`3F_=}Gl4g1Q zPo9405Sp~qiGxHcPrn|-l|aNa3>tSz0v0d)I%9v4CiatMX?9W{K$Ls{8!|Aeffl1` z6P{wyc2Ji{ujpUsDf);*J+~M7n8Xf;Z*Z%aE;X?dEx|0ChXGBL&hm?MLZHwh?z)M8 zB1j5kT>3?kgT$Bo8u1aB4Mr4|l7Ls#=VJW<)j{_d5#uxgkH`TvBm-}dydc-dYKWSk zSTVhkP34Oh(6N>3k1!8m(qB;*hz;hgq{yR8&kg2XNqB1Eh-c8+?2TwJHBXpGE1aQG zWH6YL2gnE&ctHb`hao#9U;rl56k{-t@ryJ)h2;qCj6zaJ5V!E)WMENT$Axf|s5S|t zpw21yfi7~?ff1Zc3Gt?Nl2!IBISGLw?=-`|jWj)JBKM|d%|jmY9Z^V{W{SZyk9s2> z6CNQ=YxdXyF^^~QrtJ>kYXC(bZ>EQfc?spm7L_L6WRc0q`~7enKXD0G!5VZMOa$wX z_~YyXV(vdinB?&H+o(niFdzhnR}tB<;4p%Ps550_$4vPsrs^}@BwG5{cT5GX%pTw1 z757A0X+|)ZUH_t2g9*w_P6fD^QQLe9#>xrFe^FD9B_*A5)kd4^)kdHr1L<7iynMvzgAFC)2DSRrPs< z{Xj(WElom0utMg6b13-a2SZiVD#1z@jt)JYcq5wOzO%RD^Ac>VF^@g~*Z+$;Kux)J zzGsr-XLkHpM<57h&AB`Y6eALriAJ|IRr)Eka zF#V*CoC$IiAVQe^SmKR*8uy|M0 zjBQ1x6m3Nk#xZ_X2_Pq_ROKk^1v#c~4M@|G)6|dT8k%n@{nE!vYs(_Rv56-tGI5O39ELM!Sqlgq zh)!(-L^{h9Dve$F!Y~YSQpgkv2i)8qI%&F0GS`cyRUkDRJN}PJhtdp8szg^bc6U@{$5DKjbMe?PX>3{RYLvmywO`WyihQYJKVj&dW7UARX z3m8tN$h8MlU$$XFmXWvYF}E*{Uk%4s@zY3PNfYF`4*8xta(nMDx5sjOGFMHX0!$mOD=>mfguYE zmvJ+rBW7x{xfsot?A65}Q#sJOaCm}6PA z5w}1~_1@Yv`xQwd%(ZD!szWn%$-kgNc%37wKC#nHr3tlRPJjpdDI+%e(i%UFG3_P6 zjWoN~#oA%As(!Kzp-&R88`!hhsxPo^)(rnTI&JDz;$e(Q8eH+FpC~7w#;b(O0XBEU zn3~~8xk`-AP2hPFrsqx2I1;d;bS0PATv`K7fF^)lxCt2@HxS83CkU=CDP^VzFC*n! zO8HDO^aPE4M4qO=CDbmAS@nVR0}VV}!aM2)g<@3K{&+34TSFbZE)R!womFZ`$m%(ufNIwFM75ib0b?gN6ELxv5dmELUs z41kp*tpI~9O!*K=LT4}?p&MkoT2DeS7y#279yk>TIBl-CIrRe+LLm=?=io33x8z2<4p_oMG3` z9VbqdWBQ=&7+l0b>QiO@c$%Ye@)P5cx<8zLp=11T3^aba4D~GZ39!`JV?TMO3?-`{WY2GF4Po#u9_p|1aqO;VE#^ zF4g8I@n=T?e>#N}%VX6oMYEMyFpbO%n$XDhA=~DISs7#gl2P1@)aB>|+^c4_&Vk36 zc8XKUU!6|91X-@|85p{?NpQ>4{*yx``nEzYNpX`VTwNmwpFSupkoS|#GQfHz{>)?Oa;`|N9*YzK{ zikt!=<&4Bt2OvC@sX+-!ZQRj7!VcWxcV%mShBd4-|&j7hw_Q}pS5S>l` zk92$Mkl`(9G7(V^#bDHWthInEt@JodjZ6DBx~v_%trkU%hdOGz}SF z1ISUjNxme|c|F+cvJ_6mLz7VAWBk%L)1(xrgs*iE^#?_@&ea4M|8U`a%7^;#xX9nP ze^_6e|6AV@75~0#XD*ywBUFPw9gQ!rtK7|Di4D{{eZi|3lT>Z9;@U{Xd}o81pBGnQ;o? zHv&JYVTdA5CeSG~Rmwa28Pdk%V#;$PT9VCmjien6gS5ARYrHMT3e*)}YSF!vxChiF z`lD+pPm+jXFUJHNbS>rIA6-jHU$;5+9s?bi!5ot*4ol-Rxq#vL2HkWUew_q5J#E)} zOTa^G$dBNMIV`o7*2Pu}4`0$l03HJAp$r~M&_i)N)WieKYnun2W9V730b|?Ckmli! zuj_Cgfi@;&(=pz$uS~9uBqII8=s}|gDANpJRcGvg%SHP5e2etUd727#h0C?Gq3XH3 zYxE4t^U)gJNMAM?Ol@e9uKTx(bh?7xVa8o*3?k0tsO$8^?t5XmKz)V8#7Zp|fFET* zOJv!CEYaymq<=})nGk)%>9zhKhX7U{fLh1BwHu+2aQcEQg<+JWz8I{1@sjI;NV$vL zc^AQn6`V3F&A>e5g%Nr6QMc1LQyQ%Rfe;>5F{=)z59iK}@;;D4Q~;`06)<&KM$Rds zl6pffzdqLZr=}2Mr^a_6Ml4zPglhTpePWxvs%aNeHAXUt<41{tl?r(?5}g^SM%fk* zj!Yy^AUX9R&C+_#)3kn#(6nCCRp=NBI0+&tgRV~k3QQ6z>4?Q9jcTZTng_1YJYd3< zMDqgX0RsU~;3aLWe|BP$U`T;S%V0VRu(3H7GXm7(xjQ|l;aQH0bbmZ4^@EYU6JDPE=8<(vZ@+N@~KgBGA;(&j4sC0eXEi z0DOtI&s4x}BFhvgDq@^w1Wlw!}c~ zbXS#>>7nvVZR9(Od}^vYWK&r{upQ2_R`~@c^h7wAHyyaGLnt(h4AYOeZIX9Hg;_t9 z5<7rvDy|ITiMz9Yr3GQ_2dv5;WZ-qf6WlR}Ko7}V28z+}Bfki+sS8;=qe<{i$Q$I~ zgP~03kd+2}nlDE$-j$}rO;1M);3Yu&C^qK#S^&SK7qfn0G&qt?`EIL#(gdhXJUb15 zgIqK{m)tALJSvZnCb>&$B+E*BQ)6={uV(cz5I{ez`^hChsg4yP`K;{vF9uOhIWiz4 z=@$&VQa+fF%$~)Y!X0YX{t=E$a1l{rI`+09BP9K@E#|Pw;E{5ey1xq1gJq>Jl$U-0 z3R$lpvr2y;(i|O&;|$8hn#lheL_53MDayXcr|hQGiArP3&4*) z2BtxmBnOImQw7TOM;Vgyd8iN0die|QR*ca>mhytSskp2lY;zd+w32$2tANKX0GLI9 zCm>5UAl(m`l)W_o`-{e6Vnyh=Q%m|XpBI>=~IA_84EfUu@DhG!$k zC_j$CyOKx*%tH3u66jq_-a>QqQeG-#-hIc!EZ5H*`* z>?xK0pn}E?mPmG}NOt}MNd}dnKmhqU0y5f3{^u1JqW)W4&}5y4{Eo(5QQ`fYZmS<2ABQsu%pmI@if(7Un$rkYcDtRv;2 zVQoQhAh+G*%11{0C#CQj9;h#J=NJV5%rUpWP^%`6D+;v(XF7$J`Ls~DwZRN2SCT-Sm@{2a?xI6%qtx*{3jLzQ2LkYw3ZQ9C=|q6eblC~I zjArn9V1z9Sfb<~XvsI=iKf-L*ep8J4A3fQXq{poN9{K6DH@$)r5)2cS-jzKNAYEZv zU^(sGBi%_y17h?3c-^(VAB|yn@SIJQ9^m>@eWQtm zX@p~ah%W}K5(%oC^lP94#K(~}zdLU$i}+0z_jO71P>cIoyAbw$wWYKhLJLQ`U9a>dMT-^WRXkYxgVnM{>ejAuqd&xj z4y0=+#p6`cM|mmtrThP+?%{c>XA2ag^nLiMY5x= z-ZbZahq|oO3I9>dijI`&l=9alFnc7WY(QawNeT*fg%#?vZ)e@j~ zrxjG$l~b6=9Bt14gOm|t(Y!gxmKWHIfMg0ND?enHLoor%5NoU(^uO|hnq&Q+ zu6%!sMG%@De93x+_=2QpyPGxqL)S}83%&#zmpkonXl!<3GcAzHZ%HGy2#r4miH z+_58prl3H5NJj5kTcWg;HkF8Ge32fEgLBpq=tp|D-b6bc6w1R8N_FBz9MI*q9z~TX zw4QEA%9~$Kp$j_$TP@1c0i;NjH>ko}Xrz40anzj4$ZEY&!8eX-X-p-0(jkvSv=3@s z4C3SnGef!^U{FsaFLofZuEc|bWIu?p@;xzi5^1Fq&`ORJNnCGAiKCR?Xu{j%plPcb z-4Tk}h>x~w(uhmxe#X1t>W?E~-_l9d8P@p?SrTXZX&(JD8$ zQb{t0sZcIU|3>fR&ZqgBhZslpUXV_Ro}XR+oF2)m0jlU9Em1P)ArWyk-&t)^Z=^W9E%*g$f**7+%lZge zi2&!yMwhZg7Qtbt0mw^TK^vRav4G|!^h_KGO$1P}VP3=aW0C=2v~sB@QtpsiAhtNk z(WUe!PSFTLjn)oMwoy7txg>-;N0%*#3L0fenSxv2)NyCmyQ8(Xj+9Pzs0`*ts~Eff z5n0=gC4*daSXVh}aMMXl?6Yru0sZ;Hi##5`N~q&MsftejY2oAQT0ir>vH6BUyJ6Qf6u$~0j zR&43xMq2!_m8Z@q&FffLaP1u#+MJY}4q}0JPDLtcp872A4 zBxzYmwyz}Nrg>laJAnEsn|wYQB<#tVe))s@zLV>v2cWKfc|xn?l5BGI@D(+RYGkfxdJzpw0!_BkwB0~)6fpk zM#)P-v$EdtA}#7Aju#ga`I+>B8hZnaUB8=5XDUQETV6y%@DijY`(r0E%GIpmg|;e% z=}!Ur!|2aDe}|CGaS(>iN&*tciZL8S*!6LQoq7@Qd`rm|lmOPbBin~k`#26i;& zi|j~|y#GUV$2Vll=a24q2OtugwIE3@ndukyE6$X>q)!OZ%Q8}3uhueBU5#u6smrGL zyV*EmhOfSiHdk~h;wL>|!z<_OLo-z-Lk+2YI1cZ&QxtEBBGrvUJ6B6l?oaRq)OzBr z(^_pS>B<)(&8d4}y&*FW9#Lkav=1vHcQC?CJMxE93osAj>qPL4n)e#NIPB`>4EW?n zAE^y57MuT{u4fOX&rga%Il33F?vFmMJ)NF`Hb)BFz_uBPm z0EL-58g$VhL0FcAlKAd#psWc1mxNwK`zAaZZLOV;(%B49togftgrEZN!6BPuIq~O_ zY`_rIqq2Oh#Hub-x)MzoU2;fv#QqIcU;x*sf_C`D#4K@@!54@|$9XM?lCl=YQgrfx zu9P_9ArTuc*{-7HK#OQ#f}tptV@Id8=lH zXKdPdoI=NH>Q|)G2gFAAgkE%m8;&i~#+f08ZcnpkrnxfWS|N(dZaXh;FG5KKllF}@ z4|TT{pxez4A|}^;HoSxy&Ji3L2iErlUfWF?C1l4W&Q?dp6iHcOLmH0X*!7{v6O-mI zjRqrG$xg`v6YcskL>Y}HhU0rmGW{Y@K7x5;VwbZ@qOd9s8@~a++kIIU+tyry6ZO^*W_nrQMF+kuK zlNPoZ#~owR0$O$yq7`dBuuey6fIYDqY$BX~NN!LNLCeiy-M-Sgq!UMjPtq~GdMJck zx)gHKO6L{1fnaar)Han;5p2wZpk#go&pL|MtyxLWO^)0~yZ)OF;@j4&c+cbU`KtO^ zB}kV-KG>I${RqVybZD240 zq!T*x=YU|Pf`hrZRxp1BEpsbqOclf`kBZ0|0{>CNYOj1XNPMW_RaYm`H5H z>Ue4)bIRi#*uT^cX)k_lgYR7@!3`a~*RK9UKgXjfFhuffIC_H$Tz*a@q-nF;LmpFK z>r$`b9lU&tcbqSMg&&uMPBEb&FY%;HISzS4?w_2wZy+}fz(WAg0IY=Zw;RuUMaSB# zSX3<9D`#!HemEX&uW|fz4w6Y~V@x4)@d#{9@gwWba9|q(fOBK@%l7KV>a#Yuef~VQ zRuyETtyOHS(zl3pkymL$zQ;7}U9^!9rTC%QN+`B~bia9)ph}X%is8ovM|DF(isDI^ zat~-|`?NT+TFX)Xg2+#hqIk*DUI8v<)#4d6pKRA8m*`VSplsO4!)^c-{1dek1paXp zTrCU2mWNvIBF_M52ecIX+OfdWonh?Lp6Zao@I162$MJSh0f9WzRA#7KoH9U3HD?4EvY#m4rmar2AV-nH%_U%n|5mqDoOR(R<4 z%dOz6%9bEC|4wv8KZD&P;1Jr_FUvLH1b(R*!A)c{+O2U{+qoCZ4W$*@1i`}$r?hn- z=<-PfN|GE+^g*-NcSQ`LXTO6N=A1nsI!3JO!2Gx+Mr3zj&G@s{qHsr6k++QzwL7xF znu#D!nr{7C(Zg#)@&R|JfP4rFfH|z4^21J%(vd}dHi-I#780eXX%u@;T`}Nb3?fgf zLAR~~n>5I9Jelqt&5joKjx3uOXeADP&H^;G3){ux&siDGCOqgmu{5uWg-Hs>T{EO5 zDvBP^e|2JYD*Iti(_rohy@L<^V7B<%x?_AbL&dZht|kq}`v$=~vj)6bv}oL!wbdN1 zRA5$cr*fVl#%!`BYfCYw z3ma6p9#W-waI=Ld+?AE&Yg>rQU0Ex>K3fd$%8r$CEe+Y2>W9b*qv!e9+x54RO1fW^{q&!VVeBj)h<%Iv-9Q7MLLL`L@5;4 zA(x&=q87=ti0T8`m%Pd@%d~;4xu(dA zN?_iEH(sbiH*u@j9LGvnUJho(8BechDLjPDWSR+2T*cC%Y-r_!TI{AI5nJm%%>kL8 z;#ky05i*Q57)UTK9ci3Rba*F{7U7`#jo zs+C24+ zI6Q$VWe44n6`}@bg=C50{Ia}bO#m8J5TC&cAc z)~Dj$r|6nZ6}H8gy34xgs-u1xSqPT#PA8 z*?oy1VPSwM?XHUPli6RICl9owIz2+JK0rxQos%wzDO1>3{#b7*U}E)Md->cWoL7|Z zaA*%T`#D|*sIOJ!D`tW#N@o>fu~LoruK0@bs|wj8A$H}=d5K{oUJY2mA;YXQVu6`e zDA*-~*rK26Z8>OWUYtk&Db7uUNd9%(DUz*ES~SyH7%MW_?&=aT;k7y*e-p-;%-0e> zgB50)&N;4@gqf@~n{{=Xu6Q8~o8&S0u7SK#es+C{LIq~A>E0YaKGQeE6of8J8_^Co z$>W^zE2U~2Ci&zs2A2dMykbuD{cs-59#9Fn^TRSfOqKD|Mba!xxVPNJ@>y&nujej0 z%x1y9)}FMWg^L2)X;;-*wwFD?<|Y=*hJ+`%i2`$2c^>B`s?1^KiZ^wW%GR#`3=f!v zl=#D<>l`+ck3T6+%waz4M~gCteaER__;FXl zbKH3Z)UF@NMS}&bE!T33X#tC7yjoRpdLe`N@`z}&h}9~Z(TP_5{lh8FDf&&>B+QGL zKU*obEMmcBmq965jhO0GKd3HcFXT=7m?!6G=PqWI_=Vp^z+zU*V;LjDF|W3>Yb|k$ zSsc@3K~d~m!hAITE_QKg2{ZFs+2ZS^tbDP|>>M~250a;V%v+_eT`XA2N^8!d#Ezw` ziq~M70Yej8m@kz!e~QOTF(Wt67KN9AYZbFajb$)uaz!o!(bu ziFNo5oRXDewi(k-w@$?#2jE z@wZd%V!zUQipqV10tQnFiHKI*$_8vz$`3u2jt9iL<*a5qf0-o{H$@yxC<-SHy2`G! zZAbo;SGId<=Opx%9W#NHFDl!-E61+f&y+PfJGU%K*xZ$~nIdon3t&q`n-$EbY!~Dt z2htSOZPyodl#ihDV>2xiS71!>@(0C|m27IE3g45;>Y=Ury$D&w{>95?itDRbRo-mB z@L0{h;ivYAF{{}GpLY9z6XPh;fM7K8gbWS|HQFbNuVJ++Hh%#UrQzAj`pat)1^C`! zU{(j5hGTb&=(`3&J}X1)T7&)^l_Acq!GzN`L)h0a3;!%rY+uWw3b!38iSg_*(sfo+ z&oYHu;sC^9ie6fKw5a_vYgQ*2J|et(?2Agx7}fNt?u&i+^w#rsA{G1m5aOG9?il6j zGNNC~$wY*u>7wmEQmpuyMf2Y2;^EKCpWjQjc>jxeGG01ER9eSE3O~nU9L@K9H$v7Z zjO$oIetM+%VI8Yqr|z9Rwc&mmwYE93h+5k~5Imxnv$nzs;@LVD=K2T3La=n^Eez|K zH($D2^jeR(X!I^|W<8t1=l&!HY+zl*u=T8waVGi*pGO}bQH_qI*p+afR~G-E^~L-Ibp9YfG1 zrfLR!#wmEBU=+W_6v8p_H-6tWla)25nqthZ?#5_@)v@ox338rzQPxVHDw`=H`AtvU z-jq%5De79k?wM3yLf5Qt)SC3BC-%Oi6R88z;eb`q7Q!33tCH_Gkdc?)8q(URMr!Dx5p|-|I{O;ZT`H@m zWt=TU;6`ppw%7b_fI+t+XN14l1TAd5vK=EAmn+$8CN{t~ytvW`J}fx0po4->DlX5- z5QSR`ZW&Et6MDopULNurbdTtj8J?7VB0R}nXhd&k(Vf%`W&g80r4#ncU_#P4ORQ{V z=!@?QjEBG!`hp<~jN!InxcJPlPbr_+Gx%ic$;6hX)~>nyl~rLn?({oG68thc8a7@~ zU^c&83>u_|$aJVPnMoOckkkRx>PzJ&2B@`(G@=|eGVG7(wfPHdga(!Lr=CnmvK@p? zD>62)DwSW{BQdfmK2j#H<&~e_@09kZr?!>)US(FZGCGNQ?e*0DaI&yI2IFmm|FR5;&In*~@jM;*D_RUuDVhdZtdk+wc zx3X>gr!DIH^ztI-SN2Vv>wo3v=IJIft8q1<>Ov8AMZ!F7y zW1AT_{b=d5o%LtDqFL~g)*>(Olt8^ocJuA)h*Y3I~sv|yKt`}#X`?gzTA*2cwM%L$>}VZ&ps`7rn4Hn z$z`FWvzlCYS@`T_YJ1Ll3X%OWF#^%Cj(FlGJuUM$UG6^ag7 z2pZv~_mYQ`mAHi>Za-VVxtd&j$%1^z)A1_2_Q96x`%acOy5!IBWPXl)xD0l%AO}Nx|UCZU&;W-*06BhZ0xV)Xm`N(3N_#CNh`DWHaxoQyTi%V z!cXEsCL7Nq9LZPac2w zq_cL}#5zCa{wbhPCnu~L8Y8ID{y#~MG5U7kflMyzEEq-p5ESXFi0Bg0Iigd<=h!L{ zmk+U0o)1rh^7E4N1t*2(Fg##e{uF;4#@cq?pW@D8HjbD4Q}q9x&FedDCMv^v>X$sk z9+?edTZ7V~KT1k%8=aI^KduafniM^G=e}3G=>&F~TJdXz&T<$Z_#O5djWp5yNj=rM-m&(#M5N$7PR`YkK zJM*l&PvqG1E`?@gAoLdut`g6t-KLW#jp!(mlyeXw%?3bRrynT}P5 zJCM`M=XgOxey#M|(PFqc03?h!5+dy*xnt;M*VjF&_DkPsvP;CZBWwsCKTov#gKgpW zE(zD8tP$VvL_{2AmHDD0V&GA>iQ9e`rH?^D_dG4yA7h{CLe_wcjWb2~lu8f+l?ky!EQ&`fUFs`HOvciW`ttw!-h?_A$1l+&70QUk|iB_scxeE)DRl zky3Dp*mj&%>RN;HU^?xQ<`|D-{lk^3>LjPM1W=iXG2~bU^;V(|(=edXXWJ_$LZ#A& zf_}rR!YtRv`qWmAMgxEHRl{A*afUx#K7be@g>r$U0F(Ri8 z!Zzl}CV_T`VMhVv8(^XjXJkowAyw`&`I6=(h#x^BcGD|%oQB2pHuQc%Bgl0k4hlgW zV@x1s6s{XLb@jsK8aQ{pQtfi=dB0NrCN&r(Tk4A%tjEcVND~Sj#lr4JEAUy0LP-#U zp=0x=`%GX}Qcq#~nG7XtC{f=#?8mhbi`GLBz_ z{Z~$D0KG!pt~R@fWQDA94cSnTm~&G|z>|mrBIyi-{NW7C<1_Gu@)iRvf#;cl@iKo| zhF@S>#?QPKqc5^Pe0U9U`63&}uiX%BF2N#r^tD)W3Cq%1X`=OImc*YN5=SqyUaY97 z@fRz}(nX8ESQq>({tKSZEhbUp3JYO-#FtlCMv-Da5ySMQ@uT#bC4#QP-?Uo{xys7n z$9$Ega9frrevJ((p1zg35?@4uRItddPu(SETw~#Em*wO&HjeReH$?jz>_us523^3i z+LChpAiyORHUbb`>Sx$su!iCfzFIYqI+EI(%=SSS zMdnQw#(T#Lev9>XOULR`*=804Z?Uetbh602#o{XNoJg#OP*^Lz`Vge+>~L3?NQkAo za&fANyv_XFwgRGTohk<1W{df$38L^F7*i`Jh=zAqb+_*(U?82(5l!!~vciNv{IdyS z?HxFRpC^fXcVG=pO%fgMvevw5l2~$=H7Ki1LWy`#O>)+o)@BersdbclM)CYEt5tSu zzV_-;++QL$WW1eSe{QR2s<0@xu0~+l@s8>(DOM`1f}0OPF27eH-zm(~?cR7mTFX2o zgwBf1j?LpmO)D!`e+E+2roVX#)aEePKCUj-36h0jtB1I{*hT^P3q1_YW$GVg#Y8J> z*0=!_BF0j0VqNlU*Tn90ht2>ios~hM5ELSXk5t5}9PE!nv>5U#M~j%|-ecX1Trf)! zx`7`hV2T)ck8R*tV}-Ac#g*8Erk^M2Zzp$1pyP4Y65ci&8ys+R4BA7h+6Y81ODyXM z?hxWXRidkjA7$7iQSLsgS#$JEF&ZmKO5jdw{DEn0T+c0;IZsrCdak zZM1~bV%;$fyi)dmFUmY%fx0PaR7-+?94U%1M$z#BeEM5u`YJj4{2ohg^A?#U?ntFwKi!*YwhA0@6ogqP^XDDmzgYgl4ET$U!v zKt>JM;$w}p5 zTQ533W=%octjEky2uziJyf4;^+mBgnjrlkrZ}W7R2@jPc-;kL=({eq3B^@u)qL6Q! zz7f5iK+6plqn^Np`!BKR32TiX>l0Ro{Vd8nWxvkaG~0)NQAdn^#%}SJYen31R-O$P z#^rp9)U#9{vclzBf)RCtMvwD8s1SuJM-1$eGT^O75>etC?5_p4B8 z+6Y*RV*+E%h$NR*^WG}4`z4Fv3s;FkuUI8f^EHboyc(zEz*4)u(`qsLH9O8-kBC-p zSX+-V6OcfbjWYUsvFZ(0`PWv6tT$L-Zdf7Syuo5`>gA6QP6Vro_@Ix^ks zms`?T@pJ&s^b-Z&!YYfcA%fpxkDzsb(eW({=;99pJ|=A>OygcKjV~qC90gy~JD8@) z;D&5vofYIy$wpLAn&C_48i`a5vPMV%oFCvb2AI6}>gsZF{4J}XzxB`NT}Tw#ci=$h z>Y~9rHqhtNI3gcPju;2$2U+&Y+%aPNJJ#!q_)jHPfYzbeUb+f%J#uD!s3WtAMDlV{ zf|@n2S5KZs73J-$X0Z%t{CxWjFGVjqcyo4%NV2ms1&1LWF6ydZi&b_OSm_ZaFoZg{ z-IIGMIO6~j7_o!|^{%p_4+QMWK=Ihle1i`xN3}Shj2uq)$!370JuRWia=enzcB5i* z2F}6eDPQ~ou`m7jfyHxX@)~v8ydZzeeVa8;WfK?6VXAEDjZzOF(jUe$7D#g|@tSV6 z$3Q>d>o4k6;$QLths3-}d<1{8KzRD|&v~P4(bu1!=KB32G=Nv&Pxpx)0lW*}yg+OV z;1Pv}ERbEUEbjSRW!@0`GxJ5`Dtw`FD$Eddi9<5yZaeHSjo?whwpE;O-cOw}6Kczq zT5(#aMVoTuLant#@QUOM&NriRef?t=Mnudab;i@ z+;iu2y<(pY-rs^4^kkXZ&%}n_P8bqHYr|`ZOLQ#+uI$dF^%f^)TZ_`HYHz$WJtrB?&t5G_=C+Xj#cF4jKx7B7?EeM zFWc=Ef7roAGr&zHD%Wa6-6Xj!YO}rtStR1XeF_tlmtD#T#(vkvJ4b_2G zzy*ONDQ@&I>uV4?@ZXO*KoLMGRd_lTQ08?bBUp9AixW?=gbNpdyk^m<_!xwgC4A{{R4XrriclY3dT?OZ4=rtHKi4%6@29s@@C8<$>%igL_tkm1 zkxu~<+6CV1kS`oq-Q^*@sNf7`&EFcFpG!hPoU+S92yjTR3#-UEk6){CzT^1ED* z>ADdAyULocs96dA;iOY$_JDbE9QIEHb~MR0!KxSD38i5#OG06upecI#wFWYT&vs?1 zwrsq3Q-p_jUV$!hEU32j6peJe5AV8CEY|U___?dX&5H+e?PL+`#e4G=lf*19{+)ld zNvIdbMK=Q#l@^S@3w z-{T0xrM04X4m-tHPY_BD^DF56IW#RD>6x{xZ*kGjhsT%w8?%y4N4~pQ2Aj~XOi<@Y zl9;pNt`G0>tT#gRnx3-9BCT>3KY#)Jjaz^P2o_C1KI?dj5xDL69cW zj{=RKlIWlaheQV&MdcE_T%({K#6Xh8TnG26Hn|-+zawO!Jgi)GulY!LUIi*@hw?~1 zVVor~l<#Jqby4Ns1~AL*`fFn?O&ak)#*2&<0~$jBR*kXz*_fLdFY&$T-2{Yh9Bnz& zgr8!bhex9n6bnj?{LZpAoX_CAQKD$noZsVZ6GW9rUX52u5FI0VGG9Exaz2v(OT(WI z6)6Tjj3*5hFAe-DpE^X`i{?*S2G@p}+Z+<`sG9U3T3f@z7y>a>&IKc-s;Zo*Hl+Y% zC1>McHES=*8bn#w2aAfWd8b0-2C2fip{AJ7n%D5O49RU@zrkW>Yks#>cgiy8*{~S) zGJ4RNVTQDxC*-W&e2~}`!+(xjTLYYgWf2YBPA~4QT8@gV1xgxSDN(v`8-gU&g_}HX zy9;KzyvOFE2BIpQL$d1|;fbVfP<4?R%R@Z}49?}C{%cXN4Y!qWN0#V8&vWS_OXK+f zQK&6n$U6@ZYuobk#;H{$ZDm{Jq3uw0&_+(%A_0;%CCW)#IkZGNZMBp?kJEMz6(K%e z@|Q)nx z>e1xXC?u`%KUr9_L4{7Ykx=4g{#PpXdSm8kzCr+D)Hzz?n~Oq0H` zp%2McNbDD#j^RQ7bcVF@2HXT0v77v&>NlknalIc_urZ;}bd$@_gSp0ENG0EGi+E}_ zK5;a@=nGDer`QEK5Z)hmL^bx;CD+E93>Tc2Bw$*c7Yw36nJZbY;afvv;@-U&b8ov7 zDYjt_67uM(M-ikpx+?ZCX<(ac9E5^`y+{tbI(f|x^0xW4Lg2l(ui;#dF&VvZy22eF zM&Z)P*Z73Ru1mH9AND)3H~(Ch+>B~YYw6c2&EpzHN$%0cRPS0etkh6^&8|zPkMU?X z6BjDnY?bDEldAW_MGH4^#RQyd*x~3$Ek)y`x7k2fIyZ9D8B<4sJY|M#C#A)?TM;75 z9vymFm;8?EH~mefaMk0wi7Gj*+*_r2<3pi(Z(5n^T{oD#-{4eopcUt)5dppPdqahb zh8@>{0soBA6^`OCq4wD=jZ5%hJbAvs6CJ5!w>2))@o*k7%uiok zA7=^tqC<}+UbC(Q0o36ix}Un9+>{A*rCEiIEF&z{Y<%w8hkEoiHITN2DZvj$J3j3| z;D;D8PT`mCCVNls8bP4D${ztaMs$tzE-v%OY%#VZ4&QNJ;z_4Ui&sT_)l)qZjO^%lJ0x9nd0#+u&e6+&DjF%;(6>FY>W z@O$rBgwE)Hn*I(x?S(7MT>;F*vGByRhR_#8L_!&Z{qTqBG1}zuoTBpE4qQEw9lqYuR_!egcI4$tPDh?_-EVgL^-i?+YC44tXw!RTMkmYr zjy%hiSM4qic16BvzFML<7(pAYh|6z|UyHSDA?4d9*m27~4L06w4VQ=h_~{msKY^{{UCepBpe zoGKllwM-wxJ2L)rq_{np$MSBGmU`a+spf0uYS9hjr(9Wkq_kM8wqQ?Q5-U%)K&r43IY%g4HN?hL}`N2WRj&kekSujLGkp%H1?4^SS|7w zU=FX%4^JqUb}Qu~Uz}r0w7JZvP4Pg0pSpUv7IM}_4cMS&d+8{NW*$+q{R)V5 z`KH5)cTx0ezREJ+anvwHnR#F0TLF+t`V$qU&8;W zV(DPwg|&rhR|4zI+Ve%_ps?-9gPLzraO8E~9CKn4CtSYXGP&JysNit2(2q%+sYRESYx%U1T{Rq7}4 z*)%?i52+wFP2)anoj5d&Pk|LcYrl@~oS(ZP_`Z_Dt?)^rRH?xJ^pJ4T6WQ%E1=Mc15=0iNxw8c!f&dN*p+* z-g_?d>&H1>Z>z7J9j|BA*V>NPZ1vUM@w#1oy`F=sHwf3y>ibUAO|SFS*IAC&H1+j! z$Lm=2wWoXq)2}FnRLWXQ$W`Tt%%Js_mB}_otB^V$AoW#9lJNs5K*d{GLXImX)YtDE zuWo7qzYqCeTqh>!he$}aaz}kt^JgpPDzu`4rXZd3mKgz0%ERU9l zw)Deyn`v=@uoU1^Jy(kQ@O+HvDf}WmvFZ3@8IduESN1GYj@A&6X@oK;EuPKcRr#}W zqTF0ww$=`mQP(Gwu5|e8y}B5{j6{qo1|S*9-7dd}$M2RHIF~n!xLy+VBun=fMH?kA zs~UDhu0k@AN=2Ss0xHKB99Z9b6Z*E_X|Q0ErCV zI=QWjNwJBV$*vD^pzVLj@`d0T%&R%67#gUs*$*iPFI%eqM6*RRACWMhf5-cu6A$Kd z-_kw@G%$a)NtDq%PBEBcukDn>`-)ZgE#M7(C!W{1?1Tb!!A^z4b%!Nymx5yW0$$zo z98z&T*FFr<=jX-07Vu=8`K`5(cl0QJ4&_RsoDzIRm>2RB9<|OmFe#n>5(5|UFZjtb z;=m&Qd94wrohkD#=g5!u7D=;nj2|r-N}|5blmjZ&MHzKjv{=l;_=UM5buph7`imCc zBucTj^QF&M=*~%;y7wa>+FLMPXjbOMCn}+lisjMX~!b zFC~sG<@I%Ge>w`0Kj&W;g!eLD&12^t042i{E{Imkc%VmFXEr7Lf*4N_PpV%kVUG)9 z%Q9X$w2ZS9jy=NnR~H;QLdD2QNCW=Bd945Rk*0AOl!Rcr9(bP zhLpq6x~RvhmsuHfRv54Frk0s2`8w`ZAIR0YnLe}GcS^*r;Sm*nL)=PC7cr)j%7|0% zvGCIF21Lz=W3_Ro#4l@jBVMPV<^CEn(W0}&uj^sCl*E^mn#M*}xTM7T;{-J|0b9I>=XQ{UN4r5g>`{K76JPVgV$5b<-=p|5C&cBjNZ-s$d2I4< z3c$rf;?`#P5x4FWMYe!_A<4)qTa|W5)ZM~^Ji6!NYkNox-NJo664xV-q@wyEF>4EN z=rL`gv()WuabXL(XLW|~+=_ln$hNfF%A2xcr;;3592-iNvc!U4`D8XpRNBU?d;CU& zf(E71@1W?mjYoLAA&BIr&q1+f8z1cVek~wWua&-$&z1DHs-(y55p{p#Mvs4ucT}Xf z91!Wh@x>lrKXBsioGAuacwPQxmsn&0hdpys-s~3#Ej+;E@_lFCL;J-`f~?Dhi2b6< zc0j(*g~aa{UAJRQ>d^%%^`Gq*Q?~Q+9-USLB6(DJzu3B+SMiuQ77)qV^ZUe|?L4XR z%P~$!%sx4SQ`+Kq1O*pVrK~KZqmmOdl$BSmvMRGrEZ>28-`{iAdp|>*+`+5*9|pww z85kMyD|6x@X&r+|5h{gCGDP{Ey#8lHbIbO~Q1wS6Fw#l~Nl5B1== zWgqMnf9~WymAYrwDR=ahtD7=eTuiio96uq%~;R($<8fdOap}M3z z_uMP`pnyj^g2+)?d9RoUNT|(;lvb4Y({t>&Jp#cB8qnqFQBhby2F6x>IHD)84IPcL z#B<8IKV7`r#cT8)kf%7f8koCT($hK`$Z*28)QFN1$yt3OO^KVx8YE-k>R+hFlx^Dp ztX1+LP+Fh}Dxc%{&eW6L6;pP@o&I8*7`=xd;mvQ07U_Jt$E1-CQk2WP#Eo<~%ohJD zitgnl8qC=R#X@eaheW73WX-RR1{2YvkSLp{OH~g&UWHGb_7389iMYKyf&a8soY~83 zmfV6;n9OPArMn0PnubLf%CFcdN@wu$C7zxqHTz2WeWx^RM_hnm+hm7mm%(3o$8SM} zjtNm!?0t8riam3LRP4_;f27z8?~saJJl)6F6`FRI^mVrK)poIFKlkSiH;JSBc}X6B z-lFX1eVJPXPLL~&w^=$I;A^?ZmqVP4*uGU1$mSJGrIJ2z)Q?XWoCmwDW4DTC+5AD! z`x8X_4Mg}?PHvIpzs9&lLl+*E)0GxKn63_`R|f-?rkh0EA?)K+UMI|lpn;xm7QY|D z)+R#)Y{52Ni@f0k62 zfDHv3*?I^wlypyv)gt>iZ|oUOyVIs990k;79THwAc&#F5k;!1*GXO(UnYdEKoZw|} za%12Lc+J1U*Kd%xSoCh0EV29qW{g4oMDaiQq*|vkftohQlnX1!pmuB(s-I2cuvN&< zH$XcLrk>VUYen{-ymp27l{jje(KFNZB>8$7d~cc&oI&oTX?;}p5DCxqn^Z(RQH zVj?l&p!`fPcQ(pY2k{~y(y#LpeD-2d?J}R_vA(NQjQC7HQzuK3q_SHu&`crmIE;m8S+_JO&3~VVTCdNVwOn13h6#LU%a`>E!=08*m4cr zKJk-ycnyYe<~V0FQhySm*U^l2GezI)JiK&C9OjnIn2nm9y1P8s{A7mO1ZAE$d!4WK zxYEJdgnIMD4>x#?if^VnoA3@NNFkGB9XIxUPRFCoVNa%u^EY@9|5b=WH@V`rcNzxP z!MWniO&;hr^hdnRoht%w@j-6QDaC)Tn1720Hnbtd$p%9neQ*QE+#_CQvxET%BOzB8 zaUgSv*^yh>KS#W~#mkldp32hzB?H@@;Db!H+W{me-_$uG^frIt@#R!!v6R_3(#Xs5 zZf5bt9T*2ev&D~h_yM0nZ6wbdA!Ev6gDwh-#%!h2EHUsdzs4hf5Umwh5Pj2thAelb z^-M8B;oWqqN)B`@BdHNx+Kn#6; z@kEK8YT5UYH`I7MStDUE=G{!f?I|Bv?7#}x05ql?S*K31Onl0J;)NoRK!eYkY6*D3 z+iMDSMp+uMIb%iAYwm08CviPeE{~NXI844~D@XAf!`|ScZG*1mG3eOFvfOdyS4uEz zqn0>~4pc~2UtlJwHc_TDrxdj#oGD&Nf$wJ*YzHW$@A36LFw{jp%rSEiX5OwYi@rBX z1eF$jKtRntH9>;?2~2(IXyR@aylU|Wu!E506D)(@aFd45O%$)*VGH)rKFh%Oycg#O zhKs{?z9y*BaE(h?$9C|>rdTjWBXky5e2f=Tn83{>cxfK+vv21t)#zJZ`WC4%pHrdX zc~Ub8+){!mZod=PI+MTN-j0%rZ@#o*Rx(`3|@`#;Hf>3nyiIqD-+m zQ!q5ZB3dhsR8zVTE}0$QwonOUFEwE(64cxj!r_9_QU$*z66kbw;vooM9RRWS*9lZA zchn35DMN2$2&0ECc<2F~wt9FN0fAvrqA2FBdCk8cZ1E_lsm%D6Bul+Qn(MCIZMpE#X=3@(t% zV=$QD|ALw06-O2ij!Wa&dx~gPT+@)h9w;Ui*Hm|}3i(N^d32%JQCzcDQ!j{eJ5T9AJ=SbRF0q>T07gtWfy|C@#gYC4Xchj$T)n1x z`M~^zPC`GT&^drnsPO{PO0PM|9~(ru5}IS&&mbO_(5&T8(!@L;&3fKEO|CBiy0M3A4RlK8H?#uU^x1uYqlmL$Ik)J>j@-67LqTtZ&SIu?&;Q*ET9 zP1noHqMfHx(e-=;jurW7+IU?Hlc}+4YO84~;Wle3q zW2`t)88r-(za>QNDySiRtOyU(_=-MNP=i{mkV%ZKqS;cc#7D@Z$57VlZuBZL!sv*G=}I>#LN%{tFqyuoJ13Q-aT9jz=LSrbg1uzVAuYOGPgO>Yt65v?rQh^Cz0|PBTR3uFj_I)mfn1)p(8aV_~gqNg&Lu@^znL;`{0~Segj!2Zw5~ zT8?uK=1fh~ELqAv$j5uL(e-q2V;nG@)oZ8hYT38raNNdE?DHM6-vthIP<;vuKUbCm zX|FfyU%W$K?t#ZCLsJXvbH)gT(d(2g)vL)o4AFy7&s{MQ?&IGhG1r#jh*lh-3F@Tz zJRJx`07sNCQMBcVDjZSRgt*8NCG#POBMJe5`0NM6u3b7y^LaJ^c77Gj64i;)obmz3 zlr+Wl2Pe765$61q8%Vh|nU~=Pr(DcE%9tqY@{CU72)+PDO8XHsbF3v(4dI*<2s7!5Ex0yTS`9hNFm$AZY;pmyx{8m}fv64wk;Vi_Y zoxwtyF(9EI=VZHiebAU`a&Z_TDGv|T0g3WCh}P6*Wee>aqqVs5JbOpPmuxf?Z> z>at$qjBdv9by*)lOo*k7dTfn2IhJPCXR%`aSbAHZ1(!*VN2@{D%yg z0;H=mZJ_^U7XZEEMLnO(6gh5j5Y z$V|v{xN(=+-1I$BJH2TRMSq$8pbANOo_P(;J=>C2<_>L1dp0b^qNfv z=?hB`x5PF{`IL0uKJdtE2&3p*zo6=&EXXD!3SL-k`aF%I#8B3&$50@&wYIY23_8bO z&VG+!IQc0O%)-YXE*h*!B7Lf%Ez*bN_rO~J0nljH1&9ynCBfHppyyJBf}67dF))fI zHD{F^uH(=bE29w~P{K4H+bH9q=B%+GPU=ZwOBPYEMgkA-GoT|yw?x;sPM~ouSsRCO z^>eWwPr!p-_28>?pt~)ZujAkz8ds?LKCti=FqNh9VQBuhU#k!GmWpfVy!p2mp)O%) z-qr2sNf=uw&T*jet-w2Sdpgz%%bIKL=zS};+4&(hewoEA2ZfeZp+h^`*BUK4aUel2 z_fXNhEiGt+9l@eVGPXfW4)mk@ZCDSnNdyIlvt;pLIQ<$9{Ey-EAsj8Yk4hMFiN9|aqcqoc4w#P;?!&tlnvlB&^x>U9k_;O@TYTAjF7FX4zew|o1F}(ww z@5B~~wjJq*DD>mael#nJl@m{PpgmE{UHn*;Zbo54*S!N3>x`krw*!TAW@Qc3m~$}X z`%fxT39a6i0)vX@n71w(=8B1ie{va*ibCPMf#I(n>sNg9>Sgk(1`K3n`S^|A>Y%tI z*36n9js8Yj*F>5bD21BZ|4Sgx#XHVATA=4h*xsVp%|lt-5?|koDv!Ye=KB*K$`s%@ zf6Oe7x;yKhOFms#b>Rv1>cVapJ`};l;Ec8{prdemTWS@}c2|7_J-z-}_Q@OZy*;y1 zhyWR<5l{t@r!23E@dVcbaJ^4cIG<*h@QA!)n5XE~mfFU!uqL-Ln3&o8qsHd^M55vs zKCNO{I;q%HLK(O5E3(!O;LOmJ_dKO3wZZi*(uzUvlsr3X<858Vse%-;Mxx~&@i4pQS%bUeY#)*zo?iC`qR@aCVAk1$Ue#RR6B*cZ=id zDy8$|PI6p1#Xdi7JI57OvYo$jmbJ1BIL=vX9l%)+;4_QgpNoVD@vpqL`6^1)k7|ux zl3YGLei+Zlp56oEJ$eltIw&NCoeZ)b3fh&NmTzHHeXG_K+nss1#(zV3YQfXvq3s0n z26i>;(~|DYz2{LRXx*hll@{R7T}z7Q(!1WQMsaVrGTIh5Up3$g ziBo)fwIu&O%)jboZP-6?D)=f&i|iy^)e5mnLmy8{yefny-Qk0gs&5!d%loh(=i6T= z*07XMd`*(L0;{|Ojp<7tR#rUGm^|Z{hj7c-IF9uYgx_gxUv^cvNn?IsDmVJ!_-0Qm zmFUlYDFXGZAw6u3A;M?%LCaJhuVA{`A0ub~VEWRZ)p0Lrm;YSPq$?Rpp;G-IIkr5A zS`R>XHJHr~O-$}OO=!yi1}T=>ymld{ImbSaUJk&_xz3jg3}ha*^L#a*70MD_F4Y>y zf^BRuE^0+&jVWm$^Q?IvgQ14Fp^b`JN&$&CFc!m!H~A+3S@{SaLJ;DI^$uoeDW+6UowfZLTIB%fBhl((uKy6^SE zy^p{>I4Dtrz0aunB|Im1E+IaQmyLe}yvtuH$PFZy!Dt4j#?){yDAQE52($ID7YZ=#I0-Oj~V#gwA-n71<8OYJ6)Qsy3AM7USyB{GqHF8&n{d z?!&|GeIs;jKdtC18<5>FR>iq9P!K-U%2OdEEkt+|f;Z2D8&UW$<{mkq;yHdz%DIH20KA_^QPlJ|~Zk z4r5W`^7`Z+&nkw7H9$=ys|jO5Qhh>UKNb7}$?(c~flG22?^A-6WPG`;ixqon^o!)% z3BI5#^*v>)0Gbugiq)8k5M20ypOmEB!~5+hOu%g*<83k8sroorksa+@v8u0IpKyXS zTVI}Ilb~1z9i&A)-ZXNK4_!>aVds(rR>t9u55@&2%;6Z&)8wa`S0iy_4iz1a{)? zyjxFP6Dh0pD1DX>%^b~wT}FasEGuM9#31QKBh+?a1$sG}H5KojdDE41)3wd$j5B%kVGOGbm2sY9v3(tq zM`2@GyDDB7^}uCQAA)UKrtPvqCbiaStc`}zQ^7LY&its-<~5xk%WBm8h|y3ZS;I*X zb`pQS5H~tAj&k-rqIw|jqgo}y-oe2R4wiG2H(jrD|FurkdX$2{s~ zY1btarz(X%=BUajh{u0D0@?T;4TL#A|Fy*xXdm7`FTTrBv0jHt*ge^#Gq8qY<1)r( z1~yp`i@YQIG}hVmEwVG=_*9-U^e(08qY79AB~HJirW06qzdO~wmo-`=n7XCdzni>V{T;d0wEW!h`1oy4wN-*VGFY<;GplUOz3Fa=Ix#YMYX z)M*mt;bpg=E}gk}_OQZI3Zk_Be9JBC@twsiQ42=+Qwg@pqkWUG1%394vL-RF?rlJ) zZFeDh9Fh`!acc0*lfl1RA%-WvW37`Wdm)C9G+aNmNLBMn--H%Mhf(m6&uVJ$Gf?k0 zQSXH88z`UcgNN2UNqA==nYF5zjn?Iix(*+ML(-OVMny!bzGP*T6%z`kBCe^g%2Rrq z%xc(N=2gJ`Csd}YlUZ4)pNgE!DvBTL(74H1yhG-9YPPmeshJDK^3+uX=`3iGd4kMmec^wyRsiF;tCBWM9r(Aip0-0w^a+iDVIFyxY9 z3M*Bx!3*uw!pn`;O~F2Vc?Ehlg_IR@3d(J$QGkoK|w>vBC!f(_0Vfg(P*9 z{gf5FPSfP)`{;cNd*)U|!0m6-fRpb%iMeILB*%J_(q1T2@*t&VwRI}PAVHCh?kzyh4C!Pjq$q^`Fl)>Y-W}wp8AX?r!m!KL}^KT z9mTeB4?%dius`Ssu?8i=Tu{4w-l>BxoU@z}DFhplZ@kjrvVcX1o-Qii3J$ZAC#b{3)nDmv@>;H2=^B!P@n95 z2CFrD{wC}h?R73wJPFk^UA$3g-k0~9JkzlmGJ9dz@kR7WeuWbXo|Gd|A|Z3;Nx8YR zaqME|ED7t3^OiGvL0C&`Rk5_!&^7;JCeW^G^g2luAy!}VVJ6B zPg8$lA6%vgoXt-uQ!Z)g%`!VjUZVM{STA7}<*Z_vxVU9*L#5-8;D;}zO0g@nejOVp_DrKnKeH-DH^1al^+A(pN4r@{ z@oN)ozyQW6px}q+8FbiOY(U?|SfO{7ss+9=o8OA87r0 z)Geq|ginIr5Sf`7Izl02A?vT5@59Y`Q0cs#84UQyQ#VAYe^ zG=2m2nI|sM-VLm@$5u>&}jb11BsV^5inEy}f*Ib6!^ z%t^U%0V=(bH7hd*{KQv&d^y2au=Z1p5Ox5g4?$Q=m8#xDrWqUAQE_@QMQp-(`cp?eA>iPbCHOXP%DS9XCV@DLXq3rRZYi~?A$2?dD~HQCIjJC;E%u9m{)M6h>YX`j}T3dCHYz zIcmmoT!q3hb=F5_)gxX6G8eUOg((_3c)5oq(82tH z!tnZ&rV$E7f`)D8=3rdzrJ1ids!gM98&S zG;1&Nbo--m!(Mh=5~Cl}kVEK~)gROHLzp`{JfabYS*U3Hh>jjclusX$au`F$;fG|O z&aT<4cx{fU*&BMD&c^Cf-s0#{+Zkzpd}(%m6^%cFkNR(3)3PJX&Hl_Q6!b~Uv|-Sl z7I5Gdojn5PPkKe{D64B|xGhJ`ER6SHQFPuftHX^AQ|duJm?+(z30BGrBS`Q@fPvpy z(`vgYV$`atAdfnhYc+xaGCaHk6B&M4%tnl6{6>sFITt7&{oF=IWzhV~e_lnk@WMjlf z8C3cdYv@^*2L@B)0dY-9>llZ?0~l5ibLq?&tW7g0=@e^Pa=&73s_n3t6Klf9s)!7F ze2SHBmuMk~heh)#nE^^v)Z+Ch6o2~;C#2dyhMwG7qek$iJUS2!HxX$8ADVI9*BiT> zW=%x9wutF)EHZ^Y_l!23WvOD5YsR|g*ls3HUrle%<5T~D2d0RYt|HG1SkP5}Kpify zY9-YB$hbvsAda!Bo0SKae@M$OutwsV`^E6cg+cPle7t36dFE$2Pu``y~`JEk12 zU1oH?inF<*2Vz9b4~?J_V<4UwiMKltV{CjKB0Hk(O&WcZ)e`UBq%AjDqS)~URr?+L zobflQ>F=zfgZ>6$O2rXmYQh3+>Ti+(SnGL-SMx-KQ)gPF6DHggt70KOW5a^g#rbPvlBkIS_79xGMo z@I|cyDX-_yg?p%qkqFlKH#xyuu?IdS8b8`q-0H^FA9P){3AF_gQOk z`w7aqkEB{54$S%}X+p8g2Ij7l7SQzswSK_L1dlqTZE-_WZg7gLTs34>ZNwXVfrmSZ z892V?$i7%5bhxZ#M%C*OBS&|_);QBu)jvH?dmgaTHro(F3%cSwJ$=C3?fs^FWjUz( zolTApAr0qz!MN-pZyNUtbow!CAuc>Z`X`XwtapxXKfz*S%2{%H%99pl3Yhok(X7<>6mhzs0D-zC<%QN=c`^7m1X+~G9 z;?WSQejdW1WEnWg;;WzY@C`hOoux^00VK)JkiUZS&E$i$QepK9cZ4TKZ*87=oN9co zuqlF=8B4>SW3nE(oz^~QRjY??=Y5gyvM{s>*v50^L#qC#lgOix$@gF56AxG=K%p#V zn@J}~%w+q;Q^)CWCWH{)ouC#k_-^zB4S9i2(UZ2)q8H4+-Y=8Ed3BWT5DeEAL3}$p z4>+txUqM-dvT!>5ouGRiM>q$y-FoUSYcLX5v0L~ht5rMcI5^M={TyrSlwem0g(EP`g(g$NOGpK*K1-P0Lc~6(%a>TIR6R>UudsnGaF!;&g8!8>bmA3;z6EE< z<~2sMwPz^sHAD~loS|v2L67^K7r1-TX>xu8ch}Pt{RSWG>Yk?AZ_vvtoi=WM!&VAn zfs+*e7Hn|p1TA}u=AVCpZoNgoA5Ku|cX0ckpsw$bP~LG`^9~8wo}k?|Znjj?sSZK5>k)-XozE$EflLBsA-oG2{az811~!jEDIP_NxBEKDzf23+x&D z=4UaFvOV`gS510KvLC7r4egN4ja_kgNpjyOu!X8`frEN7d|<^Lz%TO;e2ph)AN9*Z zJmNm%^ei?&tdY8#SF6^Wt>W#_C~dTU9dANIO(4bxj^-U0uWaw8ieK2FB5UBCu>vWD zq>h3j*%`a&(--Dibnq@7**68VvAc}rvRRDS!fJ<>iuET;bPlj#4YOTCdwfM#(9oAn zXg`j&pR`evv^v;G^D=Iy3%P8onAXZTArBo!Xtryy6eqC@d`(XRI`SDl7>?$$Ao24ydS6^>CJx+20Zvj~vC%dft z89|JjPwpk8rh$H)(0$!ZpD{UOD&;u>kN4_7PC@Vf9L1-WOpi==%#%3tI%oyPRB8nZ zRv800b*ETbQ$lJb7L272C8ScNl&*Xk6MQjnw*Wo=DF;0`JEv-fs*jvUwMt6e#Xen$ zN=n}1fgkB?NhzTIhZs{JM4ygEP}&6{dRHF(ZL?2CN4WCm;4Q>TGe;`>V<^Z~@^P|- z5&{e*TdnMqxo$H}7E~D&Lo;0^XR%!j{o*Q>6L-y}ORkc;Xct3&x=PJlTF%v2g-?6= zJBrU-3U@=Wl5?q_8|v~Nr2O5aO0{;(0eNcjNaQdi$&c%jGdog8qQlJ7e6g=S-WIE8 zwF~~fbTLZqQWe?0=Kvgv@w>i!aN80=4NFVU#NKJf3S}fg@)Oy3BaS3&9{PMtz|6(qL;D?@?2 z6HZN|KjY zb~1IVB$X;8P1Yt`rFIBSt0Y|(J5DynRF)(`j7T>2tRms6;^ZV+P!((!JdPezl^jLq zFyqInQeP%G8DqVr-{od4Mrl=C0p};CDy{(=9+Nkr8+}EGXlOSRy5mR~krL1i_k;8cAV-weeCTNs?T*aJko%TNnyV zi93J`O^MIzM}Ia!`;O^HHo;OUF{+=jO0Wb~ro-aMF+}PsTF2425UG)PvJa()NXx`> zeJD0m+9VqK810)&5Rj`YJIO*X=T7GPL8CSKGR_VkB(G(IXdAXK|#&{YJ zb<1c|_W*PI^OZthuN;~jDRuT-T;GgUW_w};*mGYsix=<`Dv70XNNg)zaoqxWJ(PkE z;#T%QrZ=&^p8#)>j>+6{L)_nmUNuTqdoPG zl0wAm?PzP1R6^X>j(&}jD!J=Cz;TWF>MBWFNLt1Fp^D;=493285JZyj5=lEu-S|?6 z=WY&q@SgeZmT5cJ&k1a`wv_e-6n+=^px|`F8cXcI4~wer)RwAtkzDFEfH&8bp*>udbS#c{8L#W6n4^lD5==U# zeQ>pbWCGt?1_7;%bI+j(UBKi&Mw-$|jQrbldPkB+w4`s_ntOAFGcovv*%*$miKdN- zQz~B5IxBh_e0Uli_2#mUu~uxam_2RGp7r3V40p>hrbJ8Sa7>X&+qz*;cFr_@>?Y+3 z!c&^nQz}N*J*6qa6C>;kWU)(As`P_2%H;vS*phniB5IeP5e48i$_h9oKO0O3en1?_ z!A7B<^phYyfUtIdsj|2;h%Wb+s*6cMMm9j&#l*IEjQa*lmznsf9?ghH1K+7 z5j;{~^I=unFJJ<*Po9m6)&_cQYzfOH@8ME{c-NnP9xk=4@iP?0hNax+pPF#sRiz~^ zfqf(Hmy@k#L}Lub(h++PRo~s8%8Zc8i~jx;HUeX+i$4t*AtjaBh!a0;oYo|h`QaQt zR2!b1t!DNEKVtha)Q^gdl!B`UnJD-0)GFiX%aYh>r68#7ZCa)b*ppdQn=!5YX!1y@ zxwx|qT^=d5DcPNWiAl+9nYRBvvczR#c#ps*k`ZGHK)kNG~0BTZW=9Bcr~8E)Vw1U0`@P|26NjFxJNAAD%`Xh~n`u8)bP zu!RPz`Li5{((zS6xgi>ND6g{U;%KRgIMs*pMoUiOx-4=UBQ3HC^5dCDCwuEd2gXQc z#Em}maEw&P<~B-v$?A1X5zbsOk|Pq$JPn=uxTN}LJW*;O{Az5MC_NAiy5~8n`O8M0;{QGyI&}H+y*g}}#ZC6o++K+5*3DC# zU^fYLX~#miT9AmHb3t!D2$-gz@!Exh#Hp(OU5#&8Oia81FfsiBEU!SjciFgiN4pHt z95L9NLdQ#OtNVO%p_YAo7gDFq3iX(7TT+;;hN%J^}jbXpRQ84pj9Y-Hh( z@!WJNNf2AzFt(W?1q;G{nm<#TR->l}62O3k?qZry;$VY}feFp$W)ph6hW7QKHnTA1 zlrnkq!Oi47SR3Z7J!sr4skr#26wRL{RTI-o8BflVim{SM@vXtM@3c?mAH6l+&6_9H^tyI|^FKe|#=9dX_RmvR`~mJ;jCfT2 z(h`(0Px7wuLy5mFN2Ce;eL3ov(8?i}EkWJpOPMwO3W2BhKSs+{M3qLtg?@g~w5sNY zGV?zC>`Yw>4Q_RZ#*ySIK6j=iB-L=)gBx9$B{KDyj@nuL?fAmlmT;vrJt3)%a~!;b zCw)vP7{Ud3hT?t>pSY&oqS^~2AF-q}^;-bG{oq8i7ohh3aH3xpNP*%aCwj3!8YjKo z232{QxWt>^(TyzP5|4hT#S5j9WzQGqB_GpKZtXOv!3E+Nh= zPFV}36*kc>yvtzEYU@NlEt1+*PSd0Ojn0F)O;!9<8Fo|au_0*(Z{lGnWs)Y4bWd4w zgS-|?B(2(F$$G@@R7pJSNT-%yH9f(Ra0*tE zJeEo=#BJwkz*4EbjW@Ed4Jr2>>HJcuqBzfyJ};HJ2%D(=GO4Pt(Ku^70lmmv^1yCN?Wb4cCF??F-U~by)nZ zF|J=HO_9aUlCjn%X&N+gipIm6r36{*m}~UiDV-O@so6%|Zs|D_TjFE=Ua4YP@2B`i zVph_zW*=%gP45d726;YmUI9iqf1eudlZqAc!$BhFy4XF&n0-=D!KUUXjsO!p_&~qy zm+Ds-^Io)4s%^^01l^1p6KGWt4PRcYwB6)=K)Nc-BHcl0gfQDU?x184@Bnz;A?c_9 zT~|larD$O~l{zA23Db>EM(~!H8zmPy1?n=K1!;S6kVQnqM)6xghNnx0==R;`^69$vpQ)!tn*m&S6 zu)-iCUS)nOiuNaHM5fe9+#o%vk zrff2`ne$-dT2a2Pvv;2IHMQ4DkN(Egf-(-XTdXz)*~#Fz!#^1>6qZ9IF=3g}#Zk5u z#Xd`n<@It;N!&J>8W`j%4d+hQI4&0BJ)R&3e>I-Kw^U%f+3*45Dc->HQIMaRrQF5W z09C&X-`zOxo$P4*Pvzdw@gMNPa)rseba>aHe{blq{B7klH&SAx>Hlm6*r8f+qIz9=GqwJw%T%YysA5z za((2|u02Qd@&-Kl0elb}q%Aj~DxbR|f~dZ)Tt##pO+Wa`wZvzmXt^&kGh`I$>&Qvs zi;<>G?jA`O>j2+qq_L@=ypY-NYHMZ^UNoz^^s1iR+Qv9SYeH4uJHZ%IU&fKjr6IJd zfjrJVw6?}@I4$_LYzY{GTp{#;TE_5(@?@dFyjbLDs~_EMBws1%%vUP4A&Z=+K7~?l zT;cyLk5~v6D(6DK2FjfTXDZiNt|&Mg+ccK#gaTR72wch6*fU5TDhhhzjbM3hA?fWG zh@Y9JMX!83^vH`Aw~;H0%VOzt8#&f@NfggG9$PMhrZ3=Z9>)2qtuBITi;~-Xw8uc` z-PJ_e%`=-~!sSS@UsvOfaM?pBkWmVO%T%CS5%Ozse-}CziMDvs*|@STZ;ORdrnZPj zQfP}0&5_-~rOj=ja##1#RHePFO5X3yd3LDonlH~@uGz-u4su;)6WvMUNzBY0OuhVg zlw6|XS_r=@oeJmc<k zjJ(<=7dg-xXk&z_Qp$8ir8EskrF<@!uTrWYlx06s0lvIaIyz=kpRRHj@kkpw(N%6# z#LxyF*jgx?iz1DT64_KdR<15KY(vdr z+?c6qrvQ(5?qlz#D5HyHliXcyR>QBAR<7@>*3*RkzG@4#q8Bw}mr}38{^yu-1+Tnu zzO`}4rk&m87;$JA73(3_b?^l-A544OBV#L_vW+o4linW zlCKJ4Az$O9G4dWJT6t>oMx%!wsF*bEU_@#E;R|1>JOhfCa0;Bt#FBg(;oT8!gWegE z+U{pn|H9iuQtX{^y+JN4iC3!{_a(`51+ifjW9wvjt|YE6Z+x608$_{US!1v1avPaV zDJ;@PJDodSpDo)r>Qh)tm>>5jw;yT=_Xmn6Z2Jfa7XSxow@khzoUZDJl;R|+vhcpK z>>PP4WBueDrT zy}UqLzQU5frKj@+n01<7t(To#3^MOPW1O|69OTOQRYNT0;$(8)AbX2?8)Nte`4`l^ zt1)Mzd|9w5#{=+P{xy+)-z>XTm@ne)TZ%iLImQbPF<(_y9#g7fIsEmDTkLUi-y#Qy zE3Z)W7WueX`7D**3O4d8N~^b`4Ngp=Gh5{~;@xx_yiIN<&dZ{`+vGrT;yKFMCQlWW zk2HBZhQDKHY5#V(Lnl)34mm^&wxgLl?sQ`slK!!G-;l$FNQ@m+EcyLM-?F&rfP3?&y)@2K`}Iap+;Xv}W; z2Uj|23cMQ|)xQQ#hP<6oepUZ9K8Nq+0-2~N?Mu7TZ$H?kLhyb2BCJWuu-Xtgods`g zU4x#Pa8WP|ojy81mG{VD#ck2_2yAJN+$^Y)Db)mb4TWm#;wOO&(Rd>-;4wYeBdcs< zmY^v$@F;-}rU`N$x5(3)o;>>Di2yx8U5=60KDlKfdrQh)|IQ4Hn36e~90h1o_t=mqWpnIS0x0fP7D!{ho3U$XD@uOm$);7q7KOo#P@G$(IGVY;kR_*knALu%OK^D>{>qr0hC&Mwcdo+ z)X_;kD(`n8j}U&;Ujf2gj`ZDM_*ywN47Y~^9k>wYxE<8!usp)%H_U(NT6ntg4MeZx z%Fbj#%+M^u@U9r z>qTe0*iUng$QA959OUz4@EaU;w0cITkH`aT%m0Z6-JOnobIL*rJSxxYcOK~mfTJ4P zPdSAeL4)#DnmDJKz-s9+Ml`+r2_#@nfVn_D6i2-_VR>vZwP0lt!6~=Faek4_fvy zM?AYrr;f>iwqsuiR)J~FgsF1#*HrAd>=n>-i&n!a$G*nY1p(CaDR0Um{NVteUtG0G z3kc@%gxp1wVUf^gGbV!6-8AgDT*EOOK7oh0eAZ$92dT0bwEMVR$)O2S<7H^CHB{jj z^yau+O5F5_Q0J#oB}_i z$7ke*VyCB6_N?q#F9x*V-jxjXKV?5@D~b@3D!1f5DZwisS{@23&Vu5DP^1+t)8agE z+fjZ>6VA$k4oA^XL6Xh$k{Xcnj4q#*D;o~MsY!@o3owRv{nTWK_o%xRc`1A_N+ess ziFbtLM7Tpv$GaT;a8YhAhbu=D_PQ(= zll&s@XjNGs2E>-6ga}kmJvg}J&7qt80t@pg4>tyfH~6NdC5=<`^fLwif#F zZMt<`E@4=3izgGj0zAx#!~F9SWaSE84neVYf|P%lXl*WAXy5B!%XX88OWXA|0q_~b zJAlNiS18^DEtRFbBP7(pfV}N0IGf*ua9+ko?62YMIfJTtM;_ZLIpnr+;tiQeqI%Uh z<+hxF)3+fFVa;?GMo?=H3!TP_I@am7au{?SR*ued}{?_x1M;u6{4 zlV^&rE*V$clP8G6D5KXyIZLp)eE})(p>Nu6ROT@jdi8#z$j5SsO(%4KHvFzb? z@>hJg$0HI}vn=9~xP}LHFSj6_`YSzpEWZ$Uoux}pmE-cwx~c@CAbVY6e!|i-%)UOy^!W@7avWY|&UCwg91>;UD5ds4^Tx;P_|U|A zeb^e<&%W@?>Y%Qz)z!DD9v%hBT$X3FvcBdCoVDrSLqt zm`G!(PaYQXL&wteJlUnny=~~E!CIKqwjDmJVk}RSI}Op+fm=h; zf`Qgyl1e?zqE;-FkK^Y<|bUf)m0R8 z<{Kjng^rB;NwMV966bi~iK<+>A>K=w=Gykmbgz1$FBM z@x)U4y^yYFp#p@;!p8d4AZli#^Np$lJ<_JORNnVRuN2e}|2(MW%>z64fFIuE)_OvD z*zrme*J<*r2~$-Ic}h8`1;Z1FmS9zQuyuXuhKA8PNtiU6ndd=JUXsz>1R5o3FaG2ROe;wqvcBVu@=!ELbEeTYFOB!a656>IojX z-2aPN?SVzj!AQj4WFa*!tgBLC_*^Y>5JqdaLJ`^+1n>;-}BRCm-zxU ztuB`>ESPe`*R_h=o6Z#0HEuU*rMV=7VNoJup>|+dU;Kr_W?mj@E81`iT?nVM>4&W8 z@CRl#>w1Xeq2B!f{IY%&YY*-!w!oB0-#Pg*d1S!fXY%(S&6zx+)%QUylP7wbGU>m< zoXHxnD2`o`QcN8F3c{8@lSM3a)|~DfSV1|-+2d;_EAUV|I6meFTH&CpSz`t&-9EV* zzsS1Z)IY7phos7@c%vj8%L_@qX`c)QskmJD5-Z%EKap5O=ga!f7v}!a)}1;P(Z#zP zmw^^yYFMekhMPVYLsOeM+wVrp+cj%WnMHJ!#o?vNxu|Z5c+-oUmFRv^UF#z4mJ3!~ zarPuWDx}G4f8-n4fj-xlXndgRe}jcb7d+`DHa@BH3D6@$30?SQv*~c~%VzGlY&L$m zK;hMeVur+}Jln@d{^hcGKqN?&Taz%y;3KK3TulCg=O|7Dj{hKJ;N!lk90mdh9g9!O zRX{0ZLd!R2)03e|iPn}Oua+rOcv}{GgRg!%NQ1+}RCvh}3u5;*y z9z$5qIaH{)&aK3}E?Rru$EH-(w^@X=Q*QFzcw!d{ERJx_OKEs<-QWVRW^=DJy5yvD zrGienaz%Gd1BSncls3?>w3DuM;reOZU)$~}jiPCWldhV>cx^ne zTnW52i+**|wJLfx<*zYC^r9-xh^hBv*;?Ha7R7qX3qvsk( zijF|F?9nEzG&{WJ)zwwEr1B6%_cwd+D4>;j;2XY*TiOmC>Si>RbJK;^)MH4}(zoSf z6MXB#_uGJnMfN+42O7n5Sd{2!TIr^1)WQ)l9(F)aQ1$6NBzEhL-(I%Yqw%yOF4Mw= z(ry zcdJq@n%a1vt|50i5N4q-Gfl*mBIrjCov(Ow3jOM#3m58M z1Gj`ieqa%0%Q7{1yncx5T`DFSErJ5y^fok!t6T7OMl4o{^-FI|bEyYk`uQ?W)W zM-#ks#S5q9!~Z#renUKlgeZhJU3C9mnP-m;IGfOP*qZ#gQo+Ay5pBZ>iu3XAD?yt6 zM>MWkCis-%6oHOlIu(Cs@BSqaiqivj#A032J~ZWeV7t&1mk)t$15KT3%Y$&2vdBu*KKM+I!F9vEb}7CGRjF-!mXv?^#B7&t`kbU@>CngfHrU z1INtt7L$i;%p`{9W0FD94p zcRzT&{FY3OuZ?O40cYZ%^kDmUG zHoxEJeE1ol3&E-Ao_$}`Py4^9b_c$w&up{RdI!E3)C&Bh_Usq6euHeaZa}tL!)!in zlC3)8Grc>2+e>uJR%`tmsM0B0)d4JYUis`6++R(@H{!o~{4?J7Zz6R7?i^Opt`OY# zPaxbvXRuhlGar8EwnQ49@XrYxgF*~gpB{C*bu;S7B1!O+pOVvgFTT8=ua zN{(viT{TD5!QxasM-{zu)YEWpshy*)22oGA8&%Fx%fbH|Wc=?}%25l#e796KD@W1>4X3fn}SN$7!1po8-Xzt{*|GTi?`5XQ*vFCpi={4XcUC&XM z09pd316JXRdElKKwfC(YwdU;{b@J~yswec%tOZ;Jv<2)0+pTi`?1Y%TbrX%mv&9ku9|BOo#ab5CE-3 zQGi#-L0J;EphMwNfQLP>z zvoH;R;_ohUYKv;*)%gMt{RVb`EucT3@1Jl3TmfSNjo%@H_ozaccL5UtK5%yhyoNmm z=4zN@U=D#f7Z476EX>~ktpSd34|tFMzXp~vfFJ+?b^vyP;Mc4i_2uUr^^Z?EYQmQs z^(^f30H4k7da$p7Js##Hm|bADfY}330`|5Da}f4oQ1w^=x`f)pECkaZW(Syi3g)VP z3K{s{U!j8L@eQ}`JNNhWt-tecYIbjh*%1(DldG-()BubJ6oHD(%1|oW9%dl}EWWTr z!8`+{zAK?P_`d;rXlng$=)b2me2@75!~M$XYo-l<$^F@H9vXh)ckb^qxAr^#aC2?c zfLRC{Z*QVDZJ>Mcp8@Uvz30CV^gqLVAIE>}HvG%n{pSh&Q>MSifA2PYpVRLNEN*Ug zEt#wS0EOMYfX`)eRS%fC zQ(yw2?na>e@C3L?@IPnvZv?Y2+>HTIuyglBgvk#(;MWnb%{*x&S1A6`()h3}|Cv_5_3gTmdZsPk=XE#NU3v zQNT_>96Z_r-osuh7^5Iy3!tmngGy`yY?&gpLNCosWyQjch4p?kOnBcZ0BKy<+GQEJ;>}{l@DJ!pWTvs&Ru zO|Ci?!0ioSP6l)e%T<4Dm8<>&*bdV^Bv7B(UFuM;DmC09FFHeKBAQ;38l*;HcSt0Onc1mMEOW1Il*(tG!)E49s_&;9snCaQGf){gXS&F z=773;5O6;RtrMsjv&roo+~MxI7YXe{rr_RhHo2WcBHXcubJbpenSjE;8Ah2c+`(Zj z+&2J!9>UmtJXcLfH<{ed;RF08W`GwI0L)o{Du7pjbssV1{&!$duV&?{#!vrQz*NwA zf&LO82M~|M{@=jx&k>A8V*PM6p%TkeV{>y=H?zs@9Q@!eP$Ey=Rx(e0R6S4q(`<4( zha+$&!M_U3V{rF3o7~Q!KHL=zwer*zwe!>#Fweq#2&f3V1uT9R_y48I<2LYw{|P5ll~*Lt(Br!1FWU3kZh6oC%N8Fk8Ug90dIl;Y8Rk^9W#{)hSQy2$~uIS7hV` z659jZeZWc3mjc}@n2QldF~E8F{{c7*um*iI#PJ;XVW680yJ0IlS^$xYKx=^5gwkAt zc>v})0LNLt;%9OHADGzxnG>)?&J+J0cxE`=_jpU#0F>+)z@{@w1k(j(XF!9odFq5g zdFt|^dFqlOs2!MHxH&9OO$3Y^hV~B|o~OPC;fxU`liNAu0iS~a&lA8SaC^a@n-&O% zzY8L(0{@-B&oi6c&LIqL8VQ~lm8TwuJ94zifwRDW9+;A4vo)KccT{lU^;@n zFU&2-L@oHaAZ(2(ph?VAdjlrI-+f%3S_ffkBh2h%^nV{D{s@kz2sj*`Sx8_%0_DO! z3}zGpjYYx@VNV9#ZbV)fk*$II{~ZwFe-p_8Bs>ss9hn_F{lAC*5zg>8k^B#$zoGee ziCf~ZnEym?_}-)czey~AgnwtZ;4CJOKfnFE^bA@x{tJTtRW*N?$iIjK?f-YgJVPhI zbb+b=J^Ux(zS0=}7YSH0Wr@UMTJZnQ^v;*2;qQp^NBDPk3(jKlIP%;7U7Y#+4gaeu z&KJS|Iy}$t!bG(HK@21T7=)IaO>XCK6K*?{Yylpn|2wFiX2OpFjRh=pUS>aD)r-t_ zi{E3jTZ`N950N{V@o%U7ed3nLEE#-l≥)`NJ$j_j3eT=+DB={cFMGnKP`#AGcTt zV)N$?nWznzp05O!1iv>wB0`Sikz0Ug#u6_#Et%x$nbS4=g@6-v1mq8t-`*=Lz9M0bJP3-=tje}#P>?8z|8!8{DyK+t@E=>YQ)%uO5z z7!MFZ^9HnG@M{h85^&XEKL>ma;12)+-wjY1UB4hJi@n|Ksdiz^bUau;(0(GoaiQ5LCn?prRR~ zlA?J*#ao(Dk+)2-(9}#R$*8DsdSKmWfUkMG{^T6^vLWoGsaogpNQ0}|gf67eq}Go*6@aidt5crr}P?P zk*0DAoAz#jpY&Z;A}t%2N&g?eA2&DZ-SGQ=jO+S;jc`MK{r@A=E3?u5|F<0f{~5+E zH%HojX-lL24SttiY}5yAZPbV3*BQTuP+Q$y>^TeNfABkqUoJ}WDdQkS+7F$OmodF= zlSqF06@mX0@)p1YFX4KDk^J-vLb&mxM&08GE@*(i;^z-P;Vrr~%u4VOn!h&@ z7QZ0DbB+1~_^rV&3O|A0Px!ropWAtKCdPY!d+}R^--q}eVmu2Qhv9!7zn}3dXFT~E zGWZ8BRDZ|*-?s{X4itgT_`Uciu4I7^1Koaa)HhwklnU&F-!~U-{5uaDru?BMdhOvK z4;!M1+|(l+TZQ%?NQolg2gT($!9u?r{POY3#A%i{aoUA`$M7r1?|mHCNT03i8z0p5 zW%#YZuW-4pJ61IRy9=`EEZ+ir3ct7c=NQ){(r#9Ih;bG0qqlTD@oindg5Roxy8d&q zuFpD(bC389#&6Qcy8b1w3O|uOMNJeQQ`Ng7x;q!mS@_ueB%w6Ps zbk@TQ7tfsUoVy@tR;s}qymzX)-eyw{Uc6V`;obUynaMLBn49v1bK&AykItUI@Nv<) zK#dW}`D)AH#j_shwJ3Sk12bkny71u{b5mwLI%C1CMJcnAA|G8ON?ub(h>(ZWuy&6v zn3W1G@J`a1nX>Rv=c1IEDYNj|J++hjqDLR-_3+HO3&hHKYM*P?JBl3-sqbI265yFU z>(PfFL#(+A7vOJg&sR$h_Iy}Px3`U%+q>V~e$jIm&3$s#V=<5Q?)Mn^#NmbNPk6_S zoXAa9BigLNX}AseZNu+X{9=AP_(QTf{3iFleICQVe$fXba#Xu|Fmk8br>nYRp01b7 z6|;}2oosW(@?&Zrd*(dx?lGK`%$|4fhhu76hfBd6DfZ@PO zoX{(#F#h63B`^!<*j20h;%s6bFf9=Jd9`{rFacNqEC!YUqhG4kbAzA(|`%DA|n2XSr#xA;XGg#$$|O6xIIXK z^neAyumkiDfgUgl=>J-+J|36=^y~yXz@D%dx3^X=o{pD;9IwOE8990b83{$s_n{_$ z8Tlw_7z`e$)s-&LD?kF>5D{1jOn3_!?G8O))huKj=-w0Zw`+ALuoxHvj6R4OnT?Db zs@2QqA_KsLc?iFQ0=Nbsr|={X#CtRE*6Mjc$9uJU#vmjB^qh|bKC0EvBOHFDR*xTo z3;~ZVfZ=1c`U-@jKSjoX3BXcd9x!SlGIG3DuK|XiKmlSA4;ViJ_CJSRd~~n?=!%0U z`pa5<&Rxhc@DMPgq*h;j532a9TD=fh`ZcO-5_0@atzP{U@LN=AL8TiW)d(f zt4@zb=dR3h)#(ZFWNfU{bAegF0-(RB(>>AK3xLtUjO;o+6<7?+2bKa$fYrb%p#O_? zdJINIDKH6`w53kpO$lwq+zRyHj`MZkqk#p$1YkL^zy*&xhD&KK5&~8O6M$Je>hz1i zf}O~SF^uq%0O6~3dNR&mAI2&|d=U}= z`hScJ0}Fuu9g)GKb$To??h~vkz?5Tk`d(nhr*(Q6FyT1P%yz{1&w^)82on0TPR|A= zeO0Fy0sX(O(<_1D_=raY#%)3=UUdRye2W?Y7N4xsOMwO7q2`>(2rzj%?3AHKXP_Xz zz%KZdUy(2{>9;z48u$WW8Nvw{>U2kE7ZiR+4l(`<{zL?fd&ghs3&4a+Pz07!gi@pMRTpemI9K2qqPX<=n@EHqWG`=!Z z0J#HSrYUrRC}>fy_rwH~qSot^fzcWifu+EFV1|3WUJi`&sMp=`wFgJbdOaFg3`_*3 zw5r!v1EX>DICv&9>RqoV10DE+&mrQ0#R%u&gFycGgU3gc<^ZGH)$6W8cnbQ~>%I?w zxTRh%MT7#N6Ej>!KO{hWOuc>#m;$T;CJcZ9nh6KujC&H~INn@C;UV>U0^S498d|UC zfKQ37*B!H94;T$h8dk4oL!JlBK{zF@-lgvakua)W_rz412TTB#0_PBpM#8{qU^y^h z3~HbUsuGw0bc}@_FdUc$j00u@lYn`^4B#PP9`FRP7+68!+v@f3n~<^Fa87m*JjEd5 zfz`kypkrLUo(fFD!EEPj$Z-xj9T~KV6qrPF{PcP~7nn2y$6cY9kcbQ; zoC2%@W&s^HBVk|=&~ZO%?m_6&Iq5Wb@)o15fz?mc>(fZ_N%R3=LK=8r7O)cXtaOxw zc)ZC_19U7!0U{9w&H=_PgFRpdFb9~m98C&ztiY#@fC)hVUPyQ~3IKFv;7D;2JdQ*4 zdKR#t5S;~>aTo~z{g0rkqEG|H=nFvqkI|NMpa)C>`X5D`0;7S4fO)_bq<0))U@7n- zg+D_R_eKW4uGiy$4jlGP>5cJU4No2s_|9fIu(TXG>;r>8*6Xo^I1!uzECuEO9hLQZ z8L;v<80-uFA`Ala{zS=vNq^Prdx6!!GU6|z>Z1{^uGeFL1y|6tz=Rq!Sv1DKqZVxn zBBLH924?B?dc-Y=px`($(BIaeCsP=h3rtcQ^b%lnOPm($hc@=aX<}fwH}rs&z+7O4 z4-V@B3)(=hKhVEHj{*AQE7wWD@L(MHbdewervic1IA+{H3jJ^-F$OsV#sTB{BLXl9 zm;ua-MFxQW!yEKUU^K7>m;m$~fEodwM^jXeK!S6D_u-f$Fd-3#Jm;anIB}c?3{Ps% zUEzoj2Tu$z512v*aBTSmFnn%S`AJ~Lss_Cpn6w_}B;y{2!iEMt8xd20dx2TN zB48e{6j%WC9}M{msA`~p77{?bxJ@|7iEsiio#enQU7P&EsX5HK2oV&EWzvvQFDFnT942=w2DFy!IDYG4Ae0esR+XtJS* z4~zp=1E(!SO}-3$gdMvPAK@%u4#IJ-q6rZWckRL1XAnui3Sb5>I2Jhu#sR}$!vS7k z9I$6HGV(@)z5?L{;BH`6J~|!o`w<3aya~gkf1p9H0ww{=!H2(tW9q|Da$x*$WawSQ z1G>WD$%H2kcnFvTECc2NYk;MNIO#hA^24ZdVBUvF2dkD>$kyFAqmn;t@{z z4M&tH;R`4b!f|yt_ya8P#4QWb_riS=$89hGPfwtKK%<@j%xaHkpup%3xaR>Z0GIfIB(2BcXe6`x*uuGjOH>;bPzkgi{i6xPrnnkq|KYei%-L z!C82)i*UhggeSo8gGdM%{Sa>SL!P$)&&LstTZsGS!0==o1OOI1iu)uN;c?)ZcJLjD zfN(Cb5;%t<;(6IIgyZl~B^%-JXL0Wvn6eDFp1@ZFgW{pLyitz@#^D}$4de;9K|dd1 z|5eDq6OgaL{c~W!THH|rCapt_0<&D1jd~Uo6L8Nm{7Ga4_b(>{^MHlmD{+G@3*l1vl%bCR9p57! zFazkHfOtRPBoHtKm<04MN7VyMfhUOnu~GNDht_|1qVGYKoWgM=VDxDyP|krD5l+F) zw&00C++>Rb<^hv{rN9heG;Y}C0~3Bl9|EQT8-Pi;88?3t@B-RC4ORUI3J6TVZLwlt zG47oePQv)Fgr^*e34fvOC;}lVTt+W{8VS@OgOky=z-b5<^wRZAU{W7lPXHf&tFGsh z99RLq6zKU3GBQ}#2LTI);8qu~dZ?~v5RTCGGU%to>3WR|o&tFMryxP#At+>xgaRp! z*Yzmy{M|RT1^fPLE6(4R@R-}t(&Zwi>=fYo}UZ<d>l~@$mr@1xEFCD9Xjv zl!1_zR97(wmp&`*KdW|Cli;7TRIELVjAp@~x%LvTXRR_FjYc5*c`?0`YDW4|!cdPi_?@e3^?9;H*$Ac}*hdEG54&e0 zI9mow5WHvvKm1AcQ!gTz`@A^(3p$4H&#J@26ZM$F8r_A2*`?{4-2ff!e!TG(z0PQ&W@@q)LCX zgc@(TeS{ap3N?5HWC<_S=>g)5r{-tWZuc=J2l+Mq@t&GH>Ekzt<+^y}oEj)He^EU> zvcT;|!}b$K$?_5O+(;#Bua=O1lQ{JYO3(nmGn=Zvt#F=G@uz`clsW`v<$u$HHypnh z$Q;>X)H!sjMEJ{J#8t{$r?4sJp{LdEr;VK;I{J@OC)1-L6ULpI{6b7~v)Jh;lO z)NXCPf}rE>fsTR7cuclPID#=qpBpF^!PN-j4~UKM6+!>CxN~ZBSF+@e z&`3k5<|8<0tvGxhz4rwC&aL8y^J<7%1^@W9q7l4@XG_>a2IGvHYcmHqALgn08ij?Z zV4E2AD-{<06WhdsU(wE)@b}y)w*9Jxcoe{&jqG^H6iW~c&J(A8h2@LzJ9pRV>&2Ot zT3eqbR3{NAu74{;*)2kUL*>W9U$|RL_)Q(Crj!3wv6lF~@E7e7@50}`4F0^=>h#YM zSDD6r#s0cc#Z-MYkSE|i+(20){+<}JsLg&d7=_?`1m$4pIv&9y1P79_CLNF3Y7S)K zFyw`g!=j%DWIMPrWZP#$?AR|}zJTf|fqd#?Y$D=O-9)BZQM)~AJ(W{#Mh_QJ}sw#F9m=8ZSl-Sb*dWe zjj7=v{vd!Fsm_5v>8NP)yE;VO3xCluG4Xd)RT=!HpNeP6@9;s{kBc{cR~M`4@L&8) z^!Nk%#qh_h5(~)hj-?`HwRrOn=tsj}vs#?`L+#O$R+l*l$9*pD`csW`Q8*XjXHS#q>-zJ%~f`$|w!oV0@b5Vcs9+9TevsztnB5-7dU?Q0-U< zhIz-^XEzB9f~s$8G!}Z(S9V1N{;BqE83i()$gjm=WJ=9|fA=?{UzIwrWh`XJ5H2pI z@DSCrjiN-I6i-&EqtwapNB>V8twJF);Gg`XxKyR~Q48UZJuQ0t1%1U2gMCSmEsHdyBvZ)4!2Ed3@W2(flcssKO$GjQphIb zhT~&$mF$06?c(7KzzhMdgx2$}V%24}oyQz-mADc4i_{CORy%w3gt-FnG&=eV+uxY* zzNuDQcdP_A8JwM33l4iEr3PGdK)v2l-29j76!-kC_A)9wp)Go6fH?fOI;4Fz{KY}_ zdOwOZ&)b)J?_O|49Ymj6b>duVJkLN>c*lBu0_n(}fZYJ~geS}@8|CecGyMuWjuMMF zp2!d7SBdj>5=S@b$G}b)anog5bHF8mtLAc_#wwX!tp?pj5spFV2&vZrigK5?FU>sV z2>61d0$!h)G&2LpoUCYfOf7LCV(mZbC|3gfD-f^g#%d0@EDM(gF4w|kg4=81^1vOk zaEHJZTeuV8N`K|z2E^e<;hfWP|hv}hHy^b~8n9N3C2S#Nd1U?x&t*A7~ z5f=|`KDh40aiu1KTLBJR7sHLm|Eb_II*BK*sJD3>gFmS^ZpDf_Yf!g8T~R|GME_IV zQ4z(@*Xl_WE6zKSHh{!C!3VFzvmnVQ+aH%aJpsu8-&Eho6ZmPc4=`K|{!c<+PVaiL zu7!4smW#k@1U_q_^&Ukr%0VRK_Bnc|;fSjOmj(_)`x@uyfRcg3yl1#r{2v4^qpxUL zqYi9ELyMkvW{$5HK2EKph^1wyHfZMO{&-D^YEk`oCoJG*4L?FBC#HOO?m_rGeyj+ zSNpY(hu`x))T+_^SgN6)1TN?S@ov4^!y^m+Ik+$V2EIduEnSNS)kiG`cYd{K-+(z5 zkBO9wwPHd8rls%@@EgRF4Qf9%5&mge;_U{ke=Fek#|Eka+kr#yC%h=Uud0zjtfWM#L*;;u$`_X! zQG_!1!}p6w9qOK)F}M!a>u2SbF-=#4JmSEuc&A?f$k?^Mr6aX8P|iZiWmjVtzZc>B z_cz)#ti160AKnptj8+~Ns7h0$XJ?Ywu zd;Bj^CWVhp^H7hG6rYc`X+6{$_%n`*3FME!T$glQJY&;BHG1%s^I5&v9-;-bqy|kk z2yqmO9=Y(3KT)qQFd8&y%)i?f(ky{=bqT&JBOdux4Qls<+qF&2AV_If+Y^rj2H3@r zHd>%~#IAKxV~IQ|HrbJjWcc@fCyv7JkqiHzGTg_PnRlbigDNN!A#Pe94|;09_kZ>J zWm%)iZd#BU11{x9vDOWhlm!3o)AhPdHs-f(T42i@aOrr&u+~!?uE);Z%c1$SJOQ!- zANB0*Wo*R6V31k^$f~RL;)AnlJ1w}Y3@mbJowYaw+#Bn~7KheOOG2Owff<#Uu}U18 zL(5?1e-3S=s|0})8ot(Gqrzi3z^@Wqga>YG8jR#>z(s?@++sMI-aNZuZ36c+F-h3I zO!lVDloAfUrbmO0Nx*Q_>tet;Zo;irVvH+aT>mM zctB5zao*`R`%=u}SV!PyRCT%;RJW zMVyY*!o|Jr+C6Gw4~(>t;%#>=WL^gRsn68vw;?HVR0-MO3hu!tDGWw(d%+cfYdTvO zfja@tt%+VKxMSeBO^glHd2m_FCfL76%=6H?YO(P5oY)|?{H1p8O9{^hQ8F1f zshb!~2X_J-W=X>lW`Qf4+#vq!qP0^C5I8nnv}~!hS4*4ttDE@!F?ygg4>}&v@TcN= z4*mcFU4+rI$P*bjw_*gf`3UAFH3)ouGC*B{K;HA>R7>>u0*Z!(&rf{lsRgQ~Mo{!< zg;cBIk4O@?wbDY>;G5C0W{XG39|!-O*HL}FRSerh1qMU!hD3|%D{vc!4fQBO)5 z{>k&iW=}0drH7jZ^YFD%WOW7nc@K+In-qk6BYUHjX{Y)>^_1^M_# z8;qgYau6EVw-*Lc>cQDw+CZB}d=%y-JlSe;ncpp$In4V1=5(T5J}9@DQZ7!PRXZ9t zhv<=K#^b`<2U#nV{z&rIz@PWHnBaq2boNFz@#yR!V`UGx({^nFB$M%wdOk1j(x6F{ zRD_G46kqsY?%NH2_0yt}{AKV*FBOr#@VocH;LZ@!eYMb*QSj$1Z_vND8xxp+vAp)j z092+y6#qQFPeL{J6Q)vb7c(S`NRmwn4v##(>)}o4MgY zIg}d6;?{`?t+mLOJ^P}otjDt;nVi_#TJ!Ok2D0h}Jj0XyU@I1b(; z5B}XM-XNEgcYvSfqn3iJX(M|0A)W(EeoR|2)lZw~kqCcq`c>>Oj5NOTLmFA&veU(H ze&}e0@H>}?HvUNCJp38YiXr~seQ_lbyp(i8j4Qtwg!6Eh%wLOC=fj`8hWtH@3mJOk zTZr>{_sg{?Kx?g*fNEGHx&~;YJc9dUNUp^sv~BPJ>{PzO_YVB! zTDT?<362Mu`-|8ah`LIJzxurRf+FsQKm2#mh=@0r!%wdg-%p(fSJ?7)+OVZ=_M=7_ zsWU1V{d*g)>TWa&0!4C==Bf1rm#JSBOM|q|Y61cgir5pR4OesEk5$B_AZ%XB;rF$P zKJBo=``?PrW5cTgvR0S1Lutlxmeqs~p(8KA90pd64h>Bp% zp=N=o8A=4YISGe}HXXI$9*&{tD|qbIq(7=L;0o|j*N(I|fLig^3%vQkf`Gn|4e)nO>*b`!UCp391{N;EI_mOcI zPweZYd8zZkr7xu>4>e}tY=r$Y#2=lI;34?4GK9Ai30A-#ysS~I^VhnIL?^1pe>kdP z8A;`h_(2F)EfX7{ttP?mzFfRZ{?+iOKPP^2q640QUs)kSJ8J_yJV&5`@lw^(;=>R; zu6Uxe*2*Iq+#ztuG!wMF%beY*UnN4+0MTbs{qxS~BdftxJz0z0vT>`pp)>kJA-JeC z(IyoA>LUEO^w#0p*sNLW=%;cHk(NJV94*rPsVsj`8n*@K%dhss&9vSeD!DGcH zx9in>aB1s>8itJvJ&jM^&}giN#thaV3B6KI4G)KDKBMr6Uder`R!7glAI|21{4k1i z6g3HIJmf`BH!lx|d_Lq=8^lLp$Y>V)hg{+}_%(W>ue`{K2f<%134M;dIbMSxuHhzx zqaeo9eQYf8$7L(=1Y{IC^mefl5;XyS_aDTbaCFEF_$Rx>58)UE`S8#GNvK_rY8m|g z7er(iSgL_P1^L%)uqwGZy|8_!JQ4NkmFmQvxbyez(e zB{duV(#zrx@*je~__FZs3V)gON0PsW^#2wU;P(g`Lw>yBEB9;B2$mt(M%aF(?Hht= z*uUN>W_H3gPf=IRvuhf-d{RN>T-zkwE|&;egmVnJxYSh}(&;%x>Cgcpe0`Umvl~wa za+cNV>)>}>2p1jxQQL`Rphm3lm@7qy_qVFV$A7Yw%|W*SRgXyZNo8XMSzlW_shQ zj`D&iK{wawt$~iw;KX-=y{QsevRFag6%h9wFnrLU9H;Fz+BvI7;JK8bJjf>75BuSyiV~ zXLs}!p*^)o?-Ssnv+DFajAX@ha6a@{Z49`X#C;PXw)E6)_K5?T1~Qz=v@1gFJB_<7 zbv;o7dm%}Akxh&fLvDuI^Wai(diVsH=pYtEXa20k@h-h)9C2YN9$u$5IvAm%IoxxD3K9euhsrdBPg@s zt+4jrQLB%pxPDzNH-#U%?xyfG=p;aY5Y8zlQl>8>>|GG zqYWM!K^8r+QHZbAPm%wrF2(>D?VV}EBa}>jU_vv9@${P>Qzha=^Pi;qU>DJ;uQs3! z%{yrj6yRv{-I$mhw~ABIT34TZa1}Uxd+Kh*-ATMkj{BdNKqPM9_L4AQ~54q);z6-i!1%Kj{SXouuxB^)t@1| zM`XHj-m$3B30$MG#*7>Ty)pmP%emzM=GOj5Y6Zv`M3xh-*xX-h@0|}W9g6rP@rvUC z@lk&a<%{5w8te66F*qC}L}PzsDb^QyIBEJlWhu z5TgzAX#iOaazBx;gbAMkm=0oFV^&0$>q^E~g^9ZdXajs!fUJS$C#2~XPJK9Bd@w-k z=u-wsY7o8}iiyDSzcAT`#4UA8S{gf0f~1iBthW~`Z+Q= zRID6`S$Z$HtgiSFFBPJjI6P2m?^6n{94BJ`CJR2{q7xp^i>m|Cm)!lZ?BH|0fz(`= z!$gN$(S|V~gQ6PrwPf}}m>72}7Mo;{=^*iEoX{J>#8PP9nhVm|r$NWxJWw15!v5_d zw=H$7b+hA+gYlr*xIKLWx>yE(0kALtSJlL|YLZv`7iKmU8jL-m7l)L-qB30x6+;JWU3?Nj`r;m173Jx> zP#TQ^V#Q#Z+aU?YowN;P_O(!PaIn_hyBuT&$Wdg&CH@#p%O}QB`fwb)q(+z_LWgLf z-m&1S?yZ%J;AnBz5Uq!ID!4h*YV}1FH%zPq=aWx*a~t$df#7hf$e0YiAENnqS3**r zj!)H6@^^@ThM-=2p{3xw@L}rjy^PGZ6$6H%x#J*Ny{th$LOm^vi!pzw=IN6H$$Z?K z*h2+u6)L;PR*JJ5lJlA9;g|v)^_=!sL$yHfN=OFnz%4zB(^vd6R15X-$Hp@lCwLE1 z3W1#V;8@rh1Ig}J@zGsWvg1}M>G+b@b#+e<*W&Pb+tE}E^`T$e4D@Hx$;e)M*Ynsi+zXPW3 zXk7PG=bd4ebp(|alJhZF^~K1iW1N(Pv}k>;tT9L+4La30A8TwqmWGOTBd{Mi1hVX& ztGWxzqvKw#q!{;W5vhY-giZlIKo^B&50%t70=abJ!XbGPzE(+{22BoQor32g+9;g! zz>J7!)1Xp-PrkX6iY6AsVNa9|ZVb-MuB7Bn%eD=1^SCa#6BO@9oaZH-4?2s}aTpH@ zYNP5LK3qpNwY{@=XtdVZk1`hxK~Ef#ji1rcms68L-8>qil=v?zKu1&i2{S{WE82B%+8%-kozP^ zE_TQHPHLWSI?1VF%@|a8HY72(HtL^HKRF;JA%k09-?-4px(8oWqAvVGCvkd=)}_DF z86)liy!fqvT!`$LoBD^Y#WiNi2osO0C{VI=GM+ zi_V$}PQkI^P-?wE8Be@37FniuMjIZ%tL>Ds4;S-ylGH%5;!%9QjqLmxVx$$)_20Cp z_w)?Kte=8!R8bRu86t+>hQ(|e$Siz*>wYTXkr1))Hgw-ikQGnjFl&EF8uRHpw^0rt z34Rk_Y$F?eopPeTc$?PK+dmAYc^}_rpuD+>uH(=TW5E@H!=GQmMNB7g_c*QdBzkNY z^#R^QC-PjA>KnV%xGZzRKo)ey7u{Hgs_qbU=HRUHUv7pSqsT+X{QGDR<5IZ@9mi38 z(F}gapH0#lxzsHj9K9@80Qub-zV^@bti)>dzs^Or?_lWXuz*+g+Lc!#yuNwS=0Zg#o<2)68IFJkit1iX6CuB;mCei~_Cy~F(+$Yt^Gp{j}|dHwJf zlAZJms_4KJugeJh?_T$l&^vqY$5`5S-|zJ?+ZU*p(hWARmyqb04Yp%gi(lPl^SX!}@7iH={}35_ zZHH|csxm*%=JhS|bntbX*IHz%_;uS?m>M=7w0Tt?aM_ie2W=mq`p*>FyjCEo-JjZ8 zHXtV_kK4TdKy7R~Ve=|NjeqpH&Ff|0_utsOs!(T}zOx0q3ViK5TOm?8_k(SVjp{qQ zT=K7%+df9CT&l1I{0+6|&e(oHmd~7(!GF%$4kOV5*RL|X?++R4>pyKqQ=ITrB*zQL)U9owRbX$$h z^B`C1eOSYK~zLG8S?!PfdYSoyfY<}y;ty=r@clG)a13m}CHjkeZ*?6xbb)9tI= zNKn4i9-xD0%&;4ZE0^16euD7H74}cwUX&9Jm>s*SK|AgJM5PBDZ>AjK1cZZ zE%uI;7?odcv3FaAPFTOi?)l!^c4dE#y+tYd&6mJ42xo7#w+--JG0b)OJCGewdcAvk{eNS$)x3~zb*k<=S2Yhb3-T!|`@ZIfp&r*ax z+inkA2mQ<2?LH?+o@@8a2fhj1ijG>5YY*6r@W~zaj(O<+mv`8`D8n0f+5;L9@0*== zuWT0-pWS5-I1S9%W%oP*#n*S)0}lgF?y`HW1fJhz_bEWH$jGyID@8ZT&$D}e3H%h{ zw}2~OvImqyp7)a7YdOM)U$S?*h}q@0m+ZqR!OEBI{yWhJuD*;CV<>EX#omeZ-+RUG z<2nN31Uy^Ndw&F8M!(#)+a7Qj;fy_YuT6XGwhw`)fqP%GdmVnwZaWGr1J(hz?zP*F z?X|Z&^t#1x!1BV@_V$#M ze+uoJuqkn2WjOfV-P#xS*1FAC`55EkW4rs8n0M_?v2&s}rgv{!Po*3aL%A(-g+uwm z?ta-#IcXn>@#EI&4>#pyw|GVQ$F0>;hi!=icVk@2!M>BUqc+iTiZ)HOo{Bs2?%u8X zD(xc0;wjqD0Nv(4rEQT-S$Go;DA~f{J8Z*~rYTyOxNoYKW#r-%rsq?d`*}_IMQeNT zkEvRZwj%vIcVAag9OgZ^#MK7i^u5IS<1kgj&6YuhZl>J~fATs>xHE5JypOa~L6_HX zCU0W?Oet?UUfRV}PtK(b$NypYSp>?x;AppM_*8JQ&F~fALYv{=12@9tsk3lZD24D3GzCrgFK)me1>al(-+;%S=|2T4 z14o92D^+hrS?d{cv@*nl7HT&?_7aMVOi+zab4aXWjf z?ykh!rMYy*Al5QFP%iVz1gYn|Lt^|unHkeCE^HMn%F2A^sSlA0fjaiZ%1Y)<`>zPzvDVcU znK^WQJ>llltT^FB{I^ZosE7 zznl{``{r`kpgDOLcuHTv7%`kxk%UybJ!}92$#99>MA$nU4u7B5K<=!{p>!FRJqZ_u z9N(M@k8rrFTu!l^_{#|9Z15{9NEQbEF9!c)eX}X8`YmFmC9^ZPK+dRHov|({=pqYZ z^w@re;CdblzfluK)CO?WJmj#H8-6Q44LoJ$JnPf6Mb0)##vdSqifA`h5;K_Cj5?(r zEXImGPim2F@mFjwEV1Lf941k>Dh@=YX&$!K;PGM@_|}=LW$b_0Snvz$?SUfkDb0Ou zVwG&%?zhQw%o^Cr;Y3dN5!473JdPLedzr^3)EJdlSs!=(4d=}pi0K@@^@SyFfr2t- zZ<1qI$rafLtosd`?dj^<%F`3&=J-13(gZN^c3F%EIayq=D^!y^_G>s61LjdU7{0r`|kNE|tp4zYjhdytPA|0!NFKl|KXSzugeiBrWTm@q3>W zHrI%+IeZ6X2!9@U-Ar^`C-%*BkIweM(Yr;FQQ@W)Y z{uS_jX%AtIup0te&6?wBQ)=a3$7t-*4DZ0ibbB-W64<9LfV6}9H{1S22xx+^Ch!7y znjftE0q|kX@Na>qJ%d&MJ@DPZhutCPk9hcv&hRo2GG);n!oC#y23&6$`-Y0Autrtd zh3^48`^|qub+;#`%9X>A?6Zsd#ac&KVkIUTxLn4}pJn(jPAAfkD#~CWEf-dP;0<_c zgy!^TgQrV6w&S8V0Svbg|98XBnm{r*nh32tEzwl5&GBQJ;TM6U9&o)rTJRZ=%_UF( zzUzM?;Du>$%_Y1N{McsnSApwv1HKoH;OldwC|ki%r?>LYf}_2wmCpdzr5T{rk19E?J*xctt#PbYpR?RdRrcTl%hepH?z*8Gq z`49{i+Adi6Fz}S9l@AAhU6aW2E6Q^aP_C>IN->e%+zfvl8zJJKyi>L6 z?`sjn&gZpAm%6|Mq z$X5P!@HBl{`8x14`iVDYoS6uaqxTyrr`8B=C>ibFth@*Kfz9v}!J7#{teE!afT!I| zlffWoJk*5YoTHHgxF!ll*p$bz+|-Lr!^RA*g0Gf~7=2mK##cl{bt6wLVV|D z^bayWW`pcQv@nyK$l=>%P`RH@a1BSm?FFf5PLO%5Xjat(=ybx*%8vm@7aN`&KM{U% zw5y|`JO2gIW33ix&z~%M@4#CWpFgR2+EZJIEos-M|POptrC5=OSJZr z(L6m{dD^n{N0RxQWDRwtTp@jh9_Z6{;W`1rbXJMRs+E5nJoRoXe-;VTR`Mh#U~WMs zL!WBJ${zsdqRGZ4WRA?0@*!}v@U&%~Zpe|Fz+s9*?rRQHrzGbAh@ql0i_lkh3b5qC}LDD{LUt(2|$RWCR=JHE8hpbnyyg3VSSknI#(%5M749h04pwdS}CpkRq$P# z;aj4lH0-VVbSAzzJ{^2_>fF`{OCX?*Y2`P8r|I6xue|}Ea|1iYP4p@KYgPV!6G2n{ zO*V9atIoXgux6wTqn)ZgV5{s?W|M9MN2AWle^2dCou(N9bq-|`8=~qa7m7}EGw^BV z&5V!@a>>8S`NnC4{(bPD$$Zg9DQ7Vb?TVFIn)lfBmnEOcIC`5b0F6L$Q#mYIMM<=T zf8&a=$!+zcEH)>L=QwOGZyPyGbwchF4%04$T&!i6+H$*0*o^-bCx?z`ILkBeo=Q88 zK+7Sy8!h-zjVcoidRY;PjQNa3j5n&_Dl=Y#jGum0VlFQLi#S8(=)XY; zT-P4&f6PUsEiXB1gw<@&oM%7aF!c^{7dcELo}8tMl+L-bN~5J%q36g^ia51NYvU1P zRM1I*L3PmVT1)REUQKY~cOuQ@zO294V$G}x^??G9T=xW=ysPtEn2j9^^1kp5BpXp3m&b67u$FEi)xYX1adEc5XO zVzOVwf^CfXjPEmk##q64k?{(n=Vn=Rof)GThcMpB_&DP}#$$xE*qmnK0%J8}BImS` z!vR}lLXnJv7{@Zk@Emy`hf^7o+0Jqf=P>RE8jD>a6JIl)X8es&c}*rf1DPc^gTrz{ z!|`_xn{A)VVRMvy%3)ga$))a*_U@I7H!cG}I`OMwMQT@a-g~8@xf7e?FW%jvP3%)( zle{^17R}TQeRD@-ghUaK5+ZypvjushOHdfLy61K$NGE9A&T*R%S%~oxJ zZ6MufpKM>H8F%8VwracmayezGi0eX|df>0o6Kz8?rLp+las`U-ClQ@dI+kAxPLkuE z!|5;lH9Epx5_fQ3t~SwTFB>a9+Mz9WndLNN5xUVeiLWgG^?Yk({2R^jZIt+;=6F9P zCeo~ee=WIT`;jZ_)94)6ByQNqm=%4EgvSl{cAMt|QSr?s5Tukf%ePaGt!k>z{mBgz zM%kLC0-3xU-6FNrD@ojNC76tFD(uomwM$+tbjGd={5WEoA`O z%Mh=T#23t`GoQFt5;VP%JI1_uta6>Cl>npl|69S`z>|GGrs-1N^8QslKNx05F8u)870w-WDAYr(OB109- zk77eE@N~J8c%31!spk)~84PWGDBW%;+!755(kwli_FyjzBlV$k9ioDMCr5 zBuu`J`AoJ+PPg7OdZnTaxoHJs5Ns1njo?rSu*GSR33McKf7=5DDN&-3=D9%z4~>OPSI zdRL#^_?D8d;^Q{Pr=!51VBVLGW57Mc3CCf6COf&gl6ac=LguTABthR2AUD)g>c_jb zO9Abo$Q@)s#S|HSO@d;>7K;+VpWZT<66R-{JRQG)Gizus^Rde%zlZh71lcKBDtQ+@ zfg(4O1(``QoF+j@W4=g8#ow6!2t2jr3T`qsqnu^^bh>Q=_b>}a_{fBdE=l5h2`*(D z3#$H<%tvzADs{}qzbW~bB`7!ha@8G@ggJja!o2T0k~eGaE#@nFNW%@XYbf*vxm|Ig zw-?9=MMi!Vh2A1Js`@9%)4_6byO=NH1PUaroMYbkw^XcVp8oWtVTbMSXFeCa4NI6W zZAxf?K}HPs7e`2cR~n*MlnjR}Su*70reRR}F`ry06@TQK8;e~h=|^3X1ik)G?q%ll zX@Y@k#LrmZzGH#!z0wffOefdKyz?GeXPud!fJd2>KswJ%iOfI8yzeqeoRwyjUrc_M zB+LSY1R{MhRKbmF8k&UrFh+@&$OuE&&<^GkmtN0bU_O1Z?MucS)iT^UpA!ezzoMbNb4M zEQp#Y1&?urYUWGskp$aQA~8`>!X=|*RS#tS2brh8;z6#T1ZAJ8KV1@EabI!K4#F_R zB^b>ShA>~nIW-rRhnUah1k6Qf8}m_IbLh*4JIQ>$DpNF@xJR&&F3Qg{)GZw0IVn(L zd9J>Lc{f~-QAU*nH-j-Vbdve_iIRVw^~11*CjES#SInB&%zSP&)g02Fz!7R7pmn{_ zvdXpUBn@T8$f|D5RkwusDlPzh9f#Z}%%@+F`Yk0W_c__lc}ZL#J4j!-Yzlb5-^LN> z9bPIyIuDyu&?WaSc*^LpXt`Qah}`GkX-XbrSw-81;vp=yu2E9$-_;eoiv%$iL1>sX zG}$t@(;K#wA>Z3&RUhSyzQ_9UmV`aRrGCW(sZSTOS zM^Lz8vCA+VeOa9~j%BPFqg?ffZrr=AC1tomCQ)D(3UYN`k&VL+)mbb)(}L8&OO$yopkF8IypYGUygf zYV!YYbA*a<(hz-YjNI?6KgXg^cjAqn!7n{w&NJNI%$qMs1rke>TrLaDx1^>S>eyp2 zZ@x7Z%n48LBNK?_x2Vh(-O0T9b`|zOhWm#3syk(P9`hahB7LJzvY~dIKsp4(7hRHF z%p5}-n9sdSTA0m-zGGgwQyMU)Z^(QR(WvT4l22g$NhW`{jmnRAw7KoqvLK2z@8b*| zXFhp~R5Yp%@%*teQUWDYrDA|#4If2dzRJSC#eC*HQn8;=|M1%}9grbkPSLE1Sq8Puri*-~?c5e@g&vG5S@ zd{FZK%+F+={uUZJvk7)GA3I6%Qza zJ6S54dykza&l!D{?bxyHq5@PgAICK@g!x!G?Ek&R`y2~=`Ld`dSJD5N&wNZ;Fn8KL zQ4~rbK1K4qIf1#%$1apa8(xvNGoMIbF@jrf{Ezc$$W4_36x49sp`|0;dJFf9;Xe9*Ht5+BElp7`I_L&@k{&*_!&Ydy=bHWK3 zLkfOMjiqFLz>|J5>zj5yVSPvc>uchFhCXg`RB(iFnU3=4SebyYCBky%ofdvGcuKfr zfGoi27P18NH&=c1uRx3hjN^Uk*=zZX2EuM8f~C1Ak^9AOD~s_H?O1U9mMJm1I)XZ;qp zOZ`NPemmwfE&R>58|nKgc^1JiM4%GKe=Kv_ovW^x4V|~>mznytQh&Oa%+b9Qq@CP< zB)^t3`Vx35m@~$8eW!SbBcz{|5gy|R9gW`Sr$i07UcWcpyUefBBVO9^t&v0->kIHy;tGo$*J~_r47|Q% zA29E<@ZT^$$iklkPX$o83FIt-?KU=@#G9{1hH-|rOWvh8mrI7{8|71uV7@yU$PtEP z(@BQRcPn{u#9u^`eAE?L;`Xes-7Wce3qOJRWP?Zjn-Mm$Akz||ocVkUABIgTWvIl$ z&ttyI!tV!9O_12M=BWKY<_NxAb=|oHZLv$G1S&4e8HK)>MeZ)<&9_SBw1clBFrUj; zhIHPN+!^Mp9+crD5|p9XLmK;kUw5fFi!E(|fD%rpL(y>Q#{am3ZtNBD)TC(%Alt)y zmW4kIp6nFyqk=YU#~p1)d}f@R?Ek(jxQhkJfifrbDO7SRjK=m;c3Tq2VSV$syV;bN znKxfu<+V$>2RkRDV0=72l&?ROcZ@9+HV&2qKIaJLyTt)AKZ@T}X{drr=EHm(^YOm2 z=IHae1>@L~(MIOuZoPgO zw=vdvKV`Co?+TvmyQ(-xya*~&Sdh!l1O{6SZXUzmpY;rTf>6gWdBkl9xzFqn+MB{J`PPKNJv zNZieQ3F~_qcF_NRXF)}@G&G+Rh)I+Qm{04CwFkDIVm_Cj;=6MKA2OdmP}*tFQ=m3e z+9`>Z{3Bgp1a2hp^pG>2J~#~b1{+#pMwl=8mCP41uRJ9A4{b&?rJngjzN*~A{DS*s z!piS*{-Ce_k$aB$MBMU1`SIx@`VVfiDM*nCw3W2-qiKkFbOpopet^JFSz&4Wk<9P5 z@KeAW!;kGeWbXfuvY_a~^*JqLLj@K?7nv`%@JfE=sZjEiOk=`qbiYk=PIesS7J&I^YPD0zLM?y$UHt3Z7?ihtnJ#(W!YOYtpPniCO`JfD$#aHu?koKt*q>!)f z*Ia4Hxn43oIpHxTzeV!q!_PI$=hGQ?xaW=kap#%&*j&lyu^m5LK2sA{?T|d*4pi

q`bp?egSwI zf1h!cM$DInD)8wxa+A0O>CBhl%WUMlxsBZ)mimeGnxq1Z;b!e3#6e~Hq?#{O=CXsV`<37o%9{%l@BFcRsA9D@I5C*XWslkOOP~=ze^?) zPOp;s?Tz|}FpCBFnu($MG)Jgq-uaj0y}7C*7I6uN%Fc6+^(TR+DY?WlGd{%niJheW z3D$SL%z}z;Qt-V@M>)fMdKbx0V!mUFOgPH;@u3IKq94Ny|^Ew3O@-=F10Zq;SnG18)pVZ)xW>mVD3p z4Tip1{}BGp0{_9+8?rr4G5wTq3m*WU5-`7>NAE3=OJ+WkZ&q`QD#uN}jZ`#tY?OY_ zRB0%g1sx1C*ffHtga=s?&Sw1*^loySB`BAeH{Uu9<-RiU32uU6GU4s4zZtx-|9ARJ z!5v%z!--$r<}0pcL8s-Jts_m^?+Owcq+h#bzA`KzYMp*6!6QLh;6tc z(@-Co@F$X1{sd16S6dQxOOt#7ofm830+}zj@He?w5HU z@)Ks0U%*5>B8!k#Mz~K&B-DuV-b3l3732(8dXE=5sM2 z8I?Yq1+TK83IcLMf^w1hSUz9!E%OVOk_Wf%ED1jgo|-Jvn85HdicLl+WImannRn*& z6&(WAc(X|=IwdG0GibbaP$Kw2f-mzc!BYYie6Zzp=D%Qlr{%%J-{2{u7cCj}SZ0*p zPjMf1eW&ZVOhyQ@@V%MuY2inLH%iPWx63#~D_B3qqMyV3c$YHKbO5cBiFQ^^u7cG6kjd>1#yE+ZUc-h3m~wSf(dcupEhwHTTNo=PU)ZB@);0?KEs zpJCBI&HCmmuP>X1W;+Z=hVt2o{(hnWaBQLT-!6^KP||q?IeoS254`FOJ}U)$uSi+2Rwkers~x_3#3>&EPbDk1lsWZmXA1;7PX8lCw zbC*lr%upWl@tjd(uZuXY{x3*F>73L4Y-kI3s_HUJRqtc{idVL%g9X683&!tQot}!K{7X-#I+aV(qvUCx6S_@*hX`+}fVg0C`QlI|B z5ILVs^uM1n$6`MWJlU^$i4;-)M^sW&lm#sCHFh+La#+&JG3H~RlqEaLyiJhDPsy|- z5CEPM$b4PqgbzR{4=}InlRUn!F2~=SEQqB~xxtw?iQKZKq1-2A!dna}PN;#Wgm+sK zzL)j$_e(qU=aI;rWj>x>WQIG)1^z#MT?u>?Mb_``B+LthFd^J$FyVH?^z<>)AR!## zj1UArbroT*KsZ8xhzN@rT~H8NVFe3N*?@=$3K3iz7SVvLf@fS&@Iv(S`+Y8-B8m!f zxcjQ>b$|}v*6&BE=Ko&3_v%%>SKZA_^<8UGKeyCRPaybvka*dP1ZN8P51g^M52b%b zoGX6Yqd0w?;7`)(bfOeF?xWY8ATp}e{1 z5GCCIvcQLtqc#zqyhq^f711|@*R2)!zDyyf&tHW6s+wB`fxj_|m*Vc6Cuwc{T<@s`#%>%cxa-Z^aoBr1%Xxqpl|$2 zhX~JEEIe_FT$;H>@aOasIDan~FAE4iAqjjK$v;eZAw~g?A6F;;gQe-S{4d_s4)O6u zed&TYH)us4A>lKkB#FOA(bfE`2p{6Xw~(A1pD^g6$ba}yh&oL2bT~;J0{KGmtEAY% zOC2X45RM7XDN6`X(v-QL`LNN6QHL!qp{2mxXE>d)CpWY%`)d3i zM`E^A;qlyZ;xDv>;(w5#Z#r|0x9i03%s;0cM*>%n(^qsjPaW=at|0L4c<9x0yc%)N zKj}5Wf48tpdY5o%ufSbbH-AC!7oHF}J$S8>h7qBflKL{bGgwV{;0w{gO)N(UuURAT zY>I;HokBq^DIQ4!Q1M2#J_zvZ{V*rJcb@XoEzkSP{`kb*8$G_U!)E_+}(r(`S%DxH+b+o#JQl_5IlUjK>OC9EUIyCtkdE$cLch|%i!V5`` zo6wdMUUy#bPjaN=F9ivg-W0!Bm2j7IO6^{95p}?iPD#aO%ni-?Um?i-@3aDEdE*xs zJ%-*y{Iy>Re%ef>rG!hT1pbh_DBdQ*inD?cbyxL1p;-D=;O=Gh&4}|kuJ$M{BmU|X z5s&;8iM%{VcnN(u6obMkRi{YMiylGBE1ZtMr19Xb2tVe*dm+v}l>D7g>}I-h;{Vpe zf4|_bl1g$#hhDUZ_ELw7o(}Jkf{K9PcXNC1SB2tQ+CJxqfk~5V#Q-w+%c`#^3u^fX z{3@4&*XKXHm_&lJ7$*n?{3Y1D9IoYmv!x*(L7x+Ts|Wu9aW1E}SjgdF#!K)0{BO22 z?@E5Ie*h64zLJ3WIKsDh@X3T9^56>yKjy(7L7W>@O&2DeWo`%_(0ss&f86gq9TG_3 zmgbGP+I9C3Ug*J>BhD36j1wc~FMi~uj_^YA=ner(&0pgUd}uX-uKx*_x*@@tg18XG z-xJA88R3O<1df}eQ??OaJ5%8FU_<(m6jXW)ZFG>PW}e{RC-NWu#{dzMbgR{t6g)?` ze}U*QMMRis*OFm9toUzFSn%Xg)_(sJPA}D2fEG*gYK(w|S?0Xt$Ilf=$AyB8G=j|cI2r%X#-rdm#5sS>3BkXI zT(OAwbBeEa^%~+2Jiz6NH%ajN6yX(BbkbS}j$*m9jw~g7iGZa&h;xGqJqq3-{+yG- zAo@1Gl>WZpFZS?vBs@vDGyj}bCBi08ha$xJhzm~%1wYWLW-d;@5nI58VEGbp{+Ox%i56E5$q6FP zJ=ey=KNNB2`rkdnts`})bqIK8Lua%NBYfx)VW@k?o1gGE2zSqXYfAX24+Vc6$!SOU zO2VBp;JE*zM9BO|beKek!nL5dS%Wwq!79%Po+Ezu@VLz+M?EU|w|MxA5a*u8Q|ug0 zZjdwo%ZN}y1dIo#yhM1C@aBYnk9ex|!@uZbeRZDrYnIhRPU9lsCp>twW2m27WJc~$o@P;P`-HteyQh~InU(S8SScm~rEM^3{Ni~^tY0W}#U-a&PRjVG}~ za~x>}9W^CpQj*Rio^{1pOKytp&sKitYb}ozRk^5=qIy(R+eP)ZQ>A_Tu_Wc-kyhC< zu&nQWt|D6bAdaJ?l{su#S=u8_R3w%;tm8Nxw(52e~K_&*fYH=_ENsIu9HAJHM*a5Zq#B^pwKMr=A{wHp29$7oqW zna_3eT_1Yg8#Uv zesiic|8caOFP^mS^{syOt&;lI)cRJkzO}l(WxK78N3q47=r#7gU5>PK^{p0Pkn7X( zu_de+Q{S3W-%4W38A93{L@o?{ZUO&Rx?h^M;Mjq?GG{5TSknxvJ4}lMRe0DjYJ~xNK7qa_YHvAY&&OYWp8ZpXJhu9ruY~nD>ubqglc5`XM}ZtRPJ!|4(O-N;?5k7=34*3TCzkoMFHg}Y zvdzExIyWfvycE0@0XisAeJ&jf6@yE>UO60vx&ev#*RjUI0DWJE{2b z6F*Gy2enYZjFd*gW~fxv0)pMXHx1^>tX&%PV@-;oEz@p<6ZVfjXe+bL>0mK+9dx)n zzpPU~TpnZ6p*5SD4wLMZ#xP4}s|#=tpZ@IY%BE+4maomJj9V3v^0L^pvhvBQ8cj?t zE1&29Xtjo>22|Aw#UsIBX*ABh?1p~m-xRvDo*B@TooNCl+tma#c25&9gzRR&`1)|_ zU_K$;+3Oi_H!E)nW7(>oePLE~&R3k8uZ6;FZ&R4SI%L99_G2b|z<$fcWr>Ts13Qw5 zUQqlH&58sYbmTf2H&D^RIF~t!c|Ljc92o5e$SQaV=C@ z#AXeIZ2Q?P*sriLmwajLU`yy~uWA9q0VC-14luWotg-RAFhbVt^|^3F9zOB+qm9V) z=`&;6+(b+@|0@y8D;}RWK5uZ|K(57{otQCmZbH=}bLQZ(m{=Mqo1Ta}?u)9X8i-n| z5s#>`7`voG5zFX?Yj<{6jEw7ZpfOv}4Vu|wyFnyP4%oYT!vddTS+=UdBeHDR`*qly z!h>TT+c^~mu+3py{Z7m7hhcFz3_~;Y*v#1pt)gGDVcDOG*+bXC^!v+tYn3xw!R8&r^j6|m=)SzLX>Idx< z*LxM~`v6;70L^cF=|*^Xy$goRIGUV3Fc z06Uojn0rP*dsZ+2^5vj21F|kE!^r{7qM8}j!-ipn%~DGbvo52dg`%6RO(DFQqM@yP z*5nZDyamRw;(?IYKitr4;~GOTg4f{F*vf1e$W9Fe9djwRJF}C6Fx3H} zvF0_znmQ{(IKhHaSs=t4)taVwM#TN3}@6 z(Br0~(fZGQ*>`V1Mpn!+HB-|f;Zh@E8fv(RC5tej{#FDZW?H3TD-a5qiBM3-iT`pa ztY(XgKxKPJLT;C^%7^tIk_fAQ6}tF?rLod@ED=(pn9>qvko7nS1#JF4=;pTqxD{xz zK*-V)x}j-o!!WpmukhQMQVbpJsl(w$87F?$NL;wN-7#q@LvUy7+#SN~jzO3yS`EhZ zGkz&sirpkP3|2B!01$rjtheWM}C^I;!<*bK*_@t}%3YdE4M45pSr zwh{VBILC1zz zT`s!5&jQF|D&{WJ{XJ$ZiIlywJ!9WEsCDiFSdGn^h zU6^E?cI`AsXBpGsx)d!C;LxW7aO-VgiFS?oz#Z}$d^#I719qb{!){a#Z^(+Ov01|~ zHGMx5wT4*oCw#yCf(odRm0*y`voXPFqB2>_*-(Hi56{NYR?LPia@cM=2j<9%7O?+R z35(?(TEmIaG_#=@Z0rHZPSG$ej-3lPx6-d+sjjn^=fb;+p|a}v{7U9-Sg#m5D}Na` ziPz^ru3{M6@@)IJd63}IP}>so%>4Oq7(F}?cTPFXe#nIz>{Sckl&pjT_M(Mwr{c4C zbcU?Q?uT4EBMHx^DmXisLUZ^|n`WO_1h>gb*jcw4D}MlLQCycL7~GQ{#CFKW_7GXY z5W+@p*6tw~V86c<1|g)f_6uPMyKNbKtynr6@Gz9KpVmPS`@@GJ)&#}0TMPZ{$~CZC zW^2}C{_ndEEw|Y64Vde8uY-Pyj!R()bhkS^4)aq|^nh-!T@RyWY#H{+zrahfY}y+( zK-Vnf1r5=+B_eZr=u-Y-1aAR}{^#?|dFw$*#y1+hJkPhQX!lR)gu)ji6ZKWji1b zhyAA=FiJ6T85KiEhPf->?p6a6WyLhuKVN|U?BENiJ%roYF3c3`cH;iObt9&;-p}Jz zS@)fA9nLKu1e>`FdSaWi;oQFcC=6v=R$^i-UxkdtyCC;kfvaC5aCHn^XYh_Y-r9xk zzzM&M#9xHxai*L973%Qy2rptxR=)(z?Q|QOV!#AhWH(+N_1}$`LFiweNmI-a?%+I( z9{U^YV-Hv2%D8tg`5L3YnU=nsa*jkbbp!0S-#?7?=IzrzJZF#|052Fzf~4nZ%) zRGn?Y{`L@TO1GE)0|%hsx!gOCKz?KQ{O)5T?2||MU2xYm) zWK%tIJIt~Tz;*>3&K3~`6eCf(Mk4f7vG zLGB^iZk_B*H{q_J669)3M`}yeW7*WBU^aFiAJvX!TRopo^3U$$sJgMN*D)~LxF=>G z8!Mi3o$h(1AuEnSRyv)XeP{vOdJID8%LU`Hh3q8%NDpG9BRpA$C;;d~@r*`<%6cjGM|2-2=EShHQ|!;(*+jdJ@gHuV$e_HV>W4zd6M diff --git a/posthog/user_scripts/latest_user_defined_function.xml b/posthog/user_scripts/latest_user_defined_function.xml index 9a0b6001786fa..609b4d91dc2f1 100644 --- a/posthog/user_scripts/latest_user_defined_function.xml +++ b/posthog/user_scripts/latest_user_defined_function.xml @@ -1,7 +1,7 @@ - - executable + executable_pool aggregate_funnel Array(Tuple(Int8, Nullable(String), Array(Float64), Array(Array(UUID)))) result @@ -35,7 +35,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort Array(Tuple(Int8, UInt64, Array(Float64), Array(Array(UUID)))) result @@ -69,7 +69,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array Array(Tuple(Int8, Array(String), Array(Float64), Array(Array(UUID)))) result @@ -103,7 +103,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_test String result @@ -137,7 +137,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_trends Array(Tuple(DateTime, Int8, Nullable(String))) result @@ -179,7 +179,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends Array(Tuple(DateTime, Int8, Array(String))) @@ -218,7 +218,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort_trends Array(Tuple(DateTime, Int8, UInt64)) @@ -257,7 +257,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends_test String result @@ -294,7 +294,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the 600 - executable + executable_pool aggregate_funnel_v0 Array(Tuple(Int8, Nullable(String), Array(Float64), Array(Array(UUID)))) result @@ -328,7 +328,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort_v0 Array(Tuple(Int8, UInt64, Array(Float64), Array(Array(UUID)))) result @@ -362,7 +362,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_v0 Array(Tuple(Int8, Array(String), Array(Float64), Array(Array(UUID)))) result @@ -396,7 +396,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_test_v0 String result @@ -430,7 +430,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_trends_v0 Array(Tuple(DateTime, Int8, Nullable(String))) result @@ -472,7 +472,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends_v0 Array(Tuple(DateTime, Int8, Array(String))) @@ -511,7 +511,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort_trends_v0 Array(Tuple(DateTime, Int8, UInt64)) @@ -550,7 +550,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends_test_v0 String result diff --git a/posthog/user_scripts/v0/aggregate_funnel_aarch64 b/posthog/user_scripts/v0/aggregate_funnel_aarch64 index c86bfbb060d3203d0ce30d60e292202325ea2d03..c6169148db0b6d1e25a98baba5934b638d7765f8 100755 GIT binary patch delta 195186 zcmd444OmrW);E6deNYaNiSj9aaKKd1KKvF^;vBF-Ej6WdrY2B(3T37=o3hOI0sA;c zHJmnuA5EgpOq^2|6{OTOg7&;aInz*1=}gmjz?${}oj^d&R`L9QYk$CI>v^94^Iq?J zU2oS#d*8qNd);fTd#!cf`zZZ2)%pDR^2Cv`Pcr2{yZT`4WB26W37xCJ9G??d!* z6H`zBafnjX{QPc&0<-T%h-bOQsh8JQdyLjb7=5j{n!hnNA*Z(M-%CeK_~6;0i%0(E z={#}6{L^9$Uzn75tJ<9L{}z_SSi}31Eni0p(G_ZYmN2zM&1UtPXu_K{+-Aa;o>ixu zX2KJV&I+ zQ~&)-pi|#KujJX6D?2XLi-_Li2BIFLc(Vi(7#4cSci z!F+WFQ%(4%8lGpuzt`|W6K=UvP#fB0%9wJeif=dJUmjHP>Ja!r6JDgX(;Nbq1XIQ{ zT7`fKpKzDjv9)qANo}{Oc%lj4s+CVO;X5@vL&F7K{Pc$MOckbT9jr9r`?Q9NOt{`b zxe2#TSC?}8U|e8%Gu4c$!5OSc!>djBs1@pr8%+324L=bA?=a!NRVWSqvu+tD^2)6G`xs#;(vV!>>8;zg{a^%;g__Awwv(Sl^T?Lac&FKXojCj7up)bi0g22&ciD*tJhXHfsl9s*w( z0$*>!zt%c-neZ+R-@Zc_JR-e9jj6(UtwMtduhFES*@Q3EmeOa!$L>@q(V^plw(VMf zs%EgAg9j9&;n61Cx=1Z=GvS$9`Kcyc@7Qj_1-;{?ri^T@p+Xa`H?-b_->H>%neg=* zUUg+$UFnnU1{3D3~V2Tb@K8gAV+n4&+d(I}_j;6Gjb^o*sZ3NO^F3TnLx zUw=%+%T4$g&GM-+;Tal!(1;WN-*TTiBA+n>c*Snj!jVk)<68NE2|uIZ(XS1j@l~3E zV>jU$n3y7eg{F)#+Kh`#_{tyE0Xa?hRdp&QTqfM9;pHaWr?p@6TDU6z`Vw^1stp}9 zRk&TV4Vz53&WtBa_&r*A$%I>Ws+11&<7)q`SgTq~ILgzHP-GvWFY2$Ct|S&b3_6Rw-*(XS7tkdCK? z!0jenw9DzCHx*3V~OJzz>GNn}I9*uZy2PQYl0QRyCLsdilf<_|y=1 z-TftQECt3%)om*JZIf8w$Xb&Weh;MP4?ri3j7o)H3HDul== z3V}OK_B|LYmP5ETL;@aVl)rbJo@+&&l=`qy~n;EaBJeF)qY z0PpSsntf4uLnAa6xbAgegPEJ3`>r-wmdOUfvc0&$u!!u%)Jqi`vWzO?ZRm zYOOcnv(~9x?KI&lG`!q|>jSM;aPXfletJejhzch{;2j}w>l+~`WWsMO&VTf!X-3B{~a{pMcP0cO!!k8-t>ma z{?|KhHdT0Bt8l`EKceA26Mjk?kYvL3_B;Aj57__rX&Kv16?7M@+Jyf?D}T_0>*bqF_@i3+ z6N7N9zg|Hal)+euRv}=*br&q!Gngk_AFJDLq6xpK;WiUKlTJ>E|I$nu*Hvn3X2J`! zwOknjUmpT@g}@I2SNLDofV%tLV5*=G=!6N^2h?H0zu%!wfO!W~XsQ;piXOmK`D@oI z*ak8b`O^c9Q%!h}R^D#Hb^m^;3BOwN`wLCDWv6EUuQz3^*W!0h6Rz`0xd|Vqbx<7w zZwP^(5JF^hgutzD4koEn>(~|o&j^7p)$o4%UvH==LqDb@u3%QI!@6$(u_i&KMBMJ8Nr2*#=j*9Wr6 zgqQ14G{TAhb@BVTmf(eH#>!2&-q3avzFljl%7kC?suVqFz!mw^Gnx$#!r#&G<}2d@ldhb> z*gJz~c3f*H(S++AOf}(?-c|>cXTmG4QYT!f;NU-9{H6|5Gm1gTzkAFv{M8_3yMgFWcgGsGd zu!X=gO!z#lgQX$xA`||AR^C}-vj3mdGRjRAYBjvdglE)fOlQJZXn2ze4|`WF@9W1^ z{likUjE?>cQdf-W6|CoF?3@mERr$ zuL*%Sneex@b_Aa(XELg4l*;lzI{uap6NeF)rT!V5IYZ8zci zfND(mHm!UUaE1SM4X9`MLR1Kt@S|Eo(Y1pq(W&8SCOrCjb&2f*xGH~kEhBFrgQSmz zY5RSV2@hOUHRE~{ZtYX?O(y)K57drbCj9dcH2Z(MDdVqNg=!OiuTDA>{x7Zk2@`%n z!zB|QaX=kVKrm&@)iSJggGp-FT&F}6{*+ce&4k~q4K%}q>vzBMG`!#be^F~_rKtjJ zG-WB)oA9X)bq3`ooMo$cjR`NhNyQHi#s%imG8zVFux1T!GU1YjH=A&sxg--lRU1%; z34cMs!G8f$hRd#wlzlLG2{xTmnagUzf6yqGXu|a+NHgL35@Zlg{I82&rnY8zrV4sP zD@`~%s1CHqgwN3MO(wih!(9ekkw1Mz+YK4C=&xuM8cg_Gn$Gu`@K3e!9VT4AHOCGN zrfB_#+WntsQ^wH`)tRN4@CzDlH{mlrQp+zj;qx@S(1fqh@bw2w_Wx!r!)dDE)$r{m z{Dg*An{d5+LkRqYj_dZnUZF$JPP#Gq?NW<+WT+lNL zO&LzDLXipIuHoxL;G0bNQ`$gXCcIk31-9LkaZamHZNig2R%hH00zYBG@7KypCj4Cu zXZ1u;@V_p8-)k9(O2z;_;tv`#n(%xLw}-%2n(%}}YCG$VILV)$;WB0bzg??P6#_qK z!mDo6C}hI_p&6e|Cj1O=MgE&j88uo%Cro&QhWku-Q0qW4;UnKu2h?H0uhH;8y~+Nc zre&}X2XlM2hFeW|zJ^De@InnwG~vY>ZtKUD`7`F!GSd1pK!5!AiiS@$;lI)F3={sI zhTBc}e`i+P5537-)_QdG`vd12kifkwTx<0g$&K|sWIVSXyp%@@Y5RJV8VUc#F_@- zSpV}{h2}vS_=ZE<jr&@lh3Abyw9k{~(x&~aUWh^ySNYpwgGU4S~`AsJLZ4Gys@FHy~4<72D zm@0qHh3ZIs0~w0=F%6eYxYeqLVp&6fLuz>~Z4ZGjHQ@(`2v(v(r8ebv&Sb< z23)1FTKHcdt%3ih8Zt&2I><2KF$Uajz+(+~o&k?D;7bj7 zya8WHIPpI%L4qNp&`=@KfEO9?Q3iay0mp}3{r_z;;G_FdY<31*7-PtA88XHi@Nxq_ z&VX+>;K>HO%7EJpc(nmfQE^rNYYZ8wh6)D__;>@}V8E|2;7ta6f&p(f;70K~@dsgm z8z&hm_zV@&47g;#(+zls0iSHZ0|tDG0cVPU-Bv0Ry_lF|F?(8i8kQZ8yZS9 z;5Qg>n*lfKl{5oB4dqq(<7KKLW4fWC30z-Jlor3QTV04}hVhKxA_ z8LZHNXBhAz1Ae0cUvI$Y8t_d9e4YV!Dmdm(;+JK}a2YDxWWdV}c(wuGZousZyvl$( z40yEx&k<)Wh!GYIV-<^E1z%Iaj?ITWH5S$};-9z3(XF%HHa_|5WSiwH{ZGkj zMIJv#$y3t*LORuuZbLf9kp4H)s}1S1NS7*U;cKOVa{%@l3Y z7#P4%q*D#)NThQN>8p`mZAgzqx|GsFvO0ix0DBDul8`=ZNRLCh)sRj_n$H{Fg;+=6tfA-xdk z97B3B(yI;WWk{Fa=oV;wl>yuZV6UOTPmw-sNZ*Tet0BD_X@2g&0PaUR)sS9?bdDkY zAkwQ1>4%XnohuA9@F;-2h5}C@eb|tG3h7oux)f=C-a!3lkxn(FpGP{!kbVj2)rR!T zNS6u*;8g&74e6aoA2y_4N4nLJ-itJ!H86nRA)RVSdy&pDq~At*wITf;(n6^L_yE9O zL;6Fc4;#{dK)ThCK8!Sf)4%}!7t*PQ^k+!t7}B33z1ooe@>cn(=+Xf|%)k8+-ZH)N z58>QIvE%kc{)Xvd~T1=CoCSa^F8KUK>+xOkX2cFB|*p16^#2p-|S=-WKh)3=p# zOBUu{%UDhN7qi{flWy`z;~AdWo@xuLv8JSmk1e?|Ue7|&{;WNt#XlOdjJ5WphzU#A z3N9Wk3`ue?WX@&2aOPd{8goPI|N(%~K{m*HvE zMll8>bS{H26l<*K472Z7>K@VBdN;*gYHi3;>RZ?~G%ck;-*X%aT9rmy6Pzdzp*Gwy zMjU^~b>YfX_e>Fgl|NQ|r;+;zvy3Cd6v0%2%+%PnQh(_#tp zYEEav9?wI>hCRbr& zw!gw_fucX;>9a65DwVT;B)SO;=F<9w0q5LJVa{cqT=vCY3oFm@C5bbZPYVa-%dNj0 zB^ECq6|Ug+(PG*1^n@R7L|qHjwRrkKHB7wRx+hhcxU_uK#LL&Y+}e_=5T6!s4_G3_ z7cK@Dj}qpRtekThl@W{bZA4U5CZam{sMp2GcaDpTwy@Z43-kIc zG2YMYtiorB^^O&X+&P}_`85@YKhF;osMQLDUlQ-VbBncO-9+a|?muo8Pu-aw?garC zI);cbcWnvFyqU#{>sMYAd1*y?^E>;aq$Z$0}Md+U_N{crLBD8SQiK zBuFW|tbd#r9Bc&jCfd8A20i22H`t7eElu8xa-0*iiWhFzxqY1bF;yXRVNq3FIR zDXe25H|za(?sw8B}nt&`?%VIo(Te&YJ3v(+oT4i-ZoyaOHOczhyJ7LP+V6gJi6wG9++EwWCGpPLP zm(1lY{1=y&ge&4gd>6zt*s7cRitz3ic%p8mw1&AopkDf(X>Q*P))ySjrm&i-P6Rj!lY$mK!r<7ZUqH+g#uM1Z6`*%_o10Jh11=?0b4&^b&b0f zQ)@xBYq6jtq0(@>+y#~{VyxB|W0(I7>JrUkE{J!nnlQA3=w2k2teWD;RA_1YE0y-4 zmAY3+0AjqosNp2`fLvZ$V)3NWq-Y{r8L>bI8{%9>t=ul2S#`tMvBp-$GxrXym2c3> zePZV7i7Cz2lq|P@yos+ylh#vssz!9KPK<<X88D%aF^9!wYVXTzb9AH+!}d-#pN4 z)n4|MAqAGQe1C?#ShlrDBOff_7Q;t8>3n6Bk^__k&2f|Q(^RS zE;Sg&I!hSKr%4{lw0zYCjr~44E!t_9ck|Zv7UV&~nyp!m`s5*Ony)LEZymx$(6V6H zY|in8xuFRsr(l48nT_@s+t7G6nE%L5CPPYxEThK6N7jv>-262^#2ta|n@a7OpYlw$ zhOwDS)4wH-WVQ6pJa+j;v2oq?xy{x_wPh}axsy3CC$(;68}^(Ds?$f$`t--6_lGh4 z`-8!RDr&|y#GUv%ae8r%FbpsWVHBQ6k&Ov&C;-G{R1zJ@hi?cN9$K#cDGSOs81-7^3nFcJ%`S;MyTZcwi# z$7XRiMmd)~ggix44mD^>Xpw*Rs)PRro}^KT=dMJuzluNk`J`npV(paKH-xz##yet} zv6X0n!0{Nra z#e+u<5Tjju!Y1DLP$K`_Eb*y_#__@|@%4vtB66sk3&9Vr75xuQ$x^0VHAIoA$sEg! z+`@a38(Ajf*#1{%X{6C*4CRcZnJC_|ey$LbA8f-1@y>%pE8+*|aQ96hDowS2G0-wC z#?SR0^~Estc=4RNS)RpHGfSuGc4>{Z#L~}{x?!2pU>P)s+c5~m$r<@u#Q};uM=e4J zcApc!-jK>~Oc&4N@tjv2_wYFD-RFV{WEP0?AHJ4bEN*ysQp9{^1hL-d#XS!v^Nw`! z@WT`NZ{88x9-hoEpDo63yfI9vk-ssGixRhNymg4&6HJ&Xe!4L`Lh4Z&_UQTl10}tpJ?VO*8J05w1pYj`J&=JG6K@Sr*{IZz; zafbNMU*5zYn<0*SbYVmlIw6*PUfl5LipdK9tr`$kjV|RWRjGsMaQ;@5_g~YlgFm;kIxXFeJnL|DdvSQvX*hIdKqo%KSy12 zy^nxV8#>pJYRVB?9!p48%q8nA68!QUHr`zVr^JV$E8MgvOcb6N8%FP9#wPoiTD(;3 z4Z9MT*CMF+<_Z_0`Icel#rC&WFQlWjAGDjKzhv8NT&CAagOKLIY#&GKfwIK*-{T#2 zR4lPNhpzuYgB%b8rANveu8%|&gUd#;yE`~cF&bY%0}7R%e8Q%ZtLh1zTvt>IM6L{e z$8?2M?&*p`$fr?2A>0H~jbBO4F{f%lzTO3f9Re~TUHFuutuPi}d1{%=w4Nz~FQcbk!N7)>sXc@X7P1@fpql77k!9?8(|V7NlLi(TU1pPbEC z?GZJCp1Fy8s)M38{ltjmc@E`>V+Hs6p;a(EO? zcF-={u~$>NYxBn*DxZY%q}A*e=a&h1CzG%RJ7giB^ImHLlYcs0Y}=A5DD&#RRaU%( zj$37T3*}G$<5rofEW@9YfBIIX`CGbz3H@CbHmDAqiV!~k^Hd~(=DUKIXheg=s>~z+d!ZX-H;xgeg?qhMRbB^_$v%yL- zfW&eq+->Jff!tXshV@-~z~TuMgn7=b7~<((GSpL#?(bsuO!5}8PdPG40>}d`Km#n2 z)gFYAP9%+IEYmvLUT;md_qjMW6BgKpRp^x~%TZinhJ7z#A*&!ADp^>4!g0GNj%Rh4 zcWnIe9KlluL-Uu`zya#S2ZQ2PbYQcx+FmqH);1chUx3Q~@hqJx|B}iN9VkB+<NF1NU&cV}XNr0OZ4|6}x+twMZ>^aFiJ@7Jv+8}!Rf_1Z*Kn3GbFhMEkLqTZy z{WMxWrXkBUj`g_~Gmo>hWL`;W>AcdB&Ma3XlV@@86H$)>3g-||kYen8fr9y-OAq8K z!G#CU1`{ZcaLPC45f{)v$sUz{g3+sPdYspXcbv(_Mo(ZBxzWtV9QGR{^`U%)NoC!DQ(Y5_Oy>akRQ=*@65V!gjySm|BBrSc=rR(`&K z8zcVCxolz&s3ya3EgGlxLFGDt6@f>TK4CMS?i9zBOypZT#al|oaeo*6&y5#dB{TU? zI>kSfjN^abDgG0W+D`FO$?Dm=Xc;im?j;TcG!9o+b_Nrw(MS&{Kn+p{FKcDWm1msy z>rV02((B_(`>W<)Wl%jlH?XBs{CnwS{^3q>=+=n~i4b?xAtC21iQe->MU>VjIa=u}&WmwXww)Cpd^$dEBwlGTUD{$^TGCiO zOx*GGoUrdGUbSgkm`$vDHY}{CBbZRR-o?j?(a*%D{RMe?2QF<{h{E4>5T)b1lhNw? zSZ1;Mncwg)=821+oz8nZD#uk}srPhiZvQp0VCj$}`5>qCDfhlf;|1T_YSx z`T}Y}4RGxVb6fjPx{^mgkEL742}5jkNIzBrOfipqWW%anWzO|vWWifW)5_m$my;o7 z4wMbXW208Vi%Ngcwa(pbS?SWR1 zEMpYqd;Xgox%+sItm?;6UMHVa*8XqByeVH{8C7=3lGr|W8}y+iu1vCw_4c9-dm76R z;Qf%r+RkGq=tIb66qk7PIG1;zoBFj%7!ra-G zi1t;%VE#sBHp!D!Egs`_V^(mx-u0mYT;OTH2qE3P)wxqyB1f1zfX0$HFoe^`ct=Mt zd53s=*<7J8R&6zzQ(H|(kR2!2u`aY`W$NBCr^M<_p1?LZFSv>T)umv5+7`Q*jW)PAhQP6N8MP@5wF@s_#IPbF*caImtkWU~ z^QAzrR=!ja=!TpRh~g+Ow)Ihd_-r=s=}LGRntzuhyth1#`!^9<-$-BhSj5 z^+Z(%>R|$cw-2)jKFmB9u!Fv08GE#PsQqQw3!SGa*uhv`b&fqV`9*aZZbc*2H>qic z26B-mvd#io^?JVm*^@!lJs3dsrNU!ax=tDl?hV8$X|%OxDDu!xM8q(4365Y1su6NH z`6@q7z*5jQ{vKx7fSE7gtYz6lz+qD?`Prb*3~(cVttJ0lKjq>u@YrB05d*ZlA;|+PWNa0mnKee)mofU$HZmb$ls7 z2JtHlb)-y=@ocsSzCtzZl?aQqEWNWJ19stt6i_?@*MK|yB(OtRexxO$%nBx5hyn$( z*aj=NZD%)l#Sh`NqEUYrQ-W<(s5lz~?6O37iI<9$-8j*^8Wi+*+2sJJVJyE2ZNt0( z`BmzZRyy@{5Q3lRt^WFfU@g2=r7;>UHCPR^?B7_5u7X^-JKXV}!mz;nz$Ea`l?H{Q zfqdNUiVZ-)zm0M|pw4^BGtv72B<+2%^Th-q?@M*DXJDwmx`fuyI>iqgI{Ey*7>8^{ zkkWrQF9!?ow}_W}m+*361>TqQvSf+L3`B8qcLFE-c}p#gB%-(|BdX!)FQ>4Evu&xL z?#wP>8y0d=&(GgsKkoyB5IL-1APMbdb5^0f1ng=}WU<-5;5OK2vHa{a+=iALEctC( z!EVyn!=1|#X%SJ0w6kC7*dhl$;^e(qEdRns4*C2c2i7Hda^De$d@(9ljzama;kmLL zlPePg1UGYX@G(wq--tf}e=Bn2pRW}!l~0<8R`*~On>ZG``(4&ahI+(4el`u0dZS(s zw{zkm_gI0{GvNiCRl%v=9m7$thuh%%7nlEOH9u$fHt4(8d8J&}L!3{Z{MDT35 zyA@Av6?4B(&MG9ZYFDpACV_SSj?Zto;E=QJtn+pVX!d;8X}=23dst^fG@eClVrNM_ z%U^WcOJ5dEw0qvX_N}w)BJFbVi+?)Xe&*k2lewt&qi4##ED}e&e1p&ugP~&Lw3?2C ze7^r~D7u0i5HO-W73nq*;NK59x{{E}ZpVMMH{gFS^>f%*N$C z;lzNV77r9`ANKayb;xT!`uW)@FPu?^Z{rp|k#aTIWFuG)gQt#D4hOi_W9Z3>(%K?e|}LMUNKpC3><+i!1>1D zu5)TG@49;TzYKbXmM0n347YdiAWMIH`5;}}FCU~GbS6tF!@)tO7!`pma8mVUJlXs~s6)t%} zEv)P|=veP?gQ3G@F1FqFmfBf2HW50f_T%ujO6#!XB}0c~grL*EWwD9?222y7K@-gj zL5e;M_&he23zkH|Tdp;*>RCersQmUURzU%ZDvvf|Xu)T+vMIkZl+`+m&Se#TjXJvg zu~I%?ChA~!^Fw=H#B(rH2$Ahu{}RmaJHo*kjrDd%Ib_;9EKZFWV7P)J<1*+I0 z=Yw-dozu#b8mIk0RqGM$ehk<@7s{6QVeBKccUa%&@@XH@_ml!Is;%%nZty-LKueTB z6X(&IbYZR0PzyBeec059m1z%20{Pl4%QJl^8?sJgw&!8#7vn&z>o_mBV<7FLu}@>) zZ$F8@WV`I-w$5umh4%!z+}4Zd5BS60B46O;GA=CO57Gu`kpemrJZHpSFjuMv!|S=N*O{5Zg4_i$fp8utm%%44U;MZPqt}8a?oJ z=HfXIe>5m!9tQIEV7F|+Y^c$~`K)4-VzJ=(`!vOkV!h{wu$kT1MX3&LAce44da#=Y zu$!L8Hc!@UwH6@xAS#+7B=O#^_F!cJ`7jpoiS5Zl9&mV$r&dlbo0As7U!;D65y@V9cBoP)I@nYLq?W*zdc$B#Uj! z+22wCc3V@D#g*9iw>|*2T#odL5`}%K1NJSS@4Fi;UC@}-2KT)6fyOKe+kqcug#8(A zzi$olmN#aR3O`p2n|T|*-?ySMtLq_Nc9gLFB*7)f&wgZLw)G7pyQZ`>FMST^61Pxtt&HwMwG}YRJ+@A_ozq2ut!k%!4}uaKf<+dzo@JNl)A5yq7;Yo3_Arb~k1(!OH3%{TjP`HwMq%rL~S7 zwASaa8TXvZk#|SJ)qEdRgar7&`N55NPU2+zP;9;uD*8cy6O3)h-=5P4pVhmfYs;}6 z|9 z;hdb06M~aa%@;6`F(9Q6!4$pBS6Z3w^TArzrOn5xpH~{k1*%a><8idCF1{95IK3A^ zoxh-+M<#%Q(H2cuiIsq11=9JnHnbno&QIJaVQXR*$T9Xp7LF(OKo;f90+nRUOb%c# zuwp>$JN8i93^1bwD)@~h9%9CBbU+a~V#*klk%rS|e3e^@!4pjDkAkWOr5Hrsiv96r zT+NlO;m&1goOKs&h{!>3a9aY_YR!nu-WgPxlk-a8mKJlev@Zv)E}rjTP2a#f#8+B@ zzc=xoi{~0nUXbGOEK0YhlSk=4#N>1b>SlAQYx_pLa?!2*0AJsBgsYd9paJ68o*6kZ zIeZ?hs~?@_x%oP2JWSC7%m{{AG8fm5(Y{P340ic^+GvRa&l9&Nljj!K?vG;mG(+MQ zIz?J_JF7SkQa4-*)=E(C{tJ#iScG1pQr=F!wrhBfEIXnQFo^*ZCgjL;2dinEy*?t` zUJsR9+yE}kw#2ko!P7j4b+%m`~!LdaAv9l4Ls5DgM_gv3Kh|~5Y3s&s0B;<~Z z!TcuNOp3sMoR2jn5?Jw9F_d*O>`XjomnY+|fJL^qAT*d#QV?MHo41O&$KvmI#O2_6 z#Ztst`@wj&OTn3c0uvohLX?2|Vel^yn&Qw$1Nz$G4+@peKyX&WID4NT%oMPU_S#Yk z$2mbRA`vl{l*}tehVKH`*9$V78_mmfu-1sJP@06@D;xn8-g=aFt!iA0lGfajS$#G* z>-=b}FUtPCD_D09WRpNOB4sic-!5(BWP3Q5-y1`+3^JbI2vZOw!cUgODh+f5XVJRQ zn%oJCzYMb=uGa_I3r6eS>`=LLF5f3*gCQ->Wh4nbxjZ_8aeZWj`^L3t0pgLKY;7{& zg*!)bweqd#6UNC70fG(zZ)UiKKmtAiWOP2s*W#`oEOT5lf*bDI%-2;<$NF(e3PV-n z*46?ni31`kquGu(2g&JITPzMqIM=?3QCv!tNw)%a&s#-x7AO zyWm<%#W}L8gs;2!5Y87sRdPcQ{501?BAcFUueT0CVEsDIT5N|~5ai?A?I&2i3`O6Y z!?~+d?ec*nCP(Op+>mSKkbB}CjWFJ;C)(xo(f0aC4sF2g=MWu4?0thBQjBHYfJ+1U zli>nI3?u1|Xy<@Wf$L*Mb0pmnknAG-k#yU|1-~6FWI%>Vj;ke{!a;ufWf@{gfy@B* zYF}S)W)o(Ss<+G{+DihEXS8ZV>pM`FVQFkQC$C17pCo2aq`mLJQ24}=tgjmOoeekJ zNlF?zg89=SZS=l17!(q8M%#w%Pv-^h3XI0S57y;L!P>WDpaW3W%p-Yzl}B9P8+hbf zl}8p~@Y^&d^B7G(CsX^xD~d>hUnBwUkH4MoBQuAr8{M)=Tfx_MW6$jY zqq-nYMIfOQqzIQGUN?z6OxslJ0!dBgeNjwj}@Jybv>-qF4 zyV6J5`|S90WUd{@Y-`1nZ$t<#aH$piLF&H+6tmfi#^Fpk-}Se^90@_;)7VvAxI85t z0#9PB&^=#}bhbbuT#yF;fgDJKwb%jOg7C&_aZ_ zZ1)>?d9-~_=&VAaR8R(DrEGK@F*i0j8~zd?93&*ZAA*%+_>fRYkXB-H+UV*qU>C&N zf5f2-!VdjaLl`=3MW?j+mGlJjOVFwBkX`;9?XVtfGayU^2qZrSidLtl?;7>3-BSr; zR=V`$G0D!?oriv*ErPZOAGB{dDmG;EwPccYV*q68lWjp(R3doFhk>hD2=O{C8wry=#x7R?hwunl{xuF6_ouRm zfyt1yYlf^HhAc@?9t_!>>F^=3)%#b#rJtC#s}^R%?uo2(HSJfBQ1Urefq}`dd

5}l|vFa?o?)aefPVmNIbWV_(uE7K#
zg|yyutA)r-ZWcPPe>2LX;7c6x&A_`AqZcY<32YHf
zO~_zGias^|Gnju11}$U2#Hh1DdLlmsne@IY(3e1+HC+tW{Q$A)!YMr|s$YT{@tEbP
zVKHT~5X(c@C>!RhN^BcQZ(aLnfTvB8IvE4wd%dD4;r-xo6{mpGX+fx#i^|
ze(X^?>W-VXV<&!HA~uh{&MS#_?}obPw%PYHt|7}gle)_*-^nbZwah$y5P_@
z&i-=y5%%7~Em6DdxXrPr*smPWr;M>ZnEG_&N~u85bw$<|sq!+qV
zBX+qAo8~Lv3EDKRy}?@AG~Gy(+9uzW{1!5%iK=9iJQIfR`>;fk0DG~AHoGE70T}6N
zRHCEmEhs=+xZ=3(18=;+%cGDEzKJzPxW^B_vkRoGzTdh_M!p}oA88roq<47v&gWR|
z40vZWXPw(@qPpBhl#9~?8r83
zIFI5`f6kKm9v8?7tPeI{F=@G;V$5JS6b|fDg(}PWdNPzrj4xs^%5f|-rwngR$;U)#
zAcf;uev`Vp3xl@z*RbX8#uBW)#3|eR+CHv-d*3!i-P$K>TYF08I&AIh#*tXkR$gS-
z-hCiDVNKZH^Ii+g@qdJYf~vH|L-p7C4>{y?^p(vCSG9BGv|?}fm*v!7z^Eaj#T@qh
z?&Uf4kctfrFas%{_IVClxRecfzrmqc?zHdA`|k4iciZL6aMsxkO8f*tE~7u&2qxD3
z*;&})lkj)yd%9i}=j{=QgwS|4C=&dtODyf>5*nxdyoK~U_VSi+hkP6r=z|
z>k6NvZ_%*KVD*eo-!=LxT&%!GQ4PBwZ#ZA~p}J4h=yt(#7=Z6;zE;Mlb(?@xt`f+^
zAjlq|(9SHBlZ$z|?~hFG!#3QrQMC&;&s29`vIQa@pLMZf9d5VI;RM+a8*wK(&I@in
zCPngfKl@R!1+6N5qvG}_cA)6&gmKqbxx-hK6Ks!k!*#|h}Sy5aD>TZ1Dw}7
zZ^3a3ZT$BccIxuNc|C
z_vZrZ!Je-u8VpLe1J2;`33W#M{VatC;`$qYRc%-30w*dxJtgIKc$W;=|Kn1KL_*YJ!22}
z=Wz0`+wGnYC$L&l+wWVF6eoX1fL&k2a|P!W%pMS5;kqdTc;l8@w^`
zFK>(r>3vl{{Qr>mwHdtgf5Q7(i*~N)ePw`I$pbq8g-0Hk)yL@`Sj|N!GF{uIqXFMP
zIYsG_7e-MX%GW)y(TZhGicgiMuTH6+7%4s76B|QH4|+^-{{(|8cH!h;PpkriAy2I7
z@0{+5Y1(&;Hyi_L_?tQqvo|)NeMu#Z@z!Dx{odF&gS|1q?2bKw!_fbm-qt@xV|t9_Jocz~sDn_R*ny&5S&=laz+;Q9QG|WBax9-+=2HnFe@8@9f)PKDiCI{#WkVB&|az!n%K9mi>+k
zG;6=laspQ#b)V%p0vo!|f~$}P@G*!vi(sjdm96AEzl{1
zyTU}Fn~`n=g~-pz!ad5W(!=xilpdMK((nT=rSk)OOB?6Wrn#@QVV*C$B#S;2ZOO&q
zY#A@N&MnCjPR%J%tTcp-YWCv0qAGk>w5N3IJWpv@U|*?o9%<3PwnEJR7gtLsg)O)_
zL$*gNZrr%gn20|XhARoM$XcKZ_-JQV>x{-MQfM6hu^kGiv`oRR8@lp9;iDTGv*eF(
zhacA+df&(q<{Uw_so1nahgTazsb
z?Gy&IK4%XE-~x_CI6!)zmp{bvP$%c%2KHSD*7bnY`Yo#`affENWmJ0=?$eRVb{PhC
z@ql4qEzCX2Ffj7u$>=8ISrU#R7J(>?z2xvbN6C?S3rgk(Xr1gOjq~W<7p;|}1Z#)2
zYRv^Hv0T2ncxE?douVKc){zWqM@i|toRY1m9~Q_han7S)KO)TUHq>+ZMdyWMYayP`
zV@W7x67h_FK>o;R!E!SJ4;56G--DdJ|5_tS@QhOE-0(DkuhaGODwYD0Ijef
zFoJK;KuQWBJRccbMrSC9ak@{fNpky_TjU%#N@s9OO3KZZn?B&`dTz~yQyuHwOX5q1
zJrmU!`-+=c#TgmK7-ks+z9BwauK7ENx2$cA6b0-#bCmryIAL8M11Ii2Zfg=!=v}y
zFGTF3yV~v8`yai7;q0yBOWM#jMliV$WF@XH1Zj!$Pa~{Y2onE%0+SayIGnk1xQ-mr
z&Tyi$9C0nNlB-?B-4j@ZXFlRui%RYeyudeP7IO`mySb?1b_(8
znE7oJ5O*xbEFZ!eZ!AGaI7hWamHCD`_P35~$ZDO4(}8C#`&)SX{+3dH|C_iKaGJj-
zaNDz7?X@WPB6Z^6zIcH;L}!P%tBRx0*%agfcYlT@(1RrhTDi=l0gxX9NaJFTF7f}s
zzeiUHJUe!rJqNN_w|u%Qw(pi>C;8a+LhJz~I>#4VcS%V&wuaa>fTR)+s~*Y+r-`*a
zIP@b5o`*}()0!jx@oy=@XowfxI-@eQECK8T$_9u%C>D7VjPkF^>4KW~AqqGg*tb+3
zDnKa*l@JNc80G1PgwZ}lw+o10`4>2x93bM%6v9=>LL7s980>b4Wel9n+Y!q69F)n?`#>nep`FXy
z6L2P5h6)Wh$fK_yY8U5vcJndqHfTN>GpNAnWEDPE(Xvg0o@~{8x3Dk*%^pu;INj$RZg7c>fRtRKl#q&
z6>?DFR8HU`9QXma80GfCA2L`)@3FA@zo27dBc$&yk`%yirACOWyU_^UF{N_GHuj^9
zF0}CtwL#Z`PwF4F#CtQb)c6QZxsN&<@4iX!_=C92LnHbYKSN>cvQS2%6`&c@@waq5tUeC|moT_WZGznvVU#!FPKT!lzm+rVIr#u=h;lki
zA)Da?AD9gaif$e_wF*Q9nmKh~ZgjU+h61spe&+DZ!3BU7&**KccdGWFP21C4E#14J
zyE#PyT}y?0cOiq67(P_4#aFs=&kA5zYy6%}tfJ1rA5Hk%jh(h4N4*ARLnx^lCp>i!
zy9Tvp6DDHqWqIxod3LiY@DXZ|EsHw+A$
zbNTWJS6rXZ4Wy*y`6_|cz9Q(PRF`K723Yu%x;zK8^5!K`B1NXbbf>a9MO9<)Np}hk
z&nq_U0Z$HYJOhn8pHv&~L*tnwyNc>nziXFHA`}JhnnbbZF~l`N6&c}0QZPX&37OInc;28u3k)ARb|*iBZXPxW&vN_e4BmeVM?*j0?km2ol#UE@GpOA8
zSen<-Io-yl!&ue5Pw;+<&U?|hj4S*U)9l3n{wJL}(dTZhPk}lup&e1{bP`!Wf5d2U
zvy^OxBAjcqAy8I;HD?jn#$i3PVR*uNW+QOt?gApX=}ac`j2IOWVD(IXglxU%CWy2xAUqH+qu=EZs$$fc20MCu$6nx
zW68v(zuAO%bfE9&B*PRsCK=ZE^s8w^5Di*K+ShSZpzP~b;NO_`bqg@v7`Z@u`f`Vy
zW@U9b#Dth#ObSke5sXtKd$APR6Wp$$?29xU1<|J)#>id=ttC1zGuXnA3>Nc0V=#rk
zgaHOKuJ!M5JKVh1|2u0;6i_G$MfQ6PT(Us#hv&$#U}Cz+Ss_mNB;lrak)y0ZP^Wv2
z{1z|@D*Obyf`@7%u%NUA75H09Lk%kQTqB-qoG5GuMAY&D_P;|vn1S16v8-+nIvJoK
z#h>E5Xa1>*D2XMBC`rhb^g-<3=!Pne!ZmmZTCfE7wj#uZM^Xh6dk4He
zx-1}J{B$7TTaH`rP!*&aq?5pDZV*b`?NDrLQXvBF2uRQnu5fn8ndOM@L!~reWsQrO
zY+Otj{Uo>yLZ}NR1ohehNrm*1bdm)3OEjdNR$p5^ar7rCj_W}H^O~gtqcx+TP#)H8
z0;RED^yM2Z2+_xnUnEJyq7;cg`y^MHHh!@wpM@pfyAR73fg1P%)Jw-ybX-Mo@`wqM
zyJohqFDSO@!gV`}##FQ-CQiSKOVJ%{Y)+VPF8n5j6=88_s$K4-B}RJ`Mrg;g3+XGg
zD_pT*$ND1H2VKR6`wpj}GaczHHwp?)_{-J7`n*0p9Cy+yFy{zKr`hW}N*?ZUYDJ
z9iM}HS0Rp(QgwrSHJ&-P8{CcXBi7^hPlBgLWYUKt-{3ptE_|5fy!}!geNxd@kfYqH
zrHCTkJeaa}U54;3zREnp)UPsyhJnvAeTQJ~!>p&z@!AU1PbL(rsEh0W_URxLTs}Ia
znQNe#Ybr|*sI_=yG9
zj1S>(ouvpNANsh8Ha3c~oEU=Zlz4{f$oOFv4M_h|>?I78Ixj+fqAs1sW-nl!MM?C9
zSfuv|q_T}Zl=mY7vk2SoRS4xR+H!Q)3ytiKpe3rkJ$!f4?g=0jDi&MVoP`mQWwb;H
z^L?=pS2PQ_tKMA5T~%g*FB(hBbTdR5&N3SHMsfDcVzeVreUSb(w2yk7R#120maBFx
zbWAC0As?206Q}(nPHsEI$zR`piT-sucx|p;9oeo2zT%sK#kyWP9{b4
z!?%b6U&qzDUVLd;HxxzJBu=(k+2eU{zp$+6?YAHH;Sk@-SbgjwRx7q`K1TI>{s=2g
zxx*FXJuaXD+YGLj)@3&|P~kMhv9Y~(BOc+p%eqVX5FYV}AYu>_4*n@%Dvi!nG*iWD
z&fRttpXWjxt#L7Bew5kGN8(E`D2^3S7|<~6R(za-P8mMYkT5ph(SHa_^OF_Sw!YYnK0r_EX
z26o|QAtp9!mc2fWmJZaQAZfy?`JPQ+{1ZZqtc?mc(y>1Iv_3J|)rbOE8scpF8Gs9#
z+fIx~0S^}%p$|>S*Q9}y;Y!Eel%`K-Bv6aW5aq~MRR)1H`4#F9oFsZX|AD(!G`pU=
z@x3ZWMzhO9x(#IP#%y~qvjApB9x1JDSmlL#)htPb4Vn~L=;#V`M1
z9qC$v9Xg=liCUkdwtCz%*dG>R&K1`|v!4zMv5aZ^tUds6v){O3Iw}BEwW*QXv1+-@dF#|tF
zOlg{({~`Ri76#^rcTQv{D%ucIB#9yW6YmD4m>bk
zl@gc0t0F?mA#Q{&9x)0pe2Mj8JahxpayNpdqz`-uh6WKZ^%djN21Ja$a8Y!^a_9tf
zM*e1-#nG=5XvVtP??*jFCnU3cx~=%gXx#TDS(X0A>U8Z?4}VkFPVZp=g@426R57t-
zJ-BVuJr_(i1^jRU_x$mt1Ad8F?K~SIiO=sc;h(lm8R$IQqIDi@P?V7$oW?VuV;Vo*JvqvrIc2y#b6DT}W7c8zm#@b&ZCz1DI^LJyeLeoJjj?As@pS~g
z*Q`%lw>Bd^!Ja97G!*Z@$hdAa-cJ$!V20FA&xbOkF1|hw_1EE#+M0qkCa+tkvoB=CnuwUV|={}h5Y*{vnWG4iZm);JIS7@q(36OIAh9qls&-LSEKA&
z{3&^FB8{@wP0`BkLg&a^sCAA|GwQC(@B)*z()-$sZS*dPe0>oLO}Rm>n2ro-v)1uk
z74sMK_0*BHMr~yB414BmDz{PXbb;2%9K!C_+M8p~%)%sS5bN-_7G-WkIxX9tITPhF
zFqr3o3)_$pg%>NHbCEFvg-A`#LVp}uFT(HP
z;WzQ53pw>;e~$Msytn7nC#}c374IM2QlIb$-mk)YS5AH6CcH=E{hOTn*eCHG%LG!V
z^>I!l5|MEvr+)O)cproJqdE0sTzJ0*?;qvVLvCej8s6yw7-ZRcE#CLx9kOhliuX5j
z>a7`g?!t2(o@IF2@!X2%d^|-wZ^Lslo=fp8#xq~Q%No3_#B(E_g?QeL=Y1@*8oy$0
zwP64=@t23cdyqFbM~M%Rypy(EB|(Yc`<||JR$(Xofqe2^Ne57Dnk*pFB|YH|vI7S6
z2_(r635pP)Q2ZC5OduNcp-_7epEj?#+N~>-t`!JMLz(zD&aYqi?(GLjk*wudy?;rr
zylxshsCXYxK%{I^@DmO`9GEm?Fqi1o3CaxW6gq?sQ%H&Vi_s_af#64l_Fth>XtMrJ
z7opQ!e8s=-=K6)xZa>(43VQ>zAH^b-&T$Gu1EJ1o5RAhw20{(fiOsv{+>d+-Cp6?9
zt%EvmgXU{P9(_?mh6)`ywUxm|OwJV$Vu!T?|B5t|gbwK=U_ab9e*$&Vkd=qoQpZ*d
zggR`u#AW88Q#zTWj#Ca|*839OB0By<9~=$TqtEhaCnT3A2un)REGVLEJIYF@a_bko
zJO5x|$#o6$djA4P;;5y5!8FT3-;d5LQmY9OFrSa%);YG|bN`zEcpSdpYZ@9ae)M^g
z4Ht~EaWyxIi#VD(&9(D`Z}5@wlXj8r{3WhoU(|^`pHC5baHm-U1I(l^jjoO>!v{8B
zBxE_{e}YMTXyFail8)xq`!OQlj}2Mr({c~e5>T9&qM%ZIT-gH5k~S2I4>DZdC6cot
zcm|zW5%{I#1=H*YBZhz%j?J$}NKcL!!44`Od`&7`BD#~|0e
zhr`?#;M46!ypK4PI-r5HErQEy8I@@qkV1O=TwB*9i%f$dawouQ=>|o7B8~|w@d0@Z
zuDx9kj|qojbm~e!BPJgg@ZAL5?Ow2X5Nv*!KFmNzG1$qLVQuKg0ca2-@TIA7U{7FB
zqW5klZvz@cvu*Ygkh!!g=3uTKaTJ`
z8jL`v5`LljE*SrJrr=xa_{cIDjnNs@Hh3Y_SoQt*mkSWCC5OKSrZfF|!&nTI226o1
z3A0r0V`C!KC@(==X%a0E*^J4sAHZmnQGPogd9(q}y+G$c*6_qK9v18?f5#91p^^_D
zaP#e(k5OmzqwnSp%TSKYZN#HS(9*2p*?#F?VUJ&rN%X|v(^>EaQLO?zWFABU4FZFt
zlYJO2LM(M?9OZDWfORbUA|e7d9!8(Pc5NUMGu#I1#7&6V^(xj?jJK8@ha<8Ult~1|
zq@f9AG#NH2au-czH3q4#y)!={D4HXWe%CJhc41JvOR_$~7X@^;x#d%q|Hu#XA5UtT|2T25
zWjf1G!;A|O*oF(R_ekT>5w8n_wxRI`7@-BD*@hAv)m0O9VgD&S0I?kj>UN>8M?k@f
zVPsapqbcNvx>td;M7B6kq7ih*?fEm-;ZUCi@Jd{UV+esozW7;+OhpVs&^@?5PEZhb)9u+e4KVM%4B__DnU
z1ERq9DU_kRPA=5f%TasO2Dx3Xt4Ei88YwXMXIN9*KWR_M;^gPChN6B#K)k$SP
z#fa%6&~yhhWg>WQPKyqjrPv68lc&XusGc|q^sDIS}9LzaZHZMh>e@#prR#K$ltTtLD<
zCS-M3-1M>>m#DCXdTx|{r3V`11J@_wu&^t~E~bg3^`aPJi2W
zkczgeTk$1{;YOFdJ>Q1#QGt2qz70yOLTrbXQYoQuhqGlwY2zAmwXr}dO{E*#5*(&N
zJTT6e?!2>5S~d}JDc4-&>v^re3$;{)lo|{fD;)&IMIb+&}bM*KLn!*Qp1qyMbVJLXo9q)
zebCP3P`@_BpkF@vaWa@_I~s6FQB3J-oBeDrATAq+DiqGFXRdx8;pL-~tz^fhoXYgm
z3&CKH#t|tU`oJ`V2=xTg$SDM!oT&hPY{`goz*oa{T#F&;cW;OL$Qx+uiun9XwRTO=
zHRehKyk!%_vlc*-7+Cr`S9w}MP|lDDTfr!UZ^j5b+ANdI@Bt4xmWfF6Tw!%>UNoJFd1WCFE$g~^@7DhNr6=w>
z0~DlQg?A~usB;rq(%=7Msnvoye}gUu`g2m7{spz^8mt3KfzCd%2?Z`f%b{yb`vbVn
zOLWAj@^r7JIg31CBn6&AHw-5J>N1lUBR}kfdd@?pO9CDei0D+->PgYvEI;zP$ZIerpzl*C<>(nGj}L~x0;($G#}jW&plj$XK?!@o_T(c(
z+}WE{y%yvLHLg{lRiqssg>E42_*$$diaH(6hbnP0`wu6FhF?XkR>wk;
zps`!pk24y~rYorTI&P?3rF-Cx(5!BkA!u$!WsgQawtoc)2#%8L0;qZgNpwP5|--T
z8lWWr^gl)577)-|60emX`@bl!m%ZVMeu&fnNT9<16oFa*umac0S7G|n03)_f)3Pqv
z8z7JM3gizA$Tv)?8$t)Rj5jP`o#fl@Iym2YWY8JTRJ2B)Qj_)n;9PW2rNo1^PSF43
zNM{`WlRMCd#{hL}I=bYM-2f_Deg)sAF+hE&b2swsvk%BT3gEv1tJ@`~HJAjC1}dE|
z(ZIbs<9?@~{tnktGf)Tt!cw`~g3RP@ur3MflAMH|e;|#)2er!o3h5i8EVUweU9)~D_b3rMXWrKLgrMy(q4|AD}(8Uplul#g9Dq}+$*J!*
z+J73rU)B8gvz@|!5{PvqqJYF-K;ri8q-sLKf_kuzunZ=k6fl9$r=p@Hh>3Lcnv5?U
zy|#vK;5G<>>h6b8nq&g~gn5ukIcRt8jVJcS1&I`2L-ejxGG#;m7kEmd!sM2Pe4u
zS;NZ`ohxEJ=>hk;o_>PQF6RDhU=70g4$PDM@va^E4dLP{Vzy)~9pkg^)2)P10uvA2NpD2Vv5NrGTa
zr(!zkBSCs{s!jQc^t7_3g4#s+))c4!*$N+lOJai`T;wU}j$0Pk|PNsw&<`v9gol_N5wXtMP{rzmnR6og7reYa_NVOU$r
zlQVsx9Z)ZkRp1Z~k>$x@zVdDSzc&Vx2Cb(Fcx$`4`tDFn#QwM)jUhNe19VRX^k3nS
zC%EeY>h?{CL$L4nCz%I&v72p@s(k@;Fo0J41-%KjA9OxS!Zwt(1NQ>+r~i)mf35J}
z5dRBsx=Ke^g~!q+ir!pJrvwXxh2
zK*pFG%a7wb-|VMsI+HLV2Lh-EfKn@|FBK#l)QwGT%hBHzf7GR5fYMSr%$*4MaO~wv
z-ARs8V3B4~16Rz|&Lfn2rLaDBytXQlXPo@{uPa{v*WeeR>h$O$gJ#WSCZd
z2yfKW9gXC32;+$>-+SLn)to3r2z^hK3bc73Z6(r({4J4YskmUSZi_z^C)F+>(pI05
zs)=jUwA80w4at;gg8};lNdC1`ECMjvtXG_&3Ic3@B5pq7?jKSscScEY0RuMyhhP(h
zv^S~p&(&^gPyJ7rv_WNh>ox(ySCNVOxq73dc=;nlsI}Bp3VAo0tG@*>=pB43)~Ue0
zLUTLCAdMAql++NJY$n6iwT)4pc*aaX`UT
zF)dl41a-ko6!t_}z2feu*bQNY{J1JN1?dKaDX)<{L54zv#jzAo!hv(L3Y@oH4>ee-
z=@t^*K6()aZsTG)k&7_BISAwii7Ez}t4AWO3TZSZNNGC^;olHm?1t66CO*J681cU#
zJRk8tY4Hb8rXS)R2w%sF3{kX7I^I!bZeC96nRAulcuDdeX?=Z)}sgcYXbXpI-kvHQod=Sr<>$sIe*084)eR9Wa>tnJWF
z8s!*}ixvvf0ML;$UfPDcXndWp_92C1CZ-N5ZG{S!paQHFRnom65>fFI?_=^BJXiEG
z(%#aLVK~;h@fGSoSY1cdir1nZJZEgN(ZHFdXND&C5;D&~I2wzPyI+881c$4Pa`k`5eOI5hg*6cm;hGhDL)Kg4+XsX&J=Y(tJ;;
z^h=yDISiHn+mom;2oXmSp-6bp)Z9Qu7-q%NK%_WLYg0-NR3Y*BlIQL&C`Y52(rHJh
z>>~IA_>V54(K-8A}`dev4aq%nKe_BrDH8fFprVsGni(?YH58ZJNiG-ikoOn
z7~X|Thpv(;QI=_JEt;VWNkzf<=4k*4#zDZKx3qsnIl9zEZX9T9LS<<$Oj}P19>AVqt+isWmEyhtwkIi
z^xiz<6mY}t`zg{GzG}Lkvf$AFy?{gUpo9he(gcRJIH1u;@(T*rBH?2zLFVmRC4{-|
z2cB9$j`8S`0(y)f6~e9qmj?Er>nh>zo#HAD`K0lVC*@lv#Y5avoviB)*l}q41u9f5
zdKJ&?xbGH&0_d~Di0_3lu?&@W)LVf;G(JRHMi+aORv|nF;VuX-){v|na9(mG8m2eq
z4KySRc^tD@_4ml%8oz7{`j!TS+P6(?8&N4OqaHOC;9LYE`jbFls#d{Vts98HlbB72
zM3Z(NJGmOkX^rTbrc00*jBc~w>2^j
z(JHJv4t$=HN=Il2=q-Z+-nR&fbpl98fDu(&QGO+Uui~fYqx5IAvTzeZ@fKyoljWZ-
zf+@F1>d1ee*HFS=9)cFoG#ZaMDnoPn8N`i1B?O$f9Ir$27_Bl|$Xt-`H1ZKwC5;kw
zLy%-q|1|#_mOeGe{7nu-0qU0J$a@_Fi2&Sq4QK8MchUe7LI+@5_imkV3@X+eNB1-8
z!n{TDKVjY$fC(T0Xhy9>1?2pwq27A5pfBPh>*U*rie{kv!~7wDW)S$pUF4cH9!Skyh1^I#$h>2UhA1Cic{{0?!Sr@TFNrqo0s{FHkLW3S|yxdA?G<7UEoF9vWnO4Gs%WA~GMXr%Y5?2Nx+p
zZwWCRy&vngx_Wj}bzdN>=c95zqdeJF(*JZ^p(E*O8!D$Wbv@qL5b@;sWHS<8)+(fX
z*SbVR7CuMN$2#CwkSF_Z{KDV%0m@PX3h*T(U9x=zFDa#2^d=z%%VGNeI{p(fQD<=ukW
z07gQo+Lc~%c%81KMXHxJ&h*Am_aZOxe}DM`4CnQP?_;FW{zyzj0U{Ok-w~<=lp2jn
zd;m<3M@yCX5s0Vqggi~LzFMQyw?KStgON@%O0VNFq!Zjbr1PdF_fh>ycV-n*zNUCx
zg0RrUmJC3&e_O9rpaXnVdCI5j7S?L{|B=tqA0!46JCQ*j7<#Lsv{wBag0#aW#`AEw
zCUsi-j|x#=0m`VYhtB)ofD`r^$XJBwm5l@CqQ_MJj>FfjwA7L*Lo&X59+O~V451Zd
z|Lat3H~!mfnFp8<*GeR8RuaxQEsBJ*9~_GY4rDWBlupA4CaTbZ3X%P10H!d_5_N$2
z%V#lWHX>YtvbP3G*6U=-1T7kkatGXQkfQ+yTyM~&K?mG#kPp*JxZwCZMF5l(N(3C!
zNdCxIS~eI!3p9jia85x*Ifz?P6Xzwrgg9DMO7UHQw0DrU6hBZzL2!>E%rv#xfjHOV
z2-fv!yt+Uu`vhi)!-7i3;197!VvWm?2$33(fj5%nI{)cvN!?|5ke5_CM*}<-U+T}g
z*WTVo#nc7FImsy=vEaYJ2NWKVldPPfXF~63DSGHGWDW{q%~q~KX5&9m~y|IIAYWrLYzv%R%tx9m?PX;6gs?&Wjr0N~d;!igKxlqibI`P*D-?Pu;+;
z1D-@26j6&L90i>E0e2zlA~{e;auf>oN8nYog5Is@i~k)SxFrzRBPK$Z;^==gY$JOk
ztaG>Dq4^|ay|po+^8vDpZAbl!tyUZQ!UD84eZk6G(ihTk2h`z!y-0RDX_+gn{|rDR
z{G6D#5Adto&sfob7+yby@lGl3zTg_#sXV8%wAKv4*A_|uVTMo+^($r&^5Yy0ailAT
zIV0(cbB(=JT`}2Btj3lv6FTH*j1PYZkh!F%RrYq;o41F&^6vf=Vi}~sZDM;8-_yX0
z?)*L|hsu}iu%gflk;VewM_iFJf?YV_HG%3P`-^DpdfXo&x%^a
z^WOX`G-xtnK7^2$j^@D*p*o#QyB$c~b0Kf%U!(b&3wc5JVjHeN57GwsV<6G%bl$*w
z?HhXIFLb5tDqwH2x;*dQd-0YjT2vOM%2NC{;z8I@T(DvYD~g2mt>h%sa}h%63@vYJ
zjC!bULVQ}C@^k(#<;UseqW~LaQQm9dGg_nx#}bpu^M>3HwbZF{kd`X7fVpO%4q@-u0
zYTswf;F~Z&HL&8&NTgmkcrwrbLAd_`ly|I?c4tONoo-E2vdQaD=Jk6pvF|5Hsx&}s
z!?e7J5)M>H%4@qGUr#Cx#>7wt?y00iZyTUm7+f>B(3p<2qTcG1PD5B}
zU1I}m^vsFIB=emk5SSF84i;!hLuQi(=m7!I?6LItx<^AxQTgr@;6<2+H6%9~6^
z(JM^C{Lsrny%&4wzr!>PoOx2rVl2p#3PdCzk^v=D-4;0DT4^KV9&SYuv2BzwPC!es
zozFzYg>cRCIgl$r{=z$M3Uec!m3w1D;(0g4hYh003Y7=x027;K8v1!k0y%0lI@%f(gB{Q24jqMm_;
z#OD)bG1At+sK_zTpp3MXH>71rDDx-YH>5qz4P3oddLxX>5)`dj4a6`5>t25i+O-{r
zenddA-oTnSUyc~sIh{bsx+O-T7Ob@LlSGBz46JM1&6%}S=!W)13t8
zaGp9_%yMBtK^`Y?5NO$N?Gc8%r{E=y2uU_&X6Hwr5L;YW3U6^j+;w3gyuk?(its3Nwy1Dr0iwAZYZT@U5@eq>!xY`$_6ItDyd*7CmC8Q0_DB=v
zJud0uV>cG<(SWG6&}{WQA+EZy_{d=Zmk!Iu3CWUdhMWi``lSFKXY7g5B7Y^Dz!t@YjZaZ+;$s8$^(E*$SUe}M?Vlwvh_Z0
zU*N$SvLKAITq&%H+#D?%;H1r{YQi4;rP$)hp7jK&)oY@yj*Eu%StNJ)QpDG1DdC?V
zgB}mhT~~5;tRg^Cys;M%^BW_{qXFd7aj~{Oo8|unYQHfw*_z4yU_bA#+$b$LCfYY(
zv3$>QF|q+mZoUe67@AiJ4O@-k%g0>Z1o70>qwp2w{;s|P{BBc6;N+)t{Sx2+fudAJ
z+E2+x_bMrTbDm^%dP&tPX#<44A?qAkE-mpPR-8M=96+o%cYY=`lrYjh0RLO)sD@rF
zDD)FRrCJTB_5SSrcv@bP!+f@M5i9Y{#4d;*~%aD0016G_U?b?D1l8
zJm9$fvKNbFhILf8$jn87j?0R
zdb4vLJgt}gucpkKbHsRodo#BxB+3R4k>7?#h?!yRid#5JB%51`3==!tyehdc1Y^f&
zV>sg~{rvA}8_E6sbYC$joON}nO8HN?JujTi;2v+s%AQEdXx&oyv|veYNq}f7@fO2d
zu#PU}rU6YwUJEuLtURWonS6N+OLVxg;>su&E5`V;R7j9|enTW1>{4Y~BH4#WvK#NKW7*>@K
zD89QjnocJUM9PbRVq*;NQ4)jpnNi;I7!5|B%z62jP#&|qZ1ETYf(>jN~E`Ejd=bcF{?dG@+wa*i%~kC3?+IU7GJbyHXe6a%;>If$1G41f85@8N(l(5M~#awL(h)pf&uC=5GVNqf{5GCH}$bNGv
zF}>s~ES=bbhF^A)>o=9W8(?A-H%Jg0Y=!-z|4WY
z5f?_9gRofnBL3_1)nkC)AiA^A5(h=7rQFz#i$k5+5ts5MNiO1@MAkR71k*>Z7?wHk
z!U3%=N30x%{3iU)9kAa`WZhU)i7_keZ>-vc+Dv6(z_ONAryUR@yRbPtz07{T3yZHe
zQ9-@1IW{0Zw(|WNc&UXFKCXvTTWHb3`DHo~!Y(oz>&a%@Nnc_ClEkM+q|}
z&zJkfoMhIle#w&mgv+hb&f*qr?HiN9T5BfSZ;!J^4S`RtKG*nuyGJiJh7os}+?%D)
zAAvu=i=sZPkGS_F3*$XM5g{p9m)m|KCSoG+h)=|-6c)#(iZ4@G0B`(>_$7sn9(Lzr
zl~ckQnDxQXgGDz!h7Ft%u1FvXNiH~K*5a=sUjJWa7uk9BN&QHqM
z&ldZMSo;*4%fH!YkL=6tGJeBdg!f~;dGCEo!V(*DEZezz#Nt7yr2Z%NErS?mA*=S_MjQ^>L!XzXBw_nV
zM-H8Ev5-);9~CDCV=5otEoME-BD=p1;tWR(3PytDNEZd8sfZ_6ZH_}+o_W+M1|d%o
zVci9J0Ea4{@Ub}mEc5pmgS_t$g-l5jO<#*wY`2enXhQ
zq2ya9$z2fF5^=}(isYdzyvZRg4Oc&|m60R^SUb?Y2gI90SwQ&D
zyVa^%0_+YgEA@UUZrYIA=cB!1=TH_B5{}V8MgIbwXf%*le?l<;%Ni<~k05(nY(G%twqE#sXb?>ft~k4`Ua&%U&^cID0Jo!Y*wFxh3DZ_c{I(F#+_u
zU4#@y*KToQI2*>#?h&yg*m>S}ukd(*4dO5M5~E&VF>W}t&P_>j74N*j68T$uMBGT`
zFB~thzWiAw@POyzJBszE5|&XY5seZ?l=x^AO5EQqE{|fP`A@q=zf?BMbqi>bEz(A_
z2vL>F`mpy!Vj3&p8N0>VG!`!2dyzHm-TAJ17|g&B2-THd(y|@Og!i#2%m7X4YOie;
zF$TR^@piRYZ`eh@aS*G5?}|C&FmXcd#Frw)k#Vd!uWuJuDN#b?coyz)d#75cn_b*}
zk;O&+P_1^DUfIt(Yw31T%NuPMDdW+w|LhW5#^4Jk|rUiO&!u5STrdQ#DK&*=szr|N}fLdlEo)aOjfRTIc660QBaV}la590VM5Ti?8ff$)k6AYWeTDGi`mPsop72=>K#e5M-z>kj*jDHtF8-BOW14GY$!QgLD$o64Ul6(gs!ATi=q7Rb}8
z#f(?6NDQhLTV7@F2Y@oII`iS@w~3w8S?~78x2Y=Z
zbm$@VW$Saq-NcBVhUJU~$c_GZXly8!!cm8c%C?EP87#QNURx0OjdwH8U@jDL^O@DyM7YCp}u>Rua
zYpk94;&s+tjC~!u(v4IHb`u|N6X$25LkG-cBZP4#Qs!tWZzw5qF>{J%vM7&dwCMg?
zfudRLIrc!L%|cp?mKLR?wGnSu7zRP)RMPGdQmE+eR<)wSH`oBSNliPerG2HP0f`WC
zKGUT;o2zCh)-r6;GIV~Eq07}Yi7UBK~@jA|xLU3^eVlmeu|
z!6->o-G0*bTI*2#=)C#Iz;INh|;+aJf#6pO8_7P{tOnp
zcq|O~B|J9wFr2pu4(=s{!IR#V04ag~JJ2|Sb+93>Z#)G+7I@M-0*qQk9Lrz*F9>A-
z5b|Bi0zxFMl7vdS6!oE~(AaNpLR5o(q(=jIZ}d|=G4y~dw?e=r504C$hGs=yNrK@&
zc0RH9493_tWXh5b|8IEddP*|x0qOw5LBhjz0OG2Z;@W`#LT{-6T!Z`s&;)=4jDR{_
zrH&BpeM1?)))ReQ87x&-uagp?)0ZoHF!>@lRu?e;CR(0w3xHAK9xhVHe2r8|x1ED0
z<3AO?`%vLlC`^}GNPD}8OC5>4kE$eG%Qurr9q0cGtR~1~(Sc3yg7;M%FNA9MdT7Rf
zb07TM&0PKQ#Lk@qapANjD$T}gWOuYdweEK~)J7Z)-U@9X55t7=M}>-nT_=7p;z=)j
z3w8M7t4!3wwibmfv;g$N=%)i$a$?mLqT#~;9tQwoMTDfXzyPrra1rvApN*Mf#2xf(
z%o2ii?&yWFyEiRyzgzjH@TM;yg*hqaI{~j(|SK3e~RSYhp6R6
zjQtoAZbF-$$IhUCjS)ybi>&oZ3i>nLS5Y|}q%4Qwd
zbTKoV?d3f_67h@J5dPbBv2+pZ-tg8RkS<2`vXWLxDi-G!VF&N|k*K#A-^-gt`^9Xa
z_t%?&Z5Lho2)3QtELJRLGrSp=B9kMI*0x-_54NpDgygVT|MkdEhwhay=2s<_rnc3a
z#mF2;A*I*FmK>Jc@MWsEB}p$$ZLe(>4{}(z_oRpFnTYzAu$|m|S6p6#O?mKUG3jj<
z>EBp`?;PU>m!GZ*quO;x*xqJIo+Z-e#>$oKZ%V|iw^?kQ3&=|Qwruom4Aw5zcu{^O
zgHmZxWBC%J6TBcftZ{b!eM6Jj10|y0QtV|e77N=_)-1S0Du=U!?Mb`u{%WpX0|1wN
zpeqZ--KP?9aVdm6d$Abu4ph^{#bVAoEY5XJF_iow>mAmTFE1A7-+{=tuvqx#0{L;p
zB0rZs7L>F}BM!#peFKzVM{)cd3j5tn;%YAT6upWCG?VT9I|Eb?5b>`~U;$_(jnxjt
zVw{X>qL3Cx)da7|Wk&I_%zRw~5hp8gq6(VL2K5o9H|fxEyWbS;av|>{-c`iB%RISr
zvxt8eCpx!5$zAZ+goVFE%5E&T2BGjjS1J#Z$!UiY4;n<#e~r>V)Dw%}#fHUSeDW^q
z(){5;Ukh_R44Y|Hs#Li_bvhB0T`^n#^WwZ^T#%bjGv{RC8flxx(qD
z^hrpuShXDDbhFK3-*V<332W44l|A%K
z_$m+avtcnLb4pMIdLt!Wi@h+tXjeRV=rJlk-Go|uhh62yiTJH^bDl_
z&vk`p4Cb2f3&1G4lMFH!=zrH0#7l26xTE)@=C=6Y#
z;#>jx;eL_0TYz(rJ=eu~q105Z7Ca`=N+7A-%lHsUq5WSbj`Oj4Bd;egGcx^+pl23Jk@o
zNDN%Xl6$*-ub$SbTCf1yDF6(^u0qw2`Psfin6AIj8uEo+Sp+<^LWuya5i~_UwzrGK
zIly#HKzeoFmkWb=o1(lGZ|#J!4(rHoY!o4dAkUSJc^ekQiB}5Q>5oQMf
z1SqPe<>=D9MsN9wJB2K;p?#xDr~CLWu!@j1tbM&d@c(ryDOCK$OKVtaTs9(Tg)o8%
zsw;$_Z64x`SQE*?JiQMlp%dmH4(4blo@7nMkgaT>f0|aX?q+zzMzL=zHnPud%)7iW
zG;eF>6Omo?Y&HH?d~NZFOV>_jQ?YMuzcDW)E0RZV%@^^fjl!(?)w(b?X3s`9DGE_yRq%1lq_aZL34k^Px%AJU-5HnL)SkUB8i#7w@8+fJo*3h0n
z3QqxyM>8jXD&DcWn8dwxtfg0Z4{R!Z;q8T5+j)bCU(a&b4RLTiYlnlMR49iI(FV)k
zmTD9HLrlbh8$=ww{!eHK(UGc-Q1S*b=|dK8>4-FPN$xPfDG{`^sqi+I4rO>1;=sUg
zD%Tb;al!!MtS2T9iJD|IMbD6{V;T3+07IW_KS0`{aXe0oYpR`22O&&5j<2-zk!re^
zjUrEKkGZgilFU+hDZ)
zY!xXR*#Q@*n%W7kBId^jZxWG3?8`0`qv+uDtTQmvnxgyZkiHKpQ%Ht9N;;;tq6>_W
z61_KxI4kRfx*kfDHi;Ql7UD^V$`FF8=ur90K^&XSUoZM?;BowB5t`4tyDYtqHL`32
zcknjr#iYY5w*DFn2@jmy&j72Wt3O?yNJ~T}uJD~boK=>}2
zQ@8q|TJ;~`8@WLY+{%XdcgKI4Vs)8L;(CSYbZZZ&HQG0~o_A=EqEWZpI6YV~NT|U1
zcm#YQq)MSm+7Y7sH9t97wA;pf+z>1=rnM4nw!u28-1u^^oJ^9=Lxs>?aC-GQ}<_!xj$f_R&9r3_>T|8z#ZWJ%X*6$JJ@G#
z^>Nm^#2R4lS;~H6E+vVZf<*^AtEh<&-o?%_ZVMK-ceAtH8Z1ukVM#$nQkf0X40Eo)sO<|}Jzh5JU9favla-rCA5L)EiPuVNH(^|nlV~_E#l12B=U;|{t
zzKe!D@vV)k33cK_zBXu`G{Y2M+v|m*;dbWVcM#HF8zxy_1-I21{{t|7XHTZ=tHDap
z72r%@BLwpnXhJmIgt7k_Xr`aLr2=VDvW9*kmG?>H7;
zgVpI~K4ttWcVNuDq%%&~Q$|aLQ+wd;wdl6l`J_PWkd>WpIgRHGuU1`w$Um>ZjvX#Y?|RmTeQpIsWv=lZibf_bb_0)rl&qzSm#r
zFEWpMunH0yIdet7k%^SE9=irMXQfI=y+}iij*NdqLvSt$fy)>9LT&35jQ}+Jk3b@6
zHT6{-u5V*${sgU8mrZ}$9{g9=5lL=(QKilt#JPbwqAJV~*v&c5^ORA#^X_ef0cusQ
znDX^-F<*<8j5$9=!D_jO1RqlHXIXFXF%KA!_F(KVKY(==etwX1JP&Z$Jqzz8B(YQ1W_!qgMsP*hdZ`Vw-tK_W
z3GL412!nMI54{2#z`+E=Ve%`egjR^UCYj=*%)5`cN%6Q-c+CioLaS}(cG;HvA3?^S
zpf0QcYyTM)-ZFT>abF|4wz-UECRIk2db$Pzp&;8IpwbxNPp;}o2^We7Cd1)flSLS)v5FzcjX6l<$dmI65GXkabv~FeS0;H<0Uh
zgGVeOu#}mrCz+%jOWE){arDKDgeQRYg{;;YEoopvXIYCGz6-^oKA%BN+8KZ*W^nVx-y>R-Qf^GdP^Zvx+!yuInnelAJwQwliN!d!L_R4FO`0wfJ
zZmUF#E(gPpuIux{>oGI}CBV%D2?x;T?8XR2@&Bj1^HCL1auBQw7{#1NcA(z*oAd@AXpng20Dtg0te
z1TFjuMldPQpmXcW@`
zHqHzkM1$AGFLa=jS)^s(-FHhckMS!=M}Z1M@V{n;aV897)k^2Vig`6E?}_gyL-3VB
z9SZU-K>7%D;Lj5*a4HL)-6f_%i$+sWlwP?tb$Bm-J}5?M`MhZkw_s>jLQ{9UgG#n1
z4I}Oxh1F{ezKigEa^CPevG}!weOO=oVlcQ;@aq9GPT0C6|FV5`kSQyv`1#hQ1rdwP
z#g{j(4jR_3w18!?;vR;;D~3fjOguI`Ik^JL&YZ7(gFi2X7gCFAxA2f$RjF{
z=phANk=)+u>bUA0+~md50&-((!t*uLP$|_l1>d%GC+G%tU02$`$u{@8^8Bo}cVZ{D
z?Ys#6{)g*RIgPMetu}bt79q`w_q)%)d4W4|Kjv4^^1fd|3iHglkGSW#lmIz%n*-y7
z8uYLIJdf+yo8N9joZR8RFNDD`M7Wwt_FaEWC?x
zpuX_3=v@p`=))_;;$k+?<0FNFKDL`H?Y|T=f5ykJ6c4tr1@(I=2|+fJK}G5!-qdmn
zOD!GZ(sJ8lKAwXFeb7mR>7C;-UEr)RnH(MaUpZoy2Jd>1bY_1D);rI@9AbZh8*^sh
zceE+E3p$Kgo)?xm^#H=(26=s1HrQS$3%s(fnKZbaa>dmZliekDRx&#k?2(y)msI?0
zx0^^E4bGltjPR2ZYGBvW1{e#By)y@n#+Me`Z0!2*tb;P&jwDDs`e5E=Lm=P_U(7HTS$i6mI2iu3pks+OY({`r_Nl6|d0~FmN8?R`kwtF@CFA
zgN&E9W#?~hA6!tnVR?Qgo`eg}G8bpzkw=s3r(MZ+-P|mx7{}t>#(;T@0=3J)8widq
zJh&jdaap%8*CedQl76D=kH>+o(74T>TUzZ|_QIl8z(0a?2H
zUC9K%qJhB$moZe_GlB~UAkMF^5pYQXe19_sT$VL=vkC5MOF^wrl#6JE8!9OGGo)8#
zaj`($`NfL`_AijGQTHeR;T${?l@r;Hpa7ArS|ghc2-162M6n!D$`j&GOHl4u
zB)*{7l7}d!KZTvKHaov%o8W>(`}_H+8pYB;F^ftvk1L}=vNVlk`>_Z((85?$Or#Ho
z_(SAN#KGu&P$>xuMc7*hHz#4CFr$ozLN$*o^FXz0Ks?&B$=|3}mhTjPc(*0{wt=eaS5<^LuK$AxNO$kq+*gM=6aNMOPkAz
zGwYd)M|)&;qo=%~tim(iNyP`f5w2I(?Eq)R4m>bh<{=f+L_CPAGw$xjoXgnV)6455
zuUx^zb-0;}e}m97dxB)0>5eNs^ftLC;Nxv_?M-q^HTlwwfv84fEE96(Q+SUp13;E=
zyaSG3rmMMlE`~%J-tou+5mP~|%m!uMW}?T^AVa1uV#V*n_rObd#Jn%Qv(SSt;Z5vEUFV{kaYd#$E9*^62B|m|uUIc(L+`xgFYO@Y`l6DItRZ*wA#%mgz;*=_CWAM~
zp_@`x9(as|+R>Q79-xpXmWSUkTz?0pQqagobT{=XA}S1KAW_!s50LA7kn1~;s|;j1
z0FoWV^X(^FNY%xfP@92av>!mrK&>tA4RTHZ&>y^E&I$Y+pw>lD>j0=#0cw?jS`JX_
zJ5cKYs8t4PC3;CaPJ&v0z&V2D9LRMNXC67DnAa6)!JJ~wFLdP?BNE>7VsqatW_V~K3>0g`hu&%1K-xih
zpFuq*(11pBhurxiiCN@E98wi;lyMAXJAn-QKsF*%rF-VP$HCEy!#xWwVK7|=!%GKQ
zPk?csfPfT-7A?j11Q>Q3G@=p0v<09>Twq=qC$=rh{{syB04Co-P=Y4h0cI|~3IWQD
zNtXo5y>t>-rt9Br|wfHo(DTs!T}_kDJIMsm^xgm-`=wz7svtXDG3w^!z|&Tf(f&PsE`JCnK^*>HH2v5jSbMiR_#{g
zU+ofw_Y#5O(H6x8R})qT%>|}spD!*TY-!}{?12iI(BEJoGMOnHz@|tv6+FyxkdE%^
zh)p42uMfvHK^&=Ar(wvFy3!rh`y^Tm9?s{5e9AV7j8(JZHA8(@w{Qs@@Nfz_7WguV0
zJacg+hWtfCS-05}yI5!PrgzfkhT?VkRbA8Oe$aLF+?8F^kdI_G+OcNi6+qgt_MvYk
z7B)|Gpz%3#^%3~npe?LBmTiwWz|#lUE}?xfB4~or5>9gaC5(#I*t2>1!Z~CJY*e=9
z%P^8i0WQ{XCw!WQzfi=pA-MTn`6Gjll<)G530OIW`?HGvfX?QT9_{~6s
z^1cjM#;uF;`{BzlK&*gA^8T0jnliD;U=1=NvZZ2+1i?QYfN^4mr823QGMOZo;s~Qb
z*%b6aMPa^m(lYB6M`XJyX<^Y|Zzvzu6}%N?=e|v4-HXHP8;f=E_~9|KQ0b1QckZZ)
zr+7v4q-$0zxR7XobT$1RBrix?_BqA^8UsI~zXN}sb%sfJx<(4fsS>R}a|vLV#kEwI
z0CD7&p^*jpq#Y!b^}#otgfo1N=*>l#-xKkpb1PjaB7uy~u&77>DU_`aN@vED3CukO
zE~KVR@LrZr=USooFEv_}Pv>4F6F<~0S6gp1Y()!~5E1k_55R==yXWw=dHzN5@LY)7CCA*&yPC*;~
z`r^x@EWmC)#tK=;7(`ZHoO!13AZhyiY`k&6G2AJyE}F!vM+ouYI8H+Q2@(G#l%$D5
zEdCPu-Z&w4ehEQ6SlIaqHi>cP3NiI0+sEHoA=;Ot>NzXKjB@Cghp!O)6iZGShe~J?
zLtx7#uW?F4HLr1BA&oR!64{Jw#}h9IlR>AFcV!bbZz>Erp%pwT#L`o2n8)rFs$LEC
z{&|YE@F^#GUfKw;8X8)X^PgEEVo$TBK|e3YhOJ>HPK$hO2PPT|sHN1x3(Mi!fQ=h*
z34nkM&TnchN_%Mt6yo%NEJ~GhsE!$|lLsO#3>!SE?|m2zD2LXN%?4=*J)?YmZ318M
z#t$q^>^Q@k@?TNi8MYwk%{u6?j9PSFTP~)41!JMK<>HI4*y7QnAA#0B#F4GUqtH5r
zG*YlDsA{&O+Ym=|d{qAZE5y#PS%~jG;6tSOSoxy!QHJiqHW7($!j2{IEPIYeKQ8vU
zu}F{lNF&Q&j}>C=S(e~c*hM)(Bx{{fNV21~5jd_`@_Z+;?mz6Pn+@bkIcl;`JI|&w
z{vc0WzJNX5A9?mj2YcYcKYG_5`7>L{_*RqsqhHugL&N28`q$X}F3OdsYz-CDF0<$A
z=Sn|I{nz{`onkNdvVU`#wd1^@EIfW^b`QTmc@wf`Yz?y?`<>n8d_ph#=T$I$X;cye
zP#yrKriY5}|74pRl^9XN3#OVybijjB;yT-E95QsNG6Jy8?}ql^eq|r@GU6QADUN+d
zNy8O2GC^TBQzF``=2zl=#s4F8Kw_eFu3TZe2pB__idFY;r)Gejx6ECpvYqv?R1f)K
zngBIWhdOZ-Zli|_$yX_vqBOO2SK|~M3BO1^j{--{W76Z`x>AE}v9{1^8dU>Oi6g{}w#|CJFg($OPENzmsK=R&U>3o)s<(9lc_ABIZeV^c2CWK
zn`vF;OaQ3ARD5=yE%yF+2^8j|_TaS>I9trk9Jp(Vee?si#lVl}2;7R@?FstQQI(Hv
z#}fNl&YLjAD83AQY#B>%8H!J`lwpyV%w6TdsN*PlA!ZF3Z*c|ri_nMuf;uoEN|FmH
z-e?<&FnTN3$h!XRy59lhF)4X!uloh+Wy|Q4#%SxPhf~#X+a-`P6yTB?(MrvLhW1bj
zMC)O*8g8zK9o?1mFg={9hMVf)Fg2_pkei&U4ohD>y^O*JG}7W@+w!+aoU5zhlH@R=
zozdo|haGCzpodfGrkl~`%+V-hhQ9*P`Y%0PM(IAb+j_W=(v7z3dN>UC(mH{}f9MgJ
zbWzR2c0~{4sUIrvs~$E}7eMKnV-Kpw$FRmN+A8l#nXq%j;kmVRu~_QH{rRtp#RfNS
zYF*Poa8sW@#$Q`378rSm$7IBjA^-?%F#-YaCE_z9kMH`t
z60hF(&|wY+%p$D%Bs``o24lXq{#w4Lm9L*I4_`5$B@44Z=g!A6eqynGp9c?T{PjiR
zq9@Plk!uQUp=k7Ca86Am3S&LaI$%nW1$JqYn#%ACkvqSHoTWeTRbXyRR%)YjTNHj=s0yZfQdvgNVRbIUH8YAMO`%K0I7})sT;BS(9z=@VU76Y)*1%QP?zK*C6L;l>igQmoRA918ssTAWww(!%#wZgi(8HORX3&C
zuVWxI=FNQQIFcQ$i{&GtoCBzXOYN2`!`XbVY>=Rdgco1Luneg!C5tffpA)<-4JyBHp
z$}J%c2+_lcNfj`QnkUI`evJ6rZc3jNI-EO{CTB}EC|kTYNXz9OAwoVdE;i~kun~x&mAMxmk;ARKddP{+Q9q+Giq

s9}HtUw}>2Kg_59BK3=X`ma(y&!@k6n~6Wd zsq-1UM@P**zLk!y^*Q>CeB4QWYp9)m+I@b$r~bhqpTRE_obKa)l3(rPQQM`W;+$)I zd@u65d_2K$?5>{eLp|+3k_Hd@43cRuiw34$@NWWnfsaq6pK70Fwy?B@V05;SG{NAb+3m2Qkic5~mUQ{2pB8--LmFBY~bo zK6I9&&5UO;(8?ZxWWU&n`q4zc8}%EA{$HP}`-iW?ArAk=KqZ8?W1#IM(5I-cBl@pU zf0*dMLw(DvsPoQ#vJJDH+&CofFU2@U0x{(A_d5H;VC0#ECn3+Vx!wLJqhXe9kon+e zjN%^SzobADQC~&$KGc6t^kLMmB>L!CXlNjYb1_gS36zKYXToP-oUJ6z6{zQPV)Py1 zza^SSjYMKtfDRGDs}zUTk|ekh^=A?NEvV1A&)GC9P+y?*cKt)C?nH+w;&3-QRFMSt zp?)RNuSdR)@C_KJzGobne+wEqdK!Q~jeOe`F<5E((u(~0LMML(`OAdwLH@~Hr~Yl= zkl$m(7dZ{@C zwX0(wa{KT6+B^aI=cO?=QrA?)VfDM;?i1MlqtKAvWzc@(kcSD+LOzS|Ao3c*Pe}VkT`RZFC@Gid2|UeT!V(22)|zO6yq>S{72N^ zL-fm$uO)mX@~1TK9)EYC;Vt5@2Kk4CuSI^C@JEn$5dKfaVG4KL0OWrH4e<}yokF+$ zGsrUu-+_EO;jbZ|PxxEN8wlSoc@%neUb(aF-$#e-#NiXgJ;s$p|0U`V5dBfqFC_Y7 zs6Ph!?)rzq{Emjywa%It@VTrJkFk^lic=g;wM0J@`7FXmAg}4>I{ztXXy`V`jK&1y z+X?q7?lJyC3KT;9VWK}1_4gBf?&m5$bPD2d5eABXkg0;$|6GPVlkhp1z*XUF%3{># z5dD1A4~i0pYte8O;WuEQStL*`>IKpN8TC~}|5xNIk$1KK-_fv+INXZ~UL*WzOeyH(Wn4b%sSejn=NiT;4%QIBye zarh7&GKs@y$a4tqKt7A`|3h9y_%Fy;O5Rofa1ZXo@T!po2!A$%cnzCPMr74nW3qal60li!3qO!zY7`Gntwd_LiSQ`}=bVfQ(^{p-8;nD4A*i8&CB0os@tH_TL-iAE>A!p6~ z2YDvp?Sg{+bT~{LrXV*Sc2=o{Je}}t-LtaGqg}@=d z#~8K1*@^z396ZKY!e=A*5?+M-bi%JfelFpQ6i+dhJAzA^s6(RsREdCgG1F&mp`C`7FYpLS9Aqb5UjR7|Y2Wzl07e ziNov2*Ac!K`F6tphkP&LA0Uq&CWb?3FdCgT@wMXcppg{xd(@{A{m-cXGts-gl%otr zjpp6$&(ZJ~;^0Ak8{x^wR}!9v{C2`K6i+d{j)Fq|EHq4a7!1QiUPO2{Cb)wn$VGh( z(dQvwMR%hJAv$NMjP@~gzx`S z<@Xr#NuYL2u#oUW$QKdbf&4ncJCQFT{5bNPqQnsMl`OBvxS8-cgnT*SX~=IS zJOlZkkax9zmNGzRBvoW0-$8gb@&knDB0oZS9&-Meqpw7b0%d@+94Tob@_!Isj(i>A zmB`ldzK!rkNbHe~a)A&QA-~5sObl7*@D<@E@(#kY6^Ey2q^fgK{{zwIA)iGD5W)4KIKe>CcOnl*iQzaJ z@(GXmR^B%((}c$%Uq*Nm@&>}wkZ(oa)&3c1*i9U=6o)@1Mef+Nx&23`WclDH+vX2C z`%o_OjsI})sF8<;Elz_`fP5?Ag~*>Eyd3#cgjXVen(%7HVgJ9K7;4esS;AK!f1dC< zDTKEoA4B*q#Z!!fJ(^hMZ$rbe z9tPQ#`!P@k3Dl1I_)X5M21T- z))5}3xW_oQ(U~9#`7ea0VVv!o$n*aUH0&h_vM|tZB#?=`@So1a*%-%Ya@J5T>gU;d zyZ!Uf!Al$pkWVMP5P1>d<(Oczoj_I?{8Xa8hUlZ!=dbPxDNS1 z!t0SQAbhQkA2rsI1dZBYY$d!2`EJ6SF~MSzpcVB8iGCOA-&T6N{-KK6(BT+y*pEDZ zv$HMRk$VY0gb4yo&Kl}KeSS~9%-@L)bBV)obg1g-0Q#7t@DGfFm4wGB?lDS9oFvq* zBlT{5iTp6(*@}CN|B?8)s5iEd=l^+VNZsPB>H-WjezUVm z3sLVS`f}9g6MZG}!Y+>OUyX+OT?Sdjup>f#1K}%>uOPe*`6|Nek>5}F+9(<}5qx{X=1@(eND! zREs>k)!FtdFu^*KpbqsjiM}57+q?BT|5|i-lsGgZ@4wBNxCsO8?hYh7gVBuo(L~>h z`h!Hj>pPVnj!4ANhJj|0K>Lvw65ft{A>oHGG2hm+)4`4i^>tC=(1{M|#NjwPY$p1c z@8x~N6i#@Y;_%3o#7RQ^`>5|~|1>mokU$y8eNeN4|{kO2uLSUr!9x=PwaA|!d#Bcf#kFWSLmGPgG$Ox@@FwK*2yaF{pYT@Xi+gaDe-|2V?_rSn+mJs*_j}?B{zseJ?VpQ=Wwt@y5d7pJUqN^Q@>PTvDh}_Eka1s*`qe}q ztwe`q#Gx8_1L3vEw-UYr6Ko<0>X2_Hyk5tN8oNoLwdn8~acI;I#zCTQLVk?!W{mR* ziPMVu_$TC)Y}Y@$+=Yg8WssGxf7+0T3Ez(iBF)Y|)Q&ut@I%PY>&a#Q4m8Z|X#n1- zIJ6~6d>r|F!ecsRf#G8{gvTLoAUp{;?S+|`9Z?7kRK!5L>~X7bM?zc zp7|u%|L3A1hdAURpG9~9@+!g$755nPNR5`Ger1;)+rJVG4P6FVm9QhiK(~`XwW!}p z^ed3>CcF;$LBi{!n4pm)Sc`lQ;f)yR7zxyb`lCeOjQaSeoK4V*`sj$KoxOJ#I%E=u zHstw)@5ex=k~rx>gXjxT ze>%|@qW&1sm!rNK^xgFjg{eftEhJDi@;eEy#RRB4bjK^D2o7_TcVFs96o1&`mXj*Lc>-P zC=CO>P6B11elO8yq5e&xH<7>XzApf55LgemeoJ~`% zIPCw25r~-c|q5TJ>nSk~pkIUO{*x^6Lq2LLMc& z8TqOnT;*>?!$Um`vis~pzLoGceH)zmV_@>`B|1Dq9I6#hF^m?cz83X+h<*j? z*4!hPNE|xQVOdWH(08K#0MQ>ueFM?Q`~*(0&)7hC9B{~= zV(cJ>By>1HcpCB}gl8!3Fd-$3*$P(QrYSwnSEbl5-~ z>e1nB;;hlqYZ>Wycd z1!_lq!V9VLmXa|n<5S-N_RRgO52KMoD+9R@fT zAm2)O8uAwj&p_Trcoy=5gqxDf&;NW+4B6=5dePZgaux6XYq;`$3w?TW)U zEJVAjLf!NsbV%LlZ2J!6MK3vdC-VOGcROR%9k=5U9`lP7z;{8vqT_V6e;gWW(13A_ zB;@M|PecAY;Tej<{@^+1xX(g;|6Pt7h#Dq3q!Wj1bjWc!7`dqT5`7-(7Z7~`>T`&` zP;vPFpL}8{$3Uw{pi1PM39rTk^GTdq)IUS?E0Dh?c~|{Ieb=F(ojBAZ|B~>vHh-9CmWaB0I&BVpG5fjK6v!;J`CIX;Lr5IclNp zXZPVy+y`IK2fwZlep5H^QU7qq%exKmTldfHeel(N@O%5<>w9s`|9CHhPQI-V{!AZy zXCM6aKKMR4|DC7_-syvX)Cd2f4}P={{uA;j9eys?i3=3l2T$mO59@=EcJj-*8hL`# z(8Yay@QC6G#`JfctKU?Y7w4zO6_%uadD33-B%UopUzs_Sx>n6rY7j210*VB=0KDCK`c zzKZB|d4?MXSy9%belO9t0GDfYk=E-DD~q55B^z`v{Hk*3%{wqbYOV7I>>8ze?VnVf zB*V#rcH#u%(|4RrY}YiFv@1)hTh-jeRp*>YpI{V~J8Swh)bHp+UqkeBP=BNk{VJkg ziu&|vCoazhqF<-!|o@UnD{F&z|)UjahWs1_qg!p5A9-psgg()k+_4eEpiU`D-!3))$@UpEl>v}b<>(xAXm9Apm46ow5C+M!y-rl8swCmM&`&HUo@G8OG zy)4-$phLDIrvCuDB*WmW|9o`m0v8%W%+e^S^T|`}BTVwn?kr(3f_7`B`6viG7g3m8CD2=nKqP z65g#ZkLpW{zUr%a~;eT`%g~AAssaK|VuH z7wGSlTiTF;y2u=ET>fNYOQir;-On{ad{hT%&exYBeOat8EA(ZJzHHQ&R&_~x8zRV> zqlrHNuB-T%)+dO4e}e>meSs#Ezm@1qwZ7Ep%R~C|w7Rq$_&dA6og67H4+g@aKqOER zG((|at}(V|vR^!Y4?Ec%_KMf;VHy1|SyWmszP*Qy9}_7pzUG>F*TV0<(ELCsh%@Thc>^q2hLG4-&n`_38|R&K&b;!P(($fAVd26^7WKH?X1N+zC8Uyt$smioE;b z*Le+Wn7I62Hdy?nfz5E40>0RFoY-$7{XQJp-K1l4}!VXMb}0ed5w)mJ02f_5hpc3JBMS?EIG153qwSvHWAUNX+~POA^;T$VwARiz9(h z*s>}Dr4fIi%$F-_KW5j9_x=I5le`{EU$>5pm9Hj=nGdlMV%LxCWHCapL9+e^iB;=a zy0o4v-hPOk8xQ>f1nzyjR{*(Q_ zkXYQv$`Z=0l9ICG5;G7gudvLLK(2s44{;s{yneCmGnNQFu57YDC?0r%-Rv@3QlDfO zy5m&e^{f8v4~cC7o+Lh4&4-FnPqQmqI-A)=1E#26>kk|A<}I$ie#tx-Z`YQpF&VMZ z;_@yS@QSME*(i6wCssetBJO}+SLW}}vnt(X5_S9hy_F>o3(7bFQ~j&T5PbVzLk~`>adHY2+#pMb5jCpe|4tnQ}y=cbR>z9btFTx}(iL54^-yGAIKKeoIVxnVm5_ z)NAAhpmbmli{&qaYedcjYE*Gi_yS81iLbC!x9JtDUxB%#@)dT9+l2dng_XHYy#I;2 z*m-dQd2^<^Y10(GSJ{;=OZ@HMY?>JKDjUnxy_;ci->d9Q$N`se;&%`Yx4hEP@~MAe ze8X!M$_k2o0y_Ndudyt+kzcELL>zgIwG2~rX{mUYYCbC>y+U5C|Icog4Rc4`zge!B z`8pd5Lwf1!tPI-cqt{tUoGM^QwMxhm`ENiig@kJlyFj$R!N$14^7XY)!0YxfQ+&S% z?!RRZ%a#*Q&E#-E{IZ9Ip>|ctroYKfg_7w~zR6BYRI^i9l`pKy7Y<8nqWI!XWq=TqP@&3Hoe74p)%wD#ZHI9 zUi2@v?PR!d(87(woaAbz{CGq<#K2A*8$NXH~w-4r*1^d`F5bVGE z*m)3au-oSqZ@$egiS_BAFb6$t-a_{PW^_Y>$eV zBJTkT6@YmH#%1ZXi^~I5OBVaY@(xHB|2_+fi#Nj5{$x8lJu49DHNe!(gMZNL^!NJ2 z=&fvm`0#x;6nfs~b~Z5;mX%Oxaap(`T;ePBmj%=^?+g0GU*BiLVuQNy!GOql4@TC= z_t+RQ=mR#{6>K^E1D5TEC^sF1)#tkjJRtse5T*uQ?NC=9o+*}fLMfkp1n%wIgDi4M z#AlYktQ#r!Tb5N`7SXx%Q1n?kl^%>fE7<$4;7(<&SZXl&EK4uPwbNlG{^lb#(*xa7 znsxhIVYy5$PK8Ctw8|=LtieW|o+&%c_UFPmgyW4EQ^qVpC;L3Sv(V>-eyZQB&Bm zZ=v>|{2JEN=0mJh4E~l`{d_RchknMchhZt#GOW0WE^Q>N9l!gAB_;UekcScQ&v{a%$C zMyIT(@p68J#nr1*pJ3~!8mvsDeZlUE5Bhz9vWgPFue8Kp0#ku_{tI>|^y<<;uq;#z zTd$zMBvhhW-S4wR!ad9q)4zm0bV$8U4Vz`fu%oH)L!X3xS%FH}@=4_ysZ^75y14l$ zOOExcuHuL7+ZA84D@6Ggb}`Obe!pMre2k@u%3MCVrKE$6a#cX9!j1TK{rFGNrDcOl zg{~CbI{0;Mz%D^nz5fK`TXq6)3+TeEDdowc=rq`sE-T=JE22}s@2LfX24DOHjhDw80X({Xki-TS-EKgGD3F|fq==KYUgfC&E_34)^ zIY%zsP$Ipr@Wbq+emg<=B5LUPVg8W{*j}iMf6Z)ec|s+jU|D$xHlKlVv$$BiP|SlK zZ$+@I)E9(puD>J__N#)VM4(pSE`z%Ig1S6GQ`~oyoePsQ>^^UF$NM6>xN>yKT&fHC z^-ARr>MmsVK0T-&4Z%%aHrQF%Tb$Lf-kkXZ8!1wLVB=uvh1W~tvGba`0?d$j;|Ep~ zYw8*>BjUmzncw9VxBSQ+_j`4L^t9%OX^kFnRL_RodaCoo#$)y|_VaT^YLQw&upa2*RuL&*6tds8$G0} zF{Gj(A7^nbFZ{|Lak)cbiPFWNe`Awk^_=C0IZLvZ1HZAE?uokQ z^pxe-Q|*~Q%lu`3I7Q^kq6TAlgzNbQcFwDQ zh7-pfF?>AK$uCv>H1Tx|Pj-95qW=JXwHr1cRRee}*kI$8C>Ab+&2Bv7DX?e*NEg%P zYriOB+z&;$oAKA7D-?3@gbj$S?lrMInZuStF3=&h5f<-%0lj7ha5tPHu8HMAw=XOn zisff}RKo>)sy7B;nwvb3Pi#pW$lr786)50Wl^K93WM3>#^r+zz(DOzB`t;5pp-A0F zy5cxKIUZI>=nVlqbp-sD_EbYApl6VPAGU)Kj;q-s0DBWy#HWY{UxY-9#ls?g3?I!@U^sM0=VTaHkVelO0lkX}z%E9fSH&I=Um~N!DI;L&v&kRh zpQnG%kB#af;vGE z&hZI+7=u|sCK5X{cw#?)*eCW5;iKhX0jXZR0(#g7^so<@YKnn#t(cw2gR!PAsO1%F z68ST+dfE(FJ~3-3{}K95ei9Es-+@b#SewMtp``FSS^Ou7kA{-k^NmNOCiCPR81S+* zdiN5L2MzgK@43|<(XFZXEP;^jIB*tI-S%(E(4ql-^qg`xj}@;Z!@%3Ll=OCX@gS zc{;%b^b{D-(?CFNSw#9UJ`&UFElL2kD7C}*nK%R@I(Wn{-WbO3gqyf_7*tl}aGvin zTV5CrbvDwcd$+6ss2sgt3CQ_h{uU9@1$=@ScQT*DRm%op>C0cuhl{xFEKY1Wnctu{ zIzg|R5QDH^X&b?ZpQLIhsOAkZA&rkrQe7@6@1bULQ13!uKRkLQPfU?3G#p*^DJAIB zfqc-&tNA4H5jf|bpf+FyS59hD48okztG?tyF2kx07KB-2+DLxYSUq0^^_BznIjTJR zpb~_+0Y(_~y@dJPw`LSK0nN^=gN5TWjarWxzaTn`nX?2M)SG)p@8KVXN`tB-#(f@Ah(zD8OYVA z@Yf@9D8VhluA)~Fr+|E^qGJpn zCl7p34Z)xoF_zyB!);Oqbp02`^2fx#$M7Mdc08Q&{+Yp((_rJ!t3-16Ly6!K#RnPu zG!EMcy^f6@3+qT+Kj@cLU&*(#@Ko^2d2ZS3Z@~7~d(_nV$RdP8WLCIk=vd`o`G0Y1So>#9D zfAaD|J@VmsvT7EXu9Pa>$H%}(hu0IuA|Iaw6{cQy2BF)$?G|RO#^pGgw@ZH{-4)2>TB^=*97BX@F->x*RZ*U1st{F=~Mj zJA&M7`Nib(PLdM{#DkO91!wcq#oDQSm^*B>Y@f>i;F4RIqjRA5=YHsI z;fPp#4qvR7@1TD05`-r&J`?<5VjDb-2jO<%*TliP#1TDOBVyGA*tPw28ecjvsHeAJ zB-E0S%g^qI3rjE(Zn<(gpX%xtgsE=dIXtzW>GifecRqjI1#LZbI&2$MsZBk-nchIl zZ5Q&4m}I!svdN}bWr7F3`Zr5^sP1O!v4C<)A9wD$yM>7+@GT2s$q zCd^?_p6PP>+EEOBYEl6o)r=JZ!uiQ-!I{ z&0^hbsK#wG;QrVg@Ppljy4n0We5^k5!`ld|^7VsD6E+ydb9ka$wji*aFls~%h~0sZ zcyJC+liO$$9!^3Zor!I)N1dskWSF6VIFbuf*pe&wggo6V^l8A9+X0BFhmk3ViVW3z z4+0aT5{7lJ(1Xc@!PI^JHLv7FF03nkHZa4mkG+B?j@7+D?+r}-IK$MBGvFNql@i90 zs_*`D`Ir>FaF}{DnR=IC!eElIJZc6r<#7{A3Xd|u?V@;jEQ8yS$45ymC7q+YirzH9 z8#BH8psZi%rfS>LPdqGm$l0s2>Pgzt#F=w>YJ#Oln57ROme&&BoWai(Z=A-j z6sxk~c+szjj}dRp<(#W&%7Wqc!CXFkqRw}M#~eYq87zZG)v8}x@Jv*m+Qp_KwZ~Jv zq!VHe($7mQpKMEUZ4u8F#l_IRFBS1LF=9547mpP4>(bOvxAefY^uVA$Fe z<0+gCa;~yBkd}VmLA?n9PqZw2=K!7+$R}C@Eq#WyV8vH^?=|z)OtE=Bui&cdS*E4; zq`3=t@>n=M(4M4bS7>1U9KnL;2(X3Cfd>J-g!<@jXM_$YH-El)0=|@KxafB^KNs%* zf~)xgz4Tb}YzWO}!TTcVa9GY*C|7Vx6fS_NrfLto$pPoxj1w*6rhJGFC4yyK-Ba5_ zo|5P_)O%G`#fuiG_o^&-K>FiCekSx6>l&VYf~|#`@gYnoD8BlfP3~vG`n~2Fz9J1~ zm|piLceGNAhmczU8=+g4z-pu3${T$> z+?`)syo4_h#}C8NezODt4W=yNBcU3=cn8$L;G^&!)8J}2pnKH;9UcR3tLB&JIDZE& zSJ#cz{H`=O{PZfDd;lkl5s{Kg0il_A;|88`@(KE@TpmFSZ_lfo{u@;pFSwEW#;S=W zq$Vp6picF=kbc`E1m~YSZsa3$Uk>R855Z>Oxf}TexS0=b`=PFeOgC3zq((yZEy6wS!vhA&*+^ z)m7&;_2B}z&TGo23(BtZntJ&Hdqn(n7cXv^xSAj6C;qaEKQdnF^aD?yTByKJtyJo& z7FPJ(FOJ{Czi4@S4gc<>yVKn2CM*fOz8(K6&8usrn-_)29k|Bew?HA3jSx zPh7V!CRxmCpkDkv+6IsDrET4bY|3wxIQ$roPq05R>_tB={YHt*4Lp8`{eRGf z(N;8GT(E&piN*i9s^K|t>joZ*ZPg#`+t4arf?wn9&ob}O1u4=6Nq?LVi$zn>4srJ5 z{49=&KZn8mB&> KBqj%AhW|fq7f+}0pIXsu-et)9ss(gq! z_38c!u0NMfR+4PPP3lvx$0JEKc2%hLc2%hL^k+O3z<=6PT*S>Ec6A@3pBtb4%m=Yb zQS;M2Oo5qwnApnQm40Imfhc%5 z3LgK=wKO}O5H$AmvIUY+|g6JDO7;=2eZ`PZkw zo~dS3nkx9Tjt`sgKWcbG6kIal({zrm@8zhXe|ko;A%oQKwAMj}37_OpS?w_44{G>g z6aIvT7n<&$Nb?nkwiG6`AmVYvtW0oaL*8?uvpRFybWtl?7^rdSiwG z5181$3QQS}4^=}{WWqOSxW|NF)_l4u6P~PUH#=a$ zztRXg^?ZM|ggyn=c~l=QV5(rdQyoFG3E!;Y)=hm!eBC0oe5wimRx3ZH2iGV(uVobW zWRNw${y-bC+k~I{K;@9fgzHXhWfc6d85cC?v%!=xW3f5{DGHwaLf;WrZc{mzWx^l+ zPnA%I3D?^fmPW}~W5U1G2;F7EAJ#Z@*o2pB_$d>9orVWgyx0HNeYg0{eMk1M_tg<3 zoA4NI3Q|qDbU-biVZ#6Wpo&lFgJb@suT|C+^vPiCUq{sni%s~_V=BJXgu69XZ!qEY zT03qN-a#BC`71MJ=mw|KgvV%7eZYio(@bow2~W_^n6&C#)u1=R~{-*VQtk7gzO9&v5i+DC(!Q~GA~;000e!YKHLD0ta5amZiQH8U8ije?(wf;UIOttHpyL@MDV z|GN6=oXCn&VR00^C<^X2;Wultwab7j`ln|cFl3Pa=>*kB!2?n7a1=a#%T=6E^*`h4 z3`PGPQShZv@HJ6zPZYdzOBCrn9Hl~o3AZfKmT41ykA^32?K|RWTAUzD$94aI?k05+ zTB>Jg{{P&8D!##lSKqGUyG-~KT04~{{D6j6nQ-C23u;3LOc^J%5!6S)15xmB6g>W= zzMRn77g)x%GngX^zBCHHCJOF}f>$aypiNFmR$o^^snj~8Ki%;QSehy@a8DE^{0I~p%aw)Q)2{=bG-MZs&M;HRSC&1PKCIoct{)A6cZlS@GRhp{OcC*-{tDuI7}5{HG&FE zxK7Yw6MkOnV5tc&);Ltyi>vx)(K^`Bo1v)RUj&sPj|o4g;k!(@zO$(^;T>A}!zO(1 zRxP96lyODFPnmGN1IdKjbeS;W`i?4odtaGI{*}7wCJUyFSG0^&6aGi-m@UJEf1q{j zFyY^8Q@Jz>zDC1){C~ZICrX9GCS2du)|zmgpi?GXZ@;-OF0f*4_sVwkWl{Z9b?)O$ z_)}W>R1>a;Xs4L)jaqq!30HT`kiW&IjMudag(h6zX0I{fsaoXIZNha~DKp{wwDzkA zC;8XaPtT~0QsGn-yg3SPE$z$En3q(JrW$ZX|MZM3Lk8)ejxUaa7e&F{QSe*S4S3(g2%tomlJyVj8{zlzpk$iQ-vuuHRe-b!u9fnCj1Vqe31#) z+gYRIy8oY{0UPuT&Hqn5r5Xyi3BRlbtvn_?S?jpWgzN3>GT}mo)RJ~mi1+| zUOqJno@K)SsI|XX#e4mKJ)_7};euAdZNg*TQVA_H;d3;+(uCJ)_~AY{=Kr*oQQs$n zu~&Ah9Ry6cPs5u{xL)4s?aQGyt$eBpA6|j>N&d1-88ayZ{}r0>LHgYd6RvwcZWBI2 zE59oWegL>4|GEX#E7Y4R%+MMNnD9j!9yZ~6`S{oSa$>bsKD8HD^-r&m)tjMkK*txG z@Xxe{ilX4|DEO|caLvy@a8-sn1@%$zKomS21&;RPnV zT=N5pOt|OYs%2bb!gnb+O?dJI6|Xhng&JOO!Z&Dmg9+DVOM1iP z|JQ02noSi(XQ&;AO?a|4GWKTQDX^Yah0bcipVBy&tmC@>uTMdSo}t$7nSv}6UZ+i2 zfeF{!S!%*PTE|5uT=+!G*kH<#G~8ptb(z>@!pH2>rpAO%(+CZi@VnLW0&6y9=ob#z z&b};l{!;BA-h|(y;Ta};tA;yF_$L~^SivFxy88W5%P2HeIIZDpOt|GgRD#?l{1y%0 zWx@+Jyvm4^{^=EJjTr{~R1~~93U1ximvh^-j;EOLz-i6@&oX6%kwO1CqTq!lJU*b7 zFEZg78onk9zG0Wi|L^xJm8Bk2g((`o%Y?tE;Z-JlpN7{)!B6$ziv45kyjDT#$xs-; zvVW}-#NO)5(kC@M*@VBP;Tb0U9~$m3B z_Jvwg#*Z}Tv)+V1tKkhMyijZClnGy>;Q0L|@m~Lbx0cats_+{P51a5B4QIRi zPR*Y*+-k!Aq2cj;aLm7?WhD2>V608UQ%$(_=PK(mO!ycLpJKvq(C{o1K8tXYKZhyf zPOUJ&9U2rU)0Kb zO!&}WsGKY7#Z~>AsAcTx%~13&OT#No_-z_qWx^XY&*y*%e?TjL*n~gx3(fzpHD#1( zBda&zTeOB6O!zJhKV`!AYk0tfAJ%Y5FlGEz%V;*?-)MN)gny^utfH?LWW1-cI^KkD z)9_Rc@A3a%(K4o(D%5MZ!-ThJ_+k@YxnJ$L$b=`qui_i};sTqYWqA5#Fg8uCaClGO zk;Q9s95CT!x2WYM6YjdKnsQdzw;eSN`Ey(|1Lp!Jyx_dL$}KhFVeM$9R_mC9>t@sX zq({90A7a282q*cc3h{=FQ-%uT4S2HwpJKqn20Yt<*Zx8sF^~4u*r}RXZ@_i`Q^{#C zWT?zkE1WXm>U(c09x&i)S1K+UaH|R_|8F+naVoCrf7p;Qz)*p`-8+JT2Ha}E2O02q z13uV*CmV3%6r{c_^mbsJf(%22p@xR07;x=9Jta5GfF~HrI}EtpfEVaE@qf4>W3iqA zJkfwJHQ-4GywHFr8}K3nKEi;nG2nO!P__S*zrm0(%22^=z(*T!j{zTJz{?EySOdPx zfT#500;@D+r1oa8Dg&Nozz-PkbOV0afR8iawFdk;175G-kUvt7>kSzVh6+aYJ7vHp z7|I6>c!mL&40xsiZ#Li)RryzQ!-kAWh6+s6ZzA*t18z0o+S{N?ZoC24-eXnpWCO0h z*Q)fNYRE7emJ9=av!UZD2K*KSo@KzN8gPdJpJu=d_Gtb;jcmFhW3i#a3duQ~8vyQc^8n z>VHb!ncHO{o}Z~?D!_LroNh=rA)RMPpGSIyA>E4f7DKud>3xRuMWl}!(pQi^V@UJ! z1Ua71>LsAxJP=?=$040(NDoGOg&{o*=`DtIBGUT|>5)huHKfNPea4VZ7Xa|ty#$O$ zI^B?-h;*JIJsIg0hV(5+Z!x52Aid9!o`v*LL;6;v&lu7|9sqt;F9EkBoo+}kKswKm zz6Go+tK`luoOBGPAi(!znETLAELdI|U`(&>iu4y5x8>DQ26VMxD@^cF+< zO{Di3(z}s9YDmA0^cg)ZKt=We;B$Hj_!ZLWhO{5)JVW|Dq*oZyA0oZQkp2kieTMXJ zkv?ij{{iVUTKXQ+(W3zPTYCxkFQn5A=|3Z#XGs4Q=@o|b=SXkqO^eqpe1_jKS^Vo$ z_vpjbU5`TQBeLQFNR!z9oCyirx=K4Gq)@fmMt2dc+JXV#0z)K>dAtY>S?zm zNzA>oH>2An6hE~PgJn}@dY8w#mpm1T?2K&XPIPaI^>ycS-kvdIEYzyP7;d_jH! zR+_+kR?fXdDVv%kTJFwFS~`=JM=V5x#n%nKDSRmz3to&73-2B=P9F}~r%h>twQg)5 zQ4xQB_YKKvy%evpACe<6aCfT4Y+~>j@wz3~@vrU_7c3b&e58fh+bzr=uq60j%wgpL zi{0N}+`MEQzg{Vj7-J|iag4Fg!xBIT#f159IjPriY#4StjN^E%h>a0I7@w=r* z`R%uhKl$4?cYT-~%*C9Tv`?(&)_oqxkB>DZ`$*7}?o&lKI+(FqumJ`*raT504Z)=q(k} zMyz_7dOpUTxMG1uzr(4z1Cf#@#3&k`$+70l#);lQKJzvXU_NO=EDi90wPc2Oz9O+4 z%5mOcg!f4aO#T=eb7d;1Jf<>CAC=OMIw~s->wuM;NyA$44pq>GMI4S;_!Cza^(WOj zK%yD}?ab%wF2e7ifKJ}wvZa;GyB%$0)=l#Urn2tHC?-e1E+TF@>ZN(6dIQIp_e?+c zl75rLS&xofQvN!)w`iUZ%_|e>CM}mZ?^ZOk(mmb#614NB%ImymEEp%Mr9%oxo24NR zxeXdn#8`Czg0LSfB}TVh6o2*T_5I3-*#+W>M<+Q)C|phbf=UO_%8g~DFA4syQR6wZ zMml_Dk;RumgQ9_KWr|f|urt)kjbhHSo5psdH1(^rGLCtNYOQ<%{K3oOUkk5KE3+<7 z@dn44=4X@}R#cgr2C-pTa(oK$H6Ig^uMSvjC>%HIf30eDW?pkzW5gSmKgCabQ+#{* z3jU5?i2RDD`73W6d~QWQ?ye_9R*s#8-OW2>4~ay7CO-}g+Z=n4$E+cJ#$xra0ERu6 zeLKyhhCKr8I$9)9j_N%iiho}D;)ra(mvS4=_umoTTbJ*t?u))9(TCvYGU^ZYhd#}rpoi_uI2 zbvhDBa)NzlU^6jU+!S$S(a0NGh_o2iQi2QnqzXr}Enn{4 z!^#h$@uJ-hc@J+5H6l+Tv9KuV`m(%0j5lCOnV5!dH){i{YmK;`+0A597^*4$yl5O( zCe{`u4J)($2gw?%rWTmbAukq#MU(T(taWNLTpII&7jhXjWo2vk{t!_|fNt~=j6<^? zjQBC|w33=hjrIQ59`V8AJYgJQGRtXr9!ECDvbF?F@Q@qUn0D0?npVkydghHpUS(xHSf>X%PSM#ApZj(U6up!yDMjyzyv~3}0h^)!;Jb ze*N%Gy>jGESLLV(3ppv~%jkuaX+&B1zF5UEdjrZu14+c>e@yc7#FJV4^qa&-H;(0R z1Zf-dY)ero^-|=cJaOnxCdJTnh>o8mr_ZH+G(jgYtunSpOlY`sx*+JY1ftt|^^{@= zL;6hrZuEi~Y+&vo-c{PLh%mir;_yhlOK@1njT|0^HfWeuJT>ocvcdhc^vD&ti)f%B zt=7_qh>ha%r&jZ|Z_|z=ZXmS=GZoh*UU@2=j~_3luSw^p|5Cht%~LE-rq0AYXF5_~_H=+(@zH>6A1Py{^lP z_mYN_mUj^+XHCFiw1H-D>_9H;Vn9xs@-uQM~<`1-6gS z7l~Y@_|`M`PgLesVi?WVy)j;E_h}C;08WSs%8QGh8$>G9 zEEYcJ7!4I}F5ee(l>+Rf0>67Mi*LL^QGmbQASSNQv&Dc0Qh@W~^7Z3JQ!9+Uq_`h< zwMTa1GkVolq)8yf%JrkuLN}Ndqt||tVw@G7&(Hq<)(a;(*Y#p31_+PVPR!!aWh=VZ z{T@{xV4;xrd!2$FJ?Trqq4r3;qA2g80hMF1V(xQeR1Vc|&^dIqiBLH7#FMGw+>Ocn zJe5niHz;Oe6%hk2xxl5A2NJ*)g;Q?TQLAo)AjX1INEg1ytDA9H3^&LXUSvKYo%0Dt zS@sg0bjZcERD8kziFs9^NoU>$FtUevpyGk6m=~)s?>e1=9_ID6YopsE{T1d7YZv)V zGx)nI#95ot`Ghv{(M^;2M-#=@HjRBO?JZWWM<}lhNeMIF{rN1gfTddRpDc!c|_dY5=8Rf|WIxOzbC*e2wc8ypN3;Dbc&m=Lq-wk5i3+aL~ zzV>-Cobra1>f%DgVbjnW{X7KPCU%Jf-;u+agIlUAj4SZN(Fj-B7JM-iEZmJb5`t zX@)n(5Ny3x`839kr(YIMItP}B@4hf7hO|)p-3zH?oSF-_II|rbo33>HuX(Z?jo~+x z#vy+xcE2#bUzzodY2Fn(m)CF!ad7y`tUsRS-NNm>QVkPVxH)!EtV7PtWG!htb6Gj@ z)}!OaU7LT&oe* z9;FNWIpi9opSdqj+_NQ(f4@~+uw^X&>(-qw{)S5tw{Mx5=hd4)(RvOW7c|knD9_h< zzteZ6e|~s-E1E~1gyx=!bI9iWl2(!5x?)BVnxQp_jMbIH%)v8df1GA@7woEJY zi1Hlnf2UR4y>)WpqMoX~y)I}K|G9M{pW76PhzY(xtcU-0kPa4t^f#fF;R)3F4I0 zCZ6Ax7<2J9v=;Dl_QB&cI*akNw(a~w6_+GBJt@P7Azi;^S=bthRM$f01|!`ee(M>< z^&_IhLEC3@t>TI83EU6jBiko&XTb8VkD6Ho2_2FkG=Z;m91_c?vfiF>!C@O91N z;2o1k9MXy$RGta`50q!3ze~J#$8~~b)QN~ii+C6oy|jPw{>^|68s6Z4Nm_cC>OR8#&EtsEy#U!QQ5=BfTjt`PmuViysE_lf2SSgbgfq zuaL<(J{*8Q>DX{@@Dh`QTSs`UNaw(+OYaZ&W#IRG%DjhyL7wxZ z(rulX^h zxw)*R-tsteQN_w3%oU`LRuyNZWU{qv1asj!tvAaNtxIP1vmrYeFP)I z?&yPng}IhlSm+Xj`@ya5-Q8%zL(}Gr@m>beDNi!rNE?$kig&&;Td1_Ft)_5ltC{)i z1kNDsF0=+ktf9ct%o3|V<$AUj-jR#EpSoqtb;L3%v^9qIl)V((q*1W^V#GBUzip_W zi}ICMBCd?h4lx&P>>bWrdYe)|hw%C(kY}^86S+3l0*`^9J$OyF~Y^H*n|0{jW|E+S>ADDU=7HN;savjR>V@y3A zJ+pw_dNo+Cudp zLglV1z=s8F(3C4v1$nmc@V|}=NF}!Ez7dtR;+b`{WeTfkaXO} zTusQgLarBZw$cSCvurwBYvs1@4nn@9?_t!?hh(w&bGcC=77JDH?-V|y5mmm+yChT& zevgRZdR~sKz) zr@M1Y*xCi$fE{z+a$I!xLkDQjim8y$W-ez9Er(o;N~AD_%}+vK#BRkZNK|7^+rSp- zGuUlVhnSr*nu1uz>#`-TM_X5Z9&u4RHW&x#!_R5g%f4EMKS!nRzAB(*)B zFH8NMA4)V{bW?CSa_B51Mj;#Nh?y7Ik{w1qP0~;C%zctun>r0&oA`lSTZa{-NgGBo z@`FR%OA_&)-8sP`cTVNyJXC6%nkRS9%#&wgQZLWWlP^ukmzyC47boS* z7jDXzVP7ZGq$4yghc@94MnAL-&!@yr@1*f)b}zW}OODz1e85_klg1VDGl+}S9d&zj zATQ2)eT?9MwI+NSI7>LydmtY5I=Hng%(;qJ@-z2rhnPIXE9IKc@--LMb261xX48^@ zz&^=jf|py)yythYatSioe!(e|_NEr{uEsW}oax&sd7>R11}HT1*z))a0)==cw%pL;}`c#}|)z+4h$CWT85 zl|0M4g6kla>+_Jdg|;Bw^i-aFe!Wv(2w@U}Y;DyF-gPeh`0havvKt{}d$>VOg~{rW zNtpB@zn#GBxpo*Bup{?6*5Zt3u0QVwMets>r#rI!KfK$z>b1WLt^R>TQe0@&VE2+# zZo%_z6dZ6@Z^6)@_gX{8|Jpj~l^+z+sZ8S_5WNZf0MXnFeTiPN`5Ht&b~VxN(M0ql zETaahzXjDaISgUWqG8N_{5L8OkN;(V>khHFd{S|v6s(W6uwx=hf$4Rc7q?U*1D++#7R*t>_4*r1nRBJ36}R8Xettbw`EZlewb;bc9K$ z`SavgmTVv9$lkI4y{Ezd)l~>0r*Z|)yOp&(!TXQttXv?!xb_+T&;>|9^Vz(bCQUjr zp++=mmPY6(46KAa5V|(7td~$(kt#^h<5v^wx#wzP-LRD=VsCOy2DPVoh~^``4U=dd z#f)AhnoGpT-b@iRoy$N?2y9Eqd{xg#Fs)=FpxN3|?hP*_-O7V*<;gbWQOoWp&=b}O zD%Xy4t{~2^ehc$MZE@zdU?KFj{;jno7#mB)L@AA-U=W9XRa!%J1-(RJH|M==8Y?IM z(otb*5q~HH{FPveaF>PwphPQHX? zdJP((*(A%{eRgm*v0uVSVVSd~Z-{L}yp8bvKfHxTkzZ4fd(0g$an%@c0*Fh(A1)i( zrLXgAti#!~z&B3Wnvv%tvw8+AZ1|h}8rwjJJgIn9mhb^vTR)O_{bn5Ol3+AUBY4*W z%|^%BnPQ6zh_xY3ZUg~NEO0x~Njuj6Bi&e{uv*YE)ddCbp2^GP0$iTWW3{lz7bjV; zQdqD$s`Y8wsEsLTgVbaea2f95(%JYi2Y)2C43xHE*=fX*L%a4xd93_7WsjNQ?>vXo z!+va98V5@&HTCWe_Nydn-BaU#PUqG9pyVi-UKk z2rHewuMvYM*RyK{^CtIaCr+Vx30p-Ewu%85+Y_ikp^GZSaWlFjhnnzPhI`zFt;Z#c zePTPtdwD>94UKi3w*CDh>O?L&S|v-x$RfsO1O_$$Wnqu4klUTj!_m_p53p882T-U-3K}R)*?$CXOdZ_K zHi%k#e>Dno`wgs|eBfy|q%otv!x_E^X3@XTYAyc=objoBs<4IL9O z(ouCe=P<%HjBYhX*@2OTFv?JhL%xU+DjXQ)klh$r2qO(8fkPNw2%`-_D!MSrQjC~J zDq*ByZKVDCj`Z3C75 zsc@+!5IW$I+AJZKyZJ9}-nAV#`vvFPF_N{c9>az$-^saz{cNpX-;F)IaEPsS;7M4+ zK*#_zgCX7`1aG>-C%p@6J`*Km-Olph=1S-Am3dCc4EJE&{7k8*CAtsO1_mMOHgYG z#3_NG%^a?*Smf?Y0$g_JF>VJ<+rIMR%$_;h=8(r?&agOEQ$w`7BNMH&4H1sD5~Mr; z>5i1*c@wzw4wLiY&QnB*qVmf6hz|cN3o6@Hn-e_ikjwi~blwduv4;YeYI6#Oe(sz= z1V7?%(ZqtC+ejv_JHxzbkbK1&l6`>nlPvjiVL!Hi1*wrIIPxVu+7;`#CY!DtK0kM$rm9jT?5d`kUW{RrhcrWCbPez23oVa7Ez#O7JJBz zJ!Pk5r2oNu&U=wKg#Zwuk1cf&QVD*=#8+PZT%3C})wUddOZ`YRAl|$$MaY0hKt7ri zrlvj-k&yuo`5}l*rjvQC_}kPU<_60R&!GwYEn{(^Mwphgl4W6rzv;`FcQpQOrq_;jbj$Oca%x)P zSc-5-;FA?ZNJ+d*{!2G_5oqJOI}><072~gik(N?m=_5{r8m-5bjaeJSdfeUFQfp*- zI04}alud4r9O?xB+QGj)d}63pVlJJJVHoC%{h&m({Z!edTTV&?&5`LeIW#c^@c&*V z2Mok@LmCuk3VFe)e76LL(~9egZE!e)TZekHiZj9_8J#mV;xX35CRW{*n+Go;PhOAP zL$Eky&?n`vxHE|JKVc?c4F!Y?&iwV5CF@N7Q2i~mOeHH~RF4Q*CfHvN5v52OUSO9` zk_4b^{WZ!45v8Rv67~h8C$>TJ!W7%~24=c1WEK&tQbf!qHL8^$j&RS?zQ? z8mNVfN};$)jGj8`0&z4QL`-TTtJaBGziDb%Av!}PdL4AaNO%?GL~qxJY1P}^fp+nd znSC#Z$Z7oyhx{po!D`Xw5w@`Yj5E6)a`E*<93oAGiNZ-SjfyUvQJ&NVnIR&bR})w` zlvNA7J(Pyv*PedtgW8!m%(y6=+z#SO!NDD^n@_^0@}7l&rf`BSbSZ>+e7Sr#`M0+2 z7fw=`)m~D%I~Q@pEhsD{?HS2!-h7g@n3k32u=kU;A|5uKKXhqZo_qxI(f$pS|F$IG zrz~kuPHQXz=$v&mL-ZjR%2|61f`>gqHk7kI-68)CM%@;RV@(@0v=JJro@?G+Lb^E~ zOB(buj-xi>R&%(sMk|m58j7i03m4bb1dXx{BF(jhIN%{-pv@sah*`2hYm4yrAZf1n zG6h2~PIAiRM0ucUMPR8LY@Q9advJ8Bq;YgQ6KPtYg?{y#1p(&P;A$JCI6@0era3e;58-QzV_j(?j1aEI0oHv2zLi?_8u3R=C z^l3NeYQjK@Cni?x7~|-sAndBl85Nl$9NiB5IkSt$veaT>EbZV@Q;WvdLsYF07_#vc zN+v7=nCFx5OlcVw4rMP-i>NImt#o1y*fX+#XEP*pk_-;f0$Y@xSdn-FN5HTNo+~e$ z^u+Rqc0x_bw5kqxlD}NSJ%<>+`Z5exhefda@miz(90HKi!Tg%-Bh-a+`A{6GK(7i% z;t9Q~!?Wpor|cO)dWA4MqJ9$gyW595WS3z@gjd(%hRpO?5eKtAD`Fy6#CJdh31p%G z%8g6n6CkDwwz&g;qaio}ESu_CV3L0tMp$2lDSE`gBCM94Vo6sKaR2!gW73WAux)ViGQ z{h6{0AlV8d^2j#+ZVox$L8_BqC*;^l_%O{aD=C>nqnMsHqryWD6f~oYG=tOz=C?}I zfJ$h<3n01*8bA&O2`p(sB{bnV;G_w`MNYXAnm~RnX+x!^4V9WUR6!e7v~n)1MF9KZ z?~+HdZoIXEE^F*bXDvn8qZXk<+Q}`4K+?`F2BRuM=Yeq;6$oS-tLGHsI8Tus3?LV0 zzem4|xM(mmkZBmmo8LKnZ-M<85wN|LJ2bV;D2xgIRsn1+Y~`wH84W|lXbNa;rfEo0 z<{zh75EQxxw+Cg#r}KQ|PW~KTJk4oA3wP+vK69Gr{}$!k7dRYfGrz3J4;^$S`hP}K zsBtTBzf+zBZV6;u+D|9@Hi3nKhn(_wfD+dCCX9mSi$>Q@J}!vsf?rHF;1c#gU5E?P zIZ@;3h|3A0J3%lx;&iQ$!V>D~YNG#Xty2$9i2e<=mOh4-jZw9%@JvK-eE{%`u3B&c z>pCK?5~x)G>h%pOr*=f9+fb$nmWwBA*|-vq-=?-CuAiES!qI; zijMRX2C7H;WPWMs9lmi|m^NKB)$Pb9KcI4efFrcC_h-ZDo*uk}a6%W>YzR#uoNzkR z!}i=u4ekNP(MAlFzo956!ma0MG+<{Cl1j(kE2!jVsyGJ>@HtbjL12KvC z99Qdd{_(flwMrhUFo?C-;GmIc)3nekMTD?A=Cv_MHTj$V}(bS}3YG_IpU`m23dAWC5gdNJf`~?mkvASdtZBvp%4m<-%=x_i| zG#M8vsO4N_tsFS3PiTgteh(IWTmmAN>(RBJe}@GC9!=p9uA0JqWJ=x7uM3mMN*5RB z??)I3zI78cEOTRFc>czz;h=Djlja6TcnP7s6lX-5wkr4seOwRn@H}fc?4I-Gk7~H; zdUzEXpKyo1fLB4CxAk;x_ixrZulxk%5De=Cf%VwfHGvR{KNImzjgakCoNFNn8Hj-h zwixF90ThvI6M$`9UHnA&-QpX<3Nff$h{p60WMCS8HweKotw9>4G2izQmNjjJ_-KsR zicvO&m|U|%n!8E{@L>$ z=c3UD5JMnV=VPR)?{U>KRv0q;6zE!JNenq??+rUb63*d{M;knoCbWlzK8K7^<2Gz&d0S>JXD5P|-MfSN zoLsPvt1f!k5f0@laY=a^{OTZF=XUtk(rmca<5~5tI4drSB`NdYjYZ%R7K)3jIc3L6 zL+dH;no59+QrxKwQEhj_VR1^+Mj|5#n(2u+Awf-x<=O~ z;D5svUu(GLYmM>GMWr^_?GD|&g+%`Y$2jC;w{L}GuxZX^nhs+Bic-#10JhuEK}QOV z(l}d1;*aw*$8Xa~&#Q3!;G^|&Cc*y<+E0CztEO{Sp##1ZxhRw%o$Tv><9;yMmrA1m z>&X9ZAB704=6|0Z6WO-`%(LR%^XEew!sR5Sh5ox z>6t=BmBF|E6*^MZci43DsrMo6ffN7R=WyhG7w5~s?~l^)dgUUgj91~hp*5TNiRa%L5el4vD7u??Bnq z7rAO`SD>4tx|1Im$H~7!rqNN)qq&EnAhtnj6+bDFSnvPYAhBq63VMZYbg|H{Qsx`|H7tiSTT# z@NAcHOz~_xK%8PJ;WO-eMpez>smXnW{5%FiLXKTYb)^6)ZvxOrqk?53V^oP{mSQjz ze<7npu^0}kTO$k+afhxyP!M=7USsV(7E`0RWgq9)7~?k5pJHm@?Jkj!PXVS*)N4ci z8Q2hh^s@Xh2`CLlAanUNvGyf(BS;=BU2dbBu=+kK9r>MD3A?ZYc0PqI4jQ#G_pr8u z;0W|vx+k_qdY9tsc5ma%x|}$0G6Gv6VC8IrjY!4GY%W~-CJ}XaYO84eAaBPMCX`F> zJ7pWr8v|IIi?EF1CGHxA~V?ya&&rSW@4`lA6hs0}fi#g9phFAs_2Hi^UPp zJ>bkQaaUoTQ+^W!Ux#}bIv0t9w5X9CpcSwSiyD=$x*|jljYL6xQ!o?1Nz&Epi;#Bg zwYH+MHqAY^!_XkGulV#a*?AYG{(DB!or_dVnU!NR$w^hT1g_b$sHa?nfOAS5rgdK? z--36)9&F=2Y{SytMfN{qoV}v>-Zxq=K^3i=KdrEK&pjz&9WPozcPlw%CqtVQa!9%$ zd~R&VB3g9eTCuiD+|~*wVMm2&$X?D>FGpY@g78@0QY_U7zCwO*u^iSB0R@5fYs#{S z@D)RBj52L(!&q__snWjtIM*Gp)#q+WtZ*ZLAoBL$c*pv|j@`N3z|{*7QOc})WhCw_ zv@C}MTlEN6P4=GZ^2=0z@FLp2@1(;wf0U#988S|AWa}SAK&W-IU|5-w{U^@CGCj&6 zK1Q2(iaaFy14*2`75IJY^L)RDM_UEU=6QrWbhwXIOZFe7y6F5OOl*f`jGy*@6v?2| zI@MbCw)d3YUN`1nk2=xC8*(!BjvqrI32dZb(B%*UK@<$a6wn4f0?F>cZs%eG+$jVA zx)DANfK!U>+=Mb1u}53jC~Vq>wmi^z5xRLiB!8u|IZ?hDq2ew>%pD25nSnFLqR-VGZ)Ne|v_@UMJ%YS4!{EpaY(g0GB;%8I|OB zAe=c0#}CUUv6f2~UQVMsnLOU6niy*xMW=yJ9!(PHY$0xxc3MXI6>4LPD=iy}?ys$$rZhV0V-%Px%z#{!;T>Q4<1MZ+`)I7Lp-lZY+md z*;1hU7hunE#7oDTuu$FD?^PpCJ~kl0KQDsi2`3<$04b&OPg*W%*(6tn&V(O<5VS*I zPhoMVGL%n8lN64kG9=l5Mj3sTbHikkz$tDwe)q0!c4c+jf=}aQV0t7=Y>luK9Z~d- zDoFif)ujy?OMtg)OB>#9t#&=4FKu>Zv-rUo7=?$q>P{F1d@jyTl1+i24dlr{dO1Z7 z@@fhOA(#(euLz;du7}{gE`$za5O`sqB49!e1y)Iqs_@s9NBebqs1SCz9zoczV5ci- z_qhn`HC?G#$l-k(^%>x6ai6Rt=S$e&dg$U2Jv;(E+%}n&pM-H34!zuW{)Ll;XqT4U zxgeTOAp+YT**^_8@p2I30cLQb1^EF2wrVs&zV1a@{JP&vMH%4Y1)yNoudhs( zPCMoLudY;60QwG;IfyszY5&;<8Mk9Rbo-|q;)amXiGv`32%!s_M2G|Bi5G(~S&9XR z{C2Fws~H2PKzuNjKfr~fHv{N}qR{|DmfiisJvjC! zi2?}>f};z|eMVuy#4Eyr$tPC_3-*+ppixR7If|6P6Ab9m1PLSQG6+OkOXJl7aSj;W zQP=?Hc*F<(lCKQS-5GJQnB+(OKm*4HdB2*)%G)Xs`~SK&N1`F(ni(Cv(B@VWIXMmX zNLr@NG4Ma3RXPuVC*4qs*6BQel&~0h=LLB62&THHan+tzUt7{Hf^|4@wOZN6g7;n# zmK42*i*}6FAe>t*HfKa{3-koWBXHx#BRd1m6Tc z4lfD0bKJMVvZHf`cY&z$S?CyBwd$fJZd@YA^|2%9h-NH$!4KXK-6qiu4gRuD9 z@w;*Bqx*xOaB?@ww4KH6PR8~Z-QuX}yuj#4VXXskY_jls@x=DP|IBz?Tf`O&KH&2w zizaY~o=eAPAv6x?qaOhxD#w6z{p_XfFgvuLBscDFFpy-)?YO-cfEl8rmcS=W4fVrQ z4{QT5aH8qlubxjRg{HJ*<4yF-h-t9I6D!8a57{c*gqWHiu*hX}YB zfB}ssKQc+VYi%j;Ln^(Dh2-ZfSyB3U8gZ|ER(th#QsVNo55 zvTf*u=3U>b5=V$LWSH%w^Y|klmAK*oaba+R%2Ju$Re;_r6_WiMwGnid;vyV`hDJei z+lCQ!VgzA~fY>QvUfaRq%6{Db`{CBl#9e0zQzs>GhuXAhb76a+Pu;s{H!uWSNo@bR zuryKU6s46&mayyue$C{FP|7O-NadUso!Gd9QrcB_;q;T{2~O31TAvEx2)Uns?y~L5 zh0h}(FdgRyTy5dl)YKJY{{u!VG{7lSNS{txlp8J=RR5(3W<7)zp$T=G5Q8FNJq>}U zt6HG}xOfo4=;&e@#frOeNBN(aAG$h2oPb)|>2QiJS<)sp2&yR^B+5Y)ZF2QnoT(+X zb4UxqALBYjeo{8$O6D}ee35LNfk_K=!FGUyd*k^1F2pgWjZ*|)I7#NXC=Q_+jfG2x z_OkFHVqq*-P3}KQd;2#C<*u++1WrR0&th7jS`zMd&<2Or3$mBVka9YtB;6;SPq9F~ zX>+VCs6o_YDa>Vuzw8<%sOndBf%OoOgkY9FhqFoeE?}OA+3}_%=6@;-`y_PS#c)9v z0ybMcU6XL3jINZDKh!iyy?RKUN65BN=eSx*HYSMjwRn+C;vLz+eva&$2jPoHHy^co zo8lbVp}}I}S7U{9c!S7|3uw{+OHBsePYxzJvZdqw@cYTEA4cML2-hK3{Ui&E5uQ(F zNfry7HplC$@RzZwC~Fe-%oA6w%F0AqI!k5NWM!hArsI4~(^%C1hOhCUOz>UGE6S2i zB8{@=#yhf+7P6#cl(9N1gm=oMPxzWD6fVY}QurX!DEz}jrEr$?4qsD%yaih4l1Oz| zW%+?grS!WvYdig};A@IdCNx>CM+D)%uG;Y|6~hadHO1YFP^W3$liKNg zt&?oRR%#8-aAfCT6hy=-{1v0jG#VA&+?|1Pv&53G6ND`MnhKQc`*e^u8^fbH{_SZ9 zSOexalvk7R3d`OOzPHWJlSdb^Y)F5$G$6lb%oF$>gWqj=HTE_5ZN=~7x7Un#2EPa5 zw2`f0rZzu(1gsHSxaejm)Mv1Z}<4xV%H6!CQ6S&HXeJS*_L1JA8^F2-{m zo-RCBiO;-FU#E+HY?o*?Lso3knp&`vATen<-_TzG7^F+L(LE4XeTD8g13|rJl zF*%%ltO4{O2>Dm60d#@%fG${uE@8FDU2xqRP@IjdL2%{VnguuCd6=w)K;BO=$7+J# zKx1JR$V*yvGdoNsBP{@8aKZR5Am$3=6QSyD>N5_;OJc2TC z+g8Dhgx<&Y8rs};68B%VI9n=dGs-s|!wnyFMk5Z=t7AuCA@FSl8kjQ3l^VtQ{xiHO5V(ZnMSQ+*{>`@? zw#7mSEVtF*U)7UsLqRgRyX5v_Aw)d^cMk5)ruzj5Y_FvWdJgLQ)lqyZ3a6&zP@Kn0 zH#%kkMlV@7u~O0t8M*WEzBqh6>j>sO3pby5Xv3a%0eCuKxFhS&Fa8+;R=gxz$L&0e_1p2)mkhKe;dcvu(;IU1 zW}Nx0xGBP7HjS=T(ajc8MtZ?;{=IlR9y<#<4!(?Aug?3Ki$c2e@>D8)JuiN`*NN3~ z6L#}-e~`|sD4Iua(9NKa*d~yK%&iIC?~q>sMpH&pcWZ0JUfgwKH#LwY)AT4aF~ z@_x8vRDlYTX17C>r~=*7l@3~f^M{)j;fmH|Y~4nc(mw+Q^fb3co&$6+pI83JIBi!-Txq)Czji(KjG`*D6=-r0-%j&H*=w z*l0pV??8~8V5grLEv2l+)~*$JBs z8iT9&uHxb_o!oE7bR^*P{!Pfx8;RD^udxbkhiWEcdZd0yrNl8dA+CZrw)I=Qct9)* zyo>9zU|Hk)xU>Ktkvh}aMrnmY>C524^HVl&)sMd+}W{Ss{MK!+cJop0i`l=^;&rIma??_RK( zSU43NISP(dVjn>qJNS8`@9-a(d?WC{JGi;^>$;qy*qBtN$9ebiao$S2&NLo*kD_zx zOE7k}9i3S*=fpOe%h}ji&L}Pl{~evzqr>UrCRRL*w`hZy&+Yw3mlh&Tou-43p=k8r z66QOZIHI&Raa8FRNd93=9nE(f^6JmnygEVLpi!mPK_g1HwBbOc5EL;d^KvLe>tQ__ z&El5#lt&zQ$_nBoEu`sv^irP+AWcV{x4IQsj5 z$=QuD8OK z{k@=m{{KLI7uZgMs}oNGX(XIJA+*>)0I6sPyayT2cKDx;l9D;jk}Y$DHW&xm!_ymF z!2#IC4neRqftTNSpH-8+4#Txz52-f$Z~s7;1~>)QQ*fY>asW{enCszlIJxpeCLcKy zvysfS1T($-eT4Y_HMiH4-^kXU*^f`WK(6BO8Y3An8|NEP5yB|q(8v$@;+e*&!b0t> z9awp~5aa`c=@u~VR%YYRn^D4r>lr>rRdT(T+^_<83)baBz1T3R!5TQ9b_n5s1B~HQVQ}iKFR}1~ zmst5KU<+DeyzS!lvnfIyTBA=T(c#BMyV^@5dZCZ6;t1(L8a@$BA2_DZf3|`smqm}9-2@x%dV%3tR;&EXXpYJQc*gFusqEB{p zAdFL!&cug49W&*pXjchI_k7}SxW5FLgb-~Jg{b>EV;r=IqouI8rDF}ma*Rft}033Y`dCyW~FeWSUTY5;#>+h78QCn>Y>3rspGhXaQ znGya6ft?RUl4zwE_qIAYMe?_ik)ap5qeja|pX3L1gzwIl&s({ zzk!A%jV|4%t@|foz)rz{y$!tJtZo4jkmLr9R!rbNT;n3D$S{xgs9U zMtn@1)^76uXwCi-Yqsw1rGY>R1F#NY(<1m*1QFkeYd}@>0dlA{eP&@U+^pRw7r`=o z3BKfB8k|5*eVPBr}{> zhx==mIBzjH6Sr|=n9MMJxavU|u?JwpmLsgrVWU@{j>Qb0>JS3wyAP>u)nM<$aR@|f zwwbLMz;y3k= z5*RnaJ9HE)rzu`fS|kiW5E_Dpd9Dr&amZ!(-HC`m*?Ej~J=D*zTcJ;K(04p&b3xzo zppC`~IGnU?HI_4e>m!=Oxlbcnpclr-<)Ujo4m5b4I)(-Q;h@9EO8zvb>85-+gy1uU z+=2+zP-vnZgq|la_JTX7=`1I|hdLLrcIa~S7KBTN9Cc!iCpn^bL;^T$hsuWP@KSCe z`hkU~&r%59Jp7>VQrrm^bz=Lp8G=KjA)%{BLGxbi{7tw)1C=tm&`)Wj5mA>uyF&G@ z#Sy)3uL~}~8-Xq_(;C=`2LAsm|3A*YJg&;(`~R7_z`g9qE+Sk&5l}HVaK#HQ8I_v* zQi@B8nwn;oT3%c-(-t04r_>Z&lKZ%!f>+H2w8fL)|F80O5l^-21uOp-1#7Xct+=IL{^S2o z#O+Z!fKnh(0qsvXy3f6Nn!^t1=K|*JeUb@qukxAk6WgtYml_Mysas>dTIo}~;M+ua z&pO0>{>*~oMOfj0(Rg0=4APv`JcS{+q;tM99AuhN4_)_B2_0P7L#$te@wM8#(!+ft5E4kgvs^SJAmr*U`t|Q5iw-?Eiw@ z{{+k+xI!J^XbjuGA@w+nx{ceYm+gk3TfHo@!UQKOAO|^MLwdRt&&J?i$`}U|<+i+Txm3YOAuFu~A8;xY) zmow;^9i}4**modY(}AMFa#u%bejh0>v0PeqZJ53zP>uwVTp;s!LvKJJ@uoRRy1gm& zrDY9rc3cgq?_0d!+ZLGFUq^klaxJ{nY-PTSH;{9hobs!ob@BBCzAnJ`5@6Sz>aK_< zMhFeQdPGDWL3rN}k<{g*0v%X^P9>3obXd2?nHJ_U21cG9>S8OLv?K!O?Ci z7E+8H^ahJ7y8 zgg}VTmH_4)&UouqVMCr^STI;5sc*%3X|FHx`XgL?&zu|w)W@S-+%;4CZ_(8ISP4}+ z!Lu498>;^^d}N%c+OIMmVFTJPzUS zjj;OgLjz?L;_&uVV0rJDxeyGpTU_>LK&_A8Uk@@;A}a$o_~F25O!d+#>;uGL*l8^zV>& z3*{vR5`>RZg!Ao)&|2psw?XCPY0`!%C1rWZ7ZJ`i!nxk^d4yLZTsB2-navw26G@3h z{0ak3nU~taD8$K#qw4{*IUzR;g{bgd#FYosHpET|)ZI5iQmeKg!vd-Rg{TEpu&yLb z%fV5hst#`9_9Q~zT;ds6G~K&hmcH(GjA-woY>rYr|=CIU_#RyG_x z*aLf_wU)WfR39_6p(VMD4nk^jK<`WN{Q$0@h}jf3L(2l?696y_0B&B^Dxxmeda$MZ zE7JNSO%3O^kZ&T~hw|!OP8RsDAns|z<)hIRm+kDKNE8Mk5|5u$(Jd);<2Men+Ye+1 zguNLU4tR5cWNR&V1u~;^-Q{lhF39zhFJPCF3a<1ly1Xp_Yr~f)w?nuk!gUFe?BTGe zDEoj!0;So{+f*VUR!O#qb-RiPrzbL6DI?OzsG$6NnCd<)sk>MjI+f5)Bd&>5Ro28- z)d~3nDgOyAAF*QE-_Z=9;ZPBH8X9h}+Lp*`$1z7>`Iof-njsR;80uZ>i7t->@kA=` zBfc6yh*k(-G{W}~{?QDVGx!?*05n?gl!;Ehz}#1R!{#pELjJ4BNA$!nwfzzPQ^NQg z-SrnDNLyKkc}vCtbemnJjVLeGZ}){KZP3L@%1g-ivkvIS5~+%|Oa_dfk#cn=~S;Kt;_d{C91b z)MskN)?GtJfV_kZA$22JM=BY2{((d< z#A!?|K$-S~y+))XqU>?N#{8CQks1WRxEHF;jN*7%9TFU_nsvTW20no)CclP-oQ8-t zCm`TTOR7a*WQ;*g(;xhUl!HCWE4V0Ao-qr7Xa&I zRu31|tMd>R!J>=E*_KMt*jmp;UMMXSdY~*JNoCgj z+CdUvP=itcgW#6gag^Bv&LZff`Z-icQzjXS3}32%5FtiGzCI{UEo2Il8vzjRi$hr$ zsB}fS7m*f^a@3n-bgOr7-y7&5LWmF~n~6Y(=VbGY*`(T2*e?V_Ty8?Q<^pk-bY&QN zA&JH+pW?AdkT_Y_9cUaXQ!l-#@i}1?;mD8fk4F5jsN4c1RK!6W^&+KZ=MiUwy%8oK zQ8geuwIN-7?eabdpfrZ}^hjfTNLct7#G_>lky|4lOuQ3ljQ>RZQRrtW&4MlTQRt6l zz|eupbOk6G;oo$a#2uf)ETiErgZ?k0su`AJ_7&- zLR6R}AhLp^|BFENbS0mzcI`*`GIVbveCwm!myb1O63RVFXQ9g3v!F9O*7XliU^#ww z|9HkhfQ<@i%xC;759yS)(dZE{nhn$=;02^;9`rCiP>w)6^%A*-$VI$PNAk5G zX&JHYTp-s8Us}-Y1ayNAoOU$la3)VeVjXOYZr;EP7C?fQPY1B3@Dm8rqC(}D8s*U& zc2wRQ;kWf3D8pICEW|ZH+&silrA9;1ID6Ht%}{#-Q}SR!EhcY!!~^KE;)h17fY>Gg zp^9j4N0rjJlUVjD;wVfrCs_Y)^a|0CCt!FWo+Ka2Hvr*U+QEHtBAyY*@Pb~LjxtEJ zN`t*KoiYy4TSD|e0EXae1`xM8$YlXJ;_XyP%@iO%Nb?FvS3k)`4>czFVSZmZ3Hend z?HD`4l2*5ldPQ}yKc`np<5g!!Fz*)LP7M^;R9t?c?~7*`*o}Q`|Dl1oBjE z#DRCHaZx~sdV*xa0HjlX^@;$9JxotiH6kScQKHf1zhm!C+pv1bMAUl?g>Qqg7-RPu zcs>>U4e=x$n~{&g<$5LP4Y?^Ak*9~(8}XnY@_-bAZl^rw@FgO=y$X_#^>WXXYT6p4k(@!BkPi3}0v)xlU4df)01i-=BU2kavA-JEP{~GI3q7v&eC;~L zD;IFc1RF2x^9nQddmb6M=kXTuX6iTUGO#wxMVM~X(Q(=w{BNkksXLTq^X>KITpGV4 zKJps%>n*RQFKlgA(iezLL1FSCu>uY%%DQ^V2NC}#fYEZ@2fyC<^}_F2{EY4=Zbz-8 zORMUvcmXY(Es;ERrs=nHp=qYMb)QK5Zq4nqS6!pZ$1fo zj;A#TCLuWQv%^DW84f01?;Elz=ntD?A51FyIt%`jvy(;YMZ_SbIpQxNo-QJ)tpMT0AT? zt^<)kLad#p99{mnGYFTZpN0fg1F!YLI5Z^e&H~FKjO!V+>?p>MuE`Yh5qJFnNIjpu zQ+s8|6LGXM_UBR?y2aH$Mv`3sgY;u`-6kJ!Dgfse-30&riL>bBW4QkF3|vbFtU^6>%gvr7 z)EYx*1zb`-{Y>eI&jgGz`>@oi9}v;ngbQuEx;{L$&ZkrB^T>o#>zfx-)e9r^rHe(& zqqQC@LQBco@ibbR52UEaa?x|20iN1pRG#Pr!*mF7WI2k*rv(tJqPivYh6 zm+1(JHAuJPvTQMkGbkj0Of^I0e7s3PXZ2VC`!9l{*SqM>!l9|cK8Howm!JeSg&@W0 zPV6g28|Uhw%KSQ*K@d=zItOue&(~*B_#3?p&e>;B7)sNg7!!4WrYlX0fxvIryJbO{ zN&*Q4#(tHg!c_NXx+c}>RcobS#2Zh=l9j@C04L4GMtp0I;PrIuLT~o=aA>ZbnQRM$ z?|(Ld9TT6sv8LWn5RbvMgknRE6%X846Ym&`!h`J;bzihJv&gVeMA42R6BWSPh=^iu z8Is?;ASRgEC_d(bIB8}HK6Go1I96L!{@oejW?`*-mYh~!pK3|YK|{Y2JuEEA{-+){ z7tF|JL(n#A`+cQHVcRr~DoK|DVdAH_A8BPD)YU zx9@R1G738OKTZn|cNS@{Q|DKDveXz>vXQo#78aNEI9^BeJY_M+)8gOr?9=lUM?;kl z>5Oo1uEp&bP}=c~74gMjm|GAROr`0MF)34 zxNN<&H!DVJd;1kNpWN`Y;2vxOPdY6od$8a(PasbPn5P9#M$uBBlG&-$_Tc1gb<(Pg zKgHy@2Wu@~ye=)Hbg{>SMLpTlSFSKgRrLYj0xB&7#-x1<(gEhfkmr8_w#5Ku^<+^E zbzn8$g1w`LfM=ms0&pUo(KE$xPnMMSh9~IRcpZE_S6>HjHVZcPbbeUTm{c^`qiyXp z4NVr8FU;q1i3RF?kWCRY8V?hdsLt2gSU!QvQ?&I!b)~}U#Twa9;h6~-1Gq10Ho#j(2g%gB1{l0^-M5qu(h|O+Tb6U7A?e$M;^MgOSXn} z@~0v#N{8)Bje@}v#KMXFjfA?h)C`1Lp;v0aTJ-TKjgK)(n*Zx+@L+LlrYh@6C`u5P2s;0)ITs-njmmaneH=&mep7TBj6rE>3@ zi7}PNWANSUncA293zo%IHEl}~e;xUYs~*iyZg|IPV_qPB^=8)4H(rD2G6*A&5&jJS z{WO{^5aB);qDhqcnp6c2Tv)c=B)7u&pF^BV16d(*1B|t%){T^F0HqvI%0LISt5)Ns zNLh^=ZBo;lV904TMo$z-#gXzDl(#}7(;sm}1`_PSNEf|}delV{hkRK5q|I938eKdr zn{QGHao9|0St<&d;4POJ=uMd3s6%4q*zIR6UFsoxd|8TD7#h0^CS%lTVz4i3)W`=h zwBnq{vo&=~yo_3~;K<1;De>#4IB$Yn=Sm}OhebIdC>mt#Qg@w&O zRTENeKji3Qg%LgYOIZwD@MkSrnRiNz@?+_I%_*_Xk45l>r^HD=mJsWEim(b+wyNKt zN1lPD=Wyvo-&l7MQIBFxy-Z4F{UVs8M8 zj6Dr-nGh_-r%AGnNktLpcj`~7emdqJ^~Z`*Q60egx^6_?4AHrgHxw&^n6H={$RZQs zz+oPq<1~>R)FDxbiz>srX5ctAHE?C@DOJroC5{KO#Szm1L_$TObt<}vxj+@QFBQ{* zSQ>w(RO}97Ls+Ts3ub}AUMP}$9{*A;`mkcm*_HM42=QWdXLTrQgs92Pi+_tG=@?-mT9z zw?W-O>Oi_89&$#JhFp-EkB;ald~3essPR=FWvPs^28fAatli)OX-j)K22C5C>n}IL zcR_BTybfB661=Ff1z6C;dZil6jkyjcP@N{}n7o1SgV~214=@DQMrn?o(DhnAY9Q{1 zvF1GH8FoOyMUX)&J!ZAK4MbUxoPgH+YHx<^re^BU?&H z`Zi=AHRV(nNJ_%D#>7rNm~C=AI06si5=m{{PDg(3ek~3YByCi?w{hAOpbx$Mbog@}+YEiJj z)&PA)4#I-eyU)Fk!LJe>Q^dJv0r87&kGmD(xq@$w(TrR&!CkIXkWb6h@sxUDvuMVSU)_k}`gaR){!rj;!tn{s9M@%QjCBSgRbk<;$ zHM}Aj!N`$8{KOX<3NK9d=zS(KHja9lR$7S*Fz7%s(n8FWyDH6J{6kh$M zn9zmg^gwNn7+`#ZXi>_JAL*}Q4s6u$b3q#BkF+F3Z%G%uB{8C_ja_lcj>=CGty9@T zAMa4vD_ru&t=NLC&(;^i(pZ$JNM+sm*9}FBG(7dZ+^aUjEpN&&fxChf)7V%aS2$U~ z)8x{Eyk5+`M0D)R%AU_o%ZOFmGT{pdqUb&?%oNrL!uIEn#?1}Q>U#!p;CIl!#>!Js z#+aENeS-SY1;nN}$WiCU)JEd9ZtSQ_!Mk%~Mc3}^naJ!kYZJMAXjb3J2HfIU*^2y8 z_>KSExve|v%$j6dMuuyLR<)cwJ1RFESk`j#%Fl%VlWYzzKjK{aBujF$bMU9MEUQdx zcf=8R4x@0H79M7VZBhEL1t zJs!Nj3Kz*0FZAxa4PE8tG(6pdox^&8Z$F%yxqX^#_qjb`9PVoE>u zm+*^3&`m<2)LWFAQO%QBqKNL_) zb>Vr%IVp1^d_JOlZrvTUqFo8fnJ_n+zgL{|{@iep^gZ(z8~QUVpII!v=#NEcY_aek zz*_MkT82>rn19%M=<8{P&!@!&U86qzZce-VWU+XE02`H%2>1`<@g#ggF;q4dEx&~B zF`kNVR4l>o8Hi$5sb zhO;Qo0_s1i{fh&l)o>Qg2NydB4riS4?+%EeBUl(u{z}Xkf$7f=i0BL!8~fQIn24Z8 z+Aq_a;^pgIKudwjLBx~0v9cpvt~wzmXRzQ7MTkpA-kuOw%CYUHd=(v~s_lrU!_~0E zI0eM5*Q1ZY+H^=9M0uWfNSw`J5n&%8e*?fFv4Fu` zi8n{F5VvQM?aD#1aumDGBM&*3j%Gs{-+5469Rr!7|Vw8=!4?hv3PO*!-HbyIM&ayHT8i7PQ0X? zkpp79)s=W;Q#o}==0D?ea63Zu&t&1Qr+`SDxHg6bh((#~8P_~D9Q7g#6>#%^k^g=` zoE^`4^R20($pjYf4IW!*0T?!9{6C5cX8Hsc&YwCUmQG-!T%V{d?)nOAA#4-b6a3zO zF>NB7%m#|`iEI^j?iX`jV$myJVQnbAu^93S^K@N~#>9zNC$gBJms~UpOj4ec5o8|J zRCbXEc%9-Qu2gv9l^DE3hOhcK)p+QJGNB`??w#hSIU>olueijd0!XTRtvga5;HKf; z3@$woUej5oYYo=bYVp@m79!ehVD(+`-dmiDNZH3?>{t`kxd8f11~%hS{h=bf=L;^EZkbu3aH|%jhl-32em~C_k5iY^W_A#;~?(#?|v#=H>;Dn_O z9cmc)vH*|rG3cHsUm0#pR6y+RiHfl>KyGUVOAN^_mmh+B(p73d0|ZPhiAOJ!X@>iQ z5vQ?#EINdjOX+N%!pESOfoi3#Jz9!?96&kE57l~8Yvm;=6+)C{;Lc$gO>Q7x!w_lN7w_nGii)bQIo`(beSf1CBN0melh#6Da zEdJv@G4>TmaJPe2Jf#^2B*DqIC3)`{uUnYC5PokzPu+Ed$wZSK@|%RQ&og|J>qu?m)#X% zZ(>pZ_O9srCi^HPJFPfQE|1ZOwRDfR45sZ9d=5(s9&pQOW}4&nAaLOkNAh!u#Lzjc z$H2KonzB922H+!xLcsESD>NLD5bXCt3}jBwc)yyHW}q;vJ@7H+jCei-ljnSbul*w~h3QG_p=dmUp9(tOkNc=LF z&0`M)n~&)KP;~(IQz1nM!>uP9%+6?OMS9wgLXMNZZEv>ztmaM0> zdyBoynrdmDdYXHoIQJHd2&PUYm6GAwfYeTncL|?`Y?^=Z&KePg3xRa2od$6KPO)?$ z>ow}DU8;N!@@o&S7NU%V+>-*hX3VZKtT)pB6pcJ{+o|OFG}1^{LG@5GgG*|JdRB-* zZVOlhyS4faQfA^CS|ob)F-3@sx0y>AAxM0M?(x$7yZV20n@JMkv7KVh+iZgWSRlzl z8a_mF)J}2lZA_&;J4MTqD)q)K(1=y3zYz}IET&V2<6$yh9rX5^gSm07W z822#aaNIEBB6*|>R@wrLgky9R%6B@jpKvrsYAo`-R_lzK{PzYaCy)nk zk3P6PsGDOZZj-(~fjRDtOLM$7nLBRw?(VodqN}5NN~)v6+~XhO0A#sJYPX-M0jQ7u z|42xgT4@rqp*#Bykwj6{6WaZ#K&6kGF+f>?@1>Z%lf=(I^H5RtKFj2T3!Uj7fJ^YZ z&pH>&nALnhl{mhXopT9uk(P=49M*~7^ANx1uz~D%(Rmqb%@&HO%h;#<{7#X$9J?Ib zO|ft}OYups1fLs0`?_#tae?@DIe5(GouZn)(+fnx3f9+uL;+NC9SuI%Kp9pb-e19H z_?MF=syIQTHe8zlUkes2Avh8FwKH3zOt=zXAO;HXKKGmA6T#AaO8!Qkro>0X6;>c_ z3l{Bvcl)FD+}th_K4iOj%6)O+L+tG~ZWm)eVzI%ikdNTkj5fC~>*}hCZtqozbsw=* z&ur;RYxO2-n?iB*BbJc30YgT6hefK?0}JVT@2sp~(CQp(Egxf2WqiHPvbEl#l@Cny z6Cw&lkCkj_hj}Qop02N;(UGj-yRqYbzR$?_8v_te-#bRBb1)M@5f6N+)WviH{A1+# z5`*$F>)^Tvd1S4QH$H}rX-k2)hq8Qafk<4#B9a8s_h1iLjJ8WhTgzm_hW~KF*jY}{ z{==tD8<{M`jLz+65%bnSVwoU{*07F|823_r+)L@YRC{ofB?ibVt%dJe){uAKB@)-N zmAqYnIKGzk3ckBlCyGd%qvz8t^Kj+HR*{&?hQ?Ze6CD82mVnM}Gtj4xS8{Tv*p|y; zpSXp=Kk!H&Sd>b{?a|9;K?K(DwkSuirw}fA?3u8$$U(M@`G8zer}s*?iwSwGp>JuL zwYgkgV^+KLV1ZbX$0AY=pb$L)cgx%poDDC=PYJ?2!Tl4Ctx6Yzwu6+n(X>T}akK|d z3X2X)U|Dc=dre%=V~I(v|DxF*tV~B9Dr*4}#E$6y=YTUq<*Gr-OKLi~y$@9QPBCyD zOX>a=>OYERRXm2Y9r1?|4~|}%W=nX8_cU}P;Y(I$>UIX?wntoB#{&42?c(Y>(ot*| zZtGdPH38K%0xIgJ7#f7xZrbjg!d0e*3Bqp$3$=$MR~6{Ii!WX&0sWU9ag$%&r|rkp zlAytEc!N4B zKe85>mC>5Qkdl3N{Zd+8?sNTN6(wB#-fd2nt*#|vsYC|wgB zgB9nN8bP-SNlSA3`&7r{#uaC2k40WW5LiKikhBlLO zgxoalMRmELB04}Chz=!c6R3_Tj0Rx0oj#>8jH!qF4{1#(ZsS~o|gxtmxM zUugR1P!IkUAjCA=EY5DiG2J;uRBr;q58N!0@>yDs9GrL)-DzS#h;kNCNes~Bh9Koz zgf+P#NcqO7F-DT_f$pJuT1J>I-7F5~17@eqIp-Ed@HU%8u!H@{tCXCCx1)KblGE$$ zMA2Vi-T0?l#7YI@P_QNE>f6EMXN86O>{qmY;eejEMexn6mE|VlaxkviPPflymXY`# zqG-vrVAwU9hA&{mS)e5$MQK3Gd8MO=ZA|6*;-8M2BEz7SP%O>l&rJ z3zIiuN!!7K(_Mjv8)X;{6@0bCYbE z-UNiU3hsmu!gmnH{-a1Ep8(~YLtNj1M*YJ@#7?L*slqr*D%fUMM>&oiZ{sAmdPJ%N z=bG~0uz4q##2QI4V@wWuaC3FOn7orU4nU8!mdm?HaF9TiTa+iaicfa3k_H87xy@yD za{yvgbxY8!-H4Ie=WP@7cCoKJsu?K3DER~EDIAner{<)rQ2Px))N}&~0F2ls78kO% z%~b$(6Q2bGibKI#g%Ct-P$RWPWNd{>=I7na>XY#RYo4xrf8l}faAMZY%g?EPD_Hb& zGB1yGq?6Wmgd&DJK~ByGVvdsq_ZXkAk<)g3dtvOVeIxGP?9EhLx36|%7Xyv2p{YH( zTT_8h+~-K6l_(G8Y>GJNWQpz#Q3ui3eNoTJB6%8BpTI2-#PB^V*srBt(`)z!qb9N7 zJCtgpvAo2bG7<(o~;i0@f2#)G51;mkk&{@Ldgl3(A zEi5&7g^p@ng^~P;xVndhbzb~Q9p#X@s|uLx7qAb3{Q{Fmfv#zO#-QuaZD%R%D5W*; zD5r^$ds%=P!E8&3j~KC+g{Htx;NJ_W05|?$(<&{?JIm_`U~o`@<-PyTC;A*`)|B62 z*;Roa&jf3^)i2dSeRk93u9NzWXr21)@93%wOm{Lly_P}z?rAA_JNHi|Efv&K=` z#6oa`5JDVk!#!Xc2WpD6MFtUA+5W1v^rTf?y<}29r zynC#(bHu0Y3UkSBe}akL2Uz*T_~e7^9OK8C2qz#bY7S1eH#koQF%;V$OYj6umxUt<1R3RhiD(iMKyvQv)2d!yFkWS2Xc(p!+a{ zErsXjEZ(CAv_!JVsZAo~bM`(zc~_kL9IV@Yy}0)|>)~=|T}_C{IKix9%ops5unX&I z1WjU9V z;faI1jCaoyD|WL4udhK|hM|sVc7pZyUO~%YjAlrDOT2f2#qiyEV&@4K?bp4A+6E~u z4|l52K4IzNY7HbBAlsf2(otk3%io?sfBt%uz09|)7Xy!hseavAEIr1OVlgbG231=C zLt7LdrRpE^RI2tW%&ifx4V4_zvHFXQV=OS?FyO1JB-Sv3G6}sw<9Z%Vpey5=Bp0ZZ z+JS4{gI9R8oR z&fUM0A>59&;=&&+%{_N5E^IWw`>Ep8?<~aGt%BvUh$j$Pabf1^+{4sE@KsVT`v7qEL@|F8|uA%m4ppie56epTPDq5#2S(KH>SqUkHx{i zG5xlDEYAMT#t`(CjS#Y3LuP{RW1}bClB{mT*3M*l~ zPzdmqY#~qoSPZ|0>biX_W?X}&Eb(J;@)}z_s>Q!ln};~kyQ=MctlDizqbaJQqU&9k zBkp0pJeGeQ2IM-62wd{9)-Ugib52f)_4tvpKwJO) z$KsP4EZIA&gDQuTN-AxQsQ!wD*@N}fo|as4OUAIdoIc#eY4jZ}wxUahhWftML57#U{!fzE1 zI&m9Ur9xQf2~q)IV_-iO8Y7*;+Y0h|H)+RgxT4!XyZv37KE)nz>jGfs@mAYmcm|_& z)51cRxX&GN_Z{qhN5FG%gSnA+MJ_Jn-^ixUr+>%(J-r2@`Lj;vpyS z{PyL$trxAtn$h3Tf@iu*MaHnqpkaZEuIeT#`IClkujV*gmt?q+^$ZPG9uw%nk6hz*aXvc#xzJF05p5eTch3UJ9+(R^AsKE)R&`C|S8HVF|X zznnK-4D~ocshC^GR%?fL{;SnPJ6^%XEF8?yccmTnI)_(j2X<#yX$N+rRuNR(Ct9r? z*p;lJ1H0bh+FsVcX}QVFFb{pVey9kz#g3VeL#j(JF*`5bVy`kjcBL442P@N{mChA+ zShWie`N+BAANDrm4zn||n(a3EoWgor&$d({8N=ZU|3ZAQSAq1K)I0B@bc*dwbw0^? zOU`EqG0TNJJ)Y|zZ${RM4mW2G zjvwESNWp?qW>3DuGVshYwHG1Fbwi-3FN`{lSbZRcbUrWHo}q{@&`a+6Rzy26G!2#MmU$; zppeouT-mS1sW=jT8;m?E94*gUBMxpjbl7%f+arkpbQ?62FB8Jevh1*Fc$UgtI)<2F@|U8Se5(OuW9cw217q+)i>904!c6l7jeR z|3*2W`4Riz(lj!GXJ_@b<~Y9%;@eET%~CP1KHuw^ot8y1Xn@ix$Jrx{*Jp@PZ;b>f zmzIhN4ftexE^T#eo#bx;3w9mZ@$igAqd>@Cz$R@B;xRu^H^p0&GfQzofif1>=@FAj zh*2<^q&&4IlPi!{En7^Yk44#Ogfq18wxu<8ErX;*D9EsmbLGQwqA zc&-u7Ae%Rf@}?2C;i3rodZrOBrt|>iRU@2B=@wUdDP0sF#e3}OpNByjF){N`o{3NU0@mRDX#-& zcu)tb1dC#2UCx$4IW!l%EXtNb*JHC6RJK8PN%H#9l4?bc-XOP(+5B8i=cn z_$|+ZsDC`=G)%|5!{TaV9*+pCpBxQU^s>WZO#+V=-D3HOrVq1CJ&emvky0`l)nef* z%mc(IKQEc=!07Jq_`~94EPp3Jjihcd64DNf2~Buo&(f%Z&T31N4vUpdc#OAKZQ+E& z;&>B2rMZrQ7wVD%q~!5fxT(G~9=7BRPpJ_7QyPV9BOJ;K(_t|-j$btckIJY;v0_*} zZy0d=b7-mD;a?Ah)P#3q#gcd)>#hUdRwDMr^B8v>#`+TRdpu8$(|aNZW)A4`WRfQq zovSILsv%!cQ?sKMpmRjaWK#fNSR#fd@QwzMS4*h7`gyPRlS$8U3sd5S5>cJNTec{T z%GGA>JS02|=+^t@WIES?wnKxO@2Ad@3rO!?B8E5RVf@WwVpdZgWB&{lBAWWP6^csV znUhBXp<6hMO)bay;(SSd=@8=YnAI`KEvq4rm7sx#M`4A&}dnv+F%GhVOZQ7?HtINE5*OWuI*0&6&K6-sg*rd0b6 zoaxQDnFUeL07+89K9~(dMkp+g6pP`{@KIq{$FtL;kd6cBMYQCm2Fk47;*}eU#kptr zP-mw^o?+q{?~64pd3x}(@584a)sjIsBZz@VDm~wKR=4CQO*GEgQTx5c>9)MR7AW%; z0ZDu)2hmSS;_zQ9jR zX~%=Y?jL~bM09Ogh)QCZSsIDmIw01x<1Gy0`2z-}?Re*=rST&pWDERnN8z=CchKeW zd4X~?{!FNL#mqxu?o98va$*MFbkI~`SK%8@lk^*b1o_s*;gWzU##lQn1tRo-c zt9KNcCTF-v$>NErJW#Ce$Xm8NT1&mLl={0l`A|BWDuLNkS7V<*b>KdZHbM;#i0Y1f zzA>j@ce}U~@5fj8iSwO!bN=Igr%z`-o7F>wns{r01$kEA8T-Y`6n==8hl*icc!!{J zZ)ojV0KM|-5TIubHK zr137z>p-1DMXqR5`QHX|#;0Os8js{)XrHF>q1~6qKGITpgRn0^U{8>6(-fQ_{h@)% zz*u79Ra%0){wsfvPsPx#V21h-bo^8-?8;l$MXD(vW?ZO6N~MZA)Ex7v_`NG{5?F_p zgQ*?o&B}Me&$20s$YVQWPF}8q){&}=YEw^T{rmQb#oc*Bf2`qHnt@9J3gm)s z|0e3y=OLoBI}ZrJVnsYZ;30vo-6w8$=g<18Y=;tvuovwUeS7f6E<}uCb`M@Z; zA?GJRS%IDL(S@uYPD@Aiu5 zr}#L!j~p zm9rN*SN7)DS*I}z^)B*ww2OAZs<+r(O0uFTX~w7B~RzsoU$5oX&%Nfk9*``C!CSTrfJzo0ei+AA}6qRXQ~i=ESzQ zL}Gs)?^0T%G0Y(Ga(^Dm@9h?g`}4UjrDqZCI{)j${0ACgMZ9DlKmT)+tr8Jpj z3|v|%y|d6y+@O5Rc8lnNyqfpNcMz7?&b!5&LA&!V>JlfE&K=9N>$#d zWH3L+pV;FJ8^T}UBjJ-cR2K6w7tmZXdB%PPd%IB8*3=F0o1xRHK;NrUealEWe-myj zj@H9@E%25Q(P0?kJ3I2BZyR4zQxK2)g8B0!?qWBWEq)lr8}SYE=#u2(2EP(Vca_>? z4)1?@La#ojC#F1edOR)DC4pl7^ZY&62jE1KIM9(rdg7Irgb7H)RZ2191-_>#c8Z>I zDrEDH^J=f^(Y`;AR;y(J>b#~Ko$nTkwBbC7@7N_~4d>Tz>kIP(n@}D98YZ!0Q{}h z<|J~=M)Sj}2^%C|i!PZwh?`*?mdQVLe^--O&h8Ll%j)`Wu6qT&S1R(5*2op+R~;1 zM=?-3M=CgQ%wASoYwpXa_3C!9?`8fJdQM;Be%$UvQ{7kx71}bbJC?z0$OiXAu%Qi% z440;%`&4;~L;HHx3TT`Zi&d$dsF^z9=372eC9gaX*jB$3Ayb-B1zqapBcH_!tXZQZdW z=!UCJOitL2BjOlad&(P#%Q;!b@}lp|q5)6BZEhdH1DhxuNvvJ(i3VC)5+M)OW5Uhe zJ;aHryq<`g&VOxMw|QsDiVLdG3=b*Uo#DwoY#PZRItNX71x-kP723nY+r)%d`Pqmi zh@6NeA`KR^;18wPAA_c$46Cn%8)=`(5~sM)m&NM8fO5t9Pys zvci%PjG}t0nDsiJVlcz9t>XIYJdJ;|RV2;i9n%77O$=6OO*{a5D_H9#Qy-A*fHqb} zg=+(_8);c?+TPP3lj%rb2CR3@1kG*ODlW{_Mw2!N4+WPMTSeF`o{G&*nKt9*Zxv%_ z@mCV`&Km>3wtvjYl#BR%INN#f4ZhEn$Io)kn#V72{?hAW_**WS}Zcz+Rwu(l$IUqG`+{i^MEYyma+rmuaEL!N}n0x7a7`< zr)G$I_@<{l%u54Cc!a7F_eFd=8fh5>KU|MzN?XK(sBot$RwfxP3cqHris4HEH-3hg zg>UOyIovZ@1#V5e|$8j>aVecx9^wpH z&Ra1)vbpo=6}-7C-#t~#TE&O+UQ@-nRgeG{OciOX`P(EYpIFV`r|+0>^V zJ)v@tvJ?nZP>AWqHFQn1Cj=x51f;SCQkB&OhB)_4jzuIW^;yfU(QR>dUmXL}GOz%$ zBOu$8Xs%rES{^^+@swWn$(O~NPq3=K zypF%&5v4~)>XClyAr9~q(PTZJ>T!Dt;R0ps6ydW2CHAl9QIvR|5|8SMU+9U`H}KWw zY*g&GGD?JOa_;{f(Zob5p3NIh|cp6I7=D?jpz zXr=J>5YN$$DGIVR(z8WR)*9)unI~(>T{a`Ra*|fX)k#|NcboZhL=#Q6@KN+#vW36N zya$Ll4>P12C7+=}{7OcVaw_$dD_YGRPmSEb)AfX9o~+j*A9nU^$NPQRq#Vk_VwE|n9tJPwaldh$nV@(?X$;tu|< zOJzFp+~2{Md$iN@B)y~|Rk)LH@Nm;3UG&H|cJXB%KTOmrJ3CRUtW6>R(8HleZqg$w z3$fS8jupL&cpMzIWV(ylMLgDHX)KLfLD2+vvAKv(?zc1FCPt2_#D+8E*2-PC%Cb*OlJuFan`BviCxZ7)Wd8Cq z8yBw}pZCtn$ggeJvCdUmy#3`Yws>W0wrC~oPgPp;ZXnJd!rt<^W}@a0Z^iQxMC)SS zGV(5L4{rAE>8Qs2vzo423$4~PhuYtTlK7y6H*{A6ze@52agER@F6L&)NBNRCQOvFQ z_p&7ZF6P~WOVd*$)Yoiu2nA7#@DxLBA-B>f9EGl{jX+tOT1T`?B}8UwWdrwA+TNG> z$Xv@;S;7Ncup_Vv^I;wrhi6cm$d@)V2hFees2I7TwX~PyDKaX)gfU3Kh^I4)7<3rT zoAN(3uRkNS>x<&UJTe4h39S=WN|J=>0%jvjsXObPw-58PyhrzO5Z&Q*jbsrrd;1mP zU^u|0b&$x9z8Q$4^cpxMqZM8PY2Lz(AG+Z~2dC0Fr~eoHFUAUXAH{x#!IR!G9v<5t z#mH+MfgsQq@#~kV-62!YOaZ21thjKDH%1Z5ao(_Ar3v!@(x=o!HFFz2P9z=Y5fLp= zY7zFfWZjktY9@$jV>Pz9XsmPEasH!=xgag8nQR*C?E4KL?!qsR5j#)w=3WISKdLWJ zLC637Q9W__H2;|UjTei*xffNG0?hLSenv_VDCCDj#}5IMtC4Hy-s8}GEz5muM-!7IO>Mn zQMflb#urxwAW1#BiC1;C78u1lC|aqAIh zU^OF8!)k3EOs67_kXl(M?j^(#Qp@YajYAwERXMUoe1Lem-}Yd(xE+D(_BVlvrYZCM z4^QxT&PB#7SJhQr{=<)X3`G&@q`@PF|1Z3IGfI)ktc+ZGjGrg9SGVaXRf3mPUj@;T zDVWM9ph%nl3!mz{bS!RuB0&WU?U(61Z-)x;ugiJfIMAe5z|?Pm@+7`=u_jzBJI|Y$ zqrk}EwM~3^o)6?wiij-ZZ<=4$&whZSp(sMJ;{pcP`g$>8NP^Y$^+#>3J3C zc}YcgVRcR6RO%))J_kKwRs#^b{`i9@BzGG-qbDskT5l*MlAX8SFmfYXe8!e!3aYu?7RZ_rH#bjSFj7LY$QDY!p^>|k(_~!ppD;}E9ji- zHHCEhR_zO^vFQ32@8zl2y#VR-f}b_L8y#(zwZMeaudia;+HV9r@l$-(qw(}a&@)H_ z`>!O8c^YYKZBM8g^9jRYune8@PoqTIRUX~?TTEQ_rV!R4+RjxEI*98V%<(F?wH52h zp$n<)FFcPPfcF6`%a!q;Ga;_>hWz{I#qO&-!s{C(+Mthn3YCbmtGwCJ4__c{986~* zsjb9+T5XK*3WTi>okIjFUF@3Mu>Ocw&G#|)-SM9Wzc;?v464?d5|&ES3u5fwd{pcg zs4){pv*Xhw)qyEVYv4x(v|_aZCd97ad0t%o8=Kvg&x;O~{J2LL!GytL=nKN<8owTU z8)3o&m{&?!t>o)^7+%$|lO^YK7A5Bc5qlluIwxE7y3PkCno#duEL7$F$YLu%fq~H8 zKZijS3$g9hZY|>i8k45dVfP?2?i?aI+`|-nbEuenkB4|pWwO$wi$kT+?0Y;sSnZJ92=(&KEdV2_GgHgl zwoc~wl7@1=ucSU(b18;999^YTm!KA@s4+QI5RwL>Y^dmVv8fR6I0Li;%#U_dTwpI$SfB0Z+{4)RH z?fSfmF?%rk0F>g|^n)`|*ABvRrj!dvseK}Z+t>zKdLxX`74w=g=Qrp5v z+ukk!I~_00qN)Uvio}L$ZuM`CSgIToC4nYN>mlN7HE+SA>I-uXPYJQsmtl)W5+rOm znaoBqc?AHAN}3{Se@(C9wm_B7NsZ(a;C)2lYRb_X-jiF0EcZ7h@UI6iPcbF(lEKR- zn}+kM!OKsY()fpig+DW;d%Q)c0*U!Rota`2X6oVB@THsSs?{F&Hp!_Om^)L|^dQAK zSnOt|$v#~WCYmgP75lq`MJsNaz&{=&)^JlR9y3II%T2>O`Ke_dwF>kc{NMA(P0#0| z*O87d&c<9qMV}m(2(iv&8tXNL(!kONQ5sIrf<=OhsRLg-NQ`wcP3FA^iL)-I6pzt^ zz`XIsM82=6wzFW5zXwt?gO&FNi4<4#Nv}a-f~#qF#7=;rPRryrIUp6}g(l`xQ$aq@ zBCne%mb(lQW8F*xTq-RG8jC}2rssI*AmMK|E#yl^h?Qnj!+=t0PaLf`(7BuB5>H$T z7%YyPO}%*VVgctqNf9=x-s^5MIS^=z{}(`|xljYs)#I%T#bk>q0%iqtl}@|OJA9y6 zX)$$78rfJ*2iWxTSa~zD$K$A8cc$8z0oZXrhq>9u@z03fIJD~JsG0AguBHUR!ZZh~N5#}2 zQ$XMI)$Wd?=~7igjLUm(QJ>TPEI>I0P*ep?TXidhyVlQY;FTPF4fbhSRJ%gaQSyR! z#3&C_GNc*jDi2cxP^7WMi{+lkB>-lojZ;?$phP|BvNL^)-^~8N9&G)+URbX+H%Bo(@UnQ-(ZDGzK#FFngdnxyFR8H zWOCYBrXFbVj#%t#>Nw=X2C5B@1*&XGwTWfzqd*3_O>Dvcq3cWFqO8{cXWorrL_~H0 zVL(JcL_|bH#9;{$3CV~^&2SBu5S7e~1eFZU49tAYOv$_klNoifvGN+4x5yRK%!Le< z3=I{Plx3Lr|2^*vM%~|^kK^}wp0l6xoc+A#6o;`h$m+7y7&fqRGrq{inOOCxUT?^R zu~(gM^T@JY(12n*C(Gh^n^}k=(wHKJUv9)L}8ia#$%4nSLIP7EH1Hoi$!Z~2c^~~~cfMP~z!`Iv5v#FmgD~c= zqZN+)(~j>3l-Z8Zf2kT0U&)5xnz+97yCXkDa6|H`+>v)ThB@(%sfVRorb89jZ(>KI zm1EHf>QQLAtKzLi{{f;ePZTQUaq3Y4Z_sXMehFLj^>pFAIUJcB?ZQ7XK@V@MR{d2k zTM^wRoXdPvFWvO*d62Q{@&)iwy%ccGl*BKU|d}nv@%#ZbIs0MEBpsT*?H?EHcLUhUPHIs`7PabTb)2Z z_2PGUdlvt_YWtz#?5^7GPa3Cr@SHk4Y?9?&m=Cbq|1|P720TLj()LyTnX&jdiOCOf zy~A1GhgGCt|4GJ;p8R-za8-OwZ>55I!v!jAReUz%eGDGmJht)}?>b4j-o)td!#4{0 zPtw^?-r1nIs9_J6x)|ypWB&J)9fM5a^|obt#EX5Av>h_aj^e$IbNu;5;&V7qV+X}! z$49h=$cc)B(s&(vtk`mV*`||)Gcf1^+@J7ks^c9Q}#-u z+`+sn9SG(Vxz@*t8_cJ0S01Od!BC3}j~ih)M(u>C*p{tMvqnLsEh|bh9t`1MX5W=q zQ{aQbs7%I8aqGG#aZamozi_i}%vk~ECJYQH$uQYv@?G2^{d zdN7R7_Y-@m5P)#3aYr>b*%aq;psbhaoGc1&&FN(zravu#sI znGyUDyE-YW9tRHisuOs+@p6n^rCR}Y&E2^q%G^)sK zzxYaD*3ddTQfCAo>#8^sDw9#Dws2Un+N(`>D2^tM;Bjn;d9!c?Kakr~NSjBX@A!BH zeK&%S?llxC)WI9oEJx}UiEjIFDg{LHK|nzMkMiigw?Vn=#S{he!RG%wpGcpr4?qWmrr&8!(6hD}|;7@-&&_QgAtQQn_I)SxNwiOq+f0lKhB!$`{@l*uw6OGc(?e(Fnuqxneb zhq$h$;fW+a5Qfo~XueM$e0E+m$rV2q@XzDvi)d)|r{n2dG#~6bAJKj|^Kndh1=Le_ zD21D0$9Qrb&4(uaKEC@N6<31}-<(>ilB=N@L4LrioVz2*?zf78jI}A_ku@V$YVeUO zn}*|)`|-4HG(Sy9Kt-3l#c z@nwEIJD#@3@K?13ic5FHa`-t0nV0BF#xWRzP99I%Saf?%<0&$hkN10b8Psz=#%fp- zD^L~vSTEGv^Txt=KHR$^#Kq%r?3+*0`dee-H{N(NmOsfkZcMfe;8}y%_PYMd-N|$` z4%O`SWV#-Qs(boX+8odC4d?}X4Oo7$1)A6tb)2<4Cg|G*W(K!c$I-k5MC#!<@YeO zbPKcJqr?U?d#^{KKeRd+0ZYF#2T1j+pc|P3=u-tILv0ILZo3=2VfQlRzlwUeh$7sa_G+=H?w0Qzb;nrhxa02fa(fHV3 z?$yk%cug7Z?cBur%z@z9u?+Us3LCSs_f}ZRlXEJcB?K`lFO?6mQ01w-mo@}0*vv3E ziM~ta2eIbmPAWf)YfUuzJ7w#V?ah@`zSU z*8R|U$D^8kpT@7{k{_c<6QMw`%&L$3twiS7#qbA^9Sws=!L^`ieH!mh%@cWtNxxXF zTvE_E_)7&buS2-3I3EqESbJcxW2m#*afKab&b1FZvt#XVCR?>nGrO1U@Zv z`l9XuqpRUx@wlRvX*Vv3d6dFl7?48F;x;FqU3Q@(k170e*N1UvfG19uK?Gn3*^0>F{%G&bB$5_i}B`McaYv z4Z3>+HX;bqQ+PH%gOgGSc49pqpyhmcC$7B}4>U6$i#WG8n6GL*-OlD$xfbcMLwhX_ z+lc!1GoAl2z#C?I9N~P?Dvt;6g3s((gW7>9Z}d-c^%y;^n88P_Xa)AZP7T6Q0Oq3$Hn>w?SO|An!m=#dhm03$h*{z-wi%*H)jP$c3iMUO3-#(?Mb1|4=Ef)fpV*xrJ*8!o1q7*FG0S!_7Wrs;s2G-~O zXDsE;#?s7bxM5Xj;aK`&HpYG1k@uEf79;VobbU5DojzlYo9FPAYL}{topGaZvDye| zV3lT!f7x(lsFA~u;(myuw{rNOw5t{WoD3^tDR(Z0M<2)17jyYY$LUu2bJp&cN^@K3wVGL<+T~jzc25J{aA}1 zuf@>iT(qXSN%SC>4+wpHL3jSS8p@Cqj9N-@%R*Tt(#=j}u=QlNl2>l?YUSE7l(G>0 z->NY*XCZ2SzcI7{yvIF+r(EyE^b_&D4W8+5#28jf!qoWClaH7P9li31$=1<_dEIW->*&-YlWo5A@DX#wmwXm=`u4>;XthSi9aJ0gWX z|E`bbga8RyHs zmQ@)VqfuTR-M;oQ3{9(1Ev#e`MwHB<|H{y7>}6i8~QZvCDWrw<9pm>9PD<172Q@()yI|Pn(wUt=z>?wC-j8 zi||zD)29iRVbZ@7+LIaowUz%Sp*@lD-zT(d?&GZVv4nPgG%a3^A>lXMj2^G>(Kg&i zBPnwg7O3AEY5ZappDBd*8~Jx1T^QfP$G(hbKCZU-*n1?!zK;I!ZY1q}o!@C+8;Q9{ z0Xv!PZuoB`O)KH!9eGKl)3-0O&nzt6yvU4M7>sl<2&yJ+a zYx!uu*I}PEsw$i*xl+iM*l^7S{0&mhl(-qvA}M+uAMBS8d+eTFX3yfcli@$GXE}S~ zX&4=8T(gcJq~_Xu=+ifNKR-LBC`@~u4cFkO7w+i3vRc*73X8*rV?31a{u`*FyL9Ba z9@qe7WZ3LW^BMa4(c28YpnVK8eCgVHbd}}4)aOm`MQ$|mP5u*Z<_Nm}Ccl-789}Qy z@S%RyBiNk9y88ibsshjUXEkQjs%$c3tp8@CqzXJw&PF4(h)FqKC|?jxhu`I6rCs4TJ|02RR(>-#BAm8w<^62( zA#*~gg`Ws+ba^XQ7N}WSVfGc%kQC>#ty>+;!uO1hLQI+ zK0teH7|SMSL*H;p+s1#zEgojnzR%B7b8f?oOSkiBYVN}@`fdkiBU0L>7yxqJu(@8KtNUk{-* zd-!$SmLW#RPcU>14;g}7#jEH!tbf6i;DL;vY2}}fYxie7-YBnNHD>TxCrZO9;fJB~mESMXEXSOUsdPiZ?5 zM6odEz7H~H9^fN+?$sdL@D0D&@k}7QEyX!lgAa2WaA5pYAVt>jX&zrg&PHL_D!*Ie zy2GOa%O(vt&A2a+w%72%nHv!icBrXh^|Oj*u$(A|mh6H%c7&+u8Xh%g`i}^(5nu(& zNsRus!JPH(%#6*PZ-lb>lat{xG8t~JJg$3GCclLIOGI3F2MGZ1BgwU!Dq zItR?i4uEGjDY7<&MTL#ZjRTFFkMon%es4c%c~tGMmc5^$9uD^5vRr0y@<7s_G^e!%OD>DSi8IesuX~ens#DEI6}ax&BdaQdcr|)!e zA7s6-hv6MR^1i^onSlDJDtj7^8!)O@>D6U6{xXHa14WDB$y_*mSy}dQHOv9clwuFC z+kCno-M+ww$7evs64_m=a#*cHLes2nHa&9VWf-r>*xbm|5a({GCF}iY#zm|Mj)GY} zKCx`LCY0G*%?kY33oU+hV%-=0Wbtvga0GtZX=_symSZ-5-dtbfiH3)r|3#=n?V4$;)T@{l5H6MIY6^CagcO znJjBi*ZR`!OK8SN`jTrs=4?ZKDYBj)&IS1x=hyQC)zhj_GJ14BY!uuI54#8Z77cVW z+<}ZWAKh|`U3bF`$StMJUL)ye_zgTONmjdziZvR(5?48Rg4-yVmOhhBVB`*%l{XP?j#!Rij*R zo!lA7$@wO~jGLE=J1EiOGn=IUgI{2ws{Y`e1;tkFAAC-HU4NAKTh>~Np7}WAQDGDX zNA0a;{6|*)V}1J(#y@H0pT=ffe_Gtk2Yc>=oUJ1~S`tH?svwgSY?Yy{@oY0ci*p@| zyiIMb)41ZiKc(H_qg^|^EM8QgHz;Xj`t5A^)62O14v(uj4PJD#g%9?q!wPZ3 zqXxmn;DIpGkd^Ox;XoGe9bN`cdZy=1#`j3iTE_QC&ud<|h?4&*;6-oCiwaCncEL|e z<#RH0Ax)N|cQG9IqS$-9tK&HMG4N5{@-QTL(VTmHcyuJ>tX>rWyI*U-XCFw2t6?Z) z?5sAsM}qmy^z|!6E!+%&Ui9NVKGe~RxgWrlneZh#^C#Pj9Pjf<_7dbQZ&`-$27&v8wUU$_pl=U@ zc}}RMEVIM)?IDbhw(_x9CS`n*wZ}?<+g|KinZp%+o;dL0O%2ib(Awc4@6Yw|q5(m0{`tHyWPa9h$Y8e z4WWpErG-+JZICUwdk>zoGc4g`ZsV8Rb%AAx?X-%mM_ayX$I+HERqkpF`2_A=9ev%2)sQPb z4EvFni7)2L$J0&|A2~>!j7b0=_qi>WpnTM}e`MqE4r|=X`>~7p^zD_{Jq~)pmt+$^ z+kQ2K?33k&Yq&jWJT30xBRx)xW22mZdf;qW(2ovx@z;arApo7yEV&!@!hLsOEO%Id zHP~!m0S5M?AI+Ez`M^&1D5mE+L_f%xZavq}=qmGH34F4fOrL%x^fNZ|f`HvBH{(4) z;6(l#7aZq)P8dS|TA`X>18&EDVVKdbmoSB!UZq(XX}!9RSKe(ugl@;TCbEnj$B|`$ zJ^9?Ocf}=ekjLvrVrFLf`f(g&W1g_IjI3{GDs4G<=W00N%%bLE_&37ZJVz*?M0=sZ z;|DZfti>2>RTo#~at>F+K4*&UBg6)FIm>PliGQigGPt&=3|@v%d~`CzKt?UzxD4$` zA3<%$+4>p3_Y`b-n>SSWkYw%v`q^9PvM@n|jF)_bmA$mZ_7=TbWlwVk3!&Z*ds{ND zyAo)eEfqzDTiaWD)8~T)d_)&?BiMLnun;M5aYK!R!h{*3cE+%>VimrVsR}c`JwkX* zbaB-}C9Gqm8jO`3!QZ17@N^cZ)l$l6VRG<&TPTCoN;$0D0ndh0tl)Yct+`=KCr1mJ z(Pv=Z)5}wj>^mEdgJy3L*P^juXRw%i_6{GrobJE2#%VFaRxS7aTw_O)u*k+ACwkc3 zpn`$Tz(z^9dx0%YDKd>Bi-7E{G~qRM1ufbpI6;GCnpiIQQtCv(YY->OZlf@>>aM@9 zirt0bZ0LkdmVh`>Hoi4cxXbk_!3}dx8dbY$;34B1lZ0u!vc7gVwl8ahnP>r__l+?6d9@T1=Y`~GN)k}2`;p@8Ku`@7!x<|-Vd?}dBH9?pNRCb z;gZ5*DmkH(_7(_fuJ6N+5_L49-BI7tPL8h%t28gc>^_pE#8^q_v~Jry68cO#OXw<1 z)IV%H8*5ewL-;;Dl=~T2Q7%m+uT?@AclV(&ZIy6R&HeO{yo-gU-jQu)Wl@*4FG^(4 zA%6al6_}Hu?V<7WVqt^Y?m8^VtJ%3^p^{41s-2D5YlP=HT;J0WU6#;hJXwM*HlY;9 zbd8mK~`$p~^K~hUQVG#Ex*>4v1azpOZ!Og;C&hO<`0-t{ z+g06P4L|_Gu(Za`hFy0p@x2M`dDr&)LXozWnIV{acWL+gLR$E9;Ct%&6vp?^^`{xX z)T;ZK?>>2#0=5g$_KA>p50&EXQtoyk)P6({S@>NtY!{M3b&&OJF}=WBT3Ba8KhQa5 ztKS{^c9*;4y+ha_je`9AJ=%~aOe2SKA%xx$o>N%-o8XJbTOO+zHpCOSQ_^{=qA*1G>+A{islW~bbb2_$X){7 zWX*vF)b?kYJ>)zd+>AM!R$M<;uB%Av={v2cj)pb!qUWXplkn;WATtJMUK^4qg#U< z8^i4VBgdj3`vpx16+Mp|v4-8Ey?cd}KHiYBsLlfN2vxqO_NMV_VS%yiQ^AwtPXA%t z`T^jSF+s=l`{moEGwOf-fa5b`ygDh?nnvCClBkbh4 z8NVCneJiX}bMkfLgCoKrKDNj34G*kfu^jQT4Bu9>;TvDsgfo>3epljU9lo)dw!2{o zWOFI*Yr#$1{+lHlzg{=G{(#0__#6{g^jJa0u496m+H)jZjKFeTc|TQkGCKP4_~CZ3 zF@Y!IXy^$c)AexQGNY5~bQT60FXLB$fyOp04!n0lc$}ktbn}GZ!FBqPbW+&Ko$GHj zoD{xM`|AC0l7(Hi;bt&gH`6RQDlYf6%G8E`U8i+*g11c^0`t|o(WiBShf7gl89NVp zg-6;>sVe2?RioU!=yIJfpB>tqa9U6fZO%U}MBuii9+zu<=0mDb==hu{MQe=$Zd(D`6UT_t-H~LdrTMT?OYw_o=HJE9B2j8E&)Id^$a5z55 zYHtwmZ__+{>(t#tWn=VM4u@<6Siwesxb>%d1iQ6?v{wY&1srZ1dPSIpE7LVD1C4Z5 z=+ALa|4Lsp3f{psznbZ7Tq6HAK3nZMa%5*v&trsa>6@GkUJYb!6mS8m3q}5hdhG2& zGkz0Ja(%B*_;uk$_B|l3KDaLUhqVN$UCOfYJxQ=xu1dj?dwgWyl7rs;#>c8k13tRH z@u^T*xmta?oPRH-s4|Y^>%y?Hk2zt<$3_)-2h*mxGw`W1bBvGMnWJEMfMY`~$+(JO z9yEy^9y8$d?mQ@U${~-k?4U!m@ONRDPY^u8>4lLwtMsFMMAf2|`0A%}fa)}sb8tbt zJ$>`L5QwvuO~0e>@8v`uO~MHD)2D~glTCtOIHJwI0B!n6ncbA#D3mIjhGLc@##i2Q zR1PCV5@N@aG99x$-x4)_)CAqaXPDnN2`Tn}CA$>KCZ8KHL-NMn5QaJJg9j=*%f(B+ zm@W7gxzg$zLRz11N8|e(4)d^;Rz>NCY+UrVwF? z^y@c;aQj1XT}{{2qXOY4J23hEO<^FH=}5P4LivXrqA7m})0D%&e+d1!FZ$5AKLp=i zb=^mQKfO#`voOi=`(Ky_b%{xQB&{I8xe~7BM!&4nc&bNd{?qEH2 z+!BHVc0$UIk+P>6yH7W>*J9!@qw`foC}-{q`%=nnA%H9Rg%;ly{I~(X(B|91<0ys3 z+rk7KiB-<_V~+TXG5U_+s?K^A%}zCPq{5Y4T)L>>1~o{u+?$9*AGpM#uK zF4UWq6(vQc@sH-OZ8uQEJOMkx>-!P)H1pho40_B*?m{`4=q% zA3Z)Fd%IVg5sSxIPO*G|v+W}7d??Ha8x1>b{WKe9sv52=0tITZrz-Qg5Nur$ax&B* zw1xGQ*)I5wELG{*w`poa+i=w=HiGqoVYVxtC4*dzA6vk$UOl?9;R5_sJ^;nr7#C=7 zJ9?s|i*&RdCA|G2-E0>!BU9m)wcjbgY8*^1AD}+XO4JWc#3pnJ)t6quX;{nK8lS*F z=;*#zF4EEtD0$>X+S(yxdg!cvcmti0Ty}1fx$(G2+D=O^*5bQ9mWFl;{e@( z-)s`>Sm)T{TNg`CCNu^oFHp2e2(V8;94vLp&7i+Xxh6~`zrH|gOoE^Gb9lkhT6KXw zHwn?&4;0?X@cRXNU=ns}=7G;=MOEGe zQy#2#!l!t6Zb6n&UVPb=)m~-4`l;J%1Jv!m2CLh%E)ZuHQUb@rjb`<8Z5kO*U*g!- zW8m2@^63`5hG8tYkMNk}lX2~HnUw=(;Vl>Fw5n`4wx6-4 zUacLE&qJ6|wk*_ogrCQVnvXCX^0P$+#U2mdnBiV^c`xGpJMm()<3Jc+NnlS^xEnY) z?00_O25~pe4E}746vR+9xB6$ABZ`sXFaB(qgH&wnj*Pn@53(jcuxGqTf2Nb7=wUwtb{c$e6mCFVd*RQ<2k=YHEjmY`He$HftaF&R;enkdWE74- zw4)F+29zm>wp*ycM)dNU1{*9JyW=+rd=B0N8V@BgR@;cqJQsY9j%vl-oa!7c?j`zh z4}YS~y~IfS8$VfRO;{+u{u5p7C3IT5;N6Y$xp^fy~S`fH}of3V=r#hEIn&x zXT@8Lx;~;#@T)$92B1V$Z9?r~D~>Txw(b)^9wX zcpa_3n8JN@ksQ5I+@ID_q_;Skd;KD9^cJJJWf$qBH;Q^?o$=Dq7kS-;@nTf zSGIQIkllUkO7c!Pl<903RY&$ZF`VP{d$2614Iw*#wkkk z6%B5z(PMsTPUGgcYwSHz3_-^DQy)Julsg|nWBtUr+)6m|6XUtO3v}BLF?snE*$)t> zap_i1@(1cJ>Q~9BSEvR9>s1Av+I+C zo*TEPc3VPGg@R_62Pl?sR^?6bpy63enHmAU58k;uaCqtm-T^#rue2_Jdl*`wUv7}I zu2GNSBd$3lpMhe($Sw$3Cs%=yJKHU;g`$r&`S#pojN&%k(5ohJJL(PPFW#HTOh6&Re^_SU!KNvbm2-gpZXau$c~CLsqr z+Z9)22;1X*%Vt?{cNhu$>CQJWceJ*IQ(>Pe_Nr5~4&iXaPtooGD0A>BIvRlRZBEmJ z05MYA*J|(SQxrK!3=icPZ*Bh31Xcy%TjX`96gqX$n7|-)<%m}~bkH6KJ?#BjN1F$U z-i{-gn@jrkTPH1HBCPKQiO=I12O1hEy67|*6R|6`{v3eI&Q`0+wBXqkrfIO6)#Fl{ z6)5Vs7LL{gB8Iz9k|9vsJRkrI-%aSdD*lxB{dq}W`4R?uO`LvI6OJz{MIEN1nL%hF z%T@GVkhqU4JxP;-QA?b)v?^Gf#K|X&=Yqw}oV`C<-Qx1tvTJC*=ANSEL&UTsKghZ- zAWcKO6&@EOvzsSUx_O=D;?gAW6>59s;?kb(^e4$a6ahYdf?`9(Rhp0!Ed5TwR30k2 z&@Z8)o7P7WIU8o5Aakhb;b6;#hscU1A5?9Y^uVEFh?6fO!#<}i=hdgL4p5avsc}AL zsQ9wB@wnon;p%a^IaG}0-aSFVVWLO)F{}Is1zik%5KP7DscmdDnFCEPg2ug<&o&NR z3>uhybdr{ai7EUQEaU$=OdP=>J{@6Vy!K72M{gXbq+z08@G2``1k@whWAS5moOOHp zBs`6VEc+yF90m`!c6;b;^)SimLE>?m5e^SWTlrDIcEqF&zn&i2!NdEI^*c#>!^O!q zN(Nz`*Jrrs%EgVQLBqutNB((?WrM5XKCl8)a-4}%W}?lQiB{O)<9aNJVD^HSb8M75 z95mK>+nM2FgL731q=8UjHtEG)avf&U^nHXlUAs~7*2PeIjKW7Cck7PP;t^ti-^b{j zvY`8#aX8S?uD@4vQ_lDuGAEU4QU*p*jc4eq5n^!ROso6p3OWU^C}KJ8Z1@-WWSp*H z`)@4c*al;kV#n2xX|=n=YIiGmW{>Gju2uF9WIg8=wpf|yt~2bb`y7^-!TiX|!Kj>y z!J*M*K~Jc#o`dh~28)$J=p2L8uDG{ORmpBR`(rRB-?F;$26nXP%Xn!;yzG&LK$;Lq z%ZjVU&~enFYWiceaill|Q^}f|4*JiiVqHvvM#E7$a_hmj}j(Yz_u5bqJHq{wjZf79FE|W5lG;G|)`5QWeBQl0fzek43v& z3`th=MBw0KG$&RZ)~{4~RvPm)6-IEEYT*G?k!1x{#fn}IpTKJO@`56*j}_N*BfDtc zSn&lqr5Ag1?p<_!ETUXcOMZHcT_*fMnR+om`yb{RV#xnMYxH6!chN+b^o_(nW%um;9*0(57(>0{#R!M0V0D>=0dw7yWg(>`+q{_;Swv ztI(;NLxswl_tj-hg}6Y#EZ1rJ_9LGJG01c5KfKZ5GDA!gN_}I-(98sJAa|wKxGq6_ zO>!vRgelISQ@F|zXu0?8)R0e#=)&#zFGZ$^(;SWe#U2iN7%Z3utDspUzoXqLVldhw zl}e2t)uo8#3dbarH$jZT^vv>SET142X(%g0%%&eQ#7Ticx+OUaUtBOtH)FOgMz>&g zt}bxy65Zn2FDzU-eSvOn?yT9%6p7LEDG`Gum2pp|__pum7?|mZLyDZ6@J@dIaixML35yY^DzY4y1zfhbLtYO& z3#Kj2H+N0Sg;RS_!zS6 z3#4=$QUFX{qf%vm4Fo^y08#*458MT805-#X+kZM_)9c8&Z%{PYA*lt{JP!Lc9kON} z5^|_Ro&_vF+#xrhjZFTwLoU?Mg@PUFklPlZcK#P-`~p1s4l!Q{od8yEK}3E)4uH=9 zYTi|;a*iR#z-x}9?DJGAFW~X*h$yfLcpMlAy)pq~fH5baGr(eCKClE>4%~I3qfo8} zQ3F9OFzghR1LFyyu0zfMZUZh@f~-7^ECmWbcF3(j6L1sE!_L4SR&UM&4}gz2+aYfQ z?{Ka|ZUJAy5d7Jnks!=7yvBFRdw>loopK9M^JHPC><_I^f3j2716zRkz&7A|V8c_L zvIe!FAhT1B0xo#EQ!WSY0-gsR0IE`IP6W2>MH!(TD6T?Gfo-25LO}m&UjsTnXcghvOsb2wMUIlCfFVrA`gAnn<&@rHJ1PLpIg$58uASn6{MSzCK|9b=s ztOYIuE~rHo19d+jr+_uU7T~O-ow9onQg#f=3#0CSEbLN6mhC*U5~a;g*4D~#^y z5DxgP(@5ZQ6xH=kxiAMrbrX~U7}AWg2IkyC7QsOsQ2&ZbwdYQ!yb0I{Y=peL1p%W) z%)bjQhkU_3oH0J-|)C8ekRhEbut63D^WQ z0R^>5cEBf0I-ox=1{ei&=L$`7K8QfhB$ooifEBHP)*mHk~OR0Pp>XH1{l$|OD+Mn zI(5l~jUYC;cFFz;P!_K)xfEyu)&dj#y5y)tG)M!x|Ak2)GSc0jvhr0gnS)VZUuCBK8^r4C|8HATI}&fX@o=lFtHbhIh$!YoOc_ zU2@ns^s>O?K>rbi2oMHgBfDf(G8!CU6tE5G|2iy2b;%3Bw~XqN^TC%ycga=Y!$x<> z4ZyQ8UGf3Qi^p`yf#VSXFcH`SECI&E!W`HHOfNwZ#dXQe;ERCHDJc5F1f&E8sfk^( z`&y*nF<6Ab`lK#-8?XYn0P^#xUGfp|`_oWFsR$UjAJ_(L0=iG^k{upLmI9-I1y3Nt zK+_}?1<*gUOO8lG1g0P-fkn@v$bf|z)1ed-k(D#LTV zz?X-a<+H%#VP;v#L=gg;U>*@^M*W8%Im#^W0XoN;<%lw*Aki#qo<;zVAwcjoNr(ve z1Czi5Yp0uK|11PJ!z^b2&jJg884Jv^*HpA0i_CHZaM@zmdlt$F%mA)mVwSf6_q+yk z_`7F~SuSjVAa$)-j+urCyomsSnoVXoY9l;&&n!2B_j=zfJ3I$_+s$$|P}qSK0xN*6 zz`Am?oSKd5`2hj|R()ia*TcQ;Q<#Hq0XBf&RArV!rXvBLndLlS)aT|x+5c^L@NctR z0m1QVvs}+C>_dPvpc7x3wLYqMMqzUTlv0%rfmEbC?>rQew41q=_F<^8~nqlow{ zXg%;a+?_pUmUF=8A2-W(o6r!QKv4pZoP>XnN1Q5zMG*C;%(4lnIfE3yg66DQ-ZUE) zfi2+c&Y=k20sahgV8MCl2(bEsSv~@Lc^6^6898{#EEfZ-fUS`4xrVX_zpc?MyAvV? zOb7a3hkOonr0@^K1cIV_P@?CN#rF|opso!bKpvrz+ib0PO}A0P(qzTn%*SFU!RUFrDEG zsCItv7pNP62;@Qz3>mLZ0P%*Ayai|i9%uG( zH+(Cw_(kLh zxE|O7+y%5-g0f#O%iCbD9#{>udj$~y#sC|E3xFozHlXu%Bmfu!GyxNVA$Uq+*i;Zx zLF5D11J?t+Rw95Ma0t``cL6hi)xZUG;)3WCyzQ$FISjo4`*XlA1i$@|>4EG|ho2t5 z#;?frA{I6^UsLo&F*vvZcDx{Gf8FkyddM;$*W=gn6>YdEruAlFK$}z=tZ|&B>6b)j zdjGEIq>X{>IDQoeXxb$)TC!84RrrRsUlRRnBS1T2&0{LF@1!oEl>o)39GJLc^ZA17+8X?taaXQ-KkeSxdhF?V2vF<#y4CYa=iy-yMcKsc;bMLct0aXOd&adf^8O@@T zXf^W(n{IJOeMU*XHp{?N$92*j2dO{hT@?kJa!4v5`Be=``ejIJA(^_y5>qOIL~4U1 zFOJq-M&3ptrKUJKdRa`eDF(eKsgnxrrT$8q_9z01{Z$N>>Oq$$QSPr|sEvb&!iUAT zMo439GQhNqv&NyyNfM+2W;~8%MX>E2(9W-65silZCOS#AVAEft*ap#6Y6LwCPXYfX zy85alr~<5|TT-0Yog4n(yq~dg^MWSZ&XOE?tdQfy#ano%~v?Z1O*1 z#$s*P3#GbRsT;O|pqt@LPjL-!mI&4X39>qSR2+eDmJPGxxV-&0Y+|mr+OxBG|EStT zfv

ZZdga7bB%c&>>T3@^vv(a);%JDYW6bm?X^t9XO3HUKc0%9RS_Dszd&#Uo5rf zA&knVwBM21Hqdy!BlmYP);6X$G8Aig*>uQ3a;6`C7k#8D$KH0K2pU;8o{z`8O-uvW~p0#o51b?%aYr(MPYTS zikan5zzs1?68gaDLRx+UiWCA`zldd@pO$5B7Wk?~bn=FnjrsQu8ukXW1HB^9RyZ0J?Pz?fnBy+yT(VC3O1_FcJiKB9`?wF|7sC0Yp-88c9^ z`X@M{_N>RUE?R!LO;3KC+a$arZP#eIJ+OB6C+36V(n@aYmU6|Q`ap_=!)+=&N?b`D z&0?m6y?<5tTa~9$NwQqh3xDr!sC6G1$L$(dF z)-#<>kH(Bq2)GJD(-~Uzrx;n-40_wyPMJjr3BjlgnTz_svQXoRwghIHM`jke4)P+% zm8{(^q}uDO3F>JdZes9Dg{&BM3(z0H|0*uaQad+mNV~=ET^g&EdeMVc(KTf~Y}Q?c zQo0I8`;|hPKh$h@;aF53n|xD3eIG91(aJO>`ny@ zKl9L}ap_1OGt%ZV76l^`wv<&UK_9Zcy}+J0XgB!N7C6~h6I6coPCwGQ;h8$0-kN;9%=2A4^ZC*ob8JBk?c1= z6a~L@$kVY2z}%=7{C#k`0A{sqtmb=bS!uF8-o$p+x`Zp;OC9)%4hncECOLR{gZ4DZ z^%#vpeM>3qAx3!bJrw&(IWUUy!3_{-rP=<|vH|qqA?p5F&}0n(bzD;zr6?{+& z0!*?Um0cGDXmuOhr$g2fNW0t6z?FmE5=0N$SVIPSPY`*xi@T+K9a`ov9MuGCVwAKW!fm7Q`VK}9&7jYY!tD?6knLAh z#M13fF;-%GR?cH7+$7GCk_Vv7^t9asJFTEIk|?E1Tp=w(f#{RzWEVyrM?l+Q7;8pk zLIxt=Q)!hMiJt{J>T&wYjEL<5U7SW8X87L(dfP;bmBkIV1p)A964u%2Zl~CfZp#=7 z<4R3c6=X*&vV&8kKGelaecnmO@LU*#9DDMe$x^J8406j;?`)UirBaZltaq+UDN+N- zy2*4|EqQChP&gW>SnXt*rj|k-VnDCQHUciqV_I8hDYdUL+mK3NH1#>=%uibdI^#L4 zq4RX|DQO@L<)o*ih#)A;3|hxYTcxw0x6PtpUYcPW6O6iutp|7dp)bY-pYf8XZ2@Hd z&ts#;nkpWi)<9Mc*%96{8;cO6QYn8hk~)u$3esY2Gw9|8@Q5e-VHnX)6Qwy)K9sp& zAsrQ^Y0`Pn6-y{ok|HJNAxQA6G*6O}Y_dT&tTkC?J}k*4ihwRlQmAwQbjw=2Is>`| z^nrDB+W}dV4}0#R2z?#p+DMBFOF?hJhJ`XtU}pI3f~;*5mh?fZa_rMl$1zN}tW?4p zn5AMvgEeRDk5$IA|A(vzGI#7tblXGLs!UcHivx-w4C4y=_`cXr3LJ`t{ok}gBZUS` z1zn$uU3QD5QZ;L0vXv%OnRge!Jgb_1)JPkHQTbKwNcsBzkNs4bN5Q=IARV@qQoKt* zm%y%``HjCE{I)@sa|T;TBE=?1zBEBA#o20xq0r7VRgBY1E@aS39+DoikPG<2K$<62 zffg=PbT6o5E3?}`D|$)ZneO4}fv)_OY6k=mVwIt`D1S_*hpZm!? z;Fkkg%@w-WOG29hIwnsU=-AXU`l^WyN-%}ClZ1p8NUE=4zejOvDHmZl${8}d9x`{x zDj;K7$NpHXLM-;!T(7@&83Y*+G+n35cG6OtI?#4aCd)(+pBAZ1zgrm{V%XfxB!j89 zF#V~ewmt9YHG{bU^H9fzEr#s-d5>9`vk-1de<{

zyjf1Z^=_qJ%%Qvlh%1ln{Bs&-iEW` ze~rH!rQ?tVYAMqPQEvsEt~IXk!HuQTrt#?09msotbkH^{1qF%yt@YOY8UV{|yD`;; z0{oHAXs6{L?%R4IGQjGZh`}i~RU3 zKpJbi0Kz>G2GSKf$(!5;NyDTgkQL}DZ4g3sNJCqxr>%pKp9?_yr{OfKB|o!=Ne)_8 zFuNeno=BmAP{Ah9>FKmM5Gtsfh$=FPE(c1%%3vk~e3Qb5YD+*@J&9c#p7sVxZZsrF zS}3`vBafeQ&tP8)0ie#7ldcgfi$bvE-ggU(q9!AkIg62+hOzN$YA~v6 zBg~dfrQBe|J?se-=`{K(SXwSsfp(rlCkIObih9~MKrrihwo6o&A@!p*gVE->PeS`Q zk9NY6ln6R|9&->Z<%2#ykIaLmjkb1AA{TQX>Bim=$K z&$a{|h-xa#1#Be#<=#Pyf>&*Ve9L>}JrtwKM$i#ED0e8vSz(z_vmN-1rh9c-%7SG7 zGjw&R6l_xhx*DczO2^32qzd%$y%as14S+y;0yn>x_`Fj)AjI7AYrqLla1 zh7st`lR@WyVYYORY<=1`kC}dftx~!Yi8|sODTPTjkflD2iNM!-RJq~tHLE$ZSabU6}@|0d9_2grM*nOu}TCgrpaHB`_yl+;d8q` z9=H3>&ZC|0`@iq|ak*@d^Ss`FJ$^l(&&Tume4OWbn~EPR*Kzga$K7GwA58hHwz_b9;Us;+pz>u!-x z$L3OOrXgs@LX5&@8Dte*BfUljPdVVG`Eq(k<@fDzfXTrJDP`9AShI^f)GhMH$l$Qv z#rQl2tXbGc2V)7oJT!}>9}~<X+7kkp9eJS#b( z4~}SN0t6LPusj%f=un!ZjtUO#RU?uMu!316`$kS56&#ynaLg#1io5n#fxSNjUjf06 zQP-<`;@a~fpNtCj?==ky)3La>UW`tTbU!wDWUm^q7A*A5!gL zOCt|mm>rpaZ1C7#8Ht#)eubsWVvOHXIW{&nzNQ``ui<4Q)=wMpXbm&YN?ry(@uE~xh9Tyzjs|s+$C@h48e(2Q5Nyh~TrN{xa z3A`8_$6OOSAo4bNA31=^6Sf85yM+IIXe4}G@c5)+z*Tr;XqBHmG}0|MI5?>iF#S~A zGZf85henRe4IbaC1u%O&?pL{{y(2d`vR5`9j}=b9Voqts10x^i21h3?0L#bn=RDEt zc4*|y6NBkPA4u(*pVmF0IeNeGlh14 zkTZUFy)?7EX-e8}l|{Ng(4%`|L4a=Ujqj4^Mm~H@gAap6%<@<>bRlYe5>8axj}NBx zS`Eq2tMHgW3VKci70QepIVQMwuQmwEXQBEcI509f4?FstF&L5s7(vN%+(VU6Qsk*I z=;*m%tMCBAj`_cawU7A`eI$Alq|G1%V@IC+lN zgw@w=a2YxW7E^-*U}OM`ZTIp%*Ys3@(I$qz;GY9Bk%8XFYUE@Hun_o$eh8=Lh#nDub-`EG&GL z@*gnNb^7}JU~a`pUf_`Tbx!-0nFAyKtx%B4F)nvGmasLuK#VFOqi(N{G9; zreGMp=^%mb92&Xgl;D27aAV!%%!D6-^CL@6!GWw6tPqN}L?=f+JSDh)uWev+p|{O; zeI#LQa8R$bB(Q5>i*bx2=aFNBLwXg0tjL3s01-nLaPQ^Zxn~Vv`?YM;|;gTYgPYs^lYdKh`0&7-< zB}UePCAER&W5Mh;DgH|bNBR~7`zK}f#z4X1*^tD0;^4^10*e5*$O>91*lh<#t|0%~DlGDqaIX!CJaJlZaPNA+RZnUehQN!x!-)&Py_4j@S_|4JQ|i8GNaXWkbW!f< zII$H$vH;6@dq@>;8xpzZbet}#0kVE^9o9g^gKu)8;i%ogjZGf=FxkX0hi4yi|EuqzgB z3S{+?Z<A7T9ar{+y(2FbqL~^Y8~Os;T0A|Q75T9c zZJmjcIs!|Rh}jlK0h;fT2gvn9I65B>G9!%BFBsa zoC~;i8*a^@T|?&`vI@uA(DQ?&2!kT0O$Z*IB+tG|Q?N8x>bqf3&20-OL>+zt->1PG|r4&I@d>rni;2axzObQ;kvL8+;51tj= zulFhlvrfRWR^ZSgsj)o_WhCI{!p~;~dnfIHLKPH>Md8AOB0o;X$yUDXwh_=COHKtW z#HgEob}%z(0VGAI;l4mR)c0iDzI-;?wgHm4SYT_1*?AHPs{{6mB%FgILNc_IFZ<)K zE>h_eBE!xJ9+@-&tP!k0`s7f0T6RvbUs5$B4L9Iklk@~6NfU!vk?+qzBeX%%R)*EU z7=EGO*$AVHV3WTec5ZyB(C){H2Suiw8yp_Fw*sS1*fRapEj&GQm$im;z-2W(n} zn=#Un_eFY53=WPQelCu2x%=Z3zZQ3zq@?FXCZCHLUL{!l&sbb7tnZ-6%t^r$?L@H^ zf{g)OGIfLC{ee=@6O-ioeCfFDVBy^sW<_qBh`y+VB($I06F@$GptjK46LHGk0$Gm( zurO0%95_%~JU!C)JoH6&26nZ9xWgdh>+{UW8RrF0>|F+!brdeP#MOP7aAo&RJOJ6d zS3P9KSdLpOvdbe|&kGJo+6C4OX5F8Y>D(VR2{X3B{usWvfnh7xCvxSa;Mm?ZU=6rW zVGk|u90>3Bi1Edw;6Zz|K~{b|^dm2xk3sP^+PPOD#$@&wEC7~>QzBQMk5gwgSUFgZ zR0yZe^Mm+_uk&%pX@ejOtAlMT{EQ7ZPKHfJD~v*{a>$m8qYVRGF|U~%9GJ8ilEwmD zor&bm0g;*K<1*~^$-%z88X;M5n!B_Lg(E*qMs-7Ik!mnIelM|paAKQ+@mmB*?gTW1 zbo@~R+}L_>3dYtdNHUB6{3}&<+28s{-kgHYH0Y(wYp}Ao3*htkO{(JHAxWbE%gV4= zR?2c)|Hyg8nB~s^%*Kf@EdlWI$Y;eUUrjL%h;CC1ev2r7 zk5~N2QN8d2oRW(m*;VnU+h0G=aC_d;3vf!Vgk-^8_;n9y&}XeAJ@Wnq!F_sd5y{=S zJ0ad~jO@H1I58;?J7DwQamOVQ?1YTSo6}I98>R*i?NtTIhB8W-y^3(4Ui&(Wa$MwZ7Y0Z7ss_u#!sNM9h{VXt7Y2tVHG@ri6U(-R z{hkr|>q4|<51cP1V8yb9KU=tuojT<}kcTDA6;d!%{hVpRV~;8W%z77flNS9l1OJh* zSg@eSei?Unc_4T{`Xn1@74+6ZZ&MI@@7P)m?9~i77i*YzK>;J}&uPKoNg1fz2&`wW z5r#4ycTsR)c=2u6_*5>GqX`ERQw^Tc311JM(+S@MUf2oW4vzQSxo7eWA^y88#v%Vx zEE7jVwI>mzLr~d?!3glx;6vNBaX$^83&1ya!l!{Zdwdl17l5~cFUIIK$491K92`Di zJy`xB_=+l!IUb7*5>EUK^ED&eO^_8F5_#@o++o;-&!Mf6%@+rU9h-eSI&&*l{zFHS zlqC-=9}JVK_9UMtf#-JQk&7+~J{fM@Gq7<_BTFH-bqzJX9Wbl9C9i%LZyehrv>3m= zQQkdyQ)2+@>tq#YbI;JOcLJt4d42b)H<0gJ9@q|xs{MMbe=}g#XN1-_;9L6rLuJ^i z>YpjH>H(n!z_tOQQh^7Ctig~Tbsq%G^5H$2!|=EMh)@$EHjfM~##YP7?WoSC<3bgi z0%pr`_XTOxtA5LPU6 zo6YwJLX{|F+x>ymO7uy?1A)-?*U>N!1d7AhZy$8xrU&s}Ra9cjgMmp1SQQABL%FUh zAPXdxSG!_us)jA__Ub@t1$=CKB#UMN){R?o2 z?JdHgU%DP`Cwj=sgs(>ppx3%SFlsfn*S{F(u?5==F9yOH0@_{-B-ddySN$uHxlKm( zzXCm)F&10?6$tHuV#P~=&^C;kI^<<;AHkuU#uphp?v)c!k+8UbwlcObJIimec=#dhh-7&^#TBd^Bj zY(!p$?aB?PDTY$b28>!bXhhx$yX_kSU85c@?V@_LNRX5@9q7jF#oSPZ`D^+2fQ^+2HFjX-Gm8-c(kktAC|v~G4N&}unl=_Lm*Js5J=mCd|Lx*h`jbKH1%77KpXP1w*!Idw*x)aBVYbb zAhdLIAkcuk>777ebz>m3zA+G}N8X5h$Gd2`_Ym+s>^R6<-VcO!ybr&d0zFDU2uupJ zd;su4Ag~Mh(huSALo_Y&YUJyYZ~F)Z_$Ux)`Zy4(MxbWoRmfYAFGt>ryb^io=0LI( zq+)ZR7H4Jb*yj1~uLw>^=o`uk<(lkFgKzr;x@U!20wx?=JBlkt?10(RNq*Vza90b-T6ym^`7}}I*7AH=?Q8ckfYhqw~ zBF>)SNcUO6pCkQe2a6;7%n1(a7D`VoFxlCW>t_d#OZ_X*CpWbkKl=jB1)2Cz708JE zb2e_FT{I_HVT-UB$IZpT(E6ZR8XSdRsR;hm=boj(u;mu$Jd@iHt1^k`h3$VQ& zeQ z`1AQ6Sa5-FfcH-56Um*>H549^H^4j6trUmu9&!9yMp7Y&8CybxNX^AC5^*o3gVm21UTC7#!n0y zBf)+B*>O1D8E*A2W?&tgH^YYnkc`+PxpCyx6gn~($y&7aO8nA-KfvD)R=ow5c%x$UT85nw)^}t_>sL$xII9qqbneg z)244Q4}3rzE@wqwe?Iha4DXbC`tAU=AdZ1>&j4ObuDr3gARkjKWI1rXku{_Sz}g?H2sYv0d-K<06W zu|Rqz$^etS!soJg;92W#EhpCna)IHu=wslA@6N+UZWGG6#y1dduROdKdJ-r9ft~1g z!ExZUPWTP*Bd22D|GnUc%J}yMcS9h%qtBlOKd=-24{$k;`}!|{4+c+^2^~_Q{cp^p z$c2m-9{JCT;K7Nw3W}c1^X4Z+DmLKTStT>Nh7$66M9zE>&yTZl<_Me=nfGvT|L~0Y z*t3yVYc9{9k=|vbVRp-yW0A}8+2gk;e}# zr%-9y&Uk(&d^VVjfmnU?+4X>(1y~C%yGg8p{9ttFfo}yrr4#+zzz&GR4@Lal0^AQK z!`=_P5KLy2J}(0s*a??Ofbh_G86s}NA-Np)Wp@)U@$F1Rayg@@K(_U)b#NShD!3f+ z`~VZdWH<4-w2&MyI&d4`URfU&E5H^G6gV60-3cE7elqwx)=UpDh2BNOvy{u>SJDUE z)`itUoU*^AielDc_#mpzsZ?Vh6riI%};)Bl0;OBo=Ma?~p(y%l!$ zIwRU{x6%G`Mv~;uGJl~#Jo%rL@9E5Uv)_XS!nwqzPG_ys|7rLc9Z8Fs*Yh6gGjt%k znWSKMx4V7Ame9FeKb*y64%Qj(-wE&4K|iwi@!-(*iMtbbATcB%pH^8Y4Tn>-1j*LkHmt%C7GS?Cp;Ikxos(B)EpFaYAtPBEQ@FWCs zB=Y%AaM_W4{ulV+o$y2yXmlrh5Ac!X65m`P|B%jxptAte!DVVP<`!48Vo^pKhR&A2 zKJO2n+X>$jgGDY3I_wTMzR^LGy$7lit6;ZPUO>6(Wi57lScW`sMcSp;uGCYWMfuI# zeWqA#bfKIC^o2h;C3YyOHPM2^+i&v?4%$fm#dF&x>^4XxAGo~kLzjOO_IBtE zXyMLwXyN_gz~3F8CqMG?-5pXYmR?F&jpq%2cN$N zb`bu)zsJ?80G}n@i}si9Ix6CLmZdm@VPMIb%;zt-0TVgIrBQpF~~x{IR)b$Ud1&?+u{aB6Z6HNaiDnP z(kO0=jifqm%b=9h1%c$~_&K?r3KuVP zeq=r`DS$5PgZ%MjUC*&VnnjYYD7&T<-za@h+c9d%!1 zguOw=%;nVAyM(^pP#jdHmwY}PqgpOb-l4sVgTb{4HImke72wPYE@woa{{b!o)92k$ zQ`zl(zIPm63O-l{w{LI*1Trvveh;`D)_s0Q9KJk`pZX5^62HC4U+f_0$k))(H*7kY zKS=Tm(uAQVP2c-LH|DfU9{`iR&gUOX|I481L?DC1jHe@e9Kbeyu*+UNTuZJ65g(GO z_I80Y>{j$Z)=0jTCo+n$*G49;4fZ;v{44B5NR`Y_sB(T}pi4TN+Zcj2P4c$CX2S&B z^Uy0P)`P{3+}6|EUEG#7k+gx^GOdx+d1SAOguHu+D! zkFR!enKw!r&+P$j)7*$$j@~zt4Qlk}!pY4}kmcpUM_t8qm(S+54)N=`tqrk)+uD-zQ3knB;q!06q;-FA`eAb_HY6SW zm>#VkPcsLp1qy`xEFsJ_)B}*38Y^o9m#E-t(g&#TVDvCG$!{HCzoAQ zQVwgb$Iq-;-7MExT!!6p15ZBY&x)k14~7HxN0zJ)4jWLyLF|km@8$GmtV=36BC>UT z@XEjlxmubTE@t%qqdaW>j{-9;#?1d|0c6bn|6PE}``p+_#z4D!YeHuCis5dgPS}Nq zIDG?nVP;+MxP7YHN01#i`9JWezxnxzb;0WcVI62z%72M?{F2KN70Y|HS2LFPG{vWN zB<^z}My>7#9i@r5;xBajhuhm#n1~mZ&3Q)mVWl{y;>1xdw=levh}%JJg*qpU3KMaG z6ygpSrQ#>m=#J<(<0oR)94m0*L_9;t@6=IWQ`jj_!^=rxn_N64;tX7KexK?xfLM( z$#z!R)2KiV3M}7!?8)Lkxkc%Z$qT`y0=i_mnU0>Nf91q@u?Ied;i$`*v4i^!2Jrbi z3^4ab=c0|C4zQDX)Rs1T-u68WYOI+tf7$@(dvaofw{{_=m=Ruq z@WQuFxx4{h{1@CLatuP3nR!7QKPn(Mr8G!o)9GM*le~tGws2GrwtIa~qg?&Sf*}c< z_n=c${kzCB!Vft?4lU%axYVrBtJ#P+d?dI8C|l$l%%z_dRzJ##O%3(srywPT3s{4L z_K;2N_Y}yMq(dxEF%@2ca^vumz@w>W!wdyAyYt$vg{ivGWf2BWCBoRT-mr_0?4 zB>BMvNq>=-yy!rV9V5nE(%l8loZ<8*kUv0P!L?o#oBc7lOE6*d{Uy#|Ia}f$E9hxv z$TbzxPzLUW-9c*Bu+VXNgNdZO$g`F?E-xREl!{rR=(kZou zDB_q{FJ6$2!L(2uHJ;}@R6A^LAm1_8@zvxXkZ+MCjYvHhxF2r8i=XXNoT9|Fn3UNPxvpIN*-;#ymwn;ct%1!!Ai*BqP2U z?2sGDNLsUjxP%nGRK5m+RAv7gb18WXzB?sJ?zl@@P2R+p8m)EM$lKb*Pu2ZSf0WD2 zIk-_EJoCD+BYFh9kAfat@3@7Il5vYl3@Yb3eJ!Z`4ru@roors}prSTCLjC&5Zo82D z6Y?$BIX}}G-+Y(q0^|{7aM1mMK_-EVZ*f3gH6m#ua^Vdhb+Owzs&6CTl-~isOwoj4{i|^NStOn!QIp>9SZqd!{-6$viTt{#lOW zC%)#zI(m=>n`SsJxAG-@P2O~c+s<;Cksmyj3M@av0S#~o`L5F(M+4h*ANllgvWfAZ z?znlI0{O8kNg5!4$(sa7XMk)cHp9ucdx5VeUw@`kl-*2H4Y~XQRmK)Ad%Q4(6-rd!DiUvdR4XV-j2Udsc~DC!@B$1LKf@p7j)f&6*$9^WR|SKv%`56?r}K=Nkxj>B zNY>L-kXwj<$7i^})u%c|d7qc0MH!Bdc+~L`^s`@o$D1E^U;=l`XTuasf6NK=WRpI? z8B`XFTRh;h|FNP8TxvG=9Lz*Iq}${0yDjf&ChY2C4ou3D#4sE3auKZ}`%29!AWunQV!#9Ou|JSAk35d{>^ZeQULR{*(sQEp+4* zO!`2!#Mh4ZaG99_E{<}krv?3t`ir?#J(==BgPfn$mpG8wRdV!yF$KBJ&ht>#>`w9- zUpT&;juH=YjzVk+-D^gXr*Ut}rT%R4xeTl=`?&Bhnsfn&t?rk<(x9N53tUV`duF*B zXK-6>S=z6Gpz;r$~u-Oyb&i`||yTF{<8F{O{7$}!5Neit(PqTVYr=RNfPV*A=l{;-? zUZH;J7^mOrGINZbK;W{!2#4_Y_J`H!9R=zI`2=ujq57WAgWgKH&FV+Fn%qS{*%F{b zlAnT>{k|i&@mDf{enD0bB!4h~a`lyXm@ANekrqd8n+n0D<&-;$=epF_FWAndVC`YH z{ymKf)Lq{D!XTXMjr!P3!Fxb*xMy*KhLbp#D4ki?JeCdCyRmKYt_BG>OG!cJraAp`3Qi$! z*vs)%^iyijYPe7#S3fs9j`=?wsX%_5K$=LW)ykKU>j6i3CArRSl|KM31ubEdZK7Z^ z^&Ri9|25N~OfQ{iFa!^)#ZeWx9!h7C*LwPIf=dO|58aq|$<>eL68|F#)R7u|YI*xz zbj9XN>gzS03cdxGg65s+h^;6-9(kl}N4b+?5f!+d|F_CF*O7FTK1c)QYVZiS1Q?p( z^z{hXxnOQ;Xtq5WZX1qFJEq>g5j>%Ac5*D^qDrYq1|mzn3N zuUz%lQ(wPup2h1AlbItbI7#rE=xQ(oTxy`-OfRKiDfN}B{xa(8H`w)n^CtC`tNy!T z3iM)81s~Coyv$va?)Pa&%NEYxzMq{1eXewQ9qxebEqFyfts98g9aMd z@LQNBjS022e*D;D5+1B53vtYU%Bc(Lj9T?r$2!O{gr&}5xoDugN_cj zMT_1KP)A3DOW;)bz73Mj^=49Ex$0j{{YqZ`uBTu<`2+-(RIKx#{>REDZm;R}vTPz9 zM6L#o;-<9aNhW!OJ6kG0ocdC;j_GJ}Z3&g-%F=qdf0urWb5ur21O~#Z$hlNU3ficzT=oB= zzWg@5Bn_Nb6l>d^fbz8U``^}t7he=flCGQPI3+k=w~(cm8<^qRv-8OD^89z z=y$xRM7bLDC%5V$q3#v`S*zxq?i)#Sp8!h{wuiDxW`#yjrZ2N z%#0i(AEKPt9B+AwDfRrHHO9u@T`-A`^!A!Mx)59fkGeKsQ8fjRk?%Oi@yY}z*g-yH zrQ^C6l%L@IRIlXxzlM(fpkV3a&Y+3BI?oxDp69rpjz^v-3Mpo}SF_XO@Utw(0}mFo zef~cQ0y}yqy8s!i!AkOJZ0cesHoM5@Omd)+{FIZVKzIbhM>(=3sgPVN;3)e494kmM ztImx*PhTE~&!QuJ7^RM`CD#HeF9ny1O+UvGJz{kSnDGer{x_^lQ(g)P9{Ll>&>j*sU6c?cK zB3JThR@;~-!KFo&6PvZ*h@XPr&j~8C`9^E7JAa1(b}<0YA?Dn%VurIHxjHMhJjK*| z{^vw_hxUindQ$dj*;nhhHA1BYkSEj%I<|!Sb@}<2sj_E#z~rVjq(aDRBA? z=Q$6RbW{c|EvNd#<~Hhwx6F2qmb=U(+Q(%$*^sM2cW^1t+^e19777+oU%BevVR?$# z?zO~F$LUBP604)tuj6WcYv+6h65ndTL!yS){&JMV2d;)WSmOO!@~N$15j-OxDva5~b( zC+cS;^_O~nPWO0w|DQ!ix>Q0PO#+ve*z$m@@pKC6=xCD{U{f6a3H_@dx(S=D6zCEu z4g4(~wRi#kh{LzXj-Bgl1` zo$_NW#~lza@EIL>d;7!cY&z11#OmmLaJ!pLawM06m*{Ak7vTLkd^7z|qaSf&b z)fDKM%(54oSbo>RK#UHxJjt|j)r&AmM^ay(emjaUu!BnlR`YFSTPRpL-p0q|@G&>I z)ZiI9(rLJIcU6q*1+T^pR)2T?KK@e{l7m2x^zkd45XuG&(YCwcrN|u(k%6JGP$mlP#!*=0$uf@ zg0sM-&zAED7^2`la(zQw7I{y5fzs0`S3jxX;%C8ij`W~lw9^k8U23HUC(=M)IhRhs zzZgKd>c2vLeN~-anw?;;6L4A1L$&IUC3lB#ZGTA5gg{EFFAmhel{8SUjvk~w-a2TJ z>VHjr<*NT5>SuP;m-r*>r4d%6QJ@CLfJ=e!{>7L83#qSM^>3j*-Ub+>-$;Gss{awV zz5kDQI>s36Z!b77*YgTA5L^m`Hw?z;&!E0?4KSNr_j~2@$gxnVL;FK|gB9S>==rg` z-{0f#<#eQLJv6|>gY5d=5paL~WIM*3uX1I$~`F#vYqsjGsh`M+D z4P2(%%4JK^b>upqa1?h^!*rC!l?+`tpZ|nBb&6AMqQ%h@~%tl2`76^{R9O*Ut@VRfCm02^+$Pr-ipIF(GLq0?f*|G&?S=^ zU@N%PtcGvBNv2@vB>8|T6kAC34+j_h2~W7DlpjWubSd?9^{bJN2ql) z$S-!P`v2eLl^)+{d5S6UT4GBa{w26w|D&(POlO3V=gWtlMme!L9$X5lZ?T+1!Nb&7 zuKFvfuP@`&JEFZNiw2&MY~cNWHQ0*+`C>rFBtMWU_X(zXHOr2}j{=v1>f25Ay5Ktc zR}QDH|I28g@8i@y{s#?|YoXSY>zZ!m8|Y`(tF8j_V}z0t?HIug3UWtrPiYF`OG)t( zf6LZWaykrzJRIx$MFLr$C`wmRt?pktzsHK=PuZ7Cu@Y^hh|1bmCDEPO82HQMG zkH+Co(vcQO{j8;ao9Abv$6fow>K!`LmQY6@f=f$OZ+BI#4mimkQ^~m$JC6o)z-4gZ z=MP9+HlJLVZc`XG3+X6lvop}k=2xu&zFaan_FS+z4*!Ay)Q?7Qq5m!PuU)zCg`$D` z|0+;}0T4)y7i@6_D(LSd#nvFoRsRy|Z~VjQZ%K5Lhsmo02^PrLZX|tX{Un*-kvONc zw?715SpoKYIc74J52V2~&(TqF_-Jq`XpX$63#mt<&CEX-K)Iv%u?OnsartZ;1>yZKkx#hYuIp+k z7zi!_M)h<7Mv-4ZedWYv4)y0!-~JRYgwIl6x$3WT`e9RafD15#l3(d)*WQlvIifiV zQx_>vZkprzeW01-75yCF*u^<|i#%_jA2)}1l0(V+Yw1_(Mqx67TQ>N6Z~q@z6YIpZ8) z51E_6?bu=$(7m~x`a1@4{f$0qd7lOlsIJX7lWT!g_yrxU&t_n@#2^eNX`vlQIKGMo z7m-&T>G)z@f3%8%p*c=4iB0!AdEF?-*V57GD_nq0hdZugrIft1IN9-Z9$20uZ#mWR z3`Y3Vo-D9#fHl^@U9WO}*3!S~7lPyX-_xi-4bBlkhXN^|Os)k|ei6AANcjwM?K0)F z$z?Z=wm+ovAdnjGnBrIGFAemy+j$fu+lxIcQ1tZsf=d8> zJ-6Y3Yb^EKIsKfe`U_lv!g|rUPz@;10^LG`2J+%Poa7(WSFZZ&!DZ!C+ga|wRlJvz z$S(3@R}=CB4~aA{KI&JA7zNqzlk4ZGJt_&xQNtNu^aPcC=< zU0l4+=Wn8c`M(O(;CKk6N%aRz)Zh+l5ap`>ck1UbKo%=dPkrU8|0eaj$f7bC|Jx|p zX$`Op$D0H8x<(Xs=X+Tm{cb4Tm~?Qdz^b=gjq52mhkW`gj#n|hSwcbW`%cj8GV?Zh zvS=a|lbr_4tZ#!<9unBIqO=d-*AB|iFR4}wWz<8wB&7Pkk6A{uX&<~ z%W1DA2E^fmEk{?fpe_tv8%q7sDV)Jjkk`(=`~RoXk*?WNN8`Y4L07nfuBPBoI;!>p ztc%0{P5&kI(|-KjNQ3pBqb+gxmvp2}ss;Uy`VF3+KjZLl*ZHny`h$AgTm?-rxa@w) zHNYj*pVaCU^>FcTa((G7c6XZ&T;Tk7dDQ9a-ZUQ^6P&PF>CJE!T0x50=GEYyIQ)Lg zai5<7Tv&XUoLmc}{0VUB%4U0|g#E9QlCIY|fxc#2Z zXM!VsbVZUnDq;X#(Wrb1xHMhrg^uJ>@EG-#tNv5epFn;20TD_6p+30OKbm&XAaA{E zB3&$YiB-e1Tay((>G2bwqYB}!z z&+r^gp}^-CTaNiZkL~nht{~Twjq*9is z*Mf__zF$1kVY8C@%8AW0qA&A*eF1rhl4cqxSA%aDKwnheg@Uvj)8 zKtsv5P&$tK!lUgE;kgvJLu1reF}WV;RB{Qq&I6QRX*s@rAJf$M2IcPD#lMA)^cjyj zsvy@#Gs+|2wvYM9CY^$f)Yo4~Q2lqPufINZF=RyEys$8*cJ+#lR6ie)96UQ zey@%uP`@?Sk(nNc&!ivyy1x4Po8_4Q>-X|ia2*ZwO$W-$7+{wdxH1lZkbd;d2zoT{_B`8dP1Ie-(#sw;T%&zl!bSpTQ+S7kjxK{qDyhiyWUOfsqRE zXUG2x2;|T>$_rc?hcB`m_lCcY4SWX!wEf_Cp14J7roO&yeC$)~Y}ls`v5ZPtH>{(mDzpz8QJEjrSsR!I}NJ{eX1Ik=1o zeG8{vD-O6#G@{(rBfK@LS#_spW#^HH%l*~bA9pT@p)KByLOo_uU21k6H<+#79 zqbneg8jRwBB8!4|tfMGb{SV3YfTR3Vay{TE-$srjV2AdH^m_{Q;!+KM0+#|c@h3v` zd|$9wG*Zk^ug1l3_$8J{{qXpQ;|~N9K!1aT&lAjZG*GSudXf72lOD?{$hloU;2zO0 zv7`Ms%dr;JE9m%e9Kl3zDUkbZ5WG2<2bQ-PK)Dv^1NtfS0(={Xx6*%De{-abo({aj z)j+uh7y>Q@(%&Z0E;x_+%2j_V_0!v2OJs9wucJPBwEzE2gT*G%`pBWD{qA&u>&b_* zWv?eM_|t)Aw$M8Aq14xcCR8{-GsrWnf5e{*fjnyYz`M-8$nq3Z>^14Nad@facr?QT zS^xOAi2ncd{NEv5=Koq!HTXLO($xBM0+|$SWdP->|1G)JOnEE$V%AvsZ{*UV(e{V* zPY5J%Yl7>7Qa0(4cZq3~tD|Gd_0^5a^T5SVNf+lQ%Vp*@>MMu7i~lwa3iooFWenV7 ziF1^j>Uc8w1oFymj^}oDp>F|~f+}~^m^QhJuYL4$Re-SSnH+pr|iflKPOV zzN2{KIJvf*^8V!7((V191%cFHll+bfQVwfyE)A5cqhjj!*vIJ?Q~w_F>RC>+gcbOn zepH`+!saIk0{BhBX;s}UqrN%z@3M)7M07+0{AiVYN->MQt|^Mcr+`bvX6)+MK|MsnlPP0exxzIg~t0 z11nH%%_A>bCLfZ_+@rX|lTWc6Hype%F*^>Q2QG1!9v*g0x{HoJvj!q?oY;ItgN(ns z8njYScAs<9I@;-1k^cu=25H)~*eP9LxyN^3xD>RROB<3|!AVv>$~C}+ z)R*Uil4@OMUUWQcwybkax|}=yuhtRXfb19O0kPbj_qH5+0lR?y@e5mQ(c@eb=#q|$ zsb68awm+<{puzTBXHdl>+Iku&*8m%+zm{*I-NFi_KIj7HYQ<&4btr_R}&qjc>%V^ z;Xl*Ac7gie8D@ayGUvIG0Zx9{1une71s+OXNxq=Waa7-?kH}YZhi{?yplatQ=Vk|V zmEWu6;mU>1a~+G+^$}-KdAs8!=}vGwd1$J0G=hOk$@5bkFWtlGzf7Lpg@KbC|DAk9 zoAnNI=b@-$G5RS>S!1gM1RzU&X1l&yFKZ6(T%P^#SHug zdGbA7-1%PxB~OV478-b?x-1UA4O{{*n8!YG@y%wd-+?pm7t|khiBrUZ)9rtUJnbCS zeB&H(1*Vxz-tvUw1$~{TX7UBsJFZK!hOHJk0+TbYIRab?T2tx*>)5(>wT+MGau>%Q zC~miocIQhOK=&FAa32G-T;~GlS#Br&m`h^)_j*PwaexYYg1xK&-_2m3X@GPZWP1A9 zarjZ-QsY|s$>N~Ci2jxH_=l4a9p!leO5^ZFbfgC^E$AKK61Zf6Yl$WbJ|)j*3spIe z_kXUDPq-bfQxzvLhm-4ph}et-mjH!aXRr4It|ec^y+FUj`T_Z*FWgwsuG!~V@q_vQ z;>*})2`)sD6=3(f!g1S$5Jkv$kT$l^p6yZ~TasS4 zJjLwr0(=^WZv&UE*+xHIIOrDHHji@W-IzOSZTu9IF+FzoSYZuz=a18o9uL&fYH$g> zaX;5Wm0g|WTk=tT9M^NfA^&vxnZI%VuQwKEP>_-1464{Q50RUJj%TE}06(h!QI6~B zc-T4@ppAhGS<-?R9N&JR)1RyR-|Z-x)S#Gx#WZ;NMfrePuh&8!#^Ik@j%PwVThfi$ zW_gl%!@F_{e@{nMUVy*i@Zi71B^Isnw5tJnf=dM^G`qp2m)Qm6L%(#KhXpg=@vtfX z+zCqP=?faPkxwE&^CcG`lYAO^d7b0yId4!u+rg#f>b#1Xe|O~C{$Sm#05=?X{uVV! zy~y=|<0yWDm0b64<>};FP~``bYlSVB_(Ler8mqx!R$GqS_LsWf z-%)*29~HDS`S2zK==?y1?~!XuDE}B-TB!U-*FueKko1>DBg%RFGv+`D#GvR`HYq(_ zWDRhL^_ukBIJ}er)Q=Wu5qais&cE_I!VpN{x+0YqkFTA{_93^{=YHNiQc7w2s)-FaxIX`KBuDv zCp!HSUI`6&RWvXSCs+N!;8L;e1x~+OzkggpLB;7#pxK7pdg9|3`t&+tuJp259gC{}zY;Nk2MERzF=g$|o$@;;dSE56dzC*Q-$#^r3-X z7bx!sE?uy3zUzV-3MO0q6tl%E=&U&WTFZHT(cb=WR!V_hU})e);1alawhKHx} z*9#BTe~sK;U?@cXwsQIrk&XX>6(pH5?~u8L23lkF^(DDp<0=2na=i1+Yl%PO@UE}h z_;}LEi$ZnO<8}FjC6*rV11>F5b-il|y~dkKp2O>VodZ7chWNpo&b$J>)Xu{}(P&Sv zT0t~`3f`hY#$~bkU&P@p^rHn*KR=S|jR@ENkp4=6-tSX`Kgiqf_fgR0P1h22)vhJD z*wu^%ml|h#1sxZMhtIVF9IJ0~jx=yFx!yQbehD4rc>(6f;S1@f{k9t&-2#rT44cY` z3#@mmH_*{^FTh7}_-7h`cgwV(Ur~R7=jVT7c*p(EPId4g!E@Bj@)T3)IodZ4A3#5P z-%txQm|Sn)SuXLj>8REVkQaxaN=JI%Py>vkeuL-d;yC;YaKy*ShIca6(HsWQr(MeD zgUjw$f*XgDbk2C}Te6j6TD-t##^FVlW72z1myYdk%oG}=Umm-g&5pz8(UBHNiXrI+ z>W}jL{Czjh_{-@?2cvUi%){VP<4O0s8dtJhUsGSX>i>uOJ05oWwjm)r^li~dF_XLk zoe+*A7;8D6_tTLEIFnrWd*wyoQqZN3yTGX-C%KRM%2j^_^}{vKI0OBr#J3boTN_)0 zgWnNT+#B|4a9kXIlI6JNO8*-8G;;cn_WuM5^ihi%oChv7OaG@Ukls}KC-s%9ejW8Q z`1mb@f{>juVKz+eFybdspiimP(ca|x98dXv7dRKuGmzfgL zm-)X6)ZiK#wA4AxQ8cJm{)z))FDdx03y?`(@31)q+@3}0hu9Qaj&DMsE2HfX>9h_4 zRhIm6I?@0tn*}a4t9{iKNCQ7d{T3ET^-qrY?~&^Y2jw4= z>$(Bu&8q*pE2#2s-bH-*(n-~-u8sv4xu~F1?0LS zLiu=bX}OBSH16R;OwnQcMl~dh3Nd3AGp)c*9PDvvTtU$Gqe?)^VVt~|Y|KkS2hw=g6f+r_7 zqsjFrSCyX#E(I-SjnWdh|AlFw0yVhVa$H<`gJ@YC{t*4>Pr9m~8gL2hzT}EuMn+&s zTR*h*j|v<%=354+^#cD9hj;l%ETg}MtA6$bm%uq6yOwKa;2G4<`8ezZYA~AyTNXG{ zMZxRV0M7%x0(}~XZ({)UqyE3A|FkVGZk_IbQ<_8tzw1MR8eD8S)`riBoe|B8!*2kW z8aL6;S`Nmw^sij~hhLzhG%vtAarlP}u=q1q10B5GK6VvQt^s<1OMz;rUrfP1PCwi( zP=gE_gg$oxQYk260OhJbh5G5#Z(twaLw)6{f4}9p`D`n!?GNcQF#=V`iX}SIH<+sA zC2*-(%NMSoRlJLK+-A{;a@9Y{a@hL5{{G;0{ZCP{JNxOA%L;3N2|~w`!m2tBe}Vzjj|Nx+E*038;{2~? z3#D!m4XlR0(!2kk6&0utn;&X<)R7901eX9+d%FO7%zu>n%2oeK>Nirqj`I%_`czc# zOT`qZL6YTo^Wl}TyWfFv_z-X@P(@!?AYHzDG5sr7|5t?RXpR@)hB&;Oj`T%j>gZ1D z*LZ#&j>A{dPgvjirjDMWL4)V$l{mZsTxy)Vud8td`*{D)#AB4J|AFA5-?+ck51Z+f z%!EKzG={vjTyI-PDW=sc(AGHoTg%%6Xk61uuD`J81la$6gFs3&jX%BEgC@s*E+3+t z*yMqWehKy0QgA=@m8*WW<@oaY%-AthD?E(dz!$tkNBYuPb@VD7<$I1c$KhXqOF`QP zxvt5eqw|{OgX3W`n@w$Y25AR5gM12Gz{Nm0vH1~P0u)m}je?Rd!?48f(GgCzS_ipaL~`gj|1WS@|k*eQ~k!XD#n*3TJgwWb<|8(_?wq{F{RESb^C{ zUgPn1$m=}*A$gO>KP7MV_%_SKeNAX~Y)QYTL59bFBG2*o|Bx4Y+C` z@Q(XmFA6F>gT2UCdpwNz@qe7na7ktZJ)8~70Nl#X2D zkDy?1M*;XTmiIO0Z?S`P4E2Y5`lnDo+tV*3FY)*};E11OMtP1VGeEV+FCu@;<1@$~ z_xNnf`0g&Toc=J_gkK1K5s85<5$xl z%`)>Hm}Z6FvPp@);g~Elfwbo};(Pi#`4U9o^wM+CpCK@h|D8 z)bsNl^~2A22LGi&m1odKgHJvEzsP^`c<>u(r@m&j=clLTNoK#}?cNdX|5O?@dj|W` z;ACeYUTqqnd`#?aHkf`|JwMsxp}F1E1#-kcih_(70q*sZ=Xm@?@s z>gm7njg8;eRC1d$msEPbKkAF^nd|qteugMcnisk5L zM7C2xi+sRt=2c)1@{Jx(wmivfq@VWw?@NPb&tQKVeBlKiNc~n%e<=0!7nNB~b2z#F zqB42djHKY2g4h}#&j4m#H}zrjeDXAppFy7O@w3SbJ$}CBnE#h}f@w6U@c8AHCz%hu zg3hA;YES=K@;Z-~($7!gr@j4Aw?!2E?ghAmyvu2^E%bNtJv_eL^1i0o3;ZzkGdt+p z_$w(mq=P`(`Wf;(kJpk<_V`QWGd=z)`CN~`)ne-8)F~Axxz@gM{^z@G;Z}E8eSPIO7*e1;*&+z!EVUrk=+@de~{9>3A@B=en(tL=}bzLf^ep21z@Z62>A z&%Q3U(me@}Hr#=0jsh@Ryw+`)(me@%_uIDK59fT-ok;g-p zCz+4D07=v@@$~nmevd+JQO4ho2IZbXCV7>|52m9d+Z{;(jTuh;T2KFI@D~^3E zUhMG(^0^*=Uw9a!y22A|ra_IzzaX#kcnf)x$A2Vm_4u#kp&PvU{~r`&K+rLD`ChjA znjDYsNnYsj6!PgF-^cPKb69lDx&F_fL8)hO5DhMi8rX6TBVXz9Bj~5n^D~NkwZ~8R z-p221HhO}wG-&qtndEIAFCtG~7~67F$g@0tvE?`kg}oYFK|!m>=g?8E=V(58k;fO3 zmw5aZ@^bR_{jY+8D$gK7UhDA(EKf3t<6@5=k5IqS(_cmX@v-`0^DGToJcD)Q=Em42 z{Wp1r$2XGadi)*oVvm1lc{s`3A6p{xsTCxd6&~M4{*=eRC;z9%e+D(Hz0ck-7Lp7o>zlj~_xo|MF@)jt1qP!9?;Zk5479_4sAvjUJyx-cshBB<4|IZi;P*g_b9o z!1&nFek=9UJpCoqPw@1YktZtG{$D{s7Zu1*Kw3%O)#Gc(gC1W;-p%7LTi(}Xdj$=@ zK|#JJc#pi;S;H ztsd`Zd6JoF{cHQf^#BUydVzjG} z|Bk%X<3EKd2rY^oL^~}{G9j-(2|q}&l1zV(cPAg<@f7j{J>HKzJkS#ipx_{n4nj4Y3z~lGU~TbzrFuw(ID9~m`9%O z@rC3=Jbo+rAs$~sesnAkn`IQ_#|q2}^0Pd?lKeuCuOYwKoyxWUmr)Q} z7~4cE$oKX5O7bH-zJ`3Z$JdcB@_6`V3SRI8Z;*fE@%PC0yfL<*o5}a__*U|v9{ir$3VV4|UX+_+w~L;u(yk!HSLs&>u(r3QvC`d5y=XlGl0sGH}H2YnnX4 zEE=?We4gb=rb}6D-!G(o=(gAvx|KY`<4eeMZu9Q{FQcH)Ggv`B-Qz3COFh1Zywc_W#QiY-|^xosr(K9BVqf8o#If=Gd0lth_vyZzUggODz8`Ou^xv;3x7C9^XlR zq{kC}lH%g`f<4}ye5A)y$it&NK|cycdwc+SuE&RvkMa0$@)aH*N&c|o(fw}>1=XIx zSn@|aK92lRk543j%;Qrn$HT6eE=K&zC}@ij;JSd09`_v0qki(@*wMa_`t3hpCOv|5 zD|yqcv3MB5*Qd6|MnFTfjg zbhGE^J@T(SzL|c^?XfMimHOXA^cFi-`t57ZEt@x(RUQ2iJ(LJiH@H39POljwx^%4LyVJ5lc(RE{HqS!#!sQ3wSz!h z^dtY%;{(Wh>U%at8EFW4n#YHeAL#LsJ8b=v%y3UIh6bZNK9>AckB=iS^7usZOFTZ6 z{HjWC{(l(-H+lxM$d`D0p5=W_zSji{$>)0fR`MF<+W$)^s8fM!QM1hQB=dk5cm?&F zJpGmAtsY-P9;%2PTj6ySWO#y?El)D_Uf?&VpX2GjM_%ah&Ghr5=VzH|N6QX_$Z5Ozq=vKBs}thBm@#dl41xD_IvLlLU<{lATPN{ zk+7THAcznGcoCE!zL0`EtPV!3fNfQ*3TR!#tMa;tv_7D{7SUp)6)7qLRY4TFXJ*cq zWGC6?_tW|IKj$%X=FFKh=UWorcie>Vc3q)Q6*$}%1ifp2AkD;%#fqRj#CI!r5h~cE zC@4evPZfF{>324#=lc62LwSAtjE5k;O2J2=po7f?ft>M3|C~ZU1?d|U`Y_<2AFfDB z$1D`opa`l({Bs3gjQGD5e7V5kUs$b+ce<5G-}wop{=Xg>3KfQWWU!xzx6l^E9R=Tt zasrB+9Y`N(q{sE&g&hkUJHVdsvlsCP75ot5^$LCj@joc|F~na}@Dp{|@wU=&8u3pQ z{5;~P6#Nq6|5k9{_k0@g9${m=9&d&sT6A@#iJp zwEy~J$9{!j2;v_r_$b7`Q1J1HpH=WFh+m4sL4O!KGM;5!h15AY`Y!(et{$Hxl8Uc^r- z_#wo9Q1By&|ES={5cmD2(b7`?3GARv9o%g_rxEX<;O7y~Rq#s!hc_Y0zV}_=($YL# z6#CQ)LO*;LLt)4i4DjMX!SfK`rr<@Wz*H2JA#N+Uj&csxDT4YV!;K2V5M*dj=tm)b zQNhQfoZA#RQ;mSCBLhRrM<3FU_b_zr=?_y4yk9lMa>pn~s3yg|VaA%0Q8k074@ zNc`z{4DnKqH`*UeeF8fw6o%7?Pgd~rh}S6iC4r}TzECXcyT}(b$Frgtz0jYE9UGc; zaMDbHr+G4-YTU0J_T(Y`HiflelR(>#3@1^rR+1O*?0f-Wk8Mj`#3 z3jKJbPycoN5==q*r=O~epII0gN)?7#h>un9S`_q(qIfal?Kj2CUygXGi@WQ;5<7;t z9m3Ea&w9j%DtJBOlN5Xl;!_oTtH7ZGqD-J}2Qq{ehF!?;tirGt>2Fi$4`j;7yy!pFyd`Xj^S<_v%jL41*dk3xKd zf{#c10|lP~IOtFFR4F433kJ9%DflcDL?4TvaV^rXQ0NyUeL$gKj`Xi@R`UOq$Z)H| zupSvsDGKV5evv}I1?ls*#4qJmq~Fj;kL$k!JGM1;h!ybcLP6scL3@$@1BLz&(oafvlaY2;!70#65=Zr+;@pjFwIl9QRzs6Y0_}?VnwEuMM_@~0qAMrB^J_PX}6?_!pd<){wDEL;y zUk1F%{xFyw*zvy#!!E@CqTqWG|5U*bA$~@|k09>-ZR65N{l~DQeNzW_PtOU&a}@kE z;w1`x9`Sw(ehG0?!F@jp{hd95+W3f;`Xg7L<{7LoWC}b7{uop|eIC+JQ0R-0K3$^dFfx3q2%3fT(-rz! zq)+-?;~H~QdKM#nO+0;_XE`$H3d2fdSQgLVS&#J775aLl-=NTMLHt<--zsp(|92}L zJCNb1g6~57tb*?qc$()SWk83Jo^FWul_LVjKX}||e=yN8WXM$*P9Q!)!A}c3&C{SL zK96|Y-^VY(C6u$MnH-_tCw}c!n#Zp&qzW7^49yt$T4o~sQiVPb={G3!MM%Fzp)dPM z#2@&2t-_!q!y$#CKPuR*&<_!KmWL|acofPxqsSSLa*l3LzW+Z38G_Hmn=XtDrxeAr zkp6asz82{{kH@e5Vx)iErFYkVIWpub3@ef06PJP8AAZ&&eW^lUk9dWGZ$UY^&&E%z zZYy^5QaW}Z9#HUIh+n7RdlA1$!4DyRtAZbqc%5g7(s2wK9#ZfV0&n_@$I=c^BSQM8 z6#DZ5hi}Izmbip?jlf;|gEZe|E;7yYe~O?~fx|9P1Z5(9+H>(H%|kq2!HZDNrsi_E z{xZ?g^jB>8F!+y-f^0=lf5e9?_z;12@|3E(0CB&9j|Uv|cWQC^O+khVg&~ai2nC;o zxcWQaY6YI=nXC+SvA}aYJ0DlR|Faw!Hf@bJ=}N>;Js*#+M|{;E;_-S{j)HG-@fTcr zT>q`u;d#;RkV~)w@lplfh4_OCzE|LoA3PDi?++pUDuwHP}* zX{0}=(4R;83Wfd>;&o${4&Tpw=4qbXm*Opviny)dnF5DzN-J{mkbba2UxfIL5^vgn zW!Mo_7<9yI6}-QTZ;K!35P|1;DmKM0!6<=u_0+`Spnp7c@LxRYGElkno;wjAfeh)0 zzbWu0LHHY%J?9Z$fb=e2aYg=->O~43;&>hW_3FE@qYWxpiww&UcL%f|@f8Z*4u;SF z3ietBABy+}1-}dNO@P;Novwoa#g1nchAgjCunqC{IM5M@??yZq@h1>Jig+R72N1uA zc)+cHUcwIlX1Qh+$k5ZrjhN?|j0^~S$_3udGsy+8wGr>F-~$uj*ZS(Xf^Hs_VN3#s zNeS?23GnI!_#77(_HQ=YJKT{%`{PF$Hhb{~M*l)&Khh_=^efR}mQQ~ie=qaG)za`j%_$oYVrV1_adz9~Pnks*y@JDtyy^9xO_syd1+#0Y-q4)yfH;Z-PjZ^Ua9T&m6Uc$8yg zAkuGB=qCcs57g;MFAfPl1Su%nqb9&rf}th85*1uj6u5R&6YR%?oX#EvH&AhgXXC#3 zg>?<;j?}f&AKS3Jr%Cr-eOe>W@TA`uZ|A?YVJo+gg0is*=t~v)3vJle{~*3XA@7nb z#cxX>ezHP;O)~2b%bsZTH41qpicjy``fOJy^mWO?t9m6M-=vT~h2m=x(C<{}_aObI z1oR&$^q(XB2MOp;DfFbRwB}m9_318CHc=;}&ox@7Kd8`WA$@5A`UZs_HtFf#k%C^| zs(4$zi!wYa?qQ$pYnxNo2fBHTS%yyGN8cTcc%gz%Lfjpen~wh%;_mcZd>Z1e)4B2& zO1#eFx=Sf4UI88OSG`+o?#EG(yZ(KU{zb&yb#w83h&SK_T=_=<_j=A<*&uF8%?_T= zpo1$`@9p2BpbB&hR}dkbKHalmd;Egt0nQa{dn*3sbDGW@sqOmD6LSZ1?~=SfbNI$+BLHR;yHDQlrGPalQrj}+pvq?J{s7u+(R7(4sPY=m33p7li0+Z|p}U8# z>GtA|;vsGiUq??s*YjFGy$JF#ZtDBk-ltQj~IG1V27=MW!llua8Jpi za-AK1k6f3`9}VVxg^X+iPbNF}XEK0o{3Gd{Yk964I7{BgX`A7G!oO8(r|;oTa`7)E=rWv`GF z_V*a+>(yAhoh0Pd*~S?81-r15WU;m7l(5P0LU)AM^Xz)uA5k*Ik#3>(V!8G zI8h^H+tm?UOi45OoJJ&RL7Sc4Nv5^s$~B8WPxl2(mVTbBWuLr8GFjbD(uw(Yk?CHC zeYA_*+f@s+n1yDDaR%9czfQ_vYaQN8u3{g2M2ccxzD^pvY{DC)D%~^;)7Go3h~)&T zL%PlOyh-M>;G3i~JMbpCmp%495IKH5WNCJ_88)l^!D@@0`wrB<^c~4zbKfE*?8R3| zlzsmlNt4)w-K3ChzKWWx1AmIVLWZz&2f!YS_z0TlUVDoaCfQ~s`yfUN*z&>;;GL34ODmyoUTI}#1(rIvT=7MPLg1K{N&)4*t zXzlF_7lanfoI7W}W(=Ksd(G_Vg6IT5qE0AsM`%t}6c)~`)`G#1rbi=2bvSIWvs~7+ z%Oo?oa>^LnVcT9I2772C?ZU2qnRI18O`yHlguO7H>F<&0>A)EYhD@_M6bkvx>Hr^# z$=d8AGuU?rxc&BmYmVwbzl}xqlfFI{eH(mf+}mL7bqB~V+1Udm6kGW=sqwOte+Tks zUWV@T?+~Nhn7<$rtO?DTS#{gYIWsgZ9Gx+9PNe`~&~_*othUXFp;=YIprx~hLm+bB zAy|%A-XX(S|96Pat{fzm6#MbZq#v984jI~Pv93Qv7O`&+k#TJL%Ot?oy-G$W83COQ z{u8++w&hP`lb5Y{mxMDykt)jyYla1;uC5A2hI8*Nyr$94<>k%oTwWe$XWSfmI1mbk zonRzTt%oAfAPc3@EMjtZ)-`6mPnLM~*!K6y^*&bfI%!M5(IFm~Z2DhFFShdtnHqce z2)UsxD}SG~?PYl?D{B|dUr-sEJ$r7IptN16HrfO_EafB8-KPgw*+(Sk(}R4&7;M8w zU~RE^vbanAqKn2w7`phsP;9`!0I_0Ij*<3W3c{z1(RG7u{)GG=Ye=G5Y|71)p4`+%(w!PsxBh zv&Amw+M(ZKc@1PTq!JMS3T5D#J&_ZpA=lbr1$B-uieEkd>pkr}ukOmoU^pOb+; zJILmK10KHVa~Of2Z9fD1tMDth^jsMLVL9m}=?Ug8J4wPF#ELp%MIDnpeiBx|VxOEO zw}Vlk8O7F?(KPn-7bKhcAD|Hade9E+*cYTTb;OXJAiMAdxt54A2Q)V96bZl@ilMAM z1qXzI6F5b#&Vnhy2_q&D5EBU4vRD5XvA`a;{)fm4~znbmpHD1LRL_iRrC8L%lfa# zUe=?Ic4pJs!%0>1H5mXS9`iL!ao^YE7LeBC47m=Z%{)UYz_CG^=6AReBm6MT`G(w> z;+G;dKezKtFeWs^S;1LSLWD&%jdkn;2j#4@WKIRw4@ajaCa!5>=9(r2Yhv-V)($ei z*aMmtWIYa%GB)5GDFk!ea*pIlb7&BTjzd7(wg-HF$2oG=YsH>EPu}%Xv78!&24L*X zp8l2`Y%7(rj8dAFtNFn!Fd$v()`gKY*!n`xf$vBq%Qy#&W4|Xxny|bkcfMvAZ2k9S zDVQkr0vXSGULwV8Tq!MQZxHaSN*|@{mlsHJTW&JL;qo9IxIq5o4Y1umkS>`gUF4XKMOP5qt`JL)WxXK?1IFXuE`ybRS2KIv}>b7SLMCvgGpc$-?{tMk^H3LNX?3GAL*PX|F)b6jighIog3NX*h@y z5BX?!w$Mu_QJ516SuLrH-ZFvFox*zfo2w%@QpIjo4T`TsNjpmnsklRpyDt!f7@AaKBiO|cu< zfn>TS*Yev|m92%VLQcdCI8`XrWP`U8n?2o@j$wlcjDQmAu#frEB>wb5J2;`ACN$79 z8m$gmh7k(tnh_0}c*D`;s_?xHV}#V_>QtInE>}PouC42)Fl^%NkDV=`J#zh4AgTu; zvPB}{pb;|U0CkHkT}`iHhaaFlyW3UaNW?a5(~dY*0o?YwF4s?Ym`>?0*~C)XyD`LF zm83(qRMCOvvxYlpd8}Uw?d}C%1&4xICB|&D;!xaUfx?guCF7}X@FP9Q&Lz>#NxEUO zwP)cF`q4*Ae2__9`3GeBSw5P16}N*Shh(&}DT2}k4c4WBOl((f1f`K6(}Mh`GS%Sy zJX7@Z94M4-1f`Sm=)_?jG_z9pQ3>ONs5xLgwQ@sYr#cjF!MbTO?Ey&b zL>7=mX?BW(_w}^Gy`nhRYP2RX2kzb`RfCTq_S0Fciqc)cqwk z7N?NwZ?Vm9lm2r|I&D&>>SBm_Zk%*wXm6 z$==GKIz*uJ8T2`wANnv0c@pT5v9*{d&j&b#v=t<9?`G2V?c`wLU?^F{u3RCVQ*_7_ zIGw`=<->-0DvMV89D~8BH^T=BoVNfr*fX7I4k*4S2O{I~PB3aXzBj5b}c6}E**;gh_$+rM! zmJgjS9+6ACUcWgEXfu$*v4*jHQQ7`b7O<^X-6;Q3}XxE zxDp}AfEW$Z4H=IO8IKL_JsdO}=StaE1$02Ra7sf)VFRMDpbAK;Te{QUu#Uny&vvK5 z6o`VnA5u23QYZFvcY2ji)5XS|UP!YkR1_e}IvU&E8FI!eg*1h2ETq>`$me(jHyZIU z7#+4p5xcJkEP&m(RZ3u6{Iv%-Lx7PY`VJ^RTtsL2biZ7#1;sSm2VrhiF&*#I;g(uV zO}_uLOX!d^vDya2J3d6+ip?mY$w?i+CK3233~#V`-;t~gojVB!h5Zf4=AOKYR*&Y1 z1E3I*xB!SgEga7gC!OKm?OGmjI=>!p873QBN<;17Y6*9Ke_nwR4am(0r5->OpaP!4rRTD4JxAxs2n$hF229nnKB-7 zgflsMB|G;PXfNqaN9tULDJNn|r+~z%g--6~FcA^6Sb1N_Ybwu@cC23?S`KTI)(4h% zejn;5;#hV3Y}F+=i1+rPGt#-=1E-8rhNVg3F@5RHKG+Li_oY4R#QudMj+hVJ^jd0% zsd5=|)xhstQpnYomqEjlo-f0UA;XLTc?now^TRDP60Mvyf9@Ou9_`^1*vN94kI@H? zGd8!JzLp|S2Ezuw>_;zvONRU~$ANxY1TG2B7ED_}f8bE_oB26E&8`q}r`(Xqr@Oi9f5)Q9bbY^{Gv@3f_qgPS6>TriVuF+{7rF~>9 zf^xyCqfj678?-NbREHe`Q3;+8d3!w|=nDKjw>0@Y7Jn#b=|pa~*Wevr;6k*Sy|LzAfso0~EunQ*fd zF@oK*oNDaTB&czI8KlMOa*LWWMwt+!#Cdg7e^_3_&u;Hem*uw-qQr*f)gYZcaX+nM zw_i=O^Nm*AP~L7#2viV;i`hP)lOY*1mP=RDoOYs8Wd-8GR}QNTLYd>{0m4b=4WR!6 zH*N6MY&Pr~dN2r?3+!CeXvI=)m)NVODFTcsGb_{7&v|0KP zn#&Fhq&@i_;5`9uzo5ki-$?tgIiHe_%p62Vf}brNM0fivKdY#qQ+yDIybUCa2T5KG z(^;&7_NSIOui&tCnftJDgXz3%sZ`dWOc{kt$bW?Hvx9jO%b(YQ5@<@;fg!Yy&xSJ! zG_Y+WDS;|S+E9AE4>FypLuswgHrU}+uvbnFg#*=Ouf(7XvFwjf@qgpbB$W*xMoYl> zdq&ZoZ0#@_gy6$Ncpg<*_CR4Acw@?vbOvhr+kES~N z_w}?FZ~|!=yY2>CkhpskjAhJd8fIJgii{pjD_FNNbXPi0 znLwk=oJ@#OurR||*^P7_+dP&U3}lUA$EQ(}A}_ZlM5R7s=^R#G$uYUy0ijrneLR*< zZpSN5Tw;)2J(W)M1^uygQ|WlG53YGI8@BcbxfqLEMx@;Go3UR<=yTrUmM0>_RIV4Uk=*GmsIkFJJH-}M z(~M+Z)Z|^_gs^zI4L(7p)*>>>dyA!s;AF9JOW0o8BIdx;8bmxU+mOWy$-WC9^GyrMH9Og5orGYYi<*7S{yU zcOFy@x7E-)vxR$D5O?@+p?Jm{$c4C>*o1j>M1hPnmW->G3@?@pFBVkCUz$Q-Uw1JsN*gf#%gK0mrkmlQ}1DJf$pi$;3mdsTwnX6c`ux-J`NNk;-7t&&GQ%)|< zD~ksVn6cw9=MFjujzaNCnn7-UOKxpT9=C8A^e(1ZIie0#4pkMwAh)&!Tbq{!Vo(-% z*gIUBRPJyKZbdBy&dmWMk?Dvf(-BLiBT$SKf`bff+>JYk)?U`EEU~TG$~$R(I=^ay z0GW$ekc%`eYWPlCoW_fhykATQBDsGF?ey3Z2pnQbZ79BKOK4WM7${VQxJFynv1~|8 zc%MvxY`H;g*r0*|XWU&d-!)6&Sb1*=E%rgE{^l)Zu6{3kuPwX#E;=ut zdrY}4H?1uBeGp z)XJ8d)s~ypmbn1r?b}Dg<}Y3bw{~$d<*?8)I-pxi_u(;@tA}v?+A=zZ=L9MD&`CTe z2;D;~eUJ_8y$9~u7M{}mmeXM}>$IU-A__yc+@&_Wkm7}*6zC8`wwk@_nR_qIAtDql(zGIFEbtH=mM(*)Eu)DI7s^!+(FN?uU-1k$rq$HRKwf+iViM;u+n10GRe1PsDaup+qD{!d=b%X$QnLb z4OxQ?S;NRxbP)UeSG0@Q5k--}Dtd3WoUP0lY#Cu3c)QP)El z0dI(uJ2J#LP_E%%2jRaCRLps*%f4C5YpiP^hvAiPHvM5yLRr0r&SalGOpC!Y#4}X0 z3%CK+(3OxP@=_%my_OegWGUO?rSKc2h{J&U;Ij0uwRBCsrnfYJd_Cc41}C{)6v|o3 zda!3J?#horm;!t)1$5p39`n$8s-=nO;mCBtffseh*VC>%&jw1FAvn^39Js&HN8s)k zc!ZYmS86~Y&uj+@uJazDPVA*e=wFg$?iYkNMRz?8#USvvcOIwxe!-m}QXZ5==Af)3 z2W9vP@^b_FAUnMDxJY@2)k6X1<$5|f*7XTG-3voqu#x^PHsMKH*kPm?8@#O+=?k<% zFtp+*f>tamv?8==mUK*L#gL&D3kj{kR&AvZcEOKu&-gurKO=8h)^q+N{&qiG~T=e8;Id%xN_*sr!1Xw{Y$8Pg9 ziGP-NF=*x3ZE0`#jPiKk6?7;MUya$OGZs5gpuFN~HoncWr7OT^IeQytFQO?I$fKxe zae$T+8?l}KyIri}4O-~))mSY27A;S$87jZZRx^~QO7Y)Vr09#{SG~hO3jz&$*zNmi?{-JmOL4P0dfD22$@y&e PewynWvY8#*Pka79f90Vm diff --git a/posthog/user_scripts/v0/aggregate_funnel_x86_64 b/posthog/user_scripts/v0/aggregate_funnel_x86_64 index 7b49f9a90dd5be28235cee8890ba43a995cb3e7c..615934388b0275142af8e9f143b57d409ac25128 100755 GIT binary patch delta 174440 zcmc${3w#q*-ankCEd-%ViWH)92@oL2trc)vZl+LRf+OC={6%?v2!c1qVA{@69={*T5f1@C9GN9}5| z^3)7mvgkIy8wU(m`!vd7GHF+RIVw@VqB~2k_orSXV;F3*jAPefCas_u&s4DOFT6n8 zliF2%BID9NC)TH)ZXPl8`!{*_jMw*bc{A_ls#loj4Lz|mue?U|re=(~*F3rJsHR7| z?tS3*U1vO>QV!#hv z$NRtEfOkH?>*?gUV$vFJDc}uv846U00@eT@nU>!k;Q4I^ym%>ZI6n<8r{Idoq!p-0 zNl4-~Y4Cyn*`oyz}qk7^>0k#tbJ40MSKL!$pq?`smH|}Z#Uq>1s*ZrGX-wC zJC#C91#UOs$6r%~u$+*=F z0slzgEe2exr`>>8i26GOu4wA?rKqUWP(W+IwmOxhohyWyG~m@YbKGgb+XbGViYumz z7V{CysR<^pz$?<=H3s}^LF$Gy_(lVMAFCho*K9~QAR2Br;HNC%9qlmS0|ee_z^etR zP4%gi)ACyhC;8V{FkMt+Hx#%~;Q0ppa)G-Hcv#?N2K;XVuh8L4|F*o$M^>#%AoY7o z6sR@eM=$01*Qddo(%@|;;#~hbPE4ToOgmxD@b zMV&$m$_)i-1uv@%_=OMi4%HZNSu{{@z~>8my@vDgn@sB`@QOBS32gi()2*UFvjNu{ zZZqIo!}|@mBI=14ajl{*L&6_L0qZ@f6lxN)T;H28i4-jYEM8Dxv$pueVmEkk&MEtERqM&aQU8t@i@yA1f6#&!(w1%Yr1p!*I1wlV}+r>22r5efKPsy^S;)AKPmDz7;vrr z<^(SE&w487&Hh9Ji3omTWL*ZlR*>2jPNiIz~%JEt$ZyLc{aZ;Bp$gA`M=1 z07BGuaSDap}-+c z3)0|i23)JBBMsh_iYq3qfc1fsQ^J`BccsD0)8N%7#UX$7CrvP!8q?s-Y4G+mc*KD3 zyq?RJNyUIk{x$W>yMZU3y z4S1HIbjvz}|8EfqZH59)QBk`A-zZqH-+;d+@D2m6H5^IcLjPV81v(Q6q;~K_aHj#^CK|{$;97?Y)8LBBknp6a zNH*ZF2)xXIuWRBwE;r!YMg9r{u63x2<4OPjSy75QaME-UIuI2AA;9-&9^ygGr=|%%1`Lh}lXkAV}rvW!j z;&{FRuMl{l0k?{+9G3yVN>H>MIFovf5BU!|T)FjcM?91Fppb_9tDsGc%iT#?P>6c0S6_x{+k|3Wr1etECyV&bXEhdSvs2m@3==;OatD~%+CfC zK4kF!J4J%aP{8^!FCZIm%|KP8!E4gs4H_=SZ_+B-s3kD_XEH?uNt+G$5slOa{O`|l zHR~|oB?9j<;EI-D{c9=-7o1U4CAm(SV!8$%l;wTx+1kh;QXAXfxo}mTW%%w;K{_TX})~23!-; z&NR5CF_qM1FL27))8K`T2LE3xP?n}Zl>xt5G*FucUvI!45&4@ExX{1NBB3Rb!1eD3 zfwvp*<(q}j8SwiB-eth2zsU1jA4#RqC5lL}8xphvg$8_`$X{l_UlMqg0Y4z{1_S=A z*z#2x4GG#_aFYRlRTOATgLkCCyA1e8q8{tt7$uVa|6!5fiw7Yg2IWe}$-MeM&+SZ%Tu=rNKMW;9V!fN&c*VPZbKSL(Vj~D-B+r2Cq(o*8^wr zuUSB?;l?xtnhkhJP_E5@YxQ&(@E1h>t|ZR&PfM^qno0=`cN*}cq9T_8AGn3fM7aU) z=*jVF18(Xi{Qnw5LcZ_=)*JB7Hf|Xk4Y+9!$2S`AZ$*ci4ft;YZ&M5jrdN4S_Z#qg zG}0OHJ|e&6zfvhTQs6cNK1twCfhYX`YeYh!p+LJ>QppDV36a0tfZrrKT4lhsLuxgt zxMF%qR8*guVA6cIMg!hx=M-u&;I2U&?=av^134Zs;Oz{D{B;@_e@*>52lJlh8wyyrap5X7;HSRE^UDTYV?ntA*H}=c!f%gzX8|scc#HD8&e0YXa($s1Pw1V;DfhuN>mtd*Y(_4tuo*h0tYX$By;A;df8}M}kFEijzDI%fVkkBgd3IpCQ@G1lTslclZ_)h|_ zG2s1Q;;d0>4GHIogn9#BCGZ9Vt_Xa+0S^hh(SWZN_(qN={eP2iG@1+r{wfMI8}Jtd z-eSPt7I>Qh|5)JdDLD8a6$$%O5@32ohdK=SF_AxFz-?l3>NMb1QBRivA3^z<{5_c} z?NdcX76X31Hv1Xy#R9h(@Z|!x8*oM7PT;J4%>v#d67mfN8uxHvaT)Nqz{?DH*6Tcf zg#mX7ygG?<{kv8q)Fu;{`q@MS>kW8=$iLBmKP~WP1KuTAu>R>(N?RSm|KDgx=p4pJ zU}{P&ka#xWE%W)&3ab$>=eXT~YxU$S#)R9%G-|+IVlT%cI!fK+;blfBlU0ZFCOK}? z;k;&!+Xc?X&t8Y(xXGz2z}-%skgvl9ZCFB~4(GJu`CU3(lP^kvgzz8H?c!I{Z{! zJ&iiNuMTe_oaCR#nWsx=))nZd!&`KCe;wYY!)-deU5B5h!}sfO<#b&_hc4j^9Ujr) z19W(&4j-t)yL9*<9d3GtO9&0v&T+2)7G1&+T>+~OAF9J`I((Q8x9f0+4tMJC;W|A3 z8R7p^k4ESc3UviW>Ts71cj|CihmX?XWjg#!9bT^CH2%@LgbFPI*ccsNrNhVS@M;~- zUp(ZAH9CB}E`O~K&j-%wkDGd3LV>P8gAPAShp*S+6Lff^4xgyQH|p?7Nn9~C=@QOP zCYYLa_#bq5iw-Z;;cYs+NQbxU@X0!SKf^J8;{6m|LWi!vR2?4C;nQ?@rw(`N@Gc#G zjt=kqC)Wb{+ea^rUAlz9x&o%BpFd^pk?iO5=Q_;~l=ZsevhF6iuElGPx6A$=mhCKO zeA3HHx|`y|x8Scd*o8ZKuA;PPM;yQ5&(SkY8PRt3`~*GIR1s}q&-b9($_6TcQ)f@# zR%y=4Sl8k-4_1FE8=iSZc7{6e{2`gs%^7O(`O7me$`*+qoIg~(=7N#t-rLoN3r3$_ zoReXagR}O_;nGeySlWU6h#bzoZc_Jkxxq}Edhmkjr@z}Pk*7r@-cK#=KDOY46CLI`CsA?in3~2kh2Z&7X_+k!3E~&lhnoqvvVFhot5iE z!peO0@PZ2aufth~kaZRjpdBf9_tnOneV|NTv9SE~f*u-@La7G}#0ZZq9Gkf{vzt10 z(YTzS9IQg3~XGhQ5#f!wvkV3yzieZCw#j!HirueOw*d-_k^5?#_W=AQB0*C~gx zhx8y}k;9Y57pdo6G{Ah*cnp7V=A_T2sP|no*gU;JZMx_p^U{TC&x=Qy<%R0m7Y`rx z2qtDZoOe%gcatahqbK-HMEW%zmwLK`yX4?!k-tsO0Dd$w`G^|0IN#i|K;4Y84=qqX zzj*k>Bbe+_cDJj$DRv5O_Ti>9_yh8Lg2yA}UAlT=MZ6xrI{A{j`+qzy!&E6tH|(jD zuEu3x-9W1m{E0da$g#mFv;siV<2X%6qDE=^xEmAFBN>8)@EJ zqMmcv1oKBnRPSX|&7KnVxywE<`?~)H(Ulu(tQ+@by zUsf|-!d=(ioH6*cSwBL^L$^Dua>(njMb5@!C6gMp;i`d-`;LLm!4ijE4wpDAbmxRf zM6*cbN*wvPpLVAD?p0^?x(LPttyFD?eNoT5;!Ja9wz~3)Gt8I1|NQzZEE(&Xa?Dxk z&(};g|MjSvUv);$TT5Z5f}h3noad?ws|L^gb}l_Hc33{%n#qN^wLJr3gdca(2WHc;81TCX0Evn`o5 zZ=~wJ>a46>Au zwZJaf4%Pg4Qa5DFX+ud*IQ#RnP(k=Yhy68H=mM}r>L~}e%YG2V!|tGt{nYmzy8>_4#Gj*)q)QBkJ8tCz#JDQD0elw)xAM>S0`-oT-jpHqGpfsMjrf z%luz6)S~N)%-79Oz1KUckt?o2|v_+cyk1 zM_uYKxNLK&18zKDSv`h^M+z4Sd;!S`=e;l*2I@@QJViHCaPtaoJV7SLt}o;9=p$em z;70+M2EP?4BPeAxPq`;0r593S)hKWYRX6}Q7tjqg;a2EwF7C=xihP8w#HIr-PJy=* zJQDC;z;f^%>h(LHvtHX$GCqimI;j}#$S}pOh^+k3>oxoObA zb>Bhhf|#hoIa`P$;Zp@ePn?pcihn0FP* zK0Zw?S$?@$nyx;+eBh{Z=UIR>XQAlo?9--oH~G6VeS=_Ha;B**x8|AenWB1D%+8!^ z%2fZjqNMMsW|-zs_6BOBXuI#!P|ibB)q^We&szmVs%s*H793cGhhWYPQ`G_9A<8^F zAZamK@ZH(d@I)?O?X-o2dllI~Z6dqNkweoaP->p92z6O!HA>dfCMh>(1VAY_Tf#+- zXPYQX&XFn9+;D1cclN*VK)-^P)$HEiWtQrCP+EA>o->iS^>w7ad?)_AgkB(DJkLD2 zyD9q3G0HP-GqOZ(HD{{dc+czqOd%=I(RkitxQXVWt1vrfPEb9*Gt2`gsMq@J%7HV{ zlklp>$eACr@~4hN`A^~L#zvsF47{h(;!}#?$_WKAOBlmq&0H&-)T-pv!!XFl&K=FspC@VQ#=B){E&3x>jZa zqr%hN03%trk69xBB<%}sZzd9~&OQtxYrOuphWEv{ z^ft4Kx!4~$n@C}cPedpDJLF4>7Z)!f&t?fsQTCt5sRwVhPk3<@8{PWIV;?g9Z^d)u zV#osHtDx^)=)Xqan^~nj6si;y*s5N7+d%W>BglvCGwHk$thV+@Kh#FPk$OYf)tSM4 ze38@p@ej~OXjO+C@~i=8E9CrT zT7bp#M#DXD@1}(*uRZRY8e4(dCyZ1N+&*eV1SMIkac>a@12=}`JlopB$=^(cp9ob zVdU-cyx4g4_f?IV8Se?24Bo4wNgwr%JIZ>zG@gmeN8?n75;NC1)!bTp&lOHa+KrLB z-c_g8)|pQor*5wu>;7si;JTJO9%c?3adz6M8B9%|8woXapcj!sG@&_tq3jD#J47!j z{h(QP&Um%x&gsfAXupu)ANDZ`jsbxLm!KI)@YR?jgt6a?1SY|lx=t#=&muz_3H}o$ z86@~FVCs=Ougv)!H(O11V&BFwYUN#j==~uY*TnCQu}r4>(qwglOg}xGWV+-{J0sQl z$SZF%nSLM7ksIHG6_c2g#HNzSYj33r`D|S)lUXZHKb!79i)??B2#6W(0+z=>yX>)? zcIc9z-8OtO3!INpCqo0XPzz``9F|t29TFJrhNq()GNhv&B^hZ4Og%erWzH+O*-BsK zisx0GsXi7c?7a_-YqWcPG^5>nr1B@w?)qVjcA-IncDG^75GEDRk#XAKfpxW8Gx{ph<};d8wQc@@5=BkX09Tvq*D$p$d7z{d4_3s7U>z}=z5DMw8~1x=KSbW47u6~ zBp-8h}+v7HXu)MpvV^LaP4?5Xv6{ zOSN{p5Slsyo-^|6U3U!6I|x7dV2hh+6Qti@99Y#aW}bAm1)$sqy?K_VC3xM0btv^#jI#l3sB zmF{WO{XM~NAo=(Tx{N?ZUo)=#0~!DH3`E7D6=ia0ML9Jv3XGQhD`gXn=_tBy(~oHwa;IXcYwd>c zM05K9byv7x#6trp>lEIwC7u_?Ga9mcLMxr6;Yx_m*xLqlR|noZA+rS6`MCDRHF$4H z=I%4PtL^t*G3AOgPzB~h76({NZbN+dJYeqN?xeQ|*K#t?7tbre4MZk#`K#*n_f?&{ z^>jwGc;05*prXV`;=>;W0-19AS7C|Hea8DKcWBx@yijrcLzKW@a5nPv7S-M`Nx22I zar>v6yq~&5ij~dL@!@Au-F}?C^R~%Jv@b((o_^^A4~#k0iZ8pBhP@7#8*v;O zeSC>}&I9u@ud#Jk8y@J_<4*`i!TmjNoBGTH;~dgu$Q}GP_MIRllSs3TH*wa>uw%zs z)eLn&zriTx4*p0PDK6sM8rv0lqg9=tIy3*=zq@*=dcJj5e~eJ`5HZqq%9j}fx}63z zcIvvzpULj4{&B4>5A^So%FGY?byuHL<;;65aNp(-4ENtY}s0mZiwx8i{1p^Kng2;HhZ7;2;0}d-a!Wb6D`cQ|t zJjY?zMesIg}G8qK3wrsKsVXP4rXTMLV$` zj(9?oI$84YpKy^wV-L}#ZjaX?2R#&sT;|C41Um{2kZE>C???5LLOntxGe7>GCp>NF zgW0CiqHmU$7Q_pVc!J->^E&&Y3i7FY$wfP5X?6r0iuFMRlJF~Svj6uUzUi|HzQTg_ zCbULBPjL8hYB)G{1unCwqAxw84hQ%9J2Kr@X07&Qbj3SrS6YX8E29Mpk+StmyX=3% zoSLIDX+&#RO2fR9quI#IRzVb%Y>n8#W3MCnBZ3N*?3ajVCenvVb?r!I70Y4yxa|Mc zEZubmXvQYOKJj=iT$`C#Kq4*+lZZ9RCF&0w`YA(Cg=_@g$7-AAzW1W9<1X1BPiPC{ zi5z;0@yQ*YIv$~TPB4@#%|K+HXYm9l%_Hf}hX0 zD+YOjc=iN;qDckm3OWPzjAD<2E$c8t(K99_)fo7TRTA6HnacN9$t(cD)A4b^eMnRHyIRc;2*A>ETLx&hozR z@9H7dU1wI$U)Nt*o(*~OWrL})xt?$Zx>bY!tGl~H-N^CECTg|2{dZU(q?6vt?QZgR z15_N}2MdP3Bn&hVBfoZM=Ruyh@J#AQq7@w59~X*i+Y0uS27hsndaE?-P$akim_*WS zy=An!=xr%*2!#sv6xUAeb&3=if)UB_R(H``zJpO(z-eaMjX->N+27IS8P(zInw={ES!5-l*iu&F~6{t@xYV#eWLNR5b#;DTZH_>Y#N20*tV(G@LtV-!d z>~Q>;Gf1T^i}!OzuY($P93)h&ZCM}7D+3)@9P z6142peQKzb}WZ@3Y9oXZ`47xD{f-CUDTB7UCoYmff4=8}+sjB&C|ExAg3yXUw27 zuJogOyYAkJd(f5P`3zSOAPP9ja9_*%`pbQA8Yrcph0i@=gwp1|X4(In{g!d^Yjn_s z)gt|U--M0i#)x?-^&ZiA( z^~~` zZzM(66QI@ic62j}AC_)3S4!UOSJvR9!R8s(O6i8SN=5SK5Kt){z|`?!>$G?r4b;7k zE|(VlTdMox-Dx^>Hw5*>4xLeVVux<05gI!*hZQ%=>RFAaneWI@OB)9(Bhjd*h>L+# zmz%ETs~B00r1;?^q@Wj=PiT2Kifg2JFO*I9bGw5sIp)nFwc(M|lyXSXjWmuE`9Qvt zOY)bUbnJ)kI&ti?Pd4^k6xVt%Nc7+o>Olq`z!5zs0uPZkLGH+EL_U|W%!381D_}H8 zm6PW($Ryo0`rk05ndI|BB1ye4ob^PU6Ne-Dr4z+Ufm}UwlA%lkcTON?JCwy!^t)8& zPS?VpZl|V=-1@Q>s6lYqX@qzSRwSC-ljJU<@+9nacT%U4as(rxc?L4{k2+Y#r9tKt;tAi7%s3Zb%^` zaXIjwAB8b|KFWZjNs!S_WV|WS#)1<^NNz`hikHzwF{sBt!6r}9Z&IKc^()B+N^o!m zv_ynsEt?yNruOJ5Xs5Cvb=DHx;^vsGn@RDe@TqUbLQaX%{897oxThd(zVH-~<~O^v zdS4mJyr$^P=|0K7jX+2Wv&7-P7CF9+W>x=hX5Xtco0Bt$wh}r@gCs1{t0^8o&S!F( z+t^saOC~YFO=A5Pa5OR!1{`2MBke{-02EauQXmG64q#O;o*M)uIS*#%!|4w=Y5^XW zl-YI~dm(aqf{!{}RD!zBt{WU>41dZ|PS2W28S!t*v|X~ItH7mEO2E-hHvp&tX;i?` zBA)Zba}%yGV0;CDn>B1Sm68=CMG|f{IPC0Z)CNZ*zky%CZj?1_KCFZ!emVhFrdy4O zkv)hqwOEUrymk_EHojpHLW;=n^$8hgP=M!iGOpRYHkmRd@{r|;=beW0#Kfx4IB*%LJA-cP?$H9FlR)q{Qh|4a8s;U+B7X&l1I(6%+- zP&cu~5Q`(gNRtu$Y7r{}&9Ur^-9aa1`Xu!3b{vU--o>t=IfUtMIN+#Jdp~u$QVorP z>ca-Gn^CX|{04RcHO?O6MiF8!&zH25ZB+Mk+8rC|ZX=qFnV$L=P3|G6jl zd8~Ug7JZ1jksYqZ9sP)^WJF2(EKF}PchWQRG~!A>_!fwsqkY5#vX|RJnJ=&KUR_<# zE^VHlQ}pq+n9rh)h!7>~hTWIL4e^!LCj$zVg}*N#(eXTEmSY zDy*OiQM!yZ8gNvJ=L&ibXT3#Z!qr823JHU)DyQqxZH0v8GYqSDrYXg7=3BP|j15bP zvw3P};F!yjk zV!~0EojN1hI9Yp~H@TUVmu4;AckV5+au8J}5l}dftTekHaL|8dnKZ83os8nQ9pe-y zX@QIIyP)_Rwdev4i)u9l+hTom6nllKU<-|}9pj^GirqwqYRTx}N4mlIrNCuDz=(I1 z{1+z$=V-eG*0v2frMoII0&0>*LZV~C6?^cU66}0JnXE}ZD!_(PCs#MPbj$_vIBaxS z&1K>{{Id!U!t;^gCONS&3x@zB{0$=vj;4OpW5{xFCi;Y8a`0SuNlUR0@~FcJP4fh= zw0nXp_z{3Iv^_H)?oT27C5looouYUrmPIs15O{iGOALD=pzoZOjJ^X%hC{T+rR44T zRZzDOX*dKyu97X*o3BZVUu85!hv8OviBcD$P@WHEpgrV1P=BR>gEo+w@rQohp+wT^ zW4E!Uo+IX=DU9+mt)q+jXr?Cm7wn1k)1_;=i=82& z76~J$1)~mrkTl}v1m8b*z8!jhF4_A`QXe?Jhy<=-mPC=m#deYuW^7O+6MsCnkp$sc z6MqXNhn|@lfX!tj-(+g&cY-}?Kmt*YGLa3UAC+eI7qvH1zwz9mt*tf^0OQ4{Ja`TC z4DD+ccnz*(yx@Ppm!N7pfDpB@?2O=+HPwk5x({WbjR8>DP;nOSmNAtsW(y3N-JfrB zFWprLA}EZ=az-1B3R`n@GR;}*a8bVS3?3i{mM|SQ5Q3fHgZ)6=705?-B@R0W$`gRh zaVPDKO-izuSV$dphi7NNw*~3(kl3mqzMIg5RQGpC2@SLw$uv+;(I&PIB+X#`rV6-&&H5OXiqGOTG9a@A&7uck_!Fa++D=fDZXFTuE$V-vF9zb#W$T|my zvv4xWg*TYmNS&I|8J`(vJ>t+FjFost#KeCfbS_i}cNBPNp+vG%(T`dGX%8o{g(KCi z14uS>G~7{irOt)+NC-_vXKh0+RzkT>BoXizab){ahozVvL@F!G!~YV z5ui;*DUg>>@h9YILLTBQagTQ~QlCay#ivH1C7s*Xr}X)&udrKpq{sp-4nVJv%}0X8XoJz zSGWlEgGVdT{e*iu+Y0x`-n(GjYMEk{uiRT%vT8xGbORBLD{rr4Qq`C+MI zeTw1C445LrJ|eaUIdZ|TU*@n#flHkr`@rD$vVUip9Asr+jcdq8G8cx(LS;(Hu!nzzCKYliQ}QiyE6XG*or_kbIzu#+A~IgX$*<*fm=n4J{sb$9%2 z@0DaBk%L>v8aZJhjplJ(C(L6o>QSK56p7j%9Y@R{CrF-Bmavf3u#mSqU?QuKpG>3$ zxyVG8ILJnxYei(r0s_ypfX-wKXIi~Ox!HuNAU!6UNp&CM^?fPy8f;heP24g=!ct+O zOwq?FnjMCHg)~Zc{{v8@l?e_G$Rs5Gh*bB@@5o1+%2(6A%i5c%=2wy1D6-P(7m$IA zVixXB5L*{a92Z*!ts{qCkC>Sa_>e>BI;pq!W&}RT?&`I8g*SGUROfIYYf@jN)h^J8 zH~u0z%#L~{bmoW@n3-rCZ|5k;!SgPmywI6nugEY(<9L9F4gZ2$jG$HDJhxmr+iMq6 zPGV2LREg&cH?ewQX;$PXhRM4T8Me_KJs@_N59eUnO}H06Dp}Ozu>y9Y6$eYlAB6p7 zMi!U7RpOwg)BQxfflL$ zEGGsD+g6KbQqIK=8Go;GluH5H|0UWg^+>|87f(@#RCg*sVB}tFs#`o|HjDwyXY@{5 zpo86jV`FJChrHxWyQj>~m+Brwk~=gTOhViR_sld=d~?5Xl5@x3kphn+3o5`72*x-v zSesZmZnKV~5WTN%@an#cF|&pB6>L^>l&Z!n8Z0o@jta^0NPe37iMNmfm||jqJ$9M_ z(1co{W1yUowA)Y^&48^3Knn|ueSlQwLmQNZn1s5VIMC{K;wV1Wh*#loU+^hM6=Vqu zeux^4J7vz2RzCxF&kIjw6BWFe>!2xga4zpOj0BcOkpg!j2VXO4B#8AY-eN&j%mUDo znjM;N`;Lk}#bzoJWgA;IKgy;mlJi;;X)(P;Mnk`e_-w9&$42uRGZ^I%tK2NM#H0~H z{=7~*2<3Fjbq7h2Xbj}bR?w)DtL*MLc_^*^_cH41&+gzsck9<#7|(|9sZ&43^Ug)@ zm~DQCWuea%#Cnp$iBTM6!GhfwBs+d+VguoQQ)nbs-AL%L7aIvkPC6}F*|wM+y}=3? z|Br$bNg|_-G!r8wzXAFAq#cDbki}xAR6I8eZu`zet#DAu^2xy}8!fD>71WD$r-jjm zv_lv%)J6m#VP?Jro*8rTOs)jco7DrH$Vg5xiGe3P6(_={VoD^@^U}$4*vlS~Zde~F zHgE(Y#!x%XqkjvEDX74X4!MJK{@bNKUWHg>uf*T;tn|0LJY}(?O!D_xN|eaDgxa@I z`yp?u6;4u*6-62(hX=HJrcB_kF77{0eVvKO`%gqAg#4Xo9|f`a&a!}vh`~T@pfNI5 zdd?N|uug+1z7eJv!htOnkkq>%DdJ2hoZz@8Jn4C0jG`=As$VlNF{vffjLW)7|91XhohF3z2twR2`SxTr|s2wCt9P9hTsUKF8CIP0t-gNjjfhY+Wd2+X*X`eJ zft`MXNJEY^O}r#*WW}7g!c9?<2fpB?%K+&|ohih9!5%kTN&%|%K;fCnA^3M;vO@^| z(QBNqBm7l6*>p!InXr7S8`+uWhBvW2rO1RGob&`@fg#nsn}=KqTBYpt^AyD(8LaGl zOz+}=Lui5wcSH`&wo)ufs~vWLI!H#}#jZH*xJRnH5mQg>MVLp@f0-C(XF^4Xb!A%1 zA_xo9ipdjar4LGVafRtVR(5OI#zV;b(hBeLYYW<>7w2Y?6H{~NPgi0A*Q1r{LbMdK z^qXYA$XCPh_7=JiUxf3{Uv^=`+&$`TchNq)A8^PMzO9qLuN7NBD@>h(BU!Vf_rh7o zon@D-O7`3Q9x+5~_H($0C71=y%*rQ{&@jMWH0_S>^k8L!7;p#96SFS-Cfza-E}9gG zA!k8L;xKGU;g}5XJG4?Q+U7eHeG9269YFA`=ndbs#nR@tWaZ@y@sp)zUKcAF4;C>t zcc`yPE_hwu8b8i|a(Loo`RHDG$RT&gYqH1%Z_7A&d~}~Y()u+*Ml1n>M_;?O)Bw1j3u`0)^bY_YNO^WYkC_X2o6sYQ8do$?hOq<}wv zJC*?C7SHA{M!Jv(3Io9_4c*{!XM7B5FGDrOQ|1kj)*>cGL)X*`GnR+c%R^rI?${U} z384cCz@fS&uG6aBke|M|C|_HSE#6KIIX&1GWFCA5^Wc5?w~aG+?w640Dfmdz zC0CQ{UW&(+gQBwul_>m{>eb#w0Zel0KNtWOA56IQQs8V-MlidcNP+|GQ?S(|)sa&H zkbHR@%_43{bqIA}f?mUj7krC3^a`0nL}U?^W)!990m72x*7IO4bK}W?ldsL(9vCma zmXPCbxTLi^rNC3HD9krX$t?Ducja+x8#d}zEZh)cWI44cEoNv1q1)2nPP}B)!M^f@ zB}!>9f}m|_@MoIQOlXZXjS6Ba3CBvbIg(5!b(2Rtprep@-mE9Mop&&vxA!xxR(EiJ zT?=(LW-d7Bs{IAQOvoMlFRsP&!o7;z{#M)EP)U(}Zpc&QB%LYPo2vJS+brkFyElQzT|`LxX_?0ap;zT%Zg3Qzq6)sXWw>~pcc*!!t)b%r5> zjLk2;@mB#sAUaWvvD5L&F0YhrhQUkc8yzeYz2=Kob(c~-=qSfvQm6vV-(AEQT)EG^ zA$9e!>oB!~9aq*Gcg!{Y$j#bMtmBffT!DIC$Gyu9j(muiOfxGwN4A36^p+6;Ojsc# zojvAtPPl@+<{#ay-;#?r7UIL!RUdyM#(n&9sw7H%V(Hr@>mEU!`B1~Fc-M$y(F>)CiW@d2E$79jzku!RL~_ZY)!L_{dH

7+ zAFf6?jl^CrQAk*UHpl^yzKm!XMsu_pp)kLlqBk@@tv!Gd($qwNLi3WJ5R92@h8&#r z-!4R?+y!>H2&M2B;3v?25SAg!FEg3W4QJg<4OJ@%vgn}_Sjh5x8j}+1 zkb|r=DMC3@2nZ$?LD7$hZR`-AZvFAbzc|X*AJk!fp3ej>;C&-X1!H+VSo#ug645U@ z5}@#z^E*%Q7&C~`FYx*eLMnVl1-8SZtxAE}BX(VR)C+wUB|{2zm4MrRj)f zdG=;E7Fv^7w_xwlAzqJ|_Bfm(knshsIBf4=fy$ z%OF~TkhSqC^R>O7=i$ZS>=q3I?<1ibG|^DOc;2;WBl=fBFz<=&JgS!|p})m~*YT2d zdtV^dQ4$Mf&c^zA9;9~Fy{j zSoxhhyoCEZELM5+BJ{oJ2kGvQaFdKx64S`Mz>@1imF}e*X$p|G0 zl1;jcwt5hq%c)&VzRnCht#n4=kt_Nm5-A>!C}=}ga(n!J(S0ze79GGi;ICtX#Qq_k za060Ffw95_)>94&X-4pp4=^x_+#nvx+>b=&`Gnbx0?$XK{K!P!1dnBI0;0t-N!|^y z%mfQqB=bd4T<`r*Aqr&ru7{7~314o5Gnv?$p7$XcfzO$Tqj1*3wb>M#fj)2_clH3^ zq!{*cxQD~y1H%pR0n~v)=NKFga)uHzkrYWG9pFUtW9+r)WQgKV#8V0b>LQz1KTvqY z;+>T+YqSN2V>TDltoySrqPY#q!Dil@_QBYY(e20b>_m(SK|qWW;Y`iHz?+?j7P5(U zD6+%j5)HK(whn7d;t_X0xD}#j%FJVTq(K`o5*FX=q%>_@ zc9?n=-aH|kIsYQ;Ke_S7al(%{!gz!D?06${_o?N=z?${xC;Z_5c4`Zz@!pnVsQ~X zgIo%9DP2g@O-#b}QQwB=3Cs*P>iD@b)32JsgPLQ%BiYae zHBT1P3`Urpl#603$|$IbSR#V>U$OXeIT+);OfTs>q}6AZQ{bLQ!Qfzi&X|m;t${{J zVjf2~NlD}wB1By6k{ZXX7tLX0xT81x7LT)IMiky;VMbVX@FR=SpT395rucqD(v!qF zby0XjobwsCsz}L31ofYT5Y&Xh|)#Utwr}9Thvl#klodj*^SP zgPC$LrVY*84c}{|4B$DcW71^8km(u2W6w7Bi~ty7M~p7f3iw;ha*Zp|rr%`Dj7@-I zsRJ_rO!(aWnOu1N{{~U({VesECd&WI1R_>+h1J)emFb{0nU^ z#JYZ6nT@r7?ra3XvOf{`aB8jDEqI_9dpCS;NCd%PIB_V6V($4;z{G0B;TOcdyr5c?pvAEF)OY0gXp11vN5>ys*>qx}ho(33w7mx1Hd4;s7Y=Sgm%+@(w^!Gdq{as|E zzn9^d|0tZNrwr04oUrFNYnTD&8&4$3!hpJi=M|z4L#{3|g_( z1v41WoBs|vMkZs(LYe^(aRv-C4Kd`#SUcLOz15|?)J3QI=}Bo_NiuIXbt04a9 z2Az~cUn~HSpg63!LA~fZYPGYzc z%!2ncYf`QH;C%z0BkEJ6>tg152`$mDmQK0V#c{{RPX+0Rty%9m`09ywG8}70A{vD| z)Y}uf7B0@MR`$sj(S5A)*g6D)K@H(On`k+S9O5t(M3n+x615TzADIB01h^TX9Q+72 zfx7)2EvXb5sJ`DD@f`W&$#ne=OrSc#YsJESZbCLC>x?eRI@cmuFTfV%#ZFBmHHoBm z7HUzJIS#mDdlf;ku=4-BCQY)5(4-}Da0dDE-05PzIkn5Yd#)NzO<8g;b}BGA7ufJ; zt_6Q)bRj(EfB?{fyXZIHRrqQE7>HvNQsBDzH1HcU_3+M5)&k8MpVZ4-bTs6Us+Tox_Kg4If`Vuzi0CK?&khsxL4)Yyt(BJCn!XFv* zx0Znu)^f6E8ux_M=nWg^9*&bi_5{btkD&)EjUNv#dN7HHq`85BFc68PTO=@wg>C?F zQ-N=su;&fJN{D5@|95O!-ANy^_WzzI`L{rx5SW$2S=gfJz!pU(Z8;R|!2%(sU+Q{- zzsSKQ@V4+B#NgGK9Uwe6UL_Q@kmyUT zVk2z^G?lXg2{yWCZg%i37S6!*QZl|1GepUF7kxmeWPB4{JmXtv=DBcuJ6#r!@1RL! z@%VhYTsYp2EQ)8ml`dK1u>~&$=%oU;f6Mr47%6o0zwqjiJG5zh6&BKQ)>E7}Y(j5u zzt59#jD5;#*UTDzaF^_P1QHV+OCRJ7w$T|=C%z>lzx39d{3jp%oIcyq8tVpEk%9-d zq~&uLeJlkilz(nG>rrABUT5uI94hHVXpw;(2i3FOP&Vt4E*2d(dp=z$QfWuXq5790Q8k3HoVdJ$17m1vsrz z;9a$ti2JX=qc4*1WIDlMMNrkcHz`#Zych3 z5MyMKitlMmE*w|V9~5;bj?%l?QF?@HaZnUR;Oel0qS1-e|L{y$5sblule0JsSxBEG z`$YPM)0q>*b#A5R zJ?xBafW2KnVvdOc?UV7RDFXjdU*HVj&O034XvI4U11S*E0)zycqL-{Yh=l-(jiKdg zISL^Ps8xe3rs%J4QRh0)vTwC^*q^`li36C>$mn8{bKEJ`XeTBm79hryfwCbUO~bc3 zC@RcyQ{>Vj)y)yTpj>jO8k@0Y_?|q50RfI2%fi=f`P_tTrsz@V9jyqHo)-GAs~`W? z4b_jAnl>YvXK3bD@&VwLTBs06N+w$TgCSU&-MOtB{PY9-9}s~o z--KuWrcZwKh1ZEgI4-N})L9sbHfX8tTs#rgPdZ-7&=LCAY_-fG7rn~lMR7q9+#wNk zXOxV7lB`(4bVZabx1pa9VWdd))v#k&aL_+g%L46Rv0-4jF^gsge5&m7WCf;^H)fz} zbUxUjbq}o}7+r0HUyYYae`8Ia&YRqhe@mBbXDrv6{L#&u{1>tX4*1wk1rMT+Y(@7V z;!cX3(PHTD7$q$qOssd{39guTVg-ri0t|n0s>h7j&8^@RN^mK*mbTkqFRcg~c#6LE zF7Xsyg>9(7(`bV!Kgw_u9T#u`ebE04}A_c?a zf?Cq)+iK*27fvcc5gxJ_U$^OBFknxj)w_wu1M?EMpcuQe$jy-suLk=!-l4)a# zO$@?)7p@lkFHXRBCybmn!syB*%U$n8(vtg{7@%m3jif#l~=mxl=FgVj5RL;c+=l@S5OuV7|KaJ3rlNjB) z4~PI*R}cY~zJW*#LI#Y?7h1{^F^dtoB^In^MA!m1n){vlIU2Rs_jZLyi3+Pg6R3+6 zp!on3OzK`CzLWC%76=M0!62{b!tIV43MzgLl&k|^PT+J4=>9)Bt0@x*zLX1(bsg$V z%jTz`B8<+q~O^s8kP0OjL;XY&+Be3A!u@w`n)qUC^9+eZI80njT8lJP-J zOYG!AP^RA}SFlmOjJkl0ayD;Kzu8#YxYBK5)k$^Z@fe+eKe|u$b8|k4#+WeYkES!{ ze9=`-${MbQ0Ct&ipwOd8Gk&L!_D{iglbW&Apda=VcT1}m7326*c)&604`#DMCXSe6 zzNJypY&HrR=un?UoX^1gf;R_HKON9u#sLcr7eygS)tUvVAmLpgR!r5Kz5{ z-|n!Z1@2|y9}CG_MWqmOTH1!#5I#$V4q%Js6uim~R&0-*p0v*K?e4HLyAXRd_@Y*D z7yeDLBDVh%KoNk2ChsTMym8^*8=^kGD~D?Au}0krCZ5;lzi(?W*(%+NhZ@wicOKfq z@pvyhU25!e-q_YnXzT;n6%Y=lq#Yt)k^+xFt7w!ooOs?}kSBUCAVUM_A^VUueUV2B zd;vKqkv7lFF0Sp`D&0Ha9IBBL@UI%evaHC)lhRG9`xAIt?C%;i`v>ii0{XP(bGUi!bfn{(TKIQl@|7F2O_A=`A&j#rL;qd5=;0B(T?=)h z76ty--Glk6#-O3?7&v-}WBeLr5Y^`&>Wd7&l&`;CTL00XXpbh)D17Vhj5&1&r~?1? zZU}Dv2EP;mcl?_T??`vO1-xJn?e*cUJ36WV@+p+_=oiIOLkqra+}ppc-Tzw;e3B9H zEF86|qY}lzcDzyXvVvQD*1KiJw&=^aV@@GS2aKpp*P?wZipd}p(nENjS=u}`pFR*Q zt*!up+|4HZC4q5EfA5m~Zw*H-9FK%&%u3T2&JX$CARnAapce`muWbOs2QtSj!9lJw z{z=|UJXi8hz|=v*N$6>E2)CRjhqG#fd+~P$eGR4P9p4u(Q|mbL%l?6fS{efQMG}Os zVqpt;6RJh)g``FJ(Y-A0F&pjKq+kyZiyP?F&CiJmNtz7vx!aYs%u7|ZAaS>8`PD0Y zH1Rj$iI&NM=X#?u%}^p+6B6j3SZ)0>8}%-Se)1!6TBT2;Qucwc7jffW`gXaT$-d%3 z$1|`SfiBTL+7PHOAeH2bM_<8gHp9X9`X;@$9McuvN{gO@atsi(ah34k7 zuOpSv4%I_I@czA2_bA01<zDYTdap}xJQlBU6&%1Cn_h|wIGkWE zeRF1Z1I53n7-t|l#u-RA2$Xonu(t%W#;`Z~AUM=>|4utwAz&6m^t2~IFp@T${IVf7 zyUd1N5WH)86=s~Xr1}pFAcw)D_!q6RUk5+gE1tJLBl^FaCE{hy2>!)^Qv3@uJu4fu zFF?UR!|ST}*F;7mx8P(Z`D{dWvR-!R6J3dswBJii^i08rKQJ0v?%-8oCv==_U?;^p zBP&|rT=fQ&I-8Y*u9(laYbb?&OlEMkhS z!D|@5VnaA-=$~Q%W$@||4)a>klO~YmQAa)$f!tP7f_g6a%+2Q1!s|KghU+y8Dd1V! z%rkmuzq7c$*pBif_Qk>CA<~NhF86pCP8mt?k!K`HDDl7@EOyd8Rg8Ot7+?_jeW9ii zWQj`=XvFJ*FlMAiNx5go056(A87eIF5g;z*8^LZS<%|9)hf&2)1g_5uIr%rQq|hR}2#`T4Nv-_{5p56}7daAw$zM}OG+W3d zjVAK4g4YrmLGi>sEo3mb=rvL_LDKlk_!XPjKxC$L`q4d6MiVHFI*G4R5pghkhtWf1 zldKuE5>SXJ7&D1w7Lkx_0TjPGahyPyxu0;6UR>+=g|3@7Iga8=K4QXbknEDRC@zD1 zxQc`B;>zOS`Ll}aZ<^x{-pW43MP`~THG~0g-a-~Gfc8Qb=m0N%nM@e+VGbY+_T&WT zBN?Djq2oy*4E>Pj+;Fd(-Qiw%5ereHY<2y?LF^leE)0;_(3sop-vN7s84A;dtec@| zL`(!&mto(MGZdSkXnvgXD0^85Uxq+^v;ju%=x;?qOj0|jzJ%pwoeqyiF8cZjHl4J2 z`LU-&w--Diy1g8^n3hrt{GDk}MwA9nNO}wD?f_GW0kv!t{If^I(@w!(_WEVYu118F zVAC&c+Uo_J7;?hss3vSuA#D8=YgGz-h_ob&T8Q@VDt7S(9D^y38K1;cl1#CjkC_Ouw z4x5M0C9{xIM_IZhv*1mFWV$Sd(mk2JhMm?HSrm+;1{GXy2}Utij4T&cGJ?+gzm{f6 zn|(b>f}O1q=s<}hzu5ncguK2}5|9lDa?R>FE7)3$^2MkY&kK6Z3cgp|8pHD!^t>vk zB={>!{+23O)jQaQI{kRS1h_;h4t`wR`hAxFOHg)Mp2=1JlsK!84OfPsii!P7>Nl|? zP?)Kf9rz4;G#Vh&Uo&%5<;A;$fkR20HdZ=s8& z9{v%o&UzUhJO2b~y_`u*d}A6SM1O5RtRE9*5dcRZYE|yUW!FIr+P2^)iOR zUtsz_yuAx}Ojq|mo_%r=mpBtL#3cklf|4L42nh*im=JfZTUwe@)FlbpVnPx!O{Yb% zwWaj#ZM98X3@wqkBxqe*mugEWb&gS@>Q-d_pS90SZuP$3_xF3g|6iUbbI#e6P%Aqkz;2pY+;Hgo@&t(<@;!M+v4^!e)-rB0eX%Wqi*3Rz+~DBTg#}>LeftuVS?0 zoklrH5fMhp0}7|5gAzRDY!8<=4}GbVfIaQ{gX$^_u{w%(%L&(-ACv;H8(mIVL;}<* zs#B5J*Y)mZFy(@RIFK$VD49!8RF)Hv2~-<2ZgOgivyT2N&N}3#TY3nKjgYk@M@Sno z0E*t!BGTS*S{Iy{Nh_l-^$j~()EgmHl`?7(#XF)Qed9em-njZ8#8M6+ae5s>V(?WV zoCT1)0HF|Gt3qy3D=`NonP(mK&^q`&!AV~gaKd>PjuGGn=rn?tgcY=M_be#gNYJXD zq(c^Mqa^O6ZvU>ca1!$t99WcV!0bZ2_?!LVZt3MuKEF`YMY)i_G_{2WgZ!FeE_{^;Eo z?`~=^xWeH{XAAo21B`o{o*)GWA=5Gp^(@eviU(jSL2?MO-|!Giqv52KWA;cBVTes3 zjnuII2HI|g$ez?iDHw6%i@e$wKKwqR^ z_bc-1?L}K`C@*0jkABwp-mIV1+#9X8`4^Qi8=^&Xc&EEq0AAJ zB!!X60Cbdx6vox`gu=-2B-H4+p{Q&`fFPF)AO^B$mX04f9x=tbDkL3G5@7T2bX(^W zWc!5AiB@o#Uu}8GKGayQHGzbQQ_&yMULH!P^Zn$^GK9EwCi3eW|$1niA3O3G*L7CZ3jv{w3lGS zAe$C}px7pmYwFxYTdyM1;$otA91Jpdk*To0)|80?6ZA&L2p`=T;ggbJ6{Mhf)*&ju zf#-y7(w4N^bOIo{g@hX6Urw2*>lLvv_AuHn7*6cd%6YA9@88OO#EE z)IXcHx-IE1d8lVJ6&;Q#p1c*UbQHFCrCbQn|K}?OOiZGVVkp>#kkwAp? zGr=-JsDvW&pP#Q=m*5(K0OHrAO@M6aUx5jZSPJlg7hNz_K7XUo#Tv#EArTv<5wMBM z0e$9)Cr$*$lUR3wD3>}%6bJBhIVZ>zn9-}ki#c~kGSZ4`66+J2YaWY^nhpU#$HU!d zO$!G0cc~PN>o45HnnVN!*8-T%mMkOZ7H5H2q(d?qazbdwWj^#y4WKTjs!^4P_Jz29 zqd7*zLo7Pzs|*h@Hl`ugosek3&H0E7sA~1?tpuswJizf0!4YR07s7-aFCD{bvTdVr z8*jaAvgH$Y``db5Mp2l823T;VZi{t9wF+8bM8auOhMkv zN1iZ~>wg&-QX2u9=-W9j#al06Ezx9KvleARs8-A(>E_tJFuvtcZ>3oC2rURHb%od# zu@vieSKvJa@)bVJXfa#ulwQd!@=USXMRew#U3a!|Hk%5chKG8YYuk1YL>L9425S7GoT|F}Z=V4wU;fo7YJHa@q z;8mfUc>g4o&S~t5+|@e@dSD99zKf3$4|9xRAkHi!;#Ao6fuIlSF_b~O6yW=J>rJqR zKL5Uca+fUUc~z(Zqu!eg=<(jb5FGn$gCI}(NdtRYS6vn~V6_Pnh~8h(@p2|Duf&J_ z=G>e4n`c@V;7FuZm_}>J@qB_WYf@W@{vrLMi9vZP<5|9J0rq^9^AhS<$Z8^X#r8N> zqT5hhJg_>qa1SRpw*Vn#00Dv!&2-pLGDae8Q@|QA_=sLsC=L(tKz%xc5;!21g%=ke z!GIxFlHCR~A1$09bB%u?NL>m|`LGWE67-U}@4gUwS|CXT^!k4qo)#LP)w>xX72}ag zs6O_HE4us}DmnlSUa{8I;L1bIl>2HC(TLglr9Rdci*bV;Y|ih~Ef;C|;(x6RbVg0+ z&RzvKe{tKj*hfs5gSe!=ttuuejt0NPVZ3r1-GNw zt&l_g4e{3Bpp1dpHMiaXt1~l<4gHJ-;&-?#`H1s)71AgdOkT}}4%Qs|mx5NVhQeg) zdP7+(XnK&X*A-=PvEX~e^l)i6X=4U06ENmC6adHM4)z8~oKxqTTr1GD@+>XKb^*Gw zvJX$WUo6*xfc1a0%4A)H29^Aid-9oeK~ICMTdufd@sE`o^zhZfWDy*Xrbz)SpP*SDU^ybTX$Hf6x&D; zfk~1qnP~nVp2#WQ4QLYmQ7Ixf(s>ZIBAqXu6b%($ zJ+U4;!pY=OLSP}Wcv6-0jl_gG`V%^tNE6w13F-w4qI%WNiHL(HO6jnPqk&@m4kw?a z&(i+Yn0S}#wrm8YjkO-gq-&#-%IJQ_IcDLr1NM?u2b&q6#Abup_9p zcjKI=0V#f~SsBRjOrJYB2iHlDP>N~1!`ig0zb~A@aBfd#k?-z92>NC*e#DhT5d}`; zE|d%T3aG*VveySv(_)fV!gJ-@bzWdpF~e8-JPQ_-ZKtSW@4kp48wXPLqf|amO6Xee zP896d8o%iSkg5(|=r7{nGCz99(sFtyQ9ImY9>6cA0th%A5ggr?^I87P7(Pru;Bt)l zCXHku_RyjzB8rXlBttjc{{G#D{ptO98i?tp#fnU5FFZhKPikMh=TXE*S zh3WH&W&E#s)))oWDtXq>nZH8oK$YscC>LMT=A0#L|1VYyzKHIsRI#gGv3RG3lvn)s zPlO_Dm(`Ee&WEZ+RQN4c{?rNm2$_2o+k%i}mCYf0z$G$&k?R+dd87r9v*b|xgaRvXQpLi-|PQspAe~n!~L^U2sZVNZiH>9jHDeAe)AZKYVqp}sYbVhW^c{` ztn&z0|ChvKiLdORCU(}ju|wbq*RyK)c@RlsQ-=v~DAq!pU`BVj2WrcxYyOWQLH)^; z1O7Ju84@?1usiV|J@z~xssxEnj@VVI4MZ>gXLu}7PX8Ovy?9>2{z_+5yyr*43ue8dVEt}NT;7T9#qx(si-CqaizY>-w5Cd9=>S*}Y0*0Kblpgc z@8uMZgxg*5I^)DdUdaWI)_y=Az#Z5<8vpBFuW0kX8-I`MbsSWz)z13g==F+Dw|J0t z@;ElLKEmDMYcXa2Z)mxqRsX|SqAFd4W~t=hxDwKXc%Y_!1mf5gfD`#LLEsjum( zUqRu~Y$yKT)TuYK52`fhTBd`HjQ@XD{5XcauamoPEn+42{{f=)Mw`nkZh0I6{_pXY z^fqo@B6PFr%0N?O&}v)IF|HW(o26uzI4P_TZ-bC9t=6tCq69G?_o4aeen&Xo7vtU0 z5-+{!1>V5=3#Ps?W(ui+mlyirs3e`e00W`o$Pm(5GKfCmPiLSHosW=J9UnTkf=dM1 z*^?!~U{ zkhGCl?IFgn$$G>cQ?8|4Lby9n*S~PPi*{5`Es!XZ>S}~$*Tc2<@N`e>l-yClO75ai zCHiz~KVtS7E*!#2(jObkhe+^%@xpeEd+G7o{hr+ylXK%_aw&7_fw(HPB?76N43&G0I&9rApnLVG2JEB2uYkd#HzG=?xJuY*z#Z>PIfCacQoeBmZI<8Pllp>U#j)wO z0J)MXeasCwK0RnR6n^ww5nmyT$1%*{PpDAk_r3Vq}-9`2Z zm;9wZqVXjR5v~HM6UgSEZp@~FbrWWJ!97RdT#$B^sdZ2K;N1cZw#e(_L^F%=P=`Sq zp)N$JrKm-{I2qHMjsytlFtqF@l$c7m`?4LHR;x0oXqAA>rK&=maNE71QDhM8)c2&x zZUO{!&f`XUB2dg+*SbDjgE-+Of%5ItCl+|7Y*Fn>c>2dTb~zF7kdxI0lNuQjH$H{s!mqw3He0#fTsVZS1bc7#jau{}pMQNC2FR% zS}5-vs0^p2-%&rK=ATA55c(#O9xnZIYDE-KG2umI<@p@wU+oVccbeB@9#0ct*A_Oi z2IA2A1;CnndjRZAq24IKZ^1Ai$mX4nmblGr`GC+lBy zer%sa!+~Hd-Q#BL0?IeJIr23_KGAfnx@pNsqvr`k9&M-#UzE*JOOGIb2Rca8S8Yw% z&l{-a#>Ie*>=4P28jd;&x{cr%PNCK^HJ5O=l~U>eT8wXrnDgo!idm^oYQ@t7KLoaNn>3}J5$nedj2EGFS=fBN}>HlXpAqCs^1X>MIu17G~W_T z&3~>L0$b>*Y9p`u5v4L9U^}6T20{s*V(Wq!%Qz35!a*_f@vXL{{1@PvQ;xplh}N-0 zN7;%^39h(14ji?PJ54dLkbuq{_f^#(JCbtisX6?bo{>7 z1rgP9gD+PDmcG@ODR-m`Sm;0KY~c=Wi@EwFnl7lgVMSY9@JlV}i(b0PU}gqXQ)W)ay-S{HREg+&I@JkmR3g+ZdOER|0G=UYC}-u`>D!{trrO(6?Jg$OfC{R@ zqyXcmt#lrRaI*ia4{%D9HXIW}hj7v-8^3#$5Nb2%+eNmjOf?*YpAUde9Sxud6T8@s zfZe`mzej#75%tx^2>gIjL2SlSVFrR1PY(gNOO}$OBb8h#o)|}XLeB^6lpOL%$aTsh z-Z6Kl9O&g&sQh?Qsle2_kYM1&LeSESI*sC6Q@oDJbLx{6QXe7e?&hGkn(jC(Wvek1 zcegg;-lw=f@TLnwjCK#TImP~jFzPUU0uVC=N3nu+Uo#8VZ5#{^g_ROv|INqF0=Tje z{eMWAY%%7{(Em!c->rBD@mA)_cyGV-W(2(D3hD{s#$jSv@4 zaiz@=*Ms6L?#8}Ak)a5xLU81t=S$>Mi3QQO+J}}|e{XFn z@_Dbq7u^p6s?>hULv)OI)iQ>_?kpgP8QX=xHLf1%m3oK$y3APY@C^D7#Q5)q97CCDmr$w8NFu68Cg~Z|R#ieVhdMD*jC(U~jZCpeH@mC-dNpXRq zuX|-{52d6frX6_E7683e7BC^Q_Ci*@U^%-BO3=DfF>6qaJ_EG?Abr}7BOU)oYvppZ zjFJUYbCT%|)Bh@V^RHRN9~wt8?VyfCf`mK|)vw2ZSsK-*dJEja7IhuM2`2@VX*G?` zzad8)q7~GvOLNeyo#<*!MTvSi$|TnsHJS1Xw81L}S*iO1wInKJD++N>vPI8CU_&R| zE738PCrz&TOPVzgC%SgcX0|E5BUY_}p~mN4z0 z0>WGXIlXra%8uEDhdL=5>yM7~%5}z}PRAtmZwzay=rz~Kc6w?FAZ=|e5kdq;Dn%No zRZA(PxgJ7PB@p;2CVL>nI^ch}5hL#F2R?!Ekkd>>t3a3DAb~Uh!s9((LXxcEJrgWAu8hmD< zAnV;y-hgYUp2im@)fjGYuoI{2Yb`1Y6i}|F23f1+okdl8xPcI z%nR+;_UfQ(O)JWoMGdb>UC~>e0}0`3nW*#65vcP;v|Salqpgc*oVpa!9^zYDiB566 zK)Ku$_!S`6mQ+Jh+mR&y+9I*50+L@EDVbVZS1)07E10P7M}2~<0Hf1*2I{f)6$ODV zdQ*j2Iusg?78GTu((TBly79WhM-5aEojoF^9tjN<(>(7hljnB%^oSEv9&7a;He2n_&0I}dmz z)}>pV=%{o+q`^F(7l(PP-m1}nj#?FkDxJpXQJ@y@!mWFd%>eP0YfFL1R<474;6s;r z;rb&uqCi{X?i$xLABXor{)tNTxxNoo(%l6KDiOTRDh7g!w;e?h>XimMJ-R6?$%P6x zSRBS5trUUkE>}u-H{b#}7q4;}-<%G+j9K8{9i3iFKV$M7psz?c8wvIBB7_sp>FnW6 zgjEzq(^zdaJ`hoUK-m-|1gh7#%AvGQV;IWOGW8-Vcz~sfeMfsskJsX=QJF4YbmF-% zjrtkWX0oDFZ)vm)Ac(<5la*6{EmFG~36U2pM)4tvMy-UioDQ{ijk+5b+^2V@f^z%e zsF-_TK(LIbdE;pLRBS38goRRorMH^|q|-PWg*zHR)7yrwR9Do)F0I^Js?XhZNz`>TlvN)TA_knv+O%V$b$Ul^ubIXL;LK+jS%I;mF}yM$)wZ82GYb@+=T zCHyewp)XG&C30~^Y2}#lrFz_zwx==SZqPff4`Q9_1p!-3EhF4zPU`Pv0eC-%cg<5L zDvgFJ>Ja*=1%r_~RVNv}SF>q?P!4&y>^XvD$Ohg)^o?T-anf}`I_?vJ#!E${$2$h; zNAcCK=8r~4ZJ07|usQYgQ~2Oljzid*p&AG7Jay+6GLd*0&O!7f5D=kac)3;L8&eSG zZuKX>5FNV78Y{!3+$oNFL`~(vv~^Dr&3sHNH&SuA4IXn+oez#ROS;y;@&Z!)oAb7Y zQ)VG!NWc-hgRdLDgFVtm(nXgOXdTLz@Qf?lNX{oYPX? z`q58s{7^ecw%|HbU_FR5P|2pZjWka*@szyUX2L6ME! zeweVgt7|_;L}t+#ib=h0p+c7jjEa69AaYNTyKwj&;(8!AxFhjlX#~)lRa5=oPW73ds+*oFwQ8!*Ch>?~tYK>}q$V*bXQd+fZrGI2V;d;Y z>K%@b16snKl=Do}8d z0oZSF51BA7>KA$cfj6caT4HKt_5KMT(CtW$qFP@VNQzrg1Wn7Sirtw00>Mt>CMwQ0 z0wK`7;Ti5UuBlo9FR$QLsvuGQqJ^ko{y%C+sak_54>b&+@~lCjNUy-pG8omMMl?V@ z8N#RbW*x#$w?&Dzm|vcQMaSl+4FOhc-gC&H1HgRq`y+!oyoO8ZR^M#R_xEO<{Vtn8 zYaW<+=6dHp&As}tmKGlvU`@L*66g>8ybV>u^{CQ{-ek~;oc$w#kn7zLALi_Pa&`&6 zfy+Voh6nQiJUr4ynxiB8Q|c^ID3Mxw+LihuQWO6PL@cqNRb9>D41-63q4P?kYG77I3G!qb+o=ZGI^TL^Gek!y~N=n)x)|PoQQZdaN?`T6y`alM>p%q>!Riu+vyJIQiP*=t-BI6zO z0@DzL`Nzj%ZQXnOXzm}!LYp)QB3dd|J2t}aiO9^EejWFgpa+hl7*|6`bkEJW3U(-z@OPIM%CQJ1kIYyB-p6+go{(To2*I?&i4C z*Fs-RR-AQOA3z%?7kRZlh9@;+@nF%6r01IG?X;jdYArNcIK8koHqW?}{o!iX9@{wX24yH5+*Sr>_hd9P!@U2AY*kqs9PRPz)Px4{?S(LQ$N&aeo zHpH`0C!aUo9oft;vPLJ$yzrRX4aLVog!!HkD(`sj-YB3M)GMbN7SR?7)C_YzV1Eq(j`5uKGY5Zyj(8hC*%A9KFN3Lai zto^B5@0RgTOl(Z;6IhxNiumG@+;1Qo?HPrb=LWK7(v=Z>*+ACZ^Uj?z{;z>7%X21P z!Ur*<)OG|PFo<>V3`4-&L9BO+O}EQnyA%rk!UA!t^uhzUJ49Hp&#Q<2D&r>yu`|uu z!o)6sJw-8Sp;3rGf@c-dd}n4&rC&q2W@eo%{U9B|JF`QP#pYc%jF2?)O8zJiwtD7X z8BMuX+!u6=3*QdtbslpXjkE>~LQc4aB*dmhDs(^YqF}fP!PP4kZbFf>M|tb^`|-|_ zM zGH53+8yx#+hqNdJJ5OzqZ?)kahOl?que@1_P_pq+0ZisKc!bE?+lp}UhJ zS~rh+VbNly@%~Z1dML}0_z8RSVN8~!!H0SH2o@;CAGQx1!NQr;@_YM&k?gi-t=iCm zDNFqW{N*RvGt!kq_VZ7&ex7wNbO!Pk-#4HRdCAwDC9wwf{^MDFCVdiNA3K4qWS%oU zym{$FHnUAoL#%sb5L)YBBrqR`u{KwEb~5WBW&L3jT9CZ@PC02T%3|(c5>@|R2oy4N07EkcjliA=}(K|&0 zg5q}ag_GInT4%o1ze~QgpPS6yX43A<{N)rjS$gs^zmURAo-GYN{E6q-tJ3@T__xoo z;5zThZs_@0e&snfzUKEZH>0XccX{Fz7ArNl#9yDnp7$JH*M~ox!rDqLLV4?{XhXzr zeCSkGUwW>sJ!L9O_w2l*7J$xOcno73yY>#5t(yAU&(w<}514bGJq9ZL0x7_T6!LpJ zq|W}C`^`YVd|~HpXRxW#^sRj53|1yh-(p|+0&B^n!{_<7ne5e|s}E?KR>E?{W`pG( ztT%BjJ~x7!hSv1q6KAn->9gZ}#VjEB=?DDqEEee52@xtH>V)9pLOQy%P<`q&#}R+A zekHzyTkx#UQH%mo@ z*N^a|RQ98kd5X75V-pQ8og=LI?Zv}v^ZVsIU!BI<8=n4IgnoqwLTCQMkEOBJ_3WFe zmu`6P7S9y#;?2B9I$I~Dm+%kMS++E%gm;+Bo^ILD3w?7Mimy8Id*IP#yzU9X-vT6& zPd{p(@A(&V*odB)~gU4)Qbe*#arHjeXRMth?8QnGXW} z7db*!=^;iD;#9YSb^*JO$U^r#sj2(9VBtU@C+8i>ZaDkRilgV=>u{bKe~hk+P5ucwV7vMKX3cKC9DbK&9j-ml-8Jc z&SsxW`x^0!*(@UP#ql&bgNTAj{JbvqCOBKa+=#bd3X(H4;%k<&=F*KoZePlp)jtZa z9U=6J@B&V&-uZ^BOW9oMoxMDH8EeQk*yk@}^O%&_j^E5-qol2c++=0Z?0I{pl|7Ky zH{9|n>sEha11jgJ+bh;-^adYkC3vkzQm1bc{9%++A&-lA4i$i5<9X+4BcVmzP_LVzZ`1Vz- zHQUY4tzr#>u53pAur%bLMyK&zVyBSA#}TTo+hVV`8bmJj*~UAqVXxNuGM9MBIPV>~ zeE%Bui4?Pi&*m&ldgg1sjd%zc4-m13_D;iG~J!p45Z-(Sl* zw*0{dRR~pZf;RCFA_bf*4VCX(goB1u$0mMjEvWqKFZrN1(VuI-&_`}{HijNg@IUT>6+Z(4f*oCgll*`h;U;k(~reLH^*gEzkG6eeUB%HVctz<=H=YoB1)R z=a>BJbu6-WR<>T_tJ$JPF0W@brN5T(#_L)4&XXX)%4>^q*VbBO(QC7!=F|!9+TPCQ znd@1M*ERGMznRAZ_?h)AKq~l*UtbUD@aAUz>^p3Qbm}dB?;SRV2fxSKw%PwX38D&) z667H!mI8hLcqjD5y~k&~$6CZKLRyEvk~@!hkXq0W)B=693PajUj9Bl9z3yK40%26l z`FHuz_gHk07m5+os95WSD7b*zj0?DNcd3&5#84-1{XW|+@iKn?eb%Vff-*hp=8atX zfK8Vs{EN@}00VaXExzId)>{g?#gBi$dRgROi2tXB)F8m#L-QSd#Z8(;u=jg{Y2Wyn z;$b5aiPSuJB0GZ1Q%Fd0xN3^)80iqMg3EtRuY2^>oy>jVWA_RbZn#4F8%$5RLxOqJ z57;p2`o}tOf85l8`|PFwZo>Jqm4NGvQBV!I-6FmmxL6wQRe@VyHN_%;?*h*AD?aT* zu-_Ly;+H>UA4?fe^VdFNTct)Hy1s+?^^e(%&T;3fcJrI}%KJ%6I9oAd79*Vc>dUw6 zwoS~FNv~S$EB}Q9?;q>v_#QOf(|b@vuCWA{~C4ugGH! zr5`WzPx3IwdFL`O$z#20<;~Ft%l0|E&6li8v*F*P5jcDWClAw2Q$^~<4FWO5gztXH z=Y7c{q`yw`PrhUwq>eZEnJ?KBlJh#R@fB;<`uue_0?PfS-SoN-QG|oPmh*vMLDs#0 zgTMV1vTnY~4P>$2E7+|vh1G3xPl$s)6d$_*}kb3wv3L&=dIQzg8u|7q9Uzwy?LQ zXC8|D^>5g!M3>-`AKS*P()$lY0{`%;30l~=&CV7{H~-e77rj~~|IAl;pY4#O;jZXi zxmBV+&t0dFT9M?D{hW`ao9L3ywer_`nT)_%h4Q^MWW64;eHSj8TCk=Sc@tg zO+-Y@@b?9w&7gQeXbwb@FHx~>gsVduBP1D?h~abFE5vDh<}wis1jCbfA&F;Q2AMPh zT_7fJaiil##f^*`0mCw1u#+|Rdrbx8XP~$`jXi(o@9%_3VAy3oeHWDdsLOomF1AqG zafy59v$c~tEk$KEqwn5Iz&=3fjzJ*(9UBFtKkRWsIuxyQL_iNOr6Bku0W2J5utz;W zSui(^S*mB*Se~U>&YImUVEQI>9}y~;S+d%mRMILfTzt{h!ufb1BHomR7N#N-UDQa^ zTjOvg7l$W-agi-{sGc5o%@hY}zZ91_E37Cf%Xw)5dqHYuA`v4elJGpuc!Ibds#QR^)7$_UYtw_KJYsKXfF$C|L1=4N5a?` zdLFUp3~@&e?XAoZjgJqBN+5UUposJQ_Fnc?vxVm<-8h`Rm5&7UmqDW@ifjZ2SMmM( zSgSF;C=D3?^b8lREC`8JUq2@}ojMwUYD+g25w=biQWueVcG_M&O_z#{!RjbiV6zM! zyPpL?Ku*}tTKJiEdbH8K3UQ_+@34$@o}Q0VwGCJp$}AfzXKSF3xU#|Fa@IY(;cyR= z6&42AgTo1GdWnLTCthMHM89||;t!lvf=kOY=^#i`?x;XoG%P>X3kUd0wEuoy3SWGX z)wje^$+REe6nm$Lr6s#-p#o@89i8jS5`Dh`Ph_oK+ZaeP}*G= zScJP@43N>pGeK8%G0fhD+oigB4Rt$KVFWG3W)vNZ%_yrSXO5#%TvdxpKUCUnSgH2b zm1<9$Dr$bdqGn)MueG95xXDnh9jMms?phr)U|GokkaM602r#kuUIogBJCMFcdHNVv z`c!yR;&^Ucn32|1oM-uOvhSko2Ueqz7HVZ4BLfO~0V+MGhlHogR9Coq;18OPU}RwH zScn{#gWI9$7{=)#v8LR?!6sO{V&U@#gf6kc?nmTE%aUS*;%tMvs+tXsGfsqK8LA@u z7=_1+YZx4sQE0u#c70serCjfW)oNBsPXz|zMLS8>WcsR7%qu8{I5M~tygP9^Cm?ki zn?j+%xs`Ec{llrW)+gu?p=)XN$QIbrY;9$>z6=&M&EV*XkhB&S+Q9`Y_M!s8MI5zp zykD9O{(_R7#$!JbtGkB;B>3AcC#vY@YqqwBB08qhz9<2lv2q))gE?vw$rM9y)3Oc` zm_W-96cvvs;ii_P!!Kv|Mu-RoLRwTt$SNGTDTvWwF7Hm#JRjN#Yi`n2*ROOZYh5MT z$*Rfbx|0#ULd|2*E|O9jR70mD0OC*Z(x1-g{IBNAR48^32^?dIS>X6vSFGbD5sMte zI}^{Kwcwd25vp!n0_7p27)*XqroFTnhC6B7LVM^TrZB0)FZMa#F>faAzs+C#o;@wC zY|DT7p3Ranz0 zi;!Cn@uyF+PF;{dcW1~$QM5XliV!)w8%Bi|jUQ}$RFF;d7G&2Q_2r-V-jggwDxJrr zQ!Kg0U`)K!XEXTJQ*4a%X&T>qicM@eFir1JU#;ccilf3(vb;RvX}6pDiiO9YX2CW3 zAw->I;nPmDcce+Fy!IJrlwDJKk29=&jT))w+cRan&lwiPt@x3O=I~8tV71Je!>^u! zO5Ah~ANCU)Bpsd2H~hq+f>zH)juh8`PA1C5I1*AnGMR2x?KK_R3klchK`7 zjU=b>^nTvwEK8`N&O#|W9#XxH_y=cMiy9wO$S;rdka=gBUyYY2WROVHfXhF_G9NdK zKk+kb*1ZK{h~#~>`+uTnTg)ox=h|X{!kBIuULKD52<$^IGieA2q+Sc;OMYg3du`f- zJ&JHHru>=rywb$EW_nK9fehzyp6}6B|Hja^PSbVZ;zD2WvXkJ>2#yo8CHg2HFEZNB7oPh;h^y z*BE zBp9C4C?OK$?`R6|7SgNE0&HS}y6@wq-4u&6FL7!4Tpm-xx(D5R9zc4zSeqaaK^;`@ zug~)pC9Fy0taX)ud4DE#rr@C4(~)(HyM4sl%d^^^=NC#~K$-JAe^|nLG|0z^ctQ%0 zroL00BB;LS&&?ND3rq@LxWF1(M!yG?^b#67DPi}pOC9HzQ+oQ;=(CwRX9^OP^R;m( zmUz^w$%1oH$-$rtb=?O@Tu)6v;_?DQ^U)7Tn~bz-d&E>IQOid{q32!~SwgKsfjy_u zkK(D;d`}wMMEjNAJ;CiOOkyDKYE!pXOsAi%dDR~3>pFT;F7Uu zY~myy_dAO~TcJdj%<9F7%&YKWT4oK0v>gg8z?(bL%= zDqMDSm8humQ^@n^X};nL7K`%i-ftA!!dovSQAChp()2P0&J29xYV2UFa)l<^Qedrl z;wd}tpfa;m$If3;St}lt&FYW&aT{TC9Es1#A-*Zg6QeK7S*t-kIM($uJaEdZsuY&s zPNlA%x$&NXQ*W$lC!P%IwTu*j@Dxo8CLRqZ z=B{5;5pa`KSGp2Cr6V5n%$B zqJK_%Vm>tWhge3wwbj+i_fP@!C-_{F*44;G_2*Qo21)g43e0lAH5iLnCm-@P?{kf{ zFn0K7^PWuM>DPdPZaw*?Yiw%6_OI(0&@FEIbW`m%ornIxCQRG*+G8RMrx6bN`3ggu z)f1f*JeKDFR##%!_AqA^xb%5ePg?9L1hz=2-!f5ch1a-^``3}8q)9OR){#P_@ss%M zI#RgT14tiktnYb($i1RYO2UZiIELq4XMqtP z;A~eohX(6p3Z^hYTY(^&-s_?25`5AqZcN!OkLLHUV=?;rpFI2qs?Yk254*uSw0Z~9 z*JQos&??o}_cOuq)Hy?tF=jN4@F4#A9S}pqJ$Mt6mYWs=uog&lB~$lHAA*y3Wl@;3}? zl4p^9|3fTKNWbj0Uwg#HNz$S{s|QL~q}7GI3zPnC`?`hbF)bvPa4ijo_&W*+;;=qL zvJX}ToOrJunrt5-Nok(`A@xudcEgcM3KLHHBL0)N6zR8U1QDf>p#4VjCO*>B(wQB6 zk&pD8G~hD7;v9xt^!ikmiNdT90}${Co7FtU@IEstZvy1-3?@TH2r2 z^_4D3(mMW^uXI`pdz&AwDcz8w-s1VSB;Itxn*@1GFtDSx?@QteU+w5Cs4t&gTWTwf zdy~IaTT0=jb)^*P_kMhgEIlE$jpfT^scXoaZwrCqG%6?uldMGRZDIxuT<#cmRF9rz zJYpqWJ?^dLcVwwquQPB#B+iEEncM2>t&}&s@nGPtIe4(XEcSMfD)N{c?m>Or_$2Rm zkF^Zh_m->f)U}VVyL}H=-L2O0+EpKSWKjYz;U1jRCAz|^rQZ~dtGU?WrA>hIE)G-f7d(=)CHx= zaaCDdwh@O#d`mxy-1nRkeMXT7K(~C*H8m^&yE`(=;^O3$1&a0JjxGQ|6;_MI`>2G9 zK}7EFkbKIHF8ZHmq}djdf#cTa%{0*@_BQzPR$xKQ>Lvqvn)OZ)_UNV|Ld6ynR6caR zR{+0u-pkiA0d;{`aqPq;Oz7|iceyHcI$Qs?${QP>;o_to@9t7v=wlx~h}OW28ojJb zpPSu)gq^)B_f?8ep4`^ZYbpkOJd|;>$$G$qYgN6VjvILAWnOI};zPy&NlaiYjV{_P zO${Q%;fl1dLLBcLyFC`L0(XVfkU&QpzI2&aE82LZd#c-VXg)@hVH^~p#L~+YC9@!p zQqPLpMGPJXT_!{vG}B3km=Bn7_BfOfkU?VDmATU^o6XGDA8;)52LjhPxF+_%J)g~3blj0X~JU034z4ljk_T-~BlomtC<{sp*Fq97Fadl%rE0(|s8hJa)E z#{Xcv_bfXK_8Q|rT|%{%=vkE9go+<+@yQBu&iD})S(5>m~iKHW||R`ShcX z!IcWNRFvvQlwM$9MS*@*3tU*QT7iky2j-sRLITmh@{*r0a|Rk0c8L?(hAH!bGzac^ zXO-hu2idy5t)mU9#wq14j*IJR+2#t-U`U5-KZEWSlu^lKi-X7n+22);U&o2mAxhXD zD84DQ$}>ihzc_{eA|>J&lu~)uF(vcQJ|^o~qNCI{+B$Ij<=?~-O7$>#i4{Q=7!>wb zx>h4uziR*&{dLXIhYu^Ifihq5OrsOXwuEG$6mdE~?f#_&c!vT*8`d$EFkA|AMWHp8 zC8WL)jngfZu?j{C9MRHXTeo$f3qGMYiV>#g2aL=jFTH^t?;EwM4k`5Vk}LQ!IS&rZ z-TXaCL6h}{>G%yC<7J|@ckND`t%X&Muc>C1u=IwYa_~4RXx>ikBV^TSk=hbgN}Le) z%6OSQCx;kf@9f7OasmqbS)$T{;_#g|*<{Rogz@$XYSNy9aB(yHPRzTZMYn62ttSMF za3oq!s+ri#nrOXBnk_X;G|<&lG@8mxUxyRVcjKxEFEfK#tW?Q7z|8Wlg1F4mJ_#V= z%+f^=7UhMi5h(F-Wx;h8kS*CtlVyT&$)`jg(RVS!GRb|j}KVQEj_o)o}ZFo+&qdJ9#r@7U$78izon+dNhcW*A>Lc>%ou@DYE&r zZ0H+{;T|g+B-t>~ARC4S%Z8`IW&bhwPYR~{l4O4mG{G8&CYacxL4X)<3Ok$_ZNVh( zb+SVd%Z~n4&Yp@6(iTz(T_f5_S^)px52oO&=64& z;q2iruLEp3ISb2Rh!G~RJ*cn<4LIjSp(3Oh*@bbqmj_kuPCV#eo-Gvtc-zih( zz9c33x}0?j@;>K)V)M#?95%82i8x6$8Cxvnr?J46h>dMHhVrcNh;4v`-f0c#2N9U4bZ#|egs@N2c+5)r|OAPvyi$DOn; zLP(DY`<2K5T!3ioMVrvP)OR~V?@4l=9#d)3Q(fJ!^)IO75R2vNI2l!3dO_y3K(mEJ zleHK8I0gio!%CGMhrieVl)D#9)*nrU*L-x>jz_svsj#k-d7y%d|DBz{WNYdZmmBXR z3WWyp7`xSXG=Z3~5 zFYO8oGx@@NiaZ2PFqy~R5g&-(fvW;2{*YMGFh^gImpq3I+9_Y?ZglouKcUTGFl-LV z0F;T=N3O=i(XD(-c7WpHWs3{nJuyP9P~Uv?!`~p>rXCN+@G!GKac4tpJS|#@PN{Elh~fTtxs&!eejp~@WR-r zvRHXtQG9rjA}8$0I%tv;iX4q9q%I%@Si<0NvMyf#LER@zg;#x`a6-+plz1iDLVV(I z=bulk97d`u?gFH5+^kls^BQRD&_%4N zi^-Y>b|X~Sa~v?DaOij89m(}618tEf7d7lSo}$rLZ;$Q9$d*| zvdx!FnWbew7xbPIuf8A|Zx8(9swn)9@CI%S!h1ahxxHM)(3$T&s<~@r6N>Zb;8SMQP^5=tuIB;~3L!-AVBjTFY4|8}YEq z1}EU04jYcjRVTnj7K0oNqy?r){*bHMTm)6dks#ZPD9mKVm0MK|#Kdz9W?NHFatRbQ zV50Vu7?%~D3L?REv7&I0MG2l5<@_&UFPj*A?Vk*UWG%1(EYHdC7eA zMxq5Y4QD}+r!7Dftgx$<$h&^^a?uU-sT=B3H`J$Ys81{UC=)uZ=0mit4`fU@++=M% zmq=E|`x&&Xehj1-9D%IkKs}_485sUe#Mp)sUwabpRv~i5HV?iccgrvV-)$i^8;QK4@MDe=+%QUT z!zg6?2azLDd=#pwO5}w^HO_ zNeSbTY9vx6B2|KZRZV3vo0&#^OAO3<98Cu{td7BzM**=qixnOlF9625200YcMh+#p z6G{@uzkeX+_^<_x!XybQFAhak=y>7G!X$I_MLDYh6^}_USrKeqw~4j_r`>_k?!-8{ zqcP*U14gedpw%u2itkRS?}DFkT@XH^OE5IVv@SR;`Hq}5$cymXs|zd}G*{_`PdL(d zfuv6BvK`|w!6Uwhk8Fcg3QPVnj2;S21CxQoD3X&&cnPH!;AudhE;+47s__MY(4|xd z0dIML)Y7E@pmsS1P`i{=z@-BW$e|!5!P`ho;dXYJg~|b25gB1~cXoln*(o3+14w+2 z8Vfcm?3_lkJUYia?1B;v&mTE+GY~~HoWOYN9->1qr~D*AG&dZUX#&J@aRgI6O-6BT zVVw{h?SYvdX~osL2J;w{CgwDu_#J`CIr1bYLcdeU83uI)o16t?_aVqkGU2?sPxj!MSEX>a+142|XIgw%OEY){?~BT0#X)%pC~FF93`oXz8o> z9$x$E-A8YF|Cy{eX{iZN1e8*f18B^~2jt<`98iGY5dp{WJ1(FEvEu^52^9gs_%#O@ z@aq*|jy@!3ZAN#PGIs{J`mQm7oVzN()pwThP-OPynLyMDJsK}~!^D2n7Ey2867$@u z21b2-aM(OC5d?`gpcylUv|H!ar>z@lIAjVXbG7L31WaoO`T`4qVrqQ?sGxZ@6iqn4 zpEAql+W_8&bbi& zLL6AVnPF%aC7GcGnsZH5FWlxarT;mhhrer1k9rGH??Kc{ym|oPvl6ei;FFj&aUgIa ze{dxK8m7BHgI?Yyx-qi@rn_g{s)?DwfLol0`wivnKg9SbOcH!|RD=^1E76ByZ0ZF1L1?vvi z)t3;WuEq&9Aq0=S0kIGQxxY zj?owS2j>ofwHZ8Z!zP3)xf|}`fj*Bo5e0G*B)&)9hUwadjfjZrfwyI)loBpjviSi1 z;f^LKdJ#&D&pBZh6m7YPX-Pms*C3iKsumNtC-GW|(oO6h?8G!6Vi7^aJ}`GdL!x+p zbFP9p&=`7eByVdU7X_1_AbELP^SG$9Abw0kkHU6z4^e;vd&EVn#R+v59VRlFn_;*> z3M4E@%#FkAStLFlCvU48*8{@xAik)S`(%Bh@mtOIB>Ptx$@*|xN)&?uN4gQIM*3iT zCxb%$N6nJ`ErXL$76~$xxwJ&2uBI5gb4;XOQy?C%<%!4Ajr6Fhf1-d=Jq`7w?7zk! z9`6LxLnstWiJM?mc{~l6+>sbC!^nK}R23?zu;(PvVeHmhShMk+Gfqs#tp~(h+}0}t z2Jb%RsTXK2jt*>VvL069E)AO9&%@VxbxeT9NvqbtblON*UGNwzX6b1IbZQzzQ&zY+ z<54r}08P7LkBmBo8F_4YY1DB{$YaBA@?)vqO|fWWd~bwnaL;Z82gM-iR{3c8ik=#gfhP)`bFQpZ%DgAKgW{yFxW^(7mVy3OVpMyz)pnPpdv(htl} zn}u9!MXoh3*bg+6R!Gv=>3nQs>8dny8XwR^YA5xZ##5U}nNt4q_IpjFPduf1&+!kM zOS7bnDZD|j^ry7>SzfD!baTK^A25u5SZHn7LiqI$?FT3RNc4re%oj0X!c8cjYE#=# zQu60t6)ch#^+nQfN;=M$_iZVSthN3b-2imhi@)1a>fm?!+4A;fP2y);N>>`cOi8BQ znuUwlP18=!QgX*1gUQHgob(L;xs~*x9UHejHKKFz~gOO6Ji zNMf4yS26HDWC}}E_cKvAg z?;%ZK(r4H~-c$OsCQgyse}AU&&lC3Py(Db!eeMUlJw}?#q#MV1r#{jd$#~peGgkVP zu^;%>IO&?S-q-$qUn$)ap1qCwOQWQi{`T4ZrMD#GM>jmNWuT64_Ca~o9&6%{*i`-& zvj>}`;Y^C|%hLx+iPF_L`@VrF)N|bM=?5>im!}SsT1uyPaz0GzEPcI`e>Y6( z6_ky`S!I8mUF+-mGg&j&`hn4fdcU-jhYXjdN?%0r*N01~QjaLR*9hrfo^`!0czS@J zcZE9c?fjE}J6hW7H@t+P>I$zUznOp9bH_* z)w9e%n9GUjlOvSMyC0<@p6#OaNg|KlvZqk#yS+hLJ8a-r#2!X3~`gZk}9`vd^(RI~dKxVYO<*thwa9B^aR8)thgd){( zSO5@@5OGIPKITlg9z##JO%!(&HRPyY9TW+>xQkQ$^)zRZM$hD-2h;@Q;;tHXA_90^ zinKu**V=wDMXK#xYj-eEXY-ySi#`7Vkwp@HW6%Hiy#kTl^ax9&*&WedL>SlMn|i3% zJ^@C8_{5&}BK4_a3h{e`GKj9j;Q#KJfwQ)d8g`ubmqc!y>MBSR;-@K zr%=6k(R2w6t{IP=AvI*1_?Q{eV)ijVJ43q9KH@*VAhoDvDFfTH9#uOPa_0+D=UVT( zzHb%qo-?Ic0msF+;CsIph_;P>$al|_f-UPQ#u z0B4He~I79qw4t^!ZtGex$K zn=Kt?Z9gIz6g?mqhq~oX#M09|xNANaNHlcDZ2eOxTsYscF%H&{+W5WGgycC!hSNBx z5yy6~FsW4&{&}ht)X{`Ix~@R+>cwv$4`}MI32ElrAvy48TdJx_Bg zD#zLSA_7EVFlV{XRzIRAf;_&7~Km_I^&pD!BoL7@|#2@*XcrnQR@;e^DAV4wVaC>^!ZP_PPk`2jI&a@FC;BE|4&vq9`$Mw|6q~SuFhFFQtAlbQpC?Ll0xeY zaVJwVig;ZMLV8mDf^(aTcwdXuCc2k97xwzY53LI@bf2=pdM?Jl7q&z=jPcfm1r!~C zo-H6|I^!~b*CK_~*;Zb~uKoPDMGB5-ze~r|1hOIMc$c&4()7U=Ik(KyLo1~Sx`g1q zu**bFY3f3|n!KNP%#dQ58NUZ9*!((|(n%}cUH-xVw2l1#`1%g8s*dJw&h9ya0)oOt zP?{)KR4k}~sHj9xRO}_T*rL%`BG?fWP!X?U9eb}yRIJAiDuONc-bB=wLgxM!vWSv^_k56V2;@@1k3t>z6duv_tWThA`3111E7(~XK{R{ z%}x7aLhX^?(&1ubTpBe>V!`Z45p_b65YfgTp=Q$|xkMeIG1GxcXWo!Hok3{3K`*8= zH@g{d>Ut<%w7c{=a!z5IG@ZVruu^t|Ut5~54c$xLGgvLVnjjxD=KAlYo-AItdORhYL663)2*7bgW+?T+5IOF$YO)%NV6D>Hy(tsKX!SyulJ}rVgJyG{ne;vC-*t+sNGQe*6}btdcU5HL)A? z2h!>k9;{OPnQWZh9+5^zuU)(8`b<{AuI<5s@)CBF?JQQ$t`S56sjF^5&P)Ak}- z5T&+uJ3X1hLhULFGSNq~^LDB?mksguSPzJ(H%Bl##<`V z8gHZ4^Vo8`1vv#}FG!=0^H^3mtAXnkA*4(nI zTS*m=_1_@tw~{s=G}(N2L7Gup$#ntqvWr;*h(1Oww^GvutfJlav4H3!>$;UjEnx8t zDtv>?*rHQ#VjI}?&F!s{0nIMX0y(*k#WDerEkt;FipU4oS6el6W><{!ZrNd2*h|Kx4Qk43MzT?CoNgXs#cwBDu5h-Uin|tATn$hb?NxL z>`!`y40aO*1S^V_HsDXHw1`y=bSr2NR#?psrG%;yJek&Sg@;$d!h~5+tIE$uLgN^) zVifXpEP3;^Xt8!piKZ`R4aW~eCF)M$wNfK*U}^1LKCunYL*;>L&aDqs_$rC zr%+9DcJ5}B+Obdp+7x6#*+LFwUoKkzq}EQMHq8E7Y#KATu!reSTuSMBK7V$BwaA~LpPjzpsm@;}=`=glGCBfITwW;%|-b!2o8S*Ff zSjpVj`eW*tm8`F1ef~Tu>pn+)u!?PDc5?<748+Hkv|ugsDZOLk_xkG>nvDZ1>B3s} z%-?uMwEY22gtXVNg@asP>50Bv`JK|2tND1#1k2U&LsbQq^fJR+Xi;a`p#N>Z0GVyYgC_g}$4n z=&f!784CN{u>hIXF=_&J-pm@NzJqFQ!HTiuCt97NqlwCg?p6@bx98G?eaNiy7<}zS!0bPbyFI}hMQLz_o*vgJD zmoqdtjg?_8%V=5}hV{>-bS#bi!0MkS`)#Z`EBb^&wn0JhX*iTFgABNw*L$b|~cpZXm)ERBpXd1hREwq~@7DSzqkC;Iod$CkjnMcj{ zvIbs7VXO=v1IKbMq8r~{1Ht+smfd-Rf9?Mom(LSXs&LlMq#P zzE(U%jo-)0NzTWG?D;Q3yqZkven?mqJLrVi2?_p1h;hj(B&-mLl}n}82Z7znG)g|m zR2DXinjFGtJ5HsEhaea~99PhdV^iqFA#`KnOnP&OH7?V2ilrO#P;&wA_CgoDW}3TD zX$lQUXB+K`M-_A-I*CeWFh8G)#)2+1NYZg~LU;-nLTd+HDTp+Nco}J62J>fc6KQb< z)2yGRfUuNAN=Fjz zHln_}=!IsUPqw7iSOU#B!aTh;qGX*wg@A2>wV^SfS1W+@#rN1`I&p-(vzs@qAY0}n zI(d|NuxZJZa}>hC@JZyE$udg?wbaLa5E3ThxFQUaYqmB&o<3!=du-Gcx_Jx|#O$dk zMo8{jN*wVl*3IE8p7q%$jivW+Blp`LLxEwf+uIh z(wS`LR`GWM)YrZxv0qKehvAXpghNsjB(n@_uSjD+-y|x29246tmRcNVes&iMb~Bn? zEKNSnI@;X^(>7!59~0=|aaP-|NOM5+-Vd2TJ|~!J{cJMEd&_vbcY>9#G!6a@U|bGz zOKc=WfCIX9v3c+IIjkUSAGjDe$^5K`{0@$GcM$bD34z%$fl^O0R~EfQ{rx0cEZL`R z6jXK>*Hg4#2dQmOvmkCaX`K#(G+%K5(OEXAWQSFdKLt@)lK$CWEqRVDV#Us{6oh+5 zE817xb%C|z#U8B{fGo|k8#!KKu5oYiEe7^;SDl3a(m%7cXZVbiKEW^^*N4u4womK% z&T5xLgi*P?)FSYrLUBI?OG%4fddx);W1fkE7=OfIk$D!QFEYgzqei206zQfoIMEGE;^#am`@fT+#_wlw5OPxbQ^mdx4l5j6D%&O^Ko zsh@7Jo{YU8N?mWW4gMX6a;uQ`QBcPwsvy!(2UQqh2ZRW46CJ#=5ZHGp`QKrN=Jp~w zvV+L=TcPN8zXIG`2aD$%3JSX|z>#`k;w7E}@S(jl;|{pRUn|tzci0ui;uC1*eO8aP zRVn*E`xy6bCeUF_+b^c=(zN-)APXZ`VY7MU5QUK|GfRw|9%o*myG5Bt9?@np*RPI6 zV(ik|=`mxGLyrM<=y!+?YYWUVkC6vrjecSFh{zE)(HvnCb?K>v7uW0B&t~v69V`^F z$ANzB0IYc-N4$cd6=|Ekc^Rd9%Hy8%s-sc3l(0c<~t4O`3ACSMGyOurhQKUyX^1r()D&J&|xghdb*n|aJ zW+CSL8MFg6_?wkt*@M)Mf3q;jW;)zG_Q?6O53T-(ePpW#s!JcS3NU(}pk_X1_hm*a zXv0(RA^(;1{wa%M@0U~fGnUEf&7$JZahtLDEUNb$@B=d``8jyvvYE8;IpVJxY3K_U z%H|nq+Y2_z)7=Q2zA85vko<$V{q!~Yc*4h`qsk;_UZ&}I zQeU&GUcUd%_4)@r*HwU#>-0jp@tR#=jz7?rH|z{+97Yr0LVER>MwQ+{niw{fuDoMg z%LF&obM5rebCsA6%&1~9YN1Jtdr|s(wyaLSCfJnq1lGmFSkNJ{p|Hm`2DAng?Q|%s z=Pf@Ed8>V&S#$QSAJ{S$9ZIb~GS{N+Qv_L6OYo*)A6YsZHHZ8^u^>lREE@o4d+O!* z&7pChSbnkMc<2bwex95Te_=kfDv!mpgj6b*&kj2FTfy;qw2oQeBMTLbIoskXWT)`$C7u3LIJ~DIX;qt&df{u5%#rQA9T%vV+=w94{4~vTNQ4G>d30v;Yh6Pd$Ti(;T z6r9OG_i8c}*8}i_uf)?TTV9dfildXZyoFOLJjZ~~!U-b+f(X^JRMCz%E_!<=x@Yud z_B7m%A2f`fC|e0X5-&^_MS&M_UtFF^P3(E7ht&u@<$GJbZ%#;<>-TM|Vxby)Ze;a_ zQ|scqwo|!SOQTBbfe%^>FPc}Jj|*NfL2m-~@KD$5iguIlWWfHbXCG=#XEO9hK&Y1I zds2i0AL8Q|DNlLMN; z&ZUeUtIpqyl_}DWm1CQO)%zv6BJr#-vi{TnOAMl7&ODHf^`$Up9$K_M)N`n=o-fUE z=58h2#HT~EySb72Di@;AUT5x6e0`|7SK;VOm7nNuXI{PdFHpFP{`CgzeIN2R@LHbl z`eIgc#fbd-k%Sz;_c_pP124n&_9df%d$H+#X^VjedA{w6jj^pjmw)H+7I}*4d5VOJ zu)R5~^+DK950~y(4)xs>lKj8(?#-iE7haPc>P2ZTyj<}>xD*g=7484E7u|Q^rHYrY z@EumEmuSBSl`F+PJnV4Y4dgZub8+-P`r!S_Q zM%XFfLSU{fJWda%+|s+ZB1@%G^mGpm|foH~1f|KicDcrPI5@t(Bbi-)qr zp7hm=S7FiRsfsuEEcFS`#1zOtVz-D3cG&g0Q7>&p2QjQOX^h`HuRE}3P)GZ77R=iJn z74s93(hyY?`;o?%<8I8REG;O<2ePoP3(E89!jNW0c{Dq<3+*b8$%*Ph@5Fb-F68OU zC$SM-Xq7K7&vLp@wl81b?#;L6qs&E}xsVl&0do+@NyIEC3>XVw?r#W; z6M;bp3=@Iga43*)&mr{i3)rI=;&Ldx0&i+-5K)je14?!#`-;4O-{qY^A_BX#l_x+O z|5E=R1v{zL7Pw-zAAK#Xm0dO0;<}J76^?+cF9D2M%|v;q&k3<+onBy3x?GWmuwOfo zb0uEIu?Y0O5R68@Mh}ev^*gDpEAb{01G&tp%-a?1jl~e~MIGo_Wl(tywlWhMO3+sxl8a#E{c_Hlc>(Cim6&;Ett^t1|Q_gq_w zsmd3z?e^s82h^$Aj@tV{u94bNq95PlS`z!jT;~E^VS!NCpe?nih6&yN3+=4Ne`7tO zs8V$f&v|XAd3DTWbr%|5o%dzKThrz0Jc;GBqV6>S&uB$SH8A7xt>~Ee?%ax=*8uUq z`h`0B^I&G@OmqCXzh^gRj9&sA&cg$HbDP(~9^WQQF3*MIjE2s1&mTfod1uP=M_Jvw zQ&3G_C9ZyVP`yxZ!uZ#8PIy7BjEf<6LAZj=EG{YNsJTr9NPF=0vu+n4eA;U_2}tY1g&S`RpI% zxoWpcZ0L{VUyJ(&WrI+mq?k-EMPw`c5nDcV(6;H4v7smsf;zK)b%m;c0~I#SPid@)e8ZA$~|^C0%R1FfwO&XU!EuGQzhaTOa242JP`mRA15 zXEAQo0KvcuNJ%rKuQ`o4by|Cx>j@*2_udKbuV7JuGaSi-4xE%8dpgDAQa#CKP_ho`y%KR6`ec*>Mp1$X7c zlSjk${HxZ>7W@Rxec(I{F4%Ap&mQl&91iAX+4i>dcQ9|&Y%rLK1xKuwdQ$UOw#4~kHO5KL=86#Wze#%~=(vqCUk7u(R$5aj#44ZRKFRatBsD%+4( zjf-kixMexe9E<5W5+sKDv3BH@K3i3TbuOjtFn`w3KObv#%%8D(Vw0x4{%p#bh32Dx zwt(e2ArLMCWfAC%fO%SlPo3+?nmYj33k;2Q_$}Qa@3~%KZW|;oQBZ)52#Dz~RVd+C zZILLT0>D-Piygo;0WbjYBCv2pBLMz}2sRq44Mgap4SsJygg8uq_PiE=t+mbtKmofU zU_n~b!mum>tDw~=4BIYXCABhzVaov%m{s>VFiS#=?kcH0To!c~qb-Celc8uUP)Sn7 zjOdX9BlisWHQKxXZ@&!-Pv%)oZGJ=Anj)>(!k7$qBI#u#?(Hsr2W^WK*)yVpBa>lr zV1Fc#{2Fu5e(g}DS(p(ywZ8bwbtnW7@dXxBT1-c#4_%LEpM|)BCgY`XTsafN83EY0 zEu8KK9nKBoY)rl%ThNxqd~^jB3IF|aTQ@ZNI(j9ZxPWUI>?jjjQp-?YkKJfViJ|;> z$*89ocCkx0ufw5FKdU{Pa0lk{A^Drmk>K zZ&_Gz?^DPmu`40uMK-4CVceH>Y)sq2xVLmwy%5IxO41do+nisLE|YCIH;JDJzKm(z z>Gue}sl+8wl${zbeKOsTObOWDg!;7r=ltA+rnlhrJcFNL2)|uoGe4J>VvUs?#$z=? z>23>B7P@C4gjQtcC z`r}A;4`#b*JsRDbS9YpY4<#6536WA@SYe{hN-e>x=H7{%9 za2F$XGhRI^#ShE2wi{gH+xgoOfNw4>(65M*hk%PlF`^zD8KZ<*{uO`+93mxm0 zkAL1a({Wa4LL(`rS_DWFo^>3pvaGG5y-bD}ohg3M*4@%4{$;qyHbyrdNW{UGnD`mD zs9js$jnxREEp2(_h}&T4K#iF3;VA*v5zY=1H-5z6JT)0?07wqgl5lt%1VKBc$J3;n zF?Er0>wi<;2_o-y+}Yn15yJjKXiAdr*31x{2^kDvv>mve$?y^F1Ip^h#U{haKzb$C*Hfh0zQZKf2Y2;p!EMz-#tP7f}LGO zKv19cRl3@VC&=s?^dW~UF;55YtAfc|E@?PjrIgNmc==7$F?3IFh@tBTos4f9h{!71 zqhEMR17Dq@3{z{9R~H`Uwh$;&AR(JyczoIzqkZ0&V!QAk*pnbS)&=5YMlE{Og?HgY zKYymsuG~<=1)5-VtfDWrO~Tin_DK$}S8E-N%rpPdUjH|#MDJ=ZtpN3_&i)UzA+uQf#<1}rrJwJQC2Iequ7d`3*ged!rs{P0#S?!uM{zvX-uQ-XN zJ8UI#bt^|Zf8?S5ou7(rjkxhsT-fdk^j%uRqQ$y~O1>XwV`7y>cq6ep_YR&~3GFe) zPSdJi1jE>S^~?9y&Kg`W3m=7V5&5G;^iJ!2{_H8=G_?=+=@XB1 z`hFMY$KlC6&P@xumKAm#GZCQRIYpGQM21klvSFrkee@d)c^Ly|;hY)_Z$iFT7Q}ZiS~~ zM=lNR$Lq7(ZnUu<53GKwqA0dUT4Zu8{Fn8}K*Q^qv%;zb?ZS(kxUy|FW=MZR4*j`T z(IIf2g>(Htiy{7{D9w_0PXh@&M=*)KYYM0JHb&55!$#Ed|U#BgIEp}y5*#(H5X1-0f$7E<$UhlaUoh7(|`Mp=u*#y&A zhp8*Ym>QKQDkc8RAA7WY{>AkFzA0O-9FXMb0IJ_yDqqGCNd!-F*li73ej+%I+1R~e zh%k2drKbaV9ahd+^%=y+OU(2@P|T0+7ym;r+dpxOLGi{+av97`Zq{XGyr*J{2@p6F zp5j5kY@9H?^j516;a8Znos?mGi{IE10;Rud_cPE@qoJMtDK3bfGXpAr1EgoDcZcyn zE^Q&Vk$j!BnbJq{!}ysL&1W_VvlZpM*VgU(44u$B6xC9PJl96<|14M@Ab}RhPk`X_ zWHE>gLJ1=B{ko6Z#_)!vZ@XY{MZ4?aRL>h9k+5wuFBbc1R)n)i{)O>x!-$02Dc zHisrdBPa1vgFhT5c}FCiVUX=QxzOw}-22BSNN5#PE0LasY}oa~c$1?z!07u7msP*4(|$mUI_eM#J}WW!vsR39{(c63bDIvt^pNqh=B zlT7hb_z#AjPYT7)4@Vock%#H#6fmlbS@d-Z*3F1R6gHKY^?&{uoJweazG^KuioS;o z&Jz5^J}gIPVauP1qB^FU4|DeXK?|nx>g-HO%9;xN+Iqli6r5y0v&&!h6<9y;t}+^MK7iSRXcqm$7Jj?m2|wx z%FB5-#)=qz{+>8^s9rN;sv_eXj~3zfkul(-EL&084&2%4EfcIon}6!FF!w!?rCg9{ zM>20--Uksv@)P2OkYF9AsW5f`qDv?TE2YVh#>jIzKf>G-=)rWT(?>~EKLv7SxAzpC z!h?O^0ReQ$A__7cES#wo9)Q<9z@kgE z(#T7dDjfSQ&zO01#>neP^N7!cHtNuOsx*@a+BO9R=mQw^g@(-Jt!#63{K1Gm_)M8I zc^x}*( z&thJA*oAe%%eZ;8VKz3#d%)-QS=jnX{V%UE!p-1&{FUszh)`FW||DM z-{bZG_KLnAC}KX2vG9!T%lTOJ(ll~ez;D@Z#~|t@ZF);z7I0VFSpcNOB4^!q1X%!Hmm7PboS8`vM)q-KA?1y>)KRayjV=|n{CD&EFe3OCyBW@n*4WD$>QKX0X z*8`dXbY6>@QE0~^6c_OEr1XAD6IVeBc6&;Ht>P8y4m64b)7X1fG2RerweV!+jwmhJ z%nd+>;&*YR>4 z?F$mf2p4`ZHbLKvoCjW_}{x7g-7X?+DMBDy^JePil9{&ar}0god{8Zmq}-a|`QEP&aOa zZm#6h9~st%#*PCVhT=`*X#1x>suy;^WQQ%iOvQHd+HBYrYP6fjGVKx_+Kp|F;}yEH zn^$&RcL^n>Kn0l+vk;s5%jBS9TQKAj1*!bFYsstJN@uoM?~Yrz!3#|uL#6k?wqW!n zs<($nvriXj^Bx}N>2u{<>LlU(&5|LzRdS|Qd^9=k1)EB^NLBaZ{P^H<+O`)0$b{we zdM~fU=3F56eS8jke~!}jL3MTM99`JQ?|7BR9t2wErg%9id_Sg5{ELq+ZDtKu&(hib z;O^Vc(%bz!u2tpVFj!^a<|tXOAN7Lkh?WSAza1il<{3Uu4laZsX%!6*xCdA2BIU<3 z;0gxBiQ}D=7;qhM$&3R$$nAH;AVXt)<#a&?A&r;}bI*{|L5L%1XQ}Q%?p3S}k_ifq z8&16rVkm52Lfs@QF#t~-nV>@a0czl{#YZ=)38y+CcA0eKARo<+wxQrdybWt|lx7@4 zRd@api)l7`8snVX&_X)KkQYbj?jc@2EO4p5!;MJ3DpFh$56@K7$;e92mxUELy4ILM zkP;rFa-EkhK#Tx)#m+d_-DDVbn!?k0IonXc^{oDr@8U~1Tw)70e%Z^gD>)7 zQgU#hBBY<5hz-TnK}ag%4=l;D>BKSUHx2u#W)=@Fu0c%;?yavg@*s6w7N07yAAcr( z9E)}RRH}QNSF2GpRnVf?Wr4Lx*u@K9ASOetOtewT75lIF*aKEEc%};k?0l5w9OwI4 z=OfhS1dI?8j?&E&Vl#S#q?0&`ZZ?yuoaBMQf&GAa^|aN43hhRH0fRg`5q>bVy@L>6 zswUP&ppd>!JUk3VAnD#db0JzijKBSz2 zRlubzYIcf;Gs>cwr+8D=Ba1Gc!v48l7P*{8JkO$zr@{J;AEPCw!OgDVitlOI9uj2V zzmR_9G3xafzE5V-M)5r>lb-#B1G`@`sq7h0dACfp;TgV4V)c&D{j)%W4~MDxIZS`n zVfy79GA=$$JH+>(!}Q`DDyn{%0?wnN28U_zc}PcZGw9%X#9z;#PvW~`2DQ9^?|11m z^#Uq7mrnaHprVX)_1XnkFxqd&G^X2N+rF>D)994C{%ztJ6Wp;{j)ydf^ z+8(nKr4zPn26xdJga8W)HW{AnpuJc5p%MXz&RL0?!c)fLp)JoHH0>I9FKM+y6!u+D z`*ORw`x@`bTBfGywb*i38r;_S$3^=S@Ill*N~{?8V? zTA_pg+kzV_=yWcMPhUZgb0Hvju282v<&spU%yN!%wP%qjyY68?91?w>2(F5*u;sM& z86W1}wF$ONNg1FjXz7oO-Gtb52xY;r#%hh{oJnl_GOGOo2vB($CB48N@z@f&{{p(t zMvbY)OYZ8}5jV4Pvh@Q35PzE`)cK{*H8rByFL`PES0RY)i&&H4`C@g?OFoz}w-x03 z25I`Opk8mV5bW7PiEn`O)3?ykH$cI@TjOuL}b9Fdt+J)YMb~u!9sd zB?+SSg|OI%oyV)M|%HRyn_l)MG;^;3IF%Du+CgeNCQIyb0!p@fl1IhHbwF7qWJt^ zb4*Sbd=+lNYZ#X1j?r!g)5T9bpwv>R<-knRtQ?YVV|H|FP;FK)VdLV;VuC3!4+a-u zv+1WiY>KM|)1*A!-0fDXjw)i|D@-ZJQt7_P_D3q^=V2^|rBc0oUZr+Zkvt`73|go^ z=~klP1MaB7()C#EtUJ_#{OlEkKc$1zdHLK=acD3YYB967rMueM8uQP}kJK;j! zFz9e!V5o*Kpjs*x>kO?6pcs5~Tu4b9g+Hm3*ff)2PZB*>WPj^J6;Vra z4e~7_|IA#IXl4=F!@8ybey&d2i^#pKPgTIjg6iaIEe~V=L{YM}T-9w)6qI8rLmkCC zb&=v#BHZzLU^3iKq`$0XAGRxjURldt#pWgGtF4ylOQmh(3rtH;pWDc?#2!pm-xif| zUD0I{RkD?vv&-Z0zLV_4>Q_;x*vf;rfN5cE}MaN}>_ za2>qa4DRv`+*=1P7^ONIm5Rag$AOoR+8db(gnFl0r_Nc+)~r zfAW-N*}V~LZiOG*goEbFaWH<+T)FQcn&cz9GTTA4#7FjGPX?+7eB^)dG}i!H<0}th z3;R>?3UXuCsXs+kke9RF{pe){c{6k9udb*lSCpiDHM^4BOR|675AA8Jw`WNowRRPG zO%Y-Kcg&a+%A7q(+Au1%B^2w#9{A1aX1yF6<*L+jXo_i*XeSzqq zX8$Fh=G2rgx;N^o520Y^zI_FuAx6}py|#39yp3juE3CRuxWp293X$4^ zynNLySgs(mzg|&jBe^M?_DY@7Nd6*856Qa;-pX1SDo>Rjs0c4q*wd!8uem(dZA4R4 zkaF-G23P2aLRuL|SAv$jTT_Y*M;V5u>g;fNwZuk*Qa}s2Dr*)>eOkyhm{X`aw}rfm zv&;u-i`Mc5&ZY&DXFJUAkU%x4o&2-peG12omi+>jFNY-YN|Vm!b=GqG1|4oM$FQOG zsD1}I(qFqN6z2EDNfQ)aev!BY_C0a_bs%UDaA1gz*n5}^uj|sz4su0yqAuO;AbYY+ zb?Hk7Iib8b2i3VW%J8wQlh3sx$eueG=!oqHt4nJ;%3;+K%qjPX+NwhJRZ!V+ZPB%S zFulm+U-#se)P?2L0CMRhw_wczs81(3s?6&;Vp4J=jr;GRNx~*Z5Cl#{uGgWzI?3l- zH>?1Tnv7x~r1lYE<1MuR5c&^Bs_!4fF%b3{4vG3@C-YVciVZ*o!hOArtd@7J_tmDS zo#omrsW!n}*-&MuCC&1JG+5BP2^e}OWP%DKbnMV3UZ&n%WIyIwn`U;AOR;*FX+sxz zvF$~1+$O|}$+ao6t6ZKns7=3hmFw9KMud)5rEAm9t{B_%wQzJR``OwSMC_|Y&fO3( zq85d8lWW?->d;(N$67SD8<4}M7On3lHEY6O+;;hxsZQxF zAD5*JwM~E7R+09rKmH;oNNi@NdT+2CCP{m!!msjl|F<5fAVC*h&FjfP5RS=k+YA;f zvl$$(gR?y7-ml;}Tg}k}buK4l)!_)43=2J|_)yt}#d}bNp>lN=<)L;ODmz19PuFcb zbgB`D-WEjus<|Dcq!F0ZH*U0Jgj|>HcT;mm$n_;Q$(4LZ%C&q92L=8M{dSC5q!Q!m zzaQAQ1m&0v)m&-pNV%qeUa9}tkB4UPfBSK|l->`vs}#K&Dd*NYZVjBic^ETS8AJL6 zU!e1IW}#aAXj=~=&fZK?nDErSR6Is@XX9MRFGluv3wObkmd-WgI_bLjo6&H}jxAwL z7mA6I>$!e3aI3Jy_c2Ao#bM-c;<*v5Tc!6fHO@>xxz;H8QjLFcz znNyUbQkr==)6~)OO50~>uHL|<&QxoR+_9>M6Z#*I(>u$=Jkt1Xonpg{2d}+A{}XkC zB=|eoMcZ*>MRwX<(59NZ3bm>HE*dvhj+guFDu|BXStxqsPO3UiuEN@spw8pu2CPI0 zf)=a{rHzwYvRC`*%Q(4{?P0V73S!zthh!k->*?!16dN4^YXTPE-6a!{ZgLPDLvfSUP8bG;2 zTP1%+?}VtT_1j6)C(9vD9dRTUasIK;mtap1HREX;^~PkGOAa;CFg8FqwCugM{danr zAeY6Z#P5W>&`h!`O@>K2_T1IBZOX@UrvBq(U)IK&Iwy+$T3h;?D0{H0Md)lIP<}-b zvQC2Rw^|KKlBX&xS5l8om#0gtmzCOPh8&|XElnC-=T=@xSw;s}Dk}LZj zekcyG%ziqK5k}f*$O1?|<$ABZ0Yoc)23mdDx!CcyW&_HX^VR1h_mgaoyb}Om>4-Pf zW0BmTk}WO?XhRnlLV{QGbR^J+;@K;4lyz70m`8^f$(JM}Em$m%mS(EOm&kFFl%mdD zDj$*H?yBu_xrdZYTbIipq-1r&3b~XlB~tntNK=Wb-&%Q;BqgY8*2!hL6i>%C%9o{h zb;%}qjVw)|MqA{`Qk;5Z3-Cc2uNL1Xk1r-)99e)&uYN0pOn?7I`}WIK-P6zN4TF2) zN>9N67kw+?6#uLVZri!OZ#ytvDe^F}Uh`bwaMs#vi7tCqbq4 z57W?-m`=CDs_~?}T9*2${%2)pN$N*o=j82p!29Dl`A__;J1@7B`luhz%c~SNva7n} zsyx`b-1`~Kypa)y!pUv*DSYr921S2cs(e>oZrf_BWx77Y!ObGshu+?mOIKR(2j=Wk z`$BUzq4oDU8-rLeXB%6ob??bnia5MX{8rlwEg?cJ^H_#5`{pY3;uC>!ZC0wip331e zQ5Clhhxrl`M&+B&g+zDruZ052ed?N(J8$L-po~ zKsrG@($)Z?9qK~f_KIt<=s0~qwWZ!_6MLnY+#X)w zp!8td#?w0orEdAL<3-JdSolCQ;5!TNJRXj+l`<@7JPmPFzOdkNG`NIviS-|=hL%)X zNX%^v8J(1J?r%qn0SJj12)u~**2NEFNSyfk&6`d;DSqtF(Uj+;)MlecQw?V{MjB1) zoRtJNVU)R%Ka8T@2Efn7s8FYA z>m<0+|4A9W2q;y-g}QqwT_pqU_EIWK2KAnoVlP=w#R-Pi!byGWt^CHMlIqXpl)1%# zF9kRi4~+`Y?;gZ*O&e)NRjVsiS@oXOrMl9)u3s0?@DJJ$3AoT)v9NU^#hoANqS zCz>_7hr64TzWSO+&#No#S$=o5X${3&vL5>-&!ja|=of$G1#8)jeyoYPi0P`7Vkxw|V=gJoQ-}Tt>Hj z)=5WFnDKQmQ+Z(S{0MK`+DD$A~Y=A+z(i(#j_@U6C9C-D8Q1&2J42t$Lk3KX|y0f;e zsbh%JxP*g4}}B9G=bQkt{lEpdxnadB;_ckjPLxwxg^A*DSL`(GAv@vLr7 zsx5hzN6i{5aL3Y;);3mx9J5-8&YQINPtlIUZ}ZgWjg=8aSy(9j)Iwq6!;r!OeI<>&UWepB%k~Qe`24|`8SF)g|$>7ZCVPj%ac4>)KaO){%%A^S}H4e z(?*O!pTh&em`J6rcw=I;evn%65!#Th1pJyt-Vh{iHdizb~ffg7`KV~B`WFm zW8aHT=o{XzzpMU9$}>rBw9!&si5?pYRTqU1v*=qxrqNTC2J985T~n2Xvcm>*2Cw4l z3#Ip7PZOsprQ962pp)^gx*!LuGK(PhkN-#7|ypwc?2HrP7Ub+=$ zW`Qj&Uj2npTOKUtd*m!lBN(dIDeCzaWv6(YeM2uu!3FNe)ODfKz}^c6H5PMiccRq` zl>@Bx0qVU-ajlcNFHhV*G=}8np-=F4^f%h8Tk)S3sxwlZA#w}WBmQ zBmTwX2{i1y-|jWOghz?#hg%oqfMdns52{+LUA zwc_t|A2Feagl$&3_ydcwC-lQ=rHbRF9CN=qfdQHf%X27xwc^D>p3o{}W4Rh#Tn$;Y zmqyLkC{?TGf%W3}Q|v;ZiXJ0=lJWmN)Zai#T?=+kAFi?*En9;Eu0E!-Ym^b?`vL8- zPQcm`fe4{r760O4gK%7^Zc%lU$JB2v#Mx%=X!csH@1<|k>9xufkA1hqkS298kCeM6 z25gsD^-PB5k7(FBXn)E)qLg(?g|4;V;$lJ&T!~d z8opjR$;SOd4K_gR8Tv>d)D(TQ1Y@V&%4YOB-5dA)paK4eqBkgwSk6P*wLvKx{9Kf( zZ_UJjLwX&mT{n~w7zQi+X9sAD_OHIk%= zwxop2_=oyzRI0|c#;0x_20e(ap!}31$F~@}WThHnz$B8a@cG6ePO-;#c(zg+KjsS- z9%alBSNw>7!eHyW-4Sk_<$^JViP{*?roCI0%1%daB6CXY#_*I~0=Z0vSGUx6TNP)y{=@6~P@cvQOqS%B zb{L*h_z(s+hpvgcD?C8OH<-gq9B1iOL!*6-LU$_OMN`bVqp#C1JC)LLm9B|u!d3!@ zMdDV0Ut<*QBJR4NTg{ME97?pY^etl0m2Z6;DDoM1eX9U?W-{D_hmZ&pEVKej=(Q{p zgb-5?{5bRiFk63ReY{}U-|{&K1T`7fAR~Hl?V4J0m%?Q>>w;QpuMz{z+j({7KBb<- z?w+OW{n$QCI!nF>uzhHR)iU=VXu!pRlgZ$JmPQ?bByA0g*#pWfHu0?Le^8mkq*3bj zbmfC&JNPfuAXs1NQ?&Ciq`Xrn>EU4|+&1U5C1T-8s(VE7_ULpHNA{(0#+_PV8i^}- z?{NYP%hZ!J@`&=3g&n7#jw&uJ-~^353i&7%HkwD37OdkP`gl}n#yo*PnMyD#e}~3r z;``}sIxN0lwxq9_5Pz!QrjTP`CJwjhmt%@Iv%0O$IHs(Y>>p)=5N&|hFkd@H!P$sw zdyINyE3Qs8GSN=`Z7H$0hpgzwl;sjLskGsu za?Y>}hUJBWjo>9*3q_Gcv2b1sOY@Y_oHj>E*@S~N?3dSt;;(;6`j zmGqUyp@iZus?1hG@MBqM8k!-&_!0l~QhlNzwmj=hhi^kt%4>sO;1Et{LAy3vD*n3- z<=s|(t5*cl8eE6H3{Q@MMz$jkRl&`1`xEbYqZeu6#xQ{OyXIHM!7wX1ECaD~*U*|f z*k|+`L{IL3wJh399(S=LDmIid?_y{>?xUQ$pw>3~$o8Hxmo;ofEAJ^g%f|l!8r>#| z>nI5@zzR3P?%=jr=sf{B|EE7F=DyOHdHq4^eWfyMkf=VmuY8r5T|7N_2u7J4NB#d+ zda+7zl=Zg~Cf}cko#dpy6%W<#A7z%rYR0MuA1S{}ENMIq(v*J12#%x0zWQ+_UDcGj z-Lhwkbqz!agXrd@mjUv4dS$~Kj^feA}k)bPtMYwnjr`EoY~rKd`3!l zzh^rjvSt53Kj#1o-B;7}95itMD$2!ov4yKdeIK;rP08hnQq}%uJAGKi)&2Slsr3^j z&>?mXs?x3FJH~5!deDL=N`g-cs6t-|CW}D97R>x$kGPyWcOC`jDt`51ME{Js$)Q%X znUK5m*YLjEwL@BTD=5oRkQ($BjhoQ&T&0vp#U{X#*AaOZrw0X24#uGTOAF`%%CA&J zjDvprvg1lEv^yFP&kJxY{sXSqJ0$6+`M8xlSE4I+if{4RbM@8;$yW1iN?o5SO*$4` z0hn0QtPs%J_r`lh%VFHAMIdAm5OtaM{&GvYYY1pn!_l+~_#^5wdwLisaybe3p>TTn z6xg+PzPXWEvkEm*`tg4^a?l)0BiriZt)EG2|Ad+wd19HR@(cttC-_*_dJe>&q6igk zDpvSjHRuZ``3i!n4cSyoPy;b;rGldF?U8yZ5CE@7T z@T6-FN$|3YtA)=XldSc`XTN?+T)ZR`~ETi zx;P&45VdNH`(VvE40|#ywQn9?r#KcG|4z!mop6&D7Y7IwA4{^714t_6&dFs0YqKH} zV$GK^AZ{OBq_+|8K4P=*Ev^Vm!x3Q~jFkvE>;{`|gLfkYk!1(xnVKbIbdvGHDB$%h z@U4JyMk#@XRwzC(r=;t@!Zf|GN z;x~#%Y2P4ys_$XPYBJm-v?KYN*s0qG37u(ewsA2Pd8-V!o+$7!*+@UVQ`~9MTctwD zuxWsZw~~zWT?KK(wx{!Nv54KDPS4&d<%*x4F5>mQpw>HxJl`qR9W!)RQE*rAmzmW4 zozl8wx2gXwX-+#j@D3%tPd1lyKlyt}F@aR{y;9b{&qB+FeYCx0vmONq99S4bnj@e! z834wyTWN}LctS|ih1Bi6vaIMj(ZLjDJuUd47(7Sg>wj#PdjqVU8bGx_D1X_m1^Mf= zyksufe#9=}ONwZe54HTLtYL?`)60(#s$y4@>nDhp*H%&IPfC(g+f}e19gR`hqc3Pi z`o3fEF7>HS?9mE!UvLbVuN{$q*&@9kdZn;mL5jINS!y!a9t&uKh9#H6oqnhHAh0 z6w-Z5Bm;pAC{jMnhkcWYM;JyZ+Djb$#wyxAi^z{=eVn@p$j`{=DAr&-rtGuG8nf zHb6=nF+C%eY<_u6^{iz~?>}8#Sj$Od&}^qU6Oz^4wT(x|d_Kj_ww#}%79E&)u_~{n zui1`T2S&_z(sBNvkMh-V?fU4gYE)h0D_jHA+jWhTy7a$IroH|7Pd?El+E`|O@?eL} zLNA%_QXToK=7NXJ2-V5rqGeXB8fG=_-EYY`j#vEfCr9|whx0$)6Ip?}P7-#6FG*N( z9N|lrYzL6lwCU<)t1&xzhpTU`#);AGbOssNzROz>WwvQ{bkOnoH2#F60M(zcv_8>rLpT7HP%&U)cCENP$Y8w@EYr$ek6q$>@0C^mliR@j^ zu$I(VM?nku9Q~#ZHP#V7%3sJ{FCKC>)V3*B>egAgpt`R zV5ObT_xJkni8A)pn_w$`_~1?nSeuD2Ju_ex;*&|-KcsiS>Wae`{Q_3`X#p!7hsoeX zGU$!N`DX>JXV9l&|C8UFyh(icMfmm%SgZMI$aH=_^C+_Qx_}jnoQ~XyRJ&*N+7C{W$EX=d`z;9`yPD#h;x#{U_Dt3&`BB1J)vB1TqVG zna$`!`TX2%-vq3_`vTVEdjnQSK3w-IQih}>#mH{IN^c%S zZu|osiA4Mp_;W`k1d@PEL5BTF27l3o;6CJ0Aj#gN5Kj;9cN+qf*zeNzKD}zK-+D6szhUS>q%(5C>DAV>6t4HcEMx`` z1)2?TBncdZxX!A!V$ZI%@&?sf{XhduJdcIvd@2FPft`@}3u~<~Bo;Y*0fmDDkT@h2 z3BV2UqZ~vzhNPhH10%t&z(jBmG7?Ebl94oI6yiY)WCHRDz8PR9G8{=jMj$`Hmm(h` z>yTn(EPOgriQZv&t>wR+KTDC(8b$Zl(rKBrY-ynTd?o@h89+kY{x~=yyNy zvys=3bCEZZOOQDTK7Kp1Gl^X85Y|KBEy#Qw4`%d#gN*D|!J)A5(<@ty1Gk|roU(N|u|96)Lj(f4WHpK=*zD*i*1!Ej_;4iK}7UNkUx-W+bW8yx*x zxKVJgBQK6({3Y@%vJ@#rmLu!5z6#ueEWME@`2gvSWFy@Wg=|KukXMjp z$S;U-7ypC+9E~KPw+6jnH1ZAlT%-ay14%@hAk7eupFb^dIs#sd4E&IUt68}=(@h#h z7imvCa}@=BM5Ty-s8Mv0sl+qZ)mpbAbC77b2Q-Q<@+$GKkl)v`bZ_ENzjwW(NJQi) z#zzk_7k)v&xkxYM5b|0Tr`-Qi%BpLv6*Z?!=RYaKnK*VL!KX+7$)>RXE15!B<30X5 ztA|-*qmr*0txQ2Kg8X!~a6^_!<5l{N|w- z!CykWmNc&re~ZR^0j>hKB7Px*5;O+m|C<=hTMDly{^PZg!s;1pJy?NrXxVzC$xYN4 zYz^Lk3}*Nqxx3DK;odrHQ8v8;juO1D&YFrmj9mIaomGk7><1kbU8D{^a4*WwlbIx7 zCmaNWas>{p$*dRlV)z9bMHji0xO#{=F{REbA%4|V2Spc2Bp!kNYV3u?7i$z<#Jwh zS&g5d$8zheUHq$)m(k16m!bRL!r^Qp5qWji>tJ8-TmEPHmGXbfaa_&+s$Tvmt0CB& z1b2W-sYE|)tx22A|2$uvQ)k_VJc_;jBX!o9q)j2sJn}k&!uO;9NV@y5{RQ_SX=>5$ znZfaYJ&C4Ju>AA%$8cOu=Fw#K3i1DYp|JXetfJtF$d}Z13jfUhzm@+a-AO+F|3M*A zsbD5S{7)-7A*4KiXu|p#Hlzo+Al9!;XkNj1ecvFb(f?m}aYzRLqx4dt4a~Wrvl;)P zv;RAB68`Hn|A~)XuK!;Y9?T?&|7k@}{}*{m9`*C{*N+7CPz+7fP(^}i8nXQh&i`L` z|G&uWq`ag;Pje`L#5!^gi_i-iMHkscJW9rY;kJP%8H-A{ex1gjCL- zLQQ7XYeBHUQ2dh&g_F#KB9#edEEue0Lj?!@MxNZ;{*yAewO&ht3H}vIMP`xa4l3em zAnjERaKXwu`Refxl0j^!&`tExbN}E%au(PA;AW(R0+wkM+i2pK(84%m0P;F%rh%^^ zh1jOzvk&fl;{PDN%}-45G<_us>x-ym+E&jluAP@YDA1esTl6WeqxqQ9+e z6G$t3LurCO%eAkhlYE80cRuHTQKXRi4iXP$xEfs&gvKKmaQU6jrTJ4QWH0eFeEVVx z@<)h^9>gH;AmcBF_Ow6LEtL!A}A zkvkgjpGX|K;2Gcy#Ms25q)~K{cZsWiQr^!T|F`hxNhBFj$oI(GNTXkvJQ}YCmmo8d zwa7}1`^jWA`gG(6WUt0Af2D%IbEA5gMeGm$S0{Q$LkV()!zMfwj^bx;N@p|^4pWsFC1Eo;(=J;K}<`t1qNEVXTgWp@+&3+A$JY*3v zmHiqycUabfahCNM@+7i~AIy1XHY`_SyGh%h1s_6|YpI`RvGy6SaVEH~*GkLEc;B-A zLS}woSwC#DtXn_f=Re3f$hDg-YbSUZG1NQng|$+1^UXwc_$)J{=`CZj#@up8_5+EN z?z{Ew+a^r9U!@H+uUxTZAUm;Fj2dLV7_C~2F#D)qE-^c(FZ`I{@s}w+nOD-)*@svvJ6T7g*p=4YVDO~w-s4ensGfN2i)7Q@4fv~ z?v;F2%$jPt&G~cPo)xpFnSHyOlgC?D={U9Vb+dzOoci%~v#&d6ylV4?+1{Nue#O8y z%$6R1X;Z^Ei{GE6Osldy{BwtFuosvM#`6yQBiQ}U{;C15s~(gq$(Y-{+9^yyXBptgol0*(YLz-%ye zMimd2s0cWhcs{s9?BHfFWhMnA;0JC4)4^Wt@dJm0o>^5^CYTEPI*<=oNq#9$R#~%d zGK@-aaz}nwlRc-(TGf%7&!q)m_Pi=9p%WSAw$x~E-9}MQJ zDl0CNih&D>r!S;uz!ESOzVK-(Fpm76sj?Pf2$sLJwhzJeOR!&qNQf$vsX2h-`QW%R(+jCC=N*HB|{&=hbvJwdzz z%pzW~lAa)*`91|-OAmn+VCDxDcpU|-CO_s^@yGN8bE*s+308s=B)+D~Z>3#NM(e7q zVhCRe6={wGmgk2-~lkxj}psaQvwbGE9OxU z==lc~0t-P8hf~RX3IL1w`T8h9enFoH#yv}g!Tjf_D2G=um;si7If5@xAXr&IfndtZ z)m9vbUpklu`qN*jwkD%^_z*-PnEQIQRSH&sMqBtdXen3>`VxpQAt9IwP6o@CR$GgO zf0qJ4-*PJ4jtYalz_=CF)<`gYB|QS>zF%#XfcbnJ;BY&R|I82R0glr`Fb6E=YXn7L z_PT0o7wB7GZ3V!T5?aV{oVlUe$^`Q_(gR@bCJF{izyUXrZZi!43%{(k5)!HC*YpUO zvYQ5s#vhzT+_$INTAAp_;Ttw3a@^;BOV@JTmz9$MM;~8NSp?>TyTB5@z><$WrGhR6 z3xA+upywwl)|rYO05fO+m`mJqh>C#ehw%doe`lL6_Ut2+$GYOHq{sazB`Bp}?ok}U z)IY1Oq%LIeSGAQ6mVh~6?lC4Gm{LWBz)G-!3@d7A`564_=z6}RPzlZ=p5Ca&DhKn! zYOIu-8H&y|Rvwt|tFaRCi|bOu1|^gVlwvTuYmK!F2VYW+HRKjr1ZIJ$-D@m^6)&X+ zU$+qKS!1n%_p#Y{5G!3upBig3n44TP4a_NE3OF202S2AQ=7JT&h?8+@DqB3k zV)m(*!KbkCIj$$=u+MlTn0+z%-bMp2VT0-fl-v3T4A0tjFDFcla}1=(3U98Afrv3i|G z+>erulK*&(l>?T6i$LEK6b!~qXY(qU4hF!?8T3SN{AQ97=$lnzWrHbmDHu$Ds>Ui2 z`-e4FC78RW#`5)nUq{QqijDLXn7^4J1pO5#Q>anyCpFd*sUcVndbTj;chFVf0ZF*M z#)?m-B|De{U_2EZm~xc4(2tD40bn*Cj>?qyu^MY3m|Mf70(}80oI=51 z5}0n$VlcB&z{&$lJnSh1E5K5TH)a>!8GZ_gV9(zf)GV4Uc3^63z$ySMo8kx-H^Z?% z88&CLBAD9}Kf(BbwGgam!@g3mvI9FFg=gPFOzMY<7hEh0#@G6RzCdW z*b}@E%m&LO{T)>3EbJ3#$au=hW>@7v@?ndwe*%irKVlW#l0~01vu@~tX zF!v?4X^J11OFXlX&8(p3Rq_WbUiDKT4wVq;gDLnmhF}scd7UvMo(mR&C4%sUZ{Sb7 z>`gKtp1FiECm#1!z>3czAJBI$>E8}mL%?)!3YZNRf|cNAFm5Ro#^2+AmzF{Jz_1}? zSRAlcih-{~9R|~vGxqRtE7)`nrhvY?$Y@o-$|7F;5wm(I6#`4aO7Jikx0(tK!yg;~ zW`iTa!ZpkxFm7GI+6DTvH&F0-bj21N1-G)77cAVt)?G07Q+Bz=R9QZ#xBJPu(H=#jx+JwX(osa3PpdT5D~? zUi>xa7h*R%T!vE+m<;CcWp4@P?Ck?l=kpi^o%t*eCJM5PSCBykaLF zcMq@d!AdaUGW>a+I|3}@-SHIA!;2{6a?*oEl8%=aNyOtGqe8?hAM@8)kq=Ol>2+2* zh7xcxm_36BN#e*8@FCcJvw2=fyaJp;JbNCGX~5in@N5W7na?Ah2kB~_30C6gg~n9ak_gfuF`gQ>m5OL(>v`w)J99!d>DNqvd2!=aJ~Pm_t4yvwtH;@Qi13NH8o zLm+-Ud^$$FWHT+f0{#;!4yNwleKP3TNf(1Y@Gw{c4!M#ZF6GH8==p+{lTYQ>RE)U4 zWH&8C$=yT7pl>ge$#LAvG|D#BB)L0~F43QRdfMFoE& zAsJ>KVUmKrKN!NPw6KyczLqWq2Z6q$veg5BaERFdVv>WN02PFf0}H2e{O9(wtX&vV z`dd~phRVSfyIXKL*Rqlxrea_^m^#$5a>3kTJd*)S&a%T_%ex$ z9n1$k>6Wz@^nn2|_Zo{E3mO6*l=Ro~WGx3fm;lBZPH;VVrJ+WxG2_~lE~&Em2$|}Q z74RV`NYY(u#u~Hh`9tpF{#y(>h*LH)mM05s>T!9(*W+0KjYUWf9AC8g%514(ntNl_ zPhXiWqBAg5AW1xiNO7rsKbl?CZ`;k&qgdLE;V$aUV~HJV=sL5#x?!!^C7jg>Wz^t3HF%vl%+8mOgBqppRatl1 zdALTXjCH2ZTnL*&Ti2Ow!dIea-c@bQSMz={+p5j$%%nD+MuxEx&eWy!0^{L|?@|%# z%^oKI3f{<^s0OY#JDFMNd+%0nY@pgX>rIcj5F-5^H3MTf|K`h>HL=QSu0CCFHaGVY zEx$+YSZ}rrKZc%-shPUV^fvD<|MmF5J=IpUF7k2rc~Ry?k=`)k2}la|UD)H*Bb&@t z-gGCbVtzMUMEJ00L&V!vY9Aq;vxs=nJ^W`jv#YroJvCc>P(sy@p{GBfel9WFnDLEm zJz@hj9fCgM0q%%Z(@)G6>f#M%v^fDT=K*!o2J%>lp8cR2{0U=Puz_Z-bO_qrDZE?~ z-&tk#wVN3pAeu$=HYXZSH zY=JyAdov@j7`=>#P_L=AOb=DM*^CS?gB!^Msk@w(*KRhOm`1o^ESasMKVd2)qwk$9 zL)s=hO?bNCUT4TA5RIi+J%V{eGl|AK)vUF`{vk zsw~cuC$%7vXj}xX=TX_~^=B;7#^hre1vE`9I)oJyVEoP*d z1Gi+3n!QDO1-)cWwKYe5x6h1kF|^U~oD)gCOLCsCe%xZV4-cScQR#5I(p}=4P`L&C zP=TZ6_^_Z#x2Ir>f4bV5r@lML!EnP?Gu#{nH}`2ZZYxJh9(wW9>Zz^fKy$O`&!{hj zuS8$@ta5KdPmJUUU8H(%VCD2zf+WElE8T{)R(D$#Kgjq6BkA+do}qJE`% z5iP=&@SIw`jd3kU54@-jZDVx`i{ixevTC{AY;X2LPb^dex0{_hrK877K_Pi163@bR zuuwfrn$z;pkMYPbss5pH{=LF$({c_{ihL{j%bBRafmWPdDRP$MRoOGj^E0 z%@OE1%hl=~=3sL!`rehwy^~H@g?{*b)m!uf=$Rj=>vx*>nVB($5w=QwxfB1z=-Crg zn^N?H=tXy`n@aJIXLh7aR5MD=9ub46!wBMOlhkdWn*AeW6`4gm@gudm)a)IROuU46 z(i-*Zr}#_!81c-t67OyHYHAn->s0&C%z+U@uw@ZnwO-0yF8bn6ZJ(#gOpkleXKIG1!vMla0PIs&IrBmh1uF$0heS|tI@m6p%KGrc5(~D z801!~zu?pKRljdHdvzj}u@}n_CeIXfBQ-k9eSdVAOu|a|#doTR zFU=lpaMMDnhtfaEBvmx*e~! z6kGrqz>(>0hY|cOhC3Fac6@DKY|SGWX13Rk^SEA;vMMAkeDJDH2L2eF53Ykt_1X7JdU1HcY4P0HbB4XUwP za^)rK*s$pMq0x!cJhK};?rv1)ijE&A;%Fn8XyYWE(qN4rRQZi~^lSzl)D8*!<)6qK$voygN0{Js%4FxvoC4Jgd)f`^$X>H=n@seiQ9RGo>7@vHS(1CB z8vG5%XCeCH8`ZdP%>Hf4(Wi`|$GK*aU>`VedAF+%zA<}*``R(OJmq~`eOkr!Pvd=N zjF}EMa-M3lkJTy42l<&;=@G`h#d|P6jb4Bysca#^| zkvgoAdUmxhdVR_kPdy_+~iY1~vP8PSs(Z zxb@qlc6`ruLo#{-56WM$>#Ab*bBZ4+_RXsOe)=E}y>g4XO!SrLWjoZv`^`??a`ed3 znguU-TSUmX8l5GsHsTkajJ~S0#+qo4Yy5D}@oN$B%)lBb5RK7%nqi!Yg)8hJyk&^&i9RK{{w?I zSA2O&|A*5Ncl|&oZicJ)PR;&-j;chDuc)zH&Y*sQJ)&0^)~N$E*6c{N`Wq?`^`jXR zF$yw`r}U>ssplC9bq=I=7G#DypnfUg93yY9By!J>W=C%sk;z1!{Siwgk>QO3>Rh?m zDE-m&c;mV{%$^_3^Zgk_21W9<0hjb5O8}A$w=jywoHpZdd2oertSEMnv&AB~BDkr- zWJagCM@EN9dU@kDwoky4#X^EizRhsiaHE7^*=l>(w}}c)X?Pi>?o-J)fH$`$>iYh#@!fKA~QQ zu{|jSj@8c&fVh#b|apHuxz~AcaQ@h6@BppxoKZ}h+`wuj;bENQi)vj zg=5vlzp}_KK`$Do?iRfaeb+ek40?C~y>vWZ)o>QBy}z2VUU~PIaVJN`QML!g9x^@Q z@)B_KMBWh42Q5j$J;%ptD# zkD>b}snv(5czkcJF7HAgMHO~&T$$sh>ZNHfVUEcCObnrs~<=ayPV?GbAOmI;Rhiz7VVUY zH?q->@g|7#-=CbruCBy)Fa{XbK@##^@@NvNq_L%zlC^KvX`rOA< zvp*?gGJ56XYVe=%@(Oj<6XMg}zQfu~JdURme{#(dKrf#sdXFfM4kMZC$6o*77tHz; z^_SVylvk`1{-L`5WnK`z1U)vNA15nUHSK)zFP_uahm(zB=TDrtvfVM-JOOgZCa1pn=({$l>yJ_QRpPu^Jt}$ydeTny z#xeRWj&sE1GJZVo^u^I*^u=(viZT^lMZZl(Px(R(u3~VPpcj0>ulVDfQ{LoQqFFBer?E z>x9?LsXwM60kQ@>(j*zPwS-1?ZcDa78}#Zh($X7=*nGuPW5(av@oZJ$wXQDcs}w z!Cft*It4B{sm?l||2uPxdx254W~{jcGOY*SXEY!7CZEG1)UQq!+K%Q(BK#-a_I31r zF8YD9)WvlyjEjXIs3z7i+@WwhV$L5W?OOzg}#`VU&ov-{=%XQlg_2kY<0+D_(!7W-@l-}w`+JV zYCNy6w%C`23vTtsn8i>lr!08T+uq!Tnm%>G^WHY*G1SVN7OeJm@+J=9@R?et-ee_p zo;eSPNg1u8U0xnYqfeQt`is60-S@D%$>r@EUWz{Ai8>Y!`?=Lpmp94`z#V=<#=Voh zBqt4Job%Kn7X=PS_e@vOZc5BTpEbQs&H2;pt}@(Sk68$_%eJ<+pFeLSo-srH18=hu zeew+Tf#?atC}F1h-t8S|jzZ6xr8+h84h&y}Udm5DrY>0PjZ+Ua@-_)CflA^9T$a5m zw7ilzZZGvRwGLyOfNiVozP*t*COqLh=FE*%Tu|E=^tEv22)K-!R5OpaNB9)*q6Yq$yaxKs972M!UgIX4{fbPU%WtlfF3SyyI1lauilez?-00y3+mM94Q3bb z1R~{6)!Ems_Hrig!6B>)XPLOOu{Y)dc_&^uCcdMxbJ-CE*FwK!Y(o6noV#He#vD7maOOOs)_^+D&ZZ@|kMY&C>S0$UU3z^Et)DE&S zlh9L-sY9a6EAy0NDmn~(tfTi9Jx~0r)n(}6i*21Bvf3ANT}p@!udZ_*Dui6fm1A4N zMeh}AjLX|ftqJo+c8wg)ObPYB)agYzNB6Sr>PVP(aEIxJ(Y7rnK2mFBVw16$!Zl=L zH%G(gT~wbPEIV=$&VOfoF~-}%b3t@QsyvKnr3%`3Ju$QJFCJHA%>g|#yQs^Y9cOs9s{Rq)-Z4dR<#4Zn zo@agPMFuQ1*yG1m>xx9S7oQW~?KJi^x|?PArRa4}hM zIdEfy^LJPMqP?fZEP*VB^huj8@2)0Ad#{U*GdYOaB5bGrOMMwl>NL1ScJpqN)SHrI zlsc>aF?5)`u?pjb!9mGlQj#-;zKQk5sDc=8Voao$i_>SSt#711&rMQ0W4uG7Wq30% zWU#OIN{@<;@wQe^#ClJUDTbWzB0Ep1m#2x&j9b6xMva<#>;Y{hQ&-Z5F=!)S~U}Nqt(v}Xob{{qK_G~U!0SB;7?pI}Th_Z{e zo4P#C+pc*Mk@yGC5xY3}tz@5PXc;blp2t&?JTb8p|~qa?c~RFCby8Pe`1-IPDhdqqs4ko&5w zTbYTT&%3H`s!V5!y1$LI^HN2zUQcsbEQ@g+<>70?&2aU1 zQ6GQ9;r~$!Z%oVqEO{+?x+vu=bu8_QPu#Id>I4eNkFT-Lm(HG}qFZ`9*;6A8J|m;X zdKMqg{aw`jmdrC*nMxsV6!P{iYJE#@mp;Xi0mzm@p4;W51w|%VnYND?d!H_g;&dpTt-yTUFyMWc|9RrR&VrDt4V2qh@06pM^PxG*czb zA3nR7e~;cRWi=8Qeiv!1^c0zGuCz<7us_V^UMR z*{}Mw)bV93a(1$Zc5Z9$fS7#D+uE=NQ3}1>K67+d)valE1(p$Q1J*lY>84K|=e407 zNsMb=a+TFp3Txt160qa{?sYdI%-O<+DGdcrHLAQYQljQcXda13qYyW(>gdTi6S6cF#(?9MDr>B)XCCC@GXfUhpXW&PsjUgrz8Es$Og7X?^0q$pmw3uMv%P2V`yr+>N}?$b6~{3mWVrZ&d>~l3Z#%1) z_3{9;oi{$Z7|TH(|4kRm3iW3@Z;$AMaFv62Vj$d1)w{j7O-vj!e-^oZD%HJTSK`k0 zRAM-mg@f6LDV4akvmBZ&R6%>rF|)7~o)@tGA^G&|tai4iGfE&MQv=ook`cFa=T^#z zyaOk4d9fCD0UJA|lfP91J9s-qC$-{P+&NXw>3EYG*MaI~z!kyWBRRaRUV@8R2simo z9s_|Me6l}1d5`fGWSRI&b-MFTJO`I! zyH`9b+4O+5UMAbiYGNYw8U;*_FdEp*@Eodur2UX4R;hy3+78xST)vG@@{OIyy)4>tqp~izN`jrvoI5 z6>4KwW=JtyuSm9X3a8YcUA^t2%i+r5>|EzauAQ0>z#i9u`O%#HtpM|BEr!BU7Pwl=LGB2L*97uVMIyy6?V-oJ3L~4hntK_~v z(XrepmLXV*+VK4yNwXc_P&S5MZR@NT#PG6CvR#rCVJY>=Qzb0Zv?aVd{aArz z3C~`xmo)xF6~nxZ?nr~-I*9iq?R`gg@)?08`!u#|%6MFrsOEI{c8SS>Tr;@N+9Z>q zU!rrJ@U>)Kj3t3B!avGHY_2U)J?MM+IpzWO1fLJGePQn5@JSY_3TWrempCzi=r^6utAl#H%&EbVs6!(C?$;-EH`;FEhVzaf{h=BPtG zm=*H=E|xvT@+i(TN#}T0PjZaw%;J1MAAfRVxx9nAwkM~9RLFz}ahHai*THGxbK*W3 z%h-q6A$#TtOS|uT96l zuC259-%N(*mP9qam$zd-d7-y*9Y6IJ^7h0NS3ws|v+wUr-C5Pk8`Gx@%Ph7Rua~kX)ytCYK9~%7FF0rmALjBp^^mhmGg*}rx^PYR z8QXsCaZop%#t}0DF86ajm?D`yt7f0Z!>qD9IY)k}-aXAbY>fPB!}y9H;{(CK$(21l zEEHdoSC9m``r+LNfAHr_e>_;0tL^Bcpr#q z`~{I;V}2m=S=2*;viR6q5 zn~h)IsB${zz?c@(;A+Q|MjOWVtf-wkXhYU_cz<|z)Eh(&WrwY#_J3wCj5LhdkGP_K zr!$^?#3dR2>3`G}^*Ql3A9ICm$7|VRt}i$x=FM}ps3h&`f4Eu{QZUUfyijs2I`U11+l#XnwmjWmp{Z@8jfhu!st zYbD(|_ia~HE`9gh+pe8{DsgbBD{2pktCqU9)1Zy-xuW*s{rP*Ycj7{>56)Smb|soC3oasm${I@mW#I!JJI!rUE3Mqxkp_s z-p1>dqpkqc2cL0>z0bN=dWBf}vb&pf-Y*$bw};}iYgSj8|Q! zyJLP~hCN^Gj{FfUDR#H`;4`-||2=o(%~ass_uMUBBz_pIN&c#)qiT zp5^WqTedU)N0+-}$}mh{;qJVSjDB9>ZZRL6z0%#df(#a}bVq)Ked9_p0&Bs4g6rOQ zM_SC{>i69(UZ+I`AGljQ&MbcK19z;nwBiGI)KTnzet;kR6Cb)`HiEx@=x)3QK5vyf z>dmcwxADv>cdPd?EC-i@pRIDoJSEd$mAjk8us`#WJ90U=>?3!#S>U%Hxm&HlUITsz zE?Mnvu?*a}+8y;3V_d%4-R(Va*2nHXQjyA!-Eof)e{zjGzKF4ZV~snel8L^0jl0z@ zKgw>1JZ9k`@Nw|nweA*W;G*@+yY+6@SKv{wyu=;#TZ!BC7dUT&+qGQenPrW+^$M+_c!=&aYy-|-Qsr5 z*yfIUdYjwzBDf6veY-nq+79?n-BE>~x?QWl9pJJucjVeKx9g8GchqBFxLsd<;g0$h zJPOYF(#^jRbVscR3&GW3=^l5~w|m^K-@xsA-L3$*8JxZsKk%u&?sieDQI!n6Nx_=KhJ~X>L5n+^@XBo!R-YC@d z-mq69jGw$s{s=dYc+cZ>6Sl$ye=vLyH;)lbK8$cJi{SZ(e?{8?-i@w~mMbi}Nk5}? zZ^Pg;6?a|BH7;XPPs4b|uyK@{U$_4)hLEYX^E0&w93_tF2K?^@_PtcNyzk z=aAABHplIH!Y%pgvmTkna(0q4RMJba#@yIGYT!U`_r#9IOs@QAwuybKt+A?2?2fkF z_jB>~YTL#TZ?+w$_6+nsvts($-fk_IC10H5Rz+8MWBtju(`3TjSx)w88uyIjR)}z$ zLmTp(@HF(pa{owR;kH8%*PQO;lW_`rVyu4Rq$9KMacHD4?&aR!$r#(15enVOexw({4l zuGdb?q4CAqT?S3UH;9xQ%KxDG4cb4ExP)78aPn{FM2)n*PF$`P30E2GwF6HT?66zM zxka%P(NSo?dm@)&pHTM_c>Uq@w!n?1-Qw!!yWvUva>_@C#gW! z2KcAoWy1Ur($HyeG6ycyf;==y)gX zmle_hq4vSHKZgv@hU}28GhlVH!x$Zx*(>2m9p@329bVFLRzW+g`oO7>(~n%;YI_fD z|LZ2l-a*H&q!rC5-)P7n$7U#hKIx_KQ2w4%@Y$MgIoc^WI5!_B4_S=*!AtoDQ{E1t zj$)M74b&OPaUtQbo=aS5+94;TtJ7a_3JmtdaGk!1_Pan8eb=+^oUC&SU2c!oZe8Qkp-_0o0T-tILn^~hA-U1Ux4hPygC zx-6Kk0@d_-Z-ycF4Yh})ScvGS0K5V}Ust#t?DxZXt2@^V2OR0yfn4x?h z?k{D94CR+Hx8%4#ndcz?4TJA*299&aSqey*;7rosN>)ekKj|_Z_@t3p1DWI zFUKa~Uh?e=t`lAs8M$zfz<*)3b=yEZ$VbysS-nE}abzT29LnDbFK5zF{z3RoA-t4t zh(Xr<&;$>`cRdB)kpGlZ_zT~V|I|jbzo7y!G??Hq_|6SfNO)PELmTh}d}xbtGlX|4 zq*uUTSLb19s6xUw)Pk9(C_wmz{AY#cZ}&en5<^1?o(xSO)h7twkbgG3T&RTh#5#DH zoT0pbJqDSnp$6e)#)k4ApCY3*r^rY6!Q^v99|gg+KM{_j%l|in9A@%dFtp$oIjGNQ zfPW3XUjzJNcv<5^^ATQ7bPiAd2gm*`4#zIX4JUZ{$?>HP@Q;v@oc%%zcp6?7vQU0L ze8*7!YKUbRI)@sFzXva;&d`iLgzpZ&;ZAByNVETPdkA7OT1-^#TfDs+aaX}DMhlh3 z+0$({Qd#-(a`GW>n7i~D=iDR~KhHzzhtTs{smTZuV!aL;VGhop=#)Yk<#!ljSkg|86+x?}qr{CwRXRY{5N{GBZLGJPg-R z0mblL8}NS*uAzcwz+dz?e`erJ$PRxqIK0fH&^<e$XUmk_J~DL*Gtu%D2P4lT5pO`z#?0n;&gM@Cqw5<56TRb0n1i7+!`j zlyA?$B{zJbd}sJhHm}b>!5;8okg*C)@C3Y6DwHqdXgsX}ehW<$K0eE7kPMN8n-RGz zKF&MsH{@$^_Shab5SMTjEt3|sUF2ls(Onp(F-h}_b|Ft@v^{nlGvwe{#Ej@lUEfK;4j>9SK zk7|1p$Ng{VpVW4*^AN?<<+sr8r@E0a&Ttmk42`k+v0kQ5F9($bGr~Ezp4RQY+@Xy& zG)pcEL-@|Nq2u2TM)dIn>ZLopz4@L?^c{);C*q( zeQhUgr`~o5ePqzfvHR^MLp(xnf6mt##5n2yW-s_Rcj8y;^l}VH_=|Q*hn&oFK-6G} z{F{MC1ssHbPr!K$UJl@sGct@AczZ!+@1BGV`>$d24s)(q=7;bd;fKfp8EW8DAND1E z>6=bQV{}FZRP=oGP+oR@$_Y7?e+yoguTZ`kUKV!Y?FC^B5eXKCeqd;VMzl;W>q7Z( z_<;@Z*T4r0z-$Eb9}6$nIQ0$&XF;MT>`;0QH6YY;u;W4dVQmlk4OztH8X-;JrYCD$ zqy2(>hUN$8qi3|8idP2GewodohE~*6&VQ$>$f*w7@)^$I9C}?fS3bEzIIJ=D11J8p zKje7+nIV_UhS0-qJ^l?G=Z5ed8}MJD`2#wCIWbH4Q^zlLqQ=d#NoW|6m)~(FTevo) z>UglLE`#rYg!04TtN_SxzOVj9Lmcj%b8N4_nd}xl?qY5@d8~fmkTEuh|4}2IaWjY`{8AF zhw}R=Sgs`R(**=CK}O;)y%NeVgY(N`^P)}=JX4N`lM~N!%@07Aa7f1`jfA_GJ1vn( zDIxgUW}fDQXUM^_H;^C~^%7ikxGYXLtT3;&3F#IIp^jy=NGhUq2Aym#=LTBT_ch;9 z^KukO*t0^7yNlNi@92l*O&r;emV|fc zaW0?z#vj@s6Iw#Bf}sgsq=RI##>inoxW@i#7@>Z>SZrmKLixTNeR79#k52ED!{lPu zfW0`KY~YLwFK5b7z81b~1AGK6mBT*NUp87d#OJ_wm&qNPK#n?@n4$br@Uq;8^0QCD zFFHj&CH4HJ{Nr8zQ9VO_{#~8XOx<3Hz;c7~2ipGde}$K@(T zLTSiV>Kh^H@6yFF5$$l&J6|s4jQ4c{IUPzk)d{~Kkrt?_qr7b!4{0A%{YCAZc9m*O z6cZs?};$A*7#-wc4KIn2p<+Ude{qcb~Ue_%6LHxt$?Bs$Dq}>=638`G+-5+c;m_ z;V&KUq~p^-S*BaBb^Lm3K6njqh3117HrMO8UUvP)y_%3EP{PwXE(?N$%{m^;cq^O~ z_^0NB=loE=P1-M%Pp2;=y~DE(1~X1O$#8G&@r6g=fTT(jkgmxcQ4|<5s?eUP=1m2FV#1jVe*4OLTioXnm;TkD}GOH7_2c( zqexHdgapb$pPbO2n@ZSs^ zdvNTB=y>ojyFCsMH=C&QqQq87%C%QVz zQ}~hYH@)^_`omMbi<`&ll7pUyJDbZoUVM$~T=_yNVa-*}d1%c@2bqjc(Dq=|ey4Xl zV&u&4cyW?@$n7h}<#?}gxr;7T>mKpm=MT2?gd+!}?9fhj+J7?N)Ht}UA-BUosSy_;c}{=Yv-7y65yx||qmPp)jd;S_FyD%fZ>WHHV@-p6Yh&}9_5JmH zYQ%!E>#h0*r}#$lpft2!?FuCrSw-sZY2FK37tC^egEL_)Q8y2;)~HXXd0Y9*wfP&z z%Q%bivdvFAL}Oz;&*NGJIuU*W`qXTnD`AM5ab z`|d$p-qVI0{TL*8^>aw`<@213WL=lg-#(xGeFTTYOTndc96x!`F5yYd8%G?X)Mgpr zBiazF3)o>BScQ;$Ki!_%&5cnZ1)K+O4?(`;n56ylHJ<@*Pw)%0 z;jqqVY)D3p7;?#|NM|I^?j(%Se8wLR5#-lvK5?YuC#PEp0nNwihMld;H)h%EPIDtW zq~I6e?IC!|$spK6UUC$^T=RE2X5)U%rzwXRq$~QTU4!OEaY+8*5wu@2n!C_RAm1;L z&^yAJ?UVJv6}+FnS@T7jA7Xp+LkP_$>Z8MM54JDh?TS9=IO_TueOS+hAL7q)s(4J- z^cHO>zuKPOhWyM`!dA@}yyrNk=`)Wz(#a^Xg)=9v*7-cD`J68tLVlAkp)nV6QgG2y zhxd=y38rd8qTGoSg8Wgo_S$g)-!+5kZ%4f0@^A zaf!nR56e>9-%kran2l)pPEaM-wca?qP(6b)FJlqe~~t<8RujqkJKe(X@2hgPDULyzee*p^BrQp z<8E|r=@c;JR)+|-_y*0-eb@2N(D~%Iqw{!IkGa#BJKpJ>DrG<7$VvQuDWHevRhu)jS^;oxw@; zYi%Af(&77Sb4Pk!>RWV;L*QnIbj@enUwW=;UNHGUQ5a6TDO?XOc6PdNnbcb$0yILJH1+7yq=o z9DlhDmGGAKPd@H{yJPrKCpdh!lc3C@jW#}~1;cebxUM}8Z}*%%iJVtR;<7>Wv9eW+ z;L%;(Lam+6!S+NK-LZOVIExwg@9@%r+4|hKHsI7NIJbLua}s3fj^&(hhubxubFC9UU-QpvzCc$fMe}8v zPrl9}g2zb5Bq#rz1=0~*2wbQW^jF`_p+i2#E;9tMgRZYJyt|XG^d6_a!9`#iyv#t4 z{2z*X;VJDu_ev*+U@g8GOe}VdRF6}GO<|1w0tj(WmKKSBPAHl}%p1R%oP9iw4yLvgiG1e*fem$?e zr#XD^MQx(azfo_S_Z!I%I0@!B{b*d|7>qQ%9RI(%&IQh@asU5o@6vjTYELtj5_T85 zNbT$Fy)m& z)`5q>=`<;`%P$b*Loh+~or}T}xEj0*0zV60kH>&)i~bkfJ|09X6_fhV8I21NYA-#; z4v8~}o7puqS`^D*Fc$`ulO$p%1l|W;jWvdyrjL)JO8KHLr7`fsZ+bLU&DSd_@s~kP z_k-75XwEK@ow1d4Zos!v4 z4fb~Lg~Z+m`*vBRk5wTJ*8JcCG-o4;D>_pO3-qgGQ4i ztaB#6LXsWXCy#OjjNPg;HO@zCE39vUmaE*XxeGZ_0X2M<7h9C7aK=}J%U z<#$$^48g!IR-Oc}pCu)J4365SM8E1<;e+Mii~z5{P5@PaNq2+S%r-pj^hFhY2SMH~ zVqn+9snaBYGf#L}4$ds_`Z)s3+g&Kc2Jpbm!j~ZMzrd@(M;K1+e+ku_+9vtyozM1Ej$#9}4piK&$A$}@u6$k%-{`KJbdzLGcxs6?-w1N|Muxzkl0?sI7)+7H1ShXscJ0Y|~{ z=D*IH$3E*7ZxF*gb&H)0KNe<70pL>XN=k4{7bl&vZmZI<`3<_}eStHot*L|8Q zyst|J>aO_ zdHjK{D0=iy24CahrxE82<<&}t@=$XxnWwhN(RUt$tuQFZ(`ggwIvJ-iBT?1lj?-fE z*Kr;s&KXLMZ9EE|hJM9Q5_l%`JDe~2!!AiTHs}Mvgi9Lp0pKeS0R933h2BCUYC+i%DW75 z^5u)4iVjk9y97JwX(ZsP`_<+Mf}3Da-%||Qfxi#^=HnXIP#yH`-NT)spBN>9?emeC zUYvV{rycviB;Rks>30%vhPrqHWQ`V{=iw)TPw?;*ajv-vpA^7mr~pWFA-7GHYJniAP zfN%8hdx>+2%R5Pic$ZU7A47keNB>*s2YS)^hZFBq%Bhc8bkgb4V=#a?2dL@VxGisi zeqem#&Ui2ML%E`lPVdx#=O=}KA@iT(bTdn>Y$xV%luMighf-p|`=oN31s)tFyjrmH z0(j*J;c+=Q?~D^awIc-#LDPJe9?zEa^O7fUJ$Q|WH@{T$w|V%n#5tq2$k0RR6C=PY z@WU~!IZfIGp0>Xzn;Dcuzmb41^i(ulfD(f^4O|Wo7$YV476ImfJA8{bC0dJ`^fq|@ zt@8g-&E?pZ79{K^PxRAHYd9JMLH<~Ags*7MN^tv=x6SBY@BoH|LV#WqB|z{p@zYrb zk24KCuUPo>lQ=`v|29BSvp@t-z|nWs;7;MUfp?!I0m>H%$Jy5@09NUf33OnL|==gh8$mk1gKpujyix31}_Jv3NYy&;@pSoJ$+&Wc;M2;qoam6 zH$gr5Nwet{{D;!1g(3*GX`Iu6#LblKadaVgUk|?=ehRSuwiD4F@Y?&NV9#Lv;p7!b z;L7_&K;ODeS^!=PehLcmw$*=J^l5aN)T~(i)T|M{4}36q^~1t3lRLAEsr+0OHIItm zEv!s-!@yZB99Qn0E+wL0|FrPs2r$WVWav-yl?TCVHj4fi=!|>7>(YBf@FE;_DU|>r z{N7|QEKCH?s}ucx&BWk-@TwPuf8`VYz2#pBuY&)+WhlUI;b}~4&J+kLpAkV{IC|3Z zmxSA21b+q(0xU#;=;aci8hj1>{2jdD4e@iX@U-)e71W5pR`t=7;fVIZ;`EdpoU!2L zw7(W7J8JF&577QvoE|oRX)_vd-iM3R_3#s%!v1OfAH)~<_fVyG8UbCJrwzF{;ku&p zEqEU7D8{J`+W0E^-p+lfcB}9mMoKq=f>+bdWSpi!KSaaL^f`VqgvZ~BMoho@LQoAy zoxooQul-gW`Hm7naH=>e|4R73P}L=te=U5H9Gnbz-2nlMkHPpmXbt`=f^cgIkhoF; zRO1<7o6!~E_Ss-NT(hT%zI|5sOBCQ7aQp0V4*0dxsQg?NK|E9(Z7Gt~WMF=*o!-KL zg7(6{0T0{?4o@dtC4mdFqz&zx=E}hB#vsQc@H61`?eMsXU8DUU1c7XEbR!H3$|XQ0 zcpFspo#4SD(f=0uFB9h(GwGQbKZ1VkAklxH+n-bG=@KA+xClN$n+*f6A1WLtM`s>* z6()i*=>HeIwnX&pRqtD_7XP7+A$~c>4)T z^{AonOwpGIwy6GTp<>cq5Y)xQ;5hIfh;xb8cmn@v9Z{8Yx=#+yxLFdwKEP?e*t`TD zLes2deQtl>wPH{ycU?IbpahqLn>PrM-s@~Y=}qFCp&CzyJ|WJdrrRnR9rPcQ)ODi& zvq!%RadZ8@f@U^O<~jo%OGv;~T7hdjTa1(*ON66iE^FMzk3v6xjO6?&=0Tt&kW+6p%PDq7C=93A203#gXbZzy9we{;IuLm16}j+ zI68qiH$fhL(6uGI9=yQAcY#+3Pn+AwId*i8II8j(EG5n*xWg0p5%7&3{sMT7hkpS7 z1^8`=Dwqhb>( z>+|}*jyK?;lxP0ZP4^WNpw^6Y$2kjp8hFqQD|(m&{3-Bi{M5f4JUW*@(3RDK#+`8} zaV|g&`7wFhie5Aqg1T!YLtWwM4dN}#!$Sgs;H_@r8amGDh3o$T@C%4@fc)#l&*R{C zK|gr-_W~2&d4U9+(=nc$egIzX;d{X=Jp4c4cYAoNnvm<$Y)N#&1Q&vL%fJc7~^zBoo=@$`TCj|Df(;ORkz67wle)3x+I2BmVdmT}V zn7{ObAMgV3UMSe{w~C*-g`)qA=%<}=5LEGUn3A0VSAjcl6f{!0K1iIa)Z8Eo#)qs0 z2mn7FA)5$Zxk&umdoPz8PV0XMcOHxqxzlN(II3DL05?WBW5H`45^g)uO7McE!i!Nu z+rTSirg7FGL%na~5A*{D-{C9$UnFOefZMWGi=$s)aI@w22*-Cx=Xv5hL5;%%g{F6! z-!6VC?-zZ$mP~<{KPa3hS`&XQ1a+%KU`yt|LmW8|3-6D%jDm;E$%P)0MC1j;xnzC0 zWPeF3tp20$4?iszQG7ED)`_47js`D6O{^Dw37jn>&L#f&@C0Hc??6AaLG<}`C7ik~ z7C(9Gq)n;?J7wUZ^=T1QA@I`>)bhUMlxoah+B^47RO#cw?*@MuJoJQcbCM+4_u$pw z=Ia3QlPbke;4#s!w(UO!f*J^Dt!UD3;Pog$0=!@ef1uxlJXJk|IM-rxZcvL@{U zubL`>o1wtzYiWjIgV2X!@EQ#2%*;c#{CNWW4E;I;*bM#TU2-l+qG>|l%ZRi8Dt;1{ z62B3K(>CH%fV5NV3Gf{Z3bu$Q9vpL`ROjp{&z!Ew5!+K*1YW&DO7@mu=PlDlblUa! z-wl1Imi<%xUyiEiwM+t(ZD1 z;5PBY@3S%EZ!H8h--rNDkvpepF{n5woL^tVsRF$23*q)6^9#f|aLg0<1L#+ycTyiW zsrx+?-{feoIKl-@XATK?m|f?YSoRU;0Bbz~ekIO+3YJR~@a99D#@x$)vz^Tz{mH~x zzp_tr>Hm`@zVj>u1!Kk0QZzw}`^3N*FI=I?hFIQDB7QAmXBl|yr2^)`PmJ1#16TDD z{dqF~(F_JbeN+U`_2is8&XeGwusE`(*F)gWd!j$Bo#^*mApt7(2p@_rFa^AxH_M@9 zS45AeA;{Y-f|uHeqo2WpHwfqV{BWB7fCQ*X3*dMCa9Rl-K#4aAcD@2HnAo`Hx;-d< z(!l}|oQx{n1wnqHaOFvyx5@8Z}9zUT&=?THkF7S$w_<0!omMRpW$nbQ- zuhv6XiJ*Lv7(5Dt-@yZ7&~ROE#Y3WB5fS}T=+9g&Jk(nv}zk$BPn<`WK%>1RDflL)&ao`(>Q~3|4 zD@kC+C93W>i0+5M7LTJRpCswEvIW{bc{JS{vQ+^iQ!HuV`+ zps!}05?6pXFKb+LcN6EDaCmoVO7^~>j}7PR{||_OUxCKSzflaT9uYoEurnDvi1A$n zzRT*bXsmz6v+#4T@B-*xW2RKPA&E2k*YS5N2{@+}kBPyPEyUtg7*u*nwjF$phkp-0 zp=$BNZ*b$(HzWQFUJyn{-hcA`s1%>+P~UOWav ziE~bChKNBQbfOCAmw5E=hJGDRLiVg!4<5qF>>5*_)EUkCo!0*|JdW~2;N0WkV~BHx z3MwQ+zoIiPhW-YR{zKq39=-{FLRYaL8h>_})x)69W6=B+&N6i(4?h+>Fu8FX_XQ7m zcpiA(UwN8bk0F8S6w{<+_HnEZH4=EjUkwQL0l&k;2Y|2j@X^HCf89*+&u{kQw5$f} z|7wf^nxakGNdhkM7LTJJh?^$3RUEyIKGE-0;nmB8n{ya>C&b1Z#HsjL}p!1s&pu9pH?2-d*Um`qIErI`qe7y;t4)MN#lzxXn z`?p|#Up9JyPXV|47ES- zh;vN@JT=i~iv*~>S@i#a{xIUK-@~In0s6tsQs8dQSdaSOOV-gX3}~q&2PgX-{y@Ls zdK~p6&ZEF?z-LZ|qxGXU`pbS)aFPxXxqn9~jsj{cP3sd{+Wjy(%S` z%=x1f0k?ZUDZ$R)z=PNm>LR}gRuSi~_w{tC#*9DIoU&>4f`$GHsp6(0TR#7&8@lN!EdJ3oNi9oB}z&vg3A zCHMcFAUO&^@J}P4#UXfW@L3;A0JD1=vCiO|!R?l8Mc}DzqHniovs)d!PMrJ1Z(IQB z|6;Kd26ivEN8o6}b}%rR*H}N~~pyz;{ zpZS+b7+%PTPJdCzmvC}r5f%oeNg60`kjDy)?YS0vQzqG*5?<;9W}e* z)4dI!);D~5x#80%4WE8#_|%bB2SMW{N}t5vaQf8Eu&Wx_qJ~ea8$NAn__U+p)A#n1 zuce;;zS}X}`xDwInI;q4;I_~q;$8j6Ig`;5Y_UcdvrZ>+4*hK%@4i$;BA!Lo?fmrR z37cfm>d2%y67*V#Y0oT!?P)W?oa#6ak5>E}mAulJ`JR!(+j7_^hhOB-S%?4OcJ!Ss zhl}JeLkyZj{1BzD!9|*H&H`UiM<<19T0s-fB69g$d*GNISep|PNS&P zaK6h?o>COI(1^qDQC|ztpsK|D2&uCQX4M$9{UMks&Tlmb9r(~~-k#09O|(|jHk-ro zAHi9{a5*1@&|*7@o8IF4Ui7KqKcs^6mc!|C7%hh~Iou$JWpY?AhZ;HTHHS|A&!x7! znQEiEZYA&C&Ecqlq-DCVZ=CSI%i%6Ltk+Mm^4sQrnYACgJ$+fF;Ye9YaanOPToNwJ zjmL&8zfg5><5R9IpL$JFS5Hvwb)S6INmp%iyC0t_E}S}b(pA%nOT&rDWyQJiWTd1p zUYbl4MPm^?d9T~kpO~bd8m~I)2Y0x~>EPFHOkeP$8_~b`Ra<@K0rw<*@gHtm{oO&g zzYcA8^Yk^Bsx1BJm+rZG-VQfk|5oR=*Za27r`g-xQTn=_qUdxBosTea5i4!4&t znF;N1hxv43m)l0q-|0T3Z`e&*hjzKu%_5<&UiFFlsV|~$-0jZP=LS`_uIQ#Z>&@d; z>s~Xbl=SO&5&i8kbW9KXkBX8ii5Eq}MMcGtqQcTxqEzqO?Jo7jGuM9Vw)Sgxk6W&1 zeD2n-igW^{L<~H7u2~u z8vffwkKXHc(*1_3Tz&hOZm>a+(1-TA=LF(qC555lXe^p4OT?qax%&IP?rD0?S8lsN zI8+kO4JC4u(cEaFxHPFp{o&^5x{wO!H@} zR8o{mhh+1;{kM2138rA*k-G$t`{q^%dQEr~7cR$l@yQ!3Z_b0b3D-w^VawBEY zl1MBSk41`y=&e7xmo@Jf38(bpuicVBakR8J84AUtMMWi{NLiWwVw*dxBgw)$dE_RU4=y@A;3LlX>9Z?l482=$qf&SuK2VeOYsL zt}i$9XmhpAr+c}oSWnDW39Vce%%oj4|0tj7QrVeKt<_}L9}ekJJydpPS&q8VM|RmA z)FaLG^3EzxU(}u2Vs2*&WLp2|<5W!V|H3WSU5FhME57RL(rEdNtHabs^Jl6)dUF@m z&Oaro_jFOGw9ZW=i;5yeC8?sqa9l5Gu5y}%!wLP#aq3ZjIH_;yq8{^86`a~x4c0q4 ztM1uN3LA-K1|6?D`+Z3rzC^{^BvPShsH`MWoQ#rFy=tF3oCjZj-7%=T`=cp+Mo`^s z3gf52TmHFw^-0B{RB=2JD@>JyQ_)bkY)I28jYW03t9rLtBoWp%K`LuuH#Mt${0NFE zUHET$=N{LVKOWZi{p=3nDvJ4Hdb*;7}ph)lSTI?+eHY1Any zn&3Nh-5`dOA4dmqIgj_94{;` zEgZu3opgs@YK%V>)d5Xj7L%($JYHOuOqG_zXnYjKBDs2$R$Z3gt2&%i77M3n>?BGH z!>Lp$P4!L7oyyJpaH`7qDVK9|R5s10A-yC-*`+b?mD^p{hg4dh9HaU4vxsV)=^9b* zH17~@T7i*BXn8^{ZQpcEMj{dYWS;7fc`2dBv?aeIhpJODXAV`n{p}-7G89Qfb>HEt ztWP3#1X`?$LL3R%daC?9ya_MUpZ7;z%{fpG@gLMylIySDy}y zq>hlGRBqFZrxHhCNxiUx>b3k<)n?fRstuoXXOQbuW>vm=+vg7@bk+kZsBassX8OXL zE>!be~ef2{jro@vQl-+ zti4oS?DM5$R?fLhz1zO&xQxal`l{`8ilo!Gt2gwe^J(ZP)gse>qB`XB=f*N87O07S zUoZxR$=TDf)FsfTlQ8n5(RrS^Xo}%8;l{c!M zx_*i}I&<)E)VuxEv`cPOC+q5|YH()iRCN}ic&7SF)r&qP_3mkE8aF%ZWXi8n#Yg)> z`q>-Q>6x}SsONk%nEUOgoDRI5)+mqNMblcV8&yc^iaN&yH_{w8^eWXmletm-y_tS< zwhCkh%~gYZ&7#!6x7?&Q`N&`GbQ%?-=c%@Q#+iPTnIp{Pz~}SKnuUtqS|2?;UFfBE zP-iY!sB-)?!SEcRf4@VO^1NuLlrWz;+N9M9nc79Fr{5n*=|l6>u{x_#{Xwn*OVmBt zO-E=<4;n{9KNic?TbHP@l(P@#(2O>4sfzpK37N`XU8*MeI>eiHn^>25mH zQjyVaTcRr-%FR~If+*#nL+565zk;4N{4>?I=l@>%K}J$sp2tbe^*^(LpL_+`tf z_|)L$KYuP)kNd)z$F-{E_92#|Prp~S%dELqwWjfutAD#sos`*iAD#B8F6?=uJDr_q zpnc9YSO0(-Nu?j~pn8`)PzNs7Q&&=*)4*w=m#w6kAQCsDGV|+7)xyWsd(KHT_Z+oK zE%t?J9X(v{S*7-q_jevrGyKt5X5eZS_4#Rt%vht&?-OlW;CQU*sEfzK`qcw;CjE1b z`ajCZkE^IP*7Q&(>d+(V3jJ~~b%tJZhYB={Q;R+Eh&t0BOJoi{qDq>zipQKu+-`+-`S@ycaTF8!tF Nd8>TM?Bg@={{eoZjspMy delta 167416 zcmb@v34B!589qFjEDTGy!xD|kGT?wg1rr6E256ZR&z?WdaIusYcw=x<%_U(@}|9HMq_9Jm;J{nYl3E|Nr}b-~5z& z?{l{Iyyt!2^Pcydd(-~#;l2B&w-=9Z4%x)dBQLhLhi*CYPorGRNzNlE+(Tfdnkq?uE=@vzgcQaxBrgVuVQVyfE|HH%Gv>!DGbra5NwRJ1#8P zd!sDS8#1qvzvi4je*gL+huUXO_uVhavtIMY%UbisM=r82pT2)-S$UnJrfyDTkNt#+ zKRxWNYo6^}`q1^G5BHz!OU)RhG<&w|%v!Rvvu{S>Rs z2AZ=J=*WV1Wx;#1;Df`sXx~=+J2_^yix1r6;w-ps!RxA}?$%iFW`);V@Xl&O7HF^} z*p^GY$%4C1m3XrSuQ@?>pu>V+q3|vX{&R)*8kU4^mC$d&|Df=g1)p@MYo8;W87-Ed>sqFALOU!RsyfT$R7cg1@TpwoKfxb&Xf7%S^BxyFlVw zEx1?V-4@)er`LiVvN7W`S& z0S|G?zc~aSsDxTe0TXYq;AfaD&4PDY@T980D-9R=cgjM^>fW>j+Jzew-fzJ-D?Dbw z`xL8(EV!B9acL$;?_XG;^xu;3v}&Nlg1@Zratr>0!gUKi{zTcaS_^J!P5q@tCPfQX zfhJ3VOHYy&wP(S%X2GLb@O~4Q{kPf71_sRp@*lsAs$oZcCZTsLyx4-9`AaSMUsZmO z1vl!HP}Eow2Gm&AS@7;=$%zIlt{P~z;5!xGZowTBMLmYC(~=NZ1-dM_*>Kc~tNeWy ze6PX>EVx%CgJ1lsS64GJ|ZW^2t3+`rq%3rx9 z;kU}f>K6Ps)j+KUHyfz8;3g-UEx5^vHsC`3P4zQ5(QYYVa-ze6n`+sW1@Fy*4-Df{ z|ICVph7*J|nz-|-%po{hX-!EM++)E_?Wo-kHEvpQxbi2mDbho349h;2<~kJl)?)PkoXAQs&7 zft>vsEVxnEBoo>#37)@8yvu^usfxNSc&Ean7QFpKSx>J8|6F#!uninI!8Viyclt9q zp(?VKWWhaI@LGX`|EBtx6*Xij(3S=7%z}4k!TZv2j=$M(EIlC&w>2KP$HiIj(k!@s z09?pl-2oCPe@$8Nb_-roBZYKp7Cf2-?{BpD|7JylSqeC=%jATa-<1U~&w|&aaHW4{ zMfIr!_TOgvwdz@u1#eKR<~9rdipt+%!M`#&WWh@el@PTiD7-%lK9~h}tj)rqEVxmg zC7~t@UT?u~SRu#0ISbyA1@Dshu>U_&Rn%=M;8(1UTJR=?_ge7H>m@;b7Q8^!-=BeF z{9P(xAS1zMGaHUs@PkzTK?^=k;X@YuFooNiGKWUgPx*6L5?WLboEChe!iz2VI>iZ> z1#eb(i3N8nf=Yo4`8O?~+2e9c0TcIF@X@Lw-GUb@yvBlmbBi3B+F@Mk-waitZa6`x zpV@G|1s|*OH(2nX;!u+XKV0Q+w&4Bil>gsuNidgGofiDPb+QLt7TnAqwc!6%`TH!m znSa2rBz&a`*sjkMma9yeu;Atzuf&44k5{{67JN|QdQ>9nak{c3vLoLXu-{T95-Zgu34?7i!FGm2tY#qN-PN*Re^E~ zZU&8Y3*N5s*IICM8DDR~|Dy6Y5f}O|WZh;n3$$4Z*d|D!>#*Ra5N);KkE#ZuS@8Ze zTlWNJP<81z z$KOn7N>50`+q2+XEx3NG)S7M!zEa_R7CbjnAm{%9OTs;>z)%+4xgm21#-JkpN-X#l z3a_=`UA2#4KgJ%%dKWJwq}U-r1&f;V3v@eT{#SEo2+!Q0g-nXMMwtlx-Q z6Yi8l&}+dv?~-_*1$X{V;{6uf6zah&xT9If(6ImC^(#39E}4*0OA{~8g4blh>n-@} zs{ZB-9OG{$bYvu8`)!LPsLO&Mr0}Q(U#{>z3%*C;F$?}VamwG2CE>RnvSH^0kX2H8HctUZmHxoB(X2S!S z3AUoUoDhp{%ukT#c2uD zzuOgFV!?MSyxf9&yHivQPJ63p{}O&0tRRZ*J-H(j?*3tsx8 z9D=9?@B2>T1Fe?X$)gftmI8GOAGF|I3LmoIQH9$!4zrpZHD#;V#HIbi8BkTgWhO8; zes$kVN=qzw^A8d)wcwpuIy#n^nUd z7JR$J4ck^r!Y8Ugw*@x|>b2n0I%Nk2EqJ*i)VV2h2+mabT>=OHP4zQRK$cnxEct_M zz+=JBQh2Qezh2=D7W^LyZ%)TK{*S4I_VfhcuPMCKg6~s!mjy4mTN2u9!B;B0--0v0 zkiWqLBv2?EZJDey8*pX8%PqK}8m_b8eaob?THj{z{|8khxFeX!(GLt&pxBad%pYaLr51dZ z!gUM&ONG~3@I4A|$buVfmW12?BpdF`f_Gc+=T-i`EO^X6V0VRDl``KIYeQXlgC^G=TksPV-T++4zi9zisDvgs9;SLKvsPb>M;A8J)e#l?9C81u8WydcvS>2f<8`x^Wn-y+r&&;2Cw%|2o z@oU~+>+^fY{=tV8@;lZKBPILCiNT0*Cp z0Bm9!-j#+IrQzLa_(5rSGz~vE4ew3Ei-Ak~i<`c*ghSE_^rzv6rr`r=_+e>yEDfKO zh7YFUlZSD`Hk6hyWjMiR`=t~S4vi}fcckId(r{-QK0OUDPQwpR!(9SL|Ji^$Eukc> zz>GA!G!37bhL@+|N2KANG`u7Y*VFJ>DgGlvOGqRgp(A;%V1b)0smER{)*J0G<>Q+)IUm_nE%8FKVO>2zqMve?MWjLlZpE6 z$v(a1X~#~HH97CGg(Gaq8BgF}ZK?-%e6D9&d?1P6j0gD4i6h=8p8v{c&KL2hc>eiG zk&A1tw$J)%%kNg$U6HS9X5>Ebl`S%P#kAaCd|``JuDI6z7nSzfis_M`oHp^$mw$j! z4=(A~!_|X&uzCRZF+H6Bhw&q}?44LJt(djz;dvt_f#C4^&KLz_wKlaX^4W?~`<%{* z>$IZ|op#_lzL+p#%T0gFEsm@|?ZiX>@qMbDp|p0UqaBgwQ{<@}&XZ2dv1UEzoqkj9 zJ0Isq-aCED(LdznlI{*Yyaaidbb{t(1v!MaqATb|brXTh5UC&2t`FTQRM6e{4;0dtus!u@0t)PJ@e$kjiW@lK_ujAk(M*-T!-We zf{=9yT3^zKl+}466L_0{d1)kfW$mHk_NBOx-oxom#f9+7*||^sCqMG!$~lGO?4m-B z#uKHp1o$?)3#f zifQ|jNo}+@xJwVd8{2VI4)AYc&wUk{_|sB*&AiCTKb>WtJuh<2PiM?s2`2$%&vK2h zB}U<9HEybdpCP|5_+zYgD6O8+s-6jv=YG2J;O|e$v8~p%tDar0T~w$&oDU&AxA5SH zBgT1?M=xHjo%QS&nzLx{BavU8bKl_al)%oqvS=-FcCh&Ogrn z?JJSI3l`X~m={@e!E5%{3hq9nF6W>LFWDjJyY=8ZFdjAMRGwdXLHrndPCQY7`@PQ7KF+hOyM7CcWD?Kpz%yc_!>5?Xyk zq5Y^4w(WE|l0|X7U@WPE?oGNi#No4+| z$4=RcMA;HZcEoB_nV}fU9QS!7bZP1ITke&-Db|AnCYwKfOJd~R9pfW!UwV`M{{KV< zR~~>B?d^j&MP(Ocq+28hWJ^bcy>t~OPEWGUfk-vjpfR7!AM@oKnt^Kq&9=!8s#X0uc zU6E~m%&$P(JS9RDu(;`1!HN$?$w8)IBXV|}RMSNGU zFdiAh?on%E?*BkH!bKmBhLt%IH*fJ~K5i1Y@dZT(2aqTJdv-8f^eW)$;KwRu7gILL zl*cks?nO$X0R=*=a1w5A<_(*;-r(Ib+_hyCxq(Ga1bj^fyo&G~K=Jz9eu9-3WFH@*!>$m#CGw%O;K5pl2ArhMHo{apV$xGmiBt~jeZK3=P zQzGtbryX++(i`7QTn((7^VOqVD-UY_C|tyZ9s2GiodASW70RD8CE~yKLi8=&f)1Pv1^II9b$2M`vM<%Vw*5%7%Y^+TuZ9XyMsYB9qrlKI-2{_09UOI_F39 z-xvHkSyT?Bn@jLb`CgFMGL+jm1*T=;zC7a_*>i?2%{IR84$^wkwqX z0^6wA**GCo_|Hj^$^Ju&?gOH=cKt}kPwv1&u<*B&B0m2#V*?(LbS_j)-<^LYp6KZ{ zFWgAnZ|E%xFBW%&dT8M)rWXD25Y**d(y2MCyR>nID*;IAyYN!+&#*|pY@ObgH3XFqJjmKDe3Oq_V^p(AYZkA5J!@T0><*kTm}_Q*kv zryl%nF;(W9WYO!mi5G!57@D6>h^%Wo%w9bqaz~@f7!8RP&f{}$iSmobqWp&UAmKAn zTRx;9=7NKpfVo+?X*!KT zO)Ix4|8S(3qkHh79Nm$(p?>PYtC7E4cmBxPBq=mw`N0wA+8u{2!YH&1cWbEMzNi#GxGTZxzaM6sZVJt~!X#&%UH6r5xjt01~uKBhiCkk9XCW&Ay!X zlJ7lA6`^)~elWD-?Zh?0naot2*O^4c=|EnkI6HtvR$jk(?6J5pSuyiqvZDAHG>a)h zKd18C%Jb-uoCo8R|LoYv{G|S@fw;4tsl@s zjj`Bsku^6SVSnI-$elMHF(J6uZrhb>t8V?EadHCQLt%#_@;Z{2j*ld7oH}ymcp=BJ zm60RXO|$2{5LveFxLoa9d!%{Y%6UJIBXLRpT5ouROAkNngqT0p4$dTt&OtfvZZ3;t z?a9Uki8ZKw)riRG^|NLcBU!YX^jDxS@GJU}Q_Q62C1-r$q@3oBi~Mx`!jbpkfdl<^ zq;q|VJ@8!Qx%EdKe=1rA2#MH*IjA++=t{U@5$nK&$0?YJgYa~k`GlVLC5sNjif2P- z?ux%Eo}9WTjVFsD-)*QF8P|lgjM5@!7zz8sj!0#*YxLa?!P%{`>3bqqG`HFpkBxlU zJli`F>WanchTma;9p`Xh7e0Ep(9-we96rTJO{^bHnC@FB|7O&lA^hN3{z5Hs*-a-J zv)Qr~;QQfagZjgOK!E*)f~4DF4wQAb%WAYu?kocQ9w^Hqz`K}a5#R@aMP}Z-s_-aO zyxr#F@b!<0{OabPj2(d_Q~JIu6e3;4*=awKekY$IUG=?9u&N_=&Pzh1^Ku~4+N&^P zn(&Im+!6cipP4d#9lB_?@-xv&$%C|wknt~HAOkU_-7NB?84mJpUZLb2StNOPGu95| z9T4ztE^2}DpE+uncd{C7(~NAqL-(@rj!9PD0gJ2-tSU_4W;?&vmMrQR6?r{SKDGde z*{g3y3*O~|ko|dgSFYgQ<3Cb()_24X1$!V&JMkQQ@#Qq$q4wB)&u8+k5nVJ}xlFV& z@><#z@a}=g^Q}kOSH2V(YMpLBc4TDI#$tQ-xXAGvr`q4x_25|>|7W*f*!W;|@I-J>oSZlLKlk zjkQKI1Rog}21WjPH|IqrZ9dHY&)i7G=E+Ac0cdMH>P$?~kD+4F`JsOdW~Qg`5If{) zI8GPChV06poEK@{e2jg^eUX1`o?`#ieUXyxIY=_%>M( zx$BnWa-YWa6&RQHa+l&-bL&O(-{*k9Hi@;{xwPKN87~6!26qonx|rb~llhWG z|G*8V;n;i8$fvi~pK_9TM26#KhSVgGoZ$xI4Lw&3%46x=Okw=`T;wT=U{jP99~=H z%F}C5Cid!?sG4jZM>b1Vou8ahB+3u3dD8~s+waMwZ(>KHWz#`MeP>m!tUTlN8ZTln zp$jlNmpZ+{cfBp2jOys58 z=eU1&I&ud;PW)F578*3sxtHL&Xp5eED6ZixmrQ$LpQt8T^cUPDD!rleoz;ohg`mCfKXr^hrOacLMH&%jMhVd>W_+X$5%+8_s;{@lg~azNDtoT9)M-igPYyW zytejgtwTY0N89p~-y&Z)egZYh*wes=x7i4NZAA|2C#rIj{9ERlSJP4nRuwM18u?2yn{`4!-nu@cq^X9N@V~H3K9>^{<*kJ z4)GH7xm|kev%cWPSkZHBuLTgS1b&5E*MqSoWnU-GKy1jU0v5VgwB^HEFV7(Q4wVOR?ceo+hLgQ8= zS+w_Suxl0cYADbDddtvAt@Q;mO4S(bX?ZVK53l%9Z~4|PsJJ#Cd~M7t`!rGR3)g}4 zdi-BILJ#F~w#e6mdl}%r(E+|rz7cgx=oRQ<@|ChrWk+`BzlLt&XJABTzm9u7$PNaJ z_TV0hR`zUl@O$s97xa(_oK13gTfW!y;28JXx!#JGw7_>LTZXu&vRKUvMvu}0H^Ir* zlRe&w7aBi{{|h-%>3Vqa2)$)sNT1c;IKdnCxd+RliDei9Lsj6b81QO~pDue=O>we9 z?vbNHy`s1AGf^sWiWta%wEkI}<(b2EB07CpM3Izj@TYy50jXxN|> zWG`VDwD4Z zd4|MX>n%%`eT|}($=4{2n8_=4Lj3w6eu;6(q9^_by;xQ1ZTWFzqdA(1wy_l$9>+G4GtKjdB6V>HZ-sk zfdjx`Uc8$zgq$8!nA8H*Fs;7u^*O$Z_q3K-w6pt-ml1;$tUXK*rw!J(vH!e_TC!WE zZ<9r3Pr@V`8l{1RFe;= zF+|Sc>93;Ke`Wj)6goh?NzoV!_+YM`srnm#pSaJ=P6MZO(j+5zuX~VFC-D6mYdr31 zJ#?F(E!Zct1Hu@5UCQHU*!kfDMe=>#EADN%@8;Yo2CS5R)z_Yf_^jD8q9zpE2jQ;mRNH>(cJ)anDEM4ZPvRwJYFm$9=P) zFdm>|F|3~E7;4+b@(xeK$&$L0p;}(yO8DqMyj(VZ4`(b-*&fzJZejz+^@kb9 zkb!zdPvf4rXt^bMbmPjecjRb+PyI+p?{TlzvSU@m|A*OzS?2Y4a25KST(1TGeUZ(^ zVy%TuHk+q8IbCa=WU0U$3HCy%tR9xM=;7PM=*4nwv)h!;V)DMt-4E)tz%^@81Y{48 z*6_(rtPX~Xv`xR`Y!@zk#~&%Zd#bUPPbsDHR_xMRk6*`f@B~?o;w<`#eynUHaGT(R zwr#FmZ~5^AZPU2xfo@M5^KZ14BdM6_T=Fi`9 zH8!8O8H=2&wW~0*`wI!I*51VZ8$An?Nief@&VIz{TI>JV+JT6>nS;0=aX}2SmAIwk zzaY*~j3%oVqr6GoDIf;-%SMp0Ir~#~J0%0l6%VBE$^#SRgL_r1wKN=%s*4m=P<8at zrfYX&0FO(Nl}1%r8dbl#N>Vk7R9%mUOnwFKQ1V)bI%8fQUx0mUy-^HW=4d{UL(_Hh zVaE4xT}m-l=u0{JaxdlTK+JU9P}wNChrV;2GE6JAGjNxn zQck0CBNVm@`OXTUtZI-G#5Cr`!IvQHei}rlsw3QRU`U}r`;5C+Yvpwad-u;c&4 z4G0amaXgx)jiggU=K#kHaDB+#MQ#+QD0t%kZQwBgzc>tE#~p6cI`Ec!FGV%tK9Kj$*^_OwaUJWP&Rxf6ZiUzml&$Tz`~4DFatg zWbp;>Q7fF=lnUOaR8kd#O7mIq)!;h7XK4ALP}$quMoa-+lwXYgpeV2sUcuE; zU%u3g0os-^Ig2n3HXkXDz^wDoc|A z?R)B9cu1=gmIigk>DC|&6`pr|6vG%U(r}5L_Lw+V<=rkRur;euB=4eJYKSEp63wM1 zhi5t!6b0#*kKMGg@wE)YdCZ|#tcJe*g!_c*uDD6 z|LxfQ|1osN|HrXgh_N$=&VLw(?XG`|;Y$`>@jjTp-{7#n9L)G1!78#`j;!&ue_Co- z6IdMYUox@QwB_U-i|Ju$+cl@h0ZG-1A)GZO_@}m9h&FUh$RzzbA*CL9bEmroHz=hu zTOIDdG_vN5 zHu2$`eoR)F-L!4~QFLZp+NR$$0k=4cn}|^X1rO&sFwtSbi-6LPIgQV7K>Jin{k)=O zeyoO+45;UBVK(7#sOLJBpB!NRHkE%!JktQ9Fef-pdsr6-+z#;&a5t-*hEo9zs_J6( z90Tv@0=W3RD&Q_rcr&k<{_D^O6@974ZL;XwcQeEei+&E676>ZGU2^j_cQXQs6vj4O zyK)Rry^hqje3!rGbUi%T2kG@;W^>?dfEK_;CM5Da|I14bKF^o zOEE-A=dRA52XMns$}8ZoFZeC$$wOM=hSlvM`=eZSmi21emgTj4e!<9|#yi-UKO%ZMq1F^(vuIgKx>WH!0{P6N(-z+@>A@8%g}5z_3QT- zsyTZFl$FZEo;$g$S10l*)f$X<~M zH-uLRm>5$O>K8Z+J+6W`c?(%!ka;*L;bl1xts&-G6<_Vuu0kO2qWohn6eoG}elEQf&cBNej4CMl8&G0kw z^js*RAPZ8)VGS=rP%GdbV&3o)abGHidr%L?Y2Qo1wZ!q1pRMGd9yTUlfU%QNye>e= zqDS7wSVB@o|K~&Y(SQ~>WC3&QE>g&pkOOAKsf`P`#|R~Ndh*aEkb$R`eYsj7HXqaS zX5p4{+GQ@G@noBsr@Z7yUVa(X7k6J8Hln=DGQgA~?%@a+wr*9yiGMVXfGaUnxdGIOgK z575Lhg5C&gg!X7F_yeoh9!|M2wTAt>nSWpoyI^SnJ9s92069lSQ7HC!Ay+$yk=jJH zi%Vg>rSE_n)0=?nF^EFw?ddN3a|-QbuHLe<+?(9xt9Z!Gv2FY#t0D9BpsIxph*~7O zQVvW;EwFZ!+M`?F)B@k40IqFEN_NenB>}rAa>=4s-csz^yC%~2z+{8M1%Y@L|LQ^5 z&YqITKqJD60sISWccmCfT!)D3yZJAqdwdjYj4 z6ZK%F#SjuNh7~CsDIur07d=NNb}FTy9B4cXMTJ~B z_zanD777Z-R6Tc!&cjJk&;58t7dcCh^XOtz*MlUOz$BKEI&KAFHMk*Zda#csu9mf2 zz5^Borh~L!zOx*ALf9I^$`y^FQ7ote?(dN%guPo92EC{Vvdic{Vu5lKpdPs-P;SVp z6I)M-BUAfbPq^8>H@pb1oX`LMi&NTXDt97cf-F`h_;G%3Z^@Sx~epZrDQL zNzma7-6NcLj)WKibkfDmX(wsMm26LFp~aKtFUivaKd|6bn{WQbZkHDL40TTN1^?r1 zdAf%FEOnQM8?fC%TkQ=ku|=lFrW*I-A+)%7{t{c`y|4@Z4lDy3n{a@p)Hi>9abt%b zT3kZzdi2l|tcU~dZkA5&e$^u5`@#^4LSaxoYABk*w==4&3{+Y!HbBm`&Cytzy;DBv^0oAXy z04O-PxE%lZ+%lxM=%DggAMS?p9oPEAgejFLS>i<%4iN)4K>vx_FhU4 zlJ#)-IE)-V*mTKKJ>+qa&n_}LiW~R?=@K%AL-|VeQ9rgl5M4!v?qd9dn4XkrGLSM4dzaThpQ=gt4PW9-PKqH5 z&^Wd6iWB2!Vy4=N;oI&4fb$1$D7$;~6|CYJJ%}K|Nl>TJc=qldfk8`F3NPq!%&oTS zic9OXjT><1HMp~bp*a!w>ULSEZn%)lc?(n;wJ;;pw!s@Zwc#;gzF<5$0m@z0>rn;H zt>gDN|9O)Ci12{ecKsPrTklukpifAH%k2$n zLYv!=nAQo6Z=o#qAxh$ zJ!)yaw_<6@+J9Fz8%?My2BGqXmezQ8FRfuWMEmdIMsXoA+nmBwfk%WC?v0&yyp?Jk z?Z&x}ceK{iKnr|RILbGXzZ|50=}7}8TJuKg75%!lv>&rjH*)c(VUypZ{c0C|Tyex- z*B0Rf_r$kJPR~cVa8q&c{9bnykHXL56}r)js~8&C!zh~8!&oB$i@AsJ09!{8J+x6! zAihQ~o_)cX5Ah*aT)qTwCw3|b-XjGXt#qP{H*|F=cp_#AnwG>tp_)naSHSR%LGEFe z>htdvHVPY}8rndVtK}J-IAh^cJ$lRaw18OtqIDdR8E5EEU3kS67hH1PC0FTXUuchb z9r{h5`x!_b`LfpfplJ70JycUXe_Z3AaPq|;YWWuP2FzUJRXFVeTXlAEV;C7=B&Fi{ zLKiyq<#%B&?J__B<_2q0H@W3 z8KuOKDXq`?pNh{yW;BB4R=_kKCZ4W=X>5XNd}g>z>$6(xj}QeYzo-0*O3ity)5~Ln zkRu0}WCJvAsnb83W|Q3SgikBKf206T8Luin; z@sFs3FFf848RS94&wGagLfkc7X=&m%TynjV_^TM1q3y!mLsbLPFT&6l3soKnb;fpV4GWb%9Dp=UCE-$ z2tXVM*`|^1h#x4D2uB>DG1Hy@0XxyBiJf!=JXt=38GSLP`%|140I_Z%YR+kxYPzA{ zTA&Cv!53c8hVxe*rZ!XSvC&>?aObg{0uSeccbKcGCy99&o@%@_i$FM%&3SH}8|Q*{ ze~)N;HCD5Xh0_hAIe?cBx_m+8s0=OrKG^fkzcjs9U-c{w-avI85^VBTVCV9Ui$Gz; zi`vF_absXaR-x7ByqE7qJJN{;cJ3^A^d(P!p`D8**BK!0@;$=k7zcRH^Nq*dP=@5& zxGKgN&FRx_UJm)eUEw=tV5_**gQpv(#VZ7;RvRL^qzYg=b28FeqnQ7()&u?Ab}04H zuxM=$y2-;c;2a9Rz%6Mf7jKV_k^vCR>QYs4BaNhg6-TrNk^l#*1m{YxC~;Qv)XiQF zN6M&sE1uCdb+4lN|9W}Y{VDp7dbj02|1wX*e4H%0;UzVZ{~Vco!319Oic?MHIUzLB zDVmUDG=Fh%pv#Vu$`gfD8|H0r7?Ev01ZIc#rL{ zjD{AV=6ORC$9m_VQ3%#u%G}`#X%-8IyrFetVHQ7mk@USMBnGZxZ+sEk;VPNkOG$^{ z?TKH26yYc>Df^)8O}z7n*g6icC5xI-2c?S>5M!DyEGv9L13?r4;8Z8BbmN&J*76N$ z#R8Zhdc#Z*3oNnG;unjwae0Y!j=MlwNeN!BVGEtYi+IT0fiiv^62aDI`Y4RRc~x!G z!56Xl4-LKN!#s?Dk4lLXlSSYCQ;oo<7c#?tD7q}37)gfO%04wag(YFK=sxC7>r}Dm zl-Z|yQb9uMo8}ao37xpb8WsP~LI{*g#8jti&}nZH(M1`@1jjMW=ZR1$vZ2ooZ}Mr* z%aT>nMEQbs#bgXQg;k$JaETMR@V|wTL4D)}6+cQ&P>#SOJ{!FNc*2a<&g3ZPy_#j= zNhQl#7L=sM*F5C^;PdD+4Vn-K)2iJ?6cvjG(~Vl%dVF{}o{CWz<8poT*YWJbiaML^ zlyIIwBfu3joZueJo=L~rGCe%0$2Whzd?ju6*Ccc?Hq;rbLnH~oW64v&ADrX75figg zY6S8;6d#&2Y0BpNun`CU+NHI;0w$q3skh5Q=ax8@LwTXMIBofmo9kHrymZ~&rEPM8 z$mQV$U!gkc?V&<1@Jisb?0svr(HXZRhmwEJMZVm7Q*nWIDvfvv7-7$3Q6tKDjnH{G z*?0*>*M}WN|J+oAQcPdD0PcG5V{gyLl+wRJ+{7;8PgCtZg$zP6vB-wXaZHCY+ZoHe zJbt1=chuQr(crTJHQc3OjAW2tiwy5IKwOeV|70HUM^-48`y$wb1~5I4q0aaazOR5h z4A#wGJPFI-Z}BqCY#~Bi3rRC{dTAqenb(xCG|f=5==aZw1*H*q5&6yW$}lA+S$H+` zK$eEVI8IKorQu<{dA9Jy2jX=g?7$_(oF+>ckc2gR;@`rPVpK2n82~y1lq&vszZLfm zb>C&;kkn3M;XDLPH>>?ita*W9i5SC0l#TKRzYGThbyDSE|B=9%q8zmxCCai-E0;M zR^fKE)?%oI5I~niaR1uN5Zr%cS>8>buR{EAWDnkZn8!TIjl$vInRxW}d|;O^7@nCv zB;W|Gi#K>OoB&!pJRHEfz{g(cT8)2~JNd65O`m_Ka6q)cE)?PnncRs(-m6y|X$|1K zx*sQiSL9VzU{LQ+7UaA1*4)`4Rrh#Vy6$U$=)$W~Qt6Xra3tk5!z8-Hlub79parR!k zXf2MQhYS0b<(minsnm?|CI_hZ{o+j9b2G4ZdJEqKQERGs_*_fuVS;-;2e@l#lZbS6 zf)uLuR+5M*gFZ%l;8Pk|xwZ~@CRK_^Xv9u>%pm{#bKz45{tXPT@?b{-5jlA%f|Jq& z7s3IjF%6YaNch^})YxSB<#1T>thLTUYUt!*RBdbgjc|AU&;29My%{1}Je`LLjHN*tQXsui?_`=t~_ci=u==^GMem~vG zN%wPFIQYKL$Pseu|3w6H3s`0)_}w#;*dNN$a+m@o zmO|j=>BJI1!c^(F$x#7eDnkeBR12Z_BIpt-MSPrAP_&2S_+Rn060?6|MBF{*oDOof+is|AP zfHE84qKZa^yUs6qpTHe|&ED|2ZaXJ%#;Hz$8@@o=k=(VTU`_B*j%Sdoy5eIkz#?pv zm4GJ0=XUY6IP9e2Iazf3_5z?+14RIPxPC1VU^!%SqJd=5Rmeu4M%cHKD!Aj=EMN`F zuyL;!j(r_6n`Tc~I6g#0Hkk;EH|YQtZ(vEv;w1+J1BP8Yvsr{yum?ZfWJZgIn#EyM z|1X=o%GzYgmo!<)lu`G0s8t2-k)m2YWs~9mYpvl-!(JQzjV(I0^wKC%CXqJG+CX`?)Y0`W^sd#a5npDtFVwK+DOWu~fIoi#?uOTO3 z&iOMCfbY>Zy$OuNC&gs5h?`+G`cIVO`fmz!OdM+Y=aamFdqT1pX8rn9+WHMiu@wSM z@Xu#^6z8%iEvp_v_!y_fH{7>?j~!h6OZiM*MFeGQ~k(#ecNG!G$dLkT?uST`VTK zh{!gFDbO(9o?k&!fGb! z=;4@sbWA0=WByHYEAAAH;P7xa8^NzxuRvV38FHfqR*yl+Zttyb{%7YFA zO$QY(2+u7S+K47%<3GV@<0LfFDD4~&R)pQD3bj-LD*aj9KpAjqk~fHBol)WXauKX( zg_BQt6$q~)ruvW}y2+i5&Ek%OO6g$}f<_2iJ)eO$;L0)Mx|9pMPF$s_;9YFnfGoBZ z52u93Tu1)12qK)lPT@bYVp!L_UbWCfMUX$9_K1v87Wr6NIqAlEA$VR3ya;sU`f{Ie8>AL9 zPmYiYyb`=yyZKxnAoRNv(;b%|qUI3Zr-Jjf2%@!2#K4h3OxU2*iKevx53lLryf$-# zNDnXbuph-F4Ev6vA1)cpMi3a$AuMGL1hQ)qN4JZPacEv3tlg<uG3`zWt%zxN@JrxYpcz?D37tFBacamvlYue0*_X1C8ee$X^u0GD zC^q476{OnH4dtRST!#!eWDtK7)wO4a;agV0*bXyU2oeXti0{BCi`fej-$`+WCXik8 zE8*ssfKag%pc+9+;A!JV1R$sk^RGwPI}drNE?q*_vAUo{h0ZQ94^NnCjgu3wjDid^ z(?o}XlM|!0)*jX>k4=n*ofXF>zIufHi(@UYEBz;QK<$sbP!1>w9a!84nl6Qj(^Shk>nL#*Mh8` z>iItrg5HQAb~1L&yQn<06!)K}0@|g9;FgGJ?;;h6Dv%*!td)qdUX8ui#B5qu4i7e^ zOB<(xc_riq#XVVcCOS*iTY{WIi=>H$fRS5_^wxMQKB1;js1OXFBsN`Xt3J~nU#yFpMuMEU{QmsOaVRJr1 zrFchP$?#uz+lK}b5w!eQ6WySk4h=ChS z1g*)UcOH;%sgT4J_tN9!k24F#N3@F887$gso)eS(sJs;XDw^OAP zC&5MAo#)0aHpg+|oFgNmmJsPGx8Wq{VonB8`T1LT7+djJ4Sz9k9O{DX!j$4w^Wk*> zEDYBPdS1l>yv)(txHs`~hKoifiP>BDh1d%|tBRb#$*ap>!bePS>|H4AXz1rUH&phL ze^6zyJ%qVv1uQfoGHxjDa=`IQJh!tx(1@wPUnL?!GP>1xDXA5P-_#K-Am}f}zRk*7sgS;AdC$U5GRL6IG&t7 z4&WB!^(jLw8^@jNfX_4T3b|` zJv`=6e6tB>Uh=eCqqx|$+wo@PEB;bfZ{yLau!*c_NK~|u6+vFa$1$9O;5GLb7{c0Y zMR&+f;UUw#)Wdlm{Qpc0^&~r4`fGu*-VSe=)?j33V+)!0=+X2On7) zmU8kBwGN{FY?~>mqtj6SjNO#sos=yk4$A2Q!#8?3hw)<^6<=yCWRc(?}@Uuetm zb&lP|-ej+T5#D2!4M$HTzt(aOylQN!k#~=kjmYcKgO!h>LR+P_?DLA(*FCHUhZ0>v zD|diw>0LqATHatMLQU|>6T&O6I;Qm0(8b5l6+VWpu-INFz5d0ixk>uN1s*K@!bh8% z#D^d}FV>j7_i>~#DLu)Cg4n4>&@wSW$e5TIakwDLRYXm7j)5R6f^o3;n4ZK@gUeAG ziwReC$iU19>nnqu{*zO!V`juVH=j+T;qDp_RkPmu%|eL%JF@t zMB(aod6cW{Q|uw*4K>6+ak^n2{wR`gb=XbOUy6+MLLq8V@qn|@ppv3T#SqfUE1vss zr7i0h*N5DC0iW@?3-K(sOoqj2y%;;WxLzJM=E}nd=8o8r-BmyA4F87he=z^ zs2S`=CzM2dd=Q6w?m;5?(+q<}&nrG3apE)r)nb%t9yD+kORh21yhGa|ty}BSaMh$? z`oA=4pTPGvKlFrXXo@4Cr4u@qK6{qpnX@O(<#f@Kq)nfkAY#w`Am$(%ZY0~{%W;!Y zp`1S5PXmW-WEQ*Ll;<6?0qye+((4HB{I#wsDX zI`_r!feTc}z>4(e@J=QM4*HQeak$5byNW<{6j|vgjlgQc7rY!FCEAH2*t?v*3dRXk z)l?&!|7?;_%gAGG2;Atv($2pb zXFi?2;D7a=_w(RCB#Zjc;rJR(2?%Sj4`OoV(BcNGW^pqM8`ESymtPzf5>-!03}1tD z$@}c^rhVF?A#v<{Z7$l!+ri!RGw_35gJs$3p*$1)1bW z0jMQ0WKj+bOJViWXp3pcnDBk+z4Gz0Opdqre3=h`wD zc4=-mz}(iO3i)k=e~E7!)UweRX~~DRjveB=cB(_HJ6Xg(lA2n-F@n;849TKH6kMt@ zQ<;-Rxj+MN;)T##|BM$xTNmyYf+nqDj(-yDOzSLQq9N3dEMxE(FCqrUcd<8Mo>+dj z>~ti?TM680K>t6(w!+R`tcFa)iA6B+=lC11i2-vV%s5RQD&uTl&nr}+feqqkYACgH z&4{u+@$;Dht+Hd7mkW@Pe=Xv!_?+S(!jnUYm%7&Y_@dgN9-4yn0ERYvZL;Mfykkix z499F!V;mX$YWksZs5zAJbG_xg@AYuri>tMX(Xu@hgUEufmo98aDqcd3inYaXUE=6G ztX?nbBROnFpY@{NvI`&Z`!V<0b3u!%J<|2{IYt8piEcl;C=8FF!aSIQBSr9)AskJt z0d}{a$#|wr@_9f!E6jRyO1XKgH<@+QDSfzabPuWby zDuuiitCSL~Qb?{`rQj)whGj$Y#?;#qD4;F!@}TQbp{NJa$FY%_LoVa~-%}G0sD%+E zJ(v^!QcUR{SS5r9L3FIefj{v>&T&~KY&PR(s5Vq=Gjanv7>lhf>T6=-<3C+`YBN1_gy zciXPLWKAR6WZNPd{AU%yFDG%xR*!7_V>~_o$w#$EpjyljX#aJ#&sxh)Kt|zzqqSx) z;^Bc(K;Fw=tkrYX-U0@>U33Z?Rp=4xb?^$Bow)Lss887{u6xlcwivM5R5|fx4PN6G z{Bz0ohA|EDBo8FRvWR4jTq}t>utmX^VHSP`HfVupA>?3N{)PB^(A%5QY_e#q*^%z! z>4oejQ5-PRMMC>n99TcD{84obQTM7vIH1s@_ry=ac5EL64DSHrFrE-QbvOa!58xS* zMz{=EZ_E2)ZJR7m*){?wPpni1m*!}X1l+ITel;Gz?11|XCbTUT5lmLg0Hz?t04;d7 zg%*{<)uUoKFxFb&A%G%w{yc7Ya$gsU&LIrlpbJM}6gg{p0lC5pcm*BDev5~ez1W8@ zTFbFMiE_jD`#<}g8tNtUSSyy!G&1B(FR$PYr$7F<*Mkiy9QDEXu@I8Oi)%{|mP5`? zh*Q~{iE7i4h7ZibVnE3{+^F}Sv1jI+Ew;|8W^?D89tG=%Gw;Gy8r-aPZUbKgJ`$zq zU%Q0!wJhW>$6?=#%dwFAz+b7g6(2`_A>8;jhWk2%jzHn~H*P?uEFt z)(P-@#NGg)*!er*XgsSbpV;PwwV|g z^zznDHDHyXROsdrNWd6)5%#}?S{_CAE=1U6Q;Zqwi7PpPOyH0A92bYWHwX4f0$q8a ztL)839rR@>>Ks&A$*d>QXL!2!P#rSAnfdywX-j6;h41Ne&M4?_R+-@{fEQ z){LdDVbz$Ht{OA&-N2M?96_~Gx`BXZO5Lz3NU2-7&>7GG(FeWPa;?wyq|U77Xr(Cx zLeXy?%YnovU{fqdDBm=Jb09+aBP`I1x9?PJo-6XJR~x7nXh^698^jME-_7Mq2H1xH z$DkL9`5MDe7U|z|l0By=r*F%d_ME9k@f3zQoJE5!enMi-%S!TW*NqqF03U$X!)inF z5|i5LR*(l`Wlt#-QwOePd)hEF6HtapEcrza$S+ItgcbSDgT-{pwbqk_oFZDy$Jnh= z)+W0YpMfMnI1wiEyY0ur`x7FhIx9p7_bCxlcxp4BJb=H6W3}Xy=FFiRK_bybv}Qw~ z3#Vl$g{DW3$)9q;MD(-=7wwVT+|Tg3?IHJWTp1c<3CU63VCA;72FBA<89cQT7WJ$p zm2GR6dV_{;N&^H!bWLhi12-qM0R9q*kOl)?O(tOrhlmLpteZN+s zkcYh7lJ|zkyhNs#lDFV-C$8+*R$ifGUAXc`tP2cDoRw*Ra9VJ8G1)8%q|JU&6cWag zs=5E<$9g%IY!^rLP%}1tkJuJwsj%>_rq0oI#q{y=D-{U*go1drC9i#5ZKkdAtuK4x4?;G;ZAKZbo^$ zY3igMWlL~iFl5@cC3v5+C!U8dT#cHHJ@IX0#X4j2oGjlKAI{<75Numgke&m| zY#Yu|D|3uU&%sSgGY8CEC%TNU_Fd_y3dYfCba5OkXfrZZwtT1|rEw&_w%rqh(jdxI z+42#%&^RiE6azq}C8L)Fdn!@B5>?~*%rQ%XuT=IV@ccfXFDa}Fz9*7DK>77!gF~pX zfh-(Hs%{{294dM=2<9i3i64ACD zeNZLR#(CNhvE!YScg4J$;6Z-u&mw%gZSHtJaBo%&9ZvJL%Y-o#+ct|Y6^Sv%zMSBG z-ofAnSAS?_!NKT#?N-#@rjO<=N4ZTOv&&OBn~xmeHhnB_ zIl65+SctFsfW|f*(uX+{Y--cT?edHl`8kqpI>?r%SK9Q6yd^K&^dgbp05{t7gLZii zo?X?}P+Zkke+X%FwdsfQTHK}|#(d;pn?8xROAsbK>b`Tc9oF-Tqf2nDL_Jq>}j-xgmV3NRrfCl{K z2K;R-gpE)xxblC&<@mmn^k2ksQW}_X1?KXe>qgpPt@Y=8LkSBrKE>|B;w9b$WOxAr zy+cZ(0)42WuY-;g;E(sT$tK-2xcsH)rZtz-8w01BvobGzegnJnFo#7^0uIqmT-bp@ zz`>QT2AAWbuX3&(?tr0y}zQ=((>(Yrl6q6fv6y_!G9x~n9l_# zXKFcRh?{Y~arkso-ZSDC z*v(ZGUZBTI2PBqHu`&=F{{%>S_(zBbQWHmP@r=rG8(cdeoX15XDCY|y9855dZ}_3r z7>LE}n!rUYrMaf&K0+sYh$9X3b?b18Pxm4}7ed8sVJlO^t@Rw2N5a?giD%uyV-P(+ zw(!wjefLhz`v!a(#$_)@d<(O115QoU;M=dSVl$xtN1#wnTioaU<}JD%Y$sVX=T=yn zj@GD8TQY#Zu+jPfb_u(a$ca~`v2uxl+4w67&HHlvf7Z73;Jv~k#GLs{h3@&45|<(z zXH~=j)+Wdjy7hJw+KBV5-pm4KjyDWttOBs!T?B22WDmO4KsW zYzxC9-=D%6Y39e5+&;wx3-TwyviNo5pe>)m?4S#xQI2c~hM&FQ$JTEF9$c0iw9^IY zz=LRuy@B(y2;ULGi+0;kp{4&)VZnyG*}YW%VR_J29ou5%7p7NUKU_IpnUNF%}E1T%;)_@)+q%Wk16Ox}rZ z#n(AO+P3^(f)Cp{!pWiq-~tzJ_Q3sH9vho8-S7pw8bp1~O}KzS&;^YqvxG)>ikoqy zR!7(ny+%F4MB&O`H;UQfd$lww!{_Imf*mto*!>JO!JG4muOfjHLcGWE5jH|&Lb*1) z@G2Z$*l8!>{dgP&NKwN@D*Ha~Pn3P&q$LI22%u&Z!U5?3KhnionEDtFvEsXr?^kN=QEl7Aik2OH zX2F(kN3NZ?BoJ-X@F}e1ZaqhqSo3)NzxTn!2l$~W3>3uhj?t(?4HQNXqz!+ijixX5 z+M|$z$8{ueJ>H0d%6RDEG3V&v1--(s;M`sHtZ$Zu^TaNS9-ipI0vLbP>=y`nTiCfD zH#w<5!ziQZcN8sUauW_9*@+N^-t$qup7R+sSIlBXczNb8 z3v89iXRzMPU$m=go^`5O6!gu@Srj~DUf%NHx_ORatwqeGYMyg-RZvKK{80qSFhJvU zBRn}5RcbAl;KhS71dp9>$xUFpx-#~{YWCGZ-J-7#4xWUZIuYgzVYJ3r# zrVGMf{GxlI1))4Qoz@206ucq9)dB8Y9LVPXhqW((kNJ52&&*~eNNkXmxPyea6Ql$o zAz?$}tfQ__q?Dp0N*x=LU>jSvQTI*1jykIAj1Us4j=Hb9kI!27t;qi0&wO?xQQzbB zd;PuI?C0~$%rnnC^UO2PJTsqXkcoUd#;&wu+MD2z?nwUu@-P=21`NWvRR{s?4unI^ zCs1EvVv+xn=pF+cDNP43y_7_Sm^@(*u^LiV^`Xy?D6({JP-$!3VpQ8gIflW+V&J7@ ztF?m>cFg}m=f`A{SW(AGv4UQg0nd8_AgvYrDpwH2(t3tkkVgV{H6UX1h+z>ahaC|w z0;k2woDk=|SgXdqQ%SPi@{Ut-3?d?p9Hi1Yk3@2)4WnO&{*?+@^eW2yjJwm|y>9e> z#UW`dI~`Q0l9oFY1geT22U}x8yzmzSh=sQlDVm$5B;)Akd5Vxgr$A`3F_=}Gl4g1Q zPo9405Sp~qiGxHcPrn|-l|aNa3>tSz0v0d)I%9v4CiatMX?9W{K$Ls{8!|Aeffl1` z6P{wyc2Ji{ujpUsDf);*J+~M7n8Xf;Z*Z%aE;X?dEx|0ChXGBL&hm?MLZHwh?z)M8 zB1j5kT>3?kgT$Bo8u1aB4Mr4|l7Ls#=VJW<)j{_d5#uxgkH`TvBm-}dydc-dYKWSk zSTVhkP34Oh(6N>3k1!8m(qB;*hz;hgq{yR8&kg2XNqB1Eh-c8+?2TwJHBXpGE1aQG zWH6YL2gnE&ctHb`hao#9U;rl56k{-t@ryJ)h2;qCj6zaJ5V!E)WMENT$Axf|s5S|t zpw21yfi7~?ff1Zc3Gt?Nl2!IBISGLw?=-`|jWj)JBKM|d%|jmY9Z^V{W{SZyk9s2> z6CNQ=YxdXyF^^~QrtJ>kYXC(bZ>EQfc?spm7L_L6WRc0q`~7enKXD0G!5VZMOa$wX z_~YyXV(vdinB?&H+o(niFdzhnR}tB<;4p%Ps550_$4vPsrs^}@BwG5{cT5GX%pTw1 z757A0X+|)ZUH_t2g9*w_P6fD^QQLe9#>xrFe^FD9B_*A5)kd4^)kdHr1L<7iynMvzgAFC)2DSRrPs< z{Xj(WElom0utMg6b13-a2SZiVD#1z@jt)JYcq5wOzO%RD^Ac>VF^@g~*Z+$;Kux)J zzGsr-XLkHpM<57h&AB`Y6eALriAJ|IRr)Eka zF#V*CoC$IiAVQe^SmKR*8uy|M0 zjBQ1x6m3Nk#xZ_X2_Pq_ROKk^1v#c~4M@|G)6|dT8k%n@{nE!vYs(_Rv56-tGI5O39ELM!Sqlgq zh)!(-L^{h9Dve$F!Y~YSQpgkv2i)8qI%&F0GS`cyRUkDRJN}PJhtdp8szg^bc6U@{$5DKjbMe?PX>3{RYLvmywO`WyihQYJKVj&dW7UARX z3m8tN$h8MlU$$XFmXWvYF}E*{Uk%4s@zY3PNfYF`4*8xta(nMDx5sjOGFMHX0!$mOD=>mfguYE zmvJ+rBW7x{xfsot?A65}Q#sJOaCm}6PA z5w}1~_1@Yv`xQwd%(ZD!szWn%$-kgNc%37wKC#nHr3tlRPJjpdDI+%e(i%UFG3_P6 zjWoN~#oA%As(!Kzp-&R88`!hhsxPo^)(rnTI&JDz;$e(Q8eH+FpC~7w#;b(O0XBEU zn3~~8xk`-AP2hPFrsqx2I1;d;bS0PATv`K7fF^)lxCt2@HxS83CkU=CDP^VzFC*n! zO8HDO^aPE4M4qO=CDbmAS@nVR0}VV}!aM2)g<@3K{&+34TSFbZE)R!womFZ`$m%(ufNIwFM75ib0b?gN6ELxv5dmELUs z41kp*tpI~9O!*K=LT4}?p&MkoT2DeS7y#279yk>TIBl-CIrRe+LLm=?=io33x8z2<4p_oMG3` z9VbqdWBQ=&7+l0b>QiO@c$%Ye@)P5cx<8zLp=11T3^aba4D~GZ39!`JV?TMO3?-`{WY2GF4Po#u9_p|1aqO;VE#^ zF4g8I@n=T?e>#N}%VX6oMYEMyFpbO%n$XDhA=~DISs7#gl2P1@)aB>|+^c4_&Vk36 zc8XKUU!6|91X-@|85p{?NpQ>4{*yx``nEzYNpX`VTwNmwpFSupkoS|#GQfHz{>)?Oa;`|N9*YzK{ zikt!=<&4Bt2OvC@sX+-!ZQRj7!VcWxcV%mShBd4-|&j7hw_Q}pS5S>l` zk92$Mkl`(9G7(V^#bDHWthInEt@JodjZ6DBx~v_%trkU%hdOGz}SF z1ISUjNxme|c|F+cvJ_6mLz7VAWBk%L)1(xrgs*iE^#?_@&ea4M|8U`a%7^;#xX9nP ze^_6e|6AV@75~0#XD*ywBUFPw9gQ!rtK7|Di4D{{eZi|3lT>Z9;@U{Xd}o81pBGnQ;o? zHv&JYVTdA5CeSG~Rmwa28Pdk%V#;$PT9VCmjien6gS5ARYrHMT3e*)}YSF!vxChiF z`lD+pPm+jXFUJHNbS>rIA6-jHU$;5+9s?bi!5ot*4ol-Rxq#vL2HkWUew_q5J#E)} zOTa^G$dBNMIV`o7*2Pu}4`0$l03HJAp$r~M&_i)N)WieKYnun2W9V730b|?Ckmli! zuj_Cgfi@;&(=pz$uS~9uBqII8=s}|gDANpJRcGvg%SHP5e2etUd727#h0C?Gq3XH3 zYxE4t^U)gJNMAM?Ol@e9uKTx(bh?7xVa8o*3?k0tsO$8^?t5XmKz)V8#7Zp|fFET* zOJv!CEYaymq<=})nGk)%>9zhKhX7U{fLh1BwHu+2aQcEQg<+JWz8I{1@sjI;NV$vL zc^AQn6`V3F&A>e5g%Nr6QMc1LQyQ%Rfe;>5F{=)z59iK}@;;D4Q~;`06)<&KM$Rds zl6pffzdqLZr=}2Mr^a_6Ml4zPglhTpePWxvs%aNeHAXUt<41{tl?r(?5}g^SM%fk* zj!Yy^AUX9R&C+_#)3kn#(6nCCRp=NBI0+&tgRV~k3QQ6z>4?Q9jcTZTng_1YJYd3< zMDqgX0RsU~;3aLWe|BP$U`T;S%V0VRu(3H7GXm7(xjQ|l;aQH0bbmZ4^@EYU6JDPE=8<(vZ@+N@~KgBGA;(&j4sC0eXEi z0DOtI&s4x}BFhvgDq@^w1Wlw!}c~ zbXS#>>7nvVZR9(Od}^vYWK&r{upQ2_R`~@c^h7wAHyyaGLnt(h4AYOeZIX9Hg;_t9 z5<7rvDy|ITiMz9Yr3GQ_2dv5;WZ-qf6WlR}Ko7}V28z+}Bfki+sS8;=qe<{i$Q$I~ zgP~03kd+2}nlDE$-j$}rO;1M);3Yu&C^qK#S^&SK7qfn0G&qt?`EIL#(gdhXJUb15 zgIqK{m)tALJSvZnCb>&$B+E*BQ)6={uV(cz5I{ez`^hChsg4yP`K;{vF9uOhIWiz4 z=@$&VQa+fF%$~)Y!X0YX{t=E$a1l{rI`+09BP9K@E#|Pw;E{5ey1xq1gJq>Jl$U-0 z3R$lpvr2y;(i|O&;|$8hn#lheL_53MDayXcr|hQGiArP3&4*) z2BtxmBnOImQw7TOM;Vgyd8iN0die|QR*ca>mhytSskp2lY;zd+w32$2tANKX0GLI9 zCm>5UAl(m`l)W_o`-{e6Vnyh=Q%m|XpBI>=~IA_84EfUu@DhG!$k zC_j$CyOKx*%tH3u66jq_-a>QqQeG-#-hIc!EZ5H*`* z>?xK0pn}E?mPmG}NOt}MNd}dnKmhqU0y5f3{^u1JqW)W4&}5y4{Eo(5QQ`fYZmS<2ABQsu%pmI@if(7Un$rkYcDtRv;2 zVQoQhAh+G*%11{0C#CQj9;h#J=NJV5%rUpWP^%`6D+;v(XF7$J`Ls~DwZRN2SCT-Sm@{2a?xI6%qtx*{3jLzQ2LkYw3ZQ9C=|q6eblC~I zjArn9V1z9Sfb<~XvsI=iKf-L*ep8J4A3fQXq{poN9{K6DH@$)r5)2cS-jzKNAYEZv zU^(sGBi%_y17h?3c-^(VAB|yn@SIJQ9^m>@eWQtm zX@p~ah%W}K5(%oC^lP94#K(~}zdLU$i}+0z_jO71P>cIoyAbw$wWYKhLJLQ`U9a>dMT-^WRXkYxgVnM{>ejAuqd&xj z4y0=+#p6`cM|mmtrThP+?%{c>XA2ag^nLiMY5x= z-ZbZahq|oO3I9>dijI`&l=9alFnc7WY(QawNeT*fg%#?vZ)e@j~ zrxjG$l~b6=9Bt14gOm|t(Y!gxmKWHIfMg0ND?enHLoor%5NoU(^uO|hnq&Q+ zu6%!sMG%@De93x+_=2QpyPGxqL)S}83%&#zmpkonXl!<3GcAzHZ%HGy2#r4miH z+_58prl3H5NJj5kTcWg;HkF8Ge32fEgLBpq=tp|D-b6bc6w1R8N_FBz9MI*q9z~TX zw4QEA%9~$Kp$j_$TP@1c0i;NjH>ko}Xrz40anzj4$ZEY&!8eX-X-p-0(jkvSv=3@s z4C3SnGef!^U{FsaFLofZuEc|bWIu?p@;xzi5^1Fq&`ORJNnCGAiKCR?Xu{j%plPcb z-4Tk}h>x~w(uhmxe#X1t>W?E~-_l9d8P@p?SrTXZX&(JD8$ zQb{t0sZcIU|3>fR&ZqgBhZslpUXV_Ro}XR+oF2)m0jlU9Em1P)ArWyk-&t)^Z=^W9E%*g$f**7+%lZge zi2&!yMwhZg7Qtbt0mw^TK^vRav4G|!^h_KGO$1P}VP3=aW0C=2v~sB@QtpsiAhtNk z(WUe!PSFTLjn)oMwoy7txg>-;N0%*#3L0fenSxv2)NyCmyQ8(Xj+9Pzs0`*ts~Eff z5n0=gC4*daSXVh}aMMXl?6Yru0sZ;Hi##5`N~q&MsftejY2oAQT0ir>vH6BUyJ6Qf6u$~0j zR&43xMq2!_m8Z@q&FffLaP1u#+MJY}4q}0JPDLtcp872A4 zBxzYmwyz}Nrg>laJAnEsn|wYQB<#tVe))s@zLV>v2cWKfc|xn?l5BGI@D(+RYGkfxdJzpw0!_BkwB0~)6fpk zM#)P-v$EdtA}#7Aju#ga`I+>B8hZnaUB8=5XDUQETV6y%@DijY`(r0E%GIpmg|;e% z=}!Ur!|2aDe}|CGaS(>iN&*tciZL8S*!6LQoq7@Qd`rm|lmOPbBin~k`#26i;& zi|j~|y#GUV$2Vll=a24q2OtugwIE3@ndukyE6$X>q)!OZ%Q8}3uhueBU5#u6smrGL zyV*EmhOfSiHdk~h;wL>|!z<_OLo-z-Lk+2YI1cZ&QxtEBBGrvUJ6B6l?oaRq)OzBr z(^_pS>B<)(&8d4}y&*FW9#Lkav=1vHcQC?CJMxE93osAj>qPL4n)e#NIPB`>4EW?n zAE^y57MuT{u4fOX&rga%Il33F?vFmMJ)NF`Hb)BFz_uBPm z0EL-58g$VhL0FcAlKAd#psWc1mxNwK`zAaZZLOV;(%B49togftgrEZN!6BPuIq~O_ zY`_rIqq2Oh#Hub-x)MzoU2;fv#QqIcU;x*sf_C`D#4K@@!54@|$9XM?lCl=YQgrfx zu9P_9ArTuc*{-7HK#OQ#f}tptV@Id8=lH zXKdPdoI=NH>Q|)G2gFAAgkE%m8;&i~#+f08ZcnpkrnxfWS|N(dZaXh;FG5KKllF}@ z4|TT{pxez4A|}^;HoSxy&Ji3L2iErlUfWF?C1l4W&Q?dp6iHcOLmH0X*!7{v6O-mI zjRqrG$xg`v6YcskL>Y}HhU0rmGW{Y@K7x5;VwbZ@qOd9s8@~a++kIIU+tyry6ZO^*W_nrQMF+kuK zlNPoZ#~owR0$O$yq7`dBuuey6fIYDqY$BX~NN!LNLCeiy-M-Sgq!UMjPtq~GdMJck zx)gHKO6L{1fnaar)Han;5p2wZpk#go&pL|MtyxLWO^)0~yZ)OF;@j4&c+cbU`KtO^ zB}kV-KG>I${RqVybZD240 zq!T*x=YU|Pf`hrZRxp1BEpsbqOclf`kBZ0|0{>CNYOj1XNPMW_RaYm`H5H z>Ue4)bIRi#*uT^cX)k_lgYR7@!3`a~*RK9UKgXjfFhuffIC_H$Tz*a@q-nF;LmpFK z>r$`b9lU&tcbqSMg&&uMPBEb&FY%;HISzS4?w_2wZy+}fz(WAg0IY=Zw;RuUMaSB# zSX3<9D`#!HemEX&uW|fz4w6Y~V@x4)@d#{9@gwWba9|q(fOBK@%l7KV>a#Yuef~VQ zRuyETtyOHS(zl3pkymL$zQ;7}U9^!9rTC%QN+`B~bia9)ph}X%is8ovM|DF(isDI^ zat~-|`?NT+TFX)Xg2+#hqIk*DUI8v<)#4d6pKRA8m*`VSplsO4!)^c-{1dek1paXp zTrCU2mWNvIBF_M52ecIX+OfdWonh?Lp6Zao@I162$MJSh0f9WzRA#7KoH9U3HD?4EvY#m4rmar2AV-nH%_U%n|5mqDoOR(R<4 z%dOz6%9bEC|4wv8KZD&P;1Jr_FUvLH1b(R*!A)c{+O2U{+qoCZ4W$*@1i`}$r?hn- z=<-PfN|GE+^g*-NcSQ`LXTO6N=A1nsI!3JO!2Gx+Mr3zj&G@s{qHsr6k++QzwL7xF znu#D!nr{7C(Zg#)@&R|JfP4rFfH|z4^21J%(vd}dHi-I#780eXX%u@;T`}Nb3?fgf zLAR~~n>5I9Jelqt&5joKjx3uOXeADP&H^;G3){ux&siDGCOqgmu{5uWg-Hs>T{EO5 zDvBP^e|2JYD*Iti(_rohy@L<^V7B<%x?_AbL&dZht|kq}`v$=~vj)6bv}oL!wbdN1 zRA5$cr*fVl#%!`BYfCYw z3ma6p9#W-waI=Ld+?AE&Yg>rQU0Ex>K3fd$%8r$CEe+Y2>W9b*qv!e9+x54RO1fW^{q&!VVeBj)h<%Iv-9Q7MLLL`L@5;4 zA(x&=q87=ti0T8`m%Pd@%d~;4xu(dA zN?_iEH(sbiH*u@j9LGvnUJho(8BechDLjPDWSR+2T*cC%Y-r_!TI{AI5nJm%%>kL8 z;#ky05i*Q57)UTK9ci3Rba*F{7U7`#jo zs+C24+ zI6Q$VWe44n6`}@bg=C50{Ia}bO#m8J5TC&cAc z)~Dj$r|6nZ6}H8gy34xgs-u1xSqPT#PA8 z*?oy1VPSwM?XHUPli6RICl9owIz2+JK0rxQos%wzDO1>3{#b7*U}E)Md->cWoL7|Z zaA*%T`#D|*sIOJ!D`tW#N@o>fu~LoruK0@bs|wj8A$H}=d5K{oUJY2mA;YXQVu6`e zDA*-~*rK26Z8>OWUYtk&Db7uUNd9%(DUz*ES~SyH7%MW_?&=aT;k7y*e-p-;%-0e> zgB50)&N;4@gqf@~n{{=Xu6Q8~o8&S0u7SK#es+C{LIq~A>E0YaKGQeE6of8J8_^Co z$>W^zE2U~2Ci&zs2A2dMykbuD{cs-59#9Fn^TRSfOqKD|Mba!xxVPNJ@>y&nujej0 z%x1y9)}FMWg^L2)X;;-*wwFD?<|Y=*hJ+`%i2`$2c^>B`s?1^KiZ^wW%GR#`3=f!v zl=#D<>l`+ck3T6+%waz4M~gCteaER__;FXl zbKH3Z)UF@NMS}&bE!T33X#tC7yjoRpdLe`N@`z}&h}9~Z(TP_5{lh8FDf&&>B+QGL zKU*obEMmcBmq965jhO0GKd3HcFXT=7m?!6G=PqWI_=Vp^z+zU*V;LjDF|W3>Yb|k$ zSsc@3K~d~m!hAITE_QKg2{ZFs+2ZS^tbDP|>>M~250a;V%v+_eT`XA2N^8!d#Ezw` ziq~M70Yej8m@kz!e~QOTF(Wt67KN9AYZbFajb$)uaz!o!(bu ziFNo5oRXDewi(k-w@$?#2jE z@wZd%V!zUQipqV10tQnFiHKI*$_8vz$`3u2jt9iL<*a5qf0-o{H$@yxC<-SHy2`G! zZAbo;SGId<=Opx%9W#NHFDl!-E61+f&y+PfJGU%K*xZ$~nIdon3t&q`n-$EbY!~Dt z2htSOZPyodl#ihDV>2xiS71!>@(0C|m27IE3g45;>Y=Ury$D&w{>95?itDRbRo-mB z@L0{h;ivYAF{{}GpLY9z6XPh;fM7K8gbWS|HQFbNuVJ++Hh%#UrQzAj`pat)1^C`! zU{(j5hGTb&=(`3&J}X1)T7&)^l_Acq!GzN`L)h0a3;!%rY+uWw3b!38iSg_*(sfo+ z&oYHu;sC^9ie6fKw5a_vYgQ*2J|et(?2Agx7}fNt?u&i+^w#rsA{G1m5aOG9?il6j zGNNC~$wY*u>7wmEQmpuyMf2Y2;^EKCpWjQjc>jxeGG01ER9eSE3O~nU9L@K9H$v7Z zjO$oIetM+%VI8Yqr|z9Rwc&mmwYE93h+5k~5Imxnv$nzs;@LVD=K2T3La=n^Eez|K zH($D2^jeR(X!I^|W<8t1=l&!HY+zl*u=T8waVGi*pGO}bQH_qI*p+afR~G-E^~L-Ibp9YfG1 zrfLR!#wmEBU=+W_6v8p_H-6tWla)25nqthZ?#5_@)v@ox338rzQPxVHDw`=H`AtvU z-jq%5De79k?wM3yLf5Qt)SC3BC-%Oi6R88z;eb`q7Q!33tCH_Gkdc?)8q(URMr!Dx5p|-|I{O;ZT`H@m zWt=TU;6`ppw%7b_fI+t+XN14l1TAd5vK=EAmn+$8CN{t~ytvW`J}fx0po4->DlX5- z5QSR`ZW&Et6MDopULNurbdTtj8J?7VB0R}nXhd&k(Vf%`W&g80r4#ncU_#P4ORQ{V z=!@?QjEBG!`hp<~jN!InxcJPlPbr_+Gx%ic$;6hX)~>nyl~rLn?({oG68thc8a7@~ zU^c&83>u_|$aJVPnMoOckkkRx>PzJ&2B@`(G@=|eGVG7(wfPHdga(!Lr=CnmvK@p? zD>62)DwSW{BQdfmK2j#H<&~e_@09kZr?!>)US(FZGCGNQ?e*0DaI&yI2IFmm|FR5;&In*~@jM;*D_RUuDVhdZtdk+wc zx3X>gr!DIH^ztI-SN2Vv>wo3v=IJIft8q1<>Ov8AMZ!F7y zW1AT_{b=d5o%LtDqFL~g)*>(Olt8^ocJuA)h*Y3I~sv|yKt`}#X`?gzTA*2cwM%L$>}VZ&ps`7rn4Hn z$z`FWvzlCYS@`T_YJ1Ll3X%OWF#^%Cj(FlGJuUM$UG6^ag7 z2pZv~_mYQ`mAHi>Za-VVxtd&j$%1^z)A1_2_Q96x`%acOy5!IBWPXl)xD0l%AO}Nx|UCZU&;W-*06BhZ0xV)Xm`N(3N_#CNh`DWHaxoQyTi%V z!cXEsCL7Nq9LZPac2w zq_cL}#5zCa{wbhPCnu~L8Y8ID{y#~MG5U7kflMyzEEq-p5ESXFi0Bg0Iigd<=h!L{ zmk+U0o)1rh^7E4N1t*2(Fg##e{uF;4#@cq?pW@D8HjbD4Q}q9x&FedDCMv^v>X$sk z9+?edTZ7V~KT1k%8=aI^KduafniM^G=e}3G=>&F~TJdXz&T<$Z_#O5djWp5yNj=rM-m&(#M5N$7PR`YkK zJM*l&PvqG1E`?@gAoLdut`g6t-KLW#jp!(mlyeXw%?3bRrynT}P5 zJCM`M=XgOxey#M|(PFqc03?h!5+dy*xnt;M*VjF&_DkPsvP;CZBWwsCKTov#gKgpW zE(zD8tP$VvL_{2AmHDD0V&GA>iQ9e`rH?^D_dG4yA7h{CLe_wcjWb2~lu8f+l?ky!EQ&`fUFs`HOvciW`ttw!-h?_A$1l+&70QUk|iB_scxeE)DRl zky3Dp*mj&%>RN;HU^?xQ<`|D-{lk^3>LjPM1W=iXG2~bU^;V(|(=edXXWJ_$LZ#A& zf_}rR!YtRv`qWmAMgxEHRl{A*afUx#K7be@g>r$U0F(Ri8 z!Zzl}CV_T`VMhVv8(^XjXJkowAyw`&`I6=(h#x^BcGD|%oQB2pHuQc%Bgl0k4hlgW zV@x1s6s{XLb@jsK8aQ{pQtfi=dB0NrCN&r(Tk4A%tjEcVND~Sj#lr4JEAUy0LP-#U zp=0x=`%GX}Qcq#~nG7XtC{f=#?8mhbi`GLBz_ z{Z~$D0KG!pt~R@fWQDA94cSnTm~&G|z>|mrBIyi-{NW7C<1_Gu@)iRvf#;cl@iKo| zhF@S>#?QPKqc5^Pe0U9U`63&}uiX%BF2N#r^tD)W3Cq%1X`=OImc*YN5=SqyUaY97 z@fRz}(nX8ESQq>({tKSZEhbUp3JYO-#FtlCMv-Da5ySMQ@uT#bC4#QP-?Uo{xys7n z$9$Ega9frrevJ((p1zg35?@4uRItddPu(SETw~#Em*wO&HjeReH$?jz>_us523^3i z+LChpAiyORHUbb`>Sx$su!iCfzFIYqI+EI(%=SSS zMdnQw#(T#Lev9>XOULR`*=804Z?Uetbh602#o{XNoJg#OP*^Lz`Vge+>~L3?NQkAo za&fANyv_XFwgRGTohk<1W{df$38L^F7*i`Jh=zAqb+_*(U?82(5l!!~vciNv{IdyS z?HxFRpC^fXcVG=pO%fgMvevw5l2~$=H7Ki1LWy`#O>)+o)@BersdbclM)CYEt5tSu zzV_-;++QL$WW1eSe{QR2s<0@xu0~+l@s8>(DOM`1f}0OPF27eH-zm(~?cR7mTFX2o zgwBf1j?LpmO)D!`e+E+2roVX#)aEePKCUj-36h0jtB1I{*hT^P3q1_YW$GVg#Y8J> z*0=!_BF0j0VqNlU*Tn90ht2>ios~hM5ELSXk5t5}9PE!nv>5U#M~j%|-ecX1Trf)! zx`7`hV2T)ck8R*tV}-Ac#g*8Erk^M2Zzp$1pyP4Y65ci&8ys+R4BA7h+6Y81ODyXM z?hxWXRidkjA7$7iQSLsgS#$JEF&ZmKO5jdw{DEn0T+c0;IZsrCdak zZM1~bV%;$fyi)dmFUmY%fx0PaR7-+?94U%1M$z#BeEM5u`YJj4{2ohg^A?#U?ntFwKi!*YwhA0@6ogqP^XDDmzgYgl4ET$U!v zKt>JM;$w}p5 zTQ533W=%octjEky2uziJyf4;^+mBgnjrlkrZ}W7R2@jPc-;kL=({eq3B^@u)qL6Q! zz7f5iK+6plqn^Np`!BKR32TiX>l0Ro{Vd8nWxvkaG~0)NQAdn^#%}SJYen31R-O$P z#^rp9)U#9{vclzBf)RCtMvwD8s1SuJM-1$eGT^O75>etC?5_p4B8 z+6Y*RV*+E%h$NR*^WG}4`z4Fv3s;FkuUI8f^EHboyc(zEz*4)u(`qsLH9O8-kBC-p zSX+-V6OcfbjWYUsvFZ(0`PWv6tT$L-Zdf7Syuo5`>gA6QP6Vro_@Ix^ks zms`?T@pJ&s^b-Z&!YYfcA%fpxkDzsb(eW({=;99pJ|=A>OygcKjV~qC90gy~JD8@) z;D&5vofYIy$wpLAn&C_48i`a5vPMV%oFCvb2AI6}>gsZF{4J}XzxB`NT}Tw#ci=$h z>Y~9rHqhtNI3gcPju;2$2U+&Y+%aPNJJ#!q_)jHPfYzbeUb+f%J#uD!s3WtAMDlV{ zf|@n2S5KZs73J-$X0Z%t{CxWjFGVjqcyo4%NV2ms1&1LWF6ydZi&b_OSm_ZaFoZg{ z-IIGMIO6~j7_o!|^{%p_4+QMWK=Ihle1i`xN3}Shj2uq)$!370JuRWia=enzcB5i* z2F}6eDPQ~ou`m7jfyHxX@)~v8ydZzeeVa8;WfK?6VXAEDjZzOF(jUe$7D#g|@tSV6 z$3Q>d>o4k6;$QLths3-}d<1{8KzRD|&v~P4(bu1!=KB32G=Nv&Pxpx)0lW*}yg+OV z;1Pv}ERbEUEbjSRW!@0`GxJ5`Dtw`FD$Eddi9<5yZaeHSjo?whwpE;O-cOw}6Kczq zT5(#aMVoTuLant#@QUOM&NriRef?t=Mnudab;i@ z+;iu2y<(pY-rs^4^kkXZ&%}n_P8bqHYr|`ZOLQ#+uI$dF^%f^)TZ_`HYHz$WJtrB?&t5G_=C+Xj#cF4jKx7B7?EeM zFWc=Ef7roAGr&zHD%Wa6-6Xj!YO}rtStR1XeF_tlmtD#T#(vkvJ4b_2G zzy*ONDQ@&I>uV4?@ZXO*KoLMGRd_lTQ08?bBUp9AixW?=gbNpdyk^m<_!xwgC4A{{R4XrriclY3dT?OZ4=rtHKi4%6@29s@@C8<$>%igL_tkm1 zkxu~<+6CV1kS`oq-Q^*@sNf7`&EFcFpG!hPoU+S92yjTR3#-UEk6){CzT^1ED* z>ADdAyULocs96dA;iOY$_JDbE9QIEHb~MR0!KxSD38i5#OG06upecI#wFWYT&vs?1 zwrsq3Q-p_jUV$!hEU32j6peJe5AV8CEY|U___?dX&5H+e?PL+`#e4G=lf*19{+)ld zNvIdbMK=Q#l@^S@3w z-{T0xrM04X4m-tHPY_BD^DF56IW#RD>6x{xZ*kGjhsT%w8?%y4N4~pQ2Aj~XOi<@Y zl9;pNt`G0>tT#gRnx3-9BCT>3KY#)Jjaz^P2o_C1KI?dj5xDL69cW zj{=RKlIWlaheQV&MdcE_T%({K#6Xh8TnG26Hn|-+zawO!Jgi)GulY!LUIi*@hw?~1 zVVor~l<#Jqby4Ns1~AL*`fFn?O&ak)#*2&<0~$jBR*kXz*_fLdFY&$T-2{Yh9Bnz& zgr8!bhex9n6bnj?{LZpAoX_CAQKD$noZsVZ6GW9rUX52u5FI0VGG9Exaz2v(OT(WI z6)6Tjj3*5hFAe-DpE^X`i{?*S2G@p}+Z+<`sG9U3T3f@z7y>a>&IKc-s;Zo*Hl+Y% zC1>McHES=*8bn#w2aAfWd8b0-2C2fip{AJ7n%D5O49RU@zrkW>Yks#>cgiy8*{~S) zGJ4RNVTQDxC*-W&e2~}`!+(xjTLYYgWf2YBPA~4QT8@gV1xgxSDN(v`8-gU&g_}HX zy9;KzyvOFE2BIpQL$d1|;fbVfP<4?R%R@Z}49?}C{%cXN4Y!qWN0#V8&vWS_OXK+f zQK&6n$U6@ZYuobk#;H{$ZDm{Jq3uw0&_+(%A_0;%CCW)#IkZGNZMBp?kJEMz6(K%e z@|Q)nx z>e1xXC?u`%KUr9_L4{7Ykx=4g{#PpXdSm8kzCr+D)Hzz?n~Oq0H` zp%2McNbDD#j^RQ7bcVF@2HXT0v77v&>NlknalIc_urZ;}bd$@_gSp0ENG0EGi+E}_ zK5;a@=nGDer`QEK5Z)hmL^bx;CD+E93>Tc2Bw$*c7Yw36nJZbY;afvv;@-U&b8ov7 zDYjt_67uM(M-ikpx+?ZCX<(ac9E5^`y+{tbI(f|x^0xW4Lg2l(ui;#dF&VvZy22eF zM&Z)P*Z73Ru1mH9AND)3H~(Ch+>B~YYw6c2&EpzHN$%0cRPS0etkh6^&8|zPkMU?X z6BjDnY?bDEldAW_MGH4^#RQyd*x~3$Ek)y`x7k2fIyZ9D8B<4sJY|M#C#A)?TM;75 z9vymFm;8?EH~mefaMk0wi7Gj*+*_r2<3pi(Z(5n^T{oD#-{4eopcUt)5dppPdqahb zh8@>{0soBA6^`OCq4wD=jZ5%hJbAvs6CJ5!w>2))@o*k7%uiok zA7=^tqC<}+UbC(Q0o36ix}Un9+>{A*rCEiIEF&z{Y<%w8hkEoiHITN2DZvj$J3j3| z;D;D8PT`mCCVNls8bP4D${ztaMs$tzE-v%OY%#VZ4&QNJ;z_4Ui&sT_)l)qZjO^%lJ0x9nd0#+u&e6+&DjF%;(6>FY>W z@O$rBgwE)Hn*I(x?S(7MT>;F*vGByRhR_#8L_!&Z{qTqBG1}zuoTBpE4qQEw9lqYuR_!egcI4$tPDh?_-EVgL^-i?+YC44tXw!RTMkmYr zjy%hiSM4qic16BvzFML<7(pAYh|6z|UyHSDA?4d9*m27~4L06w4VQ=h_~{msKY^{{UCepBpe zoGKllwM-wxJ2L)rq_{np$MSBGmU`a+spf0uYS9hjr(9Wkq_kM8wqQ?Q5-U%)K&r43IY%g4HN?hL}`N2WRj&kekSujLGkp%H1?4^SS|7w zU=FX%4^JqUb}Qu~Uz}r0w7JZvP4Pg0pSpUv7IM}_4cMS&d+8{NW*$+q{R)V5 z`KH5)cTx0ezREJ+anvwHnR#F0TLF+t`V$qU&8;W zV(DPwg|&rhR|4zI+Ve%_ps?-9gPLzraO8E~9CKn4CtSYXGP&JysNit2(2q%+sYRESYx%U1T{Rq7}4 z*)%?i52+wFP2)anoj5d&Pk|LcYrl@~oS(ZP_`Z_Dt?)^rRH?xJ^pJ4T6WQ%E1=Mc15=0iNxw8c!f&dN*p+* z-g_?d>&H1>Z>z7J9j|BA*V>NPZ1vUM@w#1oy`F=sHwf3y>ibUAO|SFS*IAC&H1+j! z$Lm=2wWoXq)2}FnRLWXQ$W`Tt%%Js_mB}_otB^V$AoW#9lJNs5K*d{GLXImX)YtDE zuWo7qzYqCeTqh>!he$}aaz}kt^JgpPDzu`4rXZd3mKgz0%ERU9l zw)Deyn`v=@uoU1^Jy(kQ@O+HvDf}WmvFZ3@8IduESN1GYj@A&6X@oK;EuPKcRr#}W zqTF0ww$=`mQP(Gwu5|e8y}B5{j6{qo1|S*9-7dd}$M2RHIF~n!xLy+VBun=fMH?kA zs~UDhu0k@AN=2Ss0xHKB99Z9b6Z*E_X|Q0ErCV zI=QWjNwJBV$*vD^pzVLj@`d0T%&R%67#gUs*$*iPFI%eqM6*RRACWMhf5-cu6A$Kd z-_kw@G%$a)NtDq%PBEBcukDn>`-)ZgE#M7(C!W{1?1Tb!!A^z4b%!Nymx5yW0$$zo z98z&T*FFr<=jX-07Vu=8`K`5(cl0QJ4&_RsoDzIRm>2RB9<|OmFe#n>5(5|UFZjtb z;=m&Qd94wrohkD#=g5!u7D=;nj2|r-N}|5blmjZ&MHzKjv{=l;_=UM5buph7`imCc zBucTj^QF&M=*~%;y7wa>+FLMPXjbOMCn}+lisjMX~!b zFC~sG<@I%Ge>w`0Kj&W;g!eLD&12^t042i{E{Imkc%VmFXEr7Lf*4N_PpV%kVUG)9 z%Q9X$w2ZS9jy=NnR~H;QLdD2QNCW=Bd945Rk*0AOl!Rcr9(bP zhLpq6x~RvhmsuHfRv54Frk0s2`8w`ZAIR0YnLe}GcS^*r;Sm*nL)=PC7cr)j%7|0% zvGCIF21Lz=W3_Ro#4l@jBVMPV<^CEn(W0}&uj^sCl*E^mn#M*}xTM7T;{-J|0b9I>=XQ{UN4r5g>`{K76JPVgV$5b<-=p|5C&cBjNZ-s$d2I4< z3c$rf;?`#P5x4FWMYe!_A<4)qTa|W5)ZM~^Ji6!NYkNox-NJo664xV-q@wyEF>4EN z=rL`gv()WuabXL(XLW|~+=_ln$hNfF%A2xcr;;3592-iNvc!U4`D8XpRNBU?d;CU& zf(E71@1W?mjYoLAA&BIr&q1+f8z1cVek~wWua&-$&z1DHs-(y55p{p#Mvs4ucT}Xf z91!Wh@x>lrKXBsioGAuacwPQxmsn&0hdpys-s~3#Ej+;E@_lFCL;J-`f~?Dhi2b6< zc0j(*g~aa{UAJRQ>d^%%^`Gq*Q?~Q+9-USLB6(DJzu3B+SMiuQ77)qV^ZUe|?L4XR z%P~$!%sx4SQ`+Kq1O*pVrK~KZqmmOdl$BSmvMRGrEZ>28-`{iAdp|>*+`+5*9|pww z85kMyD|6x@X&r+|5h{gCGDP{Ey#8lHbIbO~Q1wS6Fw#l~Nl5B1== zWgqMnf9~WymAYrwDR=ahtD7=eTuiio96uq%~;R($<8fdOap}M3z z_uMP`pnyj^g2+)?d9RoUNT|(;lvb4Y({t>&Jp#cB8qnqFQBhby2F6x>IHD)84IPcL z#B<8IKV7`r#cT8)kf%7f8koCT($hK`$Z*28)QFN1$yt3OO^KVx8YE-k>R+hFlx^Dp ztX1+LP+Fh}Dxc%{&eW6L6;pP@o&I8*7`=xd;mvQ07U_Jt$E1-CQk2WP#Eo<~%ohJD zitgnl8qC=R#X@eaheW73WX-RR1{2YvkSLp{OH~g&UWHGb_7389iMYKyf&a8soY~83 zmfV6;n9OPArMn0PnubLf%CFcdN@wu$C7zxqHTz2WeWx^RM_hnm+hm7mm%(3o$8SM} zjtNm!?0t8riam3LRP4_;f27z8?~saJJl)6F6`FRI^mVrK)poIFKlkSiH;JSBc}X6B z-lFX1eVJPXPLL~&w^=$I;A^?ZmqVP4*uGU1$mSJGrIJ2z)Q?XWoCmwDW4DTC+5AD! z`x8X_4Mg}?PHvIpzs9&lLl+*E)0GxKn63_`R|f-?rkh0EA?)K+UMI|lpn;xm7QY|D z)+R#)Y{52Ni@f0k62 zfDHv3*?I^wlypyv)gt>iZ|oUOyVIs990k;79THwAc&#F5k;!1*GXO(UnYdEKoZw|} za%12Lc+J1U*Kd%xSoCh0EV29qW{g4oMDaiQq*|vkftohQlnX1!pmuB(s-I2cuvN&< zH$XcLrk>VUYen{-ymp27l{jje(KFNZB>8$7d~cc&oI&oTX?;}p5DCxqn^Z(RQH zVj?l&p!`fPcQ(pY2k{~y(y#LpeD-2d?J}R_vA(NQjQC7HQzuK3q_SHu&`crmIE;m8S+_JO&3~VVTCdNVwOn13h6#LU%a`>E!=08*m4cr zKJk-ycnyYe<~V0FQhySm*U^l2GezI)JiK&C9OjnIn2nm9y1P8s{A7mO1ZAE$d!4WK zxYEJdgnIMD4>x#?if^VnoA3@NNFkGB9XIxUPRFCoVNa%u^EY@9|5b=WH@V`rcNzxP z!MWniO&;hr^hdnRoht%w@j-6QDaC)Tn1720Hnbtd$p%9neQ*QE+#_CQvxET%BOzB8 zaUgSv*^yh>KS#W~#mkldp32hzB?H@@;Db!H+W{me-_$uG^frIt@#R!!v6R_3(#Xs5 zZf5bt9T*2ev&D~h_yM0nZ6wbdA!Ev6gDwh-#%!h2EHUsdzs4hf5Umwh5Pj2thAelb z^-M8B;oWqqN)B`@BdHNx+Kn#6; z@kEK8YT5UYH`I7MStDUE=G{!f?I|Bv?7#}x05ql?S*K31Onl0J;)NoRK!eYkY6*D3 z+iMDSMp+uMIb%iAYwm08CviPeE{~NXI844~D@XAf!`|ScZG*1mG3eOFvfOdyS4uEz zqn0>~4pc~2UtlJwHc_TDrxdj#oGD&Nf$wJ*YzHW$@A36LFw{jp%rSEiX5OwYi@rBX z1eF$jKtRntH9>;?2~2(IXyR@aylU|Wu!E506D)(@aFd45O%$)*VGH)rKFh%Oycg#O zhKs{?z9y*BaE(h?$9C|>rdTjWBXky5e2f=Tn83{>cxfK+vv21t)#zJZ`WC4%pHrdX zc~Ub8+){!mZod=PI+MTN-j0%rZ@#o*Rx(`3|@`#;Hf>3nyiIqD-+m zQ!q5ZB3dhsR8zVTE}0$QwonOUFEwE(64cxj!r_9_QU$*z66kbw;vooM9RRWS*9lZA zchn35DMN2$2&0ECc<2F~wt9FN0fAvrqA2FBdCk8cZ1E_lsm%D6Bul+Qn(MCIZMpE#X=3@(t% zV=$QD|ALw06-O2ij!Wa&dx~gPT+@)h9w;Ui*Hm|}3i(N^d32%JQCzcDQ!j{eJ5T9AJ=SbRF0q>T07gtWfy|C@#gYC4Xchj$T)n1x z`M~^zPC`GT&^drnsPO{PO0PM|9~(ru5}IS&&mbO_(5&T8(!@L;&3fKEO|CBiy0M3A4RlK8H?#uU^x1uYqlmL$Ik)J>j@-67LqTtZ&SIu?&;Q*ET9 zP1noHqMfHx(e-=;jurW7+IU?Hlc}+4YO84~;Wle3q zW2`t)88r-(za>QNDySiRtOyU(_=-MNP=i{mkV%ZKqS;cc#7D@Z$57VlZuBZL!sv*G=}I>#LN%{tFqyuoJ13Q-aT9jz=LSrbg1uzVAuYOGPgO>Yt65v?rQh^Cz0|PBTR3uFj_I)mfn1)p(8aV_~gqNg&Lu@^znL;`{0~Segj!2Zw5~ zT8?uK=1fh~ELqAv$j5uL(e-q2V;nG@)oZ8hYT38raNNdE?DHM6-vthIP<;vuKUbCm zX|FfyU%W$K?t#ZCLsJXvbH)gT(d(2g)vL)o4AFy7&s{MQ?&IGhG1r#jh*lh-3F@Tz zJRJx`07sNCQMBcVDjZSRgt*8NCG#POBMJe5`0NM6u3b7y^LaJ^c77Gj64i;)obmz3 zlr+Wl2Pe765$61q8%Vh|nU~=Pr(DcE%9tqY@{CU72)+PDO8XHsbF3v(4dI*<2s7!5Ex0yTS`9hNFm$AZY;pmyx{8m}fv64wk;Vi_Y zoxwtyF(9EI=VZHiebAU`a&Z_TDGv|T0g3WCh}P6*Wee>aqqVs5JbOpPmuxf?Z> z>at$qjBdv9by*)lOo*k7dTfn2IhJPCXR%`aSbAHZ1(!*VN2@{D%yg z0;H=mZJ_^U7XZEEMLnO(6gh5j5Y z$V|v{xN(=+-1I$BJH2TRMSq$8pbANOo_P(;J=>C2<_>L1dp0b^qNfv z=?hB`x5PF{`IL0uKJdtE2&3p*zo6=&EXXD!3SL-k`aF%I#8B3&$50@&wYIY23_8bO z&VG+!IQc0O%)-YXE*h*!B7Lf%Ez*bN_rO~J0nljH1&9ynCBfHppyyJBf}67dF))fI zHD{F^uH(=bE29w~P{K4H+bH9q=B%+GPU=ZwOBPYEMgkA-GoT|yw?x;sPM~ouSsRCO z^>eWwPr!p-_28>?pt~)ZujAkz8ds?LKCti=FqNh9VQBuhU#k!GmWpfVy!p2mp)O%) z-qr2sNf=uw&T*jet-w2Sdpgz%%bIKL=zS};+4&(hewoEA2ZfeZp+h^`*BUK4aUel2 z_fXNhEiGt+9l@eVGPXfW4)mk@ZCDSnNdyIlvt;pLIQ<$9{Ey-EAsj8Yk4hMFiN9|aqcqoc4w#P;?!&tlnvlB&^x>U9k_;O@TYTAjF7FX4zew|o1F}(ww z@5B~~wjJq*DD>mael#nJl@m{PpgmE{UHn*;Zbo54*S!N3>x`krw*!TAW@Qc3m~$}X z`%fxT39a6i0)vX@n71w(=8B1ie{va*ibCPMf#I(n>sNg9>Sgk(1`K3n`S^|A>Y%tI z*36n9js8Yj*F>5bD21BZ|4Sgx#XHVATA=4h*xsVp%|lt-5?|koDv!Ye=KB*K$`s%@ zf6Oe7x;yKhOFms#b>Rv1>cVapJ`};l;Ec8{prdemTWS@}c2|7_J-z-}_Q@OZy*;y1 zhyWR<5l{t@r!23E@dVcbaJ^4cIG<*h@QA!)n5XE~mfFU!uqL-Ln3&o8qsHd^M55vs zKCNO{I;q%HLK(O5E3(!O;LOmJ_dKO3wZZi*(uzUvlsr3X<858Vse%-;Mxx~&@i4pQS%bUeY#)*zo?iC`qR@aCVAk1$Ue#RR6B*cZ=id zDy8$|PI6p1#Xdi7JI57OvYo$jmbJ1BIL=vX9l%)+;4_QgpNoVD@vpqL`6^1)k7|ux zl3YGLei+Zlp56oEJ$eltIw&NCoeZ)b3fh&NmTzHHeXG_K+nss1#(zV3YQfXvq3s0n z26i>;(~|DYz2{LRXx*hll@{R7T}z7Q(!1WQMsaVrGTIh5Up3$g ziBo)fwIu&O%)jboZP-6?D)=f&i|iy^)e5mnLmy8{yefny-Qk0gs&5!d%loh(=i6T= z*07XMd`*(L0;{|Ojp<7tR#rUGm^|Z{hj7c-IF9uYgx_gxUv^cvNn?IsDmVJ!_-0Qm zmFUlYDFXGZAw6u3A;M?%LCaJhuVA{`A0ub~VEWRZ)p0Lrm;YSPq$?Rpp;G-IIkr5A zS`R>XHJHr~O-$}OO=!yi1}T=>ymld{ImbSaUJk&_xz3jg3}ha*^L#a*70MD_F4Y>y zf^BRuE^0+&jVWm$^Q?IvgQ14Fp^b`JN&$&CFc!m!H~A+3S@{SaLJ;DI^$uoeDW+6UowfZLTIB%fBhl((uKy6^SE zy^p{>I4Dtrz0aunB|Im1E+IaQmyLe}yvtuH$PFZy!Dt4j#?){yDAQE52($ID7YZ=#I0-Oj~V#gwA-n71<8OYJ6)Qsy3AM7USyB{GqHF8&n{d z?!&|GeIs;jKdtC18<5>FR>iq9P!K-U%2OdEEkt+|f;Z2D8&UW$<{mkq;yHdz%DIH20KA_^QPlJ|~Zk z4r5W`^7`Z+&nkw7H9$=ys|jO5Qhh>UKNb7}$?(c~flG22?^A-6WPG`;ixqon^o!)% z3BI5#^*v>)0Gbugiq)8k5M20ypOmEB!~5+hOu%g*<83k8sroorksa+@v8u0IpKyXS zTVI}Ilb~1z9i&A)-ZXNK4_!>aVds(rR>t9u55@&2%;6Z&)8wa`S0iy_4iz1a{)? zyjxFP6Dh0pD1DX>%^b~wT}FasEGuM9#31QKBh+?a1$sG}H5KojdDE41)3wd$j5B%kVGOGbm2sY9v3(tq zM`2@GyDDB7^}uCQAA)UKrtPvqCbiaStc`}zQ^7LY&its-<~5xk%WBm8h|y3ZS;I*X zb`pQS5H~tAj&k-rqIw|jqgo}y-oe2R4wiG2H(jrD|FurkdX$2{s~ zY1btarz(X%=BUajh{u0D0@?T;4TL#A|Fy*xXdm7`FTTrBv0jHt*ge^#Gq8qY<1)r( z1~yp`i@YQIG}hVmEwVG=_*9-U^e(08qY79AB~HJirW06qzdO~wmo-`=n7XCdzni>V{T;d0wEW!h`1oy4wN-*VGFY<;GplUOz3Fa=Ix#YMYX z)M*mt;bpg=E}gk}_OQZI3Zk_Be9JBC@twsiQ42=+Qwg@pqkWUG1%394vL-RF?rlJ) zZFeDh9Fh`!acc0*lfl1RA%-WvW37`Wdm)C9G+aNmNLBMn--H%Mhf(m6&uVJ$Gf?k0 zQSXH88z`UcgNN2UNqA==nYF5zjn?Iix(*+ML(-OVMny!bzGP*T6%z`kBCe^g%2Rrq z%xc(N=2gJ`Csd}YlUZ4)pNgE!DvBTL(74H1yhG-9YPPmeshJDK^3+uX=`3iGd4kMmec^wyRsiF;tCBWM9r(Aip0-0w^a+iDVIFyxY9 z3M*Bx!3*uw!pn`;O~F2Vc?Ehlg_IR@3d(J$QGkoK|w>vBC!f(_0Vfg(P*9 z{gf5FPSfP)`{;cNd*)U|!0m6-fRpb%iMeILB*%J_(q1T2@*t&VwRI}PAVHCh?kzyh4C!Pjq$q^`Fl)>Y-W}wp8AX?r!m!KL}^KT z9mTeB4?%dius`Ssu?8i=Tu{4w-l>BxoU@z}DFhplZ@kjrvVcX1o-Qii3J$ZAC#b{3)nDmv@>;H2=^B!P@n95 z2CFrD{wC}h?R73wJPFk^UA$3g-k0~9JkzlmGJ9dz@kR7WeuWbXo|Gd|A|Z3;Nx8YR zaqME|ED7t3^OiGvL0C&`Rk5_!&^7;JCeW^G^g2luAy!}VVJ6B zPg8$lA6%vgoXt-uQ!Z)g%`!VjUZVM{STA7}<*Z_vxVU9*L#5-8;D;}zO0g@nejOVp_DrKnKeH-DH^1al^+A(pN4r@{ z@oN)ozyQW6px}q+8FbiOY(U?|SfO{7ss+9=o8OA87r0 z)Geq|ginIr5Sf`7Izl02A?vT5@59Y`Q0cs#84UQyQ#VAYe^ zG=2m2nI|sM-VLm@$5u>&}jb11BsV^5inEy}f*Ib6!^ z%t^U%0V=(bH7hd*{KQv&d^y2au=Z1p5Ox5g4?$Q=m8#xDrWqUAQE_@QMQp-(`cp?eA>iPbCHOXP%DS9XCV@DLXq3rRZYi~?A$2?dD~HQCIjJC;E%u9m{)M6h>YX`j}T3dCHYz zIcmmoT!q3hb=F5_)gxX6G8eUOg((_3c)5oq(82tH z!tnZ&rV$E7f`)D8=3rdzrJ1ids!gM98&S zG;1&Nbo--m!(Mh=5~Cl}kVEK~)gROHLzp`{JfabYS*U3Hh>jjclusX$au`F$;fG|O z&aT<4cx{fU*&BMD&c^Cf-s0#{+Zkzpd}(%m6^%cFkNR(3)3PJX&Hl_Q6!b~Uv|-Sl z7I5Gdojn5PPkKe{D64B|xGhJ`ER6SHQFPuftHX^AQ|duJm?+(z30BGrBS`Q@fPvpy z(`vgYV$`atAdfnhYc+xaGCaHk6B&M4%tnl6{6>sFITt7&{oF=IWzhV~e_lnk@WMjlf z8C3cdYv@^*2L@B)0dY-9>llZ?0~l5ibLq?&tW7g0=@e^Pa=&73s_n3t6Klf9s)!7F ze2SHBmuMk~heh)#nE^^v)Z+Ch6o2~;C#2dyhMwG7qek$iJUS2!HxX$8ADVI9*BiT> zW=%x9wutF)EHZ^Y_l!23WvOD5YsR|g*ls3HUrle%<5T~D2d0RYt|HG1SkP5}Kpify zY9-YB$hbvsAda!Bo0SKae@M$OutwsV`^E6cg+cPle7t36dFE$2Pu``y~`JEk12 zU1oH?inF<*2Vz9b4~?J_V<4UwiMKltV{CjKB0Hk(O&WcZ)e`UBq%AjDqS)~URr?+L zobflQ>F=zfgZ>6$O2rXmYQh3+>Ti+(SnGL-SMx-KQ)gPF6DHggt70KOW5a^g#rbPvlBkIS_79xGMo z@I|cyDX-_yg?p%qkqFlKH#xyuu?IdS8b8`q-0H^FA9P){3AF_gQOk z`w7aqkEB{54$S%}X+p8g2Ij7l7SQzswSK_L1dlqTZE-_WZg7gLTs34>ZNwXVfrmSZ z892V?$i7%5bhxZ#M%C*OBS&|_);QBu)jvH?dmgaTHro(F3%cSwJ$=C3?fs^FWjUz( zolTApAr0qz!MN-pZyNUtbow!CAuc>Z`X`XwtapxXKfz*S%2{%H%99pl3Yhok(X7<>6mhzs0D-zC<%QN=c`^7m1X+~G9 z;?WSQejdW1WEnWg;;WzY@C`hOoux^00VK)JkiUZS&E$i$QepK9cZ4TKZ*87=oN9co zuqlF=8B4>SW3nE(oz^~QRjY??=Y5gyvM{s>*v50^L#qC#lgOix$@gF56AxG=K%p#V zn@J}~%w+q;Q^)CWCWH{)ouC#k_-^zB4S9i2(UZ2)q8H4+-Y=8Ed3BWT5DeEAL3}$p z4>+txUqM-dvT!>5ouGRiM>q$y-FoUSYcLX5v0L~ht5rMcI5^M={TyrSlwem0g(EP`g(g$NOGpK*K1-P0Lc~6(%a>TIR6R>UudsnGaF!;&g8!8>bmA3;z6EE< z<~2sMwPz^sHAD~loS|v2L67^K7r1-TX>xu8ch}Pt{RSWG>Yk?AZ_vvtoi=WM!&VAn zfs+*e7Hn|p1TA}u=AVCpZoNgoA5Ku|cX0ckpsw$bP~LG`^9~8wo}k?|Znjj?sSZK5>k)-XozE$EflLBsA-oG2{az811~!jEDIP_NxBEKDzf23+x&D z=4UaFvOV`gS510KvLC7r4egN4ja_kgNpjyOu!X8`frEN7d|<^Lz%TO;e2ph)AN9*Z zJmNm%^ei?&tdY8#SF6^Wt>W#_C~dTU9dANIO(4bxj^-U0uWaw8ieK2FB5UBCu>vWD zq>h3j*%`a&(--Dibnq@7**68VvAc}rvRRDS!fJ<>iuET;bPlj#4YOTCdwfM#(9oAn zXg`j&pR`evv^v;G^D=Iy3%P8onAXZTArBo!Xtryy6eqC@d`(XRI`SDl7>?$$Ao24ydS6^>CJx+20Zvj~vC%dft z89|JjPwpk8rh$H)(0$!ZpD{UOD&;u>kN4_7PC@Vf9L1-WOpi==%#%3tI%oyPRB8nZ zRv800b*ETbQ$lJb7L272C8ScNl&*Xk6MQjnw*Wo=DF;0`JEv-fs*jvUwMt6e#Xen$ zN=n}1fgkB?NhzTIhZs{JM4ygEP}&6{dRHF(ZL?2CN4WCm;4Q>TGe;`>V<^Z~@^P|- z5&{e*TdnMqxo$H}7E~D&Lo;0^XR%!j{o*Q>6L-y}ORkc;Xct3&x=PJlTF%v2g-?6= zJBrU-3U@=Wl5?q_8|v~Nr2O5aO0{;(0eNcjNaQdi$&c%jGdog8qQlJ7e6g=S-WIE8 zwF~~fbTLZqQWe?0=Kvgv@w>i!aN80=4NFVU#NKJf3S}fg@)Oy3BaS3&9{PMtz|6(qL;D?@?2 z6HZN|KjY zb~1IVB$X;8P1Yt`rFIBSt0Y|(J5DynRF)(`j7T>2tRms6;^ZV+P!((!JdPezl^jLq zFyqInQeP%G8DqVr-{od4Mrl=C0p};CDy{(=9+Nkr8+}EGXlOSRy5mR~krL1i_k;8cAV-weeCTNs?T*aJko%TNnyV zi93J`O^MIzM}Ia!`;O^HHo;OUF{+=jO0Wb~ro-aMF+}PsTF2425UG)PvJa()NXx`> zeJD0m+9VqK810)&5Rj`YJIO*X=T7GPL8CSKGR_VkB(G(IXdAXK|#&{YJ zb<1c|_W*PI^OZthuN;~jDRuT-T;GgUW_w};*mGYsix=<`Dv70XNNg)zaoqxWJ(PkE z;#T%QrZ=&^p8#)>j>+6{L)_nmUNuTqdoPG zl0wAm?PzP1R6^X>j(&}jD!J=Cz;TWF>MBWFNLt1Fp^D;=493285JZyj5=lEu-S|?6 z=WY&q@SgeZmT5cJ&k1a`wv_e-6n+=^px|`F8cXcI4~wer)RwAtkzDFEfH&8bp*>udbS#c{8L#W6n4^lD5==U# zeQ>pbWCGt?1_7;%bI+j(UBKi&Mw-$|jQrbldPkB+w4`s_ntOAFGcovv*%*$miKdN- zQz~B5IxBh_e0Uli_2#mUu~uxam_2RGp7r3V40p>hrbJ8Sa7>X&+qz*;cFr_@>?Y+3 z!c&^nQz}N*J*6qa6C>;kWU)(As`P_2%H;vS*phniB5IeP5e48i$_h9oKO0O3en1?_ z!A7B<^phYyfUtIdsj|2;h%Wb+s*6cMMm9j&#l*IEjQa*lmznsf9?ghH1K+7 z5j;{~^I=unFJJ<*Po9m6)&_cQYzfOH@8ME{c-NnP9xk=4@iP?0hNax+pPF#sRiz~^ zfqf(Hmy@k#L}Lub(h++PRo~s8%8Zc8i~jx;HUeX+i$4t*AtjaBh!a0;oYo|h`QaQt zR2!b1t!DNEKVtha)Q^gdl!B`UnJD-0)GFiX%aYh>r68#7ZCa)b*ppdQn=!5YX!1y@ zxwx|qT^=d5DcPNWiAl+9nYRBvvczR#c#ps*k`ZGHK)kNG~0BTZW=9Bcr~8E)Vw1U0`@P|26NjFxJNAAD%`Xh~n`u8)bP zu!RPz`Li5{((zS6xgi>ND6g{U;%KRgIMs*pMoUiOx-4=UBQ3HC^5dCDCwuEd2gXQc z#Em}maEw&P<~B-v$?A1X5zbsOk|Pq$JPn=uxTN}LJW*;O{Az5MC_NAiy5~8n`O8M0;{QGyI&}H+y*g}}#ZC6o++K+5*3DC# zU^fYLX~#miT9AmHb3t!D2$-gz@!Exh#Hp(OU5#&8Oia81FfsiBEU!SjciFgiN4pHt z95L9NLdQ#OtNVO%p_YAo7gDFq3iX(7TT+;;hN%J^}jbXpRQ84pj9Y-Hh( z@!WJNNf2AzFt(W?1q;G{nm<#TR->l}62O3k?qZry;$VY}feFp$W)ph6hW7QKHnTA1 zlrnkq!Oi47SR3Z7J!sr4skr#26wRL{RTI-o8BflVim{SM@vXtM@3c?mAH6l+&6_9H^tyI|^FKe|#=9dX_RmvR`~mJ;jCfT2 z(h`(0Px7wuLy5mFN2Ce;eL3ov(8?i}EkWJpOPMwO3W2BhKSs+{M3qLtg?@g~w5sNY zGV?zC>`Yw>4Q_RZ#*ySIK6j=iB-L=)gBx9$B{KDyj@nuL?fAmlmT;vrJt3)%a~!;b zCw)vP7{Ud3hT?t>pSY&oqS^~2AF-q}^;-bG{oq8i7ohh3aH3xpNP*%aCwj3!8YjKo z232{QxWt>^(TyzP5|4hT#S5j9WzQGqB_GpKZtXOv!3E+Nh= zPFV}36*kc>yvtzEYU@NlEt1+*PSd0Ojn0F)O;!9<8Fo|au_0*(Z{lGnWs)Y4bWd4w zgS-|?B(2(F$$G@@R7pJSNT-%yH9f(Ra0*tE zJeEo=#BJwkz*4EbjW@Ed4Jr2>>HJcuqBzfyJ};HJ2%D(=GO4Pt(Ku^70lmmv^1yCN?Wb4cCF??F-U~by)nZ zF|J=HO_9aUlCjn%X&N+gipIm6r36{*m}~UiDV-O@so6%|Zs|D_TjFE=Ua4YP@2B`i zVph_zW*=%gP45d726;YmUI9iqf1eudlZqAc!$BhFy4XF&n0-=D!KUUXjsO!p_&~qy zm+Ds-^Io)4s%^^01l^1p6KGWt4PRcYwB6)=K)Nc-BHcl0gfQDU?x184@Bnz;A?c_9 zT~|larD$O~l{zA23Db>EM(~!H8zmPy1?n=K1!;S6kVQnqM)6xghNnx0==R;`^69$vpQ)!tn*m&S6 zu)-iCUS)nOiuNaHM5fe9+#o%vk zrff2`ne$-dT2a2Pvv;2IHMQ4DkN(Egf-(-XTdXz)*~#Fz!#^1>6qZ9IF=3g}#Zk5u z#Xd`n<@It;N!&J>8W`j%4d+hQI4&0BJ)R&3e>I-Kw^U%f+3*45Dc->HQIMaRrQF5W z09C&X-`zOxo$P4*Pvzdw@gMNPa)rseba>aHe{blq{B7klH&SAx>Hlm6*r8f+qIz9=GqwJw%T%YysA5z za((2|u02Qd@&-Kl0elb}q%Aj~DxbR|f~dZ)Tt##pO+Wa`wZvzmXt^&kGh`I$>&Qvs zi;<>G?jA`O>j2+qq_L@=ypY-NYHMZ^UNoz^^s1iR+Qv9SYeH4uJHZ%IU&fKjr6IJd zfjrJVw6?}@I4$_LYzY{GTp{#;TE_5(@?@dFyjbLDs~_EMBws1%%vUP4A&Z=+K7~?l zT;cyLk5~v6D(6DK2FjfTXDZiNt|&Mg+ccK#gaTR72wch6*fU5TDhhhzjbM3hA?fWG zh@Y9JMX!83^vH`Aw~;H0%VOzt8#&f@NfggG9$PMhrZ3=Z9>)2qtuBITi;~-Xw8uc` z-PJ_e%`=-~!sSS@UsvOfaM?pBkWmVO%T%CS5%Ozse-}CziMDvs*|@STZ;ORdrnZPj zQfP}0&5_-~rOj=ja##1#RHePFO5X3yd3LDonlH~@uGz-u4su;)6WvMUNzBY0OuhVg zlw6|XS_r=@oeJmc<k zjJ(<=7dg-xXk&z_Qp$8ir8EskrF<@!uTrWYlx06s0lvIaIyz=kpRRHj@kkpw(N%6# z#LxyF*jgx?iz1DT64_KdR<15KY(vdr z+?c6qrvQ(5?qlz#D5HyHliXcyR>QBAR<7@>*3*RkzG@4#q8Bw}mr}38{^yu-1+Tnu zzO`}4rk&m87;$JA73(3_b?^l-A544OBV#L_vW+o4linW zlCKJ4Az$O9G4dWJT6t>oMx%!wsF*bEU_@#E;R|1>JOhfCa0;Bt#FBg(;oT8!gWegE z+U{pn|H9iuQtX{^y+JN4iC3!{_a(`51+ifjW9wvjt|YE6Z+x608$_{US!1v1avPaV zDJ;@PJDodSpDo)r>Qh)tm>>5jw;yT=_Xmn6Z2Jfa7XSxow@khzoUZDJl;R|+vhcpK z>>PP4WBueDrT zy}UqLzQU5frKj@+n01<7t(To#3^MOPW1O|69OTOQRYNT0;$(8)AbX2?8)Nte`4`l^ zt1)Mzd|9w5#{=+P{xy+)-z>XTm@ne)TZ%iLImQbPF<(_y9#g7fIsEmDTkLUi-y#Qy zE3Z)W7WueX`7D**3O4d8N~^b`4Ngp=Gh5{~;@xx_yiIN<&dZ{`+vGrT;yKFMCQlWW zk2HBZhQDKHY5#V(Lnl)34mm^&wxgLl?sQ`slK!!G-;l$FNQ@m+EcyLM-?F&rfP3?&y)@2K`}Iap+;Xv}W; z2Uj|23cMQ|)xQQ#hP<6oepUZ9K8Nq+0-2~N?Mu7TZ$H?kLhyb2BCJWuu-Xtgods`g zU4x#Pa8WP|ojy81mG{VD#ck2_2yAJN+$^Y)Db)mb4TWm#;wOO&(Rd>-;4wYeBdcs< zmY^v$@F;-}rU`N$x5(3)o;>>Di2yx8U5=60KDlKfdrQh)|IQ4Hn36e~90h1o_t=mqWpnIS0x0fP7D!{ho3U$XD@uOm$);7q7KOo#P@G$(IGVY;kR_*knALu%OK^D>{>qr0hC&Mwcdo+ z)X_;kD(`n8j}U&;Ujf2gj`ZDM_*ywN47Y~^9k>wYxE<8!usp)%H_U(NT6ntg4MeZx z%Fbj#%+M^u@U9r z>qTe0*iUng$QA959OUz4@EaU;w0cITkH`aT%m0Z6-JOnobIL*rJSxxYcOK~mfTJ4P zPdSAeL4)#DnmDJKz-s9+Ml`+r2_#@nfVn_D6i2-_VR>vZwP0lt!6~=Faek4_fvy zM?AYrr;f>iwqsuiR)J~FgsF1#*HrAd>=n>-i&n!a$G*nY1p(CaDR0Um{NVteUtG0G z3kc@%gxp1wVUf^gGbV!6-8AgDT*EOOK7oh0eAZ$92dT0bwEMVR$)O2S<7H^CHB{jj z^yau+O5F5_Q0J#oB}_i z$7ke*VyCB6_N?q#F9x*V-jxjXKV?5@D~b@3D!1f5DZwisS{@23&Vu5DP^1+t)8agE z+fjZ>6VA$k4oA^XL6Xh$k{Xcnj4q#*D;o~MsY!@o3owRv{nTWK_o%xRc`1A_N+ess ziFbtLM7Tpv$GaT;a8YhAhbu=D_PQ(= zll&s@XjNGs2E>-6ga}kmJvg}J&7qt80t@pg4>tyfH~6NdC5=<`^fLwif#F zZMt<`E@4=3izgGj0zAx#!~F9SWaSE84neVYf|P%lXl*WAXy5B!%XX88OWXA|0q_~b zJAlNiS18^DEtRFbBP7(pfV}N0IGf*ua9+ko?62YMIfJTtM;_ZLIpnr+;tiQeqI%Uh z<+hxF)3+fFVa;?GMo?=H3!TP_I@am7au{?SR*ued}{?_x1M;u6{4 zlV^&rE*V$clP8G6D5KXyIZLp)eE})(p>Nu6ROT@jdi8#z$j5SsO(%4KHvFzb? z@>hJg$0HI}vn=9~xP}LHFSj6_`YSzpEWZ$Uoux}pmE-cwx~c@CAbVY6e!|i-%)UOy^!W@7avWY|&UCwg91>;UD5ds4^Tx;P_|U|A zeb^e<&%W@?>Y%Qz)z!DD9v%hBT$X3FvcBdCoVDrSLqt zm`G!(PaYQXL&wteJlUnny=~~E!CIKqwjDmJVk}RSI}Op+fm=h; zf`Qgyl1e?zqE;-FkK^Y<|bUf)m0R8 z<{Kjng^rB;NwMV966bi~iK<+>A>K=w=Gykmbgz1$FBM z@x)U4y^yYFp#p@;!p8d4AZli#^Np$lJ<_JORNnVRuN2e}|2(MW%>z64fFIuE)_OvD z*zrme*J<*r2~$-Ic}h8`1;Z1FmS9zQuyuXuhKA8PNtiU6ndd=JUXsz>1R5o3FaG2ROe;wqvcBVu@=!ELbEeTYFOB!a656>IojX z-2aPN?SVzj!AQj4WFa*!tgBLC_*^Y>5JqdaLJ`^+1n>;-}BRCm-zxU ztuB`>ESPe`*R_h=o6Z#0HEuU*rMV=7VNoJup>|+dU;Kr_W?mj@E81`iT?nVM>4&W8 z@CRl#>w1Xeq2B!f{IY%&YY*-!w!oB0-#Pg*d1S!fXY%(S&6zx+)%QUylP7wbGU>m< zoXHxnD2`o`QcN8F3c{8@lSM3a)|~DfSV1|-+2d;_EAUV|I6meFTH&CpSz`t&-9EV* zzsS1Z)IY7phos7@c%vj8%L_@qX`c)QskmJD5-Z%EKap5O=ga!f7v}!a)}1;P(Z#zP zmw^^yYFMekhMPVYLsOeM+wVrp+cj%WnMHJ!#o?vNxu|Z5c+-oUmFRv^UF#z4mJ3!~ zarPuWDx}G4f8-n4fj-xlXndgRe}jcb7d+`DHa@BH3D6@$30?SQv*~c~%VzGlY&L$m zK;hMeVur+}Jln@d{^hcGKqN?&Taz%y;3KK3TulCg=O|7Dj{hKJ;N!lk90mdh9g9!O zRX{0ZLd!R2)03e|iPn}Oua+rOcv}{GgRg!%NQ1+}RCvh}3u5;*y z9z$5qIaH{)&aK3}E?Rru$EH-(w^@X=Q*QFzcw!d{ERJx_OKEs<-QWVRW^=DJy5yvD zrGienaz%Gd1BSncls3?>w3DuM;reOZU)$~}jiPCWldhV>cx^ne zTnW52i+**|wJLfx<*zYC^r9-xh^hBv*;?Ha7R7qX3qvsk( zijF|F?9nEzG&{WJ)zwwEr1B6%_cwd+D4>;j;2XY*TiOmC>Si>RbJK;^)MH4}(zoSf z6MXB#_uGJnMfN+42O7n5Sd{2!TIr^1)WQ)l9(F)aQ1$6NBzEhL-(I%Yqw%yOF4Mw= z(ry zcdJq@n%a1vt|50i5N4q-Gfl*mBIrjCov(Ow3jOM#3m58M z1Gj`ieqa%0%Q7{1yncx5T`DFSErJ5y^fok!t6T7OMl4o{^-FI|bEyYk`uQ?W)W zM-#ks#S5q9!~Z#renUKlgeZhJU3C9mnP-m;IGfOP*qZ#gQo+Ay5pBZ>iu3XAD?yt6 zM>MWkCis-%6oHOlIu(Cs@BSqaiqivj#A032J~ZWeV7t&1mk)t$15KT3%Y$&2vdBu*KKM+I!F9vEb}7CGRjF-!mXv?^#B7&t`kbU@>CngfHrU z1INtt7L$i;%p`{9W0FD94p zcRzT&{FY3OuZ?O40cYZ%^kDmUG zHoxEJeE1ol3&E-Ao_$}`Py4^9b_c$w&up{RdI!E3)C&Bh_Usq6euHeaZa}tL!)!in zlC3)8Grc>2+e>uJR%`tmsM0B0)d4JYUis`6++R(@H{!o~{4?J7Zz6R7?i^Opt`OY# zPaxbvXRuhlGar8EwnQ49@XrYxgF*~gpB{C*bu;S7B1!O+pOVvgFTT8=ua zN{(viT{TD5!QxasM-{zu)YEWpshy*)22oGA8&%Fx%fbH|Wc=?}%25l#e796KD@W1>4X3fn}SN$7!1po8-Xzt{*|GTi?`5XQ*vFCpi={4XcUC&XM z09pd316JXRdElKKwfC(YwdU;{b@J~yswec%tOZ;Jv<2)0+pTi`?1Y%TbrX%mv&9ku9|BOo#ab5CE-3 zQGi#-L0J;EphMwNfQLP>z zvoH;R;_ohUYKv;*)%gMt{RVb`EucT3@1Jl3TmfSNjo%@H_ozaccL5UtK5%yhyoNmm z=4zN@U=D#f7Z476EX>~ktpSd34|tFMzXp~vfFJ+?b^vyP;Mc4i_2uUr^^Z?EYQmQs z^(^f30H4k7da$p7Js##Hm|bADfY}330`|5Da}f4oQ1w^=x`f)pECkaZW(Syi3g)VP z3K{s{U!j8L@eQ}`JNNhWt-tecYIbjh*%1(DldG-()BubJ6oHD(%1|oW9%dl}EWWTr z!8`+{zAK?P_`d;rXlng$=)b2me2@75!~M$XYo-l<$^F@H9vXh)ckb^qxAr^#aC2?c zfLRC{Z*QVDZJ>Mcp8@Uvz30CV^gqLVAIE>}HvG%n{pSh&Q>MSifA2PYpVRLNEN*Ug zEt#wS0EOMYfX`)eRS%fC zQ(yw2?na>e@C3L?@IPnvZv?Y2+>HTIuyglBgvk#(;MWnb%{*x&S1A6`()h3}|Cv_5_3gTmdZsPk=XE#NU3v zQNT_>96Z_r-osuh7^5Iy3!tmngGy`yY?&gpLNCosWyQjch4p?kOnBcZ0BKy<+GQEJ;>}{l@DJ!pWTvs&Ru zO|Ci?!0ioSP6l)e%T<4Dm8<>&*bdV^Bv7B(UFuM;DmC09FFHeKBAQ;38l*;HcSt0Onc1mMEOW1Il*(tG!)E49s_&;9snCaQGf){gXS&F z=773;5O6;RtrMsjv&roo+~MxI7YXe{rr_RhHo2WcBHXcubJbpenSjE;8Ah2c+`(Zj z+&2J!9>UmtJXcLfH<{ed;RF08W`GwI0L)o{Du7pjbssV1{&!$duV&?{#!vrQz*NwA zf&LO82M~|M{@=jx&k>A8V*PM6p%TkeV{>y=H?zs@9Q@!eP$Ey=Rx(e0R6S4q(`<4( zha+$&!M_U3V{rF3o7~Q!KHL=zwer*zwe!>#Fweq#2&f3V1uT9R_y48I<2LYw{|P5ll~*Lt(Br!1FWU3kZh6oC%N8Fk8Ug90dIl;Y8Rk^9W#{)hSQy2$~uIS7hV` z659jZeZWc3mjc}@n2QldF~E8F{{c7*um*iI#PJ;XVW680yJ0IlS^$xYKx=^5gwkAt zc>v})0LNLt;%9OHADGzxnG>)?&J+J0cxE`=_jpU#0F>+)z@{@w1k(j(XF!9odFq5g zdFt|^dFqlOs2!MHxH&9OO$3Y^hV~B|o~OPC;fxU`liNAu0iS~a&lA8SaC^a@n-&O% zzY8L(0{@-B&oi6c&LIqL8VQ~lm8TwuJ94zifwRDW9+;A4vo)KccT{lU^;@n zFU&2-L@oHaAZ(2(ph?VAdjlrI-+f%3S_ffkBh2h%^nV{D{s@kz2sj*`Sx8_%0_DO! z3}zGpjYYx@VNV9#ZbV)fk*$II{~ZwFe-p_8Bs>ss9hn_F{lAC*5zg>8k^B#$zoGee ziCf~ZnEym?_}-)czey~AgnwtZ;4CJOKfnFE^bA@x{tJTtRW*N?$iIjK?f-YgJVPhI zbb+b=J^Ux(zS0=}7YSH0Wr@UMTJZnQ^v;*2;qQp^NBDPk3(jKlIP%;7U7Y#+4gaeu z&KJS|Iy}$t!bG(HK@21T7=)IaO>XCK6K*?{Yylpn|2wFiX2OpFjRh=pUS>aD)r-t_ zi{E3jTZ`N950N{V@o%U7ed3nLEE#-l≥)`NJ$j_j3eT=+DB={cFMGnKP`#AGcTt zV)N$?nWznzp05O!1iv>wB0`Sikz0Ug#u6_#Et%x$nbS4=g@6-v1mq8t-`*=Lz9M0bJP3-=tje}#P>?8z|8!8{DyK+t@E=>YQ)%uO5z z7!MFZ^9HnG@M{h85^&XEKL>ma;12)+-wjY1UB4hJi@n|Ksdiz^bUau;(0(GoaiQ5LCn?prRR~ zlA?J*#ao(Dk+)2-(9}#R$*8DsdSKmWfUkMG{^T6^vLWoGsaogpNQ0}|gf67eq}Go*6@aidt5crr}P?P zk*0DAoAz#jpY&Z;A}t%2N&g?eA2&DZ-SGQ=jO+S;jc`MK{r@A=E3?u5|F<0f{~5+E zH%HojX-lL24SttiY}5yAZPbV3*BQTuP+Q$y>^TeNfABkqUoJ}WDdQkS+7F$OmodF= zlSqF06@mX0@)p1YFX4KDk^J-vLb&mxM&08GE@*(i;^z-P;Vrr~%u4VOn!h&@ z7QZ0DbB+1~_^rV&3O|A0Px!ropWAtKCdPY!d+}R^--q}eVmu2Qhv9!7zn}3dXFT~E zGWZ8BRDZ|*-?s{X4itgT_`Uciu4I7^1Koaa)HhwklnU&F-!~U-{5uaDru?BMdhOvK z4;!M1+|(l+TZQ%?NQolg2gT($!9u?r{POY3#A%i{aoUA`$M7r1?|mHCNT03i8z0p5 zW%#YZuW-4pJ61IRy9=`EEZ+ir3ct7c=NQ){(r#9Ih;bG0qqlTD@oindg5Roxy8d&q zuFpD(bC389#&6Qcy8b1w3O|uOMNJeQQ`Ng7x;q!mS@_ueB%w6Ps zbk@TQ7tfsUoVy@tR;s}qymzX)-eyw{Uc6V`;obUynaMLBn49v1bK&AykItUI@Nv<) zK#dW}`D)AH#j_shwJ3Sk12bkny71u{b5mwLI%C1CMJcnAA|G8ON?ub(h>(ZWuy&6v zn3W1G@J`a1nX>Rv=c1IEDYNj|J++hjqDLR-_3+HO3&hHKYM*P?JBl3-sqbI265yFU z>(PfFL#(+A7vOJg&sR$h_Iy}Px3`U%+q>V~e$jIm&3$s#V=<5Q?)Mn^#NmbNPk6_S zoXAa9BigLNX}AseZNu+X{9=AP_(QTf{3iFleICQVe$fXba#Xu|Fmk8br>nYRp01b7 z6|;}2oosW(@?&Zrd*(dx?lGK`%$|4fhhu76hfBd6DfZ@PO zoX{(#F#h63B`^!<*j20h;%s6bFf9=Jd9`{rFacNqEC!YUqhG4kbAzA(|`%DA|n2XSr#xA;XGg#$$|O6xIIXK z^neAyumkiDfgUgl=>J-+J|36=^y~yXz@D%dx3^X=o{pD;9IwOE8990b83{$s_n{_$ z8Tlw_7z`e$)s-&LD?kF>5D{1jOn3_!?G8O))huKj=-w0Zw`+ALuoxHvj6R4OnT?Db zs@2QqA_KsLc?iFQ0=Nbsr|={X#CtRE*6Mjc$9uJU#vmjB^qh|bKC0EvBOHFDR*xTo z3;~ZVfZ=1c`U-@jKSjoX3BXcd9x!SlGIG3DuK|XiKmlSA4;ViJ_CJSRd~~n?=!%0U z`pa5<&Rxhc@DMPgq*h;j532a9TD=fh`ZcO-5_0@atzP{U@LN=AL8TiW)d(f zt4@zb=dR3h)#(ZFWNfU{bAegF0-(RB(>>AK3xLtUjO;o+6<7?+2bKa$fYrb%p#O_? zdJINIDKH6`w53kpO$lwq+zRyHj`MZkqk#p$1YkL^zy*&xhD&KK5&~8O6M$Je>hz1i zf}O~SF^uq%0O6~3dNR&mAI2&|d=U}= z`hScJ0}Fuu9g)GKb$To??h~vkz?5Tk`d(nhr*(Q6FyT1P%yz{1&w^)82on0TPR|A= zeO0Fy0sX(O(<_1D_=raY#%)3=UUdRye2W?Y7N4xsOMwO7q2`>(2rzj%?3AHKXP_Xz zz%KZdUy(2{>9;z48u$WW8Nvw{>U2kE7ZiR+4l(`<{zL?fd&ghs3&4a+Pz07!gi@pMRTpemI9K2qqPX<=n@EHqWG`=!Z z0J#HSrYUrRC}>fy_rwH~qSot^fzcWifu+EFV1|3WUJi`&sMp=`wFgJbdOaFg3`_*3 zw5r!v1EX>DICv&9>RqoV10DE+&mrQ0#R%u&gFycGgU3gc<^ZGH)$6W8cnbQ~>%I?w zxTRh%MT7#N6Ej>!KO{hWOuc>#m;$T;CJcZ9nh6KujC&H~INn@C;UV>U0^S498d|UC zfKQ37*B!H94;T$h8dk4oL!JlBK{zF@-lgvakua)W_rz412TTB#0_PBpM#8{qU^y^h z3~HbUsuGw0bc}@_FdUc$j00u@lYn`^4B#PP9`FRP7+68!+v@f3n~<^Fa87m*JjEd5 zfz`kypkrLUo(fFD!EEPj$Z-xj9T~KV6qrPF{PcP~7nn2y$6cY9kcbQ; zoC2%@W&s^HBVk|=&~ZO%?m_6&Iq5Wb@)o15fz?mc>(fZ_N%R3=LK=8r7O)cXtaOxw zc)ZC_19U7!0U{9w&H=_PgFRpdFb9~m98C&ztiY#@fC)hVUPyQ~3IKFv;7D;2JdQ*4 zdKR#t5S;~>aTo~z{g0rkqEG|H=nFvqkI|NMpa)C>`X5D`0;7S4fO)_bq<0))U@7n- zg+D_R_eKW4uGiy$4jlGP>5cJU4No2s_|9fIu(TXG>;r>8*6Xo^I1!uzECuEO9hLQZ z8L;v<80-uFA`Ala{zS=vNq^Prdx6!!GU6|z>Z1{^uGeFL1y|6tz=Rq!Sv1DKqZVxn zBBLH924?B?dc-Y=px`($(BIaeCsP=h3rtcQ^b%lnOPm($hc@=aX<}fwH}rs&z+7O4 z4-V@B3)(=hKhVEHj{*AQE7wWD@L(MHbdewervic1IA+{H3jJ^-F$OsV#sTB{BLXl9 zm;ua-MFxQW!yEKUU^K7>m;m$~fEodwM^jXeK!S6D_u-f$Fd-3#Jm;anIB}c?3{Ps% zUEzoj2Tu$z512v*aBTSmFnn%S`AJ~Lss_Cpn6w_}B;y{2!iEMt8xd20dx2TN zB48e{6j%WC9}M{msA`~p77{?bxJ@|7iEsiio#enQU7P&EsX5HK2oV&EWzvvQFDFnT942=w2DFy!IDYG4Ae0esR+XtJS* z4~zp=1E(!SO}-3$gdMvPAK@%u4#IJ-q6rZWckRL1XAnui3Sb5>I2Jhu#sR}$!vS7k z9I$6HGV(@)z5?L{;BH`6J~|!o`w<3aya~gkf1p9H0ww{=!H2(tW9q|Da$x*$WawSQ z1G>WD$%H2kcnFvTECc2NYk;MNIO#hA^24ZdVBUvF2dkD>$kyFAqmn;t@{z z4M&tH;R`4b!f|yt_ya8P#4QWb_riS=$89hGPfwtKK%<@j%xaHkpup%3xaR>Z0GIfIB(2BcXe6`x*uuGjOH>;bPzkgi{i6xPrnnkq|KYei%-L z!C82)i*UhggeSo8gGdM%{Sa>SL!P$)&&LstTZsGS!0==o1OOI1iu)uN;c?)ZcJLjD zfN(Cb5;%t<;(6IIgyZl~B^%-JXL0Wvn6eDFp1@ZFgW{pLyitz@#^D}$4de;9K|dd1 z|5eDq6OgaL{c~W!THH|rCapt_0<&D1jd~Uo6L8Nm{7Ga4_b(>{^MHlmD{+G@3*l1vl%bCR9p57! zFazkHfOtRPBoHtKm<04MN7VyMfhUOnu~GNDht_|1qVGYKoWgM=VDxDyP|krD5l+F) zw&00C++>Rb<^hv{rN9heG;Y}C0~3Bl9|EQT8-Pi;88?3t@B-RC4ORUI3J6TVZLwlt zG47oePQv)Fgr^*e34fvOC;}lVTt+W{8VS@OgOky=z-b5<^wRZAU{W7lPXHf&tFGsh z99RLq6zKU3GBQ}#2LTI);8qu~dZ?~v5RTCGGU%to>3WR|o&tFMryxP#At+>xgaRp! z*Yzmy{M|RT1^fPLE6(4R@R-}t(&Zwi>=fYo}UZ<d>l~@$mr@1xEFCD9Xjv zl!1_zR97(wmp&`*KdW|Cli;7TRIELVjAp@~x%LvTXRR_FjYc5*c`?0`YDW4|!cdPi_?@e3^?9;H*$Ac}*hdEG54&e0 zI9mow5WHvvKm1AcQ!gTz`@A^(3p$4H&#J@26ZM$F8r_A2*`?{4-2ff!e!TG(z0PQ&W@@q)LCX zgc@(TeS{ap3N?5HWC<_S=>g)5r{-tWZuc=J2l+Mq@t&GH>Ekzt<+^y}oEj)He^EU> zvcT;|!}b$K$?_5O+(;#Bua=O1lQ{JYO3(nmGn=Zvt#F=G@uz`clsW`v<$u$HHypnh z$Q;>X)H!sjMEJ{J#8t{$r?4sJp{LdEr;VK;I{J@OC)1-L6ULpI{6b7~v)Jh;lO z)NXCPf}rE>fsTR7cuclPID#=qpBpF^!PN-j4~UKM6+!>CxN~ZBSF+@e z&`3k5<|8<0tvGxhz4rwC&aL8y^J<7%1^@W9q7l4@XG_>a2IGvHYcmHqALgn08ij?Z zV4E2AD-{<06WhdsU(wE)@b}y)w*9Jxcoe{&jqG^H6iW~c&J(A8h2@LzJ9pRV>&2Ot zT3eqbR3{NAu74{;*)2kUL*>W9U$|RL_)Q(Crj!3wv6lF~@E7e7@50}`4F0^=>h#YM zSDD6r#s0cc#Z-MYkSE|i+(20){+<}JsLg&d7=_?`1m$4pIv&9y1P79_CLNF3Y7S)K zFyw`g!=j%DWIMPrWZP#$?AR|}zJTf|fqd#?Y$D=O-9)BZQM)~AJ(W{#Mh_QJ}sw#F9m=8ZSl-Sb*dWe zjj7=v{vd!Fsm_5v>8NP)yE;VO3xCluG4Xd)RT=!HpNeP6@9;s{kBc{cR~M`4@L&8) z^!Nk%#qh_h5(~)hj-?`HwRrOn=tsj}vs#?`L+#O$R+l*l$9*pD`csW`Q8*XjXHS#q>-zJ%~f`$|w!oV0@b5Vcs9+9TevsztnB5-7dU?Q0-U< zhIz-^XEzB9f~s$8G!}Z(S9V1N{;BqE83i()$gjm=WJ=9|fA=?{UzIwrWh`XJ5H2pI z@DSCrjiN-I6i-&EqtwapNB>V8twJF);Gg`XxKyR~Q48UZJuQ0t1%1U2gMCSmEsHdyBvZ)4!2Ed3@W2(flcssKO$GjQphIb zhT~&$mF$06?c(7KzzhMdgx2$}V%24}oyQz-mADc4i_{CORy%w3gt-FnG&=eV+uxY* zzNuDQcdP_A8JwM33l4iEr3PGdK)v2l-29j76!-kC_A)9wp)Go6fH?fOI;4Fz{KY}_ zdOwOZ&)b)J?_O|49Ymj6b>duVJkLN>c*lBu0_n(}fZYJ~geS}@8|CecGyMuWjuMMF zp2!d7SBdj>5=S@b$G}b)anog5bHF8mtLAc_#wwX!tp?pj5spFV2&vZrigK5?FU>sV z2>61d0$!h)G&2LpoUCYfOf7LCV(mZbC|3gfD-f^g#%d0@EDM(gF4w|kg4=81^1vOk zaEHJZTeuV8N`K|z2E^e<;hfWP|hv}hHy^b~8n9N3C2S#Nd1U?x&t*A7~ z5f=|`KDh40aiu1KTLBJR7sHLm|Eb_II*BK*sJD3>gFmS^ZpDf_Yf!g8T~R|GME_IV zQ4z(@*Xl_WE6zKSHh{!C!3VFzvmnVQ+aH%aJpsu8-&Eho6ZmPc4=`K|{!c<+PVaiL zu7!4smW#k@1U_q_^&Ukr%0VRK_Bnc|;fSjOmj(_)`x@uyfRcg3yl1#r{2v4^qpxUL zqYi9ELyMkvW{$5HK2EKph^1wyHfZMO{&-D^YEk`oCoJG*4L?FBC#HOO?m_rGeyj+ zSNpY(hu`x))T+_^SgN6)1TN?S@ov4^!y^m+Ik+$V2EIduEnSNS)kiG`cYd{K-+(z5 zkBO9wwPHd8rls%@@EgRF4Qf9%5&mge;_U{ke=Fek#|Eka+kr#yC%h=Uud0zjtfWM#L*;;u$`_X! zQG_!1!}p6w9qOK)F}M!a>u2SbF-=#4JmSEuc&A?f$k?^Mr6aX8P|iZiWmjVtzZc>B z_cz)#ti160AKnptj8+~Ns7h0$XJ?Ywu zd;Bj^CWVhp^H7hG6rYc`X+6{$_%n`*3FME!T$glQJY&;BHG1%s^I5&v9-;-bqy|kk z2yqmO9=Y(3KT)qQFd8&y%)i?f(ky{=bqT&JBOdux4Qls<+qF&2AV_If+Y^rj2H3@r zHd>%~#IAKxV~IQ|HrbJjWcc@fCyv7JkqiHzGTg_PnRlbigDNN!A#Pe94|;09_kZ>J zWm%)iZd#BU11{x9vDOWhlm!3o)AhPdHs-f(T42i@aOrr&u+~!?uE);Z%c1$SJOQ!- zANB0*Wo*R6V31k^$f~RL;)AnlJ1w}Y3@mbJowYaw+#Bn~7KheOOG2Owff<#Uu}U18 zL(5?1e-3S=s|0})8ot(Gqrzi3z^@Wqga>YG8jR#>z(s?@++sMI-aNZuZ36c+F-h3I zO!lVDloAfUrbmO0Nx*Q_>tet;Zo;irVvH+aT>mM zctB5zao*`R`%=u}SV!PyRCT%;RJW zMVyY*!o|Jr+C6Gw4~(>t;%#>=WL^gRsn68vw;?HVR0-MO3hu!tDGWw(d%+cfYdTvO zfja@tt%+VKxMSeBO^glHd2m_FCfL76%=6H?YO(P5oY)|?{H1p8O9{^hQ8F1f zshb!~2X_J-W=X>lW`Qf4+#vq!qP0^C5I8nnv}~!hS4*4ttDE@!F?ygg4>}&v@TcN= z4*mcFU4+rI$P*bjw_*gf`3UAFH3)ouGC*B{K;HA>R7>>u0*Z!(&rf{lsRgQ~Mo{!< zg;cBIk4O@?wbDY>;G5C0W{XG39|!-O*HL}FRSerh1qMU!hD3|%D{vc!4fQBO)5 z{>k&iW=}0drH7jZ^YFD%WOW7nc@K+In-qk6BYUHjX{Y)>^_1^M_# z8;qgYau6EVw-*Lc>cQDw+CZB}d=%y-JlSe;ncpp$In4V1=5(T5J}9@DQZ7!PRXZ9t zhv<=K#^b`<2U#nV{z&rIz@PWHnBaq2boNFz@#yR!V`UGx({^nFB$M%wdOk1j(x6F{ zRD_G46kqsY?%NH2_0yt}{AKV*FBOr#@VocH;LZ@!eYMb*QSj$1Z_vND8xxp+vAp)j z092+y6#qQFPeL{J6Q)vb7c(S`NRmwn4v##(>)}o4MgY zIg}d6;?{`?t+mLOJ^P}otjDt;nVi_#TJ!Ok2D0h}Jj0XyU@I1b(; z5B}XM-XNEgcYvSfqn3iJX(M|0A)W(EeoR|2)lZw~kqCcq`c>>Oj5NOTLmFA&veU(H ze&}e0@H>}?HvUNCJp38YiXr~seQ_lbyp(i8j4Qtwg!6Eh%wLOC=fj`8hWtH@3mJOk zTZr>{_sg{?Kx?g*fNEGHx&~;YJc9dUNUp^sv~BPJ>{PzO_YVB! zTDT?<362Mu`-|8ah`LIJzxurRf+FsQKm2#mh=@0r!%wdg-%p(fSJ?7)+OVZ=_M=7_ zsWU1V{d*g)>TWa&0!4C==Bf1rm#JSBOM|q|Y61cgir5pR4OesEk5$B_AZ%XB;rF$P zKJBo=``?PrW5cTgvR0S1Lutlxmeqs~p(8KA90pd64h>Bp% zp=N=o8A=4YISGe}HXXI$9*&{tD|qbIq(7=L;0o|j*N(I|fLig^3%vQkf`Gn|4e)nO>*b`!UCp391{N;EI_mOcI zPweZYd8zZkr7xu>4>e}tY=r$Y#2=lI;34?4GK9Ai30A-#ysS~I^VhnIL?^1pe>kdP z8A;`h_(2F)EfX7{ttP?mzFfRZ{?+iOKPP^2q640QUs)kSJ8J_yJV&5`@lw^(;=>R; zu6Uxe*2*Iq+#ztuG!wMF%beY*UnN4+0MTbs{qxS~BdftxJz0z0vT>`pp)>kJA-JeC z(IyoA>LUEO^w#0p*sNLW=%;cHk(NJV94*rPsVsj`8n*@K%dhss&9vSeD!DGcH zx9in>aB1s>8itJvJ&jM^&}giN#thaV3B6KI4G)KDKBMr6Uder`R!7glAI|21{4k1i z6g3HIJmf`BH!lx|d_Lq=8^lLp$Y>V)hg{+}_%(W>ue`{K2f<%134M;dIbMSxuHhzx zqaeo9eQYf8$7L(=1Y{IC^mefl5;XyS_aDTbaCFEF_$Rx>58)UE`S8#GNvK_rY8m|g z7er(iSgL_P1^L%)uqwGZy|8_!JQ4NkmFmQvxbyez(e zB{duV(#zrx@*je~__FZs3V)gON0PsW^#2wU;P(g`Lw>yBEB9;B2$mt(M%aF(?Hht= z*uUN>W_H3gPf=IRvuhf-d{RN>T-zkwE|&;egmVnJxYSh}(&;%x>Cgcpe0`Umvl~wa za+cNV>)>}>2p1jxQQL`Rphm3lm@7qy_qVFV$A7Yw%|W*SRgXyZNo8XMSzlW_shQ zj`D&iK{wawt$~iw;KX-=y{QsevRFag6%h9wFnrLU9H;Fz+BvI7;JK8bJjf>75BuSyiV~ zXLs}!p*^)o?-Ssnv+DFajAX@ha6a@{Z49`X#C;PXw)E6)_K5?T1~Qz=v@1gFJB_<7 zbv;o7dm%}Akxh&fLvDuI^Wai(diVsH=pYtEXa20k@h-h)9C2YN9$u$5IvAm%IoxxD3K9euhsrdBPg@s zt+4jrQLB%pxPDzNH-#U%?xyfG=p;aY5Y8zlQl>8>>|GG zqYWM!K^8r+QHZbAPm%wrF2(>D?VV}EBa}>jU_vv9@${P>Qzha=^Pi;qU>DJ;uQs3! z%{yrj6yRv{-I$mhw~ABIT34TZa1}Uxd+Kh*-ATMkj{BdNKqPM9_L4AQ~54q);z6-i!1%Kj{SXouuxB^)t@1| zM`XHj-m$3B30$MG#*7>Ty)pmP%emzM=GOj5Y6Zv`M3xh-*xX-h@0|}W9g6rP@rvUC z@lk&a<%{5w8te66F*qC}L}PzsDb^QyIBEJlWhu z5TgzAX#iOaazBx;gbAMkm=0oFV^&0$>q^E~g^9ZdXajs!fUJS$C#2~XPJK9Bd@w-k z=u-wsY7o8}iiyDSzcAT`#4UA8S{gf0f~1iBthW~`Z+Q= zRID6`S$Z$HtgiSFFBPJjI6P2m?^6n{94BJ`CJR2{q7xp^i>m|Cm)!lZ?BH|0fz(`= z!$gN$(S|V~gQ6PrwPf}}m>72}7Mo;{=^*iEoX{J>#8PP9nhVm|r$NWxJWw15!v5_d zw=H$7b+hA+gYlr*xIKLWx>yE(0kALtSJlL|YLZv`7iKmU8jL-m7l)L-qB30x6+;JWU3?Nj`r;m173Jx> zP#TQ^V#Q#Z+aU?YowN;P_O(!PaIn_hyBuT&$Wdg&CH@#p%O}QB`fwb)q(+z_LWgLf z-m&1S?yZ%J;AnBz5Uq!ID!4h*YV}1FH%zPq=aWx*a~t$df#7hf$e0YiAENnqS3**r zj!)H6@^^@ThM-=2p{3xw@L}rjy^PGZ6$6H%x#J*Ny{th$LOm^vi!pzw=IN6H$$Z?K z*h2+u6)L;PR*JJ5lJlA9;g|v)^_=!sL$yHfN=OFnz%4zB(^vd6R15X-$Hp@lCwLE1 z3W1#V;8@rh1Ig}J@zGsWvg1}M>G+b@b#+e<*W&Pb+tE}E^`T$e4D@Hx$;e)M*Ynsi+zXPW3 zXk7PG=bd4ebp(|alJhZF^~K1iW1N(Pv}k>;tT9L+4La30A8TwqmWGOTBd{Mi1hVX& ztGWxzqvKw#q!{;W5vhY-giZlIKo^B&50%t70=abJ!XbGPzE(+{22BoQor32g+9;g! zz>J7!)1Xp-PrkX6iY6AsVNa9|ZVb-MuB7Bn%eD=1^SCa#6BO@9oaZH-4?2s}aTpH@ zYNP5LK3qpNwY{@=XtdVZk1`hxK~Ef#ji1rcms68L-8>qil=v?zKu1&i2{S{WE82B%+8%-kozP^ zE_TQHPHLWSI?1VF%@|a8HY72(HtL^HKRF;JA%k09-?-4px(8oWqAvVGCvkd=)}_DF z86)liy!fqvT!`$LoBD^Y#WiNi2osO0C{VI=GM+ zi_V$}PQkI^P-?wE8Be@37FniuMjIZ%tL>Ds4;S-ylGH%5;!%9QjqLmxVx$$)_20Cp z_w)?Kte=8!R8bRu86t+>hQ(|e$Siz*>wYTXkr1))Hgw-ikQGnjFl&EF8uRHpw^0rt z34Rk_Y$F?eopPeTc$?PK+dmAYc^}_rpuD+>uH(=TW5E@H!=GQmMNB7g_c*QdBzkNY z^#R^QC-PjA>KnV%xGZzRKo)ey7u{Hgs_qbU=HRUHUv7pSqsT+X{QGDR<5IZ@9mi38 z(F}gapH0#lxzsHj9K9@80Qub-zV^@bti)>dzs^Or?_lWXuz*+g+Lc!#yuNwS=0Zg#o<2)68IFJkit1iX6CuB;mCei~_Cy~F(+$Yt^Gp{j}|dHwJf zlAZJms_4KJugeJh?_T$l&^vqY$5`5S-|zJ?+ZU*p(hWARmyqb04Yp%gi(lPl^SX!}@7iH={}35_ zZHH|csxm*%=JhS|bntbX*IHz%_;uS?m>M=7w0Tt?aM_ie2W=mq`p*>FyjCEo-JjZ8 zHXtV_kK4TdKy7R~Ve=|NjeqpH&Ff|0_utsOs!(T}zOx0q3ViK5TOm?8_k(SVjp{qQ zT=K7%+df9CT&l1I{0+6|&e(oHmd~7(!GF%$4kOV5*RL|X?++R4>pyKqQ=ITrB*zQL)U9owRbX$$h z^B`C1eOSYK~zLG8S?!PfdYSoyfY<}y;ty=r@clG)a13m}CHjkeZ*?6xbb)9tI= zNKn4i9-xD0%&;4ZE0^16euD7H74}cwUX&9Jm>s*SK|AgJM5PBDZ>AjK1cZZ zE%uI;7?odcv3FaAPFTOi?)l!^c4dE#y+tYd&6mJ42xo7#w+--JG0b)OJCGewdcAvk{eNS$)x3~zb*k<=S2Yhb3-T!|`@ZIfp&r*ax z+inkA2mQ<2?LH?+o@@8a2fhj1ijG>5YY*6r@W~zaj(O<+mv`8`D8n0f+5;L9@0*== zuWT0-pWS5-I1S9%W%oP*#n*S)0}lgF?y`HW1fJhz_bEWH$jGyID@8ZT&$D}e3H%h{ zw}2~OvImqyp7)a7YdOM)U$S?*h}q@0m+ZqR!OEBI{yWhJuD*;CV<>EX#omeZ-+RUG z<2nN31Uy^Ndw&F8M!(#)+a7Qj;fy_YuT6XGwhw`)fqP%GdmVnwZaWGr1J(hz?zP*F z?X|Z&^t#1x!1BV@_V$#M ze+uoJuqkn2WjOfV-P#xS*1FAC`55EkW4rs8n0M_?v2&s}rgv{!Po*3aL%A(-g+uwm z?ta-#IcXn>@#EI&4>#pyw|GVQ$F0>;hi!=icVk@2!M>BUqc+iTiZ)HOo{Bs2?%u8X zD(xc0;wjqD0Nv(4rEQT-S$Go;DA~f{J8Z*~rYTyOxNoYKW#r-%rsq?d`*}_IMQeNT zkEvRZwj%vIcVAag9OgZ^#MK7i^u5IS<1kgj&6YuhZl>J~fATs>xHE5JypOa~L6_HX zCU0W?Oet?UUfRV}PtK(b$NypYSp>?x;AppM_*8JQ&F~fALYv{=12@9tsk3lZD24D3GzCrgFK)me1>al(-+;%S=|2T4 z14o92D^+hrS?d{cv@*nl7HT&?_7aMVOi+zab4aXWjf z?ykh!rMYy*Al5QFP%iVz1gYn|Lt^|unHkeCE^HMn%F2A^sSlA0fjaiZ%1Y)<`>zPzvDVcU znK^WQJ>llltT^FB{I^ZosE7 zznl{``{r`kpgDOLcuHTv7%`kxk%UybJ!}92$#99>MA$nU4u7B5K<=!{p>!FRJqZ_u z9N(M@k8rrFTu!l^_{#|9Z15{9NEQbEF9!c)eX}X8`YmFmC9^ZPK+dRHov|({=pqYZ z^w@re;CdblzfluK)CO?WJmj#H8-6Q44LoJ$JnPf6Mb0)##vdSqifA`h5;K_Cj5?(r zEXImGPim2F@mFjwEV1Lf941k>Dh@=YX&$!K;PGM@_|}=LW$b_0Snvz$?SUfkDb0Ou zVwG&%?zhQw%o^Cr;Y3dN5!473JdPLedzr^3)EJdlSs!=(4d=}pi0K@@^@SyFfr2t- zZ<1qI$rafLtosd`?dj^<%F`3&=J-13(gZN^c3F%EIayq=D^!y^_G>s61LjdU7{0r`|kNE|tp4zYjhdytPA|0!NFKl|KXSzugeiBrWTm@q3>W zHrI%+IeZ6X2!9@U-Ar^`C-%*BkIweM(Yr;FQQ@W)Y z{uS_jX%AtIup0te&6?wBQ)=a3$7t-*4DZ0ibbB-W64<9LfV6}9H{1S22xx+^Ch!7y znjftE0q|kX@Na>qJ%d&MJ@DPZhutCPk9hcv&hRo2GG);n!oC#y23&6$`-Y0Autrtd zh3^48`^|qub+;#`%9X>A?6Zsd#ac&KVkIUTxLn4}pJn(jPAAfkD#~CWEf-dP;0<_c zgy!^TgQrV6w&S8V0Svbg|98XBnm{r*nh32tEzwl5&GBQJ;TM6U9&o)rTJRZ=%_UF( zzUzM?;Du>$%_Y1N{McsnSApwv1HKoH;OldwC|ki%r?>LYf}_2wmCpdzr5T{rk19E?J*xctt#PbYpR?RdRrcTl%hepH?z*8Gq z`49{i+Adi6Fz}S9l@AAhU6aW2E6Q^aP_C>IN->e%+zfvl8zJJKyi>L6 z?`sjn&gZpAm%6|Mq z$X5P!@HBl{`8x14`iVDYoS6uaqxTyrr`8B=C>ibFth@*Kfz9v}!J7#{teE!afT!I| zlffWoJk*5YoTHHgxF!ll*p$bz+|-Lr!^RA*g0Gf~7=2mK##cl{bt6wLVV|D z^bayWW`pcQv@nyK$l=>%P`RH@a1BSm?FFf5PLO%5Xjat(=ybx*%8vm@7aN`&KM{U% zw5y|`JO2gIW33ix&z~%M@4#CWpFgR2+EZJIEos-M|POptrC5=OSJZr z(L6m{dD^n{N0RxQWDRwtTp@jh9_Z6{;W`1rbXJMRs+E5nJoRoXe-;VTR`Mh#U~WMs zL!WBJ${zsdqRGZ4WRA?0@*!}v@U&%~Zpe|Fz+s9*?rRQHrzGbAh@ql0i_lkh3b5qC}LDD{LUt(2|$RWCR=JHE8hpbnyyg3VSSknI#(%5M749h04pwdS}CpkRq$P# z;aj4lH0-VVbSAzzJ{^2_>fF`{OCX?*Y2`P8r|I6xue|}Ea|1iYP4p@KYgPV!6G2n{ zO*V9atIoXgux6wTqn)ZgV5{s?W|M9MN2AWle^2dCou(N9bq-|`8=~qa7m7}EGw^BV z&5V!@a>>8S`NnC4{(bPD$$Zg9DQ7Vb?TVFIn)lfBmnEOcIC`5b0F6L$Q#mYIMM<=T zf8&a=$!+zcEH)>L=QwOGZyPyGbwchF4%04$T&!i6+H$*0*o^-bCx?z`ILkBeo=Q88 zK+7Sy8!h-zjVcoidRY;PjQNa3j5n&_Dl=Y#jGum0VlFQLi#S8(=)XY; zT-P4&f6PUsEiXB1gw<@&oM%7aF!c^{7dcELo}8tMl+L-bN~5J%q36g^ia51NYvU1P zRM1I*L3PmVT1)REUQKY~cOuQ@zO294V$G}x^??G9T=xW=ysPtEn2j9^^1kp5BpXp3m&b67u$FEi)xYX1adEc5XO zVzOVwf^CfXjPEmk##q64k?{(n=Vn=Rof)GThcMpB_&DP}#$$xE*qmnK0%J8}BImS` z!vR}lLXnJv7{@Zk@Emy`hf^7o+0Jqf=P>RE8jD>a6JIl)X8es&c}*rf1DPc^gTrz{ z!|`_xn{A)VVRMvy%3)ga$))a*_U@I7H!cG}I`OMwMQT@a-g~8@xf7e?FW%jvP3%)( zle{^17R}TQeRD@-ghUaK5+ZypvjushOHdfLy61K$NGE9A&T*R%S%~oxJ zZ6MufpKM>H8F%8VwracmayezGi0eX|df>0o6Kz8?rLp+las`U-ClQ@dI+kAxPLkuE z!|5;lH9Epx5_fQ3t~SwTFB>a9+Mz9WndLNN5xUVeiLWgG^?Yk({2R^jZIt+;=6F9P zCeo~ee=WIT`;jZ_)94)6ByQNqm=%4EgvSl{cAMt|QSr?s5Tukf%ePaGt!k>z{mBgz zM%kLC0-3xU-6FNrD@ojNC76tFD(uomwM$+tbjGd={5WEoA`O z%Mh=T#23t`GoQFt5;VP%JI1_uta6>Cl>npl|69S`z>|GGrs-1N^8QslKNx05F8u)870w-WDAYr(OB109- zk77eE@N~J8c%31!spk)~84PWGDBW%;+!755(kwli_FyjzBlV$k9ioDMCr5 zBuu`J`AoJ+PPg7OdZnTaxoHJs5Ns1njo?rSu*GSR33McKf7=5DDN&-3=D9%z4~>OPSI zdRL#^_?D8d;^Q{Pr=!51VBVLGW57Mc3CCf6COf&gl6ac=LguTABthR2AUD)g>c_jb zO9Abo$Q@)s#S|HSO@d;>7K;+VpWZT<66R-{JRQG)Gizus^Rde%zlZh71lcKBDtQ+@ zfg(4O1(``QoF+j@W4=g8#ow6!2t2jr3T`qsqnu^^bh>Q=_b>}a_{fBdE=l5h2`*(D z3#$H<%tvzADs{}qzbW~bB`7!ha@8G@ggJja!o2T0k~eGaE#@nFNW%@XYbf*vxm|Ig zw-?9=MMi!Vh2A1Js`@9%)4_6byO=NH1PUaroMYbkw^XcVp8oWtVTbMSXFeCa4NI6W zZAxf?K}HPs7e`2cR~n*MlnjR}Su*70reRR}F`ry06@TQK8;e~h=|^3X1ik)G?q%ll zX@Y@k#LrmZzGH#!z0wffOefdKyz?GeXPud!fJd2>KswJ%iOfI8yzeqeoRwyjUrc_M zB+LSY1R{MhRKbmF8k&UrFh+@&$OuE&&<^GkmtN0bU_O1Z?MucS)iT^UpA!ezzoMbNb4M zEQp#Y1&?urYUWGskp$aQA~8`>!X=|*RS#tS2brh8;z6#T1ZAJ8KV1@EabI!K4#F_R zB^b>ShA>~nIW-rRhnUah1k6Qf8}m_IbLh*4JIQ>$DpNF@xJR&&F3Qg{)GZw0IVn(L zd9J>Lc{f~-QAU*nH-j-Vbdve_iIRVw^~11*CjES#SInB&%zSP&)g02Fz!7R7pmn{_ zvdXpUBn@T8$f|D5RkwusDlPzh9f#Z}%%@+F`Yk0W_c__lc}ZL#J4j!-Yzlb5-^LN> z9bPIyIuDyu&?WaSc*^LpXt`Qah}`GkX-XbrSw-81;vp=yu2E9$-_;eoiv%$iL1>sX zG}$t@(;K#wA>Z3&RUhSyzQ_9UmV`aRrGCW(sZSTOS zM^Lz8vCA+VeOa9~j%BPFqg?ffZrr=AC1tomCQ)D(3UYN`k&VL+)mbb)(}L8&OO$yopkF8IypYGUygf zYV!YYbA*a<(hz-YjNI?6KgXg^cjAqn!7n{w&NJNI%$qMs1rke>TrLaDx1^>S>eyp2 zZ@x7Z%n48LBNK?_x2Vh(-O0T9b`|zOhWm#3syk(P9`hahB7LJzvY~dIKsp4(7hRHF z%p5}-n9sdSTA0m-zGGgwQyMU)Z^(QR(WvT4l22g$NhW`{jmnRAw7KoqvLK2z@8b*| zXFhp~R5Yp%@%*teQUWDYrDA|#4If2dzRJSC#eC*HQn8;=|M1%}9grbkPSLE1Sq8Puri*-~?c5e@g&vG5S@ zd{FZK%+F+={uUZJvk7)GA3I6%Qza zJ6S54dykza&l!D{?bxyHq5@PgAICK@g!x!G?Ek&R`y2~=`Ld`dSJD5N&wNZ;Fn8KL zQ4~rbK1K4qIf1#%$1apa8(xvNGoMIbF@jrf{Ezc$$W4_36x49sp`|0;dJFf9;Xe9*Ht5+BElp7`I_L&@k{&*_!&Ydy=bHWK3 zLkfOMjiqFLz>|J5>zj5yVSPvc>uchFhCXg`RB(iFnU3=4SebyYCBky%ofdvGcuKfr zfGoi27P18NH&=c1uRx3hjN^Uk*=zZX2EuM8f~C1Ak^9AOD~s_H?O1U9mMJm1I)XZ;qp zOZ`NPemmwfE&R>58|nKgc^1JiM4%GKe=Kv_ovW^x4V|~>mznytQh&Oa%+b9Qq@CP< zB)^t3`Vx35m@~$8eW!SbBcz{|5gy|R9gW`Sr$i07UcWcpyUefBBVO9^t&v0->kIHy;tGo$*J~_r47|Q% zA29E<@ZT^$$iklkPX$o83FIt-?KU=@#G9{1hH-|rOWvh8mrI7{8|71uV7@yU$PtEP z(@BQRcPn{u#9u^`eAE?L;`Xes-7Wce3qOJRWP?Zjn-Mm$Akz||ocVkUABIgTWvIl$ z&ttyI!tV!9O_12M=BWKY<_NxAb=|oHZLv$G1S&4e8HK)>MeZ)<&9_SBw1clBFrUj; zhIHPN+!^Mp9+crD5|p9XLmK;kUw5fFi!E(|fD%rpL(y>Q#{am3ZtNBD)TC(%Alt)y zmW4kIp6nFyqk=YU#~p1)d}f@R?Ek(jxQhkJfifrbDO7SRjK=m;c3Tq2VSV$syV;bN znKxfu<+V$>2RkRDV0=72l&?ROcZ@9+HV&2qKIaJLyTt)AKZ@T}X{drr=EHm(^YOm2 z=IHae1>@L~(MIOuZoPgO zw=vdvKV`Co?+TvmyQ(-xya*~&Sdh!l1O{6SZXUzmpY;rTf>6gWdBkl9xzFqn+MB{J`PPKNJv zNZieQ3F~_qcF_NRXF)}@G&G+Rh)I+Qm{04CwFkDIVm_Cj;=6MKA2OdmP}*tFQ=m3e z+9`>Z{3Bgp1a2hp^pG>2J~#~b1{+#pMwl=8mCP41uRJ9A4{b&?rJngjzN*~A{DS*s z!piS*{-Ce_k$aB$MBMU1`SIx@`VVfiDM*nCw3W2-qiKkFbOpopet^JFSz&4Wk<9P5 z@KeAW!;kGeWbXfuvY_a~^*JqLLj@K?7nv`%@JfE=sZjEiOk=`qbiYk=PIesS7J&I^YPD0zLM?y$UHt3Z7?ihtnJ#(W!YOYtpPniCO`JfD$#aHu?koKt*q>!)f z*Ia4Hxn43oIpHxTzeV!q!_PI$=hGQ?xaW=kap#%&*j&lyu^m5LK2sA{?T|d*4pi

q`bp?egSwI zf1h!cM$DInD)8wxa+A0O>CBhl%WUMlxsBZ)mimeGnxq1Z;b!e3#6e~Hq?#{O=CXsV`<37o%9{%l@BFcRsA9D@I5C*XWslkOOP~=ze^?) zPOp;s?Tz|}FpCBFnu($MG)Jgq-uaj0y}7C*7I6uN%Fc6+^(TR+DY?WlGd{%niJheW z3D$SL%z}z;Qt-V@M>)fMdKbx0V!mUFOgPH;@u3IKq94Ny|^Ew3O@-=F10Zq;SnG18)pVZ)xW>mVD3p z4Tip1{}BGp0{_9+8?rr4G5wTq3m*WU5-`7>NAE3=OJ+WkZ&q`QD#uN}jZ`#tY?OY_ zRB0%g1sx1C*ffHtga=s?&Sw1*^loySB`BAeH{Uu9<-RiU32uU6GU4s4zZtx-|9ARJ z!5v%z!--$r<}0pcL8s-Jts_m^?+Owcq+h#bzA`KzYMp*6!6QLh;6tc z(@-Co@F$X1{sd16S6dQxOOt#7ofm830+}zj@He?w5HU z@)Ks0U%*5>B8!k#Mz~K&B-DuV-b3l3732(8dXE=5sM2 z8I?Yq1+TK83IcLMf^w1hSUz9!E%OVOk_Wf%ED1jgo|-Jvn85HdicLl+WImannRn*& z6&(WAc(X|=IwdG0GibbaP$Kw2f-mzc!BYYie6Zzp=D%Qlr{%%J-{2{u7cCj}SZ0*p zPjMf1eW&ZVOhyQ@@V%MuY2inLH%iPWx63#~D_B3qqMyV3c$YHKbO5cBiFQ^^u7cG6kjd>1#yE+ZUc-h3m~wSf(dcupEhwHTTNo=PU)ZB@);0?KEs zpJCBI&HCmmuP>X1W;+Z=hVt2o{(hnWaBQLT-!6^KP||q?IeoS254`FOJ}U)$uSi+2Rwkers~x_3#3>&EPbDk1lsWZmXA1;7PX8lCw zbC*lr%upWl@tjd(uZuXY{x3*F>73L4Y-kI3s_HUJRqtc{idVL%g9X683&!tQot}!K{7X-#I+aV(qvUCx6S_@*hX`+}fVg0C`QlI|B z5ILVs^uM1n$6`MWJlU^$i4;-)M^sW&lm#sCHFh+La#+&JG3H~RlqEaLyiJhDPsy|- z5CEPM$b4PqgbzR{4=}InlRUn!F2~=SEQqB~xxtw?iQKZKq1-2A!dna}PN;#Wgm+sK zzL)j$_e(qU=aI;rWj>x>WQIG)1^z#MT?u>?Mb_``B+LthFd^J$FyVH?^z<>)AR!## zj1UArbroT*KsZ8xhzN@rT~H8NVFe3N*?@=$3K3iz7SVvLf@fS&@Iv(S`+Y8-B8m!f zxcjQ>b$|}v*6&BE=Ko&3_v%%>SKZA_^<8UGKeyCRPaybvka*dP1ZN8P51g^M52b%b zoGX6Yqd0w?;7`)(bfOeF?xWY8ATp}e{1 z5GCCIvcQLtqc#zqyhq^f711|@*R2)!zDyyf&tHW6s+wB`fxj_|m*Vc6Cuwc{T<@s`#%>%cxa-Z^aoBr1%Xxqpl|$2 zhX~JEEIe_FT$;H>@aOasIDan~FAE4iAqjjK$v;eZAw~g?A6F;;gQe-S{4d_s4)O6u zed&TYH)us4A>lKkB#FOA(bfE`2p{6Xw~(A1pD^g6$ba}yh&oL2bT~;J0{KGmtEAY% zOC2X45RM7XDN6`X(v-QL`LNN6QHL!qp{2mxXE>d)CpWY%`)d3i zM`E^A;qlyZ;xDv>;(w5#Z#r|0x9i03%s;0cM*>%n(^qsjPaW=at|0L4c<9x0yc%)N zKj}5Wf48tpdY5o%ufSbbH-AC!7oHF}J$S8>h7qBflKL{bGgwV{;0w{gO)N(UuURAT zY>I;HokBq^DIQ4!Q1M2#J_zvZ{V*rJcb@XoEzkSP{`kb*8$G_U!)E_+}(r(`S%DxH+b+o#JQl_5IlUjK>OC9EUIyCtkdE$cLch|%i!V5`` zo6wdMUUy#bPjaN=F9ivg-W0!Bm2j7IO6^{95p}?iPD#aO%ni-?Um?i-@3aDEdE*xs zJ%-*y{Iy>Re%ef>rG!hT1pbh_DBdQ*inD?cbyxL1p;-D=;O=Gh&4}|kuJ$M{BmU|X z5s&;8iM%{VcnN(u6obMkRi{YMiylGBE1ZtMr19Xb2tVe*dm+v}l>D7g>}I-h;{Vpe zf4|_bl1g$#hhDUZ_ELw7o(}Jkf{K9PcXNC1SB2tQ+CJxqfk~5V#Q-w+%c`#^3u^fX z{3@4&*XKXHm_&lJ7$*n?{3Y1D9IoYmv!x*(L7x+Ts|Wu9aW1E}SjgdF#!K)0{BO22 z?@E5Ie*h64zLJ3WIKsDh@X3T9^56>yKjy(7L7W>@O&2DeWo`%_(0ss&f86gq9TG_3 zmgbGP+I9C3Ug*J>BhD36j1wc~FMi~uj_^YA=ner(&0pgUd}uX-uKx*_x*@@tg18XG z-xJA88R3O<1df}eQ??OaJ5%8FU_<(m6jXW)ZFG>PW}e{RC-NWu#{dzMbgR{t6g)?` ze}U*QMMRis*OFm9toUzFSn%Xg)_(sJPA}D2fEG*gYK(w|S?0Xt$Ilf=$AyB8G=j|cI2r%X#-rdm#5sS>3BkXI zT(OAwbBeEa^%~+2Jiz6NH%ajN6yX(BbkbS}j$*m9jw~g7iGZa&h;xGqJqq3-{+yG- zAo@1Gl>WZpFZS?vBs@vDGyj}bCBi08ha$xJhzm~%1wYWLW-d;@5nI58VEGbp{+Ox%i56E5$q6FP zJ=ey=KNNB2`rkdnts`})bqIK8Lua%NBYfx)VW@k?o1gGE2zSqXYfAX24+Vc6$!SOU zO2VBp;JE*zM9BO|beKek!nL5dS%Wwq!79%Po+Ezu@VLz+M?EU|w|MxA5a*u8Q|ug0 zZjdwo%ZN}y1dIo#yhM1C@aBYnk9ex|!@uZbeRZDrYnIhRPU9lsCp>twW2m27WJc~$o@P;P`-HteyQh~InU(S8SScm~rEM^3{Ni~^tY0W}#U-a&PRjVG}~ za~x>}9W^CpQj*Rio^{1pOKytp&sKitYb}ozRk^5=qIy(R+eP)ZQ>A_Tu_Wc-kyhC< zu&nQWt|D6bAdaJ?l{su#S=u8_R3w%;tm8Nxw(52e~K_&*fYH=_ENsIu9HAJHM*a5Zq#B^pwKMr=A{wHp29$7oqW zna_3eT_1Yg8#Uv zesiic|8caOFP^mS^{syOt&;lI)cRJkzO}l(WxK78N3q47=r#7gU5>PK^{p0Pkn7X( zu_de+Q{S3W-%4W38A93{L@o?{ZUO&Rx?h^M;Mjq?GG{5TSknxvJ4}lMRe0DjYJ~xNK7qa_YHvAY&&OYWp8ZpXJhu9ruY~nD>ubqglc5`XM}ZtRPJ!|4(O-N;?5k7=34*3TCzkoMFHg}Y zvdzExIyWfvycE0@0XisAeJ&jf6@yE>UO60vx&ev#*RjUI0DWJE{2b z6F*Gy2enYZjFd*gW~fxv0)pMXHx1^>tX&%PV@-;oEz@p<6ZVfjXe+bL>0mK+9dx)n zzpPU~TpnZ6p*5SD4wLMZ#xP4}s|#=tpZ@IY%BE+4maomJj9V3v^0L^pvhvBQ8cj?t zE1&29Xtjo>22|Aw#UsIBX*ABh?1p~m-xRvDo*B@TooNCl+tma#c25&9gzRR&`1)|_ zU_K$;+3Oi_H!E)nW7(>oePLE~&R3k8uZ6;FZ&R4SI%L99_G2b|z<$fcWr>Ts13Qw5 zUQqlH&58sYbmTf2H&D^RIF~t!c|Ljc92o5e$SQaV=C@ z#AXeIZ2Q?P*sriLmwajLU`yy~uWA9q0VC-14luWotg-RAFhbVt^|^3F9zOB+qm9V) z=`&;6+(b+@|0@y8D;}RWK5uZ|K(57{otQCmZbH=}bLQZ(m{=Mqo1Ta}?u)9X8i-n| z5s#>`7`voG5zFX?Yj<{6jEw7ZpfOv}4Vu|wyFnyP4%oYT!vddTS+=UdBeHDR`*qly z!h>TT+c^~mu+3py{Z7m7hhcFz3_~;Y*v#1pt)gGDVcDOG*+bXC^!v+tYn3xw!R8&r^j6|m=)SzLX>Idx< z*LxM~`v6;70L^cF=|*^Xy$goRIGUV3Fc z06Uojn0rP*dsZ+2^5vj21F|kE!^r{7qM8}j!-ipn%~DGbvo52dg`%6RO(DFQqM@yP z*5nZDyamRw;(?IYKitr4;~GOTg4f{F*vf1e$W9Fe9djwRJF}C6Fx3H} zvF0_znmQ{(IKhHaSs=t4)taVwM#TN3}@6 z(Br0~(fZGQ*>`V1Mpn!+HB-|f;Zh@E8fv(RC5tej{#FDZW?H3TD-a5qiBM3-iT`pa ztY(XgKxKPJLT;C^%7^tIk_fAQ6}tF?rLod@ED=(pn9>qvko7nS1#JF4=;pTqxD{xz zK*-V)x}j-o!!WpmukhQMQVbpJsl(w$87F?$NL;wN-7#q@LvUy7+#SN~jzO3yS`EhZ zGkz&sirpkP3|2B!01$rjtheWM}C^I;!<*bK*_@t}%3YdE4M45pSr zwh{VBILC1zz zT`s!5&jQF|D&{WJ{XJ$ZiIlywJ!9WEsCDiFSdGn^h zU6^E?cI`AsXBpGsx)d!C;LxW7aO-VgiFS?oz#Z}$d^#I719qb{!){a#Z^(+Ov01|~ zHGMx5wT4*oCw#yCf(odRm0*y`voXPFqB2>_*-(Hi56{NYR?LPia@cM=2j<9%7O?+R z35(?(TEmIaG_#=@Z0rHZPSG$ej-3lPx6-d+sjjn^=fb;+p|a}v{7U9-Sg#m5D}Na` ziPz^ru3{M6@@)IJd63}IP}>so%>4Oq7(F}?cTPFXe#nIz>{Sckl&pjT_M(Mwr{c4C zbcU?Q?uT4EBMHx^DmXisLUZ^|n`WO_1h>gb*jcw4D}MlLQCycL7~GQ{#CFKW_7GXY z5W+@p*6tw~V86c<1|g)f_6uPMyKNbKtynr6@Gz9KpVmPS`@@GJ)&#}0TMPZ{$~CZC zW^2}C{_ndEEw|Y64Vde8uY-Pyj!R()bhkS^4)aq|^nh-!T@RyWY#H{+zrahfY}y+( zK-Vnf1r5=+B_eZr=u-Y-1aAR}{^#?|dFw$*#y1+hJkPhQX!lR)gu)ji6ZKWji1b zhyAA=FiJ6T85KiEhPf->?p6a6WyLhuKVN|U?BENiJ%roYF3c3`cH;iObt9&;-p}Jz zS@)fA9nLKu1e>`FdSaWi;oQFcC=6v=R$^i-UxkdtyCC;kfvaC5aCHn^XYh_Y-r9xk zzzM&M#9xHxai*L973%Qy2rptxR=)(z?Q|QOV!#AhWH(+N_1}$`LFiweNmI-a?%+I( z9{U^YV-Hv2%D8tg`5L3YnU=nsa*jkbbp!0S-#?7?=IzrzJZF#|052Fzf~4nZ%) zRGn?Y{`L@TO1GE)0|%hsx!gOCKz?KQ{O)5T?2||MU2xYm) zWK%tIJIt~Tz;*>3&K3~`6eCf(Mk4f7vG zLGB^iZk_B*H{q_J669)3M`}yeW7*WBU^aFiAJvX!TRopo^3U$$sJgMN*D)~LxF=>G z8!Mi3o$h(1AuEnSRyv)XeP{vOdJID8%LU`Hh3q8%NDpG9BRpA$C;;d~@r*`<%6cjGM|2-2=EShHQ|!;(*+jdJ@gHuV$e_HV>W4zd6M diff --git a/posthog/user_scripts/v0/user_defined_function.xml b/posthog/user_scripts/v0/user_defined_function.xml index 9a0b6001786fa..609b4d91dc2f1 100644 --- a/posthog/user_scripts/v0/user_defined_function.xml +++ b/posthog/user_scripts/v0/user_defined_function.xml @@ -1,7 +1,7 @@ - - executable + executable_pool aggregate_funnel Array(Tuple(Int8, Nullable(String), Array(Float64), Array(Array(UUID)))) result @@ -35,7 +35,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort Array(Tuple(Int8, UInt64, Array(Float64), Array(Array(UUID)))) result @@ -69,7 +69,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array Array(Tuple(Int8, Array(String), Array(Float64), Array(Array(UUID)))) result @@ -103,7 +103,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_test String result @@ -137,7 +137,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_trends Array(Tuple(DateTime, Int8, Nullable(String))) result @@ -179,7 +179,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends Array(Tuple(DateTime, Int8, Array(String))) @@ -218,7 +218,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort_trends Array(Tuple(DateTime, Int8, UInt64)) @@ -257,7 +257,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends_test String result @@ -294,7 +294,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the 600 - executable + executable_pool aggregate_funnel_v0 Array(Tuple(Int8, Nullable(String), Array(Float64), Array(Array(UUID)))) result @@ -328,7 +328,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort_v0 Array(Tuple(Int8, UInt64, Array(Float64), Array(Array(UUID)))) result @@ -362,7 +362,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_v0 Array(Tuple(Int8, Array(String), Array(Float64), Array(Array(UUID)))) result @@ -396,7 +396,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_test_v0 String result @@ -430,7 +430,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_trends_v0 Array(Tuple(DateTime, Int8, Nullable(String))) result @@ -472,7 +472,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends_v0 Array(Tuple(DateTime, Int8, Array(String))) @@ -511,7 +511,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_cohort_trends_v0 Array(Tuple(DateTime, Int8, UInt64)) @@ -550,7 +550,7 @@ This file is autogenerated by udf_versioner.py. Do not edit this, only edit the - executable + executable_pool aggregate_funnel_array_trends_test_v0 String result

?PS(?aegW88+YQv z?8p6is)1izDB^;6ivP)lir?a@`5M*WSjqOqLi^Gne!}3EMN{K&lo%7j6V*Uvlvo$S zhjEN~X9$nspB)mhp*)r^d&@o~l)E#}a#OB4;u7|XiD7(<`%h{YHO1xOFdoIKS>?DN>=l6&cCf>3^quPO9zDLvx=PjC7nKEm;>ChgGJFOebd?d9S&EBGa zI1gzMR$F-c9x*GN2e&>PD<1^G=~AxLNJ@A^nXpslT`->`!4+B=CA7HH;>-*{t?Z+#a#CIgIEvfS!9qPzyeY zFO3zGTkzKW@^1U87JN2qjC5KRh-VTLSh!onM)CdJ6fdqs@yGlwMaftwVo7qLAxS1; zJ$H-8qJg;vx#ez=AI)2PmFwKD*=|sgiB-|Ce7zCPJ0Qhai@VEi5g)^6H<({bu8X_G zz8IcXuiT`tgtX4whacJ{9*gC}V>CY%Hr&=NyXKbg5>!TYTAQm6thE3U%X_r01N9v$ z@|!x9wT>*W+mQ=*iP)Arl!Ku?)sheE^%m^1>a;W-Egb_8IP8kOVepTYepNxGqA!(+ z*DUfM`G@Wjms=7uRJ)+>F5%ybcc_b0S3pd?Sc?=!MY*ZYXzsX6%xJ}#6pO4XJ5tWMC2RxFQzo@O((2&-t6kVy^U!C^-ZiZ72NX*^>z45K85P6K zGfv|Bdi;Ho*g$oTv;n{{a3CYG4>I)O4YaE9I zM;5Tg5EmeQ``BQexVvn>NN&ShM=L|OF6=3UF&t$bdGPp|&`;g3Vb_DJ7+vh54R>XH zcd1y_mX}7A&o2xDO1?0E@w55O1tBm)Qz2-fziTLdeW#e!jt9GG++LJ);J*Clonl)% zKAt>F1h?noOz^Zp)hDBATXv!!z|^wf4~XLE%9wShSPLf;l>PJeJcYhK9e5|+uvDaU z;0evZM~Sa0EdW2YVoy%ImiX+%onlo7-rB8Pax|5rvuNqzEJzuUk63^FchdSZ-miA) zf!a`AdTy86vGl}Xbx>Pvw0)K-R>kvBzu%&Oyh>EE18PqFv(J7yoQIvf3avZf^sj7)+LR(9LBArGIuE+Dlg^bBch|mO-mGRfq3!CV<6nl9b(dBJk~26 zrTrzI6CmNN{L(?O@iG2>!@3~n?6CLi0u1n!Dc&tuhh55;%m(s$eCL}z<+VMfQ)$ph z9GoNeb>%TGC8a9I^jp!52XfCHBB&eBaw$2B*wx*@N6&8;C%W;1pl#mrIV|BTFdUF3 zlZ*i$4JEhd#EN;{k?+IpVq16qfWL@u4=l0$wu^>l-a%)gJ+_M!Gw-jj1~J>kT63*b z6@p64{2WgzwcC>T3p}kvasrrak=U8SA9GVg43xQ@!ehJwaWBme3$-5E<|!WIgm?powVSAN|h;52@`KJJcf5v!i( zz3r(3p$mAF+&O8BSURY7-eY#vh@Abm*gXdGLyT}a`7Flx>0Q(ABz zQ8fgU=AB~UF%;~$rVTT3wM+6ADFbjjKXWJw>bRyBi_eDg?F4Z4b3BQM7mIz*@%R{J z`f3emOod&gPPC<{1V^P4@>ekimetnU_IcEbD`F|n^Cvw^bXIbCvmH&X#~xFlEm6tw z;byRNgA_j#e&@En&wjMo=8YQpee!7{w1JRR5)SIHU9pXS3ii*v(yKg$y+fW1sSIqFmviO92cE}LNk5@AlHyuP@~n+Z~=h(XjM-=csg!Pgkwwg8i$v>HmZ zQq$rIdE9`Az0tq77(R?Q7Q069pIX*!-mjaW#l{kr>LDe#<4DjQTSmI?F%3#XpaP0UT>$=qQRpQZ6G$@OVL^j8+d6E-Xw zZt4b_BnJg^U$p~1M;gwUm7V9`h6%9lIT~ZG)F!5mMsKd6*;c`$PWGuYpmhhE! zqG69ko?*!(Zn? zk=VpxSygj^4bZIsr36?7ow(wENphn|QSdsi;Gawub7u1JIM-M3f)!9*XK21ku|JaSm3hhhgmo-Ln>BKci89+bc`@5c&Nh+;i zEe#LWA>S{PMaV3acOVbdtH76hYw}U~@AUGHy5*aoe30$*WHD(L)F#I!i^ce+9InYr zeWjlKhAU(bw0)wdr4sz6iU^K(h-`MmV?5<1lV>>OHQ~2qlDJB6H%%5Evw2X5^?KPb zzZ%@sF%V??0BO2MI=YU6b*IFb*xEjHHlM;=d_RN%+JJGmlI zK_0MwKZmz=<^3{*$9z7L|1wcboR6iyX@)pGpD!cY!>*z*hCI1hubY$}r^+!x3MVP;B7``G`xm2`>clGY2MfX&`KDI>@>B8vnkki3( zTMVVAXtkBhiI*ZIIraC0-4wa1I9PnM2v?m>2Mf2w{A-Vj2?W~5c42~udxrU}K1eV1?(B~D&~#8fSDq?YLNHec!X3OIYC z#a+b4xA}*415b;>rI?=!B(Z%d%AQPDG5Io`FzG_^8Sn6!9;>zZ57Ncecla3Z_960e z^k*q#X2{FAXO~K1OfJ96ZQaEv8E~3w`NI`BLE?rC?2+jCE)uK9tCie_VJVs`3g6{n zyvxht;JZB0rRpr&;iPOIYuWaUR~xx3k0+?fC-ab;ttBtelE*IN&(N5#FXN->8@!yq z$=`ZO(5+)`g^`4)~?{Ay}ubp$obj6R=z&AGvkE6 zfYt>NGXzE>zQ+VaTzs#*$74jR_jsnq%Ub4)aVo0i@9_Ya@(eYP)uV%!+)hb;UQPMy zeZB;4HIXMLpD*_)e^DjMi5FF*9xveQJyvOvE49e_EBP{ySG34UFRGOtUdi9{NYWyE zXpxy8V3Uz4i3=a_XqQ~*H-+aa9_fKwYUsDz(v%3%Wfh;wL6VPFfh6T)wc5w3Fdtvd z&D=g#ELhEFc)zbD3gzo-`_EX>un>czeTaCfkQaLl)6$Kkg;OYT1wY>|)Fr0`-=sW|woVhU0E+?Zc#|N=`p%YcZUeE(2~!w)KJ7`2-kGmq&w(4A#T1nB=(-YzFAzmqPKrTjs#NdbrReA4GZ| zJuIo=$MtZdDB-rAhge30E9E-t@o>gE8gOu*N3AzKOsSVRhl0n)5%OH%(Mk_5X(7+W zWs_(<{8gwTD$>AmkKqEc8 zLq&*=(3J`XM=A&!_3#fUkG(d|X4LdTtlb~-*r-#T~<-l zp4Ey$cqzionohm*+=o1DU}ZjJRG42Jfdohv7SLwK68r6_b~U8i6u8)Y5>m73fkhEb zz73Yg0K%BeNO>&2^Wkyy;vr}%&>G8F@x_NcE+j}BDyjGy*Z-ZZr>^=xl)VdFRK@#0 zes<0|fJlh2EVv+d5fBj(5s+|2L_$=&q-17@W@u(+-f!UD%!*a>@uOyFehM}->Z`7r zC0fRI^(^sG(~7#<)Qw8M(CF9<+CZS|8oyDfP-(vX{Tf8t_j)8 z`_KPJ_M*Kw+wCahE9UD_Ja!89nlMcADKOu33xXg!Y%wqUik)XEzh+_1j+=h?OoS`r z%o~H-fuK&#hfOYg9Bbi+yDPZ;8smGu0Kxq03XXK)I2Zq1x<1RibqD)hqbWD%7h@%( zDW}Nqn0N4bgyWZ&^Ei0iT>t4D;^+5rsJb_ZCohYpe#bf>$olVC`?i(s(F`!f6gsNf zH8vW6R%zHYv0W}_wc_Gn7l z&pJEhm%ZQ3_zp_`kM}##-2LoRwj!JKKd|5kd9>}=9LgpZXAt>)74C>s-SY>n8)!VC zWv}7Esn==p59}IyW0ZN|kL;<&=VgfgJ}7AMLY;>-Nh*%U@t${uqfqy1Hf5Kv0G6Fi z%Su=Z`{`9GD`ETCq%pMeAWLL5*>wCM3(YO&R|Pif^xqqWS7b4HN-}h|O(mhv-Au#f zA69?uObft`>QDHaHoKWtz{7H-i*}}Ipf%^DX0!FltkW%Me{7D%+serDi5A5B@W_kd zSPR;3pz&ge1v3X50apm74v0k8Yro3Cw;%!WMw4imjF}31V+*txT*8Tj5bCMg5I4}8$ zw=2Fww(%zx=GfK8s5WzKQ=R&W{p|R7B+rkBsd^-BJ<9rq7h?m<_!(>UAQU1#VUZAr z1$ucXLLGdn44V$2n}NIM5JNfNKyy3B-fiQUrFfLUL5&u>UTcoASR@G3sbj2z>qc7w z|4|XM&yjTPa~yUJDrMpJY9CPUMI%bta8}|)-;}cX_8GQGNGR)8Cb6qtgAhy*wuWhHS+{3 zVxNzufRpTHYr=C+vLOuRd+H?S7>lqIc@n!2$bO$Pd>PFcO_R%5KZZ=2%h*7kN&8dG z*fNvZ0k)4(c__u!kHDFeu&WkFo4FmOs;U+Qma_r&ss+=nndK}j!uB!w_V3ugaJL@c z=FM-if^CAw$}Vg%>k9TA zD@~*W6)c!lCDPprESWDRlG{1dF!qA`ox^xpl1RDdumr43q>VtI@O?wwjtz9QrM(31-%OJH(=cA=FRAQ;*2b5;{1c1bbS_gNm7e!c`YpJZ z!3|@6KHnxspOT+sh*Y=9AC91P=dqDIbQGOA&jPxoq19UOty#e+#I;5(BYuR!tEOl= zx@td+CI@czj+-+a1psYT49eA^yS%951=gOuGn`T`FdwHk;b_1%9EP@hnsR}KkBl6_ zH{qaviCJ0{%zUhAwnxD3_xut~t*IBzN%Ez`L&Rd$qsG0tFmu{nAl8BEbtD1?%8kS6 z=>?V@JR0zUqJp)VYIq2 zlxu6@ZEg=??;_A}dGlWDJy1}df--J{$7lLsFPpQku-zJ;iSWQZ^DP0sl{DUF=Lfz& zdzoT>#p0l?6TSB<`074hf>uz8b1f*CV7 z^zU{>fA!>AR}Mu@i#0|*wov#@=IuRkC3i57sZzuuH44ERMfNb- ze2WF!{Q)j#y-de%VQu~HP>Q(C;-fLu&370h8{bD9yjiIB30_8i#a-2xc*kWdK33UY zhN${d;Pd@_+H{*`TI;a(HjB-8rk39i*caEJr84!o5ndWR&jJ=?ycs+g!X*HQ?U2qw z9MRz8vhae#K(2p24Sv@qF{6Wvz@IfMro7)+x^5o&0X{uY8!BSqN3FLFrJKL807nnx zgg2_!hLZCg^lw9kQv4k@)yh-x9oEqoN~;>9ThsMYB(&>2L|X8kU_9U}>>U$ksC#_~ zJ-x$x{6=u5@RhmJLFm5I*Rb~j?9*Ba(MT$cPkSi(F7tE!1yr6m8cP^&EWZq)@poAl zb|Z%9E=%;l#V623<9!-*I5kGi7>zLkK`5^h%|tCPsfro2YM&{;$5Gt#}Aot1;vQ%N0!Z?IR-@{-R3HN&}&9Oaa7mRp;{d+7ptgX%72FOpp ztD1Z0#`&Z*3vH;C>h5kDG=x&_v&qgE23zIwAWqH)4WUE#F=TZgOc4(ti~k)$*$>$0 zghKe?y+X@2^!ecb-G(;X{B2SZ_e%iriA?guL(ov@k9vq!|0j1t*Y?L?>u0n*4_N~1 zF_>Q;?L*q%Sq8hC!S9TYpsL@Q3v0-rFMh}LV|fN=L1*ePm|q+n#_nYBJEQfaeZ)5T zL?chUkH}#LqZ{5<%t1a2d43^BzhzL_BXo<)P(F_^eJFSMv=$%Q zQ`fd>6DHI;;P;@Lrl=EI-&?(-Xyc!lhFw}tyZ>Z~(Q}dB6EvkK(1+s>v#8&W{X>nV z!4CWgkTDcXv(Ot*$+k->Dh+PMU-hY`kjFTfv~xMSCa8#};!}t1B_eeRE~q8yh9&r5 zG}4Ts#@7eo1J*!XhAOE;ERVrKo+&k1lW0cyfH)Q+V12JjQ_@q4#m<5juhDnx0Zjhj2=H`JuufZcY4BnG|ex2XpT`= zCVuAg*%IFWrs576>&h7wgOAnLkGL;nP{3a-JHCf4)|PS^HRylKW#{J9RdhGKJBZ63 z+SDL=^cR*%!XR>e%3jo9OX(@gbbU0CGpjW{1oB+Xv?FJGjV@M|jY@kgW8F=g5mhO~ zj9-hxn--r5T(yO}tfI~|(+;as_#?k~?1pj+LKXq|_%gvUAJ(B1r#MMA< zD(ww>Q=lPP`JOF|m92c0tw6ze3;idb+uws<{#GLQe*^3?d}%LW;@32#j=A_055ROe z5bXvo|AqN^yMcY(LfTx1_FOiAj@L1-*Q(m8%6>&jLfvzQ1JyFgf37LqTqcSCEE6-B zP|HMcwPtJzMmi+%IfRv$@i)sUsh)N4D{j_T3PGV?L>Jj>i|}qv8=MTq#4+uYSXy4s zk{qzA;qE1L<4@)m)(dPu!Sy~*Fp@TxJ4VL*B~ws8ydW0i5H!-5x15|C*a+A6(L%c$ z_6nVoiZCC?*O5*%vw?M%zCk<`*}y!_B@L{vrmKs$T7HH2^2C*+RmJE&c$orGC)u*O z1?(%h(ogB9iR-a~Vw%`{eEavSCQKR(uJlV28{Gd8qCE&Lo#r;U3ic{Ymu+ouXC8y2bWXpwFRv--W?Aw=M?L^ zVGj82p-!{MNUGX3SGiX03tK_uI)yLb(5RQue2uuixnZ+m3u$}}+w2p*@PD;z93il_ z><#JWM@+<$q_n^FfzmDb=05@jTLC*#-P>xu;cTzAcqS0^gY1PU-cGTl6t=&=IY1I$ z5<8VAmAs&~&;o=&i!Dz=)8KbjY-+aSs~)$n@2}qfY2Ils-j)nq()i3?Yf`(J5=S+L zA)xX2Tha!ArrHgyj0W(<7dd{Q@mr33%uk)fcjed7PdP#IxC)7*X-}#xfM09StSTf| znrV5QX8jnoqnT!KGFq$A*HqL@t8D$8wk?&8x{8z8=l!rC5_OdBCML60{b;qDn8s%J zqib%W4|}(t8ICV%+`vjJqud;g;Amv2=7H|wY<{%FmV+(*=!{N`Vm(sKb{^sz8ci`x z@)W1p`DWnd$rQTdDehp4Q)pc~@ust5inUfWI8EFwg_h~X;jC*4UDJyJ&Q*P_ZZ}Xf zZBxkEOZ>?BoR#+N7|`zbC6kwkFPDAksFxVeYLkh1i^KJI(OXW(Kz$o~SSZ>2BtLtj z20P(nQfR8T_@W!O9K4JLw=m42i{+YXa~IGdZ!v&*&Z9ftVrM6HO4-U!Fyb;C@@+5r zIJeas`83smw>%7_{o9Kjyzb9Klfca#r8wwW;vmFi1tb+|AoBiB;q*a!F;S-ml8RDM zsb~ttg^Rsw{igS&>+MCTF7ap-M8kfVN5MW~1lvEChWm)|t-nPM1wHZ3>Pkhx^>6+=7V zH7{>Itu7A5Hmv$gnTYLlOY;M4z4(}(PwBW#OX`fwP>D0vIP?5Ox39(Jk?z6v-^`^Hh zJ7MSf6}4vW!lzw)I`exU8t5-}VtW&5u0JY%c3(Q{FZO2d^&wq=nCS2_*xQxlVqAAk zitYFW5%DqcPoO(rHuSJgLDf|>XsLXOfw$AX4-jMQ=J#!?>Xu9&UzfU4NTArJO|jsI z2NLk(>Aus4QUW2={rb@OKru|$4jtIDu>-L1mC#H*Arl@ZDByk(C`P2@B5FS4eFmQW z@D#4KCFxAw$qH=|adWVZM}Dht1^n_Z8oQ`4{a}5K5FDrR}Si9V?H!+DuhM|@DCee&AwA`C>D7cfjBlz>) zJfPmR1=z?Jsr_vA(ce(FiMt`}oZeL52~2(5+Z-G&W{YgJjw#g$%Cy5H!C$}h%DNc3lZ?HxlYM{if z;`j(I#F;ou-7Hmn@x-6QIQTBMK+ra=HL8OSjbK{Ri_UaKOZPI+)2?DLYil5%C}hsj zK=DyxKyDu5=Bq(|mxfjOL1&r^8ea==>3HW2VJ**i6((xdNt(`7)5}_dkAa;Kwy=fU z@2Z>L^Z*zHx0yH$UfIb|cNca)*zqK>wnbdIxJuR<*>&LiY6*LquEM+wX8z%=6p@GU zNFY4tRllC53gCi&F?C9H!?nsBKi3$0gYoG+KA00>p342VsIkM`-||?)-YvEO-Qj5h z4}e_Hhj}~?0C_6J)U@^;UfQ0f-SC?Q`s=-EbvH50xrp0Y@47K|z3YzlqT}7f2;BzI z)Z0iMn_r&Snm*}8&e7<*7O7HTtTC0 zu2}R0bJnKAf|1ueZHpE&rI%n@*If)(4HVg3^maUg0DM_^s22_IE_URdOKx|uD_fOl z{;s#(O-vMjggRss$0h3eRwU|!Hm+^ga5 z@mV|S>YkL;TTE)R31d?wj^|?rTJG?2qIpek4E!v^aR}0fSWvUW9muq$)etQu^NJJ{ovNk@z$)}5UsBcGDF^w!SUwJDVVBs1pDpOeMjDh#`8XT5|#~m z^ax<&Cfqn7tL*hj8a#lgJ6 zI(O6BSnC~a1=x?ChU-;BFkj~K#;f2bxbe+7zQW~$-pg@h86YM)PEhIYCi^%_8Hjn` zH*Yn|k=8UPj%EzRsPkp4IT#nAYTU~UxAu<9am1VUd@j~L(b!F$YGjCUY<^$z9W0)6 z^itz;HFeQ|$17x@q1 zp#*nESs#M7jvR?S=>8Clk3)LUzM*3O)(fLlEqmmzm|TX5UeG+wf}SE)Z3(gFm|@}q z4Lch{yI;l}dcJ|!a4{-k3I-Bhw|p<7`GsdeqyONI82*U5#|Kqygu1x`*CN)La270k zxVVg6i=q3&#qmD*@KYcO-Jhbq)}jCR$EA+kAD^nUkDHM_>Td_aNN#9?ZUtA)ic9@4XV~@wTfD za^QCv?7UAo0pu%__VjKhWWo?fA7_dk>=`DkNz>x!`%KZ_YAJ}LThA@pcxwGhvx^~~ zy1mjIW>P%8{@i6=Pg?%mvaKh5^V|}WKo?#S+gqb9jHia@mhyP=9Vxc&sB+-MZ&TnO z<~>ZGV~V)VQCQ1ASa2a#-3>iy>`3t^Yc$}!%d5>yf%g&5E#N)(xdps``D!z};Qj1# z7x3Qj+ydSMM>WR=-uuJSNli2)0Ze(TEpRcI@&r>D_W$(Iezu`rT)08?gDlag=KLW` z-#@oF^`z^g#CA58dy+Cr{59<7X!IVxKtLC_H8OvGW>zrG1Fh=4q4*@BQEM#^wdc{D z>_(%1Er46WPk;O8g)%hSyl%93NE5J#2RvXKoe%e}TfpsK+U#HTuA380bF#!C?2TwD z%Mt@TM!`L&#r|(5?8;()^{z(>8Y9-QsohAQE$)uG-3>8))5j`A|I??glH0$x*$?!o zyUOjy`qVi*!ExMQ`qW+QMr;>k(4+^-0GYK_owsVhS|$&k@RSanB@9&4Av};!Y%=o z!x!JjA}Q%zaW|XOh3enM!fbLE3Y#h>v8FCGd8!y6upa(-w<3qgLp$# z6%!BBoi22Isu&iq6#g1){;YAknO=rJt160@>1-FX?me+19(|0XUem;YfQQY}i#wDN zr?(`my|GxRAmR~E%SQP zXWU*AO~=~D0g*Ig7Sfv%L0e~u5m7P7EH8k{KF&scvamKD zadx%QQejrzAi?V~#pBgnz#lZ5)wgMNHX5Hm^Ej$D z;R?iiNPJ=)>7C5xgEmJ!`DLCxnTH2 z(EZt{@E0N|VvZOPCUHM&u-~Pzr2{ih$I_Xn<7G;Q`$?49i7w_jbHpcND_rWb1=Een z&gK~l#Q%yt%9F5{a3kEhw77vDTrJs3!FUxkK7#X^QlU7ymQP4Z!AZg5(X~=I4Ot{6 z+9klxPn~JqB5@r%*@+?+ivjjdpnZbW+pNPSOOqFiNp>}z6q?`J`T^v`VljtRM^nra ztgLQE)8r+nqzhrRYzc(m?M}1<=r|Su_`qD$iLNaX_c6y#<_$~5i5m7znAvW*I8eiS zhSBg9Sd}M*n|G`bUvp?(F5vDgK9DzbH0zA$I@sio=<;guGW)6{?Op?~pL8VWwc?i6 zl_wdjl{PL&q1E-adoPe*=O7B+CeC2*2GZtjqE7!-AbOCF zhB_!jgcXXf*5;_y^=j=Kfm8-}_IjZC&NeYdWbT3FQz));eJcQuY%wjaV#df0-SIaA z=xCuhP(KuOJ`2Ma^FLSUKGPRs+oTeTjROMk8ip7)Tnk3{RR@KOUsL6RmqY<%Bmv>L zI#JcF-s;S{ic3CbfQ3Ll`-_GC+2rtzvB(e*3&U|xX->QlE zgcqrFt?3V7b9!&6>A9J<+T5yu@1nN&GQ(`QU0lsr&Ihz%2O7o556owFh)-oU-j|N= z6k~!jeNh?MZ7H;1Y`~&y80`FtQ{JU;Uih?pkgqv-mx#B|w`Eu#66XVewPS~B8UpNM zd~SscbA4#TZZSsp5&ZFyxdd83&HhIF(w*I6cH&VVlqTj?Ji#oC)Q4c_3G(mTTK&^E32s^__a})uC4TOv3DAUS<{_GgjZ;JZB$ov!Au6JKu{D&THCRmqUDaAiupi zV86Ip!?r9p@BTsD%5;mok;z{Sb)JiL)-Oz_40ZFpX>y6UAk2QCb(_xcJWqPl1^DM{ zh^LrN@$rDK35I%;?w}YIRgEpPhkUJq{N?e1kgNmY0fgZ_T0Q*oH46{(3^z;0d()hQ zVt{K`xbu55AO6S(VZEudH|;nme%Obf%`skrVgR?+tb=ig1qVR zA?yjBhTAlpw5+@(8J%rzT5tZcZ2%2BEcR?&&HFc=y^&tD;V_27R$lbuVX-q`HG8P* zojzXllsi1uL$Oe7-wzI+szW5=&TGyA0?42U}&l-Xz7-JNr` zu#K>*3bSjzmgOlzCwRdT^y8=W^u-Y@+K%bz=n=6Sn~-2`I3jk?j6R5(F<|uJli*cq z2CcJws|0$OmV?IokLFc?)vu>%A?Vgx=C3idGtGyc*CcOVW>p&9Om%u%a8%U!JprS< zkKt~(JwqL@9)ZT2do;qS-!ACPQSseY#c^VS@d?&ylzt3byMG-re|$`=bM&smb}e_( zwu1HxTDbcdAWu~Vsl?2pR?%lxlbxTD^9YS3>BhQQhjAsh&f3)9;ok+ zZLJgo^EhrtgHc;8T5lG&e`&LSWvCm)?L%$$5r(=<4`S!UFs}ii^G%ZHn_}Q86*Q$! zD|KdT9(PV0$8;Ik9pbspgLa>_MBPn(9(3)37_VEcvqn*f>*ZJ6=JKsIeW)`>Tol7J zEJ;TrFNtCL8~spB&pQNn({~6o5JlNl2gNMWCn^;7mig(x?Je@-&+RSp(_Tl9E{Xes zpMh7dScU=UItDi$Kfg?Z$IZj;Q^mr=RN_wASHuX{ojiOXGd8NPe?ZCFowi;PQ=B)0 z&dZh;$VQ|EzYZVUXt)qU(vsn-=){(~n+IMMkF)H-JRWS_4FWdJ{jumtYwY*o(}jcI z-kGI(q@u`*DQ*1+48p5{bcgq|q+^6LlFFrIS5C^Uc zu8TqJOD)ykL~Hs?OLwkgaQ*-xtsMVTM+MKu4XhMDbW^_&@#bjVkI|WAEj_9fQ$qS{ z@dhToX<3V0XF7_Ati>!C@jQT9yemQDN-R|k}JFHX65;e8|iOhS{~4kCf^cYX5g;9O{^gE?0##Yk+SliBPkWy_9RosCNb9R00+!5ru#EtTDOh^TqnoTvnIA`|S7 zsp+QZ#kPucIZzrI_5nzRHZKM6vYdNS8~Q(+%|ekz2T8-@-}+aSZ$9_0HI0Kg2Wsi1 z=v26x6<5J|I)cwJ4;Q{BnjZy88ys2qTjrHrq<8Iu@EMw~mL(HkXnn6(Xe)*oE6qWr z?FP-ahqP2zihAKy*s`?_vCy>@QUG1;A$fP)V6k|lb^W_y+!4}RF?h(sVo`sBMWTeq5(YO|%%3Gm?KNU6m=5%j zUXVn&-eNx0PwJ{+d+I1+ixg#!eo2}q_H18op&i(u;}VCw;G^~57cU3E?VnPJW>U*J zb2oiaXE8rWmlleSjhuphq`uZXBSRV~vn5Z>XI_@#HLUwnv-1dPyv(lDm}iWV9MMQ> z%!^)=zGl6u|Go`cR28xo@-Nu@Ht56r9!Cl1NT~sxLCYC!>yxp(2<7&PHanIV9crj-jHF|}HFRf;l@M{hc<9~f_zz7RB+c+C z`_rQC=5n!CixLB28fd&9&HvM!^tSYohUGjqpP4KzW%jrtDBk$fpXSl;N}&v=xdyi} zfAF4k$-&X`yH&y(f2YawBs~87J8hmPEeH#EWFd`TbN8+_oq#`{)HJo$bf%YhI3)G) z@02oMN@EWnnpe-4>@;@qa5(&kHZPENu%8~%z+7o0+wqW0xza#+!^7t#z6M5=a4|1& zEz+`;IKJt}8(kimqw*wIjpI>x;^Kegp?Tm!X_x3${qI)aaNpYMW0y+aA$=ZLJf2{U z(L7N9_~7qWp8zxGcM-6~MZVjYN_mc}xEqpL`G8`UNdu#v-G?-_knbnl-a@|r;Px3d zdB^hk!F}4jOp15D3VQRL=+b?%ER!OfPqv^PzfbYYrId(0ptbCUcfxM%Vzs7kVa~DG z;;uK;eSM#HESEmAKMndw6bJgh*Qj8Hl)zf4Sx@XR_`bzM(<0X=;Ip}}OM_Vz2>SDL zY%68Hk0#djK5c$qN^B})7x#w}2*E44s=wKIdkVB)`Qw|u(%pM#y)JmuiLS1c5?s!}h4*WoNNLMI%Rlo2 zl)pdc1m*90pKg92r3HKl|1Hb^eQs}2{!8y!`oT=o2YQ?{(KgS<8}& z+S}f`dmZ+wDhu7MBIk9|NV^o+PgT+6b<&LF>vvF#|0=7?pv^#8mD;#1L0O&S{{B@~ z<)Hn_`wN#0TzvKO9ZDoA&1E;Jyqp?=OOdMeqs*my;v}>GddUkHIsax(+#o>>*8ey2 z=#A2SNxX9l4E2*b)2sq%C;R%A8N<&TY|Q@}0>=CwL%^%IEtLJQAs`#HyV%^}&GDbZ z6PD;Vt?b=Kn(8FUJY}17gt1OH%`sncu5R5h4=q7{Qp2dyyzFb~p!hnj z9BO`{fT?2=7{bX~j7f|6m?RceL1*$&rOJ?P*2dS-(2)Uc4mM)`FNTcQtqjeqG=K3e zdU@$Ec2+oNq2>>F;>LQfRCHi`f3Bb%-VDNY>m~efqu7ff3;WW6UD9yfYQboB6HaGh z!Z91adFYTf1qM^6-O>P7sG~W%B|UTUqjkHbt?aJe9B-BmX#BH+p}WE_+VC)igS&AU z*_V`bu+cQ8j+La}EBV;_V^-mx7frqPN_uw#?w$_hXIqimQ$msQrVtm}jwbJw-eOSG zyaNARO|xOI6y4?@H)owd2@p)+RFMn)&iR>37)|hcfwOf}iSfGe!TZ7Mx1s z_Q+q&cS4azxTQes`Idl-9KV3I+Tm?P^<`l*dC`Z*8f*Vy#a8w66Ye0hMQu0_TFV$S1KJ!I>;l1zXQO3Ea`C0sd-e&!VEGN?tB(l$Y+7L z?5KGb8+DmB{452sF;3<07S;lzhTIyJ(>`c))lK&a1YZ zJ#vlHd##&?CtQYoi9R|h#jto+I(iZfx{E74J}H&4HJ9m7nKYfB2gH>Jr=+0B^3dK< z#w?s439~5qX*c|gI9y+zgE9TyS3;2qhwbluBNY`b)|@U7Hzwqb=}RAf)}C;vdUO%$402MD1?vt2UMnGR$Fnj93p@WDFY0j`a)lF^BTh?c z&i~9deZ$?5y;G;9u5L#Wf#77lYYB3Tmf4Nm&PW5>tO>*UC6vuAsR$2X71NY6k{{dJ zhE|`E;@#FDRDtaxFEvRdb^VMKZDrcytQ6%u9ZGw*G;tw_lWRwgI4gBvZ?>TYXCe5S z!}Q>+G+I^tt&rNWV@@=_Lh^52-K^)^^&+jVkcMbqzrdx}&D1P7uU^3LlGjEPD{en; zq3_#hI*b2BLwwLi)0?)PlWw!&=V|46DNM`&wO3n>JHI^;n&Hu^QhmNNT|O@bGwlUp z7o-68{do$$APqplPrD%Xhsvv}ystHly1s{k?!wxVI%PNS zlK7_h=V+Vv(3xX-ozG8)pijkA-+DUaDp*(PYjsw7WgFu`=&hc=U|I9cn>hM@Nh)M> z&Qbj(=?q(U9(y@bwomvuo?l;6Dbs$SLzQMh;H-MBI+N>78hAyDXS#E=@(OCO;Rczm zND;m6qa>2}inqiUezm4oVdr}~D$Nhd`>F=9)-)e<&N!ceJK&8Xs9n@uL3--=iQDQj z9cQYkyPKw;qmQmiQ64!q`*;;SOw({+;VWAR^TVrBy2jDb=EcDpMAuNkGvIEjQ_&hE z5e#3G^uC`u@t2;t(wCijJT4J>(fn*mE3U(845P!>q>tEinTB4MA{~nF_b|k1tV$qp zbmY2}9CjIT=0OFJk4_@Yyr(fh17%I?S~E6=u~T9_R})=9QI%45yH0jDq->U0K@)FCF)rLIFFT&3 z>b3ZW6zHaU{fqa{NI3^y{yax_Zb<$;iUk8dd!{kfb`#?GQgt6#$--k)TF^E4aiy7^ zi+dK$0v!q+SPkh+v(HlEO^iC%&eF)6sNa|hnsZYc9&;Ohc#+-&?u6EHL7?U|ujc@C z2}$VwcV3wbU041#6|Pl*9|-6!w+gbm1qm-ZOJTR9;rczcICcZI$S&84uW=| zh(&d0Xy9!rz+oOD=PiZGZ)0mK|F-00jknm7KEI9DKmIHozbyqjU$Ob)d)gPzlI1oQ zj6=_o?l&pGG0nzk>RC$qO^SDnQ|)f1+_SXoH)*Q_gMAvWtH?W&SHI%-&_3|g--CsC z+KgAN5VTPJrupR=lv%V!6_uxp-Eh_pSEJxEC?U_^cw@1*@p^!!t~6LvcRfr~SAB+7 z-;vTnZXk@q;@4{(6lJFXrL13J{|k+~dEIjfcAf@Fcco74@pMM>LRM$`nJ4!pUiKW? zyKVyal73hE*d2=6xT67|ma)IR)A1$s+K^(ZzAKgLkDtcuQj9ho3}}V+i4Qnw3w4>A5Fh8R5u4=bv{@av5^NZ#_?BLCAg`jMn}t}h46zEE`w>|N(^+Pv(M z6r*9S%IU{Hq$uykQ&_Dr?Sm>56%-umdSm-A(1;x!E*VbMlDGF$_}~TE%;0_4bL@m~ zk^6jea~XTYrjJ4P^1-Qxsey#W*j7S9Ps~eeeaRXkrayea0#C${K07 zhWVW`-~3C8(y(J?qS4_G1WY(xX%-z*X!Z_WBRTzjhV z1U+uRY#`<&g*Bq9zkPy+G)mvG5huy634S|q>J@}$Oi<6#KX7h9lL4ZWWy(D~6q_oot-wI$mDr>W2p zNB6`9HF-lmg&(e3QiqPBN1jTGf&iNY_|z;iBOPb(&G$h6Nr zi>hv6`wa<9|GZsMDklBBKT3o@zcgFI@W+eu&*NqvN&ZmdTYYb_pW17kJ#HaG;y>A2 zcASpO@@zMJ!qPui$K*HWZfvCCc5)hfxq`OZ$pNh2aXM-zk7V5|D8OEhXTcRT+Ftfy z?T(ul*vm}g^DCNiAu{KAGO4Jh0ZMdF@X=9pgVLqeboyu7>>x+6pU=`6j{DD&T`PG3 z+l20-H6r`)EX6y?dbaRqy4FfIdAw?i%i%OGdAlXlrD7Tcbwc{gQI24H?C6xEJb`sV zpw@B{^EpckT7#3eNTjtqnpw_RBmLuyHPTz9JW?G^a*{{oj{n(OqNTrE&`bBO%ZB}} zXBf!(KtH&TRdw?!Xgwf|2HpcfHSbrHTQ}JGZiEIsQU%;G8W$&Jbx>P-0Ni2EPb%qU z>;|*y52`w+;E#XFQT<|Fftf%Kx^-WZuGtf+>L@(%y8={C&_rqTJhB9%1_}DYu5C^n zk?woWOeFHlGa%kXlmlM(S%?on7m|O(r<6I#o>-=MIDFm%RhY1io z^ZOAwt>yuam?DA|7>ap`wELMXFUY#lzacDl&#?KcgaE(#zCuf3zh-l*gd9HkLZP9c zU9!=t&~yZU!I%NRP^{H*spIh!V1m^jG$_?-%Z}4jXE}@=K2DpRWqtPjQVX>Oy%3YL zRyY`kSIv#BUJ&ZS!6y&f+$9LG^N-#t4QcB?wSEiR-ke4s@ic$orMV}pLA+eB^*@uo z4dm;WQi^ERjYJPjx zUxB#SY_YoVP**Kgoiw~17gLRk?8h_$>Dq$B#8QfHE3XUep~0jIqgUZy%C5hz7>Z_M zwpRts&MK%UqpFJq7d<^`E61=rJ#})Gcd?LCy6K8m(!riw-Q*#Rj+w{1$?KSNG9+|K zL86gg3HhLu9Ch-*6jf?%*Nm1QYnI|@%sU-SCz|cH>q$#tFVr}z*ORskcjy>x)*->) zj#8OUUgU7(C@*ujFpBh$-RW%)*~8JSlC-9#qx6x7tarJGqTn}NSNWo8^Q`ajki*@2 zgF${?wm{UJzKP-OB@6alJ>=Pr^QWKuYmRg%BHa(#&0`dxM}%KBM_6Es@Y+vm8JMpAM341|@T|>#8aNjL z=lm-|Cqy_Mw2j9o(Muj_uNKe^Kl;o|)-g>xGJDC>dwi%y0tIMbAy(ngcvBaj53%?x zv=_P;V*dlHm;BH%9dI5_D01MPCYEYrz2!=6ej2DD5MjQ~#i!z8ES_mZdwGmwm>R9S zDdZ{=m$(Qq6>h>V z)!Qb2*)v|1`#cBr(4&I%FQS8ce&!#E@&v1Ro_MPn?EgFw2Y>p}Pww!~WNtTDlfj9; zn6)poNG7CZGMA9dcSzdOf^gkWs^@uV(SmvuRL|FzwHP68;u!~fC=1t+yf+$mjoF)%HA%0 z5k>PMXO%jsqr8fx)>C~)`E6X1j*SMs*>`g>Zd|CR86jX@dx*XZ!4&4!A-WSH2RpuO z<7VR_(uK;y*@HTo94eqHhdv5kY&@NKZ{U;h@RW2?Huqe1i&7QvLB!_xs{3{wiTwsXR zLcYJZ2R-Q|cVJ@=nf2lFQahL8wTE1cf4$0zx}%T4IZ*m0N_J;)2k2;&Jlb{80gJ~? zbx%{krW&R~2PmeS97Z|ath~NL<7x_R@1pG1D^0M5~obj@(?c@|Yy<61u znCR&|7W1S$d9tJV+Dh5c->zzfq7-5AzBzbYY@(`1|O5l2TL8t+i1D%%CE8}#6PzX%LeCHy!14EY9D|x_KKKC|*dju8& zZv&42!{4u0oN%LJEHD9`Uo!Bj9~c1E0?UC~e?f3qiG%|LApsbI_|^m01FM0NL4uI6 zs$R(f)&TQ^as|QfBP5Kuai+0enbZ*;fliQ=bZGa609OIifkA8Ql`LQ*a0QOvjQXTr zxefbvpn$E~Mqml(``00Xg@O=C^~yzH0dT(oCAq#{nFRs94crdQ_)op!or_gnJ`7V9 z34-^h^-6RyIt8F#iXc1$1}zf=m(S~!OxU&CkT5U-SPaYu9)WxMf9sWs<$_@N5(%Ii zn*}_P4*!MqO6B{gQeeVLBxphvzbOc{z;yIf#b4DcxzZ$X`gOfx0H*IiolF*lO~6&_ z!ANnvG7om?TSNrA=l|3zRj}^?u6;)kjsUl9Mvj0bz-c>C6_BTGKqptwfqua7UG+*b zFcLTgm=0V5%-L0+tE`9N;%;z06&&rUSE^y(zZWHe!99N;iV_&|J#xg|fe!D1z8{PN zSNs4Of%|^oPT0r&Sg$k!4cL1t2YnAvf>8S%tXDFo3Bsg9@DI#PY*3sq`S&*DHYgb| zB;$&jSa@6o90m-3sXMs&>XK$ zhCiU^I}OTuVCK{Y1Q zuy{j*BH(04>Zc9LBw)hk2E}D2ssflFn2Vf$)}ZWxgRljh0sTG)Bj_M@ZUcw#==t9U zB@d|mqCqJK<^Z$MUr2?h5@10Q7z1j*f(!&hD7QB#4zo}Kzzoaff>M% zP;ht~bl^5%K5*7=4a!xxPq~Alg+1dgqj?;!)|2`}D9D?v{v{SG_O4_F9X3#MMA(T zpkD%tz6d!7W^G5tUIe4x!2TkN(qU(#5}Sy+{jpJ*1}r^}k^oi%tANqJpzeABPc$kS zzzpClVDULH3Jkh{92y|yKtEt1FdDcYxGfK(;6*S3T>A&AU?CS)Fbv%G7~ygkp~LtK zMS}pFfTMsePaBmLz){Z-5O}d3403<0NjcFQ47P4kDq*j2YEra|Av&&2N*3^;wn^C! zT;$wC)+ObL54niKV1h@@Y0o)G6 z>Q5*JHUd|LG%2GnK?w?NQc@8vJgiAs0($~*Junrx9XJkH0-Oh&wFL8o&WMN8yEG}| z`hpHz1zg*uDOcGE!*&p^0*?SsAV76wlj6A)33hE#q!d&QFbnpKZb$&QC%Q?w4J_&2 zq$I%o1n?rzI|d0ZL*2(VDNg;+?t#g`YG5PiC*t4^dsTdslD-@n?wN}s10f`#NqGo7 z@nVy*77p_go0RBOBmm3+Rsq)nqkA#6Vxt4|HGwF!_D7o&iltIqb!`gHT8PkudNGuo_qcbjfH^ywg#nz+~XGAx%mi zux2Q70`wlyq!0B@6=3H0CPf&863J;&rU9!b zf^lGO?OTW_19dnVbqN%vzyUZ7xE_;6&&8-apms@<5LL9fL#P`NA8?PW zML7bD)maqJd{lKOi;{tG6~MJabCF<#MX3Ox5GV~pPu0buBmn0D^MDtDIS5!8iPb6W z>$_T%N}wUiqPV<_Is}%$J-mlS83wx{&Z6uAI`y(BPDW&)uSL-gNBk)kB@6aF{VYm4 z>^p~ofm|3$vMoyF2qZWb9)Kr+D}d>5Ta=Jj(Sc00C=Y>C-h;nU5Jq4YaLF`_vJJRx z84^Uq+m^#0?8z%FO6qH1;3Fge6!46u%W8z*gs7` zVBZfcg}w9wWZ@Iw#awuRVcI1m2;6?zqLjm9&J}oAha$XcQAPm^fx;Uo;@e0F_O-vE zD1kM=5P}fig-pP`@*y$+`>ZFBq46l<8ZZp>tg|R5Ko8g8=1thEMBHHu+?gvWO30fi zdVCt51>A;PxQ+mi0KIcCS~)7pJYZvM+|C7@%ov#QM40acUP2jV1iCjmH-70 zMcEGA54;Kt_k{n6ND!C-3~vX2z#KjN0doyric$-M+m%D`EmYu`kP0$1f==VTK~ZUWqaMU^H+)a2&7#xEA;jcmybI2BSc4V9KI4yawED7S%Yfu5fsJzzL+J5ZBMOApJwVbiu^cLd_Y|6K43$1fi=1Camu z;b*{a|5hqHjBUcot@QY?92T|*e!M~Ff6d`aTF}r>s^LnvQpyo|ptAvXNRzM!dlpU_ z8eJiKQFr`v6;AVZ@zL!Ya_KXiPnr+#he(mg0`e9`=$4VQ3#WYgK;@q&$=T?e@cRHHH=(lLPE;!(1NIKp#KE7Gp-Bon+^P92C4w zWyj<&y9Aim$B^Zi9B(%d=Aa&wSSqL3mBL)nlU9}@hHBN^Ks_wT11}yw8C_#fEV*dQ z9qfWOmCA1R!$5ONXrR`9b}!nmhsE>725MuG{q0M*3oN_{969UNAY@LWvY%z0os%6} z?j-uLOx6X2z+5m1J2CKs>`h6v6+;lW!Y$}UI(h}lLDazaum zTnZ2>8ot0zMl(BHoNvQz58NuOZk3v5N7y;L5~QW7ueAcgVZPeeS~K>;U^xNHAU`{Q zyLqsb^rev}iFmL-iHA8cRJhm#hO(Fl|vae$-tS+t4c2num2|30t9p+JK6n0V$ z2*6d~LL{O{h95AT)f@%SKt9|IX*Bty9A#Gub7neyaZ*mSb8y+f~9mZlYRuuQkHFexj|dw-8NU-=3|l zZmBT0tkF{x8wbfLjgR40QTu*)3oate(i8Et==`5rSw|z|Cb4jX#Btfje#f zoUF+`Zza6w%HcL{5bZdHqH}_WDH(MClpJAafO%F1`IJNSSHV1KC`~MvhXmY)xn=^o zqJPDuc4;r5&wrV&m7^&9V6Mz0OSzobY8=d&uV4>|zU^wKrB_bNzIFxN?<=(6w47#F z4YS`UDmyI)+j%>qBYBPLPb1wNm_tUZExpwen6q*qd*pvic577t>nvC^>0lX3Vn3`q zVdXhOfj*ZN-f%9$t@<_EeMTN=7v2UP@;Guli~evJ%uB}e5(;qSC71_$&3HW1A*TeK zfcfg1&n5kNe5&lh?Z}(-=qv<)+g)5Bk9d#|akh$u*$MlA<#Z#|&YNaeAgA*{TMq%L zK<)}*E}TinE9CKZ+P3IWW>d^L*~cyx=9)P)@*Mcef_chZT67K_Pyx)sd^&UvLV6MA zJ>#jKn_XN{x^Gg{dH9coc~lOKJTHf}%7Hl(+k$K9^m!=Fe12Z`cg#c-+dJvR#w)mT0hhzNhT|ksbmNt8Nv|8>9C{po=k>h7}qciI}F3HqaB7h zh9S-lLnp&13>}7HnEE`Puh-+bwX3~vpWFAJ@8@;9ZP#_(pTAzeuGjUte!i33ac1IJ z4^Ywlhds=$?459|aFctOW#hbuKF9vsiuKa*-8gi7mWg`)@$=|}fpSkX2bYCoys^#EZ5W8EuwFOWduSWRg=Va8p6tD|E&3sM&A#Xh)4eOVN1rulJJvI1;7V}7duNE7 zbo&N(FK_S{(Q$5SGNuIwc`08+&l!}5^+gBcvT~Hngc7}VUtq{<0IWaMd+`e#Bs;O* zd8qf}7w}#&7KO|7?)?&Ti0xRPG1vS2OAPZlDHxvTd5gb-PB+$9WqZGW6}{Fi+z&(V z0&i_Q4s@HbJ_R4uZ;w99oq@4)`(khU*Qkk=SYLXqcfr><6jWoqGRJ%LYxwWLdi`vtfld`+w1Qxvf~Q zKiAv(KNLD^B6h<(Z{By&vj-MoJ?ng2QD%X@gR>^E2C&9RFZQ+QQQr3NqD$O0lTgZ) z-j&}+UpA-{>oK?hpBU*~9N{Kd4Gi8;Enk) z`m#_N*6VJ^)o|IXkYqfr221pC_d-BkY!1fiEE%h1Xp%c6D5vxw!<+gOvcCfBttWY7I-`dVDuZ$nwhNy1 z*8LQHt0`*oG!*?Ld$hrB>MX%_`pKd{%3X!^+6He4cYNwS`!hOT`T@}S z-246K=rJcP!Ft`c-a<-caWq4yDN`zn zf@}loT@E;?<77R3k>0f(vNhl1heL46D5b7h=Dpd4)7NyY7h(Pk-|kYop7I;k3$ecA zJ8!{n=ohtEANGT{_BT|}4y@86Rc!>=5$SCd#T{7h zj`W`089j4g+HCM>+(+={HlcUN?}|=ySAkWY>do1OimAkUhwELuD|*h5cC5Dy#H|6l zL$n3F&pty3`Dho$MY+q?6zlEWg<*CL*308>pYLAmc3`!8=uuA$dn4WP1D(U+ z7I$_kWDHL89v|Sw4MIn8vcU5X^J1gi>4Vl_z3pfeIKq2kp}UWFPLzA-pjN%ENm3)wq`v=}ir~sots4?kKkgF!fsR%4m1qz%Hye ze~7zQcKH7!+QsR^Tnu?zJlAz+N3X$ps(0J~cfw%Fkvtbvz7==LysLkU9_npy-6VHA zSkqST3D=$Drp-fJZ}on3-AMzMVm+<(Pw&%scidhw%ohXH08ING7h(|CsS}Nl#d7wb zTx@q^yZJNkwE^yOH}?qasoT6sLHDEq^;mEC{!j18M0fu|vMY1~toXruG3d^7GXN6* z#5)r19qtya7w_ebNpi=Vo)$b36&vHdI}oKx$9hK0rr!s;cSX9***GR8cy|qUUvx{4 zMnewwmd7IhTd+QBg!e$KJ858W0dg<`cOGojd>8Ac3|a)X1uQwi*5{l!)MqJR%_MJm zoO_bnf%VQQ-g9xN{EUTYzVw~DI$f(gH6HC=23Ro1J1gED@0uHGhkG02-39KjMJRiw z_es1v$(0*wOXqsAL)_&9tFT@+SFYlpE+gP=L)-&g=NRny^Sn2QAjl%Dx6i{J90r+_ zfFM+@xU4lE^U553TCElq+-AP`~P$-rF zv>1TnG1kjZ^fnH4Z*Z3!i<&&m8#BzE?UrJ_=rnJ+thZo2b(weXFn7M2oP$&@KTy>h?~tUGx-O}m?3lyn?+sq=Bm!<#$D9qC03cZa)6z=A8iF~i;Q0}8Re zXr(pq@`oc^H6V?cd<=JQb+eC0?d01=mTpym?UujV?N)48VY}iA@3Il@)`7(*U@W~F z_fEY1gP0XvnTX2SF6<`niA3yGNlVaoZ^HexDBDD_Bi*Uv3jmkgwR2Z}1iVGoM(jQ! z$WBlOc}KZ8nCg{`MDCgaith358i{sHJ`r=3`@L^Qy7LFD#QK)|ckXuSvlIl=)8(Vk z%8gjBxZm3->z!DyoaenL>!~MU9&x|--6(hJpru&PdJy-3Fs^QkPW2Y-Z>%7Iu*oUiy*Wvz{^R)}F4?u0nNpWwayC!4QwA==dL;~2dV`bQ5nlROH`OfxOUv?39qY~>(2VuA z7xCI4=BnlltqY(9w+wSp9F(Weh2&wUqoH2%J{jw#%+J7jQr4bV_mW|@6tX3+;>M`; z(Q{CW**9M1wI8F)IcuP^;YV+NihI1WjythaVQ*B#IR)(9=R&_AG^Cg!St z{28dHXnar$-DVcntE0V$RIKM>y=Z_p3F|4vSl=)Jcf@@06R|C~atjCGIe2uN2CT0c z;N6hw9vup-4~Vvtv`@^wj*KOe*`=trE#SfTWDGz;JulJA3ErTxm^)+djgQVAHYhTq z_q6;IH#=*uK}+={`@d!zBRa24)4ogFKe339<&CC9IOSB}Hgv9>90 z+TPNPa<`{x5uSM!;WIdMvX_Kz8t)z$w*+HQ{V{khb~R)dPWJvi9_?EbJ18>g94~T$ zJ2tis3R!R9L0d8Y%9}pHof+E=*7zo#4i197?OG1 z!`|XFcTAjvU3tj}Jgx_M&`bALr@05kWdU{##bsoHXQz8FrMaWy@&QYsnS(AFJR#lt zHqAY3NIhWc1VGUof!uocx8UYT&O|ppwiB`y`{Thok$sCd2;A{Q7h!!0rh+Bl-NIi= z_dcEI&W&9Im@ySkTnY?%4~g;&tk;4U;O?2>h&pYOJ8{Sk!1lv%nMsrnrI(mqP9a2vl7)Lg?o6K_wfGiA*TLj z48$q*NIY;ZUapwt{V~-YH#7|!DtYU&!ltm~BHq2Ogw*I(`LJvkM1oeh6e zuxBi}6;*-82)?wxY5MV%1N!tKWCuH{1c!3wJ!ndlIC!@=HQk*T+XUABFdj*jLO$lL zO?T%fBn?J8;mKgrvKQ_TOK7(%q56rElazVa10q!g}yPqhJn~8*o!8t{TE7 z2+ck?0(m{aJ!rbzSZW7M6pK%%;6KuCC)wh4SS%(t(8*y{8X! z$H#6LRsdV;|y5;_h8jqy#s58)vxV;~K!W zT#qLsWem7(itLuj-oBIFXs>;Sn;0821Y`6@Tz8XjH+r!Lxr<}-z|wB%-c=ziWlV(fC22AlL%yf^8EdcCz zlsUtJXQn$Rwh^rPF1-9cUT(uUqX_9rH=)F42oe$=rEq-pz-)$HcV*mIInY>b_~-w*s@UgS6l(S0&mZ_~8U^TDm*IyEwxg zHKY`hw61P@2-`J5>-OmkH)UiSWEH>R$y;gOS0Iy=726T%eRc@0)khrWPK}itO`E%M zolWv}vp453cV65Eu;M6OT#%BUJi)v9FqEnVu+7D3zYHk{nk{yYp5j*L+VGZc@z zN_h{T;2l24Jtz_1Z*peg8cCzH>(HLq4fbO6V{!`&m}3IMaS@T4RP5sXapRv(UBH2^jM<_i3Df_GoGJ29>cupQ8h z4zB~wMxRQ|bW`K9lQ7&Z{&QD8mci>7^0G{KRBRbY8J-|*6-OU>pJd`1_0yTy$y*^v z#}lPdQaa4SGu@eS$)mB0fGt8>1@QwAra2bOMb70m$&PinASG1}$xBDLY2Mv)QCfN7 zD{(!Zp+%1jUO}7h#AetS?63FX(NBpGn&8D9;ZFCCp68B;TM0?~{eSLSBlU3T1n-i0 z*v0AqyB@~{8fh_9n+dUP9>y)qb)C$A{JBdWG(s6-k8l^ntpMBJi6>=+-8SCacfNb% z&>R%99)k7=oF~dab?kU?b7Wk6GUh&bs?@+Z1Jk@ivylB2kfe>lGmBC`|Hj>dOq|!O z&B9Ee3X&cP8;X#%5sk!k~0>~I13LoilY(Z zw8P>S$ne+_$O`3IQ;~h;-Fl=uDXt03nN9carCRr29qCStO+vpZ#uKTgXg7Of=DQ1q ztODDSiIWuxxiD47f?MXh6Ji@7tHN~;qrb1{&rgt=z{g&koFniQq(uG58*>!STl2xH zz-Dbw*{;b%yiyuDYo-=W~1|_qOx*N zL`_J;Ka*k`{@A0@I0cYoorxsT?$9msSz`lI=F=&W!YlkGOhlJ@q_&2h)LQ8%Hows|+?xKmG`h4oe2@FYBRBuQQtf#re48)E;- z`U>z|AIBr5`{V|zBAvW)bhDz!Af8n$iNucMWu9hQLy(zw#Rca)MrbOCVT^?3UaTkOrtN4Fq>|K)pN=Y{xvW`<_5>3%0xN z4Fu~@oR0ef!Od81c`y*uAYT5CUvS5=>hFO9>$>QS+E&jx~0w$^6@@uhIlx+xGJ>crfuyv`Q7=%0c3Hf)zY zhuE;&{#+nY3^qJ(w=3!ctr4boy%b0kg{p=?YoyU=v>F>;4P;lrL-lKcxg?><8|0>Ll!j;eU`Z zEUW$#7}f;6Em)SIt94-6hQ1Rjem4-)ij^`f8>KU0S&wC-EO($e-U|dbAaLb-ftYe^ zH(*%o zy#vdlErDT$Se9ei0{hx6=U3Knzyv1_UbxIc)=i9Rr>A0odsqgOvgB_JKG6Rt}1(8x*J+6k~&XXPxSHdv~1X zemihiAee-^f9dJog3~eCUboCW!y9tCo8g^vx|@86$)ZfQY6I~_5l&+u3*jPyWf6gr z2>LQb*%gR)y7t1!#9lKRW1Q-k8Qv$$+=-9Y>6Az7Vi|s4=Wtvl*YrumiQ=u z*7!wU=ydmvO%s>9-w)jspYH}E!u|Ik?evn%+@P0ynHvn%tTt8XG;^8rBgPP<#H+~H z*yY{lV9Z83*KRsjqA2@gz5E&raHug!mnF|xKf}VvH^iX@qz(h;TE{4}N!j>64}X5X z0&Ks2_`P75{qCXv7#Pm(`sHXxa&#aK{5s&G9zic}0z0A~-UgP|51$K`+z*dJjp5|l z!J;>1Vs*nNTLUWampmGRDbVwC7d+{eEcS4 zKzgvB-vKTi=I_3O${r<+Kpt|Eu_5U`{7GSsWdP~TlJ4gAG`s06$F3(?dy`yitd-j- z)XVf$fM34T=wlbcWi*q9Ewv>c%1Eqo&E%}Z~FO>ILVXXe*Tv*9G{Ce`X4Z`_RaP1A%oFR;1b`lgPr5} z6-6ke`<)JCzmQbJzJ!^wNjbi%9(`Benh2h1w{@@@%I)!b3=A%`3c9x+LIo7Sr2m`4 z!>O-L>DNDza_!7IDCM6TvVq4jko4G@_{*n(@3lWvZ0qn0g1bmb{0OTTxEykl6&%F+4J3eqfTn z<(B#6k$2f*Xji(BavlF+?e6#87jD+YE6))T%Cq zpuY?W@2>)#13yex+<_6xBfJYQcSna>Z^aIclzbbPrFMD3`8L2* zBXyj^u#|z{&u53>QvLnup9?N02=o(jG{7Z*xPhR89-a>-1CBrNWnfaJ{qgL6_+P-J z?fdGZ%dZ9OFF-B0>>hm$^_U^!z3gSe%`UG3>A5Pvs(g*qq{;Z-A72PABb`6MsbI2q__@@Oj1#+Y z6W@_D!UOsWumuJ#Q@Py?443nyqp;S^lIg(G;F}yDbb;-}DaOiiPUg1GSZ+9yqTry8W^A^EWY1 z5Wa;ak|vUL7XDF4KzNfY{c=meBZO7PJgwyGLgg-u+xljkweS}s-ZZ`@wR`;v3|`~Yzm(f;3~==Yb`}~sl0c_EE4VF39!dV` z?nVC>`Q?=FWzF}nznXF!G9-nZaFg@m7tq*9-CWM#m9zalTSD2bB)Nm^?sW!Bu5-3g zI9kb6t3N)rheJaRVtWj{8?JY!hV%r$A8;Mw%l_$id^b8sj^9#_ ze4FGyEC1QbLuSwr4gVQMEYC3Fto2D%`f?N*4lIppAjNNonQE~IIetxyB0Uh`j3?k4fSq!Cp@#p8+U^0KdjQY7)mn8Ey z*+t*2_I|$!lg(;3vgblc^oC{>6Ca9q_X({SWNXG>)2E=O$6rbSqT;-75a$d+f@k-C4jOMlZJ1)K`G zpUeE@VDRBj+Ki4DCy4LJ6)2e-^buhDEEve{?dMm3OGozeZ3ry$#ta6~`NmS{ORf0% zGhiVZpw6R#9)&LilVi*p@>y7yRLgC#kupowL5UG){qy9-o z+j0DQQ!uVdEBX0!^lCXBIZDn+k!&1vt`?L?YNxLN2Ul=8DEj%I;L6m_g2e^#ve!ejbFAw9VaW{R5-&5r;>?YWqKTStxu#3(rdM}vlb$OV%tlh?LMzvCcrE2>e*Uz-4i!-7fgW*63Hvk?#Isih$eUHpk8( zETc?sCHW1UIvYR-+5@>Q-9u6?w`G@?u^?2{slZlGx`S}vmMK8w~sE%{`2zvS1;L@^zdc3jdj^8`y zSKBr7c3;}bR3TBBZ@ISKR)UT#RovDt{0q0W4o1CT^|caLBfMOB@bj<0q+-vt`XOf_ zHY7bo2igJT$SX(bFUV!+mn1u(gc|aq4S0k=q$S+Wu$$&OiR|!xE;B?qB^vRP*kinv zONx?;QE7)@X$8If7I0~?V~l}4-x2-S@F&wmKbMDnM8AS(m90GD|BuTW@&;MTc)yE+ zgr{wZQn@^U%fq=`!euU(7jjw1H%t?NN1S; zs4;F^ib>e3ED`0I+}0iKd~VBhKvE*@Pt*HPcLFBkuY`)yu9BYMw$8@#7J4__?cytL zM_IY-m6EC!*bL}!8eclVaGj+kn=RY%WHr6iODK0kffu}6%iXCd*>kPj8eu)wZtDRj z`#|sWa`)oEXt~mUNhqDs{~s+w&i|vpjEg?=|1<;AXaE1sK-u%QZxo`V-S&CJxIyf2 z!Jyob86MFvK<<$b^F~y-lSVf5^dr-a>Z0wDGVjMtr&hRY10n5hwv*Zak+`j*y}B=t z=_zbqKG;duv3oZ`B>JxI4ZC3{5~ubJHq&;WS79VRo3-P0-IJB#A|4JO&HwF>Cpg*X z>CA6;ef;S7ZGB5F(n!o`b$7Jdj30@)abJNIN8;W~ol#g&`^~)0-ZJxgRBb}-j z`pM8}IsW+CNDL!!i9NT$yX--CdBm_a-n$RFQx_~+v%5>jc@~=ym<%Y_nf5ZHwr3Ms zE=}SssC1`}N54_il~Bnsqj086(d#6Y4$tWtY$q0LF^misOoA9TF8$z`%EuaKTF890Lo9HA9*s?)PFH0 z579s$nN{xovcmV0K~@7=f8=`zIPfg0spOU{V+^uggkJ ztSPc?y6CNt0hDW$_0(S>m&cJ*f2)i5;%LfSR!~QaW!Q^_Cmv^ko&?=)_U;6y$d}QF z!|=!HXVtaVkIr>>8vVh}st^T?>&W-EO&gO}QIKn9254iS{<(%HIF-H({Vfa+fs6l~ z_11By)fU|Om;qFv8QMmJv`4I>>K}zH-%nm-;c+Te64+KF&<>LWA*xO{!{_CmR9ICG$*g z>JeTLhUbGzhL+x7RpkpSl5RKpz1&hfJVyN$mw-tBFQ?>Z3X-PUEasSz59b!>eG;He zE|ek7ina!~kTLIvLE}a%kPnAQIs&s& zF<5eOuYvIA$z$ZHF{A>Fl#=GeS^c!*?e;ReaEi%ioMZJn$=``Xd0 z!VAu|yplYRJo!V*tL(ygg}kxF0(+3b=_F1_#ZT8-%jI^Oq~pm8@s5}Q8sH8JO4rfA zNF8S@`F4&INpv(4-9!SUUTcAlAE%SIo?;znO_Y+$uhvM)vEv_}gr=aZuy2N9aRn@Z znN%Uqs8CRo&H$Idtqh#U3|)-Fn((aaEJ$V_x*AO?yt%>x`4EPr&?gjR$PIct=i>3a+A{i2HG-k>aUT@?}rhMr^cagtIUQ%s&3;Evr%1VM0-!rt^_)gj$0&6i9Tmodtt1d{5cHvYS{a$XhaUqZT zi%znN@+~S!9poi)*9B3H-Id1bfd8f`1M_|G=lxN*Sc{8Zy#?n*=&q=)xwt-uNHw2tIc4wBvn7e|?9 zt2d)Zjk6ieW!KOEHY5R z^&=+Qj3y4@un?qVGz4-^3W4gH<$g5CpKBX2kq)Qj6TqeI^`RE6p;IS|MlVTpT4c zz-9_AroM93Uq$^L)R&KdnEw9-4OE~8-_l^k2wQ?-cCT{MrisR2N4e@x1(yuXSZwt> z?82!u`U%b&-|n|bcnG_nU+@AQ6^yhV^69A4IKpI^Tm%2k09C#K<`+TW-_Payc9Kzf zjZ?;-o{7=95J<`N{Z84Ew2}t;z9&Xvlm1D4<;2cQ)Ytbw3n&# zQXUU38LgUW2O_hxK$t?_xX5be(BM4!QLcV2qJC|Nfiqa*W*X?+R1H2Luk~fLm5w^( ziD;w}cGA7g-CCRilB=J6z@-3FxEqIxDLIRR!m-v-H67haN6OXFedOv$`QOQP@~iwY zaveX&L(WqwI7U_Nauc}erQ|xb`-%GPKKNZ%!0+w5^^0GZ!{d21y=eD zwnHFouZ}cZ-;k>#~vG#oN(FxZ)Fh&*__rdw%bMztH**%hLl^W zuUz$Cq<-6Et1knZi9dR#4WI%w*biK)u5*8DU=HUH7E)ii>aQo) z6DZ~K>)BEOz22ex)|rSe0am8l0JV1E{F?^K)!=<_(bxB&)7VK5JJ<$LuKGuSi@v@u zT}Z*>)X$#5%K~cf3=Q79$!p2%n<9a@7xQqQM#(45Q>HV~~K~5bZl@bPo}Q zJ$a1bI5pIw(oX`p&XS3pL~s-=j&TF zu=`&f4OE~OFVbLHmNi(yjK&>mGn6;if==!Qi@>D-dX}$#j;DST2MC?I6@#PvD4C8N zwAe_4MVU50J^B0OK?cqxpKYoHx2~v9>|~OwzVi8o$2s{6`ILYPj-i2;O!*RWtqJ9) zfy+g+4Y~GkTtvasVsJ5Noo@|vce;=I z%2od%a=j#^{BinOahxSfDHs}>BWsvUQb6vE0++z`MK|Mv#3ApD4Wsb0aj8@ zedVhEAbChx91g2W5%|uGo;Peg?T-v$h=ZkAOf*un2 z%kB0myKox7Wdz*p8yQ=|@Q)0~n6LY1o9GeW#mBAb%Vp+HMuDNex*PjFK6`<9?E48M+k^u~(%*+{M#>S_PmA&`=_ zEwQS4RqR`E8Dewpk2ZoT?j4EqWwW;?tQGFWfSb0bpGpcYr@qz@amcxff(#m{qat$6 zkn$TDAm&0_m!+)I$fHC97tzU8e~|K%l-T7EaP&W1Q$E9jm6Ts@1h|Itoo!s@|0J(i zX(O(n{$ew88H^u-W=_aYG(5pc_chu1yYrsE*;utl|Z-?P6^rN>C)z3%Z61X|n60Om>3rzgp z8mXYe8M6UNMl%H;R0>0+%Dl*GUdw8g?pn#6_PODmlz@-F< zTWz0c;C}x#dD4~a6XXZuB9`c9onk$d(a|mD;ug+?sL$~aH%4esxW+mvXTjsm1qDn> z$u&bGz$HKuZ+Ny+U@utoa@D`v_(^b9_-Z2bWEjD-46Jul)zJ&&S~BJK3ddka?`tZ5 zi~4#jQ2s7B3J`MIc#x>2WVku6$3R4`2BXRKj;-=kaH)wI*VrmkKY7$wuKJfyzuoAg z|7+VkLjx;Nb;o&*ys(IELeb0QdP`U>n#lEzt@3{xj%QzXZvxr>w)H0f|AvnA7Oy(` z0bEMF;(D8*R0_^C*9{Wzn*@D3^qrA!1Azo6W`GPk^gFIO3GL;oKiKdD zr^y$1Y#5$KKX?qmj6dqfnFfIb?%-#A%r62%`ZReF??P#l*|T5Vz__$;iQ~+5KG7=8#H=`(3sf=qBJKtq)ckVio!{p9%k*M-6eUSa@sqyb(d z*Qa!pzfC_Yd;vZS!`ta6q>uHeqwi@@>2nlmPDJqhk&h2D9CN+Lv`Vd~A>_FAtFFz` zo{nQSTy1!7`|nEweRNB$#xrn(FYxRzd@lXyEoAkRO|Ca!El2!gA&@qc-xQOSY5wDq z2?MnG0>2uDZ)N~}6ohV^_vk;3Z|%oWFdB2k2$}!KFSA5vv8&A_vNuEOXq^~9zknGn zW6i!nedVg(Yf3(?CecN&$aLG{4 zo0jWANPa_4^!2#~*^+d#;R#N6Z;jgc5AG>229MH_K1E|7{yhaQffHG!@f7?)eSHQ+ z^&`yO5|5nnCcE;1OH6#^^mIjb8E*^{@HPFu<3VZ|J{epxq#sI1qPukrtX!j9PyLwC zKWx>lqU2i|==a#v;71zh_t_S)B~Cg~G%!D)AJspdT+fh{uP{8pNoK)%+P}ajP=gR1 z>8Y9W_0-Sy)zDpG`2F;wr*iZYa;j*s!sn40nr@nI4ucbb|Uf0L}AJjLzr~aW5ak4D|J|o!2 z;An7ZW93GOe>v307x4_z0(jI{uKM>SEhhUm5e%3Tu zu7}+jW~5AT3VnSmyNCPc|Hm2u?szbO20n=f)jmfTgyEOck7h{y*dZEs*O}3t@i#;R z9ZJ<;J-I%&to$YhuJ>i={xH0Xe)PFU^%Hs$0;xLZZCiD@9PqxTqgG#lKf~~-)5K-3 z9}TbBQS#-}-O^cBum?Y%0rWW^HOXfHd~eer-3vBRU%BeP zNNyh`+im}Ql>&VxO$|1a>ocdy-!mMOOkV-M3d6snAAP3P_>uTOQ=nC(0pu4(rRwrO zvnAVL7tRsOEbso@f++H9!R0DfhA%^RhT#?9h~FDP13zpGaMOtals^G(`pR}2QGPK_ z(s$H%$Sp;|sK4V!t6xvSAvoHKpRVsX|F2*PvT>!cixAZu&l4rv3=jcakLjAlPf$sJV z;e18Gu#uJ*L|K!RGi4d)oaCEKPBa{oRr;~oc#V!+{U|>eTmo+xZT#bn!+;e%N&^a1 z-+78$pN>;O9UYY(U}v{U^z$F`ql0qf|<&tIY=&8Rwh4P2^p77LJ1!GUL6 zUXb3mhK>Z60wg0Y%HNw#*-g+(@j#;Cy#Z8q9`)0G1y~)1uLGBiRxzXa7>9uKLDg0oCp^m1b7eCk)rXY-IUD0qtc%88x26^JhZ+G((YXFR{sK)D+1 zWPqlbHo!^>W}jpItek3j7CYl=@|tOu>zf|WkUOF2R?rk>Jq6RX zbjwAy0F9AG5^^%mvx4pmtss$AcO`lKk(TG%h0{bnjE))@VJ}SjBtX|325_xlA^EWL zEN`ZtV#9I$fBX5CBvSAu1X6XiYy$ds{zFGA4zr4S<6*@4R$sa5k0I9#DIW(e8J&_1 zKT`jRwkn(pC|Gff4Ny&sN66zBTdwnf4)XG2Enmi(JNyD0Af1QLTKZW>p0&{GOJ6bV z-$+3hyQjW?5PzX{w4II`>F5OVsuQgveOBxaa5+X4`;HO!7>*H;O-VOS<%K3bt3oN3 zkI_JLs-maKwT6`clU!>^c>}mqb-~#-qw<{|Nz=@Ji`#q0hOEG1d>H})G2q*)4u=J2 zBMr2MRR4DBFImc(V1UsVi3XmO))@>9QpvSOIsV}*L=@)yA+qbshkC6=EAleEtz^1pZYmMuvsM#52BkJbNff-M`sAwrHuW>ea|c?< zZ`4=L@ejhw>|O5smG-!rySFtsg1q@^@?n;*1D6t%-(xehiUoKDTuQ9mYU8ax>c`(_ z^-V=WI4$%SYoG!(mDQT{qz0A-u*9^0g}G3Rn*7;cav{vwYPP<#@x_sTs?~H(7@YSv%U8W%HHXpDOW-oY z`Q10edCl+yr^Z)+tzr0QhRbIG7=Zqrb_z5@%D)Gfj0VrKq>O^2ue4l04XyVD9tW2u zQ%-fKmih%9wqPN3)I|e*8b%Fv(IERJYaqXdA?cjeHgFgBjx82DuYgNn)hBk|G(653 zx|q9*6@;As&_Erj!4`788KwMF!}0i?ud06v!+)or=3Mn)mw`31#Qa~M`%`fc0;%eZ zGcC!aAjcT=a@9YX`fI2!KZYUc4(cnn)N$^ie(igxQR)9vIP}I}B`OJyb4B0%J~a%V zY&cF-#L;ePI=DQt^?5m~n&MpL{Pr|TY4^X443Osw{6HA~h~XFyTKWdAp?;aqPh%MV z)>S5c?_)Juh<9nw?sN2I82&%Q2RnL&`%j|aQtB&L{Q~Ou z{IDhkZ%|*k>NkTU{$MBX$G-dVhjdi)v*mhuK4qPl;)O+DiD!o4hZ&Bi>gh)VXMsz> z@bOAp|A~~`M+5z;r5Zd$1O3XS>PKHA8YsZ85vppIG2jwFKY^+G=TTod$3Hv~Ndx^> zrW(9W1LYc}h1|YGt`t9tVR(X5;;W(W!|-21bfhnxtE)e0Q15dzq)0UI`i+k#8jgXB zFR81aeaZD@bmim8L;8Nd3ifA!CSQQLVR$wj>GuHC&#~ar1j(FoRa0;!{p_&*d-uPM zeFf^rxt#&@!vZQOXMic@EEylmq2L4Z7B|w6MdW*5D=T=U<8Qkk3lM)|zXF{18{Ydm zJ^eTbgG#46F98mod>Zh%=WC;aZh0FYZ4F&ZSj9za8wDQXcI-=bNE+t6q zQv%0%CJcYhaLfU5d2jdhGP!&hrMLc}(L{mfRE_=(E+w9Yi%60>19my^I?L<+w7Xx9 z#m*vd*)5e5J13Cqryam;{8J%t&BxBIYBDV<7+__jZObb1?Zy!XB415(h2gsl@2zRN zaol1PKfx*YIT}?gDtq#=hU0b}n?@a_k?V(1luskqrc^$Y{%b`Ctc8B4SHzK~|$o~9A z@2i+s+$eUvT)nL|JOPi-_YH7s7=9ul;PSN#j9-_4(ZnWgRj8U-_WGl;kIoC!DC0QGy@jJ9#; zJ)69ojx?i>k++g3GxC6qvJ&Tv>wEg`gkp^#!I|Q#(v&cKqT#rg!wk{Clg{B{horFJzT| zN&_oUb;tRV+qqcoGey}U$-z-WvJ0@3?E5Idi!xXDnBypwv|3N{`WGjeg zhW;cE?rV8wv~@MTM3%S~L~be0BfuqqlW)mp3N}(-x$55zj`#^qmapv}F$R0`8gL1a zooX|b#98+j44_;C?4bUN@m61cn@-Ycw^%>3CPmu*U%(Ro0|F_T8pxKUdU6<8!2hq1 zYt5(Vj<^KVf0%T3KerDYO& z{m}LUtw9z`_AeSJSA#cbkT%omH?jy(rJ^y|QLg%XflG#V9BlPhQLv2qO#QBHwvoXNPMFIUg-QWRyQ#+o!O*#dq++{P=M1A>zV@YM; z(ixS@mZbX($0J+5j6QRhDgT~==NKR-$9j(EaXtEOafd4xJJB( zj4oSh^+Ofh@6Vxu3e;dF4N~*0K`Tq}0(s4USTY&}?va%QXNIo;d-u;nP7(x?qwZ5} zfa;+2RA~%)IsH42QQx`1>hlL5oWt(5ei|>aJT1jqTti-Uw&j^D@C)GRe;7)AIsK0j z?8&znj@xW>q#62@0W#S3RTRv)&t|Co92;04C%PG2n#SC z^k9YcpTp;Gn(4oz0`VnqLzxXcg@U>FTZ0|)L!y${=?ck{E39HW#m(eZ_gE0aHooKm z>nC`Z)o(D)@Z%rK|IYb;ev*}ZLqWmctfLtWJn=#6sBW_5g#)erh2$0cFmRmZmE@Ii zmUr)E`M2anzeO5QrR_hh(gvv9Y6XRM;Z&2Sequq#FzaY5`HpfMpx$C<%-^k_+WRcm znme65_kPP~v0$O04_Sk<2l_fXnY^^p^6mjP!d>LUezF;=X5gU@TR$y)V7QXJfxJsT zc8`Q%!=!cCP0;iqr?$ch)X}U*tfM6~P)8qUmml(@NuiJ ze54742Nu@2(A4_J>i&HM^n%3YB(8>nBs)+*|u;sf%;UDnSq z>7)`r>2Vui=_^*SXoU52J^7B?E!StcJ_MHmM7h<*>wDBsshVVgOHC}j-Ih#`at|`VW?z6!Vfc&mk0#r#{t@L>8g%#^{WlD6 z1D6b?-EC_~A7Y7pQnK93wE%k?jtdlPwI&#U;vNEPl1@iDa;c+(!6k6pZMG)1P_Ul7 zuGI1h^7rXS2QKx~8loWhcUuA#Y@?$F?q+&DAf?6zPFigPlyN^^OTM|w`q8HO7kT&9 zY%=L|NP(xUpHOX)6+q9VObS+9Yk3>_8uG05mT#b=XUKQ_Yy)ej`<1-8*y^LYOq%z! z4P0}T;UTA+0scur?HX&KeCRWxfTz`a+RPeE2*am>OVbp*Y%|nK{}o4_T&l7HC%QEYjYaC!dUFF2l#bUaW;r+`b~ zSqIq~D&zHlTgj^@Szb?r56DYnE!XP`dvCISN{)@P<6i~aW&s8H8P*^kY?GL-1%x z|IST@$2p_yFQq7UO6g~%&wthPX8*@2l~3>_1L$E_Jv|F9C2RcEw!NNa|4d%~yXEY| zPAb|$^t*nsJfDtkAkPcY(J~5tr=XMuE65Lc(K?#K0P5&=aH+Wsz5+ZFhSwO58yFl{ zddA;P6zG7%xXz2@x_>Kwm0UBbd^5S0SowS8c`Pt_$k|GPR;e0%1};@MC15+tX7=`d zUJ?yl=l7L(S{OdlaC`xoBcu91%y2yaqjND8WYIvYTKPh9ZDZxf8y@Fm`#RkzhU4Sv3 z|1rcC_TBwb!tjZP<20L C5%gZ#%=b2;G1mZXClY(@(&uz`m$%%u$`K28>Wfv<&u zWXR7q7>@m((`wO@bX$Y`#|aFl;j$&^t_B-;L!J$+^Mg;wYtIk$R)rWOy)6IZoOgX~ zxu4;<7tA46v?NU;*AdWC{N6ITo-Zh$1ujk2ex8k6Y3d)+n~g!7Gw67Bb6VU%M>^)K zqkE{o=7hfbmDJZER`nkv*CAPX=xGQfM=czy%PHCTFInm3YH&Nb9wU^OgNvVZmP{ug zZPYK(V+Resvic#X>qJ`>o$M52Ul9%b>Xxrh92ACU7>);wa5URJ9RY4KbiCzFkyd&o z_0tX4`iJal3N%CNteB2=EVGXCc_#E0^_8psyVPH{!s=HBtYl=P&5(1h<>|sD{%H_M ziI@3Gd`TF7x#2jxGJsf1x{6!_Sn4=Mk%1VNh@2jDs!tlk0;|eGJ(2a8< zxt^qQ{KLyHbX4REkROI$K}UL>Km)8L*OOG`*OKeBUHOI(1C;p!SA^jY(~-`S)zK5= zoK`D#{z0xMqbjH;59x`43SNOg+H4ljY;-<9;5E^}$K-t(-6srBF&q!4^87;mPbBC0 zMQ{73Lm+{hZ?K&uFKA89r-7b$sH1%Ho)ZiTu24=ts=wCoIH#r0h>7^u(m+eB1{=ur z98dXeh9@|6zM6PA41a=t^qf%r{6n~0|JT!L71To@H4%S@tqDEHOMKn(D$jDA11@P2 zOMHIIm!Z5c{1U^h|K9qCi_2-S!{_LRF#Hxe(hO+<%EQD$jsFk@J=gna z@i@5DM0&j~af@9zUm1OTNXVDbKf>^cH^d>Xn3U52{SSP@#K$#VUN|Io;*9|=6kP2p zGRxF3d@=*{Ty~??bZ{v^@JSn3FIKOlpBW)vfI=1c_;nh9SIab48>ye|(|;fge}sPY zI$%%x*U(^v&rxF-{uUkSbwkb2yX1QLPWgZ7r>L+0kn_L(1x_a&>4ige^au6J`Z{vr z-V_a7%<}Q!hT|Yq*|+US8;<$E-tE!=sSrq;weiBCP8z>tfLdRG-^1|0TjFuBqZc66 zf6#Dz##j9G)IX$2y9-ns%l+s`Go+$P)DN!f+ezny;YZSsc1H1I;xB?gO1!z&R#6%K z-An`J8u$+CuXx_-Z>C_2(NA#Fd>PsihX3@IiI4mJbff`(XJFm$l?M<>G8*-=4LmGp zC0Qm1y$`;W%Yu01f;LdCOZiu<}32 z^-fDq`$xTPOSYle8t6r(v%sYjDOX45Q@@0_-;ybKiu%e`zm8mQQYjC;M1kJNQ^9NG zdPh_F+vIvjQ+W%xG)?<^wg8;+IV0Z@jb5($$>7k(`@dVPK?ZB;A}er~er$ox0iPr< zZMD3So_=Hi^`ilPGaPS1_?mL?zePj&Zu{RbBfvnx0P1KTa4A_D@9S&eGpMf_RsD0x zRbTl<w7}V zSJ6)>kB;;j?FTeat_B~|ppE)Dtckhri3X0AT2vZjgG)x!zq18PBL5rp$$Q(skOqrr z&_RPX8Ym}rz9QG3303|b{S;qriGJ22twpx+swcUXBK|ZZz{#br6CDwTFQ6m+9a8Ja zagGC*5_hmF&HX(HZ=}9*)xTA^T>mff1%AjF?8zUeBmD(ebyN#3fvY-fRk!gdxA*&^ z(aTjo3EcFd)ja-JP;!|uz!b`tq4i<-O$-nPN4uv|1}Of?W~hO};0LmTCLqU#ZT~nU zz{BCb!25;alfb0}*_}2(Ej#1s44|BGofY)6-RHmH0}~$?lYN2^19bn~xA)dCfN~A6 ziTa7ZSp8fI{zZM|s{aP{3qmv)MoHj9S;1W@ay1BoONlp-Z)O`GPJQL7Ki_a%eD)RW zR3Goz|ITE9O15PTJ>AFv$~C~P)L;6WEis?Sa1yqN22N_pRez-6xW;@o+|_LZ#M9Hw44_;C+(G@|pH{zNl9hZzedVhEgVD$QziEO!Eb1N2v;Qk9m>~4c zsdGgbzSi(wKN{d#a4Eo&N!EXTq?LR{Kk?W4=Kp`}E>JI6Mr@UhUPme#2rdEQr`iB| zEchGsm8*Us^)sp80D)=$k7%F*HTc|c{DOhxbhq?N82%>%IMZx~^s(MUJ`!Ub@5$9w z=0_%ef|KbBu*4Yb$xovr{i2yVI*a^*B(gQ*^r|PJP24y}+WnuVz;L-#& z)2*Kpw($=Ppj`cTQ9qNvADE~6-@I0tqm6QczGuAd2ba!R@5|6X!ti>-2Rj--1HVG9 zKU}Ch^dze=u=t21x|8|+{5TDi z6Far!`h&#EpC{KZ9V`Er;Uk<`rM!7W!5bk8a(sej@_Zlvki5jlKQTPcNnT?6ga-bC z`c*#t9ps?~pWr77w)pt(A%_Mf0^(Q zs;c)|eu(;CK)<*Ck(u>0$o3iBM1#l^`*yBU z>gW3O@237fKK%#CNB43Y{}Bo%_X