From 48fabdcd5f29e389e9e339a12fe8b62d9a0eb278 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Thu, 22 Oct 2015 21:17:34 -0500 Subject: [PATCH 01/44] WIP Almost certainly broken --- src/machi_merkle_tree_mgr.erl | 335 ++++++++++++++++++++++++++++++++++ 1 file changed, 335 insertions(+) create mode 100644 src/machi_merkle_tree_mgr.erl diff --git a/src/machi_merkle_tree_mgr.erl b/src/machi_merkle_tree_mgr.erl new file mode 100644 index 0000000..f2ed795 --- /dev/null +++ b/src/machi_merkle_tree_mgr.erl @@ -0,0 +1,335 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. +%% +%% This file is provided to you under the Apache License, +%% Version 2.0 (the "License"); you may not use this file +%% except in compliance with the License. You may obtain +%% a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, +%% software distributed under the License is distributed on an +%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +%% KIND, either express or implied. See the License for the +%% specific language governing permissions and limitations +%% under the License. +%% +%% ------------------------------------------------------------------- + +%% @doc This manager maintains a Merkle tree per file per FLU. The leaf +%% nodes are stored in the same manner as the checksum data files, except +%% they are represented as +%% +%% `<>' for unwritten bytes +%% `<>' for trimmed bytes +%% `<>' for written bytes +%% +%% In this case, the checksum tag is thrown away. The tree feeds these +%% leaf nodes into hashes representing 10 GB ranges, called Level 1. We aim for +%% around %% 10 hashes at level 2, and then 2 hashes level 3 and finally the +%% root. + +-module(machi_merkle_tree_mgr). +-behaviour(gen_server). + +-include("machi.hrl"). + +-export([ + child_spec/3, + start_link/3, + initialize/2, + update/5, + fetch/2, + fetch/3 +]). + +%% gen_server callbacks +-export([ + init/1, + handle_call/3, + handle_cast/2, + handle_info/2, + terminate/2, + code_change/3 +]). + +-record(state, { + fluname :: atom(), + datadir :: string(), + tid :: ets:tid() + }). + +-record(mt, { + filename :: string(), + recalc = true :: boolean(), + root :: 'undefined' | binary(), + lvl1 = [] :: [ binary() ], + lvl2 = [] :: [ binary() ], + lvl3 = [] :: [ binary() ], + leaves = orddict:new() :: mt_entry() + }). + +-type mt_entry() :: orddict:orddict(). + +-define(WRITTEN(Offset, Size, Csum), <>). +-define(TRIMMED(Offset, Size), <>). +-define(UNWRITTEN(Offset, Size), <>). + +-define(CHUNK_SIZE, (10*1024*1024)). +-define(LEVEL_SIZE, 10). +-define(H, sha). + +-define(TIMEOUT, (10*1000)). + +%% public API + +child_spec(FluName, DataDir, Options) -> + Name = make_merkle_tree_mgr_name(FluName), + {Name, + {?MODULE, start_link, [FluName, DataDir, Options]}, + permanent, 5000, worker, [?MODULE]}. + +start_link(FluName, DataDir, Options) -> + gen_server:start_link({local, make_merkle_tree_mgr_name(FluName)}, + ?MODULE, + {FluName, DataDir, Options}, + []). + +-spec initialize( FluName :: atom(), + Filename :: string() ) -> ok. +%% @doc A heads-up hint to the manager that it ought to compute a merkle +%% tree for the given file (if it hasn't already). +initialize(FluName, Filename) -> + gen_server:cast(make_merkle_tree_mgr_name(FluName), + {initialize, Filename}, ?TIMEOUT). + +-spec update( FluName :: atom(), + Filename :: string(), + Offset :: non_neg_integer(), + Length :: pos_integer(), + Csum :: binary() ) -> ok. +%% @doc A new leaf node ought to be added file the given filename, +%% with the particular information. +update(FluName, Filename, Offset, Length, Csum) -> + gen_server:cast(make_merkle_tree_mgr_name(FluName), + {update, Filename, Offset, Length, Csum}, ?TIMEOUT). + +-spec fetch ( FluName :: atom(), + Filename :: string() ) -> {ok, [ Data :: binary() ]}. +%% @doc Returns the entire merkle tree for the given filename. +%% {@link fetch/3} +fetch(FluName, Filename) -> + fetch(FluName, Filename, all). + +-spec fetch( FluName :: atom(), + Filename :: string(), + Level :: 'root' | 'all' + ) -> {ok, [ Data :: binary() ]}. +%% @doc Return the current merkle tree for the given filename. +%% If `root' is specified, returns a list with 1 element, the root +%% checksum of the tree. If `all' is specified, returns a list +%% with all levels. +fetch(FluName, Filename, Level) -> + gen_server:call(make_merkle_tree_mgr_name(FluName), + {fetch, Filename, Level}, ?TIMEOUT). + +%% gen_server callbacks +init({FluName, DataDir, Options}) -> + Tid = ets:new(make_merkle_tree_mgr_name(FluName), [{keypos, 2}, {read_concurrency, true}]), + case proplists:get_value(no_load, Options, false) of + true -> + ok; + false -> + handle_load(Tid, DataDir) + end, + {ok, #state{fluname=FluName, datadir=DataDir, tid = Tid}}. + +handle_call({fetch, Filename, Level}, _From, S = #state{ tid = Tid }) -> + Res = handle_fetch(Tid, Filename, Level), + {reply, {ok, [ Res ]}, S}; +handle_call(Req, _From, State) -> + lager:warning("Unknown call: ~p", [Req]), + {reply, whoaaaaaaaaaaaa, State}. + +handle_cast({initialize, Filename}, S = #state{ datadir = D, tid = Tid }) -> + load_filename(Tid, D, Filename), + {noreply, S}; + +handle_cast({update, Filename, Offset, Length, Csum}, S = #state{ tid = Tid }) -> + %% XXX FIXME: Not sure about the correctness of this + insert(Tid, Filename, {Offset, Length, Csum}), + {noreply, S}; + +handle_cast(Cast, State) -> + lager:warning("Unknown cast: ~p", [Cast]), + {noreply, State}. + +handle_info(Req, State) -> + lager:warning("Unknown info message: ~p", [Req]), + {noreply, State}. + +terminate(Reason, #state{fluname = F}) -> + lager:debug("Shutting down merkle tree manager for FLU ~p because ~p", + [F, Reason]), + ok. + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%% private + +make_merkle_tree_mgr_name(FluName) -> + list_to_atom(atom_to_list(FluName) ++ "_merkle_tree_mgr"). + +handle_load(Tid, DataDir) -> + Files = get_files(DataDir), + lists:foreach(fun(F) -> load_filename(Tid, DataDir, F) end, Files). + +get_files(_DataDir) -> + []. + +load_filename(Tid, DataDir, Filename) -> + CsumFile = machi_util:make_checksum_filename(DataDir, Filename), + case file:read_file(CsumFile) of + {error, enoent} -> + insert(Tid, Filename, {0, ?MINIMUM_OFFSET, unwritten}); + {ok, Bin} -> + load_bin(Tid, Filename, Bin); + Error -> + throw(Error) + end. + +load_bin(Tid, Filename, Bin) -> + CsumL = machi_csum_table:split_checksum_list_blob_decode(Bin), + iter_csum_list(Tid, Filename, CsumL). + +iter_csum_list(Tid, Filename, []) -> + insert(Tid, Filename, {0, ?MINIMUM_OFFSET, unwritten}); +iter_csum_list(Tid, Filename, L = [ {Last, _, _} | _ ]) -> + make_insert(Tid, Filename, Last, L). + +make_insert(_Tid, _Filename, _Last, []) -> + ok; +%% case where Last offset matches Current, just insert current +make_insert(Tid, Filename, Last, [H={Last, Len, _Csum}|T]) -> + insert(Tid, Filename, H), + make_insert(Tid, Filename, Last+Len, T); +%% case where we have a hole +make_insert(Tid, Filename, Last, [H={Off, Len, _Csum}|T]) -> + Hole = Off - Last, + insert(Tid, Filename, {Last, Hole, unwritten}), + insert(Tid, Filename, H), + make_insert(Tid, Filename, Off+Len, T). + +insert(Tid, Filename, {Offset, Length, trimmed}) -> + do_insert(Tid, Filename, Offset, Length, ?TRIMMED(Offset, Length)); + +insert(Tid, Filename, {Offset, Length, unwritten}) -> + do_insert(Tid, Filename, Offset, Length, ?UNWRITTEN(Offset, Length)); + +insert(Tid, Filename, {Offset, Length, <<_Tag:8, Csum/binary>>}) -> + do_insert(Tid, Filename, Offset, Length, ?WRITTEN(Offset, Length, Csum)). + +do_insert(Tid, Filename, Offset, Length, Csum) -> + MT = case find(Tid, Filename) of + [] -> + #mt{ filename = Filename }; + V -> + V + end, + ok = maybe_update(Tid, Offset, Length, Csum, MT), + ok. + +maybe_update(Tid, Offset, Length, Csum, MT) -> + case orddict:find({Offset, Length}, MT#mt.leaves) of + error -> + %% range not found in our orddict, so fill it in + do_update(Tid, Offset, Length, Csum, MT); + {ok, Csum} -> + %% trying to insert a value we already have that + %% matches + ok; + {ok, ?UNWRITTEN(Offset, Length)} -> + %% old value was unwritten, now we are filling it in + %% so that's legit + do_update(Tid, Offset, Length, Csum, MT); + {ok, ?TRIMMED(Offset, Length)} -> + %% XXX FIXME + %% Scott - range was trimmed - do we fill it in with new stuff? + ok; + {ok, Other} -> + %% we found a checksum that is different + %% this shouldn't happen because once we write a range, it's written + %% TODO - file corruption? insanity? + lager:error("Tried to update merkle tree for file ~p at offset ~p, length ~p, got checksum ~p and tried to insert ~p", + [MT#mt.filename, Offset, Length, Other, Csum]), + throw({weird, Other}) + end. + +do_update(Tid, Offset, Length, Csum, MT) -> + D = orddict:store({Offset, Length}, Csum, MT#mt.leaves), + true = ets:insert(Tid, MT#mt{ recalc = true, leaves = D }), + ok. + +handle_fetch(Tid, Filename, root) -> + case find(Tid, Filename) of + [] -> undefined; + #mt{ root = undefined } -> undefined; + #mt{ recalc = true } = MT -> hd(build_tree(Tid, MT)); + #mt{ root = R, recalc = false } -> R + end; + +handle_fetch(Tid, Filename, all) -> + case find(Tid, Filename) of + [] -> undefined; + #mt{ recalc = true } = MT -> build_tree(Tid, MT); + #mt{ recalc = false, + root = R, + lvl1 = L1, + lvl2 = L2, + lvl3 = L3 } -> [ R, L3, L2, L1 ] + end. + +find(Tid, Filename) -> + case ets:lookup(Tid, Filename) of + [] -> []; + [R] -> R + end. + +build_tree(Tid, MT = #mt{ leaves = D }) -> + Lvl1s = build_level_1(?CHUNK_SIZE, lists:map(fun map_dict/1, orddict:to_list(D)), 1, [ crypto:hash_init(?H) ]), + Mod2 = length(Lvl1s) div ?LEVEL_SIZE, + Lvl2s = build_int_level(Mod2, Lvl1s, 1, [ crypto:hash_init(?H) ]), + Mod3 = length(Lvl2s) div 2, + Lvl3s = build_int_level(Mod3, Lvl2s, 1, [ crypto:hash_init(?H) ]), + Root = build_root(Lvl3s, crypto:hash_init(?H)), + ets:insert(Tid, MT#mt{ root = Root, lvl1 = Lvl1s, lvl2 = Lvl2s, lvl3 = Lvl3s, recalc = false }), + [Root, Lvl3s, Lvl2s, Lvl1s]. + +build_root([], Ctx) -> + crypto:hash_final(Ctx); +build_root([H|T], Ctx) -> + build_root(T, crypto:hash_update(H, Ctx)). + +build_int_level(_Mod, [], _Cnt, [ Ctx | Rest ]) -> + lists:reverse( [ crypto:hash_final(Ctx) | Rest ] ); +build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) when Cnt rem Mod == 0 -> + NewCtx = crypto:hash_init(?H), + build_int_level(Mod, T, Cnt + 1, [ crypto:hash_update(H, NewCtx), crypto:hash_final(Ctx) | Rest ]); +build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) -> + build_int_level(Mod, T, Cnt+1, [ crypto:hash_update(H, Ctx) | Rest ]). + +map_dict({{Offset, Len}, Hash}) -> + {Offset + Len, Hash}. + +build_level_1(_Size, [], _Multiple, [ Ctx | Rest ]) -> + lists:reverse([ crypto:hash_final(Ctx) | Rest ]); +build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos > Size -> + NewCtx = crypto:hash_init(?H), + build_level_1(Size*(Multiple+1), T, Multiple+1, + [ crypto:hash_update(Hash, NewCtx), crypto:hash_final(Ctx) | Rest ]); +build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos =< Size -> + build_level_1(Size, T, Multiple, [ crypto:hash_update(Hash, Ctx) | Rest ]). + -- 2.45.2 From 1b8401e7de170eb2d29251133d1f3d346265320a Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Fri, 23 Oct 2015 22:59:23 -0500 Subject: [PATCH 02/44] Initial smoke test --- test/machi_merkle_tree_mgr_test.erl | 99 +++++++++++++++++++++++++++++ 1 file changed, 99 insertions(+) create mode 100644 test/machi_merkle_tree_mgr_test.erl diff --git a/test/machi_merkle_tree_mgr_test.erl b/test/machi_merkle_tree_mgr_test.erl new file mode 100644 index 0000000..8e8336c --- /dev/null +++ b/test/machi_merkle_tree_mgr_test.erl @@ -0,0 +1,99 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. +%% +%% This file is provided to you under the Apache License, +%% Version 2.0 (the "License"); you may not use this file +%% except in compliance with the License. You may obtain +%% a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, +%% software distributed under the License is distributed on an +%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +%% KIND, either express or implied. See the License for the +%% specific language governing permissions and limitations +%% under the License. +%% +%% ------------------------------------------------------------------- + +-module(machi_merkle_tree_mgr_test). +-compile([export_all]). + +-include_lib("eunit/include/eunit.hrl"). +-include_lib("kernel/include/file.hrl"). + +-define(TESTFILE, "yza^4c784dc2-19bf-4ac6-91f6-58bbe5aa88e0^1"). +-define(GAP_CHANCE, 0.10). + +make_csum_file(DataDir, Filename, Offsets) -> + Path = machi_util:make_checksum_filename(DataDir, Filename), + filelib:ensure_dir(Path), + {ok, MC} = machi_csum_table:open(Path, []), + lists:foreach(fun({Offset, Size, Checksum}) -> + machi_csum_table:write(MC, Offset, Size, Checksum) end, + Offsets), + machi_csum_table:close(MC). + +choose_int(Factor) -> + random:uniform(1024*Factor). + +small_int() -> + choose_int(10). + +medium_int() -> + choose_int(1024). + +large_int() -> + choose_int(4096). + +make_offsets(Filename) -> + {ok, Info} = file:read_file_info(Filename), + Filesize = Info#file_info.size, + {ok, FH} = file:open(Filename, [read, raw, binary]), + Offsets = generate_offsets(FH, Filesize, 1024, []), + file:close(FH), + Offsets. + +random_from_list(L) -> + N = random:uniform(length(L)), + lists:nth(N, L). + +choose_size() -> + F = random_from_list([fun small_int/0, fun medium_int/0, fun large_int/0]), + F(). + +maybe_gap(Chance) when Chance < ?GAP_CHANCE -> + choose_size(); +maybe_gap(_) -> 0. + +generate_offsets(FH, Filesize, Current, Acc) when Current < Filesize -> + Length0 = choose_size(), + + Length = case Length0 + Current > Filesize of + false -> Length0; + true -> Filesize - Current + end, + {ok, Data} = file:pread(FH, Current, Length), + Checksum = machi_util:make_tagged_csum(client_sha, machi_util:checksum_chunk(Data)), + Gap = maybe_gap(random:uniform()), + generate_offsets(FH, Filesize, Current + Length + Gap, [ {Current, Length, Checksum} | Acc ]); +generate_offsets(_FH, _Filesize, _Current, Acc) -> + lists:reverse(Acc). + +test() -> + random:seed(os:timestamp()), + O = make_offsets("test/" ++ ?TESTFILE), + ?debugFmt("Offsets: ~p", [O]), + make_csum_file(".", ?TESTFILE, O), + + _ = machi_merkle_tree_mgr:start_link(test, ".", []), + machi_merkle_tree_mgr:initialize(test, ?TESTFILE), + timer:sleep(1000), + Root = machi_merkle_tree_mgr:fetch(test, ?TESTFILE, root), + ?debugFmt("Root: ~p~n", [Root]), + All = machi_merkle_tree_mgr:fetch(test, ?TESTFILE), + ?debugFmt("All: ~p~n", [All]), + ok. + -- 2.45.2 From b710517c649436d97a862aa1087a6eb385fc9995 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Fri, 23 Oct 2015 22:59:40 -0500 Subject: [PATCH 03/44] Fixes after testing --- src/machi_merkle_tree_mgr.erl | 38 +++++++++++++++++++++-------------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/src/machi_merkle_tree_mgr.erl b/src/machi_merkle_tree_mgr.erl index f2ed795..7a1e833 100644 --- a/src/machi_merkle_tree_mgr.erl +++ b/src/machi_merkle_tree_mgr.erl @@ -103,7 +103,7 @@ start_link(FluName, DataDir, Options) -> %% tree for the given file (if it hasn't already). initialize(FluName, Filename) -> gen_server:cast(make_merkle_tree_mgr_name(FluName), - {initialize, Filename}, ?TIMEOUT). + {initialize, Filename}). -spec update( FluName :: atom(), Filename :: string(), @@ -114,7 +114,7 @@ initialize(FluName, Filename) -> %% with the particular information. update(FluName, Filename, Offset, Length, Csum) -> gen_server:cast(make_merkle_tree_mgr_name(FluName), - {update, Filename, Offset, Length, Csum}, ?TIMEOUT). + {update, Filename, Offset, Length, Csum}). -spec fetch ( FluName :: atom(), Filename :: string() ) -> {ok, [ Data :: binary() ]}. @@ -148,7 +148,7 @@ init({FluName, DataDir, Options}) -> handle_call({fetch, Filename, Level}, _From, S = #state{ tid = Tid }) -> Res = handle_fetch(Tid, Filename, Level), - {reply, {ok, [ Res ]}, S}; + {reply, {ok, Res}, S}; handle_call(Req, _From, State) -> lager:warning("Unknown call: ~p", [Req]), {reply, whoaaaaaaaaaaaa, State}. @@ -187,8 +187,9 @@ handle_load(Tid, DataDir) -> Files = get_files(DataDir), lists:foreach(fun(F) -> load_filename(Tid, DataDir, F) end, Files). -get_files(_DataDir) -> - []. +get_files(DataDir) -> + {_, WildPath} = machi_util:make_data_filename(DataDir, ""), + filelib:wildcard("*", WildPath). load_filename(Tid, DataDir, Filename) -> CsumFile = machi_util:make_checksum_filename(DataDir, Filename), @@ -202,7 +203,7 @@ load_filename(Tid, DataDir, Filename) -> end. load_bin(Tid, Filename, Bin) -> - CsumL = machi_csum_table:split_checksum_list_blob_decode(Bin), + {CsumL, _} = machi_csum_table:split_checksum_list_blob_decode(Bin), iter_csum_list(Tid, Filename, CsumL). iter_csum_list(Tid, Filename, []) -> @@ -299,37 +300,44 @@ find(Tid, Filename) -> end. build_tree(Tid, MT = #mt{ leaves = D }) -> - Lvl1s = build_level_1(?CHUNK_SIZE, lists:map(fun map_dict/1, orddict:to_list(D)), 1, [ crypto:hash_init(?H) ]), + Leaves = lists:map(fun map_dict/1, orddict:to_list(D)), + io:format(user, "Leaves: ~p~n", [Leaves]), + Lvl1s = build_level_1(?CHUNK_SIZE, Leaves, 1, [ crypto:hash_init(?H) ]), + io:format(user, "Lvl1: ~p~n", [Lvl1s]), Mod2 = length(Lvl1s) div ?LEVEL_SIZE, Lvl2s = build_int_level(Mod2, Lvl1s, 1, [ crypto:hash_init(?H) ]), + io:format(user, "Lvl2: ~p~n", [Lvl2s]), Mod3 = length(Lvl2s) div 2, Lvl3s = build_int_level(Mod3, Lvl2s, 1, [ crypto:hash_init(?H) ]), + io:format(user, "Lvl3: ~p~n", [Lvl3s]), Root = build_root(Lvl3s, crypto:hash_init(?H)), + io:format(user, "Root: ~p~n", [Root]), ets:insert(Tid, MT#mt{ root = Root, lvl1 = Lvl1s, lvl2 = Lvl2s, lvl3 = Lvl3s, recalc = false }), [Root, Lvl3s, Lvl2s, Lvl1s]. build_root([], Ctx) -> crypto:hash_final(Ctx); build_root([H|T], Ctx) -> - build_root(T, crypto:hash_update(H, Ctx)). + build_root(T, crypto:hash_update(Ctx, H)). build_int_level(_Mod, [], _Cnt, [ Ctx | Rest ]) -> lists:reverse( [ crypto:hash_final(Ctx) | Rest ] ); build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) when Cnt rem Mod == 0 -> NewCtx = crypto:hash_init(?H), - build_int_level(Mod, T, Cnt + 1, [ crypto:hash_update(H, NewCtx), crypto:hash_final(Ctx) | Rest ]); + build_int_level(Mod, T, Cnt + 1, [ crypto:hash_update(NewCtx, H), crypto:hash_final(Ctx) | Rest ]); build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) -> - build_int_level(Mod, T, Cnt+1, [ crypto:hash_update(H, Ctx) | Rest ]). + build_int_level(Mod, T, Cnt+1, [ crypto:hash_update(Ctx, H) | Rest ]). map_dict({{Offset, Len}, Hash}) -> {Offset + Len, Hash}. build_level_1(_Size, [], _Multiple, [ Ctx | Rest ]) -> lists:reverse([ crypto:hash_final(Ctx) | Rest ]); -build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos > Size -> +build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos > ( Size * Multiple ) -> NewCtx = crypto:hash_init(?H), - build_level_1(Size*(Multiple+1), T, Multiple+1, - [ crypto:hash_update(Hash, NewCtx), crypto:hash_final(Ctx) | Rest ]); -build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos =< Size -> - build_level_1(Size, T, Multiple, [ crypto:hash_update(Hash, Ctx) | Rest ]). + build_level_1(Size, T, Multiple+1, + [ crypto:hash_update(NewCtx, Hash), crypto:hash_final(Ctx) | Rest ]); +build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos =< ( Size * Multiple ) -> + io:format(user, "Size: ~p, Pos: ~p, Multiple: ~p~n", [Size, Pos, Multiple]), + build_level_1(Size, T, Multiple, [ crypto:hash_update(Ctx, Hash) | Rest ]). -- 2.45.2 From 77096c5f828abf98e57deae3a426cd1cc3911ba6 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Tue, 27 Oct 2015 11:56:54 -0500 Subject: [PATCH 04/44] Add merklet as a dependency --- rebar.config | 1 + 1 file changed, 1 insertion(+) diff --git a/rebar.config b/rebar.config index 2a77381..f03a43a 100644 --- a/rebar.config +++ b/rebar.config @@ -6,6 +6,7 @@ {deps, [ {lager, ".*", {git, "git://github.com/basho/lager.git", {tag, "2.2.0"}}}, + {merklet, ".*", {git, "https://github.com/ferd/merklet.git", {branch, "master"}}}, {protobuffs, "0.8.*", {git, "git://github.com/basho/erlang_protobuffs.git", {tag, "0.8.1p4"}}}, {riak_dt, ".*", {git, "git://github.com/basho/riak_dt.git", {branch, "develop"}}}, {node_package, ".*", {git, "git://github.com/basho/node_package.git", {branch, "develop"}}}, -- 2.45.2 From 7f561f34e0762fa4673bfff898224066b26f2b57 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Tue, 27 Oct 2015 16:33:07 -0500 Subject: [PATCH 05/44] Ignore vim cruft too --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 80da416..0f6b627 100644 --- a/.gitignore +++ b/.gitignore @@ -26,3 +26,4 @@ rel/machi current_counterexample.eqc foo* typescript* +*.swp -- 2.45.2 From 5e571f6009c52bad000cb4738e6dc6f894876307 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Tue, 27 Oct 2015 16:33:18 -0500 Subject: [PATCH 06/44] Switch to merklet Still a WIP --- src/machi_merkle_tree_mgr.erl | 239 ++++++---------------------- test/machi_merkle_tree_mgr_test.erl | 5 +- 2 files changed, 55 insertions(+), 189 deletions(-) diff --git a/src/machi_merkle_tree_mgr.erl b/src/machi_merkle_tree_mgr.erl index 7a1e833..2e7a144 100644 --- a/src/machi_merkle_tree_mgr.erl +++ b/src/machi_merkle_tree_mgr.erl @@ -18,18 +18,10 @@ %% %% ------------------------------------------------------------------- -%% @doc This manager maintains a Merkle tree per file per FLU. The leaf -%% nodes are stored in the same manner as the checksum data files, except -%% they are represented as -%% -%% `<>' for unwritten bytes -%% `<>' for trimmed bytes -%% `<>' for written bytes -%% -%% In this case, the checksum tag is thrown away. The tree feeds these -%% leaf nodes into hashes representing 10 GB ranges, called Level 1. We aim for -%% around %% 10 hashes at level 2, and then 2 hashes level 3 and finally the -%% root. +%% @doc This manager maintains a Merkle tree per file per FLU as implemented +%% by the `merklet' library. Keys are encoded as `<>' +%% values encoded as `<>' *or* as <<0>> for unwritten +%% bytes, or <<1>> for trimmed bytes. -module(machi_merkle_tree_mgr). -behaviour(gen_server). @@ -41,8 +33,7 @@ start_link/3, initialize/2, update/5, - fetch/2, - fetch/3 + fetch/2 ]). %% gen_server callbacks @@ -63,24 +54,14 @@ -record(mt, { filename :: string(), - recalc = true :: boolean(), - root :: 'undefined' | binary(), - lvl1 = [] :: [ binary() ], - lvl2 = [] :: [ binary() ], - lvl3 = [] :: [ binary() ], - leaves = orddict:new() :: mt_entry() + tree :: merklet:tree() }). --type mt_entry() :: orddict:orddict(). - --define(WRITTEN(Offset, Size, Csum), <>). --define(TRIMMED(Offset, Size), <>). --define(UNWRITTEN(Offset, Size), <>). - --define(CHUNK_SIZE, (10*1024*1024)). --define(LEVEL_SIZE, 10). --define(H, sha). +-define(TRIMMED, <<1>>). +-define(UNWRITTEN, <<0>>). +-define(ENCODE(Offset, Size), <>). +-define(NEW_MERKLET, undefined). -define(TIMEOUT, (10*1000)). %% public API @@ -117,23 +98,11 @@ update(FluName, Filename, Offset, Length, Csum) -> {update, Filename, Offset, Length, Csum}). -spec fetch ( FluName :: atom(), - Filename :: string() ) -> {ok, [ Data :: binary() ]}. -%% @doc Returns the entire merkle tree for the given filename. -%% {@link fetch/3} + Filename :: string() ) -> {ok, 'undefined'|merklet:tree()}. +%% @doc Returns the merkle tree for the given filename. fetch(FluName, Filename) -> - fetch(FluName, Filename, all). - --spec fetch( FluName :: atom(), - Filename :: string(), - Level :: 'root' | 'all' - ) -> {ok, [ Data :: binary() ]}. -%% @doc Return the current merkle tree for the given filename. -%% If `root' is specified, returns a list with 1 element, the root -%% checksum of the tree. If `all' is specified, returns a list -%% with all levels. -fetch(FluName, Filename, Level) -> gen_server:call(make_merkle_tree_mgr_name(FluName), - {fetch, Filename, Level}, ?TIMEOUT). + {fetch, Filename}, ?TIMEOUT). %% gen_server callbacks init({FluName, DataDir, Options}) -> @@ -146,8 +115,8 @@ init({FluName, DataDir, Options}) -> end, {ok, #state{fluname=FluName, datadir=DataDir, tid = Tid}}. -handle_call({fetch, Filename, Level}, _From, S = #state{ tid = Tid }) -> - Res = handle_fetch(Tid, Filename, Level), +handle_call({fetch, Filename}, _From, S = #state{ tid = Tid }) -> + Res = handle_fetch(Tid, Filename), {reply, {ok, Res}, S}; handle_call(Req, _From, State) -> lager:warning("Unknown call: ~p", [Req]), @@ -193,151 +162,47 @@ get_files(DataDir) -> load_filename(Tid, DataDir, Filename) -> CsumFile = machi_util:make_checksum_filename(DataDir, Filename), - case file:read_file(CsumFile) of - {error, enoent} -> - insert(Tid, Filename, {0, ?MINIMUM_OFFSET, unwritten}); - {ok, Bin} -> - load_bin(Tid, Filename, Bin); - Error -> - throw(Error) - end. - -load_bin(Tid, Filename, Bin) -> - {CsumL, _} = machi_csum_table:split_checksum_list_blob_decode(Bin), - iter_csum_list(Tid, Filename, CsumL). - -iter_csum_list(Tid, Filename, []) -> - insert(Tid, Filename, {0, ?MINIMUM_OFFSET, unwritten}); -iter_csum_list(Tid, Filename, L = [ {Last, _, _} | _ ]) -> - make_insert(Tid, Filename, Last, L). - -make_insert(_Tid, _Filename, _Last, []) -> - ok; -%% case where Last offset matches Current, just insert current -make_insert(Tid, Filename, Last, [H={Last, Len, _Csum}|T]) -> - insert(Tid, Filename, H), - make_insert(Tid, Filename, Last+Len, T); -%% case where we have a hole -make_insert(Tid, Filename, Last, [H={Off, Len, _Csum}|T]) -> - Hole = Off - Last, - insert(Tid, Filename, {Last, Hole, unwritten}), - insert(Tid, Filename, H), - make_insert(Tid, Filename, Off+Len, T). - -insert(Tid, Filename, {Offset, Length, trimmed}) -> - do_insert(Tid, Filename, Offset, Length, ?TRIMMED(Offset, Length)); - -insert(Tid, Filename, {Offset, Length, unwritten}) -> - do_insert(Tid, Filename, Offset, Length, ?UNWRITTEN(Offset, Length)); - -insert(Tid, Filename, {Offset, Length, <<_Tag:8, Csum/binary>>}) -> - do_insert(Tid, Filename, Offset, Length, ?WRITTEN(Offset, Length, Csum)). - -do_insert(Tid, Filename, Offset, Length, Csum) -> - MT = case find(Tid, Filename) of - [] -> - #mt{ filename = Filename }; - V -> - V - end, - ok = maybe_update(Tid, Offset, Length, Csum, MT), - ok. - -maybe_update(Tid, Offset, Length, Csum, MT) -> - case orddict:find({Offset, Length}, MT#mt.leaves) of - error -> - %% range not found in our orddict, so fill it in - do_update(Tid, Offset, Length, Csum, MT); - {ok, Csum} -> - %% trying to insert a value we already have that - %% matches - ok; - {ok, ?UNWRITTEN(Offset, Length)} -> - %% old value was unwritten, now we are filling it in - %% so that's legit - do_update(Tid, Offset, Length, Csum, MT); - {ok, ?TRIMMED(Offset, Length)} -> - %% XXX FIXME - %% Scott - range was trimmed - do we fill it in with new stuff? - ok; - {ok, Other} -> - %% we found a checksum that is different - %% this shouldn't happen because once we write a range, it's written - %% TODO - file corruption? insanity? - lager:error("Tried to update merkle tree for file ~p at offset ~p, length ~p, got checksum ~p and tried to insert ~p", - [MT#mt.filename, Offset, Length, Other, Csum]), - throw({weird, Other}) - end. - -do_update(Tid, Offset, Length, Csum, MT) -> - D = orddict:store({Offset, Length}, Csum, MT#mt.leaves), - true = ets:insert(Tid, MT#mt{ recalc = true, leaves = D }), + {ok, T} = machi_csum_table:open(CsumFile, []), + %% docs say that the traversal order of ets:foldl is non-determinstic + %% but hopefully since csum_table uses an ordered set that's not true... + {_LastPosition, M} = machi_csum_table:foldl_chunks(fun insert_csum/2, + {?MINIMUM_OFFSET, ?NEW_MERKLET}, T), + true = ets:insert_new(Tid, #mt{ filename = Filename, tree = M}), + ok = machi_csum_table:close(T), ok. -handle_fetch(Tid, Filename, root) -> - case find(Tid, Filename) of - [] -> undefined; - #mt{ root = undefined } -> undefined; - #mt{ recalc = true } = MT -> hd(build_tree(Tid, MT)); - #mt{ root = R, recalc = false } -> R - end; +insert_csum({Last, Size, _Csum}=In, {Last, MT}) -> + %% no gap here, insert a record + {Last+Size, update_merkle_tree(In, MT)}; +insert_csum({Offset, Size, _Csum}=In, {Last, MT}) -> + %% gap here, insert unwritten record + %% *AND* insert written record + Hole = Offset - Last, + MT0 = update_merkle_tree({Last, Hole, unwritten}, MT), + {Offset+Size, update_merkle_tree(In, MT0)}. -handle_fetch(Tid, Filename, all) -> - case find(Tid, Filename) of - [] -> undefined; - #mt{ recalc = true } = MT -> build_tree(Tid, MT); - #mt{ recalc = false, - root = R, - lvl1 = L1, - lvl2 = L2, - lvl3 = L3 } -> [ R, L3, L2, L1 ] +insert(Tid, Filename, Term) -> + case ets:lookup(Tid, Filename) of + [] -> error(not_found); %% TODO: Something better? + [R] -> + NewMT = update_merkle_tree(Term, R#mt.tree), + %% we choose update_element because it + %% makes atomic changes so it is concurrent + %% safe. The regular 'insert' function + %% does not provide that guarantee. + true = ets:update_element(Tid, Filename, {#mt.tree, NewMT}), + ok end. -find(Tid, Filename) -> - case ets:lookup(Tid, Filename) of - [] -> []; - [R] -> R +handle_fetch(Tid, Filename) -> + case ets:lookup(Tid, Filename) of + [] -> undefined; + [R] -> R#mt.tree end. -build_tree(Tid, MT = #mt{ leaves = D }) -> - Leaves = lists:map(fun map_dict/1, orddict:to_list(D)), - io:format(user, "Leaves: ~p~n", [Leaves]), - Lvl1s = build_level_1(?CHUNK_SIZE, Leaves, 1, [ crypto:hash_init(?H) ]), - io:format(user, "Lvl1: ~p~n", [Lvl1s]), - Mod2 = length(Lvl1s) div ?LEVEL_SIZE, - Lvl2s = build_int_level(Mod2, Lvl1s, 1, [ crypto:hash_init(?H) ]), - io:format(user, "Lvl2: ~p~n", [Lvl2s]), - Mod3 = length(Lvl2s) div 2, - Lvl3s = build_int_level(Mod3, Lvl2s, 1, [ crypto:hash_init(?H) ]), - io:format(user, "Lvl3: ~p~n", [Lvl3s]), - Root = build_root(Lvl3s, crypto:hash_init(?H)), - io:format(user, "Root: ~p~n", [Root]), - ets:insert(Tid, MT#mt{ root = Root, lvl1 = Lvl1s, lvl2 = Lvl2s, lvl3 = Lvl3s, recalc = false }), - [Root, Lvl3s, Lvl2s, Lvl1s]. - -build_root([], Ctx) -> - crypto:hash_final(Ctx); -build_root([H|T], Ctx) -> - build_root(T, crypto:hash_update(Ctx, H)). - -build_int_level(_Mod, [], _Cnt, [ Ctx | Rest ]) -> - lists:reverse( [ crypto:hash_final(Ctx) | Rest ] ); -build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) when Cnt rem Mod == 0 -> - NewCtx = crypto:hash_init(?H), - build_int_level(Mod, T, Cnt + 1, [ crypto:hash_update(NewCtx, H), crypto:hash_final(Ctx) | Rest ]); -build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) -> - build_int_level(Mod, T, Cnt+1, [ crypto:hash_update(Ctx, H) | Rest ]). - -map_dict({{Offset, Len}, Hash}) -> - {Offset + Len, Hash}. - -build_level_1(_Size, [], _Multiple, [ Ctx | Rest ]) -> - lists:reverse([ crypto:hash_final(Ctx) | Rest ]); -build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos > ( Size * Multiple ) -> - NewCtx = crypto:hash_init(?H), - build_level_1(Size, T, Multiple+1, - [ crypto:hash_update(NewCtx, Hash), crypto:hash_final(Ctx) | Rest ]); -build_level_1(Size, [{Pos, Hash}|T], Multiple, [ Ctx | Rest ]) when Pos =< ( Size * Multiple ) -> - io:format(user, "Size: ~p, Pos: ~p, Multiple: ~p~n", [Size, Pos, Multiple]), - build_level_1(Size, T, Multiple, [ crypto:hash_update(Ctx, Hash) | Rest ]). - +update_merkle_tree({Offset, Size, unwritten}, MT) -> + merklet:insert({?ENCODE(Offset, Size), ?UNWRITTEN}, MT); +update_merkle_tree({Offset, Size, trimmed}, MT) -> + merklet:insert({?ENCODE(Offset, Size), ?TRIMMED}, MT); +update_merkle_tree({Offset, Size, Csum}, MT) -> + merklet:insert({?ENCODE(Offset, Size), Csum}, MT). diff --git a/test/machi_merkle_tree_mgr_test.erl b/test/machi_merkle_tree_mgr_test.erl index 8e8336c..bc9e0db 100644 --- a/test/machi_merkle_tree_mgr_test.erl +++ b/test/machi_merkle_tree_mgr_test.erl @@ -91,8 +91,9 @@ test() -> _ = machi_merkle_tree_mgr:start_link(test, ".", []), machi_merkle_tree_mgr:initialize(test, ?TESTFILE), timer:sleep(1000), - Root = machi_merkle_tree_mgr:fetch(test, ?TESTFILE, root), - ?debugFmt("Root: ~p~n", [Root]), + All = machi_merkle_tree_mgr:fetch(test, ?TESTFILE), + ?debugFmt("All: ~p~n", [All]), + timer:sleep(1000), All = machi_merkle_tree_mgr:fetch(test, ?TESTFILE), ?debugFmt("All: ~p~n", [All]), ok. -- 2.45.2 From f7358424e4475d47a90b8e59d8882eeda06b0f4c Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Fri, 23 Oct 2015 18:19:25 +0900 Subject: [PATCH 07/44] Trim command and GC prototype implementation * maybe_gc/2 is triggered at machi_file_proxy, when chunk is deleted and the file is larger than `max_file_size` * A file is deleted if all chunks except 1024 bytes header are trimmed * If a file is going to be deleted, file_proxy notifies metadata_mgr to remember the filename persistently, whose filename is `known_files_` * Such trimmed filenames are stored in a machi_plist file per flu * machi_file_proxy could not be started if the filename is in the manager's list. Consequently, any write, read and trim operations cannot happen against deleted file. * After the file was trimmed, any read request to the file returns `{error, trimmed}` * Disclaimer: no tests written yet and machi_plist does not support any recovery from partial writes. * Add some thoughts as comments for repairing trims. * State diagram of every byte is as follows: ``` state\action| write/append | read_chunk | trim_chunk ------------+----------------+------------------+--------------- unwritten | -> written | fail (+repair) | -> trimmed written | noop or repair | return content | -> trimmed trimmed | fail | fail | noop ``` --- src/machi.proto | 44 +++++-- src/machi_cr_client.erl | 167 ++++++++++++++++++++++-- src/machi_csum_table.erl | 9 +- src/machi_file_proxy.erl | 197 ++++++++++++++++++++++------- src/machi_file_proxy_sup.erl | 3 +- src/machi_flu1.erl | 61 +++++++-- src/machi_flu1_client.erl | 26 ++++ src/machi_flu_filename_mgr.erl | 38 +++--- src/machi_flu_metadata_mgr.erl | 70 +++++++--- src/machi_pb_high_client.erl | 9 ++ src/machi_pb_translate.erl | 41 +++++- src/machi_plist.erl | 63 +++++++++ src/machi_proxy_flu1_client.erl | 16 +++ test/machi_csum_table_test.erl | 3 +- test/machi_file_proxy_eqc.erl | 2 +- test/machi_file_proxy_test.erl | 4 +- test/machi_pb_high_client_test.erl | 50 ++++++-- test/machi_plist_test.erl | 17 +++ 18 files changed, 687 insertions(+), 133 deletions(-) create mode 100644 src/machi_plist.erl create mode 100644 test/machi_plist_test.erl diff --git a/src/machi.proto b/src/machi.proto index e5d77d9..e583ba5 100644 --- a/src/machi.proto +++ b/src/machi.proto @@ -48,9 +48,10 @@ enum Mpb_GeneralStatusCode { PARTITION = 4; NOT_WRITTEN = 5; WRITTEN = 6; - NO_SUCH_FILE = 7; - PARTIAL_READ = 8; - BAD_EPOCH = 9; + TRIMMED = 7; // The whole file was trimmed + NO_SUCH_FILE = 8; + PARTIAL_READ = 9; + BAD_EPOCH = 10; BAD_JOSS = 255; // Only for testing by the Taipan } @@ -355,6 +356,7 @@ message Mpb_ProjectionV1 { // append_chunk() // write_chunk() // read_chunk() +// trim_chunk() // checksum_list() // list_files() // wedge_status() @@ -424,6 +426,20 @@ message Mpb_LL_ReadChunkResp { repeated Mpb_ChunkPos trimmed = 3; } +// Low level API: trim_chunk() + +message Mpb_LL_TrimChunkReq { + required Mpb_EpochID epoch_id = 1; + required string file = 2; + required uint64 offset = 3; + required uint32 size = 4; + optional uint32 trigger_gc = 5 [default=1]; +} + +message Mpb_LL_TrimChunkResp { + required Mpb_GeneralStatusCode status = 1; +} + // Low level API: checksum_list() message Mpb_LL_ChecksumListReq { @@ -588,11 +604,12 @@ message Mpb_LL_Request { optional Mpb_LL_AppendChunkReq append_chunk = 30; optional Mpb_LL_WriteChunkReq write_chunk = 31; optional Mpb_LL_ReadChunkReq read_chunk = 32; - optional Mpb_LL_ChecksumListReq checksum_list = 33; - optional Mpb_LL_ListFilesReq list_files = 34; - optional Mpb_LL_WedgeStatusReq wedge_status = 35; - optional Mpb_LL_DeleteMigrationReq delete_migration = 36; - optional Mpb_LL_TruncHackReq trunc_hack = 37; + optional Mpb_LL_TrimChunkReq trim_chunk = 33; + optional Mpb_LL_ChecksumListReq checksum_list = 34; + optional Mpb_LL_ListFilesReq list_files = 35; + optional Mpb_LL_WedgeStatusReq wedge_status = 36; + optional Mpb_LL_DeleteMigrationReq delete_migration = 37; + optional Mpb_LL_TruncHackReq trunc_hack = 38; } message Mpb_LL_Response { @@ -622,9 +639,10 @@ message Mpb_LL_Response { optional Mpb_LL_AppendChunkResp append_chunk = 30; optional Mpb_LL_WriteChunkResp write_chunk = 31; optional Mpb_LL_ReadChunkResp read_chunk = 32; - optional Mpb_LL_ChecksumListResp checksum_list = 33; - optional Mpb_LL_ListFilesResp list_files = 34; - optional Mpb_LL_WedgeStatusResp wedge_status = 35; - optional Mpb_LL_DeleteMigrationResp delete_migration = 36; - optional Mpb_LL_TruncHackResp trunc_hack = 37; + optional Mpb_LL_TrimChunkResp trim_chunk = 33; + optional Mpb_LL_ChecksumListResp checksum_list = 34; + optional Mpb_LL_ListFilesResp list_files = 35; + optional Mpb_LL_WedgeStatusResp wedge_status = 36; + optional Mpb_LL_DeleteMigrationResp delete_migration = 37; + optional Mpb_LL_TruncHackResp trunc_hack = 38; } diff --git a/src/machi_cr_client.erl b/src/machi_cr_client.erl index 7b88f45..e198c40 100644 --- a/src/machi_cr_client.erl +++ b/src/machi_cr_client.erl @@ -360,6 +360,9 @@ do_append_head3(Prefix, Chunk, ChunkExtra, Depth, STime, TO, %% written block is. But we lost a race. Repeat, with a new %% sequencer assignment. do_append_head(Prefix, Chunk, ChunkExtra, Depth, STime, TO, S); + {error, trimmed} = Err -> + %% TODO: behaviour + {reply, Err, S}; {error, not_written} -> exit({todo_should_never_happen,?MODULE,?LINE, Prefix,iolist_size(Chunk)}) @@ -406,7 +409,7 @@ do_append_midtail(_RestFLUs, Prefix, File, Offset, Chunk, ChunkExtra, Ws, Depth + 1, STime, TO, S2) end end - end. + end. do_append_midtail2([], _Prefix, File, Offset, Chunk, _ChunkExtra, _Ws, _Depth, _STime, _TO, S) -> @@ -434,6 +437,9 @@ do_append_midtail2([FLU|RestFLUs]=FLUs, Prefix, File, Offset, Chunk, Resume = {append, Offset, iolist_size(Chunk), File}, do_repair_chunk(FLUs, Resume, Chunk, [], File, Offset, iolist_size(Chunk), Depth, STime, S); + {error, trimmed} = Err -> + %% TODO: nothing can be done + {reply, Err, S}; {error, not_written} -> exit({todo_should_never_happen,?MODULE,?LINE,File,Offset}) end. @@ -497,6 +503,8 @@ do_write_head2(File, Offset, Chunk, Depth, STime, TO, do_write_head(File, Offset, Chunk, Depth, STime, TO, S); {error, written}=Err -> {reply, Err, S}; + {error, trimmed}=Err -> + {reply, Err, S}; {error, not_written} -> exit({todo_should_never_happen,?MODULE,?LINE, iolist_size(Chunk)}) @@ -528,18 +536,26 @@ do_read_chunk2(File, Offset, Size, Opts, Depth, STime, TO, ConsistencyMode = P#projection_v1.mode, case ?FLU_PC:read_chunk(orddict:fetch(Tail, PD), EpochID, File, Offset, Size, Opts, ?TIMEOUT) of - {ok, {Chunks, []}} when is_list(Chunks) -> - {reply, {ok, {Chunks, []}}, S}; + {ok, {Chunks, Trimmed}} when is_list(Chunks), is_list(Trimmed) -> + %% After partition heal, there could happen that heads may + %% have chunk trimmed but tails may have chunk written - + %% such repair couldn't be triggered in read time (because + %% there's data!). In this case, repair should happen by + %% partition heal event or some background + %% hashtree-n-repair service. TODO. FIXME. + {reply, {ok, {Chunks, Trimmed}}, S}; %% {ok, BadChunk} -> %% %% TODO cleaner handling of bad chunks %% exit({todo, bad_chunk_size, ?MODULE, ?LINE, File, Offset, Size, %% got, byte_size(BadChunk)}); - {error, bad_arg} = BadArg -> + {error, bad_arg} = BadArg -> {reply, BadArg, S}; {error, partial_read}=Err -> + %% TODO: maybe this case we might need another repair? {reply, Err, S}; {error, bad_checksum}=BadCS -> %% TODO: alternate strategy? + %% Maybe we need read repair here, too? {reply, BadCS, S}; {error, Retry} when Retry == partition; Retry == bad_epoch; Retry == wedged -> @@ -548,12 +564,125 @@ do_read_chunk2(File, Offset, Size, Opts, Depth, STime, TO, read_repair(ConsistencyMode, read, File, Offset, Size, Depth, STime, S); %% {reply, {error, not_written}, S}; {error, written} -> - exit({todo_should_never_happen,?MODULE,?LINE,File,Offset,Size}) + exit({todo_should_never_happen,?MODULE,?LINE,File,Offset,Size}); + {error, trimmed}=Err -> + {reply, Err, S} + end. + +do_trim_chunk(File, Offset, Size, 0=Depth, STime, TO, S) -> + do_trim_chunk(File, Offset, Size, Depth+1, STime, TO, S); + +do_trim_chunk(File, Offset, Size, Depth, STime, TO, #state{proj=P}=S) -> + sleep_a_while(Depth), + DiffMs = timer:now_diff(os:timestamp(), STime) div 1000, + if DiffMs > TO -> + {reply, {error, partition}, S}; + true -> + %% This is suboptimal for performance: there are some paths + %% through this point where our current projection is good + %% enough. But we're going to try to keep the code as simple + %% as we can for now. + S2 = update_proj(S#state{proj=undefined, bad_proj=P}), + case S2#state.proj of + P2 when P2 == undefined orelse + P2#projection_v1.upi == [] -> + do_trim_chunk(File, Offset, Size, Depth + 1, + STime, TO, S2); + _ -> + do_trim_chunk2(File, Offset, Size, Depth + 1, + STime, TO, S2) + end + end. + +do_trim_chunk2(File, Offset, Size, Depth, STime, TO, + #state{epoch_id=EpochID, proj=P, proxies_dict=PD}=S) -> + [HeadFLU|RestFLUs] = mutation_flus(P), + Proxy = orddict:fetch(HeadFLU, PD), + case ?FLU_PC:trim_chunk(Proxy, EpochID, File, Offset, Size, ?TIMEOUT) of + ok -> + %% From this point onward, we use the same code & logic path as + %% append does. + do_trim_midtail(RestFLUs, undefined, File, Offset, Size, + [HeadFLU], 0, STime, TO, S); + {error, trimmed} -> + %% Maybe the trim had failed in the middle of the tail so re-run + %% trim accross the whole chain. + do_trim_midtail(RestFLUs, undefined, File, Offset, Size, + [HeadFLU], 0, STime, TO, S); + {error, bad_checksum}=BadCS -> + {reply, BadCS, S}; + {error, Retry} + when Retry == partition; Retry == bad_epoch; Retry == wedged -> + do_trim_chunk(File, Offset, Size, Depth, STime, TO, S) + end. + +do_trim_midtail(RestFLUs, Prefix, File, Offset, Size, + Ws, Depth, STime, TO, S) + when RestFLUs == [] orelse Depth == 0 -> + do_trim_midtail2(RestFLUs, Prefix, File, Offset, Size, + Ws, Depth + 1, STime, TO, S); +do_trim_midtail(_RestFLUs, Prefix, File, Offset, Size, + Ws, Depth, STime, TO, #state{proj=P}=S) -> + %% io:format(user, "midtail sleep2,", []), + sleep_a_while(Depth), + DiffMs = timer:now_diff(os:timestamp(), STime) div 1000, + if DiffMs > TO -> + {reply, {error, partition}, S}; + true -> + S2 = update_proj(S#state{proj=undefined, bad_proj=P}), + case S2#state.proj of + undefined -> + {reply, {error, partition}, S}; + P2 -> + RestFLUs2 = mutation_flus(P2), + case RestFLUs2 -- Ws of + RestFLUs2 -> + %% None of the writes that we have done so far + %% are to FLUs that are in the RestFLUs2 list. + %% We are pessimistic here and assume that + %% those FLUs are permanently dead. Start + %% over with a new sequencer assignment, at + %% the 2nd have of the impl (we have already + %% slept & refreshed the projection). + + if Prefix == undefined -> % atom! not binary()!! + {error, partition}; + true -> + do_trim_chunk(Prefix, Offset, Size, + Depth, STime, TO, S2) + end; + RestFLUs3 -> + do_trim_midtail2(RestFLUs3, Prefix, File, Offset, Size, + Ws, Depth + 1, STime, TO, S2) + end + end + end. + +do_trim_midtail2([], _Prefix, _File, _Offset, _Size, + _Ws, _Depth, _STime, _TO, S) -> + %% io:format(user, "ok!\n", []), + {reply, ok, S}; +do_trim_midtail2([FLU|RestFLUs]=FLUs, Prefix, File, Offset, Size, + Ws, Depth, STime, TO, + #state{epoch_id=EpochID, proxies_dict=PD}=S) -> + Proxy = orddict:fetch(FLU, PD), + case ?FLU_PC:trim_chunk(Proxy, EpochID, File, Offset, Size, ?TIMEOUT) of + ok -> + %% io:format(user, "write ~w,", [FLU]), + do_trim_midtail2(RestFLUs, Prefix, File, Offset, Size, + [FLU|Ws], Depth, STime, TO, S); + {error, trimmed} -> + do_trim_midtail2(RestFLUs, Prefix, File, Offset, Size, + [FLU|Ws], Depth, STime, TO, S); + {error, bad_checksum}=BadCS -> + %% TODO: alternate strategy? + {reply, BadCS, S}; + {error, Retry} + when Retry == partition; Retry == bad_epoch; Retry == wedged -> + do_trim_midtail(FLUs, Prefix, File, Offset, Size, + Ws, Depth, STime, TO, S) end. -do_trim_chunk(_File, _Offset, _Size, _Depth, _STime, _TO, S) -> - %% This is just a stub to reach CR client from high level client - {reply, {error, bad_joss}, S}. %% Read repair: depends on the consistency mode that we're in: %% @@ -597,6 +726,7 @@ read_repair2(cp_mode=ConsistencyMode, case ?FLU_PC:read_chunk(orddict:fetch(Tail, PD), EpochID, File, Offset, Size, [], ?TIMEOUT) of {ok, Chunks} when is_list(Chunks) -> + %% TODO: change to {Chunks, Trimmed} and have them repaired ToRepair = mutation_flus(P) -- [Tail], {Reply, S1} = do_repair_chunks(Chunks, ToRepair, ReturnMode, [Tail], File, Depth, STime, S, {ok, Chunks}), @@ -614,7 +744,12 @@ read_repair2(cp_mode=ConsistencyMode, {error, not_written} -> {reply, {error, not_written}, S}; {error, written} -> - exit({todo_should_never_happen,?MODULE,?LINE,File,Offset,Size}) + exit({todo_should_never_happen,?MODULE,?LINE,File,Offset,Size}); + {error, trimmed} -> + %% TODO: Again, whole file was trimmed. Needs repair. How + %% do we repair trimmed file (which was already unlinked) + %% across the flu servers? + exit({todo_should_repair_unlinked_files, ?MODULE, ?LINE, File}) end; read_repair2(ap_mode=ConsistencyMode, ReturnMode, File, Offset, Size, Depth, STime, @@ -622,6 +757,7 @@ read_repair2(ap_mode=ConsistencyMode, Eligible = mutation_flus(P), case try_to_find_chunk(Eligible, File, Offset, Size, S) of {ok, {Chunks, _Trimmed}, GotItFrom} when is_list(Chunks) -> + %% TODO: Repair trimmed chunks ToRepair = mutation_flus(P) -- [GotItFrom], {Reply0, S1} = do_repair_chunks(Chunks, ToRepair, ReturnMode, [GotItFrom], File, Depth, STime, S, {ok, Chunks}), @@ -638,7 +774,11 @@ read_repair2(ap_mode=ConsistencyMode, {error, not_written} -> {reply, {error, not_written}, S}; {error, written} -> - exit({todo_should_never_happen,?MODULE,?LINE,File,Offset,Size}) + exit({todo_should_never_happen,?MODULE,?LINE,File,Offset,Size}); + {error, trimmed} -> + %% TODO: Again, whole file was trimmed. Needs repair. How + %% do we repair trimmed file across the flu servers? + exit({todo_should_repair_unlinked_files, ?MODULE, ?LINE, File}) end. do_repair_chunks([], _, _, _, _, _, _, S, Reply) -> @@ -703,6 +843,9 @@ do_repair_chunk2([First|Rest]=ToRepair, ReturnMode, Chunk, Repaired, File, Offse %% that it is exactly our Chunk. do_repair_chunk2(Rest, ReturnMode, Chunk, Repaired, File, Offset, Size, Depth, STime, S); + {error, trimmed} = _Error -> + %% TODO + exit(todo_should_repair_trimmed); {error, not_written} -> exit({todo_should_never_happen,?MODULE,?LINE,File,Offset,Size}) end. @@ -872,7 +1015,9 @@ try_to_find_chunk(Eligible, File, Offset, Size, [{FoundFLU, {ok, ChunkAndTrimmed}}|_] -> {ok, ChunkAndTrimmed, FoundFLU}; [] -> - RetryErrs = [partition, bad_epoch, wedged], + RetryErrs = [partition, bad_epoch, wedged, trimmed], + %% Adding 'trimmed' to return so as to trigger repair, + %% once all other retry errors fixed case [Err || {error, Err} <- Rs, lists:member(Err, RetryErrs)] of [SomeErr|_] -> {error, SomeErr}; diff --git a/src/machi_csum_table.erl b/src/machi_csum_table.erl index 80f1765..7921df3 100644 --- a/src/machi_csum_table.erl +++ b/src/machi_csum_table.erl @@ -177,7 +177,14 @@ all_trimmed(#machi_csum_table{table=T}, Left, Right) -> -spec all_trimmed(table(), machi_dt:chunk_pos()) -> boolean(). all_trimmed(#machi_csum_table{table=T}, Pos) -> - runthru(ets:tab2list(T), 0, Pos). + case ets:tab2list(T) of + [{0, ?MINIMUM_OFFSET, _}|L] -> + %% tl/1 to remove header space {0, 1024, <<0>>} + runthru(L, ?MINIMUM_OFFSET, Pos); + List -> + %% In case a header is removed; + runthru(List, 0, Pos) + end. -spec any_trimmed(table(), pos_integer(), diff --git a/src/machi_file_proxy.erl b/src/machi_file_proxy.erl index ed9a933..a7bad0f 100644 --- a/src/machi_file_proxy.erl +++ b/src/machi_file_proxy.erl @@ -22,20 +22,20 @@ %% controlled files. In particular, it manages the "write-once register" %% conceit at the heart of Machi's design. %% -%% Read, write and append requests for a single file will be managed +%% Read, write and append requests for a single file will be managed %% through this proxy. Clients can also request syncs for specific %% types of filehandles. %% %% As operations are requested, the proxy keeps track of how many %% operations it has performed (and how many errors were generated.) -%% After a sufficient number of inactivity, the server terminates +%% After a sufficient number of inactivity, the server terminates %% itself. %% %% TODO: -%% 1. Some way to transition the proxy into a wedged state that +%% 1. Some way to transition the proxy into a wedged state that %% doesn't rely on message delivery. %% -%% 2. Check max file size on appends. Writes we take on faith we can +%% 2. Check max file size on appends. Writes we take on faith we can %% and should handle. %% %% 3. Async checksum reads on startup. @@ -47,7 +47,7 @@ %% public API -export([ - start_link/2, + start_link/3, stop/1, sync/1, sync/2, @@ -55,6 +55,7 @@ read/4, write/3, write/4, + trim/4, append/2, append/4 ]). @@ -74,10 +75,11 @@ -define(TIMEOUT, 10*1000). -define(TOO_MANY_ERRORS_RATIO, 50). --type op_stats() :: { Total :: non_neg_integer(), +-type op_stats() :: { Total :: non_neg_integer(), Errors :: non_neg_integer() }. -record(state, { + fluname :: atom(), data_dir :: string() | undefined, filename :: string() | undefined, data_path :: string() | undefined, @@ -93,17 +95,18 @@ ops = 0 :: non_neg_integer(), %% sum of all ops reads = {0, 0} :: op_stats(), writes = {0, 0} :: op_stats(), - appends = {0, 0} :: op_stats() + appends = {0, 0} :: op_stats(), + trims = {0, 0} :: op_stats() }). %% Public API -% @doc Start a new instance of the file proxy service. Takes the filename +% @doc Start a new instance of the file proxy service. Takes the filename % and data directory as arguments. This function is typically called by the % `machi_file_proxy_sup:start_proxy/2' function. --spec start_link(Filename :: string(), DataDir :: string()) -> any(). -start_link(Filename, DataDir) -> - gen_server:start_link(?MODULE, {Filename, DataDir}, []). +-spec start_link(FluName :: atom(), Filename :: string(), DataDir :: string()) -> any(). +start_link(FluName, Filename, DataDir) -> + gen_server:start_link(?MODULE, {FluName, Filename, DataDir}, []). % @doc Request to stop an instance of the file proxy service. -spec stop(Pid :: pid()) -> ok. @@ -120,7 +123,7 @@ sync(_Pid) -> % @doc Force a sync of a specific filehandle type. Valid types are `all', `csum' and `data'. -spec sync(Pid :: pid(), Type :: all|data|csum) -> ok|{error, term()}. -sync(Pid, Type) when is_pid(Pid) andalso +sync(Pid, Type) when is_pid(Pid) andalso ( Type =:= all orelse Type =:= csum orelse Type =:= data ) -> gen_server:call(Pid, {sync, Type}, ?TIMEOUT); sync(_Pid, Type) -> @@ -147,7 +150,7 @@ read(Pid, Offset, Length) -> {ok, [{Filename::string(), Offset :: non_neg_integer(), Data :: binary(), Checksum :: binary()}]} | {error, Reason :: term()}. -read(Pid, Offset, Length, Opts) when is_pid(Pid) andalso is_integer(Offset) andalso Offset >= 0 +read(Pid, Offset, Length, Opts) when is_pid(Pid) andalso is_integer(Offset) andalso Offset >= 0 andalso is_integer(Length) andalso Length > 0 andalso is_list(Opts) -> gen_server:call(Pid, {read, Offset, Length, Opts}, ?TIMEOUT); @@ -179,6 +182,12 @@ write(_Pid, Offset, ClientMeta, _Data) -> lager:warning("Bad arg to write: Offset ~p, ClientMeta: ~p", [Offset, ClientMeta]), {error, bad_arg}. +trim(Pid, Offset, Size, TriggerGC) when is_pid(Pid), + is_integer(Offset) andalso Offset >= 0, + is_integer(Size) andalso Size > 0, + is_boolean(TriggerGC) -> + gen_server:call(Pid, {trim ,Offset, Size, TriggerGC}, ?TIMEOUT). + % @doc Append data -spec append(Pid :: pid(), Data :: binary()) -> {ok, File :: string(), Offset :: non_neg_integer()} |{error, term()}. @@ -194,8 +203,8 @@ append(_Pid, _Data) -> -spec append(Pid :: pid(), ClientMeta :: proplists:proplist(), Extra :: non_neg_integer(), Data :: binary()) -> {ok, File :: string(), Offset :: non_neg_integer()} |{error, term()}. -append(Pid, ClientMeta, Extra, Data) when is_pid(Pid) andalso is_list(ClientMeta) - andalso is_integer(Extra) andalso Extra >= 0 +append(Pid, ClientMeta, Extra, Data) when is_pid(Pid) andalso is_list(ClientMeta) + andalso is_integer(Extra) andalso Extra >= 0 andalso is_binary(Data) -> gen_server:call(Pid, {append, ClientMeta, Extra, Data}, ?TIMEOUT); append(_Pid, ClientMeta, Extra, _Data) -> @@ -205,7 +214,7 @@ append(_Pid, ClientMeta, Extra, _Data) -> %% gen_server callbacks % @private -init({Filename, DataDir}) -> +init({FluName, Filename, DataDir}) -> CsumFile = machi_util:make_checksum_filename(DataDir, Filename), {_, DPath} = machi_util:make_data_filename(DataDir, Filename), ok = filelib:ensure_dir(CsumFile), @@ -216,6 +225,7 @@ init({Filename, DataDir}) -> {ok, FHd} = file:open(DPath, [read, write, binary, raw]), Tref = schedule_tick(), St = #state{ + fluname = FluName, filename = Filename, data_dir = DataDir, data_path = DPath, @@ -250,13 +260,13 @@ handle_call({sync, all}, _From, State = #state{filename = F, R1 = file:sync(FHd), Resp = case {R, R1} of {ok, ok} -> ok; - {ok, O1} -> - lager:error("Got ~p during a data file sync on file ~p", [O1, F]), + {ok, O1} -> + lager:error("Got ~p during a data file sync on file ~p", [O1, F]), O1; - {O2, ok} -> - lager:error("Got ~p during a csum file sync on file ~p", [O2, F]), + {O2, ok} -> + lager:error("Got ~p during a csum file sync on file ~p", [O2, F]), O2; - {O3, O4} -> + {O3, O4} -> lager:error("Got ~p ~p syncing all files for file ~p", [O3, O4, F]), {O3, O4} end, @@ -285,16 +295,21 @@ handle_call({read, Offset, Length, Opts}, _From, csum_table = CsumTable, reads = {T, Err} }) -> + %% TODO: use these options - NoChunk prevents reading from disks + %% NoChecksum doesn't check checksums NoChecksum = proplists:get_value(no_checksum, Opts, false), NoChunk = proplists:get_value(no_chunk, Opts, false), - NeedsMerge = proplists:get_value(needs_trimmed, Opts, false), {Resp, NewErr} = - case do_read(FH, F, CsumTable, Offset, Length, NoChecksum, NoChunk, NeedsMerge) of + case do_read(FH, F, CsumTable, Offset, Length, NoChunk, NoChecksum) of {ok, {[], []}} -> {{error, not_written}, Err + 1}; {ok, {Chunks0, Trimmed0}} -> Chunks = slice_both_side(Chunks0, Offset, Offset+Length), - {{ok, {Chunks, Trimmed0}}, Err}; + Trimmed = case proplists:get_value(needs_trimmed, Opts, false) of + true -> Trimmed0; + false -> [] + end, + {{ok, {Chunks, Trimmed}}, Err}; Error -> lager:error("Can't read ~p, ~p at File ~p", [Offset, Length, F]), {Error, Err + 1} @@ -338,6 +353,45 @@ handle_call({write, Offset, ClientMeta, Data}, _From, {reply, Resp, State#state{writes = {T+1, NewErr}, eof_position = NewEof}}; + +%%% TRIMS + +handle_call({trim, _Offset, _ClientMeta, _Data}, _From, + State = #state{wedged = true, + writes = {T, Err} + }) -> + {reply, {error, wedged}, State#state{writes = {T + 1, Err + 1}}}; + +handle_call({trim, Offset, Size, _TriggerGC}, _From, + State = #state{data_filehandle=FHd, + ops = Ops, + trims = {T, Err}, + csum_table = CsumTable}) -> + + case machi_csum_table:all_trimmed(CsumTable, Offset, Size) of + true -> + NewState = State#state{ops=Ops+1, trims={T, Err+1}}, + %% All bytes of that range was already trimmed returns ok + %% here, not {error, trimmed}, which means the whole file + %% was trimmed + maybe_gc(ok, NewState); + false -> + LUpdate = maybe_regenerate_checksum( + FHd, + machi_csum_table:find_leftneighbor(CsumTable, Offset)), + RUpdate = maybe_regenerate_checksum( + FHd, + machi_csum_table:find_rightneighbor(CsumTable, Offset+Size)), + + case machi_csum_table:trim(CsumTable, Offset, Size, LUpdate, RUpdate) of + ok -> + NewState = State#state{ops=Ops+1, trims={T+1, Err}}, + maybe_gc(ok, NewState); + Error -> + {reply, Error, State#state{ops=Ops+1, trims={T, Err+1}}} + end + end; + %% APPENDS handle_call({append, _ClientMeta, _Extra, _Data}, _From, @@ -476,10 +530,20 @@ terminate(Reason, #state{filename = F, lager:info(" Reads: ~p/~p", [RT, RE]), lager:info(" Writes: ~p/~p", [WT, WE]), lager:info("Appends: ~p/~p", [AT, AE]), - ok = file:sync(FHd), - ok = file:close(FHd), - ok = machi_csum_table:sync(T), - ok = machi_csum_table:close(T), + case FHd of + undefined -> + noop; %% file deleted + _ -> + ok = file:sync(FHd), + ok = file:close(FHd) + end, + case T of + undefined -> + noop; %% file deleted + _ -> + ok = machi_csum_table:sync(T), + ok = machi_csum_table:close(T) + end, ok. % @private @@ -512,15 +576,14 @@ check_or_make_tagged_csum(Tag, InCsum, Data) when Tag == ?CSUM_TAG_CLIENT_SHA; check_or_make_tagged_csum(OtherTag, _ClientCsum, _Data) -> lager:warning("Unknown checksum tag ~p", [OtherTag]), {error, bad_checksum}. - + -spec do_read(FHd :: file:io_device(), Filename :: string(), CsumTable :: machi_csum_table:table(), Offset :: non_neg_integer(), Size :: non_neg_integer(), - NoChecksum :: boolean(), NoChunk :: boolean(), - NeedsTrimmed :: boolean() + NoChecksum :: boolean() ) -> {ok, Chunks :: [{string(), Offset::non_neg_integer(), binary(), Csum :: binary()}]} | {error, bad_checksum} | {error, partial_read} | @@ -539,23 +602,23 @@ check_or_make_tagged_csum(OtherTag, _ClientCsum, _Data) -> % tuple is returned. % % -do_read(FHd, Filename, CsumTable, Offset, Size, _, _, _) -> - do_read(FHd, Filename, CsumTable, Offset, Size). - -do_read(FHd, Filename, CsumTable, Offset, Size) -> +do_read(FHd, Filename, CsumTable, Offset, Size, _, _) -> %% Note that find/3 only returns overlapping chunks, both borders %% are not aligned to original Offset and Size. ChunkCsums = machi_csum_table:find(CsumTable, Offset, Size), - read_all_ranges(FHd, Filename, ChunkCsums, []). + read_all_ranges(FHd, Filename, ChunkCsums, [], []). -read_all_ranges(_, _, [], ReadChunks) -> +read_all_ranges(_, _, [], ReadChunks, TrimmedChunks) -> %% TODO: currently returns empty list of trimmed chunks - {ok, {lists:reverse(ReadChunks), []}}; + {ok, {lists:reverse(ReadChunks), lists:reverse(TrimmedChunks)}}; -read_all_ranges(FHd, Filename, [{Offset, Size, TaggedCsum}|T], ReadChunks) -> +read_all_ranges(FHd, Filename, [{Offset, Size, trimmed}|T], ReadChunks, TrimmedChunks) -> + read_all_ranges(FHd, Filename, T, ReadChunks, [{Filename, Offset, Size}|TrimmedChunks]); + +read_all_ranges(FHd, Filename, [{Offset, Size, TaggedCsum}|T], ReadChunks, TrimmedChunks) -> case file:pread(FHd, Offset, Size) of eof -> - read_all_ranges(FHd, Filename, T, ReadChunks); + read_all_ranges(FHd, Filename, T, ReadChunks, TrimmedChunks); {ok, Bytes} when byte_size(Bytes) == Size -> {Tag, Ck} = machi_util:unmake_tagged_csum(TaggedCsum), case check_or_make_tagged_csum(Tag, Ck, Bytes) of @@ -565,19 +628,21 @@ read_all_ranges(FHd, Filename, [{Offset, Size, TaggedCsum}|T], ReadChunks) -> {error, bad_checksum}; TaggedCsum -> read_all_ranges(FHd, Filename, T, - [{Filename, Offset, Bytes, TaggedCsum}|ReadChunks]); + [{Filename, Offset, Bytes, TaggedCsum}|ReadChunks], + TrimmedChunks); OtherCsum when Tag =:= ?CSUM_TAG_NONE -> %% XXX FIXME: Should we return something other than %% {ok, ....} in this case? read_all_ranges(FHd, Filename, T, - [{Filename, Offset, Bytes, OtherCsum}|ReadChunks]) + [{Filename, Offset, Bytes, OtherCsum}|ReadChunks], + TrimmedChunks) end; {ok, Partial} -> - lager:error("In file ~p, offset ~p, wanted to read ~p bytes, but got ~p", + lager:error("In file ~p, offset ~p, wanted to read ~p bytes, but got ~p", [Filename, Offset, Size, byte_size(Partial)]), {error, partial_read}; Other -> - lager:error("While reading file ~p, offset ~p, length ~p, got ~p", + lager:error("While reading file ~p, offset ~p, length ~p, got ~p", [Filename, Offset, Size, Other]), {error, Other} end. @@ -616,7 +681,7 @@ handle_write(FHd, CsumTable, Filename, TaggedCsum, Offset, Data) -> {error, Reason} end; [{Offset, Size, TaggedCsum}] -> - case do_read(FHd, Filename, CsumTable, Offset, Size, false, false, false) of + case do_read(FHd, Filename, CsumTable, Offset, Size, false, false) of {error, _} = E -> lager:warning("This should never happen: got ~p while reading" " at offset ~p in file ~p that's supposedly written", @@ -693,6 +758,8 @@ do_write(FHd, CsumTable, Filename, TaggedCsum, Offset, Size, Data) -> %% Dialyzer 'can never match': slice_both_side([], _, _) -> %% []; +slice_both_side([], _, _) -> + []; slice_both_side([{F, Offset, Chunk, _Csum}|L], LeftPos, RightPos) when Offset < LeftPos andalso LeftPos < RightPos -> TrashLen = 8 * (LeftPos - Offset), @@ -729,3 +796,43 @@ maybe_regenerate_checksum(FHd, {Offset, Size, _Csum}) -> Error -> throw(Error) end. + +%% GC: make sure unwritte bytes = [{Eof, infinity}] and Eof is > max file size +%% walk through the checksum table and make sure all chunks trimmed +%% Then unlink the file +-spec maybe_gc(term(), #state{}) -> + {reply, term(), #state{}} | {stop, normal, term(), #state{}}. +maybe_gc(Reply, S = #state{eof_position = Eof, + max_file_size = MaxFileSize}) when Eof < MaxFileSize -> + lager:debug("The file is still small; not trying GC (Eof, MaxFileSize) = (~p, ~p)~n", + [Eof, MaxFileSize]), + {reply, Reply, S}; +maybe_gc(Reply, S = #state{fluname=FluName, + data_filehandle = FHd, + data_dir = DataDir, + filename = Filename, + eof_position = Eof, + csum_table=CsumTable}) -> + case machi_csum_table:all_trimmed(CsumTable, Eof) of + true -> + lager:debug("GC? Let's do it: ~p.~n", [Filename]), + %% Before unlinking a file, it should inform + %% machi_flu_filename_mgr that this file is + %% deleted and mark it as "trimmed" to avoid + %% filename reuse and resurrection. Maybe garbage + %% will remain if a process crashed but it also + %% should be recovered at filename_mgr startup. + + %% Also, this should be informed *before* file proxy + %% deletes files. + ok = machi_flu_metadata_mgr:trim_file(FluName, {file, Filename}), + ok = file:close(FHd), + {_, DPath} = machi_util:make_data_filename(DataDir, Filename), + ok = file:delete(DPath), + machi_csum_table:delete(CsumTable), + {stop, normal, Reply, + S#state{data_filehandle=undefined, + csum_table=undefined}}; + false -> + {reply, Reply, S} + end. diff --git a/src/machi_file_proxy_sup.erl b/src/machi_file_proxy_sup.erl index dbb0fa6..a165a68 100644 --- a/src/machi_file_proxy_sup.erl +++ b/src/machi_file_proxy_sup.erl @@ -44,7 +44,8 @@ start_link(FluName) -> supervisor:start_link({local, make_proxy_name(FluName)}, ?MODULE, []). start_proxy(FluName, DataDir, Filename) -> - supervisor:start_child(make_proxy_name(FluName), [Filename, DataDir]). + supervisor:start_child(make_proxy_name(FluName), + [FluName, Filename, DataDir]). init([]) -> SupFlags = {simple_one_for_one, 1000, 10}, diff --git a/src/machi_flu1.erl b/src/machi_flu1.erl index cb269f2..042eaed 100644 --- a/src/machi_flu1.erl +++ b/src/machi_flu1.erl @@ -75,6 +75,7 @@ epoch_id :: 'undefined' | machi_dt:epoch_id(), pb_mode = undefined :: 'undefined' | 'high' | 'low', high_clnt :: 'undefined' | pid(), + trim_table :: ets:tid(), props = [] :: list() % proplist }). @@ -148,6 +149,7 @@ main2(FluName, TcpPort, DataDir, Props) -> {true, undefined} end, Witness_p = proplists:get_value(witness_mode, Props, false), + S0 = #state{flu_name=FluName, proj_store=ProjectionPid, tcp_port=TcpPort, @@ -409,8 +411,11 @@ do_pb_ll_request3({low_write_chunk, _EpochID, File, Offset, Chunk, CSum_tag, #state{witness=false}=S) -> {do_server_write_chunk(File, Offset, Chunk, CSum_tag, CSum, S), S}; do_pb_ll_request3({low_read_chunk, _EpochID, File, Offset, Size, Opts}, - #state{witness=false}=S) -> + #state{witness=false} = S) -> {do_server_read_chunk(File, Offset, Size, Opts, S), S}; +do_pb_ll_request3({low_trim_chunk, _EpochID, File, Offset, Size, TriggerGC}, + #state{witness=false}=S) -> + {do_server_trim_chunk(File, Offset, Size, TriggerGC, S), S}; do_pb_ll_request3({low_checksum_list, _EpochID, File}, #state{witness=false}=S) -> {do_server_checksum_listing(File, S), S}; @@ -541,21 +546,47 @@ do_server_append_chunk2(_PKey, Prefix, Chunk, CSum_tag, Client_CSum, do_server_write_chunk(File, Offset, Chunk, CSum_tag, CSum, #state{flu_name=FluName}) -> case sanitize_file_string(File) of ok -> - {ok, Pid} = machi_flu_metadata_mgr:start_proxy_pid(FluName, {file, File}), - Meta = [{client_csum_tag, CSum_tag}, {client_csum, CSum}], - machi_file_proxy:write(Pid, Offset, Meta, Chunk); + case machi_flu_metadata_mgr:start_proxy_pid(FluName, {file, File}) of + {ok, Pid} -> + Meta = [{client_csum_tag, CSum_tag}, {client_csum, CSum}], + machi_file_proxy:write(Pid, Offset, Meta, Chunk); + {error, trimmed} = Error -> + Error + end; _ -> {error, bad_arg} end. do_server_read_chunk(File, Offset, Size, Opts, #state{flu_name=FluName})-> - %% TODO: Look inside Opts someday. case sanitize_file_string(File) of ok -> - {ok, Pid} = machi_flu_metadata_mgr:start_proxy_pid(FluName, {file, File}), - case machi_file_proxy:read(Pid, Offset, Size, Opts) of - {ok, ChunksAndTrimmed} -> {ok, ChunksAndTrimmed}; - Other -> Other + case machi_flu_metadata_mgr:start_proxy_pid(FluName, {file, File}) of + {ok, Pid} -> + case machi_file_proxy:read(Pid, Offset, Size, Opts) of + %% XXX FIXME + %% For now we are omiting the checksum data because it blows up + %% protobufs. + {ok, ChunksAndTrimmed} -> {ok, ChunksAndTrimmed}; + Other -> Other + end; + {error, trimmed} = Error -> + Error + end; + _ -> + {error, bad_arg} + end. + +do_server_trim_chunk(File, Offset, Size, TriggerGC, #state{flu_name=FluName}) -> + lager:debug("Hi there! I'm trimming this: ~s, (~p, ~p), ~p~n", + [File, Offset, Size, TriggerGC]), + case sanitize_file_string(File) of + ok -> + case machi_flu_metadata_mgr:start_proxy_pid(FluName, {file, File}) of + {ok, Pid} -> + machi_file_proxy:trim(Pid, Offset, Size, TriggerGC); + {error, trimmed} = Trimmed -> + %% Should be returned back to (maybe) trigger repair + Trimmed end; _ -> {error, bad_arg} @@ -662,10 +693,14 @@ handle_append(Prefix, Chunk, Csum, Extra, FluName, EpochId) -> Res = machi_flu_filename_mgr:find_or_make_filename_from_prefix(FluName, EpochId, {prefix, Prefix}), case Res of {file, F} -> - {ok, Pid} = machi_flu_metadata_mgr:start_proxy_pid(FluName, {file, F}), - {Tag, CS} = machi_util:unmake_tagged_csum(Csum), - Meta = [{client_csum_tag, Tag}, {client_csum, CS}], - machi_file_proxy:append(Pid, Meta, Extra, Chunk); + case machi_flu_metadata_mgr:start_proxy_pid(FluName, {file, F}) of + {ok, Pid} -> + {Tag, CS} = machi_util:unmake_tagged_csum(Csum), + Meta = [{client_csum_tag, Tag}, {client_csum, CS}], + machi_file_proxy:append(Pid, Meta, Extra, Chunk); + {error, trimmed} = E -> + E + end; Error -> Error end. diff --git a/src/machi_flu1_client.erl b/src/machi_flu1_client.erl index da90618..ed8808c 100644 --- a/src/machi_flu1_client.erl +++ b/src/machi_flu1_client.erl @@ -80,6 +80,7 @@ %% For "internal" replication only. -export([ write_chunk/5, write_chunk/6, + trim_chunk/5, trim_chunk/6, delete_migration/3, delete_migration/4, trunc_hack/3, trunc_hack/4 ]). @@ -474,6 +475,31 @@ write_chunk(Host, TcpPort, EpochID, File, Offset, Chunk) disconnect(Sock) end. +%% @doc Restricted API: Write a chunk of already-sequenced data to +%% `File' at `Offset'. + +-spec trim_chunk(port_wrap(), machi_dt:epoch_id(), machi_dt:file_name(), machi_dt:file_offset(), machi_dt:chunk_size()) -> + ok | {error, machi_dt:error_general()} | {error, term()}. +trim_chunk(Sock, EpochID, File0, Offset, Size) + when Offset >= ?MINIMUM_OFFSET -> + ReqID = <<"id">>, + File = machi_util:make_binary(File0), + true = (Offset >= ?MINIMUM_OFFSET), + Req = machi_pb_translate:to_pb_request( + ReqID, + {low_trim_chunk, EpochID, File, Offset, Size, 0}), + do_pb_request_common(Sock, ReqID, Req). + +%% @doc Restricted API: Write a chunk of already-sequenced data to +%% `File' at `Offset'. + +-spec trim_chunk(machi_dt:inet_host(), machi_dt:inet_port(), + machi_dt:epoch_id(), machi_dt:file_name(), machi_dt:file_offset(), machi_dt:chunk_size()) -> + ok | {error, machi_dt:error_general()} | {error, term()}. +trim_chunk(_Host, _TcpPort, _EpochID, _File, _Offset, _Size) -> + not_used. + + %% @doc Restricted API: Delete a file after it has been successfully %% migrated. diff --git a/src/machi_flu_filename_mgr.erl b/src/machi_flu_filename_mgr.erl index ac505a9..54fdcfe 100644 --- a/src/machi_flu_filename_mgr.erl +++ b/src/machi_flu_filename_mgr.erl @@ -17,8 +17,8 @@ %% under the License. %% %% ------------------------------------------------------------------- -%% -%% @doc This process is responsible for managing filenames assigned to +%% +%% @doc This process is responsible for managing filenames assigned to %% prefixes. It's started out of `machi_flu_psup'. %% %% Supported operations include finding the "current" filename assigned to @@ -32,7 +32,7 @@ %% First it looks up the sequence number from the prefix name. If %% no sequence file is found, it uses 0 as the sequence number and searches %% for a matching file with the prefix and 0 as the sequence number. -%% If no file is found, the it generates a new filename by incorporating +%% If no file is found, the it generates a new filename by incorporating %% the given prefix, a randomly generated (v4) UUID and 0 as the %% sequence number. %% @@ -79,7 +79,7 @@ child_spec(FluName, DataDir) -> Name = make_filename_mgr_name(FluName), - {Name, + {Name, {?MODULE, start_link, [FluName, DataDir]}, permanent, 5000, worker, [?MODULE]}. @@ -87,8 +87,8 @@ start_link(FluName, DataDir) when is_atom(FluName) andalso is_list(DataDir) -> N = make_filename_mgr_name(FluName), gen_server:start_link({local, N}, ?MODULE, [FluName, DataDir], []). --spec find_or_make_filename_from_prefix( FluName :: atom(), - EpochId :: pv1_epoch_n(), +-spec find_or_make_filename_from_prefix( FluName :: atom(), + EpochId :: pv1_epoch_n(), Prefix :: {prefix, string()} ) -> {file, Filename :: string()} | {error, Reason :: term() } | timeout. % @doc Find the latest available or make a filename from a prefix. A prefix @@ -96,7 +96,7 @@ start_link(FluName, DataDir) when is_atom(FluName) andalso is_list(DataDir) -> % tuple in the form of `{file, F}' or an `{error, Reason}' find_or_make_filename_from_prefix(FluName, EpochId, {prefix, Prefix}) when is_atom(FluName) -> N = make_filename_mgr_name(FluName), - gen_server:call(N, {find_filename, EpochId, Prefix}, ?TIMEOUT); + gen_server:call(N, {find_filename, EpochId, Prefix}, ?TIMEOUT); find_or_make_filename_from_prefix(_FluName, _EpochId, Other) -> lager:error("~p is not a valid prefix.", [Other]), error(badarg). @@ -104,9 +104,9 @@ find_or_make_filename_from_prefix(_FluName, _EpochId, Other) -> -spec increment_prefix_sequence( FluName :: atom(), Prefix :: {prefix, string()} ) -> ok | {error, Reason :: term() } | timeout. % @doc Increment the sequence counter for a given prefix. Prefix should -% be in the form of `{prefix, P}'. +% be in the form of `{prefix, P}'. increment_prefix_sequence(FluName, {prefix, Prefix}) when is_atom(FluName) -> - gen_server:call(make_filename_mgr_name(FluName), {increment_sequence, Prefix}, ?TIMEOUT); + gen_server:call(make_filename_mgr_name(FluName), {increment_sequence, Prefix}, ?TIMEOUT); increment_prefix_sequence(_FluName, Other) -> lager:error("~p is not a valid prefix.", [Other]), error(badarg). @@ -117,7 +117,7 @@ increment_prefix_sequence(_FluName, Other) -> % all the data files associated with that prefix. Returns % a list. list_files_by_prefix(FluName, {prefix, Prefix}) when is_atom(FluName) -> - gen_server:call(make_filename_mgr_name(FluName), {list_files, Prefix}, ?TIMEOUT); + gen_server:call(make_filename_mgr_name(FluName), {list_files, Prefix}, ?TIMEOUT); list_files_by_prefix(_FluName, Other) -> lager:error("~p is not a valid prefix.", [Other]), error(badarg). @@ -125,7 +125,10 @@ list_files_by_prefix(_FluName, Other) -> %% gen_server API init([FluName, DataDir]) -> Tid = ets:new(make_filename_mgr_name(FluName), [named_table, {read_concurrency, true}]), - {ok, #state{ fluname = FluName, epoch = 0, datadir = DataDir, tid = Tid }}. + {ok, #state{fluname = FluName, + epoch = 0, + datadir = DataDir, + tid = Tid}}. handle_cast(Req, State) -> lager:warning("Got unknown cast ~p", [Req]), @@ -135,9 +138,9 @@ handle_cast(Req, State) -> %% the FLU has already validated that the caller's epoch id and the FLU's epoch id %% are the same. So we *assume* that remains the case here - that is to say, we %% are not wedged. -handle_call({find_filename, EpochId, Prefix}, _From, S = #state{ datadir = DataDir, - epoch = EpochId, - tid = Tid }) -> +handle_call({find_filename, EpochId, Prefix}, _From, S = #state{ datadir = DataDir, + epoch = EpochId, + tid = Tid}) -> %% Our state and the caller's epoch ids are the same. Business as usual. File = handle_find_file(Tid, Prefix, DataDir), {reply, {file, File}, S}; @@ -154,7 +157,7 @@ handle_call({increment_sequence, Prefix}, _From, S = #state{ datadir = DataDir } ok = machi_util:increment_max_filenum(DataDir, Prefix), {reply, ok, S}; handle_call({list_files, Prefix}, From, S = #state{ datadir = DataDir }) -> - spawn(fun() -> + spawn(fun() -> L = list_files(DataDir, Prefix), gen_server:reply(From, L) end), @@ -181,7 +184,7 @@ code_change(_OldVsn, State, _Extra) -> %% MIT License generate_uuid_v4_str() -> <> = crypto:strong_rand_bytes(16), - io_lib:format("~8.16.0b-~4.16.0b-4~3.16.0b-~4.16.0b-~12.16.0b", + io_lib:format("~8.16.0b-~4.16.0b-4~3.16.0b-~4.16.0b-~12.16.0b", [A, B, C band 16#0fff, D band 16#3fff bor 16#8000, E]). find_file(DataDir, Prefix, N) -> @@ -201,7 +204,7 @@ handle_find_file(Tid, Prefix, DataDir) -> [] -> {find_or_make_filename(Tid, DataDir, Prefix, N), false}; [H] -> {H, true}; - [Fn | _ ] = L -> + [Fn | _ ] = L -> lager:warning( "Searching for a matching file to prefix ~p and sequence number ~p gave multiples: ~p", [Prefix, N, L]), @@ -245,4 +248,3 @@ increment_and_cache_filename(Tid, DataDir, Prefix) -> -ifdef(TEST). -endif. - diff --git a/src/machi_flu_metadata_mgr.erl b/src/machi_flu_metadata_mgr.erl index c851f84..d4447ae 100644 --- a/src/machi_flu_metadata_mgr.erl +++ b/src/machi_flu_metadata_mgr.erl @@ -39,10 +39,13 @@ -define(HASH(X), erlang:phash2(X)). %% hash algorithm to use -define(TIMEOUT, 10 * 1000). %% 10 second timeout +-define(KNOWN_FILES_LIST_PREFIX, "known_files_"). + -record(state, {fluname :: atom(), datadir :: string(), tid :: ets:tid(), - cnt :: non_neg_integer() + cnt :: non_neg_integer(), + trimmed_files :: machi_plist:plist() }). %% This record goes in the ets table where filename is the key @@ -59,7 +62,8 @@ lookup_proxy_pid/2, start_proxy_pid/2, stop_proxy_pid/2, - build_metadata_mgr_name/2 + build_metadata_mgr_name/2, + trim_file/2 ]). %% gen_server callbacks @@ -97,10 +101,24 @@ start_proxy_pid(FluName, {file, Filename}) -> stop_proxy_pid(FluName, {file, Filename}) -> gen_server:call(get_manager_atom(FluName, Filename), {stop_proxy_pid, Filename}, ?TIMEOUT). +trim_file(FluName, {file, Filename}) -> + gen_server:call(get_manager_atom(FluName, Filename), {trim_file, Filename}, ?TIMEOUT). + %% gen_server callbacks init([FluName, Name, DataDir, Num]) -> + %% important: we'll need another persistent storage to + %% remember deleted (trimmed) file, to prevent resurrection after + %% flu restart and append. + FileListFileName = + filename:join([DataDir, ?KNOWN_FILES_LIST_PREFIX ++ atom_to_list(FluName)]), + {ok, PList} = machi_plist:open(FileListFileName, []), + %% TODO make sure all files non-existent, if any remaining files + %% here, just delete it. They're in the list *because* they're all + %% trimmed. + Tid = ets:new(Name, [{keypos, 2}, {read_concurrency, true}, {write_concurrency, true}]), - {ok, #state{ fluname = FluName, datadir = DataDir, tid = Tid, cnt = Num}}. + {ok, #state{fluname = FluName, datadir = DataDir, tid = Tid, cnt = Num, + trimmed_files=PList}}. handle_cast(Req, State) -> lager:warning("Got unknown cast ~p", [Req]), @@ -113,17 +131,25 @@ handle_call({proxy_pid, Filename}, _From, State = #state{ tid = Tid }) -> end, {reply, Reply, State}; -handle_call({start_proxy_pid, Filename}, _From, State = #state{ fluname = N, tid = Tid, datadir = D }) -> - NewR = case lookup_md(Tid, Filename) of - not_found -> - start_file_proxy(N, D, Filename); - #md{ proxy_pid = undefined } = R0 -> - start_file_proxy(N, D, R0); - #md{ proxy_pid = _Pid } = R1 -> - R1 - end, - update_ets(Tid, NewR), - {reply, {ok, NewR#md.proxy_pid}, State}; +handle_call({start_proxy_pid, Filename}, _From, + State = #state{ fluname = N, tid = Tid, datadir = D, + trimmed_files=TrimmedFiles}) -> + case machi_plist:find(TrimmedFiles, Filename) of + false -> + NewR = case lookup_md(Tid, Filename) of + not_found -> + start_file_proxy(N, D, Filename); + #md{ proxy_pid = undefined } = R0 -> + start_file_proxy(N, D, R0); + #md{ proxy_pid = _Pid } = R1 -> + R1 + end, + update_ets(Tid, NewR), + {reply, {ok, NewR#md.proxy_pid}, State}; + true -> + {reply, {error, trimmed}, State} + end; + handle_call({stop_proxy_pid, Filename}, _From, State = #state{ tid = Tid }) -> case lookup_md(Tid, Filename) of not_found -> @@ -137,6 +163,15 @@ handle_call({stop_proxy_pid, Filename}, _From, State = #state{ tid = Tid }) -> end, {reply, ok, State}; +handle_call({trim_file, Filename}, _, + S = #state{trimmed_files = TrimmedFiles }) -> + case machi_plist:add(TrimmedFiles, Filename) of + {ok, TrimmedFiles2} -> + {reply, ok, S#state{trimmed_files=TrimmedFiles2}}; + Error -> + {reply, Error, S} + end; + handle_call(Req, From, State) -> lager:warning("Got unknown call ~p from ~p", [Req, From]), {reply, hoge, State}. @@ -169,18 +204,21 @@ handle_info({'DOWN', Mref, process, Pid, wedged}, State = #state{ tid = Tid }) - lager:error("file proxy ~p shutdown because it's wedged", [Pid]), clear_ets(Tid, Mref), {noreply, State}; +handle_info({'DOWN', _Mref, process, Pid, trimmed}, State = #state{ tid = _Tid }) -> + lager:debug("file proxy ~p shutdown because the file was trimmed", [Pid]), + {noreply, State}; handle_info({'DOWN', Mref, process, Pid, Error}, State = #state{ tid = Tid }) -> lager:error("file proxy ~p shutdown because ~p", [Pid, Error]), clear_ets(Tid, Mref), {noreply, State}; - handle_info(Info, State) -> lager:warning("Got unknown info ~p", [Info]), {noreply, State}. -terminate(Reason, _State) -> +terminate(Reason, _State = #state{trimmed_files=TrimmedFiles}) -> lager:info("Shutting down because ~p", [Reason]), + machi_plist:close(TrimmedFiles), ok. code_change(_OldVsn, State, _Extra) -> diff --git a/src/machi_pb_high_client.erl b/src/machi_pb_high_client.erl index 5509803..ec2dfc6 100644 --- a/src/machi_pb_high_client.erl +++ b/src/machi_pb_high_client.erl @@ -94,6 +94,9 @@ write_chunk(PidSpec, File, Offset, Chunk, CSum) -> write_chunk(PidSpec, File, Offset, Chunk, CSum, Timeout) -> send_sync(PidSpec, {write_chunk, File, Offset, Chunk, CSum}, Timeout). +%% @doc Tries to read a chunk of a specified file. It returns `{ok, +%% {Chunks, TrimmedChunks}}' for live file while it returns `{error, +%% trimmed}' if all bytes of the file was trimmed. -spec read_chunk(pid(), string(), pos_integer(), pos_integer(), [{flag_no_checksum | flag_no_chunk | needs_trimmed, boolean()}]) -> {ok, {list(), list()}} | {error, term()}. @@ -107,6 +110,10 @@ read_chunk(PidSpec, File, Offset, Size, Options) -> read_chunk(PidSpec, File, Offset, Size, Options, Timeout) -> send_sync(PidSpec, {read_chunk, File, Offset, Size, Options}, Timeout). +%% @doc Trims arbitrary binary range of any file. TODO: Add option +%% specifying whether to trigger GC. +-spec trim_chunk(pid(), string(), non_neg_integer(), machi_dt:chunk_size()) -> + ok | {error, term()}. trim_chunk(PidSpec, File, Offset, Size) -> trim_chunk(PidSpec, File, Offset, Size, ?DEFAULT_TIMEOUT). @@ -415,6 +422,8 @@ convert_general_status_code('NOT_WRITTEN') -> {error, not_written}; convert_general_status_code('WRITTEN') -> {error, written}; +convert_general_status_code('TRIMMED') -> + {error, trimmed}; convert_general_status_code('NO_SUCH_FILE') -> {error, no_such_file}; convert_general_status_code('PARTIAL_READ') -> diff --git a/src/machi_pb_translate.erl b/src/machi_pb_translate.erl index cc26766..0b49908 100644 --- a/src/machi_pb_translate.erl +++ b/src/machi_pb_translate.erl @@ -88,6 +88,17 @@ from_pb_request(#mpb_ll_request{ offset=Offset, chunk_size=Size} = ChunkPos, {ReqID, {low_read_chunk, EpochID, File, Offset, Size, Opts}}; +from_pb_request(#mpb_ll_request{ + req_id=ReqID, + trim_chunk=#mpb_ll_trimchunkreq{ + epoch_id=PB_EpochID, + file=File, + offset=Offset, + size=Size, + trigger_gc=PB_TriggerGC}}) -> + EpochID = conv_to_epoch_id(PB_EpochID), + TriggerGC = conv_to_boolean(PB_TriggerGC), + {ReqID, {low_trim_chunk, EpochID, File, Offset, Size, TriggerGC}}; from_pb_request(#mpb_ll_request{ req_id=ReqID, checksum_list=#mpb_ll_checksumlistreq{ @@ -262,6 +273,10 @@ from_pb_response(#mpb_ll_response{ _ -> {ReqID, machi_pb_high_client:convert_general_status_code(Status)} end; +from_pb_response(#mpb_ll_response{ + req_id=ReqID, + trim_chunk=#mpb_ll_trimchunkresp{status=Status}}) -> + {ReqID, machi_pb_high_client:convert_general_status_code(Status)}; from_pb_response(#mpb_ll_response{ req_id=ReqID, checksum_list=#mpb_ll_checksumlistresp{ @@ -398,11 +413,10 @@ to_pb_request(ReqID, {low_write_chunk, EpochID, File, Offset, Chunk, CSum_tag, C chunk=Chunk, csum=PB_CSum}}}; to_pb_request(ReqID, {low_read_chunk, EpochID, File, Offset, Size, Opts}) -> - %% TODO: stop ignoring Opts ^_^ PB_EpochID = conv_from_epoch_id(EpochID), FNChecksum = proplists:get_value(no_checksum, Opts, false), FNChunk = proplists:get_value(no_chunk, Opts, false), - NeedsTrimmed = proplists:get_value(needs_merge, Opts, false), + NeedsTrimmed = proplists:get_value(needs_trimmed, Opts, false), #mpb_ll_request{ req_id=ReqID, do_not_alter=2, read_chunk=#mpb_ll_readchunkreq{ @@ -414,6 +428,15 @@ to_pb_request(ReqID, {low_read_chunk, EpochID, File, Offset, Size, Opts}) -> flag_no_checksum=machi_util:bool2int(FNChecksum), flag_no_chunk=machi_util:bool2int(FNChunk), flag_needs_trimmed=machi_util:bool2int(NeedsTrimmed)}}; +to_pb_request(ReqID, {low_trim_chunk, EpochID, File, Offset, Size, TriggerGC}) -> + PB_EpochID = conv_from_epoch_id(EpochID), + #mpb_ll_request{req_id=ReqID, do_not_alter=2, + trim_chunk=#mpb_ll_trimchunkreq{ + epoch_id=PB_EpochID, + file=File, + offset=Offset, + size=Size, + trigger_gc=TriggerGC}}; to_pb_request(ReqID, {low_checksum_list, EpochID, File}) -> PB_EpochID = conv_from_epoch_id(EpochID), #mpb_ll_request{req_id=ReqID, do_not_alter=2, @@ -524,6 +547,18 @@ to_pb_response(ReqID, {low_read_chunk, _EID, _Fl, _Off, _Sz, _Opts}, Resp)-> _Else -> make_ll_error_resp(ReqID, 66, io_lib:format("err ~p", [_Else])) end; +to_pb_response(ReqID, {low_trim_chunk, _, _, _, _, _}, Resp) -> + case Resp of + ok -> + #mpb_ll_response{req_id=ReqID, + trim_chunk=#mpb_ll_trimchunkresp{status='OK'}}; + {error, _}=Error -> + Status = conv_from_status(Error), + #mpb_ll_response{req_id=ReqID, + read_chunk=#mpb_ll_trimchunkresp{status=Status}}; + _Else -> + make_ll_error_resp(ReqID, 66, io_lib:format("err ~p", [_Else])) + end; to_pb_response(ReqID, {low_checksum_list, _EpochID, _File}, Resp) -> case Resp of {ok, Chunk} -> @@ -909,6 +944,8 @@ conv_from_status({error, not_written}) -> 'NOT_WRITTEN'; conv_from_status({error, written}) -> 'WRITTEN'; +conv_from_status({error, trimmed}) -> + 'TRIMMED'; conv_from_status({error, no_such_file}) -> 'NO_SUCH_FILE'; conv_from_status({error, partial_read}) -> diff --git a/src/machi_plist.erl b/src/machi_plist.erl new file mode 100644 index 0000000..4bf745e --- /dev/null +++ b/src/machi_plist.erl @@ -0,0 +1,63 @@ +-module(machi_plist). + +%%% @doc persistent list of binaries that support mutual exclusion + +-export([open/2, close/1, find/2, add/2]). + +-ifdef(TEST). +-export([all/1]). +-endif. + +-record(machi_plist, + {filename :: string(), + fd :: file:descriptor(), + list}). + +-type plist() :: #machi_plist{}. +-export_type([plist/0]). + +-spec open(filename:filename(), proplists:proplist()) -> + {ok, plist()} | {error, file:posix()}. +open(Filename, _Opt) -> + List = case file:read_file(Filename) of + {ok, <<>>} -> []; + {ok, Bin} -> binary_to_term(Bin); + {error, enoent} -> [] + end, + case file:open(Filename, [read, write, raw, binary, sync]) of + {ok, Fd} -> + {ok, #machi_plist{filename=Filename, + fd=Fd, + list=List}}; + Error -> + Error + end. + +-spec close(plist()) -> ok. +close(#machi_plist{fd=Fd}) -> + _ = file:close(Fd). + +-spec find(plist(), string()) -> boolean(). +find(#machi_plist{list=List}, Name) -> + lists:member(Name, List). + +-spec add(plist(), string()) -> {ok, plist()} | {error, file:posix()}. +add(Plist = #machi_plist{list=List0, fd=Fd}, Name) -> + case find(Plist, Name) of + true -> + {ok, Plist}; + false -> + List = lists:append(List0, [Name]), + case file:pwrite(Fd, 0, term_to_binary(List)) of + ok -> + {ok, Plist#machi_plist{list=List}}; + Error -> + Error + end + end. + +-ifdef(TEST). +-spec all(plist()) -> [file:filename()]. +all(#machi_plist{list=List}) -> + List. +-endif. diff --git a/src/machi_proxy_flu1_client.erl b/src/machi_proxy_flu1_client.erl index 93f3b95..2cbaabd 100644 --- a/src/machi_proxy_flu1_client.erl +++ b/src/machi_proxy_flu1_client.erl @@ -79,6 +79,7 @@ %% Internal API write_chunk/5, write_chunk/6, + trim_chunk/5, trim_chunk/6, %% Helpers stop_proxies/1, start_proxies/1 @@ -310,6 +311,18 @@ write_chunk(PidSpec, EpochID, File, Offset, Chunk, Timeout) -> Else end. + +trim_chunk(PidSpec, EpochID, File, Offset, Size) -> + trim_chunk(PidSpec, EpochID, File, Offset, Size, infinity). + +%% @doc Write a chunk (binary- or iolist-style) of data to a file +%% with `Prefix' at `Offset'. + +trim_chunk(PidSpec, EpochID, File, Offset, Chunk, Timeout) -> + gen_server:call(PidSpec, + {req, {trim_chunk, EpochID, File, Offset, Chunk}}, + Timeout). + %%%%%%%%%%%%%%%%%%%%%%%%%%% init([I]) -> @@ -383,6 +396,9 @@ make_req_fun({read_chunk, EpochID, File, Offset, Size, Opts}, make_req_fun({write_chunk, EpochID, File, Offset, Chunk}, #state{sock=Sock,i=#p_srvr{proto_mod=Mod}}) -> fun() -> Mod:write_chunk(Sock, EpochID, File, Offset, Chunk) end; +make_req_fun({trim_chunk, EpochID, File, Offset, Size}, + #state{sock=Sock,i=#p_srvr{proto_mod=Mod}}) -> + fun() -> Mod:trim_chunk(Sock, EpochID, File, Offset, Size) end; make_req_fun({checksum_list, EpochID, File}, #state{sock=Sock,i=#p_srvr{proto_mod=Mod}}) -> fun() -> Mod:checksum_list(Sock, EpochID, File) end; diff --git a/test/machi_csum_table_test.erl b/test/machi_csum_table_test.erl index f34d955..683d512 100644 --- a/test/machi_csum_table_test.erl +++ b/test/machi_csum_table_test.erl @@ -76,13 +76,14 @@ smoke3_test() -> {?LINE, trim, {0, 1024, <<>>}, undefined, undefined} ], [ begin - %% ?debugVal({Line, Chunk}), + %% ?debugVal({_Line, Chunk}), {Offset, Size, Csum} = Chunk, ?assertEqual(LeftN0, machi_csum_table:find_leftneighbor(MC, Offset)), ?assertEqual(RightN0, machi_csum_table:find_rightneighbor(MC, Offset+Size)), LeftN = case LeftN0 of + {OffsL, SizeL, trimmed} -> {OffsL, SizeL, trimmed}; {OffsL, SizeL, _} -> {OffsL, SizeL, <<"boom">>}; OtherL -> OtherL end, diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index bf57043..e2eb955 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -170,7 +170,7 @@ start_command(S) -> start(_S) -> {_, _, MS} = os:timestamp(), File = test_server:temp_name("eqc_data") ++ "." ++ integer_to_list(MS), - {ok, Pid} = machi_file_proxy:start_link(File, ?TESTDIR), + {ok, Pid} = machi_file_proxy:start_link(some_flu, File, ?TESTDIR), unlink(Pid), Pid. diff --git a/test/machi_file_proxy_test.erl b/test/machi_file_proxy_test.erl index 8269483..cbf2014 100644 --- a/test/machi_file_proxy_test.erl +++ b/test/machi_file_proxy_test.erl @@ -78,7 +78,7 @@ random_binary(Start, End) -> machi_file_proxy_test_() -> clean_up_data_dir(?TESTDIR), - {ok, Pid} = machi_file_proxy:start_link("test", ?TESTDIR), + {ok, Pid} = machi_file_proxy:start_link(fluname, "test", ?TESTDIR), [ ?_assertEqual({error, bad_arg}, machi_file_proxy:read(Pid, -1, -1)), ?_assertEqual({error, bad_arg}, machi_file_proxy:write(Pid, -1, <<"yo">>)), @@ -100,7 +100,7 @@ machi_file_proxy_test_() -> multiple_chunks_read_test_() -> clean_up_data_dir(?TESTDIR), - {ok, Pid} = machi_file_proxy:start_link("test", ?TESTDIR), + {ok, Pid} = machi_file_proxy:start_link(fluname, "test", ?TESTDIR), [ ?_assertMatch({ok, "test", _}, machi_file_proxy:append(Pid, random_binary(0, 1024))), ?_assertEqual(ok, machi_file_proxy:write(Pid, 10000, <<"fail">>)), diff --git a/test/machi_pb_high_client_test.erl b/test/machi_pb_high_client_test.erl index 25c79fd..361eb55 100644 --- a/test/machi_pb_high_client_test.erl +++ b/test/machi_pb_high_client_test.erl @@ -38,6 +38,7 @@ smoke_test2() -> Ps = [#p_srvr{name=a, address="localhost", port=Port, props="./data.a"} ], D = orddict:from_list([{P#p_srvr.name, P} || P <- Ps]), + ok = application:set_env(machi, max_file_size, 1024*1024), [os:cmd("rm -rf " ++ P#p_srvr.props) || P <- Ps], {ok, SupPid} = machi_flu_sup:start_link(), @@ -90,23 +91,54 @@ smoke_test2() -> {ok, [{File1Size,File1}]} = ?C:list_files(Clnt), true = is_integer(File1Size), + File1Bin = binary_to_list(File1), [begin - %% ok = ?C:trim_chunk(Clnt, Fl, Off, Sz) - %% This gets an error as trim API is still a stub - ?assertMatch({bummer, - {throw, - {error, bad_joss_taipan_fixme}, - _Boring_stack_trace}}, - ?C:trim_chunk(Clnt, Fl, Off, Sz)) - end || {Ch, Fl, Off, Sz} <- Reads], + #p_srvr{name=Name, port=Port, props=Dir} = P, + ?assertEqual({ok, [File1Bin]}, + file:list_dir(filename:join([Dir, "data"]))), + FileListFileName = filename:join([Dir, "known_files_" ++ atom_to_list(Name)]), + {ok, Plist} = machi_plist:open(FileListFileName, []), + ?assertEqual([], machi_plist:all(Plist)) + end || P <- Ps], + [begin + ok = ?C:trim_chunk(Clnt, Fl, Off, Sz) + end || {_Ch, Fl, Off, Sz} <- Reads], + [begin + {ok, {[], Trimmed}} = + ?C:read_chunk(Clnt, Fl, Off, Sz, [{needs_trimmed, true}]), + Filename = binary_to_list(Fl), + ?assertEqual([{Filename, Off, Sz}], Trimmed) + end || {_Ch, Fl, Off, Sz} <- Reads], + + LargeBytes = binary:copy(<<"x">>, 1024*1024), + LBCsum = {client_sha, machi_util:checksum_chunk(LargeBytes)}, + {ok, {Offx, Sizex, Filex}} = + ?C:append_chunk(Clnt, PK, Prefix, LargeBytes, LBCsum, 0), + ok = ?C:trim_chunk(Clnt, Filex, Offx, Sizex), + + %% Make sure everything was trimmed + File = binary_to_list(Filex), + [begin + #p_srvr{name=Name, port=_Port, props=Dir} = P, + ?assertEqual({ok, []}, + file:list_dir(filename:join([Dir, "data"]))), + FileListFileName = filename:join([Dir, "known_files_" ++ atom_to_list(Name)]), + {ok, Plist} = machi_plist:open(FileListFileName, []), + ?assertEqual([File], machi_plist:all(Plist)) + end || P <- Ps], + + [begin + {error, trimmed} = + ?C:read_chunk(Clnt, Fl, Off, Sz, []) + end || {_Ch, Fl, Off, Sz} <- Reads], ok after (catch ?C:quit(Clnt)) end after exit(SupPid, normal), - [os:cmd("rm -rf " ++ P#p_srvr.props) || P <- Ps], + [os:cmd("rm -rf " ++ P#p_srvr.props) || P <- Ps], machi_util:wait_for_death(SupPid, 100), ok end. diff --git a/test/machi_plist_test.erl b/test/machi_plist_test.erl new file mode 100644 index 0000000..a796c1b --- /dev/null +++ b/test/machi_plist_test.erl @@ -0,0 +1,17 @@ +-module(machi_plist_test). + +-include_lib("eunit/include/eunit.hrl"). + +open_close_test() -> + FileName = "bark-bark-one", + file:delete(FileName), + {ok, PList0} = machi_plist:open(FileName, []), + {ok, PList1} = machi_plist:add(PList0, "boomar"), + ?assertEqual(["boomar"], machi_plist:all(PList1)), + ok = machi_plist:close(PList1), + + {ok, PList2} = machi_plist:open(FileName, []), + ?assertEqual(["boomar"], machi_plist:all(PList2)), + ok = machi_plist:close(PList2), + file:delete(FileName), + ok. -- 2.45.2 From 170b3cd797658dc22af099c73d53bb5154db7945 Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Wed, 28 Oct 2015 12:48:50 +0900 Subject: [PATCH 08/44] Dialyzer fix --- src/machi_csum_table.erl | 4 ++-- src/machi_flu1_client.erl | 12 +----------- src/machi_plist.erl | 8 ++++---- 3 files changed, 7 insertions(+), 17 deletions(-) diff --git a/src/machi_csum_table.erl b/src/machi_csum_table.erl index 7921df3..9585429 100644 --- a/src/machi_csum_table.erl +++ b/src/machi_csum_table.erl @@ -171,11 +171,11 @@ trim(#machi_csum_table{fd=Fd, table=T}, Offset, Size) -> Error end. --spec all_trimmed(table(), machi_dt:chunk_pos(), machi_dt:chunk_pos()) -> boolean(). +-spec all_trimmed(table(), non_neg_integer(), non_neg_integer()) -> boolean(). all_trimmed(#machi_csum_table{table=T}, Left, Right) -> runthru(ets:tab2list(T), Left, Right). --spec all_trimmed(table(), machi_dt:chunk_pos()) -> boolean(). +-spec all_trimmed(table(), non_neg_integer()) -> boolean(). all_trimmed(#machi_csum_table{table=T}, Pos) -> case ets:tab2list(T) of [{0, ?MINIMUM_OFFSET, _}|L] -> diff --git a/src/machi_flu1_client.erl b/src/machi_flu1_client.erl index ed8808c..119e154 100644 --- a/src/machi_flu1_client.erl +++ b/src/machi_flu1_client.erl @@ -80,7 +80,7 @@ %% For "internal" replication only. -export([ write_chunk/5, write_chunk/6, - trim_chunk/5, trim_chunk/6, + trim_chunk/5, delete_migration/3, delete_migration/4, trunc_hack/3, trunc_hack/4 ]). @@ -490,16 +490,6 @@ trim_chunk(Sock, EpochID, File0, Offset, Size) {low_trim_chunk, EpochID, File, Offset, Size, 0}), do_pb_request_common(Sock, ReqID, Req). -%% @doc Restricted API: Write a chunk of already-sequenced data to -%% `File' at `Offset'. - --spec trim_chunk(machi_dt:inet_host(), machi_dt:inet_port(), - machi_dt:epoch_id(), machi_dt:file_name(), machi_dt:file_offset(), machi_dt:chunk_size()) -> - ok | {error, machi_dt:error_general()} | {error, term()}. -trim_chunk(_Host, _TcpPort, _EpochID, _File, _Offset, _Size) -> - not_used. - - %% @doc Restricted API: Delete a file after it has been successfully %% migrated. diff --git a/src/machi_plist.erl b/src/machi_plist.erl index 4bf745e..9bd2352 100644 --- a/src/machi_plist.erl +++ b/src/machi_plist.erl @@ -9,14 +9,14 @@ -endif. -record(machi_plist, - {filename :: string(), - fd :: file:descriptor(), - list}). + {filename :: file:filename_all(), + fd :: file:io_device(), + list = [] :: list(string)}). -type plist() :: #machi_plist{}. -export_type([plist/0]). --spec open(filename:filename(), proplists:proplist()) -> +-spec open(file:filename_all(), proplists:proplist()) -> {ok, plist()} | {error, file:posix()}. open(Filename, _Opt) -> List = case file:read_file(Filename) of -- 2.45.2 From 70868999414ecbb409391baf9454cc939f787711 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Wed, 28 Oct 2015 16:59:49 -0500 Subject: [PATCH 09/44] Reorg merkle tree code into a library Was a service previously. Now contains both merklet and the naive implementations. Put construction timing stuff into the test. Tests are not truly meaningful yet. --- src/machi_merkle_tree.erl | 203 +++++++++++++++++ src/machi_merkle_tree_mgr.erl | 208 ------------------ ...gr_test.erl => machi_merkle_tree_test.erl} | 57 +++-- 3 files changed, 246 insertions(+), 222 deletions(-) create mode 100644 src/machi_merkle_tree.erl delete mode 100644 src/machi_merkle_tree_mgr.erl rename test/{machi_merkle_tree_mgr_test.erl => machi_merkle_tree_test.erl} (63%) diff --git a/src/machi_merkle_tree.erl b/src/machi_merkle_tree.erl new file mode 100644 index 0000000..a799edc --- /dev/null +++ b/src/machi_merkle_tree.erl @@ -0,0 +1,203 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. +%% +%% This file is provided to you under the Apache License, +%% Version 2.0 (the "License"); you may not use this file +%% except in compliance with the License. You may obtain +%% a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, +%% software distributed under the License is distributed on an +%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +%% KIND, either express or implied. See the License for the +%% specific language governing permissions and limitations +%% under the License. +%% +%% ------------------------------------------------------------------- + +%% @doc Creates a Merkle tree per file based on the checksum data for +%% a given data file. +%% +%% Has selectable backend, chosen at open. +%% +%% The default 'merklet' implementation uses the `merklet' library. Keys are +%% encoded as `<>' values encoded as `<>' +%% *or* as `<<0>>' for unwritten bytes, or `<<1>>' for trimmed bytes. +%% +%% The `naive' implementation representation is: +%% +%% `<>' for unwritten bytes +%% `<>' for trimmed bytes +%% `<>' for written bytes +%% +%% The tree feeds these leaf nodes into hashes representing chunks of a minimum +%% size of at least 1024 KB (1 MB), but if the file size is larger, we will try +%% to get about 100 chunks for called "Level 1." We aim for around 10 hashes at +%% level 2, and then 2 hashes level 3 and finally the root. + +-module(machi_merkle_tree). + +-include("machi.hrl"). + +-export([ + open/2, + open/3, + tree/1, + filename/1, + diff/2 +]). + + +-record(naive, { + chunk_size = 1048576 :: pos_integer(), %% default 1 MB + recalc = true :: boolean(), + root :: 'undefined' | binary(), + lvl1 = [] :: [ binary() ], + lvl2 = [] :: [ binary() ], + lvl3 = [] :: [ binary() ], + leaves = [] :: [ { Offset :: pos_integer(), + Size :: pos_integer(), + Csum :: binary()} ] + }). + +-record(mt, { + filename :: string(), + tree :: #naive{}|merklet:tree(), + backend = 'merklet' :: 'naive'|'merklet' + }). + +-define(TRIMMED, <<1>>). +-define(UNWRITTEN, <<0>>). +-define(ENCODE(Offset, Size), <>). +-define(NAIVE_ENCODE(Offset, Size, Data), <>). + +-define(NEW_MERKLET, undefined). +-define(TIMEOUT, (10*1000)). + +-define(MINIMUM_CHUNK, 1048576). %% 1024 * 1024 +-define(LEVEL_SIZE, 10). +-define(H, sha). + +%% public API + +open(Filename, DataDir) -> + open(Filename, DataDir, merklet). + +open(Filename, DataDir, Type) -> + Tree = load_filename(Filename, DataDir, Type), + {ok, #mt{ filename = Filename, tree = Tree, backend = Type}}. + +tree(#mt{ tree = T, backend = merklet }) -> T; +tree(#mt{ tree = T, backend = naive }) -> + case T#naive.recalc of + true -> build_tree(T); + false -> T + end. + +filename(#mt{ filename = F }) -> F. + +diff(#mt{backend = naive, tree = T1}, #mt{backend = naive, tree = T2}) -> + case T1#naive.root == T2#naive.root of + true -> same; + false -> different %% TODO: implement diff + end; +diff(#mt{backend = merklet, tree = T1}, #mt{backend = merklet, tree = T2}) -> + case merklet:diff(T1, T2) of + [] -> same; + Diff -> Diff + end; +diff(_, _) -> error(badarg). + +%% private + +% @private +load_filename(Filename, DataDir, merklet) -> + {_Last, M} = do_load(Filename, DataDir, fun insert_csum/2, ?NEW_MERKLET), + M; +load_filename(Filename, DataDir, naive) -> + {Last, M} = do_load(Filename, DataDir, fun insert_csum_naive/2, []), + ChunkSize = max(?MINIMUM_CHUNK, Last div 100), + T = #naive{ leaves = lists:reverse(M), chunk_size = ChunkSize, recalc = true }, + build_tree(T). + +do_load(Filename, DataDir, FoldFun, AccInit) -> + CsumFile = machi_util:make_checksum_filename(DataDir, Filename), + {ok, T} = machi_csum_table:open(CsumFile, []), + Acc = machi_csum_table:foldl_chunks(FoldFun, {0, AccInit}, T), + ok = machi_csum_table:close(T), + Acc. + +% @private +insert_csum({Last, Size, _Csum}=In, {Last, MT}) -> + %% no gap here, insert a record + {Last+Size, update_merkle_tree(In, MT)}; +insert_csum({Offset, Size, _Csum}=In, {Last, MT}) -> + %% gap here, insert unwritten record + %% *AND* insert written record + Hole = Offset - Last, + MT0 = update_merkle_tree({Last, Hole, unwritten}, MT), + {Offset+Size, update_merkle_tree(In, MT0)}. + +insert_csum_naive({Last, Size, _Csum}=In, {Last, MT}) -> + %% no gap + {Last+Size, update_acc(In, MT)}; +insert_csum_naive({Offset, Size, _Csum}=In, {Last, MT}) -> + Hole = Offset - Last, + MT0 = update_acc({Last, Hole, unwritten}, MT), + {Offset+Size, update_acc(In, MT0)}. + +% @private +update_merkle_tree({Offset, Size, unwritten}, MT) -> + merklet:insert({?ENCODE(Offset, Size), ?UNWRITTEN}, MT); +update_merkle_tree({Offset, Size, trimmed}, MT) -> + merklet:insert({?ENCODE(Offset, Size), ?TRIMMED}, MT); +update_merkle_tree({Offset, Size, Csum}, MT) -> + merklet:insert({?ENCODE(Offset, Size), Csum}, MT). + +update_acc({Offset, Size, unwritten}, MT) -> + [ {Offset, Size, ?NAIVE_ENCODE(Offset, Size, ?UNWRITTEN)} | MT ]; +update_acc({Offset, Size, trimmed}, MT) -> + [ {Offset, Size, ?NAIVE_ENCODE(Offset, Size, ?TRIMMED)} | MT ]; +update_acc({Offset, Size, Csum}, MT) -> + [ {Offset, Size, ?NAIVE_ENCODE(Offset, Size, Csum)} | MT ]. + +build_tree(MT = #naive{ leaves = L, chunk_size = ChunkSize }) -> + lager:debug("Leaves: ~p~n", [L]), + Lvl1s = build_level_1(ChunkSize, L, 1, [ crypto:hash_init(?H) ]), + lager:debug("Lvl1: ~p~n", [Lvl1s]), + Mod2 = length(Lvl1s) div ?LEVEL_SIZE, + Lvl2s = build_int_level(Mod2, Lvl1s, 1, [ crypto:hash_init(?H) ]), + lager:debug("Lvl2: ~p~n", [Lvl2s]), + Mod3 = length(Lvl2s) div 2, + Lvl3s = build_int_level(Mod3, Lvl2s, 1, [ crypto:hash_init(?H) ]), + lager:debug("Lvl3: ~p~n", [Lvl3s]), + Root = build_root(Lvl3s, crypto:hash_init(?H)), + lager:debug("Root: ~p~n", [Root]), + MT#naive{ root = Root, lvl1 = Lvl1s, lvl2 = Lvl2s, lvl3 = Lvl3s, recalc = false }. + +build_root([], Ctx) -> + crypto:hash_final(Ctx); +build_root([H|T], Ctx) -> + build_root(T, crypto:hash_update(Ctx, H)). + +build_int_level(_Mod, [], _Cnt, [ Ctx | Rest ]) -> + lists:reverse( [ crypto:hash_final(Ctx) | Rest ] ); +build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) when Cnt rem Mod == 0 -> + NewCtx = crypto:hash_init(?H), + build_int_level(Mod, T, Cnt + 1, [ crypto:hash_update(NewCtx, H), crypto:hash_final(Ctx) | Rest ]); +build_int_level(Mod, [H|T], Cnt, [ Ctx | Rest ]) -> + build_int_level(Mod, T, Cnt+1, [ crypto:hash_update(Ctx, H) | Rest ]). + +build_level_1(_Size, [], _Multiple, [ Ctx | Rest ]) -> + lists:reverse([ crypto:hash_final(Ctx) | Rest ]); +build_level_1(Size, [{Pos, Len, Hash}|T], Multiple, [ Ctx | Rest ]) + when ( Pos + Len ) > ( Size * Multiple ) -> + NewCtx = crypto:hash_init(?H), + build_level_1(Size, T, Multiple+1, + [ crypto:hash_update(NewCtx, Hash), crypto:hash_final(Ctx) | Rest ]); +build_level_1(Size, [{Pos, Len, Hash}|T], Multiple, [ Ctx | Rest ]) + when ( Pos + Len ) =< ( Size * Multiple ) -> + build_level_1(Size, T, Multiple, [ crypto:hash_update(Ctx, Hash) | Rest ]). diff --git a/src/machi_merkle_tree_mgr.erl b/src/machi_merkle_tree_mgr.erl deleted file mode 100644 index 2e7a144..0000000 --- a/src/machi_merkle_tree_mgr.erl +++ /dev/null @@ -1,208 +0,0 @@ -%% ------------------------------------------------------------------- -%% -%% Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. -%% -%% This file is provided to you under the Apache License, -%% Version 2.0 (the "License"); you may not use this file -%% except in compliance with the License. You may obtain -%% a copy of the License at -%% -%% http://www.apache.org/licenses/LICENSE-2.0 -%% -%% Unless required by applicable law or agreed to in writing, -%% software distributed under the License is distributed on an -%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -%% KIND, either express or implied. See the License for the -%% specific language governing permissions and limitations -%% under the License. -%% -%% ------------------------------------------------------------------- - -%% @doc This manager maintains a Merkle tree per file per FLU as implemented -%% by the `merklet' library. Keys are encoded as `<>' -%% values encoded as `<>' *or* as <<0>> for unwritten -%% bytes, or <<1>> for trimmed bytes. - --module(machi_merkle_tree_mgr). --behaviour(gen_server). - --include("machi.hrl"). - --export([ - child_spec/3, - start_link/3, - initialize/2, - update/5, - fetch/2 -]). - -%% gen_server callbacks --export([ - init/1, - handle_call/3, - handle_cast/2, - handle_info/2, - terminate/2, - code_change/3 -]). - --record(state, { - fluname :: atom(), - datadir :: string(), - tid :: ets:tid() - }). - --record(mt, { - filename :: string(), - tree :: merklet:tree() - }). - --define(TRIMMED, <<1>>). --define(UNWRITTEN, <<0>>). --define(ENCODE(Offset, Size), <>). - --define(NEW_MERKLET, undefined). --define(TIMEOUT, (10*1000)). - -%% public API - -child_spec(FluName, DataDir, Options) -> - Name = make_merkle_tree_mgr_name(FluName), - {Name, - {?MODULE, start_link, [FluName, DataDir, Options]}, - permanent, 5000, worker, [?MODULE]}. - -start_link(FluName, DataDir, Options) -> - gen_server:start_link({local, make_merkle_tree_mgr_name(FluName)}, - ?MODULE, - {FluName, DataDir, Options}, - []). - --spec initialize( FluName :: atom(), - Filename :: string() ) -> ok. -%% @doc A heads-up hint to the manager that it ought to compute a merkle -%% tree for the given file (if it hasn't already). -initialize(FluName, Filename) -> - gen_server:cast(make_merkle_tree_mgr_name(FluName), - {initialize, Filename}). - --spec update( FluName :: atom(), - Filename :: string(), - Offset :: non_neg_integer(), - Length :: pos_integer(), - Csum :: binary() ) -> ok. -%% @doc A new leaf node ought to be added file the given filename, -%% with the particular information. -update(FluName, Filename, Offset, Length, Csum) -> - gen_server:cast(make_merkle_tree_mgr_name(FluName), - {update, Filename, Offset, Length, Csum}). - --spec fetch ( FluName :: atom(), - Filename :: string() ) -> {ok, 'undefined'|merklet:tree()}. -%% @doc Returns the merkle tree for the given filename. -fetch(FluName, Filename) -> - gen_server:call(make_merkle_tree_mgr_name(FluName), - {fetch, Filename}, ?TIMEOUT). - -%% gen_server callbacks -init({FluName, DataDir, Options}) -> - Tid = ets:new(make_merkle_tree_mgr_name(FluName), [{keypos, 2}, {read_concurrency, true}]), - case proplists:get_value(no_load, Options, false) of - true -> - ok; - false -> - handle_load(Tid, DataDir) - end, - {ok, #state{fluname=FluName, datadir=DataDir, tid = Tid}}. - -handle_call({fetch, Filename}, _From, S = #state{ tid = Tid }) -> - Res = handle_fetch(Tid, Filename), - {reply, {ok, Res}, S}; -handle_call(Req, _From, State) -> - lager:warning("Unknown call: ~p", [Req]), - {reply, whoaaaaaaaaaaaa, State}. - -handle_cast({initialize, Filename}, S = #state{ datadir = D, tid = Tid }) -> - load_filename(Tid, D, Filename), - {noreply, S}; - -handle_cast({update, Filename, Offset, Length, Csum}, S = #state{ tid = Tid }) -> - %% XXX FIXME: Not sure about the correctness of this - insert(Tid, Filename, {Offset, Length, Csum}), - {noreply, S}; - -handle_cast(Cast, State) -> - lager:warning("Unknown cast: ~p", [Cast]), - {noreply, State}. - -handle_info(Req, State) -> - lager:warning("Unknown info message: ~p", [Req]), - {noreply, State}. - -terminate(Reason, #state{fluname = F}) -> - lager:debug("Shutting down merkle tree manager for FLU ~p because ~p", - [F, Reason]), - ok. - -code_change(_OldVsn, State, _Extra) -> - {ok, State}. - -%% private - -make_merkle_tree_mgr_name(FluName) -> - list_to_atom(atom_to_list(FluName) ++ "_merkle_tree_mgr"). - -handle_load(Tid, DataDir) -> - Files = get_files(DataDir), - lists:foreach(fun(F) -> load_filename(Tid, DataDir, F) end, Files). - -get_files(DataDir) -> - {_, WildPath} = machi_util:make_data_filename(DataDir, ""), - filelib:wildcard("*", WildPath). - -load_filename(Tid, DataDir, Filename) -> - CsumFile = machi_util:make_checksum_filename(DataDir, Filename), - {ok, T} = machi_csum_table:open(CsumFile, []), - %% docs say that the traversal order of ets:foldl is non-determinstic - %% but hopefully since csum_table uses an ordered set that's not true... - {_LastPosition, M} = machi_csum_table:foldl_chunks(fun insert_csum/2, - {?MINIMUM_OFFSET, ?NEW_MERKLET}, T), - true = ets:insert_new(Tid, #mt{ filename = Filename, tree = M}), - ok = machi_csum_table:close(T), - ok. - -insert_csum({Last, Size, _Csum}=In, {Last, MT}) -> - %% no gap here, insert a record - {Last+Size, update_merkle_tree(In, MT)}; -insert_csum({Offset, Size, _Csum}=In, {Last, MT}) -> - %% gap here, insert unwritten record - %% *AND* insert written record - Hole = Offset - Last, - MT0 = update_merkle_tree({Last, Hole, unwritten}, MT), - {Offset+Size, update_merkle_tree(In, MT0)}. - -insert(Tid, Filename, Term) -> - case ets:lookup(Tid, Filename) of - [] -> error(not_found); %% TODO: Something better? - [R] -> - NewMT = update_merkle_tree(Term, R#mt.tree), - %% we choose update_element because it - %% makes atomic changes so it is concurrent - %% safe. The regular 'insert' function - %% does not provide that guarantee. - true = ets:update_element(Tid, Filename, {#mt.tree, NewMT}), - ok - end. - -handle_fetch(Tid, Filename) -> - case ets:lookup(Tid, Filename) of - [] -> undefined; - [R] -> R#mt.tree - end. - -update_merkle_tree({Offset, Size, unwritten}, MT) -> - merklet:insert({?ENCODE(Offset, Size), ?UNWRITTEN}, MT); -update_merkle_tree({Offset, Size, trimmed}, MT) -> - merklet:insert({?ENCODE(Offset, Size), ?TRIMMED}, MT); -update_merkle_tree({Offset, Size, Csum}, MT) -> - merklet:insert({?ENCODE(Offset, Size), Csum}, MT). diff --git a/test/machi_merkle_tree_mgr_test.erl b/test/machi_merkle_tree_test.erl similarity index 63% rename from test/machi_merkle_tree_mgr_test.erl rename to test/machi_merkle_tree_test.erl index bc9e0db..382c262 100644 --- a/test/machi_merkle_tree_mgr_test.erl +++ b/test/machi_merkle_tree_test.erl @@ -18,7 +18,7 @@ %% %% ------------------------------------------------------------------- --module(machi_merkle_tree_mgr_test). +-module(machi_merkle_tree_test). -compile([export_all]). -include_lib("eunit/include/eunit.hrl"). @@ -27,6 +27,16 @@ -define(TESTFILE, "yza^4c784dc2-19bf-4ac6-91f6-58bbe5aa88e0^1"). -define(GAP_CHANCE, 0.10). +choose_filename() -> + random_from_list([ + "def^c5ea7511-d649-47d6-a8c3-2b619379c237^1", + "jkl^b077eff7-b2be-4773-a73f-fea4acb8a732^1", + "stu^553fa47a-157c-4fac-b10f-2252c7d8c37a^1", + "vwx^ae015d68-7689-4c9f-9677-926c6664f513^1", + "yza^4c784dc2-19bf-4ac6-91f6-58bbe5aa88e0^1" + ]). + + make_csum_file(DataDir, Filename, Offsets) -> Path = machi_util:make_checksum_filename(DataDir, Filename), filelib:ensure_dir(Path), @@ -82,19 +92,38 @@ generate_offsets(FH, Filesize, Current, Acc) when Current < Filesize -> generate_offsets(_FH, _Filesize, _Current, Acc) -> lists:reverse(Acc). -test() -> +test() -> + test(100). + +test(N) -> + {ok, F} = file:open("results.txt", [raw, write]), + lists:foreach(fun(X) -> format_and_store(F, run_test(X)) end, lists:seq(1, N)). + +format_and_store(F, {OffsetNum, {MTime, MSize}, {NTime, NSize}}) -> + S = io_lib:format("~w\t~w\t~w\t~w\t~w\n", [OffsetNum, MTime, MSize, NTime, NSize]), + ok = file:write(F, S). + +run_test(C) -> random:seed(os:timestamp()), - O = make_offsets("test/" ++ ?TESTFILE), - ?debugFmt("Offsets: ~p", [O]), - make_csum_file(".", ?TESTFILE, O), + OffsetFn = "test/" ++ choose_filename(), + O = make_offsets(OffsetFn), + Fn = "csum_" ++ integer_to_list(C), + make_csum_file(".", Fn, O), - _ = machi_merkle_tree_mgr:start_link(test, ".", []), - machi_merkle_tree_mgr:initialize(test, ?TESTFILE), - timer:sleep(1000), - All = machi_merkle_tree_mgr:fetch(test, ?TESTFILE), - ?debugFmt("All: ~p~n", [All]), - timer:sleep(1000), - All = machi_merkle_tree_mgr:fetch(test, ?TESTFILE), - ?debugFmt("All: ~p~n", [All]), - ok. + Osize = length(O), + {MTime, {ok, M}} = timer:tc(fun() -> machi_merkle_tree:open(Fn, ".", merklet) end), + {NTime, {ok, N}} = timer:tc(fun() -> machi_merkle_tree:open(Fn, ".", naive) end), + + ?assertEqual(Fn, machi_merkle_tree:filename(M)), + ?assertEqual(Fn, machi_merkle_tree:filename(N)), + + MTree = machi_merkle_tree:tree(M), + MSize = byte_size(term_to_binary(MTree)), + + NTree = machi_merkle_tree:tree(N), + NSize = byte_size(term_to_binary(NTree)), + + ?assertEqual(same, machi_merkle_tree:diff(N, N)), + ?assertEqual(same, machi_merkle_tree:diff(M, M)), + {Osize, {MTime, MSize}, {NTime, NSize}}. -- 2.45.2 From 028135d927ab66f0dc9d2cd42b003a926ab5f33a Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Thu, 29 Oct 2015 12:07:23 +0900 Subject: [PATCH 10/44] Update some comments for concise and sound description --- src/machi_cr_client.erl | 2 -- src/machi_plist.erl | 8 +++++++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/machi_cr_client.erl b/src/machi_cr_client.erl index e198c40..2b32308 100644 --- a/src/machi_cr_client.erl +++ b/src/machi_cr_client.erl @@ -600,8 +600,6 @@ do_trim_chunk2(File, Offset, Size, Depth, STime, TO, Proxy = orddict:fetch(HeadFLU, PD), case ?FLU_PC:trim_chunk(Proxy, EpochID, File, Offset, Size, ?TIMEOUT) of ok -> - %% From this point onward, we use the same code & logic path as - %% append does. do_trim_midtail(RestFLUs, undefined, File, Offset, Size, [HeadFLU], 0, STime, TO, S); {error, trimmed} -> diff --git a/src/machi_plist.erl b/src/machi_plist.erl index 9bd2352..7750b0a 100644 --- a/src/machi_plist.erl +++ b/src/machi_plist.erl @@ -1,6 +1,6 @@ -module(machi_plist). -%%% @doc persistent list of binaries that support mutual exclusion +%%% @doc persistent list of binaries -export([open/2, close/1, find/2, add/2]). @@ -19,6 +19,9 @@ -spec open(file:filename_all(), proplists:proplist()) -> {ok, plist()} | {error, file:posix()}. open(Filename, _Opt) -> + %% TODO: This decode could fail if the file didn't finish writing + %% whole contents, which should be fixed by some persistent + %% solution. List = case file:read_file(Filename) of {ok, <<>>} -> []; {ok, Bin} -> binary_to_term(Bin); @@ -48,6 +51,9 @@ add(Plist = #machi_plist{list=List0, fd=Fd}, Name) -> {ok, Plist}; false -> List = lists:append(List0, [Name]), + %% TODO: partial write could break the file with other + %% persistent info (even lose data of trimmed states); + %% needs a solution. case file:pwrite(Fd, 0, term_to_binary(List)) of ok -> {ok, Plist#machi_plist{list=List}}; -- 2.45.2 From 611f33e81bae6097d07e8808b2691a6405279918 Mon Sep 17 00:00:00 2001 From: Scott Lystig Fritchie Date: Thu, 29 Oct 2015 15:05:15 +0900 Subject: [PATCH 11/44] Change trigger_gc default -> 0 --- src/machi.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/machi.proto b/src/machi.proto index e583ba5..c9251cb 100644 --- a/src/machi.proto +++ b/src/machi.proto @@ -433,7 +433,7 @@ message Mpb_LL_TrimChunkReq { required string file = 2; required uint64 offset = 3; required uint32 size = 4; - optional uint32 trigger_gc = 5 [default=1]; + optional uint32 trigger_gc = 5 [default=0]; } message Mpb_LL_TrimChunkResp { -- 2.45.2 From b859e23a370f6a219452c86ec6a677c715e23e6c Mon Sep 17 00:00:00 2001 From: Scott Lystig Fritchie Date: Thu, 29 Oct 2015 18:58:34 +0900 Subject: [PATCH 12/44] Some long-overdue minor editing, prior to working on issue #4 --- doc/src.high-level/high-level-chain-mgr.tex | 689 +++++++------------- doc/src.high-level/high-level-machi.tex | 2 +- 2 files changed, 245 insertions(+), 446 deletions(-) diff --git a/doc/src.high-level/high-level-chain-mgr.tex b/doc/src.high-level/high-level-chain-mgr.tex index b6e35a6..f139862 100644 --- a/doc/src.high-level/high-level-chain-mgr.tex +++ b/doc/src.high-level/high-level-chain-mgr.tex @@ -23,8 +23,8 @@ \copyrightdata{978-1-nnnn-nnnn-n/yy/mm} \doi{nnnnnnn.nnnnnnn} -\titlebanner{Draft \#0.91, June 2015} -\preprintfooter{Draft \#0.91, June 2015} +\titlebanner{Draft \#0.92, October 2015} +\preprintfooter{Draft \#0.92, October 2015} \title{Chain Replication metadata management in Machi, an immutable file store} @@ -50,19 +50,23 @@ For an overview of the design of the larger Machi system, please see TODO Fix, after all of the recent changes to this document. Machi is an immutable file store, now in active development by Basho -Japan KK. Machi uses Chain Replication to maintain strong consistency -of file updates to all replica servers in a Machi cluster. Chain +Japan KK. Machi uses Chain Replication\footnote{Chain Replication is a variation of primary/backup replication where the order of updates between the primary server and each of the backup -servers is strictly ordered into a single ``chain''. Management of -Chain Replication's metadata, e.g., ``What is the current order of -servers in the chain?'', remains an open research problem. The -current state of the art for Chain Replication metadata management -relies on an external oracle (e.g., ZooKeeper) or the Elastic -Replication algorithm. +servers is strictly ordered into a single ``chain''.} +to maintain strong consistency +of file updates to all replica servers in a Machi cluster. This document describes the Machi chain manager, the component -responsible for managing Chain Replication metadata state. The chain +responsible for managing Chain Replication metadata state. +Management of +chain metadata, e.g., ``What is the current order of +servers in the chain?'', remains an open research problem. The +current state of the art for Chain Replication metadata management +relies on an external oracle (e.g., based on ZooKeeper) or the Elastic +Replication \cite{elastic-chain-replication} algorithm. + +The chain manager uses a new technique, based on a variation of CORFU, called ``humming consensus''. Humming consensus does not require active participation by all or even @@ -89,20 +93,18 @@ to perform these management tasks. Chain metadata state and state management tasks include: \begin{itemize} -\item Preserving data integrity of all metadata and data stored within - the chain. Data loss is not an option. \item Preserving stable knowledge of chain membership (i.e. all nodes in - the chain, regardless of operational status). A systems - administrator is expected to make ``permanent'' decisions about + the chain, regardless of operational status). We expect that a systems + administrator will make all ``permanent'' decisions about chain membership. \item Using passive and/or active techniques to track operational - state/status, e.g., up, down, restarting, full data sync, partial - data sync, etc. + state/status, e.g., up, down, restarting, full data sync in progress, partial + data sync in progress, etc. \item Choosing the run-time replica ordering/state of the chain, based on current member status and past operational history. All chain state transitions must be done safely and without data loss or corruption. -\item As a new node is added to the chain administratively or old node is +\item When a new node is added to the chain administratively or old node is restarted, adding the node to the chain safely and perform any data synchronization/repair required to bring the node's data into full synchronization with the other nodes. @@ -111,39 +113,27 @@ management tasks include: \subsection{Ultimate goal: Preserve data integrity of Chain Replicated data} Preservation of data integrity is paramount to any chain state -management technique for Machi. Even when operating in an eventually -consistent mode, Machi must not lose data without cause outside of all -design, e.g., all particpants crash permanently. +management technique for Machi. Loss or corruption of chain data must +be avoided. \subsection{Goal: Contribute to Chain Replication metadata management research} We believe that this new self-management algorithm, humming consensus, contributes a novel approach to Chain Replication metadata management. -The ``monitor -and mangage your neighbor'' technique proposed in Elastic Replication -(Section \ref{ssec:elastic-replication}) appears to be the current -state of the art in the distributed systems research community. Typical practice in the IT industry appears to favor using an external -oracle, e.g., using ZooKeeper as a trusted coordinator. +oracle, e.g., built on top of ZooKeeper as a trusted coordinator. -See Section~\ref{sec:cr-management-review} for a brief review. +See Section~\ref{sec:cr-management-review} for a brief review of +techniques used today. \subsection{Goal: Support both eventually consistent \& strongly consistent modes of operation} -Machi's first use cases are all for use as a file store in an eventually -consistent environment. -In eventually consistent mode, humming consensus -allows a Machi cluster to fragment into -arbitrary islands of network partition, all the way down to 100\% of -members running in complete network isolation from each other. -Furthermore, it provides enough agreement to allow -formerly-partitioned members to coordinate the reintegration and -reconciliation of their data when partitions are healed. - -Later, we wish the option of supporting strong consistency -applications such as CORFU-style logging while reusing all (or most) -of Machi's infrastructure. Such strongly consistent operation is the -main focus of this document. +Chain Replication was originally designed by van Renesse and Schneider +\cite{chain-replication} for applications that require strong +consistency, e.g. sequential consistency. However, Machi has use +cases where more relaxed eventual consistency semantics are +sufficient. We wish to use the same Chain Replication management +technique for both strong and eventual consistency environments. \subsection{Anti-goal: Minimize churn} @@ -204,6 +194,18 @@ would probably be preferable to add the feature to Riak Ensemble rather than to use ZooKeeper (and for Basho to document ZK, package ZK, provide commercial ZK support, etc.). +\subsection{An external management oracle, implemented by + active/standby application failover} + +This technique has been used in production of HibariDB. The customer +very carefully deployed the oracle using the Erlang/OTP ``application +controller'' on two machines to provide active/standby failover of the +management oracle. The customer was willing to monitor this service +very closely and was prepared to intervene manually during network +partitions. (This controller is very susceptible to ``split brain +syndrome''.) While this feature of Erlang/OTP is useful in other +environments, we believe is it not sufficient for Machi's needs. + \section{Assumptions} \label{sec:assumptions} @@ -212,8 +214,8 @@ Paxos, Raft, et al.), why bother with a slightly different set of assumptions and a slightly different protocol? The answer lies in one of our explicit goals: to have an option of -running in an ``eventually consistent'' manner. We wish to be able to -make progress, i.e., remain available in the CAP sense, even if we are +running in an ``eventually consistent'' manner. We wish to be +remain available, even if we are partitioned down to a single isolated node. VR, Paxos, and Raft alone are not sufficient to coordinate service availability at such small scale. The humming consensus algorithm can manage @@ -247,13 +249,15 @@ synchronized by NTP. The protocol and algorithm presented here do not specify or require any timestamps, physical or logical. Any mention of time inside of data -structures are for human/historic/diagnostic purposes only. +structures are for human and/or diagnostic purposes only. -Having said that, some notion of physical time is suggested for -purposes of efficiency. It's recommended that there be some ``sleep +Having said that, some notion of physical time is suggested +occasionally for +purposes of efficiency. For example, some ``sleep time'' between iterations of the algorithm: there is no need to ``busy -wait'' by executing the algorithm as quickly as possible. See also -Section~\ref{ssub:when-to-calc}. +wait'' by executing the algorithm as many times per minute as +possible. +See also Section~\ref{ssub:when-to-calc}. \subsection{Failure detector model} @@ -276,55 +280,73 @@ eventual consistency. Discussion of strongly consistent CP mode is always the default; exploration of AP mode features in this document will always be explictly noted. -\subsection{Use of the ``wedge state''} +%%\subsection{Use of the ``wedge state''} +%% +%%A participant in Chain Replication will enter ``wedge state'', as +%%described by the Machi high level design \cite{machi-design} and by CORFU, +%%when it receives information that +%%a newer projection (i.e., run-time chain state reconfiguration) is +%%available. The new projection may be created by a system +%%administrator or calculated by the self-management algorithm. +%%Notification may arrive via the projection store API or via the file +%%I/O API. +%% +%%When in wedge state, the server will refuse all file write I/O API +%%requests until the self-management algorithm has determined that +%%humming consensus has been decided (see next bullet item). The server +%%may also refuse file read I/O API requests, depending on its CP/AP +%%operation mode. +%% +%%\subsection{Use of ``humming consensus''} +%% +%%CS literature uses the word ``consensus'' in the context of the problem +%%description at \cite{wikipedia-consensus} +%%. +%%This traditional definition differs from what is described here as +%%``humming consensus''. +%% +%%``Humming consensus'' describes +%%consensus that is derived only from data that is visible/known at the current +%%time. +%%The algorithm will calculate +%%a rough consensus despite not having input from a quorum majority +%%of chain members. Humming consensus may proceed to make a +%%decision based on data from only a single participant, i.e., only the local +%%node. +%% +%%See Section~\ref{sec:humming-consensus} for detailed discussion. -A participant in Chain Replication will enter ``wedge state'', as -described by the Machi high level design \cite{machi-design} and by CORFU, -when it receives information that -a newer projection (i.e., run-time chain state reconfiguration) is -available. The new projection may be created by a system -administrator or calculated by the self-management algorithm. -Notification may arrive via the projection store API or via the file -I/O API. +%%\subsection{Concurrent chain managers execute humming consensus independently} +%% +%%Each Machi file server has its own concurrent chain manager +%%process embedded within it. Each chain manager process will +%%execute the humming consensus algorithm using only local state (e.g., +%%the $P_{current}$ projection currently used by the local server) and +%%values observed in everyone's projection stores +%%(Section~\ref{sec:projection-store}). +%% +%%The chain manager communicates with the local Machi +%%file server using the wedge and un-wedge request API. When humming +%%consensus has chosen a projection $P_{new}$ to replace $P_{current}$, +%%the value of $P_{new}$ is included in the un-wedge request. -When in wedge state, the server will refuse all file write I/O API -requests until the self-management algorithm has determined that -humming consensus has been decided (see next bullet item). The server -may also refuse file read I/O API requests, depending on its CP/AP -operation mode. +\subsection{The reader is familiar with CORFU} -\subsection{Use of ``humming consensus''} +Machi borrows heavily from the techniques and data structures used by +CORFU \cite[corfu1],\cite[corfu2]. We hope that the reader is +familiar with CORFU's features, including: -CS literature uses the word ``consensus'' in the context of the problem -description at \cite{wikipedia-consensus} -. -This traditional definition differs from what is described here as -``humming consensus''. - -``Humming consensus'' describes -consensus that is derived only from data that is visible/known at the current -time. -The algorithm will calculate -a rough consensus despite not having input from all/majority -of chain members. Humming consensus may proceed to make a -decision based on data from only a single participant, i.e., only the local -node. - -See Section~\ref{sec:humming-consensus} for detailed discussion. - -\subsection{Concurrent chain managers execute humming consensus independently} - -Each Machi file server has its own concurrent chain manager -process embedded within it. Each chain manager process will -execute the humming consensus algorithm using only local state (e.g., -the $P_{current}$ projection currently used by the local server) and -values observed in everyone's projection stores -(Section~\ref{sec:projection-store}). - -The chain manager communicates with the local Machi -file server using the wedge and un-wedge request API. When humming -consensus has chosen a projection $P_{new}$ to replace $P_{current}$, -the value of $P_{new}$ is included in the un-wedge request. +\begin{itemize} +\item write-once registers for log data storage, +\item the epoch, which defines a period of time when a cluster's configuration +is stable, +\item strictly increasing epoch numbers, which are identifiers +for particular epochs, +\item projections, which define the chain order and other details of + data replication within the cluster, and +\item the wedge state, used by servers to coordinate cluster changes + during epoch transitions. +\end{itemize} \section{The projection store} \label{sec:projection-store} @@ -343,19 +365,15 @@ this key. The store's value is either the special `unwritten' value\footnote{We use $\bot$ to denote the unwritten value.} or else a binary blob that is immutable thereafter; the projection data structure is -serialized and stored in this binary blob. - -The projection store is vital for the correct implementation of humming -consensus (Section~\ref{sec:humming-consensus}). The write-once -register primitive allows us to reason about the store's behavior -using the same logical tools and techniques as the CORFU ordered log. +serialized and stored in this binary blob. See +\ref{sub:the-projection} for more detail. \subsection{The publicly-writable half of the projection store} The publicly-writable projection store is used to share information during the first half of humming consensus algorithm. Projections in the public half of the store form a log of -suggestions\footnote{I hesitate to use the word ``propose'' or ``proposal'' +suggestions\footnote{I hesitate to use the words ``propose'' or ``proposal'' anywhere in this document \ldots until I've done a more formal analysis of the protocol. Those words have too many connotations in the context of consensus protocols such as Paxos and Raft.} @@ -369,8 +387,9 @@ Any chain member may read from the public half of the store. The privately-writable projection store is used to store the Chain Replication metadata state (as chosen by humming consensus) -that is in use now by the local Machi server as well as previous -operation states. +that is in use now by the local Machi server. Earlier projections +remain in the private half to keep a historical +record of chain state transitions by the local server. Only the local server may write values into the private half of store. Any chain member may read from the private half of the store. @@ -386,35 +405,30 @@ The private projection store serves multiple purposes, including: its sequence of $P_{current}$ projection changes. \end{itemize} -The private half of the projection store is not replicated. - \section{Projections: calculation, storage, and use} \label{sec:projections} Machi uses a ``projection'' to determine how its Chain Replication replicas -should operate; see \cite{machi-design} and -\cite{corfu1}. At runtime, a cluster must be able to respond both to -administrative changes (e.g., substituting a failed server with -replacement hardware) as well as local network conditions (e.g., is -there a network partition?). - -The projection defines the operational state of Chain Replication's -chain order as well the (re-)synchronization of data managed by by -newly-added/failed-and-now-recovering members of the chain. This -chain metadata, together with computational processes that manage the -chain, must be managed in a safe manner in order to avoid unintended -data loss of data managed by the chain. - +should operate; see \cite{machi-design} and \cite{corfu1}. The concept of a projection is borrowed from CORFU but has a longer history, e.g., the Hibari key-value store \cite{cr-theory-and-practice} and goes back in research for decades, e.g., Porcupine \cite{porcupine}. +The projection defines the operational state of Chain Replication's +chain order as well the (re-)synchronization of data managed by by +newly-added/failed-and-now-recovering members of the chain. +At runtime, a cluster must be able to respond both to +administrative changes (e.g., substituting a failed server with +replacement hardware) as well as local network conditions (e.g., is +there a network partition?). + \subsection{The projection data structure} \label{sub:the-projection} {\bf NOTE:} This section is a duplicate of the ``The Projection and -the Projection Epoch Number'' section of \cite{machi-design}. +the Projection Epoch Number'' section of the ``Machi: an immutable +file store'' design doc \cite{machi-design}. The projection data structure defines the current administration \& operational/runtime @@ -445,6 +459,7 @@ Figure~\ref{fig:projection}. To summarize the major components: active_upi :: [m_server()], repairing :: [m_server()], down_members :: [m_server()], + witness_servers :: [m_server()], dbg_annotations :: proplist() }). \end{verbatim} @@ -454,13 +469,12 @@ Figure~\ref{fig:projection}. To summarize the major components: \begin{itemize} \item {\tt epoch\_number} and {\tt epoch\_csum} The epoch number and - projection checksum are unique identifiers for this projection. + projection checksum together form the unique identifier for this projection. \item {\tt creation\_time} Wall-clock time, useful for humans and general debugging effort. \item {\tt author\_server} Name of the server that calculated the projection. \item {\tt all\_members} All servers in the chain, regardless of current - operation status. If all operating conditions are perfect, the - chain should operate in the order specified here. + operation status. \item {\tt active\_upi} All active chain members that we know are fully repaired/in-sync with each other and therefore the Update Propagation Invariant (Section~\ref{sub:upi}) is always true. @@ -468,7 +482,10 @@ Figure~\ref{fig:projection}. To summarize the major components: are in active data repair procedures. \item {\tt down\_members} All members that the {\tt author\_server} believes are currently down or partitioned. -\item {\tt dbg\_annotations} A ``kitchen sink'' proplist, for code to +\item {\tt witness\_servers} If witness servers (Section~\ref{zzz}) + are used in strong consistency mode, then they are listed here. The + set of {\tt witness\_servers} is a subset of {\tt all\_members}. +\item {\tt dbg\_annotations} A ``kitchen sink'' property list, for code to add any hints for why the projection change was made, delay/retry information, etc. \end{itemize} @@ -478,7 +495,8 @@ Figure~\ref{fig:projection}. To summarize the major components: According to the CORFU research papers, if a server node $S$ or client node $C$ believes that epoch $E$ is the latest epoch, then any information that $S$ or $C$ receives from any source that an epoch $E+\delta$ (where -$\delta > 0$) exists will push $S$ into the ``wedge'' state and $C$ into a mode +$\delta > 0$) exists will push $S$ into the ``wedge'' state +and force $C$ into a mode of searching for the projection definition for the newest epoch. In the humming consensus description in @@ -506,7 +524,7 @@ Humming consensus requires that any projection be identified by both the epoch number and the projection checksum, as described in Section~\ref{sub:the-projection}. -\section{Managing multiple projection store replicas} +\section{Managing projection store replicas} \label{sec:managing-multiple-projection-stores} An independent replica management technique very similar to the style @@ -515,11 +533,63 @@ replicas of Machi's projection data structures. The major difference is that humming consensus {\em does not necessarily require} successful return status from a minimum number of participants (e.g., -a quorum). +a majority quorum). + +\subsection{Writing to public projection stores} +\label{sub:proj-store-writing} + +Writing replicas of a projection $P_{new}$ to the cluster's public +projection stores is similar to writing in a Dynamo-like system. +The significant difference with Chain Replication is how we interpret +the return status of each write operation. + +In cases of {\tt error\_written} status, +the process may be aborted and read repair +triggered. The most common reason for {\tt error\_written} status +is that another actor in the system has concurrently +already calculated another +(perhaps different\footnote{The {\tt error\_written} may also +indicate that another server has performed read repair on the exact +projection $P_{new}$ that the local server is trying to write!}) +projection using the same projection epoch number. + +\subsection{Writing to private projection stores} + +Only the local server/owner may write to the private half of a +projection store. Private projection store values are never subject +to read repair. + +\subsection{Reading from public projection stores} +\label{sub:proj-store-reading} + +A read is simple: for an epoch $E$, send a public projection read API +operation to all participants. Usually, the ``get latest epoch'' +variety is used. + +The minimum number of non-error responses is only one.\footnote{The local +projection store should always be available, even if no other remote +replica projection stores are available.} If all available servers +return a single, unanimous value $V_u, V_u \ne \bot$, then $V_u$ is +the final result for epoch $E$. +Any non-unanimous values are considered unresolvable for the +epoch. This disagreement is resolved by newer +writes to the public projection stores during subsequent iterations of +humming consensus. + +Unavailable servers may not necessarily interfere with making a decision. +Humming consensus +only uses as many public projections as are available at the present +moment of time. Assume that some server $S$ is unavailable at time $t$ and +becomes available at some later $t+\delta$. +If at $t+\delta$ we +discover that $S$'s public projection store for key $E$ +contains some disagreeing value $V_{weird}$, then the disagreement +will be resolved in the exact same manner that would have been used as if we +had seen the disagreeing values at the earlier time $t$. \subsection{Read repair: repair only unwritten values} -The idea of ``read repair'' is also shared with Riak Core and Dynamo +The ``read repair'' concept is also shared with Riak Core and Dynamo systems. However, Machi has situations where read repair cannot truly ``fix'' a key because two different values have been written by two different replicas. @@ -530,85 +600,24 @@ values, all participants in humming consensus merely agree that there were multiple suggestions at that epoch which must be resolved by the creation and writing of newer projections with later epoch numbers.} Machi's projection store read repair can only repair values that are -unwritten, i.e., storing $\bot$. +unwritten, i.e., currently storing $\bot$. -The value used to repair $\bot$ values is the ``best'' projection that +The value used to repair unwritten $\bot$ values is the ``best'' projection that is currently available for the current epoch $E$. If there is a single, unanimous value $V_{u}$ for the projection at epoch $E$, then $V_{u}$ -is use to repair all projections stores at $E$ that contain $\bot$ +is used to repair all projections stores at $E$ that contain $\bot$ values. If the value of $K$ is not unanimous, then the ``highest ranked value'' $V_{best}$ is used for the repair; see Section~\ref{sub:ranking-projections} for a description of projection ranking. -\subsection{Writing to public projection stores} -\label{sub:proj-store-writing} - -Writing replicas of a projection $P_{new}$ to the cluster's public -projection stores is similar, in principle, to writing a Chain -Replication-managed system or Dynamo-like system. But unlike Chain -Replication, the order doesn't really matter. -In fact, the two steps below may be performed in parallel. -The significant difference with Chain Replication is how we interpret -the return status of each write operation. - -\begin{enumerate} -\item Write $P_{new}$ to the local server's public projection store - using $P_{new}$'s epoch number $E$ as the key. - As a side effect, a successful write will trigger - ``wedge'' status in the local server, which will then cascade to other - projection-related activity by the local chain manager. -\item Write $P_{new}$ to key $E$ of each remote public projection store of - all participants in the chain. -\end{enumerate} - -In cases of {\tt error\_written} status, -the process may be aborted and read repair -triggered. The most common reason for {\tt error\_written} status -is that another actor in the system has -already calculated another (perhaps different) projection using the -same projection epoch number and that -read repair is necessary. The {\tt error\_written} may also -indicate that another server has performed read repair on the exact -projection $P_{new}$ that the local server is trying to write! - -\subsection{Writing to private projection stores} - -Only the local server/owner may write to the private half of a -projection store. Also, the private projection store is not replicated. - -\subsection{Reading from public projection stores} -\label{sub:proj-store-reading} - -A read is simple: for an epoch $E$, send a public projection read API -request to all participants. As when writing to the public projection -stores, we can ignore any timeout/unavailable return -status.\footnote{The success/failure status of projection reads and - writes is {\em not} ignored with respect to the chain manager's - internal liveness tracker. However, the liveness tracker's state is - typically only used when calculating new projections.} If we -discover any unwritten values $\bot$, the read repair protocol is -followed. - -The minimum number of non-error responses is only one.\footnote{The local -projection store should always be available, even if no other remote -replica projection stores are available.} If all available servers -return a single, unanimous value $V_u, V_u \ne \bot$, then $V_u$ is -the final result for epoch $E$. -Any non-unanimous values are considered complete disagreement for the -epoch. This disagreement is resolved by humming consensus by later -writes to the public projection stores during subsequent iterations of +If a non-$\bot$ value exists, then by definition\footnote{Definition + of a write-once register} this value is immutable. The only +conflict resolution path is to write a new projection with a newer and +larger epoch number. Once a public projection with epoch number $E$ is +written, projections with epochs smaller than $E$ are ignored by humming consensus. -We are not concerned with unavailable servers. Humming consensus -only uses as many public projections as are available at the present -moment of time. If some server $S$ is unavailable at time $t$ and -becomes available at some later $t+\delta$, and if at $t+\delta$ we -discover that $S$'s public projection store for key $E$ -contains some disagreeing value $V_{weird}$, then the disagreement -will be resolved in the exact same manner that would be used as if we -had found the disagreeing values at the earlier time $t$. - \section{Phases of projection change, a prelude to Humming Consensus} \label{sec:phases-of-projection-change} @@ -671,7 +680,7 @@ straightforward; see Section~\ref{sub:proj-store-writing} for the technique for writing projections to all participating servers' projection stores. Humming Consensus does not care -if the writes succeed or not: its final phase, adopting a +if the writes succeed or not. The next phase, adopting a new projection, will determine which write operations are usable. \subsection{Adoption a new projection} @@ -685,8 +694,8 @@ to avoid direct parallels with protocols such as Raft and Paxos.) In general, a projection $P_{new}$ at epoch $E_{new}$ is adopted by a server only if the change in state from the local server's current projection to new -projection, $P_{current} \rightarrow P_{new}$ will not cause data loss, -e.g., the Update Propagation Invariant and all other safety checks +projection, $P_{current} \rightarrow P_{new}$, will not cause data loss: +the Update Propagation Invariant and all other safety checks required by chain repair in Section~\ref{sec:repair-entire-files} are correct. For example, any new epoch must be strictly larger than the current epoch, i.e., $E_{new} > E_{current}$. @@ -696,16 +705,12 @@ available public projection stores. If the result is not a single unanmous projection, then we return to the step in Section~\ref{sub:projection-calculation}. If the result is a {\em unanimous} projection $P_{new}$ in epoch $E_{new}$, and if $P_{new}$ -does not violate chain safety checks, then the local node may -replace its local $P_{current}$ projection with $P_{new}$. +does not violate chain safety checks, then the local node will: -Not all safe projection transitions are useful, however. For example, -it's trivally safe to suggest projection $P_{zero}$, where the chain -length is zero. In an eventual consistency environment, projection -$P_{one}$ where the chain length is exactly one is also trivially -safe.\footnote{Although, if the total number of participants is more - than one, eventual consistency would demand that $P_{self}$ cannot - be used forever.} +\begin{itemize} +\item write $P_{current}$ to the local private projection store, and +\item set its local operating state $P_{current} \leftarrow P_{new}$. +\end{itemize} \section{Humming Consensus} \label{sec:humming-consensus} @@ -714,13 +719,11 @@ Humming consensus describes consensus that is derived only from data that is visible/available at the current time. It's OK if a network partition is in effect and not all chain members are available; the algorithm will calculate a rough consensus despite not -having input from all chain members. Humming consensus -may proceed to make a decision based on data from only one -participant, i.e., only the local node. +having input from all chain members. \begin{itemize} -\item When operating in AP mode, i.e., in eventual consistency mode, humming +\item When operating in eventual consistency mode, humming consensus may reconfigure a chain of length $N$ into $N$ independent chains of length 1. When a network partition heals, the humming consensus is sufficient to manage the chain so that each @@ -728,11 +731,12 @@ replica's data can be repaired/merged/reconciled safely. Other features of the Machi system are designed to assist such repair safely. -\item When operating in CP mode, i.e., in strong consistency mode, humming -consensus would require additional restrictions. For example, any -chain that didn't have a minimum length of the quorum majority size of -all members would be invalid and therefore would not move itself out -of wedged state. In very general terms, this requirement for a quorum +\item When operating in strong consistency mode, any +chain shorter than the quorum majority of +all members is invalid and therefore cannot be used. Any server with +a too-short chain cannot not move itself out +of wedged state and is therefore unavailable for general file service. +In very general terms, this requirement for a quorum majority of surviving participants is also a requirement for Paxos, Raft, and ZAB. See Section~\ref{sec:split-brain-management} for a proposal to handle ``split brain'' scenarios while in CP mode. @@ -752,8 +756,6 @@ Section~\ref{sec:phases-of-projection-change}: network monitoring, calculating new projections, writing projections, then perhaps adopting the newest projection (which may or may not be the projection that we just wrote). -Beginning with Section~\ref{sub:flapping-state}, we provide -additional detail to the rough outline of humming consensus. \begin{figure*}[htp] \resizebox{\textwidth}{!}{ @@ -801,15 +803,15 @@ is used by the flowchart and throughout this section. \item[$\mathbf{P_{current}}$] The projection actively used by the local node right now. It is also the projection with largest - epoch number in the local node's private projection store. + epoch number in the local node's {\em private} projection store. \item[$\mathbf{P_{newprop}}$] A new projection suggestion, as calculated by the local server (Section~\ref{sub:humming-projection-calculation}). \item[$\mathbf{P_{latest}}$] The highest-ranked projection with the largest - single epoch number that has been read from all available public - projection stores, including the local node's public projection store. + single epoch number that has been read from all available {\em public} + projection stores. \item[Unanimous] The $P_{latest}$ projection is unanimous if all replicas in all accessible public projection stores are effectively @@ -828,7 +830,7 @@ is used by the flowchart and throughout this section. The flowchart has three columns, from left to right: \begin{description} -\item[Column A] Is there any reason to change? +\item[Column A] Is there any reason to act? \item[Column B] Do I act? \item[Column C] How do I act? \begin{description} @@ -863,12 +865,12 @@ In today's implementation, there is only a single criterion for determining the alive/perhaps-not-alive status of a remote server $S$: is $S$'s projection store available now? This question is answered by attemping to read the projection store on server $S$. -If successful, then we assume that all -$S$ is available. If $S$'s projection store is not available for any -reason (including timeout), we assume $S$ is entirely unavailable. -This simple single -criterion appears to be sufficient for humming consensus, according to -simulations of arbitrary network partitions. +If successful, then we assume that $S$ and all of $S$'s network services +are available. If $S$'s projection store is not available for any +reason (including timeout), we inform the local ``fitness server'' +that we have had a problem querying $S$. The fitness service may then +take additional monitoring/querying actions before informing us (in a +later iteration) that $S$ should be considered down. %% {\bf NOTE:} The projection store API is accessed via TCP. The network %% partition simulator, mentioned above and described at @@ -883,64 +885,10 @@ Column~A of Figure~\ref{fig:flowchart}. See also, Section~\ref{sub:projection-calculation}. Execution starts at ``Start'' state of Column~A of -Figure~\ref{fig:flowchart}. Rule $A20$'s uses recent success \& -failures in accessing other public projection stores to select a hard +Figure~\ref{fig:flowchart}. Rule $A20$'s uses judgement from the +local ``fitness server'' to select a definite boolean up/down status for each participating server. -\subsubsection{Calculating flapping state} - -Also at this stage, the chain manager calculates its local -``flapping'' state. The name ``flapping'' is borrowed from IP network -engineer jargon ``route flapping'': - -\begin{quotation} -``Route flapping is caused by pathological conditions -(hardware errors, software errors, configuration errors, intermittent -errors in communications links, unreliable connections, etc.) within -the network which cause certain reachability information to be -repeatedly advertised and withdrawn.'' \cite{wikipedia-route-flapping} -\end{quotation} - -\paragraph{Flapping due to constantly changing network partitions and/or server crashes and restarts} - -Currently, Machi does not attempt to dampen, smooth, or ignore recent -history of constantly flapping peer servers. If necessary, a failure -detector such as the $\phi$ accrual failure detector -\cite{phi-accrual-failure-detector} can be used to help mange such -situations. - -\paragraph{Flapping due to asymmetric network partitions} - -The simulator's behavior during stable periods where at least one node -is the victim of an asymmetric network partition is \ldots weird, -wonderful, and something I don't completely understand yet. This is -another place where we need more eyes reviewing and trying to poke -holes in the algorithm. - -In cases where any node is a victim of an asymmetric network -partition, the algorithm oscillates in a very predictable way: each -server $S$ makes the same $P_{new}$ projection at epoch $E$ that $S$ made -during a previous recent epoch $E-\delta$ (where $\delta$ is small, usually -much less than 10). However, at least one node makes a suggestion that -makes rough consensus impossible. When any epoch $E$ is not -acceptable (because some node disagrees about something, e.g., -which nodes are down), -the result is more new rounds of suggestions that create a repeating -loop that lasts as long as the asymmetric partition lasts. - -From the perspective of $S$'s chain manager, the pattern of this -infinite loop is easy to detect: $S$ inspects the pattern of the last -$L$ projections that it has suggested, e.g., the last 10. -Tiny details such as the epoch number and creation timestamp will -differ, but the major details such as UPI list and repairing list are -the same. - -If the major details of the last $L$ projections authored and -suggested by $S$ are the same, then $S$ unilaterally decides that it -is ``flapping'' and enters flapping state. See -Section~\ref{sub:flapping-state} for additional disucssion of the -flapping state. - \subsubsection{When to calculate a new projection} \label{ssub:when-to-calc} @@ -949,7 +897,7 @@ calculate a new projection. The timer interval is typically 0.5--2.0 seconds, if the cluster has been stable. A client may call an external API call to trigger a new projection, e.g., if that client knows that an environment change has happened and wishes to trigger a -response prior to the next timer firing. +response prior to the next timer firing (e.g.~at state $C200$). It's recommended that the timer interval be staggered according to the participant ranking rules in Section~\ref{sub:ranking-projections}; @@ -970,15 +918,14 @@ done by state $C110$ and that writing a public projection is done by states $C300$ and $C310$. Broadly speaking, there are a number of decisions made in all three -columns of Figure~\ref{fig:flowchart} to decide if and when any type -of projection should be written at all. Sometimes, the best action is +columns of Figure~\ref{fig:flowchart} to decide if and when a +projection should be written at all. Sometimes, the best action is to do nothing. \subsubsection{Column A: Is there any reason to change?} The main tasks of the flowchart states in Column~A is to calculate a -new projection $P_{new}$ and perhaps also the inner projection -$P_{new2}$ if we're in flapping mode. Then we try to figure out which +new projection $P_{new}$. Then we try to figure out which projection has the greatest merit: our current projection $P_{current}$, the new projection $P_{new}$, or the latest epoch $P_{latest}$. If our local $P_{current}$ projection is best, then @@ -1011,7 +958,7 @@ The main decisions that states in Column B need to make are: It's notable that if $P_{new}$ is truly the best projection available at the moment, it must always first be written to everyone's -public projection stores and only then processed through another +public projection stores and only afterward processed through another monitor \& calculate loop through the flowchart. \subsubsection{Column C: How do I act?} @@ -1053,14 +1000,14 @@ therefore the suggested projections at epoch $E$ are not unanimous. \paragraph{\#2: The transition from current $\rightarrow$ new projection is safe} -Given the projection that the server is currently using, +Given the current projection $P_{current}$, the projection $P_{latest}$ is evaluated by numerous rules and invariants, relative to $P_{current}$. If such rule or invariant is violated/false, then the local server will discard $P_{latest}$. -The transition from $P_{current} \rightarrow P_{latest}$ is checked -for safety and sanity. The conditions used for the check include: +The transition from $P_{current} \rightarrow P_{latest}$ is protected +by rules and invariants that include: \begin{enumerate} \item The Erlang data types of all record members are correct. @@ -1073,161 +1020,13 @@ for safety and sanity. The conditions used for the check include: The same re-reordering restriction applies to all servers in $P_{latest}$'s repairing list relative to $P_{current}$'s repairing list. -\item Any server $S$ that was added to $P_{latest}$'s UPI list must +\item Any server $S$ that is newly-added to $P_{latest}$'s UPI list must appear in the tail the UPI list. Furthermore, $S$ must have been in $P_{current}$'s repairing list and had successfully completed file - repair prior to the transition. + repair prior to $S$'s promotion from the repairing list to the tail + of the UPI list. \end{enumerate} -\subsection{Additional discussion of flapping state} -\label{sub:flapping-state} -All $P_{new}$ projections -calculated while in flapping state have additional diagnostic -information added, including: - -\begin{itemize} -\item Flag: server $S$ is in flapping state. -\item Epoch number \& wall clock timestamp when $S$ entered flapping state. -\item The collection of all other known participants who are also - flapping (with respective starting epoch numbers). -\item A list of nodes that are suspected of being partitioned, called the - ``hosed list''. The hosed list is a union of all other hosed list - members that are ever witnessed, directly or indirectly, by a server - while in flapping state. -\end{itemize} - -\subsubsection{Flapping diagnostic data accumulates} - -While in flapping state, this diagnostic data is gathered from -all available participants and merged together in a CRDT-like manner. -Once added to the diagnostic data list, a datum remains until -$S$ drops out of flapping state. When flapping state stops, all -accumulated diagnostic data is discarded. - -This accumulation of diagnostic data in the projection data -structure acts in part as a substitute for a separate gossip protocol. -However, since all participants are already communicating with each -other via read \& writes to each others' projection stores, the diagnostic -data can propagate in a gossip-like manner via the projection stores. - -\subsubsection{Flapping example (part 1)} -\label{ssec:flapping-example} - -Any server listed in the ``hosed list'' is suspected of having some -kind of network communication problem with some other server. For -example, let's examine a scenario involving a Machi cluster of servers -$a$, $b$, $c$, $d$, and $e$. Assume there exists an asymmetric network -partition such that messages from $a \rightarrow b$ are dropped, but -messages from $b \rightarrow a$ are delivered.\footnote{If this - partition were happening at or below the level of a reliable - delivery network protocol like TCP, then communication in {\em both} - directions would be affected by an asymmetric partition. - However, in this model, we are - assuming that a ``message'' lost during a network partition is a - uni-directional projection API call or its response.} - -Once a participant $S$ enters flapping state, it starts gathering the -flapping starting epochs and hosed lists from all of the other -projection stores that are available. The sum of this info is added -to all projections calculated by $S$. -For example, projections authored by $a$ will say that $a$ believes -that $b$ is down. -Likewise, projections authored by $b$ will say that $b$ believes -that $a$ is down. - -\subsubsection{The inner projection (flapping example, part 2)} -\label{ssec:inner-projection} - -\ldots We continue the example started in the previous subsection\ldots - -Eventually, in a gossip-like manner, all other participants will -eventually find that their hosed list is equal to $[a,b]$. Any other -server, for example server $c$, will then calculate another -projection, $P_{new2}$, using the assumption that both $a$ and $b$ -are down in addition to all other known unavailable servers. - -\begin{itemize} -\item If operating in the default CP mode, both $a$ and $b$ are down - and therefore not eligible to participate in Chain Replication. - %% The chain may continue service if a $c$, $d$, $e$ and/or witness - %% servers can try to form a correct UPI list for the chain. - This may cause an availability problem for the chain: we may not - have a quorum of participants (real or witness-only) to form a - correct UPI chain. -\item If operating in AP mode, $a$ and $b$ can still form two separate - chains of length one, using UPI lists of $[a]$ and $[b]$, respectively. -\end{itemize} - -This re-calculation, $P_{new2}$, of the new projection is called an -``inner projection''. The inner projection definition is nested -inside of its parent projection, using the same flapping disagnostic -data used for other flapping status tracking. - -When humming consensus has determined that a projection state change -is necessary and is also safe (relative to both the outer and inner -projections), then the outer projection\footnote{With the inner - projection $P_{new2}$ nested inside of it.} is written to -the local private projection store. -With respect to future iterations of -humming consensus, the innter projection is ignored. -However, with respect to Chain Replication, the server's subsequent -behavior -{\em will consider the inner projection only}. The inner projection -is used to order the UPI and repairing parts of the chain and trigger -wedge/un-wedge behavior. The inner projection is also -advertised to Machi clients. - -The epoch of the inner projection, $E^{inner}$ is always less than or -equal to the epoch of the outer projection, $E$. The $E^{inner}$ -epoch typically only changes when new servers are added to the hosed -list. - -To attempt a rough analogy, the outer projection is the carrier wave -that is used to transmit the inner projection and its accompanying -gossip of diagnostic data. - -\subsubsection{Outer projection churn, inner projection stability} - -One of the intriguing features of humming consensus's reaction to -asymmetric partition: flapping behavior continues for as long as -an any asymmetric partition exists. - -\subsubsection{Stability in symmetric partition cases} - -Although humming consensus hasn't been formally proven to handle all -asymmetric and symmetric partition cases, the current implementation -appears to converge rapidly to a single chain state in all symmetric -partition cases. This is in contrast to asymmetric partition cases, -where ``flapping'' will continue on every humming consensus iteration -until all asymmetric partition disappears. A formal proof is an area of -future work. - -\subsubsection{Leaving flapping state and discarding inner projection} - -There are two events that can trigger leaving flapping state. - -\begin{itemize} - -\item A server $S$ in flapping state notices that its long history of - repeatedly suggesting the same projection will be broken: - $S$ instead calculates some differing projection instead. - This change in projection history happens whenever a perceived network - partition changes in any way. - -\item Server $S$ reads a public projection suggestion, $P_{noflap}$, that is - authored by another server $S'$, and that $P_{noflap}$ no longer - contains the flapping start epoch for $S'$ that is present in the - history that $S$ has maintained while $S$ has been in - flapping state. - -\end{itemize} - -When either trigger event happens, server $S$ will exit flapping state. All -new projections authored by $S$ will have all flapping diagnostic data -removed. This includes stopping use of the inner projection: the UPI -list of the inner projection is copied to the outer projection's UPI -list, to avoid a drastic change in UPI membership. - \subsection{Ranking projections} \label{sub:ranking-projections} @@ -1789,7 +1588,7 @@ Manageability, availability and performance in Porcupine: a highly scalable, clu {\tt http://homes.cs.washington.edu/\%7Elevy/ porcupine.pdf} \bibitem{chain-replication} -van Renesse, Robbert et al. +van Renesse, Robbert and Schneider, Fred. Chain Replication for Supporting High Throughput and Availability. Proceedings of the 6th Conference on Symposium on Operating Systems Design \& Implementation (OSDI'04) - Volume 6, 2004. diff --git a/doc/src.high-level/high-level-machi.tex b/doc/src.high-level/high-level-machi.tex index a95642a..9c7c87b 100644 --- a/doc/src.high-level/high-level-machi.tex +++ b/doc/src.high-level/high-level-machi.tex @@ -1489,7 +1489,7 @@ In Usenix ATC 2009. {\tt https://www.usenix.org/legacy/event/usenix09/ tech/full\_papers/terrace/terrace.pdf} \bibitem{chain-replication} -van Renesse, Robbert et al. +van Renesse, Robbert and Schneider, Fred. Chain Replication for Supporting High Throughput and Availability. Proceedings of the 6th Conference on Symposium on Operating Systems Design \& Implementation (OSDI'04) - Volume 6, 2004. -- 2.45.2 From 447c8c8d48462feafc8961cb0afe15c2ba27afa7 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Tue, 27 Oct 2015 11:27:08 +0900 Subject: [PATCH 13/44] Add 2-tuple timeout setting to CR client --- src/machi_cr_client.erl | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/machi_cr_client.erl b/src/machi_cr_client.erl index 2b32308..b8478dc 100644 --- a/src/machi_cr_client.erl +++ b/src/machi_cr_client.erl @@ -1065,5 +1065,7 @@ chunk_wrapper_size(Chunk) -> timeout(infinity) -> timeout(15*60*1000); % close enough to infinity +timeout({_, _}=Timeout) -> + Timeout; timeout(Timeout0) -> {Timeout0, Timeout0 + 30*1000}. -- 2.45.2 From bf5768eb4772a066fb120d50fa101cea49eb5475 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Tue, 27 Oct 2015 11:30:29 +0900 Subject: [PATCH 14/44] Make CR client partition-simulator-aware --- src/machi_cr_client.erl | 32 ++++++++++++++++++++++++-------- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/src/machi_cr_client.erl b/src/machi_cr_client.erl index b8478dc..8630d29 100644 --- a/src/machi_cr_client.erl +++ b/src/machi_cr_client.erl @@ -114,7 +114,7 @@ -include_lib("eunit/include/eunit.hrl"). -endif. % TEST. --export([start_link/1]). +-export([start_link/1, start_link/2]). %% FLU1 API -export([ %% File API @@ -146,7 +146,8 @@ proxies_dict :: orddict:orddict(), epoch_id, proj, - bad_proj + bad_proj, + opts :: proplists:proplist() }). %% @doc Start a local, long-lived process that will be our steady @@ -154,7 +155,10 @@ %% remote Machi server. start_link(P_srvr_list) -> - gen_server:start_link(?MODULE, [P_srvr_list], []). + gen_server:start_link(?MODULE, [P_srvr_list, []], []). + +start_link(P_srvr_list, Opts) -> + gen_server:start_link(?MODULE, [P_srvr_list, Opts], []). %% @doc Append a chunk (binary- or iolist-style) of data to a file %% with `Prefix'. @@ -254,10 +258,10 @@ quit(PidSpec) -> %%%%%%%%%%%%%%%%%%%%%%%%%%% -init([P_srvr_list]) -> +init([P_srvr_list, Opts]) -> MembersDict = orddict:from_list([{P#p_srvr.name, P} || P <- P_srvr_list]), ProxiesDict = ?FLU_PC:start_proxies(MembersDict), - {ok, #state{members_dict=MembersDict, proxies_dict=ProxiesDict}}. + {ok, #state{members_dict=MembersDict, proxies_dict=ProxiesDict, opts=Opts}}. handle_call({req, Req}, From, S) -> handle_call2(Req, From, update_proj(S)); @@ -923,10 +927,22 @@ update_proj(#state{proj=undefined}=S) -> update_proj(S) -> S. -update_proj2(Count, #state{bad_proj=BadProj, proxies_dict=ProxiesDict}=S) -> +update_proj2(Count, #state{bad_proj=BadProj, proxies_dict=ProxiesDict, + opts=Opts}=S) -> Timeout = 2*1000, WTimeout = 2*Timeout, - Proxies = orddict:to_list(ProxiesDict), + SimName = proplists:get_value(simulator_self_name, Opts, cr_client), + ExcludedFLUs = + case proplists:get_value(use_partition_simulator, Opts, false) of + true -> + Members = proplists:get_value(simulator_members, Opts, []), + {Partitions, _Islands} = machi_partition_simulator:get(Members), + [B || {A, B} <- Partitions, A =:= SimName]; + false -> [] + end, + Proxies = lists:foldl(fun(Name, Dict) -> + orddict:erase(Name, Dict) + end, ProxiesDict, ExcludedFLUs), Work = fun({_K, Proxy}) -> ?FLU_PC:read_latest_projection(Proxy, private, Timeout) end, @@ -950,7 +966,7 @@ update_proj2(Count, #state{bad_proj=BadProj, proxies_dict=ProxiesDict}=S) -> NewProxiesDict = ?FLU_PC:start_proxies(NewMembersDict), S#state{bad_proj=undefined, proj=P, epoch_id=EpochID, members_dict=NewMembersDict, proxies_dict=NewProxiesDict}; - _ -> + _P -> sleep_a_while(Count), update_proj2(Count + 1, S) end. -- 2.45.2 From 93b168415d1a55327f135dc5f24329a9c0107468 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Tue, 27 Oct 2015 11:36:12 +0900 Subject: [PATCH 15/44] Change log level to debug for multiple files with certain seq num --- src/machi_flu_filename_mgr.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/machi_flu_filename_mgr.erl b/src/machi_flu_filename_mgr.erl index 54fdcfe..7e8bb9d 100644 --- a/src/machi_flu_filename_mgr.erl +++ b/src/machi_flu_filename_mgr.erl @@ -205,7 +205,7 @@ handle_find_file(Tid, Prefix, DataDir) -> {find_or_make_filename(Tid, DataDir, Prefix, N), false}; [H] -> {H, true}; [Fn | _ ] = L -> - lager:warning( + lager:debug( "Searching for a matching file to prefix ~p and sequence number ~p gave multiples: ~p", [Prefix, N, L]), {Fn, true} -- 2.45.2 From 6fa2de28cdd46550d381d328b266d2f134f4a005 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Thu, 29 Oct 2015 14:46:07 +0900 Subject: [PATCH 16/44] Add self pid to debug print --- src/machi_cr_client.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/machi_cr_client.erl b/src/machi_cr_client.erl index 8630d29..c4274b2 100644 --- a/src/machi_cr_client.erl +++ b/src/machi_cr_client.erl @@ -955,7 +955,7 @@ update_proj2(Count, #state{bad_proj=BadProj, proxies_dict=ProxiesDict, %% b's projection. case choose_best_proj(Rs) of P when P == ?WORST_PROJ -> - io:format(user, "TODO: Using ?WORST_PROJ, chain is not available\n", []), + io:format(user, "TODO: Using ?WORST_PROJ, chain is not available ~w\n", [self()]), sleep_a_while(Count), update_proj2(Count + 1, S); P when P >= BadProj -> -- 2.45.2 From b5005c35263e79389c42b1808fce1171b44f4fb3 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Tue, 27 Oct 2015 11:59:11 +0900 Subject: [PATCH 17/44] Add EQC test case for AP mode repair w/ part. sim. --- test/machi_ap_repair_eqc.erl | 568 +++++++++++++++++++++++++++++++++++ 1 file changed, 568 insertions(+) create mode 100644 test/machi_ap_repair_eqc.erl diff --git a/test/machi_ap_repair_eqc.erl b/test/machi_ap_repair_eqc.erl new file mode 100644 index 0000000..03fee9e --- /dev/null +++ b/test/machi_ap_repair_eqc.erl @@ -0,0 +1,568 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. +%% +%% This file is provided to you under the Apache License, +%% Version 2.0 (the "License"); you may not use this file +%% except in compliance with the License. You may obtain +%% a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, +%% software distributed under the License is distributed on an +%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +%% KIND, either express or implied. See the License for the +%% specific language governing permissions and limitations +%% under the License. +%% +%% ------------------------------------------------------------------- + +%% EQC single-threaded and concurrent test for file operations and repair +%% under simulated network partition. + +%% The main purpose is to confirm no dataloss, i.e. every chunk that +%% has been successfully written (ACK received) by append/write +%% opration will be read after partition heals. +%% +%% All updating -- append, write and trim -- operations are executed +%% through CR client, not directly by flu1 client, in order to be +%% end-to-end test (in single chain point of veiw.) There may be churn +%% for projections by simulated network partition. +%% +%% Test steps +%% 1. Setup single chain. +%% 2. Execute updating operations and simulated partition (by eqc_statem). +%% Every updating results are recorded in ETS tables. +%% 3. When {error, timeout|partition} happens, trigger management tick for +%% every chain manager process. +%% 4. After commands are executed, remove patition and wait for the chain +%% without down nodes nor repairing nodes. +%% 5. Asserting written results so that each record be read from the +%% chain and data be the same with written one. + +%% Improvements to-do's +%% - Use higher concurrency, e.g. 10+ +%% - Random length for binary to write +%% - Operations other than append, write, trim +%% - Use checksum instead of binary to save memory +%% - More variety for partitioning pattern: non-constant failure + +-module(machi_ap_repair_eqc). + +-ifdef(TEST). +-ifdef(EQC). +-compile(export_all). +-include("machi.hrl"). +-include("machi_projection.hrl"). +-include("machi_verbose.hrl"). +-include_lib("eqc/include/eqc.hrl"). +-include_lib("eqc/include/eqc_statem.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +%% ETS table names +-define(WRITTEN_TAB, written). % Successfully written data +-define(ACCPT_TAB, accpt). % Errors with no harm, e.g. timeout +-define(FAILED_TAB, failed). % Uncategorized errors, when happenes + % it should be re-categorized to accept or critical +-define(CRITICAL_TAB, critical). % Critical errors, e.g. double write to the same key + +-define(QC_OUT(P), + eqc:on_output(fun(Str, Args) -> io:format(user, Str, Args) end, P)). + +%% EUNIT TEST DEFINITION +prop_repair_test_() -> + {PropTO, EUnitTO} = eqc_timeout(60), + Verbose = eqc_verbose(), + {spawn, + [{timeout, EUnitTO, + ?_assertEqual( + true, + eqc:quickcheck(eqc:testing_time( + PropTO, ?QC_OUT(noshrink(prop_repair(Verbose))))))}]}. + +prop_repair_par_test_() -> + {PropTO, EUnitTO} = eqc_timeout(60), + Verbose = eqc_verbose(), + {spawn, + [{timeout, EUnitTO, + ?_assertEqual( + true, + eqc:quickcheck(eqc:testing_time( + PropTO, ?QC_OUT(noshrink(prop_repair_par(Verbose))))))}]}. + +%% SHELL HELPERS +test() -> test(100). +test(N) -> test(N, true). +test(N, Verbose) -> quickcheck(numtests(N, noshrink(prop_repair_par(Verbose)))). +check() -> check(prop_repair_par(true), current_counterexample()). + +-record(state, {num, % Number of FLU servers + seed, % Seed for partition simulator + verbose=false, % Verbose output for debugging + flu_names, % List of FLU names + mgr_names, % List of chain manager names + fc_list, % List of FLU1 proxy clients + cr_list}). % List of CR clients + +weight(_S, change_partition) -> 20; +weight(_S, _) -> 100. + +%% append + +append_args(#state{cr_list=CRList}=S) -> + [elements(CRList), chunk(), S]. + +append({_SimSelfName, C}, Bin, #state{verbose=V}=S) -> + Prefix = <<"pre">>, + Len = byte_size(Bin), + Res = (catch machi_cr_client:append_chunk(C, Prefix, Bin, {sec(1), sec(1)})), + case Res of + {ok, {_Off, Len, _FileName}=Key} -> + case ets:insert_new(?WRITTEN_TAB, {Key, Bin}) of + true -> + [?V("", []) || V], + ok; + false -> + %% The Key is alread written, WHY!!!???? + case ets:lookup(?WRITTEN_TAB, Key) of + [{Key, Bin}] -> + %% TODO: The identical binary is alread inserted in + %% written table. Is this acceptable??? Hmm, maybe NO... + [?V("", [Key]) || V], + true = ets:insert_new(?ACCPT_TAB, + {make_ref(), double_write_same, Key}), + {acceptable_error, doublewrite_the_same}; + [{Key, OtherBin}] -> + [?V("", [Key, {OtherBin, Bin}]) || V], + true = ets:insert_new(?CRITICAL_TAB, + {make_ref(), double_write_diff, Key}), + R = {critical_error, + {doublewrite_diff, Key, {OtherBin, Bin}}}, + %% TODO: when double write happens, it seems that + %% repair process got stack with endless loop. To + %% avoit it, return error here. + %% If this error/1 will be removed, one can possibly + %% know double write frequency/rate. + error(R) + end + end; + {error, partition} -> + [?V("", []) || V], + true = ets:insert_new(?ACCPT_TAB, {make_ref(), timeout}), + _ = tick(S), + {acceptable_error, partition}; + {'EXIT', {timeout, _}} -> + [?V("", [_SimSelfName, C]) || V], + true = ets:insert_new(?ACCPT_TAB, {make_ref(), timeout}), + _ = tick(S), + {acceptable_error, timeout}; + {ok, {_Off, UnexpectedLen, _FileName}=Key} -> + [?V("", []) || V], + true = ets:insert_new(?CRITICAL_TAB, {make_ref(), unexpected_len, Key}), + {critical_error, {unexpected_len, Key, Len, UnexpectedLen}}; + {error, _Reason} = Error -> + [?V("", []) || V], + true = ets:insert_new(?FAILED_TAB, {make_ref(), Error}), + {other_error, Error}; + Other -> + [?V("", []) || V], + true = ets:insert_new(?FAILED_TAB, {make_ref(), Other}), + {other_error, Other} + end. + +%% change partition + +change_partition_args(#state{flu_names=FLUNames}=S) -> + %% [partition(FLUNames), S]. + [partition_sym(FLUNames), S]. + +change_partition(Partition, + #state{verbose=Verbose, flu_names=FLUNames}=S) -> + [case Partition of + [] -> ?V("## Turn OFF partition: ~w~n", [Partition]); + _ -> ?V("## Turn ON partition: ~w~n", [Partition]) + end || Verbose], + machi_partition_simulator:always_these_partitions(Partition), + _ = machi_partition_simulator:get(FLUNames), + %% Don't wait for stable chain, tick will be executed on demand + %% in append oprations + _ = tick(S), + ok. + +%% Generators + +num() -> + choose(2, 5). + %% return(3). + +%% Generator for possibly assymmetric partition information +partition(FLUNames) -> + frequency([{10, return([])}, + {20, non_empty(sublist(flu_ordered_pairs(FLUNames)))}]). + +%% Generator for symmetric partition information +partition_sym(FLUNames) -> + ?LET(Pairs, non_empty(sublist(flu_pairs(FLUNames))), + lists:flatmap(fun({One, Another}) -> [{One, Another}, {Another, One}] end, + Pairs)). + +flu_ordered_pairs(FLUNames) -> + [{From, To} || From <- FLUNames, To <- FLUNames, From =/= To]. + +flu_pairs(FLUNames) -> + [{One, Another} || One <- FLUNames, Another <- FLUNames, One > Another]. + +chunk() -> + non_empty(binary(10)). + +%% Property + +prop_repair(Verbose) -> + error_logger:tty(false), + application:load(sasl), + application:set_env(sasl, sasl_error_logger, false), + + Seed = {1445,935441,287549}, + ?FORALL(Num, num(), + ?FORALL(Cmds, commands(?MODULE, initial_state(Num, Seed, Verbose)), + begin + SetupState = setup_chain(Num, Seed, Verbose), + {H, S1, Res} = run_commands(?MODULE, Cmds), + %% ?V("S1=~w~n", [S1]), + ?V("==== Start post operations, stabilize and confirm results~n", []), + {_Res2, S2} = stabilize(commands_len(Cmds), SetupState), + {Dataloss, Critical} = confirm_result(S2), + pretty_commands( + ?MODULE, Cmds, {H, S1, Res}, + aggregate(with_title(cmds), command_names(Cmds), + collect(with_title(length5), (length(Cmds) div 5) * 5, + {Dataloss, Critical} =:= {0, 0}))) + end)). + +prop_repair_par(Verbose) -> + error_logger:tty(false), + application:load(sasl), + application:set_env(sasl, sasl_error_logger, false), + + Seed = {1445,935441,287549}, + ?FORALL(Num, num(), + ?FORALL(Cmds, + %% Now try-and-err'ing, how to control command length and concurrency? + ?SUCHTHAT(Cmds0, ?SIZED(Size, resize(Size, + parallel_commands(?MODULE, initial_state(Num, Seed, Verbose)))), + commands_len(Cmds0) > 20 + andalso + concurrency(Cmds0) > 2), + begin + CmdsLen= commands_len(Cmds), + SetupState = setup_chain(Num, Seed, Verbose), + {Seq, Par, Res} = run_parallel_commands(?MODULE, Cmds), + %% ?V("Seq=~w~n", [Seq]), + %% ?V("Par=~w~n", [Par]), + ?V("==== Start post operations, stabilize and confirm results~n", []), + {Dataloss, Critical} = + case Res of + ok -> + {_Res2, S2} = stabilize(CmdsLen, SetupState), + confirm_result(S2); + _ -> + ?V("Res=~w~n", [Res]), + {undefined, undefined} + end, + pretty_commands( + ?MODULE, Cmds, {Seq, Par, Res}, + aggregate(with_title(cmds), command_names(Cmds), + collect(with_title(length5), (CmdsLen div 5) * 5, + collect(with_title(conc), concurrency(Cmds), + {Dataloss, Critical} =:= {0, 0}))) + ) + end)). + +%% Initilization / setup + +%% Fake initialization function for debugging in shell like: +%% > eqc_gen:sample(eqc_statem:commands(machi_ap_repair_eqc)). +%% but not so helpful. +initial_state() -> + #state{cr_list=[a,b,c]}. + +initial_state(Num, Seed, Verbose) -> + AllListE = all_list_extra(Num), + FLUNames = [P#p_srvr.name || {P, _Dir} <- AllListE], + MgrNames = [{Name, machi_flu_psup:make_mgr_supname(Name)} || Name <- FLUNames], + Dict = orddict:from_list([{P#p_srvr.name, P} || {P, _Dir} <- AllListE]), + + FCList = [begin + {ok, PCPid} = machi_proxy_flu1_client:start_link(P), + {Name, PCPid} + end || {_, #p_srvr{name=Name}=P} <- Dict], + %% CR clients are pooled, each has "name" which is interpreted "From" + %% side of simulated partition. + CRListCount = 10, % ad-hoc + SimSelfNames = lists:append(lists:duplicate(CRListCount div Num +1, FLUNames)), + CRList = [begin + {ok, C} = machi_cr_client:start_link( + [P || {_, P} <- Dict], + [{use_partition_simulator, true}, + {simulator_self_name, SimSelfName}, + {simulator_members, FLUNames}]), + {SimSelfName, C} + end || SimSelfName <- SimSelfNames], + #state{num=Num, seed=Seed, verbose=Verbose, + flu_names=FLUNames, mgr_names=MgrNames, + cr_list=CRList, fc_list=FCList}. + +setup_chain(Num, Seed, Verbose) -> + %% ?V("setup_chain(Num=~w, Seed=~w~nn", [Num, Seed]), + AllListE = all_list_extra(Num), + FLUNames = [P#p_srvr.name || {P, _Dir} <- AllListE], + MgrNames = [{Name, machi_flu_psup:make_mgr_supname(Name)} || Name <- FLUNames], + Dict = orddict:from_list([{P#p_srvr.name, P} || {P, _Dir} <- AllListE]), + + ok = shutdown_hard(), + [begin + machi_flu1_test:clean_up_data_dir(Dir), + filelib:ensure_dir(Dir ++ "/not-used") + end || {_P, Dir} <- AllListE], + [catch ets:delete(T) || T <- tabs()], + + [ets:new(T, [set, public, named_table, + {write_concurrency, true}, {read_concurrency, true}]) || + T <- tabs()], + {ok, _} = application:ensure_all_started(machi), + + SimSpec = {part_sim, + {machi_partition_simulator, start_link, [{0,0,0}, 0, 100]}, + permanent, 500, worker, []}, + {ok, _PSimPid} = supervisor:start_child(machi_sup, SimSpec), + ok = machi_partition_simulator:set_seed(Seed), + _Partitions = machi_partition_simulator:get(FLUNames), + + %% Start FLUs and setup the chain + FLUOpts = [{use_partition_simulator, true}, + %% {private_write_verbose, true}, + {active_mode, false}, + {simulate_repair, false}], + [{ok, _} = machi_flu_psup:start_flu_package(Name, Port, Dir, FLUOpts) || + {#p_srvr{name=Name, port=Port}, Dir} <- AllListE], + [machi_chain_manager1:set_chain_members(MgrName, Dict) || {_, MgrName} <- MgrNames], + + State = initial_state(Num, Seed, Verbose), + %% Don't wait for complete chain. Even partialy completed, the chain + %% should work fine. Right? + wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, + State#state.fc_list, 20), + State. + +%% Post commands + +stabilize(0, S) -> + {ok, S}; +stabilize(_CmdsLen, #state{flu_names=FLUNames, mgr_names=MgrNames, fc_list=FCList}=S) -> + machi_partition_simulator:no_partitions(), + wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, FCList, 100), + {ok, S}. + +chain_state_all_ok(FLUNames) -> + [{FLUName, {FLUNames, [], []}} || FLUName <- FLUNames]. + +confirm_result(#state{cr_list=[{_, C}|_]}=_S) -> + [{written, _Written}, {accpt, Accpt}, + {failed, Failed}, {critical, Critical}] = tab_counts(), + {OK, Dataloss} = confirm_written(C), + ?V(" Written=~w, DATALOSS=~w, Acceptable=~w~n", [OK, Dataloss, Accpt]), + ?V(" Failed=~w, Critical=~w~n~n", [Failed, Critical]), + DirBase = get_dir_base(), + Suffix = dump_file_suffix(), + case Failed of + 0 -> ok; + _ -> + DumpFailed = filename:join(DirBase, "dump-failed-" ++ Suffix), + ?V("Dump failed ETS tab to: ~w~n", [DumpFailed]), + ets:tab2file(?FAILED_TAB, DumpFailed) + end, + case Critical of + 0 -> ok; + _ -> + DumpCritical = filename:join(DirBase, "dump-critical-" ++ Suffix), + ?V("Dump critical ETS tab to: ~w~n", [DumpCritical]), + ets:tab2file(?CRITICAL_TAB, DumpCritical) + end, + {Dataloss, Critical}. + +confirm_written(C) -> + ets:foldl( + fun({Key, Bin}, {OK, NG}) -> + case assert_chunk(C, Key, Bin) of + ok -> {OK+1, NG}; + {error, _} -> {OK, NG+1} + end + end, {0, 0}, ?WRITTEN_TAB). + +assert_chunk(C, {Off, Len, FileName}=Key, Bin) -> + %% TODO: This probably a bug, read_chunk respnds with filename of `string()' type + FileNameStr = binary_to_list(FileName), + %% TODO : Use CSum instead of binary (after disuccsion about CSum is calmed down?) + case (catch machi_cr_client:read_chunk(C, FileName, Off, Len, [], sec(3))) of + {ok, {[{FileNameStr, Off, Bin, _}], []}} -> + ok; + {ok, Got} -> + ?V("read_chunk got different binary for Key=~p~n", [Key]), + ?V(" Expected: ~p~n", [{[{FileNameStr, Off, Bin, <<"CSum-NYI">>}], []}]), + ?V(" Got: ~p~n", [Got]), + {error, different_binary}; + {error, Reason} -> + ?V("read_chunk error for Key=~p: ~p~n", [Key, Reason]), + {error, Reason}; + Other -> + ?V("read_chunk other error for Key=~p: ~p~n", [Key, Other]), + {error, Other} + end. + +%% Internal utilities + +eqc_verbose() -> + os:getenv("EQC_VERBOSE") =:= "true". + +eqc_timeout(Default) -> + PropTimeout = case os:getenv("EQC_TIMEOUT") of + false -> Default; + V -> list_to_integer(V) + end, + {PropTimeout, PropTimeout * 300}. + +%% Returns a list like +%% `[{#p_srvr{name=a, port=7501, ..}, "./eqc/data.eqc.a/"}, ...]' +all_list_extra(Num) -> + {PortBase, DirBase} = get_port_dir_base(), + [begin + FLUNameStr = [$a + I - 1], + FLUName = list_to_atom(FLUNameStr), + MgrName = machi_flu_psup:make_mgr_supname(FLUName), + {#p_srvr{name=FLUName, address="localhost", port=PortBase+I, + props=[{chmgr, MgrName}]}, + DirBase ++ "/data.eqc." ++ FLUNameStr} + end || I <- lists:seq(1, Num)]. + +get_port_dir_base() -> + I = case os:getenv("EQC_BASE_PORT") of + false -> 0; + V -> list_to_integer(V) + end, + D = get_dir_base(), + {7400 + (I * 100), D ++ "/" ++ integer_to_list(I)}. + +get_dir_base() -> + case os:getenv("EQC_BASE_DIR") of + false -> "./eqc"; + DD -> DD + end. + +shutdown_hard() -> + _STOP = application:stop(machi), + timer:sleep(100). + +tick(#state{flu_names=FLUNames, mgr_names=MgrNames, + fc_list=FCList, verbose=Verbose}) -> + tick(FLUNames, MgrNames, FCList, Verbose). + +tick(FLUNames, MgrNames, FCList, Verbose) -> + tick(FLUNames, MgrNames, 2, 100, FCList, Verbose). + +tick(FLUNames, MgrNames, Iter, SleepMax, FCList, Verbose) -> + TickFun = tick_fun(FLUNames, MgrNames, self()), + TickFun(Iter, 0, SleepMax), + [?V("## Chain state after tick()=~w~n", [chain_state(FCList)]) || Verbose]. + +tick_fun(FLUNames, MgrNames, Parent) -> + fun(Iters, SleepMin, SleepMax) -> + %% ?V("^", []), + Trigger = + fun(FLUName, MgrName) -> + random:seed(now()), + [begin + erlang:yield(), + SleepMaxRand = random:uniform(SleepMax + 1), + %% io:format(user, "{t}", []), + Elapsed = machi_chain_manager1:sleep_ranked_order( + SleepMin, SleepMaxRand, + FLUName, FLUNames), + MgrName ! tick_check_environment, + %% Be more unfair by not sleeping here. + timer:sleep(max(SleepMax - Elapsed, 1)), + ok + end || _ <- lists:seq(1, Iters)], + Parent ! {done, self()} + end, + Pids = [{spawn(fun() -> Trigger(FLUName, MgrName) end), FLUName} || + {FLUName, MgrName} <- MgrNames ], + [receive + {done, ThePid} -> + ok + after 120*1000 -> + exit({icky_timeout, M_name}) + end || {ThePid, M_name} <- Pids] + end. + +wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList) -> + wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, 20). + +wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, Retries) -> + TickFun = tick_fun(FLUNames, MgrNames, self()), + wait_until_stable1(ExpectedChainState, TickFun, FCList, Retries). + +wait_until_stable1(_ExpectedChainState, _TickFun, FCList, 0) -> + ?V(" [ERROR] wait_until_stable failed.... : ~p~n", [chain_state(FCList)]), + false; +wait_until_stable1(ExpectedChainState, TickFun, FCList, Reties) -> + [TickFun(3, 0, 100) || _ <- lists:seq(1, 3)], + Normalized = normalize_chain_state(chain_state(FCList)), + case Normalized of + ExpectedChainState -> + ?V(" Got stable chain: ~w~n", [chain_state(FCList)]), + true; + _ -> + ?V(" NOT YET stable chain: ~w~n", [chain_state(FCList)]), + wait_until_stable1(ExpectedChainState, TickFun, FCList, Reties-1) + end. + +normalize_chain_state(ChainState) -> + lists:usort([{FLUName, + {lists:usort(UPI), lists:usort(Repairing), lists:usort(Down)}} || + {FLUName, {_EpochNo, UPI, Repairing, Down}} <- ChainState]). + +chain_state(FCList) -> + lists:usort( + [case (catch machi_proxy_flu1_client:read_latest_projection(C, private, sec(5))) of + {ok, #projection_v1{epoch_number=EpochNo, upi=UPI, + repairing=Repairing, down=Down}} -> + {FLUName, {EpochNo, UPI, Repairing, Down}}; + Other -> + {FLUName, Other} + end || {FLUName, C} <- FCList]). + +tabs() -> [?WRITTEN_TAB, ?ACCPT_TAB, ?FAILED_TAB, ?CRITICAL_TAB]. + +tab_counts() -> + [{T, ets:info(T, size)} || T <- tabs()]. + +sec(Sec) -> + timer:seconds(Sec). + +commands_len({SeqCmds, ParCmdsList} = _Cmds) -> + lists:sum([length(SeqCmds) | [length(P) || P <- ParCmdsList]]); +commands_len(Cmds) -> + length(Cmds). + +concurrency({_SeqCmds, ParCmdsList} = _Cmds) -> length(ParCmdsList); +concurrency(_) -> 1. + +dump_file_suffix() -> + {{Year, Month, Day}, {Hour, Min, Sec}} = calendar:local_time(), + lists:flatten( + io_lib:format("~4.10.0B-~2.10.0B-~2.10.0BT~2.10.0B:~2.10.0B:~2.10.0B.000Z", + [Year, Month, Day, Hour, Min, Sec])). + +-endif. % EQC +-endif. % TEST -- 2.45.2 From 72a4fab49d60963e6c8e2e2e9ce45ff5a0558acb Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Thu, 29 Oct 2015 22:18:20 -0500 Subject: [PATCH 18/44] Add a naive diff function --- src/machi_merkle_tree.erl | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/src/machi_merkle_tree.erl b/src/machi_merkle_tree.erl index a799edc..c4a4516 100644 --- a/src/machi_merkle_tree.erl +++ b/src/machi_merkle_tree.erl @@ -102,7 +102,7 @@ filename(#mt{ filename = F }) -> F. diff(#mt{backend = naive, tree = T1}, #mt{backend = naive, tree = T2}) -> case T1#naive.root == T2#naive.root of true -> same; - false -> different %% TODO: implement diff + false -> naive_diff(T1, T2) end; diff(#mt{backend = merklet, tree = T1}, #mt{backend = merklet, tree = T2}) -> case merklet:diff(T1, T2) of @@ -201,3 +201,21 @@ build_level_1(Size, [{Pos, Len, Hash}|T], Multiple, [ Ctx | Rest ]) build_level_1(Size, [{Pos, Len, Hash}|T], Multiple, [ Ctx | Rest ]) when ( Pos + Len ) =< ( Size * Multiple ) -> build_level_1(Size, T, Multiple, [ crypto:hash_update(Ctx, Hash) | Rest ]). + +naive_diff(#naive{lvl1 = L1}, #naive{lvl1=L2, chunk_size=CS2}) -> + lager:debug("naive diff: Our lvl1: ~p~n", [L1]), + lager:debug("naive diff: Their chunk size: ~p, lvl1: ~p~n", [CS2, L2]), + + Set1 = gb_sets:from_list(lists:zip(lists:seq(1, length(L1), L1))), + Set2 = gb_sets:from_list(lists:zip(lists:seq(1, length(L2), L2))), + + %% The byte ranges in list 2 that do not match in list 1 + %% + %% We have to decide what to do now - should we filter the + %% leaf nodes using these ranges and find specific divergence + %% between Tree1 and Tree2? + %% + %% Or should we do something else? + [ {(X-1)*CS2, CS2, SHA} || {X, SHA} <- gb_sets:to_list(gb_sets:subtract(Set1, Set2)) ]. + + -- 2.45.2 From 3c5a9e6f53f1f5f3f015e8f125b24c5ccc2c3735 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Mon, 2 Nov 2015 00:12:58 -0600 Subject: [PATCH 19/44] Torture tests for merkle tree 1,000,000 entries - timings and size --- src/machi_merkle_tree.erl | 24 +++++------------------ test/machi_merkle_tree_test.erl | 34 +++++++++++++++++++++++++++++++++ 2 files changed, 39 insertions(+), 19 deletions(-) diff --git a/src/machi_merkle_tree.erl b/src/machi_merkle_tree.erl index c4a4516..0b681ca 100644 --- a/src/machi_merkle_tree.erl +++ b/src/machi_merkle_tree.erl @@ -41,7 +41,11 @@ -module(machi_merkle_tree). -include("machi.hrl"). +-include("machi_merkle_tree.hrl"). +-ifdef(TEST). +-compile(export_all). +-else. -export([ open/2, open/3, @@ -49,25 +53,7 @@ filename/1, diff/2 ]). - - --record(naive, { - chunk_size = 1048576 :: pos_integer(), %% default 1 MB - recalc = true :: boolean(), - root :: 'undefined' | binary(), - lvl1 = [] :: [ binary() ], - lvl2 = [] :: [ binary() ], - lvl3 = [] :: [ binary() ], - leaves = [] :: [ { Offset :: pos_integer(), - Size :: pos_integer(), - Csum :: binary()} ] - }). - --record(mt, { - filename :: string(), - tree :: #naive{}|merklet:tree(), - backend = 'merklet' :: 'naive'|'merklet' - }). +-endif. -define(TRIMMED, <<1>>). -define(UNWRITTEN, <<0>>). diff --git a/test/machi_merkle_tree_test.erl b/test/machi_merkle_tree_test.erl index 382c262..d4020d0 100644 --- a/test/machi_merkle_tree_test.erl +++ b/test/machi_merkle_tree_test.erl @@ -21,6 +21,8 @@ -module(machi_merkle_tree_test). -compile([export_all]). +-include("machi_merkle_tree.hrl"). + -include_lib("eunit/include/eunit.hrl"). -include_lib("kernel/include/file.hrl"). @@ -127,3 +129,35 @@ run_test(C) -> ?assertEqual(same, machi_merkle_tree:diff(N, N)), ?assertEqual(same, machi_merkle_tree:diff(M, M)), {Osize, {MTime, MSize}, {NTime, NSize}}. + +torture_test(C) -> + Results = [ run_torture_test() || _ <- lists:seq(1, C) ], + {ok, F} = file:open("torture_results.txt", [raw, write]), + lists:foreach(fun({MSize, MTime, NSize, NTime}) -> + file:write(F, io_lib:format("~p\t~p\t~p\t~p\n", + [MSize, MTime, NSize, NTime])) + end, Results), + ok = file:close(F). + +run_torture_test() -> + {MTime, M} = timer:tc(fun() -> merklet_torture() end), + {NTime, N} = timer:tc(fun() -> naive_torture() end), + + MSize = byte_size(term_to_binary(M)), + NSize = byte_size(term_to_binary(N)), + + {MSize, MTime, NSize, NTime}. + +merklet_torture() -> + lists:foldl( + fun({O, S, Sha}, Acc) -> + merklet:insert({<>, Sha}, Acc) + end, undefined, torture_generator()). + +naive_torture() -> + N = lists:foldl(fun(T, Acc) -> machi_merkle_tree:update_acc(T, Acc) end, [], torture_generator()), + T = #naive{ leaves = lists:reverse(N), chunk_size = 10010, recalc = true }, + machi_merkle_tree:build_tree(T). + +torture_generator() -> + [ {O, 1, crypto:hash(sha, term_to_binary(now()))} || O <- lists:seq(1024, 1000000) ]. -- 2.45.2 From 3122f2cf54c97ffada8416dd063dd21be82705fd Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Mon, 2 Nov 2015 17:43:38 +0900 Subject: [PATCH 20/44] Support OTP 18.1 --- .travis.yml | 1 + rebar.config | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 7bb3465..6c8a6c7 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,3 +4,4 @@ notifications: script: "priv/test-for-gh-pr.sh" otp_release: - 17.5 + - 18.1 diff --git a/rebar.config b/rebar.config index 2a77381..d6debc0 100644 --- a/rebar.config +++ b/rebar.config @@ -1,4 +1,4 @@ -{require_otp_vsn, "17"}. +{require_otp_vsn, "17|18"}. %%% {erl_opts, [warnings_as_errors, {parse_transform, lager_transform}, debug_info]}. {erl_opts, [{parse_transform, lager_transform}, debug_info]}. -- 2.45.2 From 059f591d3f894caa91ab333f36226f18dd32e259 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Mon, 2 Nov 2015 17:57:16 +0900 Subject: [PATCH 21/44] Exclude FLUs which can not send to cr client as well as receive from in terms of partition simulator --- src/machi_cr_client.erl | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/machi_cr_client.erl b/src/machi_cr_client.erl index c4274b2..0d74f61 100644 --- a/src/machi_cr_client.erl +++ b/src/machi_cr_client.erl @@ -937,7 +937,10 @@ update_proj2(Count, #state{bad_proj=BadProj, proxies_dict=ProxiesDict, true -> Members = proplists:get_value(simulator_members, Opts, []), {Partitions, _Islands} = machi_partition_simulator:get(Members), - [B || {A, B} <- Partitions, A =:= SimName]; + lists:filtermap(fun({A, B}) when A =:= SimName -> {true, B}; + ({A, B}) when B =:= SimName -> {true, A}; + (_) -> false + end, Partitions); false -> [] end, Proxies = lists:foldl(fun(Name, Dict) -> -- 2.45.2 From 30000d6602a350fdc3e1ff12e0450a63ee68c76f Mon Sep 17 00:00:00 2001 From: Scott Lystig Fritchie Date: Tue, 3 Nov 2015 00:22:43 +0900 Subject: [PATCH 22/44] Add doc/machi_chain_manager1_converge_demo.md --- README.md | 7 +- doc/machi_chain_manager1_converge_demo.md | 171 ++++++++++++++++++++ test/machi_chain_manager1_converge_demo.erl | 108 ++++++------- 3 files changed, 228 insertions(+), 58 deletions(-) create mode 100644 doc/machi_chain_manager1_converge_demo.md diff --git a/README.md b/README.md index 8cd8354..d802080 100644 --- a/README.md +++ b/README.md @@ -28,13 +28,16 @@ doc](./doc/high-level-machi.pdf) for further references.) ## Status: mid-October 2015: work is underway -* The chain manager is ready for both eventual consistency use ("AP - mode") and strong constency use ("CP mode"). Both modes use a new +* The chain manager is ready for both eventual consistency use ("available + mode") and strong consistency use ("consistent mode"). Both modes use a new consensus technique, Humming Consensus. * Scott will be [speaking about Humming Consensus](http://ricon.io/agenda/#managing-chain-replication-metadata-with-humming-consensus) at the [Ricon 2015 conference] (http://ricon.io) in San Francisco, CA, USA on Thursday, November 5th, 2015. + * If you would like to run the network partition simulator + mentioned in that Ricon presentation, please see the + [partition simulator convergence test doc.](./doc/machi_chain_manager1_converge_demo.md) * Implementation of the file repair process for strong consistency is still in progress. diff --git a/doc/machi_chain_manager1_converge_demo.md b/doc/machi_chain_manager1_converge_demo.md new file mode 100644 index 0000000..59ea13b --- /dev/null +++ b/doc/machi_chain_manager1_converge_demo.md @@ -0,0 +1,171 @@ + +# Using the network partition simulator and convergence demo test code + +## A complete example of all input and output + +If you don't have an Erlang/OTP 17 runtime environment available, +please see this file for full input and output of a strong consistency +length=3 chain test: +https://gist.github.com/slfritchie/8352efc88cc18e62c72c +This file contains all commands input and all simulator output from a +sample run of the simulator. + +To help interpret the output of the test, please skip ahead to the +"The test output is very verbose" section. + +## Prerequisites + +1. You'll need the `git` source management +2. You'll need the Erlang/OTP 17 runtime environment. Please don't + use earlier or later versions until we have a chance to fix the + compilation warnings that versions R16B and 18 will trigger. + +All of the commands that should be run at your login shell (e.g. Bash, +c-shell) can be cut-and-pasted from this document directly to your +login shell prompt. + +## Clone and compile the code + +Clone the Machi source repo and compile the source and test code. Run +the following commands at your login shell: + + cd /tmp + git clone https://github.com/basho/machi.git + cd machi + git checkout master + make + +Then run the unit test suite. This may take up to two minutes or so +to finish. Most of the tests will be silent; please be patient until +the tests finish. + + make test + +## Run an interactive Erlang CLI shell + +Run the following command at your login shell: + + erl -pz .eunit ebin deps/*/ebin + +If you are using Erlang/OTP version 17, you should see some CLI output +that looks like this: + + Erlang/OTP 17 [erts-6.4] [source] [64-bit] [smp:8:8] [async-threads:10] [hipe] [kernel-poll:false] [dtrace] + + Eshell V6.4 (abort with ^G) + 1> + +## The test output is very verbose ... what are the important parts? + +The output of the Erlang command +`machi_chain_manager1_converge_demo:help()` will display the following +guide to the output of the tests. + + A visualization of the convergence behavior of the chain self-management + algorithm for Machi. + + 1. Set up some server and chain manager pairs. + 2. Create a number of different network partition scenarios, where + (simulated) partitions may be symmetric or asymmetric. Then stop changing + the partitions and keep the simulated network stable (and perhaps broken). + 3. Run a number of iterations of the algorithm in parallel by poking each + of the manager processes on a random'ish basis. + 4. Afterward, fetch the chain transition changes made by each FLU and + verify that no transition was unsafe. + + During the iteration periods, the following is a cheatsheet for the output. + See the internal source for interpreting the rest of the output. + + 'SET partitions = ' + + A pair-wise list of actors which cannot send messages. The + list is uni-directional. If there are three servers (a,b,c), + and if the partitions list is '[{a,b},{b,c}]' then all + messages from a->b and b->c will be dropped, but any other + sender->recipient messages will be delivered successfully. + + 'x uses:' + + The FLU x has made an internal state transition and is using + this epoch's projection as operating chain configuration. The + rest of the line is a summary of the projection. + + 'CONFIRM epoch {N}' + + This message confirms that all of the servers listed in the + UPI and repairing lists of the projection at epoch {N} have + agreed to use this projection because they all have written + this projection to their respective private projection stores. + The chain is now usable by/available to all clients. + + 'Sweet, private projections are stable' + + This report announces that this iteration of the test cycle + has passed successfully. The report that follows briefly + summarizes the latest private projection used by each + participating server. For example, when in strong consistency + mode with 'a' as a witness and 'b' and 'c' as real servers: + + %% Legend: + %% server name, epoch ID, UPI list, repairing list, down list, ... + %% ... witness list, 'false' (a constant value) + + [{a,{{1116,<<23,143,246,55>>},[a,b],[],[c],[a],false}}, + {b,{{1116,<<23,143,246,55>>},[a,b],[],[c],[a],false}}] + + Both servers 'a' and 'b' agree on epoch 1116 with epoch ID + {1116,<<23,143,246,55>>} where UPI=[a,b], repairing=[], + down=[c], and witnesses=[a]. + + Server 'c' is not shown because 'c' has wedged itself OOS (out + of service) by configuring a chain length of zero. + + If no servers are listed in the report (i.e. only '[]' is + displayed), then all servers have wedged themselves OOS, and + the chain is unavailable. + + 'DoIt,' + + This marks a group of tick events which trigger the manager + processes to evaluate their environment and perhaps make a + state transition. + + A long chain of 'DoIt,DoIt,DoIt,' means that the chain state has + (probably) settled to a stable configuration, which is the goal of the + algorithm. + + Press control-c to interrupt the test....". + +## Run a test in eventual consistency mode + +Run the following command at the Erlang CLI prompt: + + machi_chain_manager1_converge_demo:t(3, [{private_write_verbose,true}]). + +The first argument, `3`, is the number of servers to participate in +the chain. Chain lengths as long as 7 or 9 can be used, but they may +suffer from longer periods of churn/instability before all chain +managers reach agreement via humming consensus. + +## Run a test in strong consistency mode (with witnesses): + +*NOTE:* Due to a bug in the test code, please do not try to run the + convergence test in strong consistency mode and also without the + correct minority number of witness servers! If in doubt, please run + the commands shown below exactly. + +Run the following command at the Erlang CLI prompt: + + machi_chain_manager1_converge_demo:t(3, [{private_write_verbose,true}, {consistency_mode, cp_mode}, {witnesses, [a]}]). + +The first argument, `3`, is the number of servers to participate in +the chain. Chain lengths as long as 7 or 9 can be used, but they may +suffer from longer periods of churn/instability before all chain +managers reach agreement via humming consensus. + +Due to the bug mentioned above, please use the following +commands when running with chain lengths of 5 or 7, respectively. + + machi_chain_manager1_converge_demo:t(5, [{private_write_verbose,true}, {consistency_mode, cp_mode}, {witnesses, [a,b]}]). + machi_chain_manager1_converge_demo:t(7, [{private_write_verbose,true}, {consistency_mode, cp_mode}, {witnesses, [a,b,c]}]). + diff --git a/test/machi_chain_manager1_converge_demo.erl b/test/machi_chain_manager1_converge_demo.erl index 782e7be..9303701 100644 --- a/test/machi_chain_manager1_converge_demo.erl +++ b/test/machi_chain_manager1_converge_demo.erl @@ -45,14 +45,18 @@ -include_lib("eunit/include/eunit.hrl"). +help() -> + io:format("~s\n", [short_doc()]). + short_doc() -> " A visualization of the convergence behavior of the chain self-management algorithm for Machi. - 1. Set up 4 FLUs and chain manager pairs. + + 1. Set up some server and chain manager pairs. 2. Create a number of different network partition scenarios, where - (simulated) partitions may be symmetric or asymmetric. Then halt changing - the partitions and keep the simulated network stable and broken. + (simulated) partitions may be symmetric or asymmetric. Then stop changing + the partitions and keep the simulated network stable (and perhaps broken). 3. Run a number of iterations of the algorithm in parallel by poking each of the manager processes on a random'ish basis. 4. Afterward, fetch the chain transition changes made by each FLU and @@ -61,73 +65,65 @@ algorithm for Machi. During the iteration periods, the following is a cheatsheet for the output. See the internal source for interpreting the rest of the output. - 'Let loose the dogs of war!' Network instability - 'SET partitions = ' Network stability (but broken) - 'x uses:' The FLU x has made an internal state transition. The rest of - the line is a dump of internal state. - '{t}' This is a tick event which triggers one of the manager processes - to evaluate its environment and perhaps make a state transition. + 'SET partitions = ' -A long chain of '{t}{t}{t}{t}' means that the chain state has settled -to a stable configuration, which is the goal of the algorithm. -Press control-c to interrupt....". + A pair-wise list of actors which cannot send messages. The + list is uni-directional. If there are three servers (a,b,c), + and if the partitions list is '[{a,b},{b,c}]' then all + messages from a->b and b->c will be dropped, but any other + sender->recipient messages will be delivered successfully. -long_doc() -> - " -'Let loose the dogs of war!' + 'x uses:' - The simulated network is very unstable for a few seconds. + The FLU x has made an internal state transition and is using + this epoch's projection as operating chain configuration. The + rest of the line is a summary of the projection. -'x uses' + 'CONFIRM epoch {N}' - After a single iteration, server x has determined that the chain - should be defined by the upi, repair, and down list in this record. - If all participants reach the same conclusion at the same epoch - number (and checksum, see next item below), then the chain is - stable, fully configured, and can provide full service. + This message confirms that all of the servers listed in the + UPI and repairing lists of the projection at epoch {N} have + agreed to use this projection because they all have written + this projection to their respective private projection stores. + The chain is now usable by/available to all clients. -'epoch,E' + 'Sweet, private projections are stable' - The epoch number for this decision is E. The checksum of the full - record is not shown. For purposes of the protocol, a server will - 'wedge' itself and refuse service (until a new config is chosen) - whenever: a). it sees a bigger epoch number mentioned somewhere, or - b). it sees the same epoch number but a different checksum. In case - of b), there was a network partition that has healed, and both sides - had chosen to operate with an identical epoch number but different - chain configs. + This report announces that this iteration of the test cycle + has passed successfully. The report that follows briefly + summarizes the latest private projection used by each + participating server. For example, when in strong consistency + mode with 'a' as a witness and 'b' and 'c' as real servers: -'upi', 'repair', and 'down' + %% Legend: + %% server name, epoch ID, UPI list, repairing list, down list, ... + %% ... witness list, 'false' (a constant value) - Members in the chain that are fully in sync and thus preserving the - Update Propagation Invariant, up but under repair (simulated), and - down, respectively. + [{a,{{1116,<<23,143,246,55>>},[a,b],[],[c],[a],false}}, + {b,{{1116,<<23,143,246,55>>},[a,b],[],[c],[a],false}}] -'ps,[some list]' + Both servers 'a' and 'b' agree on epoch 1116 with epoch ID + {1116,<<23,143,246,55>>} where UPI=[a,b], repairing=[], + down=[c], and witnesses=[a]. - The list of asymmetric network partitions. {a,b} means that a - cannot send to b, but b can send to a. + Server 'c' is not shown because 'c' has wedged itself OOS (out + of service) by configuring a chain length of zero. - This partition list is recorded for debugging purposes but is *not* - used by the algorithm. The algorithm only 'feels' its effects via - simulated timeout whenever there's a partition in one of the - messaging directions. + If no servers are listed in the report (i.e. only '[]' is + displayed), then all servers have wedged themselves OOS, and + the chain is unavailable. -'nodes_up,[list]' + 'DoIt,' - The best guess right now of which ndoes are up, relative to the - author node, specified by '{author,X}' + This marks a group of tick events which trigger the manager + processes to evaluate their environment and perhaps make a + state transition. -'SET partitions = [some list]' +A long chain of 'DoIt,DoIt,DoIt,' means that the chain state has +(probably) settled to a stable configuration, which is the goal of the +algorithm. - All subsequent iterations should have a stable list of partitions, - i.e. the 'ps' list described should be stable. - -'{FLAP: x flaps n}!' - - Server x has detected that it's flapping/oscillating after iteration - n of a naive/1st draft detection algorithm. -". +Press control-c to interrupt the test....". %% ' silly Emacs syntax highlighting.... @@ -295,7 +291,7 @@ convergence_demo_testfun(NumFLUs, MgrOpts0) -> private_projections_are_stable(Namez, DoIt) end, false, lists:seq(0, MaxIters)), io:format(user, "\n~s Sweet, private projections are stable\n", [machi_util:pretty_time()]), - io:format(user, "\t~P\n", [get(stable), 14]), + io:format(user, "\t~P\n", [get(stable), 24]), io:format(user, "Rolling sanity check ... ", []), PrivProjs = [{Name, begin {ok, Ps8} = ?FLU_PC:get_all_projections( @@ -719,7 +715,7 @@ private_projections_are_stable(Namez, PollFunc) -> true end, - io:format(user, "\nPriv1 ~p\nPriv2 ~p\n1==2 ~w ap_disjoint ~w u_all_peers ~w cp_mode_agree ~w\n", [lists:sort(Private1), lists:sort(Private2), Private1 == Private2, AP_mode_disjoint_test_p, Unanimous_with_all_peers_p, CP_mode_agree_test_p]), + %% io:format(user, "\nPriv1 ~p\nPriv2 ~p\n1==2 ~w ap_disjoint ~w u_all_peers ~w cp_mode_agree ~w\n", [lists:sort(Private1), lists:sort(Private2), Private1 == Private2, AP_mode_disjoint_test_p, Unanimous_with_all_peers_p, CP_mode_agree_test_p]), Private1 == Private2 andalso AP_mode_disjoint_test_p andalso ( -- 2.45.2 From 557525af051cd681e8e03f1915156a209fb9493a Mon Sep 17 00:00:00 2001 From: Scott Lystig Fritchie Date: Tue, 3 Nov 2015 00:34:51 +0900 Subject: [PATCH 23/44] Clarify chain length --- doc/machi_chain_manager1_converge_demo.md | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/doc/machi_chain_manager1_converge_demo.md b/doc/machi_chain_manager1_converge_demo.md index 59ea13b..2844bfa 100644 --- a/doc/machi_chain_manager1_converge_demo.md +++ b/doc/machi_chain_manager1_converge_demo.md @@ -143,9 +143,23 @@ Run the following command at the Erlang CLI prompt: machi_chain_manager1_converge_demo:t(3, [{private_write_verbose,true}]). The first argument, `3`, is the number of servers to participate in -the chain. Chain lengths as long as 7 or 9 can be used, but they may -suffer from longer periods of churn/instability before all chain -managers reach agreement via humming consensus. +the chain. Please note: + +* Chain lengths as short as 1 or 2 are valid, but the results are a + bit boring. +* Chain lengths as long as 7 or 9 can be used, but they may + suffer from longer periods of churn/instability before all chain + managers reach agreement via humming consensus. (It is future work + to shorten the worst of the unstable churn latencies.) +* In eventual consistency mode, chain lengths may be even numbers, + e.g. 2, 4, or 6. +* The simulator will choose partition events from the permutations of + all 1, 2, and 3 node partition pairs. The total runtime will + increase *dramatically* with chain length. + * Chain length 2: about 3 partition cases + * Chain length 3: about 35 partition cases + * Chain length 4: about 230 partition cases + * Chain length 5: about 1100 partition cases ## Run a test in strong consistency mode (with witnesses): -- 2.45.2 From b956f9e1f04e2e98623a9e0ffcc67d5b45c4ec4a Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Wed, 4 Nov 2015 11:43:00 +0900 Subject: [PATCH 24/44] Fix dialyzer issue --- Makefile | 2 +- dialyzer.ignore-warnings | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/Makefile b/Makefile index a93a383..4dbde70 100644 --- a/Makefile +++ b/Makefile @@ -53,7 +53,7 @@ relclean: stage : rel $(foreach dep,$(wildcard deps/*), rm -rf rel/$(REPO)/lib/$(shell basename $(dep))* && ln -sf $(abspath $(dep)) rel/$(REPO)/lib;) -DIALYZER_APPS = kernel stdlib sasl erts ssl compiler eunit crypto +DIALYZER_APPS = kernel stdlib sasl erts ssl compiler eunit crypto public_key syntax_tools PLT = $(HOME)/.machi_dialyzer_plt include tools.mk diff --git a/dialyzer.ignore-warnings b/dialyzer.ignore-warnings index db73995..a5ee352 100644 --- a/dialyzer.ignore-warnings +++ b/dialyzer.ignore-warnings @@ -3,8 +3,10 @@ machi_pb.erl:0: ################################################## ######## Specific types ##################### ################################################## +Unknown types: basho_bench_config:get/2 machi_partition_simulator:get/1 + hamcrest:matchspec/0 ################################################## ######## Specific messages ##################### ################################################## -- 2.45.2 From 3b087c0388807b3adfff8ef644e2801719e8a91b Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Fri, 30 Oct 2015 18:20:48 +0900 Subject: [PATCH 25/44] Add eqc trim tests to machi_file_proxy * Add description on high client APIs * Add notes to rethink high client specification --- src/machi_csum_table.erl | 1 - src/machi_file_proxy.erl | 32 +++-- src/machi_pb_high_client.erl | 60 +++++++-- test/machi_csum_table_test.erl | 2 - test/machi_file_proxy_eqc.erl | 234 ++++++++++++++++++++++++++++----- test/machi_file_proxy_test.erl | 11 +- 6 files changed, 282 insertions(+), 58 deletions(-) diff --git a/src/machi_csum_table.erl b/src/machi_csum_table.erl index 9585429..6e5e2b8 100644 --- a/src/machi_csum_table.erl +++ b/src/machi_csum_table.erl @@ -125,7 +125,6 @@ write(#machi_csum_table{fd=Fd, table=T} = CsumT, true = ets:insert(T, {Offset, Size, CSum}), ok; Error -> - io:format(user, "boob *********************", []), Error end. diff --git a/src/machi_file_proxy.erl b/src/machi_file_proxy.erl index a7bad0f..0ff7b83 100644 --- a/src/machi_file_proxy.erl +++ b/src/machi_file_proxy.erl @@ -131,9 +131,8 @@ sync(_Pid, Type) -> {error, bad_arg}. % @doc Read file at offset for length. This returns a sequence of all -% chunks that overlaps with requested offset and length. Note that -% borders are not aligned, not to mess up repair at cr_client with -% checksums. They should be cut at cr_client. +% written and trimmed (optional) bytes that overlaps with requested +% offset and length. Borders are not aligned. -spec read(Pid :: pid(), Offset :: non_neg_integer(), Length :: non_neg_integer()) -> @@ -223,6 +222,8 @@ init({FluName, Filename, DataDir}) -> UnwrittenBytes = machi_csum_table:calc_unwritten_bytes(CsumTable), {Eof, infinity} = lists:last(UnwrittenBytes), {ok, FHd} = file:open(DPath, [read, write, binary, raw]), + %% Reserve for EC and stuff, to prevent eof when read + ok = file:pwrite(FHd, 0, binary:copy(<<"so what?">>, ?MINIMUM_OFFSET div 8)), Tref = schedule_tick(), St = #state{ fluname = FluName, @@ -385,7 +386,10 @@ handle_call({trim, Offset, Size, _TriggerGC}, _From, case machi_csum_table:trim(CsumTable, Offset, Size, LUpdate, RUpdate) of ok -> - NewState = State#state{ops=Ops+1, trims={T+1, Err}}, + {NewEof, infinity} = lists:last(machi_csum_table:calc_unwritten_bytes(CsumTable)), + NewState = State#state{ops=Ops+1, + trims={T+1, Err}, + eof_position=NewEof}, maybe_gc(ok, NewState); Error -> {reply, Error, State#state{ops=Ops+1, trims={T, Err+1}}} @@ -573,6 +577,15 @@ check_or_make_tagged_csum(Tag, InCsum, Data) when Tag == ?CSUM_TAG_CLIENT_SHA; false -> {error, {bad_csum, Csum}} end; +check_or_make_tagged_csum(?CSUM_TAG_SERVER_REGEN_SHA, + InCsum, Data) -> + Csum = machi_util:checksum_chunk(Data), + case Csum =:= InCsum of + true -> + machi_util:make_tagged_csum(server_regen_sha, Csum); + false -> + {error, {bad_csum, Csum}} + end; check_or_make_tagged_csum(OtherTag, _ClientCsum, _Data) -> lager:warning("Unknown checksum tag ~p", [OtherTag]), {error, bad_checksum}. @@ -669,6 +682,7 @@ read_all_ranges(FHd, Filename, [{Offset, Size, TaggedCsum}|T], ReadChunks, Trimm % caller as `ok' handle_write(FHd, CsumTable, Filename, TaggedCsum, Offset, Data) -> Size = iolist_size(Data), + case machi_csum_table:find(CsumTable, Offset, Size) of [] -> %% Nothing should be there try @@ -762,7 +776,7 @@ slice_both_side([], _, _) -> []; slice_both_side([{F, Offset, Chunk, _Csum}|L], LeftPos, RightPos) when Offset < LeftPos andalso LeftPos < RightPos -> - TrashLen = 8 * (LeftPos - Offset), + TrashLen = (LeftPos - Offset), <<_:TrashLen/binary, NewChunk/binary>> = Chunk, NewChecksum = machi_util:make_tagged_csum(?CSUM_TAG_SERVER_REGEN_SHA_ATOM, Chunk), NewH = {F, LeftPos, NewChunk, NewChecksum}, @@ -787,7 +801,7 @@ maybe_regenerate_checksum(_, {_, _, trimmed} = Change) -> maybe_regenerate_checksum(FHd, {Offset, Size, _Csum}) -> case file:pread(FHd, Offset, Size) of eof -> - error(eof); + error({eof, Offset, Size}); {ok, Bytes} when byte_size(Bytes) =:= Size -> TaggedCsum = machi_util:make_tagged_csum(server_regen_sha, @@ -797,9 +811,9 @@ maybe_regenerate_checksum(FHd, {Offset, Size, _Csum}) -> throw(Error) end. -%% GC: make sure unwritte bytes = [{Eof, infinity}] and Eof is > max file size -%% walk through the checksum table and make sure all chunks trimmed -%% Then unlink the file +%% GC: make sure unwritten bytes = [{Eof, infinity}] and Eof is > max +%% file size walk through the checksum table and make sure all chunks +%% trimmed Then unlink the file -spec maybe_gc(term(), #state{}) -> {reply, term(), #state{}} | {stop, normal, term(), #state{}}. maybe_gc(Reply, S = #state{eof_position = Eof, diff --git a/src/machi_pb_high_client.erl b/src/machi_pb_high_client.erl index ec2dfc6..91e467c 100644 --- a/src/machi_pb_high_client.erl +++ b/src/machi_pb_high_client.erl @@ -58,68 +58,110 @@ count=0 :: non_neg_integer() }). +%% @doc official error types that is specific in Machi +-type error_atoms() :: bad_arg | wedged | bad_checksum | + partition | not_written | written | + trimmed | no_such_file | partial_read | + bad_epoch | inet:posix(). + +%% @doc Creates a client process +-spec start_link(p_srvr_dict()) -> {ok, pid()} | {error, error_atoms()}. start_link(P_srvr_list) -> gen_server:start_link(?MODULE, [P_srvr_list], []). +%% @doc Stops a client process. +-spec quit(pid()) -> ok. quit(PidSpec) -> gen_server:call(PidSpec, quit, infinity). connected_p(PidSpec) -> gen_server:call(PidSpec, connected_p, infinity). +-spec echo(pid(), string()) -> {ok, string()} | {error, error_atoms()}. echo(PidSpec, String) -> echo(PidSpec, String, ?DEFAULT_TIMEOUT). +-spec echo(pid(), string(), non_neg_integer()) -> {ok, string()} | {error, error_atoms()}. echo(PidSpec, String, Timeout) -> send_sync(PidSpec, {echo, String}, Timeout). %% TODO: auth() is not implemented. Auth requires SSL, and this client %% doesn't support SSL yet. This is just a placeholder and reminder. +-spec auth(pid(), string(), string()) -> ok | {error, error_atoms()}. auth(PidSpec, User, Pass) -> auth(PidSpec, User, Pass, ?DEFAULT_TIMEOUT). +-spec auth(pid(), string(), string(), non_neg_integer()) -> ok | {error, error_atoms()}. auth(PidSpec, User, Pass, Timeout) -> send_sync(PidSpec, {auth, User, Pass}, Timeout). +-spec append_chunk(pid(), PlacementKey::binary(), Prefix::binary(), Chunk::binary(), + CSum::binary(), ChunkExtra::non_neg_integer()) -> + {ok, Filename::string(), Offset::machi_dt:file_offset()} | + {error, error_atoms()}. append_chunk(PidSpec, PlacementKey, Prefix, Chunk, CSum, ChunkExtra) -> append_chunk(PidSpec, PlacementKey, Prefix, Chunk, CSum, ChunkExtra, ?DEFAULT_TIMEOUT). +-spec append_chunk(pid(), PlacementKey::binary(), Prefix::binary(), + Chunk::binary(), CSum::binary(), + ChunkExtra::non_neg_integer(), + Timeout::non_neg_integer()) -> + {ok, Filename::string(), Offset::machi_dt:file_offset()} | + {error, error_atoms()}. append_chunk(PidSpec, PlacementKey, Prefix, Chunk, CSum, ChunkExtra, Timeout) -> send_sync(PidSpec, {append_chunk, PlacementKey, Prefix, Chunk, CSum, ChunkExtra}, Timeout). +-spec write_chunk(pid(), File::string(), machi_dt:file_offset(), + Chunk::binary(), CSum::binary()) -> + ok | {error, error_atoms()}. write_chunk(PidSpec, File, Offset, Chunk, CSum) -> write_chunk(PidSpec, File, Offset, Chunk, CSum, ?DEFAULT_TIMEOUT). +-spec write_chunk(pid(), File::string(), machi_dt:file_offset(), + Chunk::binary(), CSum::binary(), Timeout::non_neg_integer()) -> + ok | {error, error_atoms()}. write_chunk(PidSpec, File, Offset, Chunk, CSum, Timeout) -> send_sync(PidSpec, {write_chunk, File, Offset, Chunk, CSum}, Timeout). %% @doc Tries to read a chunk of a specified file. It returns `{ok, %% {Chunks, TrimmedChunks}}' for live file while it returns `{error, %% trimmed}' if all bytes of the file was trimmed. --spec read_chunk(pid(), string(), pos_integer(), pos_integer(), +-spec read_chunk(pid(), File::string(), machi_dt:file_offset(), machi_dt:chunk_size(), [{flag_no_checksum | flag_no_chunk | needs_trimmed, boolean()}]) -> - {ok, {list(), list()}} | {error, term()}. + {ok, {Chunks::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size(), binary()}], + Trimmed::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size()}]}} | + {error, error_atoms()}. read_chunk(PidSpec, File, Offset, Size, Options) -> read_chunk(PidSpec, File, Offset, Size, Options, ?DEFAULT_TIMEOUT). --spec read_chunk(pid(), string(), pos_integer(), pos_integer(), - [{no_checksum | no_chunk | needs_trimmed, boolean()}], - pos_integer()) -> - {ok, {list(), list()}} | {error, term()}. +-spec read_chunk(pid(), File::string(), machi_dt:file_offset(), machi_dt:chunk_size(), + [{flag_no_checksum | flag_no_chunk | needs_trimmed, boolean()}], + Timeout::non_neg_integer()) -> + {ok, {Chunks::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size(), binary()}], + Trimmed::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size()}]}} | + {error, error_atoms()}. read_chunk(PidSpec, File, Offset, Size, Options, Timeout) -> send_sync(PidSpec, {read_chunk, File, Offset, Size, Options}, Timeout). -%% @doc Trims arbitrary binary range of any file. TODO: Add option -%% specifying whether to trigger GC. +%% @doc Trims arbitrary binary range of any file. If a specified range +%% has any byte trimmed, it fails and returns `{error, trimmed}`. +%% Otherwise it trims all bytes in that range. If there are +%% overlapping chunks with client-specified checksum, they will cut +%% off and checksum are re-calculated in server side. TODO: Add +%% option specifying whether to trigger GC. -spec trim_chunk(pid(), string(), non_neg_integer(), machi_dt:chunk_size()) -> - ok | {error, term()}. + ok | {error, error_atoms()}. trim_chunk(PidSpec, File, Offset, Size) -> trim_chunk(PidSpec, File, Offset, Size, ?DEFAULT_TIMEOUT). trim_chunk(PidSpec, File, Offset, Size, Timeout) -> send_sync(PidSpec, {trim_chunk, File, Offset, Size}, Timeout). +%% @doc Returns a binary that has checksums and chunks encoded inside +%% (This is because encoding-decoding them are inefficient). TODO: +%% return a structured list of them. +-spec checksum_list(pid(), string()) -> {ok, binary()} | {error, error_atoms()}. checksum_list(PidSpec, File) -> checksum_list(PidSpec, File, ?DEFAULT_TIMEOUT). diff --git a/test/machi_csum_table_test.erl b/test/machi_csum_table_test.erl index 683d512..c168d45 100644 --- a/test/machi_csum_table_test.erl +++ b/test/machi_csum_table_test.erl @@ -54,7 +54,6 @@ smoke2_test() -> ok = machi_csum_table:close(MC), ok = machi_csum_table:delete(MC). - smoke3_test() -> Filename = "./temp-checksum-dumb-file-4", _ = file:delete(Filename), @@ -107,5 +106,4 @@ smoke3_test() -> ok = machi_csum_table:close(MC), ok = machi_csum_table:delete(MC). - %% TODO: add quickcheck test here diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index e2eb955..11978af 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -37,7 +37,7 @@ eqc_test_() -> {timeout, 60, {spawn, [ - {timeout, 30, ?_assertEqual(true, eqc:quickcheck(eqc:testing_time(15, ?QC_OUT(prop_ok()))))} + {timeout, 30, ?_assertEqual(true, eqc:quickcheck(eqc:testing_time(15, ?QC_OUT(prop_ok()))))} ] }}. @@ -103,10 +103,16 @@ get_written_interval(L) -> %% INITIALIZATION --record(state, {pid, prev_extra = 0, planned_writes=[], written=[]}). +-record(state, {pid, prev_extra = 0, + planned_writes=[], + planned_trims=[], + written=[], + trimmed=[]}). initial_state() -> #state{written=[{0,1024}]}. -initial_state(I) -> #state{written=[{0,1024}], planned_writes=I}. +initial_state(I, T) -> #state{written=[{0,1024}], + planned_writes=I, + planned_trims=T}. weight(_S, rewrite) -> 1; weight(_S, _) -> 2. @@ -130,6 +136,80 @@ check_writes(write, [{Pos, Sz}|_T], Off, L) when ( Off + L ) > Pos check_writes(Op, [_H|T], Off, L) -> check_writes(Op, T, Off, L). +get_overlaps(_Offset, _Len, [], Acc) -> lists:reverse(Acc); +get_overlaps(Offset, Len, [{Pos, Sz} = Ck|T], Acc0) +%% Overlap judgement differnt from the one in machi_csum_table +%% [a=Offset, b), [x=Pos, y) ... + when + %% (a-y) * (b-x) + %%(Offset - Pos - Sz) * (Offset + Len - Pos) < 0 -> + %% a x b y + (Offset =< Pos andalso Pos < Offset + Len andalso Offset + Len =< Pos + Sz) orelse + %% a x y b + (Offset =< Pos andalso Pos + Sz < Offset + Len) orelse + %% x a y b + (Pos < Offset andalso Offset < Pos + Sz andalso Pos + Sz =< Offset + Len) orelse + %% x a b y + (Pos < Offset + Len andalso Offset + Len < Pos + Sz) -> + get_overlaps(Offset, Len, T, [Ck|Acc0]); +get_overlaps(Offset, Len, [_Ck|T], Acc0) -> + %% ?debugVal({Offset, Len, _Ck}), + %% ?debugVal(Offset =< Pos andalso Pos < Offset + Len andalso Offset + Len =< Pos + Sz), + %% ?debugVal(Offset =< Pos andalso Pos + Sz < Offset + Len), + %% ?debugVal(Pos < Offset andalso Offset < Pos + Sz andalso Pos + Sz < Offset + Len), + %% ?debugVal(Pos < Offset + Len andalso Offset + Len < Pos + Sz), + get_overlaps(Offset, Len, T, Acc0). + +%% Inefficient but simple easy code to verify by eyes - returns all +%% bytes that fits in (Offset, Len) +chop(Offset, Len, List) -> + ChopLeft = fun({Pos, Sz}) when Pos < Offset andalso Offset =< Pos + Sz -> + {Offset, Sz + Pos - Offset}; + ({Pos, Sz}) when Offset =< Pos andalso Pos + Sz < Offset + Len -> + {Pos, Sz}; + ({Pos, _Sz}) when Offset =< Pos -> + {Pos, Offset + Len - Pos} + end, + ChopRight = fun({Pos, Sz}) when Offset + Len < Pos + Sz -> + {Pos, Offset + Len - Pos}; + ({Pos, Sz}) -> + {Pos, Sz} + end, + Filter0 = fun({_, 0}) -> false; + (Other) -> {true, Other} end, + lists:filtermap(fun(E) -> Filter0(ChopRight(ChopLeft(E))) end, + List). + +%% Returns all bytes that are at left side of the Offset +chopped_left(_Offset, []) -> undefined; +chopped_left(Offset, [{Pos,_Sz}|_]) when Pos < Offset -> + {Pos, Offset - Pos}; +chopped_left(_, _) -> + undefined. + +chopped_right(_Offset, []) -> undefined; +chopped_right(Offset, List) -> + {Pos, Sz} = lists:last(List), + if Offset < Pos + Sz -> + {Offset, Pos + Sz - Offset}; + true -> + undefined + end. + +cleanup_chunk(Offset, Length, ChunkList) -> + Overlaps = get_overlaps(Offset, Length, ChunkList, []), + NewCL0 = lists:foldl(fun lists:delete/2, + ChunkList, Overlaps), + NewCL1 = case chopped_left(Offset, Overlaps) of + undefined -> NewCL0; + LeftRemain -> [LeftRemain|NewCL0] + end, + NewCL2 = case chopped_right(Offset+Length, Overlaps) of + undefined -> NewCL1; + RightRemain -> [RightRemain|NewCL1] + end, + lists:sort(NewCL2). + is_error({error, _}) -> true; is_error({error, _, _}) -> true; is_error(Other) -> {expected_ERROR, Other}. @@ -144,9 +224,10 @@ is_ok(Other) -> {expected_OK, Other}. get_offset({ok, _Filename, Offset}) -> Offset; get_offset(_) -> error(badarg). -offset_valid(Offset, Extra, L) -> - {Pos, Sz} = lists:last(L), - Offset == Pos + Sz + Extra. +last_byte([]) -> 0; +last_byte(L0) -> + L1 = lists:map(fun({Pos, Sz}) -> Pos + Sz end, L0), + lists:last(lists:sort(L1)). -define(TESTDIR, "./eqc"). @@ -162,7 +243,7 @@ cleanup() -> %% start start_pre(S) -> - S#state.pid == undefined. + S#state.pid =:= undefined. start_command(S) -> {call, ?MODULE, start, [S]}. @@ -185,29 +266,39 @@ read_pre(S) -> read_args(S) -> [S#state.pid, offset(), len()]. -read_ok(S, Off, L) -> - case S#state.written of - [{0, 1024}] -> false; - W -> check_writes(read, W, Off, L) - end. - read_post(S, [_Pid, Off, L], Res) -> - case read_ok(S, Off, L) of - true -> is_ok(Res); - mostly_true -> is_ok(Res); - false -> is_error(Res) + Written = get_overlaps(Off, L, S#state.written, []), + Chopped = chop(Off, L, Written), + Trimmed = get_overlaps(Off, L, S#state.trimmed, []), + Eof = lists:max([Pos+Sz||{Pos,Sz}<-S#state.written]), + %% ?debugVal({Off, L}), + %% ?debugVal(S), + case Res of + {ok, {Written0, Trimmed0}} -> + Written1 = lists:map(fun({_, Pos, Chunk, _}) -> + {Pos, iolist_size(Chunk)} + end, Written0), + Trimmed1 = lists:map(fun({_, Pos, Sz}) -> {Pos, Sz} end, Trimmed0), + %% ?debugVal({Written, Chopped, Written1}), + %% ?debugVal({Trimmed, Trimmed1}), + %% ?assertEqual(Chopped, Written1), + %% ?assertEqual(Trimmed, Trimmed1), + Chopped =:= Written1 + andalso Trimmed =:= Trimmed1; + %% TODO: such response are ugly, rethink the SPEC + {error, not_written} when Eof < Off + L -> + true; + {error, not_written} when Chopped =:= [] andalso Trimmed =:= [] -> + true; + Other -> + ?debugVal(Other), + is_error(Res) end. read_next(S, _Res, _Args) -> S. read(Pid, Offset, Length) -> - case machi_file_proxy:read(Pid, Offset, Length) of - {ok, {Chunks, _}} -> - [{_, Offset, Data, Csum}] = Chunks, - {ok, Data, Csum}; - E -> - E - end. + machi_file_proxy:read(Pid, Offset, Length, [{needs_trimmed, true}]). %% write @@ -216,6 +307,7 @@ write_pre(S) -> %% do not allow writes with empty data write_pre(_S, [_Pid, _Extra, {<<>>, _Tag, _Csum}]) -> + ?assert(false), false; write_pre(_S, _Args) -> true. @@ -224,28 +316,35 @@ write_args(S) -> {Off, Len} = hd(S#state.planned_writes), [S#state.pid, Off, data_with_csum(Len)]. -write_ok(_S, [_Pid, Off, _Data]) when Off < 1024 -> false; write_ok(S, [_Pid, Off, {Bin, _Tag, _Csum}]) -> Size = iolist_size(Bin), %% Check writes checks if a byte range is *written* %% So writes are ok IFF they are NOT written, so %% we want not check_writes/3 to be true. - check_writes(write, S#state.written, Off, Size). + case {get_overlaps(Off, Size, S#state.written, []), + get_overlaps(Off, Size, S#state.trimmed, [])} of + {[], []} -> + true; + {[{Off, Size}], []} -> + true; + _Other -> + false + end. write_post(S, Args, Res) -> case write_ok(S, Args) of %% false means this range has NOT been written before, so %% it should succeed - false -> eq(Res, ok); + true -> eq(Res, ok); %% mostly true means we've written this range before BUT %% as a special case if we get a call to write the EXACT %% same data that's already on the disk, we return "ok" %% instead of {error, written}. - mostly_true -> probably_error(Res); + %% mostly_true -> probably_error(Res); %% If we get true, then we've already written this section %% or a portion of this range to disk and should return an %% error. - true -> is_error(Res) + false -> is_error(Res) end. write_next(S, Res, [_Pid, Offset, {Bin, _Tag, _Csum}]) -> @@ -266,6 +365,7 @@ write(Pid, Offset, {Bin, Tag, Csum}) -> %% append append_pre(S) -> + ?assert(undefined =/= S#state.written), S#state.pid /= undefined. %% do not allow appends with empty binary data @@ -286,7 +386,9 @@ append_next(S, Res, [_Pid, Extra, {Bin, _Tag, _Csum}]) -> case is_ok(Res) of true -> Offset = get_offset(Res), - true = offset_valid(Offset, S#state.prev_extra, S#state.written), + Expected = erlang:max(last_byte(S#state.written) + S#state.prev_extra, + last_byte(S#state.trimmed)), + ?assertEqual(Expected, Offset), S#state{prev_extra = Extra, written = lists:sort(S#state.written ++ [{Offset, iolist_size(Bin)}])}; _ -> S @@ -304,7 +406,7 @@ rewrite_pre(S) -> rewrite_args(S) -> ?LET({Off, Len}, get_written_interval(S#state.written), - [S#state.pid, Off, data_with_csum(Len)]). + [S#state.pid, Off, data_with_csum(Len)]). rewrite(Pid, Offset, {Bin, Tag, Csum}) -> Meta = [{client_csum_tag, Tag}, @@ -317,18 +419,80 @@ rewrite_post(_S, _Args, Res) -> rewrite_next(S, _Res, _Args) -> S#state{prev_extra = 0}. +%% trim + +trim_pre(S) -> + S#state.pid /= undefined. %% andalso S#state.planned_trims /= []. + +trim_args(S) -> + %% {Offset, Length} = hd(S#state.planned_trims), + %% [S#state.pid, Offset, Length]. + [S#state.pid, offset(), len()]. + +trim(Pid, Offset, Length) -> + machi_file_proxy:trim(Pid, Offset, Length, false). + +trim_post(_S, [_Pid, _Offset, _Length], ok) -> + true; +trim_post(_S, [_Pid, _Offset, _Length], _Res) -> + false. + +trim_next(S, Res, [_Pid, Offset, Length]) -> + case is_ok(Res) of + true -> + NewWritten = cleanup_chunk(Offset, Length, S#state.written), + Trimmed1 = cleanup_chunk(Offset, Length, S#state.trimmed), + NewTrimmed = lists:sort([{Offset, Length}|Trimmed1]), + S#state{trimmed=NewTrimmed, + written=NewWritten, + prev_extra=0}; + %% planned_trims=tl(S#state.planned_trims)}; + _Other -> + S + end. + %% Property prop_ok() -> cleanup(), - ?FORALL(I, shuffle_interval(), - ?FORALL(Cmds, parallel_commands(?MODULE, initial_state(I)), + ?FORALL({I, T}, + {shuffle_interval(), shuffle_interval()}, + ?FORALL(Cmds, parallel_commands(?MODULE, initial_state(I, T)), begin - {H, S, Res} = run_parallel_commands(?MODULE, Cmds), + {H, S, Res} = run_parallel_commands(?MODULE, Cmds), + %% case S#state.pid of + %% undefined -> noop; + %% Pid -> + %% machi_file_proxy:stop(Pid) + %% end, pretty_commands(?MODULE, Cmds, {H, S, Res}, - aggregate(command_names(Cmds), Res == ok)) + aggregate(command_names(Cmds), Res == ok)) end) ). +%% Test for tester functions +chopper_test_() -> + [?_assertEqual([{0, 1024}], + get_overlaps(1, 1, [{0, 1024}], [])), + ?_assertEqual([], + get_overlaps(10, 5, [{9, 1}, {15, 1}], [])), + ?_assertEqual([{9,2},{14,1}], + get_overlaps(10, 5, [{9, 2}, {14, 1}], [])), + ?_assertEqual([], chop(0, 0, [{0,2}])), + ?_assertEqual([{0, 1}], chop(0, 1, [{0,2}])), + ?_assertEqual([], chop(1, 0, [{0,2}])), + ?_assertEqual([{1, 1}], chop(1, 1, [{0,2}])), + ?_assertEqual([{1, 1}], chop(1, 2, [{0,2}])), + ?_assertEqual([], chop(2, 1, [{0,2}])), + ?_assertEqual([], chop(2, 2, [{0,2}])), + ?_assertEqual([{1, 1}], chop(1, 3, [{0,2}])), + ?_assertError(_, chop(3, 1, [{0,2}])), + ?_assertEqual([], chop(2, 3, [{0,2}])), + ?_assertEqual({0, 1}, chopped_left(1, [{0, 1024}])), + ?_assertEqual([{0, 1}, {2, 1022}], cleanup_chunk(1, 1, [{0, 1024}])), + ?_assertEqual([{2, 1022}], cleanup_chunk(0, 2, [{0, 1}, {2, 1022}])), + ?_assert(true) + ]. + -endif. % EQC -endif. % TEST diff --git a/test/machi_file_proxy_test.erl b/test/machi_file_proxy_test.erl index cbf2014..8c4b60b 100644 --- a/test/machi_file_proxy_test.erl +++ b/test/machi_file_proxy_test.erl @@ -83,9 +83,9 @@ machi_file_proxy_test_() -> ?_assertEqual({error, bad_arg}, machi_file_proxy:read(Pid, -1, -1)), ?_assertEqual({error, bad_arg}, machi_file_proxy:write(Pid, -1, <<"yo">>)), ?_assertEqual({error, bad_arg}, machi_file_proxy:append(Pid, [], -1, <<"krep">>)), - ?_assertEqual({error, not_written}, machi_file_proxy:read(Pid, 1, 1)), + ?_assertMatch({ok, {_, []}}, machi_file_proxy:read(Pid, 1, 1)), ?_assertEqual({error, not_written}, machi_file_proxy:read(Pid, 1024, 1)), - ?_assertEqual({error, not_written}, machi_file_proxy:read(Pid, 1, 1024)), + ?_assertMatch({ok, {_, []}}, machi_file_proxy:read(Pid, 1, 1024)), ?_assertEqual({error, not_written}, machi_file_proxy:read(Pid, 1024, ?HYOOGE)), ?_assertEqual({error, not_written}, machi_file_proxy:read(Pid, ?HYOOGE, 1)), ?_assertEqual({error, written}, machi_file_proxy:write(Pid, 1, random_binary(0, ?HYOOGE))), @@ -102,6 +102,10 @@ multiple_chunks_read_test_() -> clean_up_data_dir(?TESTDIR), {ok, Pid} = machi_file_proxy:start_link(fluname, "test", ?TESTDIR), [ + ?_assertEqual(ok, machi_file_proxy:trim(Pid, 0, 1, false)), + ?_assertMatch({ok, {[], [{"test", 0, 1}]}}, + machi_file_proxy:read(Pid, 0, 1, + [{needs_trimmed, true}])), ?_assertMatch({ok, "test", _}, machi_file_proxy:append(Pid, random_binary(0, 1024))), ?_assertEqual(ok, machi_file_proxy:write(Pid, 10000, <<"fail">>)), ?_assertEqual(ok, machi_file_proxy:write(Pid, 20000, <<"fail">>)), @@ -114,6 +118,9 @@ multiple_chunks_read_test_() -> {"test", 30000, <<"fail">>, _}, {"test", 530000, <<"fail">>, _}], []}}, machi_file_proxy:read(Pid, 1024, 530000)), + ?_assertMatch({ok, {[{"test", 1, _, _}], [{"test", 0, 1}]}}, + machi_file_proxy:read(Pid, 0, 1024, + [{needs_trimmed, true}])), ?_assertException(exit, {normal, _}, machi_file_proxy:stop(Pid)) ]. -- 2.45.2 From c1e54260347c5537373decffcb9eee9450d23719 Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Wed, 4 Nov 2015 16:08:09 +0900 Subject: [PATCH 26/44] Address PR comments --- src/machi_pb_high_client.erl | 34 +++++++++++++++---------------- test/machi_file_proxy_eqc.erl | 38 +++++++---------------------------- 2 files changed, 24 insertions(+), 48 deletions(-) diff --git a/src/machi_pb_high_client.erl b/src/machi_pb_high_client.erl index 91e467c..ef1e740 100644 --- a/src/machi_pb_high_client.erl +++ b/src/machi_pb_high_client.erl @@ -59,13 +59,13 @@ }). %% @doc official error types that is specific in Machi --type error_atoms() :: bad_arg | wedged | bad_checksum | - partition | not_written | written | - trimmed | no_such_file | partial_read | - bad_epoch | inet:posix(). +-type machi_client_error_reason() :: bad_arg | wedged | bad_checksum | + partition | not_written | written | + trimmed | no_such_file | partial_read | + bad_epoch | inet:posix(). %% @doc Creates a client process --spec start_link(p_srvr_dict()) -> {ok, pid()} | {error, error_atoms()}. +-spec start_link(p_srvr_dict()) -> {ok, pid()} | {error, machi_client_error_reason()}. start_link(P_srvr_list) -> gen_server:start_link(?MODULE, [P_srvr_list], []). @@ -77,29 +77,29 @@ quit(PidSpec) -> connected_p(PidSpec) -> gen_server:call(PidSpec, connected_p, infinity). --spec echo(pid(), string()) -> {ok, string()} | {error, error_atoms()}. +-spec echo(pid(), string()) -> {ok, string()} | {error, machi_client_error_reason()}. echo(PidSpec, String) -> echo(PidSpec, String, ?DEFAULT_TIMEOUT). --spec echo(pid(), string(), non_neg_integer()) -> {ok, string()} | {error, error_atoms()}. +-spec echo(pid(), string(), non_neg_integer()) -> {ok, string()} | {error, machi_client_error_reason()}. echo(PidSpec, String, Timeout) -> send_sync(PidSpec, {echo, String}, Timeout). %% TODO: auth() is not implemented. Auth requires SSL, and this client %% doesn't support SSL yet. This is just a placeholder and reminder. --spec auth(pid(), string(), string()) -> ok | {error, error_atoms()}. +-spec auth(pid(), string(), string()) -> ok | {error, machi_client_error_reason()}. auth(PidSpec, User, Pass) -> auth(PidSpec, User, Pass, ?DEFAULT_TIMEOUT). --spec auth(pid(), string(), string(), non_neg_integer()) -> ok | {error, error_atoms()}. +-spec auth(pid(), string(), string(), non_neg_integer()) -> ok | {error, machi_client_error_reason()}. auth(PidSpec, User, Pass, Timeout) -> send_sync(PidSpec, {auth, User, Pass}, Timeout). -spec append_chunk(pid(), PlacementKey::binary(), Prefix::binary(), Chunk::binary(), CSum::binary(), ChunkExtra::non_neg_integer()) -> {ok, Filename::string(), Offset::machi_dt:file_offset()} | - {error, error_atoms()}. + {error, machi_client_error_reason()}. append_chunk(PidSpec, PlacementKey, Prefix, Chunk, CSum, ChunkExtra) -> append_chunk(PidSpec, PlacementKey, Prefix, Chunk, CSum, ChunkExtra, ?DEFAULT_TIMEOUT). @@ -108,19 +108,19 @@ append_chunk(PidSpec, PlacementKey, Prefix, Chunk, CSum, ChunkExtra) -> ChunkExtra::non_neg_integer(), Timeout::non_neg_integer()) -> {ok, Filename::string(), Offset::machi_dt:file_offset()} | - {error, error_atoms()}. + {error, machi_client_error_reason()}. append_chunk(PidSpec, PlacementKey, Prefix, Chunk, CSum, ChunkExtra, Timeout) -> send_sync(PidSpec, {append_chunk, PlacementKey, Prefix, Chunk, CSum, ChunkExtra}, Timeout). -spec write_chunk(pid(), File::string(), machi_dt:file_offset(), Chunk::binary(), CSum::binary()) -> - ok | {error, error_atoms()}. + ok | {error, machi_client_error_reason()}. write_chunk(PidSpec, File, Offset, Chunk, CSum) -> write_chunk(PidSpec, File, Offset, Chunk, CSum, ?DEFAULT_TIMEOUT). -spec write_chunk(pid(), File::string(), machi_dt:file_offset(), Chunk::binary(), CSum::binary(), Timeout::non_neg_integer()) -> - ok | {error, error_atoms()}. + ok | {error, machi_client_error_reason()}. write_chunk(PidSpec, File, Offset, Chunk, CSum, Timeout) -> send_sync(PidSpec, {write_chunk, File, Offset, Chunk, CSum}, Timeout). @@ -131,7 +131,7 @@ write_chunk(PidSpec, File, Offset, Chunk, CSum, Timeout) -> [{flag_no_checksum | flag_no_chunk | needs_trimmed, boolean()}]) -> {ok, {Chunks::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size(), binary()}], Trimmed::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size()}]}} | - {error, error_atoms()}. + {error, machi_client_error_reason()}. read_chunk(PidSpec, File, Offset, Size, Options) -> read_chunk(PidSpec, File, Offset, Size, Options, ?DEFAULT_TIMEOUT). @@ -140,7 +140,7 @@ read_chunk(PidSpec, File, Offset, Size, Options) -> Timeout::non_neg_integer()) -> {ok, {Chunks::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size(), binary()}], Trimmed::[{File::string(), machi_dt:file_offset(), machi_dt:chunk_size()}]}} | - {error, error_atoms()}. + {error, machi_client_error_reason()}. read_chunk(PidSpec, File, Offset, Size, Options, Timeout) -> send_sync(PidSpec, {read_chunk, File, Offset, Size, Options}, Timeout). @@ -151,7 +151,7 @@ read_chunk(PidSpec, File, Offset, Size, Options, Timeout) -> %% off and checksum are re-calculated in server side. TODO: Add %% option specifying whether to trigger GC. -spec trim_chunk(pid(), string(), non_neg_integer(), machi_dt:chunk_size()) -> - ok | {error, error_atoms()}. + ok | {error, machi_client_error_reason()}. trim_chunk(PidSpec, File, Offset, Size) -> trim_chunk(PidSpec, File, Offset, Size, ?DEFAULT_TIMEOUT). @@ -161,7 +161,7 @@ trim_chunk(PidSpec, File, Offset, Size, Timeout) -> %% @doc Returns a binary that has checksums and chunks encoded inside %% (This is because encoding-decoding them are inefficient). TODO: %% return a structured list of them. --spec checksum_list(pid(), string()) -> {ok, binary()} | {error, error_atoms()}. +-spec checksum_list(pid(), string()) -> {ok, binary()} | {error, machi_client_error_reason()}. checksum_list(PidSpec, File) -> checksum_list(PidSpec, File, ?DEFAULT_TIMEOUT). diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index 11978af..ff67bc4 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -119,37 +119,18 @@ weight(_S, _) -> 2. %% HELPERS -%% check if an operation is permitted based on whether a write has -%% occurred -check_writes(_Op, [], _Off, _L) -> - false; -check_writes(_Op, [{Pos, Sz}|_T], Off, L) when Pos == Off - andalso Sz == L -> - mostly_true; -check_writes(read, [{Pos, Sz}|_T], Off, L) when Off >= Pos - andalso Off < (Pos + Sz) - andalso Sz >= ( L - ( Off - Pos ) ) -> - true; -check_writes(write, [{Pos, Sz}|_T], Off, L) when ( Off + L ) > Pos - andalso Off < (Pos + Sz) -> - true; -check_writes(Op, [_H|T], Off, L) -> - check_writes(Op, T, Off, L). - get_overlaps(_Offset, _Len, [], Acc) -> lists:reverse(Acc); get_overlaps(Offset, Len, [{Pos, Sz} = Ck|T], Acc0) %% Overlap judgement differnt from the one in machi_csum_table %% [a=Offset, b), [x=Pos, y) ... when - %% (a-y) * (b-x) - %%(Offset - Pos - Sz) * (Offset + Len - Pos) < 0 -> - %% a x b y + %% a =< x && x < b && b =< y (Offset =< Pos andalso Pos < Offset + Len andalso Offset + Len =< Pos + Sz) orelse - %% a x y b + %% a =< x && y < b (Offset =< Pos andalso Pos + Sz < Offset + Len) orelse - %% x a y b + %% x < a && a < y && y =< b (Pos < Offset andalso Offset < Pos + Sz andalso Pos + Sz =< Offset + Len) orelse - %% x a b y + %% x < a && b < y (Pos < Offset + Len andalso Offset + Len < Pos + Sz) -> get_overlaps(Offset, Len, T, [Ck|Acc0]); get_overlaps(Offset, Len, [_Ck|T], Acc0) -> @@ -336,14 +317,9 @@ write_post(S, Args, Res) -> %% false means this range has NOT been written before, so %% it should succeed true -> eq(Res, ok); - %% mostly true means we've written this range before BUT - %% as a special case if we get a call to write the EXACT - %% same data that's already on the disk, we return "ok" - %% instead of {error, written}. - %% mostly_true -> probably_error(Res); - %% If we get true, then we've already written this section - %% or a portion of this range to disk and should return an - %% error. + %% If we get true, then we've already written or trimmed this + %% section or a portion of this range to disk and should + %% return an error. false -> is_error(Res) end. -- 2.45.2 From 3f6f9e2c6b5aeca462b05b1c9659a25d48753d9f Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Wed, 4 Nov 2015 16:08:43 +0900 Subject: [PATCH 27/44] Address one offset+length issue --- src/machi_file_proxy.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/machi_file_proxy.erl b/src/machi_file_proxy.erl index 0ff7b83..ff6748f 100644 --- a/src/machi_file_proxy.erl +++ b/src/machi_file_proxy.erl @@ -369,7 +369,7 @@ handle_call({trim, Offset, Size, _TriggerGC}, _From, trims = {T, Err}, csum_table = CsumTable}) -> - case machi_csum_table:all_trimmed(CsumTable, Offset, Size) of + case machi_csum_table:all_trimmed(CsumTable, Offset, Offset+Size) of true -> NewState = State#state{ops=Ops+1, trims={T, Err+1}}, %% All bytes of that range was already trimmed returns ok -- 2.45.2 From d0e6417f5d2043dd5adc57b37127ad31b3a39223 Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Wed, 4 Nov 2015 16:15:33 +0900 Subject: [PATCH 28/44] Remove unnecessary output --- src/machi_csum_table.erl | 1 - 1 file changed, 1 deletion(-) diff --git a/src/machi_csum_table.erl b/src/machi_csum_table.erl index 9585429..6e5e2b8 100644 --- a/src/machi_csum_table.erl +++ b/src/machi_csum_table.erl @@ -125,7 +125,6 @@ write(#machi_csum_table{fd=Fd, table=T} = CsumT, true = ets:insert(T, {Offset, Size, CSum}), ok; Error -> - io:format(user, "boob *********************", []), Error end. -- 2.45.2 From f56037240ed4b139d6c011684edacbd24587fc0e Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Wed, 4 Nov 2015 16:32:53 +0900 Subject: [PATCH 29/44] Plan trim commands in eqc tests --- test/machi_file_proxy_eqc.erl | 37 ++++++++++++++++++----------------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index ff67bc4..5550b54 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -378,10 +378,12 @@ append_post(_S, _Args, Res) -> %% rewrite rewrite_pre(S) -> - S#state.pid /= undefined andalso S#state.written /= []. + S#state.pid /= undefined andalso + (S#state.written ++ S#state.trimmed) /= [] . rewrite_args(S) -> - ?LET({Off, Len}, get_written_interval(S#state.written), + ?LET({Off, Len}, + get_written_interval(S#state.written ++ S#state.trimmed), [S#state.pid, Off, data_with_csum(Len)]). rewrite(Pid, Offset, {Bin, Tag, Csum}) -> @@ -398,12 +400,11 @@ rewrite_next(S, _Res, _Args) -> %% trim trim_pre(S) -> - S#state.pid /= undefined. %% andalso S#state.planned_trims /= []. + S#state.pid /= undefined andalso S#state.planned_trims /= []. trim_args(S) -> - %% {Offset, Length} = hd(S#state.planned_trims), - %% [S#state.pid, Offset, Length]. - [S#state.pid, offset(), len()]. + {Offset, Length} = hd(S#state.planned_trims), + [S#state.pid, Offset, Length]. trim(Pid, Offset, Length) -> machi_file_proxy:trim(Pid, Offset, Length, false). @@ -414,18 +415,18 @@ trim_post(_S, [_Pid, _Offset, _Length], _Res) -> false. trim_next(S, Res, [_Pid, Offset, Length]) -> - case is_ok(Res) of - true -> - NewWritten = cleanup_chunk(Offset, Length, S#state.written), - Trimmed1 = cleanup_chunk(Offset, Length, S#state.trimmed), - NewTrimmed = lists:sort([{Offset, Length}|Trimmed1]), - S#state{trimmed=NewTrimmed, - written=NewWritten, - prev_extra=0}; - %% planned_trims=tl(S#state.planned_trims)}; - _Other -> - S - end. + S1 = case is_ok(Res) of + true -> + NewWritten = cleanup_chunk(Offset, Length, S#state.written), + Trimmed1 = cleanup_chunk(Offset, Length, S#state.trimmed), + NewTrimmed = lists:sort([{Offset, Length}|Trimmed1]), + S#state{trimmed=NewTrimmed, + written=NewWritten}; + _Other -> + S + end, + S1#state{prev_extra=0, + planned_trims=tl(S#state.planned_trims)}. %% Property -- 2.45.2 From 1b0711f15188fbe2ec8aa2e9716e34cd89f2c5f9 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Thu, 5 Nov 2015 11:46:22 +0900 Subject: [PATCH 30/44] Stop flu1 client under CR client --- src/machi_cr_client.erl | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/machi_cr_client.erl b/src/machi_cr_client.erl index 0d74f61..e03262b 100644 --- a/src/machi_cr_client.erl +++ b/src/machi_cr_client.erl @@ -265,7 +265,8 @@ init([P_srvr_list, Opts]) -> handle_call({req, Req}, From, S) -> handle_call2(Req, From, update_proj(S)); -handle_call(quit, _From, S) -> +handle_call(quit, _From, #state{members_dict=MembersDict}=S) -> + ?FLU_PC:stop_proxies(MembersDict), {stop, normal, ok, S}; handle_call(_Request, _From, S) -> Reply = whaaaaaaaaaaaaaaaaaaaa, -- 2.45.2 From 9e4dc83f2a9176e9c1e9222e2ca7a79259f44ca3 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Thu, 5 Nov 2015 11:47:47 +0900 Subject: [PATCH 31/44] Add missing cleanup tasks, suppress some not-so-useful logs --- test/machi_ap_repair_eqc.erl | 35 +++++++++++++++++++++++------------ 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/test/machi_ap_repair_eqc.erl b/test/machi_ap_repair_eqc.erl index 03fee9e..85bb697 100644 --- a/test/machi_ap_repair_eqc.erl +++ b/test/machi_ap_repair_eqc.erl @@ -233,6 +233,7 @@ prop_repair(Verbose) -> ?V("==== Start post operations, stabilize and confirm results~n", []), {_Res2, S2} = stabilize(commands_len(Cmds), SetupState), {Dataloss, Critical} = confirm_result(S2), + _ = cleanup(SetupState), pretty_commands( ?MODULE, Cmds, {H, S1, Res}, aggregate(with_title(cmds), command_names(Cmds), @@ -270,6 +271,9 @@ prop_repair_par(Verbose) -> ?V("Res=~w~n", [Res]), {undefined, undefined} end, + _ = cleanup(SetupState), + %% Process is leaking? This log line can be removed after fix. + ?V("process_count=~w~n", [erlang:system_info(process_count)]), pretty_commands( ?MODULE, Cmds, {Seq, Par, Res}, aggregate(with_title(cmds), command_names(Cmds), @@ -352,16 +356,18 @@ setup_chain(Num, Seed, Verbose) -> %% Don't wait for complete chain. Even partialy completed, the chain %% should work fine. Right? wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, - State#state.fc_list, 20), + State#state.fc_list, 20, Verbose), State. %% Post commands stabilize(0, S) -> {ok, S}; -stabilize(_CmdsLen, #state{flu_names=FLUNames, mgr_names=MgrNames, fc_list=FCList}=S) -> +stabilize(_CmdsLen, #state{flu_names=FLUNames, mgr_names=MgrNames, + fc_list=FCList, verbose=Verbose}=S) -> machi_partition_simulator:no_partitions(), - wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, FCList, 100), + wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, + FCList, 100, Verbose), {ok, S}. chain_state_all_ok(FLUNames) -> @@ -420,6 +426,11 @@ assert_chunk(C, {Off, Len, FileName}=Key, Bin) -> {error, Other} end. +cleanup(#state{fc_list=FCList, cr_list=CRList}=_S) -> + [catch machi_proxy_flu1_client:quit(FC) || FC <- FCList], + [catch machi_cr_client:quit(CR) || CR <- CRList], + _ = shutdown_hard(). + %% Internal utilities eqc_verbose() -> @@ -505,26 +516,26 @@ tick_fun(FLUNames, MgrNames, Parent) -> end || {ThePid, M_name} <- Pids] end. -wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList) -> - wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, 20). +wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, Verbose) -> + wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, 20, Verbose). -wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, Retries) -> +wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, Retries, Verbose) -> TickFun = tick_fun(FLUNames, MgrNames, self()), - wait_until_stable1(ExpectedChainState, TickFun, FCList, Retries). + wait_until_stable1(ExpectedChainState, TickFun, FCList, Retries, Verbose). -wait_until_stable1(_ExpectedChainState, _TickFun, FCList, 0) -> +wait_until_stable1(_ExpectedChainState, _TickFun, FCList, 0, _Verbose) -> ?V(" [ERROR] wait_until_stable failed.... : ~p~n", [chain_state(FCList)]), false; -wait_until_stable1(ExpectedChainState, TickFun, FCList, Reties) -> +wait_until_stable1(ExpectedChainState, TickFun, FCList, Reties, Verbose) -> [TickFun(3, 0, 100) || _ <- lists:seq(1, 3)], Normalized = normalize_chain_state(chain_state(FCList)), case Normalized of ExpectedChainState -> - ?V(" Got stable chain: ~w~n", [chain_state(FCList)]), + [?V(" Got stable chain: ~w~n", [chain_state(FCList)]) || Verbose], true; _ -> - ?V(" NOT YET stable chain: ~w~n", [chain_state(FCList)]), - wait_until_stable1(ExpectedChainState, TickFun, FCList, Reties-1) + [?V(" NOT YET stable chain: ~w~n", [chain_state(FCList)]) || Verbose], + wait_until_stable1(ExpectedChainState, TickFun, FCList, Reties-1, Verbose) end. normalize_chain_state(ChainState) -> -- 2.45.2 From 922baaf433b6cc0f00b249d522b3d3c63570167f Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Thu, 5 Nov 2015 11:55:47 +0900 Subject: [PATCH 32/44] Make rebar unit output verbose --- Makefile | 1 + tools.mk | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 4dbde70..8cf5072 100644 --- a/Makefile +++ b/Makefile @@ -8,6 +8,7 @@ ifeq ($(REBAR),) REBAR = $(BASE_DIR)/rebar endif OVERLAY_VARS ?= +EUNIT_OPTS = -v .PHONY: rel deps package pkgclean edoc diff --git a/tools.mk b/tools.mk index 1c40f8e..661a3e9 100644 --- a/tools.mk +++ b/tools.mk @@ -27,6 +27,7 @@ REBAR ?= ./rebar REVISION ?= $(shell git rev-parse --short HEAD) PROJECT ?= $(shell basename `find src -name "*.app.src"` .app.src) +EUNIT_OPTS ?= .PHONY: compile-no-deps test docs xref dialyzer-run dialyzer-quick dialyzer \ cleanplt upload-docs @@ -35,7 +36,7 @@ compile-no-deps: ${REBAR} compile skip_deps=true test: compile - ${REBAR} eunit skip_deps=true + ${REBAR} ${EUNIT_OPTS} eunit skip_deps=true upload-docs: docs @if [ -z "${BUCKET}" -o -z "${PROJECT}" -o -z "${REVISION}" ]; then \ -- 2.45.2 From ce41f9005e1cacaf040aebc5805fd0c654a979cd Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Thu, 5 Nov 2015 14:48:35 +0900 Subject: [PATCH 33/44] Fix machi_file_proxy_eqc:write_post to proper assertion --- test/machi_file_proxy_eqc.erl | 36 +++++++++++++---------------------- 1 file changed, 13 insertions(+), 23 deletions(-) diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index 5550b54..0002270 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -88,12 +88,14 @@ data_with_csum(Limit) -> intervals([]) -> []; intervals([N]) -> - [{N, choose(1,150)}]; + [{N, choose(1,1)}]; intervals([A,B|T]) -> - [{A, choose(1, B-A)}|intervals([B|T])]. + [{A, oneof([choose(1, B-A), B-A])}|intervals([B|T])]. interval_list() -> - ?LET(L, list(choose(1024, 4096)), intervals(lists:usort(L))). + ?LET(L, + oneof([list(choose(1025, 1033)), list(choose(1024, 4096))]), + intervals(lists:usort(L))). shuffle_interval() -> ?LET(L, interval_list(), shuffle(L)). @@ -297,30 +299,18 @@ write_args(S) -> {Off, Len} = hd(S#state.planned_writes), [S#state.pid, Off, data_with_csum(Len)]. -write_ok(S, [_Pid, Off, {Bin, _Tag, _Csum}]) -> +write_post(S, [_Pid, Off, {Bin, _Tag, _Csum}] = _Args, Res) -> Size = iolist_size(Bin), - %% Check writes checks if a byte range is *written* - %% So writes are ok IFF they are NOT written, so - %% we want not check_writes/3 to be true. case {get_overlaps(Off, Size, S#state.written, []), get_overlaps(Off, Size, S#state.trimmed, [])} of {[], []} -> - true; - {[{Off, Size}], []} -> - true; - _Other -> - false - end. - -write_post(S, Args, Res) -> - case write_ok(S, Args) of - %% false means this range has NOT been written before, so - %% it should succeed - true -> eq(Res, ok); - %% If we get true, then we've already written or trimmed this - %% section or a portion of this range to disk and should - %% return an error. - false -> is_error(Res) + %% No overlap neither with written ranges nor trimmed + %% ranges; OK to write things. + eq(Res, ok); + {_, _} -> + %% overlap found in either or both at written or at + %% trimmed ranges; can't write. + is_error(Res) end. write_next(S, Res, [_Pid, Offset, {Bin, _Tag, _Csum}]) -> -- 2.45.2 From 81fae3253927751f615fdfe19b29740ffaaf8ec0 Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Thu, 5 Nov 2015 16:19:46 +0900 Subject: [PATCH 34/44] Remove unused test function --- test/machi_file_proxy_eqc.erl | 3 --- 1 file changed, 3 deletions(-) diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index 0002270..c0625e3 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -197,9 +197,6 @@ is_error({error, _}) -> true; is_error({error, _, _}) -> true; is_error(Other) -> {expected_ERROR, Other}. -probably_error(ok) -> true; -probably_error(V) -> is_error(V). - is_ok({ok, _, _}) -> true; is_ok(ok) -> true; is_ok(Other) -> {expected_OK, Other}. -- 2.45.2 From 919a408e1793a04934327bcad1021e8be5f608b3 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Fri, 6 Nov 2015 12:35:02 +0900 Subject: [PATCH 35/44] Ubuntu /bin/sh is dash then something wrong happens sometimes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It seems dash does not understand {a,b,...} file pattern ... % echo 'cp /home/shino/local/erlang/17.5.6_basho_hipe/lib/erlang/lib/stdlib-*/src/{lists,dict}.erl ./.ebin.native' | sh -x + cp /home/shino/local/erlang/17.5.6_basho_hipe/lib/erlang/lib/stdlib-*/src/{lists,dict}.erl ./.ebin.native cp: cannot stat ‘/home/shino/local/erlang/17.5.6_basho_hipe/lib/erlang/lib/stdlib-*/src/{lists,dict}.erl’: No such file or directory --- tools.mk | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tools.mk b/tools.mk index 661a3e9..5a8afd0 100644 --- a/tools.mk +++ b/tools.mk @@ -64,10 +64,18 @@ ERL_LIB_DIR = $(shell erl -eval '{io:format("~s\n", [code:lib_dir()]), erlang:ha native-ebin: mkdir -p $(NATIVE_EBIN) rm -f $(NATIVE_EBIN)/*.erl $(NATIVE_EBIN)/*.hrl $(NATIVE_EBIN)/*.beam - cp $(ERL_LIB_DIR)/stdlib-*/src/{lists,dict,digraph,digraph_utils,ets,gb_sets,gb_trees,ordsets,sets,sofs}.erl $(NATIVE_EBIN) - cp $(ERL_LIB_DIR)/compiler-*/src/{cerl,cerl_trees,core_parse}.?rl $(NATIVE_EBIN) - cp $(ERL_LIB_DIR)/dialyzer-*/src/{dialyzer_analysis_callgraph,dialyzer,dialyzer_behaviours,dialyzer_codeserver,dialyzer_contracts,dialyzer_coordinator,dialyzer_dataflow,dialyzer_dep,dialyzer_plt,dialyzer_succ_typings,dialyzer_typesig,dialyzer_worker}.?rl $(NATIVE_EBIN) - cp $(ERL_LIB_DIR)/hipe-*/*/{erl_types,erl_bif_types}.?rl $(NATIVE_EBIN) + @for mod in lists dict digraph digraph_utils ets gb_sets gb_trees ordsets sets sofs; do \ + cp $(ERL_LIB_DIR)/stdlib-*/src/"$$mod".erl $(NATIVE_EBIN); \ + done + @for mod in cerl cerl_trees core_parse; do \ + cp $(ERL_LIB_DIR)/compiler-*/src/"$$mod".?rl $(NATIVE_EBIN); \ + done + @for mod in dialyzer_analysis_callgraph dialyzer dialyzer_behaviours dialyzer_codeserver dialyzer_contracts dialyzer_coordinator dialyzer_dataflow dialyzer_dep dialyzer_plt dialyzer_succ_typings dialyzer_typesig dialyzer_worker; do \ + cp $(ERL_LIB_DIR)/dialyzer-*/src/"$$mod".?rl $(NATIVE_EBIN); \ + done + @for mod in erl_types erl_bif_types; do \ + cp $(ERL_LIB_DIR)/hipe-*/*/"$$mod".?rl $(NATIVE_EBIN); \ + done erlc -o $(NATIVE_EBIN) -smp +native -DVSN='"$(DIALYZER_VERSION)"' $(NATIVE_EBIN)/*erl ${PLT}: compile -- 2.45.2 From 73890171ba345ec65b0f380a15f68bdc1f4ededd Mon Sep 17 00:00:00 2001 From: Scott Lystig Fritchie Date: Fri, 6 Nov 2015 07:21:44 -0800 Subject: [PATCH 36/44] Format PDF version of high-level-chain-mgr doc --- doc/high-level-chain-mgr.pdf | Bin 434413 -> 410468 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/doc/high-level-chain-mgr.pdf b/doc/high-level-chain-mgr.pdf index aa70095db5bfd7b8c8137f793ff629b9d1c18742..7276f5745f58e69aab4ec0e9a34abb6888ec19ec 100644 GIT binary patch delta 408172 zcmY(KQ*)pVlm>%|ZQJI=wkNi2TW{=4Y&)446WdNEwrv~xRqf60Pk63QbyuIuKHP&a zocNyrJ12{P0IZ9vvzd_{tmj&_wrtY+FiQ8aMjdT}lr`t2gPEdoJ<7FTlKhosG{dullm%%Gt+}DR1Zq*#QN29M=uDZc? z?N#o3<;GN!quY1g-TcB7>YI}b;AYU}-hB#0g&TDt7|*)p(t2eRVm|XmQJ|FsJ@wOg$@` zt@01u$AxvOmlrBS77wDEKXvnPTimOhvqAXZ=-iuYJN(dsx*dd6RPgj8oheTwRqs!_|_7_<~ zeAIsT<=L)dX1DlAT*lAt<2J(XA?MzfZpJrJI=CLD=~mcGCVe?y0xUE=tAnG4 zHIL;%ySRh7#rO8pkzc=q9#8!1vn2U|&o9O2hd@5JKh2%YNfl|_M9Xa|H&Xq!=ia!orcJ ziFSVZy%bxHLd88!M58U=3JZgEf7h80wt#a5UY8BClx7W@>%VEAMirm;+T@}MMhrm2>5>T}6{XS3vg@L~T- zK=GQroqmIlM$Ak&uf`BTwB!$5q+f~3Lq$ouctVoL;V4E!poI(-u2hlMb80Qr8xCwV z{vuIPu`((LE}&eLM_dFl3zvU({Tn`isrDIAJUJKMdjkdd>nQIQYlzwl@xLY|9JAB5 zJCpbMYJp|&cz3NH4Ixnbtx~oUXq1{=MRuS0^O&XSsuW`iMl9quhQ9#EsMNG(LL~ML z@4bugRjfav=#7gx*=4C&>_2{2hRq&Gtzv4Elnm;Tr_MugI}0jQmMHHqCsGd7vZ7dI zX+NxTCjC~C#xThyStN;o@&u4UFY-E8=?%VE;w0i2lD=8_L8^DzdOJ775UjGIxdDk^ z4++8oKhDWL5Jy5e@ZAA<@5N-rBgqUzZsXYo)XRyCphTs4qAI$&((SQT2Ti^VNOulB zCS29ZBdj~N`RhB+(*hNz178uCLBSAd=6fWgeuc`5vz&jz>VmTjBNbY-c}YT|)-8x^ z7T{5WnWxI5WgUM;n;6zI7A^pKFu0c2}iwqx8Po?4gd?98z z=P9O`AvtH7ItyGIQeoyOB+lY$o@+bq!nz34SU3wgOus|dA8F#L8FUoRZl8~$h~co* zuP|QgHpoW#9#2tIpIwZGYs@9b`W^S!y@P#?q&J>!RS7?E-5Dy@$izCti3=`J@bt}} zzf9YFU*64;A-)$o898j@sl{Y2sh00o&WlvW=M8JdodGD6?1u#v+XgF}Z}W@3U731E z2^J-4;u^qs{h4^bCIQN)fY0mwy#ePO@`y?(@@auC|i!b9nFgeABC<_Z%cay4l?2MYHfmeY5=<`GO#lT$pN&rUtTs- zD3nOnvX6;q{a5cXvxmR}`jz1n-YVi#%R%442@p?erm(8S&dGnQ`hyUDf2rArs)!zW zbVGqUQwDcGatF2mj#!3RdrO~;klU`|_u0Zifa!lz0>pf1s0A_BrGu6FMbMkz1vGf< zCZ#RyNyyv3)@E|>1gu(_*AS!)AIGF7ve)kdcthDxObjWFWBjV(+TClGR#0ByPSiHH z0n`_KcAMNcVm`aE4mY-^-*1TH1-%qir{em5(uOrj1TpNyB)yn%XDz2kYlb$<4iHXl zS)`&Ju4UdAg;Be7<=FBr1xx*sxI{!266K7I0XKOSyVMv<+is9inhz-ALp05Rb(#dQ z{Wc7bOq!rVaU!IjlGAbu&=EG~$R74dfaW4o21qc^O*4pTEXT=97fGKiDwpIr*{p^U z@zIiG=oUKsu`wFj`;-5p|6!PL_nQ<^sV@IC8|Rot83va4UV@|UXTzKHICEcAVjO#t z_74W>&Xvd9!mZ>|Lx}^4a-!A@L8dAp*kvV_8Od98QeM2UVVsP1otq8;jLrQ`poV%| zZ(8I$*Zw)7X;BkG`GdM_Y*Q-m_f7?IjOjUZt|2=Q?;UO8KV?Z{-k%TWa~T~a#PDh| z>>&T7C+zv{nF-&mKdAv8eV!oUX`=8NMyMLzL0GWr0~$Na+8gg*?Lf8~z*Q(!r*}X< z%FD)?9M|RVL&k=Xh+G)kfDOS4$idlpM-`$e|3c3-r%CvZWdG^vr}rXLm7yY&WxrS4 z#Vue({gFiPIr%-nH}?^Qdn^mpA-0wfjekSO%`j}2+K>R66+And!eJEi ztK|XJg&H>z_jX!+i5R`(ZPXD|b+{{EJG9e>aN0W4=;#RNGA{PJv9pc{XjRG|kk`5W zdow1{vq0Yx!_PS>*ky9%?e6NsKIP99k+IUh2_L03nZd`xE|?J~z0#tl6bcrk)f@n~m+v+Z@`|=FaTb7T=^h_<4r-+a%cmJ@h`u=*)B45*px;1veaM9j}!qRFd2#`m5{)0#6 zbMa$Y;>^us9FiVjAA1}m_ZPcc@#bV?2UF?%*fx>@z!>!5V_m5@k%m4=2U~3jy~G;cgOpxqfxW4XJu|J5gZ5unScf8{ zws33l>Uk@Al^5lOtyY7#?fAJi#%?Wt)Nu1t)Fo`F$eCfQz{I)nwaWRV;@8hRPj}08 zeg4H05E{XDb{+A=B%%}&#IvTo>7`%3)p;*tP5SbdB*FDEK!SA&Y1X1?1oasuI4OY@ zoO&e4u}_Y-nKw4i#w6;fG2ycGMK6D1)@Kw!uWBZ=AMg-#)35jMAu)Dr&Tmj z+G=vZzM?r1aMa-I3dv{pAj&pCj!7>{B8PprP3F8xyx>qcM_p zbH<+%P|-e|bz4>nf8sp8<0LF-JW#1`F;F%&Et883x0i%c3F(QAvdq;%yC2jARkwdq=|pb&dhy&h5e#$VFMdz2jfHb zIdJGYFj!)ap(4PZD#T%8fSnp>#2@kT@w?t!y`KAQGy{yQY}JXn4T{o#I!h$o%Y``4 zi>;pKuBIcXpZ9bfMazBpOR%;TKRQ1)d6mhLgPu5TGQ!IVTejwORV!?*@f`06<{R?PHr&Ec7vsDa{J zWqt{F#v+9>sg#A1BH#<9U73<8| zG}hYlK@7`iRJgygH~oB#s^k2A>hKMf>jt%*w)+bbEr}6<5@6+E{=Wd%rL7gW&V}l~ zZLm88*1Nki6SxjtZxX0MSoIn%g!`PEeh ze0X-j1QXwMH@17XZ;46obm2`~tUqn6tbPq9YlYW8I}v~_J_OhNf!Y~q(^x5`>mW?$ zyShF{YWda{Pau-K`HL(dmv{E&PGHZ!jxYFVZ1-GhGDmpe#Zi4FyFQ!!t2>qw&&fdQ zgRhjy*KN+Maf=$m(_yNO7S10HyTGFdnC9PfDzlp~S-pX*L+aKh#A`g>%5fB|DO#D| z_l-T}5v_#w)PV6*;k?mqlJE2ws$eCmW zP|=WcQ<->9ptC>GTjr#-Nb9aQU7&r+U^j>+3qCasb1j;*6~cb>@Kp$WIr{8G|b_`$Qi9Efmkmi0eBDG01Si*(WmYa5dAK zwhc&RCc{K#IGX$9!N4Mnv)iQj5Xq*aq|Lq>|3teVkS1S)Z#GH z`&xFaSSKjUia%wqa68?IW#;MO!$KkxMtSZoFjCtDyS8cj>r3Lk4u^SK)& ze&Nf>zW1);O;oH4va>__#Gj~c^gKv~5j}2EGD$@K_Ns>(R$FquMbls=z_2fJywj!9s1!^* zE5Ip7;FQscSr^MTHgYX!-CA}-V6L3*#oNStqs|UUSyiGMdADQJ3Qd2N=9F}-GS+OL>1Fzd0ojoG#%deaUA6y|!z&lxfl0WP zb`Yrv+?uw0qks^15HddtRYGS@CLDuf^UT-s(8|uWo z+=U&(=+V&c>dy;DhKOiVWy6lIe=|Ru8D+l~BxYQ9dW0Q<3z0qSSIYLBRSn3LTUc(b zNcmFW4M5Dlh-I4=fAAHjnHcz9=|HNMv{}G#PpZH(*wWqu5`EFUgWWVWB*_^0Oxq#j z40gtW5#^j?+&svSIn%RShRHJ*0#380m zlEb<#2HVBR5l9Yef$1mPa76o+56!EmBiDf(qGi)eHXAHWpZyq*8JCYH^_|DK@PhL% z>^pyMRX?v9$E1W8TUY1=W7A===o;G@22^#4E~KH=TiX1^4v%bgP}q`67)^_HJ-(+D zc7whE*9i}$Oi@jxghc#h$K{76iWBj&m@a#BVgoyoyb6mu>!BCH!}QSHt_N%|IH|Sa z$x>RNqmAoFBr>BD3zTxJg0H~vBc9_r64jlP`ji_HdjS;k zp?sB`L#i`Y0jPkS2&mh$D%vCNOX18WC?7K~~pD0lk`neKapN`1-e?qq<@i9kO-JQ3F(RQ%?8WA(*t-`|(WXOc{+xr+`RoIp;- zm!y3#OlsPj-qOnz*6Uv-Y&W5fR_Rz9`>&{WOQR1)!{DW1@q)U_g4EieJ!-X7x zl&2;1LUpNIlAj21?I%nDhv@5`SgTK_-SK&Jx=~rN@~heTRb>=;r1043LQZZ7IVbC^ zt}W_5KaWYh6tq1c4kV?B?YD|xWLd`^HglY=ZS28T;?n_a5rR2rGi}W3HggA6tfW9i zKKS5hdRRJlG~RZdD6inZoKrE|;&e6O@BC2h#fOf{3NA+zmC4J4S@{8L%96aM z;^j*lXv&w%dcQTAqyj0Fpi!K`6&0YD@?O#eodr2a*ZHNngjCgg7}GD?8lVf5?e|KG z5?v}YcG=xvtD02Vti{EIFtf74Dc#{Tzfzy03A0D8U^2po%PG6ZLciv^Z7Q;nCNi1a za2IZPa7Q>E|9h_;CZiZ-BaW=mp%r8Aq7TEy`*o_Kowy~cTT1igx0XC`XI;h zQ3n$Fdl>$DW;+mTRuf!(ib@a+1=Se0qZM)8TO$ew_O(eAPY0Cwh%5jL^a`nVKP=mn zcf=(=Cn+2xlCCJV&t}{_;iHI)M~IQPov%oOZgP7G_-Z%78*%*wSdG&y9cp5dCA4rh zwK}zCGbVrQsDL2^H726kXE3uSFuQZZyl>oQs5&Hi9tKLL@C)`y;y%O7Z{sxvcnUYE z>*_VRNy067HZAP=2mk>4+nVN}CK(ou#D3cphd*;^p{)J+Cr2{>rVleW@%|G{qZi3& z7yXad#7~ef{yo(_1bGM&WEYwX5g6jltZ5=X< ziNttLZgksjuA3wm8Yw*}>(1i>P{}!*pls_q6bNk34?d2Q>0kimy@9TjFfHMSYcfAl zG5dm|Qhx^X1S0*B5e%#0N-zwL3&_Y+IINxWBl1<o*G|uS+H8l#%z--TFUvTW%c+;v{SU+G2_TBAjC9Sxt#w{9NJ@o0cm|wj+*R3ANQ-x3@5QqHhCWW z>-kWj<#2fh>w^Wt97b#rLX6fhhoOZLMIkYWEZFCRSrPb)l(l1TWq|bJXZm1|ib*>k zvx6yu8pAH?SZ!C5wV$-C>DcBsz!##ZVJv2Dl-2zgIj9-FEPllA#*9^@xUU$6_osQ^ zCGyc^zYT>LB*C#Jbvu69_>b*<&@e26UrY(Ajfx^!^zDpVy8-SMYs&Eh$}-Z-@B;g! zE~W;sRVcu#meVg5CL+j%W*Q&5B8uF=E0lgxAHLR5O>`03Fa<_wXzX!6_BG?et?b?6 zl5jb~T;y`xF?HqkACUcFdXKn}7f{jyeVwtVSj_^I7oZ$^s82aFF}&gObDqvi8}1*` z0(*Fyvlx)GQ188}kv>t<0FPpxe;-^+_FiCCu@7MR&yP&`9GLbUdwkSezO;8DC4C9%fI9 zJ-DltpNgskDv8DBB}azfY=AI0o&?|Pc9lcwjmJ?_+5&O>jpeuIHE$s#v0}*E22<{MRG9@_u)gcV7nHv z<9rROMqK4OGx?MsIB{#nD%$3j;}wNhssydUV`F8v$5S|=Iz=sFFiS@|{4F-0& z+C2rF>sctPrThdzSB(SiGKoA>3O%C82FsfFOu3%e-GBAL#0z$H85#fCe}SZm3luU- z^P=OiCS8Zfh?k5^wCpPqlO7E!G@#b3T6Ct{#l?u+>qqS2-MPuZXma^#i$5*?L1OyDxzj9< zwfBS+Ce`zKoJZLiboswu0{8(v=uj^NtPzL=O9&BMu9%MoTCkfggFxmL$>5gEK3uDLN(z-MsIz zYzAcHztWnryS7(jR^P7)SS;PvS>mKQJC4%2hD$!0-;-leMmIG--!yQ#RiisMNYjlG zUR#(+`t?P7exGwKAW<*}5CW*v2=mInVIX`$7E#Bn2cEUiYUti?4l8CVfn@X~^M599_CaesmBB zXg&#pTNVpTm9e|UIVdLEsL-rk&%C#KxxP5F7MMneC)TImX7|y@%g@u z!~T;vL4OtUdoq{Z44EQN(-~HFP2lawbtte$El7YAb#OIgK!Q@kv%wc0bBygSGyHd(w!1s&3n`jK2#grE)nD<8NZE693$xY_eTd z-WLsEf@O3$d6k4d`0L2wWdKk5pp((N!zEN9V0^5#-buuBNaHHM=oN~ETD9UloF+G9 zHz|48&!yw-+29LG(LZnbf8B;6Atk`Z$@zcX#$Ronxcy;g|Cw6kcqrN4h=KPZe`sG@ zCSR~@&);3bgzi|-;nvc8rBVz`yi2c-6@O|oi^Zm{C>@!5>pI>Fug=*m*)KD!HGOMHJ&zKxE;v7EIj!d2EygLl+Ohyni)(6KD7lHN7jz@>I8qTsQj+?nh zZin{x##WYEqIx2 zeE-fqn04|pbASrhDuAEExj}LGi?wNiJ9ddbObKS79J_V`vAEI7%Vs?@Q^;#|cKDCA zj;j1EZ&Lp%yvPM1v*aQ>!FA^H?zG3M-r3#?huiC~#-f^p4*%|*cOr*8j@;v#`db{I zy6qOpubf_ky!BFts8SJzTCYsvd^dv3=8HJ0^$FL~D3uf^=@g;+|+v8h~k2?SN{@Qcx;&}^9e_G#m*&7 zx_p`r^neVa=KF1^_486Pw_cc>@F%&EwSuJu*KVg)J|_B)W8eLb^Iv9u#Q%;%4-bFK zZnX;@eS$~n4RpY==EVg+GpTN20JYd-E&(tbgRHb@@Zlu_Mu$*57^9l)_EDtQmj0+) zaMXrS7)Mz$eTr|H{B+ko%a74>3lMZHb{yKgmVp0&WH?UZ-`#N{a*!0SRF}!tRzmlM|6uUj-8R2bAD`G|e1g@u z&L4p-RJKdmQ;*qrGn9trq4T1ihmy}=7DERkNgci`n}Ks8v(>LOM0s&rKr2+sQ8HPw z1ej=|&K~zZK^Skdvu&^Ot%(Ufa~N*c&@+*SAdjHSJ%9QA+zu1F7!qbJvVCK9Y$ooW zK;N!Cmvd@0JzR4}cX0;Tp&li)hIXDqCa4?J%XTu*STWI;5-nWK$4x=TgT;cIS0Nl! zML2jMt;cmfMvno2Q2j;Es@SgfYil?JV4*HR4K)-XBl)>hj7OCNJ%{LSPT0b~icjRmdVwVOFGeJxQP+R%1%-{SOl{xtq9A_p zr6DioAsz@xC^mwyQj=aPU}W455;0HevK&C25@cRzQnWeIY}6m0j_HxceeezKGvTQm*w9hf5!5J zAM9IdG!!vMnER-=pg|T?h2cq)3k8rFp`P0M8>;lB9K&@36VWCWcTF$1d+GPS`k09= zd>W?BS|kLeVnx=TTD=W$g=+{v=H*J$=M#0qD8k@A{@P@V6Ad=RI&&#o(JRf*6XN4zqYkzc`wK~I)m{qPGshk%PtA)$+2F^L z=@<9`W>}O#EO!j%C6d5I+bsuvXzFJ942xMv^8{{^xe^YV&t#*DkIV*Pfn4xY?JInA z=L!^^ZFi@g=m+}HmNH?9O&5vRw`$@_gACV%O0)BU4@!L7_0S3)R<9xLBt+D?F9)CU z%hlANly)T2PZ?Ui{+~6#wY?>sYDr`ZO&z&<0|g+!X9SCco{vDAoD&Iwas?w-KY1WOaMi}+DE4O9R2 zYJNmx5AJa0{m5DF&FoQVkMc3tJmZ`O#y>`q`q`Dd6*(Z>#ygjAwc6@QL|XX~$CC}RFBt(C}P)s@>{$l)68@WJ@O z{_}g(2)laV;shF9Km%BFm%dRu4~}P$6`q^x+g8%^3^3-;&OPNLR8$H3QSg|`=Q(oA z$&@`~XJk4Iu!%WamRGUF6P^zd5k(lARx7dB{EQkI5Cw<-k$~aKQf(!!kq*n& z$36_{9^@ktWcW9cg7qNPLwleq_TKsmma<4wOA zHU}d=T_J%U^SJ@`OHfy9fn|iCSEi7hav^;NslktejMo zKtsA(d3$xm2$EnG03wq8~WXuSrNd zm=Q}BbKbf|iwI5?^g&x&_*lYwZ2WB^pEKtXzQL4{lEblp3a(gjd=Qly=LKQ3XmmZY zBOcD51#1KJc!3OU9EP>`1^`;D8tC32DoV4!5+Zba1x6UX;oy3{;z6c^_wEmLBCQ`k(@#~&Q(_;|*q^V0)tI0^_@6^+vX z8ktjH^7&PDO{!XrK}d@s#!R1b<{NEE1<*w`d~I5v@O(8}(86EMSxnkVGTu<_C8+$H zS9(XCY^qD+7f=5$h`kZ&oj(QdQVxN_d zG<^8&TTghVJaVYub^!B~HG|cKsWdfUx#X5JXaJ#bF(LNFFN6nKdATBr7)$vxWpK7Y zN*kHdCJOET32e?~k*}zDtONN-sGBOQ*j;5L`lyN@p-bMz3YK#_>EDMvc|J3zwyp*h z!LUj{5rT2xiL8HoD_ILY+>+J!3_%>~%F3Ht^wPyu@ni=X(si#=?5jr5SJXIA#%EP& z$2)Mc=TCgppiT2w!E3hmMlMqNCmfz$MYTUZ7JS#oMi*o`=VF-fOv^nHotS3Y$vBQ5yv3pbW6Tb5is_`fcLh>7NG zw(7s(_N_aeer|mpER7E0c)w&L+@{^MZ`@3Q9>;mR*va-^d@rXCZoLhc7~rL>baBwu zbJ8kRdGVYTO8jRH{w(JJSM7sD54H`apXYq8CjI0-x@fEW#xm(_*FyBa_cZP0*f z4u&G~O08?oOgbM+hEz zX4IL24)Fvz4tOymh}bAZ#0N>GVXhG&In2lsAyD@sV0_Th4nf!h#KLHQ6MBkW5@gRH z(zI0+7rYoOITRZy4N;Wz1*5SK=kCHsHkl6@o4>>xX4I!j@>Lw2IELt!$TNgg!v&y? zgUEAM)9|`Cp%#|BAkI32{&l}-arrzS&z$;rSUHO5?xG-(81O^nGX)vh`G@yrcGWOb>>M(V%#>G7IW&O+DUa^t+Qk@>T%xIse89(Hh6Vs9?d-rS7;X>P|(*hDK>uAt9?Uu1PpCcr=g)rm7a-VA=Y@KrykQ{#8ymB zIE3pm7&47Q;kj(=V6-es-*zyP&6ap%Vblh_)9gcM|hHf`EDbz|Y-9yAnDSf2WJ zf|2QpmlvRXn0~&4|M{m>K=(Lf5vziDq~Kr*W&!7T=Ol-F5(UGbG4H-v%xk<#x5R?V zbTkLpq9A%O*MAWS6%GC%t!7KJ$ZGBG0axd7hUOUJhkW2wIq0~BovKNQ`X#ReZxI37 zqkl7plqY*Hp6fAmJbHi3sl2gKlqc9VXy}71{iU^0-E?^y8n6k4&-TI#X(-GS6iwWi zMxi_T_i-qK0xZlTk{OQx&yUCg&)nWx;nn8zNpV&Bkh1a<8wSQZ>Bisos4DxMZ z>lo<@p`pj|vXu%BsesF(uI{e|^?5&lcv)B2+;qEtrq+X_m9FE#S zTcG{f{MO<3EjON5YztGWRDl-Csm0tGWga4oX?MDgnX$!+eB)$iUo&aYp2A!EfmfVjHs1`;4IiqyUNq46pMdl5B(< zOp<0;G9qDEEXPAJh@DshGR`4B@~)U=L*zENN@$>LxU-=;*N>tq&Q`?lXZUuVgKgd5 zB+MqMiQ1mTy=)(_y4o`xkJ|nZ=Bkv>~_p^KYuNT)}E9x%hAeSc++5yQ^nl z!s$8!#G_mtQx(DNwzICK8NW$5Q>&TjFJz_ca-_#P^wJ0U;Qp5v6>HOk%Y8d-HnZX&4!dG^47 z>aAx*)!pKX$yXCPl?$KAo7$SBS8Hg1=Iv-zK;=e{NxQVs656WDTBz(9 z=$L<4ee?T^=^H#Sgq$@Anu#0W;Q0TgKc1Gu#_(%bk8vOv1J>eP_U_HNC^&_iB9`Ue zFzuZo4&)Gl70&(2P2C^N&+E_M86-5$DL)V%>bCK-&pVJV{nsM(Kmg;kQ9V=(|IEuM z@8`v-t3}RL#`{LDjm?c~MAA(HW5J}3tbLzrX1<(9;0sy$aFNJo({E{Wq@m5hp=+Jr zU2+S{;N}|mT6+>rT7HXKcUZS?z4QC-`T81|+BVz{M)t?N{gcDEcIA1V)A1y!2XD5n zdrSFUFjoJNlr9)?3hrkgJwIYzb?w?BD24LFe*3KmXOK+qzu@3sA(0KPtf=6$Atd50 zswW#hklQVIQ@r{mUgpAgI8sox%Pp2Hhvl#~0;hAl?`X|?Z#&c7P}}yF%sE+kW>S6I z&cDH#+jal`nnu*@9{T)ICHVXJZZEWDNzd{=Ayk|B*`=g*M}Gy~_#Pxm%2r{t~f1;ya3S7H&@;RzL4M#C0d$#2YfF@gar= z=qDyGhZ^W(1?(lK#m?M3_V#S95M${&(Q%?x))Pf=7AS$oC?0KQxZQP(3yF3}IG4S< ztm8k2EA~=dIvPWl_6vhE2`-s|N1TB~`{cLJ_@L}gga#q5@SWsRZF#Q&H?=S4{+;LH z&e%}ohWZxr#Lw@O6B9B6TeMPq#id<uBMBGfMYYB~pt-p>+9tzzqb#!Z+v&?N<`gE<{Y9OGNNoot$%)c)O*+}DuW zN3ycraMmGYQm1gK*O5#@%n-eaoM;S};(JpkFsZupXw(0$WD@ddw6E8%pNx)x&CV}l z`PqWI`B~(LDVSQcVH&=+RmvE)I9|5@pc`M(aF&#aoil;wHmw{jP={s!Z{3r(9z==9 zffK7j%!rm{yVCUw=C47~AHy3R<)3wddk$}}d^`~{WuZ63z)ZBt>rfaM2ffzPBNw)S z&<12+lcQjhka!aoOYPMJ)Wj%Y*Vp@g0=}Qf#NL5L#t5G#Lad5u1tKpJ0L6Z_MZ9j9 z)J70se8EHNXp3FLdt)>0erSWjd8ZKg4E2Gc@3XX#rBNG`yyHY{yMYpt`z}>i7r32$ zJp1m11e-LFsVCI^;Y8GDrx*!c3pHGdg+BL=dx##qZ9|E>1;vvM5B>tUDUhP)51B;9 zufn}e6xk!%&?8dlgB*s8Dqmkk0xn;H#|+U0tT|@EOCrCFq1MIWehI%3;pMM*VTg{b zRAgKsADGzbZH>8GKqQ1%BB-{g(>|E_4L8nmA(yJ4HX$TtV8GE8E5QG|M{&_cF9y%< zz@{@4cX9Uxy;%Op2bluQsHU+m;XrgakDYe3p7nHUsiPc5MNmC);8{Hd^VL(ER|1QD zhw}Mhh){iQ=wECOf-(spPAKjywMSj+hHY~=U%|U9ko$a|wAnn%Dlz{~k*!(G7HJ0T z#CQA}d?ty=!wG0>C(Rb#eq87L!Yz}FC8=XU%`k@~-eicDxMO?+8-BN*Ki2mwxfp6QZD-sWg3>HxhrtS_< zTw>o~t;7o?YOK_{&>d7UmZ zwD@&arV=dY6NQHkS~>Vd1Z87R1CtwhiXzHj)}hb?ABt?>=~jm3lSL;n3hai>CiWsC;Z8}~gAy$9MZ=)9 z510OUvcLu~3jTTH1s!amjyg7koi?@fP^JVIfE|9rD%a~j!{Qk2(K#*@D!x>J=$7}! z882^1gl3V&3RFK*;`XoP?}q!~<~MPn=Vy#InRCY`Dv9TBKYT_MiqYKT&U)br=R9nj z&Z3t|>9W!DpFqNwjBESQHxUv_8IT&B`ItIry1oF_`iW#HHZ{@G#$rlGR5vk30X6h3 z&Hqj7%gUM^POZ*c@%Z4d*ffWKnZ>=awqgr0I!iWauH$s1^zV;@@rkfY{GKAL{Ihhy z=pP}MswOfuP>;6Bzumnm*fs+V^)!4e45|~2Tos7&F?bAEajE{ zUI##(7xAjMb8g2+#;W`?nBX&+kgR~4)%?zi@h6tB=+hY?CmR0Z^wEHo(8$~V~?Ce2sXUO30^_;lS!+6Q^jPODh9*V4ow^|7mt?N68m@PWm{)J$pe;=Pf z#K1Q>jm!fL+1$^HCN@8!Vdip@ml@>4T?Ygp)hxw#XPOkg+4PqlJjZ^=m{v@eje#M)iP5N<4;PI@ z?p>MxK<2qbmoP4?GPZbh${h*<6LNU^zhTy;R`?LJZe_S9Xd39nx=t}8qs|`3FhKBh z5N6s?kWwxqy3RxzGDKm^l2E|(Tt$nXhhO7_3|k!wTNK%#c$KNu=U}hMrbddj2^x`P zmTTNRLq_no?R$_g`xNNFiFiiQU=T}kx{^SoPN6SF_Ku!yawzsSjJ`4htUg=r?^@DR zsEckBlJ9feFCvK^|Bed=35!zJ8k&Jp?Tg7)hNybZhNID@Vyk!8KM{f0K^D0F|QC1z^G6#xnB>|R1X zIs|9b-_L_NKU)hMBDg>{ zFY*Dk5y83Bko7$(dF>4a0MPRG^Xv6zwi~21Os1SFJ;ap_f@!;J^mGp0WyHHZX zRpUMLO0a2!i2Lvhm|-$$M~XmkMbEgrY5Q=F{oQ$I`M58qJZ&!rTmhWj7`s1%zI9~> zgA&XaqhIon{?aLeU1txoI2n;v+cnej4XI#JqV)*|T+q^^5NTW3AiWfnV>bh1TViG^ zokn-ccBZ--BM8zatO&jf=U^(M*&BgBX|%{~7vKYGd)~jY09NQf+Q$T|TKRa>-1((F zdj)s7{~b0hE^^WbTtEyqk4!grfPb}vA(FbP*K`E=-h={g)$#gLTq#wPY3KYg6D&r{ zDy+pdr_o)NBifMOT}pYo;&VBRx4f-2fny+0wvcXVCXI7UA0-%}6ZwX|kR@nm6w?Hr z?W|rHitFku2Af&>l7=IXW304H4`B3t+{5GPS~(S&q3_K_0)}>VLomay*X!w;5ZcLG z$NN#KYJWo0EC=^sAqPFdpX%nL%G8p0$4$_pA`gj?5|?5L6w#)*u_7% zHZk~+J@6dB^PTHZ94U;iH=g{Yaa1xF@2+V^G{f$H63o??kl+f~}xWY;k7x4YW2T|p4jVjo;$@hqL#ruzSg`w^Cbj%MSUgAh(MfoWEZy{ot*BD?5_*(JuS>VnR5 zsgi*tQvuf-jVZzJDTtPgJZyWdc?2s7M?-ehl}&)G(Djyj5}g!n=10lsL8(e|NO#HR zGq6m=;YpTByWy+@IU7IbH+;?wPVP4F zc+VLx<85HXy_o4|Ea(LzJuT8Y%Ez?1kyF4(q%in3zey9_uU_KXuOgi#lE9^jI~!#r z0f_V^9G@PLvx}RLzjq}HJo}bJ`(DVm05nPg_P-%1ERVp&U7Q0kJ@>9c|NFZpkKuqt z!_#x_OvT66Iha4~)IL_>{!5TLX zQ@piHH0#1;_4j9q4B^umR=t*q&2b8m;&Pr$kn zNATknyhf_xSrwbf7@t9Qe@=RlTN4^+s#(djllL<=7SM` zn8B*oPcaW8)>>Mpqw~0DY4@%UiXY*xp(m}B4}cxo7ZQ10k3aS*c9R*27y+nXew-}_ zXa;f#sO=J_vMeSst-IqD4ngF=NEEsM9Nb~LCi5sndUg(R`uNscv=SO(p&lpw+W1j< zUp6-@1w|sP?yisR|FHFyF?|K!wz#`PvEmeWcXx;4?(WXP-Q8V^yL<6sKir`>6f5p< z`@gxlc`xrx=EHm$ImwyK%HC_O4X1{w@elRkQmYXJDv%TAd%4W=-iUT4G>AZPn+Wkc z4;t9~9DqyuJ7OqGtoYhDCFH!bqPGYG@&&#MF%LTsf&tz(H(iHH9X_-4-Bc&brQ9bv zNS+V9KO{6ctO&n}x-mxTbeVA{sfogM2A2kuG)=0PNw%ei+pOYjj{BQC%B@HbjLuC_ zhKvp?#x7YooP?-Yw*GBgviO>cOLe2Tj~EBdS~%OCoJ`6J(F7kPk|_3A#Bij)4i3N{ ztwV}ELI=!HqCPc-cXJ`RW@j3!p;33bA$`sj+5K8i?IH9U@rJ*v$(++{O{t>$NhV@&$1{mFl`0ZvV5dkw$zl)5Nmwd!4_ zTJi}(^F$KfL9iY~kI2ZN2Pfwi*%;Vq*}={crAe<$eSwshaH%b-2)64uV>&e(QDr9O zH!|{$ncG^<%4u#}>XZ`^o+>h!bzwDaWhLnRlSs}qw>7;lp64eYz5MZhn23gA@v?dXx}_pR?+ z1@;iZ4z}3YstgR73)C)7s8YFbla3=zS++Dvh8N{<^(vLY$94B7xW|SHCxW?y+5fmn z{2Mu%*dVYmlQNS!nxtqFlK>pdEUf=0qo~{0m(yefg^T3}WCkZ;{MsB1 zyREHdDgGL&gHWKEJ0>w&q0^7{`}>({C$y9NsKgLdX?5&BORZLf3-BTept=b80IeIvwtxq$;H<{g@J^?!$o=ej+!*dUb?R65&Ys-&Z zpSP!uf3-OSEEY(kjXmCmITW@)iR}fP{SFW*WjSNWHC9gB6l=N0h(WD4nht^v0ek-P zRD_b_g2Xw+XL^)$N}P5>!qiPJ8;ANk+c)vuDf{mR*)P9!*8O?}j+HjMJlR(ivzVo^ z__3#?n}LIW?hl*S78T^(({$hj8K@viCW27oq;fAWr#L>C%}01#iQjtdL}C#yesy%j zC@C!*BxDDa*1?w|;_`n72W_^~)d+PqM48&6vivL1K6tQcjEx@|->ig!7Rd;WM1CfKLT+a~M(1a-|hn&|+ zkSr%vBz9aQT?c#rx2o>z`FjQp{T(s zA>hh*R+vACzK)8pYEO9N%Yap(_d7d?iw`?|_o4qc=NN}=zp`_RJv8*asM^x8!rA73b2Jk$A>GG!;N_ICm`Z|I0;nGS-eBS07G-$ zYbjG=AW!x9V{jLb>Ey!ofCRM}rh%AvP)z)N4Dzw3&4D5iULV2Nd+f_M$?))x3ihP? z0m=1}jXL&4;zyw?TK<)vCw#TjA!lBM2nB7r=v0x&c-GINnakNf-IX?llT(Hvsu16Dm6&

`JRQh>YU7@ygg1XWDIh{Z1o^ zhmjw6FcUZoObHhu#>0FElgv27WKJclJiaPNuNW+Nl9-jCn~qU;f3;`Flc2pOx5!uK zwFeCpnjgOSg3JfWn;wNQYyuMdz^_6x{_j|&KMo*s<{|@l6hdpbke(2o0Fq2ZIyPEn z?(<*j*RcZ&&Q+_;KHP_b<)46l0Mf0LY}coKoHtA>Ql@H_$hX?`u{o;0`+dYj&F-oA zvw={Qwk%G#;a#Fh2~5KN{%b7qqN3)rh!JQVD}HHCa86{s{`y~c`L7Y@dzIbN2jHo( z1WND(Fn4*?>3_Ogfd;H%0CEC*{)T#^Pxp!aFe8ovNN^m8zeGed->V9gsQ2uY6~~IF zNeGT-thAKnzDmZ`1R^<^nRnpok0Tt;ok^?4-OQyq$oHu0N#|#g#bDDlrSF>VBt9@V zN?t5>8TzghJq)VSQ<&2u;1YWNnYDbga#k15t6uNZA7|PgXVQpI2bOc+YvGU29pg-N z!N8e!&;oPZt;#|M=*ud882K7?^91mblLl|(<>Y?xQ4E0Z=6GY(h*v--|*gUj!{Yl3rDxQrRam* zg4Z)Bg%b6&l=U)X{JOKkbDK=_AW-)$Ew`fjM6`Cj8b~rJUba19cOH)+&%5n3U0$-4 zigz=rdtkwybw<-Y{m9@z$Dzn_+X?$-;4B;BmTKJb>iJXK8)zr1OhJF2wHtj)nl25x ze>qUeH9)pq&w__8K2s(S^IGtzSjd#IuB5duJiooeD2bWq-PsIWi0MhBA|EO}-Fyu4 zZ0tcd>qH!|bkZSoII@m2voBmYPJ<&Id4y>8^j2wgaRLc8yCPO*s(3ZGurp;7;%6Q| zRJWdyH~n2U2ByHocbz-+nO~QNA36^2W$Jyc`lQkkM+50J0(VDL(fxb{Qbv76tvi$j z@w<;Ixt!F(BHK2AgM&`tey>Vq2q@B5M`mfg04x8{?#fqH3qE*0vR$Xo?M@rodeGl6 z{p!pEa={qi072e>d>4EWFK_parPj)0$zEqdcu~Y9@CPx2%e?WZtQL_PIehE337&pC zs4PQ}=ezFdrhhZ$qe8Bb%LoYkEq$xGF=$5uy(mWcxg)j%Ba86PoGX3{7F2dp-=Jm| zhhCo|fax~|7t`(bCRzuIVPPIWEL@2jlI28md+}CAo2xU-cH7 z9ur3i?1`b^^t>xKSBAK3#NdBPZie1@&m>WV4k1MRc@$JOf;mwK<swn4pwf|GP&DNivr&`&n(A0iU=V7>s-Ii##@&2}WtH*>j6 zq;lvCs>2AnJg9@!;t)cQ@prEYt12vWNY5Nlk<#j#i84e|9gTo(bgXi=`m*03apzRd z#+*&V3n~2fMaxf{D12|~PZkNMnucr&+a!4%OH}+xvxf(}-Jp{ht_*Xu6hc z(9DT?x21)Fq=`L=>~q6`^QH7>Eff|6E*y7Urb#L~=MnN#r)#Ae|CHQzSzZ*6G99P% z6c!@kempfXP9FjK6$&1dSYq1fVHOM`& z=ArH)GWiLL1<$wnX&^Us?`wLVv>RVOu&Y*^bHyBOk6rKLROmNWJL@$-a7n7zKM!fa z%FZ&XOXVqSoH6c&B9S;G7fwj(6+4(ZKBZ^s>R>U4^HPp z&C9B#;&_H^(ovdNyuME<5YbBkw@{zJU@;pewI_Tggh3+H=!iqYLUl(Kzx$gB5QGhY z%?vh!nY0fjnBILCjE8>9-n+B8fctlC+{cB*!6FT>E(vK_dQQ?`ghzg<>NjO}h>%*b8~m zC@_PkS#ir4GqAL4{Eg2wGSYI~0#r7^Aqkb5kF!I2Lzlw1WtmUeu;GS<02Om__*A0I zF#%zB&x`oq;Y%o#kX?K)yv#Zun@(${Asm;d^jB&b{OpY;OK`)&-e4svI9}!}H*Sah z8TaGxHzXa+GkD(F5;#{Q*q}2aXeM&T4~)HSnf0gd-EA*00-^Dg$2?cO#l|OoW~$VP zVv7z3?xYYVLVK245Bu_OK#DO6@|8HAX>vhD?GKm5crFq(LTRBy8F`Hk|LtMdgxw z_M)cD)Y|ld=|u{x&BO}qnrZU!i4X}-@SpzXJRlScjLc!{@%^Dw*p%btFL<SUu0@VJw(QH#}GOC>pJ$awKP5CGBQCGthZ^=xBuWTLm~meVlj6p7U4IZB_}?0I`n1VK%&&xN3?v6?^_5m}0Nn7>5F@Zoo<;{@|R7HCe zj>w8E)vQOjYD0^$57n)O&l4-FYI#5>>sIJxh9fNw+zdt8J|E@w))4ayOb~Bg$g}_D z)_RblrNE$5jX3cf{CMjd4gG@T!*=qVmYU$l1I1Dyl9{LBL;MpW|JYxJ8k(r%zVGsj z!%ua|eO~K+kahoI-7j}K4I?uKjsljT1BtR!-iCW=rb zRONHqX@t4GQRF6;R<8#NU+zL9Kbz;2rM_dG7*UT1`p3&HO)hXYB+##72y?eGbL)s$A(1DRr09e#l^7mCN0i+U=x)e}pq3~6Mr zY^JokHT~TVrIdxQN$GuP9iPsG(#2`37-jXTOG$zGQ^>)1GamHeAg}QvNvIRX-!f^s znKVA}Wb!7XZ=5~WCQyGnAp%}~^?8U#*hhyFcQmRWxD)x+LgNNw(|zzD8;aJ8kz4-I(9I>{`4K|-IC#DQgEKhNsE1- zTLF84s?acbQJ&KM8(0hZCvp;!_?pvE_Vlp~iTG8PQ9eNiN#Q!W6)TfF<$kS~;Ej?g zJc4!ZwF@hpFx2W8rK0MoRmymD(TvL|9vEw+$Th7(iZdyIS6nvHGz$oiO?VWPOt$a; zOfdA)Z^@9oTK5?hF1fvZkz;DLVXWB8yAn zhjL@oshelgD~Q?)roU~!(4B*w7H{Dc$xDR9QQp$meV6G7wULW}=f8>oj-LF+>clzl z4>rr(OojhbtVkWjMpO6PW>uK$1Ih-9+#ec-{&l5fS>M`B*>KuIxXFn_sjDeNEt16W z6kd#{IOOT#AswDp<;H!eygqbR%E+&C%l4DOOk*Rydg!vT*CK$-2^T^iUOygp3h?O9 zBKDjnndy6GA0^{69~#^q04!NoU)?p(cRlCLTt12_f6i-Y0S zPdM~zwUL^Fsf#+jM`N^(@v4sQRWTtcCe^+Hmr}W*w|pbv=rXFDIV|M3(|TDWs=xl+ z8oXA}@%S)wQH|yxaUU8`>`Y?=I49jX$L;`q*G|TE|25kyoYEyEf5L3eYN5Ouw>QkUXvPVnrfT07APpNGK#Uw)1}pT|QJrhgg)RadM%#Ek>UgoE-;zg=z;2mA63S{qA( znT@PaufYh%Kd&H#5AuN1Y-(Z!n^fJkFYS2LdP%P4A9qy@dEJq+v~MKsKNz*i0!^+I zU0c^B@jQ1%(m(cKsC4Z@Xz&S^7h3nVXw!|RAd^$nRU|9P7#4Zp9bKT;-sK--(*(9J zM_&B{5lQqRZ0GP$M^^EUT-=X737_;s_HSR~Ek7|EQgDJ^K0r&p1qq;{Xw5&4cH!>0 zyY$A~+NkGG172si6<%eF)vNor6~K#-Q*3P3|0m4Hk)lFO32<_7{@(#U>x6^0w4GbJ zZ{LkdAi6lnDi^>ZPW%>RO&gV zjgKey${|yTt{bT3^Y!^BJi}Bb`>EY4k*1l*&fgVHY)Gw*hr^CM~vo0t9y?&$[}*!HkBg>Td81Ms+7Uy)k6>1=pp)w)pB_m@A+t^Z_8{`D4q8 zo7?jMhWq7D53?~G+?>qcd}bK1O)|n=LxFUYzBWFG#<8Ps8S!O}-Fmza#0hj_`Eug8 zqEah-g`;F0h9ty_mzhQdvxmF%r1>F(M}KtAt{^6Q(!z_H6VedF8kT|^+_8>iGbwQ_ zyyd71C!dx!Md9WLM*_qLpb~(1x|zDJ{P1YHB%6jCFkhyGy0rD?JAGyRw%A{?HRBEf!=UfQI`IQ5(CIVte!s%COAUzqeHWyI$j0wLqCFfk7}^3dq=@<5rbxKG z5WmvUWaIEw_Y{^4{#hw+#x9Tpht1WGX^ADROW4T?!(yIpv@>4VM3P5Ax~htgnKcK} z!`n`C#2}GvcR9Q;+C=Mj*43Ce?vsKX6DQ95vl;zX7qIvPJn}t7p_S@j{?uPS#WSP~ zBQ?vHTaYb~iTzBlN_GTFc{a)lc}?sThw#$j!#;x^wA|`$KxTY9BM&{L-JDK!OEy%9 zrsUP{-n}|F*#~UtlVgX{JOhm~tAQs!F9WaXr;fM--Xd>WNO8hvDZCnyLN&GKgN93J zLoL9$bM4SOL#g3U&x#h-d@cIkwxP97+KSUW#ls zna0E{y7MY-#F9HUCd_5Y5rW?U_HV`xfVZ0gTN1Ml=x!`TVY^-TDSujYrz(;iqr7|? zE`>e$#-Zm~ckDSKi_A-O4&s4HXsf8!2z;-56$OoP6$b-)2`1DyOsNDi)TN| z?G;J3MJ9^;-J`Zy3QL3Pdej2q1ZuL{(yJ)2NC*+f?ki$4*qi`Z8a4rIQ-rPp^Pl^c z?}|ARC4h)bm?!n+@Rwi&D}t(gV~m=t)x_1wUr2?Tbf-d#ZQD(=*+u(m^vL`_0?Q5p zXG24gmd_(J|BMIk8f?NH{YnIoJPmhm@LV0hDyyJCfaT%I$8?q)G zjZ!AbuPJq{f%~c`c5#1b>%*~plKoE5n;mA&PYZNqbA2QmC=7|eUuhppqEWMnOzq(w zDJ;FY$b8G@meS~44}-a=6Rbtq+tawuCxewE@vv|bpb`fYOjVG$Aw(ooK)*A|tX4tV zaTbY;|0<|eJmgiR(X%WG+0Wzi2(bvlZ5r{)5Zg(0;G9Cw^3H7Zd`GLa^H1cDJ!wGa z0|U+rYFNtWq(1dx{ljxve~-zZkYk!KrdS^Lv{6=pr_+>@?3WD;C{i$AQrWtN#lEtm za;)1qfAiD{y zz%*i+WWrgd@R;klKi!d2UiKSK){&$_)BN*`O~yud}-*C9u{yW zPL|_lqO)OOwGPXQ>E`dknGi^zqzW3i=mZk(_wLjp{#1gmIrljwiBZ6T_Y=aThw`KQ z>)d6x$BxiSLX|?U1Q6|0+NnS~9ZfR_jhy_|gSEzxf-MuKwEKgs4R&a?`A*CZSdm;%VsY33H2AM?_ryxxPct$bQV*hFPJT{+Fq-!m$dRu9&G^Gz3$Ur z4RrDk{I&0VVw^ZpTZKwoXa!%}P$_S{MT%-U_}111JHy;)aE^qBG*ZT$afpa&Ajc5p z&G=;D(bx?n2MIMTyxjU8Vb%(dkK!E}74?euzmTPfX%`LQ9>cGZlOitQmGY|(53ZcE zj`Cza&IGT?sp86t^LhJx`Jn@NVU@xQLAV<_1ZNNYI#j!$P(Hx!yGX<~Tg5XbAey1j zv2(0l3L`gdF%}AOOu^sDR-b)dE)5T(@f*yu^V^EnRjA@`H+em2-1B3KT)f z5$9J<@%jW0Gz>2%Vni}LSEBi6@c^$9AM)MGv^ituJI%Z2Wbbnzlub3D`sRazDUPM; zxK&x?TV$D-4NW(YpvvWTiO2fn;Wo|^EF46J`4kBOi!yPW(V27f5w>4K$0>;|Up^Mr zr~`&(fy_8YmXb4Yr_SHKkC}|Mtni@K;sGVhiPm&u5)eQijM!Q_;hvPIL%>i}il@X?MQf*F}3SS`cq=FN{EpA`YEBAGINg>1>b=qS}@y zU#(xltQ1}vanO5}7QlNGxNiSqR8ks~xwzZ2&!@EIxmlUYBrP5dx=rToTEkS&ZQIrx z1u^BzSVyB^S~(GS?2+SsJU=9S72BDs#*NL+HUFUzP51nppFju@yqvC;lbNK3?c>!X zvp+!x3$2JuAHx}sCqJT4L#b=CUvt~PjU_o|2xIC;j{(QaJB(L!$Fu(1r;v%20$DH* zbKUGDqZ}FQh^9xCLEMKc7w|=RtjPQel|KtgEkelh$GmaA3}-aCgsD6R6+ta5Xwe6$ zbIKkL^c#^+u52672?^It2{W+)?SJ=(z;|rZlXJFudZ8)#l^)t!jKQ)M+Hd-sM^Y*l zmHUk+-Q;J}d#wM_k5a~9?odNv!pI6{1p|p(&m__O$QB546umYILpXG^4l<+HU#YMv z_}YD|1RlncjG9j_V|>J^h5nm2(r!4*ktL>p2VGwDo>iH;FBbPL0_eBG zNYe?`5qlm@kAMAc$^vB7k^dy*xAZ#FxvVBOmN}^8jV&HC@?s_Vwxr7?*^|6cEL4$PebSKf=KbDjy|vp59! zx4}^h&p2^-bu^8ikJZUU*rE4C3C()q*=EV$A>j6~K#iRu-A!iiX7=}4`bV!_Dr4t6 zC6(7L%@6;yi ztA_&hfFR{IS?AH_+YX;w+f@8^QtX! zP3@m5wEyO~mQYC)f zHSeYz=~+r2UT6xX!_^0H7#*oSB^jLrDdR)TqNkJgI~TeSgZ@9XA%pAuI;oySu!UHM6CAq0?gMhN6p1wh2_2kwU5G4tu5842q;0wmv%sNMCz-rdcl z>j*@$2wl65;LT){pHqjie?SnfbER#uvS&i^)WYG5J)7}OVUvZLPV0#nDfI*<0FN0>*e3&$C ziWcSB;hpH2A64#USOW(5F8TLr*;yrHOQSQj*=NNuxc66U1|v2*?Ilie|1O3uIIsP*O+6hY!t~A(<8_-Ynwiq$SkkAh%GMyG3wE(C4W^A zRr_>P$RrUl5h_ljijQBEY!97f$W}JW`$kwAWZGAai*;)czDbujnDhIHmKm{P{Fz_3 zl``=T6Bi0O`B)6}eb93}uA--5#E+s^DLG{C+xgMCE#3#K!UOD^Vzyn^z6r3tfTV{l zd`L_-nBf(ArjTwNCmV(V>}T8T+%K9B&E|akStbiceZ9f(pLmFFLUP{Y850hL^pBZ( zO~%`2%mwqCaOTckQ!)u^pn;g$lK7uG58u&zQVBR0z!JIap=c`uLrAqV8G35kO4(Od zjp88}D7ekPuJokfu!6IMBvELbt6`ykx5NB3)y-`+werU-owmFgkUx9yaMz0s>Z*+e z=MQzd{N`(MO$gXeqArxYv+{l2_?Nm@MQ*#|Q+bQcloy%HVDj?`7y})R64N}|o=z>7f-eKsUt*0~n#K^?C}pWvgZF7%=|5mSC#d|_ z7EgGFcFn@pIH_<2VDm^B{@~_@)czUxijy}Td@#Ujxl?gY-fAp&1zZ!*ijo5>Y_YdSrK z<*rKuH-#BZ_te;vn1x~kqwMny8Bs2xfj-OIl&~wz5Uz8z^LEqQi%<=%S!_rg(RQS? z2c4u+d|T|83;ga}HPp6Gq?O@q}o#|(DK;!7v`O!xnz5H1Au`3*H_06*c-mM z?*BLP#Ln{H1Tz~1etsrod2a+>-Hot>fbu%)5Sd`N#XUV(O{x<;)V z&`mj$v}@~fdEQ1}VcksJ6HM#>nCbsp{+0;jEzsC!%hHQ6#!*pP@ZPJOv61UmrX+-%64mtHfv*~xj+2Os?S6bg8>zN zGD@5kWW-+7zRyvM+_((p{>nnv4|Qo2C|yyPY2Ui&84n2+ys9#qc;5Nc!aF-QV5$3= zl$@6{0zy%Heo3=xb)RPSdQ95<;j?_Y7*7$`n)aHvOI*Iux7SnZ+1t!G0obLF_3pm( z`m&$l?Ky?V+>dNX+oUSA--h!%aTkocMo9{_#e1DndwUDMERn6%8Eh*@{ZosG-Xi2c zX{bKhaUBinyCj^^)wtm?v^BRgUwfdBJQ{R^z?^AEj9BFB#^|X>0NY|6K zj)vmQUNJAb-I)hi+7P}+asApj^Aou4TTr!66I>L{V7eAxog= zFV^0@)@Fri9YN&+K(8s)+-mjq+U+0GT7%=YI%=JBB;<>u1N!>ibb%KvFnUOZ42oBK z^*>ziMx-Cll&_h&T7<4Ol|f5jiZ*|~QTJ+38NnkX%&l+8cnu@l-m}MOalcJ?s}40s^lgt%erTvV&KBy$tp1 z2*!S%$e?FhoL*0)$?BtyNoA~^**>HTJG8dP2)P&ynO(YRT~j#A%I(m%$D6!foQCPN zbDmL~T~MmqGCl9hyuG%<9jcGloP**X*R~8pJ?AFoO~?Xs6mZ!d;EGd2?i9_i*6;M7 z%TfyLo*V7}j_-Q|ckgDxV@|n})vEN&E+Vea!u>AoUOcz(OC?GA6ObcO-dr7c9qA_D zvU!;O40z8K&aIeuM3v;86s{J2Qe_qJv@F={7USZTI+Dejepv)urzFzJ#^6 zFEBOCk$1h%$F%rIw8#0F#xU~$0q`8qz3!+mIJ5#lQ3hK<8xJtff-H1gcC)WNWU-Nb zh{k?+et1{w@Dzw#ASz-HLiAto+9kSTsiXd7ZX!RgMDHT_$rxU-UMh^aU51to%X&TE zwv?=KbYykr#iyY)b%IA^cMyWET?=k^^I<0EJXD*w)^3hpf%y=&07*_2BhA z;#vz}Hn^`5jys0)pM^DWj#lckKs#yKyTYo^P}6Z@xAa9GI^k~KHbD?P3~ZXq2xC1j zK$+cl`q(5~6JoOWm429&vqfEfPy+mu&X*#~ds^7LE(+qOhi#P}lp@?Mbs^{f&ilVa zdEW5Fde#&Dn`5b7uk_>X!-4Nv$F4U#3LBVg zZGibi%t&hE2{Rqvr8K#58-w*4u7QK&3Ht{`^r|f)T)b|4oNgJEo)@%#F(rJ3dEewq zlyWJ%J=RJm0Z@)+`Lrs1(9Aws+O<@9SpfqbkJ6d~h45=kqzyp^n2|Q6h}$nY&j6e; z+(caEMfzkJ@v15E#_@&opIe{3rI;;S@X7j!?hgBc_+{^ekZVj;`whz!;}<_!0gJ<} z=YqD25MaW($dG$uADj7%oF24oIv9R<>tc4lapP@SWszf5sUvpxl6eE*BgJ{ zk<)K9@=&_bcY?JoDgJne*1xX5-SmFw27igIhoPX6%S~Ibgz%q+4P^XmH`7K3(-!;S zsYvsByX>gA%0sSK@}FOwcz!4Led9IHFt@=MMa%>Bgtf123$kkiC3y=Hm_dju2_l*t z8uk{UVDsz1VpF*|9=!BE#qdD=ps{rl|g&gb>?S-Ho*7RJv#EnB+{er<3OySKCpShx3tQHfj4#8O`ETg{AT zc<5z=AwruYTwQ;YZyOGT@xAphO9SWC@(^x9^J6b$7Js| z{l)2L!55AeYBD<0hFmbcW-&8d{a&BTP@(Rb9@#9+}1+lD}IbymG) zKwE_UCOW}7%4f1MZfKLbYgE~HG_G2n4PAh+0&)yKIyOxG7R5&lb>;2*Q%wb(-$l=K z)C=!C+uc(&q%a_4Y+@;xps#Yb(UAQzAt*mbYgd>3^plTvSE82>@uYMS;<&x9xRt1r z_erwM@}_F|rw|`QAoq*{ll?_R2zSx;-vP4CNh(_A@IcSIuTpjvjS0GY%L0v5?2#Th?EeLclay`)?Q7n&+C|RZJK>(v9W;DISDIa=uWPdk!U?dw5UN* z$0E#FOtidR%X%l2q`wFsr@}tuyW7*^5q#+2*Y5&BtJ+W|FyRjzNt?eko*;sK9S*yl zg!wrBO8W&1Hn;LXb|d#I_YU1x7W`-|aPMOwVzbVkF?X-(J-Q?FAa}al#IR)Ayb+lP zpQxhz8?Oh*%$(MiN?_MbRFg2zC-esD@pm^Ci3YGXk(BSC+7{axCG4%mp)xA=xyxE6 zJg7(LI^PBMMw3Jbbrlb9GE;(^J99&7%1N*n@d89h{ir_N z0vxnTa45>!9)5WHOC1F5Gm|f4@628Wu+-rk*xLXWxI}Tx_ys?9tDX?2XNu_!qOR{=P7lur z{46d46k+CtTQ4}2d{6TVABHXMMf7vqp;upR^~Y?3P8L1c)7wpJ-ZB(9Jmenw;z5e| z7PhqlQ|@(z8GbjtKW$nC4*UDr9%EMnY^MQ6d zs%O~lF9-ZnS^-WlHl)&0U8RX`&k%%Wo@8OcbXiKNt2Y_jk#{j9S9Dl*#|2uq`=|zv zJLosLSQ`YD=sYxpBR(?nz5E0d>gI+2@T6i*hCuerAHf>7=Loj<^S@Z@Es9M@e7itW zTpil#?s?KC>q8DE>w_%Wp3qmZ)S_xtO6%CQO0r_H=-q4Ze?kCnyL9v)s(zFiRp9Ks|fFJ)TnWJH##d z<^~mB^Gb1gDt-7TE_XGm;wgopJEsv^#PWggl+j)ouL9MzIbt*hRwtE`-L3}PxdH?E zH^v_Ulfy6@3q-};}-s@SW#uA$OxP)Di%oF$GZJD+b_$@xP%$Noc-v|(iCEgdK z`oOAQUc`)Ub$|=(D|JQ<0TqnDQSq?EftP%t>HgoyTEUX7I+4@5~7RhX52LP^Ssu2yL1WhypBEU_H)N$=p#u+G26JQ5mA!qlf7*7 z*F8W~<_kND#V)8`luM<;0fZWxG_p`W=Qq_OS-zL&czBfj0yM_*_ML9$Cp#(wq(uB} z)hDo$cflgXztBvu{$)8N)%!GBeI&P_L&OUYH6#*Gl8vOI0)DX{zPayeh+^|3^6pSp zo)_&8;YRSFfpF}OgoS+(ol&52vAegckN(iJ2u}qC5+}5C@K`nH0}wHuC&F~DGlbH4 ztFiQF^9^jjTv8z`n+ui#zxtm(vmCjKzkyJvGR?D)*rHv?pjB5eCHYlJNi9)K&)x^t zIb3(ENr`U=|6z7hM`%N9bP20GA-;%K6k7H#M*Et*d8T>96=80bd&Kx-7VvM zph{ia9!FynkGmvV;NNKtmQo2>l`S1)q95=1%GmaKSs zzs?qAf^rcu@nVhC5a*NC`&;L5pV&^}9A;czbjpoSwyb|K_!avy^`BBIk3g~JgLKD# zaqlE1|G~0O@>4D|2_-Dgy|_Gi{TB=O5)E}fUUc)Zld?7=p&${ww%zTZVP@^LX~#HK zl*bM=;ALif$m#us2lFYatQ2BsEtRq(o^yM92j-Lrdl}h!TD zNXUn~{`@scS+(%dv6P^Ms1^h>A}KljSi%+El%@aG-YHd5o?mXTQLj#wm?94c--TP3 zNJW&rF)p}m-PCATUMW>J#MFO}T8CTqL-h3t2tf8!MgF1E6hh|v>qqfGHt$4?1&IuT zU_XsakgQ{zknr@`shAH#j(9W^@#>ATZK1$C(6Pr8odeKD1iUsTnX=>RY`Qf!zV*VifF<>7WCsYPKnka*{~hk)_O|+=z5@O z7?U*xJ@r>l<4*Z5Ecj4y8g9(D11i%09S&sWR35*cY7kiWPbh^)NO zh$8Ab!kU#h(b`mmfjK6u(ov1&$n4qysCO=<7k60`%~6cSqAVk5YB?<@_v+?hh8kOW z{^|4T@(J^zo~Nru$6PAAU6~eBLGQu!(%ndRKqA0#JWE~S;hJgjQwz;k6aod^6Etok z`g$W}$!;pmSA@@urZR|(*w2ukR#iC$i>BnBO^fPzm}TPyuFatt|J>fnrma8=2*x7Y z(PuNiMMVUKDHP0Y5*|qR&)`~U)o0H~W)|T0gXgVN5$Uxh4Ke5G#~6Ux-TEtCze1@! zbEVU@S3dE>Rz1b@E~F#a0#IpPN^kJ4hw>N$rshp~?; zcIRgQ)#jIy`}0kNiUxX<;e?DNuwpF#4SveILOr##yk!i{~y+19++z$QXx5n*ak@(4UO!Kv2Z(y@ zudmABcu~|8Uky!L)1>|PzRa`Y*_}dJ7iFTHRoafmGsejDa?@#IhKLh+U-9LC|7~@UU4!+7lKVNy1hZP2hSv9g82JDJ}Ju*bXH@n z+bpYglmt=53%>;aHYlCBh_6m84riXYnJ9>g$=oc`Kug7_0C$Gcw z7b`TeMR}XulrgM`9mo9Tqf|M`rOU41&Pk@P`wReXJ$+m9Geg+G_ptJs7%6nx60vJ}&@jW?z zhAVjC2{~4(mFPZaA*<=twf7zu#bxjy77v|~r6t{%6zGApj~{~Mz%q6QbZd|j;4qOs zutBjv)^etf*gw2@W_UB89mUnOW3-FNW4KPwI~282%B3_=fv3!>+{1`3p`2fA%(R{> z#ARu6$%aLs`lo~L|MzuMu{9^phmA^Qvtkmp#tuQaJ zMqv~QP&{JiP%j7+wx+i%k7D8o(~l<6-&~7p5{Hm{FQv4dZgr~Xz~T!i2_^sM9%xsy zGfrNt_2+SN?cL1zd_-hwG`DxG-xn!Bf>%!yY<=!{^QJ1j*(x+0c74}mtbq=s)k;T? z>VW&s55D!jN?i3{0Bk^$znC!>uA=YsZ8OeutS}|s2TFe!OsG`7NY^?ZPN$2&_lMWZ zqhF}gyTF!-ExR$rz3=@I6DWNwTK#T^fq|LBivbI8l-O1(i)}EnEViM3!Tf}5L)+Dx zHbliw>HotyXLu3MOvInXsp{#H+)&W4d8t)i)J3Q%Xttc=PV*Gg;~iBNQ%Gue&F&HI zcwp#cu62J8Xz!@yjHUPeEX~ptBt;wN*IBUY#6Q2f#ifsQ*Cl9gS6?rPN!p_-4+jxR zb4qM#?_WT(Zk{IUJ07UApWfu?&VW<91%C2@6qYt-Y1U(TYcHRyq2hW-{5zvhY}7mL zGyEx8(Rf}(CR`rxG|%vgE&Rxl=3|EAg-7*0djWqfyPAyy44OIH)-<8pSE$s&I3&_- z8T572U8Nishp|t5!M*}eR7f{z&}G|VOqDQ!^9nhd(6l-pD!}=~{`=U{o5|y%DP=bq zr@+Rnagqw&!JsE~uL`)`vV`NiDupwMH3i1TDuCY9RR= zGFg8*laBaNFLSwOI1v)UE%Sah$5%QFe$Z?dy<>T>V=m_at-+&+c!gnmfr#`_KAE&V zj97Azk$$a73|{l*4)^M#c9hX z1jpjJw*N$0A<6EFQ?yH4PG=otj%0D@Nc4ZS&0C3L(#xeqvD{XCLCdiBJB*2xQs_H{A zu{7gDc_CHHXId=t#r+$aCoj{5Ly`UG8j8e5gWS9=C5>Sf&{{xy+0GOjDUOa(Zlr&R zUwUu?nR*1N&0$7l2Bxjnbqd?5uAA;UBk;k2w@Xc6Kc!1Zww01YNO-C%ESs55JY`7O z$A@WzqJ(sLGLV`5fi9UzVU5+$>BWwUcL6D%#zR&=r-y-_wRD`5T9&5fQZj|J`u^;i zI-KB`3+TKfE?k6bgdfPtPS9GWR^)$ccu{UuznX)_@dV0g%5DT`MHiY&J>S`Jj0$eT z7f8FBT9bz!HWN3e^AoS&t)!xczj4CBIB)&tELA6?@eUCD=Ipm(RR6-+zrNr4#c9(E z>Tm`ZA!td-AfcT?fhCPAYb_GX`cZNs2(>sZc+f_^$_1sZ?k9pjB2 zyCk!7oZTnrOj?7w1$h~iY(Fuh6Rai$-D}wV>cl11hNWI6MR{ov64f@$!W)<5~`$oE*wgpunUbQWIAg&T5m^4!@k>c9S^iHQ|VC^ zru$9r!IZ?vqvBc|Hhq7EOXtv)J*pqw5(ZQM-T|10YyNO5bROt!40j_e*^LuMXK5>tH*dvWTW8EJU3N7(tf(_M_EP0D3 zuMdYOCJFB(YZ^!4Nju2d4O)V-3i&$*gvvryM7sbWi7AYB8Xzi%up;@wk+WC~cub&k z9=oUiaI9qpA?bh8IO#HlI})~-94B>@(A`tqmCrqNQcM|dJoP8XvqKJlvYT$2UIh+d z)mso(g#D-bxq}*jIDBa@(yYJ$Mk;kX3R6&85rGC^Wht?EimVi2Z&?pKl9)YRb$tDt zKB6d=!P5XNBYRs{6*+qc6ftAg#{m4K?_}X5z)FtfU#x$C&4FqxpB&(x5FkY*Dj1+H zI)dGg;7;-t?iSii&eoJmv?rk$e6+_#D6h0Iqhco%SQIa5q4~iH zyZ8FeZ%%*pX4Hm?M;p(GeSsd~>q0>{pO&NV>)t;8=Hh5Lou&;G!I<)~#Zs29O_+%b zyW71H{K-+!YH%2|?MX*Qw{YRrcGF1-)}P`RC^)vW!rmNDxM;BxACBzJ@px~ry*Zw+ z(8JywDcm~QnlHfnpe4=*pYu(o!B`)%}ykwhrO&pbg3O~C^s2q z=jMOd?8N&KynciWwzfXlGBF~It{{-v zxrN(i=b^*w{2~6BoDm6#userp{=@7XBT}F;J8_1qN4&1O-QFZ+c8(n{mFC-$x^?50 z?sy`H(Js3a8I5qwJ(^GG2icuilVKdsud{z-x83=pU&01jF^SZCz<)7R($Sa8$PE|g ziLf1Pc5vo6k$OFYIM3%>h=G;!TSD3?9}x&Z>x`&Fd4#>QblCDdgNmU3!KkXmJukuE z4@vVgpB-hvTZ$3~MD=*i=WrNp2OW`(uHsRps#^Df8XU5np$<@sm?Lbb1A{nq6q$eh zu3E&%@icCFpu;34@QdLIs0sK2{NmWafV1$@+K9BW>oX&3@3mi_5qSc9_x5|Gw!xc&eLws+&m;u|sf~7%>pt=F2dX`IM9`e_b z5+v2r*H1a}B=--&3YkqXTtI4;l?*|Xc5374Zsg92PR)87$C^SncPc~pJ%ICm^7X_B ziLa~{6qJlU$OBY}OGG-QYR>)lw5}1-R29Y|{lyv?2?HHJG=| z*d_p^Pk8(UwR*xrXEKB!R_T8(aXFmO$pjYa{jZ$8M9+7Ab6ny=ydle86PnOAu?gmr z!l8DB3wm2aH+VeKu1^tM(5R;wg8zBeyq>r>kE%IIcZCZfghp`qKx`x(;~A6QVgha{ z2s9z<`=tWYi!%tTg*wUVP_$81Muegb+|r`dL)c&?VsBQ?46}Tw@8W+dIX{rYYw`#3 zjLNPJ?etL-gJSOy!C$oT(90#fn*%5BYro%l4D$SdLQg`(DKUKz#d7NSFuS`8TgC29 z#uM!%ANhe8ru`FpAl-;P&}!n3nGl#zzIyfV#Zej?Kf1P(?WDH{E3BBenf+-;$rc04 zgbF9a&B#CV<0k+Gw9;ur4lCP@-1M28%^3pTS~`_ObrwHikyD*G z8nJA2;7mOlxuWXe?#5QCh<~k1?E+0GBIz9cwk1ku>=AEVcHe+&a7KSn_xzX*p$SFh z$le7HN9I+_5qy_@g@c2`u9NI5Ku$;M2liD<$4P;G1(51+@nnBrwOzg``>H+P`LeHY zq%W$orHShFX&&2GbxI9*`PC72Y^=}Pzpol9kP<6fSG8TaNLBtvDAv{$t3SgR>&6O- z+HCCA}D_^^Qt8mb%%9@S+}nQ{F+^FB;)Fb6nTbw`H-vN*t~jauWOlCIQbb_ zRqcdzo!XJ9Np~%})-|w@(xDBO^t9`V9>wS~uWHjhNXNc1LQ~jRaS%5QtTQ+H=d5o7 zD}b-Uz^dM&Y+uzDg0w-P`U#Q@9h*>K(+eHF$afZwC{KUSr7TcdzJIcip1X9hdHi>Y zChV&-A2P6h%nx+|JiQe~NVKswM;ZgP11&t}3b+F=OTxPdKEj@Q=K+6ItUOK-Vt`Ne zR`l)lc>9U={P~a2RP4YdaeCP9vI<6zJSh()2*4$B%|hpC5%FOh4wx{zz!U>GcEfiw z@^Sh2L^^*NZjjGa;wr$F=ak7p9V2TotvAv0P4~xd98Ti8#`y*dJaKE%CA`ILdZKtI zSGM|c%m_hrl@qJm`vpD`g`leS0Ug#^6U0#ejsoN=vK$kr?|?a8^~lVi&5vva`wD9mjf$r?6We{8VX^l z>?{@VTLR*mM@xNh9 zGc123VNvOX3daFfJ2F$DCG~u0C<5dY)+?fn?Is@|s|$t-tMqzUPfOo_ytt&kFn;S8 zH*8=OBt7mzs1ezw-M~CN6Q%{bD}z?$FpVe^HPVz3h{1v0X2CJPkc1Fcqb{4YmRGwU zFo-^}7Lp*vvgs77#fD7&mN-~%WuCA@oH~E%o3w2n&SLNgV4Nk25+rpVB?p9^6EOt;aEF!US?c zcUipLU`Rj|j=7r%zO6|ETSGr9;dZAfD0y|Xoh+6;YkVnRXfygS~zFn2RbMb%G zr-yq#2b-FN+%=*C!QvY5+OE^9WP0HN>;@9oou7sgI~*kJ%U0P4yBJ%=5=PuO2h}KG zEJcf>Y*fc(UQYnA%XxP60%gR-2iIgt@AARUV!L|Eh9_Jys9CvJBXG5HxKCY5t(-Cw z>d~?-zZOtm>BP-1BO!nNv>3cUR9k-pqZo*^el~yFU)i~xGXlo=Jc#S}}?Rug{5TBcUdm{1> zX~o8%wykV{r~oB4tnP&`jL3g0^Z<<-r>9ISUL2mGc4??ew|$=+Er9A8g*+%>_Pj^>Jb2A6mIqJb3Eu!frK6mB^CK{ zvUB+O^CgyJNQ7$w&itn1kP;f}!$CzS9ohC1%^r`EikzEb6Qz6T_j!NCuE{x-Y@2dI z@l)(EDN#rn=|pm_d?M#%IB+`}#Yfv}F1sQv`pnVr;;ziu@Lq0j_Td8KnWx2oZ8(J}o)HYq3vTWJBx)0CgT{ zapIRTpGK&+JLUt9Yrz@wX~{S7Ip)(&D$y~Y!-)t_OHqvVn-dY9^VVNDj`{GnEzdwl zN;RAp#(dg_Y>Ai;I8I(5=7Y}nbz(jmt&b&@W>j$L3Cu`8F`s|7V^GyI9D$JhbtFUq zpPw@1`Av7g=WrD8X@qMz0UywD436<0iaiCIDg) z!9LbC7xMUS zR}=C+KBgva`pnJdFmLa^q5VX$W7BvbQ}CAF(DXu7oi-`Fn! zqhz({N5-j2o7oNb;qEFp%Ya(*imZ-7?Im05N5L+U8+TEP&_@l2A)v<;VGV?&K)Pfl zNq~QX-g9+03<~N4XP?36_f8}f#u9-W?NzmY->FOGjE3C8@2yn7dIIb^_ur$t&Ext)AK#b&B3 z=a;)>B*z=xN?2LW^CT%B#UD3E?8$d&;dC$j%0H2mm`;WBcfUH#5fKT_6jN# zFOH$#Xs4DOAh*pZKPoNP!NIbZ8(s*`9?iD%I=?uGYZ6m)zxA6FrR6gJY-++lZ~cE9 z@Psu>L>a4(?%uY@(k>2%w|^-lgfZKPNzvo%u%GBD5O!Ew&e;vE$l7umsc|y zR9k-0t}Xv5f0W5UJwGxzV%YbXZB88TIdA>u_*4m?0Cz0%2Se36*DJpmK6i4x^NSNF zG_M}~=InlU_`>-deR})3R%~rP`*eSCaE|9oc&kR1`pwyIIZyP9vD>`g`3q;*Z`{BA z7mnXLefZ6ZAAftA*2dEs)+MXh|7vg<5dw9+;|K@D5%UhG;1Qj7?4u`1>07=Qf&4VZdEjAlmAs0|n z($25vQiw&R04p3DK~+)nX2cG^1gxbY#)W;Az*_Wf0F%Qr-XxUF14HE`x*RaKC^yN% zPHev0!!+Ko^%W09nPe`4F@t|^lteQpm6c&5qYGb8MF6>I00Q?cS7Vu=9))HpK0(|w zQ~>>zQF&Isko~y@7*O%$7T{>#gZs8P7!uXeHL}e` zrJ^;Q{)rxAuYIxb#!bu?eT`BL+StN;E*n`*;!&7Z{3$@5UI|Qy)aHK_UZO=bGsbh^ zrxwy4iK4@h5(Bp*Yt|?84~4i0*Z?&$Mqg_6jAiFYX?H6;YKMVTzWEw_<#MJ8X{wiIJhX;{g)V0@_)88A5IM* zvE%{FLO>h7jRQ1+#R7aJ`@UKNg|%2l0uED19Lt_5Td%<>jIZV3F%L4Hjyn0^NU_jzWw(g$y&x;4gXv zxYp)O!bict;kKFyT##9py)+-vMKEY3sSl{DjXc#Y+j@w?N52ERjZqiX5$+wZ zZud^3e%5sUx~`YCzDDdc0$hdmF8*BTd#=FDZ0pb2;LbARkRvR`MVPG3=>%Dtk(*UB zX5b}3;|h+g+9}UGRS&SJ%h)@JWLv4vP%VA}HE@5A<)(;!wxvc5I^wKCF6vpJf%1-c z(jJ>?Y_-k6b*L1xt6Q7npCtqpenc_-x1&RVZEf2Q*$v-jaux`j6pDu?DiUJ?aVy?6 zgQ#p>?abh=h0TUX=TJ`nz!VnVBgvNSc+)Ge6U||T2C?vxNc3S=ulhU>+?!$o5Nx)| z(>Z?yt(F}a!b)%-6uiq*z+Kr83t0f+)90+PTdEJfnvH9#2;yvcH73#aL{`A4E~kFA zUa#4v1%!#xv8>_>HE<;9C2{@k>Ba{6LHjz?i&LcjMr^4vnNby&yf@K9q>NQ7#HKe^ znB7F}obg6eh&k^=lRKB8PC&t(Ce5|ezvicfHUpmOen!J-6v2!=6wwr$r z;J2G&ty9A)Tl#{`-LgRjqh1?!n}GJbrKV5^HnEarXa@K)OhMrVD*06&0q8rroy7w<)^H~Ry-_u$nfGj5`ksZ-KX(K|o;=1f z!iGy+z9`-x*pB6*v%ZgMhpiw|8K3nxt*%ce#VBM7!wR zftJZ**KXCLjO7Xd$}bdnHbz~z1|%=#EHpW9OZxk8TEd(4iVA1ug1-dxR?NW4LJ3;c z=1&B#fH&8M^AAT+#W=AAT^t9A@e%p{nE_e+s!D>gF0R;+hqAE*X}7&B$T5Gq(awIW zex6{AMslue|6NgKni;9bWbI&z!Uc1=U;#D(ml z8!s@2H*uy{8bgY1#aZ>O3fK6sqd7sf9ZiSn3HL%!Hn@>7R5$>3LaY_yIOHSjzfZBp zB#*n&Il#KCP>7JE(w2Gk*|N*v$egXe#vMk!Y$P$z$O)0gNLJ4 z(!;Csx{AAo2!^GVs2w2_`gX5W_2Hjiy%IWM4TIPLhlBJM#{I=19-~qALYkLA_97mz z1hOZXAIA*=+h(o6xFKlU78FdtxYaNW1z+^^?@MxtXcw|?KtxrI=RJSrC2TV8xuAFS zg>Z!B(q!Flcz~D_6SCX%KF~?qo6oXN3A(7w%H0TH#vbKwhj9Q%pcJP+<><5Vt8HUKSbHkND?}kDLSP%Q zYCvjl@v16BkLc&`!xilQ!6%Sj$_5zhC!#7O#cq)(>5xZ;f-wn41Kf?`aQvh13ky|d zz2&-vufYb59aBVW=el+(_9wQY#PN8JS3=;yC%%Mp$zh z9p`X(G6^tfkx+PzcR=5H<8lfC@Od%!fT=15Q})s=0dBU}c6m@N^?@A?f023xgTK`h3v(@L+b-H*NYqNDMEhaukI z!TtD!7)Uo^kF$T9_+uu7LYS^@{d;j_nG!cLdt2(F!(6Y$ZImijq^#_^FD;1EDKD53 znPmU{IlC4IK8n8f55U=Da@#G)zA-ggxal)Bo5KrF`Gss7BgJocvRRVBR-~{1g&(+s zDVKbC(d?ov|Gi$`y+&ww4TSXC{^Z@m*Pt^@|8o+!Xpnykx})?Z?$|y0Z*#D_q)$%b zuF1>GY&pGt7-)MP+1jpM>794PN{xD|LJJVIMXc!dS8zPtjD(hbv6SZznXv5Gtzf_E zHf$Jd1)a(ffEvikAt2SDi2k!-%;c_>8b`CjT`T*nyxg@?BTfN_GLaT=pVpTQ#aOT#2hz=XB5-s0E$ftpb98=1YQ)% z89C?(nakc$Ye!rkS(X6fs-ZHAFEhmWd4zcz0P+2pNiS1yeF4w~oRlvi%-Pc8?OoRt zGBbbhbCNXy4y8bjIqeq*3T2erL#4KW+>;7@wwt_=)iF- zj2D|?>?{{3;S#K!d#UGB1TA<$B~k1oiBqVgE7BR$*5;XaACkB@Ny%QkWiu`~NSyV) zIsDylewNy`oug;DCyvl$YJ(9~+A#by7LR`y!0a9@*>YZ)Bj9`$SOmR|*=NsCibR?4 zEO+n=kfT3lkUPr=lrNw^5p$NolN)pvz+EbVRWRY-ms9MR$N>XVTMvy1jF~6yjZ(rE zFkI4jsI`L4cxPcHNpTV~0=o1&F>#Q)?>%W`FnJF$F6iXEJ}! zjesuU!bu$06%+>o>kdxOPx`&}CcYJ2JY^Vw+s{^Me5?S^Qwyg$7AY zO0n5ZcfuN=Sp(Xp{QKka~N?Vjg#>d3>FT#-jiF z=bHHl(BcidNEgSJZQ%*~_b045ihh41)SXNM$%0Kd7HCuktB%vOc6`((0OAPGJbxdT ziTQ;gvkl<8Ou^YfdSI@uD8aTNwCOWXo5R~K`32Qx@VIJOfj0#sj@#%O#!Qkggf?DQ z;pu~oMD+2>=y2wCjIeW%c5z4^9wgZWlF_#Nh*lN+8|MQk$t0nmRtY6x6{ zr>O8RcST>p(6FLYoexAnqouDi(-CI8^nk;RKQ)6;F(VKa#wu>;em>H-7e{z-vA3%x z41kDsG^yH%lG&_07%>ONVvT|h0P1ZFMkx!R`mIn89?!sO^%KuL3J5q6PaHhW)j(P} zAKTBI2sqLMQrk@nMKZ{+2=RXtnt|UOgvi%V4q#25s3jX*kjqy;L{gZXlKxeh3KCRX|3od<^WH16mI;j}@}3CuBQGbWfVzD_}3QB}ue{;4NobP}9=7eOFKs2>vAbOj} zk&F{K3eiA(%I*M1W7{^E`7*8OwrrF*KYpI*k(7aOU_!RKO@70LM-q;A3cbnuuxiCgs$4SfZIIY0i zp;gdjhfXUlzqumspsgPD=g&%*itAUFKbC}>8CQSxD70GJj=a;G1s%#%*hsuQz+I0>TJw}y5@xM?V_h)?BXW;j*js=02U#|UNz1X37yX7|La=p@3#m<8IU)6~ zlr9)PC!)Xi=t2u1u81<^!05A-G&Omg_eCc`lA1NGByMRHPh9Ivo0iw;dJ|F_60VQTH1tTqSMBA2NKF?Cmlcal?N} zmVJ0akOG=oQQy`a5Lj_TjRp(!$@>9X>chrbVb1%{Z9fc?Z&e`%slFSJ2X=Vob-bWU z-UG795@>dtK@=&zzQUB{=eOe~bb4Hi8nqu?$XO1X$-YK-HGg<#Ku?PjP(HDo>!s5N z(i=wLygYHJM`r~zTl_XB6TJsz8?o7CL2&i3!t#N)a!>LkDA#eb zI%yx~F-9K2)2NPyMr-fplekrgqsla#no^9fO)LR&>9Q2KJ)BM-fmw1XR@{G-;lKB& z2V=mgu7I>u;sh^j8)s5}i_{MM*q4fF>MBeD!!cb&8&M4w$xXOx%di=x8q|C}C$rDh zw=t(VFCfLX14Rmo4 z>GDsVD&rVQS!N^WV-&TFGl73P5fxU!3lDE{7G&_`Xwr_?`H5YrmW8qRMA$cnu%jH)MkI@jtSxzD7^^Mgf@({Z4%(&YRB@zUM;If2O6|{?;H)=v<(7F z{ilIo{6QDRBmyD?L%}j{-&^ITuWR?IQU@ynMiJ0o5#~@apZ4?Pa&E-J@Q~Kkk4}{t zRlix}69GMe>aM`yP&jTit33IHn|ze(+l)*&8)}ys)~B@1^e%rkR~QsgC&vbiW6&%q zpIuA6Vh8ss^%H9cGZ2PaQ9ObVu3K~fd{wNBtWgbhdgeqhZ7EJID_Vx-z2ZziJs##Gzti_HYB$__=#=&XcR7cy8mDUNr@7*2}wZor_)aXU#w%V?5? zH73-yed0|RY}S87wk44Tob&uWLKzB|wH=j?a2Y!v2e6=DafG7nA~2o7xKnE~H15>e z#R-@f-kj?I+9R;BUBnD9qPH2n-zt-eFS1}oh-VQ#%Yw6WzB{0Ha1YfQ*Aw|Th*Maa z-cTJcgSszq44trsal%vqvx{DVTDZ@kB4ox7M4@@rPt<>Xe6)l-M8qqmH%hN+-^Lbgx(I62r;O{Ente79l2J_?4lhr`yu&%hy}MxmKZY=$4ZWkkb$p| z8COx%0t;;_k5emIB!@F<^1Gxo{MGK+l3e@m#Zs)08ECDv2$|8Qc%r^{dn ztwzY~XQzMY%^^}3XEF%!aB-$^xB}}k3n{LKFPDTz6##@PxC9E94LM+jfIQ{6g2;-? zl$Dz$lw33Z0N_`FBo|_8;I23@KuigOwngm-0kpIT5uuZ~cXX0v2<&wVfMVYj;JS zSJ`>9W*cz><1y=`5y_|6L?iaGsEDT4t|v-_{}5r|v~AQ5PeO~NC3napCfE_z*H1z6 zL|5;Li@1`TQw+yfV%I0G7FxC9Hw+4w<8-+}=k^J9t`b;5XVGXRb^yI|;_W9@frieb zE3$v*J%z%8u$Q>-u2Ah>vr}{!^O6Mm#hNeIOED?VZ>AGukrQTG?qms56kAXeaADuU z2NeqL8f7hjd&{ahVu?+|nB#!B?=;}RCg-S%s5KziM+PKKiZcrSvV;h;(+(FF#mrcNZPmx{BhC44%qVxPn4R)Zy>+b=4l0T zDsgNAtvs|#t`A4G^5+Yxr9ZzB#rCHaD#t&q^m5t#*b8d+G@gi&h%fB;`u>d=O}Ao? zwp;mQKJZ3dueE1rb5t*HsMF+fdLre26xEM7?o=+u}jTj3e1#LPmumdbm z-sEvgo`4_b_~iytoN+qHPmJ$aH?M#A^#}+?MJ0jMN>q8iO#s^Hh;#tnsQXl&z%P;Z ziac@^=--T39p(j4iBYU1$B_3+SziFMbAte?qVl3$s&_c5sU)U;l*vM~^?*l=no5Ek zM$ro35&LIV{TZmeGDiTy`-u^@xauk~7xsu9E5&RHaY2QZpl8(nzazb;+l7CX7sF+g z;J6i=lTX1f+!#nioJ3!0h0qM5uoCU`3485b0M5A##2&GySgODWnrCb5mtB!55c15T zu+kSCB-d3EUNU(1t6Jug-Oqjg{KxG&g_ORgE9ar01@e^VFs|t_`(3uJhz@!}c^qVI?>kRuxvZ zlSN`><(ca3{vCA??%xjuAvDbHlJUq@w^^eW&uPTNiCp!?(JW2pj}(>w>4=n5)r~jj zL{-t?6H(t=L~g(f>4{lxG^9pofdiHVddn#j)otc;iV|ZED3B;zf5(3da3}o=#DGyQ zIiAn)i>l~-lsRe6)~Un5#yH}B0V@eWY~;Qd+duy~oIR4GpTp}vIB^X1y!D&obTyi? z>os7SkL6r4v(;#O4Ll{eqJTLmR!7*}W^86wMi{KH+@-EC{<&XI)HoB*e07^GB4JMQ zL6XXjjbVyHvXrmBXyt#ae~KQVJW&^#PP{>&rESc6$TlZV!<@H%bK(R>f6|FS{^Yn) z>w4=q$EP%|w|;RVTY2^3H)r?touK#M|M>5`E0^(kV6r?gsKzu#WqKYZq~e){=n0WX z2k!&V*E9e1KmC99ssHsq|Ls4|7yiBN%h{3Eq{#li|H;Wm3e11`b8*U$KXTss3x{%v zw_Gs&Hx9E3A8-BQAn0B{-uesY-8Da7`o(Y`{_8)v`qssSbPplG4u`u5=t7d|Mv#mf zrCen~tb;}Tcq{}sL;t|70JI0xAaqK~f(0bZnp-6} zsLkQ^9vETT+U2jD!aM2zTyOn_1CR}JszNj107sD_illb{s3q7qk|nyE%mCmA#se8N z&_L@bzDU3o+&-15f1w$w_ zaJOgH7lD$8Xi6`lcPm-#h?M{ssN(w-69lt3o}6g&pwCTT%N(FJyo9EvpME81VU}S? zDRerQ%pt#ecjm;U0@wWWHdoKmYL;}*(&H648h_Lhl#TvTWcq;t1WuOJF4H9GB8kM~ zL*8%veX4(ynZK7fUJUxC2-xKaU~qAIR=sx7pw9^*z#Q(DlV0Q5h<|TR6K-smX@_6w z0uNs9R23GnFSbiclmfJs8)+=PX2h|xIIDeyc%s@ojW3|A@yYjZRNwCCM&J-|uBVNM z4%olLSwv#lA1GY!=O<8#phiU_N>L-CkzOIY8+(5Z?T*G1F%bU39;*Etdo10GGuCS5 zkJ-@R&Z}qt-W(ykH9}=`4OQadvYH`9(FMVqpK;a@%2X+3kowc9?1@>iq(FFi>c#3S zTF<9>;Ym3^9w)7%X@}m^vN^o3ILDPK6j=X;mW;8h(Fm`Q#U?RR>;-Wo%L|5mOXu%J zF%f_6fJ=i@y@dPn@HYK=754KCvFhXN=im$L_c@!>NJRTyN&taFK+&lZLvCM%G^<>= z?FwjeCOvzIvX+eJ=O_RfDDri=O!g5IYrUT6Pa9BKmZ!)Mu zt;;rf6Y@P5=mn2K#**qaHX$v!JyLs7c3+dWpD=P@Qqr98uGwE~fmb?X*%~NU8(HBs za%aYIsQBxjO9aXzjDEbra=#Bwj-|1Lo-5s6=;P$L|Co!*GNBpujs;BA726pv&+UJE znd0i!QR8oWr{dN|z~r&?%VT;FlLws$xk=~46CpirdSfncPWu)pvzM86m4IYE`wlO= zIm@&gP~OXMw?39>_x0}a_kQ+G0#grVwzzM`;4o{#omJ4p%pJ#0hzomTZ+@01l|{JR ztp&&WSeC2MTC?;{Ihf9EgcaKj^6P&mjB~3gR<`UG19ATR(hW4Q0U5YHTnV_Qb#Bg| zo1=R)J{ZIab>@w>mY@u6ggY1tu){UIl8PW0XV_B15RKDP^28e~o~A?&0e_Zx)csgT zLRpoA!V6~XF+{9zq`z{Id=jK=eB~D-e<#z7gN5BaRHBoLLAi&;4w_=Ro5MB(QJQD$ z?@~TvZL?h334@+4{WTo}O^V|afwDvcK0s=dk^0KfD@L5Ocr8o*4}qf)qHYY)Ci^A- z63>rG>*J{`<5Im!oEG0K0;ncpAe9?8w= z^w!P?nmYA_kzW1%9(EEIF5354Ppzv1`B6fFc%ZXr^4aBVg9X=8HV+bhAJKl@p;Pd^fA_o1<&j%s{(j>=x=IqS|Huy#+dBmv&N1hXS!6lW6TA zU3LRT%frhEJaG)^U8<}cw4?&Tl&=fc5G$eS1cidueKAvrq_gnJATG?QRM|(vBPpR52G0 z$1sS;IWKF|_oPRd@0xW7?8;;8nv3I@zd86_IYYd~8Vyxo6E$noUC}B6w0>o0pJ93> zBN^Hj%-y3E`)YsB5_kJ}5{P1ppn7_2k7Muon62Ete$F0|a8l^Hq_pVYpl|@|$3hy* zJ1#?@0z$asZGaxCHLO4zPNnA0J^wJWg1A5!o5dUgr=>9JGE6kkHawFAus%8<$)Fyv z*`CX5TBLWVQUh3glRZ5x0MD)OaMe~LnIgt2;la++91edOiGYDDK(s3dd_k?UaP|tp zqwXi#cLXBQ2?Q#$=^=>1a=HyPI2seSWdZPZ=&I7ni=#DAoc~R#WWjGJ@Roh9q7+}F zLbb;>J=I!msg3io-V|mg;&DR^p!Y)OOq;X>TAU8{<*7ql7&?|d*_6T9%Sz0&qRwnD zIpa6O0CRtNTZz~%;wDU5jf1o_D1GLCC@$M6OA;4rf<%RfW65Fga^q#2nzwG~iuzoZ zZRPZ@Fta^wD>Ql1K&?<UxOQUU;xO#{hro(bAxcyVAJcRd(4rzVrKo?h6b-c*CoK=ihGX=iQzMM%S!1G8 zIZv>TFh1a6&vxo1u9&EB$vKn0dn)PR5NNC0tRn&i#f=erBS2ya--SnyO4QwPYT1n% zHjBXEkN$y2b&*iM%P-st>efd^C7p-itseq5yoe}ZPegnNeuL$cLu3x+!Q%R)qT?eW zx}ARtW9>Dw*Xf986YiVld^LTOrq>3>bdW|+no?+uqhG@Hk{y51zOMV?2qTPZ-$A<~ z5y_!`j5XUMx59cuG_q8C_|gTMP6r6UkoV}nuj_RQj9r{8*bwS*MYBpMerp>R`uvy~ zp@Hs@8qpVV8F>js{sr>nhhF(nN~F_Y=>z zf*OgCpUg)?sdtIjj4*oUat7pQ#5k}J5A6cq!zuj6AHjZx&iKi3_UU@-H)kl*ftIoo z|8JalYmet&IPvS}tzVqX^N&9`S%9my&{?NTdPdOpOL(hJYcdW4nwe)uc`@wIvTT2p zJrR+y#75C{N{WF`!F`VIkq`Jz?}nLH0;YC{uOJ8-(vaS$j5~jK2Y9@#@S?p)3vuFj zHMd{r621}fF(6gq7Ub-|Q;{FpRciM`nCVL)BGy7tEDHE?6p)NR7lZd9@CM3=%xBUz zOPX%X1V@TI6W6=o+QQ~#51iX>ly86KXaEZNgjpi&#qL`}xNIBg9T$`#0}O#ZvH+Uo z5y2QpL>c?7CUE!H)6d_Vuq!x>XiyjW5T}`HqVGI0MFAj{cDxXA0J*|WlHx+3F|yr{ zkV!YkF+L$(iUIsX1fat_gUUwq^D=h-5!`7c#GrA3dojz=3+U(gun2jJ`bmGOVJcRH z`?3F_$kpM{A`u+n7|<~X{pbS>5M1<7?cK$}$yR>Aw$juOIDD8rG7SEExyR&Qk$H52 zm=tH*@7o^n6N0^>q0vY>f|(UuD~yp_pW*os>B2L2mxxcJ^8OI|Kb&wSzhs5}YmTkV z!P$v$S}P>&=_T)uD9>zGS%ZIbWu2Cfc*=U);>sRd|j zw~0)BH~gPpUEwOnJv_szuW^+h4li{qb8Aa3k+D-YVMO9qq~dH9MisHzb&1;8hJ6U* zsF6kU6GbQHD9(d4kK|C`g;<*--v{H|SV%xG8^gj31m8-E6|V zbqNXVKDNQ~pKTWb?YDX$y0DU2G#LZqD&4JH5bJ%8k{{#j2{1u;30CN`5(61o5hBg5 zh{vvuX!HPx&!AWZWNKQVcBy*o~-rajwB z5Xvd)Fm6%M)nXEaFGVXxn0UITQxA{Gbh|CK=(AvsbG(slW1+@00`!ltmrwLb{XNOT zY%y}my9DI+PEZoUxJRK$s67DC&L;m+*PAw1@J7htTx1Eo~AXqbN%=%7b zwtEbn>?v^(rIS6NU6JtUjf>6!c`c5v@Bo59eZSy54d=g#zOd`#QY?4Z-b{`9uI`$# z{FN4l6b^S{W^TYt}Z0>DR0$I!eC=nS&@Cz{Wm_0+EheGgs6k z$*q|)B16*lNZO2Jtl*7t{k0!Ew9IzYzSi5kDx@&RvqmgkKFy(j!>8Gt?J?kPx?2Cv z3D4ua^_vqO3@*~kM-@@eKc`x_R(^ufyQ)XZ@O4TmQ7+7RcCJa|K#D@soE_Z3}`S2NeC z*^*bYo{$uIH9#JJi7ggh%}(cqS92uSUd=7a1or~|^OaNl_vW-fK)1c4$PfQ0#jLn1ePPY++y+tlt(Aqr$BFNYYwmoWWGifC~6c}B9Bcwyt{@Z6K4wx>yXeSge z4l<7VfHMzk`uo<{$-hu+t7oBydwXHV^K>9mqCyjQE;ngR9C~f)@(w#X?aD!{PIy(p z!n>kis#BhY5(e_cVDk4&<>AC+GlP6&K?&v~EKWclX;Oyzk~U_rf6C)OgiG!b2jay+ z{}QUXN78|Ru&<@5VDmhKb(`Ktn9O)hMT3`!+x+KscLI6&nQJ%?A8fGM{w6j~dhE4?0nfieuJ!{G|nnh&hxE@1~eVvPVI zpnZD!zP3@KD_$jl!domS2mpXaN0EeB19veB+NTgp=W(kD1Un->%>wq?#tBRaJZN)5 zwbkYuWd6d5E%3bc7mf?q`P-+ChG>(I_hVo`?%+}suoG|cIh8a*n>=}64cDDTz!2w% zKu*DbOV&s>1K6btDsC6BbJ`=mKfW$0Vg33~Ir1d;@*!E9BhX@$VTPsp2!FKsCw3#B z$o>EevggRTXZFd7lk6erQBJjiLtPFi*ic&;wDbrZ8nqCf?I-|4_s*OI*4>$Z7qAl) zG=&qZX>x>*J9K3XA_8iXn05SWfdBL;;ipl5z%Jy=i>|CBZ0GMeOysQbE64s5;4R}T zwv3u)b+27C>BpQ<;~pc~HPRP{_2Hmv-=WDm)~S81g5=zVA*?_req%kMdf-2%kJUMR zK6^iqPFk2Bcuw?QKY?SUVX{Fyck#ZOc<8vK;BZ(Rem;QpM)dlxI% z1tJ$!50fBrK4_8zJGjq$J%NdpqTrT(BT)p?@e=FpW@^EF_?4Zga5;`};c9?MD=+N= z_6nD$IV_0D#HHq80%HTj&FRI1Cm`M#g*z%>2fY&#jif^Y4!~91&R#s12~QM%hc1*G zy=YfPUmT;~)i{s@@0x9jB=c}5T13UMpx$0T*LvH-v{xht@_J&WQ&BoJ75fy55db6a zsPgps&K?>A?P% z-~sJRWX`&GDQ;^RF9*jqnqU-vIjDLaL%hQ@W8L(gqs<`#?ex(&Zz%MfKen7XBAv|@ zI_Ei~b=n>)6i)4lr*pc>wI2dKGfqeK7?~cf8sPY`mLglYgr8ZnBiF|CykgmqnN_0H zMz{3D+(hX*7(KGvhC}qRm7<@B>5a73r2-sK|EUxNI6Tf7O5q4SzJ_yu+us*QrRyJz zv50$^b`e`8{oIx7WIPtWB1l)|%T&2eAjJ@^M{m^FFIc%w5ZHbWDgladOXm=lXSs@S zGM{ivG7HNw4h{Bolv70e`H;x;pr&;o1OYfAu%p3iW@ zfFc~)^am$g*XujKIjUZN=T9%2yu`kJ&Iz_>2(NT&3Z9b3LGEV#gLCvx>|WA77^iKO zQ$KO+Yq_3iab{D%bEub%=(O>5Bj+llx-C>HV84XS1?)e?j-t2^H%9^ct6iLZ4o{__ z*6`G?x48bwfu2_kv2nUTjD86dX$&nOrbdXE2mX;#t$xsYPf55>S zaJ1iR*Fv+mU}bf~A5=I9j1=d(0A7*8%0V>gPC_2HDm19jLfZ^pMZxCT-#;rV)VAN8a5T1fSizTj{1n^6u z_Iw7WX5Et^{KTCyuU`URszC}J(VLM<*e*tF0XfI^^NSOIVI+b{Kkb7vok|pkaKhPQ z49p@(QWUEThqLTs6l`iqk-`!R4lR#ixfixzjsX^eOu5Rmfx`(k7%|E0@ppA6>({Rt z6gKGR6l?f*Mv?XF7srT)Dg1Wh873#kqkf%Wz0J#SaK;oHR{c8hN|zg5VhX2Ma}x}w zi?o{tZpO)fW|yxMu|2t;K%jy!?AYtnq(=d~&{Os#M~v^9c1P=o5dBfmPUOu#H;2Dl zBEq66R$XV*uY2ceym{vC(Q>Zm5ng192#oMXb5!Tnn4f)i>SMNu z@Ax@;g5nKT_kU5p-g?>v*>s=)8RrAMTG|MUJPy5o0SdqEBf@r|2uru!AHmec8tPr} zXmN;3@Ib)1yUxr5UTv2I!OlX>SdBUk;k!jvLJkxm<(_DE5w@N!*$oM}jn^pvxDD*l zNZOsU|3u8W>K!u)W6mxb{ilnzZGLf9V9vD-DxoGNXY7S2gd^Kg8;m*QnfepMI0LmU zlr~R)9U`BY$Hp0&3l*}52Zh=4;X`^*pyzR*_H4(Z-?W`2Fq~D`oQQ+Tf5LCBI9my= zu!FbupU}uCGMbuxS@4{<;fh@kI;%bWr+TW4nuNd+MY~T#>5KRbV5o2;zIel+*lc=F z(dH0zt(B;xrT>JVu#f~wEKbRX*euS__QKnL)p~nS?UbhJ?=!4ndQcpRuj4;~Ug3{H z2V*Ji*nA>w56u1(7vC<#X~J_1>4!$3 zJ%EaXe6lS9C;T|~dl4QnxX6}& z^nj|nVFZXx7&@a1dT!Z7o_e8fXPD2|2-;|E9omR|n-RGI{6yISi%*WO^ZDQiXcu%8 zI&lT|>->r)pa*UPpz~RB83)qm0b%5fx9E7X3_vq=UxY(dHwEGtF#t*REL8r;5Y5kmov)Y{2`}?8UcbeLME|q!V^M7r3H8a1#cS2Mh9g;?vrEK{}72gq|oqo ze5e@RE>7OSRG4L-+!2QH%m-|w3(al^zm4bSQSS+sv?UK+;J93t=Qi_s&Lih8ABCx<6)WeMcvz;(Ib2H$K)IOfro+PpT`c;Vb zB`7)7QWqBZwi(AfN)dY9p7wy9{2>Swj5mX8FX!dvoT2TGTP1;@;ObE@dGcJoeIV&Hs=#1Hsx zHyn*HhH=MV$X|_8_M1Oa0=+_HupaH%D=P{IqgViu3o>4+z#X z*!aHp{^`D!dgA;Tis!0Q?{S;VS!o#it7^UHEARNaYoND(4a7Kq=Sj;Q=RZQFIR7PN zj`RN%JBlK%lCehuIYb;T&c?qoR%#g4QYodxLUaTb97iNrz1$JmIbjr#F7pVbSkm|9 zGWEC}2>;W0rPN)&-17`jC@L7xHO`aObw|;S=e))rw7l+A?2J4qe2SOOdhu3qRJ+PPOl;w7P;68|_iP*?zT64B zaLSiEFZ+giQ6?H2=(W=l%ots?Q%V;HDW~fLOi1bbajwt5 zP@{N%Dysn-_8_;b2ayy5$G#)eW*CFML1ET4NRfBrFqV^<>v7kSGk0eqqJ;T_!~EZw z(8JjviAW-Dfl!9=2#iyZN1~LYyd;i}(NCFVg%FGcbmdt?CeOubtkVDt54!fSD*L6o zy7sK>n=Fym&wL{Brd(j#B-5z0f}LOMrvG7oDL*$Qb5Nbla$tV}#oG)V?%!aUIbw%y zMTyG<-O7s|Q6s)Ni|1A13+d5Or1*{W7{`CNB0WX@a4XZpi;;FeOrsy}>3+f^vA^K4 z+P|?!)2+Bho2~pg9~#}3KPD&`1i-2UgJP?R4_}`srpGTNw^H$~Cus(=3-=ywZCAg4 zY)jYh?8GQ>Rjxk(S_(XV0z)StO*H~^if-i(dQZ>h^f@sO~G+$2?g|GRAsi7*%zcD=6Nl&1_?CXqm=LL|TbHN3arXm0j zk0$<0hNe8Z!0KhlseA}L1X^}KjQsY0(s301@m=>&#m%1>dZ5GyP*;=!^7}C=5I8wn z!6$sXGJm3wRc|UshJ0@y!=Z`)PsX1T6mHn>4fGlMq*Kp4>R~H zzd70G8rfMp&X}v1;bVAp*5gN$Hx_(r(>;AqaYO+LKW@yvRO8Tb$bo>nNvmVmTWMBJ*(mtg$47@Y;CIfq1d5;S_C z;VP#f+v25gzIv6APZn0(G|%h2mUSc-0IsL3S)A8{1I=>-pi$fhQ^ZO&v0#a%*IrX7 z!OKF$j0FVqb&Pw7soLN?;43wM@kPj$Zk{Gdu`9;^0Ry64jAhCd_`Y9!qX1|fzbFnG zne*r7xEQJ!HUyG5`At^MfNW|5y-f=$8a?GJKfD3^3!R`(6dP9V(W=<6aD8X)*Aptt z<9?MmSK4Z_t`x4>_q3y*D3Z*NY!FW72R&S}e>gPvY7UfF=~(=P;-XJ~986(tzhyKg zcb>fn=enikW4Fq2ZXlir^y5bbrJEe zzaEw#=fH$wm`L+uRfEL{t4@nGeaCDmQVX52I9m#tdi%1Bp54n*z5gC3*`;$ZZd*C- zkyNhiA*?}F3XDx&A|-EsOP=1(K54ZZlFL%|SD?E}6!R`&UOX^%IH=_Os_lsdqy<%N zdQ#XainwVnS-eprT7Btqh-Ph}L_&D0O40TMMgBaczx|)AeL0dWJI|f}t)gui9h`@C zXbCqIlF{vG|Aq=U=YY(rcJxc}jQfDj%;EronBlOi1U_#~5O9#qCK&!2{_=XMA zBxnd@1?9#rj819GJQ5eet-a2j>}MHhY_MI6jfvj8-Z5^k$mh(Y_{_-vxHc?%dALy< zy5RW)%_IJW7lpT_>%B5A-1`>t2|LM+g)b=*JYcL%AS{x|-uF^%E{9HsyI>E=F-!q9 zdBa~H6UJahbRTkmqa4&UiC+1D@vk;usPJ2&VuN!9jId@~nACx>Xn5-^+6ZsJVDpm% z+?cCmnEG|?w+Cd|R>irl++H_B8N%4v*>3COabWMUvu}d%Q|%9A$r}|g@7D~Qq0<<} z0E~blj)jT%2;)9ZqS3h1P3&!zd9udLFlV@dLkeiB3y_+Bhx}W&Rs06nVH+$>77Vkg zP>!BZ69oGL$O$zkQ-$u^ypx0kgXYIMaS+`hsvs3b-!6>TL-=vb6VWykbnPMlM?h4D z$#%^LQ3WGoqs>^#LJCX!@WZD1C&4gZD}mrvJ<(`OwlB1MEi-&_epub8lic>(q6S|= z0^)&{6mnyKGPqEMu;O(d40f}ClN7g$S|#)x8AUXD~kq-w}Ew|)<*au7tk{m6R$UT zISxhnQgR%;w6L#B zc7LP6CX%X}SMoNe{st|AhVkwV73!MxNwFRW(nqbOi9esIVGeIkYF#kgoo;ZR-Z{g6 zi$M@ZI}dmK67zFoJT~0n{AV}G zIbnmF-YA_D@g^*WX_zQPgqhHzgN~szvl_WDFxoKI;?Ncyb9KX+lLYp!u)%|Wt^Ulw zbUYFJmCD!q{-JTygtVKww}SsOvfSDtxJXN?e<+ME7?shj#yXCyKlFPl^~YhkFdlRS zjE8$9yUfrF0oT8^^YLQt9zSj%+XYOB!iK3R_n!5Ej7s}UFiIL)x%t7-uYq#(uH88g z{zAEW4jk3!nH*>ctN--6Mj)|&4DZuz1@Upz0RQu%`AeVlCo<__phP_bW_=cFzfi%A zT!SpefH5mGgWYh4mU3YRS`Z|12`~uBFVsk)(xJ2W$n+8ma(`!T-g{%qCyA7~qs>6%(B2O<~ z3@>%@v4V~f9d|Y1f8L7$76LtiTr&UU_N8>Yj*mbT0}<3liQR#@QnO&vh|dyDMA&_a zn4cS?0+hDLzJp$9&i6Qfp0E;2i5*f5ZrpNMS;6$Zx5Mbc!EL6zBpT~yC6-KzCx$MT zrdA~$m}#(6u5B3HC$niwvD|QvS#uQd+saDbiG{TdV|B^0tfVGhbcc)9$`>;yv32)C z99Cq^blz}cdyY3|^SE%riD$+~iiKwe3LYkJ z*IlaD6N5S-Mqx07#lp&;8Fd%@vNNje6afYf%C(^qh;s~YHsh%%Jup8{ zj4`2bi1d1KA?1jT zw$OF+dmB3?&Wj@7UVl#v<8^pE;Sn3z4m8u1c{&`v>(1I z`4%t_=Z3uXoRuGPv($GU(29;yfoW(PBBqJc(HKGD+60=!bRAGvH=5t1L1!%leNka! zGEqE{5cltNjDA6*^Y2rwPnIx7UX}jYG|Vs1Y*~sX^cg3sdz@1S( zH^z##nwJKOX2%3N1z#(Yk6tKE3}T>TP}5cpug7oW#i9+QI(o+-2#v#q$i`e`XZPrT zF$9~htAW^C*mw=G(Oj(YWEJ$UT9Qg3pL-u0N|~mtoV{K++zUxNZdzO1TV9af9pKt) zD1HIYq+Vcw$v%q828T3ecz32j{V943!T(7WsxHa+^&9=lwzP<*7HAM1yzDlbUdcAa z#$DJgmvFFBR3oh2Hk$F&!{Zw3HyV)OU_xPqu-ll1l_vYiHO7*K2He`UnCLszEKI4bc{ z@2dvU@#@4@6D=J6#X=*hSv9YRD zP%k+zeITUVSnH#-{>JDfE$SyizkJ^Lz=XPUW4vV(5{9MibcL&gx?>oLbju&$HHtf? z%hMd{POs-q^Da>J+Iy=7{edmP<(_%$qr^hYX!z`g&va`Q*JTW+b3s-l{0V{0? z>Q0v!M0KaLxtr7-bI30K9qP`ZVW~TBRE@qdjRJvh3fjhG$=6tLOO+1E1ekv0zUCch zok#V@GwG6N&@YLJi?RRa8F=WwCgM#h3>dOGF?zlq8B`dASc#^|ct@SH&m)gH`m}dw zv@)FU%OZ{O9*hj zoBKRg(dx;EEK9q&WgN>OVs`V*jymEYo7onJ%Pif|e8PHd%WSh8%=m!p=2nkWisJV6 z0g0L^8c_sHv-yRA*?dhIg4}HGl^!WhLAJ44m}2q;4G3^$(`>$fSn~Lo&8^L~e3cEe z)Ggx)0!7{gnawR~Eb|`og>4eBAU?%rhu!?f)b=0<+Rb+t{lac;+3hlc;_5L9SroYz zNCf@?TAcS~!iLna+fkr8C}dqiz-HsKV>b^9q1}9!Ih)Q_5f-P{VeUS{*PkZyI%=2p?Q9n8;-k=fk#xD%7nH6oty&IF0pd|J(| zzNT)EjxF!HlLk&ky#z(Bj6_9$f#}J>A-B)SDxt0_v$@6C(T-lw zjUj74&1!D(b5nkiFLXf2kC0WgX4@OpVCp4ZY6N0VIz@lB< zELNLsbZ9pB(xVibHn_$tRA|g>uA;8z$XX-$Brz-dcF}MHv-#Tkg3RWNMXY5uU;Aik zsok*5<_F_u^TP#THeV=t6IOF8$l7VKYUnBg#|y7mk4`-{F-*H(O1|-k_y7n7*PK+< z-HvL@3n1)&`erwO+ZiN)WHB0=8ZJyj3WH+6FpQQIAtYNvOG`8H7^I~1Cvb2}JK5>x z@s_!=!<&kvA9{G**kGA1AaofUT4F(L8XKAiAGxAW8o5n&?8IZBG&leNJUveL0O*r2exZf$5P#v9g#7Opv18w_T2hFcq&hn|I; z2YOmzEbxVdD-RcfEp&U>Ty1mW3j^c>tq#@(-|i(vsE|OqJV6%oYkpE#Bi-6yFwK{= zHe49m7MeRo^BV?I8>|iXD^0#_3sg_6oE>t1gA;!0At*SRNh-j}xt!^sA($WaK%*M2 z_uFwi)ymZ$^H>`U=9n_xxM0+6KNz<*_XRWupCmA?0r>a%DMZT@trxZLpQkJQ*FVc^+ef?GqguG&UH2 zb5AlA+5^Gs-PrJc8~6=Tt1egSB9qSX!YsDhsh7xWG4@^wg!Vrhn&;Ik*vY42 z7;R8SKF?foS2Tw`i7la#0p5UE07=haC_Iy?Gn=b7sVjKS&^*FqQ=w0hSrK;(LPR&n z^T0%eG|AAc|LH-7-qNxiGPL%0i6ZuYKswK^=XfzWtKOQOW$e%f{ehv1czxyr;f>{g zJddc;|6*57i2zL0>n|)W?3|%t@~nt7|3nB9>($ba0BYWoZL1OP+uVP$|fr} z83O4hh(` zVIt2qeqyql<{1Q9b(}$RH#Cdbu9&7C&@$))TBrDlSV;~dR+@u++Y6-s)N5kDHwJ9e z5;zL^?7}EwqFtF&I;6gRmCj>-p23)x&S=XIFP<#~IR!r8X5@bVLft&Z7=n2YBu+I7)WG)hvB%X-mlI^ zANR>IQ)9XRT8~-T6-6yB*q4RH$WzFXsZA~VavzFECP7IPa4>t@V6uXhGU@e1#(i*n2h}Q7J&}EL2FxE(ZaT1nJ zb|?gaLI_9v0q^R$UEk=e{v^U}=rC4}QTHp5pxCSHTf zrMexTvCs(#(bwc)>is35 zc`fNE5}K{M@VO%8w+7U_3=D3kKV^Xt73qo@cnq=lD z<)J^4a$H1>-nwSqNlO=o?j*y|d+TKA{c$Rp6hKNi^e1^s9U0}Uyljd)!~IDH55Xu; zo|NMSf08wS?b#7On;nL}Mt_na;uN**eSg}K)U7Shk|D9-7xMS4*8}NW5xmP=9&L_? z#J-Qi+N?x-$;{{$_Y3aYoWKk;9NTiVn>Eb9b{nP*u_%;TO=8O?Ow#D9cRr#`nXV+m z>@YI;1m#%rrbP9a7^eCjg)Hk!I*u!9_ph_h@+3WfFz!iuU=~^*(G!j;$0_^>(OhaA z5qp6+rVhRJJV_&>OW3?hc_QveGC~L+j&;jO;j5w}>5v8-N#5Tnc#P9Uzr$6$lU8ci z6GhsQ^D%>q{7hY>5`onfd$Yu-GP+curjr~=hDqgMer}8k?t4io+UWc%`8;7gg^rmK zo+R&o@+Fg9M{6s+KX4>yU})-F)x?qcL_dtLW9@73JvFXE0R%Xby6VKzk<`Q$>3GNab!47 z3q9N*cZr4=yyJePekWHfG+3j!Gevb`*6<^L`ED(Go~&hq(Tt)!)+QBYC1iAPoUQo1 z)NYu5qzmJHbq)(~Ka#;iJ_L;^B-qw{)C}*d;3@S4qjPPWR6#z`uP=aNM!(z;&dH?@ zkoA-8xZ?4?HQtb>v?CSck}2i;xubztYStsfMu(t+VsI3A99B-{KuaO{Y#{J-E7I71 zj*-ydNk+vsQkjxB4>4NINZ#oYrZ0KNFtoyhPgteib<7Pt@3+7>!@%jHCzdij!ON_E z*SL*@hG7Pq3q0273(+8y*l+kY~ z>x@*iQwr}+SNo(eq#H%|bk2vx@6Dqhyq<1yQ|gKJh%N0_5=I&Y#D_LS>u!MUVeFb- zR;FA0YZITqZ-j)A@#m#zR$N*oY~qbW7OMIp7G6=RqPpRs;2tsLddT3%Rye9iaPS@?Lm z7+}jsl=IEWM`h&pJ+ml!RbDc;uU;XXV1RWgDFO_z$_hyaXsz`6aXH!>!o+G?$+9&5 z#Kakk&wL;XN;=vv(~*dF6Bg;dseYOGT=#L}=E2z(9qnc-sOy{T&7TkPl0$WY%}O3E3A338f)be52J-`=#C{Xu+td})*abnA zm&C}eGSj#QxpgyY8ixtsiC%P9H1D?(rrgO=*@g+JzM~6R+-pXBhu?{lzi~k4ien$I zXF?adz@RUn#RZh(ZSf|5NmIu3FLq&~h0nN!*-_K+_jEHgNeyl4n-%2N&}ofQ9lp@F z1WW!xzl=sUggzx0Vu%N(HU414r^CwRzLxV7^2VG&z+ZC)$qk8V#FoWG^#s>JAJ96T zpU_Hj5V6u67;iVOFUVzJHB>7CADMGcmCowOn7~+#dfdH`*yfwzUtE|Zk?l^R`Cl%t9W1OVHLNIR726k zudf-*M0A?P`&=h~8Pb}-&iOEl_jwuZaLQ(Xn8mFvm_%CRSuTU#R`KpNa1`2Jz4ut8 z#rYu1NuI#99~fN$vpuci2O+EYg>kEe!t9Xwva9G6+=DxdG++Zew&@ApV zABy4M{={V#x21ZT#SKFjHQKe#aht_$t;t?QS(EJwFX6rH`p7cbv~)+|OF9)t%Q8ra%3FE1IUGAU>0TI&+r(6VgFv+BWyeNDyrd9a~3C{x_Ld&@2iVhars8Ph<_ z>x!!Wt4&&ey*1ukJJJwgdD!WM4%&X1-92BdVzUv!*>MZLAMJXg$}Pw*R5_`+<2!sQ z4XwRY^})E^ff;Q0z;tv+W2&^4lQ0V~?@!elf79Fp+dK4D+X3bY49Fd7?|)NS;#l|Jo*jI2Oh1Z(N;_Gm*R zpe(K0T2Wer^NQ1nm5S((?kkzEXc1fa^ibQ)m$WX~;l%cPu%U0agb65R=6R?x7N9#E zqn5ybKA&{QHbF<>a~`|rxvlD6GtcF z!mwZFrMWejoxba(f1b6J0%ONj;Z`OWt#$0*{>B9-F%@DG8Yikl0KyFoX2za0O7o*7 zhC}%@lFpaA0(MrcFXIiJ_bp;_W2-@aAferVUsL&~t>YHPgavL&L&_p^MM*xz8#)hY zl#;sdFbip*i~Cmen*t8KonKe9FrH9$v-7COec&-Fg`9D8Hu3~F@e^sfu(zx@|q{9pEyg;ol4Nmst;8`OLmVqC6& zPd8|8zlb0-Hf%u`2#rJs?It4}NA-XEnvbcnn`E)Nlg)!OsqkCPm%TA43m>$7EZe{F zi=y+k@MT)k^cO2$+r3)$K)D4Q)_>_m3@o|kv5W>8HJV)H*#$B#29`WDd~KA6=b9R3 z;;Q&Ttk&y2`;sMJb723K7qN@mZB<@>=yaZKgzMHMA`Q$`t3RHyemwDg%UaMgF|hyp z7j0m#&42y(zyIfda&d#+SDxL?K)?UVZF$S-?5&(^(c>92zc$CD;uT)cGq2J7|N4}T zFMZj6Holyj-8gwv_H?;0ujBmXpxGJ>^d4o|Tw5=&Ll<*|(Wp0`8uox!#B~^dO`|LU z#NdjG%`Z$bebh1DZ6w3U>D{wB;0(|UHoxFxN}6NPp2X`gJB+ffR#Ylrd&5h*1?K0% zlg5p!Y1vXyWeukuY*+I*vf=s=_b2H0tndfc>WS z^zQ0`kI)LdLJ1&fS;v6oWpCFtX zej#T~YwgF!cVRMxq zd)@-R_Z*me&%K2oF!Osn5epjxjQ?!SfKk79+bQtAY(*1k#CVbSoJf0rUS7$kj`K!Q z+cC2M<3-&wGGGk$o)<FC3a@qcsH%@GWN6T*!hw}hDFvEtq8^)PL z^IO+pKPL^vl7@%ik!rKj=q)B7dFzuC!=BFfBi5 zqBbx9?pcNG@cpC4g|2-v*Z6&H#phWf%xB#h?Z-&M=yP*iQ&|fAPf3rdc(9 zQRLAGR;)tb0h3&wPv9W=Cw++C?-e5mdP)3-H%cILi>uNr?{Jb?o7qQTes0V{=U;5_ z1g7AldcYgRw@eCuT2LN+pt&55-wFvf@9gRMT_Alce(@d=-( zcVEV!J7DnH`ORNgT$$;_YjER!dgldm5^{ToUf84;Z&@ZTgtmtl9DMk}rygv{66YSA z>j(isW<_My0pUkPzB`}(xZFCAX~EEAb4cMzgyhdALp-F~*azyY3abY_hm z_;((`P6@4lpdDWB6NgkRyiblJn7sEJxS!~L(fcTD`(J}c?%=OZallzW2zpb}EG44Qr!+3wJ zPVVasvj~$YDR6hI_>WcbKB2vH(H1vmkVIn>2A0%+_rTe_gU-$S;LlA2@;GLbWE;12 zUG&)}*7HzscTRO4SWr0(o}jG6Jtd2KVJb-;1Kwv08?dN=GEC5~V}3p;`p2n!RtnRu zy#Lk2bT3zlPa3M$KgDVtA) zvvnzdVhqTrHW$1wgGYEj9*e5G7j!I05og5*C1*V`Q|z(L!Hj48^TLWxj}!JWxvDb@ zCA#HIHN%mKNnud4oN0o*1LF;XjNa)jq z#Co1ElMVz`->MpfmhlrPKni|gXU7~78GE;XZ@6YOytyzQcZx4adK&nRkX?r8fg*z| zG4Pn33w}&$93cV>i6(qvrVu*3y){CA{Q@spYcDf^Z;r^?3Ig14MXd8X;s0R6ot;B6 z{Urt;=_DDbl^tS(1(fbDNr@cy6mxCZAF($V+C-*DtP^F&6Fr{r%h25F(#B#jJM}ex zZ^|{G3c9pQIE|RbzmJLG+%N8d-P^?{8<>UWq)SYc2gZ^dGHDsKwkl686_9mnEH5ze zg&@w!<|hCv*N_lk$gx~P-7P)ar1$xGv9KjS9&2M^QZc;M85UUx#VP}?4c&FF7II7M! z4on+iU>dI0xwuyCn@nJmXYODEiFODL0)wIyRkqUQ zdY>Db8<_@S@z+gT%M1>@X?gBs8k2A)53fA2G;FNS?Slz4xPDD!RF8Xq0h-Y6!N{oz zS$#KZDNY2NL?KOR<1V^1A&&_7I-1a~p-cHyuxn^aye!O}b3c%d@Y(?DY9Q}lkp4pv zS`wD12?ZRgkT6HrSo2)qZn+WmY#> zS4q_*`jPs5mW@x?C1b15sQ@i4?BMN$g)O`~u+6dEVOGB}Fst8~wXxw_3KGEiFsmCz z_T`QTD)0FMP=Y{RqL*OD1A}Zw4k%TwDw)-TJYZHYRT3+I1PP~&hgsdJ<9@=Z<1(uk zs15m;)k~pGk7jizm&>efTc=s?tj!z`v--ivtnM3p%<2Zlp)D9jlg%4sZruDhQVqOx zXrI@7nALs5^ZkUUo@SZVef4=kk!&gH(5!B|Enbs4;CY+XiwRekS>52ywcQ!;cJ)Ifu&di* zOYzZprNNH$S9q1muI@xbi{<7UlVk$By1~-c@&$%n{a|2M7Ym?Wz3|^-S2xVK?CLhg zdTw2xRe9XPhS~dmBF7gRNv23M9>LHr!@8}sjQ@3ieZB9Fi<8A#LGQ22^l&gRJs5l$ z^GaKBEAR}q$*GV>TY;jR$F~;pDVcn$PbP2fGs>I(flOnEOb^A}^s=1U%q$m&<_{mE zyk{YQ9*isvhS9u;xtt?@6ST}L61E2$R+e1=9IchG9s3?MGSyg|M)@Rmv2~gh^#r|2 zv^^MqtZNVYK(Jo#!Ii%2gwEZf7484XW>2`0{Ck`>7I;J->t=cuaWSmP$GQ?`-q zp|sA!pE1qJD>5y>cs*%?wg;z(E4bqI8@xHTh^~}>>-K57wPAQDjj=MlXacwuV0AE9mq6x} zFAt240eSw!DyA!x#$ZW?yfxZ!;^2u9Y7D+GzopeIQO2ef0;@>Z25k;@wxMv!=5R1H zF&HBC2V%(^KP!Gy286T$C&G1$*tCu=D6}5V^2I|VcRe@CT#EC4qLP61g%WNl{YD9Y z_j=A){6)L$SWg%xhl7F1;cSp zE*Dra8V-RkT(0de2McdQ-%r8DC97VLR7YiP)UYJJW321s6NB5OpzxWmsRz)aksa0; zd|-rE*csPg5JvGH^A5{jUe45cG5IkbEiRRr0eAT)yWSweRXv1VpIZr;9QVC{Wv;7R z`WO_6b@PoG8a$dTyS>GRsnCS|v0Y zzgx^(bD3d_nPm?L^f&VN%{1kI&k;*p2rfMwaqfo^*5kIxwj02z`SAr}Vil+SzVC=@ zcl4D{%%CUFf0*t;KM&!CaQl@Fyz8H~Za5VZi zeY<+x+|;Jl0E0k$zo749IC%w4G4ArfRRYk1>YNb0t#%laj1c#tEkY|6h%nP^e}$#c zOtWd}mYHVD?G|R5O`E68G>G^=W}3|at7E3wBRtn`P1p*VpqXa(h|;8)X7@NjZlok<5Y)a(+wN=wZqe{JyE`xtWxaPXMHiLA-=w-=1N3ak*g6;6Fnd8)QCB)2hUX zK#mj0;fGJi>ErQ$L>;oTT`6e?!;|OTtu(ucmpQxKIc7{F&CU_UKqF0Hf5J$!bHpwd zd^60k1w&9^Iz&;)hRe?nvagZ4bAV%W6M>>92y*X6dHYhj`BTs~krk8-8tnlJEJ zXaYkEjn2MIG@JI#W1`tL#8@}cY{qg(1hTd&Pvw4hIYRKjhz~T;%hxR;87pU?TQL8x zW1o3$j!WN`%_X_m_sg1ok6Zca}E{G=3MiMj|Pva*2LfviE290i8ecf>qF7B z{_q?`_VYufgCdW$lkYq}g)hvfJIOoN{{e@1bH4G3k-%fWOG|897{;QGy)kUXXhYtq z%5R~(WAJS~wxyAKe+rDrm%ranv}3OxfG2+vjLG}Yb8SZbI%Gw8=fIM8o`eT^r!a!q z>;5<~mV|o)n4cS?G^q2|C+56GOi;N#6FVWqS$MtkiMfW{`nALM;Z~nJO!yq0gRnDV zpdt~Rs9Gh}HKZ~-rttRi{L5Cuf@1fH!HHxDPq(*|B_*36f8))H<;2KN;u?P<&i-8(|7cg#pl?nk54TbAH~Mvh)+Qy%sK&ra zB?)n;-YTA#e{x=UqH-CH;aT@{Il;*IDuX<=b+v}@hK(t~Ke%=Egn;9TOd(WTmx9@n z@zoSwFeg&;n_bFYl}DUxuiRRlH7;NMFT{1Q_8;It9f1Lir2lWeY~E? zrQ-+Mf3Xbt5ZVgqFFjyMoChPWEMo^EIpQ~rNHL-uNl6&QWiE&boW0T%UKzut zCrTKR@P!hy&~GT5Nuyv8wgws5*P0g*klp-se@0)_`v=NmaFP!}{x2VrjGvIH4{=H8 z_Gvzz7-3_<7iLfsJTtMB1rp;~>f{XTxfc+-XL}mGt&{M>=zS0}de@mBM(?UfZyLR= z_3+c^ZJma1qqo7g(YuaglH*#a}%j#W6yRF`Jw1?F@;IMihOz?Xv ze{tUb#>nb@FtB<%d|AEg7;jc@!(3Kx0|#}ZLHKF)uBapuR&Q_Id|AB>?$Y17-ngvZ zzMPlUyN>l{^|lSTt=@H}ht=C){URnY^TX&}XMPyH50%L19hkG2qP0)V#m3i)^@N}P zVBn{>{>675UR{IH03nJjG6>q}tO51qe}%UO)%#)Ov!>ho=JYE^~hoFFtjO6!I_v1AKd`*?H^vY98q|mrr_t$+mY3dtRW=Wfe`@or zx+uAnTp}-hRo;Thqz1^S;+NJcVRV6)-WTS|5U7!I05844vbgflE@j!kO&=RhH@)rN zez@rkLm9;fr46US^oG}yB-|SBXT1;Vvh&-Z`<)-)f6R|+W6|Nc zob?8;M}B~<41+*-Bg?q*18iyYBR{|vKVRN@XV409Ap=)^hryp%kfy+PgX7a%|8^(1 z;L=hG-D;rlPhb$!Y_RkUJ7)$f?j9mTODUvVpZnhY0;t8{BCQ#x!z`DyFgl&>74B3Y zsaeY9PU9O#*>^k39+7|Hf9t=+9Sc+K4#wR*V9ue^3e0h(sjk=kBGh@vyBb>ug#9MC+%qK3&26RKe=nM9U?2=|p_JN% z*$^uBDQDw%&z@n0YcU7Z7D{j>ykPsqmbsx+ECK)HGL`!YI&d0nf00!N`Q~nCe}hJ< z2Ctm=W(VlRyfZI~Et~OH#*JfsE==>wgO~|K<*zy!HJ+GF+es@ZONR|7#u@o;tdWHA zRyj5SbGOpGpJ3Pm%Q87}PppI+;c-AR*J6KN2WtBl-i{oX|5z`7FS-ovArP+o|J?6< zVnPc1C>7SammRcYe?Cx~z)S!Mfhc^Y)J4x@di~$LpSeqg{Xc%{tyGv9(%ewvf}mw7 zF~XS`OZk9n*bO5SWjP8$c>cg#Zy_-74@@s4tjrAU8^hlM<^@ksZHG*h3sgi`M{NrY zvG{`L^A(=G&eA#sn{OKKMv7F03}G10^r^_CbCtLPcJZ|*e~m$y4Bavq`r=0SLFtR@M$%JRTCD+C54MUVr{`~}%j&uz4CSn0n{!uQRgFl$SU z3Q%3=85`a~VnAmG?>eiEB6Ef36Gi2`G(CM=Gh+mF(J~0^`0xwKrp&`U+xNoTw9)KI?T;Yv{ zo!F~Qkw*=HTfVCn+d$Uq#e#YxmxFK4;9!eOZ)|c_uVc}Byy1+rBQ)&dw0)3^>50Kv zOxAy*|CmJ$x*TkigP!rYS99xAjC;ike>(@}=Yym{=3@mlf9KL% zj2dUnLBD4lIHXtrZ;ujOsnI0H4uY}3eu7m(SzyA) zg!KQxAuWKJ=d&`qzfy{UN5U%(6!P+k5(VJ(`#UNWbnrp17qts1g>{VsF1*bH0d{8a z=3Tuxq>xCW;6UNpd-;hyg4|r>&?+6FWN&Iuc*52>^8hTuam2V28D{#1e{fyf%1i?F zcx7#Ni-^O(bd;NS%;--{<~eeX0fHcd;V?S?OyPP%)v673atzK=6DFqGj}$_JDD!?| zgmmR%#zwhd7x-|}HiLW`!f6bqxCII#82i<aCLG+VGEBt zMsql~&|dewAYMONy!bqbHDyXgOF4}xlXEb_IAxwiu$|Z` zlhTIvw@``jFzHmtSCA4ByDg>S@B=CpZwvxVBWrtH!2G>2Fy&eyipk&gc<zbWbA%#qGQ;z&Yy|C)e)`LR(G-z^Hnh^oq&gCUOZ&bq z=29BPjyPilu`uU}9G87|n8tX78v7dv9hOf_JFqBH{M-^oChR9cm znxr+gua-iQHL+q!2?Nl~nn`BZoTq=k;7V4x2$SLWGj%fCjlavD<96p#$!(@mUkmE1 zNl)?By!gMt6J;7WiTw2y9cY94XOB$CP~w|`$ZsXbP$+WNzvH{LS*Iz z7#>C-FThZPf09Uy!XVW64TId^8rDY_3s`*G)dr@g7n(1TeXR`q+LZiW$U7zW8HTEg zY2*f$o29f&N*c7%8K(0Mzd~Ckk0DQDdGd>EnG0oKPoOV_f1twMAftIt;_ zE^0aCMc!78>@S|E3@LZZy~-}w5SZd@+sy?;4FVEZ^C<2EHgrjRr#?_*DD<)zmCoog zT8s>CvD&t_w#~N{q4>8AhiW)Crr3 z>_q{Gf4%5nWRX!RlfN+9_ju_Y5!Rv7%O@C*Yb`$!@teK;0m?>15C$S!rbNZb*=Q$Y z4g*nPfTb)%245GZ!tn_6ivW#=N9O3^cBQcg* zhT(&O|HI(U91G709f_ip93Y0-yzM+;c|b)Y2QW~AF!SC^s1`=-*5Z;zI@j>5kO40{ zGFGbyparw#7>(B}7j!fVL9yhcKbL%rP?%g$7&F`*Od1}G0z4cb1~1-!)4u?teyA#A ze`}J^R7N-+;Qg= zG2{@q0{c2V!jBIlC+Ye)uv{Mp1J{RtfrbV!zD<5RwJLP_wzIW210O9e39|Sx=pfY- zj6iIvny`Y$^BVD1%)FPmF@uu47hnlke|1QnMDJTO>3gK_;~*M08|glDe%K%fua&$t z>@0)#!d{j8B4<_l>#7%3#}gD0>HP5O98`JYR4oZCgXfTLs6iO#l+KUmtXL|7V}wcA zY7Izafl1gfGM}JQKnnJ38)b0fH%?uJdD(;wt~9s!Fc|@PY{G@*htCtz#b6UQe^@dN z3&G5j@PT3tBcvL?2WF?CQ%2#y9GPZTIxIghaTmpBJ`ger+b?Thog8Jr+8DRoBGh=M zh;_wUtdNL(sm|@lwY>kn5;N$SU%g-|I5y$Jy3Dc(7e*4wCd~bgjPGmvK@a=V^FaA~ zhKJQfn=L;tPxx&dWzz3qQ%Ee9e`n07VRtC%hevI>T=YbxnnBTq)mbmmW|!dqF4}B{ ziAb~&d_K#e{#<5Pus*E z-_JXD58mVFfu#>*kd~D75p@~{MJbFL5rC3%_KMU>N;&&jCDwW@_$B37!tesWhy5uK zJWn}?S<+8WPEXsN1@R;b9%fTJC-gA+z9uh8XbK9btHU^aULs{=zvuc^&7N4{1mKe7`4gPLHM;Pm~h| zGPzFqaeB%LJ@L~ME6vmM+zXr@s}xC>K|^_nz+^7_qPmnAe=4=Y|HTxwwn8tR+?Xvr zm2i8U2AT?Gm6u^q{&N?hWkYA~aA$#oPNAbc9OkxD!<=`nyn)FLs*Xqe`0!N%UXTr1JPX2AYxK{pS0;WMRCw8k`zPHHP=ZA zTM=~aKG!Vfe|w$$*7+MNc0e&#m6|K>Bi1`Gu@BNUa0O+DyHo7|t2B}@&2C$BIL zfybV?2ciO<+SVeD8bd&d#p~Rgsr`PSup9`p+c(-!71CRVx0I5Ig;I40E%ArUO* zYFQ1Budo97_f8Njs7xlYAWIp+22EJE&Bv$ka@#^i3&C2?z&ca_$chv)u@*Qm}zg?FU%cD%Ni>MWI$@pp8N+9ko^q8w2RF z20Pdh_7_BAdO|q`sQPXz)%XT00E|vyuriHXc3Y}=14Zn77Lhp)agjm=6T?}(#Xz|m zYUW;18zciCGfcyIKj9fKND~8zjS+{-x!M>Jf2ZX~XB-0z6QodbH7=sj!Zuk+BN0Hs zg~&%~;_h^Q0ebk3C=L`wv&sbuq(MRho&K~~3oJ?t6M}A}z9^KTIBJ!Bv_+AhPSJuT z-3{)y{3L>ElkLcT&~l4Skd&hKoLP-bx8tHm+DJ;r)vtV8m}4V#cpur5D|-cNq38i& ze_xK#;|^01gc0Fioirs)uigaFma;p|n0VVqfXTIvz3r`yN~S##>~yat%*?Z`7woILgT!;TFJ)*uS5wMl$P2+z|?S0hcI4 zAq-Q?hvbZe?`s*D$R3ofro-IjOyAx@{l4X3ri-8WU<0vgT$@_OCCO3K%d~FwfN9>yH zna(_uQBr|%InMJYV`z}I<+<;Z)_%}SAQF2H)j4=1KQZ8Kb4IX$lhTxUz-;Tee?m3e zk)uC2EjMW~47y>K~t`jY3m7hbbnF`mRc4GZ+PH zkj!2uRl=SaabXxF9XjUc#uO%Af1a<*s#!6LW;RkK9-f z8ubo&BO|muWw)uEgw|!^UNXkKws~{w^~{|c`~Uc}l2AkMvu>V#=;B4wDGm@C!mpX2ygis^{?f4699$m)g9Rb^@ zfxcF_GVS|zG?S}^{y3Pff9*uw7_tc|si9+I;;oSoU;gi7&wlWY#6jeWT|AhP@fF|I zPP}2#!=U9EY-%$o0=ZR;x|MPp3PZ@YR~q$K#)2TBd&wl!PKv94~$EW#58WkmqVE|!l^*e zg71wF%z~pb5Kd9uy%N4t8>eV<0YTaw$rOf3Jh^@bP)@=R(OBGCkX83e&W@QRhrvk< zl1rKaJ9Q(BA)aW6e@{QM1H84LnX0z&r zZ*$}J4*rF3Bl`p)ZCywm%14Z+i^MjLUUzdXK`qX@A2<}{+@*V6H1ct7Sm6WO(p5M% zUmKV#Z0w7Ke+B&`F)+bcOXf-Pv)8poDoJISL9+3!6*FosW*)FpnSbL}58mX*`Q~4( zvO>XK&UqWuO>N!=QOK1Cf3M`tE4Go?*MT{2T=;sTo-c5r z7XQAB&S>K(co8HH(yzql=HSAa<-T5Q!zKb;u1yuD(TH~=Tv(YZL^{C`OYUsjVBD_ zJXM^Eaf-qJED@Frr%uf_kx>1v9&pD50~#w12U*mRK*J{{%AYs~vM){8a8hI+8ZC*p zIyV$;(a?0@&WO)9>T@M3EkxX36uCMDm{>K`e_Qku4GHL_jbSr0!|DQak?}lY%M;VR zX4w7KCkE{Ue=wXe9vieI(H%_T)lrSt{CrFcQ)LycR13RHr`^d%3axdCVMQhb+v^)` z4@>}daTk3|D7)E`?o6)1i(!*HB1|N}DykNOYpmBBVrBJxo1A@E z{O-?K)hRh)Yo3kjgbVR(50Y4ffcYd{Dr%fyzAFG7g_Ws7;xj|6}e_v)N!TR-}J>KR|YDDgOuDk&kv%_zUP0QfS zL+`bEav`G~#}B4jK1*if2gfz zn9cj$N1Ic8(+gT<cJqa$>4KiI0!LC#bXUq_@fgumodRJw$r9ynpImYZSrG?&%UozA8kGcmm2f@ zht&>FP_XlvPk3ROG^o?cx)A#uf7XRV3!P=O?XA?`F*4V9eE?>fy2Qg>u6%=aQqrQ7 zi>uk@)=2gnAkReUTC7Y1gxB`Uq3i(Yl7?b?CXe0vdhAx|U+=qz%3GOCuOe^p#mH>h z-=&NfA^z|G;m_0@iu-G?%=6kz>@ubM#oYK1eG%t|&WsAbj!Ix;q%e$&fBCRdWY#2; zmBPzo(I|lhgi;+=icD9K%oGL>!dyZ&Mv?RFR` zZcLvKilC7~rN?BXaQY~`IHJ7OmWeI102b{5UUQ>0iTFKgm&st@{Gy#}9TU?PO&di} z2yGM@?#0jv3{FRj_=f^o&nILao-7=eXwH#hlBIpoMq&H2WNMj!f3r-*B$`G0f@~BA zBO8TbniS80UDgvOb!4_0KQ;;*S(o}du*=dBQ0T2t;gJTQl6L2bN}dBFMP@9|u~OI` zCg}*=Q`u#Tug)N=m6gIU(jvU14wWeyDRP@(MCzOdg%}zsf*HX`all~2Pj2<#O)^p( z+6-~PXiXx+Aororf5D}Y{7_=CGLM}Xv9t3FFp-(;^*Fc8G?10TYZUSc=lONx1LZDq zNKt;0`)v9$yp~kJvQh+w2R>K~&>dG8!YaQoiE;esbBT?t=aJ#N#>prT@h;y1>c>9q}0+|7}EU0?2U`I zxhc&<3YfWG%Zx~a()34x1gjPn4cTtI=O*EzRV1G z(U+;hZj}0&=5Mex&;gLv!p>TAw8U0c{~hfdUVooHe~Adv_6lrVEL#eTfmzpD4unKs ztIZ;7=ZVGrMMSPuIcww&s4kXgq7@CkNrN{XZe9#_X37zXF#(3D2lI1bP-x=^Ume7> zjNg^$mN|}cj79%Y+N}pi!9>JQI;VPML$2l;_Y<<;!tZ4J#hJ5cnI&;^fBXg*iAE%N z%MM&se^V?lNEGugP=<0va7cbxTAM*5y4}c?TU@R?ehjJ|Aq5P{CwgKZ$)jOEAV)0P z8huB|+bVZRf?vMC#%Sy(>HnbAU8mO!^}roN{mM7pwukzYD)CLq3A`=Gk^_EXx5VJ~ zz6kXQcB1WgqQw(_6OI}lF)bxSCRiIbBc#kFe_nz;$0YuJOpC<ckHgHf3ybueyPWr0ipy z$D@pJ$G}`?P%g!T>HJx2uKlch?=~NU!$gLU(y%PyVjkwS0NS|UMs2$ zy%Ip@Of2f?cE$=gQEsFeWiCqk1-xzYf@+{hUq*l=%qbxpSthaD)Rzn{w69TkeQoBX zQt7NJwwns!1@AY47#xHMpN~r?VtHjzf0*>DsW=%{Pn;Cj+L8b^Y5#1j`8~%mS8>*t zk)G?6=>wT0rzPhdS9Tr`2+2v`@F&)B5c(IE`0bWT%8p0DEmw^%xK6to^|qx#BHeRvP=jwAmW#T8nTr-98${e|DEv|f+KhVBJ+Sl9k@!Y}T~kji z`MGT^&85XlfoE_rbWcjql8{6tNHCS407J=GL$~%DB2<-l=K~WO&q7UyPsDjgtN91! z{IuTs#DofTVO#}Tn@5!@&Z5T#69zUK*IWI3_}SHB%5EOQWO=?+e|dR#FujjC)VLqOOm+#NsjlAsntZ zrg5Z`8i+5&Ydef9Vgmo7aI|7=u?l-T#pBqbMj!|zC2%G&5@MC9Km8ti3$>3q%lakP&JG2vHCPGC#=fh z%1Ijw75!~cYC>M#%(5qP0>?h|ke@FMT*aKL(9?Xe@I(d$CZL`S6p(96t}jQI0FApYDwxM{z$YsM=t zQHCrU(WFbq{$O+1l<2*j*5s`c0qAc#_Jd_uJWdEezcF5Lf5ue%fnj5wG$2O=x?%ie zAOhV$xqx%QFebSOtD;UiMiGH-VB!lX;YQGFX_o8rD0J_0Cb~`H9wH%lS z=|xTSF++3}W*v`L!*}H~8yJ@~B9>*+$Qgm|6QfPTtEkNskrb_Jtui-pay6T*9tH42 z>W}4T2>RKce{~4=Y#WgYQ&@Hg_qq-UkQCT)T^ zLpwastdljjX5BN3je+e)?J;mYmgg4&u3dXOT>cKn8=DFde7j|(Tlw%2XW7!W z=CGv^70!~!+H35t8h_ucP{s=n%Q|QtgdQIo9gf`_1IO;cz_IJ_m$X#}Ghr4^ULz7C zymHnWp$60eSe_8l@fzVH_43cFQ_o}N{M94Qdu#I*&+aADSG&|((8fdEwRP$FexiKK z&6@#6e`sS6i?Rq*DNZW47Z3}@GRhBH24D@$&yA5=SJma@*0o;=hw1_>k1MHM!Bz0; zmaq)KLSZv~m@s^0*8M<_Kv3r?YeWPysdmlCQ_+SFIir1T7TrK{>Kdjqi32t97~@8- zuCW;DTAkAn>BFlVObA}x1InxW#-$!SNnYJUf0**>9!jfRUPkx9t7~9VtY@s2-BM7&G;u(zjC_>d!fvg!-K=Zt&8^^ow`N{ zf9z>s6nTClJe@m5ETLzlCZ*RX@Z2y-M*8vPHQKhq6A6oxB-VvfsvZuq!I2*Hj30LH*c-i)?(+n153nt4g8+xZ4 zEM{?EhuC2G^6G7s@w!!4$f8g_4 zCf&QTg;<5Ob<8JLLTxeR*i+jJ0|3JQVnw;Cf;;RiSQiC_2+SLnW#V9o~_k;iXH zWsgBnrH?T(V_cJn2=tifS=I|PiZIA6Vk@+=sHd=_VnMWX7%%K+`W1q1u#%CyJU|(t z=&>LR z+*!n--x!o-YZ1za0%wd}h&c4xS_B}XQAG{PQ?V+EzO4CGLIp;(k`lfIf0QQ7kpr-z zpjKCUg}p&fz5>y`lSdBN-eAYGBclZ7a0-?s`U7+7t|=by!YETvg~e#VV<37}UP1(- zdwnkwJ?e**_@?S0!g8WO^s4AC)%3TO;VD2}x)vecd218a6^MSeaYuKcYooaf;}M8{ zFe+3Ux%i8q!p>RzrBMh#e;~T*m?;olb<<8DdTorv=xQgDzWD-562s>b9M@V)V2WH_XtsB;xwyD`inVjOy{CoZCLlV$niX=(}-XDNGjfMpzdy$cg` zQNbS9&52UjdUTMmPGliekWy49=3}wTaLaijv z8Cyw(C=<2soM*4`b~#!7g>|x~{fQZBCJIZ9P;ANb#|Zh21$k$4IAPlk9~(IYXh#N6Wg-E zPr6_@ON0qfFq}048U4}}U5UH{4#DtAdANVpiVPxz=UM$1d3jS*geg=9eP}1C*bCsy z&8Q58=NaaT&a+xKswCm{+!imBQ<~INzpsc*v1&L{)4WWbf3(e0Lv@jZGd{20p^#P^ zN6~$(sTD>vJ)r;{mdjit;sX(!SJ&}$_yw>gzlq>HTQ^lwlB3>g zslSCnyv-AHFAQaLW8&|HfTh~NMk|$(kim-UhQ+p(OzXN|fQ{v!omDM^Q)GYfc~u?2 zJ=gN4e-sQBU@&Z&{Z-Uel?qP`pjUI2CB$IPO{QtJ=43-%qS4p0=@cO~)^5@s*6x zf4RFl!ak_y=D48Vs?y{qd(|`+BR)P1!wh1K7!0n2P1H45hB@rv>m(>C&qM};!9RNu< zFGS-!j6w|NFE$@yi#j1%{Ra`?erPi+j3`QYFs@`Z+<%Y*4P>VCqjSg3?g#^6(SB7K83c7c zR?#pOE@Xc%xA*kSD%fmGQBKUCs5_#I2exX|$M-9Pa5RFda0(7E;IlcxpcuJ-fsvM^ z1`*s8xhNHB`JmGs^mq|iuv8`gf5sCvvI+2ocF1XohzG~e*4MSn$SQc-2=LTF4{qFn zLQu3CIp*o2)llSKh*m?z^B`J{bewSuaDIqZo&G0@R-HwHBwFftkY*{V1B+-uCX-J(?=Q6J9Fbe~~7;i>QI;qkI9ke1i6WlDnP4vZY` zD;KRqBowV4M5H}_O6-AwXw{>7rHfWpXZ$1_2|R$<6 zMJw~IC_ezF#e6HbXcg64e<@nU#)N1UTOOiSr~h4|Rd11QJh^r4DO&Xwf|83?y+`O$ zbW^!1In*KoHIe({%tA@A5OG3lL~Ue z=4yz1MLD;@x+!qewIN+dQO<2h;*67WVG!kPn58J^W=6#;%DF{$f1{i__9N|zayEEh zQO<3MtH^O=m?b{6IqJ*xWU!KdNn%5l`rxWC7|}|F=SHv`UyWr zh3kvWrn7M&YaZdq^O-6{kuz7|GZ-+}O1?s0%*&bRD^1UDnqOxxBGUQBz-1YGdSO&7 z$Nio`6LgD;Xyi5xf1;vawP{(LA6?XlAizw2MVjO%C8J9$F+VqE(Qa3yQYUuLlvetJ zy(N66J2FMP&dd@Yi2;h4Ub}?e4F*@jIL{^X;z-ELHn0R3IM)X2 z2@jRcEM*4R-(bZlJ)zn?2{&d*6gF%gjIU>T!L%5{Pnt*weQ9lPn^kSlLlM$L@ zKj2rqm1SZ$KI`-4%0P8q;#?quojz;UCnu_^*g@im_~Z>b?tn=w&nFn>m(fi*v)oHwwDB5-Q`p=W zzo0T2IoiHy(hTfopFm=MZp^09(06wOgRks(VoeOg>8B(b;%%I6(=3b+cO|1BTfnE; z1J-ue`-uh?6|Yl(%aL=Ek?Eh=4>hA(v5y(x(+sakR2$`9rxsu43Y?mv4hjsdLRRE-wv>U+m z_I1$+__?h;&Q6?Gi`(HhJTN;`z~h$;&OGnrA1v3C(d@`K9Qg(IiUdK>=2Q+$5^=F7o~Y*wg(Q;!$Jun7ZrX?Y1N6B2=iV@x zrrwvzFA`kVb5DhNeZH?JhC#}ef$|wG)(pmC4ux^n1TlbQYVaT@BIu^!JQZngCZ%>A z<3jIHX2f1$M$6#?g>^#7C<=ug@sYC5pRvA$e@a$7!BR&VMea&r5`LbDnS5b7#9@m` z^Fh&Kol(iQ6eeZoq1*>y&T1C=<3BOkQx$)%xBkE^N{jZ}jc++M6qd`)5U>oMGdLLM z>o-_Iqd{8{KfPi>g&`A%NmWv@o{)7cqcW|*o6ML@sqBNt-a~T8UU1%I5Nd}F33cD00RthW)?JOy`*iSbdFj!6t{Kg@4G}hNWVo77d zh@Pkmd?~TJvy3WfY0WFkyCzG-B}hh{Pq+Zw81NvzV#d28(fAuQF2oawQ|E88OGuVI$Pi#n@bUT%W>AG^e~kpJ zj(P`EcBPg>)ZC=f*MSfKD=O@L&$9nOz1(K&P(Y*9k;dApia}|QG+`BtVj*&=t>)E zP-l73qXzXAyyOL-D7sXj&YR?@e?T2FRH#6G#1no5JL5tRujv#B;|bPcmI`D&Kb|U3 z&oOB~_?<3asX&5BQwM|!)S0O(6{weY6{s`tS}IUy4L+$r7U)>NZWX9EivOK+B+>f) z+K2uq9jHqSqYl&$nY$9CjXp}yg`pA@7%D-XMQ^19_5H2{b%_#Gf;x-Xe@F@H`&|j@ z3;r)DL7kb^Qi5c#I(iTZL~2l*hNWZ8#_cXTQYHs8l}2XrPboLizZ9w(ka&wzNF zZo{JlvHxXgK_i7kLIRQCe?%>4<3;7=5<~4z$$=WFWh8RsJfxF}?a6Bh zTRLjb7mru@opTG!9ug*=VDey^k%b-_IFrO{n9jTa$E1ZN!G39Q<^nSA~Q4Q9Q-d<#m%!e~e>rkx6^A$L-_NzQ8$HQowbP0sOPiIyih}hL zliii_X+q6>!jS?b)A@S zmgy@F0dECTQops*$qqH=T2WoGc;aZbnUus!wai#Dh#e?UGn?=Fc&Cg7w!4j_n$fWMwp>ioRp*=W|7JHeq2MI9V7JXME!jy&3f56u<>=)C;G1)I>JNAou z!(P$;#yG!ey8Whp!PV0io0Z+tT{w!N9?l1vgv8Z-e;TyZH2tRP0+|>&2)p@d`by~+ zn)3;Vxw0XZ#)I9Y9~75Ze)DGTZu47GY@$ehE8;8853?Bi<0jx0;_ozbRY2~+FwnVd zcgE0^Rd$noK^dBg7c5D9@>sS+5oi88{dwvL>*M}da-8}PtPzTl9&PvQ`>V;Lq45PA zSylqmf0d6x=bjIv_2VW=X*e^C=KQ>d=f_avKj+jZ7!7JL0xp17=zPN0&<7D~tPwtU zL5sTQb6*@IJ!C61J+nwj*-{3IkFjdn+%=~b9Xd_woXWN7-SP%YH8cDN)|9!xzu>t# z{QGN2rqcKVKKtSrT}tr_q-=ab^qH1ZN|B~re=rCc6#TAr(u&J?E3-*7P?~YK4s5*| z`zou(OSx7PHNj{Dry40tAZEZyW-SE<@Vc+n(R)hf%25+kE$8j#<9^yQYaOE!U?6 ze{TiESV;+61+!kf7eCVoXmB2a-%_9TUqclO@OE{pYIm+k(rsWv64i?MGKbLt9;vGZ zQm^1AkzMQ!r;JYy756Y>@WvpKEv!JC-j=M<4t>DpnqkIEVJx5C)-($j0DwS$zx@ld zOl0T}j8dmo%h9||ux2q1czz77N{m|z3B=;r7k^vWTHD(ljI-i-oWVu;C2QeSE#1Fb z>;-u6lO|eWo=OPH>bHieEcJk}0fWoX>W&q#w!;Q>TEO#)-$c!(ildHCo+2Xezid2$ z!TRHhwqaM~S0;s6LYy5nD#~LiMP%O?Co?4S&w)~6=Z=?_8&a!%*xrTm>~v(XX5uKo zDt{lajemi1kebk9XXiypsK8Zl`UPVu!2ZA_6e($> z2a_tR!<=*;QC1@#k^o!?x27IE2U%5T8h@Ar=pLC@`5+HO$O;+D$+{_rw1rKRC@Xs` z??JNWJHngr8|*WuMNdxF6{*KrK^Cx=g-CyRc1TSuOl)vRvSpT)2Ru}eKqO4e&lgdX z&ih9^+?SKA7 z2HsJ)tLAH05_>$$F}QM5kO^dQMDCe&T^Wir~QI3i7ho^ z*ykIw%#WsC5M}prPTBD$(co1z7k?hN>~%aWbl(sStfIp;Y4G5l;hDR`z61Miffz^p zBobYZ${jHhjj9Zld(>GXfwW~gb=X6&rICvshx$}rVslB~cSmBLdB1NxFX^G zjMJBwGHfFcs&UGMLVnXCKo4VO#=@KR&f+ISE%55={J=oToBI<3_KhyR=b<<+8);m0pz zGMn>>#8Vt&KnT(`M#0ytgnudGt^Nz<#jpB=ZVWFPP@MZx7?`NAaM-z1*kSk+5rtQa zL1}<~TO}07C+OhDQfd(W6$@>teAeq@Bb0YLQ^1=_u8Qo+4*xJc^8&n?I@#VMfR%v- z+s#b&nWl1N!-feK%v|l}I{?TO+51lk@Go;}_jxyf$q&DXJon%O7=OU@4CX^4V*Ln2 zr*FIiwgFngH{Bv!Ln036T!;hxMqEM$yL=%`#Hf+CyMv2mP{*2FMHbd`v{QCCbXb@vul5b15r5)`d5Z10no%0p*M=>6o4pJ>xKUe-`V=p~jiA;ees6^5K^#NP z8;9*C7~W&l*v9w;yJYWg*dll;caCBgUmuPVNx=Hcr=i>N@0sbjFE318EKIlI&NMxC zK{wfWT5WJA4Ew)&9Hpp$PFc`7KT?9 z1N6E$ivn~D7k{&JX$q~TVERi@!J@j_6g!z4MGvKyWGIngss~%!&@D>=5|3xZY#H1m zR5J74s0>Vmncd*b(8onq(R^`AB5NVyClJA}ie+~D4RNe3h5o-AlM@*ZF@SBL8;L`0 zlr-1W5^=(Id2Ju2MN1Bzrlc@*Q`&ch9RNM>?B6ip`+tS6%k3oet&m$=2s;WMcZ!Gw z!&Kj|L4MV1etqt+R=<$1GG&1y?EGv6afW!)xjiP`?+iwxiA{dA4Yh#OoS{Q-13QMD zVgqHBV*EW`MHtu^HY%peY5wI(iv^6W;O5R26Hf6&Ax`kwaUsr*n?&WDX}N?H7)fEo z@T;4!hkxKs_V7bwA+fv}`B(u(mce0%N61p>N{YP7s9q02oTmeRZQR6qnUE;AOJ`R2 zc}VitcBTA;yG)c8HERsd93daMlI&KKYq-`+-Z%I0veWr=AGhs>mZ{jYDMN`4iQo-N z5n@o>Hw_hVrCD&olijE*%@yR=2GwtcnKY~u>M$-SnhX=9L8EsBK^@ER6(L{&e|;1id>U{57ux8ErhOm z2bqX@N~;KuVR#6ps{qH0NVmev95SoHi1udoGvQuvY8ypf zVz>9r=2`0@G*FvoKQ;zmn0>bxyh;-7Kv;Z!LK(Q*q%3HZh?9!g2V*=Cs|=>}KIFr+ zDisr$0vAt1Fa$)2Mo>D6q;tba6DLy_`!Vi0Zg^lB{(ds3iv$yx!Qoif#cxC=QGWne zDii)de`5$QW?_=?aPBd9G`F*hO_+C^MG5EUSuObF`xZ|TqHSPlVkUArxcI}=?uO|_ zyAA7r3G(@Z-$dYhCrpC0cuv15LgVFuuf~sLdN&NN1PhxMedKa<;KE8(mzHdpIqw!d ze!=O*^==QYMaC^CBvS_-2O;73et+zZf|wG8)UfRvrsZSXM=azzwta*~mCLrTfV;5m zBPi{%9~D+sz1e%8_n+~?BrN*~?i#mcpV41o*hi2nXH^)|COAy{feG6=0vX$Hn4=AQ zuU{}>%SZTW`4w||X-U5^_t)9$7fe|9FHGtR|LruZ!MYEaROylmgJZ+GkADDS!mX*t z;>zU{Vw&19?;BjkG4B`Ic#nDCFsmN(K6ZCpRN5)|Uv%6z9gops)3qb>J_zi#?khes z%=?88?Zdo}fX&;wA4DA1{SUHM?HZZWDzr|JdLug{=I4O{FB;t)EJ$uihT zW4m+1mwcw@lF!UDW^UL#)UrD(dS&``8<)^psBjU6|opoxW^*ZO3v-tZpFP{7*KWRp< ze;3A#hzghKHy2x-hwuoQtMS-Q>@?~xl=~W6IXv7R?#I=M7IVAaL_6vR8&Cm0AijB*uD|1Dt|d*#>k6e7Z`n%{g%+` zWFGXxa8`LZc~Nmg_Xgg6E#!k|>_U(AtYU+OEjAepnZCo8OFsh3gy{%=Uybw1n#a;v zEHDhKM<#|8y(-C%?+r0%34+pxn0eJOo8fAO#(+w<5jI&$qC%wj*X*I&otFn}wS}o+ z4X!JK5TU75+J7K@!)=^d;hS&@wh(3Kq<+;Uy!wWBMy>=rAU)kAF{KJSLCC%_r~8Ua ziP+;N6>cf^cu}oLTZ1grBASP5!t!Z;0G0>aVXk7d@OI~D5mU64p=i}u*eAO?jx28# znx76+T;X6<3$i>@j*>RN@fWKsTDC+(Y;*DmZ#jXNj(>QgnPzy8;c8$}Ijlkb=pdxc zGCEt8X6PmojmlvstF+2h(Q95sVi&(#`NQpwpy7JANPeNvNT9VmZg(74wn$aSH&KVD z27ch=1;~UMezD@A3qlmCzR`KfElFT=Q^o4~%@uCL>J_Z`XJ?N25A77WDfBfFPAs5P zD5ix=3;!7dhmf_8`18S2y+W`R;RJd1AISVIni*J1YsUZu|q zA!2d+erQ6L1ol#d%;PXFMaUc~w;r@l{9IQcYAYB2-=Fbx77sbs-69{Ubr8<&p$hH9 z8^LuUc-}5u$l;?4Ip$~@Yf+#6!dx1Qdi{bqnt#uRTK$c<`fy3VG1s@*=^JtMpg>R! zGB96i&{pi;QG*OKFEwZvnwVS-G8=Y(s6kdaw_Iw_ds4dahKHfn@=$|zbt#@|&~DEw zayJkE^DA}Fjegzg<)tyyem+#XsHh9TlR@w))2suwtps9yz3q#v9;fBNY!W20O%rDtq z?lH3pL-pnEG=mo1Z!rc&JGes*Ria+uue0()_2JPo3@j>2jLtH$;SofABB{^HFr6wX zMP5aNCoN0300IuTWMJgDj9-vr6W@5aK7TO^9!IS;2kFjJpT_a7+C0~BzoOLD3g@h^ z)A8P79m{d=VwvZ0|N2JAL40l2aU%PM-WYDVd!j`V?x=Q>Ij(FcDtC4V(a9P9L6$5! zQNV)rRaACbJQMV+*f^LS_{NlWb^7`I6_uu`yhR7@*?%lxvsPe!E==&I==F^`y?-2& z-2xiFL(Kilc3b5~uWyX^LV1lV=;I%Y9z23R9C6|k1JlVg0L#OM8cDN~ISzdIuPQ0X zWd;(S)up1wocr>Jc|Fa_cYcsaOkgy4%4;A{_IidGRRs*eM=(?l;)p<2En+`B0w~yV ziBYJ}Axbjw#9)t2q@HgjDdZ&QA%7Y(ViU_IT%}j?(g21V34H`G$zg(?hXQtn3mAq` zWGD10z*^aT!`%b!9dov8bXOvuh-?2G)2TI9CYi%m?70hI4vTjH3Vd*?X*M>H{NKP; zMeh0wvh!}xTs2ZN7~QQC$Bkbfv-n7vMHY-K6uGsKx)mGepH_@5T^WY}-wLYvJ49%@&Q z?rc;%iE(I?_Du&PGWNmt$OyMQcwx(}j19ajTo(p~K2|t&Dobu;ilSl5tukx4Ew@T5 zbz82%ov>`VbukdbZMk*vc6VEDr6KXM<)Xw%JGNY_v2nS50FQpljenIfd8fg2V=t+4 z`70njg;6Yavj3BmjslF~tZUqIuE|ZaC!jfBY2*Q$UADuZ&xu{JNQ3-!p ztVD6$odr<_E*V#JDoc@Gs!(0k&1-5=jvd#Eg~+EMJ1&Za(3MrZScqQyv;(;n*I^lP z>)alS4I^${$$i!`;(w}iNf>dlua`5|#-G>Y5SY*X(P9O2z`0wScU~BA>)L*6!iXCr zEhBEgVZ{9)`X82fQP>u3xS$5!Qrzq%7qM)(sL|sN|Lp@;U~5wa7#Bz9xCr#(<|69T zqIwG$aqUo4$p<2^6_#XJaaA8DthjMf!;1Ty7>@kQ{{7E?|9>Zsi6l30sJ=%$EMEl+ zH@r_yQQ*w~-sr#nlV0}EesVD(m=*3c3isCKCu3{7Jt z5WmCk9&@97fPZ6tE=+>!9THK3K#(R_-fgl+n0KVn5ikJVLLnk^d zY2Gjz>F^&crk@);?C%V7K@T{t4P#8wz-b?Bl+6}%V;Ed1i(z~0C#-g=wg=lML+-~0 zYyICXfys?qE=-}3(%@{cc4zbVTWvn634}dD0r=i%8-Eh+4(wkCtOtRH=@S^-4cJdk z0#TZanj$9YI}SnHRw1E7AqmRLh7HdFUmGsWz2OU%d~9AeR{X%g=*Qu{#$v%RZ(BasogfcBK_q>|ACpf<)0Wh8US;BKqq`Ri|JFrF$|WB z=^6|iTpt9m7;IZ$={{-QG)&oC0<*s?R_`#_MD|r-{Kk6lx-?8}SOI&W@P?7XqarMK z5Z2!_%Y-g_tGn(N2&uy_WDF{iJQGfk^(aV!Wq%k*8U|Ok;5J*bOw4NVWxgZMQWm6y z_yuD*5}2e?oxTwnAx1wON9(|l+-l;vWKnSgR{J1*8fMmuK%K5O?biV>m}UQdQ66rz)yY_mnmbugLoBLeelAR>^1WEi zwSS_hnUO}rQ+F6&=vXsMvwNUYIW~Cq7mZ!EUQPE%ftd_F8n2?lK*hyuhA7LP@eQzH znAbS{Mg~rs5I|gZmQdH0k6}j+qr>7|6)oTggCWbTr>h}U9$%#2YxP9ZDLg_w^Ct^L zAHai?%#;`m&%;NgC*z(_k^O-mzNt$m}473dy8 ziR6lq@UHEh7qz#++w~lWw#cBO1q_Bb4@v8$+tiPo?MJZiL)pyIAK>)x+YizQp?|~C z1$k(ZH5-Pv47R*&QW?YD04MUyLe0m9Eg6k>a+^G*&hkX37gH2vpB6}Dk6);R`?tw> zoy=M#$-F70O`XS1L9dn9Ij><|`Is0_{zOS)=Xy5TVNXU{lHYpkakvhF`0$p zAl!{hT)gF(e%rLeL!TfcPeBrGKwt*ml}q*o%w0^ z#R(Q1Kn54Fc;uwu$v3@dG2u^~Yy?r>Cx7A5di;GNEiORtR0ez42lQmaFn^7oG@kc{ zr*9^Y>4c(hutFw;K{}fNu1}f&46}*h zFl_b>Cp{q#TENbxOC2zz!l=Qox>MoxxfLjM@|`lY z$Z12&L16>5;wuh2=I54BP~Ve!4ox@zi0LgbKg1MbU?5m~P@6Kl=zkzQB%12D?m_*F z)t3!S?NZlwcgC)VU51f}trm?+YGQQt+N717+~#DMBK`|$VIzzPYEG|&rQ zSd@OlTnALWe!(Q5^y{0oXG13w&CA}X?)$Wap6;bHS^O> zn-Xw_HShhqJHhs5i;Up2olsudS@Wi8<*@DzneqC2ruG)y6U2z~fwebxUPCY#BDIDX z@bLEnfsu{Of5CJqqdxzJ%uN_nc9~0~sMOcQK{bnu$7-5a3x6)d`2{Z2^$jg$jKY^% zqx{$mE$!%!?QzUvh@8O^kPx%D)AUoaX$Ub);>kjk%XT}9cxlWmxUJ27<-0|3?x|lu z8H~p_vV~tIx0n6ZW30QCVmacYb)G#Do8C;Ka zwTlcwAzc}!T+-EBRr)iJaAjbsw{W!!d@&(h?E-BtP9RHITqiW+W%cu0`5EozeU-<B=UX zFX?JCarQaaT&(-#vyk`IqPoxr>B=@RKcp+8fOJV$K_QZ^K4{4I_#s^dM$(nR=}1>L zs2 zFX;*tG=GkCWuHtB>B=zulCEsf=`wO)5)bjpCOR)ei7mvq#4B5}dW%n3$DxX1RXPG021{9G8f=trk7#MNitw!G_ik1P|vd(MOZ&KKr3gzYYib44Wl zzY9~C%y2Wy{8_o4$)etZIZe#liik?q0*|xO$bW}p@FncUQQA``uyk@ucAh~vk%eU% zy?1Nmafwg5_c85ogsplHd94Ez7AeDMo)>0GlF0Z~nTiVq-`!l%6>GM)lCr#+KhNfs z=d~_vQq4Rx*A{Q7Xt8<6I6vx%;!{D?imxP1Z+IZT-D*~JGdf%mP=X$T!Mj8Ig+(JJ zXMbVo0ZF!Um*T6sZE^E41)SYb2wJgG8_zRO0u88J#vSvs6t%#4?UdJT=0ot-7Ef9= zI^%VBlQo)E+YK!IL0M|B={!y_s>ha-31YJ#4l%MuTU=?|f%Pu7{LP+lJv)pXoi|Nv zhCVq(xFcknh8%j=t^*brThx@qUkv`gD+Z=N2!w*ZT#2fK5J zE05>y)pvOFJdgsiUZCDyXkF1jbd za!-i+5BkBAqFxNv2;dyrW4OmdFUmF_o8EWyX#WokN`LY8*YEs-$w(pe!+){Jj0hKw z!Lsn#yCMvJNUmk!1)FUl695Ix^kgCvMW${xGmDF{YTJbUHaW*zGJ_Vm`6gD9eyCid~lR(>aC(uReLo*uI!-F}5qf zlx{6X?-p*Yitr`f^405+-G6=8cPMaV6ge0U=I6r5 zXNthK_6naV(mv1E~a>eQXiYT`B4-D3ED@Vu?b3o=}$48VH1%M|bTB(_0dwGh77FK3YV!KDu&OL*=AqiGeu?~SzIN$TMQm~ zCT%@S(u?z^72>6HFOl|XQCwuDxx#||C=msrWm1iNrc6Nc3V)wzoY(M~e$0$Y;@&i; zAuGT?B03SYztP4D(!(*|kZAxjM9(npoyQT`C}$kRw45LXybB*0T%lO3)tHo$ch0nOryZ4F>fCMUZWW- z2EO?KDkzwkpHG@LGA}4pRSub#lxX`BnFDS+$Q*Dg7k`--RPh=?=74k9I^G~0IYH)c z1ag?1zXx3#oxkDAi8!4D<6QZ@s&BZs6~pNKz)t6Z2`YMHGR3!Vj8o(3^^I|=#^B*i z_{Lnlw#RdQVy>@u_ey+Vg3b?ybWXShp%b1Lp#yWSSb!zS9AI4+nR6>zkU8KvgUqpa zio9C>g?~ZCij>aXJ#qs!|Ka2X=CY-cfLf4xQNhzg>cGquq>j~74q%2pgU{gB6Tv4G>7v=!ZdI@s{70F8o zbAVm6%J|0klaDM1Sn+NXLz=f_40t_PJO?jfgMZCUrYKjel2P54DEG!>+NL1N0d)2V z7ElTUY+aTErq)ZAI~bSc@GGN+Fo(}7R>*`ozeDNDDgzxuW`YE~a4y z^?x>PVajP-EsC@^3Ub_;c?oh@XqoWr&>yOXAa_6)MaIH&5ib{&Q7nbSwG;Fgq~#<2{w_4&}wevOU6)S3%(egb4mP7&o-MF~gOc56pEjsn;(U5B|Wn zwYQ|nnz*J6{s62#j@gY2hSxxu@+2(E}NnnYtc+)Yj!>Mtf*S z=kFr|@*wv#38_c!aR-6RJ+1leOu5J5Blie%Nfj2bU4O-#E%#pEm?P!9N4xW{nA3|Z z`i;20Jv@@)19M3|j&Z4{G4dy*o`1&RtV`-?gJP|VJ#Bh?CgleY5I=HHn=Z~%?rGEg zV3d2>IVWvBMw9s!-?s1>gk$XG{-Q}L1ya(l;o}C_C z?hy?ivF9dgk~x#o9Y^XLQz!-=zTg;tgK}f+4GLj0--df?6n~GFbh(b=cYg*B!ziwV z^DZAb*ly<7NA^bzcC`W4GZmb-k!hv$M<{Ph@+Tz;0>w!8qm%dCH<{|WbNc2b6D^iU zWSchyu)li|Vf6W@g)`+14vz{ok2&DtP9d@`$DGBqg4OvqOxU-h*Dn}a5*J-lQt81&^9aeIb>$$yU%v4!Po`5}Cad`EC$&phrPdoDJ62y`g0gYQrdZSft- zf&CrIC-spcJ#h3;B+CuoOXipidJ{(B4aR@~^K9_UTUHhlHDBBTBhUZDUe4~u!wzw7 zc3!DTkAmNrrZ}R4zcUQ!JtnNbWt)_6y4Ae;4Y44Y&vMxw0hW66|9_dU#~g2v?}}2= z4476EGpJg;6%ievcf}bNcq9*8^%nbJ8t55trCYl#{lcV&IjO^@+iQqN4ZgfE4IZk9 z<%*;mj=9({iLyh;;w&kM@JP*w(qSeZuHz6lfa96a^OT`jz%&hG}3|riK%Co-V zZv@=LYkZW0!JrHQ1AnZ?8=b$<$x4!Z7{D}H4Ax`|caq>nm<&=gSW*b-!*~k^QRz=| zUNb9?+u=q=AM$kHABbj661z5zON2 zOU0;xu(;)tuz$Frf~&HJ-x%05oDCFbor&J~5>^oNLQ-<6B9U~<}YrVUGICv_G{<+$=%O|IkL~eQ1;oGHXxUMtm?^r z$UcUtNA}s7tLL)M4pGKsAIq-QuY+uP9O5+yz}gY*3yh&`x6cZodh%leh&@aQapayA&joRu zm{H=Ed)A^s#O0o~{OJL?&)Oat;QVQ)`*jfZM}7&&J-hgGLzjDYgh^5Eak_H3$KfOQ zJSh)yPk+t|S@5h6HwG7*!Rr_f04tDGI;Uau`qC0Vl#DD0FHigcxbb#n;)jy6Cwj2p zD1hlCekfL#x{w_)4Zs^ysh1A#Ayt4Zu2|vfD-neCNFi2;2?i#JA*T@$(oc*Bf#Qej z6G6^L1O`^HUebGybgfVdY#Mb|;q18Qd3g6q zw|``(4IU>u1V5`+D%xL#nKAPNFp8)FFF+A>>Kouo28$Cj^8+xl(OTvQuBV<=nUN*4 zdf7?-z{Njr>+kKd;WJQss9e+U=GI-3R3!#Yeymg_>VlS=?WtMpgZ3u;( z$E^)zM0JKTWMB^KSFvi@yycMFs;2oVQ$&hxDc8OEiSA&OBb14Jm>y>;yR+RvG9 zWhgMA3}u$Z32n$=HzngjFfv`YHh+Y=?>ED5%GM!a0&~3|%5inZ-GVQ%T_3^*%v&Eq zh4Omzp)6CO>q7?1P$PXvCCEb`GFWlL(1#3O?be5k^Jv@2S?>v3-72A4i$d)DXChkp#t`__kW zl*>_w3?B6;L;;6FbYXBqw(7Bl;-7r~dK4nVc>G)yfnO1Lm{mEj$i)5PY%${(QWlSH zw@Q>ftw}l8fYdaoL?7G3t82u&5xPCBwD`FE9w-lg!kUVeB3Jl^%b){tTKGLs{QYop zR?oROeZL>m6)TQjq=Df^?tjzW#LMYyxUxto*vbl>u%-it+ru%(t&n^7Ct07E&^@Bp zHzu4MC`-S8#hhMR(qAyw*Oxo|J}}4e5g0i>4Caltd4zRYgx>?06V?QHpRgvZe7?h) zfVtcrnYFdT?V-Z-m)pa3Q!WR%J+Qm=BGQrXOgo~E)}JkOZ9`}ue1AK5ydIfZOT+77 zu%ebAe~sd~d*t*85)QA&lc=d|nMvt&#d>55V}4y;a7QT8!@x?OFiH2)y&}@XFq0dC zNDr$ckF`;xhrx|*Cq;Uo(mdG$B0a3qeAUxdh!kdu)7-cgGR7&?BhPf3O^OvmAJ5gE z<+<8slYT|02Wp`!PJei+17Bh))Wg7`5Fwm?H={?WM|OKwgn9%fLOrZk)l-G3&q{hj zsE3W}5{2=r_0I|03dM#Y4vQ>?R2H$b&Y56Y7yiSc`ZESXN?M#u4ig za0Gh4?g&;6BrEvO2pF3QNNoU_E()ZWP5`TJ;;IBG?X%yQQj_hKZN4@Ga z_;sFiVpa+yCRbV#+W4dsi!kLalTN#oXn;>Tx!Eth=mb>xdrp)qdy*h(F}~Ns&-eDs zj(pJxjDncRr?lC8&>5iKb6&7L=OiVZ7f66R&#vsWdB)qH#2F--aKIg=wJ#mZ7O{+K?T)5<{roiQQwq<_$oo%1;yjyfy!}*?*JI!C@gHF(b zUV3+E*2m^sVCGrt&;6d$5%G8v2&DwQFmB`~^-PZtZds#HdN)4j#5jF1v7IN2RQ6+2 z9(>T!e%*gBeudCJ;A3}F9Iq)Tf%ZEK6|+fgNn~y9==7zXGd7OD?V>b zzJK%{%L;YkPczHqTvx-sA0NHJc7#-0gkAk2aK&fqRhKJ1TbXiQ@p1SlKEhmT#ZGZq ze_)P!;=R5xN6pzvAN-BEdM$rK`H8u{-aOCi2j)_I9OH_Q!MEnKWoJ8@&u(`7DQ>_$JpGy9k$c`DU?iYh_1Tdu z!BwBVJZxRVUfzSck39-e$~EaePQ^#}dC)cFpn^0bA#NK{nK9f-k=l7WpKz#DiMVXP>$yVDQe@|A21Z>gtrh;%#g1hHL zxG+-<%*te;H7RTRHeebE@lkB>WaoWp%#5}Zyp(@KKT zHyYk>a}=Dp%}e`hvcFZ|D1V##Yhtm9>D6aC0@eXGLO(5#5EA8aXXDA?4<^+dNC>2` zL3Eyj1daibWj|p0Dh{R~-UF~*_a-OtHLj>Fne4PgY1!*p9|CZ#MR(Jo{@QrAT-ao zOy?ul33omC5S}j246}D(oiKn+ywXW?I;+ET`k5ih7Ci39d@8 zO&u8ChQL6sMqP3cLBYEeTu!(~3LYck`pR6(LQ=H*4P;XAoc-K?HrI1wVs*gZq5;E4 z@QH!V=w!JWAr|P+cm<>Aih&dnN_Jp)22X-|WIg$w2WMbR{(th|G3ztvVB;K~ZS6Rz zc#3ahPcvZclWD!DMa69Z3yk(@^AjptU@L0VGnLjVDFj7tK7tR}OUr$b{13Of@ElE~ zp)-WBZxkUO+!doQQ>?+F3h(ljl&I8(ZD*N9X8_uzxAbV+=~J-wXx$B_H{gAdk%fgd zj+-wf2nE&!o_|5k66xsTj4po${+tw^yU*H181~VBL`Zf*%ARReB_xYs?W~4`WGmLQ zniZMdj+ZfdNVQcAQ)q_!Bp(v&(-T2!-$U`j=khxF{c+7^?w_2zU_U!Fcv2LfE&`-0Hpkh z3$jX-taA4gv&Cr^TU5fO671xjI<+4oLy>gyhC3Ool0Fz_Q$iLgb?<=^J_d_rSKWuAUm|`!37Rx+&pjru{)SAY zfytVDr+)-%48!vB89aE_l=R0i5Y;#Hw9Y5auBV+X4gpXVNH3yeD&CQ*zz7z&A<2?>^~MjiQTAw){3=&&9@%I};yZlcCy zjMzsi0qgAJ=ESI7^~#BK*4r{T)tKmCE_avWfcvm#k|OiL0*2Z^_}cmv8DnGXmz*N+ zUVo_0`@m!fOhoXr)R~s=GA@g)&=O!IX;QK8&6Orpoiuoas46np5^s4#czBl*nHgNq zt*qr0PeuaA*r3dc+8{n_Zenzjm+@`m_xVQH%zfiq{+nB++RK$Ps((~0_lnu~$?cbi;micxLs8c@@gUZ_q`EGH7zTJ4{1u+c1ndTVks(t&*+nk ze2<4Ptn1^Lr2;v%x__KDP*k-RQhB;ZDY+wxRMyM%V)LpG{$UF-FAig{q_U(Ml}$fL zvn#fpEDQ2A)?U*@Wia2J!WvWR-wxU&8bVMmCgycBTE z;^9xSnO^s`Ere{YT1x4m)$9Az`bVZfcv52RyK3w{cZZdj+JAJZc&HkxCGIGJ`5|Ty zgR;bpK6P6?7MGMT8CmZav!>u4!VXwEZ1%v?v`z+}qLyR}<)gCV-XsWhNcAnTJ#1cl zJQjQoI7!=SEQSb$e+Nl9=CI_t-6;Ekxz4D1{et1m^uwW4Vt6OF%C#sk93?P)WXs~SD6cAm zi*r$**5R|AR`%1PkZr~tT274pL;-dY8K(AWaSu_tQdxU zBr7u-KP4;t?~$w~b>fh$rnvl(tjtV&OIC~pK9ZH?0m!##H508zE?Ujx1_GYCv0a4q zb7`5}A1xLa%S9^-%zuhjV_t&Q2(_Dn)#x5|;(vmbFqarLD&+BR%*Bw^>le%stnAyj zU^S$VxL`FP1732~SaJ|huCm3r%6cZhqd!;?)!_)}MOGRfFLyms~Zb zdw<<2SNYPrT;-c{#o(#%WW4aB3a@COXf8;8AcyP&8Hp*LU8p}W7d|>Jl z@fK5N3>T~>pR*ppibV*HVC9SNg4J_-#0mCohWkC<3so=o>oPs9rO}5*9vGRP4DMto zFg+POIk@g+Xk|gEUTAfP_o)!n^kjNkn}3!BH$54wJuvRDGtYY1o($8;l315@BWCt2 zsM$ImE57S(jAEfp?h#B+R*6!kC^tP>uIJaWJng|F;g+X8`Nd;-+Vi;B$O;^DEKfT^ z*I2ThDW}8qWU$AgWz}qK9{vW?6DD#6vtg$B!6|nNhbvmF?co5EjdP23< z%N`FpeTp=q?bCwgS={!tSPkK_J%8EZT2wZ`ju-|V#-})~VSM_SmMWYU#>j=35EzLG z2A3lxY-BpPr3Ax_5>kSJB_Ana<5BS~C2TzGIZ}d^0#zgn3)Q4Wm$5JAM!u3GC0J$3 zTQ;{!mX|>WJ%;l|u`elMGtYPk3R_|bqy#Ib*~Tp;IOfO%R)3!Uq*;kIOA4I6_i_jbQg9C2YJ|xTOS8fola`K=z7^LAla?A0eA} z>vxOpp*1Qw+nrZPOu*XaBY!5u+DlA`-3c)vC?Ujz?NO%^6F^AwhzVOamY1JM;VprX zVDKG)hDn?o3_a%K%G#(Pbc6&{(k+*euoa5wLV_>7WQ6C|s7R{<3PXttjFeY{Ykq}t zi=wTR^u6*>X|kHK>X%hry@GeP0-LTgnzuVs*uo?E{1-BE$KRhYGk^F!2&Th1gLF9b zxWFuxYUK||x%Dp`?bpZP`mv8O4vL3oouj0Ao)+!T{R3L3>l0c@AA>KnM)=$bt?Ss& zy%9$0D=Ok%8a7JemRtKpG1PhVW0YvUuA}{oI@9K?`z+-Z?%=MXxApM}RwzA#$kRhi zRpWMg>|E@PFw(K(7k{K?tE}o#voQt{KFY-9%?r>EMhWTJj}A-GUM*^z)uxiAA}DFv zSW688)$V+)rD}idMie>SCsy+xF?wkLBB;yn7DQ-UWm3geHr4p4lb)iaGg?{yH_k@w zem4B{RJviKL31@2Q3<8IKCYQjtTd)po(-j~D*k9udVFTNRH)>}a&DG3oHA{v>m$pW{`-p~cgu zb-K={t>m~{Opo4SvB#?D_=-mY=&|JLNBEEJWmRwi%5Q165i&rUaZJL)J(ZPRZR=VWwj}j;uMp+T9{Vi;UV-mX=;dv9f$Mlf1Hp z%k5^ddc(0@j_8#1n8=>@Owk8)5lL{Xl@9je8_Kj!aUL~^>%>+WW~0UM+mwZnD-FbU zyQ;igjbH+Vx?@B6NfcqM;r$r8^y~6?!s(||@rS$5SAT4k-7>0IS+_@><86zo1{_>+gyN4->GFy{0{n3mz2c8h z5cd&(WSDxz9~pd$Vr@IIdQ~%~@rn2&)BvnNQ@{DWFBm2@x~a9XT*qlKT|;68&ZgME zh(FqwcSm(e+ZBI*ghI2A_#?+$i`j~s8$U2>;Sti$>GcaH;*U@Wxm_}P8}th)0Lfqm zuS+Sl+4U;`$=0J@YOQM6-h4|12BL*UV7?*%$?BELb|`M$i^9TH$*aS_B|{ z5FZH=++VaqvAr)3X{ai`Da?sU_FmtZtCwT=4DSBrco=^dm;wHC_xi?oum8jPU8_8f z(XJki?)&VRI7W_Gx^XxCW01p|U`?T(>S#PhOASPSV1~Pf=eB2WpYz4y%|au7%pQer zsO;k9DcVrA?GS+xZN^^55!v~UUqOfT=N8heDBso@>V)MMab5jQ)q{tGyw6^_6UHTD z=~ETY!q|UHU}!y=5EJZGnOrGTpva>R@6Z(wnz~w0Q{*ZfSboUBoXx`XxL+}`eU4ZR z>@~QG3$`X$1M|;xj~-%?@6|nC(Q?bO7^Dm=y1iDT{+LvXdNsFG^Fqa|K5n3DzOL#v ze_lSTJ~dJkD(Yhg7ejtW>z@S6 zHsVDO7hjky7>rS-lw6YavF*j9!Zf`azvF335M2o5LPfaW9IdOFJ}H^fl<3}-_iwYtvy;DOvdRH zW>7CMothQfsD+y(131aW$CMzb@x~t6E9)(_5gqEuWBMql2(zs|8Yn^%1l`hquZ?ICK9?ehV;xfJ94h5;yTe)N z(DnYsWvaZgdLF*8jjnestlSJs6(4-NR}{r^%*N*ZJTRQ6vy;c+nIw+mKSYoE(W1J{ z!Yuo#Fc>DonSvgf0jn9NCDMxmfsmv9v&b6xJYpcrVcsl~d!B#q7AJg{`!`%A&S$&C*}0)Bd@@6@g7KP{@Lbf9 z>CH#+M6T}Of!}^FDsPctbaS$bM2eU7QL@or+Slzp_AQMbI;>y{H~aF(Or$kEZ%RX~ z2cSxg_G53D&PRrViadA&3hz25qx}gT6F&foD@>|`e*n|n{}XMByF&xRdm39=?=~Tz z#P8e~CKol1Sy(@y=;WU^8uNdRq4sNJwC3IBalI-cRsJ5-4rDZ7cz{^g508*otIRBD z?;jppMPbTzyCC1|p}gMGqT?)<#e0ug6b?LL4pEOrt%S#4zpxlj@1HPfrKvm~2VtTV zKEW{}sTM$mG*bQjdnOtR@r|J)5f$_XGfE6p29h~)F*kqm_}&o}5GsEkLfGpxl(?XE z5;@p1L4^>{Q#EiL!@5@ygyW3UNc_R%>ETgD(XCrr{jTe?UEoc0dB6?0-VeA@8PJ2+v4E}KLGFOl5xLMK|5l@>*wK4Qse#M+# z?Tg>Ec$w&5_g-HZz2H7Dl}&(@Gg0ox!21O^G3Nz0;1(~q0giv`1vfW=f-U7ak5Tse z!mY&PD^i>VHS0ZB_I*806u{$oxx6uZ-7S==f-ks9Td(tgo24stf02~BcR(c$gU@sA zzZd=b1MW|XR?SWYSe@&qkLSaggOMfSAZSk|EQ19YYG)p|u%{A337JwyAIb_ufvZPU zx-wB3Mmb?vVX1$2Hz=y`V0TN9*B0*G16;{idZU@r1=`;309f>>j6jJNivP}17J6|D zY)(-fQo%adBt0INA;HT{9z#N4uElJM0QCinLA191=f-GZW{461--2#}cE7axDmWYzr@amk{3h|QJoPx4bW zLn*2rSBIL1^I(21OqbX?8c%Fj=~w0+K5=!xNETCRMy5WiDy}K-g>I1d)u8lYEK)w? zV#bY|#`MRXRTGaWMph^viYxNN4e8}^{Ks~`EL)hS&%=ruG*37YXF zwV-%2gvx){ai{<@UdNvj2zRHjFdC0Kt1!*tAwa4Cmi@Y7NFByG{fg_H*}g9{o9WZ4 zps>=@qDM3_gpdJqRYcUGNTzbTp%&ZFqk<;Q$zhMEI~XrxJ*r2uv98(s3vMr*P6~2Q zp4}}Mohx4YIteq~L3Z38o<0;#!shT3e%fZ3x><+W@L`a?2 zsq>IJ?vD(Yu+Ts%cC&)U9f@*S+;Cm|Zp7Ov=ZK>0(_(~G&3LD++)_xBh-~O>;imjy z=kVnLb?2CRK;7r0c;NT&7ruT6$w+xINyij~m#M5-;{h=Bi$&Pz=1`F~DUgi=U5j^? zOV58R9P6CgP96xecGRF7MVioyMa(cY^7KEP2;nC;1!ZsGJ#WVa|;Vn5w+RKN5Dk5wsCtcv=vh6Wn zAEpX#s%P}W;`DO2s5CK8s84TL4$0w#teSmD~!3M?>vC!n1Z&$UTzeH?R!6jxh9ucIN<525F&iZ-R2a|0Xn}B*4k@kkoiKmZi=$oF9g4^0yTyq)M-JpSEEUr0tHq-^RoE+k zt~gqhcx#OO^&&OG9Ggi_jJNs596if>{etm0T7J79p)w;U`OD*IG5C5vx50Fi3zfbg z*F~9+eaQeq-Cu|{bHTI^LlZI_g1m`dUO)| zg-H@6AJ4lwOyhv)&|NuJgW;h-v6e+= zaQX&&wEG&bv-sUhMn1NOsl83?^B_vG@*y2f|R024uyO--ShsmeoDzE!?s z2CV1sq?@fzs!&YXo1{mFUX!>>MPrGJQC#p;Z)5PbzGl}Y`HUGDxoIH$7_5hU8>~e;UVQl<6u=w89#_*D$SB@}Vem3*$(5G}H=~GH zrG~SW58Ny@5`PK(HAU?NRK;kJ~UdHSRdd##g~K(vTZPyRyHgXr^(Dv!C3qQ+Am zj?&?4XN2i0Y2?4qRL?@+@aSp|;zU<-kTnKTK0Rt1UNX;F08l5`Fu?m_I|@?-oU}4p)q}N3Wa7Os!_xXBj~Td5pn8 z3xnZI7>b2Gn1z1~*gwRKPXvBs@pH!S&EQi7H~Jd9%t0Hm1Z!)+>>;MbDj0&%gU@tM ziv^L;eiJpvEHadzJk>0*28)^|$`k@*U7tpNK2F)+#%l?$Y1P8c0G^5)8TJIg%d+(3 zPMGY~GLyt&)YNe}ybwW8VY9bsKCSqHId_>;*cV0=#(RH2Z@!N`&?+20Z6$$`Jusi= z{;G+%{9R`EB8tlsu&`uKXWDDp15sk}by);a*zd9kqKscy1X0BAVG%?bzgJiU4ZNbZ zQG>5OEP_^X?|OIfhMT8^V^!+W{N;uh2_N0P%@lA3jp%L>GH2b4PM5Iv&8{Gp@ zWf8=7To%C~Ygq&Xc8lN#-D7u@#5XL0z+4tV?9YE?5i~g87C{^L2#cU?&b;qkk?D2s zj>vJT(xKa1@c!xBv(Gq@_2o~i$XYIYU>wu12YyURRp%02TmCo*MYsI9tNrE-uK|ob z1}j{%sgmGSNM4fCfdH6`=H|H7boA(V+66N8(P2__2-=>*pZq!X&vpkNg9z49 z;sLv>p)6m`Mn@;Bfy#3n<>v&)emR0~9Rq;Lu80CLW zs4RDwJb@3VrR}~hFXsy9_ImP(s93224T{<@jKW$~Tx~`E9Pr(>{5q0J8#OG&99T0h z$|$LTpB_{l=D|94SBr=YVew5%XIvfv(~9TuUL#K4yu^xd)r%teZ7<9vPp#}RgrArr zPkFCzOhjp9q|p8qbM@N(B=8e+eZ7Bsc+m&O<2C|j+y=n?^0*DuxTE}{xD8dki@1%v zq@Xcw!$3t{>{CvVgvxrtYxwI{RtvT~>8}&Efr5l2bcAiFK;}6rYr3t=57U}=J{l5_ zTCf`AayL~}EZqK^BBt#hzW9ENda!x^K zpO~vxyJHKlU%o6={>z6R{CB=Ex7EKSJp&BeL11JrG1yCcnS5T-a$Q3Pmy+W6f!UPQ z@3by&_yk=g^$?Sd7RjYpva7wH$?sS<+(f4UmL?{Vz6#EyK0Na zrH9qU2Dy(}rd;VJ&XnSQwMk#{<{Qo~kE(l)`_+FxdCC@zAo(Lb4N&kkDPd&Q!7q^O zz(jy=aI47`FbG|Dt#fC_6$c#j6xOtsl~Hr!gypbAA7+tYHP3*J3hDXuCnFrvAMen zUVxj);7)B<<^$|E**!Ul@3uls3{DLG*W(o1OGS&}F-q+AVx{+hH$?Y~9*kKG^=Nvj zW1}%x?@yhXl<8ia7?o;L*)uo5dV}frK?x0+*IL;TU^agyXYOLWlas#+#R0GP3rei? z^T|`8*XcAke*4w6-IC|FQ-?nj1&ggi;ij-V7^rF@d=+{yp`ROO=1+{?x$$1CJu-SE zjY1{u^uC~|(3PHN15WovP$Px2UlldqnJ}?Z5;}NSXeG~8QMMOl!Bt^-==D}_=L4I* zT4Jw+(S?7wDNQcF|F&nBP#xow>N@-LX&wCZa z$U6fgpMAI1^Wvo_cgT4YV*{l*f#;}41y1sto+7!5PDQ8C=I?S0R!uyfJcr|DPhj}; zCw_lGFN5$29`50bnf#5amOEWpyCCVN^dE?%}ehx1fx( z;fgXdlupG~3LeXiiBHJJ%(RIZHYrplXTx-2VAnKwFb;J=?AnY`ltJdNtzcs0c$j}^ z9CMZ@R{0xqrKWk=zyJB~|Kz@ztt#`)t9j*-LU$DKdZsh}-y8kcf6~kT*-ws!!nhei z9Fo)A7<>J1uw|^U%g)k^j0TY<+x57Rfx$3VIJvNx24L$@5PqKrQ$zt~!coImR$WM3 z)?k??4-mD)D~muDi^Wn02qnOv1MGi;02|!2Sj~nHqGT^% ztVL}~4rn`M3HqRRX~(#Oc)$I*#~RWu!XV}K!I&h-QLao_;NTZ(OseXeQ?q|cgQK!6 z&e%hZvT6?z@Bs`)SqI6YHZ7cGI*NJzcGp<1&A|(^mA+O0Yj_K8_Bh+6mq&WV3vsDVoh2O;{(rk&y=8 z=0DC!V4&v-Q#LU+*#QR#VM`D)CbIG4H$;MygeEH!;H2|u87{|!2@$Xsa^k86CaZf9{`)*ln!gp(G6`E7p&AERNK9pB&? zH9L(lqyr=47J4{Fr(lTsI*;8j7Yjh9RT>ac$;d2qlZ~O1RVH*t$!oxs%wWKNP2RdR zP4%!JvSmr)#!RP)b=>05ojXw)Vf*jdZ95@<|MNhM`Xq?WJHy<8It9HFE=6}n_unAD zNaXpmh#MqwJ&TXYghzjNf+@m)U(C$iL4hVxhYrmiF|W#Fg0-4><4(8}&Mjc9-6oe) zgMr*1{7Efk)fkxy+m?Umqjm|{?qIPiGu#^NcPCRQ4Mea7=I6#h1r`P@#gxU$P4*U~ zfk_%-*)DDRxHsHY(&>qEaEZ=`IE?0>?)ejU~9OU%i z;%$*h)8e2|fpHc|6(Fjv+W?#LmyAb|Z@Wxt0%Y=x5f5c!Z`B3BycZybdQ`B0WRGdTdToTVRY(IeO&B0gYI1Z0r zpy#Vaj*AJmvDvJ2N3#6%pmvnyVF8>pH!M!TjYhqLsY!n+`^H=pP=CRoX6?`2>l-uv zu#B~hzWj)Ts&%>q_Bz+D?Vv~gWz zn>FeYJyw4v?4%$mCWsiApD(h80yp-iFkSHa$Vh$TT64_Ljp^h=n}z-$C@jKMyPBWs zEwTT-=5|#Y+_eRjphOYobR4@#SmA#So?;^j{j2scyLe-dFSS6n?YqVFv7oz4l!zr~ zEtjjs&?bxVY{X3SlU(wR?6^)CSSkv2BAmS?Nz{Ls_KsX!T8C+wa?U4F^zUH37cqr-E3qddCk=Vxk~sA?Uq)iiSO*gHE@KMIAQ~9J+Nto z=&(Y_DxgJQ7%{_rNiRVUMGPeNw_I2JZc;o#2-q&3c#C-ikIa>8`bMk1_|*lR@wzA* zRW*Mu7=={-e$hU*$JKjCsQ+%vqB6OqLzm5yx<@EPx-;zwuUOdN1~myEkldV7k7(z_ z9WA7OFjEp1Z|WD%awf^N;QMwUImRJy7!KZJ(yi8`iAU1MAOtPVOVQd=1s7a~}k8zCp;HdaA89o~VJu~3Se+1Ns~aW~lK z${;w!&i3||toblNgLwgv*BrRYW^rT8DqwIEQ!eU}1#3*qkZg$QV-nZilbhGX<#T_` zLWoj0%OP>H63h}F>-NnZ!nDO@D*A!GxUTv$$#N!%;RP=HYqRkR3NYE>h>a-X;>c%w z?KWO~Uhp=V=|KL$3wd7R{07_~B~DeP66c9(|J zmoJpF!zp^&M+d^vuEQR+75&kIst12yZMJ@JICboH*rlCa@1GaXc9xw+1SkY+Kl}jm z1f|3l1lHgSQ>#=9N3;cx>;xC zDGTtEN9C|)+)bI#7uJ>evzE^isxzuFQ8Qp7Zlp#f&r%KpiqC9;pJ}Y zPz0AaXj6ueJh+xwze&EjX>ll%Itov}PR3l^-zrTIM4Gc~IbBu_+u+vE=?z}rAohIR z22g`$amm@$plR}#WDDEuDaFIpplLxuZtCJdOXAR_1{v%i=yD$|Ni!4cj22TCc%9cg z;zH2z%R^TJ2Qzb*MZW0Xa@c>hpumKhF?0L#JHmC9)8`xGD$mT)r+>jjFJIJOFz4gm z>kAWl(7{j<0xW5CVHMoi=86z7M?3|b`+dhO(U2=b*b9l}#tjUfy2kJWx)Y~HG*ZS1~<< zCCOKab%uadiH0mGPEC%LoB+f0U66j}4a3HT9HhJ}UM@ceW`{rSV4&gFqTS`iV=#_R zLWUXr1`I%!vwE;aQVVGCHB%@|NkZXZLW7L(VbK(>9%jKNePn;avanYkvBc^`ppYTB z5pFqPtlc6*j~0tUVdBxVwe1>$C}7a&pQNl5$#{u{4{W!(CRuzt>kiD%jd?GjBc>2b zbm$F80lXG1Dq5BOn-|zf>Uy1Y?UOEWRVX(B#U9d>(6f9gZ&snP?~9SqTa4Ak*?>P8Qzct+xLt7rWPLbA zm{isY{ltG{CNTTJz;Mfyy7*n#san=$6m())b~8uf(FRk5Qv`2qQAn zgQsRn!fo&pkT8nY@}{iTZc5W;jJtu~Zbg$q;)k2E+R*lh)ik}2l_Zu)`o zQ8)rq45APm>QJJ?tbzIYqG!@L-e5g=?#w~c;<1R?;^%)cKuBL#tOkP#5?vThP_}5I6d=|TNmbIonsKDobcdeq`5_-+xss0VIpT5&c9X@f5fd%ppY8KH!mBat z#gAW1EDosop>O4EIg9XLNF?<0i6m88gP5-WYmfn-{c%i}C(nO@u?P5sXikiN<>0zi z?(}+#FF|F&=rz&P!o^Yl1s7LwJVnP`%*snV7Q4&c{@4zhB9mNN)FS^8a}snYLH`AL?f0g=D`BC{qqtokuY^Z{a-!{dH<;SMBscgFD>L1}eVBx^T(xA7 z6vc+*T3jM>)Ag{&h=p#dz~7il%uui2Fo=^DW(hOpwjAQBh>FsZV1?~YuAhR;DZk2$gH{mKtKQJtKjX` zHUEQwNAldZWRu|O7UL?Bu7-&?)FrG*z856RGU&|6%)4%;Wed3OT-mY(OH$h9 zMJ-Lr6lz%jUgOBlH!Lf6VB}{lO%tM~VsH~I-(6`^GV%PCgA!M+gLZ4Tj>Igeg(9;t z=DF2k$SxkL^73Wc!oYtSTvFIq=%=BB-dL);K$bKy#umnA^QnMs6UFT*BadNdB$()U zHk(G%18?bKHwJ^qVxCRz3co>j_7Avj#1YNV+~(v6ix^H00v%!>@Hd+SLUzSTaE+To z;Q&b5$fPF8CzLH_ag(e|ABI`uZc*v7h+k+Stjd#L6v~z8nJs?`+E27F%*(b})PWdT zj(>xt=fr6lU*1}BNBwUS!5~}{gBNQ!pHsIVvjoGI_%L9Gq*h^9H ztc1JZ(R`VbJPChvKe=6O20NWwXCO}U(r@I+Z5V!P#Idt1r8N2*hC(=E9v)mDUi9ZJ zKrkaggJ_Q4h9nl08@Q^^#l$wi$Y+;8&69DXJ56R0pMN(-u~cSb&72?HlQf-4M!BNQ_}(mY zJvGfB=DN58A8M~d^WiO;ILzNrMegwpMIS^jbl-pLopf;e6_q2ljO2QJ>Nw&bKn_yB z+}Wa77#6&X%nEr1L0D#W4AWcqH252QqRuybD!msm)#~MYF*LR7y*fgWd|;HHd;^Rp ztR{#r3@wgSI7PuA!qP`x?{S51Z4UOBlQ-I0Y!L2x#o(3K(pPv7rL-m>g3mtwcuXS zmp&2e6BrSun{J(M`=HI{Pt*(Uwak$I5v+9BfB;bnChV)vKN?7P%G->L`H%3h7R4-(6E@zc5xcLp? zg@NFQ%7aNEw}T0L0q3|e zpmH&EWl%Ki!ON7%2{gFHZ4eb0`DDbB^JeaW31}sT@+er;XTB+5zkmn@7G{6?WsZnp zO1B?7B#LgcoWyjig&<8xCA!$*f+`d45djTtaQY<*BP)?zSYvkaTqDYQOIzt8CwNe~ zIG&A%oFlLmn0ZqAx2gw8tANQ_ZuY)%k^ZXNq>YWa^GyX+~yJIW}h*%sddrT_tro+J1AVNQX@r& zoNFtG0KK#hj|Cd^i2!D?MUf7LqcXD~L8X!}GY_(sm=V*cTjM&E6lrh=ti#3nkePqj z_KW&;tv0imjNxpyVj*O$Q%=9L+FR+}1h?dS(}iI#eBYJC9+g{5%aK8@@=j5z%wT~* zGMHdK;qezrIcH)M@deMYn#6I}L}W?T$Wi8Uc8i=K&|Fs-jA6aBi9tv7XduS|f5DWz zY&A@f^ut6SCYd>_6F5^%Ix88R!jgYAG*N?I%~Q?>7lsj&xg7&d(PltX=-C|R^p5*v zLN*VYqFqx^;;c)T7`g+F7DGehfhQ(2yZOYpRcaGV2xcTzAJVwtFgiGPOcb zxLrhw$-!J?hT%JPvNC-{3kai2jaW-McpHMRl+Dgyf>VU8XwIQqwz`y)Oviu!>Nx_L zITfsMvhRs`w&VrXV7DWcI0U6g@VCXW%mC?vu}i?;I|407Te{#;vKZKVl&tN7OiI>b zI+{z#T1;sZO4eE-^qF-`_L#FxSW4f!feQ|?tkRZULVszqwY70qUTIhXhKA*sqrog~ z_rG956N;xkF`-c{ZEwF|;wgWh(O)p<>;BXS=IB^~k&d;7wht*;Ye;s6lC|W?=1SHY z(p|a3ESelIC2NTlUC&w+%b{m2X&y(>T4HBcw3Y&?d>35%)~iGuZM|D8e?$NBJWrRR zwUelYiq=lip6UDof$Sg5QaxO;# z4c!#BtduQG{eX?;i}t+$We`m5`kk3Pvf>Bf$*iAq*aO)kvB{=Mq9tW3)2X?oNYQl8 z(TZEy+S)!b7kT=7J~24n39}`nyE;3xKhbo&bcC!fQTL)@tnnTgImQmdg&(|N&F9Y+ zetMDD_11>xS(+|R`vQMh)xakHEP7S8nKu>4E1T|7`shE}=4_+|74c-7gIaW9Gt`{3t6(pJn`G@I%f>sYF}ewhgcYlWhw%_>t!j3Z^jZ3cVBrMjMo2ialX^ z7+S!mjiCV5VBAAP1{6MENp**U8zxWl-lx5RLKOQES$upx` zKPSFooM+NyQDqDqQZ}1NP;< zThz>V+J=A9xf~Usf1+XwF-+Tqyjj>H?@ew_8J$`@mIkJx(#ZGW?>8oIU}o+ln_4l{ zQYyPqIP^j10_$kz;H^dm29gL{MGOvg1uPbx{cZt&SbL`xSOc62;5N2TXPM|#EHXP; z);^%Qp-kubjTN0P6w(6BK~P8-1+YRBFslI5in@Pf>}`qI$lf^DAS+smK`dg(Z|0)C z0ExO=4}zRJe0$D5*`f8_J846-`)Sc8 z#d$)FF|26keWt05=!V6b``}ckg$LN|9F3Yc9|DVn)5<({JOn9Gx94MBNLud=EVT_|(40AFF>; zxmQ|m5u+HG<*Qz~>LvvmVTkvnyMF7p@nWXI%9cl)5+$4{Rj`)Js5@G;Sb;}1AK9nK#pHGgydKw8Pe`8JS? zt>LC|^ZkHb{c4aYcb)G?qRMsnzGD*$zVC~a<9zS%;d{q8-*?W8v;Ba-;l+RUBSSCD z*Yf6lG4+tYL^10C`y6soGiI;b8eU*?)7fB^8|I9pfxNCJ#%_ z@&lLA7t7Py9V|aObwFq6A!Y2ayvX{rD3pHA^5zWo;`otCEMobQsp31!JEI`e26pfmLFPfi7dUmz2dVYklRxE#wnD!zHz|9s9 zk~Vh};R8HIX2O|#*(BA7F^8&J$lbI31fisg_w|NfFc#a-?i1^IP5du)TQ^@p!#G=!&S)*bJv4uMgorc4h)RJ zFnt0e?b~XpCI|=c2dZWpoT$)FoT?v8{lksU3U33fWH2*k0ZxB}3oeEqOyMq{Sm4^= zs(vOsfwWE#3rkPHD;jJ*(IR;Z69a(D+q|?vHb+ow6|=o}#} zEXzs{+l*A&PfUN+(yUhM=$h5aJ;Zi~`gK-%!G8AlKR2ecf^A?BxJ;0oQUrE#_iV%8 zP|40Hnrg6^wQA|b-5qrU*ePa(Z*4pNf`HDi;3F# zZjmGn7Syl}-)N*sr^mg`y|(WTonqr=lVez1sm>mxntgvQTMf+jg)vbZm?eP@(-Ln# zut{+J&7a@EzGA^Ql^-(_px^VHm@v|FJo1@YcLUt;0nUOabuZRGx5a7Nxj2=fjG-2_ z!HxTa>RlUVb02G1twBX7W88o{XR0kdF#arUD0*O%V9Ww{;Dt%}`yrxk5-bn~|1Qjc z(x)~$$W?#JD5)X?2Nz6D(*brZ7a%f%W)~P+M$xoHCHu{=n@8S`(#%{i!mVmSH(6Y} zDbp51B4^Q2rFKm#kHZxOWc5U)#lh;V@(ny_P#xP+x1^VBWO?L4X0y-v>@pZ*I+83F_Q~Ub4I^2I|GCU$~)6+*ZMoB0I=;lUS)gm2< znC}ZSVD0-p4}Bo6XV#zh4p*eS9fB%9)U$ZY55#%K`b)vB^K>h~Hf#nO8B zz|*fdC|Q8f7S!+nuB*?cMc3`KX+0P?YhNvzYquz4oN6T%o5L~bn$_T;gj@MxgNL>) zEA)RHaE0|*Pe=d`7LHD=2lN}}%#c6zi5YZmM`xjaeju)AHtp@7nCsgu4p(nXYV;kr z5v6mMCM}XvleTD%tlC2=QeE)4zSv?D9-JiE=nC(0xehc4U>l@I0@S}f2~Vkio%^79 zQ&WqA%2S6wK)$0Z{mq$`8^+y2YM5ODfVh7$*R>CQf_krgw2A!G3A^S2EJ|>_Omi#D z_Xkmv&>PC~-h|%PMdfD5n-`T^!1s&D8=T_~mABz+K^kr-@O4vpgMxXrKw98K%$6QuICoD!VMHMbX)zbTqjj;QwkVT&>w z3bh8BNS~(;`0Zeqj?uR-kDUkrYkPm+(mDF3QcvqBV3`?BWq-bxT*}Zo3e2EgF=Giw z_R;T5aA<`JRk~5!w%MWckMf>yGZDU4WHdi#szG{1AOkYb=b|L2Sf8HUqDlX*+=<_a_pn%!)Re2wMZsiJ@&s%ig){ zw2=ZX_1H++@9j2HEYF(PsZ4)_dY#)u+^0nfJpkJ7khZ&n(E29Pfbjq(j?;{!;#Rki z{v6W}pPh^=v!1PvO8%He5Wu&)0v!oToOtAb-K!U)Lx;F+K{jG~4z1J=A{i%A1RTmdHu_Ca?D2@YhKLflc2v{sh+ zMqpCexHsjI7+Kn=BcBVeJ(zgWFmmF7mC4{bB2$ zya*ey2_k?K-Z$pj^MIY^o(G19L!Xw%5CLZj0537l^a*jh_kat zjSN%0OUS8V>q*}w)-F4L>#cdZD8goP-7WGT65XC!3UE-v4l19mO-kl1bV2+F$h2Z9 zxZJD3ZRzhJfJL40n;3NC!XWGhL)#*WK|bu>E%E7UK@x?7&uMdLeO|n8wfA1KA;Vs9 zHQ@GPudG_{Jt)5D$8VUzZqkz|Bc}97NWu4t2{UvjmvQ(qBH#soZ5~cC?nRlAXVWkg zIM&MwGMm>h1Zu&IWurks#6Do$8Wr4J7#to|nS^bQ*F)@8_{lD~ahLIHUGwAvLcW3Z z51$`ho`01;`<-5a*G!*s1H}|{4M=NkNoi!+cY#36_lJm}*kD+JEl1_Ga7BVIA>>lW z1`tdPIS0igxzK`t6^^e%trT_zw?qaLDD?U?z0P6z--+bZB0Gul8oF}gL9+TlBD2VO zqCmMcuvhuu@>h3FB;wkXS)u6{%(ZXmEguL=6Vx1`ZL-U6aFAi?ews$=;Akc zv&oul{s}s?)_^sWKU++`bB?05ASlnoib8k5xQ89zI6z>3zSfuXhMs$XEBTT4p%oW_1{aAl3$P-G%GY*7Vj6K~!JSfFTT#XBESLjhZ?IH@ zn;^LwoOQ`T1#LZWCL7B-G9ZYovX_HMhnN3~Ul2Y};ns z!?Q^>*S7K|eXLA}Ds7c32335+V3b;LgzaLBo$@JclveH?!q83#4t7Iq?)#(2^x8p1 z+NVWgivp_v7Ecn=iL9cS2#yGpq04Ll*2{&vJm=RP*F5imk@FdhdY@-_roIKGW6pWz zVCd3+foNhXQ-lU5J-KJwNA~;Touccy%J>KDwVF!+0a zJ~4J}tNACHZMjGfY4}Cs@I#|^>u02iO39X^@4Qc$eez^`BxaW!h!(IF;RQ*4jS`?* z-H~WIXbLtOC{JTuw0jSF;1N!`zaK~_gce&qev_lFNv^F;qmVsg#L0z@Nu%)%-bv%v z{u+jhfM~J)NR2@((pcOVYADKMliP}a&askgcAT!|g5x~>7Jc#~q;7O6LQ^)Nvp+b- z6`_f$8AlPCls%UsG{>a{%|XI)EocruLP(Y&edtkw=5#HIN|3qF?UxcXrFpp$WHqlY z4QFnuVMhmAiMoQfSQK5M1Fhnx5Z8ggXqOJOXfO#KX!(KMcc;~2lwt(O92E$EoAyzG z=++e~&?W)o!57!4I12-;yj!kzD*Lt`QGcx-&>RO&d9g|_y-3ZG$M%~@1J|C-p?(VKwf?Tam z?QOJWjgNcEcV(xdMe(hcJLyL3+`W~u+Wgr{clOPnTj@^JhR;?i%r%>fd(u~HtZ}Q`R9j=*%CH$H^!zdOe!@vK~M>jAjQY`f*KD9MK#ql5yz zE(rmf3Kz1{9pJ7O4aA2oxO1%JcMS(e9j*YDewNm4A`FBm#{MxG{DY(osZ`LVvd0Tu!WZ6F?bgfS);760ZLMzMf%+HB z^;8N1eq-XM@WJqZ+;AbNjS2I>lV`T2VU|S`-_ni(%qB5TaGI5_uYlpYl(^e3I2poK ziOQE%4PEfa$`tz(Vjr-HpmAL5GBt{4d%*2*g%gX3q`LD3H@M5o?=5L(U7fGgzESjr z%+q+dXr(_q5z>98Y@SLOZH^%h8V)Y+z4T*lcidGV*~X23S)9u4M->kAfpPBW5T$?l zMCM5Cb{^k_^>pC!3NW~H=yU{3=4>t8)iXtbO=f}*XVInH_s~_`93M=AX+Ts|_Zu^)$t1ZlI9a9B&mxo36d9@K86FoTG%gaUYPqdo$R(gG zKf-FL+tmku`k9pN!qy`Td;uP^NLmnG>m15GOGt-TIw_q45q(4Fr70c^4@B;+=LaIX z;@y^EU`mTK5#T=JOayq3b0&iST5WMC0=(i(1lP5C8-{^dBhEx{t849&cMZ9{Z;9Bb z3!D~>+cFPDZ~+_TIuy|fY{sDobCp&cisH4XN-oE3Uir}%C<51+zynZM;upf#7qeBs$ zLRTD$_=R&OLJwquXxM!k@1Ui1VfFfL7j}=ot3{`#+vM(TYqPI25mzI}nJ6e*XQD&t zekS^VN&DaQ3Y_IS6A_c+OhmuI7KbAA^EM3wQAeDKuskvdmAVe0hKe4~IxvG>5_N4Q8PM+fbb7pSg-cAU>b1?_`Y;iH0@nQmTM zXz!d+r@Y>&L>$`ebSpV6DkX($p&g~(ZY{KbXXjgLp`A8V*FqgUTBtCWj@Oer%lyO~ zUD2QV#2lToSDJ`FG1pVeck7>+>+97`%|9@g7U~$+LVKm7hqTaMsSzG6)RGKbN~kQL zu7vjHRwY+LyDpqu3AJnv^HM^Yq~O*7Wxo> zSk5mkRQ68SLRk*$XrX=js9XzmDn451o36jzA67I`uE4Zb=e=H@ys zd|&N9G1nv2Z-f2JGxKW=W`0fg40p93n3v69O>X>0RkOJ2*Zw7hD{6D`g@&>lzORZqVCgwBKa;tRc&q> zD37W(w>$7r=ue)VLRAyT9!>2%FmQfOiG1Ag(7y}Q5PNK3y{$ca)Bfs);|C*e9+?RC zyq!KDn0wJhoSTz#TF9@aD9V&oeegn&&d%JMO~JM4+}Tw-#7A3hV)9obZ3=N%aK|tM zql>Es>oh)ED2}j6@9O-2khSfY!B@h@s%hCMZVhGg6miL#J$OEg7Tne+P0oOqZs+Fk znbb*j;0r6fh3W-b-s|n{2hE8IF${b8>8k00LeLNXds1)=5wdluQ#b6!vzykz30}|` z%&W$V#349mcj|^yipB2K(x(FcF)fprfza3upEA2m8(Qnd7dlmcZaPQI#Y76e)Qj4{ z&VCJ(6<)QIyU0^gajlJA@}UJ|cx~l@PzNM+{8X^Bd&}_7fy5<;5RM@GO$g%db~xbd z>I0B=L6g}tMA`ZK5Gm=Reky!xH;2rUtPM*A*6%Q^m&;+k5K9NxJ4dG z=_LVoyIg|-mh$D-NoN}UI*B|gUs94z@ec8W0#)< z@9tWjm|)Sjdn?Nc-x`%n@QJ97hsZ(ipytxpjyX{rIk%%`3A-qsL~I`-iScnnWA2~fUb z4Qc{T_L@giJOuxx8&;cFi$?3QRnoylds`gwI9{dnU+fKb1+M1AYRl1Buo>gSO%znT z+})Oc=6XZJW4-Y@_8}9GKIn<8*5qWL**sAaSH_569*IKiaPqLp`~?A?CBg~l10$)Tl#5gtY^EmzksLD=0y4Rb z4Z)LelgS*bXO5lpr!=`RmG2Hb>T+Z(j{XRL8o7v_7F3Pn4Wnsx{1av%x%x18;)*Uh z%t7b@!x$@L)%Akz8MFBXOKd7{Aa%hAYs#}Vk6^YKgkTJt!;BW1hIoS}>iULeGCHA| zj!wSULQ^}+dv%DhT3>WRqT<}M+!`jGJWvr`t~ME+D3Hh#p=iCHG?N$=VJ*#EcMG0> zmE#42!8_OcE#>3g1D4A^JrsN|jxbZcS$YF5Oy>E9?FODwk-ac}LD1LWwRkX`lh|QN zmMA|htWwj~8>;4s6y;%Jnn~+L*34q=r+g#dYw_2;4!Nb;Wc?!o8(dLGSLKfAoABac z%gPci`P)_BRUtHW3=h`e)$h$bOCyPY7RYXoPIwMJ)1#v3!{$k}(QpG0n{4-S3Ldya z)&rH?Q6tqHD<4sXFf3tO#029V=Ml|oToUWpNROhKk^gNJZAb}hJ02Hu7H^V z*5Coy-M0<1F(;nzZ|oorCII9VZkAkuXCL(Xcb z5GbPfUL?Fi(yB$}Fq(9T#lxIks41doH4;nnem8%jB1QLdx!R-=h<=Iq{z=f`WY_9T z206yX@22hErSs-J=q0{)*IOwWWq}9^-r}r70&r zy6Er`Lc)ab)eAy6>l2swRwqJ#z^jw70caT47NLQj7MO?g0#4NR3C(15LNgtme6NLk z6U%(>mAYWG;EjBLY!bUWQTZmJ z$&5|Hr{6b;_i`xp7%t(hVA!U0KQlyK7(3i&WSH*eO@Rxb%y25JiIdGzlr@y9z?=t@ zTnSw`x8_8Su=HIwycI4U7}L37gw`8+)%jiD=sl8js26sZ$8WHgCck0g)Ho~*LDnP= zRfz8!VRyJd(80h?F1S{I$f8bn+sd@Q_Z3F448w#SGH|5N++q8W_1DTYu-i+XLYZ(l*F$sA;k?lPX&wWAuosC67lNMt{v=De z^flPY;k41*rLXM3z-LHY#zUaos>*m{GWVVHR3`sln9{k9JFeWz`iXh>N6u>N9E6lh z539)!;U+J_ftwKRzUR4;-})yrUG4&&nU%24u<1eG^3At^Ld zmngAmzGMxH#UELu#gMeR7VHveTGR`pWe=$+vIwau!nx#ssz@s&FDWks^NTY7zH=zOZz-1y_2%D=F=sD;)gYFOH8A1Vq0)|vIlUhY zGz-8LEgI_5t;Hg+g<=j*BotjYjD?jh<_&*f))QPfvH+wuuqLiffU$Fw!AY>ia4f4O zAOOt~yL0`2+|*}(oHaM`0gho7^#Iog4~!BSfX$4!RCDzj!$cl5j&gkgF7=v)0b>^D z2@hE9U(TcD95Ap?&axT54!l(~D=-DKV>xstOi{mKC}3Y0ma|Dm$TLNVdF}O_shXGsy<99b4mRh)2BxdH@mgnO z2Q!5!ncwXq2D8{&zp69d@SYd7D4SzT?Vg=|j{9d-1RVcaFcU+>MuCdX|hb25y*sFb9gLW^BkE%v}!<D9)e3G3uM=SSXy_Z zC@JBAH`I{DP00eQI*#iQWbDiR$YO&p*^DQ82w%4FeWuZ9o>(#)-&7Vr$~5ek=*7}I z;SPn#%dQS63*bH@Wp&EHAaVaU-^yLLGHq0vT6(27NpQmB4oRDoC2Q0!i^yvm$=PC= zn(^9$H>-|Z4YFi%(YCWCkty4M%n3xh4kRvJQLudT1+PEL9{{4@(83zO5)#edFBbNB z(VkMy`nG+9oeiNL;iuKwuNG5+r*k3nHcuTXMXlf->Voyr{ni>&ldQU(jp21x6sj;Q)? z%eR!Zd}-dZFIHV$l88u%xOJS0az$p#Da0WE;S?KeE^--Sl&vc1mLYG>za@MN{=iJ_K|EuBTGzO+8@n?~#_E(+A_d)$+K%4VMDbF{ESQWpLzT5EI^i&O zlH(kNr*iWZ^>W8Dl422-%w^4lnP+io^PREqQ<}%t43o= zt3acFyk}2u4hjr^ylH`HI?7XLF?h&UgQD4^{ShcK9975U8on6A&wQP*?|8Z^9aX0i z?roNIDA<@_)}DBhW2xJ82guO?d*VoBwzF&YEoq|`ed%b$v#)v23=LZh%fwXFDCDu# z+@eu#czE`gz)Ui17Xju;f-`&HmmihnvePBwSR>bM7MqHHQHKL370;-AIZ8--4pir- z?n%WYmQ^HuaDWihjg!B8dsjKmbU5knMFAA`-*3$H3{TS0-Bo7kq~Yn&kZi*@Pryxd zp7ewt1DpWFD4wvbX`d9q_%_-{j7s|#o#TomnKs&?gYemkNB+@5#toekZDw7qX+bhY zyo!V(mIYsb?{p6=AGK1}^eNjLFyA*uIgjuiSEf@SqdQ z(~$MSy~W?qw4U8bcQmT`g<2x~Z#MhAJcpN#$UXdR{FDrJ>49Crs05cL z>naN?J#wfVDTGnUJa}Ai4o9O)T)l*o>Bukn6*_@`QJU0>aO>*ix-D5Ur5ZZDXYSRR znsQJ|=E|Tt?0WI-NcyUU1F)K`n`xC~PbLM4@q3(PT*$W^7UhN zxOYu|9oB6Lc>PK5QFOyWt;=dAQO+D~x3Hg0$1-nG z|M=brcf}GLnA})r_A?XJVY+^0Hw_9V1ynTVGI&Xbq z&Qop!`NW*B`%@nnn)zRt@n_2+QCnJ*Bj*x-T}pqMZnx-vhH|KF!Cae0`%i{+(^}}< zJ>|gI1$Tl=rhuw1^gnLKFIwFDTtEF-5M*q@H2vMex9v^(%5F-Cx~gHwRjpk;&{QPV zX+io-5HT>{A7m{jsQgE!QrTKo2^)?Mp@EUDg|VnPj4ey2uw!glIs>?|Wx3T^#+Fro z|F|7IJi=USS@P!!yN8!b77I(ySv>?rQN7%~uK8R#2cH>V^F8Bc!qa@hac76I#%mBJ zm{`KpvXj(1nOb(zC&SdT^BE@zJTPHu*)HK^YFS$u6Dfmkfl8g=CpvKZ>9p=Cz-}qi?+rGh5RWEoK~_px^Cr z6sO1?*o0Y19yoQojs(ZM>514919Ld=jLb1z-=(!0ISeIASe$r>AN(2)1!i5?Y>urS z;ey(r>DE#(8>!#(m(Gsy#rna2l4o9NSC`k&TruBn=4G}D@BV5 zbqmL0y+Ty)l5XV?fS3}2shQky41`?TTi2|(f>VmXe6HY7xmWsRQeHyopIKm)ek#h` zBW$=`{+wKGEiJ{MU&0y2FF=?|u?4BG7wngC2g6IaAO$ycZ0+iC2^R!^=EtkurTvpH z4emNV)F4qeXUT0cELR$JJ}QrMft0DLiR|Ybgn4ea*_oAQPYYJh?ePY#wBA^pG`gf| zQcZBa02gVjO{`9eT5FsDGjofucG3?_lDO-V2yp|J*^92JE7|)UlgWYYTN-YjH;ynw znDX`ej9sMXdn)}B4vZ>)!_}65cGdB{-=JT@-I(SHQe`Sj?%=JBZARhnENh(i$qgA= zJU7lc1tAM`_FFhr63>1MXYdldLR3_3PAmkS!eC6nn*Ye!GIDUfUQs10V5{UU99F7{ z^E?@&dkH}TU($Y_D)VPQkaMr=%sK-B)#2D&gf0}(!lASS<*lSb0Gcc^)Y??oZ`dyCm3BOQXc z8rqa*k?Xa4KN0MI`q@32Q>gLuFV6Qq33$EtNvqj@>lrA?{^jQ&dK_zc-5ppe5zj$% zIo3Oodmz^!h&KF*I6REkl%InT%M^<^M{yTncM}?vfq{{g8JI52QU&N^WJX>~w<)VL zHS5Y8R%XD}Z(fxc(>Y+P9C8YmK^w2#*{D-rbO3h$KgNB3mmOy6stlFu=w-~}c#aQv zVg{F@!O&q?ie2GxCv40QrX3$Ge4@pjb1pYLe9;0kNf@?&y7sUL&)ZmquV!7WfiOxx zQc=C^E?2t`5Ew#qCUGv=j8l8;C+T3y;VPJKTL#aV5)F<%(jm|qyMNOc=#;Z)S( zSEVfqfLHB*S=ftf2@Qjp58W(UwVB(MIV##u=%oMuu3LCS#rf1oh0T;iQRnAg3qgP+ex&!llW1`}6>3^i0E0IM=!6B-ZfP!09%iH6Y_$%Afko&1xskYMb%3%(ZYNoWMKxzFQzrtD&?DB zNwSVCC-!pe!^>0Ox_5%$I^I0%d$B2U%qS}mlUMv|N)~1^F%eBXUA1TJ@`ioM9RmaN zB?UTvr6&uh~xyfe&cuKXGbc>BAvxxdO;D%B>n7#-g6umeGBj-X!-iF zG=IJ5dw(Cl1Eb@2TT-pmi8i$>1Piutu0TCE>@DV7y)`L3AxRT0avZlo{g%Y@K7KO} z;T*qN3LZnwj+waJyl58vqL*w+d)d8zr-RGmv> z1zDHuY9Rm-Azk}*9K0>!P2a~n4(&|RiY{0O@4)Eb-6Z_{;B5&PT)E~0T?RTGyju?+ zILKG)uh>EZsN!!Ny#cp4dSe3iee`CAR$iQa^OJp`)P=-A$fBe&}xVkM_Gr z<@-Khw;3L52UDag;K0>JxI1lS_VojQwvODbotY$Mfba z`9g~8}jknOw0FLW$Fq|3a_8J`BB~dzxtGd|Hpmi_RQIS=4Jw8U8XYg za+5TTGk5bVK%Kc;)4riIH_Gef>&)H!Hcw~nHWh`TGk2SD+bTt`tq7`E66yVaK6Ae} z0d5TAI92M*J^WfjBrtloh%}`!^#igthSAP-fO52+Gd8oL-q(&O27lcapcH&}?E=N) z-A5oM_t=O0g5%8Xm~*1-D_kpnV&Y^RPkmzIggsn;{=~#nKCQoC;_Kt+ z;Xg3v%pDk=xrbCL9l3|wW#Y(x9UeeBa*smI+BcWsAu@5~9^^+qagP%59ZH|;*N8fC zk9w}bikA1*UfhG_?2eUES!#OYc<*d27hJiy{k>>6vv zH8#v1Ysom%+!(V~L~9l#uibC$ckQ1xx9!0ZYQaR5SvTQKJb4Fy9Uxy}(Y-cwo9zl} zr$*~xKYkO=RFrqn!G+OHII|$0paXn2uBp|J%aICil`SMIqwx;Y7n8js;bUF=`bbyd#GIf5uFMHKKp(?{Ff!}r z6Lf%;atp>$JoA=I7z?5~tl*AU14cDvv9!f@EOm&NsEP66hG5_hQb=I{ zaK1{sj?D-jQ!oe+oWr&0M*I`pSgr4?{Vy179m32-4E&xG()8dNMspPqkHN|ql^DEc z7kWez7Vj~6%08pOb# zE7Qo9;oGC2-n`)QhUOE3JTE%b8@xMVWE6~lCs5A$w2&LP!3@vHqL>~*Gt_Vl zg9o4$8#D!O4N-$G^$c=)s0A?yP6no^-wI+VwA&fX5Re4vpOwC@BuzXSl~L>fCkHtk z!Vpmi^!!Zk1-u)}#K4ct{Ox#vuHZH%6wb^vP}o9(q67Ar`@n3)GBL^kg2X@Jj2)D) z3JKVM*YzTKj3MV23)rr3H7Hr4ORfqgY4r&?xfWhMLs(&DMA4gJLI+On#`varg*uhW zHmd3xn5^}t!vjRjB-|vkl)Z}E88CQ4;aDN%oJNlrbh?ff)Z+05u}PI&j@caIZg`pU zfP_vC!Fk0n{(cOMEZ?dY5yB3!PP8}smQ(3}^$;M=P?RZ*YpFEbdV@lD{{XutB7~{c zqkAFDpe=4%l&-agr1LgiGkh>qGq0y)3A)<62gd=Ap=p7A_um`CV%|efRk``|rq6~v zBeX-y?^!jo_S~VkfFW`bIas7X;mmjU2`A_-;}n6o8j~~q5Vv&&s0@NluJW}8te_cx z3MgQ{Z;V1Hn&jpi+}quwJChV#n8Dr(gHCbnH)9zUEn=_|`VL0G8>T5rhMU}0bVCz| z!;{}5CMGeBQ_0O8m1+Q!5~fn%Y-$g5raeoCtGXX+D_8reo%>n4e>bK_jLayPj2Tsw zHr!2c*v%&p9#pl|Mfm*5W@2!iJ_~7o5Tj`$IZOrdWa$PoRy#>4qR+I_2V+va9(@h) zUTtFT)!vz^Bw`&XKn`YWb#g>KjYYVsqb@-_Iwf-_-n{a#kyVNO?XeD-oKStPlH@Ja zfl;NSC_r}9KLS0%iCSc+EZ2g0^u)D*f1QP~nN6lqxgz;Sk4QeLr`m6kuCc^_9>}W0 z#dT|W<7MAM6_Y6+UhJW8u3`j{@2P7`)$u)dEo#dC`;_8SsG6_giszq5j7dQ&@7%5} zn|rhQ;%@0J$B)|e7BD0knvbDWJem)u!{u8Wea2Lw9;zD?Jvxkpk9xv4I{Khjc!@%{ zZtBtqQK>Ox-_th-?V_8m-I}C-v!`?GJuP+;<#tLHCdZJ>{>yc9+0XjhRqQ4Aa9`=e z^?|V?z&8=Sy$}WRC2tWz_7jDA(D-Bc{U$D@GR?r>dtRIMVubUxov)>Hm$zX2L{@ei zo42MeN)nnfsdEeT;?+Y{A^G%BRD891tnXCF0k>-ARl+x<#!fvkrP%U+S_Zu=Uq=!4 zaU4~7(LFF)?4v4o@x+umuGPwa^hQL1?JS-kmoGYhtj}AXY^KF8WFd$L?c7~?JLfA?#Z3h({S}JLO@ViMHVNb zOn7N6R8c4~VGhoKnR7XUYBAG?%;kGN>)oK}h}q=hTH*}p?(G$Sni$AbE~@#EEnb#0`$v{#48a7My-E!H>OOm4g5vaq6uE*@g&n-A#0A0M{ znuz)79L57q7~Y~wS*)~Q=T1trZ9yqcn^cKHU!3fUUO7&YQY~cWbBx$4XUVEJcxEcz z@QJ|*>6^W%(*LS|Qr;$sWu}kqQdfGodk!knyKAIF8(inzNuGZWsJVPMSh8<*UVSJ< zU8ZCHU#^z#fT(Momo(l^n{2FjkLiEW0O^9It8cm}JB&S|i^4Q=K^J97UngDI!E3$< zqs*|a$v>*!%PtSMi!kKgbXxQ{ZWnjWY%jtnO??t!l%_L(g9xMeRvm;D{LE}i?WWajpF1Z zUvweA*r1D8@t})04Jf&8`8xJGICpUmkTv7NEKYcpg@gt>_y=Zf3zA#a_tPP##QNvh z>)`B{v_HFlTNaA4xV;WA6>Yeyb)D{Iud`IIUv8{p8En@l(7h>ZLWt@rqh_>Nf25*J z`D;z>6)BSLQswBNYI7@-VQUKR`z5GbjsYssZ1$ads;G7Hv$R~Sq{Grw$6U2KT&v4r zspA0RoWK#5v(00M^0;+n^E53dve5x{t4U0i$$iR z=I*+G?Bfu~jXC+n=6hT!GOjBe>7`C+-~PW}1|4vD^=fUoEF4|^+w!OFRRX7gPkPmQ zk!pL5#Yxe=`gV}n!nY}0ko6MVk4jgUFpIMCbdJ*_-E^{kj~VVsi|8$qZnvs@mbOlp zybJ;&X+xq9>5?&~&?BZzXX#KK(ZXWRLCh>w#j4p%1b4r&%=#&Vb*~+(e z+3p@1J%ySLCxxE>9A3h_rE`Y%sh@X?rt@Z}*Hu4?jypK@KO#-|fI1EFD(Hf5XI#j@&MXWdCOI?oKc@Q|N1%tJ}H7@P&Eq|!gc-4l~wJJ1lV!!ld(9tH#Mb9tL zSQMPe5wHd8iKZ<~$1m14KYR9&#&vdo_0i)<=Pj0>*d}SK=zymCcxJ}IM^z1d>6J)Y zKc)s1b+&_m8*d1HlJxhT@O@eEnAt*R#;)}5V8%xMv1ut)Y!@BoAJxC6?b{~51pocd z*vvEUn9-TFPcLv#UwGO(qCy#t$7e%&xcGqJzyFCY2y;5kH~bugnM?m49oYYWcc<2$ zc7ba;>r61h|GP0rxo2=Pn}NZPeld|yu^KF?Y?x+x<4lPCng~ppgdb9z=1FS?)|o2B z+q84FscMFK1fjtkR)VE@+Tqd^HG?6Rr-g}N7zs+<#BI5rc%;TOU!+w7wkHY-GoKbF z^3>uP>9ml@AsC;q2-UfWcwRMs%_o2`0(@+SQr9q`zh4;K8Gi$Vp!W-7Z)u#aXAi(v zuwDsgChG#iE4&5?5M8ubgC3)J|G7Bo8G`GLu_ViSOuPk-HA zOIR`eRu&k&MxO$f_*7 zCb@_3ju#wH2!nGK2oGGr2;d_iJml9fO$bQBECG?BzuO-x8x0tLqSpRF8Cv{PG0x0S z6v{ngSiK=j&Jju(t4q;t!PaCVvMrPK=&01j^WA@a5|g z_6Q*QxgFRxIFUqjQ!e(6EnDnHGwNVVkIh{3z=Z*9$E&WfFz6;th}V zHapV18$1Fdt?0ymG_OnK7fqEBg%e?#!fYFgZMcO(F0C6Ll=p$imG``}Fdfz}h)%oF z^~@*6Oc#de8vG~1pNSd$1rwi+r#>(V*2WjcDnfnPq@}DO+7@SUvZ(U}}>*H#$ zeFP2i%0l4hmMVb+aGwghgnm!}CaDYRigde#YjXpGYxV}5)rE!WH)B|K4aEk#SR52x zvcXQ&UMAarVRowJI2MhtNLHn0(Jq_auM=fYT(VSuQ19eDPbstoc!03b2G4(=|m?+wI zV0&V)ntZevC>TaKz?0>MzT^Iz95VMmocuVsZ zo$KF!QB@8wk37|4F{F{$u2+ZngcS!d(lG+o+_ZkR*r2{6~Vse!iOmX=`Oud9s z=Na|FET%RwjCdT;xez?Q>2DBH=P5?MWj29-gHjatVjvW?9}OUC#U7UJuI(>MVGAbF z8`H#+ZYdR$GNFl5=!uOr&pLCi@rFy)6T-lBe@Own8Q zam@ycjIb>j94aHSz)!PAXFXxd+p8pOhCi0UX^nvG3Cm$}x5%hx$Zu4@W#|diW1Kzd zWP-3q1F60_rCNk}BTSA9%%M~!yZl6i5_I13iNTf5H#!nrwHvycb`u{Kz|x`VR%2u3 zmX}E=-DFjdkJ#);Xmzornfv{~gBBKlh~X_VS7hc8P=B#zI&Q*4-(W64H*48h^yc&b zq5>!&mf6@XAXKu=cG#o=o05%)QHP&ySbS;oy&j8XUZuYKn{s4U-+Arr=7pMGyZ^A` z^3|dr6~s1#j(eV?Vq3y-st;7LEq#0vQ<{o>V5!)F2qm`nvn&0?_^O|$J~7gNsVxot zOY1%eeGPhI zsDcLOe_!wEmdF3!gP6QxiT9i)voPxW|NO82?m07<=F;i7+bm8Q5k8LT8D8$_R%tA& z_SvfSEr;VW;8LX~?ACwt+^bti!O`vqOKtSxxVrUAtJGA{M6Xwqcenn3Z=OT9#$?*4 zh0F>60#RF58G<%IdB0*hM{MsI4AT2}Q5i6RJ&%p0D0ntu$N!t>*zp3-;=#@KdCqFw z+5uFU?+e3hPuBBAzlp$HFG?tJX&q<`kvcE(CY~h(9sj?74E^@Y7~Dph(JHOP5gg1& zyv}C4dqxdkIJ>gotYGthBD6xt`)ZMpZ^qwnV17gKaByx-EUW8p$conSZwQvN5Jy`Sr84Fa5fm%#|QZZi-Q85@!)Pu2o zf)x~jBG1u+jx7cwNTSn80pCzTlckX=V5rZ4kx8}>JyBHB^kBN(aw@=My%95}59y8V zBbq)WJ=Yza!3ts=tl(tR#mS)*M2>{4q1J#Eom1=Gf|XD31_j&{exgJ}F3ePa5gcaKH5fswL*rhaS$)u}-7pN`!HfVN9*va=uZX_tm`nwfkOHT1OyxDkXj*O%%*sAZc>iKdX{_}s zKj4_vz^Gh7XDB+-HEewl+&=FzCWcswH_LF?wJ3KF!^tgwU|;EZx9EB$v;W(ycY^xD zV5Sqpa8~I4l{~m)zzHTXZN8Yfa12S|W#aA&!zINyMJ@csU;xXYU$VhHBJuddBE)#%eqU4-xgy?_TMPzG{XcQ8S6E!C73i z^?M3YN+L^&{aTuvq$ik9->AN4Utw%Yb19a~1wJ#hzcvKbwb|lbs89chX~G~jVEPt7 zp=%{Wx_(!5!CWD*5feC3g*c@1oDyLnzvs}d;VsU84gmI@O+v1CSmR_t>BIs!22uif zsS}5z{-0x3i4rkLr~eBAN#wupXFf5@Lz8FMDt=+N67J8TnO8P@l_*M@Z-X`Msys4B>|oeVY;nuA zKatXZt_^H1Nq29sGzhx48oc$1uBF?t<+0a0p1^)IchJHt{sf7*N-6(A*54kF*vT7N z;fO*Eh0QRDqRkHC?>DItG#5c=>((MvfH~+`0XY|?udeCx7E}$+8=P?C8>>*BdM{QM zldG2jFAr^Y%woY(e}SQ|FPxxC@55^9gZij{gG`KKg%;@J^cFr3{sy0^D{xbDsZMC7 z)x+09C}OStSQ^Ku>gr}XetQFh*isB~Mwr~xI5}?%VRPtHzDHHO=;dNzhSP#+R?Zt3 zFn520N%5l>t6g0ra8{*p(Y#nu6{{lfl|fBp%!oZ4A~x3iVI6{Cn;@-VthFS}y~UP) zqFojDmx7QaM12xxSFHn|`)8?JO>7Zl8tkqQ1ID)waA*Q zOh@XJ#AMn1XlXO@BLxEBrh4V|Ua(hx&!`;fn1Pbeh}%O}^-sjvC;XXTFmUx2W)aa< z8-n~T7jIS3hVB}JUBdLu>F6`K`AnDdJ0^6!S`6ul%3z(CwmH#@Q)@i4_#NL6k*vhx zHZLmjRo-v9a?GX|ZR%oZCGtyM+E0#G)V>bK15@hVg6SfE5{r`o zb~Ug^!ICPi0mYPT^3OQ0<;g#neKuDRHWTuvse%A0i9D++SR|2BQL&M}kFW(x!bzOil0cK>nXL5OCDpoF=~ zN)p2NxgqhYs>hkfZ^=ybm{n+h%UUqy*8c!?K#ITmuO5TB3IYFLmYc*J82LMn613S(EAV$Nl42JZK~pZ~>!>?zYJOf#75<=GG( z5x?wK1cjdeH@5+Eb9_X^sGNt5+iJVP8j8G0hzqed_2Guf|LQ??tbYx9oam|Nf4^bM z|GYXi*~o6E{&6smY#x@ADXA6^^8HzxvhPm)UqARVLw3acmahSSMT@$0j3r}E&(6_i z!Y!J`1DYRaE6B>^;NSmL6lL3islCJh!C&C8f8u#U7{nPr9O-W`vzz%N>JtwB{g3y- zG2Tai!Ou|`KmETtvX=?685cf6f837K<|pwlFqQ~qJRH>IRBZWG%!Sq9`ryr!HW51a zU<^uj@6^oQ_Yqtj?n#9060A?SWLx8gE39s5u^(aESaroBYvv;ani;>JJY{4fUwm0jO`HWa6<|DAXx$F#f zV#0aAE{_=mYpB_6&?$mu7q_+2e9QOno|Ow~ycqqyudR zgJIw*H4L09SB=3QMc572J%N1fda^gd<}@`h6g&Yx)@hj1pB>-k$($h&h|>Y$6P8AN z@&O|;ymFb~Z-` zsl;KCZHRGO+q|2&*FDWE2Pm$XaR{mpWmQ_=DoIa^Y%KE=!le=o?K=>pcZsKC~Jd5y+K zbS4B5ZKY2PmPYaLo&+eD20Y)=TGZ}!7SS^l{R*zVV?@XJGXCM_$}A;UaF z?VV)P!uBeG^0 zY)!sdVg+nrf6cN&XGpwa?n?%n_~{FgBmtVQA4x@+p;s6|FL<*3g0bur*-TRL)Ca~` z$%M6G>}=D2!Q4-AE`PzCulrM<7(09*kKl#L*a{79*qX6!fFBw%YkGr4!7$o>K#jFdz3*Gfk7{)||ZTJ~v(EJH*PUgP< z%<8p$S!#*huNIx7m=ae7;uml1m7^FWmg$s4$!@U4ycle*VAe}*cuu{-e1Fih85~=q zzX@&%;EENP#MNw8>83!vX@+m+iaYV;GH?al;7dpq!Ukg=lgYIcxh^`9n*($(fRzZV z?Rt>ze{R>C%_F{+^KDCh@kpiUo5Sk8Jt2%;&ra&2_6CzqU=w}A_A@D0>V=(o*b8>i zE|H;*X#@`26CB`af$^UgaDwhna0;UnKE>Y2AB$j5ENGb5vN*=lw>_j`A2svcuwdK4 zPMD}Osf$F@)=S$g2L#{qk!p$4w!z#gd|ZO7(FlT*4%gG2TGU8ap{Y0b+CJvQA8pF=cM0!)LIL6WC!c3^ZRY2GnarXH-oTXbN2* z{cea1NKT(+6AvQGuR~L0~b4(w4Pn*-!L8;B2n)*)v=i)X|7^B_?ZSr zwsnpV1{e_;@XYoDPjHiMco{c@hOzw;Cx~uO40r{1*by>dbHDC`lFXhE#`3?yI@qkP zjY-fC%(Xk{sb4V9F)AY}0pB(q+zwhcf6Z39$o4ABykJt)s*x@5Y35)H?ptb6TrJGW#^321Xv{vSFTtl7T(V3janfkor{AQbwKZZf4BX)Y zgW?qifsZY-itR4FfXIvsPaloEaP9r<7B-4C>fz(YquIsjZNqu?v{=|RP3tq)e@L52 z>3ps~+1z%F^BEM^*$!_QW$Cf*c3(W9KgGJ6^9`K23}b4} z&Rj2xa57dl#xdAjIfJzuLmMvUhs*RO9xP{uzS#j8gBaV5;j`~Ydz^@T?vwV_AFD&X zaAQTXl`mKYdf{e}sO3OH*y#U;f1cD(WualOXDfK3C$N{z2rAQM^Iyb#!y@(HJ!mG4 zV(Ot?_#=|u4AvFvSp)MtY3uh)rT_9xORi1MGsnww7N$Qu(j!O3{bpVg{rgX`>u~t8 z`kSMTLdr3i|L#E>%(R;T_{L^EX_$nZN2dON&83Lp%q(&S$Ir0%PZ-_df4xQ*?vQ&zroxcVQy-X`>HBKF!|=@i1rtxjtp0-Wub-zrF^TS~2a_47cEj1>&2b~4 z7cj`>k>U3=;YV5t+gW_Ef4oa(;W0jdsMoA8X!AL#@!M9Fk%OC1Y9G>mt*YJjYN!(cfbx-J`mIk%=ZU9BM@Rw2Kqph z7^@p5`P`!d>a={Fn1+J7HwJekJuHmW*@KZffDv(F`GT@AmVaQ=fA~Ro07g`eV0(h* zlGCC>DIlByBcx2}>8t;N3CcXL`NVWq zeVIs2W*k|Nhs%08lip$-H{Y8^kZ~hO2doC13nUrPH*7W$awGUJ=AVRfS|I4=f)UaO z81V#lV;cRoPOx|vf55M6%_mF3{G@_!q+_Yrk$ryt{pZocZ~{Ctc43g2&@hW3v4(lx zxKL$YZ&q$DVXlm=G4PsIEvXO!k zyuo7lw7sBXH5yo_$c5G>8-ckd+Mo0MMQ(P&y5Y<)^`8X!jc;;!3eu(jBPMC2HhZ>9 z3@JM%e~N5?MH?IYQOshJJZepV@LT;W;WOqEjwCSehY_rjNBK-e_*2{d}9)R$oR`j{;ciR20JdJ2#^^r z>_CT37DO~3n3g3z;g~!*unf8U3&uQnWKhsk9~dj-HIO3Dj1Hz`e`4;Z3fvHW!JM!A zQ@>)&Sbbx7FKIC6fp(zQn0>&?_vBf0d^YKa0zW((oAn7ruFQ&M9uuht%#&}a?X)l} ze}K2>n0BEN4AhLn0&%YmJ2U4`?O3Yb?ME-YS8hGm)7~3ce zgBxLs;k2Ghc=1TQjORhVn-#et*d5)3G&xLTescnmJW<%IWibS zNlv;83ckTI$(p6jX2Xu0cE#V}p8yyae=ZlCcu5_Y3_Gg97W-;&hhb;p?2m~cv3JJU zqZA#iaF)V^Z%9~?1>tX1;x=!>bbEukagF^pp@`yjg-nGnY04kl^~eqP;R zDbE2vPmBbu=Wemp2^fzMH?1yEF4!HJI$`cMwweXpmdytC^^*er1_dmu=aZc#e@r|* znfG=F10+stvYvFc`Ovb|o~(yb>*g8>L}zS!!2b&o2h! zR|&0=&j>)$1#|MPaUD{ zAlYWL00(=_NCi}oVaKM_USt>}Y4U9WY-SRyGkQ2M6=MswaLhQ>Zn0+>S6W-FgJ zBy8eUms!yhF~-CV(KdwbEC-r5|2Dy5o3gGJw&Z-lwuz&im~{{F2^q?*e-p4o_KxdS zgMB7Vk2AP!H`tGQFy9AbZW{>}YM6E#FdHP*CFC=tj`d?fq-P$YAt2zFN0EUA$PKWh z9#}JHXNsczmA@mMsy0Hi4C4ZQ;rmq&4E(pvam02JGx%#=~Ou}~7EK{iI$(wSlnN$CwT#%@@2=8$mUYcwBP2*G zIwT0gk=YzsfKpy>gr!|Vq0+{8fEdiZ_t{qLn_0hix{0Y5E>{gEe;H@+gY~V(kShxI`(ojx|X5H&y#U-qV zn9xHv2%cb;06Uc{e;Z($*=>Y5U2tJ;+>JqZ`GV!S1fBEhDltm$XMB`GEK?n4r;=cf zyID%+V1x)7pXRl-c$|8GdMPgJX@bXds7MG%e-yKJ9}t=bxK=T12arwKLCC&2?uSPn z>NL<>C6wB>yfJe-wEvbs%;NM8#me*TB4lnckCF zf;T%Z%Pbaq67s2v#oE)4EQd8WLu}wd&)m$cnL!+OhuGmzht;Bzpd_%?8$RriN0-L< zIvoQbi~*0LLjc2&u=-^S20SplB$gC`pd!io<9tVzww(M?OaWye0(u_B8&LRGy|i*C zP{tt?e+g9>FDco0tSPsYgpzs*$wj}0uf48IWJQUus83vyW3Eo&v_e^xHTb|Pn};XL zQ5GR6@CKg1Fh#|jp~ zjM2<%wN-lZ(A-c^%7&Yb5#Oou#gCFhK3yq7f6X+`T|GJRWQbq`NwMf6kHK&>yfcnZ zI_Y3sR07S38YpWX7*B*+ZeF$4Hd|oCn5bz6>T@MdQBw>P#tZJ8A#Ik6#V}Kk;p_XL z6el)WW~mMtF#pM*0_kjwOe~OOz%bsm!YEq_m%M`t2-+H?ioU>*pCKTi>m$@g>Y!#5 zf3SlJ%Dh_0Zo+rkj)LRmP&hjtXfy-YAqs}!WwDP)4#*c=|2kgq!ju-{qOkl0BN_j? zp8A17IgKGB&?8mvC+0kb;IZ~Q=6pS#`VBK!-{)XDHK3Y24VIuNL`0op@)cjSYfTPm zF_aeFw-V9@=am5%0&Siu?ffp67r`nRe_DC9L*1bq?15q#7;-Z8M1-J-z*aDfZzLzf z=adSFIt4?KSAo&_@}<)tmKrIHSYjA%np#jfY^oDWIK{D;1lvt-hQ724L9HN^uX6e_OSY$F@tZ68{LWCjF8g|zy*LaW;6GRW>ss~9wH6(a9U)T3 zrZV+g?35@r4a_F%=L zJyR+@a+#)DFq4SI5(nX?^_6w&@wQ^P(K^2;1ue4(p5m9DN7k@mYI%6X!?1afL9ExK z1yp=qz&h~AXfKQkuFHro%5?-BNqul=`T`Z&9<~z2%T|QuJ&I2g&CS2tf9wTqucB~E zw%rO{LuHpCvs4yXgI;8eSmFpxS`1*W57wG{ZSf4@)i-7`&vIjseep-;;6zB%HW{r! zrmB1`#%dD*4FFweZooi(5PdnYH%@L>3#tEngKTwAuD{=^jGpa;0m zSVrzgV_p}gP>jgS0reJ4e?F0xOz9|;A?qSH(NOVNGY6V0swcMT;DHp|F7oBZG;BNi zYNrK^#a%G-QMw8NSJuI3VekWoK_dJVBjJ`9Qg=7}UaBQa+Qw`txHt1pjJCHN4P*C- zd*W9M6HYJutkm*<7p90SL0RoAG0x-e6iYB$pJFiFZ!`^MF%NIYe<_#B*{(crr~Uh| zOw*0&5VkkFK>N<+rHPZ&)MP2_vYr7JGfjLio(N3iG3-SQ4^bZ&y`@n*rNA^hg?W+c zRTle{^<#I#!yUot6f;3Bp^9l_St^Hhw_Z)>Q)j=fK%vS4Ddy=qi9+&A2GBh6{54nN zj7(Meyo3gF)B7b>e_*JvU?R7IeT7`7{BIZxd;DMpgn{rE5 z)piT(7)7!6@Kl~zW%(#|LjI~|TUXLyiGtjtU>ap-SpW@Fplp?o7p8^e*BQ zMv!;qWn*6H2!FLJ(E-CA4!1*Sc^ zJwXy`&;UTCMCm~`5pe5*KUXEVlhK2rFIjgqc@hb>|KxHBiGxWTj!YMFrRu1Oujq)9 zl!yome^#|vW|$~?TC9%JVzXp_u=HCxOCIS6i%vPpF5c@r$d7o^NTQ`JivmT#26FoC9Es70fZgH8RBx*e$OHQ-Rix%r;Ud#$M!*SRb zrlpyAr6N-rJ3Wdc1!z9u!8QNc6gQ+v9SWSbe>Z%m$rqm<2+;zbu4cL~!e$HQ8tj^CdWQJg5>frajiL7O-b6NWvL ze>b_T4dVe7%|9(3MttCv2=QO4cn)wWV1GF&76wd2i!arq=qwXbF1Y{Q4*2H|$`g!U*rX#+BIlKoWW$=cP3r4tc)YABf zKswgajTwNMI<=&Z0>+GX-cHL{yGI!tf53DGAEU4VK^O=DGCK=fv_$MNvpbh~F;+Ws zEh~lNCWh+61|xQnp|~5!?8g@{zLu3S2rC&Km^F+Uogun8IsqH}Bj+PK>2U`U3-kI; z){5s|w0>j8Ap!pplf4GOyokYc0K*wZ7$$1rppw#iB>@+M)Z-^Fydwg#5+Jsuf9^=g z3hR845Qs z?tr-(K)eu6_@p8p1at9m`CxP$f6+g_1YI#6V_Gjlyb{}|eeD3fc- zHm2~*s_0}eXkZZPpBEV*9${CJ_>2+=NX!srXT?yEcP@rdi>;Xt=FVV;i&JLBv!51K znXn*hEC51QpUEC%hY8DFj_pSGQg$VW#p<62?AAmBbFDifbXWZa(`f@ff6(N_GtQVh zc*sYlxZ_R%#}z_mJZeQO8E+L5YZ`@rW3GQ#(yN zn=>Q%lMA`s9sx z9fT~HKxEZ2S-ck&Ur4b7kW|uigJh^H==8F9z{me!4E54Rm_>W`e;nG&Ab(n@o&}yq zXD6=WTH0gc`9(1voE3yK79F85h+c1a+QAqY@M!vk@dk?pdXekH-KgtQIFY!`tBcJs z4ysn#GI=R7JkzyXObFImAslA6Xo>;gox4jD*Q>svF#DFd+`MTIo6JOqPQs*nh)|;9 z>jzoBIM)w9@;KVRf9gWT@gz^4@)(ALc&F3bb%h{#VpaI)cx1i=3|7Wl3cIK@`wfd1 zj8AzOPlHM#-(T|U!$_2^3om6|9BS{ zs3ya4+jP4kY%+kc-Ef;%{H`2vU?91O;RRv3JP|OCgSapZe_}RZUw?&+cIYAGBx0W^|WAlD7_H}1cZ7IF~_Cbt63qXvX_ts zD9tjotDg@zlMn0OJYx2uvm9mm1>r>1Pt2p+f7-NfQOlf{vE@NXg(!&`RqSc8f%ryN zK|W_xh&N3?0I5Ollr=9nYyid~1M?>0*PV)ZB8jso{v>Yc3r@hh+VTglz}l>JwJ2t> z@dlOOx<4TY(CLJN9*QIGfsK}r)RB65v zC@C(wP{jLSzQowZ7-th9!u{HIoSpOQf6#e9JD80{GEfNJxdLciXePx>P$|Q8PE*2` zr5QI|ZHqq`9S#StR+6}Y1$r>Vs*%%9T@E@F2TM*`wZd3Qz|bHu1_}tn8a3A8v?z2~ z0H#(wc?Dca3YFI6Lx;piA>0n(*}k*k<}je|r1|1Uu{edPIn)`aN~Gc~oIWN8f9S0= z#$YiXOGb4=ey`#Jk1XIZq?=)tNU61LShPGmdulOFsguGFa|L--Sn)H8p}>L1PT>47 z6zL2Ye1SV`>`{FE7?*;=lv$AGu)Q1@TufkuqA7}6G^et{oJ^S2jGY_~pHYj{R$d@I zr7x?}ke*P~jMK0UD_o5?&+Znof1C21tdOK|v@}PAMOWQ|^+qr~2<4H`N2W_`NR}tH zHlk-0#9Asm@e3w1OyhMP`i{Vj0A)lzEAN;7fw>-%uJ9Y?d^?`{4YL%}vN1#HNL9)I zQ!J%coiNf|k)Be;xYBx94-LKNeWJMABfDZ0?(8zGRbUy2-D2ue_~j_3f4X#?^Liqf zs4m?q^Ef)q)t8ohTtQn1YM%;k`lL3+ric}%p^UG87iPNd5`-&nR-N+vHkTSgBn1Pn zrb(b z2p)=7YR#7;_mjeA*dDhFe?xCSVBYK9<)Uf|M&VS{_yjxEcGQvXbZ!on>>OTl+<^na zre>3oLQNJt!1Ta2S?mZu5z5DFXzO23ROel+Rq-8dde}8`MVm~ar^WKc%Dl9gYr{R! zxMp8mJzZqm?3R9fyKu8l^OFF+E&?c&xgrS?fZ`^Y)>sHKS03rAe_$w(Kv{&gRCs16 zTB(@>jqFt4$zmkBaYSRPv9+6Xo_XbTb{L+dM;a_y?OD?u-$}7-2yYBHd!=*@eC#fS zi$3XUQB)mfWM?>M-;J&W@h=9gv7W(DU=U#4zg~AJoZueS(Klu&v8G8$02~$Cg6ylu zLE~IPHWa+Zt*F2WeVzu?*$no7p-FCCwAIAahxs$#*8Wv z2mu8WKk6YbMJuuRfmz=Df#6OA{a{nfn7E~~EmP^|T78L&tlYgOvln27gJbc`+8q#u zZ8Po`E(2w)i=!43KRKc*S8b0NO0+c{zMXz9HWRpCk+{h;6RX42Zm1uPf-uNs@sNLxLUC?_~BC)&H z59)SR#anT6OqGv1u(EdWMBCfPa&hnIpO_N2T>Qr<*1Q{iL@Bl_Ckz~o@%aVcjY>iuXOkM(Y z8WibVcmZ?rs<}^d^bWK3y1K{^352&bbqO?i!!DV+F)}A|b*rYaFq}%)u3=WSlV`OI zb&MO669qHE6=}W#SBi+P#nqQ5PQ#SuzzBSn&YOf?_;o7gjhX_(D_LPSjp6*rN5!*T zAyEs#f0964amB`3nPGg*^d;=mo)Az5+1F5IlTA&|n>}zCj1I>6e?!*pG!;liQt_tm zH3Hw#l1kb*1m0F3Y$TTQFd8g`+pC3$kgO_lnTvH^5r;y4fnnUul*fVWF?nNC#jQ?t zJeb5S{vt-a>R^WDlfvAxb#ZQ>DUVyyg>Z_bf9pes0EUio4D*PEDaZOHI8g4eU4mog zeT-m~5hO;}Ct{dc0rtc^HCY*EH>YV4M}K)s?l6F^EY*-@C?D$A)w@ zf9A#xX&$lKEDVctunO6{8M|0wgfRxAABv>=lmv=VWB_k$lz)uOV$~^QUJ5xZ+O4b~ z9wiFFADAgQwo2_o4(#l8m<=jrkoCOG#TbK_kIIa2tz;cIS}4I?BC0XwkOjSl@`Dt} z#^QGPDhmHT3ge|}hf?+SCWbJ;6X)z8fAMfI%7and; zyRZK?2IPw!gRqP)-AjRsMF;f~X%g_0Q+sm;5sON$!-`<-PWfV17$=o+W4JcCVfrwa=G==P%AEi>SqbGa zsnLij)}o~$sRt(lnU)V;wL%e(7GoPL7^S-u_stK8=OZMci2pHof4St_bbLFuS6h1w zUq430i&?YpOK-YS4IpaIHQjfZf1#)?g1qH1`)XRi;I&zo@-!ab$@0$=b@FKWC(Mi1!1d0=`{i@F3q}Gbe_1`h? z2@oP4`i5Y_H!qvZy8rzL;ylt>I`lWh{%$<-3!;?|#Nwsp06BcP#2R4Ke|>s`$(NsI zUWy|OBvVHGJK@Yb&N0CdtcE#1nO97n7P_;OaMLPdOA?M%Rv$7iRtMEUt0Mg8&{V=< zK@S`%6WPJ2j;R}}xfuxMfOFW;^@QN6;{NbP7yvK&OxQbSyIwh)^9L;A!48!V%50;MNTf&%oO6 zFT!N^VE7GB6TArb*a;BeMHr}SfJ+yQo&r{nQFAYzfIuMg7@nqO*wZ%KhWP~6{A!Th zDP56^W$_n8N|X0$s+0VHu7~9y)}L?d{&f)ojuk-&i4#Mkm*`*$e;)o~tAgwPW?I7J ztUgY^;VFV!*FK#y29>HXo`oT(#iaPxx+i)}W0EE|8!le_Q8vuQoTkud!2#=wgmwQh z!_7@E2!PW}Xw&J+Rm^1RHw+_lJRyuOh0ng=m?Ugiy!TOa-VvOn6R8V}TQ2bid313! z!L`RAnlUq*W(j9*fA{Rt=1o_-dK<1ROqutt&DvxPlM*=g@*C5|?ly+E&66Aq4)cL# z9{c22Oi2jV2`BuxMJ(2=8@~3Z-ushsX)oiW+9 zZp`b(bQQJMQ%jiT1(0Bi4L`-WA;|J(5mT75yZ}jXY-cUuf7pAYhBQ%wMw!4*@CEu5 zi->r?Pp_Eny9Ye($uyY%&S*Y9L}2XaABZN#4(5;0-t*M2wxm>0Qq=Um;QYEUx-z?@ zEDlQ(NuuA;Y(Oj?uE)pU!6m;6c{-aC&bE^7ZChCS`fgDz6^g!rdNU;$2Or4hJuLB= z=&Ud5MB)n`e>2qTJBkm6<9hwaXLA0Ism6eG=~g!v?Mv_qVNyT)AwHwXtRqb5xDR1c z^n_G5a_X)YtD2Q(4ZRKvljO4^-M` z@omn8*C!243JrSY-HT~BOS6b|NM_nu3n$^D%dWZf)`P@aiZ2-_-)@J*nhGsla8@YS zh2d=KZsz-n8JvJW9vYu6|wu(B=* zAf_9UH0;MONU`_)g4jCCG}iCcmA(jAawX}me+aI`W9p6s|9Ow@^qdvM*y^#(2l_y{P$aR>}Q{(&T21qO)vuP+E0Cee!fM zvaDTnvAWzs308yt*X_HY;Db7=lq|09T4Zlnb8XFzgL>*UP-3EmCB(+S0VK@adND^i zf2{i#F;k?Agkavio}tusjceV~5e4e|hjh&8Q#vxS%fgQ^X^NuWTfCD+QHQ074h?m3 z8wN}wFxj-DA#%~Xj-I-yoS)FCyoKaT(^{^j1}+ditck-&pR5X%_-|&Ghe=fmF=kRv zG+pMv+0Nd;^UtO_ZqISU$rguibotUpf5jhpWd3K2b!E;TsA!oDB(29Zcbk5z&&WFG zU}WQFM49a1W>q7jUbi@Fhl3{>46bWSe3jF?AB+U+he_-i0 z6wFA;L_>WACR3BHvg^HJpDs7tWFtSA&cE*^Whs#LJ0^DJdFm&Ih67?6S=Miu^OUFb z7tH?pe(DFN@zSnu80uQHd7hg#{{=VCEbrqWh87ig69DRX&XS`=FcV3V%1;QxBk$x* z%!E@H+_;oAnuBV4lZd7xNFWV5f13~pn>1FtTTE(9e1X7j$OD+9Ibf)8f)@3C(lp@G z4Eu?oUt(TAD4JAGS<3}@EPG_3-p^FJ}T z99@`6!%*+VO8X|Clk#BplA{-G z{)*v^65lb8y#`&(sU?F|@Z--ccdY%j+hG{Ywvn&4nG>h#vD~P~YHsEvBlzP__>wtd z;vGC(u!#L;1~RMHn_bW>p6(XeJT~4SbI1LS;suOe$V$py8m?xtf74GGb7nKh>?fuL zC6?}v4JF?TUnB3?##%Bce(&3Jf5(UUxAiSYcPCz)~78ZF2*FVliuLq(pi!?0UP zCh%yHz3Q@=G+ce`8P8hgQnlAneiAc^oITGl&5K2<6MCsmNf`k2+E7xWp`Om(Q2bsE zW~TLde?NP_NZ&>ee~Z--YE&g?KpUt85yb|BvVF9Z2A!zr1NdH+cZ6GB&9j%ksJZ!4 zYn;<=JYxOr`d})@oF4UU-sTa$Ff2U~Lhvn2X9MUrRrSfNR zBAQkvd17dp266Tcn|#Yhiy`8~8@k|-o-(?&tn)p4F||dqrDgG!;WssmuO9=GPK-FJ zw*t)wU8=-ziB(}}f`Y5H3Fa9Iu2uhDB!ownjS)ec&{+pvGXx{0KVA?7M4Gu*l=m%)ZgC_5VUIM8r1tAFLe*^NfoE9b?z+Le08>)v;_Y+v? z!J{RiA+Y(^zT2;jczq{mpZf-n$GqHJ$G+i#67v!!n_j99a?3D2=u0&S_c$J;WJL1; zr5m1t>)f!9g7c#V?|KznbU2*nlYF9Zme{GT<0AJ(w?TSJ0|RWGJ21R)ThlRrf`=}n z{V$kjf1fdm%t6CG+9@$rOf4Ljz`jproU37P0H$i}kLYA|U{B^McZ)$=k^TQ%oV8(} zwQ68|45y)eh{MH-j~uP%wc0!+6`A`qGyl$U<(jL&^zi*}Q+eA?^<+L3hOz4DHX90; zK?@I0Hy<?i?Q1;B{ml4OY6IpRyocADCLf-33{U*u@Ck7#>5t&%I|ZH?jT1 ze+04pFDX;a(<`mR8on_vVQTXfOXU|w$?^pxo%%d|R_J3qw*nvhxS^&~-7ud}fW&Dr zyy|i4)2W>^qz9HcqcyJl{LM$|01PQKTPTfrBVGsbrf8PL(JQhgpp*)Oj-U3$k?uxn+NuwpLVyfA7Lme0<*>rk0%bp@F-OE z0;knYadxc-$Sw8tog!cQWcH|16^#x#ETOV>&HVB^LAzWD3psZ}mr`pglGdhpe`;uE zPSrH-d!tFj?W09DiLMj=72lX)hzMa^eOA^x<$&eRntL2yKSt(Qut#w^AjhsKL^*Mc zc{!S@3R4`4!Z0n}kxTgT6DN3O_^qudnxeXeqGjZRF#M$;kG$>{azyf-MKII@z{z1D zat3FRKvxS!ZlvX21{^!+#?%}me`EN~FBoJRMWl$H`hnpUIl?IR;}^_%3Z-(>@0j!T zcKpcAF#cq>kj;{Ca&XKL z9Lu-d3!PmVEK>z@**2M#h`9SCJQL$mPo6laZH$QdB}@rSc2G)jEPEQ_8*UMyL~g*5G4+ICYSA9dFDp_^?uGow=bEjm5>{(DtbtIEmCsFB zaY`HVe9m_LJpabS)9c}1f0sKWn?Jk9*UP-j-uFM>SN-dz1A3ZF2V_c$k|3-ud$n+T z+Y&q+4Hx~;83iy`+Qw4GS|!0|D06tys=hwuA_hD?EyKT$786&hFx|_-mX9npfb-fg zpQ!iGI!dAo5i3aw;#FK~POSl>?vAD-O+I7Cm=4d}+<~>iEcf}@f7Gl`CewSexsOq& z-cQpJd^mOFh)13Z1(aR&%Q2$dPQ_tt?XN-31w2kma0GBV{w)>lB!#U3f1YmV@0U{b_hPr=&jRl6z82zO+ftS~)RUh^%8~NRD4g9nCdCdL%8yU5d2tUWf|r)%ZSrD^%9ewUpnMCi{{8X9 zN33i0(nNrjg5R=zU}um^T07Wl5-cHzSt5O4{u~UKt$6V8jwc1?9gx}EpRIQkpRT~C zAH&q<5>t3gKw@Bd$HrB0(OfSO*Ld9o7w5*uTimgne_0G50%w=mrRO|}@0x?`Mch1w zYq0JXO}b8+-e5YEla2WaZ_+enrSy51tTA1iI*lOijiEK4;bu}vJ`l+)S$v?HV<+{Y zdj=u`%B(vzhv3w|>4*djv*cT7fDn4WTjVVynh7I*k6|V+JVh1YBSH*RAMrL=9inNF z0Z1wRf3>iATD-VfXgVT0geVXF5a1?<)`IGjpQs*P7pQnKdCmyO0VMTK46KF z9s8OTEDj7qr_-=R%zKHaX}awtwRXMC$TfY;mi^B96^2-vNs}AH!Cex7$>v=~oq4~K zi{)nJP^Jn5=vjeoj;spZ&ulMSS2fehGCLfme`d#P{9e+hK!VrR3cC&^1%6k55x;x4 zJk#x~CN9RwQ9koS7;H?lGI8G0WKu81ihdKzDEY6oQ%!bBR-%>NgF)0=#^t5$a)cl} z)a!0ht;#nCwaEyYpC06%W$K~I%9(z)2rKP^Q}R*rV3c%%H*{-m&5?K7^6B~_^ERyiBtPII0PH%P*`T>9-nro%QN*8A`O<7Z zR*YLFot9*+)ggi9F$c5}x;$G)bw9FPgJdTThRl~%OERq=3@V569WDB{OH@ymxsi^J z*c#oPpz#n}$BZUNq-5X2`1&z0DstV~e;rR@Y}G$t8cF*pjbl{cf3TLA znC5wyYEiiqmHH0gRs{EWpjROnHSox*2bdv2kFr`Z(aUQ`%MdXPD^h&^|v4Z@6Q(zP3c$(?~|ecvHeza6aGU6XG zd2=wZCCOsxZ*Y1+sD{7(yC(d!Xt7O;;E^`$C*qZyJ=qutr$5)KSDQ4yf6;+xqbT0p z^8?y2yS!#>V-TZ6gyKv=WF%B19GG#juX$?eOT?i9lg;|DKl3nOGbha5`*X9(+|1dc zMfvMDl5}!$my(0|dd}{X{F*%{_o0eBH|+E5Mx15O{a%zCQwApg8|EqMxpuMa|J?yV zkNiUb&AFP#8SHv^*xt|zf8_&actbM`E-USTe5z~W^!n-uBw*ql3|zy#95L>!wP@{w zX=EKcPm5*t8gEcs|MiXXc=g;1EXr^%2?v2xdu%u6Ol1(-PfQRR$gwG3m($DK|6cRo zxe+fTh9;5mVBV9QRCJ~)v)faM&eR$M!q3fgV7AQr*FP9{ou;H{f3dveI_D#h@9~Ul zdYaTVe$lanJzvtNWebz$qb)E!ZOrS&B*(w)XZWr;t(CxTBDWmsYqB8z|{3;kdmi;Fji;WGxZ z%60%oJ3R))IfF`)f7W)IDQZaGU4lJm?7Q8fDSaET0{N3=O0gN8Ssh&;jmS%qT$S_p zkl=K1$2X#U>8n~BdLI87qnBv%6vIGPJ4I#7YLC02a`u`Fjm!{r$*S~x!))K~pOuiv zhdoatUN7!ei>0x`Be`p$gG=yOdjwxU#&mmMC)CZHFYjw$e~$N+7&}{W4GDroqxQw4 z2UOoSoUgIiXE^zi;A|)CBe-l|S`h-Z+rsGzIE9ab7i(1({yKhF z-2b(*ouC*@b~RY}T>;kcVo=Bflau{Wq?fFkw->DHf2TJlKjE339Qhi4g13Jq-#!?A zS72CiN`6;>wfmEeU`e}AxG)mAn$5moQlIMZhwF91#L?H~dj*EI>+E|atP&98?*WhZ z^=dHx>iI>tk=&l_f!TrFufVWqrQNT9TRQI7SpVaGoeX*!m;03$)2Z?d-LJg5PL(O> zkibUNe;w4jAIU|L?S2K!if?wm^7_G;Yr0>#nMJ)+#dyD7xUI($91dH$UxAs$Br^+P zZ6LW{1GfA1`kC%melYxXPuKl?K`?N?Vw55es`Z%a@d2x>ojy|0tm!n>vj*<@!;e?xm;r|p;Wz4E#`hrG!5s{84JJaj!> zvfG>;jJxv=Mt4lSJ}@JP`+kz6dfIp{;{9}WuRTOY)+J8idj-rQ=k~n@e0;B03;JFG z5AD<%!a*0@hVvDeoV}P3dU>9SEb8^LPg1Vdz#Qi**nEzb=DsOE!15dO_Q0}NYC$`UyrqCac)LeC?WnzB8x0B6*C>0)EeE?bdQSK;sghwo0^h$%h2aJ-Ax zF4+Ui3p(91#EJ182kc>5 z^1$9~O^&0}qNFXB@^-;eS+BDEm z>mZm-cFdNnpZIE~Co-sRs6h6HTi6{w>GJqsx!>mHg5{Q0FCHDg2Fo3X3zpnrw#x+@ z7hi`9_HZA$U~m4Et%L6tJ!}-bPC+hMfF-=V=7~e+QoCR&Im-6HdJtdVM+H?bKX%a7 z%IhE_QGo%yPcgTJlQ#f~naN>?V&|Hf;J}@x7}B6Knx55tttsejx1!-g`_G z={~T%^CC-91m!io#JqlxwMZS&TWkfw1*Ini?h5e|LX&~w!31l7h99>lk9}j5=Rgcw z$OVE`eM_r1oaq`d%zq~oEqz)jKZ$P`f=k`sBV9A7>m@W|43X9`A?TG$;MMpR}WR>4LGk+Bb$l4~1N~D1`kut%6 z&8pbb?2hFn|1&YnPR>)4m0h|@%ZFjcggRdu=`D_xLz&(c@2%XTS9hHy2E`))b*wz-p^ zm}XB|lf_hGYJWoni@~Ik=d$1M6wUv$HX|V1>j$gIDpGP=}T|n{8hyxljjz%h*h4)Z!Id@i`l3e+w|=8VruOh<`c;qJX$=aRz~D= zwx++4Byh{JNM_w7Z6V9gT>e~}Z0xaqjEtL;=6~=ZFE(Q`bL0&f^AaX1cD+YM7%R#6 zfeLuIoTAIG=16!VW~oJTi}wL0)5lXb%qLVb;k0PGDp+5F!`n`~M5cw@Pb5B3U5$%1 zfDT-jg2_KHJSOBw9u(rChy6Qzo^UPo-!SLPe&{;_!65AeAqGa@ zN`Fx}0j`i{`E^b}(XlA!U3ij3x-yLUQCw_#QpX0p3Z>i)I?Lj<>MslhegWPK9rYK7 zNkgF|B(;mYT%Ykf!HEbLxg=sYCN-8oRF~x^A_>8@U5f4^jL$5^^a5tE8!?YD(46ve z&92Hqp@bF79TaD)Kp9b*Qndd z+~|_YoJtN{uqXGk#n`=is$hRIyHhTjS!lxLau&mHDe{c~zAlR3odl)`a;3tR7k@!a z!P%cAM=4#=qoR>1%y0{?ID-K+tZ8_YO!X^r);uh4^C#xTzdtEvjZDIjEO~cD+Qqebn>X5YADhMa zjY%!`e_*DcVMuC?c^it8t=$R=Fgc>0*oETXwNgqjHyryW-z~~!*;CUn4YnTB@R-YM z_e=8y921%J!kFiwFKR#mGMI^&q(W|$kp+a8TWG7hFlSwgOcQgM!mQ+(cYipcHnL0M z3iia-by}b_e-0^>04MFs8ro-hl=%ltVvFpUNKMm$ok4amTN=h*{tl17zTGv`Z`h;F zuWr{&Bce#XQb+_s8c~K3QCtlFZwhWL!`vO_+2URlw$2K@^hRVc>U*c6)-(qm4U#1r zqzUVQA6CKUeRCV|r{c(Sz<+%S78C7eq8pM<4XE`-)fc*+(EVEuoJ(5tdexQkDl0`*XJ%TOti5x4z;*eG5K{~Fgm~|;$ zefFRP0f}Q3$dHSm-cslW+u!y{a6EE~xCk>vkDpuK>caz=e!|o4IH1iwVjVYw;X2g2 zMR9bh$q{x7!`)AKtAAD<9O(-%oQL30+D+mc(`4NQqSg;Yb1#^zMmU*ya~tLAiS9l! zqn|y*g$X9!uuP#(zvTtS{?Xvs=Y|>7VS&>(Ls=m~swcA^`EkQ6e|KTX1+>u&YpNde z_ohlL#AJ)sRMkRKh9%k5X-D_Y6vJSGMXllwHg-{%$HucXkAI!Rau`6U6S2Z%u$$)PXO_kID0bTXLic0Kk+pEDG9zP9BQ>Z=T?DqDAmF+tI(z} z-oSioaZzKMgL6@&yqJpiW5oYkl`e&;)SHWyX>D)wxw4frHmJ&8BwNG!)nJ3mr-cwY{HB@5a$Y2=}80d^i7!xnUVD(+8{9Vj6@~KJ>>4@)zB8w#{ zE>a)DuefCU5s#%=0!E8sjh@xKbD>K^Lf#KN()whU$nYA5Xrt<};jBt77ls~wy}?^* zOnpLpVA>{4XS3Cr;w60!f8qkx{=u(`tJg=1noRzSCVxLnWsPY2y#QQw7@s(~L3vGF zb;HGGhi|m`(pN)c9#ho1EPk4&&!x$0mxR>mP0T`XEj}XabOZ3Y6x^arf}3HL&=sOZ z3nRs7+6~KJ&`A=Q5`7qI!P8FVISB9`!q<;MF`jNN>aJz5Cec43aP2hPDAQ}uXZwL+ z$I??&qkrxH#AB?x#`o~{X1)1OX$tv+Pf?|#(4;HZ=l}ShJ@CmYfW$ucQ*}LZUsd0m9}eUJ3@G;qbKUbvC^*Z;Z%pcn$MXrN2{ARnIXzQ50cYhIuYT*31!d#q8%UX_{-V4I4&{IJ94Gh> z{(t>H|JQ%8`BnOUThbk;jn#7x=Jj85%{oRxRf!OW=N}CE^8e49(7Wdk{=Iug^@8Or zS5g&rUK+$OtHmpY8HvN-!NqF$REik(vmV{3#CR;$ig;yz2PZ*Z5R(!MfL3#nf!zB`^?OM%;E8{l#ha# z`3`cVH;n%W>4^-emKtI`JU+EdQh$Fgk}WFIjqk1?NfIp*nwE%o)_Zp(X zO$X3>nB-Gr%>_e2wqM9H&{KmtHH>%h$TYn`BP*LwZ_mTb%H=uLT4 zeQSJ=vrbLjw15u1m;UjPo=Vu0=?nLOS#u*^7b5*-{s>h{#^)`1d8tyiW-gl8Db^;2 z8pb!`1ZJL!pNnJR!HqFM^?!$q8p=M)v*5Qp{|19^Sf3FLKk0PA;Mt%tNbua4x5yPF z4Q{yWAa!A;7=)(-doKAK+00hcfj}3rl)a*iqBqwnzIMZ)mS~7gxIv>b$rpedNbo(VLXT@Z$^oN1OoH?_lq$_%nD>_%7m`eQFq~LVU|Wcv5x?bN z3j8==t}pioXN9LpV}IT(kq7T&rZCG5SBO_VK@2BO>#4g=e?idg`?{a`iRrrL3myO8 zAoeq!%HJ^Y>HW|*L|MA~=)kOwMQ|8SnK1KC$wsRuG#l~ew6b?tf<1aciwPUC0HYt_ z0^>(8HS95*wM9lb)}Sg?H3S$|l;QAZoYjS-HfNQK>HOY9oPV)AEZ{9drFv9dpw3UC z&m#t%L1BQ4dzAEnO|(8Rd?#a)c8gGRGEcOu@IInOpKD7h||CxX6fiVW_49mK+Lu|3`IVQHp6I%PO?18GdY`|pfF^1mabo8iDJjtgW%wPH_oXLK@08<~VTO>PVlTsfN4FL~B zsiCQK!pw82kMD${slZc(T`^Qzj0ek1uuU&EBXNP5+1gQ23o{<3pA0$}7 z1U{G*ZJ27Kt=<2Mgp|;O!%ia!wfMOO+hA1!Vt@W@3ISLaVeg#1SrI1S2cF5*Dj~q2 zH}=T)dBakLuz|H*k!B9C3?CKx2kh{O4-8JCz59s9^lW_b2?nII1?~>%aiYrXbt;$U z-&T7fywqg|;dp;NOyx*h96NVDIf#txzQjZ{eWF=zXDv# z(XZwf4WVDXFS>5B(}`E3Zv;K{rrC!agk4wKo$&u(G^efo0QyAi^p9u1oe)UPpRIio` zcN{wbBdtD;QU@bD0hJR^H|zvdXM)DKmvKN1eHL~CgmZ;H3mM1YsxT3YuoiqHdMpaUCHvM! z{~43@mPgcL`jWL^bsJj4Sb)t@gKvodgRy~i2YntzE@*nUP|ou<7La1fZGS9S&)*mr zWGJOc=(pw1wNJxV@NHPw@ipbxz05KBcVk%5v8K&Fxlt;ut3DfI#w&_$WRv8(XZ4vb zQt_rcGXJ$r1+x5(_^rCA!J=bdW^0Q|hRt!+!!GfrFswQYO>wup`}U+5qb1;l@t(so zPw(F_a3G4`@Uty^7H{PY-GAB}feI20;Ng8)lY@HN;nuYB=li%^SrTGgEoS&F{00<8 znwK)dS*&fdJisf3C9-_NG#FnPUK0EQ^Tz|au`dV>1XSmOY zPmW%U9HNqb$!o=V^1I-rt@VLnFET66Xe)`T^6=K0{zP4UAEKz9WS4O_N9eqi4Uc3+(zr!tNIF$<2cCL4P0{j^} zHM>uP`*vlYkls8B`+s_ywvyVXFt{Ki^hJaXPl4FyRVurHpJl$+*@L`dBKb<(ClWK6 zmMAP8E{9RGKH27jz@&P&XmO158}>{-fAxu~qXqkr+2C0|h5G^9Ou?u8#Kt78mG%uo z`~X`!bdoK<>2n<4*y3bCK8h~q$DW&+JczCn)zB;D8ne9vpnqBFkY5$^9Sq`V<#1Z3ok}r#n?YZ0ggL0Z%x_f9bi0I zkpRn{`84wu`hZIq+a8hCioWx_QylL6+kh(zS)I2aO64n1QM?jL6!%+}US&~4OgiC(@*4S#T5=ZN!lldlgw-wB#J5X|Lm zZJG1$#!#{W9^ngp_ZhHlD8-BpM*VV51IWf4+p}2+?*krP?a0sU%eo7=3L>UMje_h% zEMwS;#D)+LIm2u7W)jS-wDOMxM=uF?Qiwd1U$m#s^%+)H9_u z`3YzvJAYtB7cS$8!0@DrsbyYw{S9*nKA!pw!$#3KanI8cr8UkWYETtXCTX3xSS>7pROh+i<(T>|0+~S&HwkHZO){PJCoaUK zvW2+auohyEA*Mb)v3p4>tE$1GEZAcV#yH!D!+#Mo;b0fH7;?2yk>SkQS^J2?b8d48 zPp^9kmt?xv-Q@=^Og5v#GvUoBiO4utp}H_E`~rH11e(OyfbRqiN7<~uvS|o{i^?J#2%xceZp8bZ9 zeoc(u@*Bpv*Z6YDK1n@^Jz3$+0;lf%>>e7OFjPCyB!L)Td`X};Jlk#9!p!;`YB8+4 zgb0GGF*tYC6D1;Rqz@nJV6B<&^nc^ko%MsFHikty0Nb#^$|C5M=i*Tkgec&MAow5K zC4%~nwhR%}2g}8s7DF4#A_$(rFwc1vTZGL>1od_tk`O^>-JJ)0VH~H-74x1fe1rKA z&BK*0PHka6-j9<6(g%z8T#9H9D@gE+J|o=44c5_)vv$c*r;}$s1t%Qq&wn-VjCB0I zG88`}YR&!cLmMjUK`0)J?|H8%TnU=JrmKi93_}85_ftPH`>DPiO%i_bn!T6KdxMFc zw|vK3Yz?bc%y1uM8vv71IZTFG%MlpyCPJKo8h<~TWRwp5p%?Z>A z$1eC{Ds+2=RUQ0GF6_-~mI~}1W`u)($ybA-qp`UUCXVK4@;fpI>Jif%F>|t8DEg$J z4aagFjEt(%M7H9|y4`Uw)B`)fQ_AadSah%#kK*tjxTYe=o(=ku+Ed%XGB|s;Z2L9X z--?o;i&Z`_%{HWDn}2z+1HDtfh7ALcvm@m?4i6sFkHc~yCVsg_&3 zXNUDXR>?an`5;xllX-W>dKDf+`GUv3L&oNGVc__Bf+uw5Hh<>jKu^P8|7F%yTvv-} zb4aJdh=n`oO6!UiW3do=JUiW3DkkCwkVeVa4t6oCz2L^G9IyZKc6k28ycLDaatnUu z2b>g1&To>khYIhluqg}{KV!WT651#(@wCDs0aaD=KtYLEn|{rcV9aE;(IZ+EJC6tJ zC1CO79v$JgeShNo30>vSMIS79@STFZx}q50ZFOPDU|E|%3k<%*;tufzc?h3mA{+9} zJq|un5Fp1TgYt-4PdL-hgTYiuEDY5WgZ{;%)JU!nfyr&gAd?=GPx;Gn?zOSDyw^YL z+H^Lj>f7mX;Omm9ylSmXKT%eZo8FkL7K>He0cS3rOn=aVF%6F+%!R=;wEa!Zql$O; zqQfp{mZD6%^O||amwsOxM%cjYn>=tX7w(>3D-g;K@4Y>PlwNA+os0t%ky!P{*Iz^2m|iQqA87 z&*gx%(E3P8J`fq!jQ07_Kzi07|Hq1WLlj+9%xs_*7_FIXwS%RwuT?Lx1Rr!j@xU6( zMZ!RS`GaAiF6&}bm+Z_DXVv9lqE1SQVurH#=zmZS#M3upMxwp~NrH7V_rK+$MPyv$~uP9C`1Gk-Aa&v|@2N@t1>jO4Va-Gz}gi0GoZ=pMx8vwN7) z!m|)Bh-o^Xh>3J3&P1<2-`7F2IvSTo{WN(nBmFYHUfF~9%jj}k* zWq;48%w;id%L2CuSAV=A*_?E~VF%{!MHtzmmmA(A+nBZ!xAX>957#$Lanz>Ri%}0- zL@)J(*ybUzV5~WUxuS)lN%(^7sWLq&BsZ&y7yFR@Tyu@fu)Z(GS7d;RWY9$UFKk2j zb-}~YdgqCsn0O`-ozY7FE=*%#66Wr(r+z42_m)4Hse2-U(fuPgM&6D# zmB$qE{(v)K(R-VgLdXx2N~RchbUX!Ilq}U#eSPMmCjZyUfYzphJUly zj47NUOmj&=zxDK^7Dq&DIxUdT>nV)LFAd*w_Y>X>}7; za`ZSV>z4dGCWP7{wI3MKnzBK%EeV~jVkA0cABBW@ePWzEBO7#Q-FfEPXgJjw#qjkk zW0KNuyq$325>jR40PjVr8!%LpJAXsZ3oCPYfA>k)s(k(%sWG}32)bBSHrBefe+`H3 zW)P?|*5}n?c~v@VE(7*~BRj+ux0dQq!y9``^VL;xDr(_<#Gp9rmabuP$`2;#?pR=? zTKH&$ssz(I%v3-6rNOA@)&Nu=zrErrQyTZ z(>yjzPDCUiE9C~eOc@u3H}dlNa>NA7n^dcVN{dFuRh)!kiSrS#SixD(K}X|OP+7Ey z)9zrn57(z!DNCMyUHdRtIqRZrUJQ1@!y&-ms2{YxEDvdx<$hS9)YE;$1tO{GAOYA4 zW}Izt9qHb$O57T^6b7+|lz)~K-t-+UszXhwkQUwxZ0&q+FrOP4_s1=%6=m7KYs=0^ z0Q?*nH-FfCMV$YxtfnrVy_ch?%|=*e;QlxfyxaUk;zZ&@1d5sTaJtUtT#qN5F&`~j zq-cRFy~ob{G%JgYxdZE#Q>OD=N5N-f1H@hHf4#BU*zQ=eipp{pnt$w0J(}e$rb+t| z5wuV?stRD#m^)Vcy5JsUvoS34j5mng4VG77kE)k*k3gp*ieikp+^70IOPlyHkNqNk zGD3563>utsrOn-yU*&_UhClkod@x#rZ^MM^7z2i*t0yN;iEHj+((^A z?I?>P2gSeXkd9$UlYcN_V+F?t={HsPi3X6WJ?sR6#q5}*dM*4G|xh2}S_K7Zru*vrzws3*jD-2?8v9@7ga`m?5V0oTq$SdtD z*)KcG#{jkg&V~ND6T@(@lrCs}m(`5UQuX5WzDOwSMH{o(7=O}L>0+<;mR_41gz)-g z4=*Q%N`2~&DlY+}V}2Lyk7BND!`_H&95D!I5r*lnD$h{oelrA|xyzAnU(WlyTJ-Rm zs~0p}JW5vsEhiD8r=j9TO_mdFm&I<*;7ty=LGSOgy#c z^%qQh{XF#p(|=ne;a(WnOvYxm@SM=HlFjOzkNMc7sa0pnSMx;StVWJz_=&}sL90kb z0WS#JZ_ciZ-J;W+c!4&9*Z1BmBT`ZJb^@2I?P^hC6g0mB-rJ)@R-kD{*+rwC4F&2S z)%O0*0QIk%l8Jf!pl#86`0C@#H-_!}aezCBIOCmXet%-(p+HEB=&25 zVB#qxAk~ncVMjO^8BbVn+#<0k(?_PJZGK^Nprtst4Lo-FlIqjW__IZ)!*ioS3kQH+ zFzZ=kePR~-JKTxsJI|w&2M#mB!K^>$aT{0m2e$cTcfkgl=%EGb9>Ny6dysCiC&4e+ z)3iUaCx6nNI1|18d|wC6YP*K?@5ao?=8+%WwiPCJsE`Fchb@|AZ# zr>jMG1f<`9d$Vi0KQTj_aVKUZNV$^--Yc^)GVq)?$iOpuoLIuZGkb7wiCFgX+VO|L zf+wu!{EA64!O&R9Pv;88%kFdXd-(e<7C?%6GQErlU&9Q?iX)l_{$%M+Bahm z_49-ULlnS#k$xybsTcMJxJ6{`4!ZMBz6gR{^o_I+EPv<;p0N$eA0@e49adVz#%%7k z=6^Y&e!~QJyWaB?lXPBFRRHU?jPg(CP^g|O%mFrW4|NOds)VU|*?l=#^H_O3|M_ka zk(YxtNe~g@1<8?r5jP`a`pTa}SwoS*hth09`c8X{`he9g*;UvWPjWD1K)gUH6asju zS?S2Mqkt#a$4a>65dIqWGaM5MDz=ZoReyZoae^kRNBjY7xjGbKzQS@d#4RfA-b)xf z0ke3qKMLOd-2cV{%Av^Hc}&df#>76$>9}scVM3_Z+(RAOn6Stbfu4y63|435%~9FE zi;8n3j1Bdb*Ci5War4S6E34q9Anz1Y+!dG`cbc+LC1K^058jlgB^dJSRDAVXM1S7# z^?jMO2GUhV)ppc0Va4rmu``V6ULs-e zT}AeETF22-)kf;R6GWF&83T@h3xBYNO(R{ip(85lf=tiX48LO&%?CV86~%Ht#R{2+ zYlENlak%~G3R1k_DmmHQMWyehpat_S&|tJua9WGtQPKTJ1Y7Zi1q=u5^%hi8_OIQt z5m|8|i1)MqgUQ{1y}@{gBw+iUlc^8~&T3N-9*EG$&f6y{wYE_36jEp}gMW^{uRA%w8(j)B5BLT2u+UkW7GJsFP)UjhwU4)^j(YS~(#HDGQF>fg z9W9;{kN_`=ecAWdhqToHX7Q`6IxG8*SV#s4fpXN`wpFp#~26cBdObv)!ya z2`Xz_xdnCwE75qKCczO3rNhc2aB8r=(P2PC*@%ZBEh+2YS=3E~#c|Xd5n4d6vAKt+ ztXOIRE+mn$b@pH&a(@}XdcXc066q}DUTKjk_h#)+m>xRgWU&>Cx3iYdFKChL5Mg|Z zwgD{lu=C#14_FI+FibMP_P-N0R>bRM+1-E%$5dsfr9A4agF)%u?>Gqoqa1{2kzL?I zaY>lCZ7u07czJlk&Hc@C3e$}~7e}Gg^BCc1=tFSzw)8`eLVt^jBR>iYjP8)~F!TYl zb$p~F42P))`yTt47z`ddQodW12t0H$MYbi$c0T4~MBs>m@f}W8A~@f0r0r}>7|{@V zz9SyKW8&O#p8AQgQLobQnSaOZr(#xr!R)W^r+#3<%=dv|FnsV|n)wLIO=B}3Ft)kU z%vZ;z2pxl9KYxpfd-hWFpwq<`H4M;T>Ch#+sZJrV)F#6R>*c*--9Qdc|DfbIriPy4 z79NJeY@wECTLHxDpvo^l!G4dh3{%qrY|l(+(}$34=lf3D3b9XN2DKZ!PL~V|&)|&! zJDO4HN=|UV_Nm$l4vaK^Il&SA|Adud5VRg@Cpgmsihnx~U3XST@5l=dINTE4*0UW} zfk5lo!ds{5oKb{%Pq^<{+zn=Eee_EpOn!USwOtT6-`l2(%3RhH8C;h$F1Hr}GkHcy z5o_YM>B(gN_b>1N;K*0KYJFfb;obnb`D!0ogWZuUyK5DJ3 zm$|{ZbdTYj25TpgRxyLMInKVsVV@!B8@BuVJJr9q-l|v~?r&nkB#9NDy6)cIZ-6Dd z;S!m1gheIkfe{Le6gN`o2DMTVYs%ll=S<&soqr|FLeJS(q89nSxf8>tAutw36$`$N z3H!aVn!Bydg;x$749z<2*;+RDN@EGc@H&IS8fqNI5$HX??RowllQoN; zkbf5L0I5wY%E)#W@~S)Z+I)l+osLHba}##yn7!vni?YLcL^$c*D;B;?dM@8z2Ya54G? zvZU2b65g?^ZiDj>P1RsjGS-sH!hYP?C9D6um=bV}9EKWnYRek48kAjQX7*^HL4u;$K;rP2ShYhkUg08gd>D6;A*eXtcJ;>#Vn znw}`<0LTCcU{rkRVDe?_7F&?FbMRn?ezb7NF!PS2pUnTDIDyQgb2!EP z8{JpR3S8mZG&2k!g3VZmOr)40AAeYQ*RE#FEy!4tlW1g8(b}rsKNF?*aj5pq5H72%mv~@mP5VYAL8eXCS~W&M!Oz0pq^&4AgKz z#}&K>_zu;>6Hiz))#6zn>S}v-JiZS>jTNCHLLZ!Ogh`&KpbPdX=wRZE(SM#6;dehV zKC4_${lxgpgWhTee!;|3dtQIR?62>qeqiDh^nrOYXG%zh6#whLj0!P$krKK79fYpD0tm4+oSp+1 zZv*^LJpu;IaNxpN-jb(d+F0%=?af)_PT+MUrUJY7`T9bkWS73ZK2 z0vN;#Wt7!vg>d2>qJOGO7cib^4758fq)h{cp%*a1S^3893leE-0jBA`N$Q~^$ugS@ zbI8!;C;JLmAs#wwao3qD&;EkRQ^g13nICZft7R^7@sBLs0o_C}4dBM>a4S0lcxzjeW-}-H~R7F zf?$AQ#PY!u`Xd2ziL>G%(WmJzlsw%_5*adtc+K?SvH*r!i*X^hFJR3C6q?hbG3x{{ zg61)I7|+Pc%<+i)?{F^E**VTO_L661hFnWN$hfs$&|paDnU}(vFb9tyxl|L; zU_1k_1Le1X!GGq-%nE9xtL0u}Wdj!RgW^t$L6_f1!*i+Hrr3NrCC@+XvD%>N&zb$l zz4}}qUZIPOApJ=qGWrXebdG;W3DAv4p zDMIkmwU2Rk%4+8gRS`+oH;|98>gfeCQpV||K@9@U4SyG&YxmPx$G83dtneT_;2vNz zfHg%AW3$8<&UT7X?iQYfctgz7`9{oS^kPrFd-=H(cz`#rUQ1(Gx8>lAuncfBVYyqO zm6-3>mF-RO-Xwjlog%UJR@tN4n+ex%U)u}wP`tS`?>g*U-@wgFWu0DB6`$F?-0)tT zKs=0d*ncm;J?X3a6LSj{kqcf$%734>)ycD04uS2x(^Mhg&f9Vp0ntz(7yf z=gEB|3EfrJRXa-qmss-7i>W$$v#W;uY0+sq0F0=5ECzO<th+GNA(2sJz-~%7C%?!BmB{Z~TGkHe67p-A4;t zZ?e*TvY+D;k}7ss6^VPL)@nFq?=I3_>?L`aXTdSN)Y-AKgdA7 z4CiFhUIA;$zKBRnq7mi-Vz<%UzcFVn;dn8F{e~&bHG7?hejvbnc!x7EmskGQMMBq^yabiqae?sDY|I zQ$m^|M#6e3;);9i%|rl8)M}6$MiBv#-^NrA-){hQjCQ_}w?YuTKrCRwJAd8FqPV0O zMsaP9J)F^Eubur8_XL%lNlJL-|7vC)6a2DpZeaMs?vZA z%v?UpBz9Ye2t~s0VSsR?i+^LFkkCmn2b>6Qv?3jVH)Chg%$%ykqu?N5xQSLV$4(3} zxe0NNj}_aL_`g&urdtRzD<=@rumSSG1U5Ew^LDVcth|F~TGy!5?g2|X&7G5z>*yt< z*{>FED!qT9;|u^95oslrJ>eyAJ$xWR5aS`D2B0ssl^~uxAIi7hRY&V`N%>rI#PmTM zh3wJjvI={0EK#ih8h@RksF{aQU^N)m3Wq0?ae1~ld6VMyF}b*WFwaF!BlN++!PW{)Q&V&Iwg5_T4(33MlNT5L1R_AEGbMxa@M4skhnfGXp=8?H->J|FS z&UXJtr8NLvq+%rW|2_t_RT+k1PXXrO-3Hj4L6PwpxsTM_AlSmC)x}H{D1{x0@JZUm z;6I~(1K2G(lYfHS9nVl+THy#>a6}M-DRSH3IMganNp4>wUAyCt0uOnteLF2h&~>K;eT9%I(?QpwRZuGfUo@yMomf2_&RYFBXw?@PfrbXhMS~2q zWKWS{+_5Y<3hjz#JeO44^10+#$54e(JUW+L%JZ24i+{s_SgABws-fE%H)uKpdtO=- zZRi)1`5ziJk8m$!HVQu@C)+FJ&z}CG0||fLIEaLRzNfvmCr@U6j8`NH2(OJ=zg^xV zq)I>++rpQj9F!sCqJYY%DpR02Qzb)T3|B3q5E7WjoRDcnCUY4>vEE{yl#+EHR}5Q> zgDw#0+kY|7(&8BSh?^2qsL=!olb=VlD+1a_oCUtF@oT3dHb;v8htN9JN*l!?gx0ay zs&mhWgWh+EbIv0Ijje)+H|0sYw;C0HTgDuj-j6Gmci1U#kBgZ$KH`eONBW-c;0bNZ zQMp_5y%^-yalC&BpTkftG9)~#y0G_A@rPkJ9e>Br?m+jTMu%Q}1h~_zN7q5d$)N9Q z>QqXTPd*e^ngJBEFcr9}g*;E!d`U#~9fI@KP+ek-rm>ZTC=y1Ts;9p#*DVH!Aajbb~QPwK}aQV3wxI`X+_li5#8>ED%A0g zM}M#YAJyg==)+f^p+JsH+yG`vUH0rBTd!2L4O+#s(mq>v&vt5(Dinxf*3ac=Hzc;B)=K? z*!am8LuW-CoLlGNzJ`kbhA?B;m^izqO>A-;v+P8e@%&Gkh6H%hC`|#c^vmI>M_xt0 z?7OIQ5>o{|K?z#{Y^1#CBg?km{)J_-%LqTDwiidJ;tKzo?^oH*!~1KYA~x-0(|;`h zoXal=*#b>l-pSrShrCAm_fvueFZ3v4U6khiOJEhn2gUh0IG1!=XMBs{K#Dnm`6*Lc zzg;#yqUh)}@5>{j)FWt`oLFpAs3w=qv4=TzR_av8@?bZN+bQISTZX}0oMVPn!>XER z?6Dl`jy^^keG|PLSG3>VDn`INU4PTrg<>}%sjPS{t8YVMrVfw3=MMd`v&!9oX=?GVOAl29v(tuW5oE2oM+hD!a%M#cG$^V~Z6M&8LtPAEuxy2*<78-6(+ruND1pegoVNPjkQ?RJ=p z1vB*wgt2(kPNA;o+raQi?s~!;)kA-CuT}JEi*#d|-=+&wPJh0HejoK7cIaPmnu0Ex zaN#E~A6zIta(a4fjH5Y#rD{$rHJqI7S_<8IMaPuy)*2ltSkg zlS3>Yk4#=soq_7I09B((xqkwTCY&2+3f679UIVX)*7<9A*V7Y%6RYp>Hzj>LpRE(# z)C4STItZy#av@Y1Y{0Ks&NyK40eR*~P_PSoZtaapJGY%EuV)`7_@ypuKbY}B=(W)C`_2T{C7Hl9s1XHaU@M~4_)y{mpgmOsjjL# zu*>=AWbV+DKB^w%;7hIa+>|5QoXA#!!#v#^$I(q*WiTaDRnP?pXUp+ZZDXNXQlWcJQx=Go?MvL5rz^A1+-;%vSs`0zUQux;X3F z^?`D6kSG}2!T8*wdJsk8I&0Vbky1!X0Q=y4{LEk1SU$S{8NC#&| zk(C{sSrrTj4<~86I6pN#F%;N*YDG5*ELAH`69GxE9d zlP^a4IKc@Y2lTwRpYx(+<))uAyZk%5InyN&v7<}@Zyt`$9Nxz4=%7i@yA~ZCgFqdQ z4l%mgc62bO*Wr9mg&_96IHyZg_63hK_4IY7I=T;EXRfeFC`39BG+iC$`TFVVq}-;f z^C8IsU0V#bMVf!?@8&ow$y4v^&RPQQD5Tlri}Ol2BR09v)WMPfWkz|zIO~L!u@cIa z{Eo zADl>x?O%OyQYq}O_~z_-2!e6+mImqHgQLU;Xh#2C65ql>&60S5HE2}trNjM{9A`;9 z@j<3EqH<_Jf9Nh=2Zy`YT^N{5?&WPAEM|;8N!h#aLu$>!i)CadkC&0N=LBfjPJ5-?Z%dzd0Y@#RTaexi$hzr zoPB>zd9_C|jNsChw}NZ_DMXs#;n2Lw;x|WS@kEj7k(FAUbq05{EWQatAgbbtLR5B! zh$uyU(;A6+6lDdlNmRuX8^>jqs*2~}+Eqegj)O1zAGh9<6IJnh(xWV1XiMSf3i4@1 zAhOOtgxXc{gkP~i0h9&3Wk9bZ`yG4INVq8HW?*1#v4T-i=WcH{io_H&^oGzZW z8DlRS`Ez!thG9Q9XIpeuq#3R^Z}65PsIY*pGNY1K_3DIUxpqZ6urs2f?n>!L$1%gU-(ihC+dP1DD0~5%--N zizKGnl7nH#mmJ&L-ZGY}haY-ephkc2sEX%E-QK&VrT7tzWxNNPDU^?i;KIS#hwfrg zT|7^RbmV#(=3;p-!M+H(guSYUc*lI~SrSO2AL{~+XgPbn+Ua@n3LVQFh4H}d7?1Wl z6~+VPB8o1k>fn1;MfkWQ&gMNfD&xVS5V4YD$xcruRy>M(!Q32{>7 z<3ZnSI*P3~N0srogIb^Dg7dAq*N*n-@B(c*6L+QV_<4@iUY;{5jeG@0+xs7`B<6^B zNN?@SsY+K}HJ+I12pC|zV#>2Nesdy})M$XeF=UBHZ9E>U>~!}-Zim@i8{gLDaJGu$ z@g2`Nwl~19%yBMR1R&b6sONt+E@1dIq8`gHDIpZC{)*ywkk_t_Pjb&z&km8L%bVk` z7&zX(Xoo1quo=#Vd#l0=-n&O7KVdIdI>XP~6|tRORd7Ge$xANg*wzV-$bTvc zA-+hax0`{M|K`3-8R*H$;`q(cB!O1oQ^oP5Z-qG(`0PF_HHzaw-%jQfqLfr=n=Fne z(u_mK2=Ebf;@Z{m1g?u8pL^_7MdI zLPK-X>){+vTYBB&KV5%%6}9on*&f>xtoT1T;qc~56-H)Yn6a3R!1;n@$NNL z$1uptuc(X1hil`PDk36C+ig7_`gvZ_ToR_bcwiuCDxxkPV#NHC>*7U+@$L z$sM2p9c8!(;PiWYf{O5fBiWJ`;(J#uGL4l={B{rtoa$6BTn>UR3~ip+hBm!NGdK!l z=?*>PTzbY42;+ZCxR!`84pBp)I9h&1M%wtDy<{wo<3LkJu{w=ZKcaaY%bwA`PU~W2 zH1HHe)%1RfCdCy~<@OL2#&3?SY|yH{RAqdxC3Tvj_uwcrg_t|25Io?uKzgcHjq2jT zQK`LZ<9m&juG;tx6lXplYU4ZRx=DG%4eQKJ?UJ|d>d1dx8{bQGX-839a`u*Br|ln{ zj6C7aF#Ar<^ZP(+z^l;=c5!@nwfHWM2ZuGwE{@0NGOgs|cnk+ytfDv`3=Xi1<1wIg zLRB0OQhCiNj^_+rig8|^vD=kX+?PJ)7{74gTGq#(9PWNVAt+X#eCqYvO_4Hc!B7fV zAB@jbfvtba<2j3d&FNWO+Us2&&snWC%i}@QsLJwq&hyCX_+Et296>)$`?Ya0j%m-Z z;1+}k@O7kb$9m%ox6+8i$I396p5kKY0xzU84WpsrczjuHZbE`SFUzSySI1+pT=#o( ziCp#om9C;l*jQI!kHc?s9Ujh}9#u_zPT(IeW$J&u|5I^%AHH!qS+C7j1;Tc+K-=en zkJS0^P8=44D??<1SVWLWkdfzP<2WxWvuyZ4#}kMgm=iBb=kxg7%Lm)$U`}4uhuHzslK8 z|7vr>o$9QbJCpalyHUXo7U0r#o`lOx+@op?@*s~vG|TI(F3@)yC(*1#p#bIgXaIr9QYRs35#l>Y{54c!8N+83-;$mD+dRU;luYiZuGsqbp z7N(O!YYz)F$H5*}PItIiTO-)T`cx4#6P}vAD|WG-C{)ee^Vy}Vuu#Ma6cP&bW4(WO zOV4!x85)8Iu|>Sha^Om~W*@udg{osnun*-j#v8IhM?=4r3sgo&!Ddxrx%wA1(Nth@ zb)F6A{_J|vD=e+xM6v-7=^5#GW9a!CN1Eeg`PaG zrSq5qx#O$#`1CHFN3DfQQCAlu7t?>o4jL$)-N$2Wl!wC%TFTuLJVHKG#))Onjw_av zR;rN)E!RaSFTQ?(s;Q|fFu4!{)!3>4aEr-w;cs$sG(+0H7^kBpRNct#y(AJ<3-i}b z_kqFKUD5wR%fnZe1@!I474)3H-UN8Hk;Z^%yy)y&!`?WZfisO?Y)z#rN@jnou}#ck zzoEMxrxQI`rU0-m=bagExU1b@!*bg=p20a5+GMcli;=^XH~C_qu8NG?Lm?H`So^xr^!w-mUDey*aOk3^ zL=8+i-%k$xn*dNIR@9B9P^^FPfr{2wr{qQwqka+XjJPtmevXP5f%v$tA&9J!=~I_ihXd3&wwn&gc}3)iW|g z!B|GS3c<)pPs#WY1Bk{zgSRV8)v?gmvI!UE+?%70g>F38XjvFn?5)wW08WJNk(mIE z^jS1548R$>sXT7d9iz1>7>B`7S)vcC9-W z@DkdRqhkSYIi2w=oiBYIE(AaGEp3BgI$^>v?g8D$m=Eh?6egCBi~yuWs=gEDRyK2<~uh?_D2&*?a0)L4bb`JyW2Zo`mY-zy%DL znie=45sX=ouaIgrEo?fP7Cz-3(=%AWx|GGBT^j$-dpzk&6jAu zk78X4>2ntArcmbLt zOylXPOnQH~^*PF&ls#jsPllJbdxH34@RCZoQjkYIMN)JXw-Tx<5hBD3MlVt-iu-92D zJ!-U)MPB3Ns8)IaE@16iY0z3-%ASnsCb&$mjA}LO1Pq=_PtOS&&~u(C9K5M8h?V5w znc-Hn$im_xUN7Ium-xI9&EsJ929~>+TsSRcYoOw`w=W?$7K5@SX+*DY-oRM zpX0;+9vq!53mUcOxCeIb=3{;EjG=Q&eJ%*zqv^zGYPY4$Ex${orU%+RIIO}i@pD$` zKmE8r29BSt7XaD#5qiN*@3{%imF*FfANAbn36i+}OurE-!iP_c;u}`U$MFRwH8ark z`+_R<#@~C0Bu%aYi$R<+4fY_*X#Ibj1}>f3#?}(p=3jCUPYe!P1|I)TxKN;}q+BY} zxbZa6Dp;azC!xV7fTqp2)H)6tE?>}p*p>i4a$nIH5a2yg9(Xr~eDPLOmC!gnVd%|a zii3tC1C4vLJ)%o2E$YEhhZH_cjp*NuCFS_DfAbHH(#b!36OG41uM+Qyjd*{l92z9L zDL^kB)p84NZ18}l!Qns_H68*Q#dtyg;S@mI?JJ~IK(p2dQ_^*Zs&mY4#RACbx&j+2 zX%ubd#81OReTsrkH&+kuSc1#>%M-p4Rt?V28wX3b@T^A&9qiJVOUz&eArA4BBEy$L ziV8n?w^yYYBW8iqrtd41WSoCQPS0lw+y^v*=2!Ed$o&%We9p{29f3UcP=qp-{mM`l9^W*mF!W@9i$)uHaVF!!u7^ir{BVW@vN#d(3Z_9gc%mTg$O~=xMGmrI)r)D;B(I;imuj68j$>S5EgH z(O@ov>20%V!i&z6#dm*V1bN_j(|0eOFkjRpNxi}2-D8n^2MZz1Ei45)uV5oRZ_sZ} z-m;#Sh!vf6Iczm>C3<6K=SD8n`a6!3-`}L=-k2e>gaK_2H8l6zx#s<{j4Xyl{~nyl zXj*HklF7x?WO_o%V2Zw;+#V(r@EjJGw!ws2%4M_m-;&#^oYsH5LUTpY#~#!QeFDC@ z&QEUk4j|i@Ap-pAE%cU`LI6CIlFKPMxkTIkvf$i)?qtrHEoP7O0QW|qhFP^dGu|Bc zA8wMxw^i#FjgI!KLmA;zg8Cd+C{n^ZKy|mr`@ELF$8$ET$iflFE*iGMrdRX~92`g? zXsX?H`8$NV2U>pxJm5%vbzE7KJqQw(8t7}v99lXIv?=L>MMU-0)K zOmt;G@Qam3d!;$mgh}$oV3-ZA!$YBKVdPc41ASE(>1Ft>b1XTd3Uj7PY$`iClxDL_ zpI4~zOLu>Gjot~9-FfoeZ7^t91r?dC3|vov(QJm%dhFx3^|g)1K2n3zc<%7>@d&81t&975)6HtITw8sk19LjEK+}8`vnXqk}jzF5?mSNKoCqDG&FdfI`eafaIzM^kik>A<&j;wK^g3l|u0!d*z zc6Q$%rXz~xM%%U{3jgNVwqwxSc1(_J$L^5{+Y#V(Ub?m&yDAUTcI=@ptmIIW)iTk|pnH?%6Hy9v6WzgO_G75Dr~0_WfwR3ENrupYi1&Xm zWOEBscnoRNdJ6*;G=UawH9>(k${@~(#2!X$z>O3?bi6U4%pqWL+OcrKgZ@bx2zUnL zRxiOnI6TglurElVIMemU^E1rn+me}tVz&-?YRYnJRX1bi~r$#Vb2|72~<7jPf@GS8P@P)KP2ZKob$~g zqS0-;2K~i=8oPBAH|WmOTfw3J9%@JAcOtHV>Ox=%Vha79J*SZS`$P0G&M2gU=lV_# zW#{nUzm43#IL3qLChIT&${{+M2~DoBoyk4tX6#8PTBN3v_IuJQH55(w`pbWmr*`=2 zNB76sq*EMXK;AlY8)8q$KQ#MbhZ%wSck8TTFo>25F@LylO2`72jm|dG#IV9-D5$Zg ztQ9@R94 z5z>M1{zyF9Cj7{Kg}Q%;hG1ebsu*7`N={5uvY0Q4Rq7)K_z_h{3NWfXhK>`a`1EA_ z9E{1hfH;ha4}p5%o^b`Cx!Y~&Nwl!$7*VKsxH6AaG-Q{c<=z&q6(d6xomX`4soG09 z__v?&#EjEQ(^+aYA_rqK71M;s4e~CUEWv_fs1QSeMk!UZLo9#3rmQOlLT)`ckK}D2F!a#$ovLrKC=kwA@Uy#WHsPru@e+wg|qIF{e?C5a0YIg-c30gtk67MVRuE z5gh;nT5BhlYg5w$IB>|g7&)2(2G=nJBy9xowAPy(A4|aW9a;l8pQgLYAM}IZ5DzpI zfNfK4 zI;ZPc*w4uk82gg7XE@@&8}t%4B)i1+qS&ZPYR9KZVF zJjLB}c=uNS!8m^75&Z{eKifb0VhCOLU|fbx?>5peou1SM@jG~=Wb>fXJ26Oj?OPUB z(=Q&6*%1D44{N}{sF?6aB1~=Gb$-V11Pim8WfG9U;;DVgme4Mkr$dH2F%0(wQ=b7E z=Pp3FyqAAF;q1;TK9tfV2&zFir6SDp%d||Gl;9bsl6=TCT`M>VO*%pZ>_!k0BjuX+6f=TgtvdFpq!IJi_c!6(p`e1u0VyG>z9VhKMpHxIoy<5mhk@jITaIQXTsW>b+E zTYg{B7>DB{Cg-nNTzvr9P~$Ka2j}OOKZke*_Qa%%Q^uPkAD(v@GK%PQ+DoRrG6TMx zLF0dMOmTnqPCsO&9=5<(-h6z;b8Y*BfWx-}z~c$<_{Sk0dn;SIh5EYb8ZeX}@-Jj> zRu|Yn{cDbWsv)#+s$E!+Q+Uzxh=$YSb?7`0Dl#~p(0TW^=P0cAbJ}+y;CV&2l{&cn zRX$s^Wepbbt#709AJvABqMUox)&TYV_%eTiwRi7Q%cyaiqmob1w76s0Ojz3VG6bbs z?PLh&wffA+l)ZT=YqF*D@!-T2!s?QA3wJt=Cmkw8OmR75Jl_Gx)ypdgZ}_O&MjK-I|!d=iU}crq(7`)ccshlPv^FOcz5HtDtEU4OIc?`b!MzE*#WJ zjV{<;9p^*CPpGnJ`=Tv*ODjzk-g6It;d@-1oD1-`IETrQlU^uftQcIA39AGtEpL<&ew8SrmU*~$(A2y9XD^KptcsA zw{8&^D5o-pgo08u62E`C1Y;=g2AcVTp?Wba=IY>#kFrWmk0SsLQ+9Hi743iE*AXIn ziS86+@Bf2V%O65s=O_}#+eyjLZu8QPw)Nk<9L)0-&$3tdC@v5KIpS0Rwz~2SVD~*w zz7cJ-q*eH(nFZ#r;R!tS>Cl_*xxA*TjsfwB#}Y?>13YCg=wnaS z6f^jNQyy`^?xPzxEIM_L9b~yzOY=yn^#=hT*YEFYI~3P;%@9fiBN>fg$w)^27W2-&7yw-1C&u1!e$K&wQEl`asb_uL)b=Xz|9OPvJ!3bpQ~4}|p9YNrph zYy*W4gmhBfE|6;|4i|qY#W-D{55Z7^~Egxb*TDh*KThP8VsO!AC-0P z-$zv=ck&?&mneE2$@w`rG0po|-<p2)1nq!63od4ovN)5MF3N1N!E;dPX@LtnN zHP)|m=ixKShHvAlRFVxO<3~=iVJK)FW|ECrQ;e^ZY^p}+k$eKlk;o()Cy1k`B-@@q zMPEOsSmQ6xC6a$^CkLw!ugUbHlWd?B9f~9yI7*}{$;R(2HIr-@3D)gok`0`8M~;m% z3aN084KyQ$kz*rV*6~EbZK=BShkx0Ic)zGS}CB?sSidPF^T z`UFK-&tt_UJ3fah4Sf#gyj0QiDr&;#92?^FuOkFkzkY`uS?cbCcWn=IG2>sGaM^hs+NR5XHi*!J>90=eBp+- zvCWkHgfv_5k@seB(u%{7#CX0{$JBLDnno!cdm_VQRdvxs;GWLXbQLmbURgENEjrT$ ze_emwcbX~1dS$T_!<_)KZ-I`o{N!Ryc^DD^GdVGvNULcG3WPKWXV8YQIT zDU(w;N#Jmmaxw}zD%T4dI95W_4n}`i=y(Rl{vVt>jzLL|`u#}{WzkuhHr<-V zELr!LNJ^IKyn&!i_eo9YEghR_a0Y@jn?n^fm!o8pQ7Md8q#RrniYmwHqy&a%T^a7C zc4P^#Gk zcyA~sT`;n^tP>tsAs`*Zj_bS4hfc90el(*_T{x*gf4*fbNcATtzniedPyH7sM{D2b zo1>_)u&W;%@kIVYxslW3kU)HkqHcea>sMbpbOB6pj7D{Y>mk;$oUfoYn)T|(6;tha zL&tD&K4O{$!FVZoPY!?~FBfiuzUr?%^ywpvXaslj%s~mE1JK+<8;bSgS zFa%6<$i*F{02Yksg*tGyt+O z>)REhK3gbM8ANh)kY3SZ^8f`F_}Xaf0_!j=h5aKiMUeAgZJ0Uz`_?F0;Dex~FoIbkp0N8wQu zC9T?wbv~1DzjJFc!TYjkO!-WFMLnDbe&o5k?C5{-Y#IgQMi-$5 zMLM(ltk_6rZaUJLo1=6l@8I|hVMckb=C7U2q&!%1b9%=P(-VRC2-|D2o~UHP74^!f zBG+H#p;0@T*$o37KH-*XB{Mf2$;?l=|4HoSoy??EEX8Dy3L)sU>!P_t%kq+qWF|3; zn}d`?=*!yPWX4NQckzFkCT!m4745eR>XsLam88Y2)J&*Si()M{Mc(n`y;dWY`T1R3 z^{C`KRORbiqvLC<%HRw&RiB(j_lorGNCBp)wcoB5yd_+%xsp!JA)q6aN4(fpbJ`3} z-CP|#)S(Evd7B|lI=c<+?hEI#6>C~%p+JUr$?|HW_=L z&Yysx{Jat)z_*s@NtDJQgBaz{vLkr|H5-Fgso`qydW+G*CB3j+s3S;VnkrSW4`Ql{KgizRU0>3%O zubwO<`Q+ro^H<-TNTY0y(kKBX9%&TPO*wbPWC6mV$z_Q2_O zjfT@Gn7iT%ZKYAbQK6IaD4g9R0D;->d&ie}%)=^x0Lhh98inBDcIuQy0Vk^6B~amX zDUSkL`*DPcw-84jSeje1=u+ZCKvQP66a2wpBKmZ=%S%2%dLuqshpygL*RT} zgm*8*KP7+s7+|W(a;zA8IN-zkeVx3c#pr}D(je}+^q!nupJcduv2^b(*BG%OXT#&WG4FJPb^xj>{ zsj3PU?@e>0J|3uM)Un}+H8@N^RBcPEAxo&@Y4`py}s%71QW3R7jvdRpIpdT=} z2DE~gp5aSk9EzG1^D{~~T>MZ-dr_Y~$Kgc@(uCuL^<-lk*Xx+SL0~JqgOm702kV|9 zsA}&|p;SAW@b(8!+gxldx(r$;&dB3s@DI*yU-9Iy*W^+IJ7octIF}I>V3s|sYyf9TI@qWb2#8DNM3*~l z%ZekpFK1?LCWB_NU!jwZ(PC_>gf|Kf(?EYk_Cz%_>EPiYo#V{{B-4M?DmUA4h2)fX znC`Ax`i10^t#h?l|EOV%b*;s9tSKc-JO0^Xvj;PT?VFRr_7X{m*_x+FmY(utkqk~% zVc*Gd0aVe351PdxhM}f1RLek-l@Gmug$J5-W0ioR)yIs0G;mc3j7iq}fu!jGkUW31 zeBdabr9V>#fWe-Ss)MU5e6^b-D>QH}<)}&H2Sp**yuz6^+!5s}g{?qjB5|2YgIW%LXkogtFWO#`w05_$a+xgJIHY=1W(hU5HPuA9an#>RQfQn z*$(67uZ~@8&@g+-a{r8RVSKC)`T0;K@6#Iw^KGl7!?)M)#end zXGm{N<%^94EuK}`jNCTxoojC+_{C6MGp}e$S`{5uiP>x3+AxcM*o#Rgqv*_uT5khE zswQjG#mUIcarrUMqFS;**mHk_XIK%qbudhnm|xd3SlX-6r*bvm9~3eDyO>-DY{zbEH95KJC-(Ng;fUQ9YrB>zr`1xfkiWq;LR1{-+;rs7vD+)Jn z6s<0=P|Wd=O{6xTkcqA(^&vH$QA!DQLQ*>HgKaBQo)*o3;dC%Gp_)!G~7BHzPKb#5|@AM;zTt05r;Sk!#j6s3P(@*`L9GfvOnqEtoG0&q|WtuUMor*AHaCCX>_%LIl zs(hO0!p8;(xAf#-L=pm(z}!;f-LPV2_)}rUU@Gpqt(XvM8!tRL2#MQpRD!Jdf8|+A)s{ zG{cU0lxzw+CKqbjF(2Q>{e3N4zLH6_Ey$IpOyYmGOF&uNY#+fYGsf$KB~UYN-CNWM zRE=VkBDZv;!WPL0tmOq&B2okahe~*s?K+C1+ImGQteOc=rce@hYaF%YxLLGDT*|8y z9TtTC16GKYHNi?#arK)l_NSu1DH!yPim$=gg9qvqlEmp`i6%SuU|P? z47OP!9(&u3v!}W*oFm+Y!G6zk(@J%ao7RVdV6`KnfZ?_@0;pXxt!E*d3u>Dy8byDl zmyE1X5dy7OG(#|*1hMbiB?rvCqM@NRh?ahzSGZk@Zy|_!)(_xKSwn z0zEl8vx0QjcVaw9PszS=M&QJjKlFbCnvGZuRX^w+xa*oR7k1&|kb(#UVXwyQ6^Ejg zHUcM%x^n9ngSD+&Og~38mlHVi?Fs?_lANSi|4?QWN?)!P=b_pVD5Dt^VYGr~`z>Cv z=>&riV{v}I1e?NMy!E6}oPhIbF>H>EV$c+B8^!Ie+aLxyeVlHC_qyH1YcBt z9%i>xnQ0Fbzkc?#hk;&S?Q6&$1}8L2!2K%-@3Hg|rN?OV%-i#dnG-B@`ek?6_Aqe7 z8Qri!jA6QVSdYNvwH>Mj9C<*YS~e$COVeRrVGe^-3Pk2GUP3A~%wfDHjU=1H1l51G z`O6PI-F}SBb(iCUZF;~Q#>{^eDwQyY+pS|DYz^m$1U82^9rp0%$ROqop0tPY1bT64 z4^tq#FX#M@65Bq{YM4ptqpc?RX9Min9_GZ%Z4Nh|!4>B4mS@?+n+}8cQ{sP!AumfJ zEMm&TT4@o3ri9rd#yheFB`sobYS^=U?4C1{=!yx|rEVUwbwHak(uhkvtQCMtq9u@(D<9>r8B=A;?m!ynnv=0v3 zuiDC6)ia=YcvivTv;i=`A3mFq;V=I}R}5tr9en~;Jp(2sPA}(?3$fWfI+Q_QFLc9A zE14v8dvPj*Q=k=2ZnS@gaYr_9R{{O$8BJ*<*u9NMm~(zmde*qJi}pA3ttp!N2m?si zKE+4Mi(0PlVhp9~b5f8Dv+M3|(>N=z3KW1dc**18B2K#aJm1Yx z$c0B>_-Hoj&mxfZuq`ax_SCvh)tGAsYg!ylwKuC837)t@1xnoRe4a!EZv>&p!ECxG zx)Wcce?_k5DZhU{=ZD>a%}dVniaFDn5L{YUUl#N{TA=#ldKasp&MJ!_)%$u!wx_CDHatCi zCVP+V&B|o&uKTP$umPo7Q&Hc=?q*kiv5-g|nKfetUI~AwUt8^8oNe{yeZDz5C|NM& zmxWu+!!12)kTSb)VtP(t6gIM z8p0;u8yp~O<~%MiSZ5jj2VS@I$RIJI6_Ji^kt+U=P1SoNdrkzX(oep#A5K19PS^4*Yivc%MnApfId0Y)hOClu~*sZrIbS1q8+V(USM>e|^E{6f|B`wGF=sw}1w*PSK0 zc&2|WTR(GWSbR;_`bF?`}VIG4Vh zv6(8+lQtrQUkt2_u^Ypdg~UPPp=vzPS~z>LR#}%i&x|XviDs+yFnUhC!y_4nzJr~3 zS$>w|CY;PotA!;wW%--{um9pE)L}?`c*&s-;xJ(B;m8UePBwbh-raCi%Ab}EPYn$l!kV?y_(1Y&+~>-es>%u50}a>XN!tTSr&&(h z1L+Nia5=p*Y9F5L;J%y=@$QGtcL%a~#pAlcus$AzKMwPQpm%MXA5WBK+5C9i?6u90 z;A~NSwKAxpc?rnTbo!XQJ$h^`OQAfLVRoWi} zlDLlx%C;`ppk7=wnD)mkHs5T2OgF^X{>WKR`{P3n)Wba(q+7BK28Wa;5DPF*8wk_)ODQ+U~ptzI3O}-hO@ZsP9vS|fESK1aVmeD0k7pl z31S@~UsTQnaN}Vh{kWp20V@Jt=*AyM0yuiLBi@BvP1y-x&~2RwU~tBDCV)351719| zWbTL;3dos{!7*rYFKBEwza^qm0ayi8ZjVtE3)tDOnp#A=9e*K&hp zm^UGz+W-owjw|ML-up?chI%G|iPiAz1Ta_Pl?mXzP&3~3@m=Z(_d$Q9nE+YAAW1}PsYGQe!n_h%yzj<1@BVE5bLmnr%fhSNLf z*%P1)FleTeG6Rf(RQ${>rI*3JodMqTo~z&}1I#;A2d4apdS-us`4UdH!ieWf9!Zq^ zjNDUtoMxX_wBKROttIFJr@&$z^sH=Zx5hGZR9Fw7pLu402{GV}P69O2BE_zolNA|Y zFgUc%0C%1Z{VOuSIo-K!ZG{lq)~AXPUVxD$Gg8sH^3KT4<~ULT29Y%!sQ?{C)a-*t zw9$2*id3LcI)Z;4h*b1492r-nqL-0jg5|dKo~7PuWbX>2DpJvT67f7j6=<#QR&V$^ z;uPG6@%oxzIt8paClSY2?TRy+Nu^Q1qnn1A%cG5SyvnwzmqB%J2UNV1HK^Z!t7e5Clq7|IsA)f#eD+Gp}&A>yeYre}9kQ`a~dkZB%1x(ssO3Scl zOBji3?>}W^n6%v+#UR5K!lCen#F3o4=W;xx6)%f$1+Wt?eH44>p`{$Oez@L!T31FZ z2)Vllxb*F8wsXJ&L!*-E2v}hBqQPTc zeFL62VnKYAa)uW%K`JX$$bt*9DCPkRno`s1uN|`BA|7(KU|oW4J7RH=fw>`u;}xj` z9IwCw2v`&G3NX5JF@O;WREgNoa9nvi-6&$V%;&hG(-&~O0;6_2m(!n6=-v}!3y`a) zfW?33)chezMO6!tj&%RK8AX_2prZ?sFM%GTTV=HdD{1#mL_ESvSUa7X01I2k*Qmvc zE0hxBzJZahyD?YE&$ zX`Ap7SERc1<~~rBQvk#oxm`Ip)s^&n4!M7v=P`Vjolx}~fPUPcAtZecXZYvBl2BjS zGWx`(7muDzPF2VqbPZG6zqs6VE0-dBmDJlsK#N;fVDSiyLtQAgjOy~jzbRq(&fx}J zq!U3?cCmv_PcdsokywR!Uws4(Rv*D*hP60Oapw>0zGDGH&Bu(p%K^0NIBG0#_?UmU z>Jqu2W5&5-9%PS=B{{?IP=CAEl<-lw-_pi%TTYv69@ISAYfImE5%If+-0EA1{&H(KC0t6otau-H?o;MVSP4V& zRrj8cj|`tCrt10q94ltbpjbqfhJ9_B`@t)6bmv9LG=y9x0y2%hV?m;9J52RvsTZtTo+eV}DlA?cPrV{3a z!{&t8qk)wvRUNhMaRhv=GRL5s-QWf32btdwuJ#<(HlnPZ%u4czr>e zZk|z0qqiqxsR~;XOQ`KiJ>U~fmw3~bOgaLWn`6+dHnNBfQjBM0Td99kVe|Rb^_b$_ zIEVRLXXui+Lc3p`vr~4h9w$f2O>V+bqk;HQinSt_n~u=sr{Mo^zLg=2*d;lAm+EnH z=5AggqK-2ywlcfXw;a2~twrn-1KFNk ztH;&2W0!leM(pzQyBLHRkBtC-*?)fzjubrTFpnrTV(}A+wwFc{`@^Ap5*VltAM9b^pqvXK3mwyyi&iQ{=fgtDm-Si-A?=Wzo!Zrv9Li< zrO~?Pt}1f<`R^{$_VY@0p{9O-#9u4;rLb8L02c|F=$kE!F?Gz=ji0TzmJ{{6Se@uyW&uf(ls-{ zveFsG#V0$&HQ&D?02cw{C*N8{eg#Jut0?i-u}gbq2=Fvd}uuA zn*;cA0cGV2e{=Q=`)#U09fPqhZxZE^H+9_Q{Wk(9>l7S_iSLsf`PZP~RFtWj6QEha z#K&vP`opKdsw+4KYzF$6BnjECg1oIILOJ^~h3mGj7|eq44r}pkVBz8!LX5+R;$M6V z0HjS8S@0#>*kreV9i9yJUGwKe5W84s%LhD7qC}04SAcrJ$G~RcD^ATEEXn)`)52hb zddMr7F7ypJOhoR|E-GlU+VlohAYdBi$#D4RMCkso!uiRpCoW4R9N?fK2aiJGE~Ek` z4IhnXV^q3l2E|gvRRE+hu8_~?AonwaBY=c`gNL;t`w7#32XF0nkY~RIjr)P8TCGP!cTfK-1XIPm(wfq?=rJCxFdvIJBe zLJV7-)=L$u`-&?!Gku0mt8ew3DXiB&}sTBeP3`;EOsbe#R7buQmdbk11V2Ga7 zmr_@MpcSeE2gxf4D_YypQArtAJvrW)P=!;mEkW>cU}iHgpn4^WKp8MBq50LzLh!sI zjz(4%92OAa8)CYrfO7u)cXBGTl_v*L7oy`8El!>YcPT`yrTF+B)QCgpvKw9x3x^0F z*YH3*abDszyhkY)MG@hiB!`m%@@VO)NBxq27eof%K^YsaO#r<>KxUGHJLK+`7gWZ1 zc!O3rlCmBiHknCc;G=Q*6L;{Ep3wdaGE~$u4B@4DHs|NyET8>V-82ghY;QayuAl@E zMZK_rJ|1IiBY4iz@{8hdh8#+RLDnL;1XS?X3odKqQ0O=6C)m-T4)X~k4+;U>mQtI>QqYF zFb~C*cAA7t`#2$9E}i&Fx=QQlOR7&b!r;kj^LdylBFbce*4l~nirGLR2RHy%y5#wS z6zJwnT!P}7MahtshKim*U%KGpY?{1(sQz^QC8dryW-0*a9O6{oOrO&n_BSX-PfL?8 zH0$1Hi27O2eFop$=5ayKpW_XPoQc_cW+0IY#DLpO)zV$w3yC-JF*Dp|;OrG!RAU$f zP@G$*Od=CVb&aDrhzLhSUmVOT7&9IG%pHPoh01TG@CP(LRA%Mq z_}4K;`}Q%2@d7!>e!-7v(6FxKqy8S8COunahCtI3-0#Y79LAMY=Ny$HVMC{lD67@{ zs#Ltle71R9(V2nVcMw?q`HXOXC;S$G_%Ca}WYT+Xf^!RMBkyo=g%fJjQmk{|_!e@W zu%P&o{a83DV7YMcq5V0oSn>iMfdq4?ybzSp&#}~5hI^=emhQ)7aOEHp#x!R<10*b`pf#XnjN=L&V^6vX9nxEBWpED+ z`}mbpq+&)s@cFeTedYLnzl%WtEf)JR2;XNgnZ1yZ z@M!&9YTXFkc}1fr^A1=!NFGc*Q_A0G$Ks_BZLVE?*j$TnIQp=E!=Z!E@Qabgyw2s_ z9EFj^kq;)ECp^V23PijMH7~>!_p}H4s2sRDLzP=nFMg}sw7MfLaYfTX8r-x9J1Uoo zzL+s#2MYzc{Q{Z^iJPU653R4HOMp2g$7h#2ITY#^&miFXR4JlmLgJV+OdmztQjhRJ zY+K0e*n%CNoMr=m7t61Q1v-5YD8}`%Ef2xilC4?XbPE*B57{@a=TuA!V`|vZDJxQQ zuBLrPX|PCp`)woCrdj8yOtkm;7P$A`r9w*gJ{FQ4SKwSb|BA{@DVxSXE; z+upZf%O0IdBUcy*Yd*jr#PYHF--7#Gib2K|!>+RM&w#&Z`+W+mZ>a8@f?b44Ny}^$kyiwQi+x z>SL20rVi2Vs)D#mtnqyv+%o>#>zg7&0j&wTPdP2H8lT(i29AX1mSJCqjXBqsHWlt} zV|g7E*CuniBd!5O8M6(~>k+NxTf>hUIt>oO9;mc`@~htd?cCVr48MWERuxVxq!m_- z6SgE@TD^!(QBsz?SV%HaavRBh7k>h^gR-L_r~PCBj4n;m@-eF%6e~rz^PmwL2_*<9 z;QKz5N;%=znL~7 zHm53okr4QmR{RhEywvQ{Z^FC)Cq@y05PU?j;+|eD^1)L_0?_z6uedC|zC(=T;}cQ1 zL>)S&&oZXtzba2WX#?^nr)!tRR22oM7?v683uM5{!nSCXru%@GDvvd-P?sG-GaxPW z-6K_uEeo{b0`liD4pgYhU}<)T!+1+el7moxtxm%tuVF4f>djyRjygiQ1umE&4NZMN zO9a%ONa6|ceJlN2rvbcg0`w(ts;hrugAqnTuje*;aA+{j3{SK`i(SuYFH-9~+g`-M z&bGX0Fj8rh(_ow>cO#k;Q{Z%IFakmJK2v4DF@97MWqwshHyA3SV-aQg2)btvpbOwIG0_TpUW;L=`9!M44ablQs_ zqW{AQS0*a$#rb$!w7oc0XrK1t6vp&nFwWssZi|r{RNKlfZQG3V_O>9KF&Asvj2~ac#$_HHn_~8tU|`{7_@w09$)v1MAnc*@_QBDAIpW8! zuFCBP^S z^bf{O@-8|fOz*6=~`ef zZa&FnmnZ;VDs~jCRlreAhv11^l62&EH%C6k2F`x56$7Z7B=<9eqcVkk1C=T4Cq$XTeg|oCzGIQ+?IHxO3MiG}e^wnUlDGD9xuL1<TA{85y@fb9%_xZ7 zbpR?-Xs&DDMGDQ>9Z{sfx9qAgB2rQ9j!H(A0zWi7P2SUWL#|TbP~8!!TOQj8VK=__ zBV*iV)L!plXf?wR?kkpL$lfp~yy*_^{*Pk_pDj}|lT@bgxiW}<%hH!q9hI@gzX!(< zHhnp!tMXFNx6PWGAY>8pgVrVpnZJJCf-#HH(E93+EhY3_5*UqZ!KXM7BhJQ|dJ>2d z*A&jfN(S(nOkwLngahYUg3x^C=f0xY3Pmj|$>J22VIq6jcZvgp)=NA_R=XHZXQ74Q zKVF>vR-g#fJ#Jfn-<+jF*`NRNKHnT#7~F&q#+9~cgqge9FiDS@D15~+0Ohp}c^8>$ z!tm7$3=ZSeJ?SwEg|GDe=f}3tVoV3~SM05-lZta5QGhakZ5a{h!-o(V7SMiJh*fYJ z%^o5J+P5T#P#+$J;nHD=K_ssPuJ`PiVC&7*6=4G6t0TI9$_uLFeh-rr7`7(UHYV`S zHBp|&&_9|gySzI@D5au5F`*;8KIWq!ha^NjD4Cb=6K<7?`3)YI9L5VK9T~pO zQHnNi;1q5?A(OoE3Q7X!3wn{n7vvg>PmI5N1pMn5p?&ul!gziTvET1w5`^DWj&J&V za1>r4e$|vELRcN*SDiWJMmcQ1A_ELusd2>K_h;pQDtbQF83HI{9e1d9#r89*>VM)T zii<{u+PuE~Y z@q;pd!~pd{q-y7o0qdiba9nYDqgM_YAU_PnXA-$N{~Af;=7en|kwM?rnM6*`-rHt> z%a|OTOd0R{d~+g+435mJY1FZ0Q&{hWukmiEBr+Hg#vo`hP>%8u7_PhRmoO^QDsNXj z{sCc?3Sm5U4jEH+MbepbWIsqrsPVe6IbZ;*oX=COKn3L6YK~*JQ)%MIc7!#F>;K>WJnMkm5fAx zOmK4|D2SUd+3KJm9{SkCW25v{vrJVsIXES3LNv=?#Z~{>lK7#U#8@sRLt@Wb({)XO z`K9pEjtLTcZLN$63cyVj^RzOPj+o%)C??1|?1~*cLajCt6NE&B_~ryZY24u>)N$Z_ z3txEz2{BB!T{1T2v8eIa4heELE?1m?m*>}(YlQ?i9U;L_$;aF@)iq~Gu>0_j2nm8# zX`@1dm>X4OMMx0utwMf<1VJk^pnWREbnj##+Od) ziPU0^nBeDU3EE7j-u)l*$lV7%XC65j&Le}46~q-)T6#=|KP&aOV}^pX&{;x%DZ9m6 zKnid7VF`J~tpo2Okqp2k*_Dw=CW=$rnPdWeJr29$B>@!WKAUdbr{Lr|#i1ZpNWmE{ zKvcM#U{GaelJQ=*)r&I8c=Ia@^aV?F6!a6v=H5AX0g%LIuPaKr%51SMMO%Gll7*!c zhW$+plE)f~XktNUF8SoZpeN9ON?6SPMO34zC$L1bjFQQGsi|lx#*8mrbpwOR6<6~( zn~V>gcV?40#m*<(0pA3qwg~VM&PmB-w&31V;h0P&bIPAzkxS0`b}kvT)^SZBu&%P~ zW&}M?_k84o)Q#;%}U(WD1xavWFiqZ0Ts1>-XbdL@t?sC}B$+_I|#; zF|xY?Lz|wZ$R=}89_d43dic1PP=`hMYg=8GJMX~1m+T=E>Qc=DHZ@11yEcl>g>LlR zwq6WXBQGSf$(s|oV{mR;6TxZD;|wDdRokk{Cxcc&OXQOY7v`Z9JlL&PmC%;ua%n?o zOM-hH9MHvuiM&G&Wh9h;xl&#IXppyQg-Te4gEnqRa+FWzY+1nyt8Thg(2+i# z28Oxp(Ow4T*ah%QBpto99l?oRi8z0?I43IG5wsPtO-CdQ#;!jEtsTUQxVAUNEVPk! z>~l4?7+a22(YI|mR<}Wh=~(UGhV2NDgZ`YhBOrRlS$?=VX*<^G?bf!V5e3qA6a@8Q zI@TScq$Jnw(tU4#uGNLsd;0}`PwmGQm#ROFx=_|{e|JERxmwhgW9MN!yZUg`VL5J& zEJx6C{f*;O)-D*4E(8!#Ny`zhsECMEUEL-1BavZN-G6`F^kK`fMus#^M;_%ox9ONd zZPPL7G#x+0{zo`KI}!U=j8eDdSYy|+lCys`hFfL7InFENM;~7rkNW0hzBk|ai?Lt0Wh1N- z!pX_Za>ro=<$k-(`>!}UE6mDBEpLu;%iaClPA&K4jeaX(WaAV`g(K2YJ@ag*wNG!t zTQyCW4d)fx+2FSPdcOz=D7DYb@&K7;Mqky^%E zFe~BIa(72$B#!etGPT@|$DUVY12;zuBc&$nNGIjh@frKz^YGjKQlN=F7F#CxxAkc$>seHR_yC?xuKHF zpt&=DWO5lCg8@KkawONt#BR5-yi>p6&_;D~xKAO#MDlpgPGpzCVXOBnL%dqt_b>X1MaLxP8JV8zWG+{-$4W=MiaNejo}$ymc+uE*Y&dXn)27f(z! zHiJ|WMW+L9x`ri`936m&TPh7d7^LbE>1Bdx##guynplNsXswyR%7F@Vjw88?2RYDg zi6o#edU#j3&Wk@72{8fSqASoJ6u-}Z$%boNZvG%R;mCrc){M=GAj1&67rJ7d*5yNX z@41{3yk~WTJ5hSCx2!K_f6dde#W+0eyh4XjDiJ(M2YSVjpSs+Ng)IlRH)80+UMN&o z(tEaoa|@cucm`7nmX9c2D}>(SOSoJMAWxTz*u5I_D{{T!Gg}$Q6+%%~B6)Ft`l4JU z)RdE&@~$2$T=6EZMHjG2fQCbvOR#y@Fml9_n>rGXLPjR&We<`c70z4u zVp|`CFZQV+eooM;*TWG8{dF%P|NZ^1+=G~{!TEV{?n1=G*MRA4>U#o=b{w4l`CtFn zZuJIQ_0m-6MEZ4i?(KiUc#ZjguNx&tyyc650Pq_J$34G()0U0L7YESa@7KUOIyqS4 zehmK(yZVj6;Kf}9=|~QMRNgMZo_I1)tlIgTo7VA*GkBNdlJomGH;XXOuh&=Q)?aa6 zYBf0c8PLDPfx;!-O}{>e^h+*43&M?@*Fqy_Vou=w>KEwyB3(AkU%*9wKd*ozt!KQf zhZhZDC0(a~(c`V4#7S2?Q>DZ&8dWl+X?G>#m=^?t%D*^Gr7}PJSO4IQ9B2VngM*k7 zUzP(Ra1YY!CXJA=Key{SvuHf-kq;p~lk|XM=Jo|#v!vVdl=ucUe(?B1eErhjp=I{8 zKQ_-l1;N6Bx~!MHT#WdCQ8)?~gc(1Zks=m9`C=SD5yzvx?|Mvw7{{3VH{YE65#E;) zN*=8fGKv`30A6(CqCrg|qzQD@-)}0}7YFu;IQS*#rlm{eLdd`%c|8s-JO0iPjNYJ$ z#>7_T;W^H)nIc1SPK4wat(v+`2g*otnr^{Q_geYp9K+*ZeQ~;ffHr>g?zQ}jbNtFf z`VY>2di?5}qpa1zsmrSqbR`WtT=by!nzh8p)4 zj$ga$2kMa^?RTtyEq*{Dep2_piM1rXv)TM^Ny$5INdS5Jmru&_8^+$slpJ^Enj_QJc zbl0UhK+=n})_ITDfI_i0;!Qh8zsdvBhm5T#@TL7!B)#wt`FHjT-L75cQak=#^XBTAV^$yr` zwH`%ZL36$8BmR7SmR3H}zf#Cv_`w)Vv5>y!VM)P%CnXtM9OBCwV0fIQi^c{K$QKQV zici(qHssLy5MtWBRI~{iM*fR_Ef$?4`n!U28|aJPB?qGGEp-S0IfV?=DriQB%(y;8!=l1x#+(mYLR~&G>04J|JUHbo z$YaLEI3~QQWZJWNzcYrs0&z%BUGaE!_1ATOw~)iNB-)QZv&ko@^d1QYMjF+n9w@kC z*tNMh9=2U`6>}4XJk-=y{>39bP~wUy4L9aVKVE=b#5N-f;|TCFLB5`hrV-W@l?&$= z9#(R>;AxIS{FBp)4YRHAIXqfVOJ}X(TOA&e*!wrJb;U z8BiAVkY;2x7HI2H7D~SuL%P;|*q3@ALvg<7mo9z|)k;n~FKC+OA$UB{nxe%D`4{7* zsx(Oi{JSpA5P@*7t4LWjS8gZg9LS3f+czie-hH2M&R`<)M=x?|zhw0m=tpi_*Lycn z+WYvSw-zdU!7gGB=I*JXTF3sX(XZ!!6-bclo}Ht=G#g=nQjTi3eEtftYDNhSQpZaD z`4l^1IA29dHI2>JMT53@?!cSlIuy_&ZB3H19=7I-gW{jGHA%zNu&qh@nTM_U;>bys ztx0l}v^8J!lI<9_CONhZdB<|mys;@s8X7x|%@+ru#q)|HlS;}mRX>7iKI8A~h zW0UksTazP)#gw)tIq{j6qTqzB$(xKPZB0IQ^)xm~hgZA&8YQenhTHiecNmlW(ljkS zlJ=c14}Ga=YjU!#dDxmSP7AA@d1TP^qzsrFblRGm<4S7YAx%v^q8z3sU*fK5YVsWq z(dRd``)w)UsD$%^`g{eG{vcDXlims zTAF;zryQ21-g~B{$;qnYPkKy#O!93Lk0qCL2s;Sc`8oQpX)j_y=U z?2B{EfPeK5&Y;pG=hjMlSmt)oh$qQ1H_%f%pOM7zaa~rimI(&qP#7MnL!ls%>9+(4IKLu0Qm(NSMd_BBvEH*IFU~x( zBczY)$cwhjM0VuG8E1Bcw00?WHD04jvRjCi8HYk1^*uR~oMVy3ZN52?9O09XTxV`XvvK4`$k@4m5t2u4WY2iyMm}Xg`vdBk z6tKV){N0?Y!;)Tfm3=Aj{})|kbw#1!i<8L%!YEu*;kHoD2&>$u@`k%xj=$4Q%3lNj zuD{(M9k9m>JVhpTyrAipr&J`zg;Kr&#EN3)Cpgr^A9NdCeg(Z?Do}sHpzmIx{j$ar>YzoPqlf(DmBnGty90|7}m0si4|fk9U33D zwJc%}r;Wzr70x^K_>6c4pv&6v5|51Re954X=`qJW4!nax+iY@(XV~7b#5fqv3JAXC zsWL;vwo-K{DX+jtbl;)nP}S*#R*q#92j^H{)BvAuFq}HE+BbSF166l*ZL+{FGRKtA-Kl^|^}1 zKmJ~R*KJ&}PVvpSS_-_TafRa2)3{p7#2?00aJDH0l!^K;&Y_e2tAB8|amB*cZCow; zBw<{CE!lyGaiy-?ZCowI$75YDImXAzU$Rv96(iKHj4QOlz8P05H$53wAtGU1y$exH z@VjvpoG`A|(7Kj!g<6p}<7%ZbCF5!}hAnbYqk{C^xLQ?KXkJldbB1xXZpCZTxLQt) zhH(`GAI4Qo%{H!L(zkK7E>$vwakaERrE#@??EY`#YL!f^QMas}_>2#`-;FDr!*xbW zmO~j~T)pV^Zd@%{&uLu6*xR^zpB%+Oi8%D@zGT<@73UTQeO52J!y!UPG2_-c+VNJc z(;t-qJi0(xud3JPp(p%7|FjNtFyA%?JANu;srN9ztr06bx z9rCp4IFK+o_t+&gip&0kqa$*Dj$eIoio(P$jKH4p%{hOiLH!43KRtf+&7o+$Gj3)i zt5PVJPE!#*Nh8$kkwwVmn$D3{BD8?+qPm9do;eYnws5TT0&SO|nN#I|8I+_Wr^n-!^-M|Ah+ioCQ%8t`TEVsii}Uj( z(@xi`4CK}kM1K#CDhZkV!-BkowZz;ktmaD{Op(1U6iif5zGxf`b;xm(tA9v(lP=m< z+*AVl4i?<~jH>KczC~58Jl`_t;&Wg&rzvgi8{F!X|NmM0b|gt+9ZCJ2qU05S%J~1B zV~jkL4OB!%1PKBlKF)ZAg@0!h%w)=|UV<-~mnXXU8KQ9q zKlEI|fjGawp}L>J!Hl=?!ETXTzh*k&)58jRxL@M_Z2uj;rfMt9G~IOV7~Lb1>My1mZGmO6DN690DmE|vN@iSmanp= zWj&>U8!baDA{Vo>%8(nnuNZLg@eBD3l|3Rg7v-UsNGe1~6tlM0*ZXwPRJ^t(CMMQR zjS0f3ug>^=Fj%=YD%Ps~DDrZ)d!_zb`qW`R!*MLZ_&toFmM;8;32s8d2oIHoMWxGRyjQ0k;idPtn`Dgh za{WiGV_#VwdTG$nK36n!1831>=u9&RiMVhK{E^HF>R3?o9`1~jpxl9PUTo_fkq#k1Ts~U*; zKM@_Qz$%q|Xaf$6)cu*-Ukp8nJG;f(eqWJQnm*^GmqsW}X4A8UKMwU0HzJ+;qf zL^QOI!5UrK$M%wc5A7otJ+#keZK7+R-MA!<_SqPY&_3qC-`d9<_^0;SOx&C+P%IPL zXR~(T(mvJ}acdtSw1oDFBzkBcaO%F)&*2>XW3W7z{;^2FrGNJF9f0c}Yu35+&t}v& zw2OV`Q~&IWsQkG?o~`R2>%e=cAM2mH)ekVwFZCn(Qa{drarLt}GNFDp)iJ^76)Q@vq{n+H( z>c=2>p6X|#ZqgOdrmGO@MKXFRAc^(4qWTq7K-$P&3JAwTUkb=p;-!Fmp&kYF{4B2b zOuLzR0uV}n9x`1oa(>Afc$F{wUmpVCQlm4-T$FN8sA0(h8c3P-oknlfVWdN2^fy9v zaYbUSl0BlUFsJZ4y^@DsRfanTGE?Nx_K#W;PPeDG#R(0{3YGd7CzRp&olg!=+5Y9L zU^uvh@U9q6;V>9&o=={Du+i66KBusRhYbsQ3dTo&>!C`OxyoV`is>1~%m$X5 zv#GLqIIijs%Hxvf+wcu5`*C77b2#c-OLbAx3)$DR=CKn*5`XmADCe`@DEW>;&T_v{#Iml+ujP z2(=m?damF=oL}Hj-Ou1)##{Jc>m{GFfg2{bz}NhUBPbess0E|6&jX#g*0~eJYj!G9 zM!RdHzo7%VV^Te_MiIo{>@;T+$5usUJ+=7Yrx-8Ub2sa3Ac2RLcR?t`Ckb{qW86Q#u+8Gc<{ z(E)$xyAKu_t!w*%vXow9`@x!^9@~$gWBYM9I%@;JK*2J!~?ldARj(srJ2x(`WPXkbBjU$VPJaaX8$4aCWGD z=n)Q_ihgz=3Re1P`vJO0OSc%wPD4t!y>G@?nio%Lwfg?! z{llIu^K&D6wlmL@d3|-UX5z!KKKF&bt(Dlt_}ZyWoR@lWtY!Xx=%=24E%OVAeqYPl zB5*Lki&L!B)H%P0WAW4`j_IbqIM-c%Xrx!p2Uo_DRe|W=7QjBAuwJ#c>{52+y{Q%ub6AKfZZFle}kr!=v zNTAK(0=*k(HQ%j*CMn~8pnY`}+xdY;=cv(R2aP+RD6T6E6Bo2Wsl8}Je&93Nj=7j& zwD+`j0nmJHq}Xii(g1A^+7CxlA8MT6i^DYiFMqTRaI69ZkDVQ}Z3YN*$}#(Ral{Rb zW0es65MF^ef_>@?5;xer_XVanK%*IQ0R<9YYSlNm14Uo<;u#lz81gHpbd*C|B_I2G zf>Qu1!ykB&4H;8D!9XH8Eql%D$mwCpBC5pUvGc%3c^_m@|INwjCPz zRPaE<20e0J(e!fYOmE^{@FC{dWG@|b`tRaYO}I@>$5$G{HzuYR(1x7iZ@usuhV2RZ zHJ}TPckM{2Xw8*>N%py-L8vCrBE;M^Ji=wHATey5G`adJKeUL%>Mb6(eETe|`fkH1 z5wqe4)6Wmo=GPLzX7Ik~1KK%5C-k_;#>^J^d?>Hcxpp@?Dve_Xwm&&2f4J|m4%4R+ zG%ssE6q-fT-#Cx=(bEejzA`$!hK6VhNbvQn4Q0wpK5ycGn6WGE{?r9j%ll!@87l4v zrxdJq$nk{zTk?yR!O`FcST_WQSr}XDP_+PMx>hnE0Uv7}C%E2MG^!VZ|7EOz#us2O zuW?eA-oq7zvoYc39eU#XM?Fdq9}mqo>S zRfi6=XfVwHvfImw()jmQX<0Tx3J@N1Vhm#yxY?2*Smgprmo?l8a`C zAut$!L`v%7vfn7(qU>vG%O%>DVtL5^F7!gL3nbY{&zf{e_AOrgL;3U7!B?t&hshi0 z%{$tX{7HsAHvTTv%y%Z#s$QDcK%^wu(!9=@+2=MmMR}~$>0wk=>#)1jXr8-6XsSia ziY!wCCVabSzNevCuWNL?3H3a*uZHxj67gC+(&I78MRwH#Kyd4LQEIh6q2^l<_XViH_~vO8bB2>3FKU6jK!yaa12eq z5G+i0eR5``oQ*RF+%y6VC2eDPNIO0zbJ5C;GtQ!;XSJKe25aF4Vz}nzC1L{d_4n?7 zS|B4I_S%rhku7Nc%mxk+_c59 zi7J^}iGh5R{cb_22%K`{!1Hp4huJ~fRUb`jcjWF?Kvz^6>iC87Jsyw9ncjoHM5dD* zFDY7|p*z4SLekPN%%&1QKEkG$XbpOQg-I+me?y-ReZrY8eWU+fL*RF7jLxrX5aa1S z?EZGoY0#kd@|+lPh)52)a9;p0h`i$iRnVGITYnaDUf1&TFFFepiH-XTDT%nf#W(l^ z?6jBr6}UT&H+1@%744sS?gHq}?KKkYdW6{umD1$qnirmcz2rQqJVYy$JwEb({Fz=U zBjqtfFh$)*nPQZurV)CdCr-Ujf@+y7w>b?v%75gn8k}F1uxdMn&#Ztlreo+Ou$(K$ zBYAO#6fiWlrAV|31exhtfyDLsyIz0xJuAVJd{fyF2E=!6xIeS0B(^LX1eVqs*(gh0 zSR~?HcFFd&>o3mnFyePUIb8>Ti4jGRnk-eJ3T`ePma?&d=Ak)saK9k!T%%lc+nusZ>q+ zfl2H_f5P(=Ph zI+B)^ta3Apwb;Ru*RW@WbhKMmLXtr4KSh3x?Fj^4H0^L|(datNWG{_a10Duab zEEIGy#So3++gA+Yg766EkWR%*yax2=5x}gMdWvL#$~BIg`KtdePA9%jXPC~k3&FkU zc4@C=*j|&vxL9XxJ`Mvnjs>=81_JwQAQ)gJsJ8rYE-+o(B+N8_gmbgb2LcM$V%VY- z0n9DlfuPuC+p*KC1;>&m=l5_fh1DiE4j-HociA{y_Q@&mg#eMI$FbYwrA$;L4wL<2;B)x6Y6WOq zw+Gr6`fPUWGu&%;4$sE>eQ+iKc3*U(H^hP1v3GBOde0p`sWq_Tsf)&%D5Xo=m+okI zN>NVM=kFDp8N_xW9b7c(fRT7iN77s~0ukVfH9R=t5rNO1}} zUvgnO+t-PitV)H|bo2rRXg)Ru&fzIX;&g5_2}EZHo=v%?UtYEjcg=;o)1R9--34#m z)>rI*M6Iel;tW3BS75=`*}AO7(#~K;V1fB#yn)Z<)Xo@gm35)gf$zevv_4cY3L1gf zL=10YsCHMeB)R8Xv5+R!E)|5`$dgtJ;ZsN58fl_o^wL_F17IRXCxUybrLazNK*q%n zNkY=c;8iZiC51r=NwRV{z~i}GyYCR2R$tRWf-XfEPgsAA(J{uad>-)?CIdmm%28|_=@@5G~w*?u23;OUn{;p5q>8lz%Ry$pIc>npMP@Bo>(`%Gfp($Uc9uw zIB`>q=O3Kw*}Ch4bK?8KS?ar>+V%K8=zYcaH_yn3_`cDtocKQ4I_~iPR_s*aeKe1( z2=B97VP5flaC-OnKFEE7I+*l6FYyI`XWthTenRv=sI0@|`v8vg;68>;mp7n4;K+_E zxPL_VDtvzkvw8^zo#+%`7i@7pnfc(T00s0^axtl3K2gHBm|6bM>A|tdiRVyw1&RYh zo(dE%vG)oT&F!3;47vV?4^CqXy-0ScAeJI7KMz z@lZTDXBUf8gTgtEa5Xz1qUJ<@UtR9Mw{SWDT9Z4SDX&7|bbNUc%8~6VQvz$!9i~CKFUdDMER$B`TW1ynLzNpgM;3F= zwfy3%oMT{}Dt|+1bjiLHaCRY~JE7{g-n)3>l+c`&x#bv_=8D!8P3a7x#T6s01%QqU zkfB+(Fk>XW0Pl;fBsI)`)1jANte_swqc*y6CYN8Gsh_>U%N{6=4Jm%nn49gOtHSsJGa)+cYz5N8*E>BKa3SEEEpbWet` zNe!)*N~fyxp=6I-;gBt(x71XUlt*bqj&;DzOxDn**!+Nuu$E87$=YG=8a**K$CRns zyi#2{wh|9jeMo~}KuGMui0>RqydO?GAiq&*7aZ+e8&9XRr!4M_cazK|rS0VQdP!mD zS;UaBTMt^5*z-aIB%yyyQ>KWlRT}-5CzNOxnA&?6@c}1k^k{ zEbMxf*`=)17>2v^Cph}7rM0!g=Cs8Pcv2(-Gpf7*g=V{d3&(byhcWo^M!i}dTb~L{ zfwT3gl!@5-01o@H^{F^e<7|D{&vYz6eXFNq&(wWEJ1#34q8N~k%?~&w$L0rZT5HAb z=g_|W2~O-U(CXzT`;Bw zE29*bm8|nF5vTaHYX?-97oqzOsL~ONO%MTn&L+sylxGvwbP7a&>z26}@f6q1$%C)j3axUIwQ7}9 zp5;cg%GL!>wc$BfUrfez@I zpcxPy3CY7LZ2Z;_Fq;_jFVxZa^2(W2ZTa+W;!$QRqA$g-n@03K`j+ zOCdXb6tXy%x?{)JU!0>h{jLwrrG5?HCI7{Lx$nZLelpIp{hklbrH-8=bvzV~adm8T zztyoZd|R@4LEo1`9@=Cag*>VkLv!ky*MCwWkLKYI7ckqFm7|bH_Zz5g2Qzw?LLTFe zm{5MorI5#n5>zSVF}->~@)VIsD&#R;7F;1qhL1je2({3QGyNy8A`6Zx=YadFs&4py zOLIqT*ujXGew9jt2Lc@_p8am&!3~w85m!WMwtgXdye%HVokG0CV5Migqv$BZI-GK_ zpo_d!0upCW9!)_c^CeiI7w_mB0e!^TDCjHW?@I{|-pnyNKh80Xhx?%W%RMInKfmet zo@sGTk&ACSSOFKvq$T_Hfob_Gc77s%5lH8_Hv#9H&Zc0Z_4c}=dT>O)Q2aCe5df>X zULv#5EnZUe^m2fy=NF+Ez$hI)-u6#2zi zs>(0MT2+4UvjGDDVnCh0kuB)VBtEPBz&Y8yUoyC{L#V*W;@ddDqHdP#d5s@9%1E;L ze|GOv!{EidvVw7$0=E!X2(M?Q0-VO4ds1B zPRTd^h)!TKzD|7~7?qDOmk_i_JIy7ge_Q2!@AD63YMr}N*+2&Pb81|J5%qMA2l$O1 zDtJwuG_xs+?^}O(rane|AN}H2d>{CtkNCcN)+4@e^tj^tKwgpS-t{rll{(`4fJM9F z`v9I9_aZ~j`E@}~^x+lWw{GqJ2=5!NM!mxOnXoqz-nX%6SbXFKHmME8x9 zyMpMx(16Oq49+4i=eR~HLHpUD`91GuYGXDDLe4_g{D(#ByV{X>h72U^d zd_{CW1Y6Plpgp?(Ci$3|%yxx{f9`|x714bo>lNJx;P4~5Z=7;P_id`~sFq9VF<>s) z=ZfK1Ai59u=tp$lIEl+RO|8Zg-H+uP(fzm2;Z@-xnag@Mp1e5CYr=T~ z{fm>I-SMx7Z7*O7&UjO|hPBWhK6SR>E(6+)m{azwl%Lmhra3N-_V?$mf11A^!IQV| zGw}X9ADpY(>Gc;@1f6}q zaa3do|I=v2b+GZzeR10r8-L>{#f*(w=3zp|e)P!XxrOQD?sj_7f8EuG?fbJ85Vw2Z zpL?XmuPvtlRu2=M0hG^}#v2e0$lx|Ki+t?YDq#aIUB8t`E-H z`3HwPeIs{ z@yFcWjuTG{za6K(e>?-KE7tc_ieluR_rBjoN+qQdr|qwReZSH3Z1aCEA&!mzn^;rS zC;KsvuR3HUjtv(ld{v8s)XP^L&09^}Ssko>4rg`Hn{{7xbXBIgt2$_q!&M!up9@!Y z#9q!_)qxaTc&cN1n3}^mB9QF^K6XSCaK%hM!*o@7>eKD2e@^DD!d10+YPnq1srKOG zs?OrA9`33-=eVl2)7_V=I8j4;Q4x1lXS)2fpeHpE{zx%>C5SqG>Ybw;V@wOa}yzqiSpY z`ujY)F4#toemDiOWh)bWhO*ecK!bW=z7k`os&6N7o&R9lG0OZA0&ywvAc2?%9Fvl!%l zLR;b@qNO3E*ysYVhk4?glTjI<^WuI9(PK;Wx~2eYfAPU#d!3&3OAI5csk=FBHm(z1 zq2Pl0d|XpmPJeR@SMYHYx32dkj}PU?HyRE**1Vxvb~PF@aefb{M_;|)^~t&KB1a#8 z{qi*&{;czk{yQI>xa+?N6;m~^1guh{haG2P5Qj!vPNDKhYtZIM7|q&XV9S6(bfXt( zO2}w~f9SBDo8lxcr)M31=avL7U>)^#ykiEf3id_pNfRX$o7>3PinU}kfJvM_2rJ^W z?lteLEy~y@=NcZ}^}*?EaQWrS+t+_`uDdj(e{jyz{jN_=^3!&lV;RnYkz#d3)sZ(w zo0YM=>KR(Bn+nirJ=W1+81Pz0;^26c`NkoV&On{Itv8^5DM!+(7x2^dgFXWsKYWo68ZDKqn#bH{ ze|GD!h8-M2l9}?Zpsh6lp2z~}cwqv8vSaCi6Em1qZH@{v*bpWfcQj1fxq@gS_6ycQ zXmsXFv05I$PMtzaY?{GBC_5|+m_q}_Dn<+O!#USzb~;9%9FsFP>H1xt97`q(j?Z%9 z0~~Zk89k}$GXU+BpKE?tI=(lP18@xZe?Yv0plzeOA=pAQVzJ(J1s#V5#DDB_D&AlS zZDp{;xe+usq|4IRdtY>P@x95PaEA(n<&w!wjH_$RoI~U#aT1plI`068!zIOqZ@m#5 zcV8fDyGb2Q$U-(Ab{-(`pw^N0@hFr?3-ViMaA=8y7pteRqDQMZztdgoU+OIAal$ zraLCzqH16lLbCURISa5pm2ddUoco1)7LwR!OJ8E3?f}e?y1g5t0mRiLuujjy2mkc!|e$U`3KPj6MntS9$iqtymux z7OgHuo1Ru37rk)oukIiRzeg8}dXs@ObD;eFiN$BmOOsUhez`x^D|u*rak(#;@dG(a zSm=`N61tpjTxkD6CvKGd^)@YR)ixk^j-sJ#|GPNJQE&zZpb2{#BXV^#6&5MwHJE*lc z=PWKxHgzhUW)2O`f9y##*KlaEf9;Y#!@=nND3r}JVJ?+pqq!beG|X6oRtCBBN&=I~ z?CEXVh=qVu6dk{d5blsMyo4&uBk@YqlGPech6i0_m<0TiiK6Hlee>v>unG2|>(ICZ z2h*J^5-S>77XnKn;%`G-)~B6D11Pc>dwyrDXWGqg(>a?ie+Xb*V&UKeK$B*^jxt5AQD>laQJ0G0eN5(0qF*5%S zqak7o5?_vfe?T0p3O6z9z#p8=iXTl5(o^VpCR%*JJ++|ijzVOBxPqs>fL;QtLewIm z(iV4282y@vqhp~b-au_60Bw5(fl)Z;>Yzu- z{@R5jK~XQ8!5r>piP4pV#?VOY;tKiLbZ|LTyrb8^f3QC;D@N2RE9CDK z)Jh;710*A^SYu$L64lk{OhtYjrdt$4DmrerGTxOj{D<}?Xy@RR0l}Y5S|r$Xqouit zc2|pU{e`Iwe6w*`sG5Bv#?I|Zyb;kD?gd{Yt|-%Eo3vnM__oaPL`wgrV}VW7R-+a` zq~@XFe?LvX0i1$^;z7d?uWF4hoolM0>s@kHq{q{eeo{bSsmexin1&kg&bIx%k z2xr(U3wNJ=5pGD_OZgSHirpq^0tV-JN!q1sP#a$*c(S`##ZG4cU;89yxO=qjMHX!f za>!baI22{phj34qJT~7|fr3D@Iq90DTz{7$p%EpxV2|Cp&oRyH^eN`{ou@lL=0<=+ ze=N1SEP->}6m%3wx+Q3vF5s~Qjb)f&2^t$UL(e1H^{@m@`HHwgW;V_buG|$JOVAjc za!b(Ae;S6M5h?#N1kO2nzoCrj$=J6cuzHMrSpo(RZc8AgPBYM$rVAzCx+`@z1C8~3 z=Ze9Rb2HG;ns8ZxCg<6$z-wT0zpOxGf0FdF0&USQn`HL_Drl}uBpb;7+j6hquK){2s zM{*m`wp$HX`O*)coXZNh&>Sn^zkIAfQ^3ZpfW6uk9>Y2D>tO{N@KnMIG{S&|j&H0& zxU4{H&z(5B<6E;BPF#%$@1!z*pZUI{LVQ>r3Y6MuDj$?0^ z1!$fpl>un1xsE?eg}4D|1pT}WfCkYGKx2sA4Zw3+XaEMmgEmy?v%Snfc4^Jd! zEehv!bS#6z4$|s)fFsiOrRr?$eL^Gn)#`a)o-nS;4G+%@R<)k+4G*u1zgrm-wI-3C zJG+xCqLc$ca#V?<;2#qOe>elhO0)$IbATEqhA;#Q+HXgS&(NaW0V}$wQ*av7=2<48 zvh1Yy^3Z9cJR-Z88mURFg26yp8=jfMd{7hXMEc?74-k7N3&!E+L+CL%&9f3?bF)2I zKiR70oCiQzLcmW#m-ct8VV-*r1;)2f`q{OQWc1QclIplZWn}gXe@%3y#c?_7+zz#H z&L)*{<-t|HR7;re=XY`QKb2Ns)b1ZS;}O%}b9#Oq2a)U<8CF{TIowCDf-jvkMLg)* zsLcBbXE(bCrHsb)-7SKyU?aQzB6RQMVGL$&1syo6$J4oH*>z;$bvRU$_)tyfF$Uub zmsk~9`HOQ|Gyj=Se@;gj=$CJzsijCx?Px3?v-m9(d9AbA-SX490GeSu1EY1>RE_5Q z6|&n~v%z;1j;qyZ@&O&Ni-vun3t%gK%BLSs_9g@r4C_^@4_jba(Dqm;&T(t7w(ojI zK!)yzHP{6232X48-6%m+?A(v~um)RMZ?^_JyLF&7*f6`te;8~Y-Q~t$SNBk<>opT9gdrW#cca!4m>~Y=3v2n3~NC0G6rI} zEm+*Jge_Rxe`EA+3wH9DI&FbX?CrnmfS}75?CzC6x+2yESKgjXh8<(D758^z;6g29 z;N&p|PpQYmNVy$j;2aqPwd92{*xePk8-v~5JWQXPkuOy*8NG}Fh5B5;<8Q(i*l{`I zVGA~z;ldOw?##m!_(DCl;Q3iXw`00DV>tgEM$&ngf0;nFmc|-F^MX!Y0nDV9Ff0Wi z2(idTc`PEYNz8<_lC#GN3AK__LK@Fu<4zH;?^-~_c-!{{ z1o=`H3*)8=4Zop#K}JPCx(|F8E|gDA3w$k~e{`0@YqGR$AfjKVU;^6$k5mXfaOom` z%|oRp5|Tqhdb77t1u2*^@JSA-l9$ah~!lzTqydAYboh7&lTO% z$od70)AfiFE<9hthu&ZEITg5r0M_^9h*N|US-Jl zZnrFX3z4a4m!J9E%3d^;e*^Vt?Xxf}`13GsR-$+qhgMmPM}dq$Pb z@&c4svu)cvM^*|e)@y)N<@w~Ke{bK%x$by2VrOSsqn9YHm(++8Rr5E8lhl>* ztrDbu#XqkuCW}`lK^j=YfI?A^T|2VIC+{!NaIW%Nue}O?y)avoT^TC zX`IAt4q3ASdrISsf9^kBG2A+KQu8T|bGo*mG*Yxe8buyye25;UaYqTOi=#&i&wx4S zwKEu8tZRGbgfF_HN2bA8snXtDqRCIQ8tHij6qdGO@~Nz#I9xDX-@TrSf9Gi~@Ud`v zG!M#LuHvOq6XL$4-zS>D4!!XWy5rH}hKKaO7|Age#4wu-e;HUU5^=EmMtjY*UW<*r zmXL%?yLc9;*|#ow^EG>!@!?8I0_jV2`pUe6Ru!i2J{P6LtTwM0Pe*)B-`Kg`*O5Nw z3RX*14}dI{HRz2t=f+XxJmBmnC!FwdmRBdvHIfpuPtGlj2HTR3==~#L&br$@gL8D_ z#aV@*J8PWne;sJn8LS?NNvi0zBZWb0g9ukE^DtEu5ih@?x(BI8cv_o}DBkq39RH^H z?r#(LDSN8}kti~uSB`Q7uFriz`aZ2H#PsnBE@4qEPkvRxClTe$bR+N%()3m?c2*L- z3JilxAjxoM+D&xTIPJKBMr9D(YjWJGhT+g!<<7aLe=Bv@1aF*c0uKkK$>`45FX1Or z_oxCp!3gd}Ltq^k#G2yATH6+JpHx2>=wL~?uPmr2uZR;kRr4+#K_n5w!L3_Pk`bSu zRQZEv0#k8kL04ptAH%#E?cuem2}zy=6k=9#Z;|G06I10fZ{pm#nH@@>oO`KlFU)`E zuyS4?e|##boK?7P>NU(=Un5`3vCxVKZV*i{dR>8=taHtebpuLNzOU%+zY$E0XD7D6 z8#Gud<HGfwygbh)wy25dDS@x@#saKcY5%W)8pnJ`*aW&Y|#z81S*UbM1$G~$M) zwIfOJLwK!HuD^Q|zG+dyanVKC0)!~l^B5U=x74U>k7_i&0T&)e*F^o}yT`XUWaK$vT=k zdS5u@z`I^>2jd5b1>pykE~t6~`A+P5 z0jI4K12Ot4bmAvGyeZ7xy00L@u)1C_Ohuv9+I--)0-xn8M&09{Z}UPN)z-ZSk(B~~ zQLF3gWKj<%X*pCm)vOc%JsC5Uqn7cGUtTHjqUDt@jc;X{N>mEGIEw?u5eD7we*qS7 zo0kd@GLA8(A&j}JG=P|M`!Sp?`))&56rT{THFFCXLK4nBfb-~75>9unw@l%JXhiED&^5e+wKA%LPCym;*RyJVQ?C)-z@_*a5k!MoS-+P`jHX zG+F!BQZ0HO=x3R?`E`l~I2&@MIWCfGL`)Cq#RBhwkEu~1m=_C>!@wM8yn0y#juNt& zQOqlF`wnyo--6aN6qBU%(p-}3xS}o(1R(`%BS+GOj4Qya1-KX)ekaD4e=pV&*mKYC z642gorJe=iXNQ9K;QW%)$tm!}ge;=UF`~E0&E%0(zY^ z0pbiXmtk)EyZ*sAZfyL>C&w$t@FJR8 zN@Ooy6u1bER~=$V>#S2`f2V`xHLs?`nNT)Zmou%=T=l+UM9KHobnpjyK;en*nSYm~ z+_91KDL=0EQwUHw_GSP6`~R{s&dOEhgkpY4NKHU1%EP*6hgZ+pHLYP!_~Hy6XMe8w zumANw_EZ1wKmYqb>C67v`>nifpZdBCrIprTPk3VI{9c@nWe1Mce`Wu{7*PKAJ%4bZ z@PDso|G_zTSrNydoa^Db>yzW(0Prl^(LTP^)JT1NX>hiN4=`Mf9AFq!`~@^R^(|*M zN%PcSRJ&{Z?D$=Tn;L1LPnpsd&EvMc9NjopOb2_uNZSeE2<;To!v8UAJIDdpLq&8x z6L+Ozzn+KiDJncBfBzfO-27zFDDda&v-7{^_~3<~Ms)-RcjcQ#E2Hiphb>V?PtYh{ z!!t6ZrF|FOf#z6nfNoJ?5ni+bD}uBiGIBKk28;-GrSQ{UcW@x^Hk$c` zE4CRt&^?pRo3u#5IJsz8(Rh|t@vEd8JypXW2X?2f&4lc9ETbgaQYhsXrQSRKiX|3UmQDdho6NNGI(6P{8y)a z#>;)hqSVJPf1uTa&loXVTW_KHt%I})C%`@`X*Rr}(t$&@a6v|Stq$J-LO#VOv%m;;rzekJ(=<-5NXn*O#{q;UMKSY4*-|b;#22q!; zIdx7)R`9_Y-@+OzQbxm((-s9M3puTZavF_S@I~Pp0KIH!-BzKM6={wo#5dG5a-kQ} zQZ>Sae|oK)IWJ8yLi!0gmyRal;10w`&lMdAVMm$u74IbVM-*Xj0Jo)uVKh)?`V5up zAbhA;0|$%LXpKx2oi$L+Z^%V@T|eo19>Z|aEgtda6?C-C{lz&m7@R?Qr<2a-G|_r7 zJ*H_iqzt&6IfDW@>{2In`W;Q?g|L||%*l2bf6RKTa`T<93G6N-K0oEVJtnXNG5ES5 z)9Ryqe$O4h6h1YZsADhqJa(R}WDZq~Vf96U+ zAZNqf&_Fq>>rw(tou4IQJtoJg^clF9*}h!cNqYGrq9e3jkJAAS;p&rHnV@4GJ;jfm zXeHAb3XwFVqxu-N&DIo%WD;-~-r_ILxhrV`KRL1dHV>qKaP%y4fCN>2$$k;L4cZ60 z3+uJXosu`WFeL-jyhd-S0NhWXdayYbH~91xNTAWQ8`hqF z)3z=&6VSqxz{F4JDR^P0ehP^4BYtEHbkwahp{yfHS4WO zW|^fK1G)8?x$~iuBwEyR*Zswt-g1i@(1n(phiY;rnWd1`#g;3#4+F>7e}H!7Tk)BN z5>5K&lBx*O9LWPBr(~#h+`qXWVkecl z-C=4DH}`NzG5W3$zhZ9s4w*f3N3;Ojf_!n~#0F8M(46smF)G|VfGgn%h4Wpygs^-& zF4)j#+xa*ktF}exaV#S?e?14OCr@&^;FoAt;CvX!PWE7(AiU^6V%sCwouU{PzO!se zt?!!v>UHg`42VZRNp!6e0m3T0q57>?5bh4QavAdl?5(A560%P_U&)4R=NLkZppeMJ-$oBWA&cdj@D- zk-v`KWtV33x*xK8SM@{MF(oBT&YncKDq6%-Q+;9{gjLJ*IvFGLx0ZoK53C`5UE9cz z&TvwWMEeypu1H!Uf1`vV|KTFs@vF2dgqqU27mh7)-t3B530hKGL* zZe`aa^h^i(--_?VlCWK}Xa}~+^bPHu@yil70@nZ`sOt`gf1Pp;ED8h$QNq7C;n~Gq zpPY_up3thYmdoCJmnq|3^)aox=<8Fnya7T%$x-vQ;=^=SlNA>~u9y)YBp}jq-`Pig z%NlV`k8}(Q-BW&iZWXBG>InuFkkLsSXn>uzq1CmnxAefz79>)+!Zh|>*ILm?^nPDa zBIn ziv*w|Enr{?^x<(X5_KF^(fi3T?cPq98g$jmc@}B+65qS<)5ZOPl`qnBNWl=Qo~5TR zy(;zzNZS)RmD*0cXzj);-*0H&!vvDOJ~ysRAmQt@HVen5$vLgXN3+R7a5Q~m%nVXY zab%Egf6gusG4Rzpl3X`tkRTLVM>$ZOntP~%$SGRgQ!r5OHOQd!EnCM7(pIp~gRYPY z-@y0yGS?9J%@f6?NF3!rA-VrNB1Tk%b~N?$?Bkr|ZxjB=gZ7MA*gW|s3ly#^BN_Rl z;CTKBlk1VH%pZ;LK;v8J&knI97h1`m-5=>9e}3gr?LH{;zG73{;}<+J?dc<%nlAT@ zlj+VERiDvakpbZ)8vc>T_;7{XGv$wr_UQLPQ1TEk0F*p{%aHM*4+_{rIXz_1^GAo{ z`6Eb~XO#4@P2MY>)0e`Po<9PhnHXO#BFG)tP;=d!rP||p(5bykhL}6rT`OHy)E-*N zf61C`ZmO9$I&_ZNJ{)4BwNU$0;-NfJ4PDli=YcOkB@kw#-$^2U?YiwsUkryRZ3JDc z?T>;NJk2nB-U#Syo~1JIUc+azM@~k9MXCzMaCdm-oxc-xKZV~4QvwJ+6gw-D79l!9wIf7hLN^d1l zhJZAmOT;-5?mrv_|F$RH8SV$Xm0Wkl)%J?$6-0ZlF!ot=@lo^}vabceTk$ZMFs^%g zeouj&J*9}aWd^{}mC!T#Z8NZYe?+Hw#R>KsL{ruK2taPvsPzCGt4F^T>^I1!Ap<4U zCo!&pOyS4Aan2<9U7wr?_S>U<&yNk@i{2ib%Y6AETEPxO&bl&wHW;a2CVvBo=5`8f z{L6jCsUaci)z*9{*bh;P1PmZ{BwF;nFIvI=ci|HTRlj#O|8C9{I80~Se;bHIMmKFA ztZVp(_(V)%+j*Z~1T+GTeK!H5a?n45)oJ$?lE#K^JOJ?w{e1<#P||M@E@&w7z+z#; z{$Y`tV)_>c#kPVATIJj9vanZoG!>He=q$T!Y=bd3v6B` zRjxOS%Ux*6BTx(;{ZPc*1g1ujDxN6pq_-#an#-6+hmx?j>e((};4WPv8N0LzhPwKe zLkn{084l-EBEGaFQ+~xQ0VpYu0QB2QJ6htCLrwB~i%qtr`0wTb4Mu`=73%QKq}7t3 z%9igjLy)T67)(J<*L)^!SNLW7y25oyPlN_~(NoLx=#r}K z3Xo#5HhKBG-t)`}5*pcH?vuJ1fLHAqM+LrjjAhVO8l{s#kFXee+>F!b^Z_;&zyX_n)uk=`M#jUSRjt`8MH8y%J zNW1A6%MzPlf5CNiWd&QgpOAgM*gKp(Cf`xIySMNC2mt_Alz&Do{=bW3z8i31O6&=t&`;T{5NmQ&#WZ0Gd2J*l zxuvM9X@N@Q<>Sy-I%Cf0<~Ua*rAh=-i|8S47+U7Ae?ov}@&L#TtgT#v^LugJaRY+a zo_}(DPlI0`ZyYy9;Nac51Re;Ab%i_&2(l{AF#K)Rk}z7JHWmF{&~kQ?As2uKv?;%z z1>!WdN;^ftrbw>x)|7csLVnQcTQH7wM}Zv`)WrZk4Pfo`6PlwgWg*`JM{`6mxNuF9D}0#0FC_tJW6sB?RYDNKO(sS)HPU3Gt0QQ!02L&zx!)XHn}8 z0^$Tb@{H#u?dcH*ZOV~tB>+(J%C#?G5-FEzmyRWMi$6`zJl;1n*=;bK?03mPMmjb->VmXsR6;Kqz*PJ27)u1z!(^ zE_qzxoj?8yXU`(BosEHukzzhQOas?ycIjMYO$eRlwbQ!7gH&*prqcADH37$~f6$@X zT>~*~a?#!d|E8lJa;UV)#R(5!*6<=EY$W)Q5)MNTuQjKG*BkMnK_| z!A!97CX^bjc=|k{DDc%hO2=!fhc&+dR2F41cu-ar9c~&f&+@gm(9)7D zJ_76)U06PMaqt@uw$n`W#yPuvzv~Z9W7rz6p4;D?bC(A5lXIT#cm2V!f6~7PqcE(t zT=imUke2YtO|fIrHM203%OwGED#FTX;1so7@&kIJy;w(9pwCx}vJ>qF?KEw94g_#( zQLC{u?-Z%pzwE&;-M}2sxJNc z_%OjOR{bu(3IE+3WMV<*Yo!>hq3UjxEh%RSaUl@NWzdvptZI5QxtzelmYSQ z4wAs}Je-843DAyVK*&E;8HN=M&b8>y2Rf&z_BTN}+jFzje@_<(ov(xvAau*Mi{mPe zYe>?HLOUTIc~Zm2EYU)+@Q^@{Qeb7pAWGo?Dkf(I>u8m3+QA}og!8h)Ny-uHx+6&| zPhtUix}aC;tgOZ?oGycRuHIiAU*Y?`E+{=On*po|&pe?ahzTD0%_)G%tlwONb3XNp zTkFqV@20nCf8p?sO`2u=T|TSLs2ITdj49_1@X%9h>*g$nB|Lr0{6TGvmdl;ToF`ww zSMLyTq$f>M^^fpaRTUOx1g7-aQA{>{puF%mNu>7K_h4w}(T$pneqWK(cfEcfYl}FN zcA@#o{!FghE{6+fMY%@fqOhWz!+r2CAxynH>2RuNf8zye>dAV(lPB|C;(Q<_U`OlW z8y|Imjy8H-F~fwEYZ$#)fNRxjeF_KOP0+qbG^^-(ue#CFCo~39jCeqiNo%q6y*Q4ok6wnc`cSJGVdeFYzIUL_i7%FFJ ze{y0|5_kQ<@y(6B>@I(E;w~S|PtNsp-+X^^f3)2>3~qNoD?*ox=<#)JcfdJKPZ;L3 zZ^Po9FTUL|j&BqI0GW4`I${ZAp%uk;$7a$ai-c%QPy1LDl{ogXc=f}%V!H)hlB}<5 zyJHOZN+?Msnxg~?=^FyQQaxLQ{uu9jw-X&WmNNTyGnk(=Ib!K#nb1PEE+M9tgfDB_4xjV)ER~#HD`Q^O|_H%qy`n zyInbq4YyLr)xr?2p&cpKf! zzo5IPkVZpt)6SW+dSakM;tB<$;}=woaGp^G$>DEN7va}7 zRpjR{QIlRmFIh7y;r%m6tL+9K8dqo_;un0V{1GvhdWRTmz2kE>)M(f8V}61&)UGDf zwc=-ks|va-Z})DDLlke6>fAhOQN(wV_SUL(Um?vHzn~1s^$6_@;w3Vp^zo8TpAT~m zW9#^Z%?T}!XJj6S$hVN-;auQWf9clCG@S1MtscVlxI($p_=Oolr7n&bOm=yn!DhGq zb2e1(p6Lm_24kr*U9~dg{F1}_)CI-Ev28$AM6dBr(LRH9TDA_lPP8j!pDvCPvAHf)lxx2E20tGXptHEuykXeWYJ=!B8sz*wIp{GS+NT!Od4XQiaRN>7>}V-^ z7g_OwEs~LQgp&MWOGMV1e?VL1m4~=ItwBYuzCEcOH{9geDSKXdjnV#O@$NB{|B&e_$_PaXC_}UqPn! zkS)5ToTFy$;RCbOd?S^hQ5i#$d!`nv+>OzY`*qCKRJ7Eu@Fa01A2YN0i&87I9mS)$ z8M0Ssv=6#Fh{IFvb;^3zXVt+|>4gtebm;aJW|bkB4~o##a_au#&@F-ru-U4|VqH=i z7AyqYub6QKTNp>^e@7qAPJt;bmd9e$r&iHiti!P(o*$-VPwAkbJt|3FFN{hOl=LG8 zQgF`OznjB0BOD?d37Wv z9-x3vOSek0Q-qEJ!W}!zydO+Yb4o-898B+Sz=+|z)#O%`fsVmN7rNRH4sbAbmg+71 zjx7aKRJa9Bc8AB`O4=UnT1(rbg1orC1H0tH9)F_|jxQ`C!X(QZ(W&ax7evCbfrCHG z;|@S`O*EI_f2CwjQG=vUd0|_IO_ncjpNz&GMxpjQQ_k(V#{FIFG!}EdkvcJ!o<64*GCXO{yrX<;aL8DnV%NDq1c?JLMujH3Iow8r< z8EYsEFtkI0GrV?eM&RDAE%n+R8Di7_G!8mr zZEcc-f0^6vp?7zn4Y5u?0Rh@2R!WivFS72v_~arp;)^$QewC7XSOr#tB$@Q%MD8W^ z80Pijz=>2=*?q{Z%Kzg-ZpH81huk~-F*Sn?k*Xg}Y6_upWOG>lawqad=TOM%J~%EQ zTb~3rM|}+X($Ba@99^kG&dv}ehf2HU|f<8VZkBC2Z=jT6u9{aIGhvPn^ zA0+QS`f>fJ%-{N^(k7Hj$=p ze|1man@0-zO?R)%cuKvx>spNk|L^8ZO}JaG#o9#cw$g@sML$-Oo#7p7Tq`POO2VXc zMR>`zUy+U%nmK!J$cC@1CH)N-a;pN`UC3Cf!z(_XA4b&9PA_;Qctro69D=s;_d8!; zsja{7CD@uJRAJ+oFc^*Nfp%c1t!4s@e=Gu#1zdCAgrYRmS9?#RO*H|+7Z2#)TL`;2 zm;)T%!O)D+z@_hucZmKWvTd5d`Mo%%orQjj^4UK)BjL%v#Je4D9I0M%h!)&PugE&cFI>{rFiudds;z!#3i0g2_jI9r{5jlJ6WVkI;uqLPap>+qzrjH}ijiw`k}&a8$m=ONSn;H@ zm@-9cZY|x+rL2(>jgczEJc%pRQDRG3G#tAZE!1q^!1%&ls{Tot;KN-@CFrh2al(A>izd2E9YiTU~!d$+1AlO@{AuJx?fJau2bn2vafW~5ZmvJCOf3{B+-F3&-9n&Yc zRDIW^2yhsRzRIV&INpDV1USDRYC;I#bR1gre= zEISU6X84n1>Z9c+e}@e=01}tJ$sz|wT?aJ?I@>rn!^m`5(~A+QJnKFC`F3%jvNkbg zpagY?$Q5Ad#YK8+IM*e>`-rDRd{?bHzk&cKU2ktwnu-Ki&!C?Mfi*R`hZZ*Wd?6#n z4|UM4;TlOL$UitCX%^YcSw_`rss53Xx-)>4oMg!bOq`_ee+3_y>ES}D1tha8bU^l{ z9cr$V^d~a1lUz{r_eqyhrPW5)7FrZH4875U8YOq$0BpJftGC zQ`BcY?|7o#YDvH9@9PqYJmo8fNaUf{kx0DgGB1(HuA4(7^3lzbfXo#7C+8A@?2f=Y z=Msvn;y@RQtg>$xiaZ&JIzpk8mJ3DRa?pK>Hv`p2e<+G-0sRVv$#1%shftJs1dA8! zatVdkVuesVB;eNp$CGx9g<12`w9>)(y*W*2)bbo0`nbp<2x+vq^H>7y+TBYe8Z8_s zE3Fhq8B!*+S6gKmP@I!A1u;2HWm)0m;({f;DlA>bL@!-!DA1I3U7^wt{D8ew&62gO zBasZUf7#s+$TlI!;W}iJW8AL7)-hzqRS1r|3gGx=8vu5-$U<+z*w!2tS&^qooHcAI z7GFG{Ed)OGH~|{K!d-|1>9BJPZHg_g{vlr#KMGUmxeCq0>h35^QE5 z4?P^K>?!(~9hN5AHIwlB!C}@J^P}x5)}$cPv7&0cw^Smb+yzu1Uw#+ixP`ax(tWNN zdIr;H+qV~ID60%*=U7vaCs%^6)xmnf_m2|v_~N1se-*gTPd(|JjfU(u4w6g)3&)<% zUWI5@)PKhT9=^3NAtreL)|@PnK4(hKwX^+4PNH?`ic=%CbcbVtD@2`WQjJdcXg2lu zo$+oeNr)?oJVhEaz*FlPBP!B-mRTl8P3*f{D{0$4C0e!n>6N`W^SL1q<1#wB=Pyoj z32o_Ht_jl3O*Jx!nD<6+8I=|le!~M^9F4lJWq;LL*c`39*2DGS6+^pI_83{MmU}~9 zrA8=z56FaYDWr4g>BzND@uNugP4|1DatZ~IOelAyTaAW$y_5*o!BGmf*za~Gq`=_? z>DvBmK`CCkvh#qCa|NMa@e+9CrNC9-RV50EP_;z?t*k;qr>mQh;A}5K>p9%0C+C`Z z-GB8Lr~7iWuwwn2bMD#-9KLhzr#C5xPYz=B4-TP>KsVYtq{MrZBUn7o*0rxj3cPoX z8l}(zq$^CTPlP;xtSg3$6bb=)*9uFXl;=6iH0Vo{dm<2+{L&vTfKZz6E4Keamwahz zCD}UL5KrX=@e+`h9kQ_XQakpL!t8#EwSS6J5;yW2ZA}@7O6KEi9nqdKL_*#(Re zuu17Zy?&e(98MNL(8S_*($9V0*FZnyCkL&5zxRXYcYQKC`xk&1mX5nyh!}P_j^35S zwJ#~}@ZFaH1~qT-ha_# zNjnikGO9--8$!Us9-QBcKHbBmwAoB||Vp$iFiga!p}obL(@G@d)4QDj9NdGO-Qq zE!~mRB(-^EL%=B`vLXD=7TFL^=HdZX6UjbQ%O~pfc$pTBFEgs<5L1wisZPl%oOJ zJcvr+QF8#C2H?T5)>%Hmi)Jl!cnvQOjubr?@OWdWc$2=?!uJzMkPc0OrNJY7rzQC$QCVkG}`X z71_aKJUBEr-}y>{;G6oL=fhXma$V7rZkNCeJx$m}-9wN$EPudK>&2PMAiZ;90~dGw#nDC(FQ4Y$oVZJ)`pLPTuDkx?XlrwSR^OPl|gjBWwRtXfQ{PEZ8F1G;j`n=(#&#!;Bn>H>mUxJ0tE6!Kp5{ zOY?hDb(nuo)k%(ZeD%4a#cu-nXf*9mlk2+CB7==b^z#yvw2Om$nHb;X@Ptpv=2mO2 zsNpJ-_AQO3K;^}0R7Qf@Q&67RiXOog>>z!A1?2;bPk%vq&{JD-L19hLAZ6l^&8%Qu z4y42EyJ2A=o{T9jsH%+6h|U_hWk<|(7uv7ru|tRU5%OLHpmf@_4K}a8TrP#2!8gdyh3=3d-|CP8^Bwq7Q8{bCmT=4Zlag z9Mdsr=zko1&oid7q{B0&m3A@c9sqwcs>K}P@XegC}{xX%?is(hxFizhNQn1ADn0dutb72SvhI~*SS`CVN48xsKQ z?RacjfnDHh`Bq|EaikS)=29Ad>AaBj*t-yGt~~^EX=vh#=INv*#5}D8;p3HWtrS@A zRaEaLSlyMw(~?ni&RKt619x$XXV!+E>E&qx>GOpke|qKWleWD?nh>bJCauhd=Y_kd-s zoVy|&FX+-WlwWDoOUr$?#)yP9bjW>d-DkJU!R!JfO!`eiC^jv(=3=?3HY#0MgC)G+ zL9(bsSw8H_vYW!EjtSSMFzDsr#1{v<;eRD5mt(bZY>k9;YEss&?bQ;{a?<%4DkVdd z!#~0UTLkB$ODRUl2cBvGEt-wyVWXM`Qo_|ow z2g-EcSEwwO39}f_NSJZWo`e};H5L2X<4G$@7@Qf3N9waV?xy}MoxVJcJ@k|QQX`bY z>>-z0K3VZ&{^pP(8sCnpsrW|?tuZ9pIF-jnPCrTQs)Q6nEUp9;fF{&r(sCi(lhIQv zR^biqgSdtrV3SG#hi2Rsfly5;hkw%NX2@X;eKQ1nO6_Hc4!GHua5h8AWBO)j6PzGy zfWQMUD`76hZAbPvek)$HyIGLhr~1gg7uxcVd@mGq?1h38I}IHM_Q|<+EV}D2PV9WP zHw}Dp{4O8VPtN`Hq4thXPV9w(qrH&P@<*{3+CwgI?1i>^EqpJur{d1SPk)_JcI!x= zGiqM5u86wEL3(-4N{?74`l#uCn9J>^F3jcLQ84aTEdK>TZ5Hj7roh(Hw=P)x`;q|$ zLQcy1DBlX5;q|T1n_R1`mhcnZ;DT6H%hn%bG}f=e_H@P|jE4J9WOT1*B_nT^uR`!U z{UoDNp-CDys0=Z#=-d}|wSNF8k!eUEUtVM0$v{JiDJuKL!PbdGR~|2QI1>S`UlxdC zLjK^uzxuuJ`s8#SeQxiL&;NrHclnV1!8uR&yFNGrn*PCoo(8!{7SZs2Mv2<`K^Kgt zf`I106-~|H_HtzJo?1kSF-}LZwyMI0if7$HvhV?KV8dEhL;|zR8-G-}%&T265PRX@ zw%(o{FsO^}q*al>d9-WiaU-DP*`0LL9o28ZsDknPF3ZdVC&kaXSNKN`-*~ItU?+xL zjRT~R#FMX#M#Vx>UEA;qh2`pgaM3Pb%!3uYOPw{Me^^2chb*p;3K1ClLS`5w3&xht zfizB5-7-P}1FLOqRe#Zs;=Ae+tgeX{)9 zXsqX%qiUM~U&hgR#3L{`a6?FIIPWWLz!BD&Y#E3N(W#KhR})d&H+)i(C=J$>^Wi@q zLZJF>^;L0jRKQ*C(Q*bci^NFMOgIfRtAu_eRvKm^Qg@yA#(x@b{X|kl_tG3~6?ltO zTcy`{n*`&E6{-^DUpnCt+E_*7JYck|2m)2#N4IXBr*^cj6=U~)U(vW|yUHbr2ir&T zw)#|KU2--`Xc5$M1zjB^C^!reZ#C%|z4Qp!w$Yg<0|zoO^hW&iP;t1*@5UhXF1`W7 zGAN1o%1PQw-+vMG;(;0}ALlDf4XmoF2ZQf1*V9Y-oUyz-=P*!ln_Z@aI+a;i5X<`h9tq*UX`bI%70tEiZcTkB{i*Ywg?%$xZc0m zEFdea@~*_B{$v_+tKr>f6x-?mK+!%b`@Uj_K`uV>x+6#F|8kY0*=ZdKSXEdpw-dskP{D7>smhOIseNz*mPtGg;w$6=`lcq_zmjm3&V^WBb2LO}Xt`wEx(#0sElK`E{nz86u$O`ba^XBwWzUa$QQ3uX zuHb1=-Oo3#HT)ITrPfNLLwG~9Yv?Dp%76ctoGOk=Fo3}BJq7*AQJ>~0#8Ij>lm=Y< z+O)OGuGwmjO(i6t5@LY^Q{5TYwaVD@Z%a;}-?&0wNr}|$`iKSgc68rY?8+`uWw*}Y zNjgRjP2$k?2n-DeDliU$(xznfMvElYZB>kIG=}NGsdC7Mrg2M;0|cnj;*RF9M1M9K z(({Fb(_|=;=p?(~ojp~mwd)-;$%9B0HbLCGjhKKUZNLcpnPz!tK`*JBV0ec>4iBtn zJ|fFEk@|4aa)qW27i?@QUoy)J$kFlAW%vPoR$Hvmr4E>*YcxgW`b$nJPmYTG2EsmQ zi*VgMNN4wFH2fr4s=mT74t3Di@V_f7@7AG)@sV+;@)6xQyk^2*h>vr6 zski;+6#ZnDO=VY?N|eTohP4)(g6c1IZ!7AM^&bJWj6zyt@F$1d-=PjSHGfAK?UT!G z#`tz+QT26LXa6J?|+go)A< z8NR|qX_*f5GEsVF#LGnK6@G6exzkV%6Qw1J^m5-xJ(JzRL}_rpOq2(fi83%@0NhPh zkl&bCop|auMh3}^cA|e`&Qn~{KQQt3cOq2&xk-focEi_SX)_>SJCQ8R#CQ5^K znP(=s$k?ESqlPdtnkYSg=rU0n%$vMuyzXB&J}CbP`6!6s)J1MZnnP1i>JlxKk=hg% zN|ln9h0@@0Stz%vQ^`UZiz*A{o0LJ_b~5wpI1VYJakK&xr!SuRjX6&db55jw=bXs+ zH1?H9)6_QpHLpcuzl4B9d^%YeITHU|cJ#Q6m#Ch8qCYFTO)GxIxASXrGy^&RKV z*O$hKe#{O1TC4MpSWV6%R-3bY?*~_R=l2c?!|P6=VC+h2vww;&Et)6qY-}MT`gn)O zcj=C2(dGDi3Ud|KUUG2!{#dkva|ZVBxMmpe*&SiP70xe!M$bEn5+P@j#*ZT1QabbX z1!M>MKK!kP`vEK74kc}^FxIbt$;bLOR1JRb0PbEU=wD}Y(dc*1q%g-k5Ez;VSfMq{ z1FTDRnFkicrhhOGtR3Q=t6@u~jxZ0bEf0j>VCLz*_kJ;I?3f4EqMS)-dLb5$o4*)$ z9mFA5lm3n?S@Pdv4YDHut`??6r|`ve3|R*jE5jV?fH2oeGL`%{=5RMX^#?}Q0R>;k zI>1ntunw$FVK=M;Y%m(ufi-UyfVG-0mY`TBjo68nT7OyxSfS~%4(uEzM;}1Sl6fFt z*$3XFhiJIWtQi>yE=*=poo;x~@?yOP83;V9YaRo^1E_X5S;23<=jakHa&Qf9p2N(!^7GVmE;+~Wh z5Sfw~OMm7Ila8y?$Fh`h&hFM{4wuZDIcw})dbTbu%p6 zo8TgU5H-xZo*DSSo!7$#SfnFh!_CVeRoq9EFhP6#Un!wQu&8&*VJ`3(!;)`3!*jA* ze%(>|1UdPL%(4Y>%gv+zM`pQrxH69`;XQA-DnvIh4Y?kf1)AU_M-DfcA(@|&^M948 zziEnUoT6xMZhb_X7nzns>6XdWMW%l`$$jZM#mvsd9}*$f*R|bKq0E6<3eUSRo&-}` zh^CwHpcqHWw7T&MzahyK`IskWAzI2m^aq#RU!kq-t+xxI?N`xvcv3Zv^NG^Hv7w9TSbq;dq!V?X9HJU%)F!Nyo zQ&8tUctA!dY~B)4*&Ejjgix$iMG;6`86TAi@S8)1E#*8XLixI7Safp9bd>uNS5D1y zV&A*+FNz>IJzC1se?UO5>VK>mVZM<~ge^&B^x^C{_?6aUcJfUXEd(54OI8OK$m}Zs ziy$A)8aeuNW8_c}COse{%TK>k+&gxH%_9y0$ zdG(TqzY*t|!VsrFFz4NT=nuq?pY;XGWGEffQ{@96`dD~`25zv*^nZalA*5D@ECA+y z)z=6VXEm^3do$t8^%RffomQE_Hz%ZrAgQTDrWcVx>9wY;Jgg9G+Bwbm=s+|ugq9^N zAr_wv9NnCcFKNu&G+R)|2w98#!f7Q!bD7g!kU=MD%CxBx zMZ1n}8=o3~n!yB+B7e`fFgU)guvsq^@1SB_04{!4XL`ICTj61C5yt{C-1_iWON_V5NfN}C@9 zvJ2}DJ)`pXrI~196yS30=+WX(UgEa-_(#eZRaw(w%d^|DY<~*hQ;mxqF@>zuQsfga zH$0+P#7*np*l(GIas~AZv#{f=Q8G3zY;(v_bm2wSDC^F|@-IE2Y5GN&Axx2!_2&|Q zb|%t}oBk|hr)AHtvBtE8u}JwjlzvJ+?~lSMAN39s{O`tC!jFTpa3XoPgiy!QDx-vC z>U!HaQ!UJqpMMJBW4=05jmBc4evV?~fek8%bwe<$o` zAM{E6#(3l#F`{Izk{~x0t0eUqj&s$NSrlg$D+29{AiCmDPB|bMn|IW4$obmhk-`?N zN&qRuNdHYnRMVn*!9}@ffvF&Z2Zqap>G*x~tTd(LcYj$_8xA=8#QJ@{@!2!Le_W<~ z4`4FeY$#8l0(!H>hcmf;He~Jf#tVyOv{c)0Mn-Cv>f)I18`Gsj$FB^?PF>-#RQ5Z| zY0sg|zD^^^91ur4O>GbC>7Sj-B492~@7IXI+V6VDK#xBcjk*rQ;PMR~pUf}8FTBnh zsc5||2!A5YBeevvX4PxZQ@=5n|8YSeuhXPzFC$A#GtCawKTG%6hu@g>aQ2nV|N1|l z=kjyN|M;P5KE#6dJr5aa!1MP1=l|lN|81}Ovj6_i|7(wXFXEOw<<;^FO>a0D?}(1F zV>Nl;)boRrZA=1#iTwZZOE$aoW&io?SQeq0a)09=7YGod!A8~<@n!}Z_0>`7OEQdI zeHDKH2S)sy!QMY+;AGX2>8D-xt_F;ETw$t!!RFUAIQcIm<^ZhWDu^MxmjfJr4x4DU z=>8}C4-INoyGC@x-uN%Du~Fw6qE*x!z}Us@U2!;TPle;@H+V+3C1Bjh@y-5AZ-jmX zm47;_iG2ZqzcUa>P^Hhy67pgK^s%O2I z6~Ny5YM{z>9ZQWL2B~_$F=P9^2Vjc_Gk+NGSPVH_xB(1dgkW3I9ftMKyBygRX3{HI zW7RP0B7m9SU@UE58k;#G>a9+b#_&R$v9WDUX!-A>!w+v-199#UXF^z;J?{~SS2Wl^ z$>Od&5BB#ez2jt!WN*al4e6(JoCFT#!VJPDT_CxE}<^laA(>dS$?L&iW{=wdEx!7vUYEmcWG1J>7u;iK1oDUdzVNSbCg2}|@qd|LU68(N zq)@JU7Ps)=qD3WjpOOIg^?lO(bvmQEC{` z$z;)Y3k8q-OHh(ACDwCQg%BdSnrY7=0_w|J!I)DX>ux7xdmiGvhK&adR zgOjGPw9u8m;YqSqxQlScUh2D-wo;c31cgBO9y+Fw{a3C;;fTuJuVn6(O|NZQV67<7 zzhFmoxDUWP2dtIZ_lQaEtzSUlT7O5=el7lH!Tz?&X?MVzoE7J2C4Vjq&&5LRP+g)6 zO+$Qp*b5e@mBfPJGPqb$bn$vKshbErKK15WG#G%)IxuSQ<_j_4{I!LVOtWLD z5|8KN;#M9{@9Q-iPtG~6?ZXG!#C}Z<;P&D99u46N88%9x84SjWLZvsX4KE>Gdb84C z0F|5*c`(M5<`A9w_J5fqfk}4`C_DLkP-Q77*^dhI zSnCB?dKUA2xg?_?G(I(6DTUeMovK@jK3n5c?*T`X^C-x~*UiYZU8FbfJCrsxgUaO1 zO*O7w8Sc9U=KIFD>PJi=26>r_M46ZZilA3+z5nXu(j$e?^Hk6EjqiMEgSJ#H-y?W^ zs{vW!&05$gYk#p_N`>CJP(9iAZr$Ku2;jk}l^~a9CMMUWxTP4|n*26$&Z*74n6(mN z53^R<`x(Kzc~<7#FUpv3Iz&paUqUQcP$*J*+;PCC+t-Wkzp>>xhXnB-G58VeISj~t z!*{^tTJyN%8P-)!1LD(^(fUYKjdaa=~*MIgLt*kHIrP*12{(pGMkJoRYvq#~2nY)5?hzrtvH!X?MefJ=*4BdCLabJ@U&kYH%s9+j1nb zDURry#K0z5eNj|V8^lh%f!eR<=uuEm>Qt8K+Vli4JO^F$2kqBj7@Z|f+e;z+K>Ia# z(x`}bMLb!d{n}@g9XuW%N}lgkJ8ZVpmw%_0R)=75@w+bd{B>=2Bq0Qb`tObK8-v9* zr~n(-cssJI(~zzS1$eXQwG?2R!=xt3OpGBB77DPznYW|!sBv!3QT1JgzbO(K*Y#@3 zz`1VyfqbjPSUH|e&odoIno+x{P2Ca-@Rs=%3b1X~t^Ya&Eg`;dg!EtEU+SibM1Lsl z*7tv;5SZ%Fe(#&%4@y{+mk^9}OH_k8Sd3_&d_UhPxJdi;?fa7QYtWVm=NmD|d8Pe& zHd+1(?RWFiruJ)Kr{q(Z@HE4+h=jEnc0*S(f+jj8`ifN4PB5&qaNSf?}7<4F%YD986>H7s&k< z0^09k0F|xgjF5wd zck!&e+q3KA5oI`D3cmz|)~!@jcv?0_hQ~xrQMG=gTZ>DG5=KhGR2r6 z;wZ}UChPCWhZGYjYQYEg6&mM-aaOcwCG#tMRT{;^s}J)>Yr&UIhZroZQ@tpqOwmuvF7DA@$sFu z8x5%Be>AKtdMv}RVkwzb9MJ~Q9t^|E0Y~Ik3Ihexo|k>0>kSfKx+tR4y@w6P{SmkP zItD+=0c%kz(};aUkAI1E<2|3X_Jgb~k}s+2xM3yWFs!@@UrE0FE*e(QqAiDIg*8*d zvVuCkw`FB58a!!PS;uiXZ%RFaorpU|v>4GG{JPC5tA_0zS;2+Z1kEaORPRLO>#Go& zm&hS2!oCx2o=69kgk2B=oD{y0)$t;gaKs@K@ujUiE4%qb(0}CSs>BG?Z3voX$>@q0 zGk;=)zJu)BPW2dpz}vb>QEMX)4$L`y{RhaGLiDy*$n@%}1p9nf1U_)uvb1}S^gc|( znaJKW_LI##OOe7pjK#~rr9B=#p}?xYW5i-|gvTwLVXepQbiqyCr7jH89m|K&BAX91 zu;)BAJ_F%73V$hccRJc5B=Nbf`><{e?S_5> zBj~fhFJQmrEa8S%1y^`TexieEmPqQAUf20R3)PQ_3aL>mIvA<23F)f|!Mg?{5x5+$ zFw;D9ET@N*%AzkuhsJ?7E8McYX~g>Q0XGiBYGxTQQM9>E9$!tU&ljwriPp157DUd@ zhda#(WPjv)(y_vFvz!Y;bbYG1i5fZgPJ6bW!i4`%0LW6Ox@4ss!BmTfQU+nM)Je+< z^Q`RD1K~-QRJ0Jf4Ph|)?g@#$941p|bp4ueORsz?t zYN`s~U_xC9rx0@VN~lZ~S@hSPfDGz#{W~wH7lN9NPaIy(--h&J-W^Zigfvl%xicZoyu$G)ki8f3;2u3 zYYIP&6M0QM$fV-3sTBmBmld{NjF{o~oXp3Z{YB?xguCA9yqt*D@`8iXg*l{DTCn4v zm>|h`>Nh4xyUfr26A{m7P5;20ck`j2h<`)q2jO&HPQY^~cq{e599_#-SuwOuF=dfr zg3!x4krx{yLSdeOx59M!jvXp@(cYfGUf=t2hh{jYkN1eJUl<^-B5{xjcsVpzpLfmt za8f#pa$b}kG#r%vrfN_+d+%1%MELKm5tNE%Y0LXNB<{VaMV=iu*7$bT)p z=-l%B?vsF2!g#v&utxCrYx~S=0=3bZ!1)CnRUHDu+)xTAo@A$=hg(>ws9#Vy=hvB@ z#3-Nrk+Sr0@n?R#9@p}m-{<;i0*wloF~@rEM~!G{r@vNdtz`~REBe6B{U{8= z&D^m}YkOA0T>#vV)viC+89x51)PLRmdO@W$s~J>A9g*0YF%fo^44pKonfL=WLh`&g(yZZM;(z!o69HRq zmI&BDdq_quY4a&jU*s{5Cl@0qqecSQ7R#nqfp`g*_xa9H~?1bou{go&*hBQE5hnAp9*EHp_Qd?_;Fdc=#6<9gYG z%q#>46(8FO53UURQIAfy7Js|Q4%m+;+d-RQO=2i}{~qDcw{M6*xH_|n zJ2N8{?aC@2@~up!MK)@DL&7WGc>4MX|NF)KqC@M6$25Mp5WZ8p3#ROEx_)39CFiUTbaVgIidO!qZG~mO zD)*8tMz1li=^**Ap?^|rMY-qyZVbN1V7GEvHGF_Rm)exx)OgG*>Sh@1GqDon0Db#v zQCNIqU~Yv^ay?gx-((f`G=rU{YDUKFgDJ3K)r0`V2EwlBkGOPy1NH~VgEgYk{9GA{ zU+HhJi2*#GNXeHPZ#CG-w!-zhDubvYX8IGv6i*fNJjsqr`+qm4RdfnGPyK;$`v1mU z{OYwkxnnM_7cvdsX74P?>ITI3R$@Fgfu*$#_Rw`MSW}&STjTZ`&|o}mD12eN<}!XO zu?OgzTpN=Yf=;&R;vyt%bA6MRFer0xuy1rjLXl0%m8O=iX*#tvCMz_CIW%}`?Ymqh z1k9E2qa7m{Nq^B$X3W;kb#D7bZT#jdnl_#TOkX(LlN9?Nq~P+lHjJ304Fk!0Pqn&9 zOA)Hh(&&cq;P%GwfeJ!5cm?R6h3TIGs0-z9ELgg(L)f(Zh38Zo%wz0xdhZTG)jJF{ z&F#4VA(zap2f%y-Gl*d_S{1ke=Nyaq51FSfPc_7X8h`BfWy?rarH0;=Yt$LWt-qtV zrN4kMNG2Sl}G3S`-sXs8T$$Vojl}Fig zL3`tJ6l8i@5@o?}Fa^QVDES`$JVxE%CB8f(2h8<+k3fFB!`cx|Gg3plUa^@0$PvAN zon5cmfq$C{a7)0+^59%x`wuyQ&clKEzA+RIucv-v&Qru-r7$(TogYb^F_lVcKqd%!z-OsmLgWT84g>W`U z1XiQ4XDO{QwIGvn&A2<&Idj5)TT#Qs;L30A`F|_=V*15~t8a~1JAVZkePzD->iAd> zK=bjW*@|8xmaMX_yLO?T^>rIBu*EZik>WM;Bb=ESxFWGiU+qM_P!7%(+*A7kZc1xlVz9EL7tZ`lB$*rB^cCRI3wwki1-B{R(c!3H5H#U3Im3JYO1_XreM=M-i0C&Q;4b8U7A1LIerD8zhUFKO=3%S$= zQ+TsE@%CG!C-Gx^6!S7#iFK140DGE~N4YLsqmK8LyjTDsMo_J9?4r49T*>O?irTEs zSu^$`lhb>-b1Smi^M`h%SVJJ4KTd&xT3r$Nx zQixRxDMvA2B1R>Cfb}a{4Z2++xM@o_f5CGt*&^I;5E2!geGOFwytk}YcLUZ1%PV82 z*!9E_i)7Qv1=k%-PhXtsW=x)SezPvmU*-rLa!|SY?K(`h;$LDl;)0Jz1Sw=3i+?bh zDK{RUb7tu}tH`YIBT3k=w#E4HIwZ9&+{7;S@Ki&$0F@rNF~a7?pt~=^!kqVz7B_iO z4l1eS$u4|9^G+1^2I+ktm|s5DJz|_+=zurh$#M+~>Fu?~H2~hjFY#-@3-8KRM{vC?)MP!YQW_?xZY|Gn3w!%~IX>zU8E+g5=qwBm*|r3F>vs_b+;8 zDr<+ms(Rmhr*YOo0OtF~3^s=Cr0I_671q#pB4hd-6NqdJ21a0g8Z$$Y}7LYoH-Al89q5gMc zHaDGyG?tV!I8o3lRWt7r2n+USQUF2rDM>ZNI}vvbp#H%smbQ3XvRW)B74-gfGG;4! zX2x+{(Z2fiRm^5(RezPREgcL5ezmKk*;0KjkJ-{2NAkp=|KYiFGeEIPCNotr4v5B^ zB19a8+JZrGEo%$C2fZe1C{9k=c9>tJ&%TdmK&D*4{3-r>%$A;%DXwUfC?rLiEe$R_ zpd*qr9TP@MOOH(ZPt3XM^wb|1nJxX@w0@xVW(x7xV;So)N`JQa-X(c>V?p|Tu(V+3 zVAV$1RU<~|*|r&*lHsvi+CIx*& zeeLUr)N-p#{sc?D#Ry)AH2Zn}DAlqV`!0=0`_2KTm7U#@)$&k+td^eCN=)K@)%#W^ z6&|zYhJ`>@%YRK1LtIL=&^Hj*R;8IaMlcnNW=n&&4q>!s9r7>quWRwy3jDo2>;k!i zXJz&V-XVyc285XyTd%gLlfrOmuyjyHLAMo_M!!EBz5Xyv8mM=&Gb!V6^>;Ekp2OBd zRAjC|>tUz#8{7=CRQ5eBr7{01?3M>3t;||tuJtG8oPSe2^#?|FOT$P*_v@)K(U*2h z`_hmCiZA&e0<3~cy?a3#*OV#FL+rbcrN_-u?YU42C1iG;ZAZ-0;&9$h}A>u2oDyjf)ChuJ5!T5d+ zq<>90ya78GvS8DA0+?l->>C)rNM9IdsAWYO`x1<4_y(1w+Xvmtum%GoHXi`jLd;xI z{lkq0AlHBq12(|HDT%F7-YttChQ59VqaFZRdWoVKgT?3yM5jSlpotL!@FZU>OAkz$ zgT4;EK#I7*7%o>KyS{dsPk$leE(`klg?|Bk;TBl!whFL;=;(p@zA>~DC=*q7f=QzR zjRcmMdK(EWFPUgMqo`RZyUj?kbnDS zBv>UBCmIP>jw{uT1S>HZgBPfM>4C8{*Pku)=IdApW)CrxGW}$IQCSEkn7S+k0>eTe z1T6%twHg)zG!3~e1S-neuD!kW|Ea4hu2)O-yhl8X(&d~OShV)G5Uduy_RjRibz2C6 z$iqVLCVS=|(`7{i!D3}r83@eUO@C#^#k3`PGL%|XB$R$84W?K zi)49k<0iXZZ*AS5U@BLPC}AM?OsOn7%^ZKc&wlf*9VP}f!hXuo#P9 zD~~)wMXQ%Z@+anp;t0Cs&R}z;-tY{n5e6;k7mJiOBwBfQR}uf#S4a>?Mpp?g?WC=B zI>c`)x$hA@l!L{+$x;R)%W={E;B{g443nHqO1s^{fE<{Freyxe?msbdmm*bie0be7 z&T2_G%%TYSA-zbG3NDJWcYmclBWQoqAQ~gko(+n;FWurSS1e{Gukm|iq4E?WiUK;D zwY9r`80~2~3y^pzM_y%NWuE{szMZ>}ee=%HC!9;=`N%sq$zXX023AS_1)Ty`=Eg;a zlO?6U0+!{iXmck8SP+mx6p-v5`($@19$-!ET5(@&2^x(b#jQT~R)4UAxVPR6g4x_t z*6hrDZYuR~nME&My3Ek_HOIx}sd7}E-DT!rJ{%$Xzy{Co{KK~I5hdbZG83Pdc)}Mm z9=W=nzNo9Re#Q-+&a^3f>7@F782B4O8R~lGH)bj*H9_0>3v-^Da)ms&vP|SpGK@t|Xq70& zX1$k-44rjvOPFKEyhrG|Dd@BC zpk9Z}$3T2vhzRnJXMSVOLnSU(k8O&dk!kAuZV*Gh;xF1Czkl+r=B>Ax;2=68u?yqP zA!vK%-O~Ap&YoctXOBc#)&0b=W#)Az^S&aQ&;oN^Kagj{bE+BP2p5!3A~XF@lMHyO zFHvANb9#q7+t&+ZTsc#~9UC*m1$sU~SM2%77}3Ho2EEUaIbe}bTO|fTT>rZl)q5YEnA`oWVwDJueeTk z^p!04EHVQWI7xwLTZrg>=JYg-H~4*Sv)Y&M@cjA(QBrs($Nde55tMSg?HSZK>irAH zo4G{3Ta);@*10qL^0gC4#G+T=I*oPnSK~MaW{H1a7=O_iSkv^9?@UCvH*T# zE85EaU2H-pZJy;b>Wckt98u@Bdlq@8+aY=+4pKl;7TirCb;RFEArB_kdqfYLuy1&6 zVKHPmfZC{?WM*4d#^|#Bajjtz?9n?j_xD=nh^hR6IlL`?=QpN6J#!4j$;rt|$VHRN zj=+poRewN+UJ)0Zu^#`wRrA@y-k{)sFY+>V3H8aJyo`CMRUZ%qx-s5uYbs?x{L08(^8h>P zBcz1M?q_XZ>(c3zI_#v&t)~&29yR_#7^GAtBicWS`&<}Ig3beq^1X%= zkgP;RCI%Pj4FwsIy+!oJ(nn;<>Y0uFq)9h^wyyor;uuiwLL1_= z1%|R#2`*4iWVxDN(!Aw!9yIi^1AABOEz@xOqSI-8$Sa|1ETuQ9oG5bnv2gqW=YOsv zg(#3>iz~x-7h8;G!Y&;|0>+R!BELu-XfZv)=sNP(`$k~;cnzjc^?R~&n$+(}ijev} zW#A6=d$NKa)$jQ-BqKNbbx91dPYxI>k{|rYjv3KsOfVi1%xN;izoUS&OeSx0{*6GNJ7e}IGR&ns4PBkKCUBA?N#h2)ql;^;&=Ue zT!j`lQt^`6FX-GFm%R>xjJ*7X3Df6!#q41x^hwN49>Pfm9I8`bs59ypkfKe~YG|99 zfb>yKP?wHA;ap4l$QaRry0Hjcrt=Ohq|YE$nbUjk16Q?|_s+CnlTL@LDbmZ8$w*uY zb#tLUlQf!jE35HEarr#oJb%_T!m7%y>+#9D4(InL|)q6oEFtSd$V7l!n+(ZWbB_9QTYQ#T)SVhZuRxQHd#tYB-bsM}+u$Die)Ig|ISrO*$ z*?+(mIvNZcIotr|xO~7)yC)8j!TaO~`2*u~AG2&x*7|{E@CrK=1dp8Pr-N_b;Rv;~ zU&tx6I`@${rX9@pjiHake`ONWD}GfsQQhIH#tHvcUe1FKfhx|)lwpUxbUGEVk zvX-#6Is8rT-G4CYxQm>WcBI(k7fk>Dh1t;cH`fLZe!PQxxrrncd#DWnl0a?09+#oT zKI3&+Wm)HA0IM)lL$Y<_34|rbw?n#N`8uQx@BT&V^zQ#8e&3uG3ah*KfgzXiO;0Lr z;O>2&=Iz~ggrZ(o!FGRL1s6c6#{x%Ej{L)nrBNp4)ftIJr7tt8Gm3SWYc7?cUYU_zn*&G6j6_9ZZ6Ic89;=p|p0h z&BK>Af@mfpUO~BnSorKLf(*bIl%+^7kH2` z5gqK3Pj5DAddSwvcrTTsv$`O>hfrhr*+kt-`T7FDZj2JyafF3zlUcfN!dg3RzN0`A zdseY=7USz$WzW$3xes_pb=tRP958jFe3pNGH%cQuY?id{3vZlGv=24?QR@pR*^S{7 zx2L@WYWsl~Z3n_oOg8f7CGSRh^Al9Ss=oM~go^L*qPv=7XJtQfgx$gE48xxs@Nen- zxu|3?Si~L(>nk1|HDmv5H)i7qcQwTB4D=K&=`dR#V39j~h1(CR2KnFx&qAjG zcwuY6oB_L^_o4Q3jj+wE{Dp??UYvgcjUtxY+c*a&pd3?l%kt|~fQNNb?edUNTj1h# z+#aRk;WOUOvY4@`tQhnhCl593f%(2MF0mTMQbfu6Z3#`KO=9B@m^9hJ!k^al(|2ik z4q~w;q5JUMV%ZyW-rTU5Q?gtmYytR+(i~2W+z^{!(I1V{zHLLPuO$m`%j17>FrJwG ze3y4Ot$$a z=Wuy7SX(?^ft!9ZODoBgFiQ6kTNG+~d`+KQBJm9x^ZI*+@U7=73Obz^Sk0`}&J^61 z!UxP06eO~f4-Zi57QCCI>Gxazu{&3N241s`;0F}N=>W0)K)19Oqv8WU=g5;vUi z4GCKW?SY4gQ@I9A@i{ju7);4_u-A!|k<=ApUukKyxY6ApTddfqZ!&)t*Pak2mePH- z>rYH<-1(i~m<5I2-c3os1DAaEmA_(!z-GDQR4!07yM_D#qPHa+mn_lEC!7Pe9Wc9D zzDA%RgZ}`)*?GD>(W~^TNw`zZ;60`M8JSp*U25hei_WXTMmz(C^bfFY%P*WqLb3-e z>WY|Wm(7`++BL z){?~1IX@4(G9OWXou9x!f_Y(32-=d{z&zZy;BevM!pUnEJDpsci#evqj4X2H0dW;V zy0iy^FkQYhcr<=~aS(Q~SYGR<4noKBngjA$Jo8W1elM^0ue|uz)aAhW-yaFu? zwZ-1hmphOE;|hNZ?i83Evna319 zuh}_yFemSv+Rv$-!HjjT=h79!5oSr`2s@@LQTtkS19|_#iDUh&Uyv5p_rN;WKZwun z&uV>g@;U7P{f}ybVIJ40YO>~(wZTujS-R?eVWz}MEwq18Z+T-j$kt7I+kd%1yg!&9 z^T8M{H5KccE{>Nce|hX2V}3t{Z#AAvs!yf}Rx8W2dtknA42j`->Nh5y5++tLekZ=; z{IYkDdi~BX%(X*p%UY!0ZjGWw{B}bT@7N9m=G7O*^uhaVcwEnm^LTjCx_83@Q{66i zFz?x548?yM?T>j`+rIY4et52a@6V5?{7bFeeINIidNAZou?HjOcwwA5I^2Rc8pu1J zywNe`@J8>=SZgpxZ~d~%#T>oMX+6rfw^`bfq#O*b);U+T2rpeW4|6oQi{CrXv+iho{mkirHh4aQ-BOpWkc~4BEwL2c86L^^&_jyvpoh6iADHSPn{H z!5I`HDAhJ@=RYyAVDT%z5EI2BY#O1naP$gudSh($&3ET84`}iRu zOlB*&U={ipF{I_-9in6iO- zXiM{KFS_a~p4hs?rft(jbFQ#mFhh5KQsuY5l~xkusyG5<>H)AVQv-Ym+%}?1$EYN* zoO#WT={wV86ImT38W2brJE^~C5-1GV2G@8&^CyyQ3^*c9f?8NE3+dB`CffQ5UU)&p~fBkr8@XdIRZnYaGJC@8g<-o=L3OwZaDt|DC-G;Y|rj&#_v;s zc-V}=wUD{A?QoZc9XnCXBP@Rde%L09`!_$9j7#st*KW(X3 zz}VZG4@0yk28mQ7njDjmABvTP@pt$+DdsrZ_{wR!9qtkfXgQ$qeat8@8FQ|RkjZ#o zJtW^6U$GPj;N)TL;RoPuB;UEWyK-U6kn_^hf}BJPS#AgUP^@akY_PK7ih9V}w?;^fqB-k9m9|H6ORQ8ao;$LL{jl3w7^1is_a9cJT>|rcV_b9srpX;#u?HnfwDD<>^>3sY z*~H6GAWZd73su~?(AFmE7PIBM?>UQ7YzjfKEor&|Jo?+7=-PiY_hL6oiksIRabPcW zWKWE9fL!oAr}`JdkGWq2W@}!QubSTkfuTIqGku62d)s=VPCv=TAJsczAxy@$WR%Mz z_OJoPvgueh$^9BpX^qBFZyrUcFY#})xWPfS#Q2c~P%C2Y*@RUIu<+dB%Sp^_W~8-Vsu>VjhQc8Vm&*K{m}hWXR&-htZSx=(576QTBwi+QcDj zvHeuE$~|%is;w2HFTng8Dy^K`!CV$?UjwXF+wtKxFG(u-*oo0LC@E@Z_=eF01Kr_< z=j0QVFrJ0M4o|LRZ4T!yaTk+%URo5wQGF6*Nq$Y(q-lSHPCOx2Z8R|a&d%6|)O{Ve zHyII+hl;iqrE1J-$98Ejh1u?s#A*ohUp zGTF7KV+MaIINx#}c^B6U<3)VF9m7i}Fo}{7Y)|%DIrR#_F#2NF3N9ma_pi8^zxml@ zcE|P5&=o8O^ef5(B&0v%GmS#z326f>S+2|Cz#R#wFU;UslLZdo6j(%ZyPkW}=Fk-8 z=bu1h5jzt3=1tOzvxh13k=Ob4MM!*t2WKiOv-y7%mP}e6RbucC_toI$CJ7La6l3rb z$@*qzn$36N&C;`UG*5B3ZU9dM`*|anf2dh5Ub&6k7si9MnkNK~!*;B4WWQp(+89Ry zG>e9nyWE%hI;8Z(H4)ZHj&PS`KfhoZvg>|@)w8|l&_f)+91qMGtPA6zWc~{$6v^MP zb6kJI_Q0gn3P-8Wv7h64z9Nq62(uSQjF0Rtc2I1c!f5HAZ7Zpp-q_SxX?IBe+zWf6 zl4ul3=8HS{$qXhR^H8!E?BPon2E5UKfs|`vfWP7nCG$xP z8UXFyIg{)lp(g2>v1H9XzT_(QlU2@Jk0pOII4L+`;1%WA^K1hx2VDZAF_Cd;*5EQD zU%I$KDnrc8Mz3)y~8(s`OBkS0YJ1Th5~ z^ZRiyoRzJ&AMrUX)`{-rj6E26b1sZ~bK1(XM7ul3VQnHfxfjipH?H^y>WVHa7);wW zP05C#28AotjPjy>`+;;EDX=|QK81hmMz)D79qYDf73QkFT-(nRma?Ir^bt!T{jOpe z+}`i79)`MhP4SzS3xB-b22%>7JLh0xIj!TOOq)M2vVdQ|^BY5Vj$s&_e&$}GE&RsBQ)hoo|G@C=*HgbS8Ls6Q=AkOUl$R1=WoC@+D_a#_oXntL z@KBuO!=(+L;l!AH0(NiM89TC%GBYy_tTI$)Y@TJP$gDWNZ2LjzV4W8^khvNLVL=)( zF7})q#?fC3x36KA^u%QEn9RLc-bSM*8DkZVd6Dr(e(n##c3L-cOO<~pVY1|faXIbL zD`+jj&CyGdgBXHdFIk3Lmjrtb?>|U6qdv!>b@Pq%dq6oMC;kIjHVeKXC+dovLQ>`n zr+8orVHd_lN&f|TGoN3wSJ2o%S<)vVXY#X`KFPy*;v>YG{3|c!<*Oj~wNZ&JQv@&i zXWK%`JO7B1*Fw7}`E!3O>>Z2unnX!^Ymw-fu-3e#lGYxSNKw)dld{LdNDW++Oj3S< zF1j3qpXBqjOOg0>Dm*TS;Qe0`B7J%*$2DrO0{211S&E6vkOngO%VkJ|lkT&D7fcEf z+PJAGXO|JrzEFfT1jDjoz)5JZPwGm|qY!Br+EZMJG&mm_(g=Tnq61|}!%$v#8Ped; zTWWP!tF$s7pnbN9wn9xD^@d{~E==YKsf9h`6C^w9{Vi&?&mihBiJTu?W zcrd`h%;c=IwX4sG;-q(4vyT|!c?IuS;$hy_z*ZpwZ0vtHdl@xyJ$$xR(@^Mw9Y2$_nTa0oXuAv0;Ok%mAt1u&6E&Ui83! zK+zSCNanB5dBhR-(r>a~XQv28QsBs&O+B5QBvM4ZM>GXDgDE(LNU`i(+ah)L`5Y&j zqD&PB6(Z6G_by^lNA|sosFXP!jx-bZOgTAFph$n)X)Gysru46io)}be(H#932eSrB zbAWQiRf@m1Dj2EoGa|=SG!bSu9+oUk!x`C|)j9h!(@wcVKf4(N>aEipj8y0g<0`bn zIAyLv8>~CbHE2WRh=MIXAnR_oW7?H{EDgH5)gs63-}^q|eQ_|fU)OGXU&rNWSD4q# zJGp-<-6972(HQKB`O+8+CIiEX=;Jb3;fZnN2P4LidPqyRmLloI zv=|&#G>>U{R4CD?$F%fWbp}1A#o$l_+#A|08$jZ(F>$M2D8JPN%+F^`i(#@#b39k0 z!6(H%zv0dY0`a`uVJe^wfPE^a#WBH}hL1peK6Oo-kpvu2tj$?wGrRw032tX{-SdCZ zjLv09i!Vb6S;a??pMu}c_LC#erX)y?=$aJtS=*YfDPU5AUU|f{7(8U5G&1=?0RL{1 zU{s4?Xi)U97K1r&4{I@)!Xd+2d>)G18<e@wIOF!l^u;8ieBU*JAKKVJ-H1LG`Wi4fN-FSc^@7N}`9g^qG_sQnj#I%-q9ToQSs)u|Pr*AMar; zPDF!&E++7v)9Ya^zTMJov-V$z$@4FSA4QAKJJH0x`wEU9+*SR?zXa-nzFi2cex7{;^s;){z{EPllD zVpc`zg=28qMLCVWvQ=U|Sh$h!zN6m#AoXj&3YWAW#qfAAnx&hiS{IMm%*?cX<@jGP zqgf6nuDo-ce`6w^rEj@9LN!?Bbn0$lOQdvZ@>~veh{A#y%gcwV6w0;r|}&l%&^M`9wE^{x>4Fk3(0sm5Be~eAzpt@CI%xXbFn_OVYX^U z)a{$2$k*rdeXk<99lKj(u6E;*!A-1juN$7=4KIur1of9@%09LbS%D%J*w_5KLRIv2 zug6tk5HMd<#qapS9zqEh!69EXHxC$iNd5gFMfMK zR!%zzZ^dLF^~HZO9ip>uz|=Rq-ia>g=@WnqiC$tOoMg|*_wbtG#c0T=Y%h}JzwL6V z6M3+uP6A8hbxwivb2%7ok;WY!VyhVS1Z+S~#}B%245$9?xAi#L0M532Ba7Ss9hJm1&oPoz)^fEOSyl4)4}6>z|14MVikOw>`|*b z9QV)aI(v#5vk^vPvXmR^wAC$Cp4aB`%OGWoVZGbrm#^c7E80$6m<$@5D9^{hLxUqf z7MwDz6G)mcsVxKY@^cXx06I8T`V;sE#ry3V4L+FJ_P# zH8g0ZRgHhr6uHQh7RN}{|7aD)nRAftTWBx2(Vkmw&)T zrz9|*Q{}&q31De~LjxfKrP6uMOwreo=%LaAH74VLultiG(nEXMTB$to*j ztf7C*6Nt4$zg59oKH`*u7|TIsFi{+~Bf+>=oDWKeZoW4-JZh7Rcs|HQy>`c+m}mRK zxJ+ph#sVBFmrL;3^>dX-KaI= zg0k4_E;r9Z^-a#l>KE1timgzCFAVDe46lDjT>r*2-XCVSmp92jH7h5wpOXIQ#)82#YcY( zc59(w+`nQNXKL#dGc~fc)+{PQ0ad`tQ_X|~^*d)iX= zdFn0uQs=1Gti6bO&W70c-}|SpQ_ne@GQIA4&W4%megXPys-UyMtK9{iGy86;ptHei zoPy3dpN~R8XT!{L3OeUvc{5SZITwAk>NyK@syEwDDD?bb29=Y(@LH)RD+5kE5h&3lG#>FmEiNp6d_nN^Jj^+vn?Fs&a^!l{bGX%FH+OwUPwg zxF0p5Zpy;?%a5|T9VbH1A5bxRjd0sDeqaJVVc(>YEUCSKMpPYPk;>%B&O>ylx{;0N zN3nDrgqx{h?KnS5<>CRGj9h6A@MdTdYC50uqngfviJH#-6lHU;2Cr0+Rns}9u~gGJ z&d)+kXFoq=^Pr;nk*j~FR;`V04{{& z>*4oNU{EpH;BvyI9=Kz{RB$=7H@qq`XYX%y-deSIeCxBE=q?u{itZ!+K=DRk*BeUx z^m_`$lj_bC16Eb%gXv_N`~}3}`74eK!5Q$9YB5oo16bVU*V-J1MAql$}6!XUaN-YyZjoQ7N8Y0N<%Nj<`~Bd^Z$} zy8694)Ylwy>1%(_e(sM#D_m$Q?)jB+M5>I0hXo3_6at5<92!kegiFKec79at(WC%_ zqv|BD=J`>{RZyAoFVPwFz_A+_CTj$_nc2Jj9Xtw@#H^Ho807pqrwx&m_>S8)EPo(3F3}GUrgVs`;aU@B$iAC_>9$ zKTZZ%Sz}eSrOM9BlPT`X&JWXM!9_{-50l)Vd@$cPhLxQk3=7esvhxGakLu|-IB;+m z!|5EOI`(!&W{aOW`4JD?xa7eUJ6&=yLf`M+T!&?1=O*+Nlk#0?o3p@*2~=h03q$42 zr}1*HT^N5>c79cBo*I3nN{%vlM2###rjCQ!yMdZk2yNL#b_~HU@W8v z?KkFD2Uq&2!m^ikKA1y}6O9G`z_2pyg+OyWhiQNIGp)>F^6ZWSImbt)9=}NDkh{ax zkO`(2g=rnb`fB#)9=x(-z~e3|Qd@U_0e-uwX)Y>Zf3kBvj-$#;M&U$Hp}^h)&xN}RU2`VDS0!;&Ocj;22T zfO$?MmH)f9x)Yl&KZ?2GK)GptZU!a z=fF(@lQ3sXt+~zk6BDN=5N=dO4ZB6Kar5LYt3$gTlkktsAK5!o`TEJ3{Dv*) z?so1@Z}D)4a2$-nVJ-}MnH$__BevVfVe%#qhjC0~78Q)RDr01{n)u3LxX#v$LK1(d z{p-ysBHPtLiHBHV!zeb_zPhl9qn6L${9;dqHib;LH49WWv$ zXVCf0qJ%lLy4&ZK?RqeWA;s_f#srIkHzqS8#%HgxME0-25m;OCy_vHIkFpi3CshKQ zaC@J#_3yZ9(`W0>G072{Z~P!nXH0*L(^HLL-5{@6VS+l`Z}Ri|vs4xK$Oy~4FrGpA zz$d)k_Q&8ZlZ1=E#dB4@ToyIUP^mK8E|6bsAr5-nM=lk{I+w+4ZE6V(*i?q%&Ap+| z@)hX_b2f>xL)kYnEFpmpILB)k(BJ8hscHFGD6(`+5SAI|i)dh6tF1#s`*e9*$6h zfn2#E^mM^cWs_2R%JNCnPjnZ=SBM{P&go!0+W~v3AiMJ!%vW+ENy;e5A{({Sz^04v zij&U|(j*nA4A%m!3IS`Y_RD{7@d-G}?&3p5IJBA@JY-`XfJ>!y6AxeweYGnnt#S4J z`56{f%+cy39)oGNjx%br2-w;_U-nc($g9*zsQq)*$#Bg0axxY1!QB)F zkq8DSO6lV|eJ*zdK@7zpHNJ#bW8-d4C0b^E7ay2f2%3#2){SyQMIM;1tTcnUu+j*0 z7Cb(P{HzGCy|%By`+6`cwCAT0HNX{}fW z7Yn5{`NE}DEQeThj%$AaJmhRw8DBrQ58_Shk?-rZ^B;^}$}0F5Mwjh*>Nn;*)gwWZ zl9PVt@b~y-`oXW?`GvW!z}X{iZEyeWz`&Acof~B0_R}@nhd)ckG{NotPV@)LM`hrg zlBb7p;syj;nX>R0n;KglN~30LKcQ%A$`V$Xs9x9KaIq|IztMm0yUrf4RwwJVI>cO; z-N0Af7!RB|*WZ|*q3dp9Vv~R8XN(q%p(NR==*j892f3-P6*_Dhyf)4E&D<2fO3?L$1;{RlaZ)eszs0`KpB7I<`kF z+%J*gj5l0lzOH|~LNMM7g#cmm8U3TsgwFje<#kC6+{f1?LCnQL_|%OFyN|P!G2#)I z561|zCFRMXN+%n;G2laX*^f&hQ)uR5t&a*2IFBBevuJZ^$+O`H^~G1F^ZdFthcecC z`@mSX#1T1J4h9|!>m&kp0E=odb6Wm7E|6lMq<~oaG%0`2G#6Ue=spAXC+er%1M)`= z?C=AWE#eJxedaxdxjt9HvkR7Y;$R>f`7c~ndVd8|H{fOQNt`MbnB%Qf=qrv=Vc$9K z5j&{Y`*qtw>E6b7@1yMbofu0EUH!*q@EwEZL%3P|tz?nt+r7fA(g>HUD7&L!^+7#U^Mgmv+Ooyd4rxt`POGLv6)j+HM zS%sDixmgS|$y7mRg)0L$%Yn~s98BhjNwc<#C#!#iQ-y4Li5})xoHe>w3{;xA_Kpe) zT`aZ=iM$QF-NGQ9w$@2ScQ)5PpMY71Rr#Az^6g-Wt)zp+H=8d^Q^X-=SwI}o?kIo# zJQU8ytiO*s$wQ2(NiJnw)CVIrUKK|D3v;dmJ@pqx*Sp^S7bc$a*0_IRuD8cm{Kg!y z%9?)(sr(xPwF09Y;0L3iKF`<)Orf30bATO=LMhu|SPtU4F+`>8 zwmSwW<##%Kg@J55k!Zd26kvx{jc?~S-1UGEb^%v;?@ZFJr%5IvZsO;W0qpQN7rDa{ z{9y;H(u0rz?6Aroqye~-viH^#i?qE9R;hoKL)=vk=AeCB>8nwP!$aghurrD?$(&US zk7MQDh11EG?X~~^Zi-+Euw!^h8`HO+ScEfQs(Mwp&CWCi7q5n)Xaa1c!&>!S;k0P@ zay9-;ne@YJe^;%Y@|+e|_o|PL{dx%swa_-G5oN*BhXmjUhDiVkzz;Y+DkftrzLtMM z$E0b-2UG&^g~^z689~x`-w@3=Y>m}Kr%2@lV!Gtz-Kyc4uUc1Wk4OLp2@~!Giy(j4 zm^z5zI;cFld}3?+CEMiKeqkn#|H0ra%0|S6tMYs@2^jk=>7RtXuj|t{?{&fY;TiXc z^pS95(=MBi2xb9eab(wTSS$r*YB7JOq=c&Ft}`&-H->4z*aC`Om0Qp$mUT=+X+&d) zO+U$#|immKK8>@?-4^jGMvX5der2ZSB#>Imkz6%29`=N^MDWL+*Wa>e`6vK zIM=S|1qJO|kGOGp>uXz|NdB;QTM+Uw_}aT0o1G)g?t%M9&9R)|8a+g+(UgC^2Ohbg zKTzq~T3FF`2^mA$(@79?Ht6n`h>^dGdyZ2^{Z zZsMqW=^!%3hAmarNa{BnWG6%hcW&~!Cmi)gRY+Bud{QAUXVApH1%iJr5hDbGocew2O-4mie1 zQ?M2Gn(X{h`?%2$N5p@r8;#&u){|F{S5gQtjD9U?LS%+l}m>hp&WZjL2ejuWJtba+`e_^hNw4Q%p;@#s_rQwCiQh_dv zv$6-K9U|Re5sRKT&vMtuymPo*IVP1h{*#5`3v&;sCym9Ip)e~bNk@O5nQnvxDQKn$h1!H`5ANXP zyFV%`DSiP=B;MhvrNldbi7Y5C{#j`rxktL@U1lWS`Io8eTfv7TwTSsqwv=^We+OkU z@eY){uf#i?h8~G`ahjNT_w&5nyLC)Vwt6p5*}=q7j;DTOj!9=(=to>9{~3aT1iWBY zEfITZ(_MdNhu~$4u+XK>h0MKMMQt$YLJ7B~a_7>W~+79N-qklp!D6;Y{oi(ciEdS{*Z)~bV)!oLrmK|X(0l}o17yHycj0!+TDs=7|HPVozl}W zsTzjH(XmpIeKEAQGNJoLQ~O8VQQXh^>_&fm2{+oWGh!-H&-uF&iz4sN2|~Wp0s>Ju z#v+L+7$Faxo~RfLDplzT|B9Bl^vb(4%%Y{b;IM?C+5mGM;k-R(M`?F~iL^U^h(;1; z3sxE%+;|i0!JO}t!=yX=2!+x&ex`1TTD3504hY7xKD33VMm|Kl&+a-oNq5NL=DdII zl>5?gU266HiT+0}WgZNS>8s0}1n1A{Oa-K`c6B=D=*(Tc(FV(xcCDOrsnwJ;&Jj)b z5PzXD|1Dl&#eDB6Si$GZ9eiN9Q4fZ!K)*n5qQBzh4)#f0?qHjwiVZre&}W3H$3Jx3 zBSr+@_))QidbxvrDKB^MS!!4-er$h+KXDX$xP$$zmpj;Cm8{V6t7rYzG{@x*USS%7 z9CX8QKpqS)r}zc9gZ&k>VDmcb3IvAD?M$+hhAIs?at9kmB`7Nf$Pw!_Q$FgkI{1e2 zN}4n{U+&QI1%mr(X zS7nVgJJ&q8;%s+r@&niNo3=z22&XCV9XTfDyD;->$pl8y-N8t4vuX7CH%3W!H-YfA245*c58NX%r}K#uiii>CzIK*G=jqA0Tl_lZ+$}RW(WT+y z@~#msi`f6bT<82}x@}<2p2hF{#wep@?H1{2gR@s;yc;2Q4-X}h?rcTW;mt|@DnpFz zczKmFpkNVu0g=37`V5)yBAPJ+o!B6n<^tORr z?{pGUk=w$rBZCqf4&U}W1FsPL^^36J~eY?UwK(eV#$=ja$LfgtMvMxEiWvdEUqWqTbR z!$jK!E^JYtT*H4(jFWxmPqYx78g?!k*e~uRmkn`0hKWh|M;0zLOIZG={WZNAKe1{HKz&1sL)e{Q=1CC)Dnop%PdhTKUGx%+>HjE@sV|u%^@Kg8? z)DR1~z@)uQN7W!r75df0HdeF=AjZeocA$Ju^UAg)f6Q;$q#RN2-oRXs2+GTKmg z8j?5aw=~c6`}THzE*ZAT==6 zugiagYTH%!HKHNJUnW$8-)R&^0ORf zq6yXDNLubUEEDR%RG!8W2@+d$O*qFB>IUdHOq(Bd=`wxIIf#_$tPiK+fJvv-N;wXkl24snNWSdWu@&R5LHne1r(wRA&Vp?3uzuA z!sv0xvjfOKqIa&mET~S^TqmBbCDU`m(Mi+-$>HPZ$b=f0FroTWG^&Ct^e}ktaZMwP zsBv~^MD?>no2V0!aS7g06s8nbR9pR}OJrbw6iu?k>&~?U0cn4n zm@K60?U5oT3yF6J6Ss9d^&4}ZIzcuRz4SXrGKgQ61O5EYFU);`ZVK_|w|5FtEPq=V zIQR{-s2lu5oM-0xA$9|CqUHI>F9g%5U+?{ixgO&De`Bt%EGgGx;4~BWg>n0_!)oL7 z(7*#LZW#vX2a%%8>9dF>SvQ4aNq>LfaXWAAiBzw`a}RhbDB%Z~Z7UP^-aEoh{0=u! zX~q;USf=5Ffr;CHLB{39SKKCUOyZchF~=jwov)B!hwq%%hzl@ z?fTNs&EPv!(aTNTcV@r)=C=C1Jv4DY7*(~< zh<TZvE!)G%N zCUZnR7H)@CW^->nK}4HhaoB%g;r5S+=*O)+kFx)($>;O=Ql3xJQi-> z3>E5FOWSuPL7nY{i5nY!o46gL%yp*--Rb%3=c)MTjI6G^LYkeFNXJMN znKayx;1~w~o-uhJMi8*jIz)*qC{EH?7xsh?$8s7 z4hoBBaS*mBSGy0afX@rVh{}@M`dpo_UYJ;$e#&yZF%jYUU|Qs-zcKey>nI&RF!ANkclpD#?alXSsJuq1y=es}wb06GTraWQ*29~JcJm?_uK#t0o&zl3^cM<+&oIwrEX z21e}r@CH4^;q4>ze2kEDDucSJf=D@hhJ5WDr_ZQ3;T)_p(M2NtMZb&+`$5qla!bJP zaXJ2XV<@T2_R-92@uq}Z2`ZCuDUvg0Vo(9;c+A@kWlxNLhdzHlaE-ha|AoO?OYw@q zFn!Hoq^PmzQw%t6_`v*se0@oBk7=hUcFOc3g@p3!*tMw3P{Tu@bo)~!tDUm`(*<}bUrg$Ag zO4-Y7DdoKp9MpeN;w+_%=g%#qjAwTxkCks;mKB4HJ~b3lKJf5wP)NC8--{b#A!UET zLdyP$Eu;*~sUf7q7%ijpwTt|kko!7?W%WP^5EN3-aMPP*A*D}Cm9c%(ceh25y*XP* ziGmx~6OoxlA*CN@k%bi3&e%dq-z9q}qx2S$@)l7(h~|H3QACLl?hsKBf| z=($L&+753SWy51fYRGe%BO@9rz}L29_6vOh!Q?|$|t@5xy6&- zx69t*Ngr>G1&e(A>)M9-;&9#a>3+f+8RAJuB9=}1+I4nJ2z!9(>*sNhc(U5Id`#WZ zfr%{T59WWBrA#>L5^>@y0`Z)s{K4d{Fz_s61Hy!ZtwZv0$oiV;iolPgwQ;Ve)098( zvxB>;R&&KcV!%A%h&UrZx!@Kr1R`fYAU16vOv2^$edQ?=?vcQ=1kZ3j*OSlofBYU)=L6H$<*zg1%G z;tKTQ-goA5pwPK9=POJ31J7{TXZ*m75<@ylNQUaZL9yF2vXfxHKVT#&Uznr;&%Oy- zKi7XZ$ye>njP7tnT##}f-6{sM zt~)gC-4D#r@jncW7rRUl?a8YYTtu11sKw-OEFxh~8$s2a|}>nOfhp#h_)f z?Z+15^-Z_^W`(L3=kYXt9X|8x)C=ZsEc;hut=9Ip<{EGGw}!dT^l!{gQ_g4!a`pE9 zoP4y%>DQq_3!Uy9Jg{lD3bGPPcTGkco`{geWr4mu)4vRg+kJi_P5FbVf1)WJUxqBC@U)pQvH!giMA> zL?mkVPrTBUxeP%`5bCf{uQX*YWdz|5esle~tahhKOi<2g$P!QFUF3NZ4vUP_lr#Eu zg!&IX4{#Jz#RY}_9?|GWZ0}&4l+1s3<}b`n(rWr@HQWV8)63zHhWOev%E&lQS#U_N z(y@UF1DL*_y-<=Y0Y_j`U$HX5U%;#Z*UJcpwG1Pj_zf2O>i@vaSckt5j*#k=Tp`#Fc-dB%Hu1tSqG4rgY z-Q(cVv{T+eqaWc~Boo4cz$k4iPR<5C;V4Vm+#OFhyxA6l#AXjQFmhhh;=`IvExYnj zdefY9ln*9yltsBV#d9%(vX!GODzdb&Vy4@kl(cx`vJL%Ub(DqC_#$ehi2LQR4$5K$ zI9(R2l-uEkI&nE9B^j;ICgy)?+L_TtX1t76Xf*S@o_IFDOil3fGFq|2*H_rA*cRxr zS#5aRtPUnDLg@SxIsHE|VOctF{lpv#6=w6ye_^h-Vp0Er*}tC4J97YoKzzUX!i3H0 z#*mlglehf5;ILT%a}t#W24PmguS8{H%4M@+&!5X?1vn)Bt(e9&4qmabkShc5#ZgG`?hK&&wHG5N$a)Z+iJBQVvwBvBp9s_ zkcLjaf%*R5Vn9XvLXtch;eP)Q5s~O>dSldb_PjNr73DW(u+uJcT)CHsb8Me2xiPQd z00EAkzL=ff5n_vIN@7q^la;xDFr4+-r+DV~TbwD**M~*Kpg&Vh^vIuC1mgU9>g;;^ z56pQ9uA~nk{tE+d+4NiZ+ci=D)GtK;EOz5Nd%V9fXTUKUJ@fZqV11g2&pd1ydwu7} zTU?!j*1*_gHZJ^oArgCV1O71AFBC>V^Vg-K)TIsa`-QZxMO_Fa_=xb_imY`h z(|g08M2bZk8sx|(&LBokK!zR-~4rr!(h}z4Cij z_%s#@>aiMj&2)U=8Q}ceX~Z=lx7! z`1%1A1K%S+WvW6(!~f5G_cQ)JOevttqQbw6#jv^;AQSF9bs)Xw7f8@B{*5VuPg-LU zXTS(QkDm|*c1<6V*0_yrcKt>@M^tYN3=@>F^cyRSTcyvFoJRH`9>a64pYgKv2#*9b zfJPV-#UkNfz)x9!KDU#Y!B)v!<*b8>hTi+23)H?PvkEHsWid!XU{wn%wX?wVY)HgW`uuvY#(Y~EEd%GiAB>M zA-ja7PWCbk-d1IU&Lb7q?3s`(0u|72oRI(AL(@=UERxzd;Y+S*DBsH1{V)F zFA{Qx32!F;Mgi0N5N3V4e}fMc(e`IhW$1=(8G|fht1S7VX;iWSP(S7m80Uz!eZUoK zCj|rU;-gr9N=zIzzi~lTAJ@*>M|-ou)I9IN(7ONqLf}4R*d!>iQ_s= zc)p2*_Y;-Z2){T7;oSAc$U!k$DCW$PV5UG`dH(!=qr;1E3n07>A+r;I zencyO%*C>fAPWlX#^hptBcV_y5fwFcCCm{FgzOl?9SNxMU*7%XKuzoxmq^Oq4M+t= zzFeNfr3dtrg=_+cz2{^CDvyqTPD`ixk%Ay1Hw*^Sg5&}NPR=hBu^ilJhl$NtEs$*! zlNzS#HqIS_nulxNw~VSHo(E81CTk_114py!9C|)aLZM7NHD2 z?fy1OPCwtC4j))>Cke_@VJGvU?CFHF%@Zp=dUYK$3h63p9+-4Mtt&=3uO z92A`;K1^awtsCI91d}|`I*;jdYl@Kp1DO*rH*(-6{A?`hp+Ay5vve2rYbx^40~(M# z!E7MxZWxMZ^Yg;6PC7u!BP<=}AAAze{>a5ck`7iSp)=FC=(d9lgp%Ak<%9l&pZkWD z`^Xgh=;TrPdtqRe@JG!yd5=#CmYCLmv#Qr}QUiZ@KdpK6A?-%~TR=5qY-WFQn3>lY53TFuvwIj5`y8c|sKYR+K z9if$e%yA2??2T2A&)z`vv>uFw)*As1LAZ<%S_!6k z39ZCDX$!6V$8x8WRi~Uwa9yEN2*b4ttAVyF$?RvrRcE%RqHcy{vGtH-hzp{zBesg;%q_MOeqLfL zJdx!RTbHJ@#a5)6TJZH%gT1hz3$Yb0zh@k=6`S|G#8#MzQ$1oUv%D{{HQ*z*UKon4 zG&Hv(wr)1I#a2+Rt(L=t-TRSBk!AS2#8w759^(jBW(%!{HHXlDx;eRIR#D3qS%2(| zE|^T;_6WpsOnspu<uy18xIJ7DlS_ucAbZ9I*HOB6QKasO-K2<}@gJXz_ z*(SE0bOJIByHNg6I|p zm7IT`P%q;n4lXNrM;uh6=_3!Cw6;7L@R0}K#K#*5k&R8EDTo;%L9XESwVM2lNmTI( zPf0$o$9=x>{E={&Rq+W3dxuGGfV=JZ7;1WS0nu`_^adj{DO*71SO?9>2@#>mYGn6sOv#6n*Cyd1y$=kP3zv@Z z|359IJ^OM89E?=bZ+*4e?x(5gCjG!LesWylb|_vMs#r~-+l+QTcD>|)?PJ&DVqHN! zTziYhB{na4t$ceS%>C>J;jCDH7M*;9d5dp>sf^hnJjGv|*wG5()l?-K3?ke2*R}ui zGRZj3{5lSpXYcnDib{j!2x1U-r|~#gL}AFTjsvYUo3I-rHx67}`RoVfb908|ph|{ow1ku>u`gC&=LZb^DxXDxNk zGX)ssJMccFNFS$Fv-t1Vymam}QyIneFw@+B@J$3zmiERV6|*S25O5kn%<)8Of697e zl6bzkE2D+S4fkLTPiyfj&&8iV9OKXB%}Wg1EbljlB;5VPH8;(+Hc7OZCak{~Mv-fT zrBH0}(XfJqYZZ@~-ltlBZ3col@&huOn8!*r!V)7aq;Y^ALq(Bu_yZ%~jOKQg%}ERj z(Mg9(Ayu83eUQugQ~T?vml-;YpO|J9yd0DIEKtQMBKgw)ICKWA0HB=;1?j&@wdv`k zsD8nitv07pr1<~B?A7->zhG%%-x@|4=RUf#?(rOy*3E?~GUxqE56&LMjCNwR>@zv-jVY5kLeVt-}2m?|y4G1XTFbWoj4_+|*( zK-X2N4#ONr$ih74B|;^YxW7kPR+!}c#0*0}T@hHiE|E&w57*WvKCN{RU&Xw(91N|k z*Ou!h_IA~@O`xKGBHbXok_xl+A{foT$qPlU7!I=Dmv0fPzOLuWB53CchBFaH2ohKB z(P^8$Y^y`qmWe&qUUd0$cPOE!HbNqWM*mlsa-0_5!dT9QCE)}2Fh?T#kZ{BssUmsO zX`*!>2^^`lFsc^rI?5}|4f74=fnHCPxHKrO12&yZm6uC@P%1>7ur#D5ZzII}J_U#T zKeX+}T#UGGS>y+1AJq8DPk7K`{;gqTg_x3swc`lknQ0Qkkn~8{iSORlWn7mRSvNi>0?!L1Q??xxK`vFNr=L0M}c>hFOV!SbkW_c$D z8L+0|Hy$}Q;bYrhpdGHA-=I>fF>SF7PUjfo=AoX%#brIs^TZ4S@CUR|=NG(^dkC@8 zF~s*?@F$k6q-S>mQ)R8fk4{ACmIu=RBh(eO1bW0J2If1#;ltbYJ|-8Z=;d99>I>_}E)Ztn*OWU$Nik*v_cY`i3^z#KVhG;QfG%x1`b=NEh=t6@znB&#uX(}ZL- zmRt}=vieNbzai0TJ|R2nNY5l|+Bx(HFdFxN6LlYYidM79IDC%N?7?_+PXvvGZ1tuh z?{F_K*(xw0Tg@uLCuFNxoOExUC&t>23?AdLugsZU?;d$-)C0xa>AK_}qDZ7QM!DdcdxKf{tTeupMeYbEmRy?6|K4nO{IKec3 zZ`n`efF)ebqBo9kHLHdh!c{DO2v_gz5i3BQ$?dcGdm-!}1N_KSPC-gZ#+QFBA^78K zJKz#xf#%nC&?VKNjq|}59+`p8i8+|O5ktpJGl5-neQlo&K1#tdoH{@-T)#rt-h;rot$ z4bHM^ShMMV1DuqBMfY6H;5hJrVNBr@Psk(@E zVeYq{U*8etc+zs!0$+$8o-w-Cavj_GmI%*3PNB3XF{_GsL6Vm}2n{|Mg$5Juj$2EHA&>3QU}Eyvp~3uJPu$z# z80iQNCMK7N4d#B$vSWh@mwv?tbBEUs4JH_#&B`&ga0%L@7nF58dYk%xA`qBwITC4O zdR*`{Q34(g9`-o1{;b$wja&~*Bj>U#0)AfFX>LnQFrH{ru@GAS56u2h zCpefGn|$Df&&_7tX=-oV%&XYojd5%+;ptT3ne-FoPG}=$nD3ROL7mWGjya1_T8zov zmSlwnYn~@Gn9GvnidJkeSF&8O!C3nyT@wObZ1sr^-k69Dz7Uju83B)-&?eIOxmg9Mjj149?aBBQ%)AAzf9W!6FZfwZ-Id$-zM!M-C=^jQfcc zHbkr}*VPs+dls?w&E8^@i|t#r3ViOAHqp;Ud>=#v2J@ZbaJbU-jU}z4id@)W;RFVA zR9%`30)si~ddwAnn&-=WkwdNDPn18A6Q|9p;e%}>Hmi6L$(!Xt(dU#qOV*so;LL6P z+L6Jziz7#5a3(YQx*~&f-zvlq&3f(PY#ZFM!9Q5k&5$BEc+=z&81TNs?8$Gt+=vkz zjFqUic8|?M>@72ySua#@FyVaXv}r%|M#&A$aAmaTe$KTtyZGV3&R-r>QUnL9q1G-K~- zxp>GGAe`rMykWbhTxQpf8gn8-HpUSld8W8PDLeyqa*ZQGfay|3X=Dc6%GVAF0bB&H z^M*N#B$)Ys{F#?R4cM8g)=h!&L1#!v9_tPP`M~qfe5u^PgKWA(5CnzL;&k_OJ3<@D>g|}M34hOl(j_OEJ z^O~C!m#q_Lp_ci;jt$Psw`#-&2jZsR8P@F|n4|v1TfZ?UHW7gKcq#x5nd01QMvR#yjK#_3+U{T_eUoae6{YF?> zjrXm8%f=5JWjBTp>TnD(wB?w@i4Bg4o&~)y;vz*&1GnB@p7m1e%QGVq(uzMY);LI! z_}M;=>7YE@1!D=Rs0iUUKLED4LGuJA=1?>nFiOOJ*u(+2NtLDrWW#4Mjj#=6f50}D zqUMU15vD}=kF7KuWIINjn8va4aH8$lfw&NV$4tbPy6s3@%6<$@TckEcrf5FIf{pr-1jx`0h(BjV}k*wV^!tjRY%IP!3Bl3zIJGE z$yox;4h;q-?XdR`nEr$V2g{U}`1g(sE@>QJ$P^Wh9}$oxHQCpHVUDqaUmh<^mgzcw z%oiAn)0U}|V5Dtck-?Z-IhBS8uel?Giz$;L%THFjn8GwT=wt{+z(z zy?yj>{@7lxel|mw>~>)o7z{YsY5T;O#S*Pe0y9ovFef=rU@&*E-GRYemvI7vxufL- z24fw2y90x{$^8TdW98drN8sL^l;dlE*N+`x)4`4d##?o~Fj=u{5gg3lbu_WoJ4qwl z!NC}N@YTnAKIJB8B6#aUJ2<$gvfK_1Cc)de2YXX-jHM%kx&CIpgIWQ=`(8zd-D z9~&qp*_Hw>ZIA%$3SgkV#M828p7s z7S|Ic7&Z+OSlhe8P68iwkctj}OB$jL5*ywIiNJ(Gf~;EpPZ}if6G@EA1_^Gct2UIR zL1l^Q6bX4d+{wzsOh@8DyW>QEMbNZJV7RtW*doED)DyHwsQBAgSR`Vzk40i*!Xj}Y zXptzPm&qc*Fwd7sqJ(;HlLTv=Sr6bDLL7nZ?tKNll(4u~Q^+Z?Bl^%Gu}y+EuQh>U z3b_=8ppT;{C}Q@0!aA*O5+x!_woRh6`sd^;i()iJvEr8hT-nLqPRn3_wn}WwHcN1e zzqVBZpd;0)Xr^v8!g5<{f9k2T!+BhxYTEHv=$gWw?Zw3ppR7#AapNC=s-4qXbF&3YWtq#+|XuX}5kqF}GFY zF}&_dJBOp&Dp8yg6IO}$?&#JUH7)^1%K?)d_JDXRN>+}mpr`?Vw~r0are(-ZX6)o% zken_qKJGvaKL6ZNQHmSW5GLttM%L+67ki$6hhh!RL)l@N7v*8YU~;*~8)hWkgxP;V zppI_WTo<9(Y_v?*$*M7rXHl(FIJCU61xOz~%xl#FY6cKx?PG05c?ikx1E|Nlry)y#O8}ZDqZb zDcLr%UI5?5%QDNo1$?8v`NV7r*mu@TK|LI^qVC@qj%sI=4ZtNmnPF#Gx^hxn{?lUC zwlUuqW+%PiD^7B0nwet4BI%{56T+4Bf`JxGjC`QV9us`QR<4V2Ovk%!&axuaNqWGT z`uom$DPu%`{2QIyWhj+HNiPBIoR>g2<%Ms5ZJ#nf|8*t2l(2j%;RVCHa{V#>=Qtj` zQ}Y%J<`xr}?}8Qz5{Qkd-2~)l zXeYb?W4X&jStBb=a@s5rPaTrnOs^n!oC6DDOn3o*?5a;|uVt0+kz;RNX-Wqt&gxEh z`7tq``ii_LMK`X8s?)|yY47z(3QdGbs{?iWK;Wpn{}QF?F-&t7p@ZtrAX$IIu~$bE#HeE za9+seK@O7LhbhSQOmoli!g6&fz2Iu1Q&BvHeKhFlu#s>%n`TnzCG7k?|6+|~);jh% zaA?2mmTEzN_#K+Nyy~N1Cq})Wn7eF~)?r1(6?60&MX}Tgx<=xhIu-MDbI2+Jz7d^& zl>@(pDD#bBSR2AQ!TkU_;Eds(BsY7#bC8=oV&0n{YVGRVixivNV(I-v+tzU%|LO#x zun?{qnsFk(jiv7_zk0*Lp##&DC4n!vG@`5);7M{NTB%y{V8SRt6R?^glA9JE0*6&> zrq!c*k&MP_)kDLFPQ{47?_)Exbx4;YrB%6ZSn=>8X#3YaFGgqL=c~d1~Y~TjvS*=sX(}Dn>;bYWf$#TFQQZ;|F~7 z91OFre_{4JUd~_e`BA^{3$Znyjj`qfxE;lZV7(O|V#ZN?Y9v@&@c}&7QGBra&pe6` zU?)yVB!mJ9VN=7gn;+BdutlX4Nm;TwCg1GO)NoyM`HH%6NtuIC_% za7_o<9Ore=mA%`W>{`5V(@HK_;f4ms^aRS#-N8vQYpY(P4tV!5atV$>BYI; zV%fv?!YCuD499LSbbrHtp+A@(!_kkkJx698;78n}lxBrvJt`;c*U!W1^paG3)8qWh z9uTjY2tGfTUF!BX=DdSb0Kx7b7)68VEmm+gXM{Z&XzRXOAj1$6x_Tc!SxFBjD*9dt*wX zfW$8>Ef4EPmqw`+^fRV;S-m*!zL1g~A7hiqNwh zztX!^tsN`KF|dGtzZV)~M9uCWTcHy2?4*Ht4{mcUBu9|Klhx^fW1N_HOjp z9sDT67F1N8HAb_P{{u%>;>MJR&5iaKKEp({()fvz_d<9KCDWbBF+zkW#l1F)8Wnuz zU2@C|)(v15NhDKw!%|gs&S2GItXz(c3~caFWUylM4US2Fv2}ZMA_H-irE=u)Kv^nA z9v>)@GR`lcl0)Ak!0GJTD%@~@p&W7wkH}@Hn+N)~u+>vnrvokmLC`Mub`;YWsk$UO zXb6MNqjUx-G~j7n`d}n>5}rEHtmlp282g~=1$Kx4pI{#*rPPyHrrKGt&Dn>XYO06u#YjgRsr)AeBjFVAleCo+PZT8T{jP3kFE9(wBlpM zh0NLgMT*KEV~A{d@;(#lQQ$5~S!?|F!ib5nVqfJ-;9M+6U?Xrtr@y;-8lo3aVIN`# z!e0=XfpK=X%D}y<)(J-Na@n%gnUNYF5?alzk%9}d z>_NHtWOL=+{ex)dap0?{7TnyUEt#dSR>iE8oDj;{+9j<{T{wjEjBFqg+`KZ z`w5=(#;(X0Cvbesc|d{>^3b?!IXIEpK`>CH@sP5$z+U(&hWTS_$Os@27cD-*hp=5! z4ULTXzA$Rv_ zl(1aWQ+elGN!S<=lS-ekB=lylEE&y}eAxWw11qQo3^r>1vkk8pES`8&9?|Uc1IvH2 z8SDqc7Q5FIH1lK$J}d$+cDY+hR4sXbqA9T|c|%C6FXZ_Krp zCbi#~*LlmT(M&a(B~F0JQb}m6L?vyXZP8Lz{%Kmid_I^;#~f$}=1#yDb$#GMo&BP= z&qV6$3H6}Jk9rVfm28AA@0zF>pI9&q?u5N<(0UGB*^kd}G+iwa+rCwz9<=cf%(X;* z>o;c54h0Noq5czd-tw~k19N_VJ>L3-$-zwyrpg0HSQmjx^=A3$5l1?;QdzcVb~*MF zOVXRVcVzi^umdU|0!-0$lUSrNtERKnTEa^N4@rSpB7;k$FF% z958%=^?;qAo>RoGVZ9z-vYJQ#<13YR$tTM&m!o~>USVUte-XED0h7&ADQu@H;;9Ha;;%WB-@&`kdVjqyA5DxsHnLI>PVHigO z7#2LpQW~O4W|xP~8NMNZ%(a$TChFgqEi<3*{Kh0M&$q@>q-(1xGDKSm(QI>4*pr(= zv}p7*yMAP7&c4=ktn-VC_T z_jRxG@cRR3w>e%vQI!L+2T|2TW)Cu8P2<1+9n6$-Ilh1sKzj9WV2h(NKL>%0B_Hw_ z_kMU5*N%Cf=LxOD{(@KP`U$V)9z?8l4D!7jUfVE1#{L*HX+j{Jt@U(%QyqDr^`sHw zv<0qee^Q()*B$qN%+6VLEsOm`Wev*xfl3e2{Szo_(62#M`Ly>S1HSgh7)GI<52$JR z{s~A<&!!pkXx2;P7;mx0v$(i^Qz6Xr#FlJzXt7dzd;HS5s9k9w}v5G zT~axEUELp8pi0peM*De|K7=l3?*pY@;4-Bnhk7B~ct0`y2Fa+;EvG&%fIlgL7MZ&U zSA7AN&a>fvCCaq?AOSH}WVnLGwQ#6EL%M`9nEcReQ3sf0Fw`vp?T#p!56P)88_7rY z43$QQWR4gcz*z+9Hw^d9asuP{l=uR+fE`Qx8>Q?9T^=AvRSiwnGW-nFiN8k>ld9lX^A?uH1L#YOy)s-=3sP6z)%aK5;wRx|@stfbmf|4cUW>BpnbHs%#6NE0 zASa=(3)Ix2-Gcs%|I{OZnss;Gb=X{dp>Wz${W=Wejj8rN+r_pq7sr~CFn?n%8w|d@ z-WWN7jp0~^V1}QvpMexKch!Tf$Fx3^rt1TLMb^8Nb5c}}z$;4eysM?=QnEp5(e6NH zAL+F53zd{fr<$L*+2Xy!UjHOnvd{^8V-`C+q3)SA$m$`R={d}oT^jV0vURBK2IO8I zPi&!FO-qqDY^$MJ@OZhaMro=OiB)X&`?!&Wf&@yV6v$<-dO*U{^rb2I+HzyAEvF-Y zFLtx67eYZVLa37mA&Cf6*#;nG|HgVYKdqj|kmRFoHE1_d2s_{FQ4Au^3)#d{;-;D+`QE!HDFhYc2H5}?=bw3&34sJh0e~jX!}Q9tOi-_G6KG0# zZar3~BruSyTn|6bFfuXd9B`7X1(oz)_)+^bE4&HD7Bb+Wt^xB1qhhAHz6?JpEA;DVCdt-^^F)%2__$stj6(9{ zOkI~Lt&l3!Y$4g)mUR#|yfYy;rm~ATVfbLPxpvB`9GB(q|4)4Znz$Q-43m?8fwMDl zlcPwm$hpQmq$IGaA?=eA$60F_05qPqbFEZJC8UODqR~JKBEmu-X;haKD#h# zE5eV9?zQ$xL!b$w#5bg`@~q)E!vewLw{VR66!f5&EYU^fKXVVgMq^YYW=Ua-Ot$< z*{lwLp&4Tqcc=hoUFZFOKo5hd*<#;50~Zhub}q***ZmP}ntbhy>gFQmVBOY#HaqCY z`j2J^X2wmg}nqa5}oYKX>i%J__EFQMdd_Q5+gRK8(@z8Wr+4_&R5O#lCjsJl;rgbz5v1j=wX6wf3m)>OU~oua6(@iNGA= z|Hjz(&;A-BI~xB9Ye%h65hD$V#(%(L9jhIiV+gL%mBnI$jsI+y;efO5wk|g6*!a%| z8s3fWekkpxRP;lsFIuh-Ri5*N3h{^WA8@sE$JKRf+E==0Du*yb<3He{^J3$Ftne}Z z|DbKPTX2dtF81kvRyi3-myk(m+@jCt%Og8 zWi`+pBb+|&+M%w&;9Z=j8)U%yliiFYON=TOahjc};n6>Tn1^u)-}-8oXOEUk{H(>$ zw3gryA!Dj)Etf)!!9*@*(NsF_VkfS?x}T^j;|964c5PtxTKR@J#%$)TpdnAqW+3B& zdnh&=GgXg+Uy$g;yBI%VL$hzjs>U?9oz;O!>hiwto02y!i9l6E&I2qbR#fXq%}Zok zu|kfuP7-y0bpHo!ujt9q!--`Z@ek_f?o_eb$uP{n#>kW)S*NL_O!zdgJ6%?MSL$OU z771-8O%*oXLR@=pW4AYK+)q?i?4%nd34CPVaF!t!vq)o0+F>;T%FD;cCswog@7KKC zYDEQEagA5ES}`Vbh1g`G>Da0JneC2wr}VLpg`>)UqMOHa-(ZBp$ z;60s{&AV;%!AE)x)*GA#q4YPWVP~NG3o5{T{lw6L%r2G=3$gAXYQq~YTS5y=HLdp% z2UhKWdAgDjAVoF*M*pLjM|E-R3TnXRN*3KMRJJe?rEZxJmX2g&J-yOH@FT~3bVBG3 zC~T?~3D{4t6cd)QO9*up4DNI&mD-4QLbeD~LL9U?cy}EGO)FLG_6)u#a7B?#BXCH> z;6)1z@H5(Y%Zr#LN^=i?d8);E^CV~=l~7*^vdSu-SNuB@_wJ;VA#E5s-`h~CXGbEfE zjo2O}?~c|fb4OhwF-vxp*Mb6W9Gj32rV4$Mi!lL7Yo#%goGw z+e=T5u|akp+zpkCp%(@468*@8VNOJY!#?`mKEtPAa-$g!1q#EEF} z_kCts7?sUmwQ30J)<&r$9gaY7PaBnoO&ERtxq?#F7D(D-j@DKyLTL0G!Q-n<2=80K zjf9>*<)s9T6)|~Ig2oEFwUnTt%ttptq`0eJ2;HGWDhL)qNt3ArLd2^wlK zb16YXw!^Ij4Hf*_T2R183o_=?F-8hE=TFShRpPDRn4|Ly7M1^vIPYjped<}$ zn;2~6XN)`2iFEKBz21~@2mg_F+NJ!AwIzFBk7L;uAX1vorTENeD?(O$w62%l^P*;n zK2xpG>`422VeGa99L9seHa+vt%kPOcN$WU9(N$^8<2Xg%rE*-Wd_lE(kF)i{fSnSf z+abJVfG> zPBR59g%6QwpOn_NZY*6QvB58MMG$RXdsmH#(shAs0nRd5w?n?7> zBP*@WaoBSC%2#-F*@Nn9`+H;g?Ge7V=y;7|8*hv@I(*NjgK$loPLA(?ItVGe+ndl< zyznALl^{>D2Vv>t@W_wkXk)$)#^Dv;=dItE>#e|4>hkxOg}J_bw4(jiZ%n*}Q5PTN zGir*TuirQ38l&I(jk(?m%o%lj`HVU+XN>XIZ_GXFzQTTI=MvfkyhHW^Q%+wA<%I9} zde*M5FGO>2z<$Sw>$4w!MX|PjVUA}PjW}NUu&V8^G}5^noQ(2*#D2~N|tQ!~&+TLlmTQFhU@U!7<`1wKl-zwE_x4aEM8)L%{i6kAv z5B8?z+wg;t+g&Kaoaq%`%())eyVdLc#E4KRT7Iy2YuuKfY=JB+KRl3Pmf7diS8$HL z0j^ zIZP8P&4O8(3x#YTz^YJ(rNn^;29gPL#%3jq;D)o`(c9z3K=!|BVlokWsmeA$#0mo* zSSMxc{YQ!c83V(#O^jyg3HVzv1|~nE3Pdla7-q76v3S(v6QK6zVO0Q<$D!KJIp?Jc_PYQY3^+e%G;y3Y&nfcHB$!vOoIRFT(IO(NSPJ-#*Xr|aY zVcDHXL8WcX_r?^JK@tL201!-zM~Dswm6=i^s$9l- zUa(hx#-G#^T^X?t%Zc%9CcU6)liV#BR4|5Tk|H#wq~z75LkPI2y2DybU@puF1Wh@; z6-%20!}_SD0y6^R`HTe!mVFLJGO`RcnZNr*1 zLWa&0&FA%D0G;@=enKVxHb=)HsJa8hRaC<7=jJF7EHP~M+%_K!i!z^0-58cGD=!Rx zD{>3$1iMlO89NKg$%uWCrs{5hx<2 zyLP7PYOmG&XmIm4f+st&NW13x8}l%KJ)9W%&YwtjddT(xE2#ES17`3r$EiO`N_;Xm zg|i=r1??=(&7`;o&+Cc0zsdB5RQ1L5C$>NW4lW4B6Pvi!bU<^!L+tw}^-2CweAZ2o znMsZJk{M%hfEadKpnSt{(ai2Ulx7(H3A1g8F8?XnZfF}=IqL&&sYCgG?UGA>m^0Y! z2T+`wq7egba|$6OIbxQ%NjiGN8J|N!hcc7{Zjk^Mh)$tODWqby$|}f=Bju4&_!HAk z#jcYxCm)d}T{=%zPTKSAq$m!W6vGn)2bOM-t~6G4_jsb{f)XQj3$Z{PmnB{FkeNer z5logUejZRMUfya`Fx+=fR_Szq?o!bw==kC;imIKxyJAf4r?N4FJMq2gUr=4&Wzfg)yR@*3wJ088f=1=6rWLjG{JAnd|Ukt}Fgy#u`6M(7378HqO zu|8R&oX)MtS%l{5Y#ZKW6o}%WXDZ3Ox#&Q}FFc~lS~q2P(XJQ=bL2LEEi0k=2W`XR zc|XDSZ4|*{p+?phBg=Qj8&1+*V8wYQC-A+uM;B9N>cY(iDx%WwT@_+0K}zD=IuOBB zwhr`x8K8M|AYy9UIuPNMwhr`xv)-6Ha=z*w7bE$AbQ6aTMA-UEjl{LI6(Hqc=8KOY z+HAZSO<@@9^#m)Xh62QYL{QwETLEIK=sX&bQb*I)fDRlQ(7_aEf+A6%Yz>INK_Y28 z#2jYZtpIUQscZ!(l7Y7Z#92^c>xzpuN?*x8XrZ z{n-c}T0*4aoz6*CcGMpcN89QTe{rs@{t#~cQhzk2veh4sc^>_LN2GD7KO&B!{)jl{ zb^PmF9I4Nrn>iHu2BrECeDYC!xW;*H)kl<4w(7$@8%NRcF7|S%K7rY)4?psZt@i{GYPxO>9kCpmk4|K|G#_o$R(uGKt@kibH>XSQ zA!ekl_mGi1YwJCKgiF8m9<+_W^&WGct@i|c^qx0wx~MPIo{g~9!+EW|wH|SrytN)K z!>#minRwwh3jmc77@_nKer~NtTXEDLWrQESM@jy--a`xx8`68Y+p}-I=fI)&9E?K; z_=UCgp3T0t-Xpp_ijT!d!;9KyMOyY!yT#jW>n?a54By8OL4 zx?NiObL7rf2eHq;%4|XECq=MK)%)*-NlvF!fE8IU!S3s)?(Eaz8;eP2iby!TT}`!b z?vIF-dOrb4TGxqEkk|gA4D=6bG3g0Lq$4F z5efniEwA!_!OhK|%~;eYOFP7DVluk3g9zSOuRUh^@by3y8SM|$Ad3AH4Ka%v1lPL# z(V*J_KEL9U9~+@s5!i@Ex~5&0dBT-&wV;KN7s*X@Isi_Bl|9}|*btIa*n8H2mUD%$ z4;L4SW$k^Zrr28_Kb3?)Y@42-0;0}yJgt$-zjSARHSIwaH;}s0SQH!=Y}`&Pvqp?_ z(Y?<@t*$maX?(mXK~*b72-V7L5^Mueq^5zm#9B)=Wmx5wTHj#Jdvc0NgojbNnY?;wk4~K`>A>Hvr6$FaPA^D2gBqwz|&@(d~u^SBGJq#{1oi2 zF;VWL7}t{wK|N}U1SjZcnl#n*I27Uwec$GPh&;>0h{aX}_fYcGW}*F`;AbEGu9G6k zuXjm;uIbOp2#e=z#=^3ixQ_1EVe-XgAl}!2t`BvS=H;vqm5nJb(Eu-|E*<@>I)szs z$rt;K6{>?ot-*ZN!}~|SH37a+DZia3P*mpnK+=9+Kk;Z#F^FNSL<}-uO~Y?wxh9%_ zAD9-KNkaMwX}r!^FFxQ(ny+sIfaY+YSIzQzLKU&&18$^#{Xi^a3?UZUBYf`!UqCtI z_ud#YTxbQo*TYjXH3mzBVOtrhg8oE3q%i=$KtR93uqCpMd7aZa6pM16P!)vpg(9W+ z`i&wfWsf53i5O+T*X9^gT!}5dkS$t`qDw4Ae?t_ELie()0mJuTT)QT*>s}tLDs(+D zb(+FD5KFZ^)O#^=Sz|ZAC`>4N7?ZJUj>&KB?_@TY*L)ZE?t<(AV-u$ z!Oe#crYO+=NV~&Cbm||;*Ni&pg1HQ1*YiO$m&AopjBrTqy;8NfkkAp zX7&K^QsH|~T&WMmZ^$Tl~u3|P37&IJGl%1iIA z(1;M(LAc0l`|v{SEn1d5YKM?1@X``I$<`sE4uOzNd)>#@kNt5BvnY%f5}FB zxM~dBQ36@TnD2vW!{Jb@`MCapK-h1!3y1-JFAN2%z!1hSe@J<;N=K3l8toCx0+Xa# zR${@vCT5oLMCGuOC(`-~Yb6CnVCwSnEgeZ2J+idsp;DCI0H_ip`8T|SX!70a*Uvrn z2H-c?Nm9mQW5y%NDJ}iTqt5Msf1yU>$XRq(-v?l%?I+qz@=#HGZK6i$nkY#&Q&!jU zT7fLk6tvoNqG?4jvD9_UvxRrKUIc0->6!`n+I3^DT}J{;laWF#EnXPbi3p5-wh-2B zoY=mImyxMugW+ ziTAyCPInF59O49A8JId@6!;u2P>5CiM(c~noki1_-g42u5fJ0mo8`ht$8564C`*9l z##Y_uo>mG;(|uSNtE@Yo*_OEo=EBWwTdRAzTAV6V3Z_pcs}g5qMtv7O!)<~EVQHB< z$exz{ah2A+fNjZwLHaz=f3zRV0=;q%`I9ZIaU_80QZeiiK={!ZgZ=sOa<5`#k(b{H zAI5LL@B_qbT#~0s!s+3ERn(0ndp>ceN>V2ilH7qNXuO*rgYVPCqxe=@=FRuJai`;3h88MH2BXyD*$ozh4GHyCOb`GjNIw;vT zhlg)$O7HS*U1vcVqSq7h6pip`beYck+QJCQUi{&4^}A#?f0zzfcV+Gu>3>UqU*fAM zPXG79balt%Alh}nI@2sDgtcwWt3Hv-#R}WHxLnV1Pwe`MtdMg(@zBD<g(2-St`v&Ahk<(NLTe2sBq7V#|{~p zFNA!K7gHS;e+4c~+q}UJ4Mld}&nR;Hg^|nbdP}`{U(eh#RDRD>LxP3(@b86jH~aJ` zo`}8Xhv2TmkYw@)2Arll(j-Aa(D9)mA8?8@c@ey_=z>zzW$Y&{d7pVFRBqef3)6Iq04_SZN~3#NwzWMW$d_Q{Kt^4Otm?(U`$G*T+^ZX6~jU72E1C ze^%?dExevH-%qqZeu0U7>XS%VkrlvH^L!R&OHI1%<6>dnbq+!2?M(;YQCBdDULJX0 zHzuhgI0}Y!kzkY%tR`ZoJmwMXliUW?4-R)`F~4>5Zx+hx_Y;zj;(chBukS;PAJ+&H zPthKq!5UFCfl!ss{q{Iz{jBUnwK$7se@|55)9^O;^aL*csu`XKM(n;npcT4*LQ5D! zh!yq--&^1n9qF|xfoYUv^_!EQL}^)7iW6@w{)7aU?oePBo4aJQnALR$l-}M_$aCi^m=;|7z;cH^XWZx_C1y$M9Z+NToCYE6vW_1;{esRoAUYnqRg0~)AzD>t+#k`ZvNFSwtyCZDlC6MA^~hGJxdp?}K6!-&tC9o}9 zv6k2^S|#QHZ_x^6+H8xUXq6l_e|JQyjoE@!VjskRVJ@ERw|--eXvN=dN3_DwQaz#- z>zt)qv`T8=u|+Gc$91s1T-V7*{9L)8DEsKE22qt&_?)WRr16Ag6?77k)ti#M&1t

G8^(9zUrsi(J3KOxMBvNyr;5AIPExezYVX$I}jWYDYf0nGc?{iC5 zd+jY*?cLdu)!z3lS@E5DBrATh9?6Q8?$Rw>aR=uSt`gJMucKaMQ@WTz6@%GNR0*Y9 zz^ZJzehXMtp3W^`#o~v6_1+$_1K6&%c8DY;5Fq+5VuM`3_p_8+5c#VrCjBB`zL&6=l4Q=UaGKpZ9`mTl3f2{8MA>iJG=gJ>lYR|d7 z0u#5_>~aaj?KQWVEV$HcGNZ@sr6L^X_L{N{UDwwv63yfKnv6l&w--{0UzUM1r4Q`e zOR2<%Sf`9f7c~!=e~DIXZZu?`MSQw@PXlw#`t4rw{EY|!)_3_S76RvHLA$y z7v|amzx5k)-CimRaBiT&(y*YPR?E1O8hKw@%?i#i<$K8bz1nYHojk(!7?=F=1 zxmd$kVLuP%F(nJ{CuSBLSJ%v8gO95VYkS)UV(sJVdhd>|u9YPuF$2a$fYu7c_l4LV z=>5)b%znv;e{_8ANhs3cvn`2qhmc-yQS&;75{XO7?+JmNjK1r4|#vY zgN-C};waAyo>W z?knTz43Tqb35gDd$7|!Ij^LLy!D!ePir&anu9`M#sLR zCNOj|%N;cV%oQ|&sgjBuG=Y_&DzJklfN>ZFgC?>Ae zCK#TeZ_q)Ax%oz8u@W?K_T@BsESHTQj19TEw zJM7an^{f;D8%zOV0*c)`28)=yZ%$?nY6Mk9gIE__BPxQ!R}+P7rY#JVL;*gRi#N}4 zf15D)0=xNMPc+ga^DUNYTf8-uh`69hV2_Sm-z=*qx+AR5Jo1`

{mE(Gd89AfD5wgXC%e|lJgkclden;9!M-$no(FJ^_%0gwgx?y-TI z19A{wni`<(GY{)%tw-k~m6%VnBV7fm-X;kX#u5ASocCXqJ%G2#m1f^TJG(&Alt+WU z1edSk@KxHVYW-y#_a5f*b<*7`;coc1me^HOi z*33ZdmBSPG#3tz;1V7R?7KbviSSB-b z!pmbas&?6bZGO|^GzZ+HIn0|QT|}pLLWf16Zfp`)2U!W@k1EImuxejXpQR8RbRfd7 zrmHz+YAsQy%&`(BqCmI}gBTabf3o$eC{Px#&D2xzSlC$y-nwj=q=D}2mJ4>x*^OZ^ zu09~c6AR&PuC*d$KPa{ZHD8tmOo_2GTRY(CsP_{x={47Y;9>iU(BGSNFL|T@_h!>V z{ra&nyqC(;gWKD0Se_OpXDX$Dw}CkUt`XsC`YrYxgFB-ON_kY0B)B!;fBfR-%vzfF z6N-{<&RJIF0M69SznDrDmNSy2)FPTL1hBiJ9yLQ!eLAFVjelW|5|ZCqN_MO*g4e$_ zTF27<#OzmM9e-oa@5WpIz~s;_Hb(uA1&iodb%3y@?g(>W$ig1=pke;5d=AvUl`%#l zup5#YB90ErT~930ExB@uf1q*YWbqA0HP0AT(T#d9Ntw~45f2FX3I{C7CuD~M0yDkBf4BuolG?QwG_`vN z_(p$m3&6`O`0xwLH$E|esGd8<*NzE9F~w?oUzvh!Zms@_C3#_o1P11W1QtgcM@S%k zq(~ZMNFavMY$RAE20ny=kb;@ru45JwKD@$`-`}e;*&&9TWItUON26hDh?z zzXt(AfcC>`ozfdn?*p|qA5*g_9#t=w6%DLSwr&%ommLy&h%dg*xcC_{v zJy#T!b!>!wqlABzr3Ba>Ls#Ep>NMs$hnanvUzlPS`2JZ-1}Hm%QP~mgXcm^zAKRMA znO%N1!t2d&fAYLBT$_=uFBH{Tx_$%oJv|CC)}kor&O#SAXpwAV+>cY;`*b@#{s~7$ z!6fUSC=%k>EKhyW3DeucK8o_$+XkdojN<-v>WQS=Ks1sSYEq(~2M~|A1$`FmNGHLv z2ucE-*{dGrQXE^?R#n8;B6} z!I<%9Yoe-k?hEScEL)XS$i~(>2tCL4+ICEXe}#%6>*!FnElT5MPage3p~sPj_v>5$ zSr&u$g&+k8Avpb|0M*TZr2y3&#%U=)C{EN~3Q$cFwL$?>b)s_z)FgH0+!AF;Rg5Rf zj0kBdK!}34^&cS8@wFyY|DXA&KpPG%Xd{lYQCdWA{=i&Hi{JVO<|seKhTPwn^Ojfj ze>dj-^%-`DpO~W#1t!#?GUEb}I^>c#mpW8dh~^^N;>@>Dhl&-+qYjl73X;^JvMjty z8$ubT^3sOtvt@A(sy@mSh>Nf$Y5C`>toi=4+!Ub;IT+>Ag=z^)fXS)E0L2`Tx>aAg zP!RFah2CT>amWc;|GyBH(#I=(ui)_Bf5x4S*dZ*XZ{w@DfI4LD5SG$&DEAIw0VZ7s zs;5`5e{nL+bzItgRv;MJp}!12cK}O4!$PkHiKMj*RO2gvw8GzZk|yPbq4H4?v{KYc=?YrG>_$65D_D+p1+5fyM!E&R`MeTE zu_IU53bwGk!a=z0lG*4O*<|9dl^=xUP0s!63R~G2hpk``OM0(v3N@X$f5KL<@AC>< z!9>>Oz9^38xyI=jlJf+sKZcw*96@;mNo@NiX~x<+Y-R7xcE0U>-(f3!=L%clC$F#- z!0i>bfhu?H#uAV|OH(O+_y6 zu$3~06o!aE*@ztiu;2NOe>pEXAbMYM{>r)H;&W80DP{_1hzfs1ntD-TKsEXn-vRq+1f3qK?+QQ2jl`fUnIl=F!C+UqCiW_b+(2=O!4i zlsuxmg(9b@kdBMQf5xu8>Uo|gDs5i;LObh}PX@ME(k)8QN~?v;&%ID_2`km5b0U9* zfsA6zw=h{d@NjlnmA#n+;2PImWD;4S=xURRaougx#c=7*JYz?8krNi?yPi)ihgHhAZ7_Ef4?z>9VoxgOTQ6=2K{f1 z1j{jFIXZOVq6(L)sXYlV?`&4me_5`Wlzpm}D*xJyA<(N?=j-poLsZReCDlR0n##>A zy4R_5u@J8+SOI2Ln%MNUm;X%$VysoSU^z9eO6B!lPNr^P++7jy6zf%Gf}-50^6WZG z9}I*FSahq4e?4O{vh+NaZeAp77%TY(FK-xgNJ~XUdGc@pqr&#%fX6OCopjX1q>3uE z?1NM!#a~AasSM?MqNcbYnC}<$ThSSKxTJ64-3`mja?xL`LymlXmi)z}qEsc)R}SX; z!gNX@-~ZcHU_t7tucCx38l|#^1rZw58nwJ-;ZTKhf6fEq*}j!X60dv9i96>BwEX0? z72B7*RM~`)X)E$}5Qf%Sb}?MTdDn{yOpOA&N^HkDWx=Du23t5k!5eK4Db6*9>Eql} zssdbI_m(T}?k8M(yV!QX-Kjc}QDB#K>dC@XG{v;tT-Gz-!qRL1{i1$uPmT*2ZA<S_tG>9c9?FA7!#*XxNMhr%^N6+DWOe>+p%ahV`;QcF|G9ULA zj1cFjB2F{^#0+2X#l=mUU=^BaJ$Oto{ie8+9HmGdfx3B4lgaJkO7xpvPaqGcUtfUz z=KUKgHp@{g*Cyr6r>kQJymm(oW6BfdPZduJu~da&lBnh3JpCJ$4BsVhSSAncHLJ@z zeW3 zo036GcE^OKI?S+7`+8_9n_r5ebG+N7{<(;usGap81{^0|1tx{kqpFS463ZzF)|8%8 z;__&|64mdDBLpFoxjTX&aCH#lQY}hVe<~|}bMUe$>MU+z8j=|LV`6GI%Y@bC%`&^L zhW!LoLh)Q3BZOOJ2^D!4y0w_0ZackLGSb0*G>0fT7nsg_0Y^j~*MDG&o*Aluo&R); zkwI&^7=m{C6Jnz(Qa_@M;Dv2VnZYd)U%T!nP^amq--;!QlnyUQ?2Dv-C`*=ue##A?T{eIsIMhnfZoJ0(W)nWBv$;r z&%IKdcsYfycrBDa-t4Tu?z&W2lVE33?o5*QDeaU2eS}YS@uEZM?~Buy=TyQvSAjGO zW#L@XV^?jv>qZSsNRM4fW?Opff0lHQ^f;bNbfmlS5*>#bajSJPmT(lY!MCE zRxHV8PddgJ$-t=LtBSPBF+d2XsU)!7}w->_H--8&wKM}OH^o8LugRT(R_iPIx^u;HN zz#|Z`a#ZSQl~iF2f1_-B9l-4v4CrMW1Xmc!aLIFrR+!SjbW=1MN(%JBI)r5u6l3_> zaAUqN5-8Lfyq(ZA-k3EvM!&0f#4VC;(Wl%N*rjjSRH?cH4DB`zK!ADb78Y&fd#bBo zVL^4Ec0F4jpGKna8{lyW(=C2OTW0I{y@Ree%k=MrE-_f2e}868p~@&!1QR^fVT297 zj({*YiZ#D6CCan1107E{tz}g=7f!#MKven7pK95p%7lUVc=i*kD&Px_;_}ed6K-B2 zmxz739Me%z^-)L;-%x7VqyC_T$)!4La;Zw|+^`Kmv)oSy`tzv8f3TWoC3C_Q#)6t&>si>n^EW0===cB5cYb3!SLj<~;TjHC-n3=u8DS2Q zHTqFyU(c8OG|iQ(a#fz`j%zNnYuLNcUQdkprV3qGC67Pjv?r5)7yG1kWTY^x$cl=@ zgv%~#`ldmbqBXx@k=wskuLAx)@FUT;(Tto>Ag@}Se}^c6@^N{~`cTMz)>Vi0^}7!oat_2r-WByWIGs>-4+SE$x1QqXHn~ zMTt>Ee=w*em9U#pF^DwaumzOsiAJ9oe?y8<)8BgpY(ZV0XEzAru3P8-K?%X;15={K zkbExSEmF;tqYAzdTjkrzDk-c{?IQ9<1?WBoT`|`Ci7wkDj#jme>=7f`GOg;;gHa@x ztZiwt!~^GOnfaplP_!ihP5E$%>HLONr7rvRe?D5szM5j9W7~NkY&=1e{pN>l{du#) zrV<+G0U2$c=yHP)49%&=)^^^~PYpv2E5D&`>f@C^Gx+bXuoZo7ntdne)*} zIT$Eqx=>t3e$ejzdV~dRZVXcpUR-rpAh7qM-A5t9bzvQveD7C@wDrP zVCxaWbk8jsTCiZ@j9k=mfT`7`WOKicGTRxw4m)1X_Y+l&wfcfwG-hB=>{DNCU`7r@ z*ul6x*ekTIK^M2gEey@|+s44WV@ zm0@5TQ+#XSR+KbKzqrfDK5hFEc`II`)>5(h%@B6fAuz$bNGst+pL>ghfX+Vr&&29;Is8=3QZ z-Z%7OA?fRh5wHrVqxf>-0SrqLddo}DFugC)|FWt1y*;i!TY_&RY3Q#orEko)3#R%$ zVZ|vW!9hBXTCrtXTIL-Jj48);e?60QxTeYsDJg2m@PDe}+WPgxqhh;Hl&gd?Cqk(b zFA@|$Jwf9yipMTpNOa=>_J!qQ8OB^ZIjQr|Z_IU8_ji6{dURgLSZt$EIiP2WvyV7A z>T>~o-X+nJDw1Q+$t#rd{uR~RJv_r}n#=u!+FAovY1X{;5;q4e{Iqrv?t#9 zvFr1`NkJDyRtO;N7E$c9iKZB<063^n1y-Acwtfn%$}xgt`gBzV^`etYql##`**L4f z&g+3h5$Sv&v6jpI6VUMIH3&84IfHbg9?ilp5aDt#{s8$L&2v}7e)h+?6W+@&aU_t#~TfhRC(=B14o+iA1Az=Z2%9gNDfRp#f zmavjK6>kZPl|=7@$6%h<(JcL?aG`K$TSvkI{G?mNN(g}~N5n$;*jbN=1vuAR#0t!o zs1lM1f5A6H)*JuAe{2DZzuS+1g>kjE1uP7_$sPfVby(LCus{SBmqcbo zkee*K1u%STSAin=F+j30M^w-VhdnB#~Tu_?5w{8Fql&E{e)^M zYyk`&Y6_&@YLcB{l`L;v?(?uXD!2bFYIqn4xq%QDp>?B{IQv*>6iEDt=l9NLP)} zZHff!S=bZ_OfR=Ba$~|6nPk)1#z?@=I>ty0qujPg46_|mBxcdabu%gs;J!QAk?ndy z{_bsxe@yarU#3V_96sBjlEm}NTc^yUV~X4un<5hvhQGEc65p?4{JSEx-FGrYVu1M; zX-5v9b2S@L`B#&)O_3W8Q{P) zLhw?h;A&Q7OC*c_+Fc`>v@0TRGzW29SKdg$e`5T^6J}gmBmXa7f0AU!&NBg`^M4h& z0$H#p*CVs4N#4oZ|DY1?4iIrt?;A-#M#NzQg6Lt#qH$U-ywA)2Sg~4`4Vfe3q=r56 zV`4n4Vh%N z8AP9kxu@Iz^~7}N966?yS|nCXp>B4v%NcxLb~$We)*ttAn4+T9Pviho%U*n?5=L$v zh=bN2b=T#U$*a`-E0*GCQav$Xr01bb(DO-%W^Ak-<2dqpun*0S-p?%n)0y;$Jes+m( zD6{%7q5Hx8ix*sRXX)IBe{sdOln1*TaWJaEVVJIqWqvsis*fM;dwmj4l5EyFTwb4g%B3s`NX_}bSzA`mN>-!5%ZjUteKJX6?PLuH!G|Q zN7SO5`3n&t);>s;fYD z6N4YZUgVva*P^lR-7idDH2=`)5!&8IH{_6=Y|u5_cgs@ z!N$vSf{^i%pP1pt!uhW=qRn*2RF*N2<-|aFUbrRN_%NM(e~?17jJOmG>NqsMUg{Jk zQ7~Wng-GuJkqKJ)+G!{Jhwi%TD<6nAId@iRPhYl(!q@Pf_Qy33C#iOw!|27(5Hjy^kdXt)F0Tu`P(hzNt=j(~#-=vV=6|fs| zkVZ}NP$Ldf)D;$zfFlm_gZ@a9D22$|rjsTeqs%wde_piv$y%)1NTys=Mb(F_kO)4j z;)5vdG+iraN0V^v*z*>im;Q_+uo=r}4IoTxF@0I})R{U5g^LXzbR9mDvWLQZP)ijr z2vMOK+y0HlAfp$i^BI(3L5qz77`4HNOcTX5evGmMgY*KM1Dq9wwf9+RL?E8uG`RTR zVT8l$e>hTRSLgRtP27yx#GtAe8*p-fpne#)4fHDTmI8Z#f$WO9oCmvxq)rrbservH z2ph23?vq~389<#2jb{W7+2|ZL>GGSuM*|Y8o1+|-3`#?LPMuOlWt;Ng8|))t`E2eJ zEL_1g1J)XslAmaQwv<3^wA6gRr>y-5!&6~Je?ik3xm$M&7$Vm^{zjuYPR&rvfhg3J`OIFTrjI-ERSB*YLpw}q`&y2mdgB@PXHq;f>RlH;smXkC)2 z$QuMDRx$R=gv+{MNv9ac-*Ba3KJ+#%bv>BuYW=8d2(}t}LG!I|FITI0!f%Z8jqgh^ ze;YQI|NbvN3$uiika#@qfXTG>P&0kp6D(dcJ-TPu-e`Lu18YgwkillLWH~w)$y%$E z;Yx()I{$IgfBCo{1WD)beW5n6S?0K~-Gc3vrT$HDQI~>N zCri7SjlflZ>-*PtSS9XZ){NN@B;;(CffVLDVJJ_3r?-A%;w@ni@Uzp0g~;D^ze0TE z0}-$MKP-FNUfF1YQ8t=azV&RhraCn@8_h5?Wuu{{EX_8orCiq^H^k}%R_*d^f3#NF zwcfMQP*C=jjbc0ndxjStifxgNCQOotP)+JsT_Ht;%9I+`I)N=HLM=9G?RD^XwRXeeBJ)6r13wnaLcf$NoyX4_LzI$CU>l#!1O z{>xgr6h4)X<_M*up|);HM>BZ5f6~zm(^5Jb3M{8|H0<9i9SsYU($TP=Uy+VxV0)#b zq3Cl;NAoh#o{nZa(_iUmzT-C?EodN;&+M(z-k9dmyDVpEznjw0Y-IhFj%HJNQaYN! zBc-FEc=0P8t+C=+q@!U$Yp`Jw#VZ}nFgvBA-7P~p+It6Ck*+C(|9_>Uf3?1I$o{{( zcr}!%&w82_*nFGi0~30-Vd}NYThY!_!z$lU<2+mCQPs0wt9(N%n`@RgSYcPvfej??sM-~lnkNj&CU3Az>f&+K%4CxlZG>DT%1DwEh$BCse|3{dzNw(jwaGVT zw1rK+F#|4a^0rsw2D@l8`zz=vFkzGTV!v*ax37wqH!M?&A z53;@1cu@551VkDRgN?eQWvBRo8g>9WG*;K0-FkT8qZ0>A~_r z!Nh$3qG@VYKcZN#u6?Pf^9;myLNK(^z{S-^XrsZ0e|Z67GQ0A4Dz?$!K?@LK8&O`| z;{u`oNscxYog4`Um=#n5@hHQFkcSB`aHJg(8c&?uG_&gytXhl*!!kI(SR)O$sA79^G* zxZDs8e@wI9ECIaHJyTOpRE2C%2iS#wj=;vYBz!8c(O_tw_x~xdF)#{jY-%a*fsH6p ztxOuR6vPlvuTE#?G0|P$VMBtq9I6;e={4sIMJ68%(iO0o@n9FfdI1=h7y5| zSW8Z9aWPrY&jTCdoJL^d$HZ{nyS>ii+G$n_4DNn_b;@X14Q_OO4;SD2&-coa!ht73 z#Ua(5?R4vOc@|GoJh7M%W4TB;P1C*6D)-gwijyT8?Co?9G-OA?(>++StLMjwvIubm zf4ezwJ}_IiBY$_$KtI{3*~T$w$dmcHd6bj){812;^AWp)UYy1jBVhn5Yz-giunpy( zu;tXLL?ttWn=&43p99`5lqC#E9viZFAy)myW=cf!Hwwjs!;ZHY6`ughHTYpA4e_3% zbQL#j92V$%WZ*e6@I29@>!^Ck!(dcfe>gp1O$`7yik@mCb*9UrY$?QUe3L$u-!`<6$Tm(89ix?oncfmcBIlbTkRQ|%DoWv$9XSfSqO}Gtn_{o;xY)*i%ksJ75%c@(?N}wz&%CQSJw$|E7q$G}N=sl579EHOYKEOanh+47DxO3W!}RTHjQo3E zL#mP@&sJfJU|NS9+6~6OpOt4qf4sq9N*RWj3cBx;>-doD%MjdUN?F_YmlovRxw{I$ zf*jjDgzND>I34yqo?CKrG5bT4!N+v~)u)%?1<|DX{IFP$8@GGjKnWsHeYzE0>sj^b zOmPm?rm=M#KSY;a9?w7$X`dT;UI{-Zt3LganNX^a!AgL}2@g0_e;;A4Zqc9V zWcYyy1ts43g}I83jlGP2VbWV%)PG^($Ni-bL{fbc2C5I>byXjOYu0>#Xjk(wxQ(m% z^dkMyHJ{E_6QTKZRWrhmMWrI?J)K=gL+`O|ElChJ_$oP~i+nj6;asK1=*s1K@}{D~ zMAONrGm1e>p|j>%QhI`Re?#f{pk@j|Wjk^G|4#_)DsO)S+YIxHZ~DP>=8h|{&0xj2 zBCyS1RZCD{TW>m|3T!i2Z@^P&@b$6bh5G09_u>hK2puw5p+X94>+Nw19&CQSo7M^9W1yV^=H3Qe?(hXwRw+dGx%IH z1XgC=@U6dG{b%JTdgT6@0SLRH2sc`2yGo`jqRqggpTsX}$0MTcg9b=EU0mKZ=H3*7 z>#&u>x zNXjpq*Yd1}?W5=`>&!OHnRI5G#!ecsP5fM4MN7F-ngcfDT&20U?KNjy#%jyc9-%aM zb~)hIspG*_ngb)H8E>b;WcK~}Dokc0*tam54X~%EGn+zxe-$RPN!KhUc2>f6rMatZ z;Z>Tk8>ui^(6p51fL&>R6Fs(O^5}%p3`{CahEb?E7lg(>Dcpq8Y~${!Fqv)EO?c-- zvy5m|T=UHn=UI7Qf8COfWj?~tnWgNxfZZ(rJ46}0P(2+8&Zq7+Z6*VD^M_HPRx)7sU^FEI zc8@`EvL2gx^q25uc8>>^2Oc1JvybWaDJ27T`5~!DjNx&c$OP=8``LBE7bX(_`amhl zUx<8*3Cn(I_s^W~QG(e4=eAx~n$yxG}x5DHT0idlmmHUw3oD{T*ZwnSkBh(aHqGCnZHfW*pO# zFM=>9e~lB$1njp6C$A#(aerM_wu37@fkvfYkqOvYWguk&VsXDB5m0L%iGUxwL$zpa z-k}k3&^;jQ3QCT-VvxCsX!b?OLHBfah8*<8e{aZvVY&_yh;0q}gcNenSN8y6Uvu*| zC^Hx$yLB{_^YIW=O+ZAYczV2!x^z3?4Et;Z$1#McB9nSA?(w8BkaIn>E0tIv1A{` zX<1=EMbv>UgDz@C9oWJ?froqMC5-HJq!=JIZiPt6VOI2jtQ?Y$U_2#2HCrW3nFj%#bDOs6c zPMJI+Dz=RC=mTRbG6Nv`z+ey2VDy1uhNqd&FK0v^eGp(oAAE4C%$Gd+Am}op4?t&( z7T{NhmnejB(^eulVysnFZGF;N_1d*GH7%u+MEC*rgNK;LN$YP*Jqh6leLbE4f9#g- z^?3a_gdfzTFbY5D>xpW~aBny(n|HOMa*CF>cr-6B6RQRLN<6ojw|pV3 z9z~y*;N@4|o2e@R0aPPf6}02m{_wilXhnDvr}|c9n@ECk%#HihOTQ5Lo#CZdcm8Z9 zmfb!IE&5kJ5U(Vaezq5%baRQBf1zIWE4y1z-CT0eS0St=xE{W;wd6>raBIn#&L?OM zaVNqxm$;cep6E<$c6-U`jIF%(k`vVy$*qp=0XLYO>T!{7F*zy-cbDuuo)+=O9ln5&}ngoDv{s53j!(8G*vQ_DY3KKQXVpIvdD%kRV{wAIYp@3-;C%ZISoui5d39hM!b=UnB$?e?h5=0xaeturQJY z90`FRBuKJs-HYfcAuupX2po->XvhW}W=Un{8=a#+A~p?<$_5;bh76_qjmp9*kqtQN zN;Q0bK%N2R0~%OzI^d|#5+faORNaS42Q;{R-E2UIBlXV^wSHm-Q|aJFx1T74O`94p zmqsR_jR;z(G68M;f6z~u>1Xrn{p)4|j(WUH3jcnXv!+bI!KS+@6>v0V@+lS27PM-3 zpnF=kQ!b!k@J;0c+MYnIQ!b#vg}H&s1srIE_!YT;fTeVq#U)_R1`LGK0S7uUenmbY z6e~{##GI&B%BF0yR*iKe1KKKy@F1<(BAynomRW^N^+YEJe^oZ1!78P#+iKM2(iki$ zqbfE`*?@vmHeg_s4QTLEmzf8@!A7Jv8_+PfViG{-8eAD83Y&?+QX7u{h731K2((dFFMr5Q z_J~2>Vz2Jj#6tLYqy*ZksEnyn0=dJH5eWW3jbL$Me_wk8m0*|;MpbBGhZcgBckCrI zs1KrWU4#!5bSl0^lXZvSb(w zsTUR_ORzx5k4-P99f z{*6gv)_-{P%zp_3vztBgu!rsa%*RVupst3k)`xbRqwr!LaDdb4=NxQxeay2Ap>!f- z>px&{^=uq{ z19=}G{1~>uL`Lje(MhjxRi9miITTj(Ja`Fkap@${!|gG-5_C%mWI5VZi2Xray<%VE7Ph zC2UKLs&M{r+w%n@;yIA64A3s~*+r)gH-9;eFUF)tJ+a90On}EP%zIM#2Cw;p#@`rY zC$oDhC(%hl6bCU)oK;@V2E)v4nPz@Oe~h^pv#^m#CFqnQI3>c$VfLgPN>kT<$BmSX#79)I}? z(A-Z3Qfzc>>7esBuP!^YXDE8xdDKuR>&wO92Oa5&DSJ}qT3VTsuHS@J=pa{@7?bCY z6*{RLq;ZkanEX8$PiHSw&0=Dh2BRCuU>mbD3eS>74 zi(+yv#$q7&gjrk)awHc=^@vqHMSs&k47NG_jE`7Gv2jB?2|uI}OC{LeVEg*}gvf02 z4B2*&{gHlu-as}bF}5bQpJ3`bT)yGxTMg0=jE6-V2DeJW6#eXgv-=?Vo9xWkfx%=} z^v#hf5ZXAW1`f9u)}nK!P^|@%D5kF#o*2|q?UVMCg8@&|Ero`F{R<%oLw_`}upPF@ z7rq-NjCC3?#)B__28@^C7tB#&N1A+O3~_-8&UzF&;wRE4agQwS*`1!)^vN2GvvC+& zWs|LbdT1G)uq@(y>6>PCHW<>HMX|J{4W`t*I8RH=e|XA`|LV?i#`G7PNBMKX7H8Xw zNO4P={TWg}ak&C9-+#aO1SFs(Rv2hegn_OI7 zfkVbd=H}wysr2S~s(*|+yaVIIBrOT+Kk$Tsz<7gCg*60??L3=LoF3eT?c1+}pv|Zw zKS5ikWCm>C>`7+82DcS511RF*JlF}?DP%{BcU>rVHAQH@G3ng>+=T3=IT+vbA2Bq} z{74u!y^0|7OGy=dYA9IXr)#4fn<4LjpP2}$G7vA3T7_#MJ%8~a{n!R!!j^T5@6?}` zdU+RiFd5!wyKLVzpGmudL6QpE!cC-L=PoY8n&E7u#=g7VNw> zkVFNrRk0pEFDeG&`xjZ0)Z1|e=bTre)rs>8gFZ_Ag=tIz0o(yYS~eI8Vuw=mmv2-l zowoD_7hb>Q7izCIhc^mujBcN7YzSCl8^mja9}x+7ArY9a7v6 z0tKAX11L61F%Tc9h?+F1PTakrY&k?pbRBIhMr_SyIL3!#9JrEs%eZ4r|ltT$T@{CQ1L)kpZ^A`vj)wGLBqj;L1uZzsqC`+@x*cPO9&&_~?%37=I*tzU7*#b^)Dexmd~YVTV4$AUn%o4w#p( z*?D-G3^E!b5({Pez&|ia*3egeAto$Q-+vrN2SbCo%*I*BQ%wN|ylI|LpqtHX@W7eX z8S7B5=mhwzfVGfq@dGnF!ia1EmVi1!oU~w3s`s$18~l(y^#);S^?Ku>A%6tP zv;KuS)umg%F?aL|*CL-WvG`S;E>-JTa?HdSDl^<(kow%!I8imxQ_ z#o}QrLT&ALj+^^Ce-}dWp}W$?wW(_?A2~x*=Td9+C3Y<5iotV>`NmuuR|lqC|a4MdniY{m~*q^7-zlLUL3&Z5~0|GS7ugS>$_YK&85(|O95 z${a8ZcsxTc)Yu*(s&}@B^w&HH)i9N=J=zlv;|0u~giB+iB%;9@_ukZ$3V-g?A9!I6 z<$C*pCoi(%)d?PJ$(7x8fk~WeQ^@>{2}2COeBPLal5fIbY=iJvn}etGN~!xcyQhag z4DjNyGO6xipc*<@E?*rxf97ogH?+X8jq?D9tikfYDKrUNRKDnwYhSU0&iY4<|=CIWHFIK~uw{h1>KZA$G}N zBVzLz`5OyI#8{WngTa*?ibB-p?I}2IyD(|n;R-CirDeY{Y~?8oV7iVdE&L3u)3x;Y z&zxYcTYEmJg~97#u(2wuwI~K|AtO9A&|i$=C8~9&37XQ6xS;I-uYY!=S;h`cal!W< zQX)CbjT(yVyY*&Gbynx!!s>7$`d7W#pMUjGJyh&NSG6wiuTX@GiCf(;vNJ(%DRsIq z&u+DqRo#pMsv)jo>dd6RzR=Me9CLW{dMcV@NZc_*DMf4$#X>g^rjD4RR-Y{wDjY?

=1K67cU5W+_M46Tjos`IrJYKZ)P}goFTGo)wf7;_gNN8O2T2;~E4B=aJX!^j zv6Osb@{Jj10^Lis-8v(GF9Fyg=sT0dRM4iujaLV}ng$n9J%986Kq5z%i%7(65nZF` zIt1Hacqqmo76c?Bj}jUw*1@Y~x=9ujqeg?yq9PI3aGXG{4g?Yz`HAXYD)ywV#SM9; zD`>T)?y&)KWeBZ_<;JIMzh;Pu+RwhTq7ax3_n^p@B>(zj-g%%Ai_P zN(=Y+D>PJ0^M5)OwJvMqCx(mI?6J%W+4zBeVzY=r6t8WkMuZsJQdz*jq+;JK-i?r? z7oRvdjZCwXIX z`Oy2sbbprt8~}xdP?7_nwEoBAFzmK0GU~YlAmDHSIOdwyOJiA*e_%pWF4tK2fw|^> z8|~XA;F$E5mh~T)^y~A(MG+Wx064-O00#GK|1Yu>U;95W>)QVf9_L!`3|=SO{|zI@ zHB1n;H?>{+zrmtY+5ZirAezD5ZJA7esNv}^LVs38n*4-9Zeaf}DnN7X|3xKe-2Pt} zyz2IUTgtMM{XdA54_TPjwkPc zy?-#un{|xpJ{g$!Ov5eOaxS2{UCA#BSn&<8_)*~dy1@v*l=XGp`WM3>yKjEA)40sm z^rN}_N#SBtcStfC3z$dJA_SO$u=U^*Hq&do#?r7lLp;$Z&0~X#fo1Oj9#^`vP?Z(eH4ppMQFLcbWqya~iS3>F($L?UyVt(4IXW7rE$0b$bxb>ZrrMm|U%Thl0oHCRvn6NW~FroDQSPs4Vm(}=GG zHXE~?N~FrcDxV>?^^=X=+N|+%3<6jDG123rcU5jioZXMH(=nI6?d91rbOSfxQdd)|ijLu&70I1bl`oC+VX$;re& zf|}$Opk@vYIWfA~JP61+p{iTmCkC!E`vQ;Aq#Ce;p-&d@nW|?8^=D#~Fq>0>qgLc^ z2*dhG-^4uUjR{v0QntXfxPN+3&fMQIF9{9U)onGgKa*AYgW1jIo+f99Z1IwwD9+a* zC7r3tMak$KRBe&CuEA*-!&vLdvU=;4acnjVBl+O_IT?@MB_ zr09L3{ZZ}?6`_Tk_^b%2xkM;Jt5S7xMabc+2nmzajP*Dveqye=;&1)JB(-OuchfIK zeur!N4@~;){-^#65r0Y$;n0Cr_gjui(CTj8b0x?yCn-VJ`XuQ+F(mGBEO98r;rTcsC8!_2MMDsaU#))Zpwv z^(kB*t6`LK1%Lko7NK4*EtaWfKQ7j>T~BY>^}2z57vv~TJ9VAOzMC1t{dSkM$iY3< zBA4sZc6xHBcTiyW7hV*}R-_Nl^J33J>*{G0%1dCr6DGWZ@z!rldP|%lyej^kYrJ&K zKF+#_%53Z4Q z48!z(m46HylT-klHp_yBP!dXpVP?BpMrC<&S2C)q)?Xz9t1?2#s2<}jB?ApodsZ^A z-Tf*Vwh!f8CBybfetjt!C<1Y=lHt4UT1LR3W&}ohhEs#<87woA^$gHhy?Tbh$}Ge} zJ@T!Eo`Ll6t7af){aiHzIqO-=z$v&&1{U&a5P!DJ{U&RPoMfEU6T@{1ouc6yg>(#~ zhJ95GJG%W`6~m6UJ6`;SSQRN0_ReXanAWd~VLzg?NbMy$P5jDSPMNEoOX2E)iB6~kH5RWagMrD9-E z<$tOe81&96MwNa5{Q~>BL%*n{XXV1yWVEYY#F|OF0I#T7yRc<%=V}+WEz;E4ExVv{ z@l`HtZ^JrJzAmGea`AB$@KUQRbSMv)MS>tJ>4(D2sZFncMMdglpsua!r)9Qtgv`?h z9_7b^&srcjMmfK0@>Ds6k z$qs1Xu8CP&F7k_;_s7FY`9Loo&JXK!?W{(9&C|6@4U&1o03GCfeWEb6dP@_TG($C#jOk{38l2dI&B+%t_=qJcV3=qo9Ifu zY+I_5EY#Q@T8e0)#(tE0)n9}_&bC}(QPcnx&=x%i8V1G1GUd&-i zI=tM$F_%5e@Me)c%pS&hum>~eofmtUXg?F&VFv%;*;iHY#2v=Y;yl({3IVwT8y{!x zFv4;{?l6=|<=jDV;tq~+?tmXl++kcjR68~!lLFjfY`)-Ue?B=^M)UR(8GNEKw8Oc> zRGdq22M9cgJ2-JWdvN&jhkrNue{<8H^I{N=5rde#B7#A%7klCllYwW!AMC!*qSAam z?6pSB*`jMp(~|$H4jOe5e;CtcJ~9Z{cM^l}BYQc-dtT%a`#9ye+PWDCl+;#XZsJ;C z8l&j}SIK3S@b-{NuJUFbLuGseCQGGixRoU}SeOQgCXa&2a|S;yuz%u(*k{T0ZseHa zNN(it1>F&{;Y+@oZ<3ybfzxw zxUI1L?!nWB;S1Ft7?fVmc*Bb^RgOu!wYG;eeqhr6ImV^rwIizAn6SaBx1y3?z=+y# zj@IV2%;Lj8@N&sO5r2e+QNajQ>M(eO#sN^GgLdlLpEc-vxjVRqCok#= z+UllK59?~qfRs357*E*&|6bL-97sK^tN5W(>Y-!Oa!z)d{f$Xh9RJR5%!G0S3{7aJ zn;Mna*cv4rRJpR*Jk%Vm*$2kbJ*0?iWsR3V%T}zu`R7`69Dl_V%jGYWjIeDK$5k=E zx>)^hvOc4MlYTq?;_*P`A-mFMMAuYZXFC;MeFPkOSDuRP83lIkiu`sUrQO_*$1!fY zztb$#9{4`7{8m7!e%+xWyXnDN5R)=Uo?>&xv7Zb_7Uc&cWJhT$QhengV~&T_tI}oR z7?+F=3mJxFbbsWLzM|2YUYv@e5!*Lj(a5S(yhNizcVUP|vd&X3vV!q%x#%3vv5P*#5ta@p%Y>A zoJlTfPjinN`&U2RPlUSQKEWy>7X@icE($p0q7UN7_9!YguMa9sv|j+>C2#iJi&eq#A0K)L8lw-~xybY|RCkc$o-Ga(oGfepFnV_rP^ z^~sqOHrKK}xj<~+G51~#j1u|*%O!5oo5AHOtAF{17!$HOTZwxV%)#Kr`Kix&S=<&S z^c$>faX64#nVwUseJqjjkA(hK24yv^sKNE6tADoA-Bze4ng?I8?R|+Ddhc~!s^=Qx zT5W|TqWxMmSn(pdRvnX8^En>F^ntmTXnf~4#xwd2q0||qPuS4r`HGZ&8+`YOj^fkR zt%;*!1wNBOqOmf1K73|deLb=LlSqwe<#2r>rQfLO(WZO_ZN)KhI4+|P>LW$A>QP_{ zA%AEMfqfo9U?5AvGK8QtmFw^j0z=3l1-Eardd7(r8eB(amd)xJXLeer)KEka82BiZ zjS&Q`j<`vTN}K9vdjvti5kcUX>!NS9c`(r@=5D*c^$T+Y25j)9{R@-c(xUzYlYV`9 z0>~%EqX-<~VFZnJwIhrGumXA2)VnTs5`VG%dL>PH96_r*<~NSOFuGz9M_{lLu-L=7 zqD$osBQV62?Ub8alI7Ik^$P3-d8A-FgT3|{cH5&E8AM>@reG0LQ=^WnC;|gd1P%O6 z^y^Us9|SET?=-djFH?3y6}GQbJi#$uH=?0ZT3XF9y-KPoPc#oYR&9zlqRWiHP!_a!?tE?e#O}2VL=U&hP_2DTuHtlzlhEqoMS-U5S6{)wQodkjbYv-1N?=JV{P>`PJ{ss!=HpOdNvNSC!hItsRL3!UTaZJe#Fp@D7#u>ax0{x$u%iv`Q{*6h} z$yDvCUzq&X<7ep;lYYH?a>=_8y%R}IBIF&Wrzkh$x9XbACa@#FY5Ey0L z8mxm0i$e2)0#T#DFqLLAJh0jH#A_57tkOC%v$f5ExZAh60YNni+NN(UY7{gNx>AjT zwuigQ5O&aZ_YxhdQP6gAOMz+>I3^LyV$%F?%#Ad%FTbOa?6=E(27hCF&aF{ku%fZ# z+qdc1t?ESzY`XTZS1B-fry>QmV13^r1wZS1D-_sb!$Tbv3a~-_D=HMk2C78C#};e- zJB>uDQIIgGQDCrFc-b1V?m4BG+oT}qt44vr&=ggpz~DmGpc(~DHfXO=fKA=EMu9CX zPEph-FibDEMuBa{I)ArD0k(bQ8U?<|w?;t_LR2L9nGnx+!l;<1Y82piB?n;5H#^PV zA_az-yt`nwu-VE((Yd9Iw_g+~FpO#yQ*qp(c_IrHL_y6{RSK{tRP(xCP>~q~34UxM zZavUai^&-+I=nZMwy6 z)n5RUSc>I(iGQWG+%-RN{DaV`dwo0%Kst3-PBmMy9O(GPxjrOQ#O00XT^=X2Ctc|yWJGCb*;Mwt2{JY%>JlhisO)h z*DxNTXk4hb@{%cgtDcuAnXLP?P(ua-m~fmmfG-cWsa(nU8Gy#_K3F zIfCK=8h_W_xQqQ^(vfh7t(q8F_8cuLF5H<_OSF-KeQhLvgU3vKAK!t5L4r|zU zKjy__kIfE#_aA$`fAG!mGEVls-A`yY8Zg7HZGZ6$JHjQ5MW?+)`ya8_qn!fNgF)!3 z>&sR6cqVF8I>u{r-nUS%`~A#onwU0Vu9 zSuA~y-pD&_fPdMSc=M?9wlIkcAY8#{s!dDELsu}`YAYMc4*pzziSby^SBVF_ATLfl z!GC04q?&ia%AVR14`!F~l=6cV%OMA0H)Vlvv|}dbCY$uh;$FWoX%qC;KQJt<@y($; zez{JLWyuj-HwUl!=4_yb=_(l#eZ*`*p5!3)1xq}|dL5PY#CmieFsx+6ghq-c$jMgEA1Q`4k z5zMx1Dh4Dy2z$;a5Ck0yKg7y?Ylatr3^G_?Fuqy3o_(`qWrY#RepQ!m(59}KiRs&z z)9uVTyl?9nmR&uu!_o`ccghm!T`RoZxc9Y)r8grGJF(}ziKg7*-*P7y=IMd4xqt5N z1`YP5_cS3DI5k+=e>zk>(;dX~)q`^Q_u@JH6xp}JyQ9RZVN|aW z#hAK>C#c4$!Me0m`|R~_$ja2{(VzUiZ5){;lIDJ zX`+fzD1p!isTi{`-#S8eYk%GC+WrIA^6f)#+qe%gxTq$P`0R zTzcYhgS@`*L@D|at0#ufQT)JVq)|GYHi#IvulpC=zK-9R*}jfZv}<4YQ3mgHGP~UM zylYBeCCW9h`&xy8-51j>?0>%4Zejn}4)!-K-K~F;vjQVI%iyuDoMo^wjWsoE+O0e@ zb|O=5ipyEn5|WlOq&1$E#VniL@<_GHV`h^CFCsI!c>9%FUdls<8f&O znMlt1AS7*a-AT?0gybv?RYI1H$Ng$|$5nci{ng5I&$LbOT`$ovnJ=J$eYrxGt)SkJ zkcIu;SICMrmyi{^5`RKgP(TP-cxPQ93*T&4$O4sR8EZU|NYuHH*{g;CeR8_*8?0RLr!lO{vapH1eNUVd;ovI~r)TK|Gez5W|lF7{El#B_3GSWYtdd`7+zuAf1{6A>lSNPNuid!EqS$BABdE==t0I<*g4nCn^%j9jaLQ_%8oM|-$dhf4RmYt>-+2H^BEc<^x3 zwSQ^|S?<^f-n`W{388y>ls6p|hN#KCI`(i&l)5rU2)!SX-J`V^pNNZTvehZ=+Sk20 zZh{wk5pTuR%e`vw^9ovaOj^ygy{6U&CT)~|=QqaPtA<#vmtZ|=L_@F#Uvqn}+^aTS z1+GHcdgshwecrY2#*WxmJ+b|haIlUY1%LYFU^R?n1E}PW*R^A(W{1PW`aykcy}DKO za6`-e4~^O#_6s$bwH>zJ5Nvlm zA{j2GB7`Wu(3s5@zhT+x&`Dwx>SZ2-wBo@(lZ49Ul_|_{6^nd6n~Mj zgmR3OVZkVhZBtjz&i+7k`is>g(u0%8@AIDb>v2mPo@xs^#rWhYxm*3qXrhJ{9jL8xGv4AtlP zgD}-Tw?iT`-RBIp&n3nJ{j(xpns**9ti9WYq8_p3 z3nvKqjzKh)&CEZD2yYblYJVGxHT{<`?8#-INmi|C2!$i;u9eyl5wbP-{KL@wHbgNG zQvh3Z-atp@lAmCbkg);u2YZ-q4d?jr{bp10%^8kJSRb!eQ7nyQKcv2p5&hhN}41mlh4=ZQ~Pw`%=n5r2#~X7m}d%2o|$ zl>L*m_?3(Hu{&(FBL!5Xi^jzZ;j)RC9W%hIfnfqZ4zeA~sw33B`R(9ACl9uvS-TSc z4!e(5MpeZz{b7LXb&hL!pLC0g;<<*+Wx7eFmdWnI2e5*_ zZT1i6!q&9b2ZLLn-z=yvg$}-IA5HzCdS@o@PI$sdt{6_aqbjDI9MnSv1y8=2CM62#P) z-Bp`2y2cPnG)XdgY;-&YF*PRrCvsB|*L{MPDkC?vPHuFa7#UAebfbqDr^48Eqlev& z;|pH6(e(!M^ahN0HvLB7_aB(MME=$YO+O3XZ?H|~<1^I`ZG<1uezb}vH*V_T z))=o}hX%#G=OAoV(n_RneTPeaf@-pT0IljelNc*)z<)`Co!_K5)Y*d827!)+oejak ze7M_A#>9O8B5V@-q;k^`8in4gjY5EBIkOj~)l!^s!+!|_iM_ZN;TdQF@PYVviJ67= zVe$0=p4(V)PA}oR6UM{z$U(`#^c7o*{}*}GPSz0D#mjbO2L zo~d=1=%TsALVG!=`A2G?ciz^pO{_*^%f+gb=zms@K{7?8(YU9HA2Q*xwKC(%&6~yG zuUNeq3KxH8I;a(`V?Rh%D?_s;PI^nyo~(IlFci;8kCpE>+YB&H(s_lrjf+&O}~2*&@FB$i|~tG2JwrB z+kbJip%Q+q$~X0NHSlFa6Z5VZy#<`8h?pv0mVJSxdssNv;3c};%eUBt8&c-^Q?j_U zAHUt5w#|~&@)LzJ#a*e!4|s20?Xn3>pk-TX-h7DQ{n#EB&wna~r~_jNI+~)egb?2u zQP_5srFBdiMQigTKCq(W@1ZZi?qv4tB!3GJxmYzj&?irgaJy?a8$G5EU1N24D_zF3 z*AZQ+&v5MpC+0{$@U=QC-?w-{!gxG!T;OGs>cfaLv_)ib@aLh5rHH&A>HQBuw#|8p z&%zs|NE$Qn+rbXz2ar{NE33QM*Sj+|Xg}=chZ`{`+hljT) z!WwLpGjX>=`}E~*WeH;KE}FLEjE;j27w738RR7J* zE)-~X#W@H}Wp)Y1F{P!8fGVXKWq<8IiZ*LblMDA-@S%giio~c6%c|W6bTthYw0&Sv z#4mV zc2Vw?6TiW4m8(1qthi6Gj;5(xXw+m8OPqx|pgzM{EDY4;VS{xw_zZACOT-JpY_ykn zs2-Td9R0m4^5&50)Pm(=z=zIFdA}5?0JB`G%fwTf?gpXR>DVCT=n6VWgQiT5BFC?B1bJu zP%>58-(Unrv-+#ppFu7@)QHe3`)sY04*2}TS`libC05kpfKR97sTL$}W|dhJ^Ys$@ zvJ<3xwn**t9$?0L2!Eq2pDoVf8#;tCQa?^6rCI;2AYjqqc*`m_5TDHqx}z zGyI?#RtXn1oq3ooWci8c-u5t+s(R-uhgUR}b_^XofG4F3-htv|Bi2f^3vou=u=XqN z*sRJ+yfn<2`3V*mtll`#S@b}S0%z(JVcCkLG`FOX$GuL+Zl`sF+mmeT8(ScRn+fe_L0Sp3NkRw!27-aTS3|~euWt-f%PRFJ!}Ru{ zSAPsg{R&w@KTInKnxY^g(+VPSuWlmDd|6V=$y@3#%=;}w>wYEMiecQTQ1xZr($}~W z&xCcwzjLiC_J3Pk4cSS2I9KuWA-^$Wz2D;0uXQC>C#)+Lo8s0VH6DGfpn4y%>R@MK zf&YjpJuvo~)ybqO2MUM5gYWR(-VC3&!FX>iYS@M`J@5Ve(tm zmF7>({dIrqUl_BLX3U{q+Ix0&DiG|*b!a($w$B@m`l9B9i+(Cmmp9OgU;yq_{=2qP z1vp}}tA~~p*-udYUQZl;0rr_7F_WSA!i(lehliHGBsm``fOlCP6Mieh(1_0_!&cv8 ze1+Sp?0@4XF}ba*v62`jJPiiBV*#_>%n3+R?Pi$ieWJ^;#dhn~Zt8p`x0PX7beslp zw3|9tJr6kCR*uPp$l$%-nCrF*jNDd$X(-tIU1_V~wklG$)!mDI8Qo`vPFwRU`K-{? zO8)<$H#TAY;B*g^d{#x3J=|wyt6B0bsX89%mVew-M(-2H%?R*W8N5CIg3Ej4Ym?i` zNI?DwN?#}J2I@L2L|RMH+HXuY7IW*X!IjomZ!z+?MIP0I ztbduuQuq<|$3xX){PDgU$K6m2f}+U{B-ddOzl}^?f>fNaiGKkZ>>GFp&ZZ6A{z9h(X_566mv$b~H`CBs zB(>Bg8u1-tQCJ6;KvwSNemj(s*OS`e&oUveoI9Y)HkG<<(3B zZY+K*Oj^y_+{E{Rxd%JG^BXe|g8$8-P)uTbDOM%>gW`=gaDs9r*vuMvHRSy&&wm|N z!K!HeP!+Mb_5QuF-I2YG&<3*_$WgGP(GAvRD$q;?HQG>RbtS_TFr)Q>SKd2 zqsjyHiZU|9mI4fyXIt4|h!gvo!XXLj&%L;>W3$_j^gZ=3s?H8bj^b*zj|{pJTyICB z7vE!Nhaq=vSk0@|BK%4Q#fyN|+<(B4tmX!D$9`&~x(?_i`Uh=4vYK0yk)EvP)?_5v$nnB(SGb;Ffn&Ftqc;tI!rSW&6}Okw z+?s9h7H7fqu|4v{TmB7d(_uGvL|D!ZQLg12?QW9g9L+gimUA>mNtW|EWPg^Eh-C2d zvYgw}?UtW-+*HWY*nNj!z-zwt1x?N^=e17{S7d8B!Pjyg7+KEog=9Im-&ilpxe>`p zmUA2BNC!dHM(;@&!xaYpeY(|VQ{Y=X31K<+MnG;kce-}VdB9;g|DgPDbt9sUQkHXI zlI0xZNS5=eDyY|TZUZ0Ja({07ldoi(=Zl*gVP}3nN~lHX1UFcn`5Ml#l3!su$7%cu z)49&-HJyLViD!N(zG-=3&%jJJ-8EPdQLKi(3J*_r>#a4jI1NmwYp`Q?5Dz|Re{#9px+JD*n6ty(?%Za3C5 zy>Np@ljy-t%hnr*jEY?uCY!#tXwwWDlaRgjgsk~XL|Q2`xcEAohZeKd!0aB0te;fW zUb&}05e`j@c-b7%Dl!|KGz`YykH@p=2+5JR+@NcOAt4Iag?|-U;$UVK%!0UH3M4Si z?Ojz>e%_eMRb#;YlB%O9U^KP~DObbZa(Q4FssS5ZPDNuAADq)2hI(SSl-t_8qyslg zw5b|P3_mSi1b`&PoM;))2#%31%N4<OR*!da3#9)1xFu2j zuZ{=kxN5bIM^sU@S~O(1YPHy?IH^{`q;*`wW5+%)*C_38{l*|vYZ0^EMy{S!2bZPk zCY5t3Te=Bp)oh1ZH`yhJYS~(i&_%TPPyYMx52~5la0&0J(VT5bmLbJ0Hx1~G<<41G zP~LP$VSnX_g`eyCo` z0rLa1$cJEiLmt%o1g+IJ_lJmrYB1x!ONJ+5RL25d-d!`gJ%>wmgRc%v1m2)I<4x{ z)89{sN6asL+~F?=zhM?d8XfK8Hkie=D1SaApR^}yg<(qE;TJ;u<2w-yBNzU=n8rvU zJwsiMDLa3X70=csMS6<8^pc;L4nNw6$(u&$U$#}UU$PMQ z(`BE2&V?RtwJC(s699Vbqko?gIheJ`nO$X77yU^FlOfh|0v78~pue6tZlqx=B7gJb z>w~eiu9;PBt%nYrug^_kDe)#h;sbg8zl*Vu1a`cUeM&t+(eQ?&P$}kCm**go0huCp zbyqYEdK+0Dt1Md5uCqZF)p;Em<>J2uwQ7u;N~p8!E| zxitto3D`mn(vOyAagUy+8Gn!1ZhyVL-TWCpz?Jk7_|p3T-&>)D-RHGAj;U%RQ)M|~ z_XoC);!im1u2TnvozLpXs>7YtJvdcn-FTvSmZQE<-25FsAzytAg5}QcAO)w15rgc& z{S{xZplSTJe#10)@6}b@Ccyi}Mwwz(g2uv%(CWxpXIKc&VXSwhY80vVwSTM%p7*u3 zL1H7{yTRkr<(vvrP+(*VYHED%rl8iVXLHFE)NXDLOhK)&GA zVDGAs9eebSd$*OyHfMvC$bU*T=J~KR8RjHQkS%RZmY{&c5|l9Px@Vg~q2Xa1<3p=k zSb_}GlO@OowUZ_2XAb1DVF@zKOqL*n>sJ_pY!T+m5QJq(hM?A_KLJCK!S!Vb!j30H zP;;lB3_-S2GYmnkFG=G1tzRM_R49Wn3n87_8T+Vfov4l+o@4YVQ%E|$Fn5Kn%m9_w|I<=n4+@TcIx(#CR)oDVw|%ejpi3c`InI<5qC^-U=5(9xYmkTcNWQMYx-7naE%b z6Wbpp=M#x|M!q`w@#HLC3?iO%G4$jSP7CiBgTwb?aExCJegDb#g4j{I7y5CJ!V_Z} z>0WT>?Y$So3gUliPz$WM8pe%U((N#0%kkU67tU`7;}!FDJB&ya-rnkw#=+ZDQ{jz% zJq(%_;(8FL_uJuh6R0&LYtr0I@ivr^5?SAlxtZbq)^ALDOPK3=@b6p$dHmM3-}rZa zVX`T?Eorn5$n?L9sp{inuy|m=gZS<3Y#^!C}srU0+ic^LTnY z`^Zfdeem*Mfw}1jjkR9?fysNYJtFH9!x~?}9GW1#C-{0ilD;#P6jB1j!0oeD%>)E? z*m`6Gqn3Z`^2eVAe`@->SUOV)u!Hrwg#MByicAoY@|EwE%xPg!?by4$6Om{I9DA=;Ey{b3G z?li=krF~aimhYD$v%nSYn~KcxUsb4ty0@;!Q*D0=ecQ^0blGRsk4PhkDyU#I&6zLFY2EWHN!u@zd_z22K6jBIuW z4_@yO;`>uPuTxe(!+3BmDwsTBvgdj7Y26cdmWS`^2^KyI%i@_H(e0MSGdw7qs>B(5 z!Yzvqr_CWIEQ|I98TBAr916L~vZzsnW$}M3x6AYRIFZEx!%UNcf|XcbZdt@qUAoqC z59T+(vgnKLmc@_lVCSF$yX{SOG6c8Zlbx*FqZhJdr%HCP@?n9RXok4yPBvb&(^I-r zS8}eWGuhIw8?~VAt35x_Vb$}5r`Yg%Ta^wk28o0xkj%UZPflf?@ARfRwkayhxyS$D z_FlCr#7=8|_|Az6OcW9_OqDI%TVQ_<3QnNH7NH$-T=#3(W=}iS$#-^d%@O;^7eZ&w z!F}KfR}~Y%ju-Cp!(F$BayAx@Vp9>XBjoU{027#^Y&E%PRY$0+1mdkDWEiSY%w$1B zM)KAel(N&k1(OQtdmSO0RiV^}IzpJG%#~AbR9Aa(CD_8;8>M+>r!JhNX-%a{ zSsSOTDqI^e*Wsbh>}}RY)|_oA<}0cReL`D)qI#MG8;zR1M7vp*@0O%h^2qkMg%Y7F z0bddl#)}By3tv$~$bP~L*X@7hWbN)SHXo(Q(r={JbHN~O@q9fIb##209(S*;yv6Qp zwsmeYUzV@Wt>Kk5RR8Kq6~ctbXW{rs;u~y6n~Mjq)d-?|wXt&d+z>*|+=y zPgn2wfj<89*DrLo0*WzMLU$S=!6YgP;j)xocTtO%W=^@Eb$VElGSdZQ{!%h}baR4HB1Q?f}Mi2lmC z(x$AHp9sN^U+BwAWB6}alY^56@dKBU2I+9xm+8?x5cK>DF6jAhTtTn*5bSu!Y?S#( zIC0%WGfFO%z?JY*LQsDhEH3D|4cIxS);a{e_ipgiEH>!51IQ#;7x{wf!NHmmh2N)~_3??~Sin)mif(~6Qc z%+hsS`GE8}-n~15v8x*JMZHTz$r@aeB4qs#-{ohl8|2RC!(u^?E-Ap>3)(197A-$@ z@1cjPMdp5aoUFCDRDDgw$vP%4rO z>#3$f0c=91_|kCdp?9DHE79iVSy62c9g9?1>N@oT6E1(e;T6LZW~t;I__O`0ibBJo zHg3Uix!i_|=g23ArifS2X4Ax%gkrtj5(Y~n(VQRX9v&CJ(7p67$55gs7aNzAD9RCrF+d`IQ51Xx{*`Q20iJMd;AR*W0YTSs9cr%^@cYd2*2L& z2w(a2&?SHGE8Kdvw1lqB@TYPsJF{f5(URg|Ty@125moV%; zX4F9D)A$AUmAP&`&{%Top)=r@d-aBD6kV_0@cLTv>VbSD_wfA_mhWY! zpf6xzBDPIW{&NeLAWvBjQ#<{7c2EZg02Ry*va)}jXBE6S1^M*?6J9*SXxS3x(cOZE zGiA7<2Lm#?lS(df>S=#cz=t;<^^0P7C~jOOXrVar6OT)bji>+STs+!crZ{p5V&&!5 z3pm_*fstEpJmIy21=|_Y_3Pm`X}y19O!UevZ%_2g zXS9D@H+l8IEGRa}t7l8EJRkNP`+*VAB(I)rl^pc)>KSiOPQ76D+dF!`E(12jsw*mY zvV6Q@s=QHHyc%hpq=-5_0z;HrZkO$sqe@1?wx-$ypA4X+&+FLz- zSZV>KTBtk*tD;zY!BIWlG1o?BT{oZePfUNv4)NA6OcaZ^!LIcWM1Ch~e1Br@Z`V{Z z@P)b6;}Zt$AS*lDdi8jNw;}`8%1LpsUOnF6pEat-+iEFR1{DJh)*#6sYz2Sh zfKt;dr{bs{|3TO6g4SR4-2P*4%y%YMV4{Ag_s{hDp$4m7D6$WR)m&l3k?MyU#(UtP z1+wfADj#b79gkc@8h)Fqjm}N5TgUYxyfD1PoH#=pUK$q(Mrb8k(t&C%JDu=`B$arg zc%3C&$HrqkPetuKpQtd5`^fHgiH(0FfngF43?iir+AcCo{y-gZ|LX@PBihK5WsCIc zp*CVw?RPoU*tudY&}O@lHobbN_qx5+Lv0}y)lJUSR4Ao-Q&VpCtsa^%sva5`RSyMB zz2(s-hAyM&Ur@}xKQ0F>iig?|}Q>{(B zcBsJ`k(%}%kM3&z28%JMbf|yv6a^u%$99iIX>a$c5Lv|*I1}>9p#ewb&<{41nNm~^ z4XTXFp@u0^2ZzY05$;@D7QBzjp+k}GUOE&VeUn@C;MnY|R6>^@}Q# z9Hm1I15rYyLr1F~sV-8lUa(uGLj$AIq4)w#JC9zF!sUxa%B`v#I$D2o&nPb&+lQAY z6o(~ibt$~R*TkKpa%iyg;mx79XL^vkzcxqhdTSIHlT^iMw-BmEaehppY7|d)XwkA( z&E3el#`9|MG!zV%SbX1Jut`s>KRpRm)+KkngsQIWE*Gl$a2DTQ@D-~1t#l%xrAuo4 zjk&^)zx4}qWvI^LF28>e>75bYp-)Wu?Whs+fk{Hu#o%JINnXOOAQVIsXInDFmu4Y29*l9C%*l^J*4g@H<~`SZm-c6OG9DV;A`$XHk@v9Amh%mmB*7 zclnFPOmLKzbWEnn^{GGzb4yDGMx`Z(nh$wt$$@UD^;plPeA(&2K^Kr$%X>gWTNC_GmOJLmGL(z|FOWH0= zjAvWu$vW&_TQcaxYfCyJEvCrrHzr#~t!Zv~8$BK4Mo%mAkd2^kyMzD?ho9t*8U-&U#;xLdGp(LWNdtdw^;~MQajl+?xi-%AcpDmtLlK!~d+ol7wgSkooUB?4?DS&S3sA2KkEM9+`ES`?J_Ramc-^npQU)|Z_soq9m z@%$M_ds{r$Qb{*jJlA}+Y_du3xB6uP$=Tw$x|e+8?JL?z7SBb6OcqaHcei*td@Y`i zaf|0tRV}x8iW?=1=aPHQEuKCmw|FkLa7-3Yv4XI8u9@te;tI*$xt1F=#bcFh?gr1L zG?{;E@LbzHTGXwrICeI8ZpNb}gXezK%z!`PY~Br??~8z<`C%~xZ?aEcd zfLiK4UTTDW#(}{bRc~xHH+rZMY?yOy-5BfA{tBGGh{e+S-lSe_TG7RRM%QX~x&OLt zt??76>83Iu27~KXNd!B}OtN)0FzR?`rwV^v-W32kGzGOonBGW!Q^R2UZuWtvNe514 zhok@Z#S{r0`hq%$^EY_1T((xP`KCs|7$gk@DD>>pXe$gfTjf=^KcT|&taXR>_KY8O z4Sz>ODQ^Omm{O;8-wL&2zngV~ytJJeu|2>P4tysEwS*6XY2GwGNUAZ8@DgFw8*hK2 zG(xp%f6F?_n|+LBufP6(|L^}9U|h_{c(nj(-|G*%Q&@c+N5~gZ137;=3>x*DKgV8x2 zwYR4*KWES?{o+C~_=jW-wgA8%crtX=Muq?Y!~F0oRQsNV@Xt*eKm#aOq_;u}XX-;>s8m)b1l;p;Z!Uc4BeZ$QyMcD?#WNR_GC7 zRfZ`xf``E^i=C6K_$%!1f`8sXFV>l!nDI#)(-3c5Y{q0Sbk@*X3=Oc_u2y0dV;lf$ z(;Tp0rZHC?yT=}=46j@TRhNIP^y_I_BhZlP?Nf`$zgui07rA8^4o<#6jQOeN7e4?p zMfY<4!sgM_8#v$ifoF;b z7hxutDwxutsdpW3Fr{kK1a2lD`%nTYu?B(`-sQscWXTb$jtO^}ExblVr;n9fwGp^m zVzSPb<%yOHbIN~6QH>`CkF&vzYXmx&o2Sz-^q);yBbSQ08(eVkdBfY7X_a49o(y8X z69#o&i9lWVp+dgPO_~cn2D9K$=R+rD`{p16H_YN@p^)CdmoRbKT?uO`XT48sohMiX zex9_5%12{j-vg`D%3nn`=ob66m9b0s!b@v;F%vGm-KBr3PKJ+w_$ETv?db*$0Mug~ zb1OC1=*#Yn|F5n$IhLd6wE*|O6|z9Vc`&1GP(-ip=l{0r3?Nz3>y-ptW)_nKKoBF^ zb&EW+)D3(Bm0%Do<=s|odl2*zIglLt{#{KCa+GzI$%aKKjH?t@veH@({CU|dB*R~f zN6m?uk0yWn0cLzK$a(q&Sa&vvy+$~1Pm~oF3raZhJ`2|T9O}rf49hXpzLZ0bF~AE? zWJd;cCP|c(bbiue{rRmeXFWifD7G_aF?Focp5dr!w3lr6?=Xxfl+SWURKL!l1Qd>?nyRWH(f+nKG5r-RX3jS=Kl=)K;jR^LYSlj$3Zf3f)bGvUCo%C?{3Yt~uDA26QVc%%GS3&;LZ(Q>)qfuZQw^C-|E;@EtqO z6m!diQIy$UwmlBTZ*GNKP@q;Xg~-a4(~-H{9+`a>?@0CSUOi_-Qy$%ULk)+;`HkuW zKPFLan2SjsI0)@x)X)}hDB-x;`V9tFbjp8A@t0k5>H@hP%nxpW$>(w;VSTw(6_1!U zdBhu*N%Q{3Qs1dFi?i6C<@a_#jVidG(`K{i;Q0u&?8F9RU^qywc)WO*POjl?sJcQz z3}-JeMp>K}q%I!_@|K(5Sn(|v@jVSZBL=KUo|qK05*x^;W+el>RRuOQibRUtW-*V|w6V{7Mp7 zU5wfQtQrxk`%GXY1Tm}u%V{$dW-&sBy^v^nUsWKOJZsyr-)&VQ0tnY??NjR(FW|E$VrR1?h%;M6j(9&iT1eysSSi9GP zUkiD?JUuigV%5^2AmIf&{f(xn{HQi8{nB!Jhwg9l6bLTTRjM zDug8)Kk$ytOZsAVfhi^G=WM3Hv#}eAdNZ9_o5u(#U`T{deF_+gXkmZa2bRF#o$QRL z(p&gW@+wV{oRl8O%8}w{h`SSMa)WbdS5|=%b6(!Ee+cpOB0hq{;)2$7Ah=ms_7R*k z{A`sn0a&{Uj${r@7fDW)$e;(8(m+K}QV7z?7Rq7H1ut4q5YD*_=k4=jC9nL@6Ov(@_V+KiR6k)%kKd5w z-9B-O>p5*E9BEQbCor>4@fFNA*+sm(<6LFoE`7E!%oYFPp`}@}mP@I=#{lMc`Tu>z z?@>~3+y=Qys&u{?9`*OpPIac+f5EpPRPVvC&Jj(lI|QYSvSfeu#mkp$u(VT9dcy{r zEC&~ZMA(jv6GgWZQZZU~NqJ#~Su{@**)Q6^V1ds*Xt`NUDHwp!2etJO%{pdV3k4(& zVw3}&`5BS#QQZq21v7pi+p{3Hzzi}y3sw#oszqmpN}{w-KxU{Dyv_kuH6om|1SLTi zOH}{)pBf{UUq^ol8nT^mnac}j8}u(m*7H-ra5pJM)^{ltv***UC5+F?m`9t$H91|~B z{GnG!#k>xmTnwW?*dZ`m_GW#lxpA0=MNukVgoTOI1d{*I@Hrb$g?zPVgrfBC3$lwg z`u%q3o}c%Hmec!!v(C_iSgeW07l@4+OQd34Ysz}TUmqrK++9Djv#Ki{@bX}2iOoeA zV?dWV^1Odwgc5mv!dZ{;`?0=K!2aF=UB+CV0M~W)FT_Pjy?uny9Ve*)OtbG7_9L}? zIhZVa-+3zYd|hAfQ6Dt=lJ(G1C69~wYdSx?gYjOA|Hv7fY%?vnSzk*Ih4+|$pNe}~ zRhJnwXQf+tyGg1rKQAVJABtW2|G~sl*>dLp!cc$Y>#6$vt>?G?#iXU00?(ZP#hf{x z`EbmC4~EP6y36l-{(4I8@-6kN$bb3YgF&y6PEkUe9oCh{f=%Kj#a*avH&a7vk@-t) zJ%?Zj1_V2E`qerg`hYTEnnp>I^@;`GI;+&1HXTXQWX%KycRW-pQOhF6^n%l<+Zq<9 zid=s+p!$*+FJR8bdVd10cxEhP5=AG)@FaDRsK;!ocGT8r@0*&Anl52NLqs9WXqhzZ zsCE9%SxMv>FrpMI-5xmGUGiY)I9_00Gi zM637Us9cuZ^7D8zLEayTHs~>hSm?xxB!6!O{j{m^vZpW%hHoEClA|tJAD&4eqJZhC zPjr+1(GzJ31Dzx}<76gvFijxaC0KtZU0@gilvJ(vqtXBF0->IzRuq|$xL0PPVA!|}qsa3;xqLLy5^YdT|U3J7PCRx20)%Y=nzk9-&vS5WVl;71eE5EBf z{KcQ*Cj?A+`#r1HJX*c204t`5mn1Ho;sb1P(PG-UKC?zlM+~I?!t2Z^Fq412BESkY z8w}S2%=nP(&?wGAF8@x4Bc`oTnMUs&UM2Srge`6-8 zO9=jKb$}DpY0fJiXz?XAo*4QPh6-l+{O`B-L$ACz5hF@SenL5dfOiC<&Df*l(8AGw zJv*dSzrXg-9F*Q5%P)+$Jwms#dCXjxb-N&<3yz;~Xjt-=4wGJ<@mYV0ms9grF2odE zi{RW0*VxnUcam{k-|mG=KJeF`67%kU zM{gS2q^G=Gz$HFH9IvkOQ-!SnF)}MD@i(E>v{y;_y1-0_t4@!Iw_LaeiT7W}mnJVz#y^~u^rrqDc_ zGCqv?z;u7R5GF%3l@OSWefY(|z#*}Z7#XZc=+wzy*qYxE(U7xMpLn{U0M^nBokp$P zldI?nA38}b<^d}{JtZCf?TS$MYg+j?ws zIAphASMVKHp;=(?m}t|tANzx~l0xtcw)F2}mh6Ah%MCNsv7EzGBOLPR+sy1DYNk@@ zfqJ-`f4_$<#YCsLeh%A-!`0H@7oqLj5*0i**1;D!1hfz9yjzj6eR`F*4v0R_ZAz)c z7t^hJ10!wbVwCp`3|F9AZF5yPz@e-wd3ZX#eY(P1d`DLO3xFxEWE!tPJ|MX5T?Kp@ zV!eO0l5rY%uX;TrNO!&X4XW3~y_pC8BTj1V9ihNIzQJ(hKd2A@D8L9_7~57HZ62bJ zhME14TdY1hjyYC+ctoh~+`j{7M;%^FGAB+S+}mux(t>2e z^FU}X6v-$UA8@Oh%DNR zBxjE~^Z>vboB8pe_-aX0b<W2 zM6@L^!xCQXtUO1qTo>~sUleNpVjf|BUQClYotSLlyZpOc)?|{#JPh|Q-bw140$6{N zRRz_^Jbqznu{cZ)ai0t<-}0HkMe3?q40w##03CkC%UAU7I8(k)9i2q8c^>R0h?8iP z-xu-oAh@kWs5j34fw|Jf5B-bavJ;`sy2cP5w)v23M)RH8@kYc9+bp<-qgxuNJir{? zXTh@i3e2(w!VAwB=CiFQhea18ryhR@A*cc>fQN-S9~?3_O|{L&6Ej(zO!9mi>4L^S7!};QBSI|18>};=8nSA|UA}+%iS*mt zhd0mt+ZS#Xa00?|iD$*Mu9D37s>8uW5}P<3|E%I%zkS8wJOa$21O-CgOFnL;hN5V$ zql?cR55!%bC&dig+T?c4eYE}V#Fk_`jCx{V0cxlK%}Zn&t4=j=q9Ak*E>W0Zy`QlS zSFMSSIwO4dE2}Yt7FIETjhx8(j2hIJt<6(H?P6wP(_&Ivm#ej2AOkk3(z2^-{HKzCz=5)(KOu~QXIk{^!A3#2Q z4Y-?+kTk7K`~Fx%7mvV_^5HZmV!rr4+}%oG`QgxOM<(j&~c zmT^VnT9#vsB!$Fo9AU=&YVU~gJ8odv8y+_-KVZw2g&8SkTw#Akni{V#Bhig3%t(|W z)J?VH6=sL7EzB<6ONjl&8K%Pa%_|2ZNddUjD@j)B$5Ual>^_Vni?A?#Us-jQ?ntwh z%J2RTaJJ=nK^#5j7t*=2R}=bZ?-TR*VOpOmQ%^bf0v_0&aTFbAkev%(xh}3FtR>( z;BLd<#SEJr;tin>uo)U#c53|i9X(cM<8<%S>Epkw-_d_-)VfJouF6y;0E^5PfT^og zb!VL|u7TAJjHf-ewW=cOlwX|NP!G%}+&lV=7%my<3uKj_4zN{?>SF!pHz>XU@(`@c zZ9yDQPt5Y9C?V}21mBs_hOTTrkY^u}vjbVu4 zu$iK6Kf`}wdjJ{Gs*F0$d%KALc4}{YN0Lxw9IoKDVrkT#$xdW=c0GcItLn}NXc6oU zb8{C|iUUv8?l)vzz-s>;O@(a|K`pHHj&R@5X<1O+-gD+SK3yA@voNcvhk5b_7SAfY zUK0sRUMilS)h(NbT-rf*Naogfmuerr-JgDQDkpz_;cjQe#rXDC-DZ9Gh>x1agJst} zCv{6Mrkbo0WgiCS<-ST~uD(px;-;x;T#iLhiwIu;t1AAp+Ci+V8bYcB*{sb?cD5vH zRi>athE5F zbv>l9U(xfqF7W7NjSnwDGb)v)HoJS3Zgcd{^M zmFZU|e=3u*Al^$X&9uOm+?l_l`#G)kr!U`#%fxBTXGN=ak@Y!IzK$Gw3#RA6xBVw8 zIGI|QpBKYz{gK5_9hm;XJkrHa{fn8D$v=PcGcb63+AiLP8}-^2iz>lGaBjyqxr8Cb zv~}K4!?)oldKf3!JsZ^pei3Ybo+?<|?p)$vOJbHWGG0|%E8%FmDcD8@P z#;>ytBbrco8^*BAjcphq=(PaNG+D29bCr16$95Gx4m!|nc$lvc+txVxcBO}+jx;Om4m%V* zRSAXhqh8rT<483nY+twCC*7){tL1+kK3~xW+I-F%dvEWEc3#QrpE6kSlVLJwp;_`e z@BFRqyYC0qj91(im)gMMBWZ0}NlMcY9(Ji|Ic?kd41r>p3(>m_)S$w*eAqqv>*I;!33=K2^x zkNSNsTID&sca);# zW@2Z=@;l=VJJhuE8{5~Yn1$E5+D7>9gFP)&Dx8AwpCp^|V-#aGr(3%vNBT{@i%AJK zEW4_CVxeq^PlnnjC78nTpbUa~JSI{LnUp}0ukdH)X5fjTK6)pHdAfhn5t0&UaUSw7 zGAZE!PxR`=*tHNZ$Z9nNn8_WvhR&pYXKKpOlp?r(o|{j4c1f@w5M^fhB) z=Sy!TZvIm)E!Hvd2&G@CDC>of3=9k`xBiXdq=shJ&t}8(00}PHL8$|di4s}|*A`DF zv}URo1QU>!zFV1EO{THwHr1(xX~>~Y%+1(ZeHNf+t_C-U<$ImWad#{JPHwwTyJsif ze@^@CUyRd!7~X#-FVbJ?5f7p;Iet{IjX#N>;HXY4itnn*PA2Acy>Qv&SP?M$7taXu z^I{lrXN)pd?1V$Coj@+@nM3})znCUvf2I4FCk4uqX^P->_@-LWmvt2}9t8Fu7Q^Yd zFlQVc%Uknxl)Pt+95X4MX)w=|hg_t%DP!|u+HYN|t@eNZi?}kykNk@ms)I{RB$?5u zv7{6dwnC*ewsHJ|>gUKT%Bm3(1L1?n7MCtwLd>*yEY`Zjaq}qj#qbJ+zJ+|P8qj@+ zbi;9kq~iz~>tKE!Osvjb<9`rdVtUA=bKQ-Au{zavBQ6H{QNj#$;RdhR#bH*NP5z8x zIv+Xw7d(HFl2!ojiDHR!+;Ca2Vr$@_kUpXldF5CVS-F@eVw!y-)zZAVNFc)@bb_dYTQ2Ik3x`yZ7v()6R&)wJCj>rxJtqXXEt}sdABe0rJQAAVQ8SMj^!-xi2mC@vA8zl=m_?kWjCU7-pYas<<+GVt^yd%k)<#Ny{=in_A~%_sWWkMZSXD8PA^lg`KEqOwR#}!FF=Cu|@(o2H zde3i^E0b_qE!BE<{__W{o^;rPmWm!n{_}qacK06TKNGG#Y85pPj1CtsX34)J`JHSR zaAbBq7zO#CH&`<9H>TOL#BW5BZSlO5&msz-QZ3ZOZo?1VVj7Bzq98M?=3KgDzO@^- zxZj%+n6=pT;mX*K%eUCD=f1J*%Q$bSvbdcTXcC3*3wprXo>HJMRYeN4=*}3rJ5hg) z-s@nx^+Kf?bh|9eNg;<9lyveJZo<+GWnx0B z?b`lFYrf2At$9_M+v4xlO|ar|*I|Ep-p_WV&kSOjl`Z%8M#$KZX#rg$v3C>;I8%%S zCj8uhPl@`^ngQNcOOY$}K{)Mvs(m@&oo~F(LYFS7REfm5P*)PRb!0^lTgu8B=h3ZQn~2;S?hn|t9oYn!-XJ> z7?BGZL;GhKtmCV6VAe^P0#&k@_ViCI@#1{|R5!XROLC6l-?|LXPWE zM4xZMGacsXQ{Q!%r}BThOEU9Nv`T`QdAe3=M)3I3h#hqh(&XA^hx(a$jQM#mD#RW9 zw27JVyJHge>UtiTRem4-8MZp<<^j-DL}MBx{Iba~v)UOjGkWPPof>_aeX8MD{G~Ib zIgmGh9VuUjjzg%ZHUWpad^m`tNe1TU!O*@&L=Qr7qluv}IlX^_rpRPO<1fS<4I;jek$UkZ$!D6_CXE^SQ1V#4gR*R0B;-()LUgbRD z!)U~auA_){BsD6g?8;|OWOEtrFzLA;!BZqpuVi8)t#<$-TPTl=0FJm3!?@i~33!Wl^V(2id5zs_45zchA?_u*VN$A}kxU$SIpD?DVh#6o? z?Cfqk>YxsCaNjXToJqjyY^vO}#RA_|vrL&3=$?O$Y6E|b))UawB{e8tD@z+w&Rlw+ z^lR>~!%p{v)f=TI@l88y5Zorr&p-Sp_c@Z2bT+6$dLWkiaZaW3^DO92?3>oyBIU9F z?iQj&1JbnShfnMD@KZI6q{^zpB&%{4bHlQPm-G54#S{B!@PVweB-GPx85*(%d3yVi zHHbUbQB8f!Tm+E{m#A4iO1K4)luZSg|Y28@-#)2rK>P4MW$G`x|XM zQ)qwU(I|glkZ}Yic4KCcuFlI8FOWG}V9G?WeNyFK^KQLkL;SdtsU7li*%1G zyQXi-A-iz<99MQt-FqI{HFn4{9cPXGDv4N_LGZiF{=j8j?Js zYe-BS(KV9hd_~u=3KXIXp$?DaLgZ8Zl3ac3M6=}TTT`;+3h+uUV~*tV^G9;|7aqyg zMKMQmjiggIB$q-;U&%GHw5*U^I9{({@=jdAHRjjdee8dD zp9I%r2K6-UC}jJF8+!x1&?3RHLIj|h%VFnkzL;$Vd`t& zJYvidUxSW@D?8MP<`pet>C_3~Gp1D65KggfS9}fn_@5CB5qRF@rZkbK6qgJHZ0=Db&7 z3Jz1xQVS2r0;Z{eIpBgo*+ik z)EhpZf!3APPkfe-#q{PlewoiwL~c=L_?0YRQ=icR&*&$P53QXTRvo3q{}-?9scE6G z4!p9rFz53XTUIA7btTLKuH1}uV4HLI*9crKWxdL>=N5wQSq#Kyla%geZw#ln)diEr ze!U>)8D{UgHOpa>@0x_?4g-HL!J*6QVyhNnc%gwe;G;b=GrkSM{_M0y_zQ@sj%%Px zq5OrInTRd76JyJ`IdsJ;wTafRex^MKG1G?YssXGii|^cndubyT?fLDO3W|7Tl6IRl zgt2m6aJZ?q2B?2Lm!2;eGQDtX#(}+AB(XDsy*v4aWcz;Z{D~DVgf4%AaED|1cQnET z(ln{Y21F(yt~Sh$CbB8AUh?lS=9FZ<>~*S;{wlQR5lM{E$_pnJOZ5}Lzy%z?0kT91 ziex`kTVlqZ;|cNW(txH1hUz?1gmm)voHIT=S7P)F^e?a)Y4OUHP64s%2v3Mb1{=F! ztv|;y%aLiNXio~*?wNmH-x1nRzQGQip*Jl104tNsS<;Md#%jUU?LlG?Gr(St%ZLkM zXFgb`^0V1MioTK|F_v=hPTRj!Qp9vUHDYBusrKZ`!b692Rgsefj6~!!u&iK#B^D1EYXQ=aSM5IA}j4+OQMp&V%3-$>sZoV z1l4q@wk+K&dVq;uEU4vF%|_b^3u;lKq6m~=)?r?stBlVXEA_yD@Uq);0G{oa%ChDM zt|UZ1K}=J{#b zA~2Ol9AS!WhaNR1%z`9>i`&vMo-_f!7JKwjC-$`HDA5E(d zVzg{ZJCs`nFBsMlR`|w%DZ;QR8!ySCCUzdAH$v1_tQ^Aubr3%X@hna(^1qmOD;A?< zcua3*Qv@ct-O^%%IG)NR{55p`sgwO5Jz#0c-X4DosR%`CI922iT&$=&Vnr=}c-Bx! zTC6ao!-T&fMJ{Ehn64!;gQI}!gXr_}i2k{Cc!lWCOVOgwh!L;U8$Oq~EwY>{hQ4^T z6q&l|z|iqG85$xR3%|j<0mr52vofF#B{;PRQ&9CPsl*yEX*}yVSr1d?&iaK}7sS6{ zjDvp>ma^cYti{r`n}RVOU=>ixbCw913|C@RQp*3yfF`*MMMaa|EK*|es-$MMoS$tm z8jCt~U~J#n=m2?yqPO7@o8G810vqXz1S3IK(KjwSVEF_~fXPFNk5Fz69ec~fBrtYb z4GH;`*#ugoakH%!3D)a$=ygeIh%|x)vf6*C{KXP{qUgAIn8X-&3|mSo;7w!cGlv!I zaVVIAsH=?!7N)%iXDr=4TIZ4sI3s90y>DRr^ZOG@6F762Okmd(9}x7FV|>?TmyKNG zd1y5yejrQhu_T-fMN)PDqW%BUmV*tiw!D%~j2MAscwVq_DCqGMUdS^EFEl4Pe*J&F z6|$H%mT6~4+vB9U?_3valji^cv@H;olRlu zNzX3A>?oG-8Og57%8O>~2f8XpBtWJ9iZ-$Geex!O?%lHW1l*5fKQNl6Uhw&h;oetM ziViyPE{A?p=DzhQL#-zHt54V7{cL|0yPHQ1N6Oa@I~={WjdgxHb3>|D32OjfYWSbn zX%U!h13N@}pAk2pwj}}c30hd55sWPF;s0&CE%hg`q-(~$ECHOF1NCCMStj9d!@#M{ zIc|H|ieOu0sj_y;^LE@YotV^Ia}bM+m~%M$G1< zbiZLml6qYSwuc53c1JMDwNK}HG^RK3C(3cTkRr0WCcAbm2~ZT$G^xah4%1LCS_yF$ zV~?v{;@A_k_~@$jA&vD_N6;;kBT&bXlBL6uIpgDZBS5NF{6*51E)4m z2*00QY?ox6Q;;q^)TP_DZQHhO+qV6l)3$Bfwr$(C`?R~~o4@8_rsghHxkz%8?5thu zSpkNBW+JF#q^FIZ`L9O~!{=rzs+mC>f!Y>K)ro2HPbS{SgE{&=`23I0foIse`aqi# z)KIHh1<=36cz4HZ3GwBE?-DiRC4}{J3=`r$9qmajIeIYQ;{k-dm^I$5J1|hvtPjqX z@`|+?+XvOl&|b}7BkY4;dT?eof_k7C%bHx+uoWTL=UYu$sgq?uv;l2hQ4GDHEG>$v zi0cx8J7_!FPO#~G*Nsy?B_K?=qOneD!M0V>m5n3~-|073Oz%7n*UeKtQ6JokIWnFi zMga^En|)W_&j9pf5m3w=arvzN@`MC2%8*nK-Y^e}8;#H%YTKTsSgW+YeD^`*C049W z$kTlKcn7Y2(VV&qCM4E(92mF=#}6aiX2w8baud*9=L2ZCrM#?N=tB`RWYs9httBNF z{j`-tG8Hx=fqi6N(Zy$Exje+0PyB)On*cuQh_P3-Hh_pgJXuI6KR_I{_%AN(T+RAf z`As)2`01I6)}243#pQAN#%`~Gzkm;k;iyzp~yT8C8)y897ut=%`nDhS0}i0yO3A>9Y}~8Su>1PD_NRK zh{`}pBgNb)VU!GDI)202>ZA7J2w4cmMs0I zr978w=-)UI2avdX(gNGWjSXdY#qCgmt|ZTGMrG{a>5!VBDw^SgN=NXhg+Uv}l6NoK zfxN&ZXq+o6{4`s~EeUa-8Onj<+aEZOP1{!os-eDATWfI)+^;dpoj5x24B zZ~|=@S!|iFXcRLk`NHxn?g^D>h9oZ>m0XYVq_E~R_*vnPC5=wTT_*HdM+@e`gHbws znu0s&#h93#hk{^ttj<>ExW6aT0K*$f{zaZQ@b$=qy*#6sZD-2AH(NZZ zkC=XyZzz?*!vUS+Qe&@}FuCEu8*KZJB1M$z$JYiCU(m0M$2irNV3@>voK6|1pSXJC zK9*-&UP!A?P45C334@g+a$rZNx#?!o9&0U`W*yb^4! zmjCPr2VmwnIIepbf$%bM5C|8 z#7Dk={{$@s3O)eGS*N*D_1Qdb^mDHwW|yqq(*mV&#lGbbim4yNrXG;u0I9|s7YIx# z87tQX7+s*tQz$aLkWJ$;0vsf`nR3VB+WaKy6ZPD0Z2ySvt5}XCXq#UBh?Q1?r>ofX z+sVzJ!V;x2vB4pXyZ~9qo+o^($FPJc&fkUJ6$4>aA_J$paX0qi9w44UBY{=QaQd8u zZL7l;&EG(S3IOvkB4rLRoqyA+500ch5M>e9cBtk%L zGjyl3dHbcE)I-@Y)=A5!Oa$LT>B0t;0s#GhVI|jIQm>K&+~%Vop#*G`dPQDlK1(4I zeK9H+?M!8pLoh<4cHFgs#L04TV1^eY{E2P9{!5hopo>1v`Vbsw{Q9jgFQgdk03SR* zkhyn`SIvilSx~ur0Gmmp1XCYkx2$la;y4(-(5PaznTJiDsC@?z>td*FGxXMQ1Up?6 zRwfs4o$!P39+PJQ!Mb158gC%G#dGLo&g}<0wzxFx#|lkn+B^>lw6#Obf6u&YYcSf< zawv=uP-z9m{d^kYNd@AHI?(8Lb?PpaTjC72e7(JEm}~GmpaTnPZ4q>dEWndlu+Wzq z&7rpL%=p2N8xo5LF${(eh~Yo`ae7d>wgR+e|0->j!G7zV)@}tU$=!aINH8YiG9WJb zPE-<7**sN{3OQ^PFxV`vvK*H`4kDZ&U1FiFMWteLYrp zJBhg6BFmPd>mo+5h*3%Mcev!{$LR6iA>^*~L)9rW05EpIy@N0DfIuNm0JQsUygB%% zA*}}X+h{JFcyMJUQ&#(5Dcn*>)c#y-+aYknw&H*U2WmLqIu*sjL)axPP^@Ux z{X9__;JH%$BGB7#uu?hns`1jjMnC;D#V`Pb0J^^gZ{1gOb>P46*TQ?;IEv0Lr^Q*& z!Cfmu`>dr!*fm<;_)^uIYO2u%K}ZQ4@ZG}EQ5J&GW>sk5btJEMddF1^xY~~eGZC4=vXnD-ry!Pu&!|bw4&}X26DO=!nm{x zsMNwooAH|f_*bvpB}Y^b&N_2Mg$DMYM&Ta{7>0X@f^BrH(#9oxke0lZ3OM_i75pZ+DCcIp)V8L(Y!iCo+d<@lhPjNjDPm#OX%JtEd$ll3c}>lQiGa z=pMBv3-}l1uGXQ02aUR}+txCCziPliMGRmPJ-XJ7OfCW2j;$6#+*q@w&4AB3C1ji!6NyM;y96{{_=9OCqNk z`BK3fQiA0NbtN0=&SBpNP^peudj=b0{b|V`;Q1?7uKVZ9_qO=*J85;A?o@80T z0Gnf7|0l)R#mEpFLT4imE-36)HaLc!pH^&ry`V-W7~b%^77Eq}fPjqgOh9s-MJgBY zlqCs*N+O5vTl=h}H%4m+fzO}`ME!LEQ?AubJBar596hcR`2>;2IpVVOCUrEKh0!yn2$W= z=MTs)+!c%=%SF-vwjb5o&xr}VJ|D!a_wN@NnZtlL&$v(p$P(D5cOmK0hOwa|_0KeD z$;OOjnKpGy=SjC5S*Tqx4FP^UJHL04=`=;stLm{mA~-)Fx|T3|V{xLb>#K|Tat&X7lPLn`1beiv{( zeqez!48m#w`TyuZHRx+{&YVEcTOmw?CFO1Lg#pQqSCW}qV=PTE?DII9?Jt(xpp2&1!yt+I-&o{hH4^5QC$ z=BXe995`eiJ!Vg2Vf;GqW}5R1YZ3An2j zDW8ez>N@^lc8s^gVIE)!iHEYRGUOljiFZ}E-Z|YwkIe4p&bNulLgwJZoIO+|#M6gM z+H{Te%VWD0)LRy;xi2^RHO5w~oNA*}CkERFsHgN7a)pI~U~$yTRi-s;6M_bJPKGq> z_Kcwp(}i7|wM9F@ALzLU!NiK==q^Lg`=!Z;vS)DZ_k?q%T95g)s)rj z{i&J|_$_hiOmS9+Db@0P*IM=Cn<}fV{KhYEt5hN7D;mZlfnD?%LwUDT0*E0al$>V* zREGn>EfxySP~I!PVlr@&(0h`DM!xXH$@k8rVI(gzm?j%#<;gu}ny(SE3|nI*Iz59w z#FH;3L1|EBxpWW|TmRD7L=S}yZmOVzOVJu2ZFF0M@70}jKFLNaVzOQ>M%~bNYXccX z3lzHeqZ`s@Jq`-kfvP6?Zgv6+U=a@hlx^S+QQR5J85C)$3f%AIFUlzEHxTFd-SzBT zJhV+QC8RT%xmXKSy7=BzAZN}00!Vc1>tKCfHMrCBAeYk7K1e6c7`q^sR#L0Do!L6- zN`la5rx9+r9Td(|H-s6LD4J=2kMg5Pa9|pMB+NOs&4BsYGt_fab?Zm|;`o8C=*X{QfJsl5 zCuSveozy1b-&st&C>8QrX(Iz~GukaQO!5}KLy}1}?rKa~i|9W`P2Pvq{UNL%dnqnO zaPNJP{m*DRH^N4~ghe{C*fnAUIHK>G7E%EPwW?)+57Mo9wlr{fP<~&C?SQ&k$?J$W zB#qA+#5|{5)`t~5=uy#jt{l6JPbe-t+ao@yve-8g%?4eUqw0dQRILU!!m(T@Dtg#O zDnV>yIlUC%=c@eo_oWbljxmF?`8SFT@F|z#2$c#J zMoGL$k0+-rC~G7YsD7f+ffU6w)Y*=bGuG-z@TI&rK=sv_47$9j1ctQ$kwUM#L$PB=R~i0pm~uG zalTYFZ6Fyf)CiIkY^g~m*NW7ZU)5JPt2DChpvgwL+%Y^mMufCUREmuQpm9^oS7HdbF zrO2{r`!bX`pE)(a@X_~478g?RWx&u7bzB=Sjr?Y2pOpkv8Nza|#0VU5QwRhY6H^Rz zJE)(KK!^uXM#Dn-z8pdJu}y zmi8YCso$&RR(DUJLpFq}UR-xh1yXv!QRhX+V7R&ADxONf0GEf}>=y}g8l9xMDMnf9 zOJMy8Gh2-S`RcUA};ldI@ZP1GT*W z?gns0bBL9gmWt2VTi(i^fG!R~FuiQvg*y(TFFHPWA5Bdnur%-Xyw1tSfhOd%Z1R2b z*}tIBmw#&k`$kIo-}3p~hZ`b)=4(0Pg1^!vZ(1Z@dtE_hTu*h>neymv!CC$&TrGnhL z*lxExNh%b6RDS34q$KdRMUW?WAhK7?Z;4}K5FU>Il&g5$=jk^mZYxjc5nH%h(OBbw zBD6xMm1m@>JAL>r@oVhsd3bg9OT!ih+|HmXiOlR~xi*du&S^%nZ<5b2ype5VgI0TG z*i^RwF`mtZUQK~7yIdw`TLl7mGp}uO_IKtj`B&do-isBUoY&c`+C02{eOASgWeJz} zsd2@R`72(`k*Z?qr#JBI;WvW^vg`rv1rnjb#iNR`FP~@R>!^ka_eW2d-|pJJo^>c` zQmF<}M~1!Iiga?Z=`zUKbw?3o^*GouV7YU45Sutc^Ow_rMQ-BOxP zggwJlkjT#VvJ&LvVd?~u@5SW1q0>Oho^MpEnGjkjECx<}XGeeeN~*7kBJim+*xwa! zb`9kjH`16pcUst3WN6uNOdjRm?~*IBFRWffC1>97{5xsFR4&Bmuitde7Rvrlev3YT zvNlV6wDD1;W96cFg(*=+mx&#VZUpPXKQ2Y<9240i)TXk>JXbAfbY*O*+5U2JAZp%Y zf%DuMJy}yDba+@<2r9Odq0$=ThmQ>~8LJKy_syn;kth1F%?;zK7MkD5&)$k2EoRD> zw5RS~z0*M|7J&$&%Xe0#!U(J?1C{Mld}kF}!LnqeiV3=Vkh0gh1QSRvbFqCWr;aXM z1p&e}D|V$k*usUdLojuRsVh=BvF?nr@EJh#M~4to(KkAa*J#R9OV%JxY^MT%(IUD0 zFF`35m2CDxrfB;0mC;?+?{Qg!yv*$O!|C>u3CO?%;Zo?Vri=1ZG1*+~!}sp-YZBw? zRGSO8ND&sB#8a8fD&R}47rsCBKXgv~&ay*`J67%GY+iMJ$wEAW7ZVNR5N_L^Fcg`& zSC~Q&(WPR-MUlOu+#bs?I#v##h$7xBraAvwq`lb$xy_zJ@{;+!xg8li2_Eyl&*-`5 zPB-j<86qDwtn&e^Q^uXFyOY`dkP5V0c43%qp^Y=FfgjCi!fbo6zr%hiPutRX!D`I zS?kzzcNEH4Go}|~IG*EU1VQP2xah^)o|ow19PJ!4zpFOnQ7yysg=Xi$}kZWJH#3`qw(}Jua8XsehLDUVX=_^fZDt80N=-#Y3Mzaom9=3357qWLQgFgr`#tTRp!D9C9^OP&ATs&&e z%j6Yl??BCON~#-mwrSZ@KTYjp`8v6;I+Ud=X|C$b*)sfc!I&ViLOH*BVeR*Eh1URO&vx6x9zanw^MRr__r<6>Jv4xG{bRf9A-t$Mf@XN{I87 z*7H9l0pkB#6L9`l6OaSA*x3H>lAwpzHtvYaW$%gBkBf*F)ILV)k~?gliFcz*?$3|` z3;cDMn?Sj}hc0jH@rtR>=U$n?@=#(|4{MAOgnXrx@`y=~w!wF+N)^ygg?gc;boRS- z>n4Gw;BM{iw|pIoVARazKjRkV3!9`j?Z(cX?>ImjA;MKm1ZbU)*6-a7U{H-!ZY6w~CHe{`Y~vdb!J`ycNH8>js#(#7}OL;QVQwsmAdL*KWp_W)Wk z^&T24DV&K~M(ZuyYs{dF5{G=ZVDl96cGueeHc+Uo#2zO%G*}$X4nC;Bm)-F_gi_m- z@lz>5G9HSEmte93m}SRt6pPs&0b_oF=N=GnW%nJOm60Qh_A%iOcle~z*3Nnz-5MA- zNc!b8o!0^CCX{G%((;aJM2#2LbKIr>Hw>n@z%(X>Fq4@+PF=fy0A%vcXPc8^-auh` z{(h*(KYwUUK(#M#Z32x?+B+#rba&H7p%NsYo58;g0uQyrjl zD0Z#0ZQB9($;8r@_Flhbwcu6lqcIU@!VB-c!NA__99KsXyw|`{4GC;_?5qC9h;TXw z`H`Ew#Q$)%!k}vSiX0MSqFaA=vbnUT%YJJCk9Q9=k^HeKzIU=Lrtt7Q7Lj+$?&>hN z6*w+LO1_KHOk@dwZCGOtA65Os+W>fIwLNp3YHsD3nLA5fqc1|ViQSA_Dd0`r{$;}d z(0y265plh7{oH^d5@cJ63F2luAbw)c0|mG}-{kpZdp81$Z-5BEzbfmsL`d}tl4hH0 zm$sxzLURJWSb<)H26<88%tNH_H)0bHB|hx!3^1H~pFrwx-31+$8eu6a*#aKx5mMhc zWM`WhWgOSmLrY%Q9A2&BT8C?FZupm1l!r4{{<=gWbXi6s2%km9x`~eHvgI#QK090k zBS27;L~8M_^gFi`or6v7e^&CWcOMVjKF>;{fPj$j(g5L^nIqP%`*m3&J*qU&D|mNl zFCLPpflf*#fFr|e@u)NA_5x6Vze8FK#C**rFcVrdp|8>O*V<}{mhEGC%Ml!HgG1RVOcjeIfGR_}vk+NBv zo82hhm^h9F;vN~CkO#YAI9R9T{4isda!qSwv={WysoP!w;X zd;u0h0sZrWTap1+W$y|yWrFPmLQ!*;<>4}X87dYAJv}P#1qG?3xZH!ZhwLtPu4DNd zuZqd1b|V_svOgw{pBdVNps!R&BL;l}?P*)qM;b&h6dszVQwA^*y-%^-UnNEI4SR+? z$`?$Dew88G#rL2@5owxl>&Ng2DGo7Dy^-kM(O~CoHeE39kdlx=?f?gNHwZ|$xF@;{ z4o=?M^*RJC?>lF74cNep^A*D9>xE}%)7S?i@;$dJHo%2&8wQ~enya9I9)a|1l#u_o z`}VbHx!dAuPYK{#!mH*l=%e05=mXVhjZZBCJ750SRtq$^bHD8?)@FMme#n`=tU;f_ zd44)pNo$@h$JyWdS5zp`@2CSwAl79J#0vUvJYrDc zK}+MC$7%}-imusc$^+bq=TaJeP=9NQc(3sa8wFzhbQ|#8sguOB&TVBtn9I4}m})<8 zik}afdQBT9wdj34&>4z=NcLR88KKXbzMgBT-F}f=Ti-movC-MS2_3Les?a_38ns)H z;T~s+Q)l0$=bgSBjJmRAasmdP*5sKoxXtpmZ*V&K5GN+k+N3-p=m)1m)%$a0BI%*cIL9_`|;q)Y|;l zq8F6QQ&8QMrLPelmClhAPI4=KD`#CX>1sMwxekzy&&-n`aI`>pMYD&zXAa}dEIAV} z6qkP(?#n`-pd-=1f{?QFkPC(-RD8kmJlrNuT0kVneah?mmwG`d*t#E@=DPV4X9TWG zH!=_>pueQS{Q>b`OSg)1l;O^V67b^P1nn@^I`KD)>Ys5;)Sk&X297a-O~s^qZYD4RSU^tST%jJ6mVezi|J7K-~M zX^tmrL!)Gyz&E+S9a`j0%?K!W4XsjZsRKs5QUCZ~t7=bYTdQmzf1L&D+V?5-^8bTi zybVQ>!9qsbzXuWgx;G332k?l;dNAf=QIq;AAH0#xL{Vno2>MP1p^42WiSYRZBg8ga zjVhvNPi_m9Q}H(;rCp#|Tw~O5nwe*j!+-}^Xnd==IX3|{T`VtVn1r#?KP?wO%v_C; z+Zs9>NW9bF)@N{mWmvV)5q*cHlFb108aYkX%}ykQqeU;TtfpNHzqQcuY13+PlaJMX zlo`)Q2*nwHy69e{`MT_Vb84fKm)z%ZGkE>benEYD6rz&PC|-HsKLkyJq>-K-E>@q#I(^DWIWuc243Bl6KzCbR=Eywx6hK0kN7h=phr*sOr@p~rv zb)i&E=y4(Tb2}e(rU%Nqtk@3Do-(%&iIsDcPf4(f`MS-n>Jt}Y^@Vh?*%9&c(D`UZ zsd&`Vr3Z}Lfa+4X0A^TV9Zf*bHNrm!SEm$SF5|Ko85X{8_}Q+^oPYr=$npPF<=P`? z{E-e2nObF`g)2Sr$8(F=4j93>=O~bw=doFEcc|fqCEA7#D~iH~p0PN3eS4AlQs&Q} zOl+f}=dh7VB<7AbdJAzm2?|)C>4|mHKG0w|M*Am`#`fU~s2Ou7!*Kw-{UMtEZ|GFs zALf4EGvq$P%FAJi_9^~C@-;`7T*e(N-;!kTfqt2-`n$Ws|iC@Bm>Gqxa3M%lzrE(RvsVJE)Y#sYOnswGnP@GZ6C|`v*@iTiwG+BJg-}-EaQKFZOpwSz!V$E{TkZ zm1weP?b~}|F?a-Fv8Dx=0jo9V*94`4j*@2mf$HtT0>KluY|iYLTO#s+XqKsr=k1;; zD++)K+ijNO@Cy*e;hdLS_@lKhH*HP_-{<7m=DoI#OHg2oVU?35ra5~^kue?j+>vEA zFLkE!Qcs&n??uM>Mdn-}0L#!~HpXVj+O8L<>af|AjVHEImq+dmLGw0GH;z+z;*n|Y zl%{Y<*m7$C_G9_;9upqR4gpcCHl}Z8T&Gpv9q~|f><$p83VmLT5C#)qZhhYAEOOJP z<0OMJ-86D^@w(@$V0Gc%BpOOAb!Lh52zf_}Bo3{H9==82ZQJ>;i(NEUoA3p#wG2m9 zP89C#fF-u3=kR5aip_VcR_nebHS1Ww6CSs;;va$p$pcaHV-;7R%pYWu*W@r?AybT^ zFh?lVkO-)#^85yYtq85L$C-SE#9~|>XL|dCS%ywtZzVPfcsnrcz&0!xBITucJGIKV z-Ub^kJfe`UfK4#pm)tX5 zS`oj6abkLl7fJipYRw)7hPlJ^gaV=D{VEUm^#?3G+@cD!*(!;ZE0d03z;5|z=I8*^ zb5lr1o9j6uYmJ^Oy{evMMn^&3Us>JbW6b|8&z~g@cP%Vaufvdpu9YuWR@E0rHFB_f-Wrzq(~_mlc7kXNj`G(RJS$N!A9;Ymmu>yCg*R94AM;aE zL>Hhn++yF-@iazhHd+5S{ROWTo>H2*o=>Z{UaUQTE@FjH=nw%#YhIo?a0li!(ZRmm z%!ybG)KXAk4DZ1|plKA(Oz{PU(O8wEmS##1F1UVU2L}{#C2~$>FtRSuVg!K@E4Fcd zA>uEl>2kA8g81*x#tL1_4GFLa()Yb@QV`(Cm2)32_WHDso<*T}PYgS|O#EylaT18| zvLW%i+;GV$od9mgO-k%g6!1YWXqDmjIdVZFs4!~= z*Zxr0WM}omiTOy)q?g32N+U9?Z&cToJlZ~voBu?Mi`v;v=sz>Y{zWb0*=BVw;0hp< z7R+l;#3XHm`gWz0@*)XeH?O7TJ6 zT};0_FY8Pct5Ffm26LXlJnL5o;)1rg$gQ*2I}vuT#7P6oHWRQm;HK8v4>Y?}$?qE9 zb1C4hZ}X1n(Vs*hRfGOBLK%!xS4~Zz1u*0lZ0vq=`-u4+tTy&|u5CljNRK~5v zxf|TU$p#oEel3C64sdfH1&YTV|8{Qnhg}k_NUB_dm_GYr+{*S~O6Uy|baeUiY!0}` zO8TKhI>yTke^`o_m3JdTCOm#!-J>Q!B^{Rya`>x%1pY^GfvTOuP|QhS_Q+&&5p3I& ztMl502iId22`nyo77aQu697!){3PjJtfdCn2NIVkArDPT*Wf6)3g3i)swWTx@H99T zLgRR+9c56C__^NpmLqo%g)3@8Fe*repFM-AkO&)Ryn&}iE~bp;4e8acNc;Qq6MSUd zhkp;lS#6JNquuOgl#U}(fS2$h0U7;+)RpxY7@qX)`d)OCOck%lxCUtHXZ~n%Q|yzm z%N^)}ihOL-qpiG9#fnP= zC50xj7*-`z{pT#d2}ml|50F&6vF)yY|49bsZi9Iz?g!7YM^rym>A|WZ=*(VcSEJVU#se^P6@>tSP+s+?1RnvN zfub7#mJp$=LOOvQ3EBUH7wLZWY&zk==}&Tk)k$3Ji8^(b!j{khog@aJf+20g4^iDf z#cvpZZ99Fol+TlI^%Qk~{at#|USGou_K&nP9cyTMT95)eEl@5F&|`|%R7h; zF_*;;GOMwz1xh7ea*)Bwm4M;3PcRl~;f-U`dHNT7QXtH#y6-anPPf;6Mo5H8jfFa> zK3vkAM2p8{Ckjd}p3OP=>440ycqsI0#-~hV*dYL8avbf^wQXwxc?vFtc78%3@ZN^e|U1g?QQeqjp_jM2FqEo zSuA~P`F`rO`?WSU6UAwh^yyv5Nl3_Hkp3pz{-7eL$VnIn=yMSLon3OS|Mp_>%nh7G zv{zq_(-Xw-A87e`H{q$}K-9xfA4yro%ltR~(!_lyPfHr5aT*+ewfyOsC-4LQ?>wX- zPqMi?lRmF(7HmGhB>&p}ZtiaHwtY2IR=L&uo9d=ez~y9pyTa;I&6IXS!+dSZEUfgO z6{s=L`XBjVfYv_6M`5*hkvl0QA%nU%m5;AQ?hb$A_u&D<9!3}ej&d(Q;x~|$cdGQ{ zZCh#e$$GB72VYmOgg^6hS!oL|_ht$Y?1lO6MR5J;ck6j}O*33eJ*6*b$aUL>XX&>K zri895pv{GNp_!V;Ci5O5pX^v@0|)VTi|7$sXbp@UV2G-=pV=OGewMj?HC2^RQ-P6& zvLq4NJ8U|raYPTdiX0DG8@UplDuyH<45WiX+fBAAp)v@ui<$aaD(HN($k7L^oGQN) zV}&Js$Tv^NvwQ0djBnr7OOMFEOD{4f#3IFUdd~WX0hw0cITdHWf*rLK?6`J>@+8BU zkck2w&^7{Yg$S)!X0yc!B4#1l@8#>pbpAK!AewRaJYq^T=Pe>o-Ra|XG0y?T=mY8# zRWK0#t(#8{&oN=y7yCRM7M)0cJ>fS1bLib`BjLAQTFa6}@fqEB0N+rbLWL^ZuHjDV zo#yMbwI~|`526?xN$_59JJZMhGox;3<}G9gAjoicO}GK_%IKF_5|G+KhYTy*U5Wf4 z1>}9bJ@kkry${)dcQoXpDRilHaQmb7%t!2!0TH2&dw6@fK^rUSYPOHl_&B`w;hXSM zBfH8?Pkl<0P~_5=@z4Eqk?|=~hJ;LC6b0&%jHtj&V@yU_gEd|*4oN~#eGiTv z@Z5G*%HD-OIjbrt<8Z|B+EU`XiD9n|9+79kAe|f8#gu0Ti*&!6p2=+&Q*bzpcvN?w za|SyJ*Cv;@C!MiCV@XXrmoFhBRfgjMKr?2-Lnyt{D)sW-w`}lbMg+Ye!MDu+noXO0 z7E_$>Mg&pEu6^>+>uWYAU13jzEgGNuFLHbxvLZ?P&a041T-w(`tbbq{aUM-45*oCY zrnS>4L~sNJ8J#~Ctr-uY=X)oX{q|sZRUy-XR#mW&L+t*$mF7G; zu6=k#^P-&t;hU48li(Qo732^Tr@4Mn#eCtKqiLP_F<-bXbs|BXdwhZHiVXpITmUi2qVL+>2ZjT z!RMCw_ktR1KGgScqlr?gcGDu1ExOQyD4DeC3+CDUh5r!%^VVI*(RR?Xo;fS91K(Vx z%4qm^$)D!cC!mQ8t0gc2FbXn@O*t4O*ZxosId1PC>gD?56*24)Eu>Gh^DqkZrmFol zyH9wDuwKt17YHRB+Rr=->GLTl|D@S072T1JF?MVAFR4Ove4x4PaPfg-p&i>9t8umB zLXJDT0-HclfO9vaFn}i|Wy2YlE#)&u`0?wr#D?yl!ZmOW6#ls;AgPrUuL|$B%ICiS z*EGt1tiSWY%Vht!v%H%WuU;VYLuiy!;pyYsCB(DHP=bu48@5qvT>~n41O#xJf0ot- z#r666XcBAAoVldk%b$s0HYuKLM$j0P6g4c;e^y`}#tM2s)$g`5@7!ctDQpL1DQ*m# zy$~6*U<90j?8x>3fa0RiT{=d_k~<1lsXp!GOQ(%%2Yp}YE5;T=qDJz^xlyTt7#O_F zW89b3+y+8zum^JGwWA=jMxH*~OTq!W|H+tIn^9sH)t#g=wVw}PZmoL$+NoQhd!@P4#JGYcfWSJK$^l7ERd1l&U4TX+ zdPLSpVJAuU8>28UJk8SQCfB=A!q3SE5%<~uI7@)Ejo6q6IbR<26@qBAd6Z+Zm-d%F zEwd@?^OFVu_9adOa6s5cq$0Q?9fp@=9PKC0g772e;MtW}PK14C4%E`}ATN*=VaJ@H zfNbdCoSC49borVkgK2W07OvbF(2&tcQ{K0^m+;rkN91-=ybOyl;aE}ZO+v(tveWRX#UgaM}o`C9SV!)pVDw-3`?rct{3p{)evvsUXY zn|tok$HAk;P%w?$hp#eiLpDpK1_f^cu@PL-ScF$~6jM`R!c%{!i-L5jpQ9jghy#i$ z0%tS;=bB3_y<Utku*M_pnn)Qayt$ZJrBUv|FL;8ekzI? z3<{8AO?7ydzLdO30Xc2|8loWihIOEp9Y-A9{4wT=g>G>)MJem;>-KvXT>eRFVfZQq z`1}2>!92$Bx<=;8nqyt(i_~`V?-7Fxh(G#K4f#uI&Aa7^nLA1qYq*^>!Jg08(Wgs5 z@nDLJnZ7&_{Gazx;CgiGi?2@w3f3>!PFt>^-R!i_+K;`w18s@g)eB-k-y4k!RiCm5 z_Z^3fhnXZ6%OB*LcrTJKD`$F{Nl2{)FqSoaQ{#c@gzYIOHm)o0e-bRv60?%rU7@at zK7t{&OSXq%$PkFdGmnKLOL7y|Ay$4*Y}71H@}Y8sasZk-J-|Q%=#=R1yded6bhsrR}zEdH$lJOec``pKD-=4}}EiOYA4- zm1;h;Ibnnczn2M_wWF9wUgAN1Q9_N7Y3IX5P9ODSk%r z$M*c`pZIIGWxir$BUlWn&s4DPI2p|G__~U5(+XY1=a`j)|I1(jyBrdPd%65XIt}4@ zZdzvd{JFDygXH$Lry)AM{*7uH0t7ZC^@ml=X^OcAg^v!f7IsRDA=|S86r2tP#hM`i z`BvYMBgFshK2n;D9J!KQi5dGG!+h(iaylSswD&apA?wjO|BWYui@TJJPS1J2n+kKd zyQZ?>N|(8z=+6u~-N16K{F?L7gRsFQ6Sjf|yavtF{FD(BkcWwo_>pUTdjKEpmK8S- zD%VZgLf{bar%jqn&Gq2uniSyhB=Oo znD*nXRMJ2#bYE8TwZKF!C>}n6Oa+{gMRM#lh`5N?szc|#LviHj!(Bu)9C0pgC)2p2 z?)U4sZ#}}ZI*HX^8k(#sRYKXTx(H&MB!k(Jr)^FiM0G<0nypL#94WK+!!^E+CvBV~ zu(b}ewlQS^UsCi1Q*>aO39ar&B4*Zq$(8WcS+Mjj(+h^AJ`9x$_#R;a-@qp|V1`i> zJzDUXAkDtFKCZhHUXO5SNYcy#^sX>fYvSfX!m`!v1${!! zbol*f`}JkB%*p$Zxl;iX8P`Oqt688ReaSt5`08VHCm=VNJvqe)SB!>U z0tYy5Cm(n>k5(@kDLW_q5t{&t5l7X>k{tSEG?p1hXRU;GwB=FR1U*i2#JEkKd>`E` z0|R!92jx$Q_2-!a;aJf;Y-Ofr4&1oX8CPk;pr2Yw;}CV9zaatvg=ScgSx<6_zbv@p zQ)7^4d`+JK%7k8=F%$37Ag{lO!3VYRhk?PnR6vITN!5O&aWC_l_mCXf*m@&``BA%U zPAMy9jGxm5@$m@=QnZWkz{h2rb=Yop2%r$G{-J$`CIMTl^0aOEQiDc9#0T&3En1Y2 zBdP^z$@H#~{Q|4p9QlPRl;vag9V%Zs&x>}`AkZ9u3gMtpyQt2jc%W8FoyCN17UK5( zLS9!AJdH4aOP^X-j(mSQ(Ejwjz&rwt!41fRG%(Pp-sN_ucI$vPn%GmsMVeUVS68`r zIf>0!RLWoCSo{u-AyCI8bj`^t%iE8Prl0g~48;meXNu|^?gTHq`Lw0W3b3D)VqkQ# zV`KpUzy!xoiTm982KX#vNf_e#WE7P$n`Ds~sXHN?9^By2^YNvZ%%)-Jq0lEUT&z#_fu! zYnP5hA_@~a^$pVBp|%S&GabRM@n%u+6x=Sf9vz?|_s4EB=jhpb{y$}-inHlXQ-L@(%KnzQ@t{H?lwPw zsQJ@XUx{~p<0+U!O#@X-T~mAgoeDan3FY;zH;6|W(EU6|OMiHmN`SZS+S;+i_9p2> zq?9v=CD?HUw3ODN>1}Kk=`3l9rC0c9mK#T|qsIKl&#);BNy9dYMrVR; zgph7D<^ulTmk5w;kcBV4Gc7Nmvd_~bI2_Pjrc zh=KPfA-@fMYCS@B8v79bS8ZrZpk(xJW$H9K)&FAa9J@1twj>RKvOf&G#~2b7Fj{w#UX;=fs{B$b=o&>0oaA`p!Ly+0mriGUscjs0(kbrRU87mF=*NE|!#s`^Vt zUFb7h#2OwigyQT(hI+W2{+c7%O7QNAr0nlPD@HIFLK0Yp`FccB9^C7Ige=>WoG0lj&?B8azznkT3a)vJE7sp3?T%R6 zo6wHsw;!A3i`iw#tJ2SMF)yiq7GHv{M5SwK2|>e*cM{Zj6y$lfKHkEK)ubmJQeZWC zsOfn{WM-`ke_5evZbPjLg>Q&c9-P`uE8$hr8X-*){Z)T;3chCkk!=Y<*qV2xiDM-> z1sAcmew`sXKuxZ zb|27mghI&?W_n^g$Cd9ti`03pzRJ^DJ_CJz^Tt^ne_p=V6dm8Ie*V z`nIW^L_{Lz$5Ouq?|>y%2b($UeV8nlSinf8#YT@OpmTG!@uP%}HJlJ>Bv8cAZa+jP z^N|T({!Tt?PF>t zW9WVxkPht&1CULe0suSd^$XoVj%y+p9C^}7<9*-|ItDtBNNAHVH8@W#>8+4Jnu2W-;Wv*`Q2forhlnM ztLgHMIZFJn-?@?(zWfM0O;d%dA&pFKNxzvfeMGAE=pmV2`@pdeqA?EKiJK&T9rk`P zf(_Pz{6C^0+kcCOY)tI`leU}Y{RiMnB;9_Z<4g@ot)XPgQ840u&b=s@@~wsXUOBw= zA45jTq@I;Yu_LBW8umJqXVNuRR{geYhz61ApE4szbe42_bVjxFdxPbO+#qv4DP1pX zv+MD0^?sJHXJL#UobFt-AS}uLbznYzvJJRDwO_w}F~+m&-2bEM<9qp8NhQB+TGh+X zPVWi8llxR!@%(Z#d}%&ElW)G3``l!h(_Qp1y!Q8g_0*!z(||=eE`{+3jm|9*%>HOj zFI~{hk-ra=_@a@zkA&u$gN?6>HZq*qM zZQZOQTkokwZbE!Ra*U2*gy6l|Y~+OF&|EzEwZ8PVmie*9yp}(WvM-G0WBfojfu2K% zz?)|Xo|sNCd)S9f>a_+tFqps#v=5jMDL6h?zZgzaGU)wAh%4Binm$fimGqNp{T19h zF6O^>mO$7`GDs=6mu)#{FUonej6&^euvG{nac4SJVgOQ7F3Pwf70slxK1yljjF)7S z)yhtoE{UkZh}CYy<2*tc=jR3MA)nx?r$_;l<7$-=$tXN;SMT4p3E6Y&2mmnY=eP1R zGtZ>t`z*3RnW?vQ60PGHv~Zo8PwR_kds;5izg`WpB&Oe2?BvSZ{%wqY!Uzo_w0HfI z-eaxSsW%wGW6bw38pbe!tfuAwJY@Xq{9n10-Z7sCeV{_$o#a=Z0+3 zPMIhN(-WnCwDdQbILk-=cbZ)0M}kmEpc zPq-{$yEHoyn8M}K?5fI$*iiGel9F^0e(OT)+nNu@)4E+2=Nvg0O#mJrexZ8>=I+(F zQD$S2HI3Y0y}@`~Kr2A2Lbx4Z|vf8yc1S{Z-O+gocW7!agHfhA?QFuZmdO`GJ5`>1nOm zU1)<+D{l92pePyxb^(HNn_h4r^eeQyufNbR6&>(X($E!1qX%2O)|D$}s2qxB)j-YF zq^zl>LVGyVZD*&)+SlG}MLQuQ;vw~4#O6G__I7=ba+AE-Gwbo7Qb~h8?u+T6 z#u*GA4K8SfN!||}EQ}K+|FUOKhd_58*R^7<(2UBeLuP{5?%hrdl>E!#Hn1}NgQ(n& z&eQP(y)NF2M9kd&8cA@|z*0U*q89FfqQZ#0&v9Zh*$7FfBX&(Yv~_|;Qh)o^LeZ7= zb-&tfe%ZiqgaMS`w~GyQ2pI0W+58ZOBVCl_KCZmjBv)j_T6@q-=T@_94O-+f=L`ai z?%Pf~7rFkh!=+UZzEs?bC;F1Iqez{_m%#1?xdn#e8^e?QgyDWJ)53kZNe|30tNw|v zxucjG%tFoD?-xEbd6DzJk0$q3%|=heDw1+5p(P-1O#&c?wR7?=z(HN0CU>KIKplcC zy${8bGWdOC>!C+XL%Dj&lPC95Hn}C?bUlr`G&Ou+mKe7&VdS+Ueo4Lr-flnw0WVIzI= zLYrhtumcb@EKI;&ao5SSy8WU0v+WS;dAfZ!m^z#+wVs(%hMK-Sm$iu>r*MZywfcPx z+$x(;4xT?B-&aoPw*rgbJMk7Ev5;;Jv~cXXwiSvSs)v!bRb%C*Vlx(+>_Y0Nz!mt? zg$~cRB)Q-?H}W;hbWP9^?{|`gSeSH;+tn!{F%J;`+f&r~Fxkw~hMth*y$X-12OpB3 z0R$$kstk{xS@qQSVn3?r=1j$cziK6ZX=(6pGG{V71r(t_LtNWU>&wq`B>~e?C|{lL z4a4GTV0iKILXkfd=r4$CN=u%fp)ovLpJ|W$jMK1K;$l0vpsD@cV171p^66T$ZO09H zcK~1|QIM#Cg~Yg}X!i@I%NPgkJg|zW)r~8y5=i9h2{wiU%kRC2GZ=q&i2z{2=L%~``nDzAxR|3o&E~*Ec&fQ8OE?lA0^fjfZT&YpE6wJ3zt*M+W4;0k}CpY zvp`UINU%3Jr_9#Jd|%(rL9Q)AOGZeJh5TeFw8psm6N8msIs^r;YiQT9{5@O@`2qNe zyw*^>c<*lt58WOym3n|i{-Pfy=e6Kymk-dw1x><~8oqVN3q&jRToaK@iE8KK$WI z&3ir>4HXbf{ZNSN(3?JD%4)A#T{M>+3M44!kA643{;LgcSi7xlme$xRiL@&1R95TV z+CQqn5zlFy(2E-yE78#io>z-j9uu%)UNUstd=&w2x`s6LRJvpv?+La43l3;wUq>fpVkq*czn=rrLi!4Aj$qxj6 z;BturGtA}hI~Uuvki1=<@g|*1jj%`%xlXXQ=TQJz3{v)pTPv^sbFEvkX!IguG8YYk z@q+52fn5|)92W*~FGAqx2JxL%yeJcxEP)rEJiJs<8*=1A9dNU`DN@5=+?xtSCvG=4bhfqI}f^?sT&)-Y3 z5+$6Oa2_a}TAEIHvoN!B`k<0PTcLuE)EG#x*%%tc1$cTlM*i~GaC?UxMpu!;jH~$= z-Dm2%&^uEgyQpUqAHTdmPNld-4E9|DCgI7}D01$YXX(n-%PYX8VdIn1!Ej$*=5BeH z<2W(R1xey=LuuXfygmY*{%orQ4nV-All1>CDa z*ruX=-b;2^kd-lCG6;c?!Q6O^#3-hdCHMnb^nG6G`)Z`JivoNK^ z@(S`xP{%^KK-HjlMO~S00JIun(O{wn^OWJFo)#W`B5JgRbRa${j_<)?Y*UgNmO0ZZ zxI&NX%byl`R;_TUZeoCd75>NqhzpCL--}f;KKqB4aRxvh?Bt3cUzqZu(^yg#A_f+e zMHibU|l!o4X#6lm>QFjkid`-P!+r%;4fLt?t)Um6aW3# zIt!LV%0J>PfKKy%sS~g|e&qfh>@|qjg>4QJjB?MLuIW!8%E`*8NIi?#y9666fQ}p~ zFvtpeDF}ekE3E^ose8w3u>uq7;mS{Win2{3vkJMp+O4Kw?YFxBJ%lkeROtzhh}>b# z_cDE~GOq z>|h1~>7M&=ugE}1$jZqXGaBZU8iw}X9$aAQs=^6pyQtF-xGwLV&=chEv#2I+gla6w zTUx*l&kP#Q0c}twDb^S=)#i2^nNi@1&{0;<(GJ$aRA@jD!E`EUNo0X0<*sZ zmOxe!Us5_M*E2WI;rZ_`A}(#biwNp6_$Pp+y|s-m2V%CIdaxml`jSl&N3O{D0cKb- zP=7ZMNC;v%n@mCbXETY)*H05xE$jEHt#{QPtTj{2?_+o#)!t-DbvS-x(2P;;AGGN{ zWaNNN3!;0xFgq>d&E2rcP+Hi%{Y?ot$s->2DuOe(wi&xd#}sUrN`ej1qe^``rxc8JeJZ4@Pu-9oYO z3f>NT7}w*v!a-x?xXoR-cJy{oE5$DwA-&|;t7s>1E#{%?Ubgfmu*^ucF0rFM;m~22g?%)tK#u(F4zDVFf-Zz0x4d-8u}`0|wy? zwhUg0e;mvEO8kW9j=lLHS|7zpC=#@ zggLlH0XoE;xr z;1VzzNp&iO?NT~`u?_`9j~wt(K+B%S(72* zkrg^E^pnIA#~z7LqA4>IR&fwpHeu06Qvs9Q*_YSw&m)>qW&$`X#KuB`DJD8$=^one zWjnfusuw6mZ{Lr&e@QbwS)2<~0!~7r>5&>i0`gF4=DEu8q}2{6A}mqB*qU(l@6O61 z4Kxbnqc;% zk4RHvjj!`a#*9;`H?HwML>X)+_-$xtG{tGi9#JSI|YLjlWkF4t6gC<>)9ZwZfxxw3;M zNir%vZ~~#aNs<4;GW!X{tE%s<^3p_K8CMMn*5u@Yw{^}B z83jCrv;eLR?VjZz0o4p$WeOFN2mH!bZ`O9tJn-P{GOkr}2u8yK+|;v?{F&HhCd(Cy z%(0?9w$(iu3Yi^B#E&h1gvB1yuyeOVQ!StHLV}QBuSBE@`3WskV*^q;75SHU+*Iu>P zl%bk)&BB^Ko2F2mUnDG}O5u1=%)FY{ZTB{~6D+|f0tW8pWWO;@i!0L=lji$e-|#$~ zY2|aje`lCi=|&z?uE6)hDM?jlR?(@61xl(mS1MzC+jLrF7n82^kul3mu62)k?Vur% z381{cf#e3jzNAqV!E@oW)(20Uld~L|%`j>~n_aHjL;QTK3wm}l(*zwVW+_#VLGRCw zfB(Ah5^bA6R$kH&S!rG{?hoWW7#~A-Mrhl7wCrJ21>Z5k_(xK`?F4-^(4vkjrWupm zH24s`3cg-Bsl&jz+K-OZ->mB1a#_tk9RLFBW6h*)U@a+g7UAK!e{t5Q;h9D1>8$To z|IPgo8fY49Il$03R&U*CSD(39r{MyAD8Xccv>q|A_0=qzlDB~B-J0LS95{W;8p9bw z_*wE0iugqU_and{bKmyP{__>~OR89;%i2uZxkomkB7di}k%MQM3w}9~vKsz=12D1U zas=G67BQnz;UQ~!8l1vZCZ%KXC?Z8$*9SMFU2ny7#dCM%3z3+6?g7NXy|KWf@Y`0L zHSsDMt1AA?#bheo$x2AViQ3Oj9J6Lmto|3|rPdMJd%Qmx*V&St+-9lTNd!Tq^O>wJ z_&|nBl>mYVh>U=gCF_&EcTQ4awEE>hJPGOZmpk`vi%uwx{LKsjsZyVflsPs0$1DaT z?Y|1DcgwG$c|B_a>b+zM!^eR(!kxEGawnSgA1{Jjp(aW?VgPY`TUKV z*7Q><$56(z@)ECqdYsLZ=o&l8Wfez&RK1zw$GCbT#&*8il(V7!R>)U!S|^2T_fRWl zueN4uI;#H}ALL`s-u#r$`4<><<%S2KX>Q;Cp?}+axo!Ok4jy6Vkz0(F5iPYI$`)1T z)O{m^5m5d56!2M-r6$A+lhv0hrZFAgXo26S!r^^2sAW{=9-3~o7iOoUZzR^MBsX4JU0ktz)y-sY5m8F4c$g7^}Na4eid;dF9=( zM%|9PqJMU7)RMpq>#i`%QH}wN@Kwid+E&*i@d@KSefcQ>KlslgJ@8gq+RPNK1Mh_+ zW0YPw?6%SvD9~W%wQG)5VP(QhQ9)N$%0o&H7}RJqWdARJZrf_-6?g7=1%-_FS@CWD z;Wysw5bZC%OWWK~i-AuHkTe+sBW1xe?Hx`USXO=o`(mrcB~;Z<;aNZvAZu^?8|7tn z(%Q~y5uBC3KhuV1xz0@Pi)&g5U1ETBlH8PH`yVHdkFdP&ibwPH#Qh`m8;3X7u5%?X zdlz!@@st|H$UpCQJnI~a^c>@B)m?UC5~-=8T&!>y0r}Z2>8-FTLUk)>(v=;-1=c%5 z6P1)o{=;@k@m??)j5|PtwWo%VfzEIycq9LHX&C{}HD>bu!i;#v$1k+=_(#Ou+}B|x=l9K8 zeoRDN+_&S@)?%I)543FSeEbc?zyHk7yZ6IaZ!u$h+^n;{8;^h~heLt=wDK1%L`a5t zJhcusU#2;=D=B?$a1Z|Elgxos*w#KluVQN~az+za_QlMhx?b)wrZJgW8ja)z^9Yu$ z4`^^QnP1<~*Qb!jh0}Qu=tf_pteLvx1uD<7$??TsS5+P8NO(H2d~%PNskUF2g`unk z6aI>Hz z^lhfq{P|NDIm3>!Irjv2oHvz|buzf8I=Q^LGaVzZjzAX8GbvW=ud@yK2prqy(OLqU9CAf7dn&h(#|p2G zCoACxv=zW!+6w~wxW;;bLgFYec6Cl`j^Qq2!oe9J5P=+Jm{ zePT*yGz|+Jp5;y;7bWd6rSD;G!cj9t(5Yd-_e(1U^nuX-&8vJe*zcmYJy|+hf#SPx ze>f}kGK4+m!7I3D6oa@+85In45CiI;bBh=YYr-r9#L-ze;N7Et>Q)y5U!J#`X-&av zGz`!i(BD%M4a}_DDPxsZ6_PX-i`*_8xO1Dk)*6fHi9IwFC~|y6SYjcq*P+gZQaGgV z+iQyTzDb}+S&iPdmIr)uaQvi~9xM!p><#v6%zeltDuuzKt@|!j4 z3Homfev!;xC~mc+Fuuk8pDkLCIZIAM78Vlb@cJ?)Bdg%R)cQ<%XKO-;~ z9aVQlgUaU&b{~#-HTP}3pc9XlGUC@Hxf0pvWRDBT1`#Oh?^_QZO(Xyv zzUGl~(GQ2Leg9;@8}7T5>mkz9I8XIWn)T+hS5zJ>4lcsMlf)2qf>k1=TM2mM)))^O zGp6%%XqryND~Yte+L8>luulAv<-2Z*l|I+9fGZnit+fK+QVEUI|A#o*wbY|mo_}e$ zZ-vPe9;)~6&ZFM|k`bivC%P<6-UXnC@^%VG6vZeuClE$VnofR6Z!FkxyMW*Z-uGbX0 za*0I=3l3UnngzW!?&QZ->sunL=lyci^QReHzaGTus~hw(w{m~(rd=+I+yw28pH>;A z>5+qS256-Qk^tnz&<-ZXi*W*h_v2+DbzsHFHe_D+=Bg5XnuG!d3hHr?@@V)_g)01y zfB1qgsaq_izfDXcO4tODNHJi~R694E{JTGUj$y)GaSC{m6;`NF5ILit1Zs9Wu^tM> z9=5!CBbq4aZq!0%=tZQ*XM*n^Jvqthd@5PnjqskXMF@MirL?c#H)@D^TzJC4!}!$_ z>nA(~(+kzm_Vf-@#)WAfY76YWZHAm78FI&;F1@>-%lrn()BK#54Fs@c=ZDJf3+5t*OF$t-yyJ@f4tHkSzJ&;f_ znOi`lmv1z(OqGE8c-VV+rhGuFpR(X6dfakaG#&1Bf@=I(+IGPuV!{RsIjj}#VCeQ6 zy%dVCZXyC5S1bzR1p)-cKNAEiQOr+$&s_~8Y=9XGOEbRXm=Z>sUe%2bTgchNg(()T z4}fC*Rpi~18=|!$2=PATQ}eJu3Z$ui!oWCDq#w{JK>wP}Gihx`-XMqN!9U)mF^L>* z;T|90{ZK-bGlOMa`t7<250#AhTZ%nQi?lz^>_L5M2ONW2UXO`R&S-MzaPsIb{Ublv6_=he*^vxId(>6mi<( zSx|k`TQKg95}vp*ghi452WtTsXd_#6o=**L$H?c>>t@`xfeq_t??&577Y^?$uF$@s z72e_R{y{-qRF~Qi!Cw{yFv0^S$S5G$3uJvBsl5YMtu#R2_l*2ye?VGOw>W}Wwh_yz zu4M(%pHrAzrpXb5`y-YXp|{JG>P&GeDU zxaA0=#PNou{n95riXTkadJq~Ox+LqfS^Fl#Nb!0j(gUQGh8>V~sCea6!KZZ8eyOwf zI6XzWeh@%6+^OiCSC`*Eg4T3_s{KJ7HUF>nIJzy&*>@wJ3SPC#ETbMcVOy3^8xuD7Y`!Gn05oyTFJ42GkSHY+80@#25C@qYNEFg7lWk< z@+f?qk_h)Ybi6bCQjSYI_)@zDWmZr>@CCu})kcbTj zZqi#VtYNc?>tP}UXr7c`Au^|Nt%U{cAeGpQuz+O8&y#yQ!pou^(i^27J($$^9~D4a zAU$afi3M+)UqB3B^8%l&boazRY?c7^lYbwCQL;)D73YK~TQL8lN|;TG<( z`T}q?iG8nf{@Hs8XS4^fShRmqFSSVV7%|NJep~VB&s^iDNZXI7kfDJeb>2 zi>q=d2?fA)ldY*lR=1VqZ4NaLSiI{cKK$bY5Vwwg3$&2yUJiFpsg3qBqF#t+BDvqY zhM44?0z_!0%oddCl1jD4GQykhVo#~!tN~i=k@YulLO3j4bAFWDJ9=2r+8Q#TTE|aJ zS|Xu17S5eUu(xAI{7dqK3gbuMeOUTk0qTWj=jsBOce2|z`X3(FPSXQ>CnJ`B3L_Bv z?q1b$fdrDE!IvnE`LB>lEQ3wm1#@&1_@~OwXq1lG7c7#jw6U}r)wQpTgTM#zb^%V= zyb10`K(ISvBHB}oDP8ZOEZnJY-SNjpp`I;F-EK-XTf*FeLr`07LOd8)@*>6@bEO~p ze_U!UlE1K@O>2bBT15*iQT?!t``55`>uUjJ!AIcZ$uLsf;Om+w8oSm=Azb+jhC!u5 zz%2t08bSDMWC8Kv`->q(xHhjv1Ax=wKW!e1oX9dzRg{1Hs#i>su=I%JQZ6?wGR?sc*vM@@*wQq;%aquE(uh!1PFo-wLX9+J0Ka3==^vnfM8<$xDsS?u{} zxEgY2Dw>Nh+wR5Kc^7Ab{o6mOie6Koo5H_Nt9S7|K`(SD+Im$({P`wt*H7C~2< zK(?(8MKzhf92D97#_;r?u=X@9~&uC4kd|JO6McS6TkyyL$lI;OHuxBk48vj>HHv9|ey@<$gh; zjaK#+t%mKpPuD8>g;)}g!1uRIj>S_X7+tVstZ*eY)xg`pMR6vde^})_7Q2f82c%iI z^(*qvJ$L#c{)}VdM72gj>X^Mr&Ib81|VAgNIe z`!`4sK@+gnp(154Hvm32(|z?X5u+&uB3XUM$J3l1KR@0)UQYCMyLtOE!jCX$I0Q7@ zzl7mfodjV#e_9;11Z;sf-u$YdxaeOgO0GI9G~IH1|16lF(a~W(Z6IGr4Agh}>X8<1 zw;=^N|3a6xG#^ zz(<0kXiI3CwY^0gw6|ZD2rEbHEqpE@7~7^8Go*wq6iRE+H)r9sGzwR7UXWyYM9+N4 zUj2!4f0cF-82|xs?N`=+pti8RW10H%bD~8&tTJWuq;R$#CNup&3ia=_T$S>0bdXXz z62<H-GBd(2@-v)YPMGWZuT|`Y#N!zNK^cgK1u}!X2 zO}Zb#{Fh^FDZ__G=A3%dg7F~^{|1%4hkj7Gi{y{S^f4KAAZ2wd*)99VZ;M&*_4?el zlceL{IRU`5?chN_d9-y^jeBI9PpXc;OlXreugZnSoG=~jaK8D68kQ;! zs{WqbQ(?l(auKeqw2&Bj0FE7vYG|+8b6Wnw4&j5%eM?rIq>FD3STUqC=(3R=&(H_a zLfq}j8VYm1xIk6Z4`w^D9zkgPq(EeKhtE}ki%bSwG-LO4%sdU@uD;RKybXc3#A(eO zIjfDV^Mk$d9f@xL<4?kBavrANdOYa)S^Iq`fypt{?B`ZN?{O{~dlS8p-IWD)O9X(% zb*Gpqgg;6<;UFi-D3he)wp~RLX8g<^DQ?Hg3E|F;xt_LUHV0EuVJXh;v2ihOTSHF! zwH5)83OQaqGuu^w-7wYd_coE6J4@$N6(){ci` z|H=3RgS!g@k&BwnMg{H_LrD1Wo%ePfMW{vaNv9d4t zPOKqySt1wPreL6Xo9>ZiOQjFuPf6uRt_`5Xn^7jWMfEeSUz1qy$07__(7c%?A|oF$ z%PUazn9q~oWWe)?ls>5sLVQ~@*X9gJ;1+*zC(ZSD3@mHOyCoK;x3Z5>4yWfJm)i_& z${y*zT~yh(0yl(|OL~)eT(p>X*r?V1!oSHJ=77~ieffOEi4}og5Z4toqn5VSpc4Q_ z1UWEuM=NI>yHfx+3beslsXk5U#b926rub{S&dhS}vTR58H6=8g+o}lN;6_jVk;(IK z24n*%E`3$pJfdzWSc9M*j`8-xWA-g%sza;IF}6S){qUO5(eV$ZDDsNZz%BL!?9j9) zLfI7Al%+?{wb|miO>if~Or;}4MmzwG$0hbdipo^5uz}E%DVdD`cBP&RDn_3ek4JZ% z<)LqYH7_xyZiz3VkI3q-t0S81ZsF+oL!du!NcJJP=k|#7R+=;HktoQ4Vqpu_q*_Eh zwlW{uvf{0xs_077CbSqxBE}pPXm8AvDD)nSE1Uo{A8?5MgvBKWi&eqU^aN0c)4(<> zd{mW}Qz>sYf@%P5jPyH=v2^|+Vugt43;bO_*ngQTU(YVn#r5UdQU;jOP^w1O9GBfI zOpD4tqvuh_UZE(&jOUgW;F*pEFTw7F99g#YM~DO2%iW+?U(CyF9kU0Tdl^n6exjd_ zQp1!us@zZ=X>^4Kci38i!v}O`Dw#>-F_~}6VZB{hNC<#KQ~JRa<&Q0Gc~#*w*W$Wg z8NcB{(ND|Ku1@&iIvcgpg6K`4BbBYC0)hNJslEHP+m7wacsM^(_x@gCP@($1e{RBf z+FUzD8)}!zjctp5AcIuvbAWxxPtDUFZ`tA}>I(wu!o!|{k(&h)rwHH;bGnPIG%9_F zk-Y4nvn?LtvQP#xqK6h*il5JM>IuX))MG{A40ATyQR#g}GJk&=kYvZQ9^{l{uub93 zUTJN&{IiUNumwGpIjh{~uMlp0rtUQYWM%XFC<@YVU@HX81f{U*A+c~W-ca|GK#)=E z$6DT*6o3N05|1$b^tiM4z=+SmH?# za6J8}_Gxq$9J0m8m%%&rjmM;fIX(?cHxREPL7tvXyOv0~jq8JSq2(awAY?LL>)Uj} zxkF40maR>_u4Ozrt_8r`CJ?`t>^&>bgwMnS*E`j7(OrnTm~a5H+-xfT(Q&{BguVcv z4-ICcBZRrCH$(bL4l3!T{R}D=v0vhOk(kGph1o^B?TbNSOz<+cboQoSqLz7G0wQ%N zuIZE%9^nu13up)PQNT{lzt)L!67}w>y=^Ku`6OtA&e$=j*T}MwW&{g%SsbxqjS(n| zt&%>e1T5}9)IIZ$&Mxh(}#|>Ni+aW$9N`s@iw@_1;vBfB3zRl z6`oN1o;ZR+*5B*aLsY-cem|IuZ9;$?NphV)<{1ws794;!c8(T5qYVKclmt#|m$a>n zMT|lJE|eynlmDH#xO4fPhtCoNfr(WX_&grKK@(5p2m?e0lC-)QcG1Iwhd5`HM`?f} z>T`5TN^J^wmz1FXa&{x)+kpX=v7un$sAs;Pb>e#sQYM(b=+(xR$@y*Mv@ebe)oXmTE)!Ae67#H&+%=C=CSISt)f%q%K`g)@zrCjg!%*Sqj^N z2jd)d4cJR1E>+XXo`f{g9(}lN+1L_BfW5GpIK8V9jp*J?diuiRV296~Q*6@`Gpa}t zMn3|i0|iK>JkWlsNJk9?*rx*fFNM{Xd-B2#GU#vf^mPS_IrZpRccsw5n_lLb^%df^HA}$Pz{k6rhjk_@2#IXC za>4dBR$yDk^<3}!ld%wqqN}NIwxa4ob7cYuzn5FUAbuFn3J&_(u`qrUe^y~dDf@y9 zT|_08Tt~F|czRklLSTK94Aoi61@iGf$27L>vbihTLSl#a^raX;Db0wp{(JCVNBbWV z^V_IJc}sct&1-zGywH3QU(CfFHVL#s=Rk5M7C^3lG>b=F*q~fadm&fAot6T7>t+F7 zIyfv5Krr$E(2mJ!u?NFOLwJ1K(`3vt9>elar4ZOfVyGPZ?&OpoP&x0CYaOtq5-&P< zsTbhc_)7X_la6?o*1GJQXTGqz@$!!OJprL<#kOCur|CcCu>-f{DUojnDd_7_81mFc zs!)R}CTBHEPSs>CQ&WHn(`(k@NevZWGO+1(NG(&ZFjPG|T$jH<{wg0mYQ6mf z7B~E!YEEU!dt$8+vy>ud5-+pQUhTACkk^}WzEY8{o$5Bcqh~ooZh8aHsSGU}ac=64 z=A_mf?lV>Dtbb;Wnl};L2B!e>9OnX4qW{7$(2L3d>KV~z)clA^bC1Jxkp*!^{JqMy z41ds4y)_yj!w;L9jEDb{Y;m9ZZGV@i(>9Z1m+Yl%YTRIHkZG1L%ESl8o9d@? z9Q=0jT(6#@V(rG=)Q23>UA}-1f5fu|UqosEW(zvHYDp+H`xQDLyO9p)U^1pV2&t=z z?kHJG8VyxxGoOU&MELXtDsteh_gcH|s`Qa=7cl!BzZa&ned{F!1M2WuRQ-uw%y6;Y zbDD;R(97Lw$iQF_eYg1)V+~4Z+PvQan7T2ZwO_s7rNcl(MBF*7U5a>F2 z-D+WgaqQK}kD6h}i&g?0k@fUN+(PcH@n`h4B$d1X<7{ap{szGc4Oc`V0KqhLVCe1n zH-QP6$78c*13t7IvEERvo%;eNUK>-|6cbANm@ZBV&74l8Shy64i5fP5(7Y{ozI;A! z9+v?uC?;6tcXK+dcx=)b7pQGjvz5LXrVQSS734l-clxdI_1}P6a}-yPp!SOFkeOu! zmVX5z9E?HZjx(p5t(tY>0Rw#Vs7Q89QiL@)!jnpkW3YhZSE0=NLTa25b z5sQ8C7<#XFoyGD|9onzFU&ywx4Y#ESh`)#PE$)f_vPDaMMUdW;8V$$aaW8f)*YHxG zBA7s~K8zic%|Zg?rGNEj1^w%%#Kp|dI6&CeB(2iG_?kpbcq8NW3!-oT?FaEV07H!2 zj(&(8|JRg(Nqw-1lEhWS_%7de2&@Mcdx)FD^o+xa$S?R;6%2bb;WGxQlku#!0~EWx z7o8E?R!gNV6cc@93;jjy<~1gnAgP}JZG{s*^*?F z2R1Mkj!1AQAPvJ0m1V%Fuv_VJ?Vpr-L}sbd(fVp6UWuCpA_;fb&4tZe0FGcsk9@Ab$tPz9IPx6(gaZ?#*)pW40N0=D6k63#iV4gK_@-Era}@IF)8* zJVqys*Kh=YAM`+RN$NQq3$jw3RcHX0h;gf1A526!{z2jL1)v(_H%q`f5C0fBZRbpbmUYNE0H{bo6o) z7|+b#>&g%u@rFXPw(^175WC_+v|AH{X)3}lKS2cO&k+@g>@MIni$G`s?*E`Cfhmf} zpg2kOQ--F1;fW)0&`RLzp>#bG3zCAY62-WHO%s zx31QOdNH^~jL#tx_9!yvfs3ejJI+I3*l*vQ=OaZIIT9gxXvCj&Q{uK^Hqw_kVES_5 zTE7B762L2lWmBoyC*0Pp>cwI!tU*fG=zTaL1WgLTbLMc}xHARcurqik=1^3z4;~$= z%1mLF6C%IM$RoKFhfdvtiHns8W(lKt52En%=^P3Q;6Ru)HYiovxAa=v5hDP+u76U&ZrK!!NQ|(T859sJtrWLL?I)k3 z4@5m^+Nfe`_K^5c^uUZWd6=1;&)WKotZ?T*puOdW2(y3`#if1)94Yw`o_p?rufLK| z4h5KNJ7PHPdhLg644*I03-o9$rP|`K-OOdXm>(bP-(f|kmwP#IxmQUU_kn(w%Z>qn zm3kl7s7QeFkNbClkelj)?m`Zcx@cFO6OLGbbFAL{kICFkX0smBv4Ape4Un2N8kc4D zFK!Yom74Ck-2ysU9~o8;AsKo1t;>-^eU3fcp>L`#)?^5mk^<9yIoejgYBr`FS>kz;*hXogI7!z|dc$tzdkC&MwW;P8bJ_Z}$s|OOAI3AmuQUP!(dBIJ;l&Y_+vDMb96E&TFnw zz!~CzCkbahApj(lzf>c$Rn$ZCDVtyUSO`Cr%ZKi#kHJBynU1GR@aCRR{W%;4PJ7-D zpaY4?h!ShDq)kKuHrAbjFu{e7Y&1=iC8`|cy{B6?Q=={P_udVLFg5{OZ%QklH6?2y z-Rapq*K%k3269MZoH{$%dGKU6JAB4$D0nQO4$o_L0g;|h>LNFM>Us*ou(1=E!xfLs zH$ST-f@dOxU*#Xw7(>DiT^dsOk(_dOtlxG~ZN)8oi>96s zyi$$IF-jv_WtaW-+v!@1Lcfdv*`vxP{%Xb0SM8m-JJbXRbsvD2@K`R{+@6I?Nfm^g zRM|V)#*A_)iQURLY^tQe)^!Q-rG8Pb(%PH($)q=py-}R*p|n_A!K5R^F9gnmMhjr- zj8C8#mqIXm0(}$MyP;;=PQT|!R}ex1y^bE5M9tBesDk!^PPP^nyB|5 zscM~{Z^OTsBnP zv5sG@wJ)jXAhLnWPAQd$s2slS8uki_V3_-I>M~#kKtxCu&pXol8G-+hLlfK6Nc$zK2~>PZD4C z?*mfjSV2LfHZ1#gqdI4YX@Oa+^q)mFw;9xL@FAKk_GX@{j_X!;>0Jb6jHZFoeuICS zsIq4t(*%=$hWj(|*6udrYhGyhyz1*_^@Pt0fVkX3vUYt-Q0Je7vVt!bf_O$=GhG?# z8+eofIK?#`zV+2T?00wI{nu|8pMCaJ31CjwciT# zeVRCS?6hcjs;e%^e11OXZYKttb9a z(=-teunrY`lljB9_=T-&t%EBmp}pIycZzM>XuqSeh$%F(FTq!XZ7p718cRTZbz`Uzl?S10Xv+ld|!}& zrlmyBUBLn7x*zorScr<(l@zr}K76pI6l_X=EMSy8$|=$KC2`g2H(E$%_I%X&4a1vz zwW_@MdzF~dpaw*BO}5LR7JQ$&OWj3ydJeX354!Va(_2cc^&2cNbxrcFVa+Wc&+xN$ zXb^o3A*ToZrnCkjo5v;XtraiWfU{P>u#(fYseX#ZXp>Mf7qCVTV@Ih>cdYil-^2Ry zoX7A8M?~P0KU zUgy)yn)4_2UR8J1RmmbN9Fj;hkJ{Z{?6aN&^S^pD@V>#omX-T~65sPMWN7_Y_T|!> z=xomKA6+8f{Yd!b=Ox*fKR>&-S*_o?S-Fx0j9W(QH;hxQUA(#gcURadZ1E9|L(F&U zv9=Fm^JHjtUG$at!_$NL-?nL&x^u+uR$;-(Cm?)TM8$VSHvfvl*2<~_qYTkqa74QJ zcYxOAI&I~qy9;6hm=6peFeAojj**U-3XY#Ny1p!s>w^1x#LIW~et} zZ~&Rxz)T}QkkB1~rcKOY^-J2Lo)+67n;!4rK}HQlMv{dQzg4#ZOf04f(lfxUVQ56TF*OD=fe&wvnRF5q6?EqlS} zfZ%)!74do68dbIFXEMAA?E7qGwaM$+vxV){{Mx+^aGo1y^eJc$lEeZYZ>Sm~$RH%e z-UNWd>~B7|+8}1#{4hCYaf0sk>X#lsi%~S3xx9|m<`e`hrnA!pegT6y{vA86%57xzP#cJm_1z5wJDX^R zMaTAdqME-u1I_d$op_KUKe3_9K6GymFkn=ToMQmx|H*;!MPzaA#;g+8OSU^LF;`1K ztKyu8pBgNgz$86pjP?;~*mTX1w=$H0I~97YMeSkwrCUUOhDaNQf+x1*Nkri{ix9H? zOP^qBMD<0r-q7_xt7e67gsqV?a9_zie|Sn&!&>x|0A0A8{!@iUc2i7Hb@IZECmZBEI-obRm8?f2h*}=bY(gaF zTI*_!e1{X885E2zUVhZiB0PLR8@la~M)L#^?7V<-TxPAuX%8-w#7UqI`^9$39eNS2 z(i)bG7}ktFaHy4?s1_NUgvAcchR>S5Mi#Ed5CU%-cxeo3N8<6EI_wLW!l> z(;cCWbxy&wY|ccbBwa}Iju}W8-=YDCFe$B79G1(@RcJgR)@Lx+MN{Yv#Ui8AYvGx> zz;Z%?k~%^N=(-VcXz~)uLf2*LTklm*Wl_U(OHiMz21C3cBNS!GDbbRp*vbNt7)SnD zf@Q2#R031Ex@0aG?>A_V^^#olAkENYX7rq7S-40-9~5{h`UBDdLQ&c7B$EJ#s+j;u z>ZzXPvj=Y(Okj2jhRR(Hc6|{7PeTlr@H%LkKg1X|;l_Dk+)e8>Il8L|J3NxUVmocR zUF@mXLPZzXHrTeeL1uyv%dkwExIW2&y7@d6t0 zTK@IJ>&&?G$tyQcYX21`f z$z{o=?rY^e8Tj2kk0SsCH#KL#R_QI+aQFz^qXnWGmpcWixBuy;f)H+dtQeX$ZiT5B zY4a5-P9e~V$-_eRtasr|kfS*lv|)ooL%E5z;-%}DDzn+82CPphFM3{P>*B;Q2{o-GIRvi?iQ@W=p%lHQr|-r zl2hMgISE+^?<1C((V(r8Gh{=KJ362WOXFrB*jL*lW()a`m<<^g3bxE@y|D(}L)Znl z@7|mH;Ea&4r0;;UcT`x)g`_G9g(LF&p)tz~FFQzDi7Vj9bCzt%$FIr0nVRh5rA2i{ zJcB$j9#zUnJa$J3GxW&j$2eX^qeBU6ti-^1XLGkdlF(No6 z8T}Ucf1;0^<}cUzw;|^}__ZS!ABO{Bd)`X~&@w@2VI@F{I>)zV&{1@&+~>%EkH<#u zX(;+F`(%tqugb@J6W3#XU1u;^Jnj0O(wRgG}4&6p5-=sDzF$!l`PU5TtLeG$*E9hZfwU5v= z@yEzxYZWV_7&*p&!+1tigGn^!}nULaA z{O!-pa*4||)8?6RZ2^Ba)5P@3TE(JP#z!yO1oZ}O(>(Ozsx)GK%Vj>y#h?{Yx5fZ} zDId(rKlElMy&8h0Va$)5s8A;3i_Y&rJ$hCGIZ5XvteG`K982dBK12{{ zEeu$C$L7V>T=1NI&kCoQ<@Z4C9|2h&Q>2eld1aa9OLEZM4mgiGdj+EjpUkC1ct&%g zk{d3FjSxA5W=-B^K?BlM_OCRAw{9y=aG*(DU~VX{WW!_`>(AXrKPQpmA0ts@lpCYv zUNI7UU0`(oH?k{|D$kNb;F9fW-quFU)&R%^g{%o6+_LzZs%whOGU42Jt-Bp3jHN^l z1#Y=0F*=%u@^n=QhDia|n2~T#i8=L36kwSHHgw@|#9OjS+`1LiRH8xg* z)l@uQS}}?IVApaL^$fUlZdD^ABhEOq*3S*WCZu%sRca(Y=cAZ0i0%{iyk4-*&@E{$f`li6?j zD2?@O|yMHCjNbuclYAmE28*4Ks zLTe+KvkxZrvJS4y@RHKlK~P(6x%VO6UYbadV7a7dwa}HQTFKoK&5=VGK!XQjNx16D z@o4#pHAU`{2RH_S9MHf_-$T9ZW1jkPwxLXp_|iXulT7x$Sty^8uF3}y!~jF@eB53K z@+-30GSvJM#m#?>z$79<9GSKS-}PNUH=te`ZJ>!sJw%z_|C#g@Q2FqYyr02Z_Y+*x zzP5>t`tayuN+f4x_Oc>S;wY1p5)MC_4aZg;QDobMV%$vGc>R=~PPMEWsSCZC$d*L5 zO#szCr#$BPkMjMm*XUf7LVzUT;A?M&x?+J%Aj!3sf))%HbP;YcWcyw+$CgaSsbW7{ zSH5hRPR3&FY`k<1kaeP#RGJwLp@>E$bK>o6YIK}wx3;t|77dq>HX-jJ#E-)r#PqNy zPZ`jo?JG2A728YjY?3}G9h|WRoF$MK)%q_Co6{Hh1!$e>6$(yGJpoiouYQ z*yH(H~F2O=yA`cCo z{DMn1mSJqf7d95A&;V{)kts|6yqE&{@-!2!*K}V?SJT{ZkxzCg zBfaR7CK?d-CdP43n>!BQjF&ZZKhDRW(|dwuaH+EDczu>(f`F;+l~9i2Qx``@n=7NG zfTL>d!l{ch?O{HK@iDWyIQFl}j>D%FcL}{rmgFlS)4vFrW#{R4KYA2-?0@oC&Whkn zf3Aor{oINm3~&)V)TDps+5GCSEMAv@&@%^*8W+=m$;zKd=BJF2Siqn7gf%#3%Ylm; z6L7jZ?cUym!gMxvVYz(41j0{~0KA2c`neSlDQZnD<`bEL8~I$^J!ecRMR>~|zS9FS ztN#zz(B>{?xc|uSQzC*Xk-rGJ3ot4;buXCMVcFPc0&s@q7&Xj7Z7m9g3p6`g1_#9^ z2IEO48s(2_y53Qd$R~!I7>*fRpCg0{PaaCAt~r#A2CLP;wgEO6j=rKyX&bbdu&vMak|nAd{;w ziSK+80West4Ozt~p-lr}L2rtTK{_4D=(TLf z?|$Mfy=Oyyvu61j%5U9GB_m=HSiLhk(Flax0G1~=646C9FalcGkF)uS&jU4!Thv$QY0~EXF^wfuAMIQi z2dw1J1W@9bq7FAY*{_bhr@7xv60|E#lpTd~MSzJU3}VL3U`s^}W(QDxIh8~ z{N(}!76&LY@G^tUc@hnj$NgZosuL>2m z$=0YFuvdY7G4#2>!+j>s|LV|_yPyM%P(5*f+`wi%#;kmJ?R(zum)Zn;dxoD301%Gi zkG842&8en$9OD0ghu8SHvg05&N}7U zL9|Lf*S&$fDkldj(Jdmdyx~;^XJznY6`Drq_mrLg2jo!ozBI34%Jm>Zdxl_Qvh}zg z!Pf}#o~_-UK-_kucF;n=w|cje0Sw0;I!@X(8&7);Q#OOY*xel4NG~eWxpP<~t+k|+ zMYIHwe-pf2e+!$OCTdrsOjq_3@{l2C2yVMK-6~B0#XyY=zrjoHCmo~z{9z@(0I6E0 zg4E*DNH$6s)i;q|Qi<%4SBcOqfnKa2BM}@L?O7aF8)SS-LKKVWiyE4B15DjAA@{+n z!%YmU4m#jD70=Aip%d`tQ(32Z-Qo(mHuUae;3_-)M}aw(AqwZhiSIY_r;R1Bm19#F z=se);5cK*@hQdip=)a`yYBa-vdK%L^0T( z&eS@J+~Dg-sermZaEm=TuJ> z&(6YC=u?6y1GLk2)k~I!Hi7j?P%M@ZE>Ld`1J|* zAN(VHR*_I&lHU}Jm%wLsp;=|weG+dWHp(blXDl(z301+bGV z4IOmzv#Q?d^an!yXvRMf$vrSl1+6f=jk(=z$Y2~B<%aKjqcY`&2t&ZWn#-m8_f5X4zBTTZkW%6F#$9Y2&vNPhx?L%bla_PZFTP4fJ}=1d-SmmqlLyt;a# zJqa3p1i!AX_S=Zi>W}r}rLn+ZM@1-t^oSm)WJ7vt1KS2(oAq{4r!}b(4F`-U)>IAF zp8KZ0=^GTWN^#uEC)JGJtXC@m1x8kH26nJmvF&EUM9&3u-6Km(O^Kcyet&@zqq*u4 zL1dnal@%tt|h^lVt+qCU(>4VZ5!UeHT9mV9gsUSCqeq%Lo4NMweMvR^&{A3nW?QY=!DBgqBV zs_w)!7H>(5aa|wTex<9(bWbr<@}pO#SWAq@8Sl-DuvUL?n{sn?&Ism|`C`rjvd#F6 zFhs2TqQ4=^iwEChkWks$p)y)>tfr)YJm{vd*GW@H_zSJz)7@L2+cVP9(&-HgqoVrOwxgN(UiaWPZAW2-fGK2lE(l@q@XOA{Mxue95 z75gx0r!*T>NbmL>itTS`Q#OHBt#%L;5Ni%;!o1t6UJtkcq)h$$z3a8~I-i?y*JG>V zUndv2x&}`Z?~ouh-!@@`z}i*o?mDDN9;Mqlm65e&52J-=^8Cym;}Uqf`qwlM$gin{ zqz>a_K05lD?8hcjB7V`?#Dp@~gfx3D&?5S&jGiB1Cs6_c?7oV%Z&atptUe4%ZFc>p4hU0kVh(0X0#(Gj4R=p?Yl}R#wI9iY1^!R@pNje$t_Yb(@4S#QQ0W3^sWEk zM_AK6hdSQw>q8?04(C1Oe5vfu?x#a((;uw`Fu8{nB(=m;Fg_9VBjw1A2Q{?R6z@4T zJ>iZHHkj6z^?E9+jGps`S&jV?#+ql~roikF<8#3Pvft7J+RuD>IdGUM1;C$Fp6H{vfi{wul{GgT;hh7@IhfI14F zYT3mO$ZL$i!#3(0Q(}@fQ9`5WU=%kRhAOS^Q+r(ok?LjJ(9%A7<|sL{&GhD6*o}JZ zop8JVf%xtlulf+wIwCKOpK_rM|3XAIBKUlWiuux~h5Wae+j>pM+tLJ~Up-dphbs2v z5rtp%IWNU6p$p<}J#?{87A&ZA12T(kuHfSrNVwNCg!HYY1X!bJs=d4D?$=@N3~95_yyN==UY331=e<1K6;tzMOD)Y z$W%?e(zjlNuqKiwYHM)(D=0tgyXwO#6}vIE`P?Da$5d{8eHw1*4!KpNwkMy#^jKAD9O#E2y*nmO)Wh{>CKt8m>*iwCx zJ_H6V&q!B8pn2UcfqJq0V;yZeFpweEXdo~4#P;$6H~g<6-+{mf=f2%KmZMd?g|TBg ziguyWdVF)1)REkjs?Pr(<|@9i2ikHM=yeL*-biVx@QI$4Nb4VZY}qy}Skm7?KwK1i zavDy2Ga`5V%mhwVI9Cu%uUDL}dAZD(aX{M!8O5N@gfW^l`6Wa@ zJlU0h-gc|I-}%4UBD_xY>^Y?MBdO5+zz1R2(_Y~zzJ=gdGd-&D*FVT2VO zz|+gp@eBT}P1|#W_LBC$_sAJmF5;F=%-{~rzM`7f{_@E08z?vtiu^AaKr9Urm?|6b z@eRxmWrHqnRN+6ela5)tAQMt=2M6Ue!(Dm?`>@~aXnGFy`|I` z4myRuXrXBvwjsV!-3C@m!#bAL-V}N~=!@ioN&ti_6S00M0+9_sUz8Uj#TSfn&rd3vI7YTa_s@sncp^#Q;z)XXt#< zMZu#hz6Eex-9de~HpKrlfrs?AK!dw(hZ%`?zCttoU}WrAy37$K+Y8HzDY4kBotEVk zdctdTa-2C7u|-NM4pcEd3-kEWx%@z&yTB}TCYNllwm(Eie0nus^8gtvAcx11# zu#%VfL(#yj&ckpDv!~b>z65vfh3Pn;^B@GwNickO z-N_l($mZqx^AwusRS`(&tbFCZgS(J14e{!%c0O>%2>$-YJ~|sK#aCh40Huq1XUQ9!YxXsOa4bVRbcj5n* z7Ky5!gqj>BwltcvFL;HNxmuQmB%SNQpP(XnL+P!J;P0{I0X-Q#g4*qPnP_N@Z|Dg` z#ERC~w0{O3AEP)~FO)mlux(;KH$R^DMa7#KGG3mXOU?(M1(7&gT;uLY9>WYZ?0+$fS)tu_MN_EjFAKFcmNE6)HDpY`ymw!(cE%)w9GDvktSLHY zaI;hqYQrk{03w3xZGoiYzB)js=FzoPQ?OWnQ{mPYH*2gwhJxK{X28Xhl?@$vZ7(m{ zE|2J2Sqjk($AUJJd5Ra4Z&93igqRS3+Lkj<)Rh`|vCwMoVf78Fr&TWHqKuj&V>-bP zXjs`1FLJ3dC9vHpfWi^o$xteZjgn_4;lB%)Ar=Gw0z|9^VsdC+=;6a$P@xpDy6z%s z9>ufqB6U{ui6s}E=aHl}bpwXPZ}LH`twujhTEdhawGIXt_`!;c)n>U8W0b#Y`$*#0f2(ZB@0)D?tRQ+2g4}%woxAGA+2%4 z#5PM9JU4DsDV#d(@}iZWtrgvTfPAe<$dl(oeoC!WsI7$Rn<+hU;x^q#49+4P{P(e* zK6j9Akz~p2ww2mr#kTTgd%LXAJpq zo}nJ#MO$BvS+nfhOS8j~{3zJ!ctr0964s#u;&$P9RST*{|1d$BzcMIdI)l*c)pC1Y z^t%Ic6pGD3>`QRZQ)}|envo5iHax@sC{Hka8YcZD!edidTR>#jc}Sb{I3ssFo^KC&h<4 z+uh>yMxA$ei`iK7#gYZxTIkrhs?Y$$yUgR+T-xW@IAPtm1{L6`GN0)k?SojlU?DV& zW)P0`Lel{8h8%p8lOiEM54lEa9*Chm9)m!*eB&Sh_y5 zTKTZF%XJw#5j-t6g4ZP2>Uw{iI?A_X(IstzAf--MiM)Wn5gO zK2K=AN@KySLtV-P!WxtW+~G%;^{ry?$vJ4y66xuU_BepXw|$f`^s=OI(5+GkmE^gy zKEJPdF8&Hl$gfi{Mp%GKL-2(0_ZaZF$;hXR%IaC??g&`R`%r`E5iZJp*MR%+P{J;Z z%6=GOl$EC@HDJW#LdE|0Wn_xBTxL4uKJ_w>T!Kp@pT?cL#_(eH`)`CK6xuA5c?dgI zXd23#>=Eo}kZ*!-gm0#u1;re0hSHjmo*i8Y&;`Eb(GAUg*7B&(vxW^9W(u6 z8tzogoC3fal`+#j+&$erIBleFcFOXYzLp*`eKl=tj@^{4HBfE3YpTv%mBBu(ZI0iR zyU}lJ%E6pFZEg6=-LZhJRZL)obb6_U8E@7dgNSII?=oQq28e8<_pD7VmY<3-(gkY zOM04qw$R}Fd%0n-eYr!~4%rD1a35PAI>LyK!hL;z3prAdzstPqaqVmtAMLoKY{;_*&}=0Wve6#q5)7bY3w5$gLuf*hcmWC@P~-W*8`cm=tm0P z{s~L=mDt*feWK=hnWPaorrDoJG=U^ke>vWAr1w@>YW=2#xW#nBVw0ScCkFu5 z_s_vT4%hF^)a$=>S`aQH%pSG)vqb>wtaYLj=juMna#3Q+u>Fa` zukXJlkA&jO9QH60g+uXeeWN{xnsPurPiM88{lZDZL4)}pcMU_mmDQ^ zFN9Q#m$MS?>c%*CZP z0dEJwJUu=+KGcn#j7&l?(?q_09sT$D{B&?EAUnDTK7QMDz+2m?=KpSM>6@Za;N{+U zzMm(OPIZCyzKX(Mr;2oTmhaBPLccQ0_7l~7RUG7868zoa%>j%Dn5ioM@yh9%-&Wk& zb$k6m#tJcl>#gf*J8NBY+nWVgoh79Hsd7E9>_N0I&yJYzlr4rTw zhtPKl(b6F6wV1g4y`EvHDBip>l;m_PkLL*{>jiuy+ZOUye5E3MfXM(wcoPjl^AseJ zVuJ~Vu8)Yq`+W2nSHiO4*nKlDoh*Gn2^+WIP8c~m* z>-JX<+4+B_L|Vmq!wp1+e{m;SJ@yF(+3o^+82eoZrRw0QxUVxBskpcCFwzYBEz|D# zJjaPbLoU?LVhNh;h*U~IZf<<^cK%ellX`Luu+wCtAtktg=_G1or={r_TvAo`k1Jtx zXmIp5c>1|kOK>bc7F`3bueUW}dxPGls)-=B-{FZc2BffVhp&jjm!gJJDNE%>!(}mt zbkpA#YF4M-T0iTe98kqOVQ(W45Mbb<)veMBA)JmvT0>RZAvtkv0w;?iTh{%3qql_J zZ*|eGB}(Hr#{1`xo5}Fb>U<7i!VD*skz{(g#Yq=r3kbCh!ye2)+XzI4&9LS8lHBEmkkZFJ80|3*6<^e$h8_h&-U}ano#ycq0}rsWWBuA z?2`mK1oebcz3t0NZa{ew9-O^@v@QeE{@diZaEkGS$$m?-tu+jqIQH_?e?S$$jpX(_ z5ngp+#umY-&1eBwh4MOu;;}@e@cB?Ul|)LPZCXxDIU1 zf$s&qGaX8J%7+rxe5QxvpqpiWy@Lm4lzP9#>GyLefldQT#r}q()?pl8+HWIum_lOr zt|!5b4dW+p{-+{xa<{xDplJQ4;aY?wHFSSGT2Td8_WK$&-5Du?Y;i!Zc1K*Fm=smS zV+xL|l&ylMmHPt1&z})3m!5`-p?bP*SY-cBJ)P)~^uyo?;xybhe0GDlw$rqZKL!~r zd}231WOfgLfYMn3mh@C<|ljpZRG{f{Zq_?cq+pcMzP_Xh)BQdzv|Bmma^Lvl0k%LcB>JYXea?C7mR}v*Ao*QD(&3U>OwJi3aue`>A;0CAD+6$m z>$YX$sF2QE=wo4Lzg}m{by#FjyXoMwqVlqz-go3C;Lx=*t=(iP*6Elu3g~ zY=h_wTgf@O87Z0e)8S*5K?i)@VC7%?%o(7!tZCSEn~N!);0SXt6Hk8I1BNtoDBo*0 zu(b$)S7oYh4?<;v8yknRUz@U!Ky4WZ9^;_rFx@%7B*>^iqs(JWs$edJJ*wL7_50_H zAiX!552u4qQ#R~Y%O$=%P-6U>*iLuq){?X_b#ir6j6-Ep4^6}z+ig~ZNDngbD$esB zWjM{K8LmH^$sp!>lV0G;E}P|-t*yPJkqC@{44FO@UPCL$bG!N#ic5oweU{+>u3B%* zXfp5KRXJLU&H(+9>EX9O6aVqtIo0RBhqCB~(MIC0l+oeaBT?IHsS*T}Iwez6bJ5~w zHKI1gBm7qnnqhd_8xWl*31OGiR4=kN{OhiG=I9INLKSs}N==v2ZCJ9MsOfsW5?#p! z=thOd(&}dAchZ7MEO&E^Drtx71D|Etwz-C}hvP4AtZpWXfKZh2!0e6j%k@g7OSUKT z=m;{7_&7UqNHAi2yVc%D?W&U)J%bZ4tvHXDW?RejF-Sg$u%`E7K~+kNk@%Egi;5kP zF8i5a3VPf|D%`V!lTj_Ry2t76Wxo~$tWN}Ay-7^o%P>W=v>`$>rN{;VZ9u+#^)A`Tv9q`C9$6w;>e2p| zhk)GS*W{Me%O*bVtl_;Zh&jv_GF$AUfLqFox||59&32h3_Wn7cJ|P)*Bp(Az_cUqz$uut-`61$Ti=K?}Rx zxdn6OhJyw{B?S@X+SO5U6Nh*K-qI(9uHu3)K>HDXUIm`K9|&Q+86(K;g1nFJWBV>v z2K<9^QV3OI6-Xrwe=}A}tBa-a?pk8yjSu(tD>Ubc?`^B7e097D^WyB_Y8O&MG-@T> z8)N_}4IO(mTFVf$2Vin{KX3)K0?QG%r8?7Z^lS+`*cct!odln16PUo%1h&YD-rt|vpn?5EnKu1sTWseQ?rwWnimi+B)%4wDQyrRJ@ems za(pdDC6fGjDef(xAx@Q;$WfrEBqe3Z0-MZ_2E7F-4#ENPgrldo~Lb)-- zOdD2(&Md1>)c3m*OmhKMh|{*9rP-w|5f46J&#F*U@&qS2TNgkrPobEvYsPv&Js~UYPDtt8KsS*x<>6mute7f?+8{Eb6PSe^T))) ztN910YXkcgLpS(iyyqAOu;N8eSxsDR{7RY3b-9j%u^stpcV76AU6e$`t7C_06avGn zL#RCMHp^xwn_+B@ry;yNFCL@#fJ(VPKo8lxrf>oPtJflDa^3t&$VT%LRVo^W#IO^Q zy+5ISJ{O3jr0wy??Xsl1Q9Z9!+Rm)dqa61%>2?+iZ(LPYoeR%|`K6R@Tb z(8-4zF^{t~R}a>ay)`v2)SRn0v$o%S$AoWs+j>4#1PO#6yxdrR5ik3KhXTjEo02M2 zNXdXSs;v=`8`tuW049>UAo4`sZ+-*@h%A^SnsChKuADk&RW>e!WgT88Mv&Db60KKA z7|tcyqx%OTv{Z6(dKBT0=RDKEoM$?)_}82P1-cFH`qfx3!?F@jpMB|z#Y267CTe4h?q*O%7$!s>JRy3K% zi?#?1dh=lnz>e7|HeI!sB2A*}s-XpTV0>*Lh^c4TqJGfl*auA9Dq`vnJv^v%D0 zUP;$@WTc(0iQ?JQR>zvN282Ms4_I}OzE0;Vb9@=s9+rm`|Bb5CvHd%;8IzEMKpD^q zrZZ$~vtVfc5?2VBB3$s#SAjz4Gver@Iacfo%2BaoD8bKdYq~QXg|728MdKlw1z|i3 zxE;90?VBww7nSn5JOmay@T1lbh`97WORP>DS1v5{BXqoG|HsMVMLWw z;A`!b5|?}7M?5OIBah;RwV_KnDFLj06?KTLoN8NlM2z~+w^*%o?LBkj_)woh^hVUJ zxVAmFuI-HeV;v~JC=?$pniFHH@!UgRq^jZt9Wz^0&6MsDl*Ma8w<+t}9cPyjyU$iwxk zQ1d36$&BDR%tp#6;}4DtaE`lqM z(xOi@Ty4}F>K_HYCZG!%pD)&jtV$+6-9_&D5$)lP*T*IL69b^MZ44QIMZ_%Z++1?I zkTO^Oke}zl7u%tn-l9VNdjQrcetuuB(Yr{G3NO68I*^qcREM|QMEo#~x#D-@G;>xH zR#VuH-#Z)hv3)~}TIsSj|IOK9==mLkm7TZuDqiR5Zqi?Qn9Zf6S$}GSXlC=*7 zY3;?u)6>(}uGhKo+QunD?ay#-1%*HIiVUIvM|BNDb{acMWz%mhO_?F*Z`A!Gsx_!b?G za;}90Mf2q!^4RZMo?L3TZ_>$tpJe2>x>q!ej>QyVOQ7q znCgVSA<=mscp0L_244Ltjl6HNM;%!~r@0Z;Cla{@5Gr(k$_H4a__jHnBxvx!F(Vn! zx>g$Xy-HzxuZos z*#cB;WL2bHM%BQ;51Z$XK7}_|@bz2mRM(as`Nq*$PL?i~!xZisN?!?xj04DZ38HuR zyx=(ZE8C6b0+=_YF?Q^#B;3nW#OB2?RJd*OS}B&R+z9 zQwA^7z?d)1C8cfvE$c!w3)9k4a`G{YbSIR>76-rSV@IMKUJU(_x_u z^gvmMp0}Im2hq$$SNTl>2Sr>tu@Yt7cnwue=_o(!KWv0>ZDCYyokVjkKY7x1he^-y zW0jeaTlZd-j#j_Yvl?RP92NftICVO^f&kzlAhzG*44vZlhce*o)BiQu zwgb8{$V-Nrsza=F)TrhD#&RC1YZn++{yr&7SEoF{ zg*y4_U<;G0o0#0ojwBIp2@o(d>zy)>+=Y7odbA;?Y<6-3SFkKz-M8$OQ;)*o! z;aJsDg_?t%TfDr05W!)&TVxq+)9K#O#BUlmUjWg>GjjTYz2b;I2*$~IHL_!6MzQ$; zQIWzXT0_Iz#VpVOVjx(b@n>Ovhpq%v+FDCjt)09Bp{TVp6d}XZ9Z*;4Rp*MS9(=1O z_wA@)d0=3JvGNPG7$$$`Pf%y%OqPX_mzo>Ckusl9!Bxdp#D8Dh0bEqasQ)G%Dgbr;!qt80 zzZ#&U=n_<vwJJl0a$Xe9V-z=2#j@?)C~4$=E5iO*FX4y*G}F}I^tR|9Q&e4 z%8%YYhSI8q0GbNgCcaDar@Eb4HMyWY>(_>SO8fv!A8WNeOw8ScvszutJ zmbGcQ_CR}D`|+|ZSMOGaKLRy=@Sbqdu1!$~R_$*J&I=vta7}L$tF4EJogl9TVT+Fy zLE^la8u6KV+TwGL;%#~TEtRY4tD5`vlyz4diPwkM06yI;z3k8d8v!8HCPUg;_KVb#%6V*Is(!MxFF%qw#w7Mw5w)xc zVOw#>r|9UIDXJkiOXw79l`=70N>pn&Mi!ZOJV9?cC%2oT>R2bkxIZdTjfqKaRpT6O zC-uZ106U&?^sb}H2Na1~_OTm>yR(B`tIsd0FNq!F*0KL0p_mO{kCP8kuxAH4xV^Z+ zxxu*z@#nWxC-0y_G|E1zn?vMZS5{V9T4z@s3~_$5w|a+4N@BbQi$k*BFsU``2Jqx* z{3=~`fTl_lBWB%<0b^}$>hTt+Oa5V5v1w4dKnimzO?HK) zwEMmz(C)Btkn@*%Hs?T3=4-niv7DTQ0w|luv~p!q>8UvxIU!9{bQ;rf-2Rf8U z$!lL(9*Q+0j0Abui(Vgle(#c#q`y0BDGMCCa}d0?#ut)hjT65XPvNHtM1(84sRpX* zRBYaC(bd$g$tQ7sAP=Ko1~%5482`()mg|46wP0NAZ0u>#OW-srljY%{B%#-xsoM_<<5Q&utn6wa8xiu|8{ZZ|{XxlxQ4FTR zT(8sE{JTXvbj%NY9Ityrig<*1x)C-(Dh!e={jkL^XmyKufM1zN*q}ju4Jzy#NE&x{ zJ@M;{UO)r*tcL>ELL{aQPEHc8Q2r1F3$D0^mY@uqL1n4@>QUn}Fay$R)p$$HkwD{5 z=;i>_R&;|3r@YW~XpcN{)3phpFY6adQ%ab1r_rL*w_x;~FTGRt$g~Jiv<=o)to}xd5 ztOL8qza7F#fmTUib4rZ=^!i*@Z>Y9RP2PVj??7&r=Dqonf=a}yRO}IrrIwKen$|4r zA(`SEic1xg71+u2693|fDNy;aYtt5JXC%a#CV8PBqukIYwdyE%Mr&f})(EMC&{s7ra0 z8Paky>5-)+a-WP@DI5_?7P4syefP(mlnS$&{72C>b#`&*VhN|zrxnuoDu+RbY))x! z_e&uPzfaS(7PT@MGR`-FH~cvre&)B;H%*J?b=f-{pKTT`b3ZiC@gy7k8|a_!FK21U zYv4G5e?^?Vn0!6bLfV;amV0)ew9p%9o;6qXph4Mao7h^k*~arCiD}Lbt1h$l>;U?S z2wvs?!__$kR|0ieyJOq7?R4yP>~w70*v9RSlN+;RJL%ZAZQHh!>G|HOuV&`=Ie$;p zS!=KTtj~swbY<|X+Tnf?EOpCF3&}gxy5PJIgOOY-q3+#*4^Bigo}=WHM-LvNU})Ae zYw&Sns#j+t289C+KOch4_)4fFN8zd1KDf>h4dA(jw^-wMCcxpvz4sc4RP0>X@f!9e z(v6#MQb=G3tX-avb~f>-TnRfHv)8H#uGtLEFdVS=S24A{61_GYJKN>=S25lJ=Ph=gw>)ktG*yx8$`NzPH!R6qFWF@jn8XMQve0{JRYOZ6!2QD9@(JdL+Cey& z6%Y%PMkA;u&!{esvIajgHX$i}Y&AzZyvx~=34A}2uwVMSiiMK}THod zR4x;s==!LxYDVEWXF3(R2PSwq+MXB)2Rt+^YR;8$#mu7XuUM;cB}K+Yo~xwQ3bTlJ z`z%ZaqD^Z>WDLx_cc!{hsrd#iw)SBGYxk&7L6FybhzS3&KtCVde0=@h@+Vg zaoMm}_eMMi)AopY%%yYdDmp&yQx{QognSe>I~1%D(LEZTy+*pJOhCTCSS3IOkfhol z%}t-3cjH>+sPr`}B*Ma|W5#lqqP{Iu=^XhQEJ?-6SkexquiUkzhmsuN^A3q^$9Lf@ zxridoRC!UX*d*{vS2@ifLscbCFK!VQLfP`XbWr$1Q7(Ee4~O`(J1XKXuZ}CnyvC8r zV)OpzqWh2BDd);hNCtEvTYB0 zFCPpPNHO|yZxelQoAy;gwvTqXZKC46vG0N-hoyApW1PknRenkeb>wxS2nG=K)QIy7m3sN zZmdU6l4efF+{U^W`Hh@{q9s*d@))9YZAxS>=`ASfmAwhOz(70yELFz`;rdw+Rw|DJ zs$sut+uDnQ;+vmI`|pkY`%6K*_Onrk-%DLSiw@)viFWQ@K12W6Q*h85v7^?h zFwC`Cci!aBuN&2?&emZy(E;;#jVs#KzDwFIxn_i|z9}k_1w;Gh^?4_kXB2Y47DYL8 zyNvIfr<(rJc9ha;7Yrxfv|8s@=jZeF&2^a|(~VO+3gOuY!Qz4~kVyx@WgBWEqB;|P z2(>F}vLdPn{V9P-XP$}g{$B>=Yf+58?43taYbN(gJcTs)Du@~I#2;XvXuJ>wKM+m3 zM>I(Ndzfp_$LWWJrz5~l!qi3I&eX+|NQQa7q9Iat#2RJBhat+;e~YnQDu+rzm&aO{ zrsM@W$=5VE(rg9^*lz0gk*R<12d^n22TMulaP`DZ^*bU>ZdZH0-?J@_37CRu+!Ogd zp}$lbHZ$?}h+wVg1B1u0WUr5iXwt9z;nNvU)%51MLLS(oZzd05|Mo%kDd*V-S|q=J z*Tx#W%@70kgGwV^SvG(MPf&LXPm{1q^MLNDkiw|r)lz7_oDzSZS*1yxFL9)AA-2R{utz^7jH!~1h(#+hh2 z?v8oPy!x;^;6U!$;+!%3$l}(ufoS~*$8Kf<<~BX=3?THXF#{Ie&1XSX{5~Yfy z{33|sJalb}De)4jN~=FcY6fd2hb6v(p!G7@+!iKJt}^NN6a8f z@$wx44Bu}HzGK3VREpRYujk0tbg4LABA;3|8(mVKs<`Ic8OJOc#O#Ht^XzRQ({#dm zxp?H&3DGq?zsFH4*DN56D?}YiqZ~f}7t+F@^l%K_)v(mI#YcETpftl&pMG;G@w!j1 zO?M%_60mgtV3FH#F)*N!MP<@bGh^(SkFDro@)U{qkM|hd+%L=C+1I7FCwH$*6$5B& z!IxtjN=YTVXH*V$mM~fO#HIX}VSE*U6A=hy&;BMKgWv{H;AD7Gp{flSj6J$-<{vnq zuDz%?jV#bdiQsY8_{)XFPeDLxPSI1*OZBTE0vKOs6ep;v)Ns^#Kq~PlH@iG(F_A^GF+VqaC;)>)YqDq6hL_A0_B!+ z%N`v(%G_bxcZ!G@d`BtLZAQmz%#^KZ7ZZn4iZDfl*4URhU2mQ*D^+=OT+~|W8s+A* zyr&RZ8XXC?0wIGxX96YWLq=_-{9cF)QhX-KaX0L+NyZm(UbhCd*RfR4I}M4<;(3R|cGQ}DK-(7-NJ7`CDOflnnALS;FB`eYm+&Ptg8ak&OI zn0@N_2=cRFI+&4v(O`|1S9!{2cy^L%C(y_~Q~Q~ajAqoG9Y8}!^*#mpNq@;uFIAEA zWO5pd-u2BE_X2LSWn?h%{GOcF5-kU6T-s3^O;@V*zlQHuTdQQ8Nev!s`{Tne& z^@N&TTYt2=C|=opN<~xmP>67>`mAd24|E6b#_z0$!-|MQl*}5ykb0>Ay{{>*dZx7azA#CW`4XcoU>u)yk37Dn2?XW z58&_+Lu`tO5E`dQ_nse@hVC_i;6LmmjEsR#vF?e@)+=|2VL+hS!7PQpyF|j1nlDTT z4Dw#f4`1(_GT7f7bL{R$iVgoR0^aXZhK2ml2up|A3`&uS<+-B)DRQ}NDOW&*3j21X zEj4e9SByllP_yUj%j+MoOl|3V41y)P=PXb&u&a`S?RpvEb94BxHw1$^2XqpdM45+A zR+DPqZCczdY)z=q^U~jp1)%$ypJt$)D)h={`(H_%K-ysEn%;rTRr-U$@~n>e_4lI8 zaYY|nfjWtA(pvnbE$)ZBtEBZWcB@_Ln23&8F_ltV+nYFdf?E;Ka6K}y_QO=7d^=?X zR8&n~R8MTYzKjS%PTFGUr78CNQ072l;E)|R1^tdYy1y^Eo8#948Gs`E9y_mc`pP%^ zEW&aR=LQUK+(FxQ&};c-j^c~7yBTLf>M`v7sckyc9*3xk*P&qGZ{h1_xEO-Y3;x9oVRg6UtKby111N1U zP)g+ilsbRP_mp7t8!9p#DUiV6qf<;jlH`jZVDLKkMRl3r;G$Rv_0r)_`eeGu8k*IP zqr|5L}xc99eUC|UQjO$dSlx$c{D1GB!e zW9oC3iag2a)Wcm>4Af^OE*cCS{j{tC;$4-)IT+#5ZHguaE$6)#<14V)hwO`g7=J-{ zzf8 zveU)=G}b@-qT;uIDoco^8tZ*T%z|%z2KPhs!&X`!^45GkJ%7JPbWhkuLS$Sx!zn~B z!WkcrXS8EmH>HAt|CN+YjlTVwZ`3ckZ! z;^bOYq=gUK1iQKRD0ddN*S6jOPBh|CcJ7zgOuJyY)~Sl1zr)J-4S=kr5v-xUnr(s= zcP#tIj=P(0-B*rPi&_c_Bww|e9^XE|Z}Y!a!xz^ZtSR^@Vz-W=faf?jVOC6_hm?=b zM7YJ%fWmj8>7wjujDB+YCaj3VD^75f%OL#<>9%)(kAjD6hcqJ-_>gwOUdR+`cp!6!EF194R%cLS^;>peA2vt}h`qFAz7@eEn3`FRO< zLUw*Ieqcv&{wTUWP+!feG+tCu%07gxgOq|K($-H-V@lbtq|KEk5a&EhfsZmKauHgD zl;PHq@Vb?f=-1|`T(Opl%w<+Yi2)#RboY_DHVns~F=RWjzr?dpGw#QsEl^Uoi!t-$ss@qJniKa&)I`dU6b|<|QrdHd|Wi0dN=AJ8~(928B!T}T6{fJ5FJvKP{ zXFK#a3F(L_CtSy`qN)$;XjdBEH%Xt`w#v2`w~SVx2~uN^=38Ucx=VGd=BVyAHH|Ii z{AS&5mNcmUjXS096p980GUm9?-=`zx<`ha0cmaA?a)@aFnci~Tk&-M=f`1s}8T1e9 z_C`D-xLAOVv>n~3c_in_)VfTYTk#Lt(+H+ZVVLAm+~;=xb$t7#Al;k0=Andg zvBk8P)D|tKi#kWPeHeG*w6!XpyVe7R%oUOqb0TZ6(AA$O=Q@k!XDT80W9av&U^UJo zaA+9fkp$W0R}y3Lrd|l{$&%xd7)hR%FUbT3&W6KTJ7h|ZluB)4mrr_g{F6QE`;YrJ zc?(JbqMG~amKHf3xWdDp_PPtC4XJ(Z0ejV8H92=~xn64)o%+cB+yzlg7}UDACa zLC;a|2fpFq_o;A_j-irQWCWkG@S$1Kz$)Eojt1Il{wC4sYoUm?pIb%)P|oO1PrCA) zc)fX1aWC=Jz14cNip*MFPH~8YX9qta7TDNoLK3@`^*OBw^CR|r@`)aUpaNLh-a40- zX89evF4YZxx_BYD_8ffpoz>OwD)kzUiQ{SMtexEJHiliE(slEnw23j=Ih%}vU@zfLzRxsrnMv72K)MK=wI~5G zA(URT2z`^*T2lUkXlt+XrO0eX%W#u&w1K>!EYgkaM0-NwPV#{8$y;Bg0w#d@m5M zts!Ms!$Z$ZfDJan*vlwpp~kdT=Gb`)D%}Vu5^U`FdX7SynF3IF{D{P9h2x@pkEoMn zN9g`rL^>%K!i8u%Ph)%Nm(RoIa72xDiv))^{~ZOB3R$fbGlE`2+|*6O6c|{>m)OPG zlWUj1xL4n`Z3PTpZ-LlNw53VXJHiy-<>#48G#_PzY mBHwd`UM!NNlpwfYrDOt# zI7C)PmM&uVqrHcRMB_$^%XIi9GO>v9Br~d(jrIc~e z&P({1!=!g8?Y;n-3Ti$F7sdjkUfhc^W*O0kU#NY5$az>|76xF)qc^O2zB4w)IWqj2 zXY}#~#JgnZ|A=>79}fUVYsGm}P3D8xK^!ugO_0l z+V!9N?^#+c{w|EAE*5!nb;Xb|Qn3_Ymga|9I4NdFme^?TbYW{WeO>}cQ4BzSLSqwm zORI>Sbjb@uvjZIbsD_1)gNtrt^_K(@^15&vu>iT#E+LS)_2>>e$T_v>b1_2HXzFKB zWvsC1zzJr?H(A4VdMU?$RFt``onT-$~iZQJMvHfzfnq;uYFYq3mXMM{=!KY$QxOR4QOe(J(^Zsw zkCe#i$$Cb*r6LTXCFvrMjiBn7)qeIj#eq^EyYc-<=-$;jMHY{#?)Qt73=>)c{xu2n z`^2!BwMf65t(#m-+nn_)9PvRrpi~YpHs0!`SYQ8xD@!clMnR;qhMK5|(d}twB%=5+ zKDmL{fhCNV$uFEFkT3kc=|M#_4xUNQ87yINOoH`#ee4^X_^n%rLE1S;-Ymx+mQKe! z!92_e-RMsEtrl-NJ)=EyD#r|?j*5Q~@<TZaWM_G zW$feBe^}R4@oC=%(2x{fbK~>B$O|E@RSfe_G1fDzg6mjl5y$XW`ZDes z9gnQ2K9DTH)L0QaPG?{1Tj9w%MO_eFd}``L^nY7IvQ<8JW=WN&++S6K`fSq89~8Fcu2ol$NI%|XUfqH zlmQq^iqtI>75Gzgirp=gIGA`!%`KD$&?A4fWD=wu^dw^-T38x<608i6e(ia0B&@Y5 z4=GigTKeggqy20n*wcTVAwXM zxNju0j!hixtI28(q^w+|LaGENi2~RhqiW!~6(^~tGKuZ8|d`YI>m(ge;UitF6kG^+&Q zC0f!dFyUpZ%MX4s_TE(|m&!3I0f%EGIVWSyXi~-SACq5h)Z(QMa0&8|V$^MVzJ+VQ z>NUj4-xVlJNi#s`lE0c=_1w5vPJ_(q(`o3e=;$j(4*7inyB6GV%{xV$-$N%`6)%g!DQDux z{MR?@z>UiPKcq~Nor8i0WBt!MDaUtSw*R$G`qRr3Q=_q}NL)<>!?C`h7w|JIqQ@AH z7rB=Q`m8MwePq}ImV-`{6+`$?IM-~{9=9AjWm~v_3|Ve3ixx1fv>$U9Ie=On@uKmykKr4W`o&ub3Dms%BpLv)FBQ|M$zkPL>7f z=-4q`3fz7H&b>=!fIKskH%j|kz89w3BD$4uHS`ZP97Uqk#+xD<2R;hc@Lq2PUcN3R z=WAdCOKT@DjpRfxeXyt6PRDK~sUjKBjr#*X%@c1zT9!(0LLke7@@V+AX@?aLxdDaN zhfIA-2WK|ul;HO{z}6v;tUYwf$>C<9hnSRPbfqOaq}ZEVaxwlBUUH~$vzRnfFNx{X z{W75qxidc~j;x;e!#P%7qT@Phf0Re{ejiw!SKQuFQqtaDQqs|0T-?zi9sb22v=Bl3 zDHWg@TnKzQ|1w@7p}Q_0h{9T9Jh|R%p@&?~Y{Xjy_CQ zc;mQ26VFnPG=&_cS?_xxDilJ_=HdPlVse14geNDZXiP3##CVdn8$WK80*bOi8lgo~ zEkm!z7141*F)sn>@m!sh&OlOIE)e2fF`NGyxiwhK9q%U|ssjv>c5`rP*_>$7B1*<> zi}5e1QFF^QNE(fpjvH$C~O)nm_5d~hEm#nb`7Y@CUkZ4$VhWXfCr*d9hcH{wYY z>ncectRawEvddp7cdQnl7_yeyzZOOZfSHsF1wc=`=fU4xy&mo8tO_tR1O`owfXXK$ zoC}|8D!k7pNGC*5(<+%XV~|dJOiXc+Rv4WlmQc^|wx7jggD`7jbe67~uRd2y$eHZX zg;nQw+W%+~ja6?C%4`!d9VwFpa}R%_>osz?^>pk~^0BpckfUc}#I<{B;G5^iFnE^B z#b8+1SvABk=idLea{#H50tcLH#2B$`1|$LTw4yG@!hQTOAu;}s#fRnH2?j)3Nz|;m zmC$~1)-MrA4BUb$bx^}E9Ev%d_=*jATfV=kvsxT4o`*y}_ZFzgWyGmX`xwuBjec^A z5aLf7Q5BuJLS9^?NMuX7!B6(I{i4vmwode}POB(dFgd+U+EtCT1Qwke2Kr#QsRC2{ zV3W)_aYaoT>(lHInN909_EmO02p*x9Xm54ZI?TV)ni@g+q+DU;zujueJ5irj~QTfO{^h1Wt|)JKF}JoX;>y zYn)HH9lZT^AY|=alv_Sj2(zMP1SlD2L=AB@`>zdyT9=|3?G4sS`Hwxkf4vBzvr}79 z8f1zw4_mNel_VIf8%Q|M_JH7Oq%V=>RrmsaW^+OHE9OCVNSY&~Oou#X_zv#pE6FF{Nb{z)1THL=4h4F`JLafL*1$Vq8kE1 zrW=%ZY+|Dj?xO@5!jGvDBIU!)&b2LjD_wj`eZF{PL+;df?10{cuASgm<+lOE#U}+I zaW1gQoTJQ(_rlIu1gri)zlznc`SPRX8Olgxt}8NPV?Omx*SRuVcisBd)dY0BdV-C- zm(nB%0CGFlAaDM229Z@O&PP2Ui;dZKDM@(!kW*suGXqFK9)puHDhc-xTn@JzQ$*hO z^W^>#VISUsd;-`SiD;Xeo+$Ma%r^ynQ_xPKWyNG(-NLfh7aea^*1{|SuQ;yQ zQeb`%@~Ay{xp-*vGS;X@pRV9G!37>Jz2OX=U(Ub3A4kt;4W|xMg?i4LC8<~jl@LR6 zn1)GaGcMJ*gh#HPdUQmbD4s61?%yAN_Ye{@j=aAtOxA>-7Pb`IQTDTiRwyU)$T}qC zfpS>ILUrylufe~%92qoUI~6$tUVaQ5KC<%_i^!SLLd_WF26_C{$8^Xa@c1&7`Z65& zUc}C+&zpzKA#}@9JJv_V02nT76=twJdvpsSS|+s^M$cD98^jImJBHLWohtmCD(E%a z(G9N4e1i5uSqm+tWFe(1n2l|+l+@I;0-hWOym&eUAet{RbRy=D9PP;6j=tCJ+Q~X? zJ;A-LC>#lFZTRKtX5C`g?IO*K<&M;rGo?FQ{qrDgRPjI$ZMhMcaFn4TC@`If&kaVP zx5OE1M%~6bDJZi}w)X6huscZ~5}fS-)L$MyrDWU9@=yn;vlgRZvhPYB+d0!*fqlah z%fno5Q?>=NF?J_fQQ~ddi6D%AI!;F7%}0|9AdZ^&YeGYHSVk9MvSLWm3Rgi*tSvv& zt=3%t!?|E-IPHc`ldxc*R;FSw+Qiw&!P$u84#t32l4$kXVuL{>Z|w#(A*g7^k-4gi zJIA`bfdx?RcgWZjl z(4zM(baY$G^xy2e7;qEu_v*hzEgFtDr)$uzNaOv*Cbze)XNuz;)cd1jnQ*UC5|5vZ zVTg>Nd?|d4E`1>~H0b&KC;~XNW9tn!ab_!8Dsq|P7C0aRkp))GT|7h%O!Vi;SAb?8 z3OYk=f@?)+ZDA&D^I5*Uykz#0MY-2$`+YyCX7x+86|uMzGd=rvD78@!8jtXfs-(?7 z9|MA}J=b%mBk&#d$a2NPNTzAUUO5??jq1egS`ze{&f_WmpS!0(9HceewntL?}7ENer~j%>KGsF)!v=wAIlz?hUS*|tjKMB z8^6-8ZrO0b_2T^b<#@}FdDn$_cdw1oNPny{ExU|@Qfu2cUbe)YW-l7SATYtRwvgOR zbSr+XGU3{h*&LCq6j-p>3+pV3Sz55BmOYfs=-*)DEDnDP<2cIDdmttX*n6DAS;vETndp}a-P`Q=TWl2!cBs0yhTego4U8#fyb4rc%8?OcIPt{otM z2X6+%U?SROCNOPQE_vOo{~j3sp4R%8==frYLyH4*TG^_i{ur@vP11QvQQyi>NIkym zYRd0(6}5HEr33XB+frHfu;Cl0J3)R^R0y(V&IMwTEpgHKo8@m3NsTW$di*$e4%P}I zeV3LpWeuPf*7Drq!oo@+W|Lo&vP#E&YE*hRWJAv5eXFCQm{|h1ma3+DYJykJuGfLr zvu*RSvzzV>TEjB={>5e~{S3qy#d8lKvU{}fK(TR%L^S*$9Hz+H zc;QUM10e+ukjrQ=HK?E5b!Tw43W84}dl+sXka5n{Wjo+XIa7k&#G%w594C=vEIdPVpO-fj1dhVGmYwehUs|M}c~ ztOYKDOXH1)ZV4LXvuBSUZy>Mb@X1wa9xYZ4Wxh#0YFReFh#kL(lBVM$t?>FpHu@tl zM_;i*$e<+>$7c}5<>@yAsy1kKxcsthQ45U}Gfj2ObC!#yCeW|)5Bsm!7w~;$a)jdx zLE_2yj19$)=3VzOYBwT?Mm^ltrsJ!b9(?x9QxLVPOVueM?vq$NSrOH-mv>w_z18yL z*9yvtvRb(DO0FX`HMb*iG7butj-v<1qfB22y*&Z-DWY>w7+~!GPsXRyU5W0E*5er5VTU3YPJDX&X&s> zh;R~3^id_zeg=JuqYR=%t&Q~)hD68isfLUtlECCqmjEsDS?fEU$uC%^oZP>xo-BK@ z_+>gAB7JnG3Qaw2J*^+LOm#fzPkI55{)v4(Y%Xg|;DAXDq|NRgh8lWNW&E)Fekr<( zS-JCJr6+?*Mk-E~Vp51DqnSa$pA@a}F;hAz)E?LsL}22!=xAdN+OyyKGw3cQHI7A1 zy+6>h_o)nSr|rh-Uww7lTHS59x*6m^9=SHyqQ;^UTE4}=c*dQ#>TU>6kwSVWrF5=_ zEPLHNFi-pLa026EI^xq(0+T^ghW#L~bf3(SaYALd-`On^P^rbJY8ftxhhC~xZ|2rJ z_^jcSx3s586~5c=T*IzGFFU+OmQ)a~`D3=l()=O&M1#b2r;}IPw%p z*rHRQd>HG5hexJMwO=w{##tzli;V_?s~PiuP{~XxLtfL68Lp+ll@V#DbFiOnJyKaGX{DjvW#hho<=rwkNd$c_bw(qL0R+>I6I+$3Vp_yf zUVJI$&@L{I%)8QC%o8pOShHctRTgNfmBwi^p72KCOGf2WD=Vo@DPzzvKM+0$Jb-hh zril$GbBwH0nLVmn<6%h;sEv)ZYM7Y=!5czk3l>$+N+_)b8-B|cajVkJC$e6Um*`pZ zG-S7Sr%m40ePYMq-oU*JmWI?Y z3>o0@_^fX~uk5iXdIEd5vh2MNpZ4&0f|O^iIA1tHX!)oPe4zpfMDCf<8G|oCLUYN# zyX~QM zKW*XkPC)?Xx_RE136FWfU2#jdSy}q|zy#*;=3P{88SsOY4ODd(_ZjYhP!SQJcVEub zQf9c<(H;{0T{?G!Zvxv7%&gazR)R)%e*|YjN;o>mGGC>+8n4`fJ+m55`x;~v6#|UyKaBE;mx2M36sty0C)$Dz za{>2^X-q+Pr)p~`7Os|>Qz*eoMh+GRt3%dZ!D2DX@)y@8#Qr_N>s;qa_;J582m!cj zGkjqmKW3AB95fiGnfk6l-NgQ9vPZQf>kFk)e9OvB5h z8)->hM8_JbG@q0~Q({kPG4=lxM|6~g6pq-7-?-XD1p|u8f`WT~bf|&td&L0=6!!Vb z?hQ5#;`H_(N%;c|rZF)g22p1lJ}Wz-K58JlCV4);RuZIC$?Mfxn_hwwMMa}@jq`6M zu>XX;VlXwkU0)~SW1D7RR}{kRRbD#^jXAeEEmG>>(?0?^18(kx@ z>nT{g)iH!vQ1NaT|0fM?(PQBp~e1PCPByM^~V#7n78!>w=Cl zPQDfPbLOoL>5DX-G}%nZe0{+0bM~nx=K3i;0965Y{0rYu!wV*@yW(&u>+VgM4kbV5 z!sN3Jvj0kJ3R6{1znMR9sMM`y_oJJ>vVXx;5qowK_TF1h2?V~oenp{)r779MvvG)F z*j(oB6RhXU^F?gqAtD!VP$%FsY3A5Piw6KS@GU~ zSR=!~b3*)t?dov-ibm)ycWgc*^VYfRT!RJDr=>GiaW=qv=a$+Z+ob6RZ;3B?`3NG> z4+nVg?6Qqvmtvx45*{1LxUUVWWhziieG@%d~}OmeX+WgD@*zu984y$f7nD0L%w z8M&nsdJ(q`UAlRn`@YX&C>1cs`ss3q73IDF;L_Z<3ctjARF|tMu%!`m`e&2#kDq{w z!oYm4xtrW5sgvd`k7n~DZ?b+G5W1#CSX1bnhV}9SZIH{w770F|^V{*~p0O+X_#3XV zVEG4Ft)69mw*O(J+>a?aFg_LM=QPwy4`KZGg36AjWY!io00&&piBoHZd;SozrVbY4 z`kX3>_2t@X2Yy$$q$hpW_mBHl2PmNUwbM|G?#>>b`0CzxU2b>}%gm_mIyf~_jh76= zgeK86w(biv+4-f!UYoMS7|%m?wnX{}+G08f5y^Le{`^puD$jo4%o(lxX2{X4U1H{M}Ss46`ZQzZw5BgEh{oe zznt8~YHk)CFR0p=i{w|*<4IYq7+mAg;Lisg>6WbSloUU(_(+_PA z)?_~r*b8eT8Ef1Xg%l#F6w85@m+pB1(AfSL9MqTaZRuz?ox9`y>^w^KF8cHm>9&Oj zVzkWk%#SIM5wfHrOExyQcMm!Z1-ecW*RX#-mqhD#`C5CV85MIl(A6@yIm?e$NvAaq z6x&eO4Pw$IhV~%&!=pNLYyLLo(&TeV#)6-CrKX#oq%qZ6-6JB0gC2mm-oEj{$eT=^ zebgO;00Y&GkqVkra~2vQb@i`Yi=2zLR+!^Zv`Fuu{43jTOvt{Xw*>!&hCp+=Z!P#3yeqnP{3Fgv3XbZ`;dejUvV9d_- zWNC-9lWl+*3L{fD-vNl05p(gVP~Q|SAFABxH8D%DV= zKMf;zVR+XO#!&AJ+aSjUW|=fA5=D} zDz-4YD+{Fkhp~Q{<~+m(Bk!SYq(RLE@e$Xa^*!^iYrfwmPv-?Bhu%x?xhQz5X|2+?S~IklQjx4E8hrv^#68B*;*Gx15S_ zx!5)i7GA{kCmv9>*27jOgd)uzYD$D?qtZSeR7R)pwQ=0wjmz3)y6L;B|gk=b?G?f&D!?p+okEkl$|B&XYLXXY0>7Go^jF z$5|Vsbnd?kI2}NL+X}`cWeJrL@pYB6U}`{MZb7$mz99;|_1X4fT+>nQmY61SR9vTOfju=2 z{mYq=t8m@StEC{=kQxYCzKzZYDa;~?9039fUN+iL1h=^7onOwwKjV4>1{#k0)W(~k zHRphV5W_td2K_c(e2UQfKQZI>>FDX`MO{+KK24&H>pGRxI!p_OCwdsJ@0THTzewAm z{RCrBpX}C$c#2N!#xnPDqe}|=IIGIF$Y=otz@kP!-DSUjb z)@!|vk_l3#@GpY6FzsOXq7yrvy83~YL|j1BZ(U;!baP4H#YLJH=G|Q}=Z|0)a#;82 z-3yASj_2SxUW#MKFLZ6|y|L5t+*Ewt2ZCf>#gl?jb5KP{ct}X7AS091(33cv?u7wk)IiiaOPXZ6w?^&s+XF}e|LIZ-ZPP%PYKt)R6PIbgwO#V z|9yS6k1x2u!@8PVAiR_#ql{$SUxv5)D^?vi@N=7{JjooIByMORy%9(obpCn~8jno5 zWcE8_lYVTIS_@`2@byY+{|xfAFdM&Z z^&FWs5Jy1L-aD)gvdO!`plJ=?E@=f=+bwmpfT408_;*p**M{++xMKROZ}ptE95Wi$ z{zm-%7m%f5?@(=I2-hC4o4a-#$iHqzom8+Lau6cMzHteUIs^Z0UcUkFh{PU2syNHY~Sd@L2?80&f*aT z!mFYh=lTAJa-J@|Y>mMmDDJx#)2SPG|?4xO-qu9o&7#wnw5oL@plWeY!3 zrToR4`c2)>$*s7zq79r7aYKQeK&Wi2A8~i9>pDk#pk>j1H)ytJMP8oNs~8ubG8vN) zUP)!d&pb`T;6io1RIL};n}GNgU&GcZO~s5-t5uJIE{6FH|5LQI5r@7dKlto6%)GIj zyu8Bll9GPIU1^qF4~N`v$KUo-HnF8r0vxY)3nA9x-Qd*A%rd=Gd;e^HZ z@eyez=vdVL>Ely1cX&wjg4i@Uh&dF3C2r}sf~A8|KK$!AoAjuvq}@CLySQ{#rqOFh zI;ls^4jgA`w?(G=@kI-)`8)eZ{~f*W^VazmhHwKC?Cf&E&J!F+Ok5dfRc{gI{;O~q zoi#ZPWG8}yi+&|=^o z>y`9VLrsz#c*w_l_^>zxP>&LfAIQ%6znmz2B^{d{ zR(OqcowYd8lmjx&N;4Fc<~L5ATiGBrKe>~!2h}1U<>kM?*s*=oTD!l|51f$a40Ae0 zk>4n{>UL4zA=?6Lk%XAqSgNLN=6Wu5=AG1HCO=GKsJ~HVNVm8?P*(Q#z=a$9L59mu zgkX?UoGt+ZS4IIkFBAgCCufo4$XAIN$yY~NhcCMOc43mnBd=lf(pQ!!H>7MECK3d` z>*Lx@)g~WMV9wMI6aO*ic>dd*;{vkt{7-96LET4Q?I&(mO0u(LiI2w3!s0}@88QPp zlsQF6a4E)7ky_~P`Dn1_Yb}%O!GbKEpKoCy6BBDG(Og7B$V`TKXSbb+tV$-RQ-_`afNnHNJVVclptb-`BO z+emqVL7^gT0a&e*kX)IPuFhAf!o)*Q&`?HQo2_04Em&TuL5IKK94Oz|sZ7mKHhscW zmo&Q&0e;dtI~!$FAjz&>>Iw?sD#3KFte3S{t;KiF5c`{axk#s8F+#4dU&NVupT78) zD}5r^IjTn4vIN*+6g6MLPFHwUE44V?zbMP#rjy`W4AkqBofE<;QVUkhcc8p2&}h|J z%%QS31C>b5tIEpas1TH(mt~aY$HZFET;y%kheEH)dCUo|DiIYRt8l5?Lrbdpj{XqL zD?h%&8)L7)!w_I|V7Vgh3KSl12`Z&E$dU+lvZs{V+C^n)0Q(@vhlLE7MliO_q2lF= ze!W`_04h2{m^OGUQyL+;NwzkVA$qB?As!;f5D|jxI*8C}P+D_^haD-Jr8seewJP9r zsMPWU7O7d&^I2)V5JTH#_T?y?p^kq8T*fTs8vV{T9rwj?=Fpw)BVc+w*4GHyeRmz| zA5Qxqy@jr;QsA;lLmS02A>}%>2=x7cUrCc=K>fa~Uks~=@}ZS=8sK(2r+=I)Zigz5 z6|e#m)Eq!*OfJ{!-+LID5IColRVcszj92XVv+qPcNhQXf7>?H-)Xt$z-NV+ir^-nw zasa^735!r+qsB{-yPi&P+c9IXR-9d;uJcMzoo9?Fa$<}>!S3O~5fUUh?RpkJ=JQMy z0x~QQOl&*xGZh8sQ}N#AOq{ri)3`o1_e}zkw|p-+58o;bmPlE1g^m+MJcis}6z;Sy zhIb%2U|f>A3;fAhefD2bE-pgpFZjFrwBAT1eDB=f@@cMs9t7qFr+06yWU!*4Qx_jb zs7L-IdF(&4hor}Ab$jT+Jg%YppS%FC4Om)HgY8+@=_!vljwK(4F@o0nknau*C|dIjm;S%|Rol8M%69U>TH5Vw-h#X&pW)E+MQO+&v%8RlX;AdEwu} z90JHr+{*wg-JS`tiDI%-gX=#=OjyfIJ??2J7+ZU!UG)KrEP{dAmqq_;h_B{RqkOu zTyuUyOVW#ROyzVibgK|J$g+;M8W?Yv2Pb4NsTsy4(ax0~nJED58I*v%6YcaX8Dpkq z$cM=5J_3}jJS-877(wFK*U=2;P2nw;YnC;(?*PU1VCqCMl4?4Eyu1;ERTMSWf+G>(&eO^InTwLrH*AkEq{4>ar2{*o#Sc=18hN}-=2ask zeyVBWWeW5l3l%CFmRzY61gM*NiByiP9>!E=`t+pBsf@fyxoLU)nK2@Ly7v42@8#SV zrvGG{^Z)&+fHHA1{`7K?fTICQc=BCDVp2&u)lKUuAsnz2P!%h4NJ<(747Hk@T>{p~ zEK^Hm*Nay($=SUd+{Wh3e(O@wN(U%WNo(RMAc7-d#KLf$aWGH6l=1N}1ELBivEn8& zl9Il&54!HJ+r3kxC9qgYL+CQc=BQu#B}6YuW!=J-?3F!uou<)KJJFQXtL zCS24^#yY`9;#YBne`>?u+$USHK>IRz(eeh_dvJL9+pHgaFbV39ugMF4K0HY0=yYBN zKSE##oBU&I|h zwCwB*yX%LS1g)p0*sj$vnHErlG63`{qtz0=1s!UZWFK{QV!G8DeckzPhPs%Y+0 zw{ygjav@m&uY!eYLxtqTHuMO+k!QM97|e&z9zPm=LO^Bz7yo8(U$aG*RuK9@er4ME zS^B%Z*ZzSg*OC$;wYa^r5DXqC9$v2pj@IZ(0;Zz}BoY8g5aHaOG86BWG0MM|o^>a`(u%USxevY_O&r!@{XFK85CqMFm31;WOM8zh zS6;6T*vg`ZfKsikm;#CFViixCH z*GNbOG07MwC@g3z7{P7~z3`s&yUyFE5j+-TNcxiUMk50%keMP&!r~SmLW>YZ`!O#o zvc-d~2Vz)wG~vZqZ~E?B4#XG|Zr{*Ul?WE8o&wGoWi~No8eRQeui#9&hQ+2x?m4`I z=0aFmtZkU2VC8?mYL)yEgE%ya!N~^6S5nGPJgyAs2Xl<uR>K`Vtj%L^^Wy#f{O;|4aZdLM0;et7 zAPrE1VKcFt{1rVxg;3NR6U%V^&>``O#r>Dq3Bf$8JrjmF@i?jF#hncxrPz%5qB2|Sj~EcLlkYcmSkHyH%Ko!RC2Lc^`QPRv zAk9CH?-mxU9a=vu*QXDdSV5EvT8p$+sm&|{&gvQ3(cx&ajq!kcC>T-Qlh#o&b+KXR zOcOu4+Qw>?Jt~BzuzsS*v2M#Fi(-H)6Ry+3)yI#$Pf6$aZa-cvZpt2kQz$mSrsmi< zX~AOT%bWt?=&6+A=Xxt$Y{w9rQ8b6txwYK@GfJM7C?2GsJ0;(j9+`1bcNi4z1dHwy zN}9HsU;cV@DJ66tx=FGWiT1&9Gm^NOQgw<{a8;SBDW^HWXaq_?r50q65>D|^W zrY=5QXn(ky5CKxntg(^vt*oxjpX{`@B^lH+2eiI5^G4g*2Kwyub%b-QK4cQ34P zxso+yelpTTXN5;KKL$K(&KTelU%zc(-*zxcjRBXol2B}&%~fBBhybb>)fr(GK%Ta0 z*39aX>1!6Z?@wlMtNn4LHp0i?!R=>TVq!n_nIYu~JHHU=lziQ@k_;r@w-%kKqx9&N z37|F!s?(rCMO&=y`w5jSTEMa9-r$~rFj=4(N>ZbOQRPNbY->?a83C?h79^H@%D*#2 zo-nc3o`c;9aNA)I>iCKR!Tq`#&1muU&h3#bJx4>Wr*uJ>kPxz2(rW~ji|$pQkw)~&T1l=+<0;X3{rR0oj?i*| z&z(JV#%`PbAdi9wYNE;OXrw@&aUb?1bB0ew zhiKez1X|l||21=4Y`x zjZp?c7Lgqu_GM?a50e$Z@~KqO4R4OSLHN&=OaK$=7ZTL%qxGRT8edo4(B;V$WRFsw zv5|4yTH0M|nX$+26_VA}h{+OFgNZ_f_e>l1Kc-lpG4Iy4ZId7~Cxi1puFlOHoU6ds zSLf@#+~OhSa_ylRAV!qr8It5lP&ziHgeF!_1S76>>V9BwdWHq&{5zmQw7JhS@0s*@ zHVAoKtKOK|#mdmwQb3PuT&A~2cUC}udaQgUy$xGNsX*^CT;)?KFCNvfdoq6*nZr3F zu(Gqzf=wM?<&)4rl>4e`)7s4}AtSYjW%i{hW&wf8yDcQ+@ZdphM2qWlYjAQFiy<#~ zWLs7n2l{!!%I!H#UaYhrwmGHE#LTjFK*7E2_q17;Qka`R^=CCC#zR{Yg_~C*JoYR} zws``wky;lq85S`a#c>%MoTqCCa>_EW?S9vQU(0+_hiL`4;3#Rmir_415DY)wC%EvN z+h^#1E#wT~5mAKk5-9w?AdzBoMG6;-KfRut;GA{TQ@Q>nh)xz1e?iwzl#hjyb>UbaouYn!K|^y74vc0m<$95uVNU5iQy~sHgN8`K+JoC>uY4W%B2W zr%?sVqT@Jfh78*(H>HhJ_@Mh&Ys3>1f5nPliaa1ajUcn=lx$HciX?M|<|2<3-=PR? zJ~}`ts(fe>+3I&wv{Qjj#i2+$PE+Dr^jr2@pq6M&>BesaE4ZdqEpgj?HpSdU8fGbh zj0BRQMcp$1Saz?x!+=eOX5gyyy}-RaEf38aO`6uxLTV$uuinW3tbx{B{b&Ch>y!7Q z{t6&3C1i=KstQC5UIOQY`G)&Eu1hcmK zboe1u>%nAa7fC}s@By+TQGzsffxv(*?D!aei55n2_qEld{Ig#Kk_J1m?KyA|OOKD) zu7PSj;?x{9!eQa=fpenhHMk+6hRimM;5}iwx8oQdm`uXxGqc{KDFa5GVV{F&kK)F2 z7Mu1hTz1g|#y1-38{a5SN$84NzCBKYmvE$(%>3a@E5#h6q+&0}C5z4$7^Q%XlXM11 zSS4YkxYRjOtMV#@Aj~Z|Mfd0q`B`Z%z&JiXT`+tQi)eOqpK}K$y5ua3^oNjyikPNW zqo%dD&ha1N6cZ#~QPJ`!-5s~WOB?Np7I!)uosRJ-YoKpi!gt-N2({MUjVxCP?Oh;( z8Rr9orLo8DD-)3(Zc&-E=pEA5%hn|xfeQV6NAO~Eb*qUV9eGw?&*X3YH^U0DIusZEa~e`fG!m` zrvwozGZ8B*(q4ML73HqpY>Db14%5vc^>>{}!^wC;YB$qf8x`fD{w#oONnvE5yRqxG zL#bha2pzLifnb{I)LO$l;?hExlDKhH+4@}~%*YkT6;y-yp-jU>_*-H3sE4)Jl;;~M zhDT_n3T=IP@}aX2*NZMI-EwKHIYR*>&AeKKx!GAuS^<88aT!KIvEH z)Zuw}_YdZ6c-cwBZ%$$GW98KWfJu5IIf#|f_j@0Apc#0d;a;8vKc?f`oC1I~ZBOSi z89Ng zDL_)pHbE91^rRAsGVFNw)~97*44;T(5t=U6``}Qd6(&GWJ>EBd`RCgu@SUBo2+u6R|l1CK-St<2#P>%_&6PI*^6n^C=|ui2n;+dvqvPJY&pt4=)&W3 zH~30~eh1L70xqCygOWj##!!83XA1u}LHkt<;A}RT%A}PhM^uJ2-J@6jJU04iWD8)z zcQS=Qrb>JkWi7qn*3()`c4vQLVa-j%ipvVUs6;MQ0C!sHS}$+^yKfd`}pLKqNa@bI|p`*su)hbV!|0Mo_= zO5o2YW35`k-d?|LMc0vGy3To?kGmFP9tapXCHeV$+{fVX^f>%jA4hrpKfRqZ&|?|+ z{k#yKk8k$A;d8zkPag2vVbAi3iiv^y`?8Y%<^T0hB$SmQVNkF9@~i6;AczR&Llu;9 zIs(L=AaZ!d22s~hR4{d`RVE%Y^D@-&dc&dV-?x!_1!^h6RY`v8fitH!Y4hccg!G1H z+p=!4o{FQDL?Vk?{$c>1->J`G04H_T0XslHV#S4@AJzc_XJjXExpP>0w)p6-2@`e) zD2<2ba;JJ>ys-3D?3!z7cONMLC^M6B(gAq6d0JZ&TBC9+vXRA&q7)Fews({_qzMe} zne0(vrWCr}2*x7vPN?!FAy_i@i7Mx_4tRe_(z$GR96H<)$KD$OPJj7(1YX{zI9WQS zyVWcv&5qeILpTBj9-HCG5P3Wi>s?*yvCq!ilQLNQet$(YyK8p1#hkD9a6BmI!~=F7 zbj$pTZu8%jb9pa>%l za9W5UelUZx5jBX~BV@>gHcAanKLo%QUlB8{>6Ht zOgYpH8TXBcnEcL$(;1AC+a1F4nDt8UrD@xH3q%oR5q`f?ev|>$8uOzewTNE9Kg79& z8tZMfwdENUK98HSP=PmAkO+xK*K}L1)ybhm=RCXAI9Nkj02`k7E~HLN6$F4*P#;>! zEqV>UYbb#P`ng6ny|=b#4_3dz|jIQMb_1r864VpYO>JB4}|q zVZ4b^pUf82$vJ<*lNzx0#g7jO7@0-Vu+ljtF*7uvZ+R>h3JN`RpCOXoN;zg^vvjFe zCa%RM>_p`74Um-X1M$ZQH3P!!Kbne{3zj<``Dd%X`Q;nB3Hc{o9|9z&z0aIaTQ2$* zEEKB%m;d%?LEi$QQ0gHO{5Zbkjl^(pYvvZ2D0__(9GZ?VUZeQl+waqGmhE(GbuYJU zhr=f!&W0anW~7+AEHyA8>>hDl|4=CkM4BBGq?fwuy#|JhAo=l{a&%_;yBL0s#u)9EW0=56Qi&FuM-ycrt51_ z30wz%xLt0evI={U;{mv|M!cQ!IUNokFRS!%c3KTTXhvFo)k!mR{N!ff`__GO!;JRg zHH(gOSr>HTEG+S`9d14A7Qf;Y^C}J$6Qe>mV zej>=hkiN!M~1Jn1PV77RUo;pg)dWR7e10d?uBs}P#T0u=dIFi#a=v|D5{)1WY#6bAE0AQgZUj3S)_Bs!w;+U~hlGrbkB_W` z^k+^*#o}iRU9rTDEz;srXb8Sb|A<~2B44CGf(v(g7qsA31g3kw1=bS_NyuNO&g$Pgs1q7HC6L3X8)mb1SYq4Re4kgKI z(O}Q?qtMPx;Fq;yzt%NLclQP3$*zsUxu0I!B?!o!Vv)9|T-&1(ytEHQUC^=UcUfAy zhlje2L7n3~Kg%$gSlF{{G&LWu#i#5dZHfYaQnEG3fh+ccK8T2bL3nJ?J;a>EoCM*c z5swxJ0aBqwYjVWp@G5Ac$%{&LFjq02*6nfM;>71AY9mf8br5Z0U{>+hF_q2`qEGC8 zfsPIIXDoZU39CN98Ez8UfS}rXCl#)J6F5Z5ePG;()VsKYhj*V&1?Zu7=L=gU5_MC? zKZGR#NSxz`4(eCHqx^DzN+lce0D{dwNp5CMfO*ThJiczn)8katANZur>}*eKnz_Q? zp)$_bRaowld7Ay*qAwneL)?`j_A!XOHB&SXp3G>h-_u&{*Ynz&&bnlC8@l#0Pm^qO zd^#4F|ISo{hiJuI8(!r16R-eya%@#pSh#h4#oy`Hz@lDILKl^A&PN;_TOKbjp;sO2 z0E(jyhgy&z^3&PG_d*ntJ^isSms1Ruc;(QP(ZybsWQ^tH5rUkl+Mq+lD{9O+_88vZ z{!`KAJyGj;vQGK4q+ye%xr8fQi-jJx`DzciIYe#bQwgThZeYhWwEn^}uEL))8_x=u z$z94=IcU1u>n*2mYt1e{Mc`@UFvJl{0Ot~CAJ6nVrnma+M%$XP?V&zwvk`azgdIxt zblQd_wd~CH>iSPV@~ixuV-Ntnsa>)>7Za&QUj!1&|&ghmI zoN4IOGL>(Mufu}W?fyU3@L47dw`Wvx}<;lIf&|Eb%0Lc_3Hms7B zifvn8-Wa~YS|?;86x0DCWhM$au+(~zF6*wZ7 zFuy@2swnIBP%&#^`+1M-UJuM1in^z?IbgF)3KO-po(S9q6r!7|vc?&@8jg$b$wW;3 zWm+HYOsmyT596CEb6m75XzrPo;x_QN>lOIZBdRj$@Ano-^bJ9$kd61>uICExe{C{s z|Hn8ao8&8_Y>qm-Exo=@t)_A|WstN^=Zu0{AHqoM_v`NoY66tmFriUMEHP?bV>4wm zl|C)SKTfnT<7BRMEldXKA*2lB$Z1L(=hjom$qZOF*LxD1YudE*O(xl=nWXGz(qrFg z`eUMag>cBg^m;N!z-IekaR`*;Z>f-LBCezcc@?kcsFgHC?lc484-K*2ZbDeWaKxD- zG~Ue91?~o)H!AwV)c^?z#AL@f*@Ey$1xjKRXy~X%&Q+}-9u{VqKPnZe!4o8C&?!86 zWpl?SmG_5|RzYe-$~Z5?^0`{^8AXh!6T)fMaH0L_Ls7Ei0OepUbyUUt89gj?I`%`% z#yeEm1hF3NBr<*7c>8#Jg>#m>-!n+cnF{&TEZ8hwo=SqLVtLp1*rDjeAr8N_0_GLC z?3V~6XKhf!i!I;_6&Z|Xof?rPKM(_gEx4_S`oo2cCTEV^#T6(yNH7sUhO|Id2&cC= z63z&3w)lXa00!J}EwmoPR3^~vAiOpfaO4RE;=@{HfZQ> zx|r$$QYv{ge8V}RdwkuoqmHB1O%GD((w>JsN$3tw; zvM-1oS{d5NzHnW|rzFf&z1BoR?C!b*tpV-`k&;tyZVWDaT!7(cbqW7Z}eUp-h=CvQTf>gtfjqr6 z8|O?7O=*>Gh_@XLK5oVG<%`V>mz&htOx(#z0yfC)%;Sgp-J>W^IjG6y_f&~_HA|{S zSJaghyV0YSuVN_d4WMVZ13n3KZJ{3wc5&iv|JJWiD8cVGcx2;$HC}t?&NwU%h13p_ zk6gKA%D3T8EAwg>1Uu$jf7juaPuIpsVI{A!U5(F{9BMqW9O$W#4cv>79(ykJC;md~~Cua2U8eFb)9V2hQz# z=3fZ(KMJE%@BKUMZ~f~d{dBW!8wQy>?nmX`!EQXd1Lg1!?HoI6nswct3#|FFhK>0f z(^hA;%{;N(O_?Yt<)gE7Sj@@lQ*y{N0a~~KZ>Z&a?q|$)2j{c`@wZ>}h6|?;3rl%| zypL0^GckSEStytiV^Y%pl%|p2!eZ|+^7F8xI*Zc@Pzd^dRLC~BrpXlVt&}(UVQl#0 zLFp#M#ndL(3L@@LGca0?gKKaApgYyt zz1`Qdx6clDmz>TE1F93sBb@Jx2Ri9_%>2hiHoUdJRgf1_EW-8*p?SxH3iQHa?k*qP z0d;YWzB7=W<3^tj3D`Lj%1sUbUe1|zFY0`^q$9oWTetzs;0T6g;93p#%GUQDm2~2c zjnMr9S;!G5O#})v0pF+B*^8Hf8t9pR#rP#>81tVdMv(mMzgwKZ=>V!jg(c{=v_g<^ z0{Eo635dx81@Zj5aDBgkpr}O!zyktcy4 zd>_BaopYCm+93Z9FCdL@w4@;<<>j4b<}K`<=bXn7D*P2|S2oclwVQ_Y;MSCgs6bIF zMJ^{bu@a3wo(7e_*=i|Cda!_qsA>?&se(&3F&|N!!a}*^4z#9j46!FaxmdD^>o(9_ zsYz=|c_Quj;DAOt-a7U7((hfEv~cJ7#)`d5r5KeQT3O^;bpTc4UJaB}#S8qgsrx_5 z5}lNil**yGancGcB8NwP0lG)8NRI%C8@XGlXT>Nd$Q*%{xEl)K2+cv?L0sbKaMHr? zra>+v-ueWO*cA~QqQZ!p!Fv(ddRY(VRF8xouQF0ccvvT7;tg`F3QEu**?e9!slBJE zN1F(Y2y3C^HGnS9ldp~MySTrisyoZ4@0t4z?M=Fe*3b9L`hy=qXZ0h#Vb|k>@Xs)-#LJc2A{%?1 zG6Sz2(i?10_TwuJstT5M)cYdIe4jbO5MYI^A6+9m@_26tadWE>d9>*M z{K0lpf+MbSRbbI6!zT>^s^ZZ8)7A1fPb`B%E<{ydnS^j<(2x%m}A$b+;$3+h8O&yPhZfz1^JHi zHvlyihx%mc7;6;VR>`iCZmbdGBz-WKor5E!GdrAZA%KkTApj_6a4I9B-fkIVO8Y-lwZrfGM zvb@_T9GGLytcsc?6?=xPf`ww^*?NH68Uk+ThxAI!@`%9`6fOfF!DgJ_9P8GgtTVq) z7em@X(eb5=$g-Fqf_U951hFJW>f;=ZopA6jGfSWdu5Ib;WVBowRl`i z;fkWuVoY-zLJnGiIN}5gR>wVrKcJhH%`GP=w=3*vRr@UDPP>w23d}7Z&#hxWzMUji zxi7bHak8JvQX6+RikP^cL!wsRc8X@LLmo^@K_f*?I8dw}#%(T+WY4p}L@?ar8ysd? zJuaO3;3%9)6buhxzzn|na4A*l?Fh-t{A1^IK^#Ip8uoJaR!QgsemZcO8NeRT&XaHH%Z5@V*Nd zt{l4fKH+da;+A{AAAEubDBxXj;oZ=xN4Yl%=GlbrExdie5g1*O4mp7!eAZE=FP(<+ zm*=~Sy!eQ8S)03p2UPBHke#f>22afQQA?(}xrnyVDi=jjmsdo)6VmIBxDS)6GXJ~m z05#8^M~ncGxWdzPa)ILsY~$*-)y)Ogd)2~8|2-7i!59MK;cK|ZZvfX+BhZ~+bc$dh z5i@Oz`f5I+R$w*9PpNbP7|87DVvY=Ve$)U-_i5%>^>)4u9&jD_JSXa*x9v$yz&|cq z+^ENHv7v(mnD2!IsE!YcS`KO3L;}6dgzsfal&ND|b}O{K--_Ttz-fLVmNN7?^c%~o z`%y?C<+qM3SwmwUoB$Lw-`$3D^BKKX2RCUAiH9{xI>f4wFCNjg1p?*z%#1!`j7R;q zT>*5H@7dQWX0X>=uA?hIPDQ@P(U?L$i~^g+48emG>J|-YC2>WP*~^-#oXd8S8K;Te zAv`PA%j_F`@(Q2VVxkQXlg9`6C3$m`Efq1kHia(f!P@;Hl;25;Fl6siQ7Bnt`Dkw@ z9{-XmaB1GTuEoAi`zPLm|0NKogw@XxT2A?;0ldJ;ephOlZz}XO!F!#ei$B{wktdgB zxu!n2mFH#pljS~qslH_W9i%NsFVIPw@&d=D{4LYEFum=P)8(N76kCKskKt4v|V{{)U;4Wa25au zBj^90ogkaBl|gaGg}C`{I{t{lgJ&i-^WxRVWKGv1C=GgzkLmrJPV8#tty!B3z7wc z`Zj%t1-2sU<5AcW+95z+1wAc8hQMB*(rsudSM{MNttKkjJ^pW`lA1NDSXFg?#;W|u zw@c;{B#~0x&ca z-c{}9Qx6@*{we@opM-erJzE)2E^Zv0XTQd$!DD}EUH7ZAf*xy|9puyz=n;u6{#D!^ z>Lk4mL0-=P12xTrl4R$&Mw7@+%IL}DIEUTL;-1H|mBshyu6SYN z(t6GP+W#{3L#iK@=idPK!I*sXT2UW(>w#wv2L1`~DnG3|FH6i-Tule(aRTKeIQZ`P zyY^r?_-Zp1g}Km>nJ1?C5k(P(eKY8;4B!Ti3mnR2f7|z+6BIu%a=xVFl4H}3CA^2! ztGE-u?gFws8@+6s+UId1n&Fv(W*3%8@&JDK51zfJ8cD?v@(cCoea&<{09=f3Gw+wWVpe7kE`ieSx{|%In z@Xy5kR`{8?7#HZ6ySWw#1#kl((4)SfB$`?0F+o4+2(siylt-zJ=ltyeTTp zgguqR$dGJoOjK5~L6dFMam|ElIOD4EPz1BtU>pX;(9jg(ai;83r%jQl3=@}=tXWYI z9a}iKwOj63y=rtu#<|1F79fM6@6=@&d+N1b?Y&*ocqD3?NBmFgMzZ~1d$z-Hm}>N} z)jQ%52g}HKygkzUFEE>*N@)0Xapsjev4;bE! zK_pp(&rY>=z4@5WO~8fbqA}Om;XIVeDLk4$!OpK1b9cjC@pIV*e+p%wp(mU^agM@^ zz%mti?+51&H?)3ozVq~^rmm8wQfM2nv1*Xq2(SK5SovGCn+Kb0T!~oz4eLcuvf9CT-Yk`k5 z>G1g%q26CYW$TnzpuKs9*8fe#1OXE!86Z?{5_Jpj+HOV#DL_2XQ&8!X*ADd%(deF= zI)YjqXl#9V$I?)%VYf!1#TNnGi067=dk})#?Os@o`y;ZO`CWmS6w?58iXo&|3jO*DdeIzUf z^yumI`1W4$&lV2!DaTWsnk=ytmZqs~BQc^a_z8l024s^vyOLt#Z@eL08MBBDw7QoG z!AAU3t|4WORwN?bq_L{llBXopBMPKLLQxTiZfDa2!v`FHcoGyb7>%PaN#}t}yO&n) z3va+6Smnc4Zm`l(2GV{s#OeRbq`X`ftXcNXuWZ;i%Ey1n^F~Jw?PR2IZT4d`dAcYs zu*5IZ`&{Jitr296P29)-P=Ih>C)n6LzqO(N#htfcV`*(|XZbnrKqV>R9}TOGd@Yy} zIqALB+Y4|HXsCIfs1U!+R;#3elJw#BZbqPfJDB&e4E+Ni_}Xvh(AR9#V(u;Z+oKn* zq`R7Gzt>_w>7}>X$JY^?!=C~3uv2SLf6KKCa{eWRF`hm;Y-6V7A$ifm$8Cp}E=`m_ z)ww@btN__ZL0|QVUqcg}!~QftY)5_%w-=K)_zob?VWr+;mMUMXrI(=tW$W`ovjV=}s-uj+{62NgpgSj2K7h@S4=7^PAFD z8X0=fwEv#PIYXJIAKO94jLQC<`-eQhOmaI2qKkfNlf|;)Ndvy%e~gVR4-L_xhOXmyI|V>v9_JP?K}DCZf55&y~26F zo$Wr&du_LUc>lRE(?vtfl59{9(D*10UKvd1l}0I!lnJ7fERj&CS${qEa7<;9qfdrV z@0_L~OgGn@blwFW$QyzbN`Ec*@V3nAH!lD=+3&&nP967)B{b9ZW?E@K2Gd@kcaZ)- zX1{M1L%z@ijWTGk(){VB%|`p{v>K^isGRsjtJ(fgLQCb^twuFAw?MrNsL>WTKU(ncQkU{@%XC04vK zyySKMz5CV!6?emyNu+Ho|QA4iUq=teG#T$}{+dP#=Wr93h zVGl!uw2urO5hg-CoIRX&P|XMjNq7+1aS(n_mN-{}&#5gYS@} zz+6NA?~`YCj|Y@#R6U?o{&SxRO(4Fs_ zUT-r@xR(BtQ98^n2r=if?jRVCeiEjFI(;`Mn|y%H!>D6|zxO%y%Pjz2T!wzcfg+Zv zs*C1G5P|;Bu%hvo#c$`Rc_;itgfbILQL_kEDHwpVcRy2f%qH-x&cd3*!zRIR%6*m7 zN+)WYCe0MjUf5t!GW#`2+D_Cb2kPpb-Sq5%(Fnufd~>eFiOeJU+5=vnEE2ISFPmdP ztJ~ekI3?=mP3v#fq%7z{8Lbyj1as|_HuMJ&c~|iqgn1dXN7vVg0anUEeKA8!S-pK* zvu*%86UKB93}KeFi4kF1T3cCpIbuAUxNX<32bO5p?JhWJN>rl>`NnK!7hG^f3;Z{< zwIDSct`sj}2QV2}{kKxm2~%M5#P#gWGTfs)@G^Owc6le|IXjP#WzC~82&;HXQ5(l< zF5Ln(5GCUaXvp5z00KF`6?a3z9eSA5B_2TUJqQEJu(!K|6!2X)zxoV>X!cW1rd}Y%dBNa0-=G(Mqu|N0JaAV z9DAOwA9{5j*BPA@{?d8m#cbc@-dx^<7%c_YuK2|R zfa`_d<^QiROSmTtZ9dg zhxfLnh5vK^MR;K3M_=HUF>e!9Q``<9nLPeIh4Cv&w@N>WK-w|Ob#vH@jNBXak1(`W zo2kz=S|2L|qsu9db#oOYYQn~)X3iVZ7ZwBG-82U2a){B6sh)Wo1xC654Ps$2x2mO7 z4YGycSN(WjWmEutnB9Xu_SrSHceE&+547DI6R6)Wj63+ja#*40kX_8-!lu7~wzc=3 zE#@3^3h?)J@m@mDt62r$={K-O@EUDr-(%k|@twVh=WYtcwLA~uc>delb$^>$`r$H9 z<+s=vDQ(S!qCv^MqR?QVqa?=iE7JlvWtsV7q8qxTe(h9(c z!$$2WU33MhN^ZEX?%3(xjOtzh-4|cHUAzetb9BZy#uz&XI*~VWRjO!6ZX@*}>!MOD>HyJ|b&7?YMi+Ew$GDW?<%*S90R4$}GVE~5a$ow6%_mRSAZrUMW; zQ0Klq)1%FZU|WY+Trr0-OxEFDhfc=j=ov9fUpzJ_s>(C^0nAA6s!u7vKBfhQJaru2 zjxU0*C}aTHznfAL1}1T5pe>KxfWl1DTQ4 zh9u)_xQDmE4=P%yvs|=e?IW7{L(V8&vT3D6?vz-Id%t)|sn2$l4{8IKozFamNFdx( zM4jIL4P8Xw!}Ru29@$g?Ukjv&rmd?TP`RpSM~W=!wtN2K4p47e{lwFg8;czUrqxk*b^nD|f+a0nB?tH2j95x@)LbmXP95FluhW*x2g9I zNc$yvza?6F@Ppgx3(-}K)6J)&S5fm?e3`kVu-MS9!MT+7mSgT}0~2~{dJ>TqE-~A} z$0S~UP2-2`GI68<-j^J+Nx*s>H8tAkF&EG%bh%Ri^_mpb`i2KLk@*;B##(`lKvW8pn zYafG*j55~9kE#QSMo8t=D22Pzbi*%Hutz!RQceo-nb~t^dxM4=KaJmzkw>y;1&*$s zV}s@w{HeO!Lp0g#)uO;27G!_@pTf=}DvzLRw?TuudvFNu?(PzJafjgU*0?()xVyW% z1$TnGI|O$w-@V`9|3`P$gPQcJTHRHr&a-#ZAHXjgkzAZMntZ5amB@#%NZBq;k6xOg zzb57vC$}GJ5(aTxp8Xx5QpVL^L_a~v%vQX8koy9<^nCN5oEqHnTe>YRKAA zz%0V10z*tapNsvs31vLQ2=L|yb&h}Uq{GgDVBO9@;Tku&o;DK8|9jN9xwiI1p#z>h zxVLmAzSf8d9+|MXY8D40R6)Q`v04-bqIUXH%H$dg(?ivNHP{Q3cZF|My{8s~p*?T6 z30KrsFns3j>O85CR*X$ggMO}ksIEs%5uwU>#E99 z@y6{OUB94hhQb>Xti9%5IcJc-5+qPm5&9d>=k?LO2=KzDBi(qL68WT#w~97t)QM&} zin&0>g#@TG{+0yXvOlKQv)Ku(&M&x`r+49HEHSQFPCMy7cH&b=OO=lG_s2YXXd+vPUgcXvBynaI@ z`?^{nBum@CgR771h!5ZC!N0(8D(-@5Ds;N+tKg!V*p55zF9E90UtPJl1;4&IP4;E{ zvb(}M!a1<}1U4;w@8*1aL*=J9+s5&jm!eGSNmmCcQ4#ugP5X)q>xMo9-5!-UG_lSX zIIx0aIn)6(=;~@(Rg30l19SYwd5UoR)6kPxMBypc%awPVL;s%qIn1fFafFeWC$J&l z_YB_Kc;1bo-ZVV$&?J(>hns`0=ZQ$J=d_bRKC^w*v-py8n67B>HQ3B4K{AJ>6}7 zM^W3<(mcO+&PC;e*)=@z!GRK>`kK_q5-dTn3!z#$lDKMA!EPK1C;q1_n1xHcl-p`L zPsH>(lHsx7yzO1DNc=*>ij?fSLoNcF=*p4gv0fAp`;Gq<_QM)SU`StEZE;Cy6N?C` z{NMwEhYelwKSwpG&*B>s%m1o(CwM4*vVsOT$KXfQP?bF}i=WZ7PxW9K#2s)`HCBWj zFe2e2aK43*#HB#yVf7UyJfi*}jTFK&fiFS9;VGdmG=TcvA12M0IQZA>2vqq`criNarj)x?BSQ*2P)8OyxE0OO|ODYU> zYvij|W_C+i{=hdXG?eL7Nv?>BhqgjPoRyD&EOe&3-eHcnLTiwQE>yrP3s*itqlv(x zD=-Pm<0~DR$FQW`7pBrs{z^dtC!||YcmocBC6(Wa{`lW+3;IC@`>HEtL zDu}b?K;uxu4^`ncgn8a(SYy1P#y?|tAzI0$pX@3m=aWPlBY_E!!;e?KS5pDlNFMaJ zcE0N-6R=Hs4R2B9Z&)`{NcP(-QS;U!$I0k`@lS7yUET~h7^IESsF8XhWhJfJ6Dcxh z-(;MtL>!9Dh>t9Z#$MfU_A@*$;Ah-rY*%sxsYQ-U)wAG|h7u6yuRtwjvTvrE)HW&VtSN)C@)WcdD9IM~_lUvcF-WO3?!nf?S;zJ^Xi^4k2)!gp1wn)fYpGE3mF zHQe81idV{G{ZJlP3QvWj4=!nQ@z^h$Q|a1C)_`GRFC$_)f?idiX6PC;kRBOueyl8W z`;B~xhd63}arzuEd|`!V>_OJR89lY@&|KWum_}@zPTHS7p&OATK|Pj9w6r>pNyCq5 z@-X2@mGvG0Qwi%_FaW#vWQ{7`E_He*-)&JtX{5J6?qeOcaitoVC)bxLov;jMxPAT8 z`E?DG3uti!f>{8`?HFQcIGZSe`<~n|O3G{n zhb4~9*$sQCvhi9NvO9byVhcO3Sqa{&$DvARW@*$sIZLQ%72oED+t}(bIP#sttXtud zBU>n$w`}CQIh#K=J(C~Bu#Uv14jTY^5^&&&6v;JPbv62auj6cT zsq)!l`wEs60V~u9HS780TM^e$cxzYTq)DVp!3qj2jv+FZRaq|BdfVzW=QMt;??|y1 zU53|6Mm|+MU&h$+JV~eEIHt zhSM(%w>k(JA=-q{7TwB!d$c5DreVFy75e&S$$4872MfFOCRt3{&_fd=la@NpmCqooWM3}?gX{0 z@&d1R618{|2?>Yn%dOc9k9~|`@Van25i#05E*rG5u#eg#?Cc;-qkOielalC8^l-iyv{p)Ofy_6#8bmKMfjGCSk z>(cIel1eQ`P#?wy;lJ!S#xw{3?D{f)&-nKoH|@8#oD;p1i2OQ=rE1luVtmvv!v7m7Azd`W@afJ(oy2N zOGP6c!p1rpoRmS+h47fPsF0j0kenB~Ehugx!tP{t#&)VW2by2tOZm)u}FftVV> zl=i$|AQ|k5DgqRzO+nuGeC=}uPV?|LO*;*CZ?|?Rnz%RvPPPX z&zyKUeIOvsyN{ph&dh$1?O6W%Sw{B1hS8`=lx?_y`2Y}KN|HW|_Cd9zU;@T?XXlE7 zAcG*qX^@qsdV6z#5@V*_Yx^c?<4a!yrYF|A$<}V)&o`ICCK)0Wr1T;poqMG$wNeMC zbbEOtn=`HHW})U+o4LL|U8A)9UpVkg+u*WQ4#>6F|9AO(Qnr+#S(Cp4&?JCdE4H!6y+JGm z76v?+Fxh>;EstnwZlH4z|LsPo9u$U{H8r!$e66@-ba)z0GzFz{eS*^s(uejXLXN0! zB2d{(gy2@m`>^J7|9-!**^Ep&+T({7F4j+XK3-YLA+K%GX)!0Vj(xLZ-(r6>b%(ai zFXOIH?#(2%HPFWkO$;a{4f{a0s(i5OxV=qkxDiS+P+CcG%fzrMR(-w&uyJhp6=Gi>v%Zmzee zo?`W?{=UB7xbxFua!Y1#mWMUr6@_S?Sioyi3Sa^x>yz&gR za^caG@FlVqpA`2|gL@EHpFx5D?AEUV@mXobV%{U{+1LWiaV9g)e8SiFEp*WhTu0B6 z#cBAEa)j~_<$k`2JGFxV{VctiJ^k%sO! zR|>*o1cpiaxFJ9~bju-u-#G}LtDPW-per0{NR$K>E!D3tj-?7rrCcR_E={FOH9a-^ zy}usc8JH9{=RN{Q?O=}<6xm(M_LNu5j~3_(+UN|r*B80eMPqkybNNts9|#!Cmaw)| zcjPf)+82iE&0rjWq!imZ0V>&~qgIK(&d3QVhozMVMa?!4K(G6ZY}lV~v<*)DS?gFX zy7{5{fM|Q6#X(u}Ivk&HJwA>$n8hy~fkU(k3uxUJopRUBdj7pO)2_RI1-1Q}>BT%J zG%;1L?S8Xem&uR)3((4lo2RPNRjiJS82<2dhUW0t^5c3$v@K;fC9G^#Xdo<~l9HVY zAfV)}nh#a&2|&!ASbudGw$^~Mx+&IA^EKKhOqVezo0F~glCKi{E`q4V{7jf>y6yN_ z54cP<>3$|J8xDfg=| z0&iC?koXdP)Plv1mgFrZW&S31i^S2BApNLo9lqE-a7h9|?%}^7{+r;wr}T_!!5k*d zk5cdH`1$U;>OV-n89e0opvMwa^9fmS_fHsEoqamJpQR!Lq*2~3X$KvF55Yqy1wez| zn+`j^4C-y{`{v?7Gj}s%ZBfGL?yfEh7CU|PL?Q#C_mZ7PK?9JWS6?|Zb!aE-RZf_*^ zl$&)JEp;Vr*{%>_B5Jm}^PZF%6~MGRFlIKv}@3K4RXvI@YNon@&%P2;^*{Ni*NHdrhRT4mTHG?dM)(=B8f8CIvD zI=m-S#nd>#C*M5mIJ%%E7bf%d2^wHb<{Jv+Mz#-Y2C)p6x!tVDnt%HAt_5hnrNoLU zE`zh(Z+A+@LH$y~!NYDBOt_88lTy~!)lzYrzgZJ|C3+%^aodt2*8|@1(cV0f5k(W0 zV{5>gVVj`2TUc7%;ovo4X_Usj|A`--+bL*P={z3)`9(}SV~+U~qZA8|J&$Y!om>MJ zcENQ$X5Til$siv`#HT$4#@@z4osfD@B_1t&`eait38R=?6f7SisK5DmsShO#`VSO_ z@T;u_3bmG&knY6s=^;>owI8pMmFIP~oHs^j|Eslq3yEVO1pYVOp26L|?oUzr&uftY z@$sY~C#hjFVnOf8-cD0p^}+E3=!L$bx0^)WA=~wnl~$X?=uW8EZZ-`QmxI}4eP_YP z?Wl=o7xirV9hde`v5qCIqt*{ceLC;3(PbDu%aOBX6fhPw17W~Y%MuTRdH>r!OLh;* zr7mJqn8ao7wmHU104cMJ6WVV%9ig=ZzKmdm79_4rw;d+*@S?i1r9GuSG<*(Q-8T2g zgXg#MZ2d!qet&uK^u*mC4rT7pLYT;VR5Vyy1>5d~oZsO21z~rZVc`J7D-MfXc}}}c zj7FIB!2nFp$pe55ea9)lo=C8`KQl~=9lcSc`pTK(17$`Cl;^LJL#7K$K@{LE!pV49 zIx~eg^F)>D22l7l=f2T7?`6Fgmgm>CK;fnG2|+JGrQ6o8<=A!Q1vLcSraiOLvNH*~ zKDOPE$0|p_ShqnQ1VHRU_uAOxFznXVLTTqyCy!ymwE`VF%Bre*di{6xrsY_K?G#u6 zda4;{JNLnD-(_4utKfk{2F$+1)q_?uB(GpB!U>;xIo>S@Wp6%r(L6f@qI*Ll^3TBd zyRek=s9SZ{;n7sXJXPqUVaH@KZjEF{mDCo1v}l<{G{Vc$gbG9>|1c zqgRbW{$?t{bfn6FDskQ?waZ26YpPmlD|TEaY{_VKu^Lku5$M@YUtQ*uHm6qja2a|9 z5y&&j>%b*!D`%r_!d^epv)?NxDpw$r*u~AI~ zPQ{V(^XX+6zMa@#V(yuI9=lmgrT8DG+JM9?JW>r2Uj)A_pOt93AGf|w?wcP$-V^mz zFgY?M!wma4a!lsd7HNSbh$)|x6wh=YmIi^t)(AvaTzFqSRUd>M_j)wPjf`DNvPesBX}WCLJlgJOXGZ4KpS7Www`JHYkT ze*BtdO)YWy_m6}hDL+mYWSji?%nA*+=6Ze!tLs6>gUgf0+Yjh`Y@;G0&Q$7!X2hJ2 zJ4_%*u`)(Tt!M78AV{X555?$i$6>&k6+IVvVpCvt*;Ws4%%;z? zXLfIuUR`@!C0(PVj-{ilk?osNE=@y8JG91qQkyjtiC#ORkkUujZuyom7OXw(QqtE6 zkLO=HgPCsI&jjO_gJ3qO_S=NMhk2Uagp>S>@#*x^Xn#2N*SOkU)ibW|b0_@sjo6$` zAM#1MR4J`QMn$~11tf|3SPsa_mfI-=Wx5Y~zHSg$-`<$yN$;KlG+TFF#7B3!Odq_P z|D3fncQm?E=M&sl*!aMa0X9r!@2?Hm?3}E6D2^%Zh1tIM)8D6Y7Hl8#_RDRm`P@rd zz4Kr8u-Z4MmmhFz`P(aDixvOINA~Sp zbkk_{XS-JnQbPzjE|kY8WnB~53q7t|D4oD2UEDD*iyYZN1lVm&w5$3cVLH5u2WVsg znaxKPwV}et64CJYq#yg78Tv4Wve?VOfL%J}c_o!@(v-tu0ho?P`C*@Q11eHm7?#hn z2dYgM^{jb(Wz%*C4$!J+U>pW8)8rOVk~-zr-yaGS6^Phssk>fu)I+lw+h=XlEII2TY7t*r*?tMbTLUB zXWS6*{-iz^j{fxii3WH`I`L2EsX=>Aa5^k3NzP&KTW-E1T-{>!EEGpu3yy}iUMZaF z4?W^oKg~l0kmg~2@zf7xldz&i-r6Cqo+5M`?lJQxTEIa4)CcLi;T|8@hXpdJnL8%H zu_PjSn4CjM9Alrf-h#}<(UJ4Mf!upLkmKLs5)nQ7d1RYLoz>rqaR0CsHP&N3BOuAqj{ZSC2 zB(ob?%y%4;=jE3-sbilJzV}8;mKQv%#x(st2-bGu62qfEomv5X6jS&VJ}J6rCjQvF z0&$!G&mUN(#M!=8e=`9{x4l8qUEJ@x)8JYCI1k(4wzN6PDD#0 zGiG&ro(J5WX>uGKbfDhPdBMGxk4 zEUG%92WK+a)$+=p*MgCpw#3&*d6Jiw;6q3Z+~D_)0-=Pen;`ckA^pV7t*SwYknA0> zz^q4hqD6LUZ89u%nBC_P7^s2ERpL{fIzkbcf7J-Ai8nQ^e+4-YgoNZ@%cskNcu{Mj z|0T4_?=A%9^IYjzTZqJT0QO9$>G*IYf@Jjv1*@9W6b5u1D&+^On$M=zM)?ie1dk26kKq6 z@*$9j?Y_AG`v7Whlvz$?jQ*ss zMCeIb#Bk;;k_+D=BBF5=B9$DL3F~W;v>AdVtAMoRs-Zsz5< fb4A@2^N5}iBP33 zvTdwe?N`|~E@mbvW1RvBYq>zIAwF)5WS8WoO$eDx@HfFWm6X=k>8a^Oj_*&6tRTVo zA07W(O0sISEcyWSVHhUa9jVg~UiOBm7;uwU!hyBHsDZ_g&8*{l`+^s7;rXib{x%p( z5_)b{Zb67q97ji(Z6^6p6e++XObVG%N=lY|y3dvGub2SiEWbdU0jf5>`{+X&;f_2U z%<%DuhXpRTQV$9!(1kH{)S2BQ3Jpahcv929q*H+)f*;l_`f2xm+zhQ3fJ8ZOn2~#i;~{N=eZe zXFm>lg5V2;X#j?Bo3@!i)0IO?3*~1P(_f4Ps}$i z9?5_U=D363+%>t(1)?^{bB5LG>P!H#Cmq~OZPVj5))zk{wWu$G{;c^)W{^@ABh&e{ zXr-nhDF{S5Vebv=?7uRYT?fU z#|A$b4(|ufXAeduWg5VH4fRiK10!N->naS|v!^&D0wy4-L|J3-C&k;5Yvp|w?!O{{oL){tlvnsLt{-o<0Fj1MRDO^_qak-=gj~^-` zNJgS6tOA1@cx=Iq#9&Uho7IEPFRMHI+dJ0Ea+`!_tjfQ~2`bTQW!v~Y9u~VJZ(v^G ze}*0I8;nJd+68q4VG7ELrX3p-G zH9}LL- zjQ+WvOxF|9IE!n0+-znhPCj;E$4fymW=S}y#Da9V`VW5|mJigUt!!`dwR!72^n_i> z-A^woNmONqu32b+(gs^KL6Kznu~z+&7Dp4P<6Y3WpE$LqUGVp-3!XzDP2iV+!kMb~U5&Yp)od z?`!@}(d^ar&bn`arWd87ExZqq_(v&LFYnGp?4TNjiRaW zX&uA*;+?%BNAcTMln#<8@|fXjFBt*b^ihREX@jo<<=_kLWx2QRMgthvuOxAjU!^<> zdoMm^O~jwpS%LBBQgqnva%)cnP8HoYd+b9E3#;9(QN%r>x?ERj0^F#-rd^|p7vFs_ z38iByW$;2lm2F)GY@W479SZn%l;)=$V;Wk>M% z@WP+udz|VkI=b3PDZ$>PctcVvNAHxRYjtyD`xQ(~g-dVHu(}+iJZuy;^*-*H{M&l6 z150H>$=q99g^b&Stap-TmC5nm`EX4d`L8P(;-{DzJTMh%X-KI`3!WdO0qdf%t~Z4P zD?QD?iE(8#oOumC>oIh??NsbP=}r`OxO6^@rJWj?jdIEF;9B?$x!h!2KTN=BR?Rgv zj|~G4@XF>C{=`ek-`Zjri6rg(Cb#}Z2eYJTZft~6e1RdYW-LR3$Biz2URXI`(?GL5 zhn_(0dP{k6!d-r|@AKxIVs-hH;QB-~>)BcktnOaDlvedhZHcR8xDF<1D~BZBu>MIau!t_ zxQ%@p+_}_#&SmRTOW#~$;P?<|H*|ak4k||B&Wl$;hl|)ZVx3UElYs+gF$GlA9N|C-4YWcFna1&z~k8=5$H2!>YB1{?ZQT^;eXzX?z0 z?3uFX_xi_+8_KpaA^`kt0iVeZJc4U0d>zH$y<#ZPFhlE!yAJMS+T zfyekfpYlVKo^`c;d1L##v(KXdLO80R5Q3fg_89jJ?NB&^H9ZdUriE;KF%nhXm^7R> z0Y#Z)xC~8vIpwNddwcy{V3gcQ*EcIiHYN?2Cr#x(C3ZbRWrz?qziImJcK ztyJ4$(vl&zDz^F{r?h7;HgjQb#g_yfwv{h6f;YVySwS&>(@|BKKXvC(Wl-~35Lq~a zm2gETxe4joA>oE!-CBx*N?YBmU6QvL-yx*^8_l!yZX&j(xZ1Wf7c-1vuB;rKF9xIk zgNHIb<1l9fK}Lti!!wf#a1q*&LNymLAz`gB6aEgsthg~0Btgy;l0yl^pe^S^dde3h zh@DOBp5ILvJNq~%r3<;T(TK2V2{$1MOq<#^hPsB9cS`tpDwYzA+JvJ1U8OFp8B)%B zNg`*aAgA@{et5py5NONVhjEDI_fSXgJ2lM=eeZxRYObJa8Dcg9KoMdMNgdvIt0QOB zb(yJL@1<-z)*)!3c=5M?V3S;a$}{#Al?f1f4RAnLffOwttLO zEvYPmsbBdkd4h3>3r1k~zMx{g1UTCfXjyV7xHZK-7#+F_ML*Y8*5bC`rXaBA>3JS~ z__OAse}{e(c{B%f`4Z(9A$Z3(f;mzE`pcDYV11nHDkpedC#GBX`i&X zekuoYF`NgZ`p=Y(d)0C<2@kIKQV@T3@ZXt^Cv2)Q!#5>0rzc#6= zr;s)~3)cSvt{mR68&o%!wJMPk?HT6gsj*aYHYjzvv2hzC9f2O?WCy&}>{^)wc4yr* zZlHb;|0w@*meNo2+P|&BvT{w{t&>`fjt^PsJV35n-A-UUa;*rauJUkQwj6Bdz}59X2ahvSSuBlPn9Xi6NH> zyAj9dUc#8&h~3PX!<3te*@T(dn1!85fX|TW|1(0DQPslJ^ivVeoqV1JO#v`5{m+&P z_E}!JL_?gxe~0q|)q`3mP_i_^v!r_3r7Eak7GUCIg?f>o9A_3=BqW89!T1TBbvd~V z*htvGNTy?R5jg`xvXCwCZ!#?ze7e3*haGlZ-WGzW@tK6NVftD}(w<4Kv$iXuP7TPU z!l+F>rd^uE`M}~ge@tzm*f}sy_}Rxf1^!z|!h-%d=UD~hX%zZt-dm!A%gFVAvs%x7 zF(MMhXFn`bh;n0yg7`Mqyqqqp-CXA+$!<}7z%OS(yp?%(S5?(3j$snVtR*4V4|5Vc zNYNclZ%olMXl(2@UVa?H^^r|K=C~SCYAB@AEf6d75dk(~Md2~1b^!0$Y~mT=b1F{j zr&F634s+_aJN1PQ53RY8b(d-sL;yPN;+)9KN=6~zuu?l0s9Uf&+8j+}6#4j7!6@9U zuhrZ;^4kS^@@D;+C8PS9zo9=^-~v#d!EVy=w|X>K2gCL<^}%G&dtw}Ih}6s1T#-ww*k&ZQHh!6WcaUY}>YNtKVMhA9Sz32j8p)HLP`4?R{NE zrYI^-$3)KtLw0m`{RYED$Vg~!WCg>^%OGxP<6`Q>AZ}ymVk&BCY;R)9AZu!8?qWg6 z!pzCV&ky75;$&)Q3*)g-gS{TNHFD3V&m@RH0BrWJ3$Yio(T1$SVA64g5or<;x6!Z5-~VS8(rPpy|z-+j1!mmqk}`R5Awq3^EETfiwyyq|I+r|35 zY+H44aR&IRnFpe$J8k>8v#{#|)LxObOLf;yL_gbV_^d4#Lu2j9w(V6#V%&AVA`wR% zTA#xKl$W$Q{Hy!CvlUxid1=cH zTT6gy?KA7l#~Te6XzmM0x2Bn*yLm$c)9=lFt`0K|4Giy#bj_2 znxm&qKj2+rBiZQO9&Q#Wz0=;@;%`pAbI z1RrEUj<^PE3tS<=qQl!6z9~UfZVYgpz{Fzx0$7apYWqn@@t<;_bP0o?VNQ=%q9+o( z_+6FX1c$*6=W~e$CP~1kM*l%gAFf_2CT`%+RMkx|)KeWTC0`hzdNlEJMV)o70~ngb zDuH~V{9f2$!ek@(k-(fi^<|DlC!co~UJ00!c;s;C_wHYIE%Z);3wEBxne0DP=q*+x zg+|owYq4UOh7`7>XfdiCd;z6yhOL~m2h$MGVhEhleP?q#I^(}{xf(o_lEo(Snb39V zlVp~RIseUMd`_%&&fX3WUE8Vg`Y<68LXYWzRa|igetEV|b5u(Sz=2}|NkD-{;)(~$ zx`6e;B3kl~5!5|ag(k^Nh-@2z`x4}rk zbyvfqd&cvQ3=&}JRbB}ex{e?jHI3m#Jx2#dJZ;vzdIWCQclkh#A;fzGxx@WXb5TGn zEA-*m!djhf6CWC_B33vIElduu?u*pXjA!T5VKS!1xVa9 z&2d*Q68zTfgh9q?I3Q!`C&+17n|vp|8rK1nbE5kn$U%mHesrz`>ETdX8)zYJN8HRv zU_?mY<0&hmyNHtDQ1Ui$IQ}bH&rM&1d_e&wKAJH7ePn4YH8%=e3?WqD>%aC(b%6!o z{L4jGAwxODjRB9=Izc;-h(c6im|NK8*VRga6R#?c*$4joslAaFv#e=cy`1~bpfAlQ z7|=1^zJ{iD^O;R0YQc}XR^Ql{k|kYN`&XQkWj^Qq-KEfT*YV(CQ1tQb3ER;oI9OA5 zu_wj?)}pH?sqtbcCnsW{)PXiqBr}&!MrVX%d_MQ7uYktx?aeTRD3znE_9Elry8x&p zSdeBaG6zl+s^J7Jp8>>ZoslbC5pc_($$}6U@W2J~*BuW>VzV*@rEr1ROtE6UzpLWC z1WV!*Zb6FiH$$^xFU{ZxX){SwGhe>%C!^{DcJwgS0p9kSg=XVRg&y+VFwo#x}8vt4#c2#Tzl{ z;FiR45mmqw&m5}b6hs0nsS3e0II?MuVJO0d{%x~4)4VlH5cEcjVI?TN5~q+*RRxxe zsH|{`I$=W6Gsxweu-M2!HPygGHeGriw7}aBm^%}%u_K=-(;gDJH1gMTEFLZA@}x@m zi8%uyvvzG>_bG*;L}iNwH4H$&J_tgGti`w%W03tp&V~b?P;kNy_jpGl6Sy9&N-6Jc z+dO*iqoN${twuHC_c)DD8X%TQvS}i;f5`%Hzr~oz@wzHI*sve9C4r>;1n)}Msio!; z8cj?{mJ_32^;k)nfRT!xJ8`N}7+Jj&@ zK?U!Ho2G4sCZ-?v%k5g}fsOcJ`q$?5n zP?hZaNVg<{nRFWkj6q3{39sr*8FaIymGY=rMP`*Hs#+DCmUay@^x7@?388f4_C`0Z z#j+6pH$x+A@n~=ATUa1E#c*ttpu%byV7Y1z3#ZXtFaa@@v!hgw8CrrI09Jj!(ivQu zE5^1+JfPPCkqn};$8-@KpLVMu&ny70$AH%nuht3(r$pPQY+>jPf;)n7yKn?QU=>Xs zX;M=cGZuPmu*zYD9gc-#*1&)3?ejVK8>q@5*KQ*!H&lY%Nx5l<`DKgo_bIOZa(&&n zCR4`$xOuE~#0*~oMoz#}jr@JwC$17|_XQ6bT`k6i9pqpt1%CfYLK2LFQU%en(MmyLBAydqZN@uRdHq*p%cZ;tPS zj}VsUvU{%ArGo4bMU<4=;vd*qlk0Sz zyocsXl5E^UThEr$U3}uXLKvzZfr_wks%emd_`4vNz`*PzR9~zsYH43j@%URP${SuX z(jy0|T&hG$6pn{Mg~1>ryzP3-o&!d{5t+3`1U|f)V2B+(^X#-k;TG-Xw2BJ8SwVNg zj3rxUN(qqS^D)8TVcKdd+kv};J3{2-94?u8W+8}zk_3brZv>4f$bu;xvRn7rZ`s5$ zp>)LxbfL{f9$^w7f+GD8t|>mMUV}xrqZubXXsmbNwVyKgCE=xW%g|eBTGoGQbM93l zFOI2%276EpVs;yX7qK8YLDja%MAuO*J%j|E;omv^x}sqDa+q~9-QULRj(x8Yg%V5ILos>?gQqNOC&8PSL! zp(M@6JsZ3c2gnuSd_y{rqvjYR-y}JemFvFj2*Vr>=c%|-Frp_EIQ|9Da4eiljZZK&pP2u z)Adv%SEo4@Vz`RJo%7^opR*JU|K<*hbUDm^QPwN6$UG}t)R{CUVksdQPzk=Uga=sO z77tbsj!YP(+hXtmz=>)FBjh(7-;HOT;wm2d3Dml}@BQ1RZ#Z_|W4Aa2b#TKgo{o8R z8`YfW9Myz7w4cZVKjn*$ZxGZDdv0mX{7^DKutavf{j~oGhRaU_&)Qc9x_U4<^W>Q{ zIXbjFzEm5D1QPS;5o7+oMR*WNmgu!iLvCSb30}I#XF&0?_UlKoTPw_&Yd3`z&id!D zzL~;;SMwY0^F9@JzPJyksSGq;XfkBpu)yoB2LCH!L$y&akoeGVen0h{@WfDsrm)#! z@H2V>vCLBB>o1`dWckJ8oRzk$P_SwG?mPTl5sVr1xKPI(VPc^b3BwXqe(GyCThXg9 zJ@LEM`go>BXlE217=I(s8&;t>pp1q1P>GZMF@tD3>dgWcr~kDh-}>py(6k3N+nQ6G z{P_^Ql7(!TIz&et4+fjI$9MmITl5E-zMp)DF|{-K{}|`L)&Hg6WRt%=7E8HDV7eQ@aesU$q^OQfXb$- z$R>25g&T9|3a^cy=NAv&asTUi(^tVDxsyfJv`G5#ObMPPAl+NuxN^{O)2&&TG`&A% zwDiN3b%D?Qy8U`u-U|@C;@(pc?N=@>^nho_kM&4#-6D*;`fe~`&EfQ278@}9ZWC3T zHZxZUpHnmcqZH_L#a$TW@=<^pBWv@z8X3)lfrma+fSaYY9v7gyjHuhP{npNc;KL>S zlIF))rpKLgey(w>b@Ngj!-?hv1qLg`t$T9xK5;0~_aHe@U&%-;Oeslq1>itZn(4wME4qc`Ve>WiZp*0a>`ho8CIeYH z%_kSjH~J4!U8oZ|@YinhmR;xw)0zDX>%)OTB$QP|q71x!j%u@Y6CekS5F_-*M}?o0 zWC+sOr0V>;fj!A=cD4Pc_nNv1qn?FxqAV(N&2xve!EbL_+n-+MgDa^lK>(wpx#iX~ z_pzf^M^v;>W;kuuI@EJh(45h)e8z zo89lEd3xU&R^fVv5?H%iO{VxyL&D_LpWtuds}p4Z{qRCwHGg2o{*;PaUQfNbXM+pY z{4^Gj6ft`*;AnYA7aX91gf}xU!&B*CK5M*-bGq1SKjxCG>*VH=ev3%Z#B^C>H8TWZP)zL zzp@|Y1Dm^-_#Py15n6B(HS@Muun+iR?{ zgj40%LP^9ts}LKgYwOS$yVW19tuVYXk?hydiU;XIEyO~kQsSn0r(JkF2J%Cqugpc* z#bh)C18NO|2DO$O_YOKJot!Qsl*7TQ&4ewB(S=z7RSKn542Y<5nl$r6kzhgza#do0 z8jWFbI4J`AW>owrlvdqfy z5v+zS!U-x%xdFdLd35nd1MdY*Mv`6a@b4>?Gh!@HL0sq#6GOCPTe>abN4Ihm z7rAcsTKiqYhYf@y#Y$DERfHhRtI(*omB@jo$>h1NS=`FpHh~ASI`r&K>8t1b45!EP z_O1&-NTY?D4b0|pn8nC%;5JF*>Vu&(Qw{~*)bt>I1f!@N2kD^8e@?B`FFjfcm1zsu zk6_)bn!Y>*6DuyaV5fXwvn9nw+)!ke;Fx%EqlF+s51}dhVUoM%ad-TN1In{p`TlqY zIGpkari^&W85b$YlZXK@HaC34Kf8@xQ{`vaX!^IfCTK9XGcivqiB&im3)>DDoxO~R zbqMMyhqYOzJGH03uxuq)?GbE74kX7>N0__~gF3y)1!s9tFB+?aoZoE0pLH7QMzs5+ zv^#EW%rewgJrqAw>Pxjw#`(=?z0!T(#5c0u1kF(GrF4KImM(%yycRfJ`%#ekA{IBZ zKmPnl6}L-RP_BM|r?`(6LkBhYPJDqvm6q_Je=A54HqrF<`ZKrrZCWw;Md|vj=KO%1@WO^x<4e4?->aAt zcU6kSOpACRpz(5q;}Gb5k4k~8>rLe%`Lf_{>h_`|o{1;Z|JsXYBMA%aD@N+PA_POHs$2!LF60}3~@J?xhgOp0T_UAS!x|_XE&;)j5;fCwH z{}M)U(toGHnI3(7SV)HY2*93PLE(S!OVjbtMP+t0T6OoS&tcxX)0z<$)i!;>`9x7~Cjw4|MJLnQ&i3-lS- z^^l{5Y}b!$=+i*HSKXC{)f4AQnuOM8lpTJBm~lchIKhv(z|#>DXiS5(NaY=BZn;$W zLW@3WOcxu_1VVkT8;GeEuysdafZR@|%7lCiMilw%%hd(M=h)pH zecipislg8o4(uBUzjG~J2FmjLs6N41R%1#cUJTAk-6cXHjRSdE%+i7LV{9g)w;~-t zYdpD?u^8A&<>5|An-FtF>yfH}P(mOF!9A5J<3x>@0qe}bkUQHD%Ja|8`y$wOqDGi9 z2tK$9rnEoZ#D@a@UL;mq+*V067WR!Jiy!brQcjGEiT?ttU*Bh~%Cw-1+htcApL!5Q zrOX$tO<+27SG*!V8UsHFIc|VESPy4#%lCLmpP;$JHU0SORE$(jGCTVJBtFm?TNlIxB*=+lp2b*;Wyh^wR(3~ZRuNhuxcX&c?? zwYm{=!g;g2g?uZka98e;7?f0-?Q5|3Vh~~EPV901ei?jbYQe@qO-*>>rW15~?enmi zp(fkH+^DpbWlsV-g?%dtLD7-=jJ@!MkCVN6(lW!5++i6*wA0%!>BMc7@ZV_;wBp?A z+0V`+u85_ks5N{DrHgFRC(@zFNvMFCBN@aiIp;RsRxPy2YOaD%b^tUj4fBo*kFlcr zc#jIBXaR4K099C|85Cu;`6&IAz6FF3+d@x*{)UCQ-BGf63wNizp=!QFYN)t;wvwWn zV<84 z3GD|8om;3(jb;Ki9ypvTHy9{djrmy{Hs#U)ypi>Tuw)}UP<$gpsU#cR}}M|)2ww> zPj*}2f*f3H9%S?u@+ND;BE-`484CBgTd7He+Lpj;g(7EYe3USOt3{7aUa)a_M*fuve&e(S%D{>~RMOK2^RM2hm} zx6_z4yimJlJfuMz_iSbL)y0b%N;e0<2n??Q!G^6~nCRTyXzQ0tFiU3}TnUQg1? zVWItPzp--=#UzXH`7qZ3ahR^q6Vz!3I>yNv0as}*n)#ywWYst$S4_Lmo2o<~v4?d! z9O;@?w3lSSzRnu;d`R|0;YMiexGMxFlatRZVPkhgE{o*jo%PLm1dyD#5vJ{TY?dcxIkC@EF1}`KH z(GI!>F{O44p+Qo`W{u8yPnMki1v1Xs+LL#sX|zk!&Vo$_B8ZclQD)b*?7dQ1-8+}M zcj#3R1Dk=>`27VOS4a21j-H=~VDWU$R}gFJByleN{Y{O>827)`;Ej&am)k#az&`g6 zJqCvpjrVGruz>s^SnC&a66sPJBvDg?xry z(mO({v{K$AehZ1=?9EE7geX&k>rpk|*$jX6TxrsXo%54b)e49_s)^A(FU;IkW~@+0X)(ExVc@u@My4(ir+zq6~37oU^7u#c;{E)?>Bgv81)j9cm7(e?gxDYM?I z{P$_T%x5Je>j5dH(RHe5U$u4G59Ho58{nC$Zp_mtB?R$7pSOeJwz*J z<$0y?^ITC^nOPXNz%9hsSKskb+gaK2QU9Qh|NUOYs!<|!WA37T4JNE;gs*Ivf(%UDumCBd0aW>~+lz#{ECw zjk3Ewoe5jrJxa_PaZ$=+-~U_nAscKwyPna(nUynnSezF=%zIVlERcTah5C`x#aoz@8`~|U5e7lv;F4oz@ zS^4#}l-I>p<=f0{AhYH_l~~qhz5ALC9v{N10C|N9F;ih3-fClQnJ5Ybb*?jOs}3Jt zglFot2q*51Sh{%Lsm3@(Mfj0SRrT%J|A&h~MmvQ&=bcflOFU)1)LOu4W&srDNyL%& ze!Jq;MMK40|1bCqCRvBLxTq(@!rvA1N3l4Ul}g)_rzGn(?*peBsvKZ|oc=XhYj>v+ zCD=u{9c_3+uvw9=3%5(W3J_<^XrU*s>bKw%HGwf8V$vYw%+p=Yj@~i>mW?&>U-;I zAEa(Ho4Tc3+ywcj159TVQnxO`vFoX)s8&ip7|j&RTG{UcE*T6fA&n!;b*7B; zG@YbQ`{^GjY5qb9q?vKP+5Ev)h{7t##2LO;k3WD}T6nTd2niMhV4cDPjr{&gr^SUZ zW$A(lxozXveB{KMSWt*zN?}U9t9@vbI8@%Vu`=@aL}A3jf4dDG5Ve#q^h1%~yW5hS zxG~PKdWQ{=8;OT2&791^vXEO*)OzS;%KgmQPN*S8T(TFzc3KJz@pp7E z74b~)X|vVNDr`5L3OB-B=8mCY7B2P2-8enncv{V-J1WWr6DQzii`7E4<)?Os;cnT; znwi1AVc}9IADuX2SVJ+=lA;zBk6Zfu!st)11v}MF)y8cv2)@A5I-8PN&azZkNLGLy zXft7pR4B&itYnZ#+FrSE_>a=3(Z!pOp>XP`r;HY@MI_RD8ec#wBN@0Fa58tYZb;A) zAta8*&eVL0C}Ygs)pk)rk@FyONKt~2?19ldL0~}K7uIr*KW&xpssEOR-u(>+xqSB& zb-Pzht_TzE=o?D5dh>)@|3fN+G4dMTio*_>D&~R|3p;Q2l+*&f+w~W}oJc!wcsqj` zclOy-y&)%-f;+kV1`hUDgQbKFI|;wF$lyup`Z9P=f5~Pv7guPS`FSrt`qOwyyCsIT zgvGnah`T&i+4douBd_~UfN)wXi-hG@_OBBK%y(rdOc&-?T783dGSXhQ9&0-W19~1p zej!jeZ?}dKVfq=+M366jU)rSTNwxg-ZAd}$ke}#Y5||Ot8Eh=^sK3p66;ff5!-smR zP}~%>F@3)sOAPKi>5nqLr$rOF=dVWMnlRZGSy~M)xF7Al`yPc82avD<$O%X&yGL+_ zth+zC_?C=|1(@QrFj8i>&awxAQZAf>|G5ILaxySV zHfjp|@!icU*M19UyYdNdPZhS%o4Q%oAP~~CY2R?1Q6vkx+e7yLt|YD@kvUvA6Wyx;V-pi$38A`RQIpO??%Jtv)vpO{-SB; zjTXjpY5D_dc#$1FkTfb%bb>17Pb%V+5!*c+`P#!21uxc3EQyQTlan7b2U$}-K?I2c zb=ZeutG6XB{ps+G@}S!cFuhJQfMdDPIE+||G4RF_M2VidrTRHf1*8-9qA{4{1YS1O zDeM2@E`^6vR^{j(YZ;B=!cD6)|lYgL$y#GmLl9sl1EXBeP z)35@yj)Y5hF{dCATo_&!%jP6Rec)evx)F`+^zhQ~m&23s*sb?KGuZ4x2UvyakrIMZcx&(F_z-1k0cDir=ba4W@dVY@ zT@_RMObd#y=X|P@6jO0Sl^emOT`%QU2yqRaGObj;PDwb*oY)u zE3Yij3^1nnBYu|`P>Nqz5tL1pZ%{?m-tINNaLl=7_9Mm8yq3;Ec{2}FNk{*tnjS24 zPRpln82L}^bR8*cNTCgXh|soo_(I8%B<0@;I?i!j?<7ajh#BI^2)5tpzBHyC?FiGTq+rJ9HiAU zL_leY%-9&>um#mTjKYB=Xs$X&siy7Zs@gkIU;6V*9gZ~_^j}!QV>1{9APLKDos$T^ zEMwsDXRh@xn@cbCYKKn(mY3h|bOQvYMW%rf)&`5q!a&$it)|d;rO|k`cOS%FaErn{ zmq85?S#C>(W)SaVEU?pW658SG2u)-JE-aJya**{kbFwpZHV_E7I*3rU|LOEhs3 z^{DM|b~J)4;eJ{YPVOIgQAgDqVF!FdW#b<5iv$F=<(^z@GsksNpZ{t@BQFc$mGrSu$R zcrjZX`WUGiB>@IDMkd&qxR{@k;>H8NGp;DFa>gSw)#1E}sl6~_2aDN!IS|+p=spf%^L^*pvKRY3m)+f0`JyH2Ap9qU$hxFO^rS|AlsmLslwIR zY$h1&8lXi-y26?TowLTM(7}ml5OMeu`lP?b&+2T(RZis(+mkByi|i~?g=o;l^P7V zmDO$7LGfY4eyf)#L#tlK3Vo8<3N)XHMWaY3!=&VT9-tgct|TwRd5{<1z~c!pS4HOW zQx>cFXGTG=;e!b@aNOPi=Pb8Nrgg&3Qykj5PTZ7t>5ZuOFHRH$hWt$_nR_h~M1V`_ z=X}^lKtCdGR1nUmQNbier$>IQhQZ`QVUDPl)*(u8Hz#B@Y28U;kXvG|4=bfFybv9< zWN}CL7@aaALo4pv?wb(PIT^`_0eM*2C>T0rTrvP@M4n@U1nXJ4>rbKZHVmOa1hlPX zAj#VLvPq~Aq_rdWQSvRg)obhQ);&g^6Ko>gvTWU+6z`_~Ms+D`eS$(g)`&_?@_WYj zg1gnH)jw*Cdyq^&>8GIikDbOR0^+|?wHziK9T+ica;(a&x7;d_z_jCst+mp=8&Z6o z^2G5meL9G38JMiL8oXi^>IH0s*_!zS8Q`n#!ZkTD?O#YGX~4mFmhyCkTmjP&vSJ23 zD?YgAL$TR!h2)Nu)f-Z(rTm%39nzp}Sf@w#L^0Qi*H=_aghNwD#`(I#svdzIL(#Ba zcefUt*bykF`o^H^c(RY?LONcC{dSm(a9SvF{iJN5ASw`e|M%FdZl zBum+1RTdxkX>h<%c*2&K2fe8Ro*D1V$jZuFhcqfA713H(J16vSBOjSDX;+09w#2@> ztzX7HoUN8hUs@7(k}d6~NJF74@Q)@F2{zl{g8E~KnaZEWjMw&?9T;7t2z?n3bE*WS zj)L$KkXy~?UxY<~Ks$4CqC#Cefh2I~iF6d8_h_hyX;a^wfR9S}hL%hY{rR8U^hn!n zr%guKIx7FZZDaM{?iP(yu?mX8;KEbapb zSP1F&9nIfg%U7e_FriLQO$aUcHH#F+KrOzCv0#Zk2kH5uY#H(8ba1GI^^n&v^KZN5 z(}d#WY}9KN)c6#6-)+z_UY>s1@hE%u-(VYT{Go^eGS|0FqlT&2>$-TZm3=4v9bf(N zog=GaE>CQlxBKB`BsrheGHo!f1#sGLF-Q_&c0ISZ|1C6DR(mwVJqqSw!Ey;0axk8e zG@sM=og_pjg{r?b4wyWjR3?)Ne(ngUt_-#R|0;v|e^&NjWo7z5%N{k_+VNZe8vF!$ z3j;w(w>cNFl?v?;68hFX8W0@Za<-u#_L$&ehi=rWQ#U$1n4uqbRWOnY#V;HaFhQFs zlZc~-JA42~HvINOBjExKQbBSk=6ad7QI99D?pd3+!Ez_2j{n+uY+i`BuT&Uh+1>v` z74Nlc&~y9Gdk@ub#TL2VPls2D57L)L(E|#lgGd`xuZDbkkj+5^Y;U3%D{o%j`|`4T zpV~~4UNpv5tB;1$Rhb=lfRgTmSw40HUvth+^)|TEr{v<*;aoIIzE+zjiz4^bPd zSgUXK2WkqjKTq=)t-PnRYR65njp_227sj~l>~79-LfU(Rv$Mca@PmbLZp2BnMbGNA zyeOw`x+pIKIx6P=S3WoG=%(|}Q0QnaHd-4ApN1H#!9Fwp5}-e}tc%~l);qmpJ_rxQ zqyKx(O>m1Jj5z^Qi>C+ENR$_u1dd7ZOjXb6e6Gcuo{4Rs0Ak|5C(aTfa?d2tq^KO! zZBM!zffwY!qvx$L!`gM|?dK-X<911v+xp9-{F}Cc^LMiZ{V`(8LCxNJgR!?qt8JPc zMUw5OIw0S|$-NlFdRZAR@oH?66j^vNoJj%k;mH(3H2S^WHlgcdnz5Ko)nkk28*saogrOTXL`z)M&s(W;?)LBsZf&OQ_a# z<37g%7f|T4he8LG$Ek*SApEazo*RN8$UHy|9l@ezlen(^?bp+>^50t34QvzvGK8_O6-QHBtCjhPVvF(PbrhKAAfP#o=( zVvG2+Y^$Ckri2*AQ?l?FtA;GlU!ziRX870um3fEu)7x2NKGn317fW=gj{Vc|&fCk72=0@W<93fTEo7MNR&IK&zQuaWBI? zqq^gGuaU)1+H{ri>s%((Crpe^So{P#c6jOrZ=3LkqUIH@p+ft_=yxwpkAra-DG+0L zs;1++EbTrppfnnIpLw5;JQ2w2^5836b7CfzH!o9d8u1PSS#Q$#?2o?*Z0sSarq#F= z!^5D(Lfs$8zJ~E?Epk@Ym#^8muli?IAD( zLnaobfWpXqGvVqx+U0z1hwG?LoHtm8q+?(nOTj=qD8(yvhk$;}K*nrgLzq72VBsaQ zw)$5wEfD@7$DJ{4O zQhit~-pbxWL#fJl$uM+4 z0>Wynk-!6_>hX6+AewA-w*3ylnM5$S2Mr>+HDs;~j0+pplMI?0>E=_q#T1CG1MUEqPgf5P>bmTQ4=Xmp{e7i(zsMj4xkln|Nv28jIKdpkjD zCCy2g+<)HmD!q3Rv5zcBK~@0OZDa7OfppkiM#z6~KI`*qMJOR*HVtTHX?0){zmmGp z&?0T`mSW&y{lf?Nt|or`Rt4C=*FH->Is=nA2c)3}EwoMe1?wT6I*%}hx^GZfk=PK4 zL%L`Ip5FNh;GrgnS`89Ya7n;^ofb++n%JEH*_ryf)AIS)`%=b*!wH@T#il*xR-ys; zHn?l0{hLcdR8!}MkwQNGk^Y-FS){X;(2H^0andD2;M8>3QZ+~W%pY-ggYD3T!Kfik zatI{;`I&x}Q;9qRw6uJ65$|&s=1rVSkDMM2%~$6< zX7DG}P-OdCW}G_|X>=sFKxy=gpt#XfEYVb3XTN_RLAS*PbJqpe+^svbMipUMYp0!g zem_66IzD}5@q>T*e3)NAR+CKX1oy))h;FIr{Xw$wDT5o# z)8G=3{4H_mT)gsp_CTWFuftW)>j_ zovkz@u%Fx>z6=Y9`jeETxf*6@sh+?bS&Om5jNF<*8C5^`Wq9e4og8&)NJwC$1>&Vz z42Nhw!P5vP++>1D0r5^t5h40gi^ko5Gz543&GmMs_aAf1oVUw?an)3J`9vh_>1zyM z;z=?w;VCE=96zxB0=4}l(y+4Kn*CWtU$ywQSI=5tOi z7=$*=YxpIjE{ZXlr$T8y7W7a92AzA`H+^tpXuzjwecxkD?{=U{N}L zw*_jCsX7FYM;}%ZHMv_Dv%eJ0`y^|5haVpMTf53{YQ^V02E#iyjdy@MMzFJ`E<~@R z-GrCH4shJTp2kB30$v-e5U#(`=V{e?y@qG~zvf;Kp8Shp8jCB#7~#AnDYOVWVwAWvJ*{G13|Q-`?#02wLBZHVBcFrTI2QY zJ(ah6o1@z3jJNR;O_BU`UF6rx;3f!H*|cos`A-Mkr%Jh8Xwz)>HgT$k^Zp%#^es5aMB2DEraM18IW`i z226pt-}qEKL zBx!MdEwbcGlrsC|evSa=FctQOaIM?z)@53Rvl13{GI=R4R*0K&0>Y;k#B7y^#_(vD zP+7|@z&gH+q!D69k1qMoSf!1)4VW2%v7bMd3m#fd8xCMHusd?S&L2cgtDzSlzWd^K z-c}hGP5i;K!y>oZl9rmQTs#JpTPD;CNDK(yicvmQ&!MWTod1g}W-W$tpMDS!NSoXF zl=&+lFr^8F;`x4x1IKp_!7%ipTQ}0UjbR(P2~8O*+|0QA5g$5En^Du}hkd2)HCsR+ z#(#I^8q~NSr=HAb!vCxsGCJFdC(PoTz>_z z|CsH2a&i@#+-Sb<$Sa;&I0^)rI0jGtgOs0FKzzTjq{fLq7^)&)7Sdh=?u4Zt7P0M% zb4XW?){utv+9_`gB~Rog8Pc~@;+Z4bmP|e`T!IxSbS5?c(bJ5avE01Y@X=t8QICPt z3&BFm$RnM`lnxE>8Gt^Io${naox1hG|D$fSSGG4q+GsFYTbsSTY{X!<6XL+E+hGe+Ptx&ioix>Z4g<$gp{0AEXuLqt6 z3#+bNGt8U=P(_WnidhMbA}#drpR?wC^MVl<10l}Xr}>+3AlJIG%+(>Lu)scaeYM!P z0ZNa>W#41PP4`97)U?v0FtQ0gB=`9Vx0T%E>0a_5#WZ?qlvikXZ{(@jQsI)_gqczj zDM(~9T)4U-^)WJ~B8c96fpv_OetwNJq(MC#_wdnrdYdBIUf<6qEnX8mO)Im= zTVp&(0FMdHVqY|gwj+G6Tux)%nlzP8t~3?-j?@FaOCx#NWI$7f;4yO(mAX$6nBT|@ zj0qZ&leZ(d9YuGL^$s$r;zl)&41Y-aAN}bfTekhsz^j{M!VZ38L4jlTsh3QVlXLMA z|Mp(Q(W+X;C?f_p-GW+=^*=@w0q1btWvYuoB1-Hz-(4p_2#*+fmB5PC2vDDu@5P8Z z!n@?JuHMdY1vpPjw4?-5D`)cTc-UM%(AiFWI-9(a#o>o1+PCz;;OJCHL?gRzKGTw& z={6m!$U*W$kUmB`zP8or7rW>g{ipEpM~Xi)2WSakJ}Cc0FTRuPMD!f()!r{5XH|j< zIQvZn@6z8%j*{hk?PyfYJDPu!^c)7VdA*UFkPxNs^&A!Qkj9Y`*mqt`y%Zg;d4kg&n z`0Jn3BE{a|_TGMX6wTXA-l`Jn^aYp|&EUB+lK0t-(>1L93WJVg?v#_72Of})m8-J> zvb%qTNBW8B8H?!qt+_Xg0_~R_nFJ@-vobmI`?RnJnHGosrixT3x+0@}YNI*Y>*fo- z`TFC|VM=l)&sCc>Q`@)k98>$=VSYAwZF;pz%6u1eG(6o3vTniR`RU8LtvB>2#Q^b% zXENgRTTt{^8=SiQ8%l?71SmDl|J&r(ZZ;t=M(4WS~($^>R1^m{>Aq+$AlZOjD|iUdbzU|K3+c zM>;m~{@wP`?s`koPEpP{TPiDC;-J>p;G=n1D*Hmb*aL9tyaa3L8|Ol4giAlwAtj>(otZWHKkE<=V>Q6Quq5DS+s~s^_~NHVKs*3 z0&@d#6EJgijWh?x#}tz{|4N9X8|Z3Yyb?BXMn=nap||nae8$M%a%IzVan3pubJgVi zHAjz}ApP09>KU&WlXPGDF6za7&8d@gcp=$grHHX-)WzEm%*=)NCxCZxz>bsY_l(_1 z^UcLbk+(l(?5b#!^!U?E7MY&Tu-axpO_;2>>_E!k-n0Qi^8o0A8GonsxJy*6KBv7E zf05#nqJK+tSd@QTC23QM>tV@^l-3ZB3|@)`EE}F=ot^%EKyh@U91q`g z4`;%7NeU-4?@S>bG~&l!YoQFeY7OjmGVjNlvLyU=&ykv#qmNVH{qK`MxOiQ-Wa)cD z>t=>|wa6)3+vWrTldUaNbTsoD8-tZGF!=hr1w45H{3Kdh1uJ%OJ|F)1zX<{axj6M!#qm?5`7`lS@HU*5B?(Qiag{62Va9?zkk_sk1Gl5zDLO(Cxc|5C)DgUdYwv zrmXE35)ggR;}+#e;Q;6UCl67o)xmzLmQKUa-E_LEAxnEneerr`~C?mf^8(b9x2JT-@{hk_eg|}7?2CxRW<)T?&;urn@4;c zBKtPCufdJ5>+-01Asb~2)w9wy6>{HhR4lVP`8h)QLA5-V$oy_y2C3oGdOaZE`##t_a$&AKo>K=ff+U*fY zYvrmT;T*Z-mP0uH&)qhr1Ts%Rzwixl_q~>pNFyzxszMI)x;IgBZE-%BfV{s)w#TIZ zmKAXa8#6d_I~cXPO`_eM&#&(&0Vk(OVOTFf2YPJ;j7A#+NVYZ{@veulUd6w>F1z|n z6dDn{om1;#U7pkTC9?#nY0|nR6b%xHOO!G(6_HFYrJ^%55#NT7)dhl0Hf{U1T2hF?SjO*%Jd1#a_);wf? zf)KMk`Wbp+Ae^(ft;u{BY#*(B;lr-B*+kerVcQDy#n=j-7kuP6zF1Rjrb$ggxNcQB;?A?7Ez&~j)>y@Pi36@J`MQ#oRH?| z0B#O!fP4%%)1*o0cl|(39)slDa{uyJ`NZfCMF!AoSv5binggkIseqnB;gLhH(hST3 zNx=a+5^6rf=F-U@>|k)g8kvi~xTnC9l%ZC*Unu>de~b1a1ldJ|bkFa+T70l@CK zl$L))3%LR0nw$N$yj8GX80Bk&=S9eaYm!_hm4?1(f(H;ebtog6h7f~%iT#TX_0MTB zjK!||fa+II1`%#`O;37;CGsR1=ka)95X!3-XGje|wIG;+6uE_s3P( z=z}CiN}Hhrx}mQgMkaECFhA{@`z zX=X!(dKM2Qq0S$2mX|`|%MhM=1+hIDrQipI1|lSE$zkZlA8k0>;sqkP1o7V_1JdRMRsPwQ_RE1nUT@cQ*d00?Sl2ZxH$tw5dK*A6|9W7~Q-LJ&kqcNM1)W(V?xi@}juii#k zE!B3bkY_wHYPx&1GOSKOBXj0pdxJ&on?MEaveC6tq>~i`jb9X_@d|_S3ro1fQn$Bo z*GonbIXTmUn6G$dBL+;WKxWfy-?;0UM#m2lyuf|oTjK4JT5(eQ3CP@R=b?|GHk%^? zGnkV5%Fo#fIRTfyhh4SBCALgEMB=O|1H+wV#>g_HMu7wT-|DtTQu?zY<3ox*e)Q87 zV+=KTw6Rj2SffvTYzvB^@d zp^Ld!KGQ%3mw~DT9eolC{_U)SC|0MRk zHYKuj@hs8d_6gndaRx`rB_^k*>IID2SolUQ-}K%UtAgYFJs8G(K9P2a^puW)6$E?3 zn534UA`z1h$3iB=`9@$R;iDsr!S(_0c%OF1L@9e?S z)gsk-0ayySElWPQH`K8<)P9lI+R4I2sVq7@t}>^)diX-9pcS>sNe};-FM|R)=+7^4 zNT40eW$0=Y99JUBTPO8e_8N{i*KKDFtLA00?iWiJByWD169>u~wA-4?ogjB@$dB_D z^92_fk*OAZo*2FvlfA&yr3W`+RnW+oQ$Fv<``sXK#1!3wnATrF*V;I7kl-lhR)oJB z<1;mcBS~nC$q(sjxxqcOcjsW5C|43oQ#kocbZ5FmAvD5Y`JCH2Ac=;T#JJ|lyzbB^f{=9XqS=1Cw3wC>Fq zp~UnaI7WyyP@xXAZX354u`A}d2Zfa-**z?gV6+0gLG;U3NUMOc2bZe!@-YB=2#3TF z^(1#L?s@c^rQ68+a%u0)qZz+E5H*aBye#(gPmt-_!C$C^{$J-Oq5C_k}f9pa^9`ZR7N>s z^O}7E??(1>DK?q;`LNWjgnFH`m+Z>=-Y$Y}N79i5={E;SB+_RCTQm*{4Qrnse8)Gb zSF3|3KVqe~zxvXJN6BT!2m_Z}zNlExAiQ4YHw4;lyT-XkFC#9-P>c-`Ktk+O-uU04ZW@$Z|0%P#I3j#Kb zv490=ddpXK^?;Ii!~lzs2k#2d;!5S~()~#f=Q;Tnp&5q(n3GjG%2Ysahz5>x`u;wT z*9y_ou2@38)*XFopGJB(5tS80_7b^34S%;JKc_R{d2-hE@nZgo_?uz~i}RKpO$B_J zdAGzQJzQU$)e{oKiW`U3HlUHfQ)RdY@6Td$;CMXU6g4r8b*H0wB2F3@r_Sp$jYU;U%kz~hG>h}*hjzhGl zaC$Y4(e-ej_dYnls3sW@WcTQnC^(nb?&>kK14J#1DAtEsQTZi$fz0yy)*676^!f$; zF+AA1$HALi1_WukuW?J<^or>4)x#IzQlrKhH#vtE!~uv~7n4zB7kevCZfI40-EXVV zcO>&}TXWUhN-R*$!+;#GPsY}g9s6^pne3mmH~!-S=KInipch%ujww}Q($gzAQoMqk z=;Vdp6|M-z2b-`9P9v1plKA;MTyHT5%^7|6WT z!V5Ew8{r=MtpNFWZ;e7RI2uzMerrGS3~k;9#{_Hf#A} zjSku}R>hG0ggAU-@2F{u+AaEw&{3wFL(l;@~3_bQ}mYNu^6?yKZbc)Y9 zkQz3uQpKb=p7gO~yi#W-)Jf{1zJ*4HV{QpzUvh7dOyu2~SUk-Z1-d|o@aJ^&`ATp0 zooWtlBvFgWaQs&xGVHwG*YGl0o#qQv9QwsnDmrt)UjGc1)GT4mSzFvFHQ z#~5artY%34_DgT2O8ODWL0+O0tBmkt@wf7jFf}aYV=c!L<>Mk-LD)YVu%}MvMFgcc z+J;^ZJuie}Moc`i?1ZiG!C65IftLLP-8xu5xBCWOajX0By7 z&cB?8<9sq@hEoY@Y68^My>1N35dJVfFU<$-Ufz;1tWtG+P}nl^QeXIuf=#fc4aMM& zFFQp3b7m9_j1L#eKm+tcbPMs)r3Fwdl;re|yqZ7)8T*10KI=e2^#;WmaOv|)KfwK! z*N)l=5jPCuF<;MV*D`Od=aVA@E;^*jvgv;+1I0oge$r5dIKO}0eL)POjzN%PqFGf; zf=?pII`Di4f$fsRz`{pmLg9kEp)e0G&FV+>+$j=W%hfrC(MyR|*!HR4^Tnhz2_K^9 zlyc+$NR#l2h_$YVc%{H6*A>^IxY?x3w{#&X!zpx;QZf84%FVAOwf!b4&$U{dh%oHA zZ7GEVMVz!EJjvFImA0nPryD~(R?R0~6f>vSgZe%J)2uYsKJteC6-&;MxMuKD6VYo*`n$M?EkK3*}-X+WQBt@DUF+LRrRJ9Ds1|lXei|j`|xyv zNOA^!FvPXsfit=GPh8sc9ywYGtU}4f{`d^-D)%Y&p$0zEc93Z2*fv=kDzo3m(hsPIPbvI=C7`S<{}Ter#=*t@{}E7J%eal< zCojK{NDz4Ns#dZvb67MfJxG_h!3fT&Fk8Xs@`S-m5_E#?YCoSHc0Msr8@3^k2~JkK zYqy@cEgOQ^teiYs*N80gtgr#W9OEr( zH{o;co0bgYX5T|TyPv09%+tbQhH*D_FjC{Nm3cnwxtC2bX(xTn9ELxskTw@l!lP~o z;+2Xq%;GNlk%${a&dzIDN6y_an_Nl2Kg3sK6rt>4s|0Noh#U3;(M-93-c@FBYq37D zi6`*xi;pbepV*b<9Qh$x;BcNRkeQJPw{Ao|=oN+!qdy4o>gRmurem`NEu}1F%r<(y ztM3KI2wF*8;zRUYOa(aBUvbl>(HaKDdxe$bY#2IjjR=yH_c^FZyB+?l_n`Yn3 z#UBojfAaR0BXhl+Nqo#lw3+0uK6@X{h0TByGhl1gHtd<_lW)g8rvUw$^FQqB8!Z#^ zm)0|*&z~LBGeepiRL+2y0>a+xgLV;vvG#K2LE1y@*8I@QUG4i}XBEGae_q)vnuV)W zO_9O5&aftm@wP?vxh422e79V&hO@ft3zi$%81%eei-MlvB%SyI2^^JP^LHhKwjdak zcJNOP2;(GvKV`v*M6_-o_q#biSCZ~C)g{Ao@y#BcD z0BSVioB)d#WO)&iMD5apU`@)M@b^n##Y@Q~ku$T7gz^2m8akmvpFJxCLniUMptoUf zD$amIl+u^j15q$7@gEfHVSd=Z^s&5zr?I0-X_T$2*WP<1gsU!O?mQG0?wq<5XhL*| zo9s&T(}`^~7{m7=4c+WpQnF0DQnIrBFJ^3lB#B6*&2ao>RRQZ*Ri(zDFqc zx|(padc7^Jy0VF_PLbgdkxEimv#<)c*s{RPfxZ-uLGvvxrdUyy(Q}A681J1iqKiP` zp@pa$v8!zvD=sG@`GnmbLqc;5(4fzWS*0wH>D+QCCr!3_ZhBoE{s?0H;Z4+lf?45F z6yp8slGt+K04r5p=ZDVChXU$m;ltWk%hz}~#BaiqWe zJ_ZbQfW4tzaY05mnSGcs7?<}h3UVlN+EBIWQURq`b=g6M_(IRn=B$ z!%eITYUc&4W9iNK6OYGF6KSW4ivoRD{?hgv0MDG@&gYdR1OY!Il_@gWU#)XRFgykV zDIxprf6O{ycz3>iS&zY%@*&jr`6$W@G}SQ%?pM!1!)}EDS|Q&~=lh2aHWe=9xJo0*5S{na$qm%O z)4;zSq_W=m0gQ*=aR7|X^tjEpwM|I;eE3^Q*>>Be+Z@z|Fdi_y-Q_a6Aj0GlkZAgTI z2aKGoi`9y|8#E@HD6ncHbW`&JsiBG9!q?x|a5q-PvB#x{LF%#jvh)%8_9xIZXb5%z zI4vm@C_%y-%0NRNs;Lwk90HIO>+gB21|+d<-?p!Z=!4mR66d0agE9x;LQplUa4L@x zQr}_*jDtL8T*G0;eT~G;z@1Yev?SumZ^yVri8Mv3eV1<{h5R(Xvtgt zNjUuxdi?I%gaS=*99?fS#XdP9>v|tjVW(TsJQ}XBpcbt*x8#ogV2i- z!9%0>mDvfONm#**cUcziZSJDbL=`^up{=NrmjFq;0$#qGO|64_f*z7wfLkY|vpMlu zs8#I}IX%`sF~5-ho0N%-;F4b@bo}Z`u_ROc z7pM4drnC;z^zC-C(T=WoknaAg&}ejZKkkIP8U;~fx^rJ}5C7Q$$+N__Z7dkcXF+uaK4O3QdccB&~7IG)MH+zMYRIqiTEA|sW&ojbQFDqaBWtC5IM zt`RzK9fPI0fRv06pjsGod{m*H5c*G$%t*xq&Zffkt{yf3?+WGBG4cQ#kQ8Kz5E7O;VY)e8bG z2=3Yn#b0eBx5CFN)t)TuIsyCwaGmf;eShnwk7-j7VFyFED6H3~+nDi+kZBxL+zs-r z2zCDj-PtF4_vtYTM(fBW<~+m4oLPH4rV!PXDQMgaj8sY{X{r$wE~%f*$t6G=3!kXq zz~d&QrKp2+kpJq3=#z}d7ZX$tFkz=FE@wj~Y5=`+FHmB|qF@d7=cv_jG99zb1()OU zm%@WEaBTo0N{PqRtSl&4pmJERu?^c(IJ_Kvnr`L`ZLPNqkAg$DTJl)_Gr(&(_)R_d zonQ}V5Hn==!N-Z0`uJBJHni;GEsIOdc2EF1`knVMZI(J6nDl-#c#GWIafE3kQ;SK= z{XvU+voo(^EcFH}VxS?M6KX4^cCC%54PH$aw{|IJ{}+8yj@idJDAmW51U)ibP~{3Dgjo z;T^aIqQT%oBdwEhk!QDz3HXCe@T~DotehKxXpNcBf2wz@_pH>q?N!a$+%n#xHxhj9 zIW@Rx_5$&Jbt_nKyU>63r$R@QE-R$sRibdbhvUgKm0~s?c7?wjKd_72{g=+F6@@N>d9a3n7Rp1cZ7bqEn7hg-3-qD4iJfhSmTEDhFAlejdWm z;L`8)#&Xjo;nETQsmMZWsewD1?b-z!vhoV^lbzx!Y7r)PwP(@@z}dy!2|V>yYAm@( zSjhhbXR7&@Mk^y&a5~H>JCpagXhOR-3Am>8L+D{VgTWB0MAudMK6F&X(ObR0=h7N& zuv*U;|0N@HDf^-fd?Yu=Y*D?2tf#u$-=7CY;E_J}u{7Y7mJ4BCKnY#=ZI6fRFTOGV zCYCNBaOrhGDAQ4_O9DQ z1}e}y*Cnw%1UICA`Q-tZ)V>|1SyLbKcxR9(0HeXHJoz9A1FVx5WqSEZHVi4A6!o!l z4O&iub1h0{5IVhg_}WcAv+ywZgAP76J2@Y~;(oLHsMRvnV2j)6p9zV077J@0JBvG3 zq_W7h4+%Oua^CrLd#npxxTNVE#2JWEnlOqwq`FEVca@j|7G4+HMO5t_vGK*`5On$A zHa+nX{vrWEz3J8Ect~U{ff0|-SCD38zz=+WU!cf`g#(pZ%Hv`lDt+1aPXPH_1|R%T z0+5n=L&js^!jIilH}Xv#zDLtSeSu|}Y+Y7Zp*bY@I6^+cn5UH;dp9vNXR4Q>-Y)mG z7ys&Pp}`$(%@=L$h*g0n!69&0_z|~Yn2BG?ys%c3x~DEl8(}Cb91@QRWkyEmrgc)A zv^cP$isul%LZDcJmF&Nugy;HlxlzIBv&;r72$$1Om(wlXSLo9uI&|2aLePM$2KPEe zCe(3XAEZZ(R%KKJM26vM0ZGPBO!6r@hXH8aWR%DL-F#a0qa3M2oA^5_>X+&ZsIf8N zrlZI46G4yn^Y`~8N!vK#1G{eVlOZECRwk+Kht#$}b3^}tgB0Xmt91&vh%v34{aRKi z$!kGOS(q&9^@|o`o25v4Cr)5tr!JTUMk^w>Js}S~gn~w1YqVm5QRj1I@T3tmvc&6p zJO(KkMhter>Uj9DQpMHN`S^^~5ir$}v|H!=#zh-W-Bic>Ji*cuc%y`UMn+EiXp zu`t#_h!THS4Nkqh>)ke)ylh+)Rib%NJze1}5t-j1grC%j_6tE#006x;&l@hO@X_xK zI84x^=MJ^UH0U4=h+?G1x|zkP+jrs!cG81VzJ3~dIdbs1A!JaWdB#%7o6wZ7L6|sG zKV3#7!T`jd@r=gVOo3l;s<4%zS!;u~;BkH5R_wU?SNOojp`GR-Pq_1ju9{xg^{J1U zzdhD2pc>-)pCKEDxCQP(^pCq50?-G8-%+d*O`QUePl?$u+8EkS!x=F%W>Tx;LtipsIH%SgENF1N>j;JWLb7cLU9~XLV|dz2 zRHjouHXPJ`T4ol{+zjn9G@uLRGJuKzsGb2$82W-Uu3hX)t21)>h+ z-6ACCtzlFYA0YlDguzf=+OlJMo(V6YxRT#jq>?*tZ8f+OVtS8ui}f?Qzt0kG8MMBFLf8pbYkky|^{mE= z=;{o{I&N{xz`RQaA{snUnqA+XFf!hMahRz=>)b;|2S#vub|jkK$75K9@G&e?r=$hu zpXYK2V;Sb!V+J|FlsT@Ib2%58l&5#1!Epd`DX7Q33p9@*&L$}kI6hG^7b|v8uR+*u z&fb*Qa^_~+4#0RaTnpq=L(=Qd!c7XQ&+q!ND5O!|RdaS6Wr=rWZB(D|`ik|@9gu}! zpJaafucYpgN?ZM&`XaRlqu`7}G@LXYy@nIS0>9f7lV_^&fp(%k#)SsHlh^|!>MStSX zOnEcse%GrPENG%Qg8uMpdy32kLPOZOUL|2OXX9uM#fol@ypUK2+;{&h`2oM5a9;d3 zNa}wi%46eT<@|q<)DrJn?14n$%?G+R+-?a_gggb~+y@}pYcrhyV77kE0fCi9NX4vP zI3+qYu+(JG<3OH)_G<|o?%7aTh?(_1HawbEG;0Re@1QTQM|zPWLiHuyR$=M*Loti< zW|6{Wfos*9$L=}dLOjc7zjW+a^*hSRcKu4fBsXbq#7zKEo>xt`x_&3K`82Op)XB_( zv(XPt2c5?c?Fp1quc4VyVTir*BbN7{UP;>(UbOzIChr^OW8*{MGGy8I>7>BIrR2i5 z0}&;UGbjX6N6@*RQNCrM{p2$_=L4V9)MYxvZ_DS#@vLaM)c!oG^f>ZngH~d}64YSN z5>(K>+Q!MIz3NbV=k{NuS#svYiwyWy6P+i4n{`7Wgx2WaE$NkCI&Aowgp48Bnu`+; z-CscE;e2Jk9j{a#w&hh%9qd)b`L{`9jn{J{Exmd&NDaGi2?(6c&xwt(chys^7>-0I z)B@_eIm%$^ACC2y9Q^Yjs?&O>?fVFexAu_$F+$pId~>Gd&z*Wlw0zMnrIBe5#wUx5 zIn^D&FNc)V#>%O>^MiCcf{y;Pq8h1+d_;`xszJfW9EYN>80G{^kC+hTPS4u2kBK&4 zYR9YBqUg{(BIu+9cS!@qwP6jm5lgjkF*4^Q;RE-q;i|bO!*i8LC-)>jou}GJC-|GE z08#{QCT^nT>wh@Hf4tQ@zPRD9A$~>lj*jZKJ=P>PdkDm;Gs!FN|i!vxKgPxUWED5OuFETl(Lm4j6j zY=`~g9hTbVuLBO3%xAZ1O?kaXc$U#v-rV|oHlV-^Gqyn20rc+X5Aw==8LD|Tka`d@ z+MOJ-Mg|_vF!y}zb!V@!YW%qI`(qyXB-JodNawG_Nmk-SsuO}Cv|G+8O;bfUa&{(Q zkTP3OTkhWwqekauBG#jx+)4VmZ!HBE;ylgrH8(8`E8Ud)f@m&NQeb=UT925v+2tVU zJ5EJ8as`rY@|XpqO`{KdjL#Dd2V_)R84Vv*RCcTDOg0;Eh?oohWs9#EtA0?wWjX&eR1x>vx<76rK8$ zUuCf25(JREdMD02#z=q8;I!(B*)(NW<@a^qId(Scb#SrKp)6Xk-|gQXIAhmz_YU; zrXiVL>ci;MQ%Uqxjyh{vfHM*EtNRi`Q9&WwHuL_)M%k0o=th_aC12wT$A1P-55}l| zu&f%fs9B>Nq`i`>Q8_xP?ZeSWRb$>}5ZepFU~5~7{TjK717QBOaOysWoWVwfI~ocA zZsuazfO5_aGC#gE^a7VMua5!#Om%H&y%V&ojZw_l!xNiLiNNLRc|EJq-It9v(uqr> zaYHW6=0%BA9_1R=Kd`JLF|s)-*Edo^a}UF%MZzu6oO2{4B0HG(AbLRE=qP&!+b?DK zhP3pLxzu(yNp=x$Dedj^6;lT@0c;!<*YK~bH7NzKB7O6QVmuMN?iP2%;x^>=c!QnV z<7jipAZst`#ex0GPFKJVHW_WxHQ9!PvjYkK1tn|3-2ak5hLO4rQ<~33iVYGP8*@Ko z2@TJkL=rKEjzUUgwm6}#3<&BLb*@DQ9(F-=_aTZ&9SoexRfy0Nc0b;nC#nHj(_nMs zz!BXAG8s|vXRR7^#RMbZ%l;6v{^A~Rq=eaSim%RU?E?Aon{`T<#Wx2GQdb0<&{W{VjC81T?f z(&%n;3=$38HV4C?B`gDK&%T9@ph~NjS*Az-to=%;0)!l1F3~u5V%H$Evmf#lsM^zv zCW}a7Gnu!GG6H5)xt)g&PqysZNITGh3>5~)eo}K7z(M;W89L0XIsbqrK*`F}g;RJ1 zcKNB0;92?4TL>5(O+V?a&d?@f@BT~Wgx9r5VRH5HD_vCMfwmW8+YuP7F95*X`kIf5 zjUYxBfrE?wO~pVT?+eGbLS@4I%uxQ-y1qrMN-gV{ARkK92E4lwRVzxgu`oXw)8Fyi z2ql;$s#510;D}_NFn5irbKo|;4Ao|66__9ce6R^c2=uF}Q>WfYgEU3in#%6bLg$CN z1dbuaZmpCr5F4_+>Ym;;{T0KLnHoS~G(FS?UsTv78SQz%96fmt64f1e_Pyy}_NG;X z#R92W;>0_i4xMzyruR82fvUZYV{+t7YX03hqkzvN9xd32Qs-%#B@;a`3g4e`jyt8R zwI7LTotYii$&}B^F&H2%AqRF*zDq8UuD?g&s>&2ZUG6B9H3P3|CZUqHLFD+34rM;JBi38^B>$13&BdkTBq7&(SS0dfOCS^+XC`pc$S%8SXxU@;kZ)FtnY4So(^_ zw82bt@oLr2CZ$zHYIw)He%Z_HgrhCh-Zl5))C95}-VYe=i9-B^DL;&!s*!*@g1Zo9 zBOo*gt_A;oqw`3t+P($VjEovTqX3Ma|7SI8E<0uR__9}womqwx#>Ul?irkTC8+RBK zF3K>2oH+!8DjOo7bPY$#NE8lNx*RF`l8dIcF7{F*d96;DS+;thPE3&Vw(G4;C<=*o ze#8H!?k+(VNqUo(4vM-iYU+k<$6lA2+p?_NXOl58;}o~1ekSUd3P0$I%j zN7nuzN*KWm5(}L?Qu`Kmv}h2*v&Sgi?4bBSo0Evwm>Lxd)-d}?d<@6)kpg6X*LkjN zr=tL6h+CpoBFkldacTCcrJ$tBQ$KmQ1_^&JQXyJ02W!T;kWjJnHjg%{KNEJ?OW1Kn zEX4;`1VerPFXY-H`ZD?W^m_++&G6YHmdgm93wRcejhJEBgHoP^rsYjoH!c`!ss)#b zFj`+>rV$ptJy*8@$t^A@j}_>CdYIr-3ug?zm^1#T%fhNf6jM_pI26vUf7%Ogx4Yfvd)Ugk2s~1BWAE>O3y(g}Iura%q`9EVkI@h| z(6?qB=&)(=kkD7qxze;@?`>)in8EBFAl-44Pi!)o`>NVzuU5%vt3A#u37&rn-$)ST zTBl!Be$KU=TZMIA2u;!`4pLwY&K1>zS)1=u-gVP>71l%%gy^vGN8majJ~G94jL{s}a?_Wq1ApCHDYZh!v1yd#T1a<-v>Z ztyx0bl{REUHyhk-YB*ONdgWk_9j(^PG0XeAW63?|8iL zKQ%Opd(~-9e`?#8%qn)^g|_!Bkvcq$Xp^Cw4L+{N+DI8lNyOuHM$Al172{AI`|)Uo zf;tU7)zJ5DMO}&9h{HcL&EwtCjj!<48^^9R1D2a%#-dTHwMWyW_F57eX>A|^{P*AV zL04_|+`tow$_fa#aK?0TzHnVC4F@lYek3@=9NJ>N4tUL{i5>%y**s;|TzWD&ZcEDD zbIC{wOjK0Fz$u6S7Kisdnv^w^+pXJiTsW@uuv_)Ll$`TH#U_-7#f##G`Rl{)IkCt9 zWpb6eG5OM*J&F>M+LH-vx#Bf}x<70_h$7G-6=TxH-`NzcN9(++hxGUD-i0!KMwBPR$Bi+lJ%G3pzzWEjP690j%7Ktq^znUdeCq^e z_FUz&Roh?Sb&vNiYN*I3FyaVUoR}J!ipv(al(%}3mi+KyW_6#+tv3@WMqBiAFsqE4 zthx}yP=(Y#3=Ok9?*+}->)~tKk}U0p?ea4B#>*%p?q#6pA3$&Z*L`;Fnm$N`LqJ6g z3=pFTPS6m7s-p};nrB|d5OkWEiz6v@rvGqk!|lx z=5Qks73~V2+b>2yM+zkqpu$`wDnJsj5{`Q#Pd?dH66`ITc5ym?!b9+H` zyh#iQMib#{@ZYiv{bJ2*Hnl+kcWg7f(hU+;LQh1-$S}W$A`R0R{EjIeyt*jeQI%!u z`jm{+*T!GvBIk5K09LF|Zv9)0Q*Ui#%IhY3!tcOODj`@c#=N=O=sS^wZq?6~t)i-* zYXUbt@-2^2Yos$?480SoY#H1s;61`VE^&a1wMR^c;!RMBwyK1pcKg=i#TG%0$4Y6j zb8$|?U7uYVYJl*768xauud}eM(@K-=+It2$7iAF#%BfP*pKc-!0AcqPz67A%^Ck1@ zhr}qzM9#vEwcD1PFiKR~KRQXbY84!MxDtAym(Ll+j;L|TrRv0wnz}ss4#`LQB%d+n zaOAuO#dO_La1p$dMvD4vf*m})u=fdT%-Vm!WV z;IE8b^@lM?JVcs;j5E7`H&TBq^kvP%+$~jJN2T_6YvG=+Sw^Qv*k$O($ASEU>l-k`-2kmzN}!|T>r2-M|Ojd z7f)9aOpHFx12BloDWNa<{vkXC>@4?f{}AF=<<9j9%54+uFw{Tm+yjDU_a>1QhTp>N z#{f_gHLzX|^U`cKxBxe}{ZjjNj^5;tXpL`6s+Ikb?hDD92a?5hOyr&4%|d%oyeYSJ zw}Y5eP|>Hy_K4{xJiJ54_R(= zXh8jhM@qLcC9fI^cMGY$%q|$T?~1uu@BRO1h*7Ko{T}Yo+niLsgM@_x=)chZqsa7= zvciOZ-$x4H1Il6Aw>!aa%>7kouK$Xd*jWFs>gj*?Q8O`du(AJlh1F^v_NI(liblUp zjhz`)U5uRn<6Xke(dIvg|M*J8%qSva=dQy<#0106%1y+^%&EsHZ|Ll3>8`{4?@vrb z|2gV0s+c<2xi}h|I{mB0R&um6RyB3jVf9e=<^VaiyR^q3kU|_oV!n#uc5Z~QZmv%1|eFV0wICadpv? zmAL%lU9hzs@cdBmbRqx1`rff_xqI^S1)03aLv>R&y=3f-lJ%9e@%wqk9e4dI36{_z zZ=I6rDQW&(S%CB)<-(6eziwOi?d9_Bk0qt=#eNAq zLF!C!>)*U#+TTBP2=^&LSM0KJXFg1&&AxkjMGC1g%^Gr5*AQr|@qha$ad=_59f?JzgELwOWYveDU8`_WHY*h-SO?B*FlV!Ms#}?2wYy zoKce=e{Ghy%$>|qpgyaQD(-b7Wf?Mb!j8aKr}5Rtf_6>yCn19jn^g6+@j$afNs)i( zkG(R%sIi`-yhoMpTbWM2W(S9phG3f-@!KEuU-PdT*Lz~a)vnmbSE{9lEtK~hvinKy zT_$wwS5OrZO?@Q3d2T$Aj6Fsru)|l0?RDQNHA$0kF zyR^BUKwJ?j1~4byj1X$nr!zM)??j39=bNXotI}g*3XUL6jK4o&=^l#XGNo=_OVy8n~=3PBIyvY};qKEyi{yCl1)VWMFMB>+vx_%EC!k{}; zO>B04IrkEa%DV`lU%A-g0*}j9EN}Ajlm6bHMhV2j{cS|MsAMWTSh>33p~Mg|EK9R( z+LT|^IK0+X`kfDXnO*g)LsxFoQpb^MQAK$!yD-!z~`7==IOrSDHtcu zn1H)t-AY_D%za!as-aG)ot{PD1%-fYTQ_X>ARKT#yHKyshB#^8@aQ3>0{FY`-70*^ z_eIPJttozm)GMfvc7E` z{LA|gj?>hDR!hlLn~_bDZ|IXbJ*t}xD{-&rZ{p_df})n7KjRTMPz4}&ZiuDQEbXi# zM`<3{{Ir9aOL?gN`CYeuOrdL(CV9U>WcESkUl9R5N$_i>CTuhD_N0tRw0T53-I}T=YWa}Q^V`(aWxdWkjoy;K|jP;L?;pxhO)*Lwy0=A z4C|IkyW7*Ga;=1CcpYSW2)_C2{9EqA()#U=9RR&9q(w36zFicxorVY|9yXMT-bs93 zwv!w`_ONINqc%-s>PxU8d(nwsd+WHWnu1y*~XI8`L)IaDk1XA@%A((?EA4I;IXNeT+@{b5m^U;8z|gdzw09os4p9Wa^A{GD@hz)WI-i@XFM| zG}M8*9rN@LSl46e`ZyFYVrmeybOyasuM{nv0iH@-@zfoQr={ArUzX0*wxx3@v~>6b zJWFNj;1zZ%W$IuIFAsc~I%F@!L59p&{p8k$sWTwYagLw|EY;Xu>!-nP7R<)(`b6VW;IFLe3($*QvQgK*` zw$6BWJcYK-SO?y=Q?#Sj5h*`jw$6Yhb4;DF4q$1TEtEfQ zFm)~pO&zpn@^*xd?^&xDmJXKnsqa@*=iI5Kv?S`E>u8$?q0rYB48(rcIOfCVnU#^`ZS&0X&BhrM zs=Mtro_OrHxyP6hYFUdhV(qrgGqAc2ieR6N<`A@vo>4;qgweAp+r)XNV>iDkTRitu zzbV`383<^NuWj{=2tM2DSyQ3r-$oCDfv;6PjGmbt^|Vd1R!ax>wtCij)~8=RmY2Kj zq9TqoTS2mVW^-V}R?oCSY@=s$eH%SL56`_n|I8h9-V{Wc zlMnDwDM0HyDYr2>pZa5^(Ya-tKYTy-_BXHS=Wsa0W%t`_p#89h_*SnbLZ0nwE}?JL z_QF!Z^Gy-J$r7fyHeR#6Fu&$f(cYHSm~(C9l5QP6mo7+`gF2VCQGg9UfM0kbERuQf z&Njk&C{QM|MPL$xuB+XSuAoUXmjbpa)gePm(SG`4`-8uby%Y%DB|ecOhPTcl^hl1C zW?F4myK6@r%Cb-zUy&48T)Hi5P|4@SOrC}UL`!IvLYX}|Tq6VItW=mNvm-i|uO8tM zR(Q$eg_0`ZC;s_oeAZr(5gwOG=&lKV~!X%$(Ts;== z!smV}c^Ah`<~CUCmg@01xUs_DSjPm~)9%9NZ5*i`%!h?+cL5ShHdVEm{F#g;O;HB9 zpPZfIw%MEx|KidH_Ah?w2k4CSJX{j4?(*ILyD9KczgD|_dU#(AZEjqy4Slyoc|O{* zg{y!TsmA9=hhlyho04A^rNTn3!@xMj_;Tw`P9@X=6J*b4e~U*@*SIiI6qzx zk1x4jCd7F`d6l6&L%@dqIwSOM-E-LcdCyXmvkLryvrO@Zvyd9YS!j*%CoA|w`h?37 z%rRx{4e}y=6aZl=c&@`m&YV0ejBwJQ#wv$+-}-IVk!1(2gYAfb)8oQ6;} zd#qE`AYf3d?eNRhfDQ6S_jLO|>y^*vTZHUT=#?+i5CsKW($?*J^k*Vm`p`6tWS^G^1S%XBr*a_bxveOK|B{C?P z5$c*ET3uHiwZE>`MkoE2+J(W^NAGY56a2Ax`ogm}7K1WZnoll00?S7@m*0@3wGHk?l+bOcuOrLz` z$Vdt8%sn|nQ(Dtb?8RFZ!}lL|hh{^xOSbx*C!_RL|9+vgEe3x%Xv9{%^JJI&I%xD$ zqABHx0oJdR4N2)4lCX#rUlg7$Y*(`br;eEXwEHzx!C@ z;jr4Tk0K^SR-5%T$6+C*=S}%@Q|z$dSH6m~A*+EO8Ti=~nodcDgz&-aIE6BGx!a*JI_gqX2~uSfrH*PX6P!R!hA>_{M6ICy!hEv@;)#;EQ6VsMoC$<8#juVrT%cfvG5Vo=_2J<$x;=wae%yzZ` zlEj6UEeMQHu6Fvo!4(KE3ZG6JPgEb|(E}A&3N@qAKX-0?5!$Di0hH}pZIv-jyxQ5> zD#wWrWm`C@*wp(?+2_%I>Nh3g#C*4kerSAhuwo?R#H&q)oL~-;^I9g z9_r=12F3-i(W96lcXpgC^SE&YqWOBCiDTw^#EE~(0c9hnY`(A8Q#6Ks5)%I0ggBkg zGruW*XcJT}egAG0P+t@TdcHsPo8qT1wVN$tnQ#!c+($$N<|Bi50BZ)`1>M;=bcI4CT` z6b6S@`;~a9`1x=q%A*&h$|2l6%Kf;_g6jp3%-tBlqqsq>mbR&#CB{)!rsszxMdgNL zhpnq!EqUrZK30FV>d#TcJg(-EToL4=tLfn8>t8P}u9-C+Xc1cRc*0|89zC0nk8jNH zaU|uQ7=$ZtcC3-xVr`I?x`Pw6rvLo&Kh%q~Mt^JdDyTQoV{*dF8@@+^+qU%L z|MvgebN>fk7?+Z2+;XWfKoEXi!-Qr2z&L(&E9w72r|S+9GHy(6_uS-FBNv5ehzWoG zLAr`{raccP;gHUPy6&tpXa0-Hf{ycsS)L^!tlsE29KUe&f;rV*6+qqX^&}$%7lq-- z^8+2G=XnE)1C+FhF(CNRzs?wu`oI&O#M^bTD{@*rh0Tt6q%j14_LV0 z+mxjO3hN=TG;ikW&&;K&cOzDEQ?Shke=u_hDE4Z40n}};S$mN~;0vH`5))*Z+@%My zr?8}nCjjP{q0c+2*p(nFgFVt-rABe^H6Ug)`35*?lL@m71>Q6F>NgpD^EYmV5HhzU z*9dp)9_FLx)J-!}76*~7M`XQMMw*bN8oE1kQEH4Yo`MXen=o*&pv&s}GTAjB-DKL7 zpTj|)VsU}EVtnIOj+WBFcaKN;5_S3YjOLk}WLG@>@1oR*5zCa+_F~YP%NhHRfomvB z?@-3<5p3}ePAuv6+A;!*5GMF&f8KSYx}n9Ym`j^->bdEkq-m`Y6}lg zlvWdU0eS;oxU}WakX{{pZ4SsWhA3@^Yk-^s>sR{2I6Efoa?Sn}@(Rwc+DoRmO3aY< zhiNc8fJrA&)oF6=sqhl;w(0 z(wzW@-=!I(1ZNN#KUW9!vE? z;uIUhJCg+N)u4;EKNe!IBzrunj_kaGOaWwVBG;Y^)4EUi2ODJdXu8QoX02&9-_pgH zdHE9D;blKe3lfHcrp|T`k>xVf;O?W#D{aytK^uP`Q%}1UTPE6-D}-kyi_DesxA|@y zecCWA4x@Nu!U$YRaddcl7(R=h+v|Ism>@5=fEf>7v5q+^hP8nnEz$eD+G15)cvNNH zSXR6iHLFUFI)z-&{MeMN(QJ#q!xJrP&X-(%wWqVZ5&ln)J=`d2v|lmk#TyFJ;E=Gh zZdnuI&rc1ClOHVBS|OJKtqKb3u}%5&518Jo`z!@SL6n5z4{VEdC~wA#QJv8Jz#0;^ zIPcM-jjaor7-P5bL|0AL?$LB18Eps{Y!X~hYTBr#qqj8W3{j05TC3q3g zZ^6o?F?Rdkf3Uc$=kDa`Klq)RT?WdLD7z7s|}#E$XX&e(SIu`^eXA`J{#9%^favFr3i zRoxqeKA}g`Wh(53b!PUYJ8nGbc9pc8T|~wqei2f7pMns)6J# z{F3F__470!V0%iEUq+N*drI-G5wD)GJXt@B8PYBWg(rk5e=bUA_+U|RTSO;3<0W_t z&mpSz#&Og%wya_tRkh!s;HG+pa!A{Tz&!m6R+$~P@OvoWMYrX%R3La6>p+!>BTt6T zn&}I%>7mb9MS0pIEBJg-I=P8okRU$1u>VZqNzf#imIu0l5XJP0yU#ft!93YYYKvV& z(TE;d1!wpH3<|zR89rCM`Wn+@BIy$Af$eg&nb!RGm*9gtukEuzoAf_XXuGUmc{B>f z3aBF1Te7h~@YZ5>c^LFzI*WS3Ix3dZwA)x33wCrc3IKJFb_x$*s=eg3e7O6N#4g-@ z)`u^gpKG%=1sH$Mi`yRIh&OLZTXTi$U2m;p;*HD`8{Xx8W zmi5+-z^`ec{j?-$1fWK%VP!MbiYE~I_Lf)^?=5>@V_0})lKV`fnF8NrC(#UFil9y_ z2FpJqh0NQ)gsi79rPfbWt$F_s&H0@b`EL>88}6tZv6*@IZS8ShXJ zpbbeKg{-^XXqV6-ycGvZmJ%uC_n?)|pbt;cvtw1+4+4Y^Nk9tS zc(}W24V!4jhAp~V(4eGOqw*Mkna8`@9rC5_rVwC88o){i=Xl)~Ojaol19)$E7Ku{b zo={ty67L>tiwG&hS@z$XVX5fX6pD&%I2NN>nac>%Hi55{-JbdDCtDE9>0>Z4K;`-% z1<;4jj828sQQ}r?@?hbXI@3zS;K5s$#@iN7St~t?k5sE~@bEeMem(I{VYft&H_ktd zVJ+Poyh6B$TIw}g4CcOkJ4Ux!X!vor_d9}ugPvz&cUs|Z6nhreQ$Hz)Bk-Zu$OOL# z=b62xf1$*?&r`oBxGvYeQ-HSeg|_UtR}KZ*!@;AWS&!pCKYD`Kz-blZiRoYfgw(94 zA*F+Vmz{@j%3>Yt#n%$c7hpGqRo^?A>mgD`M^T7m%P_9>&13a1;UQWTo0G!D`=fLc zs7!{TX!uGzVFVtG2&iGW1vux^Uj8#`BEs#OgbVY4>_`FDs#PlqfW9DwDiNY1MF_KS6(2zY4x7N?NF7Qz zQuq}+N_3odK)DIwNRjd!M~aAoaq9Ce&fdZmhLUks$3{gcH3!PG^ey1ys{`Rv`%yO(NrH60}% z-j>|}T#Wu#f??53WstRTB^X3e5SC9Ex|FebtW}P*M&e-*msdGYLDWr!Eg9+sNZJqz zC)yCq*5$6beX-q?o9%K7L1{MM4`t|XwkdcoLF^#EX^_jw`>KUJO0NxC9+;XEc!QT; zuo&&}&g4mm5`FR6rqi*QkRbibd{&~tw0AtZcqC< zDs~`O`;4oHp=%o85VBD!@cZtb!t+Z67DALSPB)-Eu)N;UG=gz#!d}Q!1UC8?N*KTA zslQNI#0QY4$L9?q^=rLxe<68Czf(X!k6m#>Hy}X#_|(i?3t2HbM2SRG?tmqh-@l+9 z<#OgbV4RwhyR%9u12AVZWQ?yPSd%rKr;UJ(So~e|0`BIS`|w@T*dl2Y$MHp>-8ta+ zMV1UG94$Qdusa9PE@z~T0Y_7W)MvXB?FfcHdZ7Avmf1O=Bbx30#Q~ts$Q}!`69_ol z`&>F{;9w}*8wG;SNtzi6Cco#;_40KE$W!cBbOFJPu=9Ba1DJ)ks%>@-hBQ4xnw_8& zS>rZ44<*dbjTB_F6K|2>MbJxV%^JLOz!4 zPH@v!X6*cGaAn(_K#tM1l2E3$IPA02N%MX(xa9-#-%$O?fZn0F0OCCXB!VU@BD*FV8nW39vZfuWbrK zRNulF13~JVzEOsc=e4~^m@Q7<=(6k_+viavi3>_9U(rLjsj+JZ)aA!CId#PQXGHcm z^+Q2;9Zlt4UsV|*@+Do+<xd()mA(kg z5SbJ-!`r_VbJK}HMV*3eY`zU>ZL$*k7iyynz!*}!kU-5t&oH< zgtE(QKlUBsdstO~|Jdw-j%(i-{}Ovr3{#Tr<;6B$ql)trur@G!Ghoh><2*fc zm{JNnk2#C$xCR5?u?g4?j(zij9TtwjSlEExFx49Zd-Urp{L4JyPuBcP45@9vyP{>X zjPX!}BV!J+q6KZK996WSUbi)y(X`44XY=?s2!~Ug=vToC&~S1jWLeTeAfnMuk8bou zm$V3x$5NsyXF&Qsp@(0h`ZyKGk`^LWN#VPslV(P;kSAE+gQB_Fitng0dsVcVNBhRZb%~psbvqd$>y6l`X^vic%P9HXsB= znEVBoT(0NJmW%VQZ22h*uIc6#HOfpiB#kMrp%;1-Y`TY`CCl+Y4$ORXpXW zZB$tI{!NX;TkFSq16GfZ3jM;^Ss=dyct3U2JqVO>yF}BVHqq8J1Q=ur6NBd$Ca;j5 zcPRcO#Gvio3r(rE(A@~D(q2dK+B{hU=?O$P)eCGw9;0>9ZIVK=f zA{?6%sb=8??XuG@^q*l58M?L*FQ{B0!fRHcYk3kJV-0XU-s}2>HCW4TyaV2P1SWCmiqQC3C)`M9L&UgLd%|=dpwDX9}TL<5L&$cbHvvv#QX7R5kOu z+)~Tv+5|B|tBlco{a}Kyf+`c3$GW;G4 z)~Az-ub*K3fvqbRx<>&yp-i$OMBBVNlrW}PpHEBwHzgGEdFn5e61PEe1dM53acJ|H zS1hp)TiYl}nO<6$S6q-WuI-$4ZE5o(bNM><)kW#L^H9td+E*OZ47XIb2}<@bvXDIrYo%jH6XB@d4jQL^X$5lrm{(l8 zY$Z}=Fg|Kl1;ms-?bug`KVe@TO4wH$*UI*lgvt9<&`r96yzMJplkmnSlu>S+#60*zFRx zi#Fw@psOQgjv5bn?LxtG^0G2kTB0~c*gnLpR;R;Sk*&pqOi%T5XMbq4gpxj@_=l^nEA{!m%x7r@QZXWpRm86I{a6wSlK@RWI}N6TvH@v&aM< z`ik9ZX5ZC#cr6Ok?ypia+|35;=})`>ix*cjYB5-pFKATHk zc*UFOp{=7ACM~3%C~=&urRGmoiSOH`#|cm?fqBDUeS`h44JP$)Qjoa5Xg?N*7RY+T z@-QSHeSD&5mI`XOtARZZ<&_gcsCI4h*RP0ZuF?Krfm$sVOR?mB_=)=H4f->lJ0 z4We6H=_Xk*(389`D1$UYBg)7TkRNfouJZ!twxS$WwWq=>ic)tU{jdGAT)uu93s!M0 z?ivY1WQiO=$gw@VGiDU`(k6_J8vvH7~gY1A@tu!nIc$L%)%q&RnnVLxDInYk=C z{0bxXUjf|>mo$gYNYje#Zf?VBQ-~A7_Uk&k;=S~`3MiC$`#2m;Ta7NEB%camghSN= z0Wvmh^ze5mTfhuv8+O*Tvucv>YjC>uiTgQmlwXXIt%o9#lQsNLxJU0@zjjd`!4%bM zKW`YNseL@rfN|awXHr zEdIGDkw;tI!PFgdu#`mCQR#}>sEs%dY7+pt**=>7e5jN#Ad@6 zdAt%v^{N7?bsjOLGHtxQ0Cdd1-Do)7sK1?tTl zdp}o&8`mZ-#&J|V+YdGTDOmDG#Fi`y(Pkp?YCDtoQ~1fCM@ML+5=j|OM?M}#K39R( zFy|+(0F)Sagr~YOxh_AHf_fLQ9pR21b?hH#-Yrp1Aq-(`I0XWym%GZEQ7ZA;uq^=n zzRyb#adYZq-VvWeVZ;Z`xKxXo&Ty9^-~-AN$#twSOrG+!13qcCPeipeRTM#R*C^tX zCcjX+BR)w7#1-*LOQLp}IC_AYN8VTa{d~IzwKH~KIxruJ9orqQ+9kOz4p?6Q+9l3KlK+%M11h>?S619+5Vv-J{Wrh_7(BLY~#K; z73b)V5qZ~)xW}>gl`deP9@E0(h)3I$S$>6l zHiLG^XH#~_htEXC3*VAHky9pa>WHXBgnUw18V>njobx)mhLIHZ(KRA#t-mW)utPrJ z=jB*?^`D5$ex0Rv$ma)hXa?lsYqC2mzbVd{OZee4_e5iBUKjrmeB35nBh-yaIk)g@~4BuTvOx7GkfY(Uv3yERn z{b%Gr?}TX>x#3(OyiLH&g^#SIYyGqDCSU1@-zqdUs#EALsjxF zE&Eqzdr=+-pgVj(FD_P80Z#_7jv!@o4PFF(pabpzr$}lm_8ri;xlHRJ+HGv7a}(~g z2aA&ZMAQfSC!0k|uPDcD*O=NTZbW5Gx?Be0UZ%4*o8V+mkJq-^D+tdv_HjB4`uKdJ zSg8u$PbXj7z?3P1p?<&=)UwjX2L#~g-zmxGJg!U{R?w9k!?1(N&aZby3wKpV7oZHs zWkh1C<(t0?c*N^!tMDA73iyix&>Rvm8Y(}%t?epw)2czJQ2A=>$&obBTKIahsU(!_C_Z*9gD^0@3F%1RPMvo~5)X~xbX#O^@@sEGW#p^fwi!x=@+w>w zW#or&ma)&D3u{V8AWRGUM%3*mWtWlj)ze`y?%4OWs$v;AXOgU$B+29X*y_$&h;myI zquQ3;nrghCbnCJ)*Q+{t={V?ti76r`p(u*-y$I@_yu3waZD?3m9qMT9JM<$CYtKGfL_#ueZ7EFcHQvc!!BQZV%*I4gW*owEB zvf7p*g|2H-`V|zM1{i2|<}9lzc(|f(d(^~_cPTjj#b$HxycM_ig*5h;L%F3o2-n)g z8vmvosiSd0_bU<1^S_(I=a@{Sotl{br9DH_`56jtGTX~Q;M4U))or&XfKX~CDxjz6 ztd-h?OgyXzXtXB@9!01u)pkFOT>&a<=w4c#{HQrZtKuw28|ZCto$G1iiK36s?H)-Z zLc5~oFB{GH#F z+!R>ByHz78JRyutH-$1Q7MNS2Bas7w5nGFHI55axu88WlvQYa_TF3QPTkvE13G|r$ za3y{-XYx=W0J$lB%0Y+@cx}sBSw;#2 zJ<`kw6=Q}xdLYVQFfXXa*b?7gM*aNf>}!FzC^gA;T~Wc#f#8+(@^GzC@2F8VVMv&j6-;`XDr(#(5`Arg^;;jkB_bCfMQ05B7i;Q9U73#Y)p zcaNdH{?JdxvI!*W@ux3ngwPs!_))BD;l0q`Swj$v>@ToddjBK56$iHEc~dZuWP$)k zk!ehXk`fXWPE!JZ2{-Vu5lqN4+V;||9SF2!k?_lJg}F8iYV**C0~FQF-s}Jn zNd`DYTc9Vv?MmL5fR9O41=m&vK&#>fj;$_$MzyyxdOT6F`0xw3y!TtZ`T3y&sA-j+ zK81*9PXH<++j%#KN@ETsJ$j((>$FW7OTvayX!OAx0hsB^4Z;YW0A^f=MDG~*Wl~u9 z`jha>lJ8t4Axsb%IZ)u(|>5| zk_~xpEV$`CE%-5gY_92ZvuNsWfj*_xv;9xhyb%o}8+05LBKb`(Wv&vLVOXlQ=^w4c z>tfZ+0&&R5+**P1F)ytF0r3m;_u^7;lwT%dm+6jvwmA!VNugH&afHep9&nkTt6&OT zZzupAK~Y`NrKdDQ1N5l;`ic>_ zs}~Fx)~jB0t0Bk_kkWQL%|PAzuBg7LC4blTlGCcM7EsJusJup5SCMk6T8ZJC$;oF! z*LM|2NKzxZE1OvF2{F;pv3D#$@3bY0#O{&U!AQ*lHPi@_ckJUYD?=#&zzKDRI!RbJ zdA02(UlgHlQ43h_0M{zTt+J;o($%9zLPc2^qwY)VC?TZfd)k_|be}uu+mw8s@*?Pz z$KN`oLT9e|(QZMkHoJ`%{@@Ldnr5kch@SV~Mc@|$J0o4=i~*sP$L6A-&hWJi9}@t? zY5qKPDDc1_J1QQRqMmW}$f~lZO-XFDrUg;59 zSMq(_L>-SY>jHN2r&>S}mSqxToguHWg&s{?*HTj;@gd5}Ko5$1ClSXy2qE*;fapxw zM_3K(IWg^jc%2x%BKW@k?k}(Ks{)KZ>cfkN*@*$_)8_g4O&l_(fv=-ZEkGu*t$t@7UtcllX!B zN_-pzfIJ_kExbA1vviNPPwGjS#A0c{D+JbtaxxDBy1TvmbRdr6Fpg$^y(ie;0l~y%(~& zdyWw@y^WQxVc-A_?6i~3tV3aGA9>a6I>7>}*_S}m@J7Q+N0@G1gQABmFQ3?tlG)b= zMvXhj`oa)qX?M&pjp68|Zp1?1FluwMtdNM5j}6QuOjF%lO0o9^pp;tZWL?in{Ddu0 zSL1jTE?Aqf`&qEEm3(@H>iF#t;JJWUY4KKk2${%8bla3aH-**bU6q>rRVX3e{PPh? zcD;eQD_02ZeS*SXMfHbtu|W0$*;VcxK=-sNa_#`j7v~>$)f+csAJ+L3j_x{70Vby@ zZACqtrEv!q;`s+)J2G(X6I{Z8vRCTFhadUqz+Z|J6=nt^%CFC!f4PVPjWC|uZYF&) z9msG(TYH#4Bifv%%rEQ`-0;oYzQPE~372rpNq#~crmt^wo#PhQCG)&`_}(|p)79)_ zuYrKFdt9}CSUZBxi%4TWg5=%pRE=YHDExRie!-S0;`n%j4b5hAj-|YgZ~z#Rw`$wb zZtt*5xAh+~0?*ZhJsJ?NKW0TtvTR7)9dHn=+ilrz3MAzb1*$JX3XPz8G5{Wa zMH1*|C8RkETP`Vr;P4?(+X(k%*#v4^?YFV`!YnK8>X9#V@FhxHjtC7UWd_ds!yG+8 z*#t}a(7N2sR4plog6}a3#LdDCZ9N0zGP}Zvka}?n64-D0Msucj0w6hUbHE=FVGM(2 zgdww}8_;t?V!&zqg*oa9s4k=7fdHe;Q$M*6XNv!u0@Ecn2v)Y`yGOP?CwCIOdA@n9 zK=PBvBcJv_BxYuS55Kd4bq^&1rdw`)Q~oA5octwQjPwOOV+p^$WTFh}(mdhpIeL$^ z8m)_wqpC&53Z-ojVdG$dWZrsTVFlCKcUW@W?8OVwM&|AnPs=pVW>^ZfQ}BePoLDn- zwW~4S4uv>zXulDif5dg0{aw(BIM<4U^F@%7`s#tsBS22t!}EcKX-U-FBgH2<@u+~ z?iFFq`U`H5pQyVyvMDr5@m8UF zW|qJJ(omQ{Jb^LF?NQXv1&Zk1uqWls((S|h&!|*Rc|W|&OMX6S|2RO|+x2#oUp(`S z?zA^cHU-H#{Xp|hnR>%Et3pELZ9W=*e8s~1ASSRk3MTmEtHZF({V+_(1=@QtWVSXN z!DPc;V#euzG67pin8Ii zpTD>845^@B+$h=U9rn)=|D>z-5>LfK1kHtVZF#UpUfz^*k_^M1{HD%@B7K0mBgTEKnNgd$2GL+H) zgqvarI{ z(6Ei;lG&4Dzw$dXrc6%N)+N7LNYuV;FDqdD6sylHAjH~PKQgR{wlIt2eu_hBoc(>+ z(}a-uoq;@F0Q)5Gh~*!DLzkT_ET(ys^p$9QN?VX&^1}3;dprF2kT9}>ZFsrwFW**K0;ax zXKH^|)&RbPF5}66FJ_N!58wmhElh3-&@(?_;Q*iOnKqhb5nb)gtWDvEq|lb)cm&yq zH$0Irp+LYvOs>pWy9GnEn?PN4<`6a`CRvWjNai)$n4P@a7Nu~?azN=n)Z$o1r^a~4 z9m_0UKK{H3ziJ=JHHbrVfdwy~flVG!(;awFly-i)q~X=4P?>T}Ux(Gt`%M-YZv3el zgoWvkscwrE-{__e->2rLFr-pRN#2Ys5d#up35@~-!8)AIcpiTdz49V@gn0zMc)~if z)?1t44(OE+oH~f5JzhZtDn^=d{|;b*@yVx$#|+^jlO0klmPGr-z6-SuKS6qI!w^#>0yOfOB-|8evn=c0gNNt8Rb3>SOdN zs$1X_5GXV<#=Kqug3M@!kcce$v?IkcJ^{6bx_0M*qQh(4T_%wMdMTJ*$Sf6aivT5~ z6N}mTq_qi)xut`g-p##^A!5HNm!%4XIX}lLjQ>c9(5aT6_c5Ja!g8}}JhDBRZDFS7 zqLd{+l|KChvD9*SCCs=3_)d0LHckql@thA*4`7SDen$iTW8m6Bb$$sr(2**Bei+b$ zJ4uGm$@3emS*lHVeL*=JtQ)}XH|4C^`JLaCs<G_x90U-omuxJ8u>>R=L?E2+<~-Zad&vet3wyqM_yVkCgBXWn zUb4Y2Gn6A6a1+}R4SX`~skdkZUJ~eRUrtIAscqO2C7NM2 z0*sQ9#AQnkGzpv~`!3mI^f8!pTOSjEE{b{#jS2YOt3xqhg_4rMW&1iAN#d|gn-l_a zdv#&+352{spr*%gctdw{RDFM%wh#oh7|z2qz@86dqMHZo- zzq`VdSa!8tJ<`<8d?h4T+W>r#kz5pfhWjfrlB->G=|vgI(~gYfMH%8?@~!$TL^fpR z?~Eh|(ZaT`UkB7rmb*{^@A5q%36`rzF(M>0l3Ym1e-9w2;@f(136CTzBY8>o#wg5wch!~tXNu38Cm0mvF2=$vR(tskj)id^V3>J__O^W5v_AqJf6n{q=DpHAD*^vt zDU$}!rgmF)g~jXL_tZUwe}7F%+g#ssJ9cV%;0@nK#HQ@X>P2}qd4AL7D?894Sr2rSgXMg7!yf z1AD>Sv?Hp&Gc2PYq;d)%7e`hXp?!QR5iAwo+OUB_M5LCr1`$}@lxrnX!*=Nl4 zJHIIotS(B`y-h_{IkfH+BB6(x5lJGl%Gt#}BRX35$9ZE{2~h0bcfw7(dKijKk=50n zy1*KtRl;}$ONeb~*>Y(0v?H|oQ~D&4Ga?eilMp(UudN6&>G28*WKCRfOvO-CQRP?+ zMK#N%260pGyP_OA8g>t4?(Jp)Jo6dGzIsxOdt_N8@Z6HY#iEp>-R<*^yrs)~=pEkB zzbTvZ`>9_P)YIu~8P=G;Dd(wJ)4x#m+xJtyC<&41ZxqB81dyc@#FoZz@!PYl9ia~L z92B0GowV%kajnEsI;_K5tlX+MePXw;&Guy`^E`r3VUk+&7KGlUKav%G+40{^} zv#6Ttv9&qtCz8Rh@ieWR9>VttBGRR+k5T`k06F;`C#sglxAm4(3*V<38Yekne^IJX zSi8=MM30X7b!s!;CgyDcx_y0B8$D`xnycGBwvo4Rs@eB$}WrGyB-evt@R#o&){B?JUjtni*8 z;FnT#b(0@mzuPT(RF+ z7wDQ%F!R_DTjOKXJ_%qeyhip9M&vh8q&9^H4Vl%pFYtr%P$rF#HS%|gA+q5)7}RV! zgo|C3_$6rV^K9I${Ml9gO+AJSA)j)$`)5BeTn**^)!uw6dSiE)i6sDdS2{chYZ#T0 zc?hqJU!nR^(nO1gn%FPqMDi4@8sjrE!w5G=O@RnhES@qu_!J}8p6gVag}C_z&Bzj| z`i{=-@yGxx-b}?%-4Z0qOp{4cjAvAgG*gUkR&2r*mt`%=BRGUi+}o7$v{Bg!QJAtE zs9Zw?vH@^2tHoElYiHQdI&I%;KLokw`~ipPh%J;u!wz#UPw%WQz{&DhVN!Kr5?>4n zns1n+)EM!uic}*H+o?s;rG)S;v5dNw`Wh@P`w2$Y$%xW-`(Gi^jePeQAEiZJqZ!QaMe(AD7AmVv=-~wzx!}Bc#BW z$4}UZ#Fqp#XW!i^vLUes)I>#?oLD7lmh#5BF6WJwYZ=DY%krel)EQcvjxV)C5BIA_ zlWUD%7)A2Z^a>dOKn!HPa_L44oHAsDicj^%Z6d7xe}yASOQlLT7+ceso6j4kcxu<*`?m-c7KnU8*~pfiS8pV+ex0^q=RUJsxL~(Ams8yReg#Z6@8W)eg4s;1LkD^9`h$W_6En5a( zRI>-$-H>vxve1J1%62!kD!5K%k?(?TyGl=pCRe@-ebJ{gb`M<7^Ohla9Bx0eX}F`( zt#}C+zuLw<*8Ks(CPpQSCtyk)+!cQWn@HR67v%&l_V4_K(u51cSKGOGiP+dbQQHAi z4P%dNo4&EJq4DyRj)eDjDo(Uq-=fmQQpfa!(cjD58)tgey5mCu z57>8zmI?u1vhf7V#qsP9@?A1OyUDDY#to@ctV%vC2q0P_)=+Hqm_MB3B z0!;J1(*N7kb@sF24iik^LG)f24ao$Y@64 z^+lQDM*M&(>dz~p=Kc&q*|t5%y-hrgYa@B0yGJZU{6Z|%c}FZ}j3UD!k_)L-z5;x8G*(nN;kgH+^zHfA6Oxk*lU)*Z%R|cV>xi z?=Fyw(e3u(OHdwyo^}HgtEyiwh?@H?Du=hlH z-klxLN)&$&@${gT8WokRo%b;&)2#3aEv~3s4(m`FNfni=ZKT1ds9cm_N2Q+D7dwkf zDp$J-M=?q&iTf9Mavqhy0i-dQ1YWq?HINrKW$Lg+%-TC*XiVXUD=C6WoQg`$`W$C_ zna-}{LSFk8>wNCflMlOIrHB#!)Sc=nSG(JNg>8OV9rctK#nqFfgl`y><`abVlzhqK z6x=&Bj0-7?zj{1S99o#ixqrin_S=CiPrYvBko$4Nkh4JLi5J$Oj*-KabC&)6ALl`L z^DxSUM*zxUTB@2-zaO#Mp#eE^>cfcxJR5OOU~kp(#MX=+Q=wL}SPODjQ<9ahkKB~B zh;JwdtU)Svd0kD(WzW9p=}kFZnbF8<$9STglFW~bYatA;YbD zBId##kISFTz%u3j?mTrVM<$V{E@j+_|5CcG*O1ZD?$f4}>b$G+)TKD|9(n3g>^V45 zUCOf!PpFfU9sb&h>Y!X}sp74_Q1%Vf^E3Y+8Op+qtF?!+(^!8_pOok-uAju07Zxi{pa1#4{r~N!{*QnD$3M>({;lz0 zHE{nf6W|#Yjn%i#gYxH4fM$hX*bn_C@M171ymH+BM!^@!__@UM?!Qo~-hDmwi}DgI z{eJ2%6wl#*{(}#h;J$t)swKqlqM#zPp?qZ>mbno-fo=ae5J?HX{(?e$&(P40S#xJ9T^bvOLetIU6=hmfJPtKYt8RhHGD^EX^l)sZuICx-(wrE zgiG4;ZsUo}TQsLfVks=!%Rqy&!KE3ti3)@$KFaU2K2V3K{ot%nwfn`8eNGyqCb21< zQR=n61!#AFrF0b<{5enkjq)fGfmfrXI|CwjJeLivElAtaNh{ZBCx zk8yu3`va2yT@=BBw-zh`%qH$AWQfFkb(%EdS@9JNsx$^BhK7V!0+i`ZyNxFbK>(=|wFV<nR}Mv=NHephw^`3*79e2QPJ=JS)ig@~ z)Y_+YJY-Sc%wYmPk8NQ7*1}tP(Z&;<@Cv8LI9r=bGYKcf9mq%&)>az=yY%Z~Mi=|O zGv{Bz89%)8N?H@e${hwh@HBk)zSKV*lF&)Jszd~TihHU(b) zwQ&KT-(nNu2bK$(zQ3Z=(%xghht{~yJ;u|XN;V#f5~TYFCaZ?_8z3YC&W$l}FMRDG z##V_*oXzA}Mu%7gKOd`fUa;5EL)h!=5&qx=o@fNMx_GlG_&Olp*|k}kkf4iYY`&_e z=T})NhuBVn+FfFPHp`h-Y$tpy4Bnmq_y;^)@T7Z)x zr{}ArHv0|y!I6%<>|>5&s6>z1JL5=L?44Jj9J>cEQ4Ie=)=E_ibqOKHH4Bac>^O^$ z54`>4g#dsb-m&KCAQVlY(R5RuM(o{0)uIn&17B3f>dR6XL8&5v#I4YYHi_Gbu``H> z+0(Um_#K0C_70&_U+*s9X+65?q61A3n8kwi5}&UV6LNak0L^@Oxr1xPyVLx9Xz^+h zs(SDa5A`38CPXa?6-gFW9zU^CtTbL>pjYnT!ED{to<`u_6n@~3n?lFLt3f`zt4J~I z$;o8k1IOfLth}vI5MJ)u+)B#o~6I)kRp`-%5=j>Sk z;l87t9t6Rk{!M`^4=X>RZ}N`J(4Kto5|( z8t>)Bam7JO(zzw~H1?(?Zm>3~d*pM{yuuld6LSu49!v5Rer)4CFun36jB$I_WNZP8 zFZ&OlKrF(>9z$(_Uy@nV=L{d1KBY&pNZ{rmo7g&=-NRolJt85>pI=(y5Aj?d|4w)s zgyn3vm3}CbMF`(0MIeVwfU0~^T-VhFXg zYO;mFdCV0zzNm4u7+nfS3JI}?l5drT4|hG6%0gLKwNdd3aunE_9Dxu->pUz0jOCUr zI;wZO$maVgdz43f=euIj<-`0(#5yfmcs0F;hVFvcL>fVdu2!R$SoUsIaa^IGSvPp& zxc5XLQXUc*>u%4-*w%-l^D=&+>z46;M}J&Kxko`rao`|&lu#Zx9%CM6>%V8NhtW5M zUuFlsU!j?#gae=taZR1S3Or(bP@J4jDexP&&&SKugw>Qe$iycQ{+e7ZXQBYO*h<^y z8PYD}(Icsxcyd0B(+Q{w!5v@ZcKXg3#?7eF7xrClDV04+t9=XrPAz?NbDj{qVaVYf+Z z8QAhKl!!jzhx>QJ6rV*P?BV116+qhne@8h@MPlU)^torhg7;|`R$^3+u6F|4iYlaPR2+5Ty=-a{rWUX^RoZ4~tyl@q<`P$cT9#{?!ko;G^HqUV zO7|%42i^dyJK7i1cVzG|Jz{(v#as+p9*Usdg2LBM+W~oSV7)nV=(%TtH&Ar|VDJ&_ z2HV}BhjE6|;l}EWTz!tbT+V$6Cv5=R9y3;}6N)8HqKiLCz>42FNEAZ|A;TtJALjfE zcFv1HA}vh`J&H&H`B)#?3F?FI;G}cp|3}QRKE}0GE(*RDWQtCkmv!gws2&Zz|F%|% z_v#(6B&G!@`SBSCImL%UXZU+-m0%V=j$sNP4Ib0jx0v>!*x_UgJx0ySB`v|?>6^jDyo_)pv0ePW+6<#6i{MFt_eVfws zM7ME2m~I2UB5+$Wzs9b23~NZ5b%647Ac7Gb0{JC1r;z>hkr^u3u6gD#Tyi4dB2A(F zsa-6kT<6IOA}|B+H*8O|gW=jH|NirhGWs{6n8eK>RNV~!Ewe4XJq_s(A(4;4&f4$R26T^fE6_xJ8l6~*NcUP6nCTW=4b z&_t|uW#52B`qi;&P1&izf;&%~1crtEj` zr~X12UB%QFg=P-hZAb`5Wn1ewvLZ+$j9Z6m#x`OU7e?EW&PTix;+;ZDfM|VB))U=F zq&TPQ!-oKH6vFCaZNyrah(Z)-wQeB>MUm9#k*}cGq;SO&6&Fh3xXm&!L{N}` znaxr8Wf0;W&Yw3(_fTbW93B4>HgzWgmJN?N+}Ux=(mxPe9QQ}xLm8}aw>t??)$Aw- zEo6!o;PkrVrNF~Pyt7zsEFN6#X%N*-nMM5Ek!7Zf2CjpI`O_>AI}Ea3O7>oPwNa}5 z*DwVz(b-O3doVHXlO33;D7P;TlEvbCOknHRk!;{0kGq|$K7YtQ*prl7w|V-XD2z=^ zUIN1~dYFpXhsfW*4&c;ej_q;aU{D2om~C53w_&$MJ9?n3DSn`9^C(&vG2yY=;EKN-ApFuQPXVSV{u|wIL zuw`P^-IQwk9|k}p#BBDl30;bd<;e*L)?ffeI9VG{VFm-~Z?8Af{g-!&}mfOw@J zn80nG1X-7Mi^VLDkD#Sn(Uw>0BkHvM<@Sq0up_joSq8&uc!d=zT~Z8cMly$ApU0rw zgCB_>@cx=lTAew7cPX|Hankl`k3!$8$Gjzn@(;yfxwV5us8~3^gV6loxU9QzAvpUt zb#`u!tJ(y-vKNyS!GsJ*&VbudFBzDuQNP+vHs(dqJRlLnu8uK?%WLN?`xWEztuP4t zF-a0aRZ6eo5twpU`3aK*l4O7gQd?CaJ_B}9wokQ4Sw#v49&ifdl=TR8dfXZj_(=8c zA&hAKLJ`M{eIeOB_^4AP7N9fVq8*F&leoj3e$C#YLD`v2P?X$0lKR;fegMIst37b>=hAYVPb2@v@`32f7PiEoe@G zz4LB&c~2LG8FX^2IshjSX?YQd4SxwUx;hRI3Fwc*F%#F_G2-eLy=f{xMdz|$#jA1+ z425tF{Py&D&0HJ4i73|$(>JvXd0k)sS(@>EB7^pXcJ3|qO(CzX9)=412i0`!-mgpD zud#gnI1cCo&DYVt?E|3zpxd^94}={SWeM13IPoGNc@2Pmw^~Q`i|+;g{E)gT)%SOh$m$UZkDG#VRVjt~4)K zS-CT+h|LEliZ5R6Wb)?cp#;aBlHL;Gl=YUGoz&h^<+S&aLT1H(2<5*7YJObrHc`3z z9|59eHG+c}#U?0}1z_UpgfTulb9cd^|Nfx7I4b_!FDkkg@Z;5ne3=unCBi7< zq|(P3aBuJ9KgY$|Mfr+aN8%zrr;47qiQz!#k{Z!7lNc+YN<{mmaZ=gPmU)-a6P>Zx z+j;}ba-X9FcbvyL!vnB@?heEj$}XZOeq@9zhaZT_EZ*?~5T? zKoGL*8v4~9TT8v>@8{IeANf!X{Z9!B4m1~1LKf}ZO-9C#SxYp=UY=nsdZg#J*XgkHd;5y#8o zRh-yEQWiN%qcyg+DFhg95sRgjpjn*ADHnoxI;Tg^drxa&@IgWCzPfQsr(I_+^y@NK zGVeQUco8|_s{l^6G$AQ%3aissyZEUjOtL3Q+M~RE*;8wh-Z4curTyN4C^PxKMaS#- zp?i$_acz!%VGxz-eg)05JqEyq<}<|IbanG47+)-&R_*%!`CZ+<2cW}vk8kf1&*0ie zp5y7kCGhjHMCS#25j}*x$bo?Noeg~Av8{U;V+u>t*DI%^W2{_X?NGBD*xO=@*WNgo=)JWjuNY^S&ugXP@|PvpYpL@2Yw=M&|C1NmI0c_``pUa} zJ}~RD%jb#bHf%KrzU$Fp8s+muB#bL!-3W2l*#-o;c3(Xq>Rj!24_0txm(M3$7Tiaf zFvQX;!BYABYV#{0+2!-A-F!ScaJ4Arh|1?zo1ntk<@0POThBwHidrhi)nE2V<}u6X zueR&44~6CP{K;y$(kq%1@GhU{G~RkPSVs8zY`<8kvq8C*^X@@KoLxRoX2e+<_-y&( zBirD6jv#Rg=Ju|hKm3pQ5n)!}`JGZeSM@w4qWd)EJOyMQl?;}M8fTY5FUP4s=0tP9 z_bhM;>$e_{*fRx3n#LzC)d(905-_Yc-XV#2PWAk0U)%~x6Ch-g(ctQNzN322R)Zz7 zJY*kL803}P-qSdH78A|m?(yzZM(Cg86N$y?y)Z4wQ?QlTUXdSvwb=f%0sabcwzA9T z$?)!w3dL5ixjD+`Nr=Q4MWQMgrtJiz`~)>^=icA~gum(LTraX4Bm+*n2#^DdtUg@W@%5ZZkQ19M%5rmuO7 zgGrlrB$;5m@4UniK$7*Fptkbu_#yP<*uhW22UfqsPt^H|BI5TIBH<~WCtzWX2(J3~ zdtv_wc=X4@nz9|-{nTG5Q992z&tsTlVAa|{kiCQ<%uzZoNY$wH)?(z#p1@+w{!LxA z`FU=I)_MG8m(CMac;2P+gw^aW6DGQCNR>+GuXYI_>F;tUSwtB2#BU&r!@1q<%%oxM zteDSg$L98Gz)Jf%L8C<=+(3ktR=fEGsTyjuBLr|!d{p`u972d!>=?rygQF08jE++5 zS?t33J=2TL7lnoMi$a9*h?uNjVTDmdR92g%l_%!1DBmQSeH?kiH_grFXb$!~m=#fA-y6&b!D% zQvhvGL-93EQh&}uY#h!rDmtfSx@+gTki(U-kR3i$m(Q) zS#1jKY__w@7YPNJo1C;Lod-n6nVfT5Vn!Vld9$k z8p;xbsG1kvr3IZ}Jvm;%s(CKO_&P=NSSr8r5soP!CbbO+V~SFmqJ8iR=qY#0JvYq7 zI-}oxXYRj3)SN~TV9oqRp^1iOKQ;4gXYIA#Vpa+Sux6edQ#|E0`0DGnYvxHw4sL3t z`fdwf^-S_#t+S3Wno7j$tYb&`$np2x?Th0l2DmxCqj_0?)HfyN&sA2>_f89LHiNUD z=vx%Ki$tEHslK)jde%~OpPllsi7o3u3}HL-qgbAD$!yoSmhuLJ-hNJ?!=5CJz3&V( z9`N`$(EtCXVtyHV(*=m(Smj9j*s%f%UThLB`=}?hDaMTQi5f+N++Eo@Vz}kIfe}(E0A)9Bw+e(ODjy7v z7vUTtZtZp$>YF0B3NLv2ajQ6_uxnCGqMXwx;}|~i!o5z%PRXoXzI&9jjR|4c!ih2> z+z2~Sdv*+;mj=o-vyWf3*}uK0tS_H@qr5g3!SA;NTaonmb;W|B(KiO_o9T#>MxuR(36^F++zY440`ths4 zod}jvZi8FlS3wBczLre>KK1xjo8Q~7+A^{Isvk;1Kxx_D^!BR&&z8P91c$=UsghE> zEBUQKV#Lk1%~_(zRVx=k!s#qfLc$1w<%-DwX7PKtlhAte?(x|B6-MS9^sU~&2Ly`Z zjH}`VLF^yeI3rEC69VKI@~RyIVo16Hv!?~2@kQk-h9O+C76 zUpIWD<3wlL&b>cX$cxD#^J*l<;dpj3^kUtiyw^*4m78&?pCip?M*h}gS}PC1-X%{n zisp&=w~OYxF#kpX#H6nyAtP_N#`a0y7ub^an{xYBr4IYw)Or62OKp71?tzj_;j&lX zbDi`&h~O3#UBfVa&52Ka4yfSPwU1>(z_s1{SFnChyE{DAjjTo>Bwt(uc?_Z_vk0oK z1P+}wvT*2{;|3dIGbm@DA-tCUM!D9^PyME_LR7o6L>Q0aNDRE`OY5a-HpQN}^cP_Y z9oXwd3vv~!aJ^2YiWl@67+Da_VvhKT9Dc(&(ztc%dwg~Nz}WKA&4X`}An6cL?kyc} z`X(AEolKW#;1XK7M58-DL^TR3|A%SIpfz{rE0#VzJZ`X(Hm$Knx=*Pc`ehuMA!$9F+6z@pLR;`YMo zW$s5!_eJ6P!}f3)ES2YY6d#S-3#3fPZF#hfNZtxY9}tNx4r-~s;e3N`FRXU;w&v}H z)o!0Cm&$%e_wmWw3#&ao7f#A*&t2DM6+V-Xa`%`eGOK7r3dFY;7R5rX+Y75bJgKB* z&^JRa>!Ci@&D(m|l&~K7E0Go;pl82By%cn_x;=X51G3aw~VKtwf{w3-!MD$x8G{pM!DDeZ``y|~L7r3+Hh}24l zTI9`zvsdHh!p~`G*B4&yybRb1clp~fx+yy`a41R)pq>40L|V{}+w)2c)cksdlo&v} z+fT%nLOV2)5(71N&xd~|X#36$)O=3u;TP0T;OHPZP;+rOT(CNjXfWzK+P|U1IR_x~RypcceB*thGH;Ru2x3OdQ+0Oe z9BAxobDRUMq&Q{TjC`rowpsJ#%f#-F`-%PS^*BufX*Ee8apyp*B~W6fnV>8=2hP}? z10S?UdbBH-AYMYP&roEr$Ce(zp&hK&g#txMKY$kQ>V^kuxT}o=Hgl7_cw}pv77TyF znoxP>jA6NTDPP_mvo%G*Fi`p#S6R0cc}$GYHw5uO)_LSxuTAZRJLLo~xQbOEbb0 zGTvAsV8uuu^7RnNH$9WYryKPZB7GooR@pe880lv`rR%u6tIe*phyhGV2J)A+nW#8> zjC5{8=OTP;H-rVno7?i_BWXjruUsf?h$DuRr4848>4s?v-IJH5O_#ICA|@|Ixp-{S zj?|82w@`zVUTATUI)1)?dkU2K2x1(Lr~{#^Dtu){X4e9CYfcupu>ZRtb{BqGHE|{{WW-qQ>Lo?IA>XzfS{!4v{M*F~ zd?IxG8CN9|bNmkp1Hhmju&WM|0^iS`3=|*8%4YQ`SjKogGmIgZz$kY?5T<*6prDY# z`9v*@c|CK$a=&7VD06(Rf#c_eSf!qbP0$a-HpM4mBQ=S$^PJ=dChU#uwb=tV0RgwM zucF+ZRd2UUin5idQZ^!OZJJD-w>O`^$1?3Qxt3L6hTJAZv>t!ApnFJ|d>g`1y?z~q zI?MTdmW5oii=i*b0a=fG_98mK;pah~K+l(g{_(-g*{cAmQAVHF^f|Wq+YxEk6H}$5 zGR)}MpDbdoPP{DP$B?zN!x=ZhCk}9CEO;C@V}%0yA0VuWL&(nq$*MqN7OH0}EoK_J zV1@nK#`r9ytY;#Ko6!rF%F$^g!0?%QpQ_uyvAr;~7JtI3yXXl^?J%)ua*W4|(o+i@ zMV7~)GqG%8+xqch_n$q3^X@qa8`A z)(RRcVcK|YG@0zEwQuc_u^F&u@)NdI%+#rh%h+9NvK|;_&+|DE7gw7Vb~$>F*A#yW}+xYRVPefBk8~?2Ze)o$3a#Ll2n*}P*Kb_kazN%C?6nckO|)3yHG#Mq7N@R+b7 zU`?#IWk-!ApsdRdPa+sF731sds?YA{nwagB6v03mvr>GA%0 zEc2+y$LSkW8WX#m6Fm?b3*v$G9gN6=^vl){WPhP1dghEd+QwA5$0-q9gI6uCC?_H41j zY_d#+#E{)|ry!-PD5{syHo(^8Pud3C3>@7hGjAyClcmauW7tHvrEPh>D4;Mh<&38y zOAci7&o1sDT`E`RSJ>te5xMCJw&}Jh!CX?SsADjkrm8>#7D0$L=L7jPi5{(y#JAYm z&VC5Gr1+|hEJv!xuIF2-2WBXU_!F{$;J^~qE?0;+3Xg7uzYj+j2bFzTpRocRsw-&$ zDLC9egv|PeR9a37gZzx-mPEVBX@Eb}NM~V}Tx=$i&5A4?;Dw;55v)-JwC=ylTU5~+ z`r68@=d)^d*#sK*UiuV{fDe~c=m1P+c;+R7WdAz*{USv{`FTajFjO5lqaLgl6W$sZ zodMLg(eiR+My7e*GUI_zX2ckAc9a>P@nqPL%Zz{wX;z?z zrpSBBjL(=x>KrTWD%wMtk-3S|yhdd0)y;A2JIwB)75sSoF#%?DtKp)S%P zjcXt%HAF_tGs}F@l?_|(o3)FXpS!nbeN<1h#fUjdjnBCIM$o;Ptc{}XB=GN+$r>Rv zo}mkk7o-cFG`m4&#P9oQW=xoogR3`YRkIo9o6U9Q#~nuaXi%kHPmChg*)gTj7~kPC zBX+up!6#tH=^8TQHLjlJ_onQwE2O!4l}siA}^kr5l|O7rAF#1B$rypC>jp7UN3It$nZU|($mH{5?WgoWs9 zV-d49!1adxW*eC?Q`T|;RUq2H^*+qPWY>XNqE|-u2Utzan9cqMg_SK}|F#w#izNxo zKK^{xIzOPidFx(ezF-~zSi2Q-CfW63CU^TnR$n@e*`NhwBnA1oAQht4YUDN+O%d@4*|5M!xzx> z*0vLBpxIyXMrDH{GZarEpt0Dv*=((e!$IF-|Bg3;i2&GY43dHJ1DV8$$}8G$3``qZ z9c<4;$ztCJ`>0h;w^TOZXrf)GArUqXGjAt;l-Vi`EgrFRsFG=8^zLyonw27%ZFT93 z5O#7|`~vFL8g6o1F}|@*$c^=m;2t*nfuZC;EDgXeBSGx5sCoitOTomj6O7H_LQvVo zAl7KdEBZA{+8-DL#Y0SB!0K6eW*!9C-Zh>f5R;8j^`wm|?4X9=Genz)gB#eNKCy>J zR|;Qim94+6x`45YJLue6HK69D2>#D9?Up@(qs3?bQ|o|!9g_oTh;Y6bdRDsl(ogwrIKtLc{mtbaVhy`{tlskpC4`39dvYA8`h1fD0 z(4}t_wQjO_7JdVepBpkAQbec&65v2OjK($%Rc$lIoH-53*3Ruwx^HpZH`|G{Gv?Ny z15`ZGXvo4ZX#4#aSd~9be}c8fld4fRlnq7ukl;v_?OU>ld2}{>DLU{26>265MG3`F zpp5ebPkd-17!gx~&)6IhMK9c}>cyB4dUZ1|mji4msJUy9S=cqBZH}=SmF1#stP_&y zRWe8sC)xUi2mMAUBhS~lo!XvmI)qUUY0n}EHc+T3BtN!lbYc+eK)0Eb-6-&#;Z&=) z03dLhe4vr@SXValuL6OsGBeGlK#{7Uyo${bE()+N@`Y5+n5*U)^(U%OYz=%Thz91! z*1USW73Q|^2Af6ZC4gGSHs0gw6FcZG^6=WehM24&tXjQEF><>}aIcAN8+$T`3pTqh zP1K|x2pSbS<$UhGkq&2{wa`-)oPL@WOw7H5H-YxI$04F2gT@5tzK*n z)`qI8!>^5@{GQDvqgg_Ev8|8PAq4VLXxEoSr?N+k{DWp-BD`=S$=50~x*lauMOGHMiuKM>3n1qZC@P(A9%8qHzhO@pmX>&k zHMTGlnC}E$XzE)5>Co`2_O^!4hpC@2D@fQI0fE^bzfhz_Jgb|{)eMEc+Z=imE3ke% zowa`bnzvTq`<4TS`my>0rfxLd>L#{ZY9@aVW*Tljn!cE#a?v!*{MWdXZGI5FA*M-B zpm_QpArc)Er=VLugL-HGCZ}nCqG(SCf_3f8z_D(h zKsJDFVy3NwJx|(VLk&o~CK3k-2$y zI;$-;T>ThmRg<@0a|O+EQ1|hWq}4CDAl8J&hn;P$imP$h8!I=FrBm4hQ!S?u7Mj_D zrb$PmPIVxD_AK}wm+gT}UkUsQ)*Gp~6p-2Bg-U3vjMPSlmn9})>VkP*`#yWKT<{Jy z!yY+j^At0KA9#4S$=zg?Dj{f_iN3<%e9zXP*myp7(;1^A2j36`*=#oMqw)VUkBl4E8pnOqdzygw2TCFVv%cn5Z;m9dX2ZYLURa8qT`9> zs+=8Y6H}HHHy*I5uqx5!|1GZ#~L>)v@T4&oF`DJd+HY^(cSb7iaWbWlOFJ;Oe?>J^E!|lWCt95 zIiM2pjey{cvsq8x`vvK&hue4|*;i;q8wet|dC({kn(<~*1{C26I+J=xhPS4Kf?$bA zsh!so^)O9Syv}x3eVp_!YRlY9 z1!$R19+zNMbD`sLf&Pfu7g&?NE5%H1cw@}&P* zMx8Yqz*WzDLK~n5NWU z5UWf^#jxAH{GioIpZ6mp?lFg$8^ZNhzGQ)ktK83gtdsTV-3sUybQCq8*OUX`_HzSB z*@OMu%AWIbp#j~-l(J+$M=|*&S?$5!PU;RO3;H$-`qDQcyBUO&rv=^0qSGu$3;Is# z;~f_CZ6?wt#)0+^`*0p(8!(XGhLweshx>_oDZ*?RL8G6W(5*AU9xexObSjYS2OIh| zmc9CdY=48g9Ifcq%78@V);RmBE(TnYXk*cIQCLF~vz~2P)fXFI47NLvNRO4eP5fc( z8^x@3Sv7A6fa3nLq8mOCyeBJqVBC%9H(=c)K@++SiBFo)ZLBMpCUm0+M3PH4vN1_~ zs4o7HwZ#Pmn-VV;n$S@>ze`dO$>ij6+0deLcbhpVBp+4E$b%ZLOBNuId2H&oZB%(_ zL0@RtD&b{0!Bn4qfD2z~*3Ejrz?ruUi) zl3G96f}vCFsoV=xBnbPt?OyX=DB>eiPgoEu3f6 zpHNET!s#`TN$?5&YOI>N|HgJb6FX-g|6{# zmX+ax$fmF@=Az~oq`K?`D}yE4G?z-BqrVZ2ASne_hDf!MmBA8e@;X)pTkD?lMxXVd z-1~ZBh^N!cuxpqTK*Mh3$ahq5skGONsGWncKGiQyW{#Bd-qF@PHU>a#Qp3x>>+?=#cJD#ayQ7;K2N z2$>jcinDd&E8CT*vix3|m-2OwL50T?}SAjJ*3urAm)I42kvjJk$f zS1B_*Bu@@S>jLO29fPYalkC7{0NV#1L_J}?wysOduB39gqN=9^lIhU8VDuzo4Xg{- zSk{FDp>+Y+B2B`&fGoSNpa zt+>f{N$ra{S0UR~&aqN=|qy z(KZuwHP6O$qiBsOrWzw93nR|v1)`3I z`?hR7kV2&T6RMOKuP0WbyFQSb95I>#i~O7^`k0Hr8xC2F0rp=QXt;8|5tR^P7K3F7 zVwOb4S%;%b%k)3{9{B6NOXmZ=k3NOic}?++2_^W{@xo0&CYw|hlzgIhPsu7-cgIkU z1arD@f|W(lk74;*md+<~Y%Y}%8sockZLN&l?@g#IVM; zS7=N`hs!dIsQD%*_1Fv-ukP^UMB6?=j$3IL$MB2r@%}zQrvBUy;GDQ+_!%=>%`L+Z zW$o{JB4eKMA`Q?we2wD^4smNNl2l%?G8aTV^LpzS#I3{6Hm;~DAE$_<3-idjhu&14Lv!(e9W(LSST_@Y z&@&q|AV~A;$S7zrm2V;CKw9x8e^!Q3#!)0)Y-X^*_KB8H#82JP_z1cR4~m^nvk_iI zz7~(~IR%R)SXA|Hp!aBj+9(0V`-&xiLq}arGtC(6q4k_9tKL3RD zOv~1Rw9VUTvADm{BGUa<6XK?`!^O;5BR;Tz<+O!DX;>MOLoTT$R~BsAcFxG;Y}$-HPh{0E&0H_>AU8U6;MTkR=cl zHN@JI?=u@_D9JJe=mR+$nd|4)I^-ZJe&;uY0yI9&AZl<%b_|m*epN0e-vaalK@myG z&)AN7)OBoMIuX)BLB52ttdv*@(CmSvFa~w^@RSS+(1)-nL7%Zi=ugTcHO}-&jp#o9 z+A@c@I|-0{(lO9R=W-`KWAt$hcM(L1SX66jxNOizQ*3~$Pk{|s^7$)%` zQ17S|^xQUtNewZfscueJN(L#!S&(<$LS26PlRzRdoZN@~=1Ki%I((vs{s0Q-EKV2> zCD2S*6$WXhyIZW0W;{Ge?ZDdlcE?#!A1sH=fb4Gpx(Lx-IhX~?W|Z(Ju`ie>R8*3B zImSP%{L;o9pPhrS11$Q2!|_q9?Y%J6EG`~9zB~VSj9ZF>E5}UGa0(UUslW+8} zeNB=M7+22T!Tp6*ha3SYAU2Q9GAAJbnM1XG-2%|w-w|b-`9xA|Ggh{j5l~e;Kc854&NdWSp%Rg_rQ#Z8q zg#_URLbMi8zzHEgH$?f_{>sNNp2sjKJ6@0N4@hh_U-=l`bDZUSy+kx?N%YE?J&-sr z2Qt*QXp(qjva>PRd-uBM2BDCj*br(cvwR@onvIZjV|y>AA(BaKUV*ibC#U%CwXmGvIB%ZXp8vH{8; z49HYEY2^dHGfU$o?W+3oOX=DEi_6uqevc>pum^Gw*48Hq9db|@5P0M;4irv~J*KB} zW+7Q5_JeCnJSVfURCBc>ngX?{{UwLDvBy}VVN)C5lFHOitBIjI3R`=lu(3zfg~BG; zKAuS5o=S_pQ&r0h##gHIUNXK?$G0b@7~?CJb~SIt7m%}^MY7;;$T@29JHH{YVgSO* zLh$8cE;?6ZaKp?e@7c7&*PCQ*2RSsCW5w6lYvwS%()<(tzzS86Tg-aA_6coJSMdKB zY$w&TCe_)KPn}bBO7KP#r0Y}+p>3eKVP8jq)eEj|FLtov#d#(<>OJWfH}EI6h>c}n ztWj5&60Kt~i@pl#I-HLEQO-Cx_?s271x%W^h5o+_A{R?dY-s5&n3`(XE4{Z==3?Z= zv1U@91d2Y38#R`X>R5KV{`%a@^YS5CvoE(W7I2W|E7SwA1A065D1JY7Q8PIE&KZ8-;ooUpbz{aM^sd-fae>SG zVm-ZHE+CtxC$%=U?V6W^C)O3rX6KibWi8I?=J8tCO~cLO z*w4%OT`#Y97e<^y{fYhU+50$$)mQ1}@mfom@qu(!5p(nSArm)`2jpy~liP2|&CX{I z4+BCDF9)zK_+K4{4&PtQVKlg z5TE3-HrxD+D(g(*ff-r8Yj26@IswgOSfvgrS{d~vR;QRlY)jP>9WNXsEc#=0_l_6u z(1ut;g&ZDaQ*{8I@YfdY@|NrogB`vc*Ni#5#)>&S5XKy0?1AV>zu>B^x`;KmXwc$d zlX8ausUwGquD)x%Nb?!wRoyF~OfnKP2i-x3+!&>BqYYSJcS<*QdfzGDK+e2`KD3MT z%*ZKp>ziBD;o&Q+5$WCG6nBW;Z%bpH&VVS+kR9XR099L}jTj{8QT$_)toI#xh>3cn zA+D_xuF0nUCnq8QWRojp`@%OdMr@s4C}JV& z&TGW(my_5C>?DbX5CF!eH_YP>!PCddY`l2>W}DRat$_GLb4@?Yty<113vSzQ%&)qh zhqf`){~cxPAG}w~y)0gp+XB@ps#(zIoj{^>vgO%x3;l2ZbMO7%_CYqhscHgeFHWRh z)mD|}WwWZt5USQ9ts$=7nh8D_uhw{T(>b4@^j35aTt2P;nHMoHOCL8W!gh(hH8!jE z)l^hH;eSVXA#!#sx)E)|Ey@xF6D)W)G0q}G{;()QHURqH{^#D~b{QLdrj^)Bi7y(< zF?TU~_}F$0#;qdUsA^LYOlfOn@1Ml3V-|}7!yI<&f8ssPinRt&>vVL-B3v_U+vd}< zQ1(BbvJfwo_)Kigq-SDY|KDGvJ~~-q zwe*hq-}nTIb$2uZs#bIZtwHQs$KU0Rcpzx=RILn${bX-Ep*^!&bVoxsoM8wz55)3M z9|$F{Zk87f(NUs7Gx4##P@zvb+i+!NJ_C@c=+jl(qXu)O*`lK8IN^Ono!4U2=y#v0Bv?v@`FUf9K8ok2vf038UEh zb>Xmmm#UE&Y5`fmV=@;^&|qEBNvr{J&BpJi0^@APN>Ge{%F#Ys7kX*4Ma zkX31^w)(V~PDRJGV)bi5gmqBU0>myrgA{uJg@mgW=Q6wgU<)FQ(OnskBnaA?72H9b zR=4pYVp#v&-e3tR%hbP%$W0PMiSvwZlB04vhBsQ7pb}uN1 zEWTQOc^n!g*lx^8($<8^CgY^J2{>3f9B2jYXvW$2cj3J^@zn$AAXQW$E~NEQI0l34 zQ|AYji#y3ZlTZpr=y%7UIgq}{1hNVMXm?jNz@D(>0LEk*K@^IeVS!?fO*$UP>54Jl z==?xDflx{QCic0I?R3>V29fwI&>uC6BX@8zLliz?YFTkqt{jk`3oZWBJO&LfP<(Rx~So9h>Q^f;|huoZYg=5^0ebX!kc2k`hejV=9X~#ZewKV+uE(oYr3h$B=)cAHUXsoZ)4tL6@$AV8O^7i2Qc&$l)_iNYaCQm}Pct0-sme?)$jUQZB12GFgA|;S9{K8N_ zWmCt+%#9Av`;24FbHGsrhzT5{BbxoNZeFK1bEocnJ%PUToP=$E7ryFXFwX<=Z$LdN zqitSF4Gr0^aeTuejBzo_gKzc^fTtFmcYZ_6^#*_m8|^Q^c}F|?3vfQ{@BD_e!SMVG zLg@zMMG`de)y9;p6;cJr7Iw`SHC%WxtG~v5h(;X^TSRKXJ;R)wA}HL1W+V~C>I;V= zO$!BP56yxSAgmr5W_aWE1X?HX7f4%OpP<5E)K~QYWQlQ(iQ(Berlu&C?)1BzqrV9H z2Q+RK3djybVHys>JkGqQFq0TG-<&lj+q(hQG>tt1YwOwbN@*`i%#4-!kC0SUGd@mxm+ag|+ zGGdsWQ@&^&I7w6pD2)Cm$lz7JGy}it=OF3OP_Wp~09eY-!tg|wMkrCusq+YPPSO6v zT=5GKiRX|xW3n3;HZn^&L*L?wR!$Ug_t!b0T*tp(E5lK@+R#gE5`}#)2%B?R4d;2W zq{-w2Y7#){wEiW!uq=jtvej6SoE1y2<)%q1xSq@PM51wmUA+q0D>zTW@wRf^O$Wnt zRjY>&p|T?Gvsk(Bp`%5mD1uq>8tz|EB;m^j8GzDy;odmXO+b2$yDV}CqIDQie1%2Beo#ik z?H|~#Qw>gG=pb*-5Ti`rBq#{Npk36Jl)R-J<;6v}$?<5~Ci1piNKf-{KkV{L6Pj#w zc0DjDvkZD}fNFh@QjV2rpZcO*Dbr#fCTBP^k2l!;D2#PDkd7RyA40gz2a+iYDfh#^ zf_ZheX0&;936EK1J-9Q{fQ_Km#qeyYx!x?%Hly56SQ~Olf%jJ0C-_2|nx@kaf;M;& z?Rz|=Dji4{u4FOGJp5C%B+^;9NP7`;*tEEq)t9Ji8YazOhQ!WL<;fn7w0mUk06)&I zfI#kadom9twR1*z7_On6beECrphFnwR8kEqu(UMD$-wp2Rh@;~rh`8~zrpJ}tfwEh z0Tdp4R00NtLFjy!!eeYkL!8o9#o=7xF-|qb&cR)pEF;eSo9w*jeyB^rm!7CS-T=F= zy6ZyWK`s4D;jytxYklA2^9?~lsjwCRZT|@g-6Y=n4RH;}CbipNfb)*_^cUcK+28pM z36<=AN(N+G3R0XFx;?=b1VA%z=y@3|YvZn8poR@Rb2CL_SJBzaH7T$S%m% z6|9$5YBC}EBOi<8!t1_`U+6NPui}m(TF+M*;Yre@%rN#9#&bi(cA?4U=*V{C`2o1`{6J_tw{e&2 zXcUf%oUIgGj2>3~a^txTi(E$o=r25LZaue2E%D->ImtAS_^_%ph%lfBfyjUk0=aZ9 zBai&jxj-Q6exij5e`BkQgd8`Z8^Kh~%5-6)m?_O~5?5K`BRLOB81I1mT#zuJqeQm5 zFgG!rb=Er;lF8P?fWC@65j3FNIy3@}6=^LYHQLsc`-z%$iP3(J-T%7G=Y~JO5h!cc5XzIrPj1T?A5Q!biTs$ z4b%BE{0h6d25vOh5EgSk_cEB<&&sYxd-*EIG&l<gwM+~^w~L}dAbzERdt1`~S6CRo8?Ry@%uhSR>_TN&ns#rWnhFBm~h z2{||iF?RF9fpnruS6b%CBNj+HU0zw47Z&SpxOu@Q+civ@&T>I;6|S$lV2B+{-!eWl zN9z)xkWVbx8d4^f;Zh9ag5h~9Yy+1#szrot!ElyzF-okQvH{VyaQHlJ3m-QDZU?K4 zn>`W+y2sAP^viqxK&*WMAuWu(hH9$LfwBLt z1u)yVidUDyr4-q?Sy{x!V)q5ZDuYmBJ9Ms{_rt^&TG+&m!=UcIE*!eK3^FB0w59F5 zE_4+Ce`)+yJigvfx&Vh&hlY|R%SVPEni@s zTyR3G4ZLJUHwnRN2ZuUHj639PWOdK>FUY}>zx4+Myao`DlVTCe?NK&N%Ce%wMaq>H zt9xQV#ld#8V`G2DBP{B?5N-R*<)Pd?Q|ukswr5;*ZnMOI-CJN}0K?rcJQKNMSS3ro z>5imm6);|;xaub<2E>gr1}Spp;vB7_J*H7+FL}ViD6^NJ0$61fy&qN?3#Okhvy7D; z-g+Y830>4O%UE^b?bhJMH@YNJnq~GpL7CvKlls$PzB73M4gNl4gOypv!rO;LahhfJ zXvgbj2(lOm1#W;z#@H<&2h#P1dzxsL*+XuY(DLl7az)|FF(pNoceBdu;j!dyma&|L zEO$ZGGTlsLW*OU-DJzU7HJ6S#?J}sFEW?dS(SdKb%N&5)We$XPnY}JoPT}}vmN5#M ziek(pm@GFA2-l)2{OI#}d6Q(8F#_3=lCUq&*p2Em%LIYQEMwK8uQ)~!OnS4U*<>K~ zC!{4gLz_7yy&e=(oFi6}%<#ZuA}dMirdehWk0^DsOhCdcV{c8_K*}bHUDWPah(dM3 zEVF0fEk(18t)pIo^qxl)*X~c*i{gp>trxPgU8+Y;c1M>09q6QTC$o%|%`S32(k^3Z zMO6{>6;I5FGN^^)Df}i8golyCFym}Uc9|HuS?0~AZVC*m%z^BnQWs)ql>yh$;o>$> ze+Z+@Mi<7fFv=+VwT$=?y~evT4iv6m^ov& z%^V2KGr<64pRq4giA^-n*c@bDnOK92i!K9Qp^IAbEcc&RI9kBkr`yBaIs;bjaRXxh zO=6Piq92$s`Mai4-nRdm!ssoG9*9dk7lb~yHiq4+`EIgoQM_WJOt1jL1#^VS~_i97b;ZzWKel9SNd@JrH8 zwXCixPow6&<$9z+*S4acU@GiXIq?L(_3eH_yp+;TwKOi7Bp~cms~+y?um^*tDD7Nh zi94TE_XA3-a_0;Pw?>Ip2x151=Yn`r(|PMRPhf49*=^HQTP$b=(H5m?&_pP%v|Npak;#aw8MoXmP!n zDuFV~o||CQyQ-}3+mX16ZcNgPm5A;~X?Vs`q0bST@=N&>bLxapWK1UZ`5EF1?ju@Y z25BmP0wvv^A$@UuIF-yS0|+) z1Cs1A1EkS}IvDGcL>`m%*f4oXWwrQXUC6@iWQMVu1r8ZV4htOq34dZJd8BHCCoN1~ zJmG=JU0|*W9EPMx(K>Hqm$Q&l_&|ynH>3g_4y~et*hi?XMK4S_EAjZ2^ItQx<62*9 za260xcv4Np_tsLJTK2=RlTxZSXRw`z$NvZa-XCnxlBv7ovtmx33&O)_^{T5}MHaDS z6|UV)!7a2J*j*8xW-iHjNZ?@*L(b(6Dz!J#x`Rlcv8#dlg(Ly!d?Te3Wp0Svi^g!x z@{Co*=RjnxjbAXXHLq_l>loi?t`>ZqDGCd*h`>`UGoH{k;C0`n^XtBk{(awPpWho3 zcth5k3pZ}hfijU<{D_Pd8^6S05WcNBx*yy~BxheD*~eLxGnkzAOTNl1-gSNpQOD4^%(`M^*7#hOjcb>m@@Rhz#I0vtE=Nz^m1Fo?!v6RshKm$0&+ZDCK?`&n7zyOZ1qpfJXZVNcq{^b^M4A-%M zKVx?=$O8UA+#?MOIJfiIx1Vu~;Kb?(x&J)lb2+@3u(#c-#VhD8wz` z2XeMjg!T(?^YP_Mteo{11kU=3n!)p$`jYj=o%Ii-hN6R$$V!8UkmgX-L6F*Pks-$? zO60@YwpdYa&xh0Mm}>9d`f9bp>&XVF*@Y_i)*p!b<^gHt4H1dcTVJi7xT9fVUist? zl6lYSBoB5y#e98p=9GIs&_1`!;J$tNPEmZuMZd89jitCLy6c~@%++K%*I4fQ3sO0& zCFPjzdi_rWK>Cu$a&|}ZJlOyT>GaoMWBKbZh}JQN3>8qN zFy4wN=cL9wIo5-d%HgmF$w)h&#TRkh^&p{k(g8r~A@DXo3?9#~jl2W~Z2Vh(m|F*3 z4=i`0z*|pjJC_xGcrFsW^*F*)2>=r`IW?!^SEobVJrBrb@c?8*Ko{=1rI=Pa>@kmY zCj?-o$6rq(yCjA=<;Dq_LWyhffD%%uv&2q@&Vu85vH>toa#FQ@kL9kvAYG`6YykXz zxv~L3ut~&5?w}YW`0FuS#&64CPsXQke=u!Na1=AQ4MN}gfit`;PHLFj5KUi?8yX9?Th zW{V3zvW$D(9_XP<+Pq7433rGEmdKmvdZH)|$O;`lL)mt_ib>XzwyGAOI@>c|@`8ZX zKjRrYoj50lm_G<=CaS^rKOwzJ472+1#q%EVyIvlpf#!=l#(S64itlJCMy)opq{U9n z)E%khP*{sx84dgEWsNAs*y^1D!6XmJl!9Eb!Onno44TfMh67}8FM1)Iu1`^kMbHu^ z;{qB|N^LcCXz*^b^`;%;f7+pWKcV=`L0h4F7Jt4?3^n^pQNv5iJ>o?>`eZPu+ zRjaLv6o`uAoTex* z&UkuJl7OIjh5bOs5EP70!a}%{l88G?WW$2gwTe|`V8WyrTdTHv|9+v(_HNS@Y|}tY z@By$#hmG+gd&d}eUP_7~w9Ooa5L6@Ey#HWAN zCEo~ST@-47@GX>z!u<;uW}XEVl47p3nLbe{CA+I&u;xVO zooKEGl`CQ#saaF2s?30lW>pOtidtzZ7zibj@C6qBx6(Lpsm#JUHl=e>9RFjH850lB z6SxZP7f}R4)p=~ZqH0vDoYUy6DTDJeJJa?nZxdUT>i1N%f^f*s4MC?kRSuG!*$W`i zSD?xyHOW$qBlk}gvuv}6%2nAqI7F42()9zUCQ#xx{fW%vTKmZfY_-^ksuDkN(AYvZ z(3@n+X7S<}TQgGO*C8r9WPaLCDqZ{n97Vga^#urGPqD8m{K{@i{5El~rFdzpqE+|x z#DgDrp!Y->qFQxwygHz5HO=2_7k$WDMYUqwy~@X%@PAnOYlsC--cwc1ip6HmHcnb; zE~KKBc^*|WuB@2HDlB(|-$V(KL#P=3`yYq2_4mBvb(DosX)p10?PVWi60Fk3ijjs4 zEYuI{P&E9gAgI~Gg4c)SdXobyh_cI{ z;!m`tHy8>z7~jEzLIP;i!xDu-j7?q!mG*la-*CvTUZ$&Mam(M3sEZbFeM0PdZ>($7 zlLUZI$n}=*=pT^ydc5@uvXTPm12VTtMkVND#!FFC&rFJ$#exii1yEn&OpV<7wVzLwnw2fxOcdhHs2F25jdD0KtS?8PzzfWoh$q@6Ll~a5 zKY1sP?Zr(RDv}z&^uno%w>HJCx&Y2EvR=p%-zqBwB#Bn2UV|}1QwAFlpf~~MG+7*K zvf&2!CTi`(xYFctLE!knZ&pdtoFwzyp+7}+0+&%HmwiN%vri$SOxC%-o{-Odv1mzW z${5heDUGDNA3k89a*%Al4yK=DRrR<-P>%-{2v~jUhUi}1?3jgEF3~xDwNGj-)|!4f zIaCQC8PlYTOY(;;)745*1CxXN@6GkfiXvlXgQOsqzqN~vs9h*=V&gCBm?Z{*BMuWw zSTximRyY!Wxab<=%$1X8M1ztRJ$=`6$HdIz7P6(4c0bW|QKFI^c0xp$wALxYH8lem z^aoI$sc$J6XzYL&=wV8X7zTzZIyaJOgpqXV)RV3103i*q)(oPq^4CFeXcdPAxK{DO z6v!ypxSpHKYA+6%EUvLjd>9pDTo;{1`9st7*TFJ9tBJ9Q4agsOF^VY#*q@k{tVusG zQ%G?7!qAcMIgF_d<`jL%HQ)t@u&-16fF0s?ePTB$@-c@kGV(e4A@x<9W9;jUC$tUz z3%*h38@`o3i`aY3@{NmsEI5+e3p)XUjRjxj$+rKe^NQ*eUhg~$8+}9O@-^~E#*+NV zP8}5eU3nb-wmI>$^V}6XxKt8^wl6<0OYzO;NGqu^a@>4}M< zEjES1^x0IU2UV6autdVn4Vv>FtL@r>gygZz(6|1CtSosYAV_tT>$z}ZJ9wBHaXwEy#ec*^fZja?SD zLyZnu-OFeMWJVVBkb$x;j=LKhm8sk%0ExVkB-ELj47a4+dtHBREn`zK89})~(5KB$T%zWiGQQ zLX)jyC!df*&Q?}b|NIMbuoS=Z8-ki$fav7y;|_re`$%vs2_X4mPp79w&$0a~PP1j! zjC1waZk3meX$Yw0eqzXeG09O(+V?q-AGk=eA*2LtZKw?~TvJxCH|>!eR}Uj3jY9@W zG*do5v6?JJG!Cs*!uq3QrgTNO53QA?n2W0|)&RQr-kjezW27Pw=AkvKEDUZQS_&g~ z^N@|RBrlWzit+h`AYpu2$_4xj5@?UNJ|V{#wB}G8>nG%T%lGsTNPIos`UMH|P(WlJ zTFOmx^Uwk>x~xM01e3y0w?pHI_Gvc`Enehd9kQ`_B1$K4J1^tVvbIfMVH*N!F9xfK zq>BZwB9Yjis1&zk8nW@ywM)~G(VLroP1_Vuw3EmiyS`_^?_5ycOyAuwnyMmmtY_%epVe$4)8MhGaAO z-NW%=RRE^8XDw#=U%P23{g;;mFU;4KXDR$mzlgNXoWs>Z$A{(YSE)JX_^_%CQ@qarb$$YMPQ{zJ;4hK z4tFN9Sz^^4A8Vh>U)S4~{3XxF+HN|f=Of19`8W_2joMjuJa|6r6Cs%orwGdO^nBPP z%cX4?{@S?$unZQji6?TFQxKjH!%kS^hHrnDRT*iwLD)*Wjj?+^K1d(vk;N)JAArcE zUxXmPX>%bR@oIL@2dZ;u9w_gS!`WpMq0W(Y$H1_c3ga0 zelK}F&PjE*$H$U*=36-u4$Nr=X(J%E6;{}O5`vltEon1C z68C;X;!=vY{(!hW*uE{7a^+NsMVN!PMGs3r;RR%C4bRvgYkMBA znC*Oa=l*b9NM=Bxn(4`h6nr)xyaXSs15rH$LG=*oeHBl}f?ZefTrz3;6ABQK=wlR? zq_1w30Z5W^Df$>ex;TI$Kr2grtC82Q?G4%IgRGhUn&m!sd(?rz6!#2^gwfb4YfiLm6w9AxW@Ir8p3tMJ?TtTUdN!FnJx`9}FS&Q@|3*WNE>#lLWYZ>+eU zg`I#XA}diU-|34P?w7R?Q(R#!e1?)3*~8>4$*QBd5#_bSS;3FrM};5XepEQMW9&S0KSw13*8GVX zAs09+N{_q_i|?JE7T;SxE$`juY1zlz&Txwc2B%3(&T`!Ed^knL) z`$ha_EmWa_ER|%rt|@F+ot~y7>mBE7y3O=HZ)<#e$dh(^HCYh+35mT!$^MJ2a*$BY z)ccjc6?WGoxwwy9CW9&GL06tQJwPs<(+05rf>;XdW{nK}%x{Qo3c8Qe!8*B*Nr>p$ z*R7=_kCuqVx?Pev=tFeA;)2a*%*^uJl8LW-1O`1&W0Q0~t6 zQ^8qv1hP)#D}J)*Ld0n|^O@w(#1aOJ>0po|H>X+Gb<9_%9()+V zT9@M-I;1<;Tuu=iP)UOA)>j!+eB-BY^o<#2N9XdbEc@M&kE+T%g8bYNMdqKEegWbg z0>V)eKXT4UKwRU9mwo|`$s2jq>ulP87ev{LNCk#2sC?yCV3S=(WqB>l8gd{5 zhn_JzLESvx5gMcMR4Gebp-`TP}iF|4EjR@#+US>zTr4*+uv9WtC3 z7=$kS17X1UXi=A7Xqm{?E6mVEtrFEG*iv&V1NAG*sU_9%&2FmTSPABIgZdMq$U)O@ zW>rIsdGX{*DQ$q`TWYw^sVecCYPpw?K`OW)*nPJ8t6sFl#td?ro@{F&M=pcu3AYgU zYdDh*&4JW!Zd_FWIS+SsBhSprXJ>XMDu{57J$x8^p@tg@DXTSW7Y_R5-0>$0`&uTY zI~p?mVq^lH)tykF{THehFxrCgTQOdhc9RQ2xyr^%Sr15XDLF!oSce<>4;%?P@X#Qo1WwB zu`xS%%48cu6GVn`p_}9@oM3l8F0V0VdFr^F7Xzv-jlFx)7V_9-b6j0nr2%2xL6l2y z5#kqQO%-d(>Q0-~Ts03#gbTuZ9_(uj32Me?Jm0K+MSeYG9Zb|?FLSIz_I+exsm>B? zfXewU?rm3oy?iyB=?=D6*`?TVkC%jehghZ=j+PR`v)V~!)soC?NvZM_;0LLyiojIK z8IjsR4L=S4X`9yl0IeN1274Q<5N$u$czQde{LEtBR>}j_0upU>xerY#J4d%vixQBZ z8#0({$zQ1yoQweN8=S>tS%`rSBv#XiS<{B^Do7P}xl_ztv2{{3TY+BPA^yZHTKI$v ziRuTKRQJov0`I1_qJPD4Ql8)EffET`F2MgTx;)Zd&~6mj@wa|Mj(qo#DmxiGT9A)x z?E;^@FBN5Qe&Uh5^JW{k#40fHMC8f-FaFLuNwy_KFO|BA8m?3ZbvOihwy5^^{ADT} zy|Ns~Uy$<_WSZZAR^V2^?$3TO^O#tSmUk@L^T)Z5<6y__}Z(JkM>lH2J_Bc8^X|n@55>^+4d_ z!PtJqW)7-ZjH7E=dL1BU#Wu+1X9rh4D))b8?hWhOR*P}aZkfB=#x`HuP3#rJb}(i- zYF+piAfONS7PR5`0!?7u82&qiMZiUASU;&|(wl!l78+wagml$8-uMbw?K4JHsn&sc zTmOT`mD>CbLuqkr)_fbULwWcrX%suRiR0F_xA^Ax*4DyE1EQ$o>mVng_UI@25Hi{d zV?NIDY^8l1&jab|^w`*$44V(+lqym|5+=rZ2suDwdn9ZgUsOhY#!w50tj%%K@n(M< zJ8(r?YlpxNToE~@!2?&S@QWAn6b+*IASh;w02)b*8?O!|Mzgz~zq5ZTe!u8wZOxD| zcec^mhB0;4NWrV0p}DU4w8kZ#28Y>BU4vNNxRmh1(eM;1)Lb^*?412JR1?SQ^^fsz{`PDzF@H&MzdSv+C;I$ zNvWCxcpb(KuMd8IY!PsFDoOk^j+IeGAu7bA9W$l*ig+YuFku}4@>BY6G{y@$ zhsI@!PYq6BJCXE4Ko^y4%@B+(F%QQUNJeWAegcr63zE|X*<$FzY}Et2DGCpTbsXj_ zs{Wwa%Chp;*0^wAp?R#0s zy3^8?x8YD;4TJ-m=Z?HW_aEtEzPTYju&u)IxB@4zA ziF`1L$2aGHe7xyMb*`=vWwr*azUg1%w`2^0TzlEl=k5w(B(x_XE!fWoB!RpD83@S& ztFm2blceel(1N^0y6iAwYt16fiv28P;?9CPucyPK`qWSO1F!<%Cr#9ABMU)GB9Rdj z#aw$_lx4e9?_r{2`T$Uy&&hUUkZvt?AYwr zTfZUTP5|s0EP=S`iH#{QY%N@vkQ7uy=)Ej1)r^`CXkr(wstvjrIKbXwCdmU~^PHZU z*;*%Y-qV(drIKq8MEb^Js6$Vb*DLA)#lY`phuASSdC^}7`S0_A#!_GQhNjTim^h{g z+2zZZp#A$7i#!{80S33PB#PoC^A}*-!4GbXm#)sGkj$PZi7Zn>vv2bJG?wzoO&BU_ z_HL8JRhQYu#Ku{t-WY7pg&ZZxeDy%Gh!r`THXKdtZlCeBlL2YCj`0JH6v=&ZwPLI3 zB_({>gn1*xmfc{U1v9?H?`ev{Gyh@IY`xXMaFhX7Z; zkQD@IMA|2%drOM~kp=*)(1w&z2lRv92N`K$xjPJ+El z7h!R!XAF*k>&NcD-oS6L`{X;l4Q|aw(V1u5#32U1(T**|Y zNExCN3t8eb>Dlg1d;<1{gT)M|{0ExxLY9F7O?CdeAUO#uF6*j-R;PvtIfq#3?5AZP zwcW=$IVnuq&`3_g@Z{bmuQEx`!0ILN^(IQX^t4A@P-U(nG!l%?Sv)4F&vhJ z08pU<@{)-4K#IiV%z9;wjmseEaE@6Tm{zTxLB=}F@L|jBGA!6>?9H{jD4x~f=far* z)9z9;64JM^vqgfBs+rl|Z{4#8Rh05RQ1_3=m{ze$kH7fuZY)xzEov=p zkFfevloeFTq)>G{_6VzZ)cUbU47vcr9$_Vr+Pv%$!}So^BdoYr-eGUBVo`Hm_6Vzq zl;`<-?qb-qM_Bo(+;4lt$cPpWgM`(qYV}woV(cb~11RyrJ|+qKJ*Zou=!{&&87Z#X zoN_rP2~073ut;5O1$$7xCDyA<6gx7zNdon;@>iH79M#7paYMZJm_uq* z8Vi$zjk~J>L`!9=j+-QmHhA|Aet%2dr%A#fMOHQzzM7%tX4b)Ky6VYoK^Ja?Qw=vC zv$R?L?zfI$bW!^KK$03vEdy2LzK%)4DxXcwsH;w0));kW+zjLv2?LY^`MDrrkpN~T z!U(Zl51 zZ5=d7SdBSnL)u8I;4gh)43@SVQrCt`+c}@3MZzFcw1B#2;y!MXIAd8PKG@TX8M#Fw zAaV7B#9TZOuQ>(@+Zp-^h93&_*9(J$6NnomY_Bni>zr;G7io|%boI+3VI(ugZILhp z&wVTsHipeu7Kt-$0A|hSo5|>*eo29#POONP{j~wMRtn>%E;~e3G^Nc zi-hf5UYA7zY&;)}gzfaM$0G589ae_4PTYQ>uy)q)>w;4GU?>vC*gn>c!Tzp)vlRxT`Kb z!yk%qmoW;P)@7V$$6rH26*1ys==93j7}o&Hbz`g-4lUXpXlTvrqRl>Z{|EK$hQ>G_Hsj&vW3VdYU{ausRnjs@7m%M{#LVJwJBNi+1U@NK3-WVA z5N?Ap`2iGCE0TN%j8VsKg>7{(l7!Yol@ zZAqX3%b)wb57_Ll7WA)IsCC98LVj;4yl5&`YEp^;XWcJ%zW!qnbcl-T8 zs|G}3@{lp8j4!g~F*=XO&J>AbJXCvD0K4me&_UVGBqx;Y96JP`UYsP{>`^ume0d>& zzTRF!z>@pNpIGBtoC9;R06pH{wfNTe9hyl{w&^mu~!obyfkKP+QMqm&7x{|UgK z$eKdZGEp&iHJJDu>k^3YP+CZlK6Ft1vQ|iG-Q0dA@`VvegtP~fUnUZqP)>UbT)X`>c`hgK|bZK`JHP`Zx z0?OH+@m94e2Z*)UIO&eJqGP#B94o4OK4p!pEHsWORf%{el-QJMBldRC(?olzd#QhU zsYaR|>J;n_LYj?}XT;85rL>JKMjxWTrHH^2D;) zvSSo5tPBX(0g7nHoMGc7>4Sz!(AlL?&6L8!P6}}ia(hif;|}t6qTFAq^7JKE zK};>&a#Y4`Z<;iurrpek`C<3diy0b**8PCkOwKGjm%E$L`1JgiZv+JjE!@asIY40U zm)=1j4ylZZ2gF@u4mnOTyDW_A#FtPKGb_A$+j8t1qqr z8Khqt5Lzc*8jy{r#V5MiJ6g zAfv@`sX&v3&@L6|(66gN9~7+u!!068OJjlIxG!0imWqbzI}Q(5bVnx|{My9i3bmrC zG;nj%(B~O{S4gu8>t5c~3$RE#osL}hk6A@xT!!+Wp_3E7;TM;cdaaia&c52*Xe#OtT6{A__0-jz^`;7`uVfA?~d7Uyv^o zpO64uRhxI|JW$}o9Ceq~a#nF1s%WjHh_->>Yzi73s*zje3v#Eex|oGqce-zgUSHut zUg3kS<2QDM-jEfMzV;KM2=`1o%^csgQC5Vw) z8TT&>@^eE_ANlpxZ;0NqDmT@Q4emGEY_(C2sn#k=@e1~!@(r8%AAs{t%D3WvL5@e( zihg|nI%Pk(X!j$#k+v9RzB6Us6J@@snt71pcdkogDRZ1YNH09OQ=9|#SEHs(p>3A8bN5guf2+c^cn(vo7v8D);t1-XWcN{1hZVvZzR5 zlZr97IZ^iZ81tPyqK&~IwyG|7hFw~lVoW(P=C(6gx)kcw*~WRiZr9dIl+n&+_g@r7<1@u z_B6O0y0!>vI&kRjra%ulbZt+X?i9|cXX$hF=o%Wj!tzjoN|-(#-5a8+iUFy7A@9&y zK?uv!B8ZnR)mt82BMrtk3=XpEu-WOPNf^huvg*Hz+63Jc1+}}t1H#Y)HZ2gw3G>7r z%o7HROjf`OQqw3j;-pZh@xz+0lx1ge;U-#e` zzaV=%Dp}_-&zc~<_r?lumObu}+y7XX>lIPwfQT#?zyr%|*mS7^F*_?{E9$kWGN=cZ zTe-E>uPc*{6v_!KH%Kbi;az_1(pMK%lr!3 zQgrzldUW{*@v#75tvVDUNY=NyDYB;AW!YDB`2)#HFi~{*ftlj=nLJ||xTY$sSw-5* z=7PFhsf1MqmAe!Wa1V4X2Sn<1ye{?Ie?Ro|n1D3+m>?DWSV07r$11Nay&$ZHkse0I z9OH~1=}x(u#ge+xsoYZWd}fo%vRb$_Iz4IXc!r*tR=lcLJ6J<4uGS;0iRH?=)Os(D zU>Vs>7IFM!le_lxyn)yTWpb~ zJy=(&D88K4B;=Z1s!<`vuvEEm}HATJF?V4>5V)UUqqn&6F{{eAf zFd&i!F-|L_!J6Ba{mP3n8f31J1~ERuAcpyp1s}kY4VV>#Y)mPP?QV0Mip?KhL&jL? z4p|TdnA*`cQL(KY*${6O;l!WFC(A0cFpwuQ>W#YOl}IY=wUl-4B+2ISh! zniY@!2|2<<{LXKP%YyhejZ_p1tgI2`?afL`ErqAsi_%Glqb&H0mH9_W4?8w)H5I!) z^Yui}Azx0EmQh&{mjxe46JGFmkBEbM4}W|^$9LXFH^dq7n5YIraN z>*$Ovg3XS&0V|Ax@KEcg`MsQ33#cmmHS>j89_vGI+*~N z)RGCvWAL!m?p+6u6kRh0?;`W%hv7~5z>TNNF$RwS-6~ng*;*+MoC*R(5FX|oK7Oq1 z6=*^-Jz*w5SG#R+tU~6$ptG}GZ?C?ao;k(+KDXokx(@4B>BHBuaOjJt9FXBXOD$!d z09~-jQi>9V0+QRwWX9cI*mv%JhSfrEcDs8BHDssukS?(hCI@^=6)w&*dCFB3 zkkTaF$6Lj-X5*RnO1b(W?l!q}k4>@SZ)F@?wOwn)E!prjJ;M>4%W8L((PbTH53@2( zH3JsYETd{;Ass*CQ&Glh^OcbzC7KjbR3ZR@E%pT4cl&`ZN^*<)rU(w_dkuK>@b&z^IWzR0mO*C6=%hn$du3s`-ob0as!F zdeM$!cM7sdOD1+gc<^ZDCxD%b4ZCOOxV!XaM4!FtzTE&GpE!TD~rHRATOh9SG@VyWkFogkfyEQ$sEa zK)i&N)psoBxROWNxO&++$_1fAHXv0JwN+LuJ1LuvRlm#Hfe{vtE++fz`-u@bUdH#! z>beIyU6c>R?IVr6XO+k(@A!or8MZ95>yV2*&!LdwPsnvh{dayt>`o$}%5sB{I!rdn zFp4v!e1_*IIFI8!8QP6{SOs=#9oDY>yk-RqMD$wcI9Ll`D5}54ddP;tu1^wWN3|YA zz?%?oGt}bs{04(cE(kkE*HTsP3!cbloWA7?r5UW|6;5O0-c>;s;?6ba$kDZx|4Z1LXIH0l>8JwvYVDQhF!C|m5 z#q-46!H4_>t-#3~11h0-b|tM{LM-Wn!;wEQHJ0SUHw5S~@v8uG{*vSy9O0y$SQdP0_d7bDwcBMC;`Sv*S$WQok z52>o`c!rRwxa(_6RY4>nSbZa-*y3`_R)GoGsInM4RsL21{i zu5J4xgHGz7*8?v6XwjQF+6Rz)ofw`fILT3fWj zHzU7Lq7}$7)QnLz$}6Q)lQ>>)zA+rm9#2GJC@)NloTrejN@Sv1wmREUw12PN5i77a zcZV}y82rM{i#jQcZH%A}J9nM_JM|xi`&E&=~p!tItt{)|j zOzF1V*kgl&ymUcqpAOzV@ys7k(nmOjitQK2ZG@l7C!V=*z&oCqV87#;3%_;ZnF&XQ z+}v+CoL_+n_gEQGb&WiuB*xo-By@z;bERC;VCB{mLW>i_H-8N zcnPtT5dq4T>f?4Z#$N1z<`+i3%GlEzQyl_Ihi4E)_(Owij>j|zD_p8&M-|BB`i(A? zV1i2J*u%tg$%tlWE(r7Q#tiUuW5}hFlM@!#jxbYMmXrDvMl={3lbYhHm-0^mGl>8$ zy`}FdT6ToFFe%z}z$Di_B7AK!jE8r+!o#QSCW(%r5oyJ6K_TO1I!sP5U!X3$9~jd^ zs+7paa2|4Mk!v~Zt4v~AV-#JqNu;phyo2I5#@TprG_vmaF_iySGBd9DhKdJ$L{lX# z40~l=9%l??C=|*1oCNuXOKo79lzIvy7qKWj-enJoH+f;NI&K5aD0K}l_S{8qmWS)7#=0CS%D<6>7;YP zUYgbJC*+W*&N^Kw-cNhGhDU%uSN%WNWZ7v_Q>@~09$r-`Gr+uYkQ4xLDBZZsNXuK! z$|(|oHt#8_!E@l^eyHEL(@lbzO-XNlg_<;7&p1sfD@24GrwG<6D3KI!^};I65t#CI z=p#?Fs6zGP9977r~w^E12vjoVAS%^K6Ff4>$umDkvnMkCmk^AyPkm zecu$N$`qgceX-0uHlx>OZj1_oy)X*QA)G}LMIZy9m8AGrFJ#jY76&;p9BR>V1KfNt z6?P8{*=$jX)QJjEKH2<)ipcTo`xDOnENX~6b8tCvkC|P?VDp!mW!c|Pq`6N9xX4U$ zqT(BVBRPH&?GQ1IOA9`>mgt!gOO|96nSTy*pi8 z3I~5qO(vN!G!I1))ggXkL8!)>>tF$&aq}haLA{@_n^GOIKg9F=@d#7|a+b7r1xIF6 zmuvPNI9rJLMt@3{>RpWB&}5fg=`2|w2KEDprXa`SdYiLNi#if*XJkI4YT;sr)-IdF zY>w=)urZqw7LGJBtE;fZ6A?`e`4j6|U9Ug6JLLys1t2XgM-`_i>a2Fb(+MyeQ`k+D zzrdv^ibMr0qPEWX!PqK;%iN=Aaa9A9yOVUIJ;Ja}HdvTYwW_j;rV2b3{Vdl>>UdF^ z#@XDAA!q`@>F0Ly!lrURp-{WH4rf!G`MEHs3s1+DI=f>QaS3DHyxY~`zr;=9TE}6@rQVv)Wdb<(J2HH{S_Cq|-_OLng7^ z{ai%c-!$)wX*&7jJkfmZ$Zn2hqOz`fX|cZ+!&B_KuJwz>74sSZ-1?*M$P;*c=a^K+{_b1^LMXgm`^ahJw3r%eLZc&2jckH#~n z1C^!m%<1^f;2;(Z4~vod^KMR&m7YnMhefI%0~}{XFAm@6duen9ffC3J9W0ggJn%$Tjjwvj7#O& z`@B`28zYtHV5IT@E|gB_=NZKOGTpZU`8bE`5iK`L*>OuKw4h?_hgk z1o(54)FF*$t}g8;6rQdGn^5`nI-n zFrn>`#4c@z%;wT|uqNfD?M&BEu(q?Q!`cq0qrJ}Ql#{{w4)$qX`VQddrSIV2=1bqf zPV;`V!gJuzcn(HlN*sNwJabwq3zY{HE5sLvH!qC`^I0#A2NUk)e!>cs6dq7Xem?&Ct6SA8tfRd#No5*Vecyr0(UauNUy&2) zs{zb&(Kv0szRGt?*D7uj5KmOpTkE`C+;5i&{zIQ%580Kx2;U}E3 zUF?a>EmHT2ZRXkeXebRMkdRW<0Xjjp*yop(tGS}OHgcITiR_Xf*lDMck2uem%ieG> z5bljWSe~7qUv%kmO=7b>kOR{Ze3L`*1l%|ZTrgD;RI+TEmOl`}YxV#n*FPqXX6;ZD zb`fVPq$XkBUR6eDOqkY*njkZ*6=(p@J@aKYN?40x#1= zvXY#gY_7Py!pCd%y%DuXHU?6Ja)0X!Qc^Qka4zM2Z|i`+U#veA)4n4RZD9Zr-8Vi? z%Vo3R8`&1kctW8%`xjyrw~b!asadAjlq{lSZl6{>r%7~M2=ej~i)l!#Uk}lV+*qvRTNIoyNeSI)2d4X_tfq98V zWNXRhhSkoxydo)^q?r|>j(r}!EbGp^eSAPUiOuqPtA4uybaTMaU-hFZp|KG%9jsxU zGb`3?jk0Zes`R!-+5$Z(q}xanY`<6Y+kG)KOJDH>G|qheLKHJP-)J^E#3-686vikw zJWm?64!(A2|3FbK>ivo02EJzy5W&{h8Dt#9n8rD(G0}J;79xHjmg;;X7BfZ>iyfnU z?Z#gv&9?N~9wUNIN7^k_$r8t7D(Z6jbCkm@kE#0I^Nhn6^E$_~vkJw2pq&%XA84+< zdp^pv8WWZ;40wFi!{(-C<1b!ke!%Z^O=u*tWMJwaN-9)yposb7cr2nv+cV+KfTpj;VqL2(6Sh(mFNg z3(;iuzFunL(if)7fP)ysp{5~2VaONeJsdtGLkMDCo7(0K*;4bQO@zN(Dem~0pKqyF zbW&^6Gu7^zTjy&Bv!d#>gUAuO$};ON7=hqcmgID?OZygu zZ6?EfLq79(qJ31E=`CQ_8ka1=7}Q;e#TGb9$VF|5FJ|ZV9Jssa&&G@}#r21UGJYZ& zx%iDxP$@yeQosNM&k$VIH+>#Ww5gn5M=V~}u$tjcvPl+xIdN>shuJNDgD$k}atd8d zw+YL}(4XqHQwP(o&Z7X?#Nd(R!Z2T8#^rfH{jW)r zVA15DkXGDlXJgJ%2BrPP1f_u|v6qTp# zMb+{)tyzM;XsyVDbDk)ELshiUYoavK_i!5$r7QDHsLjYTv_P}o@cD!><8cuLv<`uU z_Zwy}(SGX(MroGBw4dO4J{I&B%>Hg^KS4C-yC6lEbgL4e?uoW>b}}UobFrYklLbpO}%9`owRO|9s>G7SeAtspX+2Eo{KBbiPc| zUD|2>Y&jd%d={h>!R$EDrNtVKJy_QhLv@?*o9dLHG!za&+pOfP>>RdDU0u~a4_qzG zeWO1WNmZ0;M*2r(p`(pyQF?h%l+UhbghI*7H;wBe*S*M{FDNg5Gg3FqyIAciZ&o|k zJ}meJbWW)C`UF}hkXw5U-310>j2qT0{{HX6G&f`LFLb42?N995t$Pqd*^qmXu^};y zy=8l%eqt|#e_^lGsSg`#=~0~3*C@a3#$T7IZ?FB?7~S|RWUcc**QtDsPxKYT)KODy zFjPXH{W^+!V*9Wz74bh$WFLt51=Vpo-{{Wwj8UMA!?*@{;Il!-w0J{Ds;N*nKQVo2 zm}rzTNg0TPkWF(l^-X4*?XR`(XVy!9i|5bTp z9WvVb{u3r|1Zz4s3VFaVicuyl@io_IHaDIwxm-j>eYr++_S#8p9rmRf<{PvZ?gz3W zo&mE5Lsr@)wrja)c$a?B)zWL^I2|(hjI1W< zb~j_o8mtbLE({dp9s*a5$tIsN1{xLRPx%{MzUPQPUeIe?Dn{-ncw~ygVhH;i7GZ|U z&BL6U1+s>9?E0X={_hVu5NoCMlyjlG_CM7p?i<1C&{l|wI$e+2O~SfnMh(t7Ecgt2 zwL%C7%&oRld&8ZcHoqZ^D}2W<45bwAZ$RbL3m(Lfn6L*q)z&j+q3iPRndiPgFGP3I zYjvkXCYyeuT2U=7VXfw!NVOq<&LuJP;-o-tKR?kgz>RLR?5SL7YL-P;&qafdf_@RD=yZjw!W!!T-5A#c zU#jLn2IC0N5XBTS0ZcLjc_JLU>0;wWH<$ljqrm?hHbfMjoZ%ikOzG{4Fys;JuDm*2 zeuo><6u&rqSMn1>sdR?%#z^}%oeq~Z=J2d8_x>-KZF`B3=^NvlMHt3PlDu3E0HK}D z_8Sw=ROc{~yk^_2vSr1h^NLp43_id7OfVzXQfd~gQ`kcUno6I>SG!_`lSais&-%xl#4S~(%Jc)Hb;=ti zxlSu;^RW|$A<{`05;z!u`9k%kZ}8r8VeUQe%=LB9bjojxLiLT&S53Zbg5+HQ922+7 z=#&f0Ow=et3#LANAl!=uk@Hr=6b|0^#ZS4~0kpUoUB`b)Gztm-Y>&d-zips1FO=Vz z8Bh0HnWPWNsk^*M{#TdsgyeS)%72PbtFMD{Fs#yKjMQ?ugJ*9Vj4PeauFyl=XUm~+ z#fBUmF~GU)8}9>Wn+$t|4RrIKs$@+BXH6MczZeoHMZU%3O)iz}3QCCfs+V$tkho`T zm~RwoN%2H7*QQ{H7`W;YI-+rygDgi$cI`ZjdFzI&J8K_I_Zw{MgL{QxT2;;mB5U7>=O&Z}4TG4qs!gMFZhqf2Dk zd)=Kd|H7zN`N8P!VN7Gw5n%L1Z$2W^1wBb0%eQhDeXI~$(Z2G0) zVlib4SBH^lYT-9hY*<*!wfw$w2O|S#pPM0hF}uymgO!G-mn54f&v99O9I|6micM0R z5p~Kf2aF2C-#FDxhYiJvZq~Ks01}#2IR+c`^2g=_Ekv^H@x+mB3Izown~gNX45f5s z`s>U#)vV96ou{84fEVFo3V6luFEInxuS*Jdm9WpWEwL1=;?Kaqk zy&o}i)z*1_WmR7_8gE2S9ecln@WR~BaJAz~VldXrvxMNScHaZD`GxK%QE%6__A@xl zYgH6H)~t!mdc)@v>6>jIv5HUm9kV0B_gg_{hv|`!}=dlg2(#LFgn|HW>=JQGDSty^0f8++3}dwzN_I# zyMRffD@mU}RP2f-=LuPUZ2t#&Wt)iDfZOHvpQ146k8yvne(R zO6f#=Rtoc67QDj0sIZ&d&02{0cVQ~|m?H$`UmOlV3eTZhw&Zj><2mqUQfa)RIFWo9 z1*dw%L%xyxQ9tpF?bKn-`={=ukiTU;+5TS<>UtDe6sCj?_b0l#Y>Wf`+_}~kSbihq zAeVZ7#xaN#ZF0ZF0c@cJiVz2u+ez*xa{A-DzAgpW$#Tauy}yd-4E;nQc|qSwTRm`& zaFX{4a7?GnQBQ%S{(Lwwt$x&^#V-tok<_AhiF zn6P%F-MB{{GX?NekJ=YKawiR}oUxh&e~j}Xex+^seq zd&?CDHnv|}EhVaN=hGs@mWgB@~QVOm{f? zob5g=|1HP(Dd1u%XvtM`cr~~gQTA`R^HL3YkU9;+ioYm%9m%BZ;U@T8>~lfQj4g^hO`=OS6cpKiLELA zPz0!zx4>;UNJOvvb)|(DSwgA9Z-`F~=L?}Y!tns5^si+=Q zHzV!ZK3#kzI<2^GLDVB;lNu4RR<5eH7$YiLY_PtX-I>XD`mO|Y99>@MeM?M`Ng|GJ zh%?I`sOG!aKcBRSY7Jq@Eqf?T(R*q&XpOc5>dXm6 zgP{f?PPN_l!%!I6xgNSonePX>^;)YzE8-Nk8ia7ncGRHNkJQGiaKq;l#%v`64V-fM z9TUpRdFuyeD?Nz&?DJR5c`Fw67tHy3y!8VUYS2N5M>FK~s6i2ddek6bwl|0`*o5aB zrdzGvY7j6HLZkrVM^cE@AcU|!FEt352+1TjA9=&P1xMXMQinRmx}Ruqky7v($Dr$^ zixJw|^F<)s=j6^qVeLl^+Dp6Dpl>uSIcWK2lxOX~3u7-PV2S?na;`OOk_|s7+Vp59 z@{}$pbToU*lZwVayo==L!W+szJI>b&p=POQC&W#(?+gu@p3Ap}mQCbk+xv#5PEs!9 zpTk!N7f|jyw2cbmv)lVE+bit?#JHOkM=Cu|2|i9lVB$bgH&{n1$QQWXlm{b6=KcXY zI{AqZp>@wMd#mIt@p`hNdg9r#!7~@LzzzlFsu#G3S-QK)^UZNofCKW97N+A zfer@Uj@)nrvCl!dBiaupViqqT&*A2Y`+aW+q|0!<#@H|yG(HheG%Ep33vA2BsIfjyX2TfTRI3(^iE$ff)v}L>`n2e8IuACg z4+dwdt<7xV32LgWxtAIUopvZ#AUA<#(Oj5?mP!E#zIO7GDT(u}U;fU@q z6S|GMznI%d*U?OGr;z&EwR|}?(Eo&pyDNDZ%`;U`py3i(-Fl%H?7{_9v_NQL!ziX{P%S+m!olUO4;d3;+J_|M_1*B*$m_8$7ljqWf^C z>Q*a%(&yZ&E!qG7L;uhJo-h3S{A2|``|$dKs<0JJ%nk*|9llNQ^vc18SwtO94iE!L@?uRWZVaYhY^1p@V|f=GF}%EnRn|?41#|X{($pWYW+I91B3GV`o4eA8!M!ckMz+|eCSae-h>=; z2jbPNl=mR}&&KCO%kXs&j2UzT=NASIl|SRnY7oH%ngA=g0i0UwB0$V%DKO#Zv(|(` zJ>VBuVZfkzG`{2=RnZ@66mb~P0^lTgJ^W-Qb)i@c<+cxob}V2z=q)GKeE^IA#F`(5 zIt7e3R5<&3qKm6l?R#bdD?YhjprkrD7QO&l=CJd`Akp4Z)0F=rFgIIS{oDE)gX39v zkDcH7i9yM4V=@E+YF*zDQq}U18q=82Qgh)!2+Y3= z13d*R5Ayzi4|!lkSFjIZb~J;d*M^u6Cq{;j_EzOQI$|Wi6Jc@tT;F}QKN?tJNi1ae z^oF_1ivXD9S7>iSf|glRRskKRYz@3nDIa(Ct-%OYWinvC!O+9&2|AWK1B*aib!kY* zU($x?(gbEzyz#jTM(3ahm@->uEl#R3po^H$KP4{>To;}>CF>G*yjn>kpcKL9xQR`whvWZV(UJHz87=*I%ibF|NCP? zI_+zTiHO%>k_(dDeE}g*uDmgiSeNcLUQ@*(>f`4C#!471gyT!br z3IlV>uV>I&YrQd)3-7m96>~n%yr;>jvpi#`RGKvPjX@EH50$Wl5V#K3EeU=)xmahF zhRTvb;#=dRH&)jFX<`~+R4DJm!;J(!o)r`rNj(2N?JRaTbov)xV7>vCZP0yPSp$p+ z9wrnpdFDCIEO9dL^InoUMm&**_J)R(irS(@+nFT@vI1Ksrh)UJbPs+241&v$*!3B3 zOu`semgiw958Z=5G1n&et=}+6jhVuf5RUWUSm?CJK~G$-WN8kbs2S63L-*>39dSV* zE(WXMf~}jGV2xY5pJ2W;+0k0sM2bU=^XE4EH&B6$gkVCM+@c$rR}~2489T z>d;YtblpD~B~VQ*U6=SY>2c)Za;eZ< zD1V*B&*GYUd%)l7z416;N!y(0W5l(&ruHlAVy;EsOjDQYqe*cmiJ=t3Zf55e<_uo< zH0uVn>j~7r(@pPHVTC&$!KrCw_Oa$M_4ab%A8+1!YbdQ3wnC%;Jh@qoR%0VD%{F6q zxU!8Qs}x1Sd)a+S!fH+$w&KAlK|@zU$EK#$FO$DeV>imw=BdZi^#rqg0mE-oi^8KY zgt9bI=p{pZ7y@5Wm>5+Z=p{u~D{2Dt9R%j$%BoTQ192Zx|H?0z&(>FCm~4q7L#>#N zp(LB?NXZU}BUY*?Nzv~-$a88v>R}gLuX*^qS9IJD3@Kex(m%JP2{t2aQ=2lS>;rC+ zz>|JKn?968{U!ok5lU)@Pyj2)e;b1QbkoD$YmcK z{%OTEb1gm0sbe?Emxt#WOf)8Txa=qrPxlikj=Fw9&aC?zaDZKd=v;y`$bhfyF$VPm zhe)6%vNXMbu*Q%&I3&vh?m{OSHiODP}{sOtr%+k0%-(Y!9Ev3hx z?eJ1RRJXrx2Kj2uMaGydV{MFOEcB%s^T=3$Q@Uj=HoIy^#`yg{hkV zQ@~<_vwF_87wbCtSm2)SC#D%KF9U2~&bNHUO>alO+C*ac>Khrw9?x68+8E1MRkZ=P zd<80~w|GS|$Va>a5nQU5d*NQ^9{nBLb?&h+scrd+?R(oTU%B_~man+>>&REJIa|C6 zBG}TE+Zu*+#V+;jh*$VzuD5uF?aSqeSCvZnIp&x{Pd6jL_FV!Pn$y*E#4D`Q%C~q` zlMN-}<9ppLUVYmcu>oA>Gy~%_V6z&l(Ff+=h1u5qcfo`dr@}g8;{KDY&60#;TC^*FMdGA1hV3jmQwXz zfuP@^4k=R)0)(q(4Y>gqom@%SU;hL>6IHDl2XcW>*}W>uAFxe*-0{gP6=o{$cd$Y3 z6hz?1ZF@-oJbRs&c5F8-b5A=k%wW3IHzrZ35IKym?F+##TDECz#W&zGOJY(Sgmw7V zJVvTUoz6+VAz=nr3|FeNe=d!8TIolE;pFa{qLxnW7(*1*FP*;1t9xHLW4te=vBGd9 zUOk_1!dU^SYG}rbj*M>BumY5kbiyg+AyUI6WsJ`FKVi~*W6B?Kc*;chh;IN17HkkX zqTdRMtV;;4t%%Czqv&n;!ov|(LUL{fXX@)@u1u%7ZmSqvnyi(MQ zl`gNg8OE?Xy7_Q|AW)DCJ}+!He3>{2tH=4D@(6n`wE@JLB=mFhbi(il*2M+Z7Vbs zT-shMMb49t+yG|Silo&=A@u_?J%!?}M9F3=B*a_rvX+Oe0KRA%OeilKV#|#Dy`IS7 zLqNXR2Tox}DB0h5C^z7%oJR6B0a=eQ^PKyVZ31s3nGDZm-9V#15vcc-*=sm{`ew#IQB>G-XYsR=2Cp{#b#SBI(0mZT#k#oaJlNdxeb<>FT@k2gn476raf$u zx+Ruz+ikmtg1*UQlKj3Ck~g{KH)$F3*}TWwzU?6xMu6u*&ZbEh~9$`;M&WvVRifV z@QY0b!X{#6#tmfy8L=>m)+OUEjj_)oOmGe_af9@p?k8BIfx2!lTDdG~qBpO8-u_Cs z`u5uIwSMi6m;pi_^X)io%s#waN`bK3v^m-&{H9HbWtZQyDgR;nO##!;W4~!rG|Kjy zHsvdBzbQ(Mk9F48%Vwu30+Mn*PSYj>*;yyGwYyicCuyzwiSlP1pK0s1er=y=>qEM? zeWpz%XSdI^jYlq(?K5qv1ab+a8?$|;O`&qzXNs>pb^A;)jM8qOX;bo8I88B)+eN~O z))xarL}t34Kp0RuPSaLrEQFn=4KB*#Gi{ZPy|&ZT@Z~k#7`sjJ1G#-|w<$g`br$V5 z#UShqq2H9ddkH<*zxA@gwVo9+5l{3TmN2ytwit)sw5h<`cAN%LhvRe;`F5OsBYW-+ zlZh|KDKXGz*YyiUIAxYi4!z&`fq{Rsi_K5R#r?osZ)sJ3!|ad8OWzUCqcHkFK<1)mMz~59 zl{A;6zQbx&@9#XZQpBgw9R%{q*1Dy@HvYK2ifOMR6Gf3?hLubjLLb^`)37oB zeo!?jy(c#4yeJ)I{Z~4ajz$26N$F@55YnP39jzh8Jd}>+0`rnUJEQe(N=J8sr{7P= zh!B*H1_*H}h|FSh)s|CogbJ(*J!-}?(!3Jy`@$C z4HF;VFMUUBN;hItI^lIF9Zeq`uwY6DCY3|!fJ;A=4tR{4(z~k51gQh;{g66ZdX#jN zIxwprQs>@YQUz!subb4_SYq5yu#DX%braHEfj*(v2IudT~%PkNV;Q+QvghTU&=Ow}c(=QRO=OKn{5w0Wp;1c0JH+!r%ML4v2 zpu=1uTrcU*EAkCpDf%VEp=-r!xeE>a_j0S7!u7=Za|OPkH$r|1ap;ARFCmWY56ThZ zx*~}$A+9Sw{Sx9fW(#p$(b`*xLl1^}gt+dAz9YnSwJr@Ij?=gpL}_~cun={uJ5S^x zan>?ik4STrwPX zr{6Lh#$7HM4oy9*OHxQ9p}u4|(e`;DulNBfXXr-|Z%s1HV z`Ff&7thKmbci8ie`{+G~_{_MEu7b!r?xU->{1x|M%w<~Xt?I`8j=66n|IRO%9r=N8 z4+Tpx@&lu-Upw#vZ7-&5y|%mdZZ*e&A6SqptOM(p4(t`Z#>FLfz!w8QxFx!g9r%Gg zGlI9^Nb>MI?qhFy1b)1!fP}#lTyCKUtL-Q`3O!iuRF2TYe1v`rJ>$6~9+BN)Ml^Qb z+zN*;kNW{8s2+i5v3tNR@T_@BJ7|8;E@>y=Bkc%ti4)5~{yz|BzvG=Bm?PT^RU-QZ zbKdfD{)UN<@0Y$KwzOl!mUiekNk`ftJZ@y~z~3y$%|&BkDc zd88dT__(AUn4BHYMrns!^^#?16n{BfaammQj|gWSX@{9B^_F%(YIcu7#vsc*!p^X4 zeT1DL-XrXMBWIC#&37yqNE}T#Zm{=%1_M6J3(fe1M8S~ z!pt@K)|huPBu!pW?sLa8Sv=h_S zPCE(sq@8@DJW=Cl->DHfpsxiJf|WdLAhIxY8xPgS0*hoI*67wmQn(_4$r1tSM;hIy z^I=w8Md^Qs3OPz-sQwZC8k`;KU;EyK+id&J9WD3@TU%TYJc@z+EH*`oHp3g7b%zhN6_rP) zh*E|#fxGwV=^@E@BkdHDJi8Ht-dJL)76$&Xq`h^X89jISrXz8i@*&4Z3hcVyCwX*F zO2i9}(Tf7;4^Qds<}zAL=fraN)Bu&=NPD~dL>kzxK_3mr&3hmO^}ZK#gye}N>kn9U zk=Ow|RQifLoWu-9i(qO&D8kD8Tf=1sWMZ9Ne-BMe5R}TV436V+wRayH=lKz75)Q;62-Nt#$0mG##r!~%HR*d z2Owe%kKi-OIgj82*rECqeCFfi&Mo)=<4_R_KJ&?uf=gT7JPt7{RlquG0O{f zvaEU@ezS&Dr~O0^^MoXyRsXyNpT&xvNAOuW@**VosNUHv_y}_!k+}_RZ_KR?#P9sX z9MNYf`+SQ&5X!HV+p%`4P>4RC={bx@qR(eR?zmJe*kN0-2jhOAMGg~Xp8%miUcdQR zoZYg|R2*=~KBlfM`viPspKtU>*#{N!9kJkw4;$l(4}c$A4F=(=FrQuV0Y=+24{ zD9gW`4Oj>}9wsV2p!b8VKy-q$Ps;e(6(6RW-d!ah+|Xdxd_c2-)`Qnow6N$K#PgL( z)Zv9j+$XItWBDiM3j0JOjPwnYnOh!pAEt9RqwWKSp3`0T0c{N!UFy0Iv-vBo`+&tj z7~HA*fc4{7UH1X&i<>v;wDZtW)-Z zy-N;!?fj|xuxVkJd6;%@Uzql&aDfXy@UyOK!0(`U#n)B$felZt(hry`3DTt>*y=}T z!lfUuT3KwEr61T5PK5f&Ope<+(UK0nbt2SYUiIii zuHAjXN8wFt{%rO87oMLg7M5%#7Wt;oB5sYM&J^`gLpV#Jco^(aPwTfY<|Qph}t z(Xvs;nh}Pu(ybW*lgrVJR&ff6H6xP8xHY5YaH-IYKp^ch(U3e|nXDj?e91CcINKPg z8m;W?zpWaf7s04UHNs}3dQ>A2OkJ01w6f)iP>nWXt47@5c~m3(W~)avS`k3ERil-a za%;_Kh4IA#1t!{F$;jOCbU)D|0%B_WdYEsl8L^^vKbjFX+$JF2WlN5;OEX$wps{L% zs`0%Y)d(fEbGlU{U|hn1s*zdIR*eEas?j%Qb+e|e8*O3@fqKn>sF3PCiN&sSnsF5&#W`)tQ=v>7`bQV2xQaStsG&K$2`i>3OAvZqYZE6Xk)A# zkq@+^9Ibr9;?@ej-~bU`Ba2lVx8}2CpuBV=oR~E4Crao-StqDP4u*9jvh|}HVW}O` zLe=Qo?(pJbiRzcr14XQFrw6Ko-%bx!2XD921J%LTI(`qpx8Gy2(bmoA)llm^0PVFN zzXuAXzx^J|3F7p7pzwOG$M3P>?e{R|I2KSkf8^f~Tl?7W{JV3Z5 z9;@e<+XLHryFFMW{zbsODN%Yq&|h8;z;CYyXzlWPtUhnA$BLxr?ez%iJzkG*^h{j~ zHJe8)w0Wp@nALWd$s3OK*m|_r7A-E2?&Vk1bLtacpZ0;$68%^>JIW) zl9P6j$Aa=a?jR3fR=I;bHf9HT;8(8VMvEcVJH}%z#j2bbkEJHI5#zyWTp=8sXnU~_ zB;4fti6_b_F~|co>mAeVAdfY6a37;Qmg?BdKQ|FcgmSP{`fmLOAagYahM+FvokHv4Y)iB5-=sJQtHl6Pvk8hOE z&8d;6qZkh$`Z_TlfFA`|FvbJBGv)WPe7jPpM~v}6A^6L_WIDa&y7?}Fgp&}#rik&t zNqMZ0?W5JCu|07VPpIxbsXNN!JtrLb_T;4ywaDf?7-e&Q;HmgAot6*W6x|)!obB0F zcwkZZfNWkLRM(!=Zyyg{NVG{@PvnT@MXu%t-dQ?I=4{VHYky>Oe&8YVr?NReFl(cp zY|ev;14vA8{=keV4S^D*&*RhwWpjRpk#wpfoAZN_=hE4nAGlqd`~!}6^xg-vZV&fo zZBisDL$dh5`OlrpskJostM~%MCwb*@r5`wwIS*!fOj$yOlR5c|7SWW<$&Yri?2abVT@f%faJ%^d<#KWri)`@XY~1i+ zCtJoGwB3=+$&DH+v078?u21mbPinU#nRCNCne$+j%qeOp+qyw`O0!ckr)bP-RT>(U z%BbOzCAB0e4l)*S7RluTmg%QB7>x^!IH|9b%E^VXX9uaAT*`DfAD8~BBVu~Gau&FE zjx7hv&}KUxsCu)TBIK0H`GHj_Mwy%q4I5OQQ;D1#5m}o=xX{tYIJ8WOoTNB=kHGRt zU+c=`xWeNxd=_NJ8WFL=9#MA!09OIWHuQvY2b7q%v&Try$_S zWFvF)xlAH@_T5opdTgQjVsDRW26W}T(N%8yokKAP<=>^eXBkM*RA?&0^X|6 zcaoMI#QIhtY~yNxdVyO9v8yUo4*Lq;g9Uf6slmoPT(JV%1a!lUjO!&BhFb7c9@zpT z49vr{w3N>ic?xbcix!w<_(r94}m)$azIKlZ60H!Qfp5 z9a)zKEzxOX3U#l4KqRt}3*q0Gq%M6!i-!xgpz|oK+MlJXw9ayvhz-h(a(-bW-p0j4 zCblM;Hq1Aq4DKf~yB#q)x-~5(^dB7Fsinc?#zAQA)peyCb|O@a-1hV~o$nGMF4-cW`S(jQ2s z-fvVGN4JO#FW)U2gQgY_sVtLh8`U^{@-yb z%!Y>nj?==G%yGgpgHWS7b$nr9LeX@teey2|=0sV6{Ca{Z_<$cWnMY|eXd6{}tR=@; zDEt(-+&~~r6`&6MICf!Usw!RK7e2sXSpH@ZNR{Y1HjlE9(Osl^NZ3vE+0jQ&p(Dp;_7W++UkB`n22&p{UDi0j1yL1bP59ph7y7{o!<+>R zHYZ)#N}Sm_+S;n*Y|s^1T&3*;{0J!&g$-KCBWu?=RP$qHR$-D=MXx}13$v*E!f8H- zicRS3D2|_O>0*$|4!&6^ic#JVFkFfmB?SX)tXZ1Fv{J%`jpo(~FLY_uS+&Rttfl;7 z`0a>+!!w8X`(O%vA;hS9J@%NVS#67OMS&rG;PFRbO&)gNS6bx*j7m9Ip1OeCmm(@w z-ooFAuO}4cTP5>zdw3||*etR998{yGylEP(Cej7tx>@?h zP4(Cu#9iq2V0jK9bn--DjGDyBh2+Y;f%s9YwNQ?lV}R05swz6F5~@X`2DmID>iuWs zZ1MH1om*lyt*lr_@s@`>V>uV=oeY;ffxq8p#1>ZzdCW;9w%DR@v9VCsHB*-Jq`}S6 z&I@MFDS}(R$0tpNBrr5ATPn+Q2yRmRx=C`dMfG`N$o37vj3hZITjb*YSL|pzNu!ch zTs;*IV!bc8Twf5TN-gC{V}bi0h!&Cf`<-7f^ym}Ob!YuRoOisQKQY&*{nk%Rksxwo z60P}yC+C>72CA<{T~`cn=w<=G=}k`*37$~A2@i9y65C)&G?NDoS>7S3Q`V6WB->g= zc!|S46zQg=tNK;S+H1V#}y%a?O%p+-$E+8n);s&hk z_LQbSqh*<2P}v#?sBkMo6DOV0fbs#Qo|NHuJs38S?NmB7x(cJ$6iheZW3k>a-hjy| zZ_K}g@s&YLmaP8+!?s>@6VTxH4a3HcfvIeXnkZH>BJ81ZNVus^&{LE>M^~n2i?1{m zZ|7o63&MV0<`P7&A!-H7iLGzV&ky~%hMylh#5*9%r4~Dg`j5GOecxkm0AydZQqrCq zUaHB2Ar9X&%P=l^q+ZAYhDl)=P|A96R7%-Hx&~?vVh#WYc)Jx0v}k&^xx>KFjL_7=z2dr5DB?c)(NZ!IHRu2PY3& z`>Pjt*%n3IJWFN5!7sSFi}9$ut(V^zy555l78~yYW;ib3N7q7DcP^W6jHhhIXMa(+-R7D@!=jc-vU1;;IPgV5D ztw*1#=u12IRbJ6#m-@D*DjYqf&ifWbk?0n4RQ39(fwZqC>m6={e%TWC}q zbUqptdq|d}Q3aJAh8To>9+0_1Iu%dOxOFNnWZpUz&t5q?)j6xyslH7~=Y2A^Jh&aE zIocLU#Us%IHoy32>T*NW9o&QlgWsO{NG9XOYuOO<);HA3@Qp$8g!HAQ=A50;qW2@6 zO5cSTk_gW`V{)F?BkyUd*;G<-p`yI1|1L~XBspO4ScS>5v*D1hp;tF1Y{9%nO1pI- z%kB*ICA8L|D%@yGNPSb5YIfP92_!4HGe4^1hsbV4H)rN}NvCBWxJRtb$2rUMcpZ4Q z0=ysa-nke@esq#RT5g@X=GJHn2pPp-#UK!|gocDg@W?vv#w2kBtgefJ{lr{5;J1Ea zU{GmYm=RJjE7Yg8cx!@6kPCppcfpQz#cZ>Pf&Ch?YT9o`;pIA<#El=%1IStG;zep# zDbXKIoy*?isbj}(f~JwG)(GFJC^oqYv#RLInB``R$yTPU7+#RU(`|@#t4MXArM$&$ zRQ<7F4cPa78)P{1obD%5sPatdz}i`WGZ$Vm7K045@?zSBePm;A3gdIHU56WO#b((R zq`PQIUczd9kD0ZHhx`p?us9GS7-|74L z*jdD;oU!uRoA=rp%IRoP_$0egrW)%>)iUOvFy0_F_^T?OSRoYsvw-xruVBW3Cp1q&QaX;u6ntCu?->C9L z#vpLq+cn03n^SrMfv(1C8maf8M3IX81)G}w{%?g3cEv-29r9Z3o@8)O zPwL{YO<@A(i53T3ztHI1ygotC-WfwnJbf4qbu3 zcf4+g{1**L_TcFx>5CRq7m`9+>bA}It0u{cnPT&$p`N)0{64lDL z4}-HP`Naa3>Ia=Clu@-MDn;dA5>;{2@{p)v?Xx%U-WqWLgkT;=@EzC%Q@0GCXl|I= zF*B8-G^&Bpt6k+kDpJapA8fYnY@$PNE$CS7dO!IAp1crrJ*w z)szSmQ-LpP-8QCD7=^uUOvU!LurY0T+n53qHYWT+S;xp!%63!wJYp2>*qE@r^Rh9a zIqAjNyj!bWPc%s}VPnEd+Tw2;6IMH2Hl`rzurX~S-!`Tf*>iW~=GZnSBGR@oVGL~^ zBNHg7)Z55}-Ab2{sniuVBQh>FM_o5-3&K86(y_mU_BAmNvU9}smu`!COcbixnN3754iyS zK6AZA12mMpBm;Dud@f51I#QN5-49>%@x)MP>ktjl<+7ApG$=!6$`B3E7jv#78la15 zueWFrm?H?F7w1U7VU8dWZ~ep^(E#6G<;VsY8{AbP8({P{j950neA_s(0ahg6M{25J z+;yZTTU%dGm~=umSPIs?WP_#O2-(0?vSoue9mO`6mv9i6kPfD{Ws3(>;ZsLEz%Y@& z7~(-udfYi7Zq9a{p|Sql4-C>;2nSd?N8$xDQa83O9PGunbg;K)O9z`2LOLiz#?z1v z_>)ICU`y3`3kM)aV<;S8kS7jTQ+QujQ?0G{6SGE~nU_U!+a()JRqxocK`gz6gZJLJ z1#6G+JgIIQVU4xsrubkfv~0+lZjG1yI@i&O(E#$fbY^srT;o2d=sG#a^#t-e^7RW( zu@59Z@uw#y6*G4~pY_szEgnN$07#=m(*h36>Pbv`cyu8$*Rl7^-rB!nX<*3yh;bo=VBA4kd-BXKNOd+;~O+|}hSxZq6z=m67!v_X*k4v|P zb+XELUsVWRVe>QQVKtd$nW_uK?z88Wx=v?h#^!mVZJQ}rB{yT)Lg_w;knD7R)pfEp zU3|+;2fg8lCy~&+F>2rYPna~{n5uiEx@%P;SFE=K|0CaV+VPl;yF9N38 z+v~(eJnViVg`rv#DXQ&WPqWmbmJ-!9PO6&@f?81fW5d^$8I#>8)FNk&uH0qqo2YtJ zDrpdR6-GI(+b-sDo!aVCx}Q)85w1&~`)Vzwh0&VLd+p#DK479<`UGlo>{{P_Lz~s_ zuGYlOH~28=Vzb11s(ZdRwF1xM5XGc^M=N&h_t5}1Ti6=lH%4*#2X7VcKX;=%n12Uz zf{ZrCK}LkXc8n3gZtp^N2h;TDiZNotVY!GIV^rPI{lpjnVPHwejrYW5gQw?uN`89flbNI3^79c-YO;)y;y6+1XCEpV)R?4aE$H6 zwdNB}T0PR&t+O;X1sglw zpC_Y@@-05EsSm~V1k1zm?5Itiu8tPUD*7NgjX_-ryE@!J)9O`y_PT;DO(Vo8E|VrM zHUzZKgq_R)Yon3eYCk6_2Xb>rVun z4)FnCbjh*aTXP8`-j?hEs2tkL@D@C;QjXj|{$e{CBO40F<}Ei{9JJG6`_Rw3HTGKH zd(-MS$-3EJbe_oZlWfb|tkZ7JhdzTXAI^t{kEtKdhlY^zVtj_LMXcX%mB1W3gkSoB zfCWIcjqzD3-)<6)VMS=A=8-nvYwop}L%0$NG{r^)Tf zojJ)4kW-^#!Q|#X^;O~V=oo2_rW@>a1&CL4mnB)s-@jD+k$pz>fu&IFrq0frYz%mDTi&RuCrul&H4dsV#guXx#WsA}`JsuDa0-aPyp_j3cj5kZRPh-i>j z?l{tesSF(dr>0Nehppgr z?T+X+H)P>skz#bC>l(L%^wGqx+h7No22K3x<)23@z4Z#NYRvpUPoN>6t?8r+)qKV* zY=lAA!)A{RH)-XN`NX~gUStz|Lzt4(8`fPEX(yd(oA@a4%NDS~IybXDfLp4hBr z_sjGz)V*JRfR!`ZGEq#sMMq8vlIEvqclgNiyR^gV`40J4KhQ%A?;aBjU=G9M3*tgd zhff!*!qb0$@Tr?CF>2vrTqh-ols1V`b1rZlg`RjNJ^Vtgs~k0y;8g!IThq&DU#oIjOt_oM@Zi?|VbU|brOzraX2 zeeaQ?n#LZ{qsY9XIi$O01LyQIdG+U2=79vWY<-S4oN{!oG&eVwD%{NelY-3EsPMHj zw0I$n=`DEkq`d-)yJ`hvU673VPfZr{O9tt#)O^3CVjS4Rr`^vSv4p?#4I_?1tA%KV zx%aPUJmw2SFPTi><9_6IqYl|L?b7*AP3B6wojmAM2QwrJdo8!&!;sx(j!pP-T#s6E z38Zu&Om*QKGMuSzUBHJW8UvY@eRM6+eja%0`9gcFv&0aZ>J087BIA_TrzasL_wSkK z)_ONWWy^>ut|*~?rbFFCo0}*OT?=hj(nq&FCzrWfAM@3UeE8^g$hr=RWji=#nbE_v z$M%1UPRYaF4!gquqr0Ih$ijLB+Y{g}y9ld`Z}_ZZV2Uq>ULzfr*8slJoB?xsE;{re zsRiL9S*U4UH46wb?w{Csdv_Y(r+N;;t<2sN)$4@6D--pvdJ01LarrK-Gm_KLwuWz* z1W!A^H&KdE*9vh*3D}jjk0f+atV26KJs@F09X(D4c4d)6du!M(kyVH&({^Q(+KF!2W<;VxKdxKZq7_7@ZwxXAQ@(+Nc^X=L^Oh14ExrTu0w9*U|P{zhInl zT~XYqC^ZiE-aBI|utQ3Vv?5g_tUy*~& zb$80O`=Pq}sdKO^N`XBhTlM{PK3^nZS62O(E9D^zdWR!%@B3Zpg49v(i)Zyn^?HK7 zd{H^87#N2@v2=BJs-u&z2``PI$Ea5LMU{m8ovhUW?z2$iWi%bRzO&^v=2#6jCVE1u zc=xuN0`B>?ngUbjF`ELutfuU8aT`t97h@cwDf(uN@LltZDGUKNngV094UMMg)^P{2 zq9oeOxy>@M>3*Q!mIKz6Ew|;s{VBHPV7P&HSPs}Qa$Kf^KwL}tv~Q`Jf!UWuyz~Qc zY^JP_81ar+y|in0sCSsFQ*E2+NaGkx*_h{Yu?}}|IC3!-|H{&2`@_+W!4wT$sON;i z6g^>VRO6)^R24?k4R4M68{LT(yLe;-%MBNWizuQEJ6H~IxuWF&&wfSAjVNj3Xt`0H z#^h+ZQFkD)qvZzu%3nKLZa{FC^Mg0-sQXYkS`Iz_ijFFy<SSu&Gyi zJuxENgW+;$+4bCoZqSVEYlq9BgO^0gu5OD(-pePc`{dt++2L|*Z>C^fwV>)x zcva_c54=kT{j26>QDmepH7k3rV1p5U*?rxNI`6tS+zoqj883%^lwJqB{sx2%xkY*3 zqX(vI7OCT1z&lAU}Mm)rIyn^`Al<^(iGaRpYo|sW@ zi~)04%SSOs4wz#nKBv8Ck3`rJFt-VP2h6=l|DT)7^($a*NHmOqIgFu;e~JM&MW~#B zIrbBD^2+S$QJ8ObXga<4VO=jT6ILdEA?xsT`I8+nhowxmb2p#Atb}|<%saL3nZhtfVl4!y*IIxxoL_lx$g#;)0_5wEX83>Cu9$SqX3p(#5vu;1%u!C(C51 z@|L5kTfahh)O*f`arT6I)Pte!R&vU{1kYieI`$X~4>7dWnI2hxhJ3XF!+7whVN+nT z%nAI3q)Ha-9`?+rTQQd#pLHeecXk3ttF*5u&c@z#{S<5at6p{Y6JsfI0hVGOm&Hm0 zt^J8rVud^))O~ZgKuDDl4C|g$axVC|1ICT{cVQ%N{Zl);q&g*{lO<`P#SvU;%K_`O zkAnl*QBZGH?bhnM;2vj+L%S>TabY0;>?gu>v*f3k&;SNiDmP!4Nm3b>5+=c!ltRfn zenZdhGbTiJQ@iP47${V;33eF>$T$9d7n@)uf5637Xm+_vCGyx1!@xH*E1aLXG-d|_ zXSRAvl5h~VF>4TzOUtoE0thHcNn@9H5bcx+0D7j^>5-k4W%t=3QCDbATG*t3bJtE` zdqcV2b?L4dzWWrL*ZM7Z?jyYg&u@oZ2YCLNW(7Ni%FgbzfE_9eOm-VL{ z#&54DEa^K~b~^C+1k28LB_*TD+i)Tri4x{bg|*uP9GGJ=nPAMnAdadP@BGA^I9Yso zrmr0*i+*%YW@Mb~Y%UJ6<76>a+l3gdg2hOeD`(mjcy3ZGaJ?kY+4~MaVn@NQB4j5{ zmYc47jr462i8$GBM8p?N-#borV;m9?lpT@FifYoh;e_Y;4bmPo{cXT zGFsl1bwANU>|mrUTO~S)F1sCIiCA1l%5wRg_sT|xu|Ydhc9TLx%1(8ojzC%b;$zX2 z`~_h+5-aVb!KfY`Rf_R5;%qF4vT@%llMW3UQ@x((SpiCWWy%gKba&Lg0}rWe?A4b$ z>)ZCIai5(_Coy@h9$~|F4*=s$Vi3|?CvA8Tu2DqCiOIlZSN7p;yviJnmp%>TntAVehz{N#i_}2+Kun%hB+0Kc~X-DbA+2Y*0)k%fw@kr z-};F;smT-J-jX&bp3E6_nsl^a1<_G+nAc?I!LOa0j0L$;ld)d*w>UQ$D|(HKdk`ia z5ZoTnVKAT?mIMN6$h>F)#PIIAP8xdt0tJ z=#}Oxqkm0*6xb_fLY3sU-&}Lk*XFC7Wxb!+L78d>nrZGok=lziOH|^n@FB>c&a(kaAxB z#U~neN+3N?v_E=*?vCYRT=?X2Uq%JLp>DUD#;SW5>-n&=qXLRq;M$9+GWBQ}^7si( zbV`U@T&Q(gh{`XVAGX@uB$M~%84j;RAdU?m({_o0Z+c1b; ze03hfAX@Rw%PW9Jd#}>0Y@CdDxMYWlH8B8JIZ5j+rmG?*abI{Z@b`&?Ek4JUb*dpMAU5;WLf5P?1CU6 z2in*!h*9Tq?1HGdd?x`0nO+xef7b|DM;+oSwH3KbwhQh;;I<3m?3K$dxX)_X1>dt$ z*OX_5yHklkY?}3`o&?;-T@F2&p#Mfm?w39K)E(I(XI8(VvV+mFTrrN zFbHs_t3+L)Xll5W6cU+0Xq@;sD`B{C;!s3M+od?9-nyzG6qRpONR&4m={oZ*%2NPL8#MC#Uhlx!s#~$UCI1Pv-(JdOfQPH!D zkgMC*qtt*++m1=PsDH^vbe@Mmgt9ldcd z;?aWJj^0qCx;uJ97|ScY>4|F9NUiIXE)>xl>VtWrHwcdty}=fO<%-^5W5ee}Z)kIm z6h>kwcl3su;g$P|zRS}DfsH$QL+$ZS^afWiO8e#_xZlwmhl<09{++Hx>rMyUS{}Q` zxlp;dlk)Kj=FTu*X8ffd-Vdx5SGcm?+myuy2kuPxvUGJalr zlA^psBT*041(6ZuGafOxZDsjjMik!U(eCZ80=%cRZ`KdyxlKEtMNmJh{R_>p(dQG9 z{o^s50xazX?@0yXJio@a%<59iv4!**MJ@tq(?Cb;oEC zj&>U#6`63>o-0bLp*VXaEM)#>+P1jCZ$6Prklpiltw?)pynNTZ6R#V}!Iw9gtRDBC zpQsVIP_bSPw%cJ^4K?V)7x3_fN4^4uX*CqB&zh#kWW>eU5aC#U!(5!rZ~esVKrMcH zC{RkhLShwqZ zYAEu4TMZftiKqMTCA2kkgKaD$?n9v_qlpctfq?C%p-%Wz>^OxoRBY$1TkQEem@Q!f zGuIn4Lu2Dzjxzr4LdLrsmFVsMo(y|#DBB-3YoHyW?)%fOrOD%6!LqRqe7_e@waOZ! zw;RjC+xQeORQE>f+|*Jf-VfF-Kk#z4=*BSqzv_vo{X#M*3{WphGUE%1?*f~e-mOKD zA~GuU+j*}{PGYk3)WYq}Hf0o=5;^P0_}bgMWWs|xh|A{ukuj^i3!7udey%ub#8+`_bO<4m+%Xa3<@FAYO9gG_C}C4cWcg zP1L$NM$oj{`Bmlok)YR^Ng{#$l%wwbVshPCNR*igw*zs1HUpNOzdc!6=u67_ok5n>4hKs3#lc16P^@ zcfJb0^PG&94=fJljF%5QBN$RSF&wEqF1j@xA)}<4&%~a0(oJJmi=CB2u*;^eFNob)Y`^mpgIFYH5$=lSUpZGHJeHd{td<{en589`5ffOhX;@rkTQ#^MPxGu?`8^*<}Qq z^3^}EToN#Vv?bYE;0u^Nwhwwc}gy9Dvdyx3CmOBDquL6 zY=z+))qSIFmNDX5o>=Cem?NC0FZ}zz|L1?D4<-@kR#*xZHB;FzbX3|g)O-H_|Dpfq zf6o{G;U~wSGL>guG89}GhT;?coUsqg9N02CtQ3IhO!@nHqqE4%2g6>BgcF}&sh6_t~v_!+4{z zxprXwU6_aUDT!f8lix7v%K!{J3cfKcS{j%#%}LGqsmGkje2p~J?{z?gBK`45;1*WPtd4u;N6Vv05kPNOe^MG$(G z&zePn56jK`#q&u=_TfqpCd+__uK=bWk1*M9!FlOEt_x2TwvE@Q01oZRlYJ~mS$+U~ zg$IU%NRTB8XnEBQ^9=?RohLHuDe@O^^spN|dYA@ZfW03uvxN(`mmi>nzrn{$fYKohtfqJU0~-tQDsE1nC@hIbRy8o9@G)w(mM15^ZB+vpeL&K0;U{*j65J% zy>O4Y1Vao+E2NFg^a;|6KEVPZ+|A)n5cBWClxJh6R;IAd1b_;k84vN`lk8 zEAjoQ8&YmszQBeNT$#?@H4B2B?FO?9yx)1$iz6+@lO4pC07zw7m{LXSI)1}TwS1h$ zJ{-Jxcp&H0;#R|CQiYa}>%b}yE+ctpdAo+dl5}7-f~xv6+}~xEz?o+jDayzcYWarQ zhShHngTg`0OrVJAWPg&gcxZR`poEe9HmNEKPF=I=b;F*wIAuIBwZ-ayLy{>?-_XsA zwIrD-mXp9f>0Cexmq|j7zj2Eo6wh`^xE5uy~DOL~UvN5ywz3IPEV4CbOIbhz>gN zZ-_z$`|o<^7Yt3X!1SsG{{^$(@_PP2oKMF)KQW2HnJ|KUTEIg z?iVPzJoG1W(UUL5)M6W&DYSyRWHKRjEyE_y?oS+YEZIBTF7oV4vg#q=@rYf=6N8Z= zx~Q9Ii^AvAtuO$y{03j3!bxy6!ONeV;)+4OMEv_s&(PZnV*aNubkq1O5?X}$cVn3F zLRfcx5to`(H*!kBUT3qI@JK2^gqb-)Bi#Q#?!Fv3nxI$C|4%_%#uv3O=g<-8$Qa(= z)gUv0s^&1Bq-V-v0PLQw1wdgdfQMm%f`m7wWX81EQMJ1yoSh`!+Eg2o>)v*CCTW2n z^J$fy1>Kc1J4?R1_T8VDQqOcAZXMAH?$B>roov?uNg3~%?o=slt(Ryayt*iB@r|Z! z1Lk#OEMttA!u3kDMeSb3k|LQi$C1y`fss!UOM&)oGvUZ3CU(^?1e;qss_TIz-60_W z`GyM>9;V3Qg%skj*&wDB6eE{GKMr`vLJJK2)BFWjqBg%F8$ETW3h)`v818Ysj`>}r zNrWO`7d>x9U~ira6|E?_Kl%pTcGwXP3Uqva)!p=i(%jey?AGm^y!{F5SEXMlVwmFA7@2#y`PH=Ai0e}6 zZC{6JIf?I&X_5UcvbD*mNL)LZgpSG;HwLbw*lrkPSw+<7b#H-U#mKpIF&=5p?vd;V zaqcXKgg=qxutQ_T?Uy3s4QbC8CipOC`BDz1%AiBf zY4*^E{&xZ2&OAt~oCxE=#% zV6`hCU1!t2z?@`49!%DlkT(myt@wcz>e}YgF!T%YZ3>)?iBG_uz?i{wx#{HDC?!fv z-<;=`l4v#?ZVKs)-lB;_!#3l-9(J7H=>M?ci+r5BB;2MtN~m z&dB?s024+x_-6YDJrqqLXX*rUZ|p3?Xj1U!qJN{!0!W1pmL<|?|71|mSzzOb-CI$b z1+>$p{G`emr$x9s56FtzTEco{AC|n_qHr?=dp#f-Q*r2Wp9NFx!Hpyd%S^_luL{u= zFg8Y1wi8BRz}m(kRrbh*$<&N2mvNBXFsy{-B z>0FY~*2y)((XWLpL*7WRsdkT~GpG`yhdf$gjhag%Aa z+TMUjS)CKR*L7s4*x@Bc#WX_vqnE|+;{Zm`Mm6lq2->~tgrF6)J9m^;&}K1<%L=kf zG4-;7c9)~Xr9VnTKhB3XW$x*TCZkqXkX`trpmsB8w<1dlX3$2g{430$fb;o;Fkxue zt+f0XBCM5o=MRh_2heA?gn#Dp1iFf@YzA2Y~6tYMfzhMAWcWbnSs zAY1kNFoO&;br^wW+oBdCGss{aZj$0{b4)UWu(g-QZCk${W{?rb821yE9(0*OMqKq_ z2JP`}+V#W~WDw~b&)=4+(8|I-P*xsnPyL_HtcXa&1v@tPR3PYyCH!Bso$r7=E?X1Rc zj~lN3IKdu|+wb8nkbzKG3zkN|k4vg$BG=fSKu%4ut)ngrjL2u0iav?4#) z38owaF8GCI8jDT87lNA~WdZzR!JS$;Osr!^KMRt^B229XE37BhCUrQxL0-nn45DHaIuJ zrc6!d1{DSL@T+t>BT7|B<{*jDNMjA^lR)3|d_ORtx8Mgdm!CgSWy#zA@CkZ3jM~cN zu(38B2>=CQM6PD9T)m+{p>o8!Ch@ZG8aUTMc9*Oe=?n&Sm7!9UBb22K0@Gnw8 zxw=J30-duh^@iWSz!!vIVaOMg0G%FZ`U(?i_HiOYPGIZ0z7DVe%i4)NI=tc`L#ag-SQgk9`yq%|r*C@EOT5erWw z3RXL=@(rI)2*dO}oBkd;us<-dMDf-qWnA3@9&deOnhM%qm_`q) z-8C+pCtTjVdt6MIbOKQU0C^z{VeboDbB3Yzhpm}z>xPi~te=b)*aL5h>JPTnL)ABE zZww(G$&2)-a!cO8^z3^Q7{!{%f=~@tQw|6Y@Xopd3_La-;e`(**8_{v00v>jG3q9> zv-~ZSMx?YcC*-JstwhXMI@nwcpr=dCvzRY5`q1+%p4#U9gf^j5!#1DL2eNg9EsX|3VV1A${7}x^#uR~l`usGjvbrJR?Ka%81 zJSrNZ`ad_OF^7bWq$&B07Uq<6o^rHWNRO)wn+9P-szH*iV^YsL(s$IS^c_jeqFLbU=&0^mYxgx0nH*)wJ=P-ZQGXewb->3SK9-1j^SLn(u=T{_NCz>iIE! z{{>&5^9^4@4Z>GgLws%lp6}8RVpAN`3ndT>;DG0w2V$kfm*J6GXRm}OQb>P>k2SBu zphNSBCyEG2zc5ga`2I%UG7U6}zQm#$rNh_OIL6-)BfU=#1d#n3?`?Kcp>xKL2acl2 zzad#3#1+DRCY{uJVlqt%ztFF>vh!&AdM6*n*R}keC(?5}=#fk=eWvn(Q7&x<1~FDf zNKJq-QrJ1L4x0s0KWO)TiKP)a%*4R0$s|6?az+~XUL5owP?S;LGn{30yIWKo* zQEf}VN!30!ylwJd>*0n~(6cpeN6%au-*Ys1_x!Fp(g9B_4UJ}Ank)5t5 zsO)a{dJbsjpc$OCBjTP7ahu6q0ToHCkj$6cY7u~Fk}_K>nJ;A#CEE@xlA7~=f)RN9 z!q9dLmhc=kEW`v7YN5iKXA1St7Uv)W5rQ4ZU&Mf;FHDo6!5b!P*l-mQl1;g8NVm;(EaQ1bMbLWw}d94Rjf!^+EScM^kmYvGi5A9k^~H z{3RK(2}(5D6~93Y(!|WEtN;drSfexOqN8yPvHu_mhq#B?`ujkjAeB^+?;DZd{|vL1Kdwx7L8=q&40+Pc*$T6o^4T)nZ-L zO7~cgZHIMHgXg%S@JO*9X?5N3`GznX%Wr;}!XVgR7&jBoTfZ?gAvXM?$GHB1xZd$a z{R0!<_LqJmtS5MTAci6!NISq35jU`mYy&-^8i;9_8m@0sJZ+8OYqjV{A)I#XgKZ^T znQ(O{ypaCM^KECtm@b#$n$!bjXlDs3I5wJtB8*n9|I_;jaFejoBups|Kbu~lXC3`* z01w5Tw!RcVZ2RcP+456Xb{28LdD@Vi+VCS7ff8F}W&Fcd!wWsh7hOiqQ zzZl;}MEeY%@b`bt1pNtL0Ds+==zPEz(T5Ogt|4A~AYVE**0;pL#(l zTM21v!gOOgC?Zq?pCD3PfEza%ujGVHLSJlAdl-S`CvQICZk#6;t%G0{OHX}#!$KAy zJ!h3wwCrLPhqWsKiVppCr&IsHp=2L|)4C7QdBGc?1DuqH@__T zX`EkTnZ+;nE|6dP)M;eq*Uyb8jA`{D4&)HJPP5^cyI03OA3LnzT5Gf+og2)zOMZ%b zbgfDa@+VyN8gz?Qy4P|1Rrl#P9X@=5&JTm(bFDNPplazNr7#m&DlcModt!!U2@Q=E zia{nxwxlyPdyand^-lDzGtL@o)4HUiw$JXx9qEUqS(1Ice}o&=_WNfgI}r$7See2V z+k3&*{mb9j(ikzIj)r3pBxUgnyAzg2v1d(6!3y;HAiM$N=a$k07IPGULHxyx?nZcO z%6Ss@yMXxjWGbl-(JP>`bY}H$)uF*)R+os9nlBl1PlN zJLlk(LBw*Wo*}r<2h+HnYg3xYjbaN|G=|;@E3mV*E?E&m2^UFZn`S-|TCt;A?k8xt zpI^{Qrg$N7lsi#gRiutXrmqW}N{9{wt1)1wbZRXz{glq}9t7<8Q?+_(sXP~N_^{AS_(mbQ7)t%3`>J!t^Ne9$ixQ=G0LpWzb=1C;>`=aoqsgjsNt~ zfBfUFA^G5(L4Ej#g?)nYgcd-*5K9!_h(**W&YE{!j%%0t%R72Gw&Z})u`F;gzmATz z=}EnGtPL&*4GCB0Zf=Y@1;3h1y&kaY`Ik*&7qNDGd}38;AR5FfQv5W?fH@@xr$4UO zrDEX&{p-@PFogCMIu=HF4EUHr9MZZ98UOb~TsqdCGH^o2@?~@#%a_@8tZNq-l51OL z8$2mLOpV7k$zOQT1hR~KzJQLDKonIN0+Xu_3aT*)z~hT7=Kr3FJe*Kv z@c21WRdn2v^gw;8-C}-?A{ail$rPHw{HSkc>D55IE=1B54D+4XGWES#QfDIf7%(h* zQtz+aehRwHO}Jz#M>et`+ZUF(opQl~0~9rLQ#S|lTL zEB6KG;yL1h(Vd0P$Spb*$O&dkTWVxAv)+eQHbgG{$lUCL3R>8^9U$hgx%>Wr^@qU^ zOfDKY19i{2NW6xE61mtKkScC{IndpBolnA^@=Ru&Hb25(c8j^OvyfdW58zBGoa~xD zSbn{#h}7awmdZ?Kr%b}yqP_1Y5*O4eD~UUxES2o3vZj4B4gwBUYreiYNK7*LGaP%* zi7{I!p~D-9iJWe>audNBb!>jvKwN^@tS$lS<3Kix0~vYQ^>s7G>$u3xx+y;CaU<{T zdV<$SFo;HX068xycw!C%XtxRSAD){&M4|nL;~S1CRViQQ{YmX_%#@;j-ul2;8FSvx z*_qnMCnnzVW&H~iUyrvwFgdY((7_~L=h#xI*>i^Qrc6}^{N~+tlQ@G9P^LB1T-$Ux zFf~>?K_OikN!gX*orGjJz^fD1T_adQl(ChY+x8_ytpj#_h}k`Ps>c(H3L&rZfxuu` zpZZma71Hq0+k2CW(oZufoma!Zuf-+i^+DIlOO{cM{x!@?&LKAjUt0*P9C*^T!B<`% zKcXy98H~IP_H$zz73{a|bDcg><;y6%M%|_?zM`$nJ6X;nNJy+V7g@?;y2LsPGG+dP zifmeG5rn)`tpU&)|%3f3os%Ns6 z6`RXiVVgEr6f5&n-U$e`y!QPB%YNXTR;8yc8um99PoGrfhFIKY7G=2Lp$Ox_NENNp zebbURMS5O}{NQ@j9VU!+J+V~*sfKz~tIq?`shn8nr^+do6V>6GsXgQrg|jxfH)rh_ zYnGw^W9E+Y8(F#ms$>+NW}rLm`H0GE{>|ndn8J$IS@(m=YNK%Lkwo zl}w{uch2qP9m7AtCn`-3JCm2VY;cf}5G6b?tQR+Q{TqeD1aBn{!~UXl6;(+9vtCuw zFD~Urqp-u#CKfLqTn`MM@P${6u-|?T^l*PnVo-Id$s8&}#omg>)SW^ClU#P7_V5$K zizg_HEGG=dRsN}Z8x@V>&L&*xEG(KVDmz*ODqhuFudtOU*8^oSwq9x!YvUM;zfcq> zE>1xb@I`(_hx7Ilb%9w-`zta8^JLmzVPXPGLyI6-{>>B-FYkvqP4==|NmSb-ot>A^ zQE+;4qR%Tc$?eX{zkZXvmNA9c%8k=K0;OlH4a z2gM|DpC=dwb9D;^$MUUiEsq{b-SSjpSGNv)zL9<@S60ocm%lNg6vbN~m{6`P8GDaE zG3PBU>tC4odc5_433V$lRJRt~mD074Qao;5%P_fyu4Nz`9@MoAoa&o9&HPIJFcT}rGF1)%g80&ddA9O(}J^BiL+D6)?&f0P`2ouJ<8T< z4AmWF%a`^jTTiN139wBit>CIS9GyD7hzjI+k7eD8%Ln-k)30ZOX?|fk5t*caes{{9;a&Ndd-VhgRU)Js*O_fUjzj){> zmHz%OlF%|}9?jAZ#_qoWeA^ls?YpzV@abhcOp#08>Am}_C`VM_B_E> ziu{FQFgOwL)*J8<;;Yslx@EMx!zc@2SQMBBd#4`^^u`x4qdYPCm5iGk{0bHWQ3Q;m zY4O;xxfLWlVEL>QYeQEj!gXn`C19)$l3%YUkl<;*X#lhhd=R$Zd-;>L51XY~XXu6( ztX^Cf1{Moo5J1UVd${%ivr}xNzLiZ5TQBa$oA%kHAJ(4vi-4DAfHE1`q+!7mASJ-v zz|8F~jc&~MqM&RNFrc}zxD{w$^@T+Q5f`~@vCs)mpwbLBf`h_c8*J3HjVY@0_r^fd zWi&Nj7Xh%^$OR9U+`2Gutl<}ieZ!0cebNSPRW_y%s@Nymh+%^|m|}p5mjJm?_kU4^ z3UJEYYW-LXMk{y$@o;cv&4kM!FXzGmHIlSrc>CdXarcD*C1I8pV!&7I30{5W_G28X zuZ@9n;BXFtU>Cj%=PRc<0>;8DlCVEfpd8>AK(?ikDO&cGDJW2hQDChzsg}2bFL+1@ z1p=4z@IKb29>#=#FjD01VkucYjq+p>hIFuMV_3LEy9QI>!X++KMS)?!So~Tz#j^ba zcI}i147m=j3JdgsjXIM63BsA~JZv%ZN|CT|#iTEHZnT_nyM%6S|_+H}p~H?UYJ^7#JfrkpXI z&1iSQw(vc}um=SVonV|szK+LL^b2u-&Z&>;iP^~YX;HS#91D?na1BrmCOT}n$JEhh z3grRO(3USaW^nDsqVzM4;TOOlwK9VZ6tNeXI5qAs_M5&GY#$Vds-7utjE?Z@%vdA~E>Bo1^yd zeWyWj@qN!lpCI3NBiW0qd+wZ9@O?LB;PC`vxWV_`icrp*@4Fc_7vJ~NcnFH`d+Cxj z;(J51o9_b=eBTT5GMw-26I5c!_uX}x;QO8zrL6eAQ!x#$#zc>rj5pFlwYJ@u7vJ~P zVj;-)J!SHk_`X}(^a{QYF!=sphBh>djluUl;j+lZ_nqZ)gYRvBvESnA&f2dR-`hO* z^?>X4Vtee(y4l{=I<3R@hqB4`2X?mqMf=ztYHPvuhR7G!ce>Yu>pN>oUtHf=tTed3 zn?@O29c1cUAC#^C(;^S9H%{sE=6YMnbhy5=K1^`^;Z^7Q=bXs&EW&^d?FI7;4Dv5q zx2u=r@Kg!n-+9QM>%1XBUa)a6$%Wiw`_qSvmMl%VckNH271FzmOmPD1_yOd~uqPhK z*gnvp#^DF++^7$XaKFVw+}|FIS6CU zT4>Up{aBUl)l?Q0kM0w81$GQ$baM@sDUn%A(lll|sBG zXGz}e1pQ;rdeReQk2a-WIQul0EBb#A=381hvRiRM)QVOLrVW=qYnJWWb7Syw?>Ua9 z8r!AQ2_hMdd`auO^Q$m*a^{U@kYbhxplyz`+-Fg!?QvT1!ohi>XPtMxFeP+Qt$rgv zMHNjy?1R{lDwnMJmJ_uChal9~M+-ibyQ|(}W<$ zmOBcvACE{NcEsyKj6iJJM#K0PKRI=94A6bi$*=Mb*j=}k$|RlV%Q8X)@nJjJqO9v! zz%my7!qCEJeWUnR`C-L{?-Gx=zu$~wpkCN@6~3^vaH!b^c*_Y>K@FR(V>Sc-rqIE` zQE}QHAykmVaKVc1{}xjZp@bSrx8N!z$h7_wqIXhy)_KLXNL0K^Vc@$lHHTWR#7HN9 z;$YZa!wP=gRlJl%a`J~T@euq?R`yI~eoxe(2cu_FO80|WL& z!i5DrFsAHA&89>c<2pw6_~m*=X!|%wur5i88wGq0#g0yML`2<3qaz@@L;UOa-teGt z5A%QFf*LG1>`q79)86E5RrQz)qs;K@txwE(YlpbkG3UQ?D9SORrNvvH7=Oz~y&7$Z zG-W^A4VWSf7Lb)ad0?5yU76ZJn%V-pC8nP(Mwxca>N8ox%tOd_h(COoV~_A%^NALM1f;%e7?mP*%eoyGsqgg96}54JLFhsRA+7m~vH}s* z=&$@n9Gc#p`pG9IUW)bn1LL2LmwqD-l?R4YzB^aYseFq`50&#W)S>b%Cx}z|9@Kme zmG41sj#K$Qraq{AO9ePoz851oPUU-RkfdQFgUb~s!dUHo0(w`Fxsg`CseC3kcSz+k z>=IJBtz|tl{e7)&Q28&4CY9&HP}qNQQlUtY;-}(-vT31*C+T45Go48F3zw;ZzHRiG zPLH6;XNrysl0Fk~`6uDC^+Hwng)URn?bbFmd~rxJxX*M}c?sk*MTy^Q&8=?jc$!2 z!!*Efm>vurrn4xTLx<@sa({=z)W+c2hdE49<_|(=YIhguJ(T_Fu|t2Nb$XiRF;&Zp z%VTPbO=X@c?xAaXOb_fH(_fU2?QtKfyG((>ROvEB#XBhCbeXD%)#WlpwW0hKE>l&l za@!1Lp~FiPV726fK)f!5b&ocf*UA4tG-S74uY4fv z?iXPH><8k!!dhm3VdBFv_z#2zZ~06MdE5647qdgIDLSTR=4@S<0xqo5Yib3512(}> zYf}!2VLF~bN-bd9dV?+EUl$Fs%IaBo10D3<7&v_1FtYmrTm0SN1b8b+p<2xcBFsv! z6G?!8?HH<)_OW2Rz*L#@1g&`NhkbG~$Yh`NHy8sojESQ<6PL8ok6n4C5YxhC{)0Ko z$-;zxVHz$Syso$Yz@VPLVJ4E{-x!3eF}4jtqa&nFG!z6(M~uza*x!D2+5_l~kc~o9 z=Yq8su=apIp(hqP58?+h_8GO)Fa&h91U;qAC9L)hr>~0s6c=12;eSyesIu{D-Wz8D znja;4+1}>Pw7WcF4Ht5OG!Xj{|-p|4SQq@Wth!2AZC=VzY1CDJvn9hWiL_N+n zek^@}WWh{=gw#gR4bqO%q*%=Bj9y6%nTdS*7e;{hkb0P*r5i}o!8Wi9GqiRBk@J_S zM%BC{A(KIR!ypqaPl}JioG{-S+-~~j=9wx`86F0V50*>@@y!jdUF<(Guit`zvY|CD zNS*R8TFTrWf$=0dVVE@Qrw;=cO?s!nMcH0Rbt54ubF&j{S!tL7elrU*kNW{AkANXA zSmDPB^_%%;Nk;<&Ohx`CNC&CE)sL3uq~e%3C@fJALg5n=2SvQ~iK&~NOpqdoy<(0V zMvJX#xXI2O*CAF_YUGRqDHSeAt??D+u?DCG9p!!iu_b^-JED_BDTXYsN{rGen|_je zrhTVI`u#x(q>hP1p33&CMjRbs3dg)|3|bTdUX2_H9`3NN-!S@B*oQjY)yAS0d$y## zA;LwvSMOY3qkO$E#EF|s-I=3X*-P{dFL-Cfdw2s$HHJVo;$UtmS=mYnA1)uL-0uGx zoC}!`NnTxCn|Vr25;AXLAnj}G*qNT0)E-O-xqmp2rzF`AuUDOU5T^sO00g-kzn>1t zVz`K*@Epa=nBo`b>?rz$IEeeCMD!`Ch{<_J;fD;H&)nM$6v++RX?fY za6ow92d2}qKha100wkdYw;Mk(oSR1D7;rLm7le(8*Uq2S-9x|TrQ|q7Hjmw6=|5$D zU9vGSVYeJiQPxCGXVvWf0zcW}Ckq#LKA;?#QsuE%DgWL*$DG{@jH&Vj zwRy~p%Q8J$%sNpnR1|6fJVkj3@};Deoknp+X?tsBmFv4V<(;d4yr{?BBQfYuEO9Lb9vPG&%!w^n}ISnm;f{ zPw=)s4=kJctmBih5pxbFbU>z%Df@21EPq zc7*tW?M{F}hCpf>R6_b%YN7MDNO=UNWyX^eGZHE@keH^aALOkYW4WZY8Omv0m+cIu zeHct32>c0lFXzfAy>!X^5mVT;i$h!?;_6_knS*XQKSdy=V&7CUL;vYifv;2IV5t9e z7JUf)$Ka{cllo7mizO7GZg-qq`$5K4`MR{9Za2QaLiw>2tR)o$3(=bF2v>fPa&>7x zowX>b;`Fv;W-Ij#caHTxR!<7`$CB2z^scO2$S&^!^&iWnE3#5B`K|-;qz9oI>EC=T_}rK0<_)e`Z=`W5zJE^9KABeQ$7r8b^sOA^ zS+od#j0WZ`<&gIFiHSoq-ueUMzS}-UE%M#k*l5NpiOq5mzAo3TO>e~^YKw7sYO#3v zdHO7#4e`JXrvS2j-x>wn7vZ=?j++*^Dxay#T>&W`a2!8R(qqa^W==q*bIDR}vRFK$ zPj5pwQLvfy!w*vWX)QvXDOGZ17Aiy6Aqq!+NJ=ERU6RsLoR-MHPbI<1gjH67xjr*x zC-W)`Ejthb*H~sO$<=|C+586Yx;?aug6=Tn9>yv zJQ(=MBVW+wiQFdEFu*CV{FcPTa~*|E$s#;F^~UtWb^C5)IXWsc4&Bc&m2t?;SklB$ z^+``4i6ke7zd#8!$I*{y`i}oqcbJ$qg;2cWa8%4u@kUQH5ZVYwYTtz2b;6+p?J8dO zi@xCwzIzxok}YZ%}SX<$LP3< zjf^-bZC$k{--As&eLJQ87|f?I;zPR0po_R?PX`^1%2TIKfmTIOSvPri$m!X1Y+6}l zdiCHx1iYmsgE<2(T60HOIf=>#{2(6u;F1f3$=)`kCNZ=$hwe5ZhLq&sU|EvKh9T>% z$Zv^EjGr4*nT$G)EJcFM$W(`0m+yBB*W?EU#z+c%@TRz}{3?W*`Z;u8!=kkrd$nX& zEoO?@4G;aF2k7WS0(vnMw>PaEtO`&=p8EO10GGCDd$MUrFR)~y>)HRg!xV9)U zv+TKv-%NpX{nsrYBzv~6OEEkcT0Tg>`wGLyGV8$1VfYMJTV(j4M!ReSWKb`CGH)V=cr4(70YraP~~^0C#f3S}a*$q8DP&w<_Y`9<=`IFx}F zmJcw}|a0bW)?9X<1uSF0ZOQFOep)+gRM>QZr}a{WI4yXL$r7-f&jp zhSTOWkIqG#Gmi*4^-yY=1{dk$T(u=J49ZwnX{yS%D|$MX^15mizLG0CWQ&cQaw$|$wDqAQH@+xX_KgibjYIQK}AZX zW!jp_Tg5z1-jq37W)QBF%qJngJco5l31WoXFtY^arEEW8kW;JVOpkD**&RV1!s^(a zJ;05&vU;K^1+hLt+>`BquP2&37z}Y+YM|#nd72|Lg+0WLM#7yU%v1$7KFlY=?TAp% zEMKtBKQV_a_0k6-!rMqv9A6RMMl*A{TBoNuN*H@KZ4WCx6{WHsS2$M7!_|y$&!)&g zJh4v>SJ8%uE9v1tgHrt_IH4=Xs{nUwd$`Vj5dw*0D6fP22jZAY(poH+b}()##7iHD z^NwS}wuxUkdl6rXrH|kFz+7Gd6gDqkmu`l}hxtCb4u-m!!3dv9Ih#y1am{S9y=Q1< z)*hfJxdfTTQ-)}$W^-=prBco2O0p)Yn$4Bn*pzB!UGTaLLQ~Dq1#h}&Sl#nH`Rr7D z1^?S&YGN>(^+Yc1Y}e2Wp`BTA@cMPMv)z@05bD|Pk#r#S%rT*!?OQk7CF1U;n^~#& zaRNXXvH}!R&M<}RbXy8so%8R#V&i^d4td+Dn^`jfu7T@jR^NXp+n``mlk{imW(RiN zEHKi|&~8y7I^^oVzmeVXx}6e+VmHVdUa~)=dCk=n9CF>vCZ4Ycc9t?H)oe?y9o202 zNDhK(=Jf2US-_#1Jt-cWBcsPPGecC_QC%A}{oFg4;~nbEK~2Ao}3w) zS&+0pF?akz64T7EwzZb-<;_J*sDOA`Ux!IUH+xPC+-_xylTsN`!1Tf-oWEA3dbyKl z@m46v7;I;CvE>|jQ(Ie>MtCwco26{G2VGh*`f}-I)(5gkaG5Y)?zYNJ%w)YKsWEgt z(G*TWmE>k^4tef-Ks+$Pe%AOfwI`;9C1Se3N`tGHS&7kVdV>*Ax~zT%o}&*rNJyY&|f=#tK6Q$*lWs195dc{~^%UaKhmvmj#}Fz4EIRcSFT zkZFwFMzG{Gh5Hr5g>)@Ln$pT$&O^i&-mnY{X0TztpuK{WPP|dTr)6@Jc*O&7vE|JY z;KdVcZKL^<)bnDS3;k6~m)&bks;;@%?h2ec+qLnPr}}}_^W?&48Z6MCl>q|SwItR? z1aVz?=?i6!Azn;Fv_aBal%;sBq@-3he+1nm#n{&%eRo$*s&hcr!MsFA>#{y=t)D@5 z+aDx2#S>BomClPDcl9P{gz;*MNmR%e+{n`kD2OYC@YLO<3QZTH%2CAH5cYxQX$~T4 zPlpl=r4C@=%vhbU@ zFu`RL)lcp;&gANvk?l2u2e@+tiM+%$g*!{(;MceobCrsNv%H*375aX(R@ zZ{rtGgbm^m@Kyrej3FR1$EuoZ&`BA1#%84_uas)}kcKFn!+=)uhFMuvHTz1RY~q2U ze8gG-c2k(tR`lKOP zTMOyLg?+*Bi8%BB%sg_S?Za6{aC!D_3HM?qG|Jj69RMjy93HY93R{zF;PhS@SePSq1m=W5XIky-?xp17PYN^D$vgdx% ziiig?5<1Th*df{OFJzu+K7!0*1~I^ZV^WUkdO`1-==TqF?)4#-+QcoJ$;K@oemmsl z9En*zo}j&kUszhEsc&Sb;OV2lUTB>=s|x+viEqyt)H$nygPoUp4O5vihlw_?6i3&K z5j~5$TN+mr8z2`)wd8^=deuoAjML%gI8U^tHwMSepHtT%BnZ0jn#>@??)So6w}9jK z9W$fIQ$uZzKM>tauh;#~2WFv>=8L7@M*kCY-qL!0BF?A%oevBtV82BDmMc;agy~>{cq^s$PTnU?H6r! z=Jj6Gt5EWJf=OnDBV$?0Ot`2J;u1ZBGXSqexGNd3XbeN&;L;ND`c2GAkxN{dBYhkr zxw~+=E2Q#*u78Q`)ie=GbzO=L+GS(=o4O}cp3f7sRQ(6WNYR(Dos?Soa~i*!M}CgTbfiPxvbMFZeQ@Z)hET5V7tWHZ7Aq*Cr zh!F;SHb)p84F0|!3==)m?}K3$ig`a6X5(S!j1vSFtfjl}@x+oo?FYl+6`Si|Sgg4m z2ZJxP9}Lf}P-QUOFDex&)D}!O3osBCZ&rM)u*XP40XG~Km9bp;k&N)YUfh;g!X>UY z=v$|lYZpASu4Ehx?7;I2Qw41N8@$YKZ}=!CtJ%gVS7q9Vrm?*5gHM!**8@TGWbnce ztRVbEWtAMWfjCiC^D~=ADL-r&2jw57oyrtDA?>%J`OqSJExjQNMGS^ol6ayZm##R^ zd*REW?j}Z=2-TM=W@*fY3OLr%F@3QFQK?dpvHSc+;~An{r_w^~BgKQ2H*hv1i_WLf zhL@l-cN|V{-UPxz7+B6q;onI$LQ!nQK1wGH)*2RXMMUzDVI*)q=u&BU?0I3sXv{ku zi>D1SFQ$$SQaod*mLIkH6H_&~c&RQK8iDC7C0JI}kLqj?aABUq!N;ZC(aiZ_h8N_3 zS)BSh6Df`<^y@87b;EtAP5n6t=b+61c5v7()TIv(?!+AtlUiJ%T}-^_79LFjr;7BvE1P!!db3o25GISCq#T}N#_=EuBE95 z`K~RULj=A=ea-+59~KFWn8N}2kHhTgoUtEW7uz8e=T3F+S&UtTCaeu=oc!hPgaFN5>3A`ap0cpSfgJIb) z{=lT9LZ6PY-<_RPv1YQ?3EO<*H6{HzUac# zL!#Dl;quL*U)zZS(`2*fSQxu$`k*R$SFQ&TDB%wn6Y2Xhl;H`_q_)Fzkkh3=NZ~ zN@>8oxlFii=Yd&?*;>ekPr_e>Ds%zjdAren{*5ABe0yo11v5x!nWJ3;0O=q}4Fl|h zT+IO$XG+JgtE6#AhfH{2kh6n@LmfH0dcuV(8buj|ofxLpC9%ev|F zH=WSf4{HlU#-f&4>GUJ_V*gy$57vu&o(A8nIEZ{L{0g&M>p<3xV_rAL8Vvzll|Kt9 zG++@xN@>r=kl0|2fI8+m_Z#c-bsa*Q>##IxipIu5evL;ma{a8B9?s;qaGk0y2=M?+ z9V`}D?n8dEN$@4G=j80;iOpFpwsur?J$zH*cOle|VVY)ESK~KXt{(OFs2LQW#_HOE zIENgwrk~TaFL1s=Nm@l3;eejIkV?!0lEav|rpNPhvT|a-&(|rjeoo+-^ie%ACKZmw zVt92lEya=%WQ}z9kdk8$fE{tLSk(H!Tb)}SV@$*?r7mr33F73&T3B|eMbg3%q*e{7UVNc&OQN$fjj2%Bv#4Wk%JR8nm zkh5uOv7wt`jF_nyRF36f(3e3uE6Z-?41 z*2+c5Xm5oXf}qvJSAb`F)Wq{UZ#Adh0VC(8h=Q_1VSf`G~8vyTURV^k(`jU07 z4z0AwK9R7}&PZ2656sw>tPZ)4jbp#sR%n4w=!@W__ZQzXeY|&F{c2a1eGEdg6srFI zokyAW9-b6D^TTj@ez3>EEOwzsx&w~C#U|ng(^n-;rHlQ+5hzXkVs4_V&w-wX3Zu) za*8)DOj#!%@3LT$lCe{LEsHeZq)9Ic&Gt`SL#vphf5l#lIlljMVUU9Ux-rTT$W@Y` zZR-v0qIdN=wUlC=l%3=;!~tDn*j1{3cXBCWiRyKrbd<8h7)))zD%2zLz)`BoYW+HV zZ6SKT4_}bSxSu#C%VPFi)eGtrl@d?VoS22UC6SXlgm1cslxzCUEUPjC^SUs(24-Px za#Ow>4qO_Pm`Bzz;7#FeoLN#jsZ}Y6g4&WzbcAx{8(z=?q-ttPg_V)ZMy|(=X$zimSk2mkD(b}1z`!La3nzh<)YE~g5_EP6LB@U7u3nT` zabn!G`=*I{({95J3?9*q@OI~{a@b1vcfE70^9NJl>OZ*;)Ll9#cPl>fiQNwlt-$#*s)XLO|eWacT=dAT%WPv=*ZWX0t|Gg zSXvg$`EaLLkJ>ekJEdDOnrl5r<+RR<4=7;Q6CQozF=%-<{dT5UqD>penPO=+YaC}v zpYmq8FU1i1;nNRH_)_dgYCXOb%c~kId@0KPxqK-dX+z;k!7S<}97fgq!Pa{--GcY+NYDkVIepiLo6FV{F!6^FCRvaU?Aa53+UMpNJrvwa^^*v9`0$;-Jgr-b+2f zWQ+*1_2o`MkF#}lvx_*JWvNxg%W)7cc>7P7%@K$H7A`M5FeTiELrIBp<3flUTkfwD zW$Uab6H&I#YePqh>~hO-tV|2J!Y77IL=_g?lN=mjHubZOFq^e@Z5~hGH|$}ygJ~L- zlnRB}>=!w253}{HptHhkHYy5Syxt^jJ-SaNMC*+Ee@-)u$MY@g`eP?XKKawuNGOihJ)7-S33V{C!Y3|3aGd9eUu zrs^LdHW1fkMlH8ivIqmUGp`VvQcEMmhMJdO5n(g(-qL*9b_QrteYO@cA9t}Wd3uP! z-Q=0l|HnTWSnU72LMp0u6g#wpoV4z)ls^oyp10zUlC$MI zX~*ILAyi{L${Jcg2YkMf6`gUrLXAfM#K<~1Fa1V@g@caH`%g@~qy_y05nqmXeq+My zISAT42Di)Z8QHaJw|h{c@+hQ$=as*q4waYTW0-y!K7)me!tgP8J`5jAr*6mavAwOA zmTh}KAC?cAcHa-wI6zAyY2}CEGm6YJ89oM&%kUXx+-A?AUN?Jw(X$eMn?jQAK(R^q3WMa@(Y2oLKejC6EmlMM zxzQq<39}zX>b?7qt^VQG6KSx3;x2I&S8+LE)!3@?GO+m}ZQ0-YJqa0F$1+>F~Y#E%?=;1)Ns*+U-d{ceP zsi>xk&-Sfk1UisyQHH`PDTeGAJl=BBx*q}MGqv^67{e(DxP*!nZ6vzsuOxSkOLDAVgibKkh-c%AwMik_v^^qdysAq&pms4Zr@~f zOfj=$w)>CAEKJ*x|5&}+!+&h>^zG#fNWTsTvI`-1ARpKr$iKwYEtJB6eCX00$e^?K z6%J&hsw$7ML`s#%If>Y1S;>LyB;pQaBaz-8{JJRD2}B19xPTC#a1Z&@{fX&r(5AY!so&N%iq0Nm zdq&1Lb)tD?fU8npV4C`sGt145ZTrN&ZEe)j&BxXraGTQ9whFep-=;Qd+SX%h+bxLV z8oQ>P%WZ6<#P2x9Hd@s0V{F@A?{*v8*zJ9cZSA#eZQG=>FIyWINZhp9XYbwEKCoNc z2ct=>5=lmr#8lU9ZVxy2b?7X4=wxmWlI3M?54ZSaZjZ~p<|UY8UDMvSW!dxC+k;dY z2f}VLL;bwi<4SQwrQ7TYu}j-rZ-d)v@Xjm2Ytk|dgWI^e`H7)7gbDN7FnK>Q#j`G0 z##ez<+1oaZew*7yU7m~u)oFKHoqoJ;9fNWH?-H>eK9G1!`49L?hP90<)caW5xviyt&h9Iq$E_;I_@7Cz%GfZTYUZ!EK_wlw)uoQBQ;W*I~c|q75BXZg8V^ z>N*DZ^t5*x+*Ueum)qc;O~DD2BDUbsk^KffFFZ_^$U0B3z-rjrD4Oaype!X?{r72a z8=Q~19pL#wajH{&P^)CP{>I2Ej(2{+VOUQ#&HW1#?`SFiK*V?Bo!^)+wGV=(HsEng z?a6AAm#IA!|C6OXS=08MDwV;4V`)#1`^wTb*bQal5EP}|x20{>aMNXKV_UBTVk3b% z@3V16q}reGa1?EAqbffuNxZFxw;R=m+xX|!^=yPky{+v-x^8X%B4!qFo$lXpYkSV} zOR;qf*mXWJFa`lP*>r0Xt~!RO#~Q|^X(_D!xfMY{iChmIC69d>Caq)3SpCo@;Ru;N zwhj$1WHBv77YEmr$?w8+UNImF&pEoR1b?E231{Ng3f^|lGSM43%tV#~-nt&QC=9w^ z2k;xYj=5H|*NCn>F;-ozsm!l_=M$5mV*{gH)?7%YgE{9YD%UuhET)wt8coLuTdPkc z+v`}J?C^SyUE6_t?6mndn%v`t+Ncx7VL1RkKVnCE{R-Mb*5U(hF1w!8$0DPdA~K@h zP)Q$|)dz#9_hdb<2zyK8zrxKvhH@Nl->i5Up+PUFdKo*I=EH$(E0ymfZ_CNZnD=C9 zvMc62m+U9Tym1{?=4e_s>@jc0c+7jU%2>p_A*c7(74x<+xH^dHxvaMwgwB+z-~&jx z&_}T#6peL+d;BWX7%=m};({#~FW8j^T zDht`xuzwxuvO2LhI8-6#7y0ov3#+@opTc=yB)KQuBgDKeOhNi2V2r)M&sO(hYG4_@ zb$ih{!A=uzc=N7)Y4Am~lqhUV_2%b^=dblILCwfi7=U4Q=Qm8--I`v|c5kWa8!%1Z zaHkPebO|R-PONw%u$FWo%%89$LcfXp(Ofku7gjAZ3BIu3V9N-88;DyY7B6g^6^{(V zAkJChvV>zYz75RaYURZ&b$XfsBQZMDN3Bi5ujVp4Te=G(li$Y$O0!xq*p4cIQJ@C8Q<%8RY5=ZyvO?Gsu2qbrt{ zT++xdxEx^Hd(nDJgRY_AhL-90jwdqR+QtpIZj+Tz#!}B({3;28ntDOUCU*Em z`q&$)eI{vn$%kdyi&;)Xn*(jC(z$C}(K3)h5hamuj9Gg3o;GK7^EFfFi8Zz zAm)Mg)S|4xUfjaEwcfhTQ75HQaIjy(dCZG^KPpWhNC0t^Xj*M!OwQn!55dPd+`*G*u%wm|mF(b>8Yrk~*Gw&XoD4 zYQ$05RJSfTB&kA)*~U4EA+s-(?v_ZA zIeSyZevjSbjflJ`nG+<@pLh7IxiX|R4`18$xft2Wi+3&Npc0WM*ZZy4;#|s8^}UeQ zTXguUzcLSF9Ilo*UE?ga=LfIr`2U9vHYT|s9apkZY0smORBb-U*QMQ>l~CT575zS~ zsSQ}(GSVUnN|P~NxAXL7_N@xdl(^d02Zv-WCzOi~1}gZzd{ z8nx<|TFieWg&5h@V@N^f@g3_F2Rs&X+)q^T>|p=-sYojeK;VrLih5GBY>ImJ?2PSh zEy`r>IIde|<~whQ-I%3AC9xFd95PIDq;0$5lA95}YjF-5mZN2cVkRwh}WXe1M*LJx(u3#~j1 zvV4Vl9McE|+9nFyDV|`mgTY8fobvP*wXLx!oDR;lXb+TGSIN_m}I-_#8}-|^7WgDOHx|0O8Y1U&I~!d4%Cp)HQjz?9@A1ruxJ z*e@yDRk}S2^`#WTM4PZTS0u9?OxHC?7av=tGD5uULPgsDpQQUw^_Q10&O3V8Pm1@F zRTo%kXod9&5A00|0uve(a)j5Pm;&AOfnjGpgI(xgtG9H?GCUV{=g{4lk(>YdK2GMX zx{avB`dwsQ6&^&phW)nj3DKUmc{i?|8HifSV-I4bX<-I=?{!;@*c<-=g!25xbAj>l ztwd$f^F-#WI1E`*K^)DSygDYf>52fLX7$(zskVG-pEkh$~HR@pi ztkya#qV>Fng&hbc>e>;xgS4FPV;iauE5`SB+_C4wvtplAK*|_79DAaSk!3draf~Dj zRK(FY8NrF8yB-_F;ei|{4ok+nw!R~2@gfZ>`qu+hmYXL@6^gw>8(pp!r;WD5bp_gJ zibEdSaLl0uwu$-+b11>z`UB&%p>k0{8(nh8X`?$QIB|5wf`T~uh@E2_^!1>OzV1O` zx!Xk;ee(uN5C(g{AHo>An-_iBeanL|o-PuEMQ1bSKW;dV}R;!>u=FXKX3ldIKF`!mDRpuG{6a8!YR3 zNo|zqy`Qi;u$NbFc(tkU>RB3y++A7KbMset^&)R6^aZ4YJ-m8>39p`|lJv`~XNf{v zYCSuCE59nddP9Yn+^c8vn8&MU6WyT*U}3txssCq&6Y14M-sr=tXX%zat?1Poq3%s3 zt_pUq-a*i-hab4SddMRdqj-7VEFq_iZ8iW3mEJaQKD>I!(3;m1OQC*w^{_P$uO2EH z`$4p>G&6ej4(MLJUzGo1PT{`2dO%!Wy}>T1;nlM|uq1U{d5p`w_w?$Gr6f)%^g*Vf z@&{$d6ZQOpc%?qPdd6??K*+0S?71(m-eF)*6?@D}4aF-}m{3|_S3XYAz(j!QV3cSQ zj!_1fKx2;p)ASn|3NXo@EgoPp*hP|J#(9Ja!v5GJB1UF?SljXbh%wE~yby($Ci`N> zf?9j<{s=M6>>(FDk=#mUh-s!JEx~_`m?rNhDa2&6Njhd2VwyRHPJz&FRuNh;m#x@-gqUoMw^c4LL-7!k zNss*qF+tasDuwp)qA!M+4(K7KUj#^^Y`VRgr_Lh#{*>&)I=3OSWis8OX&>lxvEOYH zEg5PPl8C4nLK-6R?3R{NEaLxl9z7DUQo~tN7%wObf?d{|IbxTBJ?#eL12u@NC=2ZJ z{Qttlw#Qo^n2s)eFXm)HYJOtQTfU@!V9wY5tq)9z9^-+bZ*Q{BJ*HWe54LpTlVj-R z<5{Q#F-6F@YI6OR>Kj-0G4YWGn0RiL%;w20htEK{TrvYQ1MTd39>=>8LO#d6NaOK< zZg3YWpSDC@b{-w&Mtx0BbkLH5xdkSjOIpMzNo z-j{3{BJWSs)sS0~k%>A3{{Bxksy{adUTnbc|J0pvBR89!@IFM}@WfS~ zjLSyd5J>;YgI@kO%pOx_c)WS{4OcqOvspn2KY@IyY^^=sN5C7-e50WU5n$viXR~+) z2Eq0*!z{K!*t${-^0&kGMupOFJ%KJ0fUScN`j`xUqwtMlsfIvU=Oi=ZHAY)UyD3=) zXEF4=Fh!ZIz>M<5!0-Nk3ocxIV@)oewFI(JG_4(6EqSn<;v2&jKxRL4x!JB7Gc5sQ zZL%GI=LsIH_yue5J{qdK;Tv#1zA;Ptho#iCQ9>%fl9_>d5D4@E#BLV$1F^ju_UlEr zsPMgsb%Cyn-E4vfzEC*>_A-0b3`aK_B)TUR)toGu#C^y;vdc!$P;b z!4~a+7wN6SpZmt394B@GBdYKvW0$5aNma@IY@QV`4Z9%6R#$dwJ@BUWdoYdNp>S>1 z=Fr$^5TCS9uq3tyWs`dIf{%LH1MHrLNn_W#de=``q=0wG*>-2dB49O&fex zIx%^ZGpnt<`uCx8N9CaOMqo07tB1QU3_PBgs&xoEU#a3=bDO+Dlrh9Gu!>lV)ycdI z9~l&Cb!+W(Etf>!?kAcFQ2YXDmc=yjjZPA(NyDsK?@c1%w+>vl3!%U)^j5Q9sMvEj za*u$hL*jg3?JDvyC{H#?g!|RW`?awpEhd2>Rkjy0&aHzmR-Zv8zn$D zxJfL|OjyFA!3u|P#~j!t5QNPQFPu|ZE-ZSbpe$^je55_Diu^tga1;(!#-^ym5y9XE zWU2KlH5+hB8e1#0!FoTGwfyX?l7rBSvRQ>V=Uh6ozuPUpc|r6%F$Skf<0so?!JH|d zC#9MdXQCK{ARC2ik>Q~@a5DJPd);q+VA^7=E!(}tIse3*w|rIq zz{J<%tq%n8`E^m8>UG)FcCOeZxOIHw3^IL z0Gv1&))b<8VZ*h=F@j*rz4%4S1rPbGNED4Fy zoTb^c9I6A9AvQgy^s<>RS3V{^9eJ=65+vT#OmXfq5i1W~1fwdzQLzm+Fc4l_%BdBi z-d|_8t8OJfF#ACe6=80`lH0~ExKvmu=qG<5aXTtBHNbU~%~RnJhS*-p0&^RIfyQh% z6P6Aqyn&Plz-9QfczEE}D2YUGSW7@$d~E^ajiPV(6PepVr$;u2W0JTO51KQW2vFG? zS_U?uAMAofEo%6F?j4xdjoE?eS{&Sl&$>-ZIr)VRP7tlh_4b`8;aUo;s`~pwr$+TSzVqr@QiKeorv)xSW)Fqm>Sjrh&!Q{!p7Std+ z;jJx(wGPPxwY9?VKa!{x$qJpElKQPn(LD%X$7x^4pe;C2P=9zugL! z$RBR$jj!KZ1H}ryxZs8#B5J$&V=_JsjQvV2A{*Q+0$inaE|O8rE<5bs>2RdIPMPMg zJzkcm&YzfEn#zM$X*E=9N!e!jNsYDB2re*5JCg!%c*&{J`GOFKO!NE%Lk&7V@QHz* z2~3s6Ho4l?=(LalE8r1I8nqsqXwi3umSX1k3f3*}fU%P0{&$fug#Eyc=z{ryT4g^_ z@sh(6uu*+%wlY?J?LM`S-eRe!d~_uHgr;q&q)7)kev}QuFHPIBGA~6NNt^wg>qzRn z4}Q4W*ZqJFNa@*#Q`E0Z&qlOJykPnwVj-gZT`stsZwM1=>}2A`pP1P7c6zzkdqHVAXCTw`?q40?2-WZ&*(zFrckn5%D4ax-9v=#gg zO&b9NX}P8%O}Y;*O&d;KsbP!zn=VCqcFE!pZAjAH_Y;zQVE67ulk3?=aYK5wDVw%- z?p&$_hXzPB`_Qnf*-xrALNCoYWpN?jWWE`caLg+(rI*QG@fjC9$A$Ty`Wam9vlw7$ z6PES)8GWY}WxM^4|L0fCTUr*=_iEjZPSU`%g=9$#W_47c{o^TXls1g2%GyYO^Jh4u z|JIB5z;G#XmFvy2sP^3h`{SvJa>wWA{gKDG{-55nVZB$6EiyKZ=bymT#L6YF|I|9@ zkNpdS`0oq9RM?yCI01c(0LdeMe{`uF0D-e0A${_f9(q4JLOp!e2RFEQIu zwnU&M+>dnrYxn7FG5zI$SMjD+!k{+3-5|5qEk!){_;G>CaVqEkJ~n^(*XUWa`L~&PE5?|vjYT2-2ANmg2J-*jur;@SgM+O$3YwO` z;qFQ4neodu8grsE`*PVGA~xq>7u_~seBcc>h-@P>ZC#gJ8uPDl8KTyJb)TZ&57c-8 z_j=4m#s`|Z+VJ+WVfIvCoyH-}$+kF)BVN%qXK{%dZ?$dq=|OiS5UQDM+H-94DTbYR#~rPDG-#s=kfe6nvNQy1nR>sTF!iet zX73X=>%_gK<{M|>{0?gy9B9f*W2!Y6b_@#mAjNwClPEv~E)HZ_r9UGm1_+83PT zj+HQr9RoYNiN9gB!l6)DQyIE;iK~Ma23v#Z-dJNja|{?1zEHy0@1kvD{J!A=+_>_z zqyVdSIi0e=BwcY_B*-CASUO>e2}Z&}!k9#1%D=Tzlki=gJRw0**s}wCXFCQSvuP`wmp&2OZ?)j=&|e6D z#ng?fqv4db^XA9o|o#O0L zlXPseif30ZZ5&Vhh6QEy1nz@2o%=eF8ZYTwM^^I z;0G!MN3ymu;H@F3RDVz|Fs~0AW(YLx9O!&Sr)}c#ORkJC& z)tH_%K9E&$)A%UYzdJ7x#p|6V39k=y7MzPAbY5bNAqG5LBsym0=n=9duyC!?zp7liy08Vs5O?-lm6!lP%vdwj|9t`cU0%NHoeP3iB-*X#umkk?yZHzE0am&FI z5QxwfCokHX{L~%2$vb++O%uIXG{rU95z41U82eInF8C9y4G_Q3c_;b$MrZO`jKZ#m z-h*`b+8W1H%1{3V79ry|B*2_n*wDl%ne(VpQN=w$U%_94^&jF1TM7NZ)KY~A5Cu;k zgDzoqA7lUey%}s6VCNOyK{oN^+W5N$YeDz=?kjunKRd#^TbW8ra^9WvN(#a zE(Wlm8+31^*e96KP|Z;tGbTZtAu>9o?Q@ZPtG(==w?n+L8gu`TzX?+ zGn{^UqaUfFYjR{(T|QL>QX8e;loO_BdKhqP5S00eiL=6A`arCu>{5IgOy*5!Q1I2b zk~jP&f#nk&;%QtWvf$rgd8|~uSl$*F3bg+p6|l(io4Z%YI7-OR`hlf_xlSJ|t1ZDH zAKzOD0F)H?xiD`|!cByvomi?3$~v^vu!;}S*&RcydVtJ4RMPii_q`=i#O_-Jw*EJF z?~NqeuCojJcNAr}Q9&)n@eiRQUOywZ)AKiaeS#t@>p3^t>Nv*PK#{p9K!Akodge?e zX6G-Kwb|LG{7R?<5xXkL z=?zEUEvfvTtAr~5qJqR2tIiy*_6R&~S`f!5v8F_m^69_p_`mtdrWBj8GVv5kdeMZK za^{2K^`G2C^A84T;5E{sBxteBC2pE3KF7eD2G69j)#9h~9_fG6Q6oilqOaIhf}R47 z2`nT0bg2mZAH|td%P*}YS`E1=FsR#M(Q@8E(#cxaxc-}t9v8hdBJPy2t)B3uVD-VW z2F7#DRz=Ak1~Z|=A}=rgKI73N=xU5Tu>Ydtn_e--O0!x-I+8@E=!L<<4F-fTNXcNi z>3McdyxYowfBrL+Q)wYkH9Wxo(LbO(CwG=ufb3YF7CjCg-N z2YkZvLpZVLeE39$0T|A0e(K1 zXqudkypo?kXZVJ28XMC&!51+#ls62n(O3)|==jBt)WYS98byX%DqTG}Q-npsg$dE- z5gjHyjnVIsN}On@8^+SGKG|u2NchJ7_+_~G1{kVymckZ`J-|EQ8Gxm%o;JLDo)!ll z^7GgTIM(FC1{vNk2xQ~lVSOWBm`vS)FE;Eh6bor`H%T#Y?|$Ae*vke|g%@k&LgKN> zFQCi@=73N^wHQy&$=?P8ggQj@KsQ1lNOE{GVlQaJn3)6YhEtW%i zE=M?3Nw$BCoHs)HjSU21JcO9e-or`u8S;VQn$vGuV3J_87~JoOJ7}r;BBLe{Da>UB z1<+*+>d3}HcRHcN_pox(bT=00jh^vp(79oNGf#~^jUODm2~T82O2VYhgP8&YF}MLZ z%7Ok^q6Bm>hB?GweAD61Zy016(@ZfOegwz8A-#F#FzC&XoD`fW)Km1lOxy#^qtij? zkzE)D3OAR!VJr>oZsoLq=d~A;B+J|ljo)Y~dVd>dis#ZpF-RKLHSGCN!hsR*$hFxR zy3*0t8isYuu9d?cYF5^II?}-T8lKZRk)LEZ^urPhD#vl*(nT(ZW}&5nuTACAO|b4i zFqa=5=}~`Aj4ohI0FEYnGa@%Ql1e9;AHDM@!8oH&CiM8x{BoK+LDuP+8#WIuaSqHZ zP;#p4WP$iLw`U%n{`FfPKnhL>7|e{0GPqeXglGzbeVNEWV{SY^0!XKtqj;@OILr|6 zB1?QSOnB{L;EOrq;`q+K;TT-57AOl0BEleWzTm42wga^wAHb8wl}<)TK8BiJSPYQd z{`daej~D2)QuTD zK2s%(!Nw<-IA{TLBme(l2k?ccK!wR%Mv-#hx!n_f-hdPooh~H63=5|K(ZYyri6-pc z#PPwifbYwROb1nhXgrS^?kQVaFv&&W5?oYW+zm(@?2kjvl)aLCQ=$>ZrY6zG&ui-` z>Cj;?OdnvOtLt7>V_qMmOy8;y>8FSz0A}*W z)qyvM$`(upwuu)7ElRz~o35whYC&!B`hhuy_Humz*FH^MW0-hi*SUwdVNU`*76xXI z{I!1pZeTe1{sg^vctkw}6a@#zTqCTEBG^+9$T@3{LhL1Q_q|B__tz@=`)AoTzTexx zFEB~K z(b6=}(b6*LBv2#;71-l9+Foq7XxUoOG9kp$z+p+iui3h}zFWog^;dl%)YTKfN3hxLwGM+=%pE)i=NZ zbt$dpsB>R{aCBg~aal2L^c#+zWX@ajAhg=fE&IWVYpUCA}tCd2(p7fewWb z{P-y>2ib&=b!N4#>)2=cRs8|EK~x?gPy>D-{-G~_L=Vg!Tw zn}gBYmz&OO&9nEBgJeo0p2?BdC>bUVe+h1~;sbrEA2@S|_Oa)tk8!sUq2d0Qi&hS2 z5Wg7a+H|f|)J=p-uM1%uXV)t-8jgF{nc?>A)+D`meZ(R#>8%P%s}gv@i<1<+oh_O% z%yu7iCGNUe0URdt-J*S4aJQzlOX37o@dj93iq*x*z52{_QonxhkFJOhNizF?<*oT2 zF?b|qBq0x#8YvJHp}g=XRW_HHwk`k1r% z?>bW5h9VNM<Ic7xIPPl5}wLN5N^qC72_c_7jYPTagZOFz`UI9dmujtt)z1bWF6yDQDcADBsw z^SXNdfw8A(L?H_#xcLjSdhKQX#GJ3MUZ0qr9PG6bQK3K^4oV9FxCxG4T1LuoA#j)h z!}0)qb@MM&f%Y4pEUTN4sT6zzn?>(Jk50*6o!80Q?;tG6AWIM$_itc zJ!FwTEtFP^>1;W~De`H}d&hdGT^NiBBpKX7Bq=8tz>2ckt1HawgRprmA6aS5>R1UV zMpn@qV_w3*Ee#kbUq(_IQ^2_UjznVxVUi}if2V&Gq&bvoK=1%e*F=NtAP=0=fF^GK zAVK`FzUTJ5v81IzA4z^qfU$lfPOa~7>gR*0Q)ZONf>VDX4t=gZpP13)3Cxjj-DLpe z$T<*Y>lmqcF834vpN+MkSs%yv*|l zb2h!06@OxST}NbN5Cku9ifch%26#>}t>l?_+u^XJspe!JwWdDJqcpCf!Zlz@xU>MS zT`+D?SiCd^NA=*DZ8A&@-k&x%c;M{K+WdkV$Gb)KcCBBKPxAc%AL~7Yv0?7cB9P$} z8Vhk2a#7hQW|7%Li!-Q-1HCo)02+f1R}1qJh9|Q}*%C+Bw?$cF@mDS$hN9D;*@8G{ zwloz-9E}lK?IIvnwMHq@BSHQA(R~1eE!MMxD&{*WW{M=N+O1*|bR&^LU4@A6f z1Ovv2QFS65BD~q~jp+5l^%$dAum~FCT$sq2cCWC{&}3}?eaePBP8f3!~rl@i05 zfhSHSLIbNMnIa&h+0f>uLq{1FlYgU{0^7;&^{x&evD3KQVK@V-OGVfRR3Z zG3W^%k}`+JkW@;BFKL{YXs?8jhG9B0a;fF;QNOKm$ttdWTIdQ86_=n_an~;{Es(eo zFvAb!IR(@5Zz2O++j=;Uo5(98VOkx8}s7bAZ@xa9au=VRz743oHjOpnVAdxdCETDU~ z%9I)8Ov&}B)i8fvY1j?32O~(X7Q>&DqA$1x2|n+W+Eo3JL;T_%PR)iow6{(FIh)>C zr^8eLAp=bC4{!jaCH_3y=WpxIE0=~CswV1me|ji$#Kmx{Yfi=Im%O9u+Q6YMs!4)- zUF(*IFdRh9qP)g}NZhm8H%v{PB4~CCdTHpJ)Har4n8%bolDMk;(y8DTOc%NX_ox86 z3!|(8_=RBFP1HuoYcZ%Etu>C3v^8`dlr->xmFXoo`z0J_l(Gv*{b^Jx6V@~D#&q$)jgfQ&90)6)28cl1wJHfSzk=%z_$}4z zu1W)VtdqaNb#D6IK0R0@Ol+hho5et#saq*M*pTUA(qI zfSB5~_|fpx0IEcRa`<6{!vch2xE2xN%;LgT^@KI4 zT7<3`E-q5KVb}IKm?I0N& zK@3XP&UxY+A!=0{lUqc~YOyT$Fa|T@4K#)&>23^vrLXTQ_J}()1h|NZ;gjz#DBvCQ<&t@Cb*u>i9IOR;Gq!B8dDZX(ufT0Bxy`1-s+YU3)70S zs_0iNe_*7fyP+?@Zo$qZc!A)f#1Y^C^{89 zgua|8;99k0H|Lra{}U7mcfs8mkTwPvNA{@uG9p;Pp<&F`qeMTJb;*v(;(T~f^`SZ3 zsx}eIYSDtoU?y=Z>dDCvWh2CvP;v}qg5I=;o9YGjDYgDD+I(rTlMd0C@|*X)Zz#tt zJrJUKLDg=EyVND`)-x6>!R@(tQ4Ke5G9}+PUAh&y2eMbu71)<+r8~mmpyU~P-$&)Q z;s{$5JoXfRe$UB@Cf!qlBFz5%XWQeAv1)~`7?WB;TJWhqm^$psu>i1@I&gazg0J<& zk`08hU2qzhdBawR#JhoS!uZ7~IB7`YSV8sUh~axR@tPMQedyG&;4#NtP4h`bB2`6xKA{$Qt^c(u4jI#R4Shq${0;JNllfSU@KbE)RjFbgX{7-0i4Fy zCNN)uem+P}WLT{RYYbQVcVTdIlV+|eDL3EIdI?f_)U%(_vgQL`k77TcyKY5z2~L50 zm?%xuU{ugUM8UuKj zE~9SObktz^UM&#&;t!^!I&%-7)~YPu)S1*TYrhEcMg6|{KQ}p;7W|ZM*3GSVakRQH zaTi)}#&nKgX}yLT;=tq;eYHc~Q!6gJ@FeZa{ui zvbLu#$A_>S3RephVyc3totlDmCMuW_l$VkZb9sPe!$su0VOOg-2v0B5(}VPCx54TY zwH&J|*Hb}{)E9H>JD*w77Xxg*9AFM4sm@$XqE>bo;y}eD*N18kabdaheRzGM7JxXqj*y%!ypaW2|SU;}F+I)H}j-(pH-0e<`m!nt*I?OLi+rT<6{r&At}mAq;zyWv#VLXU%a-IyVN5gBJNzcH=f*V~=>S3eX zzNcdNK`eqn5`v{@9GZb(tI^+D)b5d5k<_fr9)0QGT5^kK#-w)+S8H5wpG$eUFh?M+ z)BK5P&P~qe^^vRpCruO6R7b1D|J|7NJ{uBrjaoxJo_wgR(~0t?=CSj8Ki6|CX$y(> z!Hr6(bPg^OZ3#@4zRBm>Jbo%iG`WlAoGb*>mzpT%(!!$HjTv^$;zK^YWBe;^eK+$( z6zmXyTR|~g-^bBB!dYdUXiYt5x30e}@|fMysHx}l#jaH=w@es!32&~917#>-2a?oqwi(+R3+g)74F)fmdU2~Z6Xd@qGkuWT~P!W1ik#4FczTjqSg z%J)vIs;?Woi%peU;&sLZFvY#IJ20CT`8WTv$O;|=7dT*|zsYW7o!Du9VigRGxKYf& z))L$-eQV`$3;v+l$==M(aIo}bVyi`zDAGfXr*W)1c--m^b3N+Do+~Vc-=J|!Cp-!E z!fs4Zz&ZQOUl>~phA*Sebsg-}hH)gb__@?NlyK79XdXvwaHiu;jDWCny<1ci>5xt$5C3L&SqpfN%Q_=R z{HQ9Ihm#dOoi1Ld{83j6Uq_d;s&TX1XpKSM7%JXs-JGmj=g^N>@ROcLatn%sfy*Kh z)mG&BZ7ospo6tS`+BlFNrbG67yhaXrdIU?4DKPa2A zb;DZTVhy7=L#1&J+_Rm^4rW^8=*#NJ{$$V*uB$`#eQxN2&2#`avntstgUzhuD0o^_ zv$Cz&&s;?Dc1(zXeOYo z5LZb{*sWLWP(=<#pkHjLB%P+is7kmo($AUs&8~GSW=k@8Giab`3|yC7MKKU0O>o|G z`1w64`Mg#=W8ohrzklD`=`fb$M5(xKxlq6d6O?dy_ZG+djVa$-DaZszr8r?YpJ4WY z4dFD%A})NA&Rm#29N5Hv{>vG1ZQGa3H!vPsQfj% z(+5IzM(OSAg^G( z&HyL>0yklfb!9Vi<4D)WG{2KfP z!qQ{;lkA~4F9(I?=#mM5xq!;eN7(+!##FsM>0KcMG11Xj$#I0Ak+qk;(cey4kkO;XUYH_tBoaaQf^UvE8EjH_*bi4 zqg9!U&b%Ehq9Q048&m+y%_k>oM_(TG2Hy9L+w zHF!3Bj=fgDFu}RBE(~iU5@yP*#iPty;g(z1im*!$NV5Vs02e zu&HBt@h!|*%|&IOm`!D%#pc4A8Z_%v=sso4OBfzaxwu5+2U%YyUFF4PhxM+=4^llr zu*Ru8y_+v19mrwi>@dk+c7rN`J$3zMHW2-so-;@X;yvR6Rz1^~@R@a}+FA(9iCnJw z3-dY{(6Yje@GDsd3e#oEk?sSPHHZa^qNn3yt6r^fs{Fz}K1%g;A23$*npcZlwGi^w zW7W&9Q?DyH3%nMs1ZTRc!Xcw#z2T*a*_f!VrLXc=!?cN# z*|sR2UO?2&Zy*)62D;V(2T){?@~ zd*ZkZ%Si5NcY{c7%ky<+@$)e+*Jxy9B}&I3^~paK-+Fcln4%(PG)(|ztyW!W00tK? zoV4XP7UITuR4|bxH&($!*DLgGkUB9V}M5^hH8hEk0Oc}3B&DQlcSO6Zy z(X#3z8ZOKz#SX-Dq}Z3!IEz}wzc3LoGb&=~UY{6LqaLa(I;j@_19N(LRsXF+oXKswsz96jzmYZ?xNB|iG2e0eolZNF8yGyf;FH~f!bqdBtsDP#0_eewz-YM)H ztI%b^gJ3EC0COkuJKbBI2lrv{%4kwB{G;-K4-dauF#Rtl#(LCXwfsS4fMN~3l8Et` zrbdAKK~wmRswujdpy=0_*9S$5%2kiD2Y9EbIvfj>FT_iTr#UdV{dLvL2FQdKEP|E^ zH&?DqbXWvt6?Y5zMP?N1@?Kl;kLs*F^80GROAdb$SB9QwG}I>*yBw)k?oJ5sz|<9? zRv{A!^+fA(ExOyQTT0Wkhp-OveDFWVRpd{YjxOx zn0x0-T`Tqr6PR4(AaGKdk&LC+3i%du542)TWQJh~TogUWY7jJ2e7$|U!TE?-KEk_%^*7^eqBf_Qp&Fh3G5) z2X9GJc?3HzYy!9{e1@MtE5$k2F_`Xy)-|jwDFqL9iag37oE(jCI?c-5Q~3EkCp#?C z{jtDHQ7>2cwLDC7VP3*KoDLXn7aU=bsy_$WV={vS3Sw1PU@U{qfT5uys}L9*)o2jN zytys-M_zBaUM&*IoIgqDjYOR#zL=M19o;$$YqK)olR+KGPhI4(Xt*&~uX`9%g1iRd z(3Ahd;5N|g#i(8%m^`Zwafq%*`U`RQiBt&xfw{izUVmV)z3Ij6w1muQ-e@wgP`ZA$}chhTGcwEG=SkBIj5o+(r&@TxPl!=TUiCqnR>Zo`bB5n zQt_m0*M~)3-5F?q6P(=@sc0yu`?lN_g+a*z2MJeIrcx57F9mTvAI58sOHnCD0QY9E zH`cH!bYF0gDk^2beSscHl`2ob(PM)v-h=RggToK#9+{+Q@hhBE!4VOGi zIp=ZKh&daAyI_0LDdT-%P|WfvBiNh013^NDf9yVZHz@ibRP){#RBAxnj;XNH@&HVi zS(8}|Fo)EaTj1ILpS}GVDF{^e6@6Grc<8afNJq2)VL1%2RKlrvn7pax+2jyXi+-Rv zeOsmc#;KiaA!xzmIo7a}YJFF}(?vh1xLen4E8XlIWJ*maUT$HRl)(P!FXm?x&SI2 zOLQ`71lOW=6S&i~KP;#RrjBTLRe(hann}KsaMy`?{=HIia{;mM(UGG#v5jC`9$=N= zSe}MLi1``l%tEJ#><1jm6dt4+14C8jWBxiS-NCN0b`Sg@NiQ9kQ+a{woTT&j!fcl8 zKJ9zypG$dr4o43`p2JZ~H<%ciFp9#(E&s8j@nBvDlPXU$G1MyqGw32!_|SwDh;-CZ zVQ9cB8((1!dtjIrp5C{eQX&R>aDs;z{_)wlTQn6^L{dt53GgXf(#k|GHw|Et9`khL z&{{1wI_&FnYqRSl5tjs9=l27l18pZodO5h!n~&ndxR#G1RWRV7$bb0n!Z?#TBI{VF z9$dc2dnNH0wPfF;u7dAX7g)wLo>NHgwRWFV`qa(4MHV9|_9J5O4~K20KIxg5d0hOUsv0>kskR*AMvO;5)S(2nA`xeTTsB23EW|~h;d5_ zg}Acr6b7;)4D;Gs){CG&Thge@c!S6CRx_qJdXIT6u#$XPG*n^8Q-&#vLH+I67*-?fm-U{SWsVhS6%W*9eZgqRa@&oJ2 zH5u8(dkZhWDtQDR9qJ|_R1JzsYPTo~rU|n71-JF!+L&nxOuZzIH(%6IW)Ls{Tq=I# z=q4VcLl*SG#X;6i9v;Jun0E{6MDPovJA66%cI$P8o0&KtFK~#X`oNSN z)v=sh_^6T&@+TaGePeeJEd}+8>6T2V1maFM!hSi?y;B&zBDlALsS%^gVdo`b#mi4U zfSf9qA+m+^%1bM@4U34Oc}$5OZiNNqNDSukU1pkJa=-zw(BH3P@=q>>eiA6tJ~61< zE6*<~pbHKB&8}RN%%t1-uQ06~S>Otey)L>@=DM4~$?589Q5H{S0Sx#T8XkQ+tR-D_ z8~t&WCWLo5##IKrFzVv#m}I;~gAkA>nH8e+6OuTY1W@Ui2l*t%4kNGNY=1IVg^)nA z-(G~4JdGSS-F~nV%v?=x zz6<-(ISj;phSB~q{#2~E;#la}zk$l4Q&`5eP7&tTO(wf81$EWM_y&uynJBhts>0yeh-Ghz?pS!;}>C29^na?X<`=4 z9<%~pEqEPtDZd2gjamissrg`FD$7Ly5(F!MQH7Wf%rJ6ySQyrj;J7uf4~&c`udCN5 zMwAXr7+S+m%;|-j&R>Z0?bYWG421d{Le)9p7t^`$s#C}$(+?=OHO`r4`OIpA_&bWm z03^TTBJ0mZp>Pj|E?!p)npO!TA2T+xMZqLnE?J-Y#OxxDkp&}Z&L*&)68UFF_!pN}~voG{DcrbK*OYmmkY+&fYY7~k$!c8<%$bE=G?jjWGNmVd14Z*ts zPmEHEyrmOdcZv{=p9rD{R=5&O$}d?F+X)U38m4k40!|#1QVbk8D&^gUp))DroCPO= z#VaxP7n%^o?S~s6N5+)?0h&CoZwV$x-f1`|&RXoB@F3ik!B*>F42M+cV@a3?z7)}k zVD_k_wB4ZjlS4`lvXd1qKt zsS5TNyT70ez<@&(jD!;yvkxsL6X%qT$wjk<9*5fOMLnE!P(B8(TS~$p3oBMOxCNOK zwkmFzn@N6lwJ2U`I^rg{KzULouOQZ%nPn%j3SH7dlcE<4;?bBkh7TsfGmC_gA!I7| zwoZ*c5g`>Ggc2>FfEI^@FaT0?^`V*a15Nbl0WS-tUh>Bv7F3e|CT|X|A~(K99f(3N zHp1K%btK&Pz8f*b2RBAlXR-AJdAhapVy&UBosUtN#uHM-7+5?*VERy%r1$%YhTGS$j&+BPoSuuh^aS{oRf||(CQs6JsOI2<- zDH82qjyT*$`4hv#Eg8@)jnwK-#ZrbPr3~@Yi0nBBBhy7j{)aGiWH5K;i_nqcMIZ;> z^Yk5X3>G}vNAdxAyzfIhSpn%B+F8w6f<)Y;r}1JO>=E}8AiW6+1+^mHgQPpC*(BDI z7tJnAp=TGCXe@DeWB4m&eK%-aaKI4Q4aQ%?m?94(sWQ{$L7I$xAyb~7YB3KSNUepL ziruIr^_(V;?2xAi)4f_Msy!#U#F7kL&Dt)}kx_0ckGn zrgWRkVNCd)gpiVPPVrAgUU5-*1WT5mEcZo3?pxU8dxkulJRB~}?9udcC;7}Krj6`m zo1A1My5J-q;tNwXP1)j^h4i+sbt{HLoJoX-x;p8QeY!VtJi;i@ANMK*knJ0A$m|n?U%Xkbuo_ z9~a1o9L)oWc?rX-8Ov2YKcnJd;u2V`T+DQ)oToDqQ>18Kh@od0AT}Q>emWgaSBj>; zTQoCQCLYhuDZ^jvA|&Yybl1m`s=l zm^aEuSmtWXbzfP@jZ6MM!Z6e9bwnYTrTFLFA_p%+)TzgNP=dbe3(9b)D=a#swG+(8 zcfl@xb1;>qR`?6+^ayS~!H&SlOwDCxMUjz?ic7Skyl-iEq@` z)$0?JnKJ|!7{B75nA6Ma`UhrzefRprX#Q^)5vLLiXSn5&7G;Mc9KE!>v2r0O)0i?M zqc&6*gsoudsJ_}m*t57))n;k)req1t7lOr5=yuSA;q?FvULZ+pQ^QnBO;SvJYk{yA zg3$!l`~`1a(pZ4um=Le%7ENIGpS!-Wsdy@k@i$$I)~ndmh*jm_^a@TJ^AZM3Z+McF zj8ky49K;a@tOJN=xf_r${k8=>eFfx-$92Yw&){j1`7GcsWJ$bcxVeY;DhK8Fg@1qx znDqi2u_!ocV|bpl99-K^#6_R$S3WV3DxjAJf|Va>U5_!dqcfw> zwO$+H#G4OBj#8`z2TJMm!pU$(waPXWfj()S3Li02P&LwmTs#mNtlyYqW7U_dLGlNZ zbRSJRkuGx+r^eKr9Heg4-E4H~;b|D$LU*Q(Ih)@4Ua()7FxDa27&_n!Zem;6R)pEa zw36q-R|~()2d?~DMzBn2)QyTuX<%)d(dp8tQOTC{GMvN)RRxp8A|*c&zP7AfsBj0b&9 zywYZrNbFjn?wb&^m>qhPzu^T3rEmF7k=U0g70Qsq^*>|uBe;Q7`G$zmcI`v2y6yxm zJdCwPn3=RQx*|!9TGwqyFnh2(!D%5MBK?AiKKe>U?oni$LzLUEijqv!WZZE|#JUa0I+$6n`6df;t4QHqGI~Z92a35BhqrWhf zkt46G*C(b$Wf9MrQ~w2?_G>XeG3UGN@(aw)5VJrSlGeYa0nEy}l%sO@a#_T@Fzu%1 zl&bKHm;$xz(>g9-I`@eQj<_+g3C0)!b0b_f;X%kU&MM|f8=Kt%+>aZdHq0LMhq+oT zV(HL$l9)s)eQlK@n~2e%x&V@%&H}5Xo75Jy67%|?UtM~lm>3dF6C!X|)*fhEOd}0dIRlhV6Eumz7fC9rq^hV-!C%o zOG#IGEzdu{S=zN;2T@dpN6!JSakiB|YPQL`exA$Z($&l=iE+W%saX!@?6;LIe_(Q3CuC!+ip@G1 zU{2MBVhAQMD5+#IHST)0LyhX?kwMXjnstdHgVM;dLB6tb^|CIX7F~}37dK}}A2GpB z0`qVKKI>Ax`9-nU#BNcdcgw#}l7@V~QGBe|C~$Z+$Y>o}FjpP>&D-9Va*^04#xw>x zlu0AX%*uC}EI!7(^s3^a%t>0yp1r&#tiE{bT*7kk%rw82m!K4k9HbSkiA4kvz{njEm7n z%A<&0K2x-cMd}WM2KoCsVXC-xov>a_7W%vLzw+m$iS7QeqAZ{Edg`W{;P)w81G;;57JI%(ohon>`ZC9U~d$uW3?D#A7H+?r~Z#m z=q##UY=*BN&!Xu~+3yeejirTrqes5oss#XV7ypFeaV|FHs_cf?Yu7GxU%LD@R*4<1 z0|lT;>k=jl=?q_8u%Dq^nB78mTmFKb!Isd?ZRs9H(kQfc^a7!)ZuVQb?rHpdOicFw zqKY$CHvY4B!@<1%v)oOH$w`hHFoh50@+15(Y{Jd%R!Fu)1f9f`lw9x>9Lk2-0S1(- zfgB}<(uony0Mqgvk!>kiHB3137wtA=nEJ=nL5QKsp&T`4^%o{8DbCgF1Jk<7ugZyX z{E0ceBJ*tgiP>M@y*@GXd)x>v3q3r^9}^yMS$=|r%RXe4qhBD5-_M)RZc75}Z$sP0 zg1rkUj&oY5NH5?q>_RY{e|3P3N6%7RM?A^bGfUvMa^gD|S0N!astha{-(-;Y8>H^%@wW9KdCse!c{-dH(;qDIduvoj5uWPN zcrVr<*#coq1nEulzVkmfEkxxKV3*E$t8CG1Yt>l$gP*`R+w$FP%tA&)DjFpFyObs1 zV7Rn3tuTZk_o!66@kn=34oeyWu5A&^bPJB5s+t(!S-|JFE zem$r;^Sy8xhP`H^SDM7U;kKO2HfC-=+hz)ldbq11=0JyL0DT z6&s~LMvI%Rb{`q$2Tat*dnX}*x{7t_*l%zhnSM7={Vyx@HEr|q?u{He#50P#3Ks-J z)JCb?A>g_lqUXW9Zp@IsjNGU#p0cT<8>Vw`4WfD2*A{OmO^7nQ{z))BbE{nX4gcsE zy|rk~1BFic0p*_76<>)CJ-R{FQuHi}#Zr|E9z%+PgE^vbo#s!B&bRpg$YL~V^?z_e z)McN{lZm@7OyN5di7y5@o}5C0%RGWn$dJksT3v^ZZW+=1Oox)5G31f!*wSBWma9b= zW-ba{g_}}Nn|0W+jVGg#isgDY#g3vKmBd`;GWRb`XZG}$Ff+Yl{40HZmscUe_U99` zZ&r_R7G8#fX;+UpHH=+pVaZl@GZUX#m@&MA3Z`A5A`9^N0&BCYNT3 z54sh_a<^zld;rw-0-z@mhj*BF^q@dMNx*o+&06z=>2_QiVme)63-|61IbeuT#7zXeEdH)9Wb=v6!(;QrMhS zgi(tkvQ_FsZT=MNwx5)YY~RR z(-!?n@sqQei{o(dd%0Sqpex)|v_(*es&2em+Eh0itVvEr&9~%`e8Q8Ml#b3qj4A(a zV!|(sispjZBmyB~>oisg(~#sg((jwWZN6*V*hit#1sAQ`6}b7@Pb$^?O~HdCrS> zwbA8O7P}7kkd2S)VAvm+lCI<4U0e{FmQ@?kRKkS|zb_TqT;1_ypkm&lCaSi_uhjLt z!qD4mG5itaxcDv-?6r^yw79vgY)Poax*h}2iXzcO>w@{oR`vdnOkj_u6M3Y> z!l-^XF-5n&CeL9PbR&!vY{6jvwu9Z+2s-gqRW{!8ujX4Ga%s9)?rNc=9Q;CCZhuVz zzi4T4+j2YFQLvf?lW8w_xLDkUS#F@dtIlM2>ZLTR;vwXwXWqhZly={KG82E~H*QtC z$Yix>y5Hd!c!nD*ns&P`qNXmHNLD<v1xKENie_`wxF*3fq9j?;4XdD!6>-n+02IP z^q+{S@SCg8Zw$;~ELQRiQJR)d%++fx=pUH<@!jhKquSkX7`~SUPt1}!9Kr;L^!+Dqpm>a{}9ouIfN~}9fkj#(T1w(Q#QV_@v z3=+5+FxHlE&myfd{e#-Lq4-REQ)3#QG<6{ltk*N(tZ%Hv73TGuqDkaPiOZ=>MhX1` zM%o5rUc%rVm8Sw0kysa(8^Cl`i8SgHramWkWiBTMbcr00JULTlIKW(S!R*13S62&W z0M{?@8?>OTZ=y$_Kc+^77Ai-_abpR4Fu+JhgN=*GrH~fm56q@c|H3b@*d~JSbHOJ= z^Eu+kIq@;a8FkKiSbSjCMd#>?OEk}6;G!Oot=B5A26R{yJOr=ddeQlu6^2VTy~r}L zovogJRsPz15k%GL*Sdfn3gh&zmD-r@n9Gc8xsk*fU z@?7_l{lbjAPRGXZiB1@b7;q>0J7!&f@SW+Y)Xk4zWWr=45C0HQi@NrzfP_nmtf1~(FuThk`Va#dM zh)dWV%vsC&OVGU2p2%_PaZf)MTK{{`lDfj6n|4_SI9_!9VR~Hl#aWyEk#SpFi-Dp)YV2b&cO(BCjKp8Cw`trhbo1jc z^ezv|l*O=pvJk%kM7F20X>GEI*hPEQs)S$Eh)o%agU7@|B7Kgo(>z#DlEdQERkwuu z(6Z!~+>1eREow0+`v&7E2$O@X`;B0e@Vpvy+2r^Gr1`qVt^8{Acmv{#2Z-CKB5^uU zX9_h*y^ZLM4`E9f7X ztC!5He__toSFb-XQE_}1M&=D$TqAysf%!7Ivd+Nek;v%EZ_;|cXe>n?xZhNF8(rfVU02qF5r;fRS7JiiMpqW; zVum0YT~)MeTt-)7m|LSZ0kqQVy-L{e%`&<^7>2ejqbuRmx6zeVz}5#ynm{a@XdoPH z@G*?{>1A_$5JP#BWOF5)>N2`&720KWC5D+UXmuqR#xpJ{X@7~5raWre#sx7C$!EZgeJQ#-aXIjnO$x7C$!nA_^AB82g> zx)QT2;KMCi*;dyDAFJzWA*(CDA^(Ib`WsPt@nv{@Fj>i$WLD*B*BczmCcZG!i7I4w zB?&X_gzT<5Gne7@K~$Xz8D3clk7jso?(8c&lnY9t5M5~7@=92fE-p+rmNby%RfXuv zw!EtN-Zs3dE?v6}uf)uF8D3e6j;Re~d1WCw8T#YqukkXx64B;1yb_-4=x^{CUatlu zEW&-;UG;%_*>;x`(zZ0 zk&(CEl_DdgZFgPpvAdoYvb(B$UfYIO*551nGQ6^8U*C3Ct<=2X;Cw~>l<%}aT<7@{ zqZ59cU016&v+F})Z)R6A)D$nXE4fQ3+w95{quBCl4jFf3IJr4L)Z#X~u2Z?puIJn? zv+HRgvn$~-x7n4ycw1dn!ZdEH>ng#y%w1wwm;Fogx@@jDV*SMWCQYt{Yuh$gNs`Ot zy7=lcxk{4MZF7|<*)}zC)#juZ!_22i11zJfmbza?S26#%jjl^LETiiYAIs=^wP;6t z9ILCu$7OUSqSkF?)v((;Y}4Sz3@x^-u3Snbn4sR2u&s}>x)Rg3(Um-u+H)CQCDrp~ zbS0+DWppJd<6}}K=jk+I%jn9B!b*!FHX64fe+ME1A)s>f_xoxOD zigVYC#-ys+@LUy)FN|VeZFJ>Hk9iqg$qK^G^)X$2Gn8#~UCd-}=4Lm`=z3bnEXQA9 zZV4G(b?eQS(UnY4hP?G2889p0Ffw!u}^TH7|b z61KgP23LyFF_*!WATveC;HrAD`7*dlbmwhwB`l-xFntXNSK8uAOm1g)0mrnBuDm{s zbQxWVp=W@suB6#8x7C$YZ6lGauB3aK&KAF)(e&^p5pJvNIxENOx-iG;$}>t9EvqYw z^e&StnTqXku57w_p-irakICeEb2^z^uNHH2*HE|3Raf_U*=r_ zxl-CY2as&8y2h4mW7RupT<|Q*-(X{u0L?a6@vgFMuH;?rA)6~1M=jgtdRS5QfjP`d zHrLZaHdnEtWpZV$;=C>1nMaVBGZxjcU~>PusIaYRztiP5x~hzGy{xV})^k~1#Rt;1 zx{^=ya#>wht!rCd4=0k<_2yNwx}FxYy6Ok!wz~2Y<1)Ic>~r2mR|%4_jIL`AKffo1 z%Uxmh>1Pv?{H6Z^qij}5v5bYKpSxfr=4K(}DVV-d52k+eCse!xC4!*~2K;m^)tT)K zqr>1G_Ouvv|0D_r)S<*Ai6vvzgT{{sqe@3rM#cwb)r<|!q^WL9X3>FXuBCS6*ZBjZ zBJ{7T*Cz%M??g-q$WO%S<2C&QbH2TLePEJmp1onnZZ0@8TQzGlN6nfy4Pm?vai|4I zDNUxvDAhK^Boh>SL`ofCkklXWS<2xC*Hx(`YbI&QLuO5eBeo_gi1)4f$ zy#7vMCUAjiZca_nm8>f?h7yzUp*K}4`4u%Ezx#NxC_Ke3uJqR;q1RF!6mx`RN5PV z5A7X_&Jke*QCSLepgenE<`*QqdNeOIm5-Q}Q^0*<4r%V+`NV|5U_1^b@>H;LJ;$ye zNne~77cx2&i&M`&M3Hl&6aV@G*EXFmvWHjuY5@*Hm}|?2g5abcGUgG({gD+myV}X# z?U*GK-u7Pv(HJ-CwE;;~S~Tv}+L$KI{DYAZ3Y-a<@*5P9q%<=52Ue^tenSVWRi%d` z`$UhT$Is$@t)C8Ae=A>bjW(J2i`v;^^ zmY`QI_?%C-XWD*Z8dpl$T+9i$8zJ4iqB(l{v`F4B|3Xy^@AZwUbG-&p>jqJ|XOqbC z#@Lu^B}Hi;h)rk@HL^mpm$!;TK?t^uc?rVu%v1Af?{mLtaT~{XR#LxX9M=% z^z2=%nAKm?#i~&rHs62O0~OsNO;FFbn6BtL`d}0=i!XxmPz5u@BX3YPGkVLjdo&#f zVO|$z6e*$fD_MsM$DbudP5%?anhP{F&n^T5gbQj0_v`q`s&F5$(M=X^F_gs$-$PNL zMtdLc8oYeKGpu$k3yGw!eNf4j19v^(HztRqFLb5Jg|rv!1QTJ(I#A#PHKN!kzcHA_ zS82QSycp@ipTYLS$x$UAC!HA zAD9x&vhWf^Ml-TSv+58v$KiSsKOfV=iL9yiZO9&&Dly?;Ucg8p*Jb!3taL%NS3Pib zIw40+KbYe5L0EHw`boa09#|PF`+8qJ7^TTPEyB({_FVpJDssbPj-8;18K4qWE+PmDFgSFaCDrJ@g*Y;S;1%;}Yj4e_6t{q^1J0|R@?JA(Bf z9$3jlSmqcDSL8y^K&GN;OO3?B$7rDE05f=Ht#nTt=3Xdo&uJl5HsIQ|n@WNe<;%`h z8%~3jy9i|X7ZQI#M7EPM{w6z-;?U8C?{pL?F3iJdiq@ku4+#(%l-HGK3NTkg@jL*7 zs6@du@H!b7NLPA$zzvk#m8^wcDH>R3!6{+_zaae zxCEE%u#dY>XhM@^U6NS zF<6H8hSwcc#Ftq%^{&*#77#jGL_Lc(`;}788TK_0mtWxR34SRjxS|Dkc zr4M#o{t|TD>e_MLkm-HKwV2AF(&eds;6A0AJ(wdH*P;Hz=$KO!nlj&xa_A2^v)1_g50tP*AX&~ux3V7>3w+03X_b06NEq_^1(TzrZQ zAp=$tJTj1?9?{2}WL&cxqzQ9SQPv9#N<@1wLk>#plj3hov;U;N8{7h~1cKtfU1sMq z4Dr{RK6N(T_Gp!*@LSQA^Ekd#? zkz5V(79kM>%0;fpt*$8`Gn9Lbsm+>}i{%C#wEc$RQfI!I81B^tMz5jUD+N2)W+)mG zk3gfAibGCIw7W-C%29CntXWb5A+t}^DincflTY+z;fre){r8r&syu>7^{V#So~qFe zqP;Qck)&oQP*E z&BH?W9&RR+;cc1k7Q;DznZoorl1g*i$6ivW?CH{sK5(|jD(!zd)KRe3bR&W^qYtKR zH|&K`!lSORoCl7ccQWz$eIeaE2Tr?$R)zzY-E0X4&`K&_m*@a>^m4awpo$*cqz#I3 zz3{n~iW}?7$VKX4BG*AbhBr9PML)vOZpL4bdC`n%-W5Cj({!Y!NJ`HtEL1S2YlD3aC`}8S1?q-{mGAbUFH_rFyJ(hc zP|Efoo_*~v_YN@^Y+)|nDI|W&fgj3-QFJrVIe~0w(N6_pUc&HZ)`J*xOTLi~(MT)) z-7XW*gK?(b#RQx3ye|&c^>Dg)9FOJIpqZmGT%^LQ-5gd%@QMJB?7Kbgf(TF}@wcuo zjx1VlW6|j-I*&&*Y)p_oL%(8?(=gLc_Yc4T99vX}AL9(juu!C^=rVF{nb+ej1Gz9o zjCQqXstbh=^3=ORMOS__^r&WRt!Sc3@W^S9RWG<_={qjW&^Kz{hevSp5w5$3-^Mb+ zaFo}52Qd#Vy)mox&RH#RpEN7t2_8|TKAK{VxF6CNkQ*Gj8$rx!(_vc^f2YismOJ=R zG90DuAP|Lib6myo38`sA#90!0a^!x<4j$-=2oD7Ym=`takS}i4+KqYO#YK~NPs`?i zN!jXnK6qDBPAHup&ow5&?abrnV`fwVXXeBkxD+Q$wqRFa-$jiTE=V4gTEH9Qi1V6@ zip>1a(@&MJ@JorY?fmIyfJg$}m;<(Ou^Zb29){h2E2*yJe!&3<7y+D=+bX(@hJURdFFr z&Z8|oB|29=9NbvnIFpP>$KbdW2nV!yn)e+N) zU-<9;`M>`ID^Y}S$tq0v?>{j+VXrX{QKaDD_(YPzyiiH&#yq3>|LUhOzVLi&$h>rU^?v8-L@GA*R)*>&8Im^gA$j&>Sp z5B&5%@_ptLUM*0otq6WdgOSX5uHqAv8i-W(Aoy@WL~dZR*-6VEVs;brKf^7J9U$Y0 zVATo)8Pk0sFi~uWmm}_YDHQQhEDi#z z+pKR4>>Ngb$MrLXwa+Q}4RB$y1P3rQUL6Di@rp15TR&NgowIxft8i8Rz3S!(VqtLt zU^s;4X`*=77ej1m^Gr>l1D!r!Ln~D=tE`Po$QoQZ>qg^n8f%t;ESQ*w><~YkX}V3+ zA|ZMh`ili)CBc;1X@LOjT=mU|2{WR*ds=)nbov5tRJ_z-KO-)YNv;zYY&-bD1R6^D z3-o;oeBhISxs)O50*z6OVUq2j0C?>YW;*ce*tl*1W;&U^+AWxjz6*X%EDmCh{)N%J zx`BC(C!fSv|0?LYP;qm5*BMF46hsUNAF=?8$w7P~B9H3pqN5Ltc?zy1%&^7t)|m^u z!-b6oK_Xr^f^MRKCtVQ&LsUD;0PDnN#oqSiKoOX*5MbvC3d|$WZBWDHhT3<^d}2Tb zxQjDauu|YrSHz^xv&#vJD(~b4eKrj@>s-&4^rlLRXNO=5U|#Lj;CZeE{{Y6+rpGko z`$Lrj6K_wND`ZsZngmvXG1FPFwiFpcV&9n8!GJgN9@S}SXyTvXUp*+D_+)e?^TII| z_000wW$vA37M(HC=aw=r2Hl*Gd~~_NrWj&afw42bT;M7N6*YE)f88BdmEnx_;lLdQk#em7nuyZ{g7*n?I;c74!A7bPP zH%7na%Ba{19t~x>-vk|LAwHmsX1){rvnNnq(X=0<9F~b8#@N!= zIi=CLm445QhaEz=nO;~A5b@Wk8uN%B7Qkxl+!-j=O>jC&uzROcCni*dCSMpj26a21Yf2IOO-BcOnTAf<|9$XPr&Ob8dJ5s;bOI*n^JFH3r>^L z)xw$QC$*@cu+Ytlc^L3qO8x+Dg!oQg>^>9L4byowFDfSH^+DI5^}&Jd}10DhzOqCU6;IL+6 zo|OTdl5gsK#z42{n|x)5s7omj5eC_@Z33~HsV@4~Qh6@EdIATWSn zKph@@*C_hNv@{94Lk?!OiN5eZzfku5E-04>UxIueE8!SyPb6En-F)mQj{LppeST_iE5|b4V&T zHV|phn&7r5CU{6Bu_y}R95N}!LafWc0!&RC@j8fb-~6SJ;Ir`IQDU9$#kWc6>%?zL9+56u4h z?)8aT7q1J$ix=T>ZeD<^ z-+JZ_ePb>mj;j|pxE@6W6L5_C>IDk9RTZS6UiaM#B(_x-WL(Jb-_G4@#Y_6?rEtl5 z-Mua{_TB3@VdL&KSD1>G3WFNUd>&mFf=UI3pw^>QU>Nndl?n`6rFfE7fnlw5WS3Tf zfwJ||Dlk}aF$!q^L-5qd`j`u1uBl=zm|tRPX`Zo6)(Jg}IYae5)T@nP#8Jl8V6Iyl z1tLNj{noZt0XdU;oLj5FEKTuf6#&nAYZb7PDco9xjoDfSrt%p_t1#&ue$)yWM=G~k zfnk~Ts1-01s{hO=IMsaX6&U#(&ixU~U&GjX1q6$(wcz5Xr52`MLBaS(uW-TED;$jU z3j76%Cg~OM0bhWuS72V0X1xM;Hy<4-HqZJ_-vbb{zICQ(ftc=g6bsxN-glN_0lVQL zQM0hATAGCeTC?y@_1Kq4OWDyZ5Yw!r8`_yI_W6bI+#_z)f+Fn~^TNEE)aAOgkw(kB zu63KP*R>Akp)KV^;yJN0!m0(FwV`WT)k?$3;iwkQSzW4y-xK4JhabW;$Bd{qxiO@H zFQjF$Dodw@1qDsVbeKbGswEGf?<3olwW=aY=uC6Gi4yAcO9CRp)>y0vCP2oRE!M=< zfs0Z0!-_46E||2T;1}{zBv!t}HsYK=__(F|hN>m@qm$ugn) z4`#d1y$DIx`Z&C0D&YEJHA`K;8{j_Y7SHZ7bKEeaRu2^U+e+1EW@YK?C7&)hsyfMq zVMrz5Y!>`9!0t2#@gTPe$22u9Af8pmn0!;CDF|cMU9(cwO!}imAv*SVsfoe#zK{;h z0<9w;g1%(kt4` zvdGOBL+piTTSk-=eKa>F@^n9^1W$#O-Z01@*tK{MkHbeatVs7M3?pS4vLR2^U5l2H2`@))_My zlCPf0H-HRJRcmFz8!)E4)EAkX$JYx)3zhtm2lLn#PQ^P0Ht<^#WQ!JSXKrFWng=K_UUhQaeul3{gwuJ0ZsU5 zgLfucG3Fp^*E1xH#1+FI$a#__&03bJ6zje>dJ$LLS%!jtMT!8Ch z)qVjXi`Tl-flXG#51j|lhW7*LVAk^jslv3rpP2PbIlVqH>$!uhc#UzypP1ciE$Sbb z^Yzv16SE#b7lscY!q)?6(xdxUJu~5BJgR4A^{Yqq%-n?SRy{K@Tt2F2B%{mM0|=Sq zLMYb*2pC%icp$O8_l29m2IZEM4}J8Ot`>C*Qk<@~b-8-~fCAI?{sHWlJ^A-^b*TOR zxro^BpWkGS%TPpeo^`!t8N&;s%wK>_M#v<^yjj+J@YqMVF-^a39zr7=yjVyff*e9# z3LdZ+lp~g;jPzE_i&gfkATnu@mCR&!5s9&qFIdP^5>_p49hQhRSeLO8J%SwJPm7|G zuL{78OpN98><;mJ-N2B8D}>-5z{T}z!obaHsIzJs2Q&CmcovwinE4mx5dG@)2PRIH zkmI+j1TB&pG0U4}7E-D>iJL^+T(ez*6xoEZ9#j-dZ0_~if;BR=RK8k78ygcVeMF>r z>dQN&sE3M6@d6RQ4xKn{)h|l&KCes76ln<&8MH?*`kB&6X^CFblw$gU=s^tm9){@# zQ=beJzek92;-TLy%J+Ix3@=?WobuXt>C~0SifeBx@%Xwnr^>P8B7~vxaO&b)bFSL2 z?z23jXsxe1v{okM@2%~q8s<3H=;*9_hh1<7FGqpsRB^hg53ODCr8#=>(Z+{>{lSSYllyfR9r^vu8e*i6&l0M-|?2t=H&IEYa*(4Dz-^-7qmOX2$;X1#- zDtGq8iNP@&+gFE}MOM`dIWC&X3Jklbs)X+nuNEclAE0Y*;S@>OIxVFvml*uDnbEVx zz@`n$*`Gr#*D9LEJ~5ldK#5~LkB0K0T$m>QG`inxS!Zl?xs8S>x3CAZOaKh7c;L!F zrY)9Ks6~{l(GHhvUz7~B<)UPO;d`4pwHkdLN)xgw9vm|5Z$gMY-rQgwIC)SESbgC5 zZBRpMN@!0)#~7L`i7Nm<2IuRiy;jHGC9SR_+mglYZwx})n`qg;^9LqUFGh^Yaq<1f zv9gc{#ZvRwr;iM+51eiBdyXuq)5-4$9S8beVrhikgDnt9vBc8UCNBPNA%l�dsJd zOeityl)RJD+UEIAT*gpcgg#0B#gU11NC?@_t5VlElkH^4DGW*|=K;r=-pZ~Aq1b@a zAbTPfOmyRV6HI+Q`fvl}>W{kQ(zQK|>Hqo9MG8~-fwW;qk*r^|9Gd=S^l8`1I2f2> z$%9<}4rC81nS3*l&eXwIi)rdpK31%gIYJOg426wZIeu-%TI#pWkzAT>-}dYu0cF|k z9}%OJugf9xA25pfGv;++a#9S=>Gg@ZdbRIMCw=|$IZ)?Yi>saA`Hi{xTr3JxwT2wO zjrXXpozFG@fy-t9kQxzjSY{9;tXSkPPY#mNAbel$6EWuT{r1p&FiIbGT9nNnkk~M0 zUyX~j%MT)2mNL{c#x%dT>ahbJHt$)x^qu4sDFw`vQ_juBs%w#2 zNV)OGfyGL63_)fqSqzh-BpdTSVgnLmMv+U2ema2h#bITx;im_Rw&J9gvOGpO{~)tH?RggZ~S8tf+y+m^q1!h;;RP{ih+~Z~l#`z6+gRe_*0K*70e% zj#lR3VwxvXpWLGmwMCUKeuL+fiNC6UbxtmaG(vFGYg#V49}32Tf+N?fMYHo=fg`5SPm8ebRTP>b=KkhIOT0uY0c+zUg20GOM?Vg_wmksXT#cB7vqN5SM! zNEW6RmCC`!+?jR5T8TNX28+@vvm;>X=#0l7^sdtY8Vhs!aP8xoWxuw)bXh76MzJj3 zFX~e>E9u}jrI1bZ-(%V?O2stg5Y)EwZ(<4+3{$ZWiAjaAf3o@s#!9Fh;c76GwThXM z%vie2h01B2m@3QCb4tI?y5m(hzJBivu11LPkU6eQ)t~}K_O{|{AmS|spX`O@0s+f+ zzf8puRjUQdR91NeSJBQ23|y_53EeF^;$yNmq&Q<7!ms-JDq94GD-wopJWNW{RKooG zEOE%m7hVxM!{NMVaKDGc0nd7A4%XNhDLKYK8$d?>0+#+`ga{mA=U+w+`GmNw|h8*qt<1ucyxY+!K zVOaY4g)$TZw!TrFkzo%4zqbb{HhQM8810#)TkJ*Pl6i^tZ|^nq*UzeJbia1thmlJ1 zaX$O9G04A<*pxK8s2{%X;DkC2L+aRGl$?w_@u8ryq9epZuQ{2?vdm*X4Oka)oF710 zGm75*1=OA#*BF>fz3w3%aPP{(G-HTLtduVbw9$Db;&l+MD0Ru{ z^NHD=Hio8k{O-Z<9?W1Z@_6Vs5F+F#&?v_q1O-_M^>8g5E|`HRJ9T|QM2{#{FrsUU z_0_9Iw&P@Esxakx=zM|k)fJrbs67_bh&LZqM?bj;i)*?3{5gduPK z_=uu0;#BBTRJ0>tviqhaP=Kof<>(O1m8pdFdx8}?m+IZ3*%~9m1oMON)&PG7eWF{= zL0mChsuWKH;egP%h*QOHePP-WC^3TwV9%0ORu}gvw|2aR1xAM zELZbxBtGlIr;AZM^3#Q%5JPnICdg4QBE;zE!j6mBWRQ-?ATyT-6HPYXndp{+pd7`9 zC%WW!Rvf!Hz+8sV%_H!#iRa#hbe7`J38Xc`dZ=Wj9~@9IP&j9D^Z^fXdMRew!T{?` zX zBGzpQG3HF!e+k>UZp};BBH^ju3&dby=xoE6Qu)I6a8C67!e;E-Ec~B0S<{i@hLg_J zZ^FmctbnVMmCA(}9;!lW!O?3rDPE3n8Sbe}`tfRv4l%z8p&|)81+9H@-V_4`b>@3<}JTnm8s1_#VjjJa(V$iycC-kdi6o|7GpXku1q!Z9%Qj!h?wnE@U(brVWn zy5zhb!h$fLObOM9KV?(**zB-mq?h&V`&xBEaXKb3<{#HvHv@N!d_NJ08hI(tZ(49e z+vA>S)+e$v)|L`hX{TGvSj?f#J<30f$ra_#FP0{o6F8iL&3C*>pyaeEe=bVg>CRif zDf_KWkz#lMau}+vZv{)7-}yy}xBllJBvth5(*55>k+B2`G1`7Irm%Eiy3H+%(MncC z0k8&lNjplU3SWP0Pyz~HTNu!Nxu0Kv6+Y$b8?0FR9J3M&D6iAY(~evBMWNB;{DO=s z=Nn;mIlaIS?k@T$-GkVN?H;BmXBs$bF-!65XCC$aYv$R%pDUq!mT{S>{<|r#nBa@L zvdW->gcnK1Fu9WXpG&rj+PY5KOAfv zY_f;~J+CK1w}lS(wX;jm^b6T0Qk4N*$d=gJFA@fq9TnB#$(yjJQsr>}LI?qO-uVk9 z326==_O4|qh;>ZutV6h1Dby!)3{^J)Bu2s}p3h~ISq|xTsd4V-xqR-t4BFY1w_#I% z7UpSaIf=S%@DzSw&Ej%$u{O@24Xf-seAGPu~9+vx+4>otvLgeK(S z5K4oa@AD&&Z$$ZXQ8o+E!uEGc9H@(B1nNz}r@02AEP}aCwTt?$Wbtkg+BKDX2m+)# zuxGfzI;;C&E(T|(vjQ`wGsQ<45{3DcL!nW1U&JaQ<2bmPxSZlM7RKHSh+@c}tBd{^ z>j4zn-Y!bza!iZJMuQ#Nb_jJdD&if*h%7A1A?oTk*<&nVw3Rc4P&s*vm!_%uENB!n z0Fd27)T||2i#qRB3$b`Sf;*${x#UmT?R(8dxz`-WBne?W3j)A$R}BI`+I+5J`yh0l z7SkGAbUMYbQtQRAbeF5dVZ}t4u_F@|Q3P^aFsx(Iz3ihLDv_~@xVeUxr_)j4EqxY; z0iZvZkEhA%QcIKSz!hdaqJA4LH;PGnh*h|`{l@jdf#8Qmxt*B*DQqv_qV%FLM#IsW zGK#A1F{3_hAz|EI@87;LDbxj3fy30y=;PkLw-Jc(gsPxImCm831!cGR z$#kxP3dIDHb%bjm^s+e^P-(~Rsy@v{>59&;CrpMvC%tqO*)=IRdyYX2yY3if z*@V|W6;JZu7zd+5{=x_j|ZGDBz>XonAo7Sx+$u#K}l>L?0$lA z{jKSitlHiDRfl;~*e8_2UPsB`IBx^D);r%1AU9C$*o(_&!Q4UI;qBMt`?Pgc?Z?%M z&`vvAm{PT62jV&hr7<^}p}ocH>$I)G`rDFAPGHpcIa8?F@_Vigx9NXl;RR$M^CH@YtBcPkyeA1F5ItyTi2iQsUfbI!AD<`3X+m zQq9b;?!5%((aE~^w#)ZryL;P&$GT;_Dv+Efrb;u}y2p_IZexGdewof7l68+U;U#=l z>u#TKm=e}A2BOdUcgjw2*l+!!*qF!c0nYm`!g(hq^)HnD?fup-N?7+UN>eu6bX0XQ zh&kH!v^tB7dhJ(O_t@?cU%~cbxT?*2Y*+Is6E;*=*5|SOr~WbHj03p?Vp4L2sr^Jw-{C-K=T<<2h#se^og(c6MtV z#|o*D9@UT|E&i&Xc$Q<;=SU|k?vX0BDIP~@x)RI0!S?di&YkMg6B>ix*{SS3Je~NIeRptT zOgTY3Et1Nt^`!B2lFsPNjjhlu>r&Qa<`Yw!Y@NAfp#;Er0>+o+UnpAfFb-?CHI<8V z1_gJpL{VWrDlKslx9vAfsXLjkIYauQG$}>kaPi&p9kRL?kHwb-qFgJw2ruzjz2#kX zw&Ib9wQ4`^C(8G+zF(91>=z#Qa!GVmRWs(N*4=gy-iy-p=VlXHl2pp(sJ;6i8z`o8 z3KhJMly#*Xy7_n?7*McgVh3P|yZ}|+htMs3{N1B^-i9A|)R&f4x56w@@z~OAHK`S; z0>bg)l8`@@%E(3Yo=SUNhj%i6J|umo-Rbiv;LRt^$GSq?e9~Co^}6|_F>P4V%_oha zB7Y#0BNN^@|esoHU#~_lt1ei8=iXWq*6W^^3AEK8K=<58Caz_^|zP z@d2fWM(QW#ybhP4%XD0P5=$|B?Tb%Bp7Xi*pgBJ-K53NidR=^ytMcySZ?y0Eaq$6n zv|CYUx#fNF0e9t(i%%N5ocZE|=D03CiDvSz>*BK~cVB#dFg4VmoL7{sWt`i~=7s(y?BJThRHNpsP53(_A_r^{tx}qN4cw(6~UakPQ3sAsZ7jFoAT$P5D59} z&-K=C$}m1tdjyO8H%h0K{(kE(lQ8pUslNAW-ZG#t$j#?hui8ZHWX z4wcy8hkx5Vgwgijlgm-VoC;mHS59z)BX@DOkQ&?&gEaW(t`yh?)e`BnUB1i)+bvXz zUYQ)#Dj~huPn9+$#9I>uC5e%?oB0Iyb*qY4j5gj#bg%6v2ui6HL@`L(eer8W6%_}e z6iH!`qN z5H-|#w6)-<>~x->cdrL8Q-rV=4;J+Q-JZs1uTH9wL(G) zDJ;r67*fPN*I9*>PG+`UoO?01ffST;z*N)fJketpPtI9-nztSc zkTR-wczgD*YaOaq?sHvOq!-~l@c4A*v0<$arGPCPWGLTK&~Rhz)TKSm>7Oba!X5HX z9$fsho$WUoQz5Pm3S^RN%Qj}}to;LUm}$rR11JDm+UWsmp608Er)>=FLm^b(*ZGCc ztNHnaij3op=)>UM?ex*xWRGHRq&-eQv8Tbmu;*!iW6z{}v1i)7{IM2zqcAT0UY(#2 zlaOCs+OIeS+f3#$>?HScnbX%hRj4!E%h@V>SbHLLrIm9 zF-e>=e8L>Op6otg7WcH%Z~k)`uyIfY7I{TH$TE!3oydqjDPF13lTrkDvMjV!v5sw; zMLJB1!>7^Pp$=%(bf7WWg4G37cP(JoA;U|Y%Y$U7oa3Gbo{F_(+y~d#|wf(32E$)7mVRbpd?KnkyF}Y^>Atts7}9|L%!rq z=mdCfuEeBmG1a9M8)tY7g>MWBwoX93v-QqA#pc9JF75+A0oy!hVX(SNPxYLn1QYLs zTI%}7!deXL8H1+mv)@9R=i&7vgxj0a<0Qc?7au8{yUzP3Sfca5@2=wAT^Uq| z*CI>QdbVRnzSFulY$s%0x}H!q4gDr2D>J%}U3m1y!wuI}E*t8)4nHDrpY~a=O{s+X zfWpavtNIVVV5T&XZ)X;s)Ijp+iv9bHYY4Glv3>|q+)rTjs!zs*Nl1*G)mh4YkVuFD!eGWx&p63@jofGF9kl_48bYd_b^mK9( zp!^I&>-z6C)VDvw`5WDS5$0WyQ3LNDasWzzYV)8(rkJnaAw)GFDd%6}XP+5FT!W;xPDSH5H7%*HL|8lRJ)4KpARqAq zmD8OySB>B_5?AU|^C&9K6J51LDZTo-1|cU2UYNeZ$*#+3pi))Vo+us~Wf)>-Wa~q4 zVZ3h)x)KvM4@Ayk#9|DkAFN5x6JSL@#m6v2=%Gl@9?uGKNH-hWx~M4#4!B9vMhir} zsn|DC$jN#@e1WUE#KvJNHt@-{)l@C}QLVnz34B1LC7+NR9Vl+Njo{2c+>#2@h_(uN zAX!G5(N}?-gsj(~{JAJwdO5-SZ^ zzYpw>z+H^lsJxvI-#iBqy1fTjAbr{dGX`In1mWS9aDDU@n}Q{^uVu|R!+tI^-cZ0E z2)z&#EeJ_VQW+;M`B}I`-iFn+S)m(q8UR%K#WBjb($wd~1Uz=K>64F-(IK@2Ysy_T zbVve`D@I9#z`lls7z3k;>LeK?qaRnDn`9Ws{j71A1rH?*e8k+!?Ss-RK}6wqTc`xW z3X}TVqcANn>d0@Mmybb}dT9>;TmVSM;_#~T7t{6lcpaKgS@t^9xG~=dIOjHFNCQ_O zR8wF9#R7Grx&Bx{La{)v<3|fYokz;1SpIMIQ<+RZ&N2TV>*ubxdvT*& z6b99bLcN&oa3CS5FEBVRw#9+9$y+~gZR|EvfJd!v1Y(SpYdxV@Qr-WEl47aezM}xS z?~6l)yQ`O7PQB9>T$T}LM5*z{;LS-kVwM80Hd+WN2v?oyvz=AZhfEMnMUzqEd}PT| zp3A7EU5iZy$)l`Rcu|LbsFo_*>eQH)o}CFUg9vc8NmdnR)?#-j_FR;LIF z&c)?z?&8j5_$2U)>}yD;H5+Yf!Mv8B7sES3wb*C!Eu}NF2ON^aPzjEq3CU`-1TQ%x z6zNPB3C(*h&V=ue)~g53$m{lpxwCzUnmI6rQoK=uV=OCp=LE-yM_A;Y%T*qan+`K# z>$J8he=f>MhayfL^wMFKrM``CXoGY;XA74(B8OhL(lH?a>U5$z=Ih2!yTPUU@)q6*>K@v=k=g8*uU1IA-}Y??52 zB#HV*qXeMNr=YsI5O(+v$rWk<*{p}zR&vHWP-viI-#u;LrsvH!_yH->yjsl177kTd zWVB^R7@}Gi+arlEo7~8xpm=#eqZi(l40Yb2nz_m&@FjzUXRDC z$3#j=g>D-jj*n43tb7=h&CIqUjLXJ=IcaVJegy_V$72@}1h9ak0F5A}zT@4&}- z?gz5%w=7yk^drfO>RM35NAjsQi8A!)14~8Fqut^F*_2lfB-M;>#{&F=627bR)^Ex* zwk-i`q(uIr#9L=le^cV?=dIrqg!!dU3e}sb*ia6$sHGDrIZ>J#?XloY>Mk|`O~E-m zZax7cl{R}0;uiI=DNkSs_3&#BX^)_6qZQJepAm*_)wD%`Yd%IVfYLm+qxuONF@07y zvX^dwU}`3aqdwf@J%9%a(Gui>QoFf1^nuO7_6z@HYJw-!>SL`9soFsFDdAx0#IHCM zhG|U(BHN}<#Fgp;xn+sgNeFODgjX0%f|n)luttm)orQJDrUslLNsP+K*NN=6WSc_D zLUAb`4b^X;xajc1lno6hQ`B?7%J-3JcXdQ@sQ{wnZU8) zoXkql^4W!La1uxSQ2tz$uniKOB!>lMvf%vIH32t-7{qy^5dI~$&l4W~M|d-4CrJv7 zwI9iW`JC}pJ%Pxj8lLV-939Vvt%86-8d67om15<@jX0!`ZhQ$G<-ZCiIl*+j~m}3EGs(&{0(EmPYPv zgIjtxLlZ+DwYHmYI`4AMzI|chJs7U~$3X|nNk11EV8@h*1VL%$*1`={ysr>@G8{U> zC$d${$|z&t1Y&s-v!!O7rc27;{4pptvHj}_9vH+9$PxwcwWm7QFJ%!_a`%S4FRlBeaj&} zQu18@*|yxkZF|yqw<2VctSn2sQLO*bW6d|JHN+wC$<|qqTC%E;$>r)C3a%v{QuQ@z zikajIv#>+VO|4&-IP5jdSbn103Zo2AuJwvf@EhaVo`o;fyH|&B-oaauZFkeY+?4QX zPTFzD2PF}yQm&a!j+7~_(|-r<8optwC_1doaSlKk==CF}2g7uvM2JFUpO0AMcc&jW$iA$aX;+(dXCV(mmza2;YvIO{A~$Ux5t zc2IS8pCY;qtuUAP2w0D0!agMC_=qh6T+y?G_lj_&ih~;raFpXlh5HbIr*epQoQJ#J z_9ud}ZyT^-m%mf?t!2OUo3d{|vlI8_Zh-UXZ z?h6s>sofW%*}vO4Vy2Zq-GyfOGJF>T+|2ZC8~X%CkqB&)vWi^QxXKXn>2ddb0xnCT zeMBxnq@M>$IC6CtTIECt=q|MOm5q0y)#84yyAUkq^W!pf7&tCNKUkVjKBxh`ym>mo zd=7=K%w_D*WLGDVkcQQ2f$iX4rocX0JQu4w((J8*&bS#;tt$a3gNTX`&(}pMimV5! zS>}~M{<8)9I+slDHDH{{0TK>4d`E!bQ3`{l+<=SKpdH?e z63!__V3hrc>%Ef7_!ug;R#R`7qY&!Ud z-qbUS@V8=|vJ=vfi)qjxSNyL-l#< zFBCh|uD52W0s2AYRR|M1x4kq;R zslLYnYy^U)3^51Pb26^D-!oaEfOCRtxt{1sl2kywpgXGiBkFhsm6leFuIOBmc$pqx?xOG1h^CSWHbB{P|6jOdj@s3&>UjL{e-Q6+ zWEJDdRL@mb&M0D{%2!0u3jxKKav`8Z+i}^4qCk57rq30=@OG+1lFNuuJdvW(HSTtz zQqN57?P)-n>xrRxkbR1av2U>IRH5-J;y^7R+r5#33$eIW@7M@D8sM3xkHp|I_F%(L_pGX3RaTq=4F2X?`y^-49v;=rb zgfK{s(?0F9V4G5v`yP5`Gqwl55!sYpu_;eHVKTBZoMS`oLF_FHWS%QECb^boav;5? z#c$Lanl-Kc((VkmUGj1_3!disv)SkJGDGC=9SHDs);j!U zhabWy|5Hc^5~de-Jnj_)M{S$(=b}U`{YCK_;p*f-I#y(E_FF_OJt-wTDn(CNbQ95| z4GDq>r6(nyPCpqOnopa9H{r-9|@KouqE?EA-Qa^ zuMjeX3h?0s(hDCZ?72qoFy+i7l4$E+DRI9&Z~dl3AU(k)l0$CvZDN>4AU(k)d&kca^q2bqVc0l~J^=$bM+fvm&wEK&=r(GbxxB+`=0H0n#N<{<1{n)M;5?~ z2TMRfeR3r^HD?0WoCx|uk&exFv%tVnj*93rjy@g&pk(K7<9tCy^x{Z2sO6}Lp6>|y zgzDqdX-yD5tUhk56mFNSKJK>mC902ChrM;ak-sC)oZons4^KzXr~JsOvP0(%<%q4N zbL%gZZEC_Vw@*q+-xt6G`t}v!^NH2m`_~DdPdQoQGJKw69S)7+bUwP;M)*9ZE7%_@ zE~axhjNJ3DKRbM$D^wlaj$b}&%23FkYwY>A*2S2fLr8l)wq3)0)(*6ZlY>&q5`l-% zlqsdD&#Ai|P*#{1g?tXrQ~i!`da!GT0o0>p{0^5`#ujQ!Z~Kw-k%f;7^PKj*)>b~TU{U~(4O89bH#KGYqi4@TCBnL zq;{l!3K9@2@|1HB#R@5Y{>KXbtF7D>=D6eMf5DVz;G>^3L@Yd=KVSQPm0!6NdxQ%) zZ9v)=tQ>*OzF=hp-d>i6Jf<&-aly)?`)G?N>bJFr#Ag1ExOm;Kvi!~aewDkBc6`6e zefp{y_p2=A;eEf#Jxr9#KhUH`?fX?02>QNXV`@ zhYuegMi_3ol#o3^TT9^-j_Zk;_B{&ss=N=txL4({A;gAdA@#3)ui6W}?^T-h?UDB?@Ish6IE${}HEd0Q5- zcs3)ZrM1g`L6<5_TM7Sxca;65P=hebk5z^0rK*nF)k%THZjL`qUhJ?Q=vD>F@V!gy zAhlPQs=d(rQni_3U#fmc2I>YS=^nlBR0TGKkDI`ubR$%jl)Uv={^@N$+g1W8TAsPZ zhwT032@eASO5b+#tyAKnAbIP<*jb+Ew&A?j()k_vJ=}4t`|s^r^Rrs45zm?xoBz>- zt|JV*TY|E^2>`On>#vm7>`gk9%6VUF|Lo&-b>PGjFD+qyqa&2I%h(i+z+W6_FstMi zhc4FOWZLjol~q%oylvXT(FY}b>)DfeVqQ7dSMV8M;NCwEjQu{oR2~npPT$F}z~TSL z5m!>Un23LNSY&hGrtGNo&bI zaP6h&@HZ)D%k1wB=&*;#GA@$3sUpD1&Ksz6edGQK0jR3~OBR)#vkVc#}W#wgSl zf1!Lw#(vQYBAjiOPa@Y3%9%31^o!7^VmV)e&3Bz*8XdI?;bHWM{i-q0C){C6Q&Gx!b@4cunKOx1hSIy7gB-6#Dd#GZ>!C+IhH9Kqw^QucCmFQ z?muNVpPVuwmf1hdxn<7ph;g#)p6h$vPptGEVNTE=%ZiiK`3o<_T&?_tw{45Q@)zFr z{9%s;rQDiB5JGWy3Ec)UnFH{R$01NW)K>Y9L@C`3(a&x#jz+fJZ)*TdwfoQjkdJvi zFqUoMOkQ}~-P%mpVaurB5wl*n#1>vQ;heUU7Y;?q3;G2s$KZT?!=~8Pf+qqq2rJDM zzVv>q--Y_YajU@2eqxoq&Ao$gI3U1<$>iuc6B-AZ1#+D=~h$^TrRy3JildL1Y} zfx7!qePHP;`PPHZ_+EZ$$4dgj1lFCsqX8B6!L!f!z#` zz$ed%4OT#a+av$wyP(^7d0NKH)>zLa^cz@gXJaWruki z%?A#nqu;gzS|U37rNjp)*=n_!Q_<0<-Olfu60yhc&}}IEO*xAmZ~dl3Ec8pUto%(m zZ^fMcrkts}5)D+7VWVl>T8cHNZ4 zLNDhRsHg8w07mmsr`!qRG%Ao>=^zhd5%w_L#=s+gGoJ87#4p5Dop0>PR3~Dp-NnzP zzz1~uIBOGx#j^`dBJKe#6g6~FoXWq>s9KbUW@I4HDpzOiXk`h=a(IpDkxILuMv!s+ z1j6m}I==w@sLuNv0oNIwNUE~CJGt9?Sck&QIR3)mI2jwwZ*;Bo4*BOdhx;&HSXL7yh4#U6{%D3|{cZwx%`<94x)1)S~MgKFe-tQ=+q*3-szV|45xjHaq*3tCu7f1?0@Rw*6v z*54?Pf+6r}VSYT6aOkMF5(~vHmQrfkO0=6W(_;Bok)VRpfeMwrvIyIEWxg_P0qR0@^0oDBQ0j_qD?cj4b(JfJ$~j}nT&D{YZ4<`GUUYJi)< zvM#F|6RR;Ljcq1o0WVwoayWsy37~6@Bsk8d5DK)lR1lDQBMb7i!NHCvjELG)XJZWm z6k|_xy}|O*B)obR>R1!RAvO^lK9b6@%>v3Po`_9Y3u|w?L%xEuemB_}uc^i~!5O#- z7i*Mjgi&{Xs&;)){#=wnGro#hsC32Pk-X>=GJ~?Xc8$ZRMOk=Ukh&;fr1^L)d>CPv z`h_^IhGH)E1CKB^d0Jtyrc$SHT2Yoj7g&}qs3LA%fZ*r-YDQ*N5EkPXR^U)t#IA0X z)aSAM9D##@b4CkGeaFYp2)vwqP(hIDC-lUuXLq4TSefD7IEtrUU81Yz@bqFp%GJ)Q zV*8Wp|F9T)np5XMN~{2HxERQnrfv|@4C1J&5IRA&Fz{KuR&f|%1J1~h^x+g7dXBGP z>itAHKcHRG;)EpN7UuF1WoWXun$S4Xy;GO(%|FMbJukzkne1 zv)2ls2LgB!XX}Z~`x4r0D!h!lPtTh$Pm&lw#6AeLP!e7q;qmX;{#YuoYIjl)TvO;6 ze^X)^6c#VfG9mA2$x3S2Qs||~WN@BfZ}|fQO;z7yCiIZ>K+Q8K=k!!Rm z)t!>ZqS-e6zzwCQ29f2|ruwkdrJ*JZ0O!#3hU?TqVW}c8UnvV`5^W|8s7NK-&RodK zA47cvFa_3ns0k$&v#2jj2hwo0mo_L9++wYl(hRXs+I%UB+b5Y{vB^0tvD!=FD5u@+ zl|2*>BPHuU&XP(pEj9Uj`9@Zx@S{L;^n&)!Cz(I0f9Q4T++z+xH>c7U*;t)xv^5apRmj35unMJ~)5&l2;G{gaQn(y;-)4mQD^y@(6(rFS#nT9 zI@j*))3xG~5Yw4urCJ0G)idAHUPkdB=Ygt~J1lOij!f|E={i;~3ROq8Db=MqzHOkI z9?g*i$&TX0!_MQwn&aD+eR^w-FGY3atvM<&aVw5WT}_wbsN~kNHAiK?9!*gN>F)~~ z^AP(jvzx4e)*h9PduxwOMQn9zk4))YM|)&q;y6doznabG^K=SIsE_=Wnzs7L?8niz z`pAAf=LW={vnY9}j~dB2b~V;x+xS;moI2zDL=8P#zTw9y!J$9C1lkKie|%ZR*GMUl zS3As4n<5qR{RLU!tU~e=v90AoM|2E-L09ZZ?AG?P3aL!we5sJ^H{Gz$vom7nCGgCu zmq;{w?wain0S?cl1mHMS1nj9yL+D7= zs&01d`~VOIFQ#v2+B&%MXyX%)43lmG|2hc+(Ud8V6xVsDP=P#5Bhaz4Y0$V+V zCG;*40GdspIW!;iJPd7E8VwaCgqtV;j7|F!jM%_b5IhB?)TZ88x@Mt%+^d0NHVL}X zRP!3M?Y}>&(ux(LjRJMfM*g~1uoH*)#~Nz^WWg&00{`!(G%ty6@qjk$Y!gxwpLm)F zMz~azQUk~^)8~fT=yo#dB)Tcg?xe`eCNaY`rDUy)q8m)gjByfmHa)h_Rze-IRzPwa zoQYMZURMLyb}zCh_p)b({rn+5KJl$C$`q2bDg3Syr()&ibM|&(YOQR-S5Dhb=gR=#UXmEx^V7A~Tb5v1ACwI)|{RI4f zz<=`yR~r*I&KLNE(j)nkm4M|$i(?0ZTZk2jPQH6;7bV~(CbRepkMt!_T0fLOH-)zp zAdEle(Lz-aKRVSbR52nNHlYyFZt6zU?bp(tSOV3eTgc}u=;gK-yj)K#R~X?hX!89F z&hWr$?I!?2j$77rCowmOQww3F$;4%moLp#PK%2D@Aie;ii;JFmoeWp|N*!WzQY3yI zA@p+8s&_kZ$$p5b523_(tXptxJzyEx36wM?b$EwUqzX7G={w2$V;U_Tlwm2-y-6Xb zU~G{x5}OtZvv$-Y-Ycqsh&FZ@GC4RIX(`^7+nAy#A+mhK=eNY%kE`93XmL>(StmIl zxkGgP@W|klE$)+)sw6cIXdn{T8j{S}0{4d^{7nB1H_zc0QITgcXsD$)a|s%f^{Fno zPn@zQ9EP8ZLcl^w7K`deRNK5y3&uzQJXZ$vM<8>wmZ*)j&slewhcR&zt~jc~lG`&s zDk{4r`^0TuhmD>{hP-YHlM?v$lw=p!W6DB?ZAyFtTLY`wJ!%UiN{59tQfq zP$*gdTrd43K$mB`tD@&`!hU5>=3glB<@44rN@Aej2ZdS<+AN~llugJsr!cwlz}(7g z0x~b3kj|3miN>TCip*5J29hF)z=z3bs9+@lSW7@BTLJ-CEm4?+;F%wD9v7TAabj4o zL)Z2Z{&~&*Az%@|tR%f6IsFXGmej8#yz{dc2)MZY*C|PIpBWY9&o34xSaWlU7nb-Y zTNhWdH094lNvuZ?Bx-+)F;Zyj^o0r@=;9KkN#M9;_|lPxH$yz~WzKAYBW>Y8Gba$t zJwD1fz;a7PNLIB0%bjA15>dEgrmKsQd8QFIwf5hHi^gdrNU!_hn>9dqbyZBlCd64}@YP?A+oSFg$BJ7n7%v4sR@#?_oaPr$f8q0W1p-xYC4kT;v09Oi&Tj?mAbG zi-~Ctz1r21@duj=sayVe>74(Gl6+}u>V*$UT6`vvGPeGwUat_Ajq9YK11V!ffnT?} zc9_^tl(m20(XObvKRv2IGpQf>4#t@_#3;4c^>yvlru@gB!{Q#FE=Wg}9wR*jRJb7y zZu4Fu2rhvdd0`L;t_!^RN&)=Z)h~uatkwA7NMP5E>xmvVh=<8EK$X^1KeT8Ab=Zir zWZFVVXX%X36wq-|IUgQ&0)Y<7-g`ZwV-BTBrO0nVdn779DPa@KX5Tjx3$k_%nPsj;3&nPS-`=L&vSo^K!h38 zaXpcakZkuvrYJ2i-AVLeaOz9(;--1EVYmvGGlhv!Xqzd!>mAr|mc8S3V8hR~)S+3o z07ZwlwL)c02xs0zL5&AQQc&!`p20a_4Hz^flZQ*(Hvyq=)_Cb=-o~UW$bXhF#7*h( z1CK;3*((mJvBfih9$N0g%&j9GS zO|`#-a;!r`7aE`(%N8M-4@_f1>Yq~qCcbr^n9J@o-)90p``l-M7#Q|Bh=R41GhJv- zdw(J*`_cdu`1-%0#O33>^@|dB7`QNG6#q$xcg~Fdg|ffB-}*(_mzqP-r3Mjm+4nXG zSpv?Kuj^8?Tn8*JH7hJ|zSOLJIWAs@IeiRbogbBa(4}S}1h@KTQ3WUd%Cg6YA|iiK z`ZN@3uz?hg6xi1@t~KD8I4~JV_w0MMuQlMga$Rc}iyRlBwuCXh{0k${xHYG7t@+8x z3^Ssel~#4aru@f%V&Zz+(ENwd*c{fO5b5l^_2<{`D1<5b8kLa|d`GjA@aH1LDxY_L zQ{tsfU=Z)e?}~E&aCVl<^)tUH@z(#yPfhJ4TxJ(V*AaTM^H;dc=)2RG>@qvvJh{x| zoc#)y8C`qncA3$YI;@2Nc|eB0!>d^ZHDH$+J+C}m!ey48;xG-l%-*)ud|AR63i6yb zV+{u59=5IQP{P8dFsSucid|&<;rdBv3KbyXZ7Q(PJ~KLzsZfN^jE-pXt$b!2tUjSE zXUqk6WcbV!eQ=Kyj^BoA)YI8nO!T`g%bO@#$JGMqS0k`QCcj?JT5b~ zi@kEnMZ&n|ahb6#%j|Nm9cdFTGtNzTz%*A5QIXG#a~U2nM>Z8DJ=$lcc#q=|Q@qb< z-^(68vx^bzA>&)Ae^M%ATujk-?KEISRZ(u`cABy8m(z@6qbJHiYZ(3Tmaq?9qY9@P zXH;64!k*A>;hA;_NmfLNoMzV&6>r3kMO5@l6G-aiG>gi|;WZ;->^z1DAPNE|pBq6z zE9r40$)m8p^e9!58%1E9kI)qw!9W$u9j7;dloj3)=le@}6sqMX9wmD7(uet8%E#h0 z(xBB@&D+@m|LBzxs{mI}BqCKbo3noOJw!skAGcjSfhhg74zJiBEVh z%aDEAIYn5*rYM$kuDyinWRh%9UR=N$(;cEQKgUsVW+@bX&EveEXlEa1EklXBgrNSy zma+`|>n0JbW0@5s_3yRtq1BVp<|pO(4#x$NARpyiHbv5=hj$KniBfQaJyFD-e3bWX zEuN!%+HP}Ngm(~6VDVQtp3qdl2fb_BIm@v-kWA02u)xb?L6It3*uPm;e-wf5cL z$aZWxzGw+%jrV=>*OD|N)Mt#rK%kETqW!n3O_-8UT%Wd$kH)op6jVn0Wc6kW(}Fjb zEXUc=mdUtApb(u?PG%Wp($`rd*hsaoOe#iuY2mYd6oL=-B8PGEFhZC^e#!(Pk5;DY zZ^wB`KzW(>dATX+aVBSyVNZ^6$bwRJwu6tBL3+LQH9bX@@_3iX0tRm9DnK0J%V6}c zu!u*$pHQ6ydeQ}EEI0RZ5-iZDy#NG?PSduZ`kn6PFkHZN$&iX|N z#sc@@`fZ!)&?z+>0~ikHl=`ybqfoun=l8Yt902V)m%G9)Uv5U9@6;ba7vM<|^Y2P1I@inR^_nJT;p~ra!sgi+p32P7i&zRpIxjGl*0%^@J@fB97c$@{zBQT zF|(P)tT8QZf;VPth0Pnr2;RJb(3^U#{EFSVc>_@^<$fahB5B@0oXh;+4TQ1Qi#L|< zO2Hd@)i-Z^@_})P^VZvFzj{xEi7>8Dw~Uk8j%Az2UjMH^elIsCXC`(RI+a}}B)phhkZGN` z_OFYjb!9$nyD@(<{htG?yxo|hIF`A&8^vdpaFe?)lyYOT-!m+vO>oBR81WB-Zsk6B zIv#kM4-7%O<;O`R;m71goAxwlw)0mwF|U2w0>t%mP}=XUeb;k3hT#z zQktf?Sp|aDl+?~4jrHyJXnL{|J2N5%EX;ADp&hbF%$TbK=qMmyrEwdFYz97xrY7t6 zYxns==hNG6-{YRlTBRAE*{5aBivhVcmlBLHv& z;Bq?&g}ntBu*HbHCWH^}$BIJi=PR6-Z}>=vC;HlYBlOzbr>|B~Ar9cyx(EyZ90=4i z!ae5Q-ropHn5SP#_8$IC*-O6P`bDvqM8MqXZ-nzs%;;Yz``i1iUzBj0Tof7W0u$VB z6SkL+_Trg+Bq`-K;Y{an%DdfnEEr&)zw9$11ax>Irf{j_w(mo5#6?dR-|Unoo#dl$ zyMuXR<^8u|vw*bAJf@@eHWIlrzw9&NbVbY=LAB?;<~iwUPiXi|ezGxGyHD4vu9?B@ z|NR3n9by-W@aG=@>l5jv7T!*Q?oWJ@D&-EaKyvg%#! z({5FGkr#!6_xwWVRr7p9;C6T;dZC)-riPl3cdhh(yJw~)AUAlTE6Sk%b0S{}72n4Vy@bU_#Y_p(c z*-xfq0vIf}Pf|~6V=7SfVuBDK zK~}=!Fh>B?y+1veB|UdrxjYPWGNr@sN>*lcW9T568YL+-g z&I*KFaa+t(1(r$dxxtxWz#k;~qj}02>5|ogb6@iHghQP13y>#>{3X_E@A6d*sH|FF zf@d7QQX$XN2JAQ2_Xp>Z8|SnbZicWO0McH4Q~unP#QaN+I!iuaSpefVEg-mVn99^8 zA+WF*Q?Yy~YWrwu+Dp*BN*w&}0|*J7MePS^W<^2d*sYx3_Y^N~Bgti-UEb@%)#r~i zK%N=*ho6|1F3dj3(H&o%nczZf%t|bMroiN?4v(T7r0%1T;t2pFscy=jo05bL%8V3= zoJXl{Rf(Eak-g{!BMH4c|1wvKfkikXmeS{#+!%)nhN)kOL&fOl@c^|RA6t+CL=5&U zXwPG-OY$}x%fqPhKJ<4JdF%YZCQPlQtJPyyzC(C-4C4^F!Mj<3;cL`fs#aFpSbUDK z(sbHdDds-M6M%XoE-^9nxisColfD_u^dp^*hQGtoo!9k?#k)7c52pX~kMfD8#Op($ z49dt7s7>7(5I z?t)9mdYH*v#O#n{Em_imGjN{R6U%i1bVDXw94my~JO;s>(wHa3^A9K=sP z@)~PCPmBKECrHA#y;YTti{x=}4y zs&aUi&~aV#XcEkLh{Bv5h1+qEt12CPq94r1MB@}zR#uP5HUAE!L0WhiO1%Ic{k8tsS5d+;^$G$H(4n}40q zGDWk7Gg-bZU*i9wP{Tn`KeKbJ;mmIMg3xf79Ks8?G#s?Wb<%JcMxNYtB`Pd0u{0b+ zlK1NNqtJHW_N3u3_I(}|2Z7|CUQHH!d>q!EPE;&N*L1?v7%K=qM3P9hQl*e-M)126 zNH0U#H-U!pOkkwAzG*!NlPQ%w^qh-gJqHozOewRTgT#Op;+gT~$+z^-bI?;?eW~YQ zB%__nGREp(2r)R41&NN&Fu5F!+v(+*ANq9{WKKOk8J zRflOIBTnybp2(JEUFj|2W(y*e&zwgvDIvldfu2AlKP59Y9jwlfXtJh*^-!0j=^WOU zrgO2nHJzW#|A*-UI&nhNA%)^Yn$A3$P108$D<*Nu0Y18T=qWRaVF~!0;)%GP*R}8P zw*C&QR=kenm2SY&t?5i5jEACw-6>m^6rF3gMn24sK{4_guEu^C^S^%pfZU&8%q)ED zVn&>qkjhf=z?fe^NKQhrBwu!dU5m@abqGM06(jhPLG1H-pm@v4$%5c`o3H{L_IbWT zL12=?RQ@MAjYL(T)^!Wbsx*gw_RN;zjjrXh4$RU?fBss`NM-LU6%=)fOCB56m zOS>parUQz$xHRh%+e%O^cz}J-g7XJQUE4XM&ol++V>1)RLHl(-|@2<^$Q3&<*2b7Oii(3irOMuGOeY4Z)v54K{165erj}M;CQYsk0 zlSi;LMp(z@aAh{gBIdtmFAIKfFpENuXV0IOv@<>$OdwmLwBoD1*Jx9SMZ;g{UUuL< zKhF1aRStB>2va^=nUK@kJ}k}%;y781&db*mA*692ePKpqa>nV>yj#q>!un46Jh2KV z**9VXC171s9D;-U(K4CJ8CPk9366cDH`*FG839&S8|47-oW^xnT@t{D)lZo~IRU*T zE*@%n2)nF^H}Xt$_yN5dkd0=tFOzqocokdzX7zQh0#RzBne>`>{Lhk=}~ z$)?ssuz!iXnk2~=v5$P-4^4IW zkI4qIWV*x_t=-4iw|M2U%u%HUx}k$98iB#(5tJ4TtZF+WLa>0P$D#bWDHbdX#kg2T zMOrYv!ZbT;!OIAlTM0%-$E5@VCf}w3i3kjJ-5w;AC6IF9CulRyU!eqJDfd+9T!iFp z$J)m#1NoTyPCK;Vi$X1!zYu1xcNDm7_c#Z;`GNV-R)ev`T5=!u09P2NtrZh@`w5(p z+nUaaTXuE!;n{!UatlcO&DGL^FD|zh{FCqh5F1`6PiVoU6kF4pdeP)^P5O$`%C;7a z_1Y=`i$W}+MEb~Y+q7YoVDq;BZtMkX!7L$REsIJpCeoHsp;%qSK5t@1G4C|a7j(~p>I*4+WK>O&j4h^MMh9{3btdBH zkZFA>#*5iUw_6AW1LOhl69`ZaYbf%lQ97|(cUE-=M!(vC`pWG^_1*jN)nd+KcjJs5 zN(49@ruj|z3)6Hb&7NN_3J2x4i+3vYBJ{0$i?DAw`dVE`1@XY)K)?(wg9v!jpngKz zdrZYIwkb@d4|1s4n0;rJV+&2qeo}1~Y$BFucVr@eNtw zsbGk&?I`Bu0fi0nrDy|cJ|DNV)>usDClGQzt%BsmHJ8_?x{em-@v*zCX^m~!(a!j> zHcA70HJy9{Tae$GSSyV26$Ln>pnZA*p|FpKYD zN^e1Wr8e{9EZM}!VLH`!%4WKK%uEc;{jZeJe9v2dp;U=#_T>p{PcpjklwcVei+62l zF;-o=r_y=XQNNY*d14ccC4O~^?GQz=8aCcLzq;`t5+w=IIa!}4Uh`gaOSYd}AkLVI zY~9CCddB$v2#Gips3{^S2?qr9ZI~;ovH?faNNS|hJwqXRI60UT>~J~|Y3>I~P!Y!jWP{G977i&#)*R0d&^|lp8xz9HObzdhQG0>;K$p!9@ zZzjowK!LIy26WEw`1Yp0TX30u8G+k!DV`pow+}&f`!G!AfTbR`swxuwQMeXrH6}rg zb29*yY{dxb+WLgXs3w-@>C3+0T|EQgcMeJ+kBln6Lo?IGO1Bva&6~o!*_#{SJ>lXV zV_;rtS6d5}@BxZWQ5V#}v44pT7+#m!qX)kvA1cfq9n8StDa0@4?>LoIm3p&B*G&%W zu@4>2L)qRL&fDC-KQ(Q`O)Y%0y(@x^pB~b(NbdCTc@`zB%99j3%Sh+xQ@(Br6Q)T? zo);xwJl(+o(hRt~`BEL&u{F1H@wSJ><8o~PC68_OgcEbA?wrYNA;FCQ;j3@o^-|u& ziYGMpA;y%d_6*lpgbi3PumTI(LUq=LWh4k)r+f@@j zh17>g0&K-CwN$g6kEqlIT}>7;!?CHF3Y{r5+O-fNQa@#q-E898rH@P4Ut1OxvPsd_ z#EPk!uh22TH_dLV*s@B&tWNqzJKM?4&9K{%!f+@$wYC_*VOjN!;oGs36Ix9u zA#17BZzv%!?)p|jVX!29FC`=>hhLW%;xCk=a>QGIp=>2&u|R9+Ac-i(U!jAfM>%0f z3Bh0`7^m{y=dt*4uF44r;N#=&6x$=7@N$I8;oqND$fSgz9TJL};d{|T3HiYW$_y>6 zsk5zK%;tV>uh>PQ)r)Ov`LcSk9oAK)h$U$XHhVF}Jb4>2Q#7lfO_;t&SPm&;l62Ny zu<47r+}oz77ut6?+?A`{zsI>Iylra>vCELLo$B>~Gpua+Vw!fCI+gpuL~VQhnP3j5 zzHit}`7HB0%G0Ob(=Tm;*__?w!k$JzZtlC#~5spIVkTc`W)5Ecv+Fb~JjrGeh2 z10~8|4{4Ki?KF zQTkU{yx52C+mJqyy}51j+Q1Apc@Z+;c)`S>TQ4#`LELhtrX^k{mg$SjTgAT^c$ox8wJvr0?VS6_SBFO-Tn0N<1u08EN%x`+Wse5VwP=y!`2) z{N$|qrY_0zOW=aIAmnwo?Nz)e$|Axy!q%&A%)P!AIa@JS_G&`n(z@6>Ch9Ogrm|j7 zAnRNLeKP)| zKWFemn58JxrZJVw*kk$!gYie-!JJd8KO?Z6)0Dp@K{lE}JN{qdANXD+go*Y2W1UQR zq?3&2JK`3*e>^;1?oFt00#kFmj!%i``#DF(lJtrv=Hrv9Q1(q{utV@pXcUUIUa!ByDOe7XT_}Zok zw8N|V(G26nrr0_`PzHyc5aj|D)jsYziMdK&+dP3BeBG90oSD z9Tvn~*T)zCZJ!`?fP*`k_WQ7sYnUhYQg8Fbp@ey2Q*55dDO*LBR7E@a(jkdQ4vE~n zA&v>dXQ^VD<|~Bk6X@2oK|;tA_F>yVk++^428zuhHc)^?(he4jI%`(L|BWg@5Oas;}*EuQzL!tQ-`}VadVWHp}N5ES%&O)gB5FBfLM@X?u z6iomB+7^oZ6x%s9Ll|f}YE>-x{lK)IrCk?r1;;>*+Q|~+d}a}P<@CcTPO~zhJ}KR7IE@TY@|I{ zJr}yV918`eui9qOElQ?$X|3WD1>(i7#d>H*Wiy_t`Gti7E2ut%g#v3_9J*A8I7!|Q zaoA~xh2l_Tq1ckZ77Bc$MhHg=$x!VT-G=NC_PNN)w;xa;oytxYX%PH1KAe44O}Pvxw-f< z);~Lpf|C-f-Gsln+x>}cw_$MwXn>D6`Qz^Y` zKNpm#P2GH8cuQ_d+(?b6z%EQdc%+uyBnKOyRu62k%f7x~i|`;<8ko+{ zrGoW1%$P{^p^nt!c4h(GBj|FA=<$57 z`-$Ob0NW;sc;aKh3^4>N8@5S6n?#I*@K)Bo0qVvFio%EY;j<}Woix5q7$@ z_BKu)ij0%^fgJl$U)V7-Jvbr~OM3MJ-!e|=cDEE9={q1u_yn_8zQR7q#hG81eG<#F zPEg)qYa1vJn}>z+2jg>b+HKwY&yaJMB<&Yr|m=XnWe~~ZKC9E3=cNmh|RjOr#?qT!$Wz*_C{a%@1h9m=51%&_n46P zwx4Y#FlVkMBa+~B-nQ=^X2hz+-@x?Iq#Im_ZST$FB z--tIwF1`H=u58Y4*iWRw>g+`lPgB2HJK~1NLkuH8c{ot&V1n$?MBo#G^f^-%pg){U zzF;i7=}@X+Q4y=5opG<&4!_!~dQrUYF1@??LI6jHa{>bC@RmT5E^d_F;Rko}nEJwh z|L_0(&zi>Mcm!qnvTmPDdg?fdJ7WX=Ou;3JERT^Qq4GTQKmXVNv7h>X{`p`3JYV?l z?OzAB`#a2zq6`)1X-9eGcV`CW&r89DIDd5~nEv^v;WO{PzbS=TZ0D_pV9@vcy(mj1 zG=VAAm^Ir7WZo{y&;H=+nF$0P%043}QP6%2!dq3s;Wfa@0`ka@{C>~lzSn_TULQTV}t@HO$ z&+AX)EOE(*!Yt&r$&^3*l9egwY?H}gT5Dxxzh9w0;arIA!YTUin!zp&pzra2~8^Ed$Y6l%!55yD# z#fGO{%#xS}-g!C9grfs)TStT#KlrZsQ}u5Vx2#P8HV=M^{Dn6GnzxrmZM-3{fb&E# zE5|QnB3s?xNG#I0d(j_C&hFZ09b$i^y?Ef4#VijFbWZXx62UbOY$EB3t4pm26f!-2 zYUu^?zN&=n=3!tHG81a~x)u^YOGvA#rC!aCBh3W7fY9hpU8|B;kle%1R&C>!HA@eR zXLp!>AhbI@*PXVj#z|t+Al#TlBv)YHBVxRFmE%MkTj*@_6WB|gaE&{#*g@nBV|HHwA z?aov;4t_D?0R*z*8q8+gLK5zSl5R076z}suVj3Dx9pWX@7seqxT#u;&*qu(YN_4=! zG~3|$-K?x;E3iOOJb%P=64&uxfCPcrtLmZQ4Z&D}Pq}cfHdlN~BOSRjO2j$P9K*~*OUAX*Ii34Z{s21?3No8#Lyr=|FFy101X_I_fLmYvk zq3Dfk??qd41LV-4a4va5tI9lQKl>ZBV+|_HI+L1!50_2>>a} z@&ZH*KF_S#Mqh!1SYv&Os6CPe6zfz7{E|`#IWoXb+wI2moqvDHsf!O;+HBpDYb$8v zG^p4}eJe|-k9y$U+F&E#6!BCL-Y70MNe4a!W^hSW`CM=b)$y}qSLNiuNKeM|c@MBe zyod{1lGfk#d}e+TWCGV|34f_q6;MCz%CscB5tK}n1j&F#v4b>7e3I)2C(4RGA<>Wr z1x2Bii8x5;S)KP$L;qIU6wD0Q818xi$H99G>4KxCR_{=qmyfxkX&==@9rWkw@T^h! zV@`MP1OgO+J}6+vRqCtw4RGQB%pgVZA}T5H1sBtjaVkJ#8!gTu$?y?!atnE&9wPgC zKLMvuOXnAGdY5s31GC0s)xALYAy0DmGTI&n-Uy0z>HY<1892M*8`uw;z32r;+`e`% zV~@r-&MuCT?k7AG@e46m=NmDb(TkXUck**F@H{edo9Vt=JvlFC5Clz+yZUlj()r_uhG7?Y2`swaxHQI!25=46(Tfd)T?T$w_przSV7xk1)ra&WQJ zg6a+xgd^#3X=>?j6a=F@gA~2=oA3}g;}5|Td4k|d0f~i_jG=Q3dBqq=p^j0gTnM8R za92E==ip>uIpQOjEU6)CYdVo(e4bdo!&lyNxFBjtI?W|1>KhjpkR~Ct0q_`$jvml0 z`;!d>xF{V3hY+;8|A>PqnI+1fi!yn_p-o!>?mI8*PCzhYHzQ6!wdWrprh-MG3%**n zG$JHRznaH`yba2AU5g@(5U7k9rBny=EFiaow$McNe0BkHhL8yjcaeT zi4cfU&l8E+DLm+2)oT*YMTb%&p!H1V=PI!{5oAw8h)Rjiv|{s5;ohXh3cEr1b11|; zB*B&)hM}Vq9X>5IVLzOy?lWmp5>M7S&>4h_8b^`NDaCQ#hZZJomFw?GrX!qL(3cm3 zXEExBSJ8oN+@!;|b8)K-8I9kKCX^b+I&MPcO5$~LA*3w0bTK7@mwFZz0xJMnU6=F0 zh2=u~Fv7Y)y*kI$OuA{t1E_JZ#QCrsa9Nf#Mp>w)Duwf!{Vrs1Gz2MTV7IeksD84% zU_Aju8y6wLj&@sT01Edu0iH_J%scy773y^Kbw;PH@?-M>1(cG_hZDJT+{f2u=>0^A zAAl&t4uO_W+eVK)1;8To0U9xPA;oVae9zvcEpF&Pho${4T_rdSNW%ElDmXdTTLpp{ z?^N=3{&x`sZwYicfJN-Bq}2rDfyM}(raOrMdnRN+QL6g1h;l;}6nJ@1{;~|0i7{{Zs0P(}kw>SfSRv|zZCgcf4-sf< z695M&Fsw|b)9VSKA7gnXImPvOBk@WSMq@0udNFCUoy6&3b5? s-=a*@gX1%>~GxjM^Exp%=j&ngSaQLuEmcIrUJOrOE3; zCr?WA7Pu`KjM9?MP4&CWnr?6%j3cdfgdrqW{t%alS%K2KpLlFverN?av-!ju*sC`B zQfSp~ua(&Sw1+KRhawzuu9zJgtqU%>>K4?Rm~P6ThzkVad7tg@fWHBqCf5oq)tJq& z_aY{z3+0&x-Nw9F?5k;)3YJk5#bGu={$YKeBp=LU`5`C6)}v?#>!d&wVZG_kX|khW z?dU?POVNFkURLj2^m)sADXc#Z17g-Lh7cc>5eTRCzZyXg*tGG=%+D8E28@FQl=sP%d!|qwBm> zMyTXr9D&ZblKQk&M%ek8kewXLu*8V_j!~?I0b6Dl|~rx*QGU1doDxBkwM!n z27{^N31qN6<*1Eg$Zc@cM%3q;M{VRzEPbnud#4_?@pD*&=4-aPZ%+@=+}dqZU9^SN zD@XxYhpT)LDi6x*rDT89p%godx0`+$U;Lu{=*}NP$8E7Eyr&`{*b}y}%44H@1S$M{ zi$k5mey<2yAcj(ER|vqSobS6%*Iy{-E#i8OzfpGB0^Xu$?~FCR^_#Ly)w|EXDd$_! zpB=WaDLZU|C~^z}*uV4REurQhd0ZT%xP?t|+(LSA?;SgC;cerbx}El2I>(G2=m5YY}QQ0vFacw@2W@TFuI|0vCWoB{PL|(B0mC-%WAg0)GL! zNr4OeL`zrX0y;rYqazp8$rX_adlV75z+skr{>~fPj$Gh8&2~jDXeQToe(#xf|b!~!v4k% zUf8{G@Pc}ocJzYAkxuji38n1l1#7^|WyBrLHjM#*;JQ2O{ zQW*bvJ;CVt>lc_Rdw&DvV6RT_fKlYa+d!tl30x3PUEP5TY}Xcn3vA{)aDkLZenOn+ z;J0-}E}ZsZ;h@OEreAOv3V~phmUyBU)QKIuz$W&pgBLi6atAMvvhogI;1pO9(^@}8 z?WiG+SsXihfs_24OYqt2p?7Ja>h%N`vh3&uEp@x17sL#6D}!Q|6TR?rR|s>iIh^mI z%>D0Bs=-XrF1kAtgZ@F;)xo)W6bBAev9k~^1x{R% z9Ah~$Ag3+aHIhMzDl=0c`Lv|i$m5cgmHBSBtc0QFet?h_)Cb9zh53mLO7oHyhFK!)~(H8-eH;?_0(=|?;FU;@zfyVq4{2~i_Jjxbi7}_IQ)yTlK4CJrF zIL*h~CnLoow@=nGC7=TBhw9>hM}bAvlUo__E>{*8kol{j^>M}9eLd!&Q@fu)B%rRx zgm)_hh~qD+*!x2Uls|{aWr&}VSDj#JUKa{g|4_lk2K%TwKX2PqD5UoXq8GWR1nO2d{tqTo_|BF=6?5 zFE_<*TVe&xzTLXS5~}Uq)xc4D@^%dqs%Xq>3obF?wMQ2#ik~MC&?VkY7jVy@0NVm6}} zd;ZDPvdft@Y2RO@y>G1s@_ zi1Z5pbr>Jt05HTly+Au3|MF>H3*CK4=dA@ zUFJlkP;>K+^5~PJyGrOPwFsXPb#+zz^bxRZW%aD`zDtG0HlC0=ceLd;ms}w<*K++CU@-* zN@SDcsb46bV8&QVj|4NYiPnw;Grp##{4&0Fw#YAIKAsgX!F;7`vSu_53~lArV2ol- zQi2(Tlwigps?2tjiwboU%%7YNKmCnrN^-&8iD)w+;DFY>gv6l+c9C^1D zyN~Uy2M~@uUifO`eD)0(zGjDPt$^XHjq6c8VEAfRmjT=1b#NT=1`LDZ0Ym&qsSz-I zy%tb|9x;5imn)|&k<6QXJz~fYxh5q)HK&;|eW`j13-H{K;j3MIDi|^ZMi>IR3>gB> zt=&v>v?FAADP4=&Sf;C(A>L7QvKef#PQ(l`$}*F3vPe(Q>Z_O$S8|X2w?@nm>(f%m z5R=^^WEkA7nBi?JX!yzgwmJh~9ye@AiboBxI(tRf5Nk7A3L9d~?S>6s?e;g-N(G}{ zUFOUkJ^J)tpm6L!#=2oc%w)S^LvG2NZH_G)VZ+bw;^t2$l734y1Igl?n`*uU-K1z( zG5gwt0LPdu<(dhsvm@6`Fqq7{XXIM#FT(%S%)1AuW}a&Hm6X^(C|^1o3gB4Eli0|2VsRP$?4fbhAg<`)G5nz?3R6qPJ%tdQf)cm`h0 z$vuwyqPRiYa7W5c$!15|@nka)R#b>th{>Y1Cz~&&3zDUBk2Bef_LGy%cxfAvYz7$0 z(<0f7K|ebsn*p!_(avNuV0n6tWHUaz=43ObCI$Kk_jHrZn8+t*o3Y}(dbT;1Tj^%3 zmHj!=%>-pDk#3F^SGxJOJ>7gMx{ewgQ6k+;(6N%z&3NlmBHfJk+L3O?D0j*@gN1Iw z865PCNH~Mdl?X8VpL49|2m;^NZscDX}^ z7ph$Ioyg}t@Nk5>h0OSAp2#)dw&$8JMR#6fv2L;%?LAVm8E>68+00S(NH$~8qo-^$ zCEt#0Gd^@w&o*OY&7HE%Sl&H6+ssL;k!{9m&y;M&a-W-QCZ^erWHYhg)=f4$aFHjQ zue4{f`9mELfr5B6$$m7rDb>wOr3OXzV=YkPZa!LbwQN4t!bq^2kJeU^?IM~GFnW^t zSSz66!hEbRXJOZCKGrJxC(K6!n5pxcj|L|*k~lW_nB=^D5kBVR9<6u?0Oli5N8Zdw z10GRe0_G#IROU4w4QHfe>k(K!BkLijg!O3HCF_(NHht`qb{1eSG;M|?(7 zulUfnTu!Zx@AMv4N;$6aXrN@82pNySHS5=SG~BbNT;tK(*LVzyj7R&0dam&Z;Iom9 z$4b!TFdl7*R~B}a!KW(@4M=9U)PqcDJ*xi#>_^)#Bb$#Xt8?4Y z_8)=U_*59|I@_;btZ_akL5Xng?Wdp2{0#u?{zq ztgm0jMbeq_J|6U$A^wnb25LyjNoQh!)RbcWe2R!0L?pUN=a=s-gV7VXLAmd5^>MH5 zgOZ+l%Krx?lFoRl%c8cem zo7qA4to*ADWp(9o8*=cKa&BY%P3L`HZJ_QXG1UytHHoR#tNYOiF*UQCOG-JvrhrxG zN;$vUi-8MSqTY^tT~L&A#xGRQIP(oNWt=gFQE$fC#&{FX{tc0EwvR+Y)BnEXX_Muc zq_>1KrZFX)eVRSt{IL*t#r&%@y983RyI}`)12zA?_Lkv3rp2=4^|s^${CA2bIQ3X3gZ?b*d?W$KlTNT z*>-TVtBft%DhD8tFt)JaPjTamGj|@Zi&50N8BjFE7B0(dC{j#9YLArr#RJXbBkE8* z9)ipNUV_PUzYjf)w$@(-IJh$mQG)?S+!@~%fUxbvzWnEASQRS1v2b-;lptf`= zj&WFA$QVFi0WdM6ofeCfj=}H;R5;NWU!n4hGQ~m4+dhmF*dXgn`%#~fcI-w<sDB^hu2^+QF4m_aaas~ZB|WZUFKH|)9>YgpS36MX##icLsEgOs)zo7%Y`$r4tO#%!i%>WQ z%k^atU);ZNeeh*{RZDS`2$+;1LO4T2MdrbXt2OM6IWOW zQtj{C4%YGqeh}gb5j5rZH`5xBkS1Ku{G`M~HrNS7QKJ=LTXuokxANdfN;jZEwaK*; z$hHk7mSn04qM%H17CAjRAsn_|i{?4=3p5}U4p>)^No=S3NagLZ=rZD?9J z5lyVXkO|XqUqZnOn18}WMbo4#*8H?+iz`;iJrIMoHigq%l)099&?|78no;ch9NL?M zDtyB3jc9(fc`ToJqVDa(bmOl$*SYXIl+)fbnpnNAv!$G`P8?Um^Z*6Sa&N22{ zoWG6yhpmSt%+NMjjsTa9-GBnnq!aogAWC6NU{Q)!8}@>Y-AwCO8;K2k?Ba?7Ie;@h z@RRFp#trsqzRmc_(`_I!os-#(Va&L|&(kP+5m1app1=dRecXm@X~$Rln4SecvH#(E zf$*IO72^YCQNUCjHN-0(N-5)NjG>(nK={#0CV2o8@M=FCwM&M5_zQPmZ`^Z{dOQ&F zg@8BRKuph}q|m?@ym$N1#kI8}UMyzE*7ElFD^_>H(ONDb$ePl$+Eu*5>wmT7kEt$APSGFk(4#wwe~zM>-9&ZG*Cy{EJV5tASX)Z@Vs) z{e1F0)(0`6R=1*Zq*VNhxo;Ro$EVu=o9njLx38AQV0mP$mi?Y32-e_djuUN647vLD zXz>ep$1Z>Q3YI+EOu`y5#}3=B*wc@syzf$u|LeyWala2vej`fBzeFKW9NMOD8Xd}4 zyV7*v@m8BV-dA8FdC&pXBOHh|fn?q)(=*oJ!VDGTu)3~S8|5dq8Ww7OwUH8Pb`#xc ze4=6Z)CuSO)dmu|t(J+y*HcImdE1_7$lf{S`%DGSoWp@p^HY>D~RU#Y6G0xA~E#XFkyJ-gky>Mv~Azv+F`cw z=cclBtdJRZ9VZ@R0TJcde?FI33s7h-O2`N0AliXZ+LkixSCennwml0VzW6Ro1ZvRG zd^!9!Dh?W1L|A2y7G3U3_nB(mhTIsPOk~e3JyHIW(BXl>uteK}U9UD}m)dE~MM-OJ zmr=Gu$|A2O3QiU{g|8+1R=mf)5Y71VMu{eCV;~#{v0%;~J{{7jy8g4tfuYn8uEK0q zZYcE4^dU+Zixf?)pqb{ytami(46&85{6(#rUv$tjgcwktig3St?hf{Ie!* z)<`?RDHUO%Od!UowK+Gh0e1sNhAb!okppE>DA~reW~U2fwiMbAeK0AUNk|Zg7u(vi z6#sIQ?bft0Tk@t&J|Oimk*0pxW?DZ1#62ilY+HHKOE_tSO9!LgTM4ie{{J- zo@M*G*a>wq7Ajwt8Yb_3iSxu57n({b*~pycZJw-tgCZky@+qWyJbTa)>KYimy8kCfh3)5O>c*(X}`QBOss8Yao&rVHvtuF>VbQ0+{$Dzf3`Ahp4+@urn9wMnZf2^WqvTeEltPi zg=A$;9@YsfbMihhSeZCrkc~_mvxWd86KRx{M4k9E2}AL8^k_4F0k3GC+AXY1n@Mvb z+|ab02!SL+Gj?hinjasixX`~uX;@3NO~lrM zgG{eB&gQg2Y?`nCK#}}<8BTp!NRP70?8sGt{LzZ8XMGV+K+#O&+NNK1Fet*r+pFFA zKyj_N>}1De@M__ScmHw~6sV<5zlsD$^w0u66fCQ7OBJ{RcmTKFAmm`uZ$U^=QeYMe zTpT<9trwt;s{pZI4E59|_b25pfFAluKz0Rfh8G%7?Y-I^3e!sOF;!)Re~II7_|Q)2 zVajLmit7*+n5j37YO~u*?y>&L=d~D>4z>8}2M75AQ)enQ z5tM_p7uwkCtV#e2&=@nTMx%VHj4kklJUIym`_Al)4%%xtEI;_phw5c(Y6{LyBYZaTPUbLN z3ENk^tLasd)w(@uqY@s-lVPTmC}9;iCbu%kX!_tze$@iLFAq#~AiBOKL6#L@q>wlW zG?0-7Sx6@Si8P3z)b6)5$U?QpA5ViAQfg0Wkj0ysQW|90wVt^HPFhDE#K2hFTOMTf zO)gO$WHDVCc@V>Q?QeMyr(BV|Ikzy+s%X6(2jgt>?^BS_>=M= z1|_?19>iv&raXwv)_Ti>*mRDWdrY#g@*r4{KPeAlgq`vri#L8p9wgR0@*p2vKn1Z8 zA~$ZaQo(M<+Opi>nwW_3_ET-Zv1K5c5&aTAWRd36R|uk>EHA^7%~ ziyA^dFo#E&{SCAJ-FE~NN%uQIZ}St-pOp5X&eoqt`DeQ?h${(V1kLx9Mi^35-;m_2 zzA}1f65{Pj^GUE&S3M!(@J?W2%+U6{{ZMBgo%C^-;ZKs=729=6ny=<$^%G_&P8M>-PX5F>|5bD0Rl_AuYk!>qO zKArX2R=bqD@eH0ec~{omT=v{y(&dv~Ww5BVC%ekRz!rrZ-QeYM2|rb^e%n$~2=4sr-K=`3#t!G_eQ(vz&owKg*Q9XnS| zvBsY1@3<=!k3WN)HTVMJD}eVJA3&tu+Gk-zg-_rh5iW~X2371!!uKaXv+`se&db6% zs(*=s@gv)}Fi#Is!wdI`@)~O<7gytlG^&J`-~7(wW)Ac>9-ozS={*J{^ZkW^a>x7* zR{!2n5ZO7#=*^{Ti-Iyy_ZrVzF$C*2_=dS%S@;-MQSMk{T$Q;O{|dG7-Tcnzp?Qd3 z&`hOwa4KUEnrI*6`${MSe)jv?kYc5ye5RYawi|m{KeEVlR%@g&3;8!I&9LvZ zdUckV@VnH;>fpp^>VeCC(=XuEMSceWhSMl;XGR*O+Ie|IX_BLWJ;MHlQHgg{?#``B znPNw~jF~%o*yUMw`#6)u_OkCxf46Dqd>nNj3z!Yu{xR8=L-3i!+{>8%SPSOG8h>;w z0k8B?%+?ReccKiYP*I(QpM}?LZEZFMUWdibJI7%f_a%I-e6Z}LL9uH;%Y8J^^vz)Pz zEIm+j2|daQvuGI~fXZL?%TppeKyuXzrIk>+y zOC6IHPU7at=zUUE~}J^AI@)Rn1xC8~~(dkt0eH zId&slk~jxPl&-`%4~-H^oV{tlxx}$ANaBPgSXtm~>e1-2rd#?DIIH;#NZ?q0D#@D( zEV3(bynQ8(C?Rr82aY6i7EVhikz?nK))hIns67dk-5`OJ$l1UZzCF%^-5OHI7G=Ll z9jkv_N$QB1L+UtlU#auX?pT&WS$3t)K^ppsokL?i#ExC1Ub5KPw4FojY6|k~dfvom1Zx(4z zd_ZSYq4|mWYy6^$iB|G{F-L1fTeB$T%pR{gL{RF+^ke;^EWDD*Ch%3!q2 z7y#=e-WEQJTb`X&5Kf-IzT5I8jRhHvLM2lsM3`+zunW$o=E-}RXCD*X;HRLxl64;@ zXWx3f4~4_q7QM;dN>J#fO$)J(>%<`m-m;r#NqM-Aas=(zbh}m>QuOs?A!?_EOA@oV z>9nU`utY(9dxtrCC5>XN?CwT0d=;n2P4nm7i}N>JH_yl5j-ZdxrKEqwm81Thw+H7T zenB&p-l4gSQD~}tknd}OSD?uu5ucKUAPZ z zo3J7h1-zFRaUuLAcOu&1PB_Kg3H(C6?!>AR3GPnV5M~Q^VztcKjp3BU`ozJwJ*-f) zUthwe&s^c_OV||4@;zJ@a4)m$2O;3tPT~O{Fog`w|Oj#_LP?2EES2$A`TiV=uho zvpxp^XaD{ir9g~2<@>+!RIE_Jt#=TJPlfop6g$TLf9R*)kFgg!J@%Ty2NfW;Dn&tQ zQNDvx&PJ%kC69))-n@RK9v!Uu}}pdjJHq^Kr8dZ5j*O zD6iJcV@QP_@#W}+gLb!#Lug+=cpi%b_?L<+c_2AKvG+WXpje-!59M1PNKU(5X8OQ= z7zdxW_>(B_4;J+{o&Z~?9#8y&rTuNwtj1S#pGp&)_GO%GhPZGC&i<2jF23UH$9HkZ zQ=~bk{f4C64U2G<|DQew;c)L`b7uSV^03Y8MiEHCZC5ZmVeY*XMt&72nA%KPojgQ2 zR_%mmxy|>1phHJ+&n_3D$gOLWFDnoo@SBXJtIyA;Fj+VmnZMEM_}(S;r5s|cbd+Ua zr^1WqR=B}WT#$x5NyFa&@9>}ucdpr>RxwrX-tMT7>UpoX2l?^a<3V5%KRK+46fs7? zz*jRcWr{enUktZzYv=Ei+f%YZ6l}zT8MAj-fb<82ilMwymO1%o^LK|u&CouFia{T; z_^AeXgpuP2B?h+C-r%4b-%Yt(@$;5%;Ac%t!=V%VX&*8Am!&G*fs19i^`@DjKPgHDwE^C}7Jts<@LvYv+!euby!XbsYrO(3rc+3iYa)(C`Kf2G8`{h@aP z*qd&25Ri3ZQyTQ?Y&WtyT@?^!*a>KB9}cV|j>FUNII#*cMG8BJ1`;-?W6RdDuQOlpczOjTRq&q3S7%IK^;s2y4T3+TB){b_+CYd@Ph-O>k! zGd88u>6;s+iX#eon9sttStAAvwWZ`Ay#pBbpqM)Zu)2PgykSqRoy~wYadPhVm;j$d z8`@${S{_yut0%@^*#%f)3(*d?3x1$#9^m)m0Z{U662I~%<+WpgA^QE_{hgnb0`x0T z+#Cb=JBBt7&9w)Em#JkMPGi~d<#Ea5;e?EO4Zel|Q#J{)2s`(<7A3Uh!UcaA!$W1a zpI8$^29VB zd&0C^61oNn_O910DwkyLnXy^ydB6g`y}?lBYQ1AH9J;68F{-VPzynSg6yy8>aF0bo z^a{)!&;M!=wBT*UFveI8enVO>edO)HNr>k)P3aAq$QZ+&d5-aYCGY_2S<=!t0dO#A z7w&WvJU!6MbW^x8Jpay}Y&BJne~q_A!VK%w0o6+O24Fxn);8i|P?g74%AnVT47|a;+{KxxK=h|75pWzRNqz!r zZu~+?21fkOFO-K48s5E|9NfmyayKOLP$KjKe5v-p zAp4fX4aNb|u0*vji$ZqfV_yqz?!quXt=6_$K-vH(#ozF1}474ZOk)M&Ir%)DB;9g{ia zrW!71XOHgTqoQ}nw@V0m@~9Smb1*(&(EBw$gKiB=Wd}-8UKETM6EWbw)jL)AJH*Bp zi)*2U7X&nmpH0jyfkGxeR5|@SXOdgb);7cT`I81^?A!gj9`S}o#)xfdJ zZQBz|(t(ukpk&FlY7P-yiice6XJ>9s%5d!2&3aj5X5f?^ny)&v9EHSQPlsdN?c&`=Q}<7(tmabvu<^J+(m;I2Oq=)MUf*vdPlQ+ANT*LJpX-UP6LXzI&1(}52$b?~K+ zB9mL~qn#kj_!JFeQ*fV3wQo2@Vv6Na-RTST$WOvPj^(M^e{T!;x zpWo1)46LkN{?Z`<%ePOf#g!=VgqD*WZfVC;I zXrDCzQdD>&Js%j7jL3q%+EW%j{h%?nL}im=9+RcfiiPt-rQhs<*=d%>kmi!;#J$E? z4DAV^lX10uk9_@?C~zgdGi9j=wMqeOC@*V2@=Ljqz#`fPESxP027pJwAzi_M5rwgf zCJY(mcZ$yNH9$GC`lp1 zc+|w?nynq)LF-Vfx!$~lpqTkmNtv=>a2gUT{K#lUMKG^HLc3`N%U?yXQ8+srSweoP z?TzO|+n2oL9y?;-*pKUpUtm~$tca4c1{oZ~noJbMxVm%L4Y;Lfj81~t|WqHLd98P@w@MHiIU8xQD0zBaq zXhIhQ@DH|_#M^{{wox?C1>-x)-uR1wDFmUrp`ukD>fGGmwn5p>Z+9A{BphyRF ziNad`d@c^Omf`A+79rAqM7Q(oFskls85AeN)SaV6_i+C(>-0}V`v>!FSBB#<#w_IR zc)-dXilY0B_79IMMYP|}2#h}w>__>0{}aLf+m2U|lA`HjcgCK6p}2>9pZY9&=8iSIM}29A1uup>*fxnm!6>3z69kvC{cQ1ZB%xs^u*fxPpUjY0lzXX z^{_tvkw7SFAI*N#6GI}S2upMNx3S)TM{!7xdU;O^TJedM!>8Fsv3t(3Se5#d|E^Rm zi%$%)?`ZCXuvZUDe^h)TT)-$kF(kQiOHICr0@+(oz7u6fCH+8b+p~-*(L}QVk5#=X&LbK}8r}ESQumfyoITB8s8(>_BcPjxR zx3piwk{;D|EjeOANn7w_xsQ`4Mb$8sQnWYaZVJUHP@QEwaLT(^N8l<*gCtE8^CThu zrE@GL{ar;RGEL@?l@kX+s&q_WJSwt1x6pIDxmXIy;8=!Cii0?s{eBo{gL5*bz6ZE+ z7|~XKW1_8G_I5QhT(Y5D4~VR_v)%c{)SO#^AzRra^#Cc=-TXsb7c0&BkmGW#cm3qy zHsMOs@a+UVWvHVnW@&5B&DuCKTxrUnxAsy{?o#v_#tH4lHgHLl43BpF9ov1X!BI@g z7&1mx#t`9sW&66TTQpn+r=H{dOeeeey`wPDi^z|VGa3r-lORWx;=AMRYkemgWEK(+hYK^ zC%HYh9j_oIITdzhtJmKt;he-%KPchk*p<3_{Ytq{aaw<+#M|Sk4@z=-E`@H-#DU*% zd(eDydu(Q(YuKqk^ulerv>|13HeZ9Nq%dszar_mf(@9+fIq4P&Zeow+(YbaqtQ%WR&3KC=h&eKv5PF`RmNRIM2qLmW%f*#ZEIK1Bf+ z<{Z^dS;ui*r&sTfD`DH=b~AWJ3Id zJ9(V5`#}Vk%!{|Ngk;7EvlNb6fc46fdEsX15X^S&=p>6AhuE@QwsS`%WhLy~(MjPb z9GZKR%XVh$lq;AQuBg3&`L^R7O{zS^o*(l1jpE|-KJ|kVVt93>zkX8gQ=HXbDe?Ar z>VuL5^QBNQTf3AinU`m`LNen-VUo<&?r!=wwznUCIldb8N@i=W6p&iIMh&RYV}bB@ zC9|z>3V{HyI0vK_@5Q-)_r8Zm3-_B`Hg=mS8U-xoE0?V~a}8=Ru*>BSjuy1A8~!RO z6QOt|-Lji)8>gxDKMvcPYVp<@;YS`ozm|6{pRTWh1?PLzbxbJ78Ex{2-G`c$%4-ElJ!HF(j_4mFK!WPyMtf>0ib*N&_6M9Ce;yXb5)-ET4b$`1O zBIwrr8O2kdc;JTXt3YYzZGRD-xa`!UY-T1^__qZWcMJavN43a%8w6)9C`q)(@BBjX!aoDZr4PWuKVY~{In0e;r`k))4;F>)pby1d&uQyD%U8!vmYP>kH0F3vFJ%Gd{q8@umVKu9)neN5$N9r zqaTECCq$Uvn(l2_n19=)Q*t*b2RE)%`V(>f4JhM35$E4+-P&wDv6a2=wn>LT^~0e_On6##Y)m4xhH#?o)=k{=Rh>2!Z}Rp1&S~6H4>&EC~bsc7XUu z!(bLhS1!o|{Wg=&J$L$@C}N_aieEu=8-}cS6Nua6p^Fu9ej8+y1_ClP?#>B?gl~MS zP`7FFb42LTbQD%${@q$k0>k{fr5OLNuACX;S3AY{FJ)*YAC*Zn#=mi;;-3`bw;?(N)H}hCWa9cDqgXiR?QrNO6zUtL zumtkT);YC$L0NK`4{lWfwSDDzr9XX&$%`{pF;6PAwZ{V z!&7@iI!zlMAzNK-cq&eMg*rU7>L3x7-Kj;^i>t%K>d4!psMuNR@Ra3+_|%4{%*dvh zHax7X32k^v3r?Um6eX?3@e`S!lsswvJHJp|8``_4@4&{vHV+@8t;5lw4efJ_SD{x< z$X!m%U-G~-qiJB6^X{u#wI*I*wD`pEGlZrLt+|pmjD-(v_`wQfk47{|3l=;uWlvMx zC@J5WGT8J-KAN|`JKSELs9(tI;JtCEguReNd9?e<^hR(Vo}!xAu8EeR{?< z9x%t%&5g$P^tuBrTa&o~zu%fs)p4xd6}WW2-zeKpTTd-))Ixv_O0K3d!WrrBUf)XB z{|uc-a?js5ulsRV`@&fiGN$Wq?H-=~S7Kgw{Xe*w9Db@?6Sq&#Dmu*Nx?7a6P_3OY zP^^7swouQivM*_&p4I56$wED=E4S+ws)b_x@UK~@Cj#QMP!Ed%lZ9%bS+`-$LbU*J zp6{))otW?4ggl_X--NR<#J<_)qyEQ>*UUiWEQWoGy!96_5z`+%JaE1%yKb6*2EDJ9g18#Liw0yrLefhDzFKKQReiY_;SR(?~sYD)d7m zRqbS?UJ8v={6f7(s=dRD)oG+2uep?wdIUh|Ca6W#%Ud-n!ss*iSaA;1j8rVn8x#s) z5&R?^7RVW!M(S<5k@~^Qsj}KT6pGEJpU~rc7abplj2X= zsEN7@TEiJagEoUwvudV4-9W@plv0qh>6Oj_+B^{hbQuzi)2f#j;jU6^%V}rZ9vDhg zMvtZ^Hj#l`%UU#sSWx3(P_Nn*joqO3rdG(*@@>Z}oYEnw?Bn~SDE@^K6BJMVpzwA+ zql`{j_>*#<@=5)L5^s;Eeo!iw>5YQYHMB96mW8W4e^|^SE}B<`7OJ5cpn1goHfci< zpW1Yq&M1h6(2#$%1@guFSkcBhRIl7IXgug1!c1drIY68}!rGq_Z866@sfXd=vT4=h zl}8`&3RaIB`&GRY>?VJxLgHZ}i^kaX22Cv~-#>Wz+p(2kz27}_l-9X>$SJMMXSx*H zgVx5c5Mi>{eJEfW61p~;t^6x1l+ea@=>WM#8yO9Z1qP_Y9OGWox>-^FtuuD2I zYwJ?IUw1n)D0enZVf;oIxGC*BJ@b>&QK$mSqJ`X1EA?t0Dgs^owOFr4Wg$TK6Qz}x2VjimsW^)IO{|-x-Bg5)gUw-y1K#wfIympQj+bs81iZZd!S0;dXKtdcF-L9 z=ttG5cAX;lyW&dKZg;XlL0Nv|5F0ojas8BHvhi ziISidjuBo}8;(j-rAwP2RB-t^SDjc+osL3_K{c5bjhM-{7LdmlF06mN%H2xX-uc9#Z#`XSgI$PY8XcZaQI|<#o0kHHpl|nhy$9Ezbyn z)dec&lo=F(XLWi?_N!A`8$d5rnM7Xwi;)+mqZt_ikk%Pja^NoGon;FaJ$Kj=-7EH^ z{{Vsgn^!b`MueK^)%Hnq3GJnff#dh0`SSV%lBOI)? zp!f=qRS-Po`08&(!!GNu8NSK~C9IhEou8DB;&M5pUs*k$j<7;;c)U z6z0OzZH|RC^qG&M2w(-5?cgPz@OR~4bvo-!eQa%r=%Ec`Dr$4)nylG5ecd5zs4X$~ zwCRL-8LQ*S2tldvOntqyu`!psg>M9uzAZ7#vugN_5xx+R|F*CkeWPz3#l51ycz&iH zr;+A9&fUL6v9pB$SQ5%9u_|z3D*QyimVFUG2~p2y3)AVq zZhaPUVVX|Tu>HjgY@s=$M?q;WZph{hxcWI>qS32rr7ehOd!_31-oyBIhg0%w#R=Ig zE)TjJBZ;ykdm5oh>SkF$X+#8R4?Rf(T;+gOL|CYMI@9`?g5#iFBidiYxjJt>#F9l& zKxPdtwuff%QS=9bzjuX-3-*cQSm7)Dies7LmceU8wt|8zgA%1HhF_-kw^6!#5bcj& zicG0}a{D+Wi&JWqpJZ<-3NzG3I4(&3`Ve=m9dV}PxHXjpN1b zJRpBEdR(mqEU7Lo?n@aDT%9bl{2X#jlV;?l+FI>CG^Q?Z?url>4nHZ?v8gZ9pg#Lw zqI4vl#71E}i@l55yr27_>BVD*;-fdEOOgLu0+(412$!%@~|&~Q;qR)NpfA(N*!9S`8~WUM&MirOSqhHS-_(DL@m zJzTL#C^Q0;*I+uIMl&WQ6X@XMLi^99q2Ai=(uW3cife$E&hAM8EYCZ*0$7u5;tF8t zEQ-tbDJS!4fVub~(PPm1kqVfd0{NSZ<%<2O!?uZ1WGqV& zOrl7;9++-RyB?Sd>$)DWOj;81(=F*x1CE;HJ(j=+Tn)Tts*N`_u)3^ZssRIGNm^j> zXu?$jzzx1SU@5NbI$$kX?x6$Lx;cc#9`p{R1EPHrEEglwp##=rS+DDWrPI5v1D3){ zp#xTB7}RyZQeHy>ehMJ|s(>kJ8$D>cx(1j}gsL4uP}0^_fM@Sr1$g_afS^bP5T)>{ z0K;=BZz^C&0!af9_%JlU6jt=r0JF*>rv{iZ8e9X|RZ&?1%m`>e7*FkW7w=QxfmZ>n zq6m_DtV)~16~J;47z$vGaD|TUl!vM13c!~-6u?^XlM?qX8-7x*0tk)?1rR?R3SiND zarwVA84~}iar0jBzbfUk%l{>l%;o!f6d(!>Fg@C@9#d7gt^sD3H{&Y6WsFpSEnm_A z-V6mGg3JF|8S8fWkK2&maQVN4hjtZ!=F#Q9 zdC*Dxzb_17=L!)D8Y(*gB{-QgCgnR*9sv|i?$=5%BtRTW^#z!uVm&}vJe%pjI=FL= zfKaQMIOu=m;_tQDUYt z&S|$-KZ*?rt*!_}#FBb8rlNB|wex1=GO0~!V5$|$;_A?ZPAEsG7VS5;!XUtiDgsro z0zuq}T_Q!mK(*aY?m_;A57k`>3e60H7^dLOcoY{?4_gbxYKZeVVj{XGzbd!Ef8uxtf|BTwxh|ON{4KM&cD52Hjsb45n*Yb>3 z#nnV*Y&|@_!B3f2-|J%^`G9ux)~2K1@&-(o=PU#&*B=_3-h)@%p$*8mUZN6(DQsIb zR{YpDAr=SqJsxl2@Cotloey-+eTf7@69(VIK;YdiL7_?jCSSQK0oaf|sRYB017k}k z3@>z(PQab$NhKH_;k*rU6rtVMCYLr zwlLdVCD?~v1@Oh|JEBVojk`!f?|`Dk0jT5-x=*0l#oH0)Rh9 z)(AF+R+C0>hy@q;>(kt|tKnCQcMO*Hg-Y0)iIJ-k&LdeOl@NR_oe=F%3h$i%P5eHV z9E4WbLalYJVC!>|R@iQ1hgR4v3{Pl6(>Ibht-I@a6bdIEC?y@Be^Hh*AEeghq*{exbmi2W5)u z2Qgco8&(8{j~&X{+6BeGwrM8;pQ$vdY3>`K3g(S9%Z@|=I#l;$aX;Ji5|7>l+X~b< z@gJ~8f=d?BPu}^+SJ`vcm7L`ix>we2%9WQ%KJawb5tN|>BmHe-gH($}2*13G4C1Yo z+`+{GvspHCvPEV<{DnQ!|D>sBnBy*vpIQnE>u9uGTfu4IAwE(ez|snjLr?2o4xWa> z!j)U%#}9bZSd)jGQjaC2TFJ@9V~|K^Y2u4KK0pfkwAZ7hds`Ty8#%UZ1A6GcL?KYs z(oe90#~_&8{}-}V^YSzyK&3JzU`Va&!F&;FE*GuNwj;S^KhT6h8NY9*Hq-l?E*-<* zyuq=5{g)_9{)yg06W~AK6LeqUQ|N<;8P^!! zSKy0iVYR-miBl$J)KJis(F+6CIyt&wxLeNdSQh`v7SLYnEq*e?A@CmMEu9SHxRKw` zHgg~vL0i=DKEP<-r4gm-0X<(Qfw-E0f~iZyDUVDvene9YUi-m6;_jHfqr7_(j1M|W=>}cSDt-mt!tk!mJahYw$^<=XF9_(GKt;PI(^w8y`WfACSwNPThFkQ;sl|x;!gv?wYf`PfTp`kgauLq&l zrp^)&A{IDcO(EAK^e|m+YPCS*`SKJ)ft$JGTNN-op;%=n3%kPWAnIV#>ZE(2y;riaW5Deym|Q$Zk;ljN`NllJic{ z-7s)uU(K67#AI$kJ+clDKU$;O4~CEx%%pIpeQ1BHMK%z!`PQQ|$4m`}IydA|cn3y7;H{udOiMshq5rk5aH>kRKYj;`xto}1fbKjt`hgX=J zq*+A#FSl;qC|<~l4k{p{;k8Qf%s z?lBGn74o8IG31nXb%ykFSI%*MZOrA#so;?J&|zL@$*~os4VD~JN<%C;hRiAP_tb~ zg_oSa+Bdc)gsLr&d~hp43!ah4>vd~ zko=B+VHG3q=aCjS7@WsPKq$@28iOklgqX$Pu%DF7hb|i~ACAuxSwRWGcqzo4Fod0> z7zxpH1S8S591mddC;~D!dq^$<3ug@N{1O05J>;T!Cf4@s(`FDQRNbiT z;c0lFFm-Bso|pMbP*<<1^q)NPp8Lw}>F`EQxeL}-IAkA`yM#d@*6ogi6$}2UwYN+y zmf9O-DH#Pwwl-r4JDWu{giQ~OI5YGdVuuxX119l{;=qoO!Q5J-ZriQCuVE{nYgljW z9X;v}wy40WTX8DVpBS>hsN8l{DI#^7VYMB1cevJ+a;l$K6k;g)O+W{-&s}Jn$atmY705 zY#m|izAxqz*=LL*%ph0@w;hW5Ll{y(o*8c9A$*n-a0bSJsE$+|MBnjqh<*lUn=Wv- z(PrGQ4rkV2+Xzd2>h_#pw)ESt)gfA0kV2m=}4D7w0zvnkXxFqo_KPhnn zX0H`F0&JQkcU31~>|2U^DVm?6vmj!&-6DqoR@#Fzewl6Wcmh36iBd91aYI#!o%}^O zGI!z{*4UYZ2Vma0NUQA8BC%D8fFkzgc|u`A|LyDd2<>gVv&DGfHpG@J3iq|9wX=7P zoy#620#FuTwq@T}Kx^5%vW%GA<(1&f8rCYnVA`g3q|d7M>Vyfl%8KLzVqgllQ1NjVWE5-l5nIc9~DV0-^8g^F=f{2;zXDsuc1=m!13sXCKvE+ ze^T85kmF>C9ze~_oOS`4+kP`tQ5&0k2*gB%1NP?WIZu8>m5-*%0sA_iY|MfMOmCgQ z7iY56pnNAvR1VmbD5eC=rYbxu6%HWabwxFPw8Z%jKqQru^P;D$TuAUmsJoK99h-bV zyRp%pVX+J3>FHj7_+WW;!<-5Sq8)_;m%_pU`+>*LSs%W_LohQq!W~?q?bWd2xxh0H zQczZzo=Ct1geUZzDWUPEjt~lCsmiNU<-p}@nxwbwl>;A~{~d^Zsd4}m#X_1E$iv2b zi|&t$%7Ig#7zdDpiU12Goq|uwX%`1;iA)=cH#5tv-iW91k`~18;OT`Rkuidsi z(RX+w=+WK3V=>tYBTL0(u*S3k#Nlfcd2+#ypHhipvY>>9hBvT(rG#A@PyIsiVlo>G z9VHZ#u>`xQmdqZk_w|24z= zeu>B1=YRc=|6@P(|NQrV{rCK_e>T2CtdnNp8^WNLIc&2!F+4-!ta^QQ_G`o-n@mq*9R^N<`cM0%vCfCW|{SI z+nq-0C9HVh*rXjEg|Z)f?eDf92ac&M*l8B8GB?2nWz%ZmDML`Hc1!58b`v%RP~Fgj z_p#tJJKGbpfFv{8npar!F9IMFu`n-74Q;b0eXXs{kjkY@nH!fvC)ifs=59Yi605*1 zezlJfAS#Y!f`BitID*u+9ZcpJPGeKxKouh8MWLZ={LwqxU>Er5X!4AYv&ZJm3}BC; z2TZxyJd9{!;&!}&9T2eLPhzSqG>9e{xWGKa7|j^9r6^OMP{}WYU%0d~4_`paccPRY z6d;?tD1*N^@MoBHSlZUuCIc3Ph(I`avP6j#sc< zM zzv`Qc*^k71tf(j^*$5?kDZFw7?LxsM^t3`I5nC;s=_DI{TVlnkhPVv=i$K7I|dvI~)g^Xgg5tV+=sJqksm1lhu2#Vq;e)=%=b z(>m?A6h;jtGl8~>uacsi0N2HLLs41=l+9Mu?3$?U58$28jGu%MdIlLkwOPvOl(by4 zd@i6UX#s-*TtK#c0SDX9;W)P51N0d#Ay5k+DKixNRpur!7rZtVz=smTFhjToF{cBq%WnDDOt3n1G z=&WRhzd}7G&iPPn33_r@zEblun^Xd?*E8Tq4yw{}n}|VckBI+BKuI6>$mS0~Q?7WW z@44sb8;lE9lWNxp-EzL)5RyPWcF{g2fZhl6wCXMRxLhCHC-^KbVpyFY$g zlq&}=r4wPSZwahH#HhahN(=~KtfR~kvk5FNo_eM^S=oNn7GP9I>^6+C#$Yz(z$84I z8+#7;hC46A+%tr?p3K33*T4mGa;Dg;E^}ivW_80|FG}77@}SFH@l)sQ$UtH&G?t*-6{H5iKHZA}qqr zJrFhKCbD;qM@D~>I&GuIvY8TBjyk973`<+HMN)*7mJLT0DGgqF`FbKDXzjwz_cpmxwou}9B#%{Dxqx)(% zG8$vHctZYCMbB3o6~q<_fZ=IvEW~j%06S=6gw5mX&yq@`Z++Vc%;^DFdcasPYeG8? zLmGc&YB5J-BN~!;d-EHnwmb&(&{>L`o(g>X?utahFPkGuwmE` zw3&j#YI$ck)WUredXG}>QaY^{Qou?}lCc3-OkR;%B?(9FGOGd<)-nj%1;_Cm1r&L!^zAwcbBm8b{ za-0_7kUBT;K^e>!0hY$0V+4=^tJ^7bGI|l}lbnztoG(N&;ifFd?3rvon+QBjZqN*5 zDZb+o7FCgBL%#^XWG6dLhRM_3dMtewwU8n3a8yNxbVqQR-diC8&t!86VyicPmA;~* z;La)EiDEEz1Yqdy9qUYgf9UXSLYW#a(7HSAi{MaCC*|qH7OFQAW*^EQaG(#O7scNz zxgN1PYvQmbgM;i}j{IVrEq}E=a5gAR)P|?c5d~#%JB=+`Onm;g;-G}c)SyUF$#A~A<9(kYN?%tTT{R}o4_w%ydzYd z)fMxNxmjEyRX7q%sAw|`feA^gYi=fz$)JF><`Vmox#wVbBba}7Sz!8mjWF^Nvd@qs z%ukS$ia2s55)hx~#i#})#N0tZ^*Ci=dl=V@%Ha49%C+(QsoyBoaUaJK%Oib*vC(Gh z0?Mj?<(VD9)-)O~@|igyrQK;(5Qb zkT38OMQ}k*0k9O7d?|bgsk?Ad4`s*kG4^u30b_xYEP2D)DeoI-x^(d#EjG{)!k7oF zC69R5Fe4FKbp@NSt9@;3m#?=~KR7|C)v7}(s&lov)XU^MyW(^$K!!ZDN6;u&qtZwA z74I5r;=0Jx`bEyA)3- z4b%oiK55WUjC_(op{$b=N=A9cO(+?(4J83aW>_h6U&PZzTbXpW4DGtu9$KUN!#!V=7>gSYD>ieebuy-~t{ELLrZ7X+k2}=(Q z2~Y{xO*80*sO_ai3uFnUzEOV`38j7>Mgk(CWOzOBC6rLYXUcnr>3^oSmk;Wk>!vL( zgz`&G=u1PYeF(nCDV|Td6y}qx-4yq0o!oHTZY4Cy)3TOC`_YE%WS8c?xL#W1d@`YA z{K2Rc;;>*24+m%-Z~<2Hgp#pG(z4`^^CmRTh*z*jSD1?-2AkQbCz>-%@hC+n#|*bO zDmi9Lk};=bl-YywohXr0!V*(4@F$PLc~y8#ODQCQFSWnYD?oE(A>I||XNIRJfuEX~ z#U&4ooA3rH9rdV156=4_wMOVh@L5>Tx?n9nt(g|nDA!9}<7(^K*V~{@Iw(C)V zLm4Gou2XMD$=VQlkx{Y@TvAHO7JQAHQo3zVDP0OvN)Q3#rj$BUNhzgdj9ceTDcLA* z1___F*PEI-ils!4qnr>^O2#&0MM`Nr_2!domRr5~q>q)Mis1>;8#*0y=x-GC5C6{S08>NV1XBWl*pD=(3#3oO* zKDtwI^b_f$2FR!V*9C~poz*x+6aAGdiVj`L+D8ylO~nAmv3q(^N@f$5sy`HB%CZe! zQOzU{0cu1^bJkF~{R<_{ng7l&6thXbITX5dtV%Tit0O_NCO;;k5ghwad}t8prn_Xp zJ1|>!;WVA2rVfKS#xlsZZ6x^H?hM zamEZn@NPWwlj0AFkhobtlh(n1_|Cfj%n!;NH=p@wN~Z%{FDWK@C9~E+Qy-wqjv0fR}>GSu22qMA=n0U{5b3 zOB9GQ`!X}*fF3m2?m`b@)bLT&ImTH@vE;m=2eI6Zy2iegVG6fRCJ)S{9*4HcCb09Y zg$+S9;9w?30LZ4Qkj7>_*08_mEWwA>+|!?G1O)MzFD{ZJniyuG%QrP7sUrVu30bQ4 zi1nnl1BC~LL0_A<+T>RrhqKZX=Ia3LYUvj=Z8DzrB0K~U$j}CsOPSI5BFxm->CexR zkGY|LZ6UV?MD1y)QCl(K2KI25ZxWzHZJcQ315V)afk`)v5tYgImN7g9U^bZxWWhcL z;UaPnL~NBm9!B~i78QM^Mp%_1v0PZ0V_)euzE4w+%+2cLF1jI<2!>5H-jaua^hz0c z!=7@h>J%=8OCd+Esu8OK)JDB-P^>gc45qLBc$1AT9)40JLcTM{;H37y|I#l%O5048 zvO?Y$GSN!4ix|2LnI6DfUYrONl5(P&0Gw;#@V9`}RH@nR(KJ0;G(r?J=ru#kRIWQ= z>bxAPi%+~=bZfU$%(-}kiAGDD=fG{*yBvqp-`HD$ep~jgYBUO!^|wURWk|^#Btwo% z1qE?V@SY~qb0j7P9t~pDKc5REj53wHQ6~QrzSX7JqN%39WIrTiN2Ld(4UnC>zyX)!x>m$kmZZ@g_CFSp}0Fh)|?BcYlC;bq}5{F zROG~fRfAwNdEDiZmOd)3eYP;&C$GlEOpZYSs^VGYLja;j;kPP8q^T%Gqph%&#C}+P z1+cNrwj&%B#aCoEtsPbBc*oM=g~zMx=5oW_gC{d3Vrae^yw{k&QLa;Ch*0f=BGXyr z6_W-B%PSfYpdd=EOjY08X^mDCJpuBGZ)w77n)Irx2fZgFwAcW|d{SJlK78V+4$*4s zxd>jyy(n8_4W+!^78!3CsyAy&l3z;Ks-zmV!#35Q3`V5uv~jB@qR7387D7^|L1@IPXH)277&WrZHhfFxxc1`=A$eb8^ zY~KQGmbargHYzVSsEzF5<}A47iLA!#{#l+pU0WbiDKBF4KnKIETd>E>J%$1VjLEFg zto<1a?u0xVx;vMX_7tJ~6H z9Q%eRRX^HS?l%_)%bk+xHjyKp#<8h?)D{5M?9qc$FGZ{5iJ@<~IMPhc9if$H5^`-z zfIW@)T>b4d&P(Y0iOh*hAkUoKwlXIlA}R}{$edi3EWxBr$JUTJfxIbyB6DIa^r&ox zfj?uJN12i3@Pn%o{ z(vMWuUCgUanr#igF73!ltMR)CU6(td9ZER*e;B6h-B1l-8SYU-jm_O%M4wObVPm zS#;ok@Av2k@W`}Mdn>wUF`_Ds2GqaJxVsIdzzXD#@a0ZU~&O%RZEd0_mC1qabQ;*jIOfa=BlbW zv~pZ^a7=3isC&eL&;@(Ri+X4>!e(H*XlYXkWY@{!%)u7o&mF4jG?`xl&YLocjs!=m z6`N!d`OP=U#1s0trQa9-#+l|1jwa|(b^p<8P!aWLLHWKEuR-06Dxd&W3b)+?R2ZR3 zqXNO^(P?58pl%}hw_AW}3RpQ#!hKneLc7(xXeihli?k<-Dq2*4sxvz=nL0p|W?S{! zsmG4>Ff|BE{zxVfXorX{%_~rMl-e@9V(onkPQN&p<#O3~qC^Sm<_)P)f-1);N>Dd} z=2Ho(jZ}*UT%0mxP(noDOY>Q6Bu=gYDF?tSP!al65Le(ykK*-2SAE>y@V;pT!MtaEOiDb63JJIt6!V0skx? zgEh3n;y#6-20foKi8)1NwBR}3KA3ot*DuB2d3_aGdI%8Eliq>n=Y`trp>n;wgw9w0 z3PPa>NsffO^IB!dBvF(!F`d7N5s2zmMjQZ7x4poRjMz@d9RRL%Rs_Yw^rjGPNSQve zln3}+ce4cGb6dYq;)z2C&txB=BVESK)a)*loG^rbDSN^|zld$;rr8SM$t~Rp2rMge zP*EZP-1?BH-O@6KsHZx)KKP9+L}s1`0DN9YCxE@8LS&vCvBe%b>J0$Vc2pO6T%t9{ zK%lPl)mAAhR-1pXics2W`W}V`;&Q|Lo2j`R8q5j$h*l$wGxP#aeNSN*HXZt+8wJ3~ zN+Cn$@UyN>k%1kh=AE!!L^nhmP-0dy?*$UmcY)W#?9QodsesI8zP?Mst`p4BWBCzO z2#hWegF!0?P=;Ahjw%tvQ^k9zJ`vzXU;kagvxT;_6{*dX8UPhdc!`fv_Cuf{o?{i;&wT5@{{1uqCwkfKsD zmh-EWNzG4kSKs}`V-iJ!+7hOf%jZ!IhniFvk*CnGppQe4R8Bu;*6ZoqwOHY{3?IcAZ6V*Jn>p%hUDQ z^V(}IH{%603Mc4T>~U07`jg@g%6;kwB^;eSwe^2e?o*o5-zf3+ctpR% zuj_+$c72SnlIvsblUyI0_ntkUtrO$!`E1@D5}uE>PfwnYVQYG2e9_X5Y-+En>|dzK z)&8#Bp@$xY<+I!ych|?*uU^l`nmc(uo7ZzE=O;KgoSzS#7UJqw)i45b|1L$DN^3qk zX8_~%OhsQk)oGBc;QJ;_)~X0MbEO6cd5dDK|8DPztwezj3QA{;Sm`mtoIELgW|)&W zEEfqj1L))#0H)wJL*9|1GYrb!J@H_^%bYWm;P1LJhq?U2Jys>uSoijN6W4}-qPnJ&Es%(Gph!eOydOS@0 z`!6Uuo!&8Y%ECu6m{5#SdYj*YC!Df)%EEu47hbjP@8~TKUTGA4>i%(!vTvOQ(sy*f zhA_U=d-zQFFZf*Dcld1jC}RFK%J=p7;)UVxVeSeAW$I>iRvWE$c*2aAj^iDE-m6lK z38vBxWARIyuCDhlYknSX!t=cyD%}hJfmvAbabGc1xZ;BtC=b1ZjP_k1Q7Y{}{DMh> zszZB|XF=CJq(2pP8L?qEEWuNQ$@ z*0^TZ!Nho_D=tdM*Hs<%@^VNMl`N4jn+#AJui!4qY1GD={L0b^+aB~`bVARd9l1-I8Q5A!iJ{l}+Ng8Vai{f@D z0QXqBr_XM?=NuHjJ(+ce_^gU$eI#ENuNd}r^Xud;Woa3USg^n=9C$%*e!|1>H?%Ks z5ThJza;hRb|3We0%E~_SBz0!f;ydf$E_j`L>WHUm=8Fp!tc~+C12DIZVD#4yj>qB@ zrmh+y4gW-06fQ&KfWfHChm42x-LY641urWahBI-M6`8X~8&8Ozy$7$VrO?riff3tv zUx7^~qUrt7T)RBn{Lt8>QC|Q4S={cuQ`VEua(&55%CNk|D*d^W5J>x%DP8xhyS-I; zC;_t0Z!5A}xQlq%927f$@FYuWOaZQ5&b5obGjC5B*u3roM@x!!i#D8{o)>dl2i{M` z?S%t1i!MQZROWgqu|>zC5=#`Nt!n3~r&+WiaEKJclD<~5*9QJbY2#GHoOBN{F;iI< z38D3{vi<*M?c0%a>2>Y$GYZ8HRD3_iafhI)YX{ni=O4JS)&j_ul-T7uRcHTz*k7|r z5X2i?H0F5X!OCvj{9`O(5i_*UnF{u%EO5cYSyo_J&Cg(-?L`*tQeRo>dt{g_=>bVg zQ?^ak>GW>jVESeF2VfVfa%KjT$_yU-TuM)>3S)}_KUB@+U_*Y#s?^${Nd4;Z&e-S0 zqiF^m-{q=P)%}Id^VH1$zj(T_Fuu!ma#;gLg_WBQnm(e>5Popd`wGECjnunL0u9a` z%!0Rn6IICM+hmC}vT?RoM7>Kg`x^Z2>cT*EGFrOa@rz1}bS`MY7%!pTTCFhqhfAU|`qCy{MUE7?thHeyrgJCBw7b?UX#!y0+F5)iJ4kWW zHsP_k(=ZR!=B#jbN_Czg248=z+Jr^EP_v|D&oP@UV|4L$Xx=}$KSLKYOEX?VqM4Rn zcc>#IB5qyWE+X#CpoAkLs(wI-h}#t>iHL4=IU?d#hOdi=n;CZ@B5n#iM?&17{+sDg>j+r?gjpXUB@PzK4Jpx5s;CxAOdRy!g_t-} z=1)40a|%6NP&6F_TKquLAt_SkxnEn!CoU!0Is7<6qMyI&>dxco7v#YFm^pzXXmz9DY#+;BUZ{_kouqm@x@ZF3mE*?15*;PUfxT^ zVvAta8t&+CRFMpzg1lQW-qE4GJN(6}x8scXzA!zhiIP>%wG;3Mh6xkjEPdO5U|`R7 z42lPv{EDv5ljVkuUksMq_)^W|O-$kE@+A{BtDjrO?8BCF;I){$-ics(tmwoqHa>t3 zb`@>OS)X^%rKecBbp#bqd`in9gZ zbH+)qJCRX@xe>!vT7sL3G>eK>lPG3Ig>xnb@8~p=AT_07&jYwOoS1vV$M)^*=+5)P zusEAAeCZ%O8mDCvq!F{eyEAy!Ff5+Tum*$o;-`bSr}%)ajY&J`bY8iXpoUr5&S9X% zCQg{v$`*g~45Bu4Y|hXbd~F}AsZZJz23j27`34V@qN-Ob0+p&|EwL0^t*r7f=uU4{ zHT~>B*pk;mZU{7%-~#7Rp$H@cKw~%wBS07As_(u#LC%fkCtFum1zLtqFmWc+NGot$mw!mfE2pqq; zy(%z!s>Ok|tpv}Qiyz^e&2sf$*Y;0@zkL%!$~zd!i}qoS^ng<~*&(+UULXduU!Kjg z41PJvI8%uQY(lktDF}q{7fVQ&dqifX0Q@GFs*}~z4sP3z-JO}WSgp#a?ZhI*Iv88! z2)cCfc;@bpu3s;t4vI-efV+}>Fda+Z0(p_ZHGjhR;>w+h5$qT<=-%Oqh#aeq0PRDD z8Qe=BU;>OSa>60((hv^GonD81{XnTZ`NQzf+oM07B}olXKCwN`Tc2J@JMl7gqbMKY~RK9 zw|`+qQMtZ@K~h6i>73Cw_qrij9^|-GAG?B4bn5a3E^W0rKI}uCUTNE^GU|a`HmjId zec5WNa=r<-sDqB>>y1aKNrZC>ZY&Xe(RgPYr5M4609ZVh;4XTjk8fhZObV|`djWx> zXt&eI)OI-HETxm$Z;X?g5hUXi!Jh`gb*jpW2xHGF9^Eil^To8EK^quga!wC5v@*<1 zBh39$7YmNM%v+UqDHu;QT>w6!1Y^XlX4PIg=b(#IjY^Eyn(?a6C2owCGY>q)I6b&5 zFz2XEYR!LO&Z@=l{KlZ(5HMRp4PBv?iTXT>{OMV)n++3QXpQQ@mEB3Bp zMHAUUkVj)PoohsvmMQOc&Jv=XJZ#rW4)SQKF5={IHr~nO2Pu#|T9Miadm74+Yx9Zu zzA;5L!X*KWtRl!P_irp+g|MrxH-_Lg_LMSSR{`p(>f=*&m?=VcHoUPab1stZs>V$-D&-`b z6`O2c5`RnW`|&Ozo=(YHSc z-?pYo>SS){KVd3aOxHY`ZY)G`)FG|2diYCinn&j$tW{i&qCE&T2dpjI;~JqpVPHNw zEUoYz%yw;zkijgt;LaG?$^9JHr6#05zDz~Py962+WSi^D!7KwarmCVH%u*3)iA7_p zRCy9(^k8iS`Ap4LrIeR+e;li^4^vWH(++8*sth%+R;VKuL^QNuMOCPWE*rQ=P>p-t zOFCE~>soXR8@mS_WfdikRit{*Yxt!K;lf1p+u*Fhot1bCy^$2@n6sKa|6E_koU12( z=Qn1kGrhfgOEy8Fo;IaMqy~&Txd1f&Kr@LtHxY3JhQ7?^P(PeD|jW1se zn-T+Ds?&?B#2F$A9o!t#D#zX&Q!W6^!8Up=DPqNEzW4?-_C;yw?fkRd@To;C%!H2% zL0GaJ{(*^YkEecN7QXqLHEm&c`;CdGVom?R#M|SkUzh<6_=Ta3&tTpp6h4?Yg&V)d z!X{U+y6vk}v)@|F`<(Me- z&{}y8k_VzNUDS;JipvQ-6Tal{8#9krI#=@Sv0p>x()I?%bTmE{uYxu0``61^gsG)t zL1UG-z<7PB^y#eI9D|inY{xZXnA&+Uqn=n@;uLg^j5_)EaE**Ax|eHYSa#M&ct%Do zGFM0WsLFD{?im?nks-=6f_jJt{$f$Rmt({c!Kp`f()xu7FNpo__<@mLEx(A(EXnJ! zG51XF6n>FmJ!L-JEn=VQ2ON*n1A5#dwpx58-6FP{OD=@L<>#N>En@qFU!YgyY#-es zKQ>saPqp6tA_udyA6?EG`bBKgUGGpi$=b9G__ORk@{3^q%E#pwu~nfdxN4PHZovH_ zR-V-TBvc0mBh14uf`X;xBm5%R@dIBw9-fgP0e6FpqFeI949aeDjNq*<`!vpLNLus{ zPvHoGC%qP3&2)?yymc62`a=QO9Y4$axhhy)`lSquP9JqeSbT-nTho?m$0`oYR52c zCq==VoFz4|F5)ythyKnO_nG8->Z*v6wyq(T?NPrr2oS6e!LDGG7_4?f>a>kXAec#Z ztq>Vz($%FAW4kSy5hY{0m8z)p=svn8;uLlZnIZ2R_O#l-jI1X*l>m0td0;t|hi7}+ zODkCzXwxsUf#SJS@z};g89hA{i=0U%;Acu{K-nh6N*lbIuWXiI5H%`trYtsSu!!v* zUK2eq(@TIJOlH+36~ctrya>DLQl&WDTQOO@+~DjtOe4*>S!wF6IrPswNFy$Pq1;PO z1jungAo{bZ^;2yWTsJU^NiTQJRKCC&(PXHi`e94!=A#l6LrX4$nv7zGU~n`A>c}vo z?VK|(-#2Cj#;v#vkg-ip68;LlB-2(f5wM4ORBnHg5w5#lSgzIrG%=Vf5mGpKKdtu(LdspvYSqCX4I_CwfR~ z-(XU-%nw%eVoZ~T4ub%n-^Ols)j>BNk1a}XPUim^D=O$KI*T;{oVJRkyGu6dH ztq_r@($VmeR^sZx!nGXEGEK8dp_f`{aZmly`lfpST}-~x+Z+SjkVdu8!lII{E~PHL@Ux3PRad&mAhrz|tFzfbj0bNl-= zXVk}Kem6uLf%(2MZhoJW5zcOYx0hteZmcaxTUr?4=Qxh`y;w>xT?Qq5H^B}{>25&F(6*g9_~K zB3@8r6+#g!T@+j82>FhRi+PE&504kbmP=N}z3Cf1bbyw>awwnKNWhQsyePo<(FNMU zf6>F#H!Ke@pk)K4E2P!+;V)RXQng%l8KMr?Bt65&=x4|el10aErmFnvFtIy1xrH67 z2g`#Vez!f$IXL20q+E<}KC{U-_6lnnMIKbn6VF8XyCOz}h)*1E&S`q-lE8TUA?gLq zHzrHNs;v5w6i+%CZinh!ioFm++(S!4>j29A-284=MI!`9+&M&1j4-!|eqh{-=?=1~ z2Ab|5L!}kju5b_n&aN;8CQ;y6$FIz?ob%INMj{CoB8G{gb53IiOU0bs$T4&fibB{> zcYrlp0wE}mjEEq1(G&=MhkT-p!%6L%OwvuG*Iy1~JwR-ImdEZ84HH;ejltk_Dz31sDba)Rp$Mt|nTw zk~s;+6C~d=qUZ^Wjz#cr!wj185-kAP?8=tkH6}xp3nqns54__Q!dM<#V(eL=zWfW3 zv)YrzGk;*V#}ZdXTmONHr@)+lfakS7{uidxqx!g>4E6LN4wiqdgKgpf8Ud?LonSnm7)~FdD*Jc^0+b~+ zBrjQM5y>PIW|4GVISC!R|n2R#?Bfk&{MO%Pi4dcN&(U>RL zVq!FleFqfV9E3O|{;HRytp#p74Y0>5qX%s@mp&ru zRT{0cz8PYbyZ)|FWyZ7GEbszY@(ii~S+?-LYmSW0|A60Qp z-K0EQr%5%)@~9EB|L7j%iCd9p_8p)_y1pwF1Hip8Y@BCGB_S&P&2=P^2}>FS2H|P5 z4Q3grgF$()?;FFyqIJu%kXVb>$N|EAYU+Je$&o>NQ{n&HSph*Y0+!UqtO^eyvLn`@ zY@*&NAdgO%g-JyrgVo(ZAxjU;!F8B3U_6lH6&%58XZXpBF$^-CUPiMCT&tl>0Tynt zwK4CMb0RSJdBeqrUySO-mXN!4vo7=o0H30rp;Ju;S@8glNSVNGzK0*##ilxj8Db<| zN3@YhYI~p9Mb5UB7RHhs2I6?lp&7I|u!HyiuKe!Fi?*|F>;XM|H86`#)!+f1p+aDK zBBdE0Bp#T1+Q?-bi3DFn@gXD>nZeEo+N>av43bNeK(Zr|WG%k6i~kpLLR{?conZ;d zvh^c=(KHrmymmA&drK;G%t(s)D>^JJP6@b50$^yQTGNiC*mInO#H2ohUBK`te#Mcd z!6XvT0Zk0;vOwk`KBJ0(MNxoVJEaWgj?8cltqn z3_l-PcP$K~|1OO2Lq;{@?CM-t)bo%^ru>)Alq(1gHr4K;I)#d6jjQ%8cC?0eVAlw? z*nuWFm?aS6yYdQ>3$@G|VQ4ynBTv$Z%}I$1j_{x{gx>ErX6r*16QP<*OKh>!8mrEt z!MgHFP&S$@0huV1x|c6a+fs;~saRln>G$FliHt)bxsLul?TumXQetg z*xyexRhHf84B zLk#h`;HoR(U_8@kovU{drl>Pi=m%7`E-t556i|T*CrK{oc4niYB{3!s(omM5vq>y=pF;z9T%aM>M)Z2HYaa{;)f7aKHVQo8J1G=PT z-59=@=f-F{O1t??`Smg<6KW0fmh_Qm%XmJoG%Yz)c-jH{%x9u<_K$Z$y#_;w?QUQ(ZMb@VL` zp(6a2RznWZm|GEd0G#qa?*vLNQ@3%3Zyfi8Ld3F{Ob<-Ka(!P<{l=W9gt=rQ{myl5 z$GhT^j^Fu-dEIWCwX2Rzvu%l@vS~KfQV*jhau=jCi-+&}>9^TtI>fpxmTeycr;T3Z z8S&>e^m{O9TJlJhLaj?#I+`0a4fg97iGU98^9mQ&+-zu-vU~j#bC$=S`UB$z%(kYi zDZerCl$P`l%z3*%^$T+hn+HR~rX~0!8E*3wygn|&W|M;EhRwEg1^C^#SexP)Hc@4H zeuQDOE!P5u3MMD*hFdm~Kcf*z*)sdm8;lrg{|{-A2*Y;DPlem;Amt{Df-5pfpdnctY}p?yTL`*+Xbj2AgU zoL~8gxzDDRfaQjxpY^{Rqr!u?=M_*dG$~F3=8*ak zlk_XzKjR1i{-I}tQW5-COOq(^ENh2{HWmV&P@3(R?9OQc3B znG$_Fi*tN2Nz9!SoW*=Gg+pe}6!4|55uFy3MWY$kss-!96H7Lgq{7_3{*^^)-}}L$ z`c~=meKswg|8+Kn2wQ>>Zc0D_*K$k=22&`GYNj(u22wFz7YtdwHL-+6*b=O=sJk<& ziu1a=W2xwJE4MAdYL$+9q--nKJ1(Q8l{T&YMJ1!f1vph;bsoXc>fV-h3x-xIcg+wC zt&*-ABu!TbN98-c5mYWg|?52btq{pnpGU6c7 zln@x25{8m3{ZtH9ML4DeyNRm1e^DQE$!$n>ZOMN1}+%9EQ*vw+^W+^aTt-4eQ zVlX0Tk}f%-g;xf}GV7=2#1}Xm>1V0R$$c4$f_)hAH zu`;=d<2Xkd>RfO}u-2bj5=Ke-yIT^lmNHKctUk-rxwsT?m=X?#ri8)rt6@s8ci_w2 zlwgSBtaNiyTUOl(cuF&oDZ%85%aSnImM<&`_Sr8(f~`x^16%b{`2rdeK+@&oG9+NX zbsZ-a&86jlmIS#~FH6Eu#+6$VwBc?@2+-{afzUUkD%&8<2q=czZ!-c2Y#as$_fI)y z1n{YTn-Rd++A$*xw=Sy2Z+9j${Rp^Zk36_s8R)FzZ>v1|%^+-FTDeE;&&z}`JoKRk zSu8hVfN)g!OZEdh*3>@SesC~uEf`*m@4{5+2%ZgG#Di!+Fxu+Q#dQQ0S49>CBc^@C z2YhE<_eEhVQC}l6qp@^T#9GEpfulO&rtkvGZbCS)n-C6$CItI|RE`P3;B{LN@Xme& z6wGNI6N1gTxnWC(Kmv(u2=<-uRLF*4>oEM1FTTH1HUs40X7r2>8-ndOtrBiSu%o^| z!h~R(GvzW$om))KAQOUdcAk8i5DxFB3E{_Gz=YS%PC7?ivB2f=ve+Mtd)ZNJQ;ZMi zk<~b>yV((0mkI0iv@P=i-Rz?+d3NY#A8pb?>TdSYhB!Lh>;@*OR~2xR(boO!!|X!R zEAX|gy!4L2QuOyVRhNv-8d1+B)F)GlJy<}ex)vD@x^k51gK$TCz~N{=m=amd&Dm*H z#<#OZr8&1x?J^Ejb*~%!?5F|Y#wwCxRT`ok?Srkksyo_kVaVF#Xt%Y@qJOa8rotg| zwEKP^NBhCJqy1p$Xh+V7I54iMu0ku}Xh+hq)C7EaPx6Ps(QX9N+;Yg5wiS+X6vTc= zWJ|&n$%ub<|0)+i&D}X#e+i^UF{!*bQG0pbv2fMzgvkm4%f;fLm?D`KB>}2JVC82= zy8rR2Yd7Z|*>W)9XCGBr)$+637AKFmQQZPv6 z@VleXKs%m$)sOEfMqu^3bU69hEiRG*z5Cgfs#M+2j%+2@cN^jPaV;fI!I5uZJ-C&_>pBTGxxL`9WA%J-JpBg10zrSU zf8l5^kC=V&ymf#!q$$(Q4(2Q|HU~FXaAFt#Z{mZsfBf8+uPB@V2?(C4f4~H2JGE5W zB8$_h*@VHkAg2_@yu88(~keZ z#TJyHH&~99Q3^y^6M{i0BCS$fEPWFj(?}E9xgFe2`a=dR;qMD(lDZ1f;E;lIETlie z=1=O_Ns12J1sP1UG8;mU?WZ6lx?L1lGAWquDvmbzVY^8gA4>JHVirj%B-rp~L4Y`_ zlqRpkY+3B{8evK6Q@>f2Yd_>tSE6B(thy%iJk!!UAs+!=N4fq|<%5?i|6rOrG+Xw2 zmw?xq^6&O>mJ(q&j41kx+4ao>4wR068xi)>4}|=-o8IXoJh;aehWQN?T17?u(}NOqn&{p;~*6F*ykPxm=>?)P~j!qZsXSgN>efw`+SZw=l|7yzXTyG0?A@8-)<` z11v%@SYe+U4W4kRXfRtW?eM9W{BcqF*6DfEJ$6PxcN7@^39~5QHBF<)q{x@lkZY8w zm-1mhs@kzGGqTg^C2fhe&xoScWgAR8Egl2RiX4Oe%P|WmJQ13S+OB?H^%-BkK8&Zn z?$4uV|GN;0Nr{zeHkTDy@$L{;+Ppij3$yBJIHl>slJ1Ml?K*v4R45gVPzrP^%oML~ z8qc1&RkOBO1siH&IN@d`CF{LJ4PtqMb6|>kpIZwaa-yFY`_-vZ2#P^4YR2z+SUpP1 zHEY}BA?wv&vm}in%dl*z_96-|BA>d%j_D=4674+bkE0z%T?qG)lJ+Z0nI~2OHgHKU zMunDuv(sE)Ds%j+ir&%2fY&poVtAT-m1&%YI7$PQz?;PSmk5z5mgN(nCV8!me<*9cqpwgv*~P{BbE_+pR+HP z-Wad={h-niVh%=$-`7*WF~_Q%7hE07;@&H)+QMwBuT*VW%bL*9e`E44gE?yd^*lFF zL+JM}EvuqT2W+v{O)=z|!2kGvd+vWdFU%kIzyJL|pYfA{YK0Zuo?qCCw-(0o!7Q5f zqwHPPemHFy;0^|J`Tz1uHoNp;|FhW@W3;)tpHU#}=FMNbG* zqpgri&fXZ7lgMgaiML=q5lxGEY^@$8O?2zw^5M&PhZty{A^Xz21IBo?J2@j3vK`jm*HVP|-0RyUIar_5Hdfo$P0$m-c$P{y< z$;ujQaY7wAs72aSZc~bjZDq%dHgeo!v%C3kFR;n3d@xg;m{?dY589RSN_&OjeNHA; z<$*3SHf55=K+kG?Gj@)!Z}qxHES8qFUqIWd#rQjen{ONzf|*_qOuMwfdGj>)%{kTW zrx}=zEAIO`-xr1{y7r}}yyGnx!A3eoRT8pgq%A+cD|0POue1-T1o!5b&I9ZC?tG6B z8&Za`wJpQ3>D$+4P+YS~uqzoW17_6qxxnUJC@?i1Fd5Z0c-|bMC*7mycPEm-(Hh7p z2h0s)x^6aOCdtk!6BJ_5|4!{#4aWMHZtdavt_Lv1aXT^cekB9lYzzT(NN=c5x|a93riasmt+TMv_#5*`Mxp8auEjW zR2H*Wau>H=OmEFBc4M4XX>jteWHuFx)=b8W(c>ypCjoC(YbN%5z#XABTSYeldx z!_C_r28PL~1`k{pD7=Fo9NZvk3f5qd*^wdM14cHf^EG@0g>2sx1s( z;UgFpgWtqTci!whiOU(ibd5lYRLWX~&F*u4^xQ~kn3MzaePMdLFjyEB`>_qK8iAyz z#PCOh)pspXA;>6Y2s$Sp>r7xzTYN9_n==CyVnk;4nZZ1bO~24Q87vQtXj*>upfg^& z)yWUnZLhwXk8XjN7N&z0ns0EM7Q$N_%mO6FdU_{`pn04)5mP=cxO?BFH?O*>uoh1^ z54uc2sf@faOcSTU0ki2IvQ1&e!Klg_>PM_zVW?Q+& z+N{^UD!B5(6lOT-DV1$l^@V9JiD&Ny)>-S~8uNZTbJG`n4Vm4-Rkk4Kz03vlUqij#6oQ?8x*{s$u zS+yoMZ%$p0vArx)dq%8GiYjDYOWKf85XLJ#)27I-U4w750wPnn^^35YmaWMPwtBJs zz|dsz&0pQR)|pjI<78(2Pxc?1vGw@XbQHV7MAMwFOM6cJLR{|odq0u>=YZflFUWY&6~VA>@j*ji2j@DxizGSe?)+bJ?C?2Ysfi0 zlikIL9n}$NF*hcU!YI-L{DO5FJy)>Pb&SYdRaSY#sexF~mZt=d#l34&3k?GnMU(w; z4r)8i_wMgfE3izFs2ACh4~E2U@Q{4M{K>3~d_`ValGKIU0I6%;C!%DxDgxmqSr4Tx zInDAA6tQf3Qq(}HNrmoeW|Tt|+)OvkHSRVvmnsMTQqpj{Z@<6ku{dUQ39C$*sqCqo)-x$m>zjB%^*Lv|Gs|DZ0G>Y-KnJBphJnnu~j# zuyd^GyOTT5J{oMr-cXqMv3ty!v-;Tf+ROsGSLf@|vN||e7FIcD;O_EJmJhO=k#U4e zi}cSjt;9h&7*~W1Q!Yign;?BD!acJR)KP>DQ!YigTSLlA5$=ovx)kBA2wYc$duD4D zSA@HITpvZaCuUE$BHR`J>i1z!Op!Y`W=}G%#fT))kBV?p0sf;1Bg^k26ydfdcp=-7 zfUBDVssGw@?b3fO(@KU$^#R;VY%wVt(60dMzdiY_2PYrC5-<=0-iUdpfK!i^PN-oB$<%CCKCeks4UH2G3~ZSnS{ z{9@@}%CGIszLejdb^%%;pv7367-{3w{eh=)* z?+^ZTlVB*nhbU8!X_I*}DYdRhOifygBrCitzqWquQhtL@r2HD4jFuTf6A zlwaGz@=|_nldE`wn9aVF-vhhydoWObg9S+WwKuev^4nEh)|FqI@--z>!7W6kFUL4Q-!c-`ZFASv{z>5^v8{s7%>ChMV zuxzZxbVCR)wP?){^FDeOQT0%;Dr1}x z?Boo1leD_tL3P21uj+}}nWO5bWZ?K7*i}cbV(Fs^5_&9i94u)z{Nj?yww`e$EV&{0XD6I zn^HRaO@t|>D>J}NDF=2_N?>S8>D>i9no>G-JlRp~{m@)wN5RgP>vlI^>+AX-5{7-i zOi~MJNx{BQP4q?25@V40Dr%M?B|x{Mys2I3zfcLnjsnd12s;W=e0A9~xpvD^;C2+- zrR-d6Fafn3!^t7eBhEQN`-r~d7jXEkrj+CjrK|TL4Nf_|y@$02U+QNx3@Japj2WrP zPpU2ZzZ;Pm_nlD0IgW`%q;=^(L-}1vI+QZGRj(l^ZdcvJL6X}omm0As>=s6}pcaKQ zC9ckreZCBhIPoaB0~Z?w;tnKhbj&$?tAq^V7#39XNDl1xhoYcdFThjHj#|G2&c&Q1*YMIe=GktzcC7W zYG{>*C|1Q?G)<#Rd=y9AQwu|@0MaCHez%*<=fM$odHH{lCn-5_-7A^=0hR;eBqrCs z%l6#%4Ntct1ZHTfWHZY={8O|fV%zSO$JXx@;Y^J?*90aV09g_nyT{rWHeEeBV|;h{ zY`(Qq=47a~TQNMXaG#rpV>LhZ5CodS;gh1sT4EAYlI5yKcjDA_>a7y+#pv0}>3tJ0 zNvOtAlu9;c*eCIB*}`CBO|lOC?dv>oZ44ZAhLljZN~z6*QM=P zAa8kTJBUBNv>hbGyDMnl@Oej#&;)7}mNwV+-xya+&QrfIuHINM?rwh}&NEul--vi~ zJo5_^`p&^9dozo6t5*qdfjYW$<|Y59vX8T@bbhUY6Ww2Yrn%g~t{zUkZ=GU6V|D2bVV#o;OWX z1{zChRM7d5a_4kDWb|}GAIIa()fz1r zch=DlLbp4{1@(Qy=N-ZvQZfV9_!|=x8BhJf1YKKsnEV&wJmZV{2j;wcML5n6%pvu_ zc%t&s7K_v^q%prpy|b!(ka}OC>T})XED2JFFXvFY`C;;*^pS!pMd{8`oW>}Ar09neauj2LFvQKb5Oc3?;-U!U6a<;&)2RA7C{S4 z?Oae+JfJd;{>&pxZR*Va+-$>L9F3V?FJfOzB)*7!m6uB3Ifn5IbLe=)JEr6TpT#U; zV~p(365sLlp9`bT#r_M8-5BB>-SaeJ7WhG*q9m*Mh>*SFGkt_F16NI}bbj5}(cd2{ z|L&}OeSuXWXz#soO#Ng+;s-qzjTu{A9_^-mk)x?6R=rsMeUzZiCq<*cX2BWP4`^ITD1#W;mVxi_ZG0L?^MKJcowJXOC4K&U>XWayf76;alrL#zjfpE z08BOX8tqf!DKf4`i}1SaO-d!ss!`^siXGl)P&^{S65Xr^OO(R_q!;9n!I301(&zCh zp&Mv^MR!=E)hpfu%_Xbulc`>kTR+Hrt&SuzNZ^^hJ&Q;bt%$n}#Ld-oU=$5vMzqQ1 zreUs`T|z}0s%8faKDz5S<<~=-yDqPvI-d2H`**exwz40Xvu%M0h{8p!+vY=34~q%RxK zZs>_66I-dwuSs+$JTN+7K5&-#tW;+BrOsxWMZToe;R`Mm{pbNQql1FzgRoGER+Q{% zAYNG57D})^P}mgij9}h*>PmDg=xqtO>j_3-8pdicl)|cTHV33S6_(JlDeEH~(^Q(J zWELm=jaWE+-`6vLU@EiO@nW0WLh3vwhTl*YS3zHG`vI1bEDWpx#w={M7-l6&i+QbB ztQs&@lOESIf{K&<0upR0s-!1oT496zClb?!oIs|D_iy^~-JQZ9m-Jw4f>nDlStHOk zzis|HMWn)#QHcU2cAo<(-dEWYFnMXM!qB7mdl7N1LgJzk9I&zjRed#4SmTmUeP6xy zdTvj0a?B`-d$MQDZfKaYxlOf_$XDlEYau-J8RVf!SQWDL^{HtipR=Qd9Ci#z zXX?XlG)8qOFx~rv7AC#@g*m6wpZWt+?1Bdd0Z$-5v*g=c%ixLitV+;3dlu_!bz&=5 zqaQAT#RKnJUbw?B*SxUc+%$;X$m-h!#mokKV*rzx=Qjy*!&W@+X}&Kq;Jq-Z^{0(f z@{g&0COHc8ePf0?eV?a(W6o2;aOLk`K6~Mqv-1AbZ;U_n-%Q9ji}#OdBVb{!=z4(ogZ}k^d<2YshYMDd@94g4U>_q zv~?fyhPCvRI_^8TmUH4bLUi8g@v>y5DIq{4vPP!(YXm}|Wsa-)N3!)K-B*#Te3W;kPOSkm~{=I`U_zpnTKwJ%Z(U@e6~?-^43c@eSL;aQV-zfz!eGM8I6lmc^o(aA3`8ap;+& z0(kijtG#X2rJ9(v=e%e#m;1oe`^LVx)r&WwO83pbW(jI#`Vr)cPD+B9^1xgScD*y0 z0$61o4;)NndO4*`Ru`^nJ9Z!3sZ*6ndum2)M;)iEP#aBS$H8q9o>&RVh-qWQWX5*D z-VHbq$~tvkFskx7r8Cf2E#5#3rIuROM+j@i7}ET>#5w!wx$hCIm5RqnGcIG65qzD9 zt$P4xcvQaby*JD)94r+KHKuwd-xO=dZ88Rhv989a*6&t(uJ)?~KJ&y7GbDD6cJc9n zE)N%@@2oo#0~Be4IoQIfdzdl$MsC$HV`_vpr(UTjQ53r1^A2H*TQQ9>>B|0vi0e3> z`GsLeOkh}k><`R&%GdM{OuXD5`iU5bgMMM=r&=3Bh#nNt0Au&N%6}MUYkQ4aI&=FW z?`OhY0*UsmvsMEL6iI%Ga{Exa;$X^>t@?~U(Ni?rl;k%L&4Lnge$nt?NwPBn@f*0Y z$49hAHdslxV}=fcgxI0?p?&dScO8S(K-E{4nD1W{tzOETx7*Ar=!0IIIWXThhP6qM ziMPlPz@=PxWV+lJ(?h9{b#jbFt02yovi*3XYxFS9Vi_#+U!m2QE>o4)DGg9*c%2vE z$Yl9zyjNQZiPvgxSfbB}%@k;G40LYQ9X%qcQC<{Kh0SPBy_1D+>Rz8^IZ@$(6~-H` zwwjsYCT}-zs?Pr{>vN~+s_-E*)v=N!GXgTwwMBh4Ag#I@@q=| z2{B+xNcIuCN{wZNNM*wSs*2N>KTB>yyXK&526-L0Iq!VaKK6&^5=yB5h$x#E`_eT0 z;^#|>M2YPA!rLVp)*`YtvzKJ&gnD|0i?sdkC7!No@H`}Vs9wVq^RgqRvnUHvSC%Z# zYNYWPe%pdnq#o{8%B@Q6%px7ni|v^l`8d%r0;QDWFaf7A3d&uQrdWFpu+*k(7HW?>+43>?Wn_DbxImTfZSiW>@bz0DU4;Z3h7%Z_N zu7T?XQ~EQdVT0Z7$}RJ4cZ=l@LQsFgCC>s%i2b`TgJl>k>#ey6SD4F}M6?RCRaqI> z$}*EW>%JKDmRtCu7u@d@)SgUwPtfpph)?Xdy-uv;ztVG)7&ki}Jvv*sn zvg9vjiKQ&Vm6vC?RLCSNk(ycVarkL`m$TblZtsva-|XW&tyZe%=WXG}dB68o_4yt6 z?2RzPpO{i^J8D)eW<^Ql_+tOh?utH}y^e7)&V4*bVy$~h&B#g-W})^p6zI$ji+n{S zgeT2mDVKR_j(vXo+y`c67Un-6d%tDewwJyCVBETYp8AbBPYKh}v+UoEsmx9*P2zuM z=0ai9 zkaa2{1UWNx%Zch{0_36Yk1!Kh0x}g*)lP6jZI}rx54B4C!m2el{LxZiS*P92Abg;Q zrI1}|XIO5)0NRoo<1E552>U&PK`db@uq?$Ur!Jv=?LS~7xPKi zcEtCE2s=UV@-NFyVEJ=Hb_0}4=v8MXnr!x>I1t(i>?_*QkMN!271&`*ciS3!bAqIu zz>wmVe>j&dcoH@a$RC+G^=c00i#Z6$*R>~?}5#E<=HG6T4wpfe083c}k^}onp_q#_ z=XyzsmKWPZ+)TS|VHOvq%|g<&%|#E@Ofu-3(klI!AZG7MyTlk!J{+mNOSHWR>eg2s zoH3;6v0$^oG!G)+HNaUSi`FsTN_U3d!tCE93s-f{GmAXsU5#KFR2Q`E3w5Ypf#PRG zm3}r+Y8x$1#M_mZ=2SJ51Gp4%(J3SPcC%*p*F^vm<(a>UnfZ-5fi7s%4dsMd;bNxZ zmY|p;m(g7g7$E<(NL%&359!Y-ZzlgfyX*jkA2F-WxF@dK_D`MZrYz|#TA%fXW49bL zR&-XLaw6cOnN{20{QS;KCAS%t%e)+)aIO_oq4PaL4BIXy?p)CWLpgp-vB6cAb*?;Q zsd16B?jfabhWM;(&n`z0=+*aVRBx5OSjjMrlMh;TQFYP6%=g9Rq!1U^n>jG4!@9!h zTA|xa{Zztj=KXcuZDwsgN)G<;^&d1;%{t#m1xWPXi5+-T*uyY}ing2)$sd4txW)R8 zq_n3T5_FrfgIjpb0t}8MM95K_Ri6Gpgc#+I{6vI*44JC+PfR=%%lQXJuRag`M1;re zVBj$us+r>+vypPRYla%DqOzPEX6lX_4l^`+bctvB26X9BY?`Z>PdUucDMmt+dIhIL z-pmp05%q^wlWE+#oT$Q`J{+-Rlg?`G>*}F<%!Z24y2lJ13iX?4?8L6!m(wg3H=Jfa z=vl3^b(x2LXGxEAhq|VAFs{v=r+#DNDaS;RQ~XYtsseLJ;)=`ofw|(jhRW3*i$P#y zEwMI=Sv&*XBYV}pyS>HQ_f-j^?6kRPeDj0n*K~>or0crK2NQ43E;jCbD5A!H&O%HI*jmY9XiF)>TsqWp-sWaAtC&%Or!xUbTAfv=-aAy!^f zeD4IF5CgmR#xY$TFPSjo)|Ao7;=(-|XNz`S^el&h4T`wL_O<-dvsR z*Y#;ss|j!qw7#{&uLso^3-W`%xo-Va$%Gux*BZ-Q?2}#Vy5SkV`8w4 zFeEB8<@7)-bWmPZzSHDQIjJvI59YFT!7#*SX3lvCg?a@pnS2y5W{ngtOP}Sz-A+l% zleZDxIXumu`U3-T@|%g5 zJgbd)$YEdaO1$hISE`F^q688AXp<{(oql%Yd^x6V@yk6xbiUYhZ!8HG%PaMw7oB!v zpjVxUYT|Ob;347U2idV%T}zVwP&nn^jj3!WR(FUrMVVD_;YME-%dGO$jkVbo?Xn3V zRcgKzG3yp~sb=VTChAwqGmm#t1G6jCfDO-zEDCnyteshsQl=Jqju}_SvszvVk1au~ zYIHpFgN`_&g+&!HsFv=yYT~=je)Z-WV-H0OoQV%?5piFzyh^;)jBuC?xw1akNgP6$= zrTD&?70Co!zOF-+w!lK~L0p{D!KG~Y<{JFZF243@7-Nikm=h->UdLeT8>O4sf zi+v}TV4X_GgaVGgKF6ATG~zwt2czJqPKloKzvBrhc2FjCjWV z_bxXYFNw2^#m5(MVqOxq4cpyCo9xh z%OopUiU8U=0TM$d0!G=`vJl3q6*qn~^{+FYiSL|^nb1{nWRoT242CReer`Yc43ns8 z;>jX;I5OPbs%YG-gYSVfr~-HKc~9bf-2_YvA0_nN%B=7lIyR1(b;9}xmD|#*J(Hs- z`B`!BQn@YNy4-g|R}{m!8+!e`qMq3`*s+(veh0jv&p6ZV0dbh!@m#ZbH);84cH$_tf+7v!QMRJ5x!@0|+JB#e4 zOR?0UTh29=s!Hv1uJ)#J^^Lq}+(lm>G1;0Re&M+pJ+?>h81kX}EQYM(K1qi|Q#u$b zhw%$i9^)OMUdAk>fW|ES%JJ_wmy&;JjF_^R>KA;i&O5Z2J_%oY&GEGrU(B-ZuXvGz zp^9j)Tps;}#TVU$l4+A%C!Sy~|4Lpk@lk#OP3@-sJ{5SSrzfw1^Xn8*&ikkIL7!j1 z@lB2y{zM5Y+1498R6zSeL1AbH^DS2T2b_bo-ZA1>0R4a|v6Fq-FC{h>QA%tqvXs~# z+km@aWe-O2IIpF0S|`qHZ1FbNBDFGQev1eAP^w!VfWSB}TaGI+)mH0D$<%sY^TLXICi&2YyyCoU z%dRjl1Bc$g9rIoHFs~nUz={fujZ}|Yf0;zo1M_`j=+<9e5I}BygS$8!d7w2qPKrh} zS#Y5_ZFR5ys!588d-WHqhE@0KuX>a^^Xji!<=Bu{f7KotLFUz8mK1h5S5}?!RQp+5 zX5+g@uyAK}ul{05`Uw^fsun_w7S5pp% z{Q5SFawr%-nnMFO@zK6x_H!fQOOJ&WUs^+w!-%pK2|&O8D#|$zzy2yM%)X}gt)l!l z`t<`0zy86{ufNn}Tz-9fLwK^MQ$&%Q(fs z>hbH_ek@s`%U@c^d&8~2T8bdi2HI8b z5wEGBRKLc8w*BupN-R_zmMS|h-THRkmUKNVwzEaL_0Ms2xBichqFcYg#!+^MwuJf9N_tp!jA0La{T}LA?q%VjzD&(a)N|x zdFUr8Z9QrZe$kVKKdZ6>>aDi)6YAs z3<@?&1#7J?ELOHv*(RGWO+znmi~L`y^E=oP?O~XcuyO?+1b!kjDz|Gb+0?=!;#(HX zpj=1Pta6T4cB#x4=J`>_AdLPqZQmdC83QjuF-K+;m~999QJJV=U(vaJXh@X&_nP-E zHok$MR0G20B!JQ?mK@M+-!cmk*b}CBXJU}4W`-8QOq;jXQvrYBFvSxXVaETaUqC0~ z=Z6YmH5Rt3VRIJfz;5jb^d0D3{LoCz{7xozm!{)cSy31_fL5;YcsAPx)QnVZ8SQ96 z4Sj>##E8zG>;YFVUK5z4LPF0H;dm|uor@eP zg?Eb`cMMqmdqy%1)h+F#bMo>6CLI@In8yrKGZ0j60d+NCd%z4(_Xsn%Y6RO)K72s% z^pmq%;~aEy_7X>$ATS=zT)dguGxt>VRT&3IEDDt#tbiNVFncYr26reB%*qj$%pC~{ z=7k>wDS^bfA?EwSr0!WUR)V$@)A*w;FT#nZx2+}CT;k+(Vvy5fp+bf1+)M-(2?{Qi zs+>mVAH?=qhc2}0d?;}0LO`>|hrTvpY8=)A2vnR@m%#+L>q_}?=}=o5V!#&QO9>z_ zv{G3Wfzo+t*qJc{CpLgu19ay{7VvLD6+P{#-fKxBYhLed>~t6dtc(XJPe0Iy4s5)_ ziqW-oCV@!lc_w+_B^O{|N>$GoG3_!L$3FDVn!FcNuqv`Wo8;JHX7lwB4mZ~sX~t!mP_MoY<8P$E}?Ya zqrG;*Qw0wnNk&ovE#VJ6FTxu}X;|`5z)o&9u_0}p8Hhj(g^dJaV+g~e_^kK1M({!O z&9M5t50y27tX^TJ<)t@d1RLl^DO@Gfdg%^Fv}MkJ%!nAg;{HHj9=N=-`(t z88R}y$c|OOl?sm?yQR9lpK%URZrB;$W^CJ*phMU;&#ZawmeaWLD(doV$a4{IGI|B0db>&6t6oIsl^GA1O|0)`e?RC~rS%~{iP0?|9G zl5;&r-(t;|GwjR%B$VDTwSgIQgUzb#ySXnpPAx)ju(@ck$F8 zn8XIpHW!|Grcd|aigb?J$8AzXWn8)~NcUfT^xY0;>5(L`Cq56z8l8>W&wxkf>Vh9l z#p16A|6ZXlGU=dm4i6bva~){C=Ffeew_xie)f`s=8!+i3l8s;&UVGjs-|BEAy`bU^ z)?{dl^6fBL1ExC86t{_^`TmBz&Jzy?LAO=E3D~@Rl~ceI79PEq9;?8(1WNAw9uJ)bLWUL);UGn*+QHmbG_oa z)+yS}4<=8MEnlwpDYCv!rJqw|C0u$rXJjWAoct8+B&yJewRN;1K-iW_EE+6z)sb*I zi2)gz{OL;SbEXvIL~R4}=MbPr-n1 z5tO@2f`9bff%+D{!~-x%x+qY@FtmtK)427*7FO)bo|QNe+MV6X}wxfG^u9U`-sOvCc^>-1V(!0U7)s`xtp!z@?LPxrIq zw3an>m3SU_^y=*9K|9SsSX$cy`|18cdUlac%hoMNhWq!QX|ea?zHVo7Q)y0guUi-+ ztuqHxe_YZQTRCti1!kG88AdjkSRTyYlF1$Ob3Qi4vb+2+nDN=VSKtDzRhwlev(Fgv z`yqU4t;b|!=4Oi!+S#vidwC$749+vZG5$~>Vg=)8Vg&<}RmAi>^&4|`XE9Ob`e`c{ zG;dml54Z1YE6mee-Ap7*Iu-&xWkDFP?V3T<{Fsgn>5_JQZzjTy+qF6N z!+YFY@$)HsX?0HEpK7j*H?q`6?1>-vJ+ce=ruO35k-0=+2XR$ zw&XX@#1l#8EV4!O)>2O*TQvW!E?BX}gNfy|1eh0JJQ%0;^VA<0F~)`V;58Vd&EDg$pY#WsG>D{Ox ztPzjMv%`LGf=&TUyDwpTq8t*%h%E7yWqfzBH2EFxe4fZ#7aWD9ZQ@N1iUhah@yRI)v4 ztgSE&8-(o%Tl0Iw5{0rLw871V`@}t1er}pAfbF!a${%OjYo(X{Cn111V3?B?A{%?& z<{u*o0rPx&b1TF%spdyD!}_w#OE#siL(SP18<}JhFiVlO8>RIH_Ni?hNAd{xd7^i> z&uE0ZtP)uE?mMg$2nXD3QAW4Q`q*ZsbB~3+aWoz#crf@tcQ*+z-xnq!mDn(@3?wc$ z%G~R?KOfpxfS7Yo8Uhb?cdIOIOy-QHUBj(V+k~-vZ1P*JEi=o#Z1X;B*~Op%ZA-2g zGUPp1({`VMofS7>qr&ud1Gf>T$&gc52jD58zN#MeZb|2`B7k_eek;Ck;EyAPa(~t( zm)d@esULVYWLtPzl_L~(C))wJ=8*h+sH~tB>;)&0ay zQGB$bO`R#^Bk6(XeS5i0CF6RtpbT32ROx)9+i)hP*&8ItWnKW59GF|e|!JHaBffCyZ5OL#Y0JOz-x@l)CF9mTs(sGWoVEE@$%svrYx>u6@3A z?|dZjhxz~9`mby2(_p}mJwtgkp>Tk~*`rXNFSmD4+B8mvS#Fv0nGkKW4CmnnC_Yqa z2C>hIJ2F>%crM~)OSA))k_w~BZRzN~H*tSrH|Tm1Z?oMh$0q+gff9}5jIcU?Nus+k zq3JgC#G2}R7MZ^jvd%W#8e)d+fVt@Q3wUFayG(bCuZzvvSHy!xbvpK#Aln`_=6Q~ zm|uv2Tj^jd&(Jo1CKu`lH$S_11wXMo`i3oRHcB`g-<@q;b~mkNX(qROM3S6Y5Yvs% zb{iL0E-XoixVDv}(q7=nw7ukwPdUS$X!1dYz!@0GONQt!HzPUD2xPhhzwAf1f-lS% zDh5IG9&S7=|GJa}aKLt~C0{&A_XsmOy4vyVjWF)~wbIW{NWxrjP=d zT}x4X5-z%LGav9M!AkPA)2B!tK0sG`1T#DsMti{H-SnMwblUxeb!fWtM-CDa-^EgH zxy-wr2rMmwhdL|iV0XG0Ge=a#JVEf|?#oL*^+cAB0`U@uqzWPuFIc_)F3jY#Gh(x$ z&aFv;!LdeYEFN&@Pvv1NdL=o<*-hsq4~8}KVnj_m`0U*Lz{u^TYQ`rj3W?FkVUwyX z1e95;nFn0{Aby1-3Yw|fL8*f}!-ZjoM~I)cHno1bVX1*jWCezrt)CO$pTXU2^~#E9 zkXcEn<-Q+4CUe{)7@L@x{|h2%`g(?wyzIM1E=L}0*60bron54f(rgk^w6a z>tKow2!a4bXH%~zaT{Y&EJ>`d7w9$3+28MWWaivz0#j>5?RFA~To7pM@XKsayo#~Dn<7l?+p zuYiY9aV`o0p$!eQ-^Z4%@2NVxU>Kw6DB_ngvTnFQg`{FhH}l%Jft2vWKYUvbhZ*TJ z-yFu8*b}rDcII0|G+9Oxr{u6~l+qc@UbIM^DN7Md2Etfw>s}<3Dt3Z<&T6KL^d!i2 z#r!jv3@}47U`9Fc*@?iw7YrEQ5(bk2UJ4y7F)PNJRPKeHF$qGTjy`yYxNQxXC4Cax z+h+s}L;mXP(-qRi{q9==>87kSpxqofWgs;95@ak_Fy8~1LZxsrZ1>c;mWYR*L`~6T zq!)ri;B_3p!f@9gGwpLkOscK7+5wS54deya?^ zZeNBg2bi>i{If~dUBq7iw@+7nx}IpCZW&+r%^KAG+*A+DXvA>kJQ%<0d_6#+8J-ZwwS5CGQN1Ga4IfuM_ zfdiH+=O+O|1HQF+a6E- z#xSAKFnv%a{*5_L`I`R5oVWW^zcG2RUi5{5vI}^TmDV9pjI?WIhm=M$vjTM0DdG7U z)<&ufxzEoCU}zk%#no!#=D9U-b7lLM@{HW8XR-)w3dVWJ!C9+ecDYBe!~$3=Da?P5 za~h4Ba-#g0S;GNSu#Xu1)1CNP6sW}|=KB{>gV2jnlSiaOfIJw=W+l~7cTILHNuqB^ zcX`iGk6$qHFK-qu3QAt{^9s4OC#*2cWoyWje8hiXHXW~c#gx*fN#rSGg(i7mm%IWa z0TRD(G7|6(c|s^ZVh$%EfpfeB$oPuGoa{UIHDc=y(l218+Q%h8Vl74R9V4{X5+Hx< zhTc(H+?fA}S^Fa2T|cYKkOn9pCuxq2_lZPEgYzXr9@s^fz(|V3FPxMFykcA_37o*m zNjMXf6q!-%;SwVA&YvFd58fC-hDHioCnrS6Y?WjCwR3BIdVB<~W#5<(A@kBb5+nMj zM^M=NRtq*r8PYIWKVdRtUiH(1G=^wldKB4o3eJ)ZGH9G;E6I=sR&6Y0$jl;nAwwFh zRj&1bXB!+r0TPePmeM9)P_)xUGs+S%DL}>{LCUJ`EB%?McrI3!4Hw+^;0aJaWVLWDGG8G=Xq_<15ALzXsa$M6K- z2h(+_lfx9d)^uSTlqg0dh1Wj&w8X&C&N^F<_Tv%G*6E&k1%}4>%2V;knP#W77VyrX z0H&~0Rv!JbwYW6M8r->`w)O$4^K&4q5iuvkerD7uul~1@5-XwaI?vnnE8M4jy%wNr7SqpFJ z`g>d}QCczt^G zD;Zm1Q?qM?Otf^zV1u$`|Dq?7u07o_*5zy*x^nQ)U0~21jz1t%jd3=6MUj1-ce71m zxS+8}72`Vc_8Ijh``CxNI|Cy+bB)}>4l;3Br}*LyNn|90`Wd;ho7 z4l8Q4XkPkrQKtddpAC*o7QpRuzVv6qq)UG;t4bTD@>W>N|I(k!O1DP+{Ifr=2=NDA zpPueW{n;>{5NY~znUnXV^ye~ruvY5NfjR3r#%J1pVj2=;zOSc#V?uwncb^b(Oun8F zaeQuiDbQsV@oXs2wqEU0pluJ@OMxzH^ZfgWT|VojKo@I4`ck0F9`ZC$fe!le6%N=H z=pPgay#g)b?>LqVqd1nV{53cxb7jU2leL+7(0PyQhc!*;G?A}TQvTzmL*O<9V<>Ixriv4Bevz5NNaV_pTMj#7z zW_S#ro~R*`HSAUUydqhCZvNM``5wuFXO>bu_uy}MQojkfl_6}+BUv~N!lr}UESHR0 zcJ&#Y1ygmh@5-LiU-{)O5-!nCb_e(HwM!a-IpqFgzm;*1mm*uUR2d(Uc8HN?o2YJ< zR3yuT==7*DlI4M$U_5wWk7ccA9u2_~A0ok#1#-sP7Th_f^95%ts4&Ks5?Q>F4)7A%aLZ}Ayvh1zGV(00==w4v>s zVIoE>%fTp?1wWD%4BSK_OYp)0l_sfJ7OZ|3kMC2ff5O@E;qFZqpQ8)-j3~PF5XQi5@AO7^ceQYr=`alJ~**M~{u9Y+{&plGh%%$=^uX#9)u5>UyFc zKQ?}|d|qi951hkrAvQT!V?RdpeQ4UYNp(JtY*PCSyMxIlwU18^l>)7Ft#HUDfjP_B zx)}T~Oc-9{sXs6-o7h~uqS0EM?NUernEXheo^VFtq`Gg9!ig=|I&_L9zijzkl$n+gr2N1|*Ja}+ z2TZb8BpQpA5t7kc{=oLBCWF;`1nJq}ZAWG!L?6Fc^aMJa00X|nP6B|j{A=nRv7Exd zXSF~hYsrpsFz%%gXR&s+gUg3&ek0EdaMwJbgEx7)9@tl7@tj#x)d2B;6?}}p+uCrg zc-uA8oho5k2wGxwf5S}M(j=#KLK}|x`xmz!|u4qib z?KGR>lr=|INuSAkXp!=b3nlx_E%SsKQH{ulVmPPMmg`3lRQI68#W~Ji+}B`9Ev55k zTj9RWO>A4>2LBuf)uTqINNEZ#B9x`8x9bHes}Za>IM8nlnAG=0EE(>#e;1~pwTQ;J zHK}kWL*^%va0}I$2hWbha;rG#t%kvB>L9gaj?&4yqQi{B+y=HcSGVYvJ!jh^xo~lB z$+D$fK;AA*2S)nZZnB~_Y9?EOO=URGbBhh=YcGCKMyYUYAOEh>;rOJy?-4bOSwo9N zUCz0HG`q^J%%1F&KYs2BY%x~nYAewK2~-*74FmB_gntuOy*7#OV9?nrjde|lr^>3f zxtZQ%ZPfic8B*Emp}lgbvei&4uMuvRqCC~i@F;7U7N!?rkVTcPfY&}%uO7I6>f5&- z7!j9C+gD&biHbj>fMDDGrhdvyqIzbv9pt3tf0OMCH5dO6Utf|ew+>@FuT^*j4zVZK z@LIiBTlvCL-9B}cTnk4&ObRHuHywv&k_Tnl z6bbbVMgFGUh#<)WJDciYq>l0z;!p^lW)V-k5uN~LRB@Eg(3zrPZ zVK6fl=Ve(;&YbKrO7C(hYrmcd%B+f+02R~DS8cxJG2mh;4C9=mE|K-~q}{u<4FFuf z5B+pwcBIOF>l<@)x!lz0>v(-L`*EK;BDbb;ls5Tgbj6NHI6AK~I+s_gfA5no99Ji8 z^%*6mmE$t@8x}HE);Xg!cCysb@y%+pW7KudtwbIKS1Geswvx+kz%}hIOxF+M^{OMH zA=*k6b)7kc2vvF68HXr?4?O*!|t3ba(Hu5(6zcXINm>&%rjEhR3b zwPaC_w&JF(+`=3ZPxLHjkIFi~aJ9o^FH`pg6W>tlk|^t(QI?EWt|;rAQR-`-3A_<8 zKh@hEd4MkKoKY%G2$prus05aNMO9~lDm|^L&Ib`SocRkn+czc|GxW`7GfA!*)3C#O zYJbP06H+;;W$Peyq>O52QBuS7*2m^9C2Lu<{Y{hX&iaU&&bg`eL3K@MF1=(FGBj?D z>PDO8ar@#8KG7n&o`$#=_B|anoil2;c~Aqxj`(JkbSAk(oaQtEWot?->CB_pytK8L z1gZB5*m+%Yzybg2Vai7)2tNz~O}){o~w3 zRp*0=s?L0C+C);j4$gEAS-Rj<)tT@UX(ydC70^a3Ot(6arj{-)2l11- zLvWdR#KqPnL;lI_sq?;0J}w-ne7PqZIU<1a!Z0}Fg|lvI#V%&_jc#{(OzxLC>D}y% ze%JZ_?s5=Jd^ficSmTN5^f3PdDh{SQNHe2De0UBqil%0Mdz1%u=;6UAg7W-=;_&nh zvMz^EY7`dDNUY0IuhMMJJVN#1Ov}M&B~A{G*2;o8bz7}wB$VKZSZxKk{M-+IBLd?b zLaFsVH9`ohVnP8>Bm_kaB{9X+{9ZSNo2w)Ae8UdjI~c`Ko?lQ5rM?led8|Ws$Wn|b zTd;mOs7QHp5G66s$;8QMN*|dyRM*q#eP509G-zV z%G|a!$8nSyMHHiEB5$*mZSm7ev8ZC`3nK_QQjT~?S4^ww`h=A_9F1b~A%iNAlCZ=h zYTyUnyKykS-PW_&7bUp${I-KNk9c97TpXP?+i1vQS32pq{={M1gA=u>RiTtzD$9&h ztY+1dR6Kv3rIyKto3ZYoDQ?%y_Y+f0Sx1B$9))vs<&q@gwjtdb=mPFjAzlWhNu7vp$Enc8z#m%r-D&KW(+hksR4FuI(Fxo2R!O_393 zcxb47j6--H?RM7nKn;^U3p?jtZK;jI&KV^W73aVclF<~r`)W<&hN~Vo7&==<)9+oA z=@S!$oi7Zdj08Kp4o8ZO!%ci$6KM`qS7GPuN+(P-(X=BxK%Aom?;b)qu5VOcdm(FSib+`6Wtl5F$Q#I^ccA}K_5}q`C!gc z2AzFlE){UNFU79wd@!3DQK9l9Jn3MfB<;bduCrh>8T1nxPu>v_Vk*GwaJZo-dZd$8 z!6j^R@rD`r#Gljg^zwr?>P9_UVs1>|by&ppk>2yd?F2X&j@+`U zon2PKRMt78RA#n)2UmVy4(qVHc_YqIkASpq%-J}<^o_8und4Bxym)#KZm5#ZoNJWu z>>l~4=d8s$VQ<=tB*Nka&0V!7t>pz3HAr{)q0&gI%ojn}pT+ zuI_@)8U2E*9pC6^s5{{wT5PheEhS+L`Ys^O1{&caxEYknd{ z#0HK}HbXeaXIB>R3EJ^MK0(<C% zsvb~$3a$1p?DY09S`!ncdT=+5p3e#!jsn-sZ-n&5n3#+@vF+kOZ06fOR@i+4&l6+E z1apsI!3#bp8g6axBCcul9ByVxFeI>3bibL@#+>DZLGxdT&64&z|G)&78j+%{h!~D_ z*`%?sK5g3cWT&4@IK<7^v8P?^-QDIL_^7;3sa~s&agrz2AE`Y9xN6UBsHV1s`T1P8JNuo2@f7= zw$*eFjYL{>C#*IBWWHhZ@k0xCMr~IQlu0gm?kYy)q8X)O^O0CHYssF=A8$C4=4}1O zM494@inOwR9!&Xt6*qq2u(YRN97MB~TwK5{Nr`#2@{GE|s)S4#A)_K~5-5pk7CU@u z(maYY$XBCK~0K1ElF$eKwThjM86O%)T*Cwy^P-Vh2)z-^6e~j z`{LfKFS||ZRxsIEwQSjdZyZd~+Lqw^j6OQ^P})*f>0L3YWD=OZNgsbD$c`52pM=T9 z?)nWFSV^fDVs;V5a%J_F1&1^s<0UX&ESSq#VL2?Uq^c6OO&wz(9-CZO6PZ1+_@sys z4|;5H#K#r}>1LEce(Wg01<4k@dBc1oVmOKUdtsz~6T^1BH^QkkR?zX;U7e?CU=cIx zndhAiueFM6m*d-0X9-}v;P8Yz{fpuMrU#wyFuZE)&tpm>sspy)Y77iwawxQ9dlFn1y*?xtQ5WbiVpBnh6_zM9iE^46QiJ9Uwtb%s8?BjFRXHDwWuXAhrH}$YkXna&CE{8 zoHG5T^|2{>g?5134P(we2KHC2970f$&!6ia?E@#5HFk{r0c&_c zhZbgrumaZaGiH>tdYl}aho_zQn7Ay}mBGv&HPR2kM}VuX z$Ifft5$lN{fw&{qW8=}jBQ|Dpkp0$$={r9YJO$saVIgW7nVnk6b9dq|OiI5vu?VY! z{DY}S{fw&T>#61?8jge&{(c(U{5nQ#&SUMS|GIXDOzr31k)5pZY@JhXmV=Q4WA%Az z6M?FBwTp0ppf+_B@t$l?ZG*T?hNuypKin*+0UdQ-_yk10%guuF+3my4f)d+T>&zDp z7okVF;c!oEjNL5!h0D#tZ+y5}IEu?j!b;V@w$8Apv9$_JbX;DR6-Bw?2e4EhUKTFq zu-#`Ec+Mi zW#KoR!@A;y=kl^}gqN2E@UGuoVpjDMf>E(M8XGK@ap` zds$G5{3^2e!uFcU%Yqv0(d?sm|JSzsCHF&*o=04%I!{#A2s(~JZe%Srv~GD>)()Br zFAJ)jt44<`a)8_AW!doAjg6791aQB+EUVt-D)`BA`B?T*vyTOnh#bP|56oQ?mD^j_ zmW`s@d{mO~c1Klx`B=Enr`%aq_kj)bjk*h@>4|13VRUZaYCm#DeX!9tj zbEgmp-#7)e@JyQ(pJ>CZsrv;d?L{JRL2a$)4^q4O~{& zNQnn*(pDZY_hop;AHSfs^3DV1PL7Z)xGx{O|L5BM$OA5_f2oug{y?#t@SQh55PiIt z-mfmRLL(00$MWr58nf@`)?-d#&rU_NF23VukdANo-ecWD3g8%;l^@9f+NwJa{l&Oe z>3$-4r5UPOo|jQwIlwPG?22R#@C&;{rE-7|967)jhQV@#m&;2gtQ+{d4^`7b)oDeM zSyAtpy!*-leqr;%$N?@WNf`{^)N6}%B{=0Fu^DqVwm$i#x{0!J3Z01R8zSK&fSj;U@eO+ z{+va7c1h)x1V^#O?#)~~QiB=52jdK2zLaf{s-}-f&z(ubh^(X>%)x4K8=BZviA?P8 zuJXW@{CHxdopX%68D)F3l3p0y$(@f@VvJp0chnnp0`S2EcfZc^H)baQm-Kw3ZTts} z1JmQUthux9G4^Gq61ABMG7xR_U3}gyb0wO*u!HJco2fk0qJwHPz3|voObOaH&!(RrI}@0UaauH1aVAb-H_g$QmXq8H z2xQp`pvp{zsisy&rx*kp7;6768GK0LL+{7ad|YD zEEL5^@QvhLDph7GX?s9jndxAn4in#rQgMMvK9a1=M7Y@xIQf-lr`$8^i%5gl( zVpL%!g4@YmHPz{~G&1f&e}pA++|5i6GVPZhixYgK^NqSAr@%+j4(|$1!vxry(<~b- z#90b*g`-z$7LmptN~0p#*L+ASz(aEz!@^AZfyiB|{)7#GL#$~<&<3_A;>CQsvuVS* z?Ky|~d)*I>lrbA^7^~QxlWh4}ytM*QI^fjTc&M++%4VX=j_`R!OtTvH4MvNwna5 zFq<~kF;pIEimbFlagmjFFj0|7bSw7ZK}1TL$$mE>-SbS25^bDU+X6$Gzkc#JO878! zg#(h+V+#@{oa$C!_KN7vfnO*xB1Y}lLN`N5AXpik>Un*#r*^4^8pv>SMAsXNA1{yiOQ_S()YhGVZS|ZePg0hY@uV|Ux@Qgtmil8 zeA_R5A);>VLC|L?&|%J{8+JaYgjHg^2vZN7l*r;Ji=baVpMOWQJILg8y*Vzb5H9;3 zlFVgmVs+VA?sUJ;U}-l0?n7D}1G1kOC#+Xsgbj8N%8XoPM@WgG^sXg%S8o&f#o9*l z%};vPy}t9NoK$yV=$j_|*ny6bj))DNaR)`Bm9pvxM|=SmW27C7^oWBXzk|Dur+XN4 ziWq+nVb5tXFD5dgVaF_}m^&;2cfz4pI<4-)*iBGUE73;f6A?*u1!bFCB#2^tTeA_0 zNp&Sf0mDeDE376gMCA)>8=c4=dbpaLQ&&>xEX=7Zs1zKPZmQwwg{>v#QBQbM+2yJg z%5cFcc=N^-eWKV$$0${fU`4uzo>z5JCDm~b$p(W-_)ah3n)Gr1`B=6|_AOhq=pk&h z=*7U4x`N`@#e4+!DQaJbxFQt^W!C2HJ31Jr)V&BhVU<$HADzK@mWGid;Yg<~iU?No zh^UF8q$bY0AD9bMKc5JyR7xF5xY#?QaZ6@ex?HYgM2NOO4%xj?VLZ~}ZN%S$IL9~M z`NkYmP0h)P=OL)Sm~!7cFZ`H0DRUz~POXe=x)0V4^o;#g!g$Me%0Xu>&jQbxg#}zA^uI}R1kxj=Tq;TzEC~G-% zkN0$btldX89cM|2!lg2?3dM8=d=615vgx>z?WDSCrOT!o?Zr)3FUIS#pI9!qF27Wc z9yyUt_hRz?4Spj#lSt`wFYJP_kxutwyfjLudto;usow7_CBxZt6-8-JGF?SYT4nsH zejIDm=3hIT?jRy}j=vC5lLwQn2@QSHaiG9rL{Wo?ypfp4jH_~1JxU7Us?CzO>rSWR zv{mqsao}8*?U&;ueThdp-HUNDq}-ES`rbA-g2i6WGSXq-8k#|-!Z5xr4P{(r+IzZMsSu`2?LTcN!s_hqn+cSK(LcyoEMK zwT^R-I!B7;Mdj6g`(R&8*_|`M@p9B%i|-;O;`(ewh2kr(?}b$cCf<(>@AHK+cgYow z0SCKXW8ARaqX#1|uYMsMtV7>$&{Wm=uED2@2(`M9x(qgZ5@FHdEbI06UUn)z-;bJl z;p5dp+Fr^lBoT)gKOnW`dm!o5q{2tzuLW}Q2j zp4Vm_TPJzM*=yK$RgwpA#J6EE8$LU+F}~C4*gT6#Vs-2s;XLAv2z}sI)VXa6B#(C* zj~Q+CHx9Gr?p}3@a@JBM+pMOy+chub;?-Io^;ny6JGr!7>wco0&LYDb?xXFS7QVQ+ zWU6Q=E}d@pIu&MPj-U8HQt4_r;ms7~D=K?#>l4KC3*$_>%BsiCW5bC+b4i^D2)3g{?oy=E)x69~luGx)4meRN z-35#K)`>@z7gJpctVvF_75CSu)i=IeicJF zRnE9OK_Zb%cQ9u$O=(|JpPdt~Ou84&J8!OPS6kZkV)0~Yk)^CMe5E>(J+f2|=B#B|cP!_PIU5&m{R87X z6^=#zMltsiOi|f%oUA2_?!ou%J}k@YcSH&9$L&3;sC9c)m9HrE8l!@Wfl{x@!bjB$ z_Gr>HQTp(J#Lr82SNce&t0j7pDxL0yv)xfSs8I{dIV+s$yiU(BH#o;N#pq#ZHKJ+ z_LWwJ~Qg zpT}OqFU)30`>lUqf<@J6?5$Kfj&|9avA8_#xASqk2Y*~kRm*p|uel-52aBq?elUQV zUBh0Pbd|*+&4VQ8)kvRJCfy;fSl|uE3I9QUyhc%v?m5~eNOFujfT*G_TX+yL0@Q8u zl1jU*QP#>-@$^4@hKdI7F5)_lkq+*sF)$SgDt42s24c}xK&w# z&tZ&S`r)Fdwr+Bf^RI0mWu?IbCm!mK}FrhRIYDK#dg|&x~o2J z11icw?k<>kVcFPZ11ien1qf*SBPm*R!+xdq1FAQ?~r zZu5545Ox_Z?e2PP&XfgpPr@ctOmi3Q5xLE(Lt)A8}|F? zYzt~2wgHvzSgo!8$!1Oa@#$?JM$~Id! zDH|%*Uhylkp|Un?xh|8>>xNQMOHrmtsdu$KE@J5{x`WtP^nT|XvtJ6t7Pj`!Y-wO) z!eM^i`o>(hW}9VK>+x@o{Gp+4L{3UJuBbs7qVW&Rd8sxB#(ZHS{{6i5jmb+6)ARA4 zi1SX&;2U$zv*)%8Ov}7|Fftu?$Ss$78#S5Ahk3iFxs%?(n7gtvH%as-x)X;%2j#}n zW#2~WsOtw8Qh>Ypcv!d(9Cp=>v4wm8f-T(m#JDZo9KdDa?x7?-98d{UiHy%EmlqYa@m5cH zR}})6?<)9y*|+Ht#;L*pd_D}^{c+g_8@T)9>Y+Ap zgBHRuIf8(rvp^z=s~3jt+W>>V$-WIt7@a1cNIO-mlnXxlurU%i_zRbPo0SSb?Axr_ zciFf1DYJbW@J^%jH>{J`zKwN=VxHOX{IG9h?IIwoaa3bFziQTId?aybyDs}Sx6|=o z+|HD?t{^d}M~1KM+kkgDulnofncx+JB2W347;#QU{-JoIrq}#n;V3PZEFQ5%z>A6N zYd=%g51q4@W4h7R)a&!(`s!#FVBZ46JS*y=MF^9h>5q%~9!!MH?zcw%q0Ro7DY-Zb zFNL9w@trSbM6J`pa}M$Y?|FMPv|A2$m}WukmNprFO- z0MCj#Ll#7a9*>TyGU=e6l}4<%}#F)&vgs3<6$R3fnOgLMz8Z$H>1369;vaV^0gaS?lY zBPQ%8QH#*~IF(I#%nwGA!04Blz=Ba9f&>act$suelu#Ns9k*vURx+Vi`>*^!eYTaH zKTupp?R=ut-O!SQC@$R>BiwLY@*87Wg}+eFjmK};qAprj5 z#4=jmHEkrZ0F&~yfAD`89_8AdEwPNceqRF12+ytrjD(!)GrxMiL8u!8$L#g>O8LGRk{Y zA;xnvsvB}yV!=G3ba+iM^9X6@{vs{haL!t)=p8GGWn8_68b`J6UPRoCa+n{tFWqU` z+D@}eZRhdnOJX5OL{Jy$w%U)YTkprgpu{qh`^iQo@`Z6G@{2K}RwnWbM^hE%0lm8R z{On2mVyw^GJUt)vom3+73%1Xe#1+-Yu9n0v9JLg?-|%E36PcK+B{AV-H#au%z|KUz zFfw2e=8feK+%1WT2m_M;ZsBhhjW2nrt7lM7g?j$_DQ#Q@#?>_Syozlp4Vf!+?>1Jj zm75;0%I)I`-`UDT=B{*KxN?Qe2o_czGWXFIEv)5_t^adveWW4tm8d?yx_%->INkWg zPuL{C^b5WbTulyxoN>`D!O@GV^_W(4e@{gdPnR2jq!aOTJytG^Lmb0Xc3u$f#u+7N z&*iz+=k2 zns@XzGKWj~^NG~2X2caG>OPtgSJpH;wxP<3W?^usH|$*GgV{RbIndvjor_#&M4WBk z+*c%h>gpVT{Bo4DTt=hy7POA^FABt+ex=iwXuo6?N`{-=>@BITTN{y!}oPF(z_x z1S(c*Mu`F!)0{18EUIwwY`!-lA^Bh;{h06MPsG^iTD-2B!yJ{5%#Thx$#dVs@S)=a zZFg^ax`rK%rM+D(+N`ON#B5~;fbxcJE-e=M$S)>s&6jdWb2dQmlIr;J!)TMJMwLcChky7*6qt5 zqCk6ga3-HThtOBvEK{LAdd}Fp%DPH;}vZ6xVyjLrhA_fCwjQ9(Lo`4sRdy*fWv=0o`q3fmEe;_)wW1QGN^iF3y^Nsl! z3qPH3_HordC2@4>O#9R7sg@**S`ia?r8X z6&bMe*U#PHHyH5AnZM)2{!}WdC_9``4mQ#JhMK|)^)Q+0U9^3w0?%P}Zu^76setno zp>k;g#y>0G0H(5o+5Q0-(9qZ*K2e(sx_bzfP8|*KB^Z?PLXAI8VQ|tjo~KR*aU@SJ z977x^*)_tf5Qc?&hUY--l{fq15GYMS7)jbJZ;oJYr?}a9f`*jy3x&Gg{f)x*0%sJ3 zQS2Ng=LNyIY96M_LRpP7MGaeqm3bk!orOhYM1*S-8CNosjtlEm&x-5jg0U5OQ)Sm? zm3U%+JHYN`c&&X}Jt(oNhet_G0w$A_t7qY(wEPLmS*(PuPf)p~)>yj^!qRE1%WK=k zkPryWHz;TWF8Zc+!!Y+RiD=`Ansv2c0hf1t>KnmSbbuK>+C?KAMfIpY4+g8=rpY>Q z=SoZSgNsjnF^fT_$tXz`eFsHMW{gaB0q=qnehsDCHH*QLX}}be>P>?wHeg$L;)VIC z$sC-dH>k}612*%*P4N35sdCUt3Am|KPa69w1PjQh@=-5NsgSyxmv-F_`-x81fJ*2S zE|5Sl=xM&ysTOeYA3pJD@nJA17mUzoO*}B`od(}vG7b4J>f)oOMiEsu(=BRhkmrai zb$Wo<(*g}o6B8UJEBEA>1(p(R7;5%B@xn{C4}M_K0z>qj9?bJya_-ozV=N-%x zrmZQ;z~G?L31-OOB$1m^T3iMArYJ*CWb&&~#A^ie4F*46Pw2vbVJ$#kV%smrQ5RknjRG>@@5fz;Tu(7xUt6IpCICU31)#SQ# zc`S1R^fCQqU6!-#H9#*5w*l^&fTDt={CNFY!=VdndtZ%n*|@xG*TlrQ}7pfBtCVO1Hw z$PHP;xSGE)=OyI;d?St?gFN7;ePZZ&4NPWkdMDg)6mzFNXg0BucYCs0Y|%i3F4?&^ z30%Gdn3Iohz!$jm25tTj@)|nZAxA$;s4HM-dC=)LnNm)=1O{RROoW+FnAO7*bi;$c z$Ty6k7S10ahk_G5>61w!oG;{z8b-Q%Qk|LxI5$fga3PB1aVb=Per_Aq4zEK zF*HaJZ04dU{|z`vy-8k69gdcCrvrX8?*XxbgV4&9;k9JxU)l&K->}|ZsONG2ip1k8 z9gEcr1~DCsC(9@Yn0GjH!^ejI9Lh50rapwX>Tsy~)dyy92aMr&KqtP3*6FEpL|8r1ClX{Mvk9V9m? z$UwS}eSt6YK4^hBLTZsadkd*ZCdbG^NG;DJfpQC}z~m#NBGR3$~1!Q4;DFQS+j6Ym2B^Z7{b4nirG2FBvsYMlYWyW)H!cGHQM=;a+bUHP1|~ z)*@=2%sJd5>V~(7x-k|}vv2+oQS+po{Sr~}(cF%VnvsXIj*OZoW2Y`5^;M6#mLa5a zqDMx>x|Dj$s94H1kBo}VuvkhFE%y6}sT;E;)xdT)Zk3w24YEgTnRUYOetfFwlGmo%Z6#Q}tt0*;5kE~jrE<257 zRlsxJvT9NB-Yu&J3$m;l;E`27dDP94wz#@UbPEh5uyrI>@|<==R?tr=x5x^5!R2p} z6~xi%5m~X$<(61;7&9%g;w0h_gv5%i8|9W*ae8My5-VUzhmu$~xFyz&vBb(BSVv+F z9(E+wcTihgB|Tpz5X@+Nm_RV)Htr`_A!duKtnzV7t3|;DTUx~`^9L-g{@58^F_nJU z>xwB$oObW05pcY>_JKjYGh8WvNfYE2xK(wz#syO(sKca#ftNC+cV^wLfU%Hu3tgp* zpv%F7P=;its@*K5%Wekup5JKtjixKUs2SmAAEn41c=$_!+3V7mQL>}!8>GpSWu!)( zD6>gbqoCszhVBBNwa!Z8`ey5j@DBM6Va{Tz_0l)u2><(6zA=r06)`qe_xQ58>yVik zW4DrF0jCu9st0FDI6zpBwDo2#*OM;BgcYyl{QW)ukJMsG5jBaIz5`H0o1y}ol5enl zCM-mFprv!b*jWj&pQJ~11T`a4zp2r>>shZ}WyDG9SBGjR(5>GIgQH!T^GXYGFfdoK8Gdv`*S{=VttqlhG?`lG)FT(WAK`yh!AH* zIiV~DxlOt)v6d@^#crs_5o%|c_n`k!nY=N$D)oX(QPAK)Son$eam9mo#TT&OQv~5L zKr`aS;4z}J3%FCv!i)Il%=9elg@FuAj4TnTZ;O6DRfY8F5dqayurTRsQ@uh_4*^RG z9|;thvZU0aVhoo?o`pq2^B=7xm{sMy_JUFtaLP4}Dmz_v-y|TibcIIgY1zYqM4@!S zdnwm;e^I6656IopF(rPX_?i$`SWT=S1XQ$?O~ULNWzQjL4fuk@E9Bl{t+@uj7=eYu z$VI~YDeUc4t&1jI!dzC?LS~4hmUR>1Z28`-O@ts-Ft*M^4hQ9b7p6)@2TUG{V0*&4 z)W8Adj;ht}h9z|baFRevPE~0I9;636PEnYv&E2h~$wSLX%@<^d8ctYIKe$@5_<=9o zs*0!Kvdkz;aXB^d25ODe+szr6%^@mFMt#A@R8duv<^!`t$TnV7<+uaJUGTz1hKCtc zfVs?i+=sQu8ifPJT`i2dgz>^scP(suLJt*p5oub5%JD2-{XFd@TUvds4#Rxrud{#R zSwDA}->_}6hog97OwWM3%~jcOl1w|L8&_2b2)`F+iZj+g97SsZt1-qY!vHY!tuZrB zo6337lT+=of}c&I7gl@PkO0RiijCkF4MPIhc9~x))C9%P(Tc; zW3~M{8`Q+Re{A6&9py@jeyFM9P(6^mFzR<6ESHyCq71$7-u5sK>l}kX97%9vED1J6 ztA;2DkP!}B6l{;(7#0Nq_e&H6oZ2l4w#N+Hq98Kh?HHjb2>9u@DA*JMwMD`9B)wrl zuc<%IB?{KHB;ELlx~v3Q6s)UBqu!z*iW}rh6htk9K8`4eS_Wy}qF`VSQ)Q_!;AyGvJ5%R}CQ z!3z!F$uhGdxv%n#No5>$y3u30p*mWZx+E6e3iBPNnhd|7gb^0z#l#9T3)HP#q*o-J zqQDh4qZQU`-zr=yLt#N%$!p;0*8^tZP!hq?TV)X)>X5onQD1?sawa3aAo~ij#X65% zWh8M}{r_k&Q@Q;Bfl1;?#B?2XoC6L+Tnfs`gPB)_pQCBGSh+4J-~*4~)wrws?m_@P@=MJd*Bc zd&f7Xj{b9qqDkA*%tI?QwHJ&1`!tl#!&1#_P%;7^tW9c@SR@x}*dfZ;mQC7ViS2@S z#K#Zf=k9C~6i4PiVWy<2ru`~9qjAQdp4s~xRPab019#v&t#<}f^qqZ}BB*<{`-w*e z>T2pobQza+%F5f+FJmflLkChfA>%Lhf$jU?f6hxsU$$d&JWO>mNa(tSCNqgH#E^hp z6&eBze1{*8bS7n8h*o3Y%voF8p5egsMp7Wn_)7|Wxt>A1X#T)VIaaQ>)*}vip81$0 z=iN(^;EU*R?(xPTWbIvl!1$EfyJTeTSH)a(9m$@T;uNchpex}Ryo2`7o=3PlD?{sF zjH47Ee*&*Iu>o-Iu0(1Y`WYXaQ3E8~whxPAPalzbFRbHqjy>va+?ITn*APrc6sjLM zbQ0nv-%xNxvQ^U+7RV@G63(c=z};OT<_6|y(5M6f7Wa)gyAW^v1Jk%dz$C3QZ`Y2o zP|c*b561|jpEk7Am~PkIibH_)I0j!W*KPmpFzuW_XPdQb?Z*z*`%HE!nX=B}3~m~c zn*NiD_yOkI5kq6_MhvlCz%&d!vTuV19-V9Z`9K9DABn3gA%8Ol)QNfe$1v4i<9b5j zdXT4@_q?Wt*jP;Wx22NPn7Bd*^_F z5u2qyYb_H~HTG4EN%Z@w#^$wpJ-|y^L8p`FO%X@zJG2)+$?4R7?Um2Vbo{Y36w^Bj z?`j#6gz%1n>iyT{qUdUQ%o_#5DPW7D)UHqjL_Uonw^eUVCVZ=jy)ng%Fvs1>4f73! z{GBHrnHLey3AoCS3*P@|ZK~Y_YDL~_mn$a8tkbKlEb!Ur6UJX7^C3qwpgc6^cw|9sJaZU{E;f)UD8m zK4q;28yz?fGlhC5#yrLc-+u49c12wGZmt`z_bzGqe8ABddkCGWjc164Jqf;H%#^Ad zzW{BpUEhEv=ewbAp3OM!7-cZfvH#B*YliVhFTj4+OLYCZ7jb_-mOaMzxefe;qRp<2 z3CvJL3E)Tk6(*|}E>iaqJkevAOWaXk8M?5!Ff|9Pf}<~7X3 zERj<2Li77xL#;8!2gTsWE|8TD)uX)xs*SNgH9}*QKsA)Vc?ncQ6`i+0g{m>nI0BWc z3S9!#P!;JdP`SFz7N|yIG|nYZjnu4^{d<2Oi7Y4>!PW^m@dpp^V+@EDhBa-SX5X5zA9Qh{!u!@+D7g zgyku0Imcjm3I&hKyydCk?3|FNKmyl!9_m$gk20kl>8mFm{iMaIp*9ScIE95RN1EE} zZfR=o%$BACbEGM}Lutv<6n@g)XG&8;U4JiWYN*l07N~|A(C$M%)L!;t1K7Zi2gdJw zOYuiGciiHnx>T_qr1C;V?U(J695q>cgPg#At%|;kTwqFhPSjus{u>?#$ zjycpVf`Q2A0xy&z&&xh37Y4o-V%mo{W~c`5_SOvCH|6%{4CRAwkIz6LT0ah3p1GLAaou5c*KzG}5su@JtCVloAl7lU<2=MVx8qxkjj;Fe+N|TDj19dp+Vb#o znj*pt8YFWZr9o5SO5H?gSL-KHl1`^mycr-fDMgH_lp06r{XLk>MST$pyAE&&DoYHoudNMEeHlQH&mwvV#(}Rs*Q27_;yi|4Ve<6+^ zZT2Mh?u99L=-ciBRDIyUS%t7QkDj3^AN1LO9 zC*u0^i3-N{)pkb(&t7?*MO5(Yc9q(h2a@RXcIE|P9cSJqz8wRsBm9ZU%7buJFut>{ zs9=mNmpdw$W8Xo+Q`IbYP%!3hd#8hfu{zX?6%@?Hb9@luL%F(EqpSD$QTU&Og7@~c zAft~BB(qp4M?#m*=29ohv+aR6LBYVZ6BE1~_7^e1fSD1t{dDV*RVadj*HaZ@;Gp1z zGE8?&@Ooq#jhNu|r0@?gCU`xM7}^dBp6X1ygM#Op!7V7)&ebEP ziwl^^mBmJubEUC?dK|aom|&2LC4Sx;NL09Dg0apu-9f?FiB(TfFizrVCny-yLf6|4 z3dYW^a)N>Z*Rg|wH@Jg>H^xE1bCnYj91QG41e0RZ4hjZ6-x+F{(pm3yQV`p^5)ODqAVY6!e2%$850zm@dI1qq6?KlvCwcK$a zz-0RhIqYgy;W!Y$w*On=TKKQ1l5rq_^Q8c;KvBO%ubM(Bb{xp6_8~Y9WEJIl1@l;C z+zGRIqx6pi*;T5d`5gzcKw9=&m`9cr83c?2S$S!cjssb$OL4+Lj5sT)a!}uhV@HYK z_(JR$kW~~D88INsuJ(ul!MMAnK1nvqDrm%jU@h{otYE29zL`>W)}sGjo1%y9yI*yO z961PN)frAB27v@gg@-!e69)20bW}suoZ)RW9nKW)MXy$0k+dT^bQaYK2M)~j%9cbv z5l82XcfK(vAb25h_`?~okpA<(+m z&=q!xwR+SMu;F4YnVoIfSgR^~*2+Y{TbbVo%RTXaXtD8yqvysLtO@TiT2bD!*a5+N zrh8d$i~@q+x7cDUl=c^PilB7*7iNE=qxHX-rldmI1zxx)T98(Q7tFdJ!>qu8_7!%A z7nGu4s|4X^TxN^4WY-G22H+!c%sKxha9m zvZI1I*?OXawHE7&3g&v1x}$=*o~`bvU@muAS5z>!h+kDy@YzB!uAiIi9e8B8S5)x9 z1Wi|0#86Z)VcjosoW2N|FB}w17{|;(!JPOrub^NpPKuyl?dpjN=31sJD42`A<_-#e z;c{mabKB=_N7$RZ?FfeuA_SOkJ?j%wlp!F@3pa6j1qE|pWnMwS9J+&maVv3V&e=o& z9_K#fBa}U!tJ6+UFqbpQGYTgk;ilY1MFkV%0wR#}&ju+f_~#b>(P>sV^ZykV{C-a9 z>>_-G4!{*6!L0fjS1F2ny08}N73{hZ+ay6u&D$nXPylqSZIdAU91G|lyiut7*d|d@ zR+6@D62&$0k5QtqDO}%1iGmWz5juv$`CUG#joB0bQK-nY;D>{zLCQ;lcJZusL zMIJ}mMu~z#f6nQXQKF#4Ar{j{3C%QbqXZYG^vfv0#eLE?N^m=znhZr^Z}~P#984G` zHpWH?!hLL`1mWB+qXfxf)@_s^oLd+rNF%ngnmw|Nb=^h@Qi)@^r{HIYgf>x)+SZ0u zLR4~$4#kzXkKQ0MDfbgKf+@8Wsf3GOdLPNgH6qFkvZ>oH!3~@zGud0(?#em`6Ltx1 zbDP^P!EJU3&ET+-0)4_X!GT#gq7;EbnSDo0Xr`AMz@@qWo~`}RHo?WWRv<3MGhMcA zqM)Xp?If^G6t~bc8Sf-ZJ6Lf_idN>xi`G8QX641UP%{ClQjcudxD~xe2^eqKs6G(io`3_ zOSJwaR@zhxVYBa+oQn|ry;5r4f3Gc~k*O@O8=)-c`2K(>J_Nfp8Wfw|uDg8l=uKOQgrM8M|oeEcrYF6g&9 zG;AkvTBYpDJsS8Tf6n$X^qk~wdUszEX zrSYCdiy z7uT4-UldIpE3GH4b7j92)WRBfO2EM=`=zitmVzOubwzc&o&8dFwJc@75MGK?2O*ZI z6S%Wq3Ja}O$AsX4EJLAUWeTckWx>jRDJZ?>Y?sC(Jo@LF50y~8)_gj9JD!+XA?r$i zDX6tJ6~eCcm%`F$vTN{4n~acl_R9;(#J96wh-oJu=I_Dm?3Wia!r6`-kP;Q#oc+R4 zq;+M#aGaKjb$!n#g4)F<#tg<2d}Nf1v2gXv&V4B;epX#;QnnJ5&1-dUM?t~Fr09&r z)i=CzUk=6!Jqs$z6q$JWcBhxM^M>pc&VAtypYW-Ia%gGYw|`VQ^xtQV)S1jn2(B|l zYfe+pT=K^#`Q=dd@&O4rl3#w(J~u}-y>{}8Fe)(7v|Tmo+t5UWl6_fH!ZY{4?TBh3 zL}$B)yoQImsi*K9DV!r={i-FG6WlP(TV__^7Q^@-xEBYSPT=7J5A)F%GEAQ#NTsZTI$4WbUj69NS%{ zB~5|#QCE=-2gf%pz`xK;O|onBAvqM}WhXLqsWs)%Z8BD2d@jeSvc4pqCi`ydrr#wk zg!Y=OmQjk)pRr2jbSAfPRYT#TKerl$eVOb0o!tWX_8Qt}Ydt>dIrZ?r=tMF8mm@@g z5)XA#BwVikf-#8({lCK4W419P;_C)S4J}lBkD4yVQSK}o4z$WfM4L-?gl2F#P}3E8 z&Xwu6k4(J6A$)0SEnPOG>v?;M%i$f%`Q0;J@Wad3^tR+1gt?c$P?VP6>cx#9153Zk zHht5|J3S^@E_m!7Nns@nU7H!eVE8QWXOa7I`j}&^KgJA9b8a;YEYtqitMj{cL2Eo@u=$w&)bNB_y$lDhPt z4Q~A>Fk8XMVdD8hY%OKK^MyIuPA(DyzA@KZb(eeliP;~ImwqCS{u7wcf3kXzZv6*v zz15#woGB*t2cQ!-ss3QmbGp?ZV0u03PcF_?v-$(rnuqnDTy`XZ7=3YGcwB!)7s+)& z)+bWW^#mlDbo3w4R+_i|1L|`I9rd4F%TBn{8&&$zR)Au4j{@|ArX>Y+-3bx4y%F|a z7Szh@J8ZkK3~au_hfm-iQ^34@w|G4B@B_a@nrzO;2`q9|E+&{gWWoIY1$J zXl;x(3g|cU3MTK*2j(v@dWPL>8K{AdIq#4@Fw8i--|FKhWJA1ob3EG5Qdu75@(PL#53P-spX3TXJXXHnH|F}5$Kz-II~W9zC%4j}+wA_G&$qNfgGvAB zi;ElJzk^|i7uTtO1%Sbn*`^xnSgsWrbWp~zIpE|h5)Ycx@~O-`4(8f0-e@$|pC=mK z&&Yyc0V$ZkM&dVI+zD2+nmC9RjaG8wAhY_k2w{R71EPfHQ(WfCH_S(kJrNQVj-fV-T#V6l>k3y8YaZ zKk8#JmR4VBTTs_4re59A0b5SuU`h_HWmVf(Q|!+x0b$#jO#-+s5~@}v;Ku9HZxA!x zfW*JhAGLt+C;C;;o?}4gi{y&;7^0Yu<}gT`4mSrTz1xZJu#P`0Jc;KU>LeHT8CtpB zV`Eh!)P6(}@ng8u)ZQDx+)XjH{X}DpkMj#`&Uk%+-E*!XG}cMt5J@kMP2)*Frpd;N zf1KqK5!azb;5limFo#v@?xE8%*HEp3(6CTz%mm8~x5!7>m>`bE+W`Bh#v=H@KBpc~ z6`M#Nz^o*cXt>mhj*xr2sNEz|vit!y$7UwcleI)iN8NcESVXYuNpm9CcqYPXBNHcG zF!zn6Ca))ou9``wvNn406VdCW`(!^#>>#Oh4pM{qJP_(#kaW-x7V^&gzk}i8prUPJ zbs{8svHKA9M=VCM^&msF8k65q!E0!gljC4y0aUT3k#|@6M#Zmp!&%`Kg?ibqkW7p< zwbgJIY|$0}pfFZ3LbcsePv*l0ReSnol)jk!XZnNyGh#C>f8f2Q9hBU>1eF0eOWjZ5A6K4Az<-8ahau zrTul7R5BQcX|IJ@h?;1~SQ0Q-le)`(0^7m!)~K*^194@50@W^xn+`^}NG6ADl!x|{ zjJ(Bje;&b^mg5Yo(0gGAtWAex zc|Fk(Uj$gqZcxuuKUiYd`HK(Pv36b4gsi%^2lY7I^voEXF)A~A!*JBe;ujNl0h6|| zfoqcbeZ49moX3IOWziIS4mu{$KUl$H?dAPMRtg0#`PFq^cqDhkv5i}U^+M*az4ma* z=}3O;j#xoSyn|V+LOYhl)r|RjFuhS#`Fq~_#$0b1bL8guopV?M!xnDye(M{v-}-;5 zN?+&4Z!d;-9?a)>{;Fj7xJB|j z8GC$*xt*L_$Cv)%ENrQq3BFJ|8!|OjNxXjddRb^S$e}Q zWQD*S0>dZp6W-smc~t6&Nokf>>&BqVnn+JaCWohasz6*Vfxv3Je~ z=3Tljxy~PC=mgJX8-k}K09TzWa4qPZvnotMk*ss5i>gLl#~y2cKaigg2q*v)RLaY@ z0D$e~1#bZW+s_jeHP}TS2v%_6e<7}gG>v^@?lV`K?gs_qf)JZD1jgKA`5Lpv4i&W) zhYJ!SzT6l?0SvQkOJm84Lq^aj$_Jq_%Jh0d{3}=~Y)L zETpAhaHGkNZ}_rJybm~i)Z70D;w(_S^M#>CqSE3FcKR3QddrLY56t;^y!C~F7s0(? z0>i$W0w)QdcY9G|P^RIV2 zMXj@1!XlQp)3&Ad?l;ftg&-Stv5}qpqnoW(+*>!_t@1vf2T)gLTtA?0;rAyV z5m$_%C>3UpF<{T)&;J}ubK!LVLUVKA_(o@#-Jk#2gMh}uX?l>c8=l44t(fM1z{{|I zKZq4c(;31%ACGL8Ua40)@ z&%n3ipEJpGUtXB5x?dGdQ8Uf){ODlOL|Wsyn1f~$B(}=pTL)WA&;*QSQNetJ=p6g+!X)ahgtI6+u^c=i%%&u*qAJ}g zC@7h8sT@D-Vl>~OY)Pmog9~gYN92Y6OLSVs%84=nVNHNRNHi8Op_^p~9 zkpn*A>^=vTlyBb%`W(nL!X3myRvt#)0a*Fm?3__0_{5dzigY+zXFm61b2*|ZzE;9N zxiO++fZdwo6iB(e2QS55@$|{vqV^kwKBO_U220)<+kAlGO}5Cu)m>w7Rx~5DN7MuV zPW2$AP7L|q*8kA~EQKFSEqnSUzJ${@$4^X8mLD{^0Bp&$xx3zL!|Ka z$V|1hQDDFu;ORh##Wt5*`kha_@RTm!c6SP#Oli*QXEA%gtVVSuPUQvcghM*$yZ)VlxCHW_9nk&d8rb0Bzf~H)cpU2HYe>YI{}2NX7>L2DvwG zla)j@}vKD5R?sg|L%PwIp6!R?O)>E-EpHCpx8z4PBSn4G;Hae`Vq{Y{988>7@^{eVoT5{!b2twW_!jbOOsa)Pdp zF5GO`JK8d)T|b^c>^!;LEauKqo>5^T__@ZwpcgE3m34;N+mFprOM+abl81pGZ4O#H z2L7%z{B7W8i;#62_|Y&Tu-Uj}_90Ph+{knf)wnn_)`obK8KH6L4@h{BvmY4M(nDg~`>kD&C{2L<^ ze`l!aG4T_gw~3#el~dNw}~H}Gi=hQiN7o1 zdebTPzl@ubyX{~5iJWAKw22=ymilet2Nl{WKr27snU9&jt1iH0=HG<8&HSIFtwsxN z#J!FWu?>$zNn`$A7?-vLJmpfR&efUD#yxOz6Iwir1jSyu_&|1tn8G@YJcCk1y4*_i z++i)sikY!IYOlJ48C0M)BiKu=H%;7KaoA@yE2Ar}Ak0UPUaa2$^!aa=Ky zokzq4td)M zYw~y>`~appHEJpnnsU6ryc1o^oORLV6Zd@QC2~swy;^>WCmLH<4wlj_YFk1nOOB^M z>iL35lTK&bCTlAF`GR*;-4DY5t0pi>_F%Xn{xfD$qU_o(UQpq;Kx>sGD`xgmR>nnv zf8h#5SD0}`IZX7H>v|X)K z=Af zj)oj2dn;mSgp#C2>5lFG!LGifXq=LCkgQ|}<~n|zeFZ)+5}Ob!#09tI4Uvlo)@0ic z@lZLgI*l8@cH|;vu`YW)<~cs%7WxR0#1BjbM8ywGTE@=@Cd(QPHMV`h98rrj6>hlj zXz>MO9L>j_kb4O>7J!f30Z(xkoJSX;YNWFh31f|hl*g`|by^OqTbn2%4(y)&l}3x8 zX|*RCI<9p_gH|ld6h`Yhn1Tyf_aSU0`UT}szAHWS)n%^K<ztrXgKa!g7s1D#VG7Iu}EZXKcWvYje4`f z(d1)3+z^Q&>wZE8W@yszwgnME_l~YTOSi&v*a&T{OM!nI9#KaoyM8VUgrav-aB=4R~8qv$j452BHTl+a>2B~UoB*iW7{#q}GD8Cd{U$wnO z#^w?X!xvXbOEb&RB4;{wrpn+fs#b)B3mp9sbBExF_^{r1?IdJi~0}D{`h|D3v)D}jgbaK@CtGQ z{2Al~%pK$e_zrTy!hZ%i0doa84V7cx8W3Ph=+=P7Q$B*6&|c>basoBva$l6}(Nyjy z=0}jzP|otD0lhSM)E^M!7374K{0ws1B)qksPokzMgnlTsY)9IE2NPaP`MZu+6aKC~ zFSjQ=UKEAw79DD7x!tDdnT1$uUYWW80aqyf1|*I#c@Zybc z%z4L%GhqM98E+u=;PyM;m}|I8Md~gO6Z!~zIAM*z&bm*+k~ioC(xbZlM`ZnUu;TTS zG3R&Z(CPlh9Ov$|)8N7^1s|b0Y$_U`;tI|)F{y9N9ay&@tlSnLR9A=3;;h)lM01nk z*bZBOSQS2B7NAw+WDg?{;MtB52nE`4gB&9eOV3|cAQWt$>sWynrU~VK zU_<}`RUd55Shp2uow&5d-6Ca&5h%cI1Nx-=zbfJnZhlNa#I$1q;uyxU0ih8<&bJK+ zgP!9UfzU}{UU7;`b)YZJXO$6Zrk*HP*iw+bEZEx)#C8Yy*nw~y%_r;Q1J)G*myl0078b}HmKDhd;NAT~yf6f0sgG-eP!DBw351oO_w`>gt6+s7k-Q%t z!>)9E4dZ>eK3(Y;yzB95M#=y{(PSVg5WDr3ZhWQ3eIrkGReAHEi zj|)3A1p6pKDO}4X&Pz(s6Uy%$RZAHyT!G+jP5aUVu}b*TvA@Pp)xmF$L70~ezS%L2 z8oqR7&Ecpng(_nfH4{`0>ty(Dz+M?_&v}9&`GA!*Iq7=sbdZnOuBt)MmyQjdjWB)b zd5K6@4qtjkD<8aJYifozuGOY5Ju4x#D|r;$CN~lYQzU}>Ooei7u!Jl92Yb3`5V?c> zgJF%ea|ah2h0@B&bn40E4%%TkNV<#x+LeyAdj~)ZHqXLDye~j{(y`PpN5EmVCmnmy ziBRe;WVOW8O;0*D4=2AxtaMfhls??NmWh=3f_o%wILBqbF^_bLOqJvpj92ueZ|*e? zl`c{c1L20q8dMsw7nl^8bPrJ|s~F?;MCUSq!Odc>JopU<=WRAV9Wf@*ALv;<;7 z=iEs|d5AWrlZe#!xQhi5d`P-Gi6}Gfn~Z`Ec@}esGBd$911f4oeX%HPL0b0{?_!H7GKr`pasL!a zM5P4hWfGA|3s)P7B;c}>h(3u=S#5(2?{9*HpfW#}EHFLN+zUcfEe{N)bFq{rL>R-5 zj#~+%D-!C4HiD5uNZDKE2pmZ9Tk{r43R5V~0}$Bwi+wT8ka=|10Y-x7=sbRqd60Jr zY8ec|J>YJT7#QSVD`k{9yENqYDtPKlE$4$)NV=fEtS>a4sVUzGQc!g{yTcYnoU~!Q zK#&R97iq?MvFy4>nsHJvbyrdVYpq~Wd?|?{Xf67n8Ki%V`w6ONycZvC1kHfF)+AeD ztEwoKytDKYcP;IEvDt_Woce|^vR5l#RYdcR`3KPqMGjts#el#TWP~mAk=NKn(Tv!} z=!;pz@aP|%-iM_fG2e@$sFe22b6VAc?I$dxDcqTbaJOr=IZ7_4`=8cbpC56IFzr?Kz8NhmpD*IPa}5A&t-2WdC4#`DCAy9#xdxhkNO zzIzRGB{T0PC419FC;6Po;GesrtSE&{->hIQ-Lq#`1GCj1VDiy^%3|5^t^Jf&XqmV2 z0|c&tP=3l{otmxv6cjK?xAs$36|=GWQ&tI>`cF|1>!tscrIQuEodXQ7PK+ef&kZkv>?Eg z8@3h%0_o*wL4cpSwIGm39=8^RQQ@i$EeHhC>(PQp9_iMCFs(L^7DNKMn81=Oc(IA~ zNR*E!=KOthAS~-Fw+;kqN&VnDk>eI zU@JgXrJJ?_1Q<7<6d>@Myp94C{KpE=2DbwAgF#*7cNCzF*$NP50gfID5E)KC3J~Zg z)msAsz0mw%4G82>+EIau%Kls`5VxWp9S9^-+tGorc~Ccw5`@!>V-QLZU^3(q8RUky z60|W^f&f$Lu@bad*f?a`Ba^%-BvrM4VdRu5=3L3R{=06!Wx!qXs$1bhe);W7}g{ z{-E?NyQnq+yowrwIS7}24uxX{*diJ$5K6oy2oC>}59>~mAqjqBZ#!Seu1isdG}I&` z4!5onR`-a_3lN>wy_tHzWe75lC2NxHy`@z~jVZNig>VsDZQ6(}ePQ8o=?|GDu-7&x zilkc-Cm581|0vzR>mV~8Yjgf83Gv5GpetlD90R`^^)&)hoPIr7Rg!uav-4+>e7>zk zb<$_sqb^KUMBd^5a7*-$8=_BX9`Qt%4Q&5H&!H2f}f5A(2KH;_8WAIw;g?*UWU%KM4=gzyo%Yx1i+P>%IUez$c%pAtg?^zk>G%Ey zLqM-l@a8^)GyMH{58UH63k=KWjbHHtNEc#onLmMKAhi;TL3B1TgYvII zBIQC-v8I8P3o}(uH86iaFfMsfkURGNs{1d#Fza36SEczpoKep!LZ|Q@P^bD=zMwRl z++4^|(h3@)+P?ni*36GYCX5?LCb=CuDn&wiNzK<25ZKj}U1W91$MGyhoNV}7Ok->! z3An`r@UcLbP57IBE78ZKJFC;dqBBbdMrVm@kJ}Fa1YDkY*ixpKbHfGpT2m$pagXH*| zqK1UZT(dQnrwW#RPPMXHen)RC>U6U9;^Tdrhbxzxu^gq!i~oc{t4Zf_0mC%-%U)y5-;HVX4BPd?g0kP3UQ)y`u_|Nw#ISsBU^upS**okB&rCk%NUC zPe|6>n+VT{*7#$a`3CbQfT=2%8W{ELC0Lh!f{R>r>Q-2DAKohNoyJ+>lx2oiseI$) z5M6_x3zLYU8L)JL-8D1|i75Br?pWw9jB3d#nT(6@gwy80*QOhDHoY?%S7cBgvBe7` z;nWy@cZhstEkd>INjRqd&ZJhcRx-*W>to8QfPtmJVj=m{W{>)4995C`fM*v=>Lv+@ zzo{P&N8*K=)FS$9*NmE)AKRy!r?NdZZWZ?8`ezM{npoF9s2<`&V&>U(4vfJGwVO0B zDJt*)=9;kZjCvo714uYiS6xh)PEx?Ik0h0jf8j@q-or}@47qlyO#7Gqr|R1UJ) zSGm{$j=a!*QekeUvM57*W8!fB-EaK^gZ;Lj6zTqrJMM2U9q~tO7B<7A3*CNF%x~$? zl7?X9;{neWPE{f}U~d9suKfg>B^Umt)bu24Mq0tks|pok&8U)yx_WXI^$n+8dFhk> z{)Rz{{JXg`BXw;Upp5x@VTvwkz<4F=R;r5k!;NflQq8m)R*92`OP4paicS@_PeKia zP_7EQuTjFHFvy;Jy1{Fc2G|8VC=^zY1x!^verkY&ux&rAdLFP8!Pni8y9f+&DQrR< z=!Ik1YRpjGV;j$XU00yx=;Uu26%z6H#%=+61Y~yJ%khrc+^LhN+cn-EClivj9sIzQ zlIQUv=R!J2gQlm4FLgUiCeOxX=Y+ zYl_8u#sHRjjBWVA`2S?$Q%!#&E0h!-E<#(YY({)`SY=lzJTsdIs#3nvka`8TkB9zUN~X?K}1DrT>w}I^=>6LY1QcLp%r#9ElZHnnZzYl)t!v&E4=|0n!x4 zx*;qGnp1qLS2yB%DUaUNF4LA7)}kCBf=wof3UNzRLE17^A50_bc^g~Eo^9PSV;a}S zZ}|yPp)1f>Q%gfdr=i+NqQftRqm|FpJESHp-FWZ-dgIa_fFe_lF%2{@7}Gx}_Y&x? zUnuCaJB6N;n$!U`*X650<;d!_@s8#F!2uiJYMnR}+z+#H#6=_0?IllImd|8gfJU-n z$?WyxNxHWs)||V_Cy-erIZ^79g}lH}!5_L~|2qVjUIZa4kG7=)p?D+seNv0$2Jir4 z1J=4)wji)zz$eb}@s3$}B>RfA^pjKB5SX-VHLk>h51Eq!v2w&JHN%I+y3`Fo6gZj(_bi8d1@;R zI&XQ$(m}hVEAG|EX62975oVoJM^ZOfE(mP|0iW8llv*`#!qECCw|tkw7mTXdXA`O} zC<01s3f!fkJcQ<4F^_RaQv&F@qQxt8{X&)b)7(?^rDQ#$34N_8^&v6!_%%H2Z7G44 zYsw{zo8kjMDfj;L@BE}dJ(sA(qo+?`@=ha)69I3yLmGWX7+XA{x=WH+oFgpyZ6;&- zo+CIsq0lSO6^ct{_vqq^cr3u&=KLUlC-bFxi6dhSkEfpDaN8#v=y+IEbm+g((YPr| z78un=X-sq(LPUz^d){bq)WJerz|(vSZ=yNevKw)Od0jBP@8Snk?07$-Pi?$-hpxcz ze#aR1aPkHP<{-}l{Q?c)be=)8$GBZva0?JsFL{J0zv7m%`poMBr3UUF;5=Oq&^(?u z@I3D~{9K8@0{Uv}b8Woq9?!$H5HxT$e0temxpKUvy|Kc;oK@z2h*5pkC1(YtaV|jq zL%V(edHx?y7$x?h-htabUT}PV5r^48g}M=Ptc~ekHlsDewRJH@i=SSf($t)Fgz2nu zXYUJa9xw|))6>jG!MZdkmbw=18Fz{Cw-peS8a#F5)~GhcDiy|M9hIuH&-<-X!3F0% zMU|?voBUR(;E|5cn7*Ow6tGGK?-yU(DwTn>E|rSbLp>@LP3cQvqSoH;ZAr&#zOP_< z%T}p~LUw6Xa#S6ail=`*Dpj|i_^4DtISLf!?49_ZluI1@u3soerQ);OQK?`c0H<5VuK zDhQ!f^)N+C;HgzgprchOSh2M#o<~R1%Q8|wUNTuc>?@jn(pnXt)h?|H6Pwb}s`zsC zXjOY@wpO*-AhasnLaoVK6@K!`N2}tAek)b@F|zo7_suB?ia_>AC`)s?*V_ zu$FP07%V0e6ld#Hdo8R}eXfnh?6{1uv*ZGFi~jKK!2q-^lmbwJAQk zfJlMkKF%(TQ@YPCAXJob&Mv}q)O~gVp~4^QvkM3nqn)#B$Wq&97ZEAueRd7FT-G_e zMk?XT70gaz6ysuN0IRfdT>;>gdCsn(je8C+)IHRE4zD4I^nG{%4W_UA@CwRxR4pB$ zKPcBK_^w|l=kUU_DN}uT4dR^dvkRaywV$&K- zUmtIo9tQmP6~fWqXBWYEuA>XnYJ-zU7pMK4Tzg&i$+dYPPA=Xt&&fq58t3E!m$h~u zT|@8{`{)|#=d(ovzJalTEj)(m)~+j>oM%3}Muau?*@da4aQE4@*TQGl=h_H*rKk!y z3#-8pK~#7|`FAKMA{}@Aq}+Fravga7o$J8c&pLY|e&;*oQ+grwzU?d!BD>JpN)+xU zgsM&u&70!t=#xA0?RSLil2r(ra)8k+<>86&?;^;5@$b6lCnatQ0-&eVvOIpql#7?-arp<<#g^UA20la)S^P(gr}kBo{rITh2TQIS)D4~3fV zoC>TWH|Wl(z}$?rb1E=(W1XCeIcB6#IH>}Y-TFzYn1hJDJE>wyN4kQa!gGxLOT6$X7Txl~jTNkfJQh#x;Ndk+BOO z*^7X};ZGV#)?-*8E^^qEtP((srvqxW3-@R$)Y|94Tj~^wL?Q;>65>CP>Q9N-L#t=K zI;$anIF+C*f&7d=as<#uS#3#~dZP0Urm`v9h|YDaMb*|}j<}aQw#ncB9ZC)e#!V>( zlInBk5@~&hoZv08RzRr=3kdtDtI7^xf~w&YsYRP(B4XU8Sfonze8drwORx}(pw(4U zhn+K$cqCuRH)N3yzYCeXYO(~J+A@O+SknM%^AcexfKh$Wl$^fy}OBx=h zHHOc|JrhMqyi5DP$wh&~Vau>~IFz`;j37*N^f4OvQRTxAj$M-Gbw$r|7gyB)QZ&mU z7VBqO-a|AG{lHKO;OE-lyeiMbZV!~+L$G3N$KZ*^u-1$~5M>a`4COF0obFmJ!gEX= zUwFSuc(ZQyL%gpik~DM$9QsNrVx|9@U2cFJ5GM;rINKY+;Unh-4`Pp*{-aP+P$;6v zbA~DUK{&=r-1C#d^c3H+K&7)^>tl@MuyFQv*ur8L2J z+~nz$Q|29aZJtOId?_eI_=+^acf9T?FHcS~qhiWm3a|pN3!M9-M8S7FbXX??KDN*npCuwMrD9E`?ccS2-M4}*nrJO{;_bJo4I!F|JhduIzBtc&aI}1gk zAiv`|-fVv9b0-RN_AXh%<(MZ?@MQyix6%aPp<`iMj(4maaincd(*zHjN1EVw z#>diR9gsUsP?WDo6XY+MIhp$HSTF%G99B5+-u@nX2jH}DXAsQ+p)oW_#)1OX`k^}Twz*bf2}xTyI~$oG7N z7sGdsTCA$cY|i~IEUo2tBG(hKgM?pj;t)(hFR-G*PmP(CEC11<$ce1_I}R}3QS{K} zOpkczalw=p|`G)m|FeWt{_msIwxi*DcUS2pY@h?E!|_T1rN;Y ziWd1gk{L3Y5!-mYcq<~3+FWdr!ZBO{+}lY`wBo&E3fkDw(0eIK67-@xEM7v^S|$@s z72GJuKqgyP0G;pooG1mqBi+(aV9d8PtgiBjE9!8{4ipygkP;t@c2z+gGV^}hRU!j{s>q_+>Ku$ z>8Zc*3qH>RSv+bwax9*g(xuAC;(5nxylji-9c}|g7SBt`lAN-5-f@cHh)#g{k=Bem zhI4`9iqWLa%j9{-cB+NR^HS2zhnT#=y1Hzh7$2KwQ^LA5t%1`zqt z#lT5-Y#SjApy1(S0DW?`G@;e*Oxj;_TS}^rYLUauToOtv?~NgLuhbTfr;K0aw!D;=!@eT5#xXcdB;!W!uX}y49iw8Aw@VcTi$V({vVkwo3is*UP|Jo z`a!uDq4jWjf1~Wo7Cl?l;Pgl&xL^BDZQ;wWyi;4=v9o2A+QR9$YUMOtcNR9MxLk@N zmZbE^ZQ%;mJ8Oii-lYxEmJBX&Zp(K*@+#ZO%-ohuaBj;x7UG(7TgV1_f>K-F*Io)q zFtz0!jwoHBLYIL4(mr0YmnbUq6{>=^&Y0L7?oui-uTMEzC+A&x;K*$`D-*dbhYOtB z!dqN-f@nC|$QD`BOFxp^k=sH}wATayO1@;dWdlB9en=01>XiG6DG#4=Tez4_PB7)R zaB8#EFxtC&EwY*UV{NFVrKQr9o!e5<91(<9mdd8=P73ZVdb;M& zbNm>_@^cl0L(vILgN8a)07zYDohgK;R^n%6HV}sVH<3d+bD3BXHZ|#8k+Bir|IyGgqO+P1zbV)mNcrgS5a{9cTk$IIi#=;S?qQ#j0>Z8n>G80~dlQONeW@DiX)k%>|Q+Mxgh zvTw@2i;|4F4V$i##}a|A2Y)tCYCV4i_lz7E#6bvbJ?JY#jV8ez3UMRJ#DT#x38>4e zP6E2-V0vcK5@lFpV>=K6c5;^tAxIPFV(*%%TIsW^Ceh*d-3%1<}ALwU7@(f4w^W0tMrbRxJz zi~-3J3v7zpd+`@^I(V!MUo^xzEP*ct#{A-n%G!+cM?Gq;^GPLS`|pTpYr7cFn{gs; z$H4Dpb}A*-s9&40YYK36GkOnjhi|OU<&hvO>Tdm?DR}M2y%!dHG(+wf7g{i3Xt)|| zkaSSPFC2);_lY#S&gpYzAsO_(fMsy4I*S~e$D%j@*wlvzFF`n?p`-)!4JRN{Y}dJg z#Xi4Fm@ZDSe%}#t2S*pv;of~}Afn2k0yeSJhBu`dg}4%pmqq8v5Sozc*?29gCQ9o7 zdepPF6VGhd7XmR_ zJ?j8)lUY!}f;bWc#-lKrmA zP5C25Mp(XC$|CwE1_Uu8!wo>N&$ak&`n$(~@@h9PR4+tkid6G@Z?6pDN+mk(n3ODXu>GTStoWt5F=w7*Ml>cBkhLw5!}-!S$RII{r4<-XD2G}h0OBez%sMGe{{HV!@&+8P+IBZ3^H9Mb zvZ0k8$wg)?;ytX#Z$z6U`+`L$_zX@|kN4Jrf21T4s4p?eiIOS6zB^EXFAJonBx@pS(Cze92T6n^264JHs`%cHU!&(T9AShcmXz-Gz_6k_xb$1p&MEe(pW zGr63NL+ZgsO$Jj32yr3)EGPmYE)whZdF+M8erdVaf zFO+^%Mm!_sTVSDt_{#Jl2+^3Q-d7~q z%7KH|Sc!YoMa&}=NU4ie868_)+?#c)i=UrmKU>1?z#R6R0*#cEXr_gUOJDp#8ikd3 zk@&uVp)Cy5)ZK`Qt(L#Qj>4fWOzXiH_i$2CWyDU+(bo+(R3QyRSNOtoY#7@2`3O#{ zF(519D*I~D0oLGzi1A;c$(Bv{w+X6AW6QnqeS|`u$}LHBJY2 zmH}N*ggg6am<995R)CG;0G{QsL5hH9acF2P;8`lhdwf>fgtO9pY>?SZe4`Hv)2@S} zr;1w*JFD65<9s>+d24s3KR&f{xTte}K-?Xa%!Fiamr(HtWX-H3J5l4T_-$d$-4b<%^l?zJhfr7z6T_M)UAoyK&KCPba)pcbh40hGZAidbAmy?e;gt zGGy|E`!@1+{CBSYg;Ed5EEGx}WqM@M9_|*Q@F(#XnjNi31|Qs@f+=2895QuA3Q^ZX zPy92lzf<|BYy(3?T6Th^%m{p0HXfih)D1NZP8v@D%xVn4#{ih!ye3BklAykDieWeZ zC!;AH%D+QFhsPFZqwl;xO`$%S8EVE!7W!mn#1l$|^hwAF+))Ju0>vFN15MwpPiCfA zQ7y?b;=wO11;i%n=!MOiLX}^jJk}=*6VIzMw$xo_#?VIgtElb2TUtXc61ps*ci8U5 z`Xv3de4`oK_|!YI5SMLTGQTPG$=dZwb?cL+)VcLZVU-KacF7#5~Yxo^|}K3t$tEw72(wI+|>0#Z<V32HS)onluHSs5O|saTFUgviy?Og_IDCAV_WbgTNYt@M zO2@W{GiN4qjMVJn<0AlNI~{@X=T3mb7&Rt-TvC)`;zQD69iXc=_16oC+K)>{m#jY& zcH{fFX)Z~_bIs>>8AQV}A(M8!ub^ZZYoui0 zSq=j-7m1d+;W6>DjOX$S8Y0~F!jdH?XU z_8#PVFUl_Vhu-WljC1?RKwe#}Ue5s}8-#waZIi?gYryDTmd3S#Wa-Y>8WnB*BM0z} zD}THt(Zj9_ihi*Mj%?Pf7CNcD*prfj9zafe;VZCWd@sziab!(t>O|wE;*lC~pou;o z9>M~WP1I&yzBfu0USsNz6(nyW#y}jpO{g&EP{!ib^JgSj@#lHcHH8zf?KXz2_GR}R zW6qnqdwnj7@ME!~rN1lC4ni-R^3RB;P3Hz|0F13;3XQZ8OP8APG)5|W0)s-u0idcw zD`|k(EETUD2wA2OoGa>zw-AJEWuGQ2O+d)ju4g1c$Ua0l4(h7Hj?K4p=Dg8Cu?;rN z?UU{+8jXB*ohn`nj|EfQx(t_L=-4*mkdR#VdXJ49`@sN4jzxC0+zvYWf&!=LFd8Xz z_O{VU88IYJ2Xt(af;9p^_C$y{TWBH`a9$Ao`!w9C+d2Dr*gkSOn$uL z=VJU76l!lD%M+9y;ITx!1@jC1bY^{4ikzZGXNP1VpE9PD%aEP%T}QZLYmWPhN1lNA z1-Nkg$1{`+dETP43)Feb7+=dHN}Ehf+P^?N>FH;f?E*(uCu4JN6yNd^rv8j0Oov=< z@%Je+0Jmq)lliFx7h@vv0-8uoG+t0Bv>!_$^hi3>Z)IvztW5Q=4W&#Gn<-jN0~xGDJ`HG2(FCDT*?%aSdT76P@GFsVqczo@6Rq!wsXZp z7T>xQ%5zG)bt&X-KKbZUERGvTnPQd4x|OM*T-wuO&fqVUBbno_UnpCp!n4CG0pLco zLOTi-zUAq+M#X}sax^Mt5g%iW({&qT#26{(@9%oFtxz$Cx!k*nYu=7RwOPap)i);c z3a5D~RGZKbur2LPD^x5E$e7hUG>&mqatWXyEQ6Sij|{Lu{f>mO^%P4`4R$Lym9@# z%j;pCO6APAPKBxE@g}t@W0ueg_+tuuITX=MqOns>7{s~pXzfIFTla} zqjz-`EnIq6P^4*f#p%9{uAS?vf1zx>iydC)(Yx@Dm)ZruTRK)k?ZP+acGNC5Wj_*x zvvi!Ap2nK)3v!4DYF%82era7G%yG0Xc95M%>)NZm)vj;rFBHV^N^d_Sd$`uy2NRU= zGqU|~K7K~DC+2?IenwE}%d(#l?Noa{enxa@?zfu}@UpFLHzWEZ_kP@rYHoeI8PU!< z_v2`}hD9J*0vv8HV@HAL?PSDzKI3>9@fqn-Y4M$@ejJ&=Yq<~A*|v)jK))&9 zE=E8FbDr#C?5acDF2)!i7vrYb#fV=>GSf9yA>!p?1Tze&v;1f*ZW-s;BD}zTtRWlp z{2fbUKIVCVg`rDvI~lPSQ-&LzjN^{9v}?TGjNiDPDM zh_~*t^mV+W^yNCm@RjZ3oLh2l?JK4n$l+te9?AMD`xr5iO(z+F%cnwE;bYtzcl#K> zeUvtQW&g=bRooP-iWujuDGmgz`>iRC!saYnQv|qbUq@3M#X*IwDGp@o97S=IDqp%4 z#ZlPn?NSs+NnyDOvROT!Hd7CK`VQLMDsET-bI|i&O;&arn4a%+lMx|MECapV#auLFOb_DM7%Bo`+Y)~=m^*(frKLQ@=$mmf_L6Pwc36l2~) zQ~X>Q0X7d}6gsXw`m)Ky5A=wcmO93OX|JPJ6B|wvr$J$yWCuFsbaZ(fs%`4gDGRaR z0p&S*l^5~wPYxTk+LqvsPtFikbKG+_DB>C!W8fvZm*K3+|5GVz+qzu&(S1nkc_F-4`DO&TQV1R1r($AHEb0vZya7DP81ij1GY#XSLvmJp7DR!)nZYP6&OGhN zF0Be?Nh7C}txP08BS+u@PZfV$=PbG7!|Vlrl*^usxx;VrWI*kz-oXV#qg4sN;w=Y$ zu?55Pi(2E>qrIRb-FDQbi~#Y(A`c01$v;80P=BX_*_ssoNv`Ct+IDu)-Dp$CfQ5Mf zmZdO+%Xt1ySOMOy$0El=tPvBfm5icJ@Ocm`olE<12%~yGE;lT zu$fk(f99YzqjMy#SW0$V^BN8UTL1kJO}8+r2JzfXS!f9S(^oKms3<>$vPNDE+D(H{ z-jsgA@C3&cSydI!}6%ztMm1L`tP> zo5R9*_&`gy^v0p5O*r%eMYiteqUb=Gi$?_PizQ(CStHvFlz$gxX9n)Oep0TxMDbDO z-!ON0bg5Rp`X?pQYGzsrJ8mDyOcM5I@6I$M*d&DRG^F|42QpJvHo*3Q%;czyeIOV= z<#;$|s`r?8n~2+l+CSi(+|RR4+&!4_2z@};%w`v?z4FZ?L@@c6by(&#M!Pxd$y{S!F_1YRwBjESEN50Qpiu#_R3Un z65^~=hnJENC-lXyor5?}q}Fk~pm_eO0$!h^YbE0TpOl@0h)2#a;`q#|R01a<&h|tM z!C8nCaPsos3)f%J;cp zk>}Qvx8pkr{NI%7t~Kis=Tw)x&827;bFTE6i#5n7+0hmz#Q(OM*(LbgnZ_jxur!_9?&Bn2U>SvA`2XsNv%50@UJ>`taV)gSf9xsP&bf8*i4M= zw3$UUmZIBfz} zqNN1GWREAK-@YTu{*8UG?CX90h#!(wF+{PC1M z{;L)mx!qShO5R*j(6OA46J)#-h`j`rh5l&t3mIRou8L95QC$`UwN>HGk1H~g$J0?S z)l8xvNl*4=B=-%`(V5hiMl~I9`3!dfdH$%%fEFe@-B4CQUh)I0&c(K3M`VQpnf}kt zFejM`+;zKq(B|ce3}Vi0T96IOpK==lbJZwckfd5ozRDp3HrDukJ43C5(mvFO5DPyk z79IaCUTFk}`0r34wObDEiHsMn)Y`V#pkVjQ*Jz-@Z*#7{`0e?~U!T+G)Hz#bkLmzl zQUBOa)ZMXVD($C6j~6FvFEvA8&zB<4R!-MAM_0R`Kwx*@L8aVo{3##@@FU#FW@+Y; z=r_3FRcbjaN3p`C?y&_3Z0CK&O7RP7p0u(8`k4`nSZC#~NPfs=+G^~~kL96akIoaA z%^4uZuaQI?1tbEyCDzIcbzKyME7HE}-;dwX*|KDrO}b?NbDx2{DgO?|TJpbr(+|RR zj|jl5Qje_o5mT<8J?{VbmG6Xo&;KWkSeKEg*y~VK>?IUgYe~WY&N`u^*o%~$cd?i7 zS=%o55~y>zioFQ>JM%8~A~d5Aex2o$%=*5-p8O=8PNiRm(hX5x$F@(V$5l$3 zEE;|p{I4$k;yCZhFD>9z_@%iiQTV0hEW7ZFQ^|t3DEuOTa@uuYL|h)bstb#A)qL%h z-9=w#^`pk?r3}T>FN&;DQ}oETtG@VB?JD}>ThG3WzJvihcGVZZ+Id=rki)cerL)86 z=0s*0z(gj*IC4a%pJ8{Sj*7E1tPlCVcEa3+-=+NDRbS-+a=Fq4euV&e`L8=mP%x#T zFYBM%aZdQkR=4AvV5BQ;$2rHTT#j=BV3xWa=kHiHggxg3%$)V|oD)=Y&FwrVfT_m> z5l`yCdi>`?>CD@IP8`n@K^73`y4>d_*&VX40LEI`?(=u7Oxx{0e<>+SS)kp#<3~r8 z9O!2p9`r+z3!T4E=Juh#Q)RwU13|p)OS9NWt)gbZ3sd?cYS>Bm_#V#Jq0Wq8O z4suNj3|9vtJ82}r@rCD{i?)uh&C7T@-!>uq=3H432wRMWwW3LIaB`=NV!6%vT1yIU z<3n>gHB0==zKv+l%l+q}WM`<#oc7!=U6~7G-FOiGS5PkDG=IlMF^_PX6QdU9uASzD zq^-{Pg=mB{d933+W~DAsC#U(AH*%VvVK~iifwZNNJ?3V_aF_G_Yu)a0@=@R3a$)q& z%Uw=3DLvff+vP9U^v~@w=K_KG?K3BjaI1&UoJ-2=9{J2UZYg}`XBa;7 zL#al}O(VjzRCz*TZJ)VVwS~`|oIG}pEl)}{r{j`cWF+46859xmhS|cX*Xp^3*lHNl^}$hyfR2^tA@VK%7#&|nADui zT{KXvJ_qhQDASzD{f67-cAvas!5y>|xbL7WMSIvG2g{n$Va-N-pNn;Y8ClNX`xB)- zof<@Ee^Tbn&5e>=lsUI}5cU$Ahp6vKy$dQKn>U{D+=PGMxmf_(tH*(@yh4KOQY8!< zYg3$8NJ#9Y%!M=zkv+anS|Rb5vr^-!@d06*`@S0OeF&Uj=M|E&9LwS_E|nA}!!^TB z*Eku5c(V0aEvg+ClNu<->1?+lvnsp!>$!@&#Sali0Qn-cf5^qnCb7h zDKjw=3rXqo>m(L#%1$iguewGMp$CEjE83&39e9tDuFOKdXO>LkUeBe_tN zx|0j_9tR)beNE+z?EFHmm!olmXWs0p(hB7aKGC~+hRl6yZ(}M2tOQhfrzqR`g`CSr zQ4)%>AJRnN=y?Hi8+rsCqRjplapJU8BB}@phbd3Dy_1s;$^G}K zq{t~Tk;BLf+^d(8BT}vXlogG`VOx^+b5|BX^>zu$6OFAvDf5$3qojlVGw>8kJQ}ok z51W!o-{hM%`R(aW0ef8rE4gXjFb-s&tJp|WWh|R9%{~q`WUtqZL@vt?Pk-W}w#P8V zm`!jfQ$08^rc88W6-pzG-R+7`Rkj{`$?>h^;hkb?DPRacEibWz0e!$;(kACd9h!)u z^ud{9iJN@H>~_VO)3|7CciwgmV6#t|g)}!HlZW?Bvdn&z2UXM;D#KF5&igt7e#&|A z?FgX3lc9i)+v3S}MJ*pKo?K~(hQX8T2yMd#Pp&oM5o4yl{$+Iaod2(?jn@t{!TfR*mQ2MrbFq-2SJ-IsewT&K7#yCa~mZh{~^ke}f zUq%lovmT=-*K(l-apl{6Hr9UN3dlJw^`XfF#`>^$zZTZg2m_Cm7V4CWkd&a4~s(d;1 z0fa!lwP_Z)XC%y%d&ae^C->~EcjTUZ=K!(|AoSLWM!sB8p6?b!P{LHkxQ}Bhmq)*Q zY-Lam($_oG5fG+wdDv6&GL=iBxnN5fjaGU-mU2n1YuJ`DupeE=Kufu>2h!e!=m&BY zz!voX$Y%?)`^PwjGP-}%ejCcbD{TE3%0wP)$51YW-@OfGB26CC`QM=&L%A65Dhy>H z9C{z~z%N;KTE|c>!o%K%GCt%wb?}|f$Ekya(qLSP8`1&09$X66B>MVs*^ORHV2z={A$G z8e<+ac{8=m>nfDM3VuVM?PohNAljc6hRC;;Xxawf9TYOy8m7(R!) z-mU-~7n|uPjAcBrY)&M*0>V}9VlbtXCrk0jbi#o z?m$sD-km$Z@60E6pg3Lg_!QmnQ_)>PtFlZ156?yhggrWk~mx7+cAlNJaYM9f?{JTalg_Cu*fdd zmUan7l^4xv1b8!7&+)M{=_s%u$A{j51kfDFmApcNPKOo+Yi$ zBA~DtCyju8tL~~hGKzC##RWM)te8fC*{%6bBjEJ6lSY6A8ttSJu)CT;Nt8yw22Tnf zGK~OZJ8L+lG2Ceco8mMA{N!WEkJ7)k@H%M(Q zrV&si+s4Z27UGdcfO+n&Jm%^Du{1nYYRbyBjnbup7FKOuWk`pz6Eyc-KPl%fQ*>a( zZ-_@?bkwT2`X|M;B*lHOF7F46M)JLJRormAAFQW1@IwWed&U#C_k)PU2Y#ry^UL;a zRQvdj_k&G!#(hPJx1jff9k}xC{b1{=4L?+X23(Kx1EAS1l(zFDD4we>G&0z8f#1%QH$Aw#QCfL92ktk^5kTRE2eM*SU!$!EJ_ zEcjI8oib{`eBn!n$Jn*l_H85OILX+6j!~GSe3{0ya_*y>V9)8kKs8RT!Yh2X*;FE! zJLJnnc!d^+whe6;Xg3DIR~6o-cpA~drFc^dPPLqP-P)%P!`3+ie4KHO;G=M1#CX@>-JC|w zBT}*BYw^t+V!8l@ zxRhOj_l~9R>=L|}Qlrd|mT`OJo^T1?J5G#Ps}LeaQE^8?tsjIegGMEI?^s5uOYryx z_{b>1<15lTp%T2(oCn|vy8>9`RIeJm0$Ai#?<=ZePO8D(h5%#1me;JssF z5xN4e1m^WF!Q;FWjY}nX95dHJC3qY|tKTJf1z^jM@*B9xnt}dYf_KJIg10HV1ds30 zWKNPDoF3^CJT3nCIwg4gaA*F32{|!WfgPvPS>-ChdnrtBI2ROd3s>O1<420hPDiLq z`kPDe-tma!0pu^la2MenimLGV1CMfp=$onT*|DqeUW%0iRpId^oQ#MnJhE7Tp0f&1 zudUo%TAScuN-Wjk@hj!_Y%6w!G0a(v!FkzM06DC;OIiR9yw2x$1gJkl&`4q3v}dSW|m9)^4#mJ zj3`MF`;Myj=jQ@Z%_ms7P@E=p=O8q+1_ZCTa;!B|THcZ9g`Mm|SXxoc8CiQg&5FuXCVD<; zd$fsil4zyIa;XoDe*&g+jUlj&+V1ycCF2adjb3}4jyqwsVz z|0RI-{2zXX`3^9xAC<2@Q%(wB^Ybp2uO{=aUl?Brn8SL1LQwYO zQ5VqCFNEu!1gMLjgrNqlz6UY3>ofhFpA;An4~t&oeScE!_to3@n*R=Eu0SfFzZ^;EUDiY++m}j{+P>x_1hPUz zq=GS);W6&$^QL5HR`VC0rkR?4 z#^aeH1Rvf4n(CfwWIqF~_S z*7FyfX2h65_ITyShT!6~;@ksZKuok)a52I09V(UsDMF5nl_}2l%BYo~fZ10NCTIUZ zV{hF12`b7t@1UX_zM|Nrb1Q>=f+Z0-z&EnLVsd^COaW(eN{9Eh0kgXD|-~ z;Z9-`uy(0(CmLgcbt*WImfA5Vcqnw$+2o(0E)svj7J^BkFq5@P#vFC!%Qg$NoOm&fodlnCtcjg?)5 z3dGvi>7P$zjo3$v&N60}0Ja7%SkFd`FW4J3KS@e`{GF&t7!pkpGt#xOVefj&3_XAV zM3_Z<52XXwju4_4TR5@0V{&V=Z!`0;7smvv=Q?$XqT+2yz*tb`v&0YZpu*n>c%N8n z)V&E7Dfq#Ze?No+Ri3QY%E2QL7&*i1dvlE`grLzc`L3Uo`z}$oJ{>=^)yN=RZ|Hk| zQsO4O@?x*d>xWejb^Y+v6-P(Ropq-l6aY_M_cUe{UxcAl;iI1B$<`(}<>OU&Nu?%r zGSJ|CKT9!Y1{*3`ei|T-odt z{%%~IhVEb&-;6}FT8}74o>^_Tut%mD?bo;hF*T{XiLI2u`+5UO?((ohC^rXKyQtl7 zVbpb8;~vUHP!4l3EwukefvxNQxO?pzYw+tY>oNbCXW0x!> z(Lv|vs*~XzOYCVF(xQgB(#_G47tUHPi0o$)@DeC*j*Ng3`pvk714x`JSVh1JDHZO|263+C*JwGVW0Ms-C$zBjRKPlH;p3`3_`{8lZ zcf!l~k8t=mg~?tV@}OwyBY|;~W@H8cL26?z`aAb$X4XkYR$7tPYqUI2hV)*weXm`5 zvJUWESWY0e@4kaM0%Utmd!1S*vo$itZq*o^QRek`bQj|q;(!NZW&M744|sUZv`q*J ziSg2S>5dlr#9-$!v8Md{!O&vzIfFwjb770#B3ENVXZ4%V1kz5WMSX3;|Cr^k$Jo_kP z%pPgCx`rGh+N1jlq&QOFgDD5S3y!?0{4Cy}01b3TPCoLS`N}x5bFp26x@$q#2Ib6R zCy?Av0)ivm6y5WaGNtg4GWBZJPaa|`sK`)}aGY9gDdx)|+lKW?m`!!Pvrdkw8s%v0 z_`+28UuTi(+=Zuy$H2b<2u&ICH1VY>6Hhj{X)M8c?r{rjyH9>oH|0zSJ*vBFL<@CL z0ENET?!0pTW5ZU(O}Skp(`u5l2Jr$ICN;_Y4l#-@b*D|`+{^o54RiuRMvGWV|7KQQwwt<^3>mm@eBvV>dKR6$#B4owV=q&#@436Lo3iC5GN@Yng>W>AxaSv2 z4nc}Am$wJ<)A*)v`5>&r-Cv6|pk~})d@3wt$U=6OuX4xuA)Ksn6eQ1PAJ-M_k*w^} zDYzaU>xP!%C*j=Uct{JQFx_j8Y-YXGCl^SXDo=(gPovhVlIKnNrv~(eIWAJ-ng0on z4*3eRy>4d}pM+v6>fGSSX4h1gdo{*`$c}&icL+mzuWx7Ys-NG-m&tOEH-MjmljApp z9|Nw;_y!-2f6sTGX(M zA5C!5e1L2^x5aURIc+A2oGxv)8Jj;oqP1vE#Lr}OQ!=VEy_MDrbUYP&e5sg3>&4`C zBBTXmo(I*fz8^*_nH$IJS)*TebMc22bSy@2Ti_?jrHGsnFz=f3Tquh6j=MBo zjq#~V(S{Rw3(ApNj356(IF1(l?q`q`PbNEePtN10hGh~nu{1U(AQ6L}7{|1ji-li6 z`yTWt#$-&65whnU68g(+S}>nofe{`GnC_|#kv;#K9z;e;G_}9P-nTXO3Q*3~2wrZA zSsJf7WjC^Y3&zq)hx>u6a5BO%9CMQ`x1on7HT9OsGm#x~8oZuZ9QZ2w5HOQj(I2%z zK#CG&Nm3|}Nlt@f)Y!aZqJ72qT69ycMTdf=u5-AHV)3F3nr)`=xtIyqEZIw36`R|i zo}>-9`wu=0h9)%m-u3A^LXBX~T&bollg#ni2H0F0cR9q}&j=JNZHcYuV|j|i_qPT7 ztF5f_=&d1Ve^QHeE$dit~gn0EM087&x7 zuMdN?O~MR;z|K^Sr;wvX7&aO5Jxp&j@?Q4$&8iOS%PL55XYi@L+^Qdu{^+d{#bqb) za^l}?G(P)njI{CT^+Buhq3}^OmS>*hh~AFqc5ZhA+t4(%t)&x8{nI^$r(@}kxt)bG z`lWY;(8)LB5hp&;yRZ$pD3D)Y**~4ohZL1)-`P~E!#W?&)X-ESLmosa!-+rn?(|=sKpw2~^vWWhUHxbYa{A{@Qo@lT!S~6>RKz;WTzO5VX>pUu2-tlqNqTfS7AzYG?^me1_Pqj)6uQW; zMnhYhfm3h_{hc$q7+_VGOiX$cF|JbR+cm1u6=+k6)N%YGxS(4n3i$

EVhbkEZ4l z3Hsab&yQM{&5Vy8*Ixw<4rk{xER%~B+Ib7{&bSh1Sz!$JyZT%gHC>-irban9iBO^e6+K# z`MC1?(;~Ey?%{Uj2W9)p`@^*SUkKY6-uL{V9LIf6s)YMVx$g3e{zBOgkDI;|jw^pt zgb`8}0C`F7=lVAnle z`B>NI_;b3t${pVAa(eAQPkA2g$_HE3+mjEzE8}?byYTzACm-XwZcl#9>+$4&<7OID zz129t@g#6~Vb9y3{5zB`B*1qpgBS)O%qxr`{Q(7 zhk{4gnq1OClre7J&R>{2QlTz2y)4Bgmb}_saA12zYre9GaDA-`LC4>_(hZkPl&%&O zTyQ1P^_Cpd#QVT$!WUNi?pBnSu(Jq2=^rurpfu^&q$HWp#OQm6PdJa+mJP3veNxDO zgqO7T%hS5o8h$JZhnhp&g!Fjl?TSlx0hSZsU~%cTe>!(F)=>VBmf{_w^n=o(pW#Tm zXT1qF*>QVQAVyDOW2Ram%>`geAvpZKpc^omTbZ;!MS#g#nm zSs8N78r=Lmyf0!L8Fk+n9;;G)G**Ti{xe1H0a(mL)i3L?+(>aE22b|&VafD96qRh| z``&`@mv1Oalo4LOU*E9+fs|3r|g zcZZyo+JbdN^NzC7p{#v~m$9`E{hf^ruG@$FZ|4CjdFLB6-m!K*7F>dRT%#%DrZg(P zT!^Mn5_~B5!J;Pvf7#?h_Lu+Bh97fH^~_0!{Gh8;+H*Gqsk0x>h}fS zd1P;7JyOWSCr1%GAS>Ny&_GO&JV)N>QYU)-2wmN?>Kbw#@jcb>nCBTLund zuiY|`c(Klm-zHab4ng4q zD^#N)ug!3{ygR#&g^-rmj|@2%CG^P}HEvrnfJMBZf4(0A>hb){Z)r>886Rm&`e`+KM+Fw$Xe=4ao~L}tp1U;)K~qMwbWPn$Xek+sy<2&+3xoi&u5 zXH#d5kF=$}=C`yZQ0}xPEX$p?G@j>M+7c*t+R}K^owkIP|43T`Ym{;Qz1?X`V64xy zrSXi9v?Yx1v?a{rXWCNC>@jw}F*LQGQKS|qNcP_$L`ZU=Cv)2o%K;F|d`C1#R$@WK zJYNNb`gFuZM`lRC;VRsauQ6FKlkyK810%@;G@keF13bo;} ztjbp;ZsZicW{2tkoIqp0@Sfq%Mlf#(pL+-UhFaoW;lPo(zaw6VA(=$s#2D^y<0d#B ziE&vcUdabs@l3uzIswli;FKK@jZZisQ+%U?y$q^i4x$_f(HqWEy2G-V$5Bth|@s+cz16OJ@0=pOh;ZYpf-I%r+}F#b(8T*vZGN7)$p4HY>&| zVriQd1Ndf_A8gEixy5k9EO$KQ9j8b!r+P5#Qr=d@0GLhLR>c_lK^qlgC<1@1il7`L zVhn|SZ;HGKVOES`Ib^e9ObeK;_K)qeq+Kv(4KD`wA=oD0qI1p=wfy^%XKUZSQltsR z@2?`A-ev{Ha~`u|kGEN|DKaa#5apN^V<}K|%nDFaI%Wl4%CdoG1>Sk2+pHLqje53O zfzMp?*cC(YYL{Ix)SzbD6U+qA$(q;X6O@J~y}v=~%}Z_{F& z*oZl<$st4Q0TbWCmf+Pc&eC5a5MtY6u>JC7T40wtwxT_Sp5o=07TDD(-KIrQjv+CY zw%twXl-yxjj91T_{g@WuF9*teSgaK8m8DtmER|r6aYw=UovzO+hkfR;EXX|{#>G$< z=xto=iMMeP<6~TGij515S31T;$fILi;HmA{7F@Y`j0>(@yN!#G1~xA6nf=%n6b~HU_9$>TVU#OB!+Dw#B!TfT!7f zx-5Rhw-O9mfQP(11K17)@h6hK((QwGe&v@4Div~N)aeUEO+_oJtv)Z9xVL6-Q#`>%AT*4R4g}GhPSeZ4BD|&OZ zUD10SuIO(p&xNU~c-yXMQu=m9PxT<*uIN_^E6~1O(Rk^xjw^arg-}BhfGWC`rBtwWt@E%)crpbd)8wKcVq%)|5GNI zvSor3bGA%~3g{yfri#V3OqesA2$BiY<>W^utS72aB@>oFJVGWwL;8>j$c(#W0t}E# zCaiGkijWFW zsBfv@@g)_45>f%ba7zV&`rT4t%44#n!b~TniZ%ogHk%E4t0x3r-EW6r5Du9FLhg4Y32;5R(%>bIvo?q5ok5qsXc1s1< z4sWS2RrGmDg{jjwq{3vQxh)l9=WeO+?XxHqR!^sn3?JHhgpOpO2H{keZAv3JsK$y= z0!NKwXO$bj0+dI&UT_w@VY?>qCo>ilzGP$eD-CxEZ0EyQ|KhViO$Ek~ussw$a7S`rMsSB@%& zXc=T$6xH1zLxDw|@s>l#1Aodn>LSUoDwiU83$4@?P_2jf|e%Cm-yj-&FdVgHL&p5?r{PO3?ISKX{NK_Lg!glvf}S+Ly1V|StXD_Lg&FT zd?nMVm-9bIRQNRgJV6U-dCPK>v%^U3X85#jw@y3SHA zap*c~>~?e=I3qt)o%MvfovO~7p9gDXVISGebnE*HB?pG4v*ho)G#$*~*oqFe-xx>J z*&Fs~I-eg!+~=p12!~SDDNk1VMhJsV*eB(osEVHBEL9f^f#IWrQaJHXR+6===s9k3 zyzOz)LFsn052U?Tkg&{8-XIS|ipBU*vV`jBUrN<)sE+;}3nDL|FCQl+?)iQx&-Lfz za#5X>@3>2Gij(rC1Oytf)}Q2b44stkSdNB$Qf|tbNQ4vfFNA$A?t6ZrL>WDwJSFb^ zWEXd;jQ$-Ddr#d&^+{HdRYi{vk5rDPiQ0K^p8CDlxRyitwgkg{khtf9D507@#l@#` z-ED^Q#yI0RZN9ObxUj9QL@@0CQl?BWh5g_02)(0n*j=Qeu>VW2ajUTZJFXE@Vr!$P zIj$2z@ufUFBbrxKM_Cp2f5%;7G{XKbrEX*?$g)>!mrsmg|1*rR|E5G-{UtndWd9(X z1+N>o;2UKJ`Cp3i5r0yyyF8`8P|m|==%1g29riyI#o90&C+x5B9rh>1(W=ZzT&nL> zpqk@;hyA$>&k6f;+)vp59j7?Zr}??%K;mQ5A}np*u`90psbDv2^l|;20T_q<$yn_U z`;+zB3Hx*2d^#@Ov6NyPVgEg|JM90Rp~d9A$f_q}ODS^B4A{^Hqt#{F*J zR)BXDg;1_FDG^j54mO^;Y{Nnp15LN+`vmmBw{#Vjp7M-4E$w{OKA#9@d^Ci2N>=Ct zp2HdL0Y3u(!gHOsz&MrjmV11yizvpE;8@$4J(Kwev)b@vKVN|*p*P&x^kJ*mc*#De z{r44>-5e%W=xr>CJ&VSY*t2XbiSO$`$ns?O!;*ILpD2*dYwIhi%3IAA##nJ6J7g6O zXQ^n7EgEMz;+zLm~ynF`1tF#C0Acn}o3TIS&Nmi70c>WMe^FUN1-P0FnhFh|B=x2aG=P$fs<_brUW%|4 zj0W(IvkDAU>tE7P)O4c(l;jG^jRvqOhg-TDO?(^g{2%@5`k+i@tD3_1N_k@el;kqz zjRjBuAPe=*SO5j+uI3$J0dOu#zpwx}Un_4c08Vep8w-HrH4YX47lbE=umH{iile_T zF<)v0rmL_34n^r|92cEr>?emMno_QZCC!{VDFg<<4`ptqzyP?2+yVgxz{wda&;U4h z910EK9gB@N8~|sV94$D2cbs^zxY(cTQ_uKX^$iDb_#le^3(&s%_@K}--4s684phJa z@XeME+WqlOK5{oC#K%_I*lYlT$;dj;+)IVH!rnH?JU^U#&0}?{DrDmUkYdXYKBu!p z!UOoeh&oh|%{u>YNPv=ijWCle=M43)UppZP$8z2jTh0J6U2|K`7~Y0+W}4bDoG~ux z7|yv(DHYpr&P^$)VL0dJ8UP#4nK`7l;hdXckZd?-!$gMKk((g$j$)FVGD^-BPk5DS zJ7;w-zHH}Q4LoDxIalHRh4Gwg379<_&p|nJS>}hEc@xfo6ZiZ=*~T-TOuuf8Cp*4j z<2kdp&}}{A>*g_@Gdi|yZ4;AKPhJzJdk5vX+RmDy@tnI{V%v5G3mG4bwsWvl*v@;r zZRby>Qv zDJAfga3j!hbKXwccPt0RcG8wS`%c{ED$IFXaPLlznn)>?s&5o< ztFv=nQhJLOxsuM9b5M<|M(#7@;YxcS~Se9Pr&#~A_DlfeGu}9;)X$0ezL zVAQ;`+drY33o((~CuIaAG_)E%$?>#vOJl<7mQrbhu&X}UVt(e|DAy|Zu3spH6GTeUnO_+kFG{rJIm#_WFbqKigzq|I3tFgqBYBHZ9D(?LlFuv0#&gK~vKYZsQFR)S0n%YkXp-Iw~p_FU- z2&uBl(-ODuz6(WPQ|>$#e%)vvuh_No_Z6M78>bR;zN|y#z1CfUd8oc;!XNbyjwOLm zlWH6k9>XmbDLX-or{4RUc5EWajT~k$h~dubsBN@|n-kfk&-mait{>e8Ga-A+`TMAG z((}Ay|8oU&6`3y1o(N8kS?|Lf{IM)&6?9Zxx)k#9#iU!omfIp6U}(#>5VN%@ue z{JqE!u&ru7uSQYiC+Q*LTa??duuCpz#VF47c16kT( zedo$VgLqht?I=+|`)x;uLaWo1FC8ifM`HqTG2vr>P>vvuyMCc;9g0s5Uj)|doyL@p z4uy9&kf?MhfamspbSTWl&Kc88)oZK6XWRE*jR0UBs($Y)pv9g?huYj?9qJqRcx|6_ z=}?V2J#QVV_PnhLPGm3A{JpdLU1`bdM`RV~?~AJ* zIusjC*IS2TtLe3MsF?N8p+1*IJAXZYwkm;UAkd1?yA4W$yqRe08qSNXcFl?fZ2e;s z7HhCbQB@P1Uc>{=Jy4!JcDtPJsx9`A_aR&Pmz+M1RCFpYACYOc5wG_ZhMIzX!SNFX zy{D&qN>0kag^vW99J!AQoYm{d!@>+xHJ{fZu++#~(W;^5O#W#%L10X;xu+NON{W

F7h2P8 zT`v;4aO=AYUfZ^f$SRDD-Evgz=!z<+!T5pN|9t4{Xoj7ySAb%A<($O$sBD{JUDZDj zs;nPyR9(M7=)&=S-dcYDz482g{;R)!-|P9ge%v#-pB{4_FZa0ye8NYv#(x*ZS}LAe zuB4NO)~FvzC#~m*LF*~(czA7z_^PvMS`E}GUy%bDM!(RWh&aVFT!XpaqUwy@ZyDoj zVP@3oQkQP(+!3mDJmTomc!_#cX};sNWrDT#mgyTPMn1c|_ek|DeWaZ2E8+Jwt%i<0JbIl;`uj1Tg7`0pHf(`W$yGpMjGDzs>!8f2Pom)ENT-@&~sB9^u(Cu;R<>J z#?YAXawve{4$h2n66a9f6^{)?wxgcJr~KMMPk=X$%R2|SO7XB`p1?&k!1W_PbIg&y5?&GDH%-cip_u5(jSfXrv$0;*9=CVIrCuuCUh@_9t-WP&Pu zArwQ#YM;39&B}8v)k=RX(^Cx&2kJOfg(8&(atjGzp;NyV-FQB~1Q(gjkwy z&>NtwCqa7rh9w)B+TT#9rKyc9KQjpbZi3sy>*VRrE31#T3T)3N=c4rg1W|yW%}EWYveYAemf+ zLLOfe83YCmbU(|oR42y=paLRAU62EA49v0T?H1i-+;@E<%N&LlFp#Nt_sa48$79tL!A-zP_^KVf0PHa-*+@!v&ZNiv5IOCVL{ zhNu}URFGt#{($JG%i3yHAtI*7vRAHV1|wvA*5KOwRliVF)^>W9Gx&frRVgglfE0ik z$hdNUz)W}nPU&#dSt2?VRi>fGspk@7+@Ut8*AgW69{nxD6YO3z+nHAtmN&XzJcHzF^HZgA$IN;em z^^<$Nw%iDdSVI483Yzv3Zy#eOaCi0h!5dBQO}Dhn#P?z=T=l?JY6NW8;V*@o^9E7o(LO1#ya@55!`9nKa4VYEmz$_cLIc`OF z6gRR&$;7=ss}Ud04z693v#%&flH(5LMo|4U-@$F6g;2u;S}%&~%$kkeTAPz@prfDQ5dg+1v(>Ty!Urv{=}PMA9aCufEX2<~rPj zEg|9VzR-d)A@i^=G`M75%mgE*f3VwReDqwQJY%Xq>S4PNfZZ0Ca+bT9eW3?!UCc+i zhh_JL76o#`jhBbOCw>yn^u#?sC|SSDhQZ3)_({3$(wzQA*$TAVmjIz(xG0DW%|d1zBW9p82$oAKfT3AJq+=O1m0TC}i|{@7Is0!uGxTMceJ=v<V+JmCEZ@0t zp+6V2U?gkx!Rq97xG|Z^)Tn_HRa0due5e_V6PJu9aZ`e0W7#LCw~x1y5S<@6n7)PP zi_4?X>G_f(pn2@9qZzv^nu(Ql%~H%G;-ueaRuOk|<&JvOM^ z8W^MszK5FP)_e?~Mzkr6AT3t{txaL)VtgM4GDG|&zw?`Oy|lEx;(YY4obz#iXYZ~3 zBR@H}n5#?_xPMr6W3C?##t5CG#`9j0oWaeiGl2DG{!pE( zYV%!^kRH?PXofFm=x}d@=7ZPf+E7&yHGtU0pj#hN33{}UW6S|f4_ckxm3 zQw|5~Lk)mUl2P#S$&Mx{2T;`_;kAS77HeL`G1%xNHJSqh%TAIS-HD&sX@W>KYIVT5 z2-5@9=u$K=HT2{e!bjpPL1Q=PN>66;Dh98_(Hs||ruy$3z1a2jM zhsbJ~j$vIh z(PGn{4WY@(sKg2;n_f|n#2nnec?oJRZwEiH=t)OH+BHwGMwTQEA4uA15?sxpS|Qfx zu`K(qxrcF(`o+$c-YVg(fB5$20z*mdw;thkZ!jmBZAn?_obJW@19NW1;v+&L^_^Ny( znHjVis>x0Ff}- zp-N^+zGC(yZOsooeJ;Ha19NPz7|6$ZVb;ipT4?yyHl-<5!>=|adL}Eva1--pQFxr= z4Wa~4uNC}i7w#>lg7~XlEs~)Rl<*=Str0ys z=390-OY+6vWk-Z*fher4N!ZSfT35B)MOHDaQ&Ff=cq*RU;6Zijx0RH^>|9J#> zd5{>sy#o0zG2!-dTmoyAbFiSN&Lz&pWsUf$U26QXwA9eMTgnQn{|qu@VyCYF#Oc+QX`P>qKUB@Y9!huX0!;e|)a z*?EF+IB;9xt^>p*1iB&0a2ic3GafBJj07N^}pW5-kNSRr)xvRHp0|f{F0z>aJ1pW4UPs*4u?j z)Tw!m=kWE(UZ?>e(68-;8l{E{mrkewL!%rIyuf&kG%)m*)M1 zOkciz0au9yp0^8XAXM^ed!PWTH0SX^ZF+m4%()j{CW^vO#^rwshTTs?g=)9^X=FKe zj{6Cmw2u2}NK|b16A`7#*X4d9bd!DfoV;AO8D8LB$j$n?+)v<4-Cl(00Z;Ni>PDV% z={KUxd#AttTx|8X^JxH!R|Xb4pD?*gHUT=HF!!w;=M$I4@jR(I${9kftKXd49y?mO zwl9uGXv3dJ}KF-VkrAz6vmi|rry>R zsjzgowSf5nr|Z`GRKGyo(5rL0kOZr!8{a_E_%X0Oz$$!KV7+OASIAbVIGw4eHVA{x zHmLqgsQELzsrw=Tu#3FzC!TPDEqQx|puhkngIy@U^HDh!VkX$+Vax^j>0P;-NES`% zt&Hg^()@H+-Y-q2;}%cBl`#An!I2Xk7SX?@F?(# zGz~k=mAhwF;65ggPVFok!&KpJkVSLp*q$k?8Rw48&b$&U)#(%mzu^cma%(B;uE_Fah;Z&w}-)^-P=b8{Sbz|4wJ)XffqRf6Kv zN{NWSDj@>5$f3FI(!T7S0Q^dlxxdJ-Efc!(K}g!7<;vYyaZBy@3o135iYj+w+cb$h zM+E_z#NLeLotU;|5Bv1Ff`YK`>-lR_zAG-tvL?|SmLj`{!t2q*(+FW6nB@QKP|`53_GDB-LM#1bGo z*RO2=pQFhn7VP46zo$X4j`4f42JJC^&tCtuWBU$ro4&=^mT$anSz>yI+xt07 zq2_J*M(dulfnD%z?|><6IRAC_Fq%u!!D-fR%Qtat%CUTVHE+xJro;06W4V6|1Y~@e zzRAhk^bL}CG@HIBV^40=H|D6+WBSG%eVndi-7=knLT}6UgmRX`^gV|(rDOY^1~Rwp z8(XgQWBc|Ny^P;K`lVX^S#YbU44il|lLs_JuU8d<=vf1y| zw$gKli~Tj6%%!{oZ^lQ7QF=-&`DNsnBz0c*2mrwd^UFsrEK%B0*;7i!z3V2;?T3Tk zV2S_71=C>?QJ_~=09bXXtT8EpEYuusLPv?JK!of-FU$5@a#XU~9DkP^bibaOOibqk zI0@yc6re{M@=OL6O3*n#FE>Y_vz!Man#l7CEDga;gfSi;T?Bn_>3&9+D~_n}6U;aS zZP3e=h=f5oUp$#2r-U~*E3Nr1%>PVTzDA7ekXUyR5QfpTm0`}dJWp!WL%@~@g%s>h zq1L_x7YX#DEPFp)%9?Jffieq<+;_iVSjsOIgkj_?$D1W5o`@&s>eAL7^3XmJ2+UjD zGEg+;H3SDcaFripZgcc94HdVBPu$Sb}$ivy%^F{YSyxbg> z2q3M3?YbRX7Ewc*B{qJ^=YqgS$=O6ap+{9Zu^0`QAyEL%vpkz}wG#4@f3fVY)wNZXtR>8; zq4*-9Ijt}T;+jbWDxaDT1_f4*L)0I|7z>GMud&(`c7Il3xSsYCvY*9mYPojWxlIA^ zM^(0UK9!++sc%yR3|jYX%A9K`-e|t)IO6sIycD|bpJRH z5TFG#izj2GnA$X4V)02lufdpt7?CU*y0#TV?Ao)=k*5c1spupSN_hEtcv_L>?Jrp% z3q56Va+iV?ivoWY)_7>;R``vnP^pcB;^<|eJ3f>bBp*%Y(x2gU)=xY`D@w=gOSeDL zd09!~d5R+bME?~m&}^H-*|rxB`a37Lmk$MSW%d-tQbtjs3bWL=S`WE4Xc!qdW_WP9 z#P?GT8MIXS=;<4MZK=@RGYv8ZMN#MTL$mY}>NepAjQPV2N==6gbpDav&}sL3RF4?@ zKgHUb*_N51LD`lWj@XeU9MEm|;Hv7|AVVgDT#;-t4FGSipk3c&LA-6vJY$ z2~{jRyUJaCyEmt;ulJ?3ne~*Jj|;N2&B83XmV{}4Kf&gL_-SS=6o~5uV^`>4R=yX#I$Q< z)x6kQYW~@G>fZjV@&fj|#$vvHy*WuRKCb~jt^1sUzXIi{<#SSeg)7Dqsmg>PSTB3Z zA_Ndj1|vC9NBW2W%J+_lKAk;NW?a(rL{Tlu{R2g{5w1@ZIyrX}ikzamiK6dr1xF!# z{sG1{3#zL9NuK24dFg)~*{m;2DcUKozgKD%Y4&;8VA zTq9Wd`sqC!enx!jH3ERSTe^Qy)}EKn5+q^n*_>_NA?F7}>HIeL5Kd`sYtKuLP!Ix{ z43)I{w%4#1*W|G7(>-}ZQC0izLm|!k?Im~h`F$s3+1m5cNxorOdk!7ep2Jb-$HO{g zUt#UxCwS)yYY#bDF;;?KgGRQtwTEq<B9X<11GDnB_;4sk-D22v z)VGj^YvU+KE~RLb52xPBPoUyWRob!1hr4#oZSvu+&iyj^Y|b|Dyd2w7E{fbtt#RC&Ym*KAKGj2#`G-pBxq+ zZpC!s-8gWjg0W`FRuwx+Hb& z)_pt#CyoarqV3}Ws9H?diQ^%unsOWuK=Ha#GhVvVjMT8NCw5}uxy-<<3S4cExvKPJMuj}DFR02^(h}I z-7iCi_x<;w=+XW9(vh}*AH!J6IUYdE0;=O-)BAYXoZ>xa|Avo;q^j1}@c>Rv=Xgj8 z0o}(#QpCe`Jb?4`b3CL+L0EA-0Mc!a`*^^((DF9|sE+Y<=Xh8v=r|tmW9d2`mJ+=7 z@t_hm*Xgh_RmFWeXui+s5cD}6g5%Rc%i$akSpD-p9%4x2crcjnJ|30|xLwD?>N+su zcvvG!gXnlzE27JNJYZlP)W8*oq3jU{z=1w74KLZRdv$U-@;DtZ^YuO)fQ)B%L{V{$ z688Toq`aMh>BAv7=O{=@KKtMlS>}8=0FiG+*^a4ZJncRml51kd@qljROZcOgkt;eZ z9I|dF`-vxR)EMCir$f-^bO=tI4og{=`*e^zx=)8C6w){yICb-! z4xGAn9}gA>aXff7<8+{GNcZWGY#NEvfdg00=>U4w`*aA%$T%HRaxjHY2TY#;&Es^y z6tA{Vhb{SiI{Y{ZbUI`?&LWMQKz2y?)>aF1E23~!_EGY zpPb8f{ND4N*3emFPvkJXUGE_tsdp)6fsvUiRt=7?b1c4^h-mbs+DT|j&pHxDRI5(i ziH%)?Ckd{LQLmg6p=&YfRiZ1k9)57Hy>w!6bXc~}qq3#@YV~&BSCnFG*E*x_k;_#l zMg{WhR48lPCP_+o+^Xd`57e>)$vj>Q(dG>r0*q&YV!vAVp~G zs#_$MT%)^>JrmqEEuYM8j@5}0;oy$6|JY>O^$lFWFgFv`*N$sws$xPnSI-A;5KfG_cvpyq|c&<|x@| zq~uGw))CodlD`;mIib2HzeY%3z~RO8uP9fIca|fplpit_JvoGGjjtB#Eyfd6R80Ux)A`kECfL1B}_Cmk@%S|(Y3*kjW}3%x1$jnwagE2l$yo8|P8 z5BUy&PS*Vk&(OJgKJf^XY&YUjz~*kF=(}IRvE7ira2XEqje6S+*@%7&huL;zJOA7Q zXVV8EF*y4qLod?&SB_JK_?f5tW*h}37o}W$($@>6gd+QcalYzTela5VXfsry#hcE+ zeudDWJMD=5NxgLJl9wtNqv0?R;>tWOLU9M;2in?46d%i9$5E8h5sHwb2fQ!TBb&Nd$|vL@2QLXkMqy{Vs#Me`Du;d`M5^-H12Z_`1`7@cFl@7TV29O zS2^kwa6Zet6)v>Vd1(4dDUlq-J+b?=cMYl@r*b`!Wq#kkkn6sm`5P#7JFiw`WdZG0 zioSam94mGHg{{=_8|BvO*ot|yI=1rcioLvI*RY@c`g?Q!s;Ci#m#oxzAh%Yxhu&J< zkEzh6Svi7ghNlO|-7N^m`cpn=)gXVy289LY@%;oY;`xK|gzeLs3ksNiSPM#b-psSX zUHkg;ktj-#%>JR75f7$+yfsIh7V*qy4+a01?a~NsfgpHSi$(9K)=ev@Rp@QH$q;xe zJG|FHtz_7SkmFf?WnuL9L|oI_3hDe01uiF$iiJVK5*D0g(Q# z^pcmZiu2r?qNaWM^Aifqbx^$iZM$J> z-pfT8WZ9PTh-f81qRaHt(CWn$A+3jwnARW5O)Ic4H}`+)w5;vdZ&zG!!WCBq_q|YC#=E>qNY`ZhCv;Hb<@_{D4BqrDU#Q`F4wd0Gcy&gv# zCVSS~6NjlhSvfJ++Yz_v?TFhP&qGgPY@yQv^vahbj_XS@ikB6F<4FqA5r>shBrJNY z3_&f2Bd!{pG8}Q}bDT5Po96l2UN}@UojX>Jl`XFbR#vi1=dU9I=_O#tHj{@WAezh9 z_QRo+hF=Sp&l6diF~I*y1nP-Y)d?P`mTuth>XZhAZ7Hs`|bOzk-0EQ~Hk z9KU)p_tmO=9C2thvro1o&XOP;ah%Y4JaLo@?e@g6AnG`tI1b%7o;d88zJ04s!ij+Y`41zddn34g!I$Dd#B9sKZ`%2}|F~iAVSq1yx?Mhb2OF z*bC}aF5g}zUnsZr;-$07*q_s3ebf_t=@ITIUM*SlR_9M>DXZ$dkdN%8i|P$a)!1az zVGqu+gP~fh@g{ySj>-`4{Nn7Y>n~^Sx;vko^Onc--#Gi@@zM{*sl$Fa)nMXFcnq$? zPC7e@vupYpRPQqEq?#3b%7nEdYP#h;4OP}*W~#L3QQD|-8%P_0YC(vR4Y4NX}PyWP#rGJSy-O-%XK*m3fvBn zJ)k6)bPL%936U7}69k(%B9TjVISH!hV$dk}LDRnSyb&O=94&ckWm{X(jVs+d?d zHMKRWTN`GL%R`|)l(nF(A4sK}}1;bI5HD66|Rufa;k zU3n*#$938^gRq|`cC8-dbP280;i}nhDCEbezg#eif|-Q7bSbM{k&~RBHs||rc12E3 zu{m^W2PCR1a`K}~7po_lA(b!%dLuw>wt}>a7PmipV6MLw8M2)Kgpqe4IA;K1ZtQOq z8m=SyU6J#pxk67x&X+T!E@s0~mKnu`6R{PG!)T%+Cq<5ev5hb+5fl|U4;>Xbe+a2d zCPhWgvnZn?Cwo%lx#iWHB1d~m65Z9$c12FE8Rd9MbHoa1dP4B0xkmj>XWukIH$_EG z?#5G)lXEwubEqIEXLrgAI0Rq2qM%E=s3^&KII77>+I7|q4s0Pd&mzr=hZ!Ese8r_on4s&-IlU}}l%#8FX=$z?8k9sc*hFFju z)oUdGNRjF#{P>|rDb-->Q9VV$DGBfXe8MLYa`g5Jp@^?@ftOHIOH{Y89)*Ier5`;C z9F5TqTqs%#qN~_}+b@ZHg`WNTuOQ?WKC+;uK^_E8S-#Gir@zM{*r9o|uH7L^S zXi%VAzcnaudfh5iQ_}HPp;+5_-YOJ0b~%iVWVMaV=N{9MTpx#k5USTLuv2Kq7^|_LzK&{Joi?#g@X}Jw@ zn-PT@Urv@ysrx`&k9?e}+(^vXER&KdH@=(^Rz7VITvx6@mo|$sVb1zT_QGX{0!r z9Pj62=|R`M|32dm)(HnYmz4h~;P}q5rHcwUat$m=7mc=8$*$oToO>*})P6B;qbG_5 zmvkF6gA)c#fX_P4g$B)@XVlEsF$e@(M^Iv+^jtp7m4F=nyt4MRUG{#W26Py$5Th^<&fQ;Y_01jbSfWj9(XW<^F4p z0?oXMaN(@mrinfJ=Y4)Os{cX}TFvJ7NG9<1_Yg2ZKK{6 zO)Rf@Wfow0t;=5783A_Ru`mkxvUzfpuGE5_TGZ&YMH2(<{07U8^5;|<+T^xp2E*n| zkmJfLpeRbWIkWT5y{(yS(Kyyj&Re~$nHB_L&Fs1ahB32;n{pd7F}F-*urZUPmX9$L zv#o+1ad*YYKUZ1-X4}{STF=EOgWH(dT`$v{X zcd30`%7Fl0r7sLme*_&nu7VixePeeM!X2{Z6mJ1WdC7NF`^AC0`QE7Qf!Qov_p~vc ziDb?y=7IXm@3^=LaG{B9PQ5Zo0BJy$zZ%qt7qKm=t29vnPYeBA%0Sf%OV_?LAyv05 zgIlxbS@$^Zwjj5B5hK8}^}fP`7(&S}*evf4%t*Xn0<#~AKq@{OYJ`&r-dladMiKxx z$AHmQ!UxwVxtO@hJa8P>S73~DoUyo!K^`*^DCK9iu81wq=m&>$9%$&XCIT%zR!vEP zr|xB?dd3(Ut-1hY|FJBwQsEpO7)GFJ*ZR&_`ue~B=Rdf1VzyRhF_C}%(;R^sobUg@ zAj3R^h2q#?JQ9c(>H!u;Lu{N^ZW9~{rB zIRF63Fte@B)kXji%ra=MO8|g$jTK#J%~ilazRLo5Aeg|HZVHS9dJIo6aNgk|=m{1u zDo{P5n|`579Tmb18-a4OU%aJr3j5SAAUk30*?HgS3X(n36^t02J;uKnF~tA<540GH z7gXbdB7HF$tU6Q?u%l4h>m3!)w8x?P7tOg)D|o25UKrF44xdVJcRG`>uFAXZ@%%#~ zVd%Q+ixyNU16gEVS$X+cAEAq7OWG0^6Q~Dd)2+hvJ1EG;Eb}}8trSP{5_ml>nEZUE z!RdX0kblvDcP`;Z<#sC!Bf=GGXm3Gi@r}m=PzbYyOBfFdbpPe!*2eB*d?0an5lm%| zAg`#}4|K*|G;K|IBRCDY;rxNjW$qUy4=Zju+Kt!-tv5#OvW*i1}3uUtqqsSi_6+s27rzDtn{v1g3}em}*YX>S6*bVqZNRQYao6&f01g!A3LDpwziEXWo70^v>S^i)&Z$wg7U-9u?}Y1*J8)(xC0L*|t6&r07g_0CTmM}lAbF%( z6#Tt+nuWuUy~pj?QEFq#toeTDpNlhzE3Htasjc*GbOs{H7R#9K_d~twYJ!%%Pa>*} z=R*_T0OHW5E!T7;GEj}57)-IP{jn6&gfIE0yr1JgnyvKg$_gRO=!u544O^f z;5M2uJGE=%2HhlRq?{syF*w&wB#n+|AIO0wQDn3~2Kgd8^iCRGtX^ZPBD7qIzRw}2&%|M?F?xtQ3qbc4s;jm44{ z{Ux9WLjhOzHLs}tDS!6mfYx_u{oN$YYt7G#K?^xZCTJvFF!}o`=P-+O1GI<}m!_0@ z+%H)sLBZaPSYpcGU;;PZP$|Ay79P#@k;s648E;cXNoU-umIv}pvcn&RMYiU2CTL1O@HQ#?CBa~mDb z>|rvh3M9EWF6J|-KoIVn!8W|5e!=R*7=u}L9ue=Jz63l;lD8%7**MpZm((y4vXZ)5pL*ag2UlK=_S5F`!7b5S zRHd`(si{ref+*}z2A|-T1w9sfQPrrW&?NM;>o-`^1LF`REL@1y8c8(O77XQzpI92& zXPCsuj;KAcJhGUamaiv@)VnO-WTU#$-6nP$-{M*DP<}#;a5tf!7$^cPS@27~K8k!Z zn>8#nH26tovWJ#aR%kUd^Xfw=tklWt(;!%ZKxltAp+BFx9=rs-xE3v13_aZ zb{IV=g$RI>3mV+!+DwUxJqP;|12G#%40IPbi3hsuv3$R2!p9aoSRu)Ei_iqD!XhgT zn#dJXr3s)DS#LC=1hSiukHfZzEaxiMtx7#LT^+cxy@wxpA$HxSr+od~8R*1NF#Xvp z!O4QN1I>ku(+fu`u8M?_dAckIh&0q#tJDz#S~Y(<2=>tJqXH(_n$8pS)AjQUDA5vf zUf*bn1KO>Cr&WgyYYJ?dt8DlUbC`9_U!Yb)UpT+9q>OhNf7ps?h4LNU7G=(?9Pa(N z)knl&N_X=UO(HXXfkvKoe*zh3Yn?`%;ZNLNKc+)hHCaIOfT4f!s%B&vF>w2Ngl;rC zf%|YV3t;TS$+QSvZvk4VeEbgAFaoxw>$JKO&9X(3P%CK>0?Rt zuX5wP^zs`R$ynDoEn86#>_ZfVJr$<$!r~ouKhXn889&fD-IGvOi+B;zWsowXMqoiq z9>{DY77T$yle*EWPeT|0*ipK(vmn1P5NIT9%89U3VfnzqCfYotsm@C+eAO-gxiuu8 zE90&9X4xEh5YRI%b1}!%xulpVL7^K=|GshZj{?`z{V++u)W|FbHQX!}T>u z7scg}^N^!JLAc8nX`O%2 z;If$Q1o|alnbH68sNxz(eqUtkhBPe7XZYYqS2XAS^#pN2da!+}j0n1^IYROU1JQZ# zIx19zC4JDJX7X{5c@1R1LiU(2ssaV_G-MA0xNIWI@(+Wfda3mxx*2`%Prm7)+u_A& ztms-*Mvc&iQ(x>&@h?x0Hua_6s;8jpz&!A-%u4k(pVu=wYN<34F>m` z@qIBKX)P~9VZb-1aHWgEk8&)l5E{^|YFemZXm}3}Mv9I*lI!kwk_nds!$6vv^ovW; z>R2^d1Dj3GJvl9TnWe95YN|Kt$K-h{M=%(J4iBd&1q(DsyQFLHDHM!9kR?wFl_7m% ziJoq=7g*NpD?vulV?~3&$R2tpHM_~tWxTscL?m-KdsTZ&08yBj_`!b9CFyG_mF7!E zQn1a%soE*xaPHCv%^)JtBi7_FV`=Ubl3G1#kHv1?8;A8`$XOgp)yi0z$tW0EwW;gA ztAQWlaCz*h8qBWyF?BjXYEd(iP7jo~OI!MGGV?+VPOfhDFAnSwpKtxs6*%EhvF3cxD44SqAn#I5!oZR?E76c4Lb|Kv) z3=xCoH=5c+yv^zwb&MB;JkF|h>q7?5m*!5zL9{t%aAJ6WbCTR_xlYa&Wp1sUgw5f} zOqx<4t0HO@jtu8jFs&#_qKM6^(_*UhGcbmcDaOJ!O=o&n%2yjw$6C?UHllNJ)j@n>n5I=1wW+J*BZ3rh zFmaT-XwF+qgJ{lM))^2_jArd-unGugy6F>A!KzXs*$8OG^bV_yTq>@TgkE^(%<6tU z+P_l~VHk!d7Hri(Rn6UWv#W&uS@J?q_^~P2D2tG@O@gE2j~$y1k*D%9LNIlC)pFE} zLc5?xU}%!{g0Zf_tM=k#7+%W96U|DOf1z8GJm2WTs@|>WRPdswg+9)sgP-6nF@((NhG?&WXIxK=I0^g{jgIS7pc_ z(?LtylPzTu4GRB&qXJOoRG>an=q76D6_Bu!LPSeLWZwHfoW{B^`15wcXm zQ)cH1P}3ZD5yb8xzRpsjkS^Zxbk6dP?7kyp=8JT8Zb%;&U9wc3i_;j&#b2O`GTlgJ zk`c>W5s^=ohEb^sv6WY}dIjL@vWCm_Om@`a#g*ZusL%IBLjN@QJlcY2w7voVQOo4#{9`<7IY!*$d7Pl^|sQ&HH5R4i47LWS{#<-OST z26HYK%UXP~iCMe1Ez1wSpZ@sqskzLLA6WXqkrM7dXCBariL+8+>mfk4I_7zK{60+F#LUO47X4>w!93`P*Iacj7M3%(9 z$-ZpVq7}Y8XhYj5w85G`wv9O77e_PTjl@Gd(QM5hrS>CmceSEgG|!K1ld>Yk;ao1F zTT##B@43Xb;bu+r*GRIie-r`pv1Z>Fce>N<~*5fNI51);|~($xm4~G>)(+S8=#PTHQ2H zC`2rVI;DNpy5^=beVr{$K0Y4P-PQ>o7j62ucr?;YN$0#`P0`iYrKVQ(P)*7CzBp%u z_DKKcc%=V}i~`k;rdXf{ZgVo76{LBtXI{2mdP?>&Fz1U|3FDmWe4@)}h!}K`&C9ua z8vn_g8FxdMbV z)Hp@(n^rY1z7ZVUe)$7gGGEU(Ys`#d=ku}jik9~r+jvI>44v?DcVY~TQeKQ~lj3Nl z2XqZo_T7GYWY0Z`=^1O?%FLOIt=w?7^8G|8F2!?gJw&H*05HCsFRqki+n!cRbB~5M z4;NcgA0Iw3q_@=K4@~PRN%2Qo&_mvIEY3LYhS|iXeU?SGMzX52?)lVxP1+Y^k;CZ;| z>}dSW$xLU{FX-+8V(5c8riM)zKEaFTe#~W4!{pUpRL;>s>)r>Pa!3hc0ohKF&tgT}Tk_2n>Q-ZbeA;FrBzej?#=8>=jGaAnp zVa;apBf{#`&{~Am{^v-nmn=UdSc0N=%Qz(;@ApG--V!W9Q>jOSrIJ1*SjsE>Iu>C- zfs{c}A}nPmHClwFEIGa<7#K37N`j@_J|x(tw*)&JO0X0fTZtFCPv{U~Y(qaHjBOl8 zge8}ZJR%JJ%;ku%RKlYq5td5(A;K_}wHy(~p}s{}OvVvmwI81npAh2omSQd02ep^6 zOqrHqt^K)$SnEH9SPP+LA%+TS5Zk-JLhN%_un$WI(%C1QVm zN@hzjxo~bNR=r(Aia|DA_nm%qLyHP{t1_mP*%29IYryG6^Ym=#XNY zV=2a8=pjPNwB>P6O|e1~GN?XM=|e{Cy||@VZJ$z1>f!YZ^QRQ6%eR2~b;dcQ*yl_z z+jmV_0EjeOnFu?R3z;_U>tPhvWLxX#h7cUeA$&P5xt^vp&XHU!b^UQ61%4u@c%cs! zCCxT-byY^euH|9Xn<~0yN4+vP9WN9mC@fVwDY{UL7{<;LS(Vd)_m~V)i<;J>9%9oN z^V^KGuaN~mTuuIQEehANZdsNAc2C$9RSLtd?hS#!~{0 zVzQXZ5+t6V9*_Ni&OgvpVkRv@{5_8kN&seY(9LBK<2JN^?P~|kwfk~BFlZ#2v}6HJ zVNqY*m0p`-MPAM@p0#ftOSc0G8PZ50=co+7AO>s^OvyE9^TimN=GwI1mnz^A6jdbu zYTG;8C{(fG{Og!p)$>9xo2SwXZ5f){K&(q3SOi2>FCW$A@|B1q%UE1ln_hYti|LW} zIbDEPG~a0Z4L#usxcCK-d4Ey$hcNkqFIm#n6uKc`U~MP$jpBnMPUb8nh5ttoIKW zOv_yn&2Oh4&SkbhL$M`pcst-Gt2B`oeA#mOK!^zLm?qSPT)cziD=^I$JX8CLWlS2I zmzHa&o;kEavuN1XoiWB5&dvzeUzjmj0vreg3rxLJ)-ronv-pQ6#OnTy<~s%0kOjV zNRd%kj{zjB12$GgqB&Bf{i==!a&$Z>@fQY4<~g)3V>RfI<}>VxDi=7kEDq-kO^okv zPK+<@V(ViHnncDm1;IH#(*%U|FPuH@@zO8GnTQr;$UPCQeojPlZND=S?F>98qPcQ- zOhlVLjiFU-njKbiwhEq%bX`@v6&VlvLno)FUSzQsty5FeWXoDK+sE` zgq7UK>@7!ceo71B(q{PrNwS1P*c{p7@UuDQIddQD`IMv>S>9ejD z`8%d-@zoxMM5dQrowigxk_QA_RW~e8?4v=FXxBodZFlKZr0!)SqW<3Z8_Oj>hb#*S zE&K|7@ytbNl$xz}J(1)n;tym`x79zrkQAnnm^WhEs+C@?7nkN_B`sUGn)RN=H5xc_ zQ%%eP(cJ$e-Bq$0tuV8*=rPb=KX(PcU>S1uNpLE#vPp7D+Mkx55KF_(DhvZbChO9T z?HLvsRNk(EvwBCj~2Cmfu^W*e*?y7rAT1Z(mz3RejT7TJm&Vxk4U z(W@$ic3zcU8foXs)R5b&Q>$eH0uO~Q+*&3e+~jiYqs^H`9#H&%W9HRH6i88|x&8J` zJ9Hunj}aAKM)Z19Cu&pRzLyNDKsdA?RbMa+)%gc>{td+g4i-VyKy@J=AP}2FxmdMS z*#(@7ex)DKqEDdg{ea-@=cC;Usg{fAp-xPechMKBWb1Us149KC?SpKSgD92`z97{eQ)gqPeMR(o(KS z_Ak=rLXI|P^x=_CfwJqGNw2rXC~tCxj~ul*tCP5@FVTHFM^@A9KlY|5cY!`rPEEa% z9XNUMf*ufrDt~}8LE8pD(X@VsU}eEcX!RKWgenzrwM2S^*H6>d6jT6LSog?IZ%t;u z&Azv$XLCA`URDA_^P(G+Hh&CBB) zIk?J>C|aYa*x{@zJ+SnPNR87#DQRpdij705d?wy(UYcbbBOya`(XW&Z%~rafP#i6P z!74^t{zi`Dzq%349=+z&HO@b-ogzn%eN&@f$@@m+(h`20LDjaNG&M?$1XxWn5-B2f zrErvUbUJp8INukiXJ;VV0Ykw%?SIn~$u6DhV$EBk#Lnr~qzLsPI=h?6IgM+?z&QD= z&8;};G_YdDM6c#>dX9B=`WiQj>-UI?WFGKyMo zXE543r}lMq?W`HHJ9WW24rL#|G)=oN-So*#4~ekDpdFq+pc#0iX>~@$>iF;{%9iGX ztxg&hOMkdvv?AhsfX?35{X~;cTMD`1{dS0c>@^e)I?n?Na#EAQylslr8)% zt8C8q#lcyIEra!jy_ z=rkMYPvjKS&DMPyk7@~=`tZ8Texmt}qCv)>KxDQFTNX;*lqtx+xaUpob~h8)x4jlV5gnoTzB>?xBpyLyPe+54pNFKy_o~R1-$H5Tt;4q z^kxp?>N~wzgM9Lt_cWZH%89D?6H_2Q%5T<8`!XQe%C!)!;997<`zgQqrM-{k(B)dQ z92{Pq(2wOZ9*Uae}hlbiQ)S8nrD zio|3N%X8`Sd;M-{jyfXwdIFVyT0Os@IzQJZFgH{e-H4>B_?TEiWqW&H=+zAUF!+om`-3s1aAXv zXD?jy6|@7QnXB~vCBeO(bqz-IX#!@E@RpsA@OEN1WIX}rR=)x#y(%nI9 zPu(=%;K-aODzDBUn?usNe^9_p2*r5X^@&QWE!&_(2rYEb!mDDgZ*nUg>N>M}F`cP9 zMG6jxsS;vDXbuN5HD^?1Ue!R`krf=o4yv#;qkL`#4S-}Qu>qb<&COC1N)atwNM<0T z>D%$A8OS3o_g=AH_L{5YErzXrz+077#6~7apEZ7zANHNNsx_yMM9Th8B2i zGYJHw(I_w;@L$*@g2FlU2#@mSR3)a#7oFCMR36&Ov<+Ld>RmMn?#trjs)Wryha%(( z{Yq)+bmbc?g3ZmYC#rrklC&9PpBTU4k)l1BoZ@)1HbrX}(aaPy=Z5~ulBYZ3i;`?f8d&(ZMZd%d;_Yk@#Ws$zoiC8UZZu- zT^B3_;&@D70p>#5VPr!Pp+4T&vveJXgD1{?T%vyd^}wt;C2-g&?uf@uYH^fJ zs;fBC84(KKg2#zRVbG83qFo$k4lTFvQS1$VFfT2RwrNd~L$&gSkE=CkiqT_Ng zMywPJjtRe;yi4hqZa5iSzwy>nVnz5UDpg zLhYZMQ+Ku$+a^0cvy{qITb+lYJVi-SKEd=p>3v^JQlh9rpLYm`#OxzCkkG#+y_o%n z*jwqIGJj^qNr!gIs1Ci*WnyLF-Am9gHs#W98M{&}if9i-AyR+CX{=p>eoS@Zp`~4I zWRfb+TlQi)#WZ^$ij!%Rk00KD<{S$puEXVH*O@Jwwh=NoTzB-y38W?? zA6D79LOJJllpjE45~Tp4-;Z#b(SO?DpJKJIgd+g>M%pc$-p9L{X|vDzW3AE;Bhgqp zFFTF)(qv;E&Q{e-t?P-LEyWzDkek$-9m$Y~U+D6cPqG8f#Y<9uQj%90 zbx$+^*~?X0sY@7YL32>+SFCo*^|ii!?guuA8PE3)5r+y_an>W{?Bj{g*?R8?KWI)m zES5566P1P#)Yq_ClyhLm{Xn)-7(c+c`1=cq&dAdSoElaNmAm>#e?TuvydjPgp7|-M zDa)Uj5*xXj&`UU^QQo7tV4M{!2pLNz<77MkCG0K ze2PtYBv@zU(s`n1?SS}&USu`fpXl{dgzbU12>YA}xm{I!JP|68lVd=T;8c|3ZKUn< zm{dDcPp%69qc&75lx?gm_e-KG;ArYv47}yAfj|B|j(kn>{U-n{GyRI_kf0+#ZAi8Q zn7ULsdK&4efX1Lpasb2OFzHUeFhCDw1P$82Qm>DfbW-5^%PDd`Or1|%i9GBEh8BRm z%4JWAb5Fqd4PQdW8_vnbFyX@56+lbdEH_>PRxTs)Fwh_!=?M$TChZ{M!1w{!a&h6s z`zy(q`xMc`M(mOGxKor%Y3uY-)czGL6~&wC|=VI%nB)(@?LJ+5~?OPBAFZolXXQTXU?zF4V1%>9ov(c z!pD)!7)j%zHdptOW>?aJ9YQJX(KP>0)7ZBxI@_<3)29DwBgGqff~bD_1zY!@rf)Q@ zAOVDslkD8dY9x2gN?w9dk_GM{?o5n>{6*E{Ir-FZvst#00o=I&K&5``o@IsT-ZYkRnt8hCc0gj!V%m%8AKw z!tDxlYT-BOe8K^#!bc>{_|wI<25r|&%jXHsrk@zzITB&ZK~h)|Ia-mlQ#gATN(9*u zIra+7)7|pE5jB_nO-+N)>}FdY1_my1#pf8dC3^7h)!xp%ZN&M$IFPL=_RZj63G2#; zC8VpjGp93rv^3CDbh}fliwvI^cU3y#p>m@pyXv0eqaj(I$Ztce#u@vDQZW%kfpb&y zeVBY4H#~x}yTzFejzwQRXj7);F_hjSzg6PhOvzq#qC zFta(;W2#&)ZhNlPGJWzGPKd1tNix`OXw7`FgV{A2j|2Tr}(_0ft5eX)!0-o2F-}`(d#uY?erF7 zk##P_iD*CKIWw~|v(FQ?Xh}ELL4N7Z1syfvUdEERsf@UK7o(1*sv2dReC%KkMxm0z z*V)3Nu$5e=&gc?Xge;ALYe+3>Qi!thuyf5Cr`#>?GHbDGFpRkPBc z3Q-qy{OGs>CwYxLe1dySN;t7WABlhzVZ()h^EzwrLLNSZwXsTOyq0bOexdxa%+7q^ zk&vr?pdC7J|WOlRY_4J&}z9Q)kq+eK2;iXa^=I911uin5GW{$TgPy#zWDkvJ#st$<+ZjgFrTVJxf5~;=m7BXM5bFSaph1 zLlZ5n=WrjU1ya`w^kHxra(01)*0c!6)HpS3_1zS{52QI`_!!v$dqq z>*I-{RlR>9b6e~$q$I^LnvfpNmC7YvbaI)*&B^qf=ofq*uv01Pi>mq0H;OOaOGBIJ z(g=0Z}IV1C2Jeub)#PYiW`bKI_8&j|}?QEatEx z=rGOnnTJ(uQ=U&0(L!Vm7|F#u<@Lx>1YCJGFI~g?witQxavU$S=)!twSB+S#zL)Ze zHdu`0emUijYZGAf^NO5m(c1TRb&25;kBbxk(ykzeFC@@&7i3^noP>!~>Ng?&5~gk1 z6$7Oi)8_ovhXwf)W{%k{kIbhfDbY*_&56_Knx)cAv9ITlK^>YCW>&e)^jZc@d$|VY zJi&w#(%IS1S)}tTktaB>Ph=)&CkuWR^UI=j@k2pIj!at0E|?^`IWhd{qSwwnUrB4AIyt>B(M&CsH8*H`Y2pn%!SbA> z<#fsWo36$MTc{XJl1BfQnoY;Sqo56Rt9cbM6DIkD;ePC0BbBs2wa#O6dOlIMH)~mP z1w4wLKJ*hNPIK}a_F}U1`Abu~h*G=Si*_+0W#VOIrNcc`hV!CnOacTnj1n#>%xbgT z?|cGRRqp@cnxu45l5XK?8h+r3I;vR$gb6lD)=txWV^A|aPcV#Jzu>Z^l42u#89Bt( zaA8xHs0pFO1=ZY*6g#Ij_sfMdv-_RBgf)&&)Yi?N3LUk{FKwjdajT-I$FbcTM~v@_ zF|%Q%@y@E^WPNj477y>W8;rQYIz46Ns+Dtz>~AJREK|rnk4@W2S3w zZhFMl*_`D{q+Mr3O0F~7eC$N}30je}L`t&Lq+JrHF#`IQNi{@>pQzleMM$k0re+DW z{eVx%AUoxXWsr(U%bbSuE!SVxIl}BhRD$-gqejHhShx2w#o0DA zS3D5nD1K4qYIL2mW^m#du6$NFF_(LV@52} zG_?y>_`FO%rn3cIxc-pT1DCxRBwxz0>o^83Bl%m*{PzuM#wp`O&JXDNgae|PcFuS z-g`3C$=H+c(O7RqLD6IPrku>{I#0}@Jr%oh%ItgXO5x$0T{)Qrbav%@ERFWAWJ=7v zDa|;0a)t`)O*v!dYgYmS{$o?l=`9r_A$!8!a!6x6n>2>LrpH>K2rh@Yh4mh~EzVip@8gN*PrH6VDH``D zvf|TsBeJ}JY~;AZk@ohw>o+}LMR>gj1-A{Za+Iw+dnpkr6 zDduz9Bu9_!3vem^IJ;r@MbH+GcAC(O{xVF+AsUYaLeXAbHDO^Z|q za+4BVy3!JRS5>4w>3--${G6b93-63OpL&7iLOT2xD9_gc&V@IS48>odskJjcp5fJ9 z_nS^^C;X-aRAQ`qNd zKrWj>^Z-bD036{pcd{1%j<;ZdaH5)+b<@iU&THYa!S;jOMi5ul@n{r z<)#V@x3kRu)NKZd%~6%a&$x!oSN_(9Mj64wUIl4=AMj4RSh>-t`OEhgHz_ zT_t9@3mFu~R=lcVN3wr$yU`C`VTyzYrn(Ddns2ZIX{T>;X%kfKeWVO_Ry*jdK;x03 zr4yX;aPoKzf_{*HH{>e{nv+4-*Y!lj`2`xAgwqWHCT}TUu)rGe0!!~m92KporB0L_ zTV^Kp3I)=Jlz#LY$mm?M9LGmD60Ywgfi^OC4ZV&WHZ6GCHb9TG?p@(w z^p<1vAr^tJqBj@{e;p?u2M!OR(R8FB^^FmhFNZcqa54w1vBou4CVvGqzB`&Tf=1Jh zMesP5E@+5zELG$MZW_%vPbUaFzw)a;?dhk z2p*Y^5Kf&$$jFSdXIIO;^#{J%8)Hr(Av zoEq{8b9Wd0YN4wRK|%K{J+ZXq2Wl#A`d$61jlDVW z=}tZID$!W6@dGK59A46uCU36sIn-=?5=ey{>&m+5*b^zQ==anvkwhqT7J`=gP-x-? z-5-xshXI}(E{Vc2Wu&=C3pM+o`39B9;Cvqr@)V{yMaTKYAxw5~_+33f-68 z1qbl(%3XV*;v?09s4~VDbX5gwaF#ec-Y{HL@tQYBWyl`r)ykfAJn;?>G9fwOgGZ|L zil7mO)b^ANX&x(o?R7(Z?&)tx_ZZ1Kt*w+Mm27J(#C<-0-pWjaGj<5tk1fht>qJVVg24u5 zMkkd={1(9{bon90y^b`l5lbe!(ZD1W~7xMF)n}gU%I?E^;yUB63hwE-3MY7cfTMU&uTt zF^JHuQ51^mJXD%-(NJ5p6wJZhm-5ZOfPCqe{0&_~;yCCh&{L%ev4hK!)Gyk2%SiZfMKO3_EI7dRi-!yiPy!|bRxoB>0VtMW&3d~)_>q1|HN>-U@r z$_P4tXIlDsU6ly0pGQtyI-ehzCpy;X+9X=9LLI@V)`mxx@{)Y58#+W#E!cPPQMF%( z2UR8LjvD9=T#BN~*qtY)FM{z4ld9JJ4FQ~-R>YuJ@2Ss8pzxYy{19H7W8pPRzlGNn zV(%rqW?|*TExcy2FthNQg@q6<;WbP85MHx9x~y1u%{(NHw)~n3;otIW=JFxGrtHR- z{F;e&#_I`0v~Te>Q;JVmf=#7TY!NmeVlZqGHlMC;Z4ovf24#c@n>MqA2%9Pjb&0U) zyL^bS`OxmZMcAgd2n$Y#uxZmxh_ESJ^(Dg4id%%u$NG{**kmn@OMb!OGK?!?9vQ|J(~bwIKbEc=jQKQ;z?(L*I>Rl|gC)B^eV8F0W*v&C~Xb2yE5iRzB`0#PjEoy%UL!spXy^5$qHINVrSD9xbCUn|c zR9_v~12R^!`3Q(bMw{aVR1cQHzRcu%Sk)DwzN>qPVAMp~ju9l86<3%n#21)|S8&}S z^kU)#GHCiDh2wjkXe`r%z)w2i`8}i{*jKHZJh`c=u!`q_q(csYr-@`42By%6EY*W9FL`RxZoiDz_oO96MD}FhmdcD_LuM> zb7EK+T6vP7yYoQt0RUBSU#b%X?4abKfA-NtHYHo3GN`1x}pb@DoP;nlx)MVt{x=!}of{Mv#FvDzaj-&4Q`u2JG>% zv;mnh3Ib%VLR%{|iSD&V>{33jsl=g7xA#L03vo1^wNry`QCT+_G9>5+-Xa;qRzV>sdWp+AhBL$GMUx}>*l+qP}n#@V)Q+qP|- zZQHhO+w5~Adhj~>4Q@mYYErRgE3&@&Gc&B#i?;wvOWKSX95O`1e1icY?gDmE*%0ND zG~#aqIgknbKSzVy@-v9;?N^BLEq^gu&wk`^1Zu;2iKdm9gDf5wTJi`OZn3uX>Ime2v`c1bw*vqcz^JCv z8X#4JT$oTBrXo(<9ST(*!Fnr z?B*&>6rAA~9UvT&@KO^8=4!~`yr`U24t?rTIm~pktBa+l;WbDW)?N8>PZZw*3`kcC zUp3ygt{lA@m~{$SI|*E}hPpHfcoIDagf#`_r9Qs}@~})FGKphnbD;xs*RV<&8-~?? zx9>^4@xLI$U9Z-vF27dlgKiJ^Lko1X&MgdVCK__?glwU|0m>Lh^DpG!W%3x-85cKr zzBK?EvYM7Vn>`FWVE02vW|Q|!F(uXtUYP*WF`v<1-X~GM^5|)q@G}c~iW1N(VlU-? zKbT8u64&PNY@hVR9_v5>{1Wl8Vtnvzn|sYeSa~Do}IMKY~f- zHpPXKhmSK36^75DloJEXVCMli29J8yXGoZTqeP#x$%7>&#zf|5 zleVB3lk2&J6rigouR<6iw^L5Z5KhHzt$zB4RlJ5uZ1HlX*&Wu{GaFu5->+`^oe$>& zP-`qql6FBzJU#!3pAM8rb)N4fLCvl^Rl76>bT+QJd(Mvbe)dN z+a*l-OnVMdB3J+~@{8zsFf0|m6*4yteC%NaVE;vQ4QRDozcScaA z@lYUR@yvZu0A-=M_RaD`i9koSkwHlTG-@BWwozx9YA$d_8G;b+x7<}>Bl>o_-u+ii z6;*}nd$|1;*87}2sG|o26T$-;CHbwrcw|3?_S`Y!O=r%{J9w>Gmv*Q=eW(HE~6C%&hKUfFGPo#om zwpW{hkO+|xI;K%-#hNrC4v6UBC#>5&yG1i}q~&n{@XoE;xO~wZUELEPg<(_tXRXs~ z%GxP-9}(azTmrbfyqGfQa*1F(4&#P5U*Uw-iQ!d* z(gM#J+_#Fl8-&J@J~fHMwJPNqQifeD-hd5sXb>Mvu?!sb>Q!09As<7mK}8%w9yU$C z140PpkzQXaQkwmc&Ym`fx_2tuE*ou`4(eg}b#QJe&$w&2gp$9|P@K_xva1}XJBb!x zOhOBW_A>ZNbED^QP(C?98gsU}bJ=C@+S?KB_#9qyL)n9W2rIEW=<72L#blIu28A;{ks`7f&zEAn7gC<-?YMN;DO(=6>UCeDRKP zvUod!$A?gJw9(NBrFFrOz9Cb!+-jtONa7e1Ir{d6XR_f*d>OY)E@qOELLGLH>7@wk zS#6qzqjx4~=$6~g?UU!U0_&c7;eN}rQ2$qQce*hXLZOAVsYo3RLnk|K6)Q#v{-(+38d7#uI&`eE!@8k)aV zIGnzyp-U&W^q-mQ`{-3qg40orC0iJPQI9gzzVC|bjeDWV8jqIRf5;(qe&Ln!#?7<0 z*>cYRytEU+QsvQQ)X9E!EtGoHW`d}L*a4^cClZt?kytEhem!~YkFonDgLIWYb zi9+24dO#$HEr|?Bb(r3?NHGKP8Pe!(PLuHFFm{hsfdE954h}$M&c(^-NNnzF$FNOB z^`bedbRT2yBAMSv2w|Bp6GBIB@ZR=mb85(Gtd+@h4xjwgd9GnngpfIQ zMYlz+ezq_Eezi8_zREo89)Ss3NnPC|lsP`95T>u~p zOGbGSlL*iJ{U;WbbI%LzG4GIXIO$ui3p)ZR>IsSyttLvjYc_>nJ@!0-)HqWvM8BqH zFC;kNKFWZgO14~3-9uW>YLQjk&>hq_$3570-4-#^_|R3pLalw6-Di5~;m3wfdBTcM zEA<14UFpb@L#TV*Vx5gW{uE0R^oP)Iz%zrfKr#5W*5;56z1j5vPpa*gtJb@OmJ=}X z2pb`~ii+!=vjZHMVeVHcNqLN_ErA4gb~*SE)fjTOK? zQa&z7rda1`MU*n30l_|Pwl7&+-yO0KmQ~N-0nY*g)#Mx?1mlxEPsVo*fdfWfIJ?4A zp}5$R@RA>tYBeBl8b_%XCw@li^(tfRv&b@6u$AD zBO`=Fo83~T1_X5=vf2QXk7+SMj6?+Yd1ovs2C&kJ?{TFRO|SH->OUIc(Nk_3R9Pj+ z()6?D`geSYkux8vvY*^^NgCzC8Mq_#Zk>S8nLiC4U-C;4*a zZ{7%Efl4c zm~V?-H6TYFeLVosC9~Z6Y%kAe!0J|cZ0-M{GcpAM>Nu+yZ0wUvE_iJJ>8qT7{DtPx z39%r8Ho1Cc{m`tBT!>}gWuW5md?XT~YMZq1ULN`Yy=!tMpn?KQh3h*xB;#+K5B1I< z3<8?0bOjjnx4Y%)z6Z<6f~|Jnk9Mov7WaHlo41k>06zwwiY1&3<|qKYn?%#aJW{#$ zRif79{M~MRayov1hYSJv7ACTa7SV~$%=frsb07Wp^~@u=_|o!a=|eETQAd zo@WIzvU74(Z7|3t*Y7iSA{-@b;m$1_`_7T`<8!fWtb8Pq)G13KP;JZhsi=Efx&Gx; z#76j4AzR6AkqEBYLn)Vad&747Q_k)YPj2e;THmg^e!>R(;Ox};TW9lF-^r?F_bEKy zH}Cp=(|h{^gP(C;GzQ`U;&e#vm$15%4RG}Evc<-qBC5yga%$SbE1bduulC1f@X%=e z`S>Sv)8r0gYwE?Q^m}Wh9vpa6y`GA;WI*2yyJY>v)~wJ*^;V_JXC&9Tlio`km)mz6 zb~u1gnwN7n7WcT_;r3$RS+9kucIUCLkd`(##1-M;fRT1boqVl>H^;4&E3QR;`955s zYg-+j8`lP3zm{Ta=kD?|XQEX&4tB8Wa4zY)!s3{^)xi2L^q!lV_xJr(9tMY6Q2fj& z@TKa|elyifyIJ)cVw&P~{%N3v6u656Acr>vP(DYpR95iUc+QZS7UDZJ8gn+~|ATh#!= z)=zlbsU)^9+KUEExO{<=@6s_c@TqIrLJe`{ard4Q;{=<|MQpHKwlzNvEAYvUlVCdc+hVcI|ZfIenDsjPZ?KB;n_R`DVq zww>yAi9|Yu%qA3P0^~bc)+cF9KBrEI`xUL%Z$fK~nV1Cla*;Qy+HV{o+vY=1Ap%a= z+Sy=!;o~+~b0NrAcnm-h3w)xHe~T zKfVD|;Lr)+SAm_`cz9bX*ks`j1Siw;-ENun5v)gRuYhCy9nJ!?5oSNy_pY--vLJxY zs4V=dyo@WMl}~2XX2(MjbJWvvHA5gFeZvEs&_eB5?a^P{OpwkRJ{NCaC$`+6faM)1 z|J~a8udx@8Bdj^<1VnuM@TQ0a$KfEE-t~S9V&*JKY&-1>q_RIKMeeS5NwtL{(^Z~m z*bJvs3Sg$?x~h;%yNmkD+~!*}R9HAYLnAn^2xJHhBJb#Ra+6CX4t$ME;!-=?wrqfQ zJ2CY8fD<5q#Nq3T#KpJ{>nJ8eCWIx5v`n_tCTwvY zss?$!%lSE_k@ZXymn#D!fk-A=z{OwN5s$gGd^v)GO;WHIR0$EK+&(S+2OJy0vpC`n zKt5W2G0BPu%-7o0meEt~xptR0pg6BGUEduC`DVvGVka(Lyt~oG^7zo1RV$&!DJ^Gd z^)t@HCLL(cW!aZ37OYE`?Bg}oTg+?~9+rnboxRgeT`;O_o5ZkEjPUHi8id!zbWt+~ zT?K~ljXuPe9zrJII}4t0!;0d;_T=GMezmw(=#vv(7~?R}d9_Dijj?e}YbH!W7TRDo z;@5z_!TvtYx#hH4ry-nX>c7mdvb4nEfkcdPsjfTG2>cz~u)M+yW7(`#Ds5f(pgn2K zC=o=mXBa>Q_atleUmWnS;!N}d?|{+ifp15^1i|18Gd-87aTRpRnwLLx|5uT2me%Zz)dC z^-(0aJo8|;L%05fUemu_2zqT4*q7G7s$yXVoL@YU2AM;t^LKBnRbN zzA$TzIgmjCXtdj)+iyHXz4-I)t?Fsb|59Wxii>EOaol)J`HcHW82L~Fhu`gy;*js< z;%6#L-dw}BDr(Xk?fGzaq~>xEx;|7tcJl~Q;)*TqyI^8aBn09^rhR>bC+OyN%xe)c zYBZpz7B#PSM81X2>&e3kuyM0|IinzU z7v~w@b(G&_(%UJWci%B)sUbEQvp0oA07Or66xlmhP`r|y4&iDU2Nv+RF#*71L=dG3 zqpc^e{OQiJ8?K}eZ6Un{$=GAx@tuO1=7NsKy7wm5aYx|G zG3S{0v#nMNg#2AM-cP=l4(Z+!SDN!8tX>LeM3s}_7HE=FiiD#Rnjj!CYg~ ziNB3iD}=&xIOnfkvWLxc|7Sdgn=ORn`qAc0QnZseA0q8!N z91?PXK-h{HR%z$(RH4Lbfrg|9#$&E57Fag6-YmEfxogpC84$tRF+snG;7=KXj&}o9 zKI2kLZC}9^MLLSv72iS?xImcAs6U3btA)O4$3cZLceF5<@oWm5W&h{RaH7%or^hR} zb=VZo+^@_?RC)lb@L{rj=W^~fKWm2#fEu+!MwnP3{#5MWy0D^bpI1fzDV7JXQ2Z>; z06lwhm1NL}ju{!XeO1$k=xqN)$EE64jvLRw4~I0O>75b6gG*;IXiv(31@_>l=U~)X zk^*^6hjPfUw_NkOH&EO&WX!oN*ZnVmSX=wJ(a2pt(1d&(um!Rcr<)4@Awq zl+T9Ic*OKmb|fX!JOGS~ie&NzWl}&>BK520fut}f;o2qA1v(eKwml^CV1m3bj!fnY z4+pw^k6kvBA$to3^RYpdtK5MgyP&=n5vTAzpd!h7HwC~pzv>P#U zeYOi(qyU2f^|>4|Oof%{JNLGImW?D17h<6Ldyh{i6U?q52mQ26c8-~IRiB8%?4RE* zX!pPPi1Q73N-v|IjuT9B>BD4?@x--Sjyf<-GegzV4b)6tCT zQ3iu~jLjFW)JB<)?G~ScLhPIn2*-^=y3Z02WG15K40s+dAPqKr!uc%_Mw1+A74V9AOqci|!M! zy%w@{)UiwKv1g^SQ-VY`iHxD*^|Q8M%trd~ZeRUAnrj6`I*t3mgnT*E#j-3LGGy?E z-wUQfc3t%-*h}g0w0%77`)J720FMZTssC0yoDfpl(qo|41Oo7t?>NT z>>wQ-J=us7y#tS^s}EB581``J7RVHm5V0dCad0K-ca~_<@zZ533{L%;j9aLCzR4zF z&5DMQ9teZ&frq~=1Pse^^rfbu;Zi%-gFzupDTnsafcTn&B?GGaV0TIvFKmuLzLHa4 zHjw;D`yo9A@aV0PTzo%=nJfKG&;UG_-qwD4>EgOX3WqIPaGy-70_lJb0 zj<=ZO6{xnm-?)_=iEx4MyUFs7TsB$zX26mlAQrWn!duhl$(HnLCTSv&TSgrcu;knZ zc9_&bd=#d=%hIhaI)Fd_s+Rq(&VWP}I@4SN{e$?gO0DXJ|7Kw=EbE5@vJ0VY;rCeK zkke_0s9yU|!lyqLu;6Iw_)FQovulAyWc{qkU}OW_L!GHN zKnD`zn`Ya5(pE$=&xFx9rOs8_^&xfi*vV#58*(!Ds4Y9N7arv}*=g@o4paS2(v@u9 zRGei`qzmqm=YVI7#d$Pq#e~!8yZ$cG=2%XW#4uZg%1<&06_B^VTWUrua1!SSh3|qS z%VjqS7nePZIwHcc9msDTB=pDY4-ws)T~#phWgRL4YtL~J1`B`XfJvK$HCK@KqVC|?f0SlFmQFbqE_hfKNM0RUj=Xlp7E{1A|9E@$B01^*HC=?BqP^_n}?BydR&l#NAo z*vqxkIDm0~_0WQ=UUoCr>-vye360rqyH@ncmNVTKdz8mQo2S}c6*f?CpTc?sIgnV^u1=I{_pZxvmIc%{k4)GhbYzi9r9IxHyI(-+@H&MeEAAyR`Uo{(K-O3hDiIlZ; ze;}{JWta>P;w2AOBMh3}Tu#9@HcH}3{Xp4-6-E`m$m!mV0|kd!m79}d{jb10&$t8i zV*Th|kv${tULj3;dAVj4cwaWeQg8EEr5IMUu2%{#tk9Al*qrJcQN*u`Y?#xvq9h5L zps9@6U5XB|=5KoxvR3}MKe#XdV$`WR|L%COV&?sXUPblt$^v;vO>{0sJDzT<%(wKB z)N6$j#-f}2XawDnJj2*nyeg#-m=6VpzQc3rW0(x-&xK&C{)H=idq^oQKhza5!vxYU zMfoD}!zx}M$={XN=^7%>M?!>J{8t^Nl5OK0KT?N6ZxYbsi~pg1O$@2TY;!1fQ+NR{ z$2=^E@yTTPVFFq9(*H~cXL4VHEjJadB*InSh2!JwEh=c??&uAe;Vu(I3!fJRoAJlN z^LyjGF=dhbCyviQx_Q!(j2cXVg1c?upsP#JA zFg`b>+weCkPl$Ai^MmOWU`Jf@uUMOftX?%_sq}sM5O-T*ixKxbsaObC>-4(4BY`ns~@3Q zQT@NuGNJsc%E8dgp?|+@$>mOt?H#f^;|&jpp#JLpo!(W#KmZ#Pq(s3u&8Fz_C?a^G z9zsS#Wl>C*nq%mng5uF zoPy{SU!SP)L~q+=3$H_|XI}~R&{r?g$O0Wi&BX!dMajPS`zxxja#u$U!0F-h#XN_* zskPf(M8jfh+Hch?M>d-O#o~3QX=LLf4UD}jOz`{Zux{RLq4D@}DGLBgCnu*PZ>sc7 zcnv#rB(FnC+fqe2da+1eK$Rcz8$^u^c!@Etur%94k9XP6AtMvV8=-eSXXtytl_$~Z zCc+IGayBQ_H(pQZ4xY%*veHw=D}>(L>k;W$nG_&Z=62FUk@dGBtg3Pwa^I@98mpCv zHMaJ4eB0lpcjDiz3qp;aL2xwJqBtqt@jabmn!|tP!2XjU>Ayrf4t zj0#WFI2&Mq^~CX_ zN|QGM;b~*0+3>M&dZGr`-s>g0U_Z;WGhHMz!OZK})9qfKD?aa)PMR^pMlbh`?J5q4 zWDwY;Y$+k<)zP;CbRXh0!!{L+wTOK8X?#S?i^ZWF|VT=}wvFYm`nDWG)R^PfEl z6O{*%wdMgNuCv(ELR%QIZ~KiIZIv=OLs+K$`+fV@1d;0a8`^ck6b$DrKJ)An{%DcZQ^&)Xr7 z<)eThgjItlk&htuC+nhymB`+e}dW$FM$8}W`bBiHHx%3U7?L9yr~fH?`n9G z@l~@Unglp5{kn8hAh@7Tp8k-MTTH*yg<0oC)t{6pI~+*1h9`>%e#l{UhqXE#>}ee_ zm>e2Rya_;f3F4fHKRZ%Bpusm4IEBh{?z_U56=$kqmf=+^qcrxGT=)1olThd`$o_gr3y6u?99Gy00D0NT;%&9Qk*Mg+u@9B!S+qAB~aZ& zz;*<9?U|s&Dl_Cq@C!uBo)$|{|B5?91-%Y?lo2w~*f`1#^~R<8WweV6F25%{`|nO~ zPqK(CMPl`+X%Bk&X}5v{Z7>-$5i2(@b>#7^%i{BK!tUfpg~couwDwgge+HgIO@ynh=usZsfC)+4it%<7I^^=>`Z>tFZs1$j( zeo$Hm0BihGCtj!%0B&>K-sC~ewN{Xi($bC>Dby`_zrH6u;{sO4|c5hTVFt4?eJXl5)DS_XPzPOeK zOjYXD6QZg!Ya}qMTI~kSD{w`^#}Mg%Gtd2(MZl}6$Y$ml2$@>JtcETiw6Z5!2;%?w zRGC>#cv~W_e_~1Z?qm2*sE38~awbY{Zs&+LTcqMjsT2&W1=c-`Xi#q{Z%qW>?>Vx^ zA)I#^74?iOLV4l5XHqSh{#@YtUFu2j3SC<}z7Zv1o3gF&;L{Maqbt#<+%K`6)-;BX zV#fz+;K9DtRX>bI9?|{DwXxR7YqJLntTO(c0s{Y|^{5jbs~-$9n;b&7{`+>i%a6m< zylZb>`G8Ha5qN#dS!abo)bQy~?OPiMx+xpbZcYvvFa$?D;wcC3ZRRO}oV?0in~e3w zX01?2J10FTYso|G(;k3V$icwwK-xC^^}NJ}j#-5{#-6-rda@9HOBAE>6F6eL9ti`C z!GS!BD>wp6GRu&)=6-Od10%niq=AGLpvT#kPtJ$UlO|pSjwDbu&BAex1Wk_XW+v$U zrCFbA)~`fK8)2o9K9{t{*L-(j5<+0I_Y61cOR!xiP#H@Rt5N6#46C4UoaODn8z!zH z!~eAK9N`=#^~T*{I>+j!mxnEom)7jur}PPCP0~QA`srr^tg+|eKs6Iuo-YG+MImYa zmoTqRg%)1v0tT$vBI@gDgdSj&eP#?Z`GwJ}(X1Mk*@8wU`=3~bVL|!*bE4B1(Eh~j z+tp$s-vT{>LaPDi(7VG;@s6*+hm>dV1m_5Dnh8ShawE_X-)NlNh$5rS3+FtpDXLhe zp~C8D)9i6F{#{Hy*Kf=#s3hH)!%6b!6522W8-h>g(!I0vJnVOJM#33S)BtAk{Cvkt zZwv$MpZrz|wX4XP@S#%3kOI zrUEFKznW)JK3cl^(GuYiVOTVdDlNaR0h+0%Q6du^Y(XF@J^a8P!FWq z{5_-sakkLR%D)bRB9v)Q3Q8a~n{f?7aVUW4;n`t(6^yHUeRbMv>%wZO^CP!qrMlE3 z0>u1UJqX}N#nx_k1L5fh8}&eZQpTXO`LpJyRMyXdauxHlJv#` zw@MuuU> z9k{4u;=Dy|-Yl;c$`ShP_Z~EKNVd zFakDp&MqmTnV`jxukBeqo_TeO!s8HxfGA{EhylwCgQKPn2gP6Lu#ggEguCz~JEh;jQhYLzl z43j3_Haat{^a6g~E8k5}fE~9w3?$$?T&?60=4?+?k$DZpYv1kf@sjbr6LQMVgf&nU zkxbg)Y-F=?B3hPtim0_ccnRJKd516R=VeS^_v5Eb(MdS2LeszfYksrK#=`sW8Y(cd z+x`nR4_)_C9b?6u{8}7cSFihRQAITUMt@vG!J~nT@w6B`LB_LzuA_1d z2(e+%>$8m1AqLkd+wfwZlpL_q=k*PhY$X*%4VeF0qc9`9Go}K0Oe^G+w(*q{Y>5#W z_{aes$+sPn2Pk71vDrO{E}-<_4!kBvC#JUDpzWC}Hv~BqoBG-M&$#LMx>T9#7GVE< zlW^d$8}K}wscN1Ql=fGI0RBIubdo{?f$X}b@QeI>SdoY2;3 zfCz0=`Vfi0robWRJwD))OEXA@e^UX$hKyg+Z{TbJV@6{E1Rf;O8QNxSVSS>99pyr5 zt|3Uyf!PZUktTb-*IMv2Sfleh1VZ$}g;z*oUB_Dx_21V?-9aq27Id{y!4)}oIRo%q zp%WS!qOix4M@oZzd)_?)31sO>#hs_qU7%a$yI!Dfk}f1rTTM#_G2kl0g&As+ety=C zRe191jIN!Z+sWUDFk$CR67d}YG{PsoUNPv0h-^b)dMv6lA5!;r#~h3)lJ_iFOLxQ5 zFVRy+$WQOWy<+Z2`RKGIwbu|Fd_SBdGyvxmo~;41XPnFG6ik@*rl}=B5$c*9f_%-= z(;{!5~ke@AWcv*`J!mnMqD<+OV&@vVy!8hLnRd{STJBV(UH-K7#ULZ!=?}n z)76ps6S5O9A;8Y@{&g9886YW^u;T}28Myn@#u}YL6zjJ+moJDUpANoyT!;v>rz-T~ zjr#x}b6Xn)3f(+3Bh-jcq_8{nLurp9=J(=-M;$|U1sMYXN)tm6ulJaVKKaEuS^ZLc z#FnjZ#i;%^^GT zliZkJ(p|A8hTMAG4lLEc-nHO<(S3aUH$HphC@-(v#pC=PHDLU%^APi6T^L8doLvb% z`7EoDzJJ-7^@_bcV_x}>5WGRlIKZt=>FD#Iu|s{PS_xT#&X(#^N;#awe)T zc3Xat%tFj6JnA?)C9mQ-i{dNxAVvFddfG$ z+rww&d+(A+J%On&3A!+0rFw}+)~DB2Au=fm{V%UPmEPOP$k0;PI1y3p8VNE)z6zR+@275SCDu`Vs)PJ`uOjYxNOpqbHi6J|DFxO~dGch2xYxq56%iwJbB zKzq5z@`RLcB)4C%4nXI>1-sH8*ii#klgkwlq7iq^4_C;Lb2(L*^rHCY1Y461}EM5@^JonZa+-9M^?K zRQ6{Cj~wOZ-Elb`0q0?}>35<^tpBE+H5v)6A7Bz&TMGw7n??$4h>|Qh0%dW}$!6`LsqWtRHVwPniS@LU4bX`WnTvl|p$`kYeBzg0};72mGLM9riFqJZe4#mv?%9;r-)rEEbwVT#kEh1r7s^;l*A;3R;gaUr|{#vAE!3@Qm7QfX11}7YQxA<$;2-7q_4iU2uaZLR?x5e!{>Sb=LPN|IP47CbUpe zCU&Yy^RI>fA!!=#~b4!23RtWA*=F~XTOU}?J-24!&%2kZF9;s?zTAVK#_eO zmO3G@C)p}^b3@|c1Vpn6>8PmFQ!o5L!m_J019D)dDT*;hdgws5PInzanQrPhZ>5c} zA7i`N#{6HVC?kY3niQol5zd`BDggXrYwW5{fS4O5aOc=<0IndF;a0_QnTWTHu|}E5 zhNkMZYuxUdh}W(N;9+Z4=c)nP(7NI4qekP)gEr!1pCty=$Iyh?!modF#ft>z0|w0W zW(kiJ4XyP1+yZz@4sLH(17{!@GScKpD_yzalS|CM*|smHuF1Gs-{ktSoIgo(!dN%s zr=Vf@Qw#44kGQqj_%_Tuy%MrH>eI~n`ofykL)$aMfWZ;xIP?45#Qye5#MBZ_Q=2U~ z^mT@NfOWM~z8@qRxz;dh+}|;jjKW3ZP$TzZAdisT~c^F9dzAMHM8J&Hd@RNmh5s9MR((@Z=AF42*vF>VkY7RYW z6~&cpZ-{$m|5XzYAr)Igj;(c|2#ms_F`{q^4%}2aa6+mltsW-P*C$N9MI|POpuc3@ z+;&L=Y}NVu;%c&VHv4$h8twSJ`aYLc`XvD4h~fD{o8bZbOm=>Nqm<_IKgxI4i}05b zs)kj8KAy?iq+;~J=j&4V=L4< z9#I2dxtifdUOWa-|LY@ZaFGHQp{wtHn=2&tit^SngFaLkEoj45v0+0>43t08uf)48 z@95k6poU^_hs@*G?EIBf5d-98q!!zo6=2&c3tXQz#TC?TtY??v!%(<3{}Aw}RS@Pw z(+uOT-@#XYgWL}Z^-JC$o^pKc`(B@cmaaoBj7sZMa)S)nT5;b~igzT`-XH!RJmLm0 z>h}r$7xbfmN$-Dwc^v-@%wuI>;P{^~ubF#2_E0SG_7fEdEnU@sl#Myb>{hvMHRd{l zQNMGNf#JqCzgQ*0Hd5u$UiSNYB*nNvB*jXf{;97kZmf;C>7zrb3c=Sh;M^}?y zg<-V#-P4asYHiwhzayxI7F}0aQFYIALa& ztF*?#pynRQN86cq^{)Dj_RA*!#x5wyJ_|MtU%sVT=c~B+9VHYj9~3=E%h~666v?j8 zMS?r+S-(A0+Nx&dMNS;97(k%r;n{*C&w%$POf#pZv*6Ie#7^6Tyc(|Fco-PbP`x#G zdosAu3$vLoIj76H@B6mxd#&lTFp<(`5vM(44_Y=vhF+`lZj3`~bpM0n30MefGS2P2 z!=e|2L`pR2Gi?_SKvSLY~_CVI}3@z?np(@y@^qsqX8weaYOU;>mo>0mkJtF7<95GPB-922x zbS<}*5F;@w!0D`4(@cW{f|-3y`?t{&Q03Jql8~xtx#+6NPJzF+AC7k1|B`W&!Dv!T zD;<*j-9JMq1qh+`M>Z*-WKIs%tN^vWq`Sn9&pWUI5_RRZjk@_Y&naeC`WujXKT==s^82#slpvYrGt(LNaQMJmrqO~vAT%opx>uR7H8igu@&A;y znG&OK-q>))@Gw-aw4e{1bd2gk?4R4`5Q~UcguBc00Ggr8yGeHa5xA?_vqd?o6Tge2 z*Ke5t&esdMZ(ww%+LiX4B$^QP)9@`ZU8r^=BBZ}n0~IMT9GL1x_Mp5t3`?* zMtmU;D1c@twmQm%k^r%10c7;{k--Ho&}zT$B7^`yCW(k6ma!(2%&;OShzt8!YI<|k z*yMGv10?YrC>tF~dkc(oM&JdiZjZnznL$1_-ZxP{$(`YPRN|7ZVzS5vqS~h!W!Ak-3lUfY zN%+G0=K?EiIlQCnoRnY~@iHdV)F!@z$jNnFC&90cngiBj`$zCyq?2B z^2=sFS1zKkynPh#J3N-TsLtVLgiM*09PV&D(SJEiFg-=CaI^1;xll$ae9}s-V7Z`m zzV1O47@)~tZRapV<>f)CWy*nWTl4fMcW}}uNXb^R@Z~jLxyP;DBkVUJQyX)e>V{va zk5w<&y5(!gp4HpUXVXT>yCDnBFB|oP&RIGRVw3<0Qm}OJ|EW}}uGEEIC0WdJ(?;c*I zf#Kd3`d7koNUup^ZAQc@wcdi$?FFXYZLGu0O*NF zxann@wdc9Z1%r{mCynhX9R)_5@rVpxaF%hhH-+;T5K7L>6*KPPmg{yKHO#iLp z`r}}px|GAK|{()DOnkE@{Z4}p7*YQ6a?rO--LdYd2Go3!0sU~ zB)P~aHOEyXZ+#kt3A$o3)&Msj2Fs5InqJ%cW(P1+HBAvg3xVJshFzfMZ;{1M=)zhc z;O4c1L{+9*pef9Mr(?OW*?c`-R8)xYH3;>+TZ~9}zWy7XDzp+CEEJ-ooyLOocmZ2@l ziY;l{Vh{?y+g19H(odMYH3wJHqFEA}QH4VN_@PHD;6!_k+!}pB5`x+YwWvjS9s$`? zfkCc=wwK(9bK;8seu8`|_Ji$j&4BIqL##-W6Pm0H7wjsYnL8aNSyoSAx{2$afoXt3 z0Jx9>T57|w=A$G98ng;aauIV(aV|F^v+$kv((VfdiKnP)gR_2nldA^=)?%aKNfpyC z;UGstM;Y!LtS_Ycf@hGwD4Wr{J9Ry^Kt+$s#9rUXuxNG-TtTd5KnOLp1zJ8j-+Mry z{?7VUZ32RY$}FrW8q@Q@&OAg)L6oJ#hX){)6Qhh104Cw@E3av-b~|fa?#?9GEZ2kB z)!l@eohFQzxO}sFcTho?0wEEEqcGY0%YrN`-G`sp@M5Ys>Z6Yhp)vp z7MO!E?9pfYP_mtNC3m-&S<8i0Jgck(<{={z{&0r=eRI-nE}qK2mRwyp4Fdx7npCXEQB!8WfO->lKn?uVSo3uMzp0n8TV_3`F6d{{|{s55Tywd zwAr%Fw`|+CZQHhO+qTUv+je!?wz{nOW;b(Y7XLn%c}_;$jCigg#^$hcXq8eszBleQ z{w|L1^M67`h_M;`{g!1lIPOrsR^N9kfBisU!_aKsh4vz>{tu(WJ+Gs;)p=%t<)NC9 z{wGQ$49gonnBniSlGqT71lJ=aSp->u5=J%SAGC!U0r-C(Za`bqCE^vqE$K4=<`gcVAx-(TDC7#t*ibC1gV=9>TpNzrPva|kQQ>tlqZ73GqS?H$exjJyZNPN?(6=26A{)=KGSW->vBnfY zzV2k;g#)(&U+xkNdzk-5s7tRw@Y&2WrFvjggx0Z&plr8n&`2bem6eHVy~^RAq@6a5_yf|MOLB#=ps5kZ#;dt2u7Ax_c6{Lqk}n-vfcIexUUjKW%|r~moV z${gaYZ#4QalF@`<9Wo-l4?Llosrrt@`7aHnHE^?s+v#cDrI=Q zi*+)SzZZ>nXId~~c}vNO)AbsRZ3M8FZEmX^^LUQv!)1Oay#33h&DN=HfglVw%MZ-M zgWEZ<>+^vrgCuF!|8=O zZEbiPXs=-oQzGn8%vE&s6gp6s(`JX;6l#65iQ}qbJeXlrWPZoRidx3aI|Pw%9_A_H zgf@&Ry3%=#Nqn0@<%Z6}X6Her_j7uog;IK7MCHkexi^Ox1}SnQ=y^>Gv#|q(ZwO;r zd*-o(8<(7A&-M~zLLj!+xZlh*`;YS4I{r^6wjMDMTF<|ZlS&ZAn!XJVNhiRD2ofBz zJfmp@8C>JP?Su{P^cXakS-sYjt@Ykq*c5T}6f$N0@XROM8SJU_6>dm5-?29s{Ppk( z;aFJIXz9=gTlr>2iJ_+jje9VP42zP^m|{Rjj9#L;iJ=GyF$sR+m^=d*oj~7jcn-p> zzhD)v<0sG0jDe3i_=HWw!DgKrFF`YDt5&xRxJpv{3=_zwCV_n3A?LiS-pFwYRB6>G_~IW- zZz)A#2<5{B3Av~C)xzvK;RWvS@!i)DN)=9BM;`f}*a*&-w@o+%Us()Ys=I>s&(Ew_ zdj5$m$uUpz(mXF}G6W!aw{1Rjcnk?6s3Jo0gc}*5vLPu4iyF@$IYdB{j9afUgk#JE zrJ?I@X%S@o?P7V^jPCFkpg)BV7Tx=D0~UYCbu?CA zuJ*5vRKiMKq_|)ELTALuXBH!g9fIQkq8&l#fh&|3}c^ob0O($5{?ydVpduh8l>hxLf7v?8Wt|C5zLK*}AyxbCNy8AfzR{zJdy}Ttf`u4jj6;M)+1NAtJ2a zaE<`8R~vbgnAVYTE2;``x&nnBE+f)xoe^PI8GC>{bMK9t#w9r4M)e)4Le>TLm0s@o)XwQ)aLJRWpei7jg7|%%5;wN6U0}&*bYKG{OKFS3 zJ?nfNpISUrTgaUI8E-7);>ug!lk{Zp-kYUExIi-thpu;<+8>}XQ)gEDqJP73Nc=5x zbqH|tsgv~AW8XqE8`C@@D+CACDR4bkxF81a1%H=`7cs{33-0%Z2{jHYzpwIytWkx4 zwcUO57K%$;KiY$o*K{r?F@a!vq;@9=g8O<6zV97=Az5JE%l}`h$IkRWqIzuX?Efd# zo7L4y*%trbBYNlX0=6EW*kCYWGB5&;7kQdgf19+U;P@>61zM>ph=_v9;o`>m_dZb< zE)2j)EhF#%2J2znOwG*vv$WW95OWNVhWiauD&yr;>e0`tqI7A=`OYEKAu9(nZ%8YIVdcZUp?(Q zYW=z<4Ru{i^XzGqZ+(Bf_+;+#Lm5)DdMX%{Prf)#O=qW^^~@2-PG_mKsCa)yW_fOL z=&omfEx0WCtLFSTWek$;x%?!z`)y>%?Ktr_zpl3y)p{-q>`^ z;oRJH9HtkL-YK*5XMw~Gd-BU!+u^Kkuu`q|;!TTKc7B&khv8Pyf{N)nHk-C8=Pq|t zr(WJFN(t`1QE-=>fw;6kb4poLTibTLC}FLSqdNUoeebPGPvy*c;!D;Q`)Omj8P_Xp zPhHDOx$7Xj>maff`EzDUvd5q9XV+yv-z{)g_hhN7-zySYu$3H%XEyL3g)3$wfLMN( z99!Qzty=*dL|ovbhnc=B=+G2D8QO3irWC^%bTZfxzeUb8A4Hp|aal%IG28moOSFTW zDxK&{ocOwl06v@4JSmmGc4h~jX;E6VRd^R;X9Gv?itt+gdqM~cultuouhc!Svb zdcHXw9)oRhed>TlISK1v^FO~ZbO{fhSl-Z7wDaHmN-@jcYFeg5sQmow(R9xT*T?S2 zv%uE)jGD_6U*c*A7=-gdr?ls`FG}Op;PdwXBF0iZEW`ec>R}9u3h&D(Gx7J!m(Pg| z`z)ctmu&^C$Oho)ZxKJ?>qiI)3nz>qEJoQwECXvh| z>Zwy-xI%2%a~@_;z9Eo3^5t!iP+?YvtL*VN@dxL+)TjB>_a}jm-u}=qs5KWKCGTTX z4gZNq;p#DP7RM&BxCJHjtxRNDngM0AhRr{Kt57N>pX)GUrW821Ov0Iq2|CbW7|26l z;r%+B<;P*U3fY9<&V%UL+Pz6v4YG>|)p{9DhQ%!|M373pxx$wE3eEELsq~C!JWL8> zCC`bma#<8iyph#&&D9lnREos?M7eobAf<;!b9Bu14TiHPAsW@6zz^4DY2ITQS1Z%E z%?j_(w9<5U;ZHeK=QiuDN{K>mx|b#y)3M@52JIL1N-z`QI9^Sr_rc#jfG>Ywb5dpy zY6Ba@GMz|Y(2aYy=y0}1HD6(~%s#mcS%DoBVWQ(c5{n5A5Rd8ag3m3pJck~e?upa@ zL51=lr1uN>B&l>7G4?EPs`y&hDFmJY52TQ63y(mDi&TriaIF9u+sIWC5YsQW0w`A1 zy+6?}=?v*0OIIu?C~mES5&Esq_OR$~7Ch!pi{82G=3{~zu`9bN#=O-BBLVo8ON5te_n4gN8{0dBR~crNE(sQL(((TO0=#Edpf_7tbcmNH4l_L3^L5JWGD zW8A_B2YXX|&dpfla&eI7GR{jNgEpizlB)F+I!$bxz)ePFjz}k6Y%&^-un(%d=Hzb6 zBO@s;EMXqP+&x(sRBf^#8&jb1y($?vw&u*JIC-VM2iY1#RL-+CX7plJANQnv38G-d za}*m)|83WV18jSMRS5?f_rJZKL=p}T&Ewr8t{(*&;>51i)-t@LVncQtp`_Lr0!ATgz(N;S~}5E)csdjFV` z5ZsG;ilHGbAyABNkc*B6(S4C^5OZ>UHUa1OMcdwo9I)D58y`~&^F3YL2U^aTVtD+g zkb#S|F=1*}5zJ~v;fEc_C6eXG(t1rpwsFg&Vl8}|*YdyLegu{@ZT&GEH?x}}zRN24%+(rcAE7sShr+?^W^^F>da+Z|+ zsHL3v`#k!3prZuicFkZo7y1Z;7B|kehc{ks$|=%(4VLk#kJibfOcE0MGtrf(+gR`H z2gHN){@>%!U~jZvA%o$5W9G`Mbs(1-!9>9mSPc>sx8Sj+y7bN?T%NMBUzG5|ic=FH zgdGmsc+bp#-*6vvcNhXR2**8Sctnuj@sYT8jTE#wc@TaX$Om`48QAss22bJqPoWMp zG#8AOk+z}6MBOf)MLa>G2@W3m2}+t~p7E@QeRg57P|>}eZ8yYYI14t0BB_~$OOA%V zrijgh$2QOOdPnyAhOcv(k<5C~;?cx-1(E+MEMCCOTaf5tnic5ckw^A+CTxfvmehj+ zJ1-SGe@LnmnU5k{kc(%a(6thF$wATlJ1nz&L&R;+rrrygdSmx+mu)GU{qYsOH?4pH zoy%WH`c#i2*NGL$9|V|^Of16s9I9S9{K0OVYm*Z#E8eZO_?d;F1A?{%6t~b)tXGId zhecZjOk&9p9A)%;;(Z(VRs*8r7B3WKW0C%_owG}qwgZ1C_;O_=?vuCz=cnU6f zT7R1?AFzA^qz|1>h}ABRyc9AMHF;>9YpT|uK!&u@p&7$}+7EIOJzltS{o;mu#zg3u zzO1{e40MS5;x+SFXmY2{ zuA?4)%h@2ST4;d_7ZS%J2tKDdG2(5|hVW{#i`|0Y5C}zDcMgU~TME zjokg7t%AP-c-^aM-v0N8#zt)q=b66ZVEsgQrsENE?x@POaYRW32sOc6(@5msuU@#e z*`B74x)4_iWizD=^Ta^6vqTk*c=-7m4yWM00vl@osS@o^c@1a6lx)J(FuxRi(0-E%Nn@RqKz>#cN3O7#+34ZC~B(P494*mmJKT)l; zf$U9?FA(;`+qs;o&Iu5Nm*Z7N9$=-<=B{Crk`gXr8|n=rNCm_)XW!b77x1Undss64 zFqfAY$1>yRyx4+*JB{=~y}1E+|00Bpd)o%nYc>xYSu#s9raUE>Y|Oj>o2VO^QsIw+ zctP(O2koA|!ecyW+jp?#$-SWy1sAM>Up-805-E|M%CFsxmf?!6wZn|W&GVD+H-lP! zS&LWtAtkBc<~|qz|l1Y`0c&C7*o#jDKU~>z;j{#a!+V57*}`~@;dG8 zrK^v9Nw$sE6y#qHSM7isIor7imv6_RbNLiLBQv}asIKF)4#5Z z3IU(#BP6~I?rfs9)im=5nfVn3R>sD z>6ETY+IH+L5}Pw3c?2bOhr^h?MzSb@=P&k}0_`PJ{m%iogi@)*=amZioWU9{OFU%X zbOG3C^P<>d1{D%X`bSldqd#k>`ME4rY-9kLcPzizq^Ln)rJ#*OkdJ0s;`5*FlcYoj z{X4El2K;(-NpN!!^Rf*C^$?__URy)4>@ai?`T#SoW#xX4pgSitY7WD~ritezr z3?naylnC9vX44n^40UYe&1_s5TXq766K+zC`_WL{Ydg%Illiknt zCXEkwWaSw`cLZpeE|DvLEeiQIO5Cj^Yh}VEUyaeTyIhwgMois zh9^&_6T=5zn8TE?RU>dBS+JXg_Cuuv@1ZfKE z^T_Ta&-AN7zur2O97(@`gdAfVzj_v{UO?C|IA1S8UYp~5!PB$nQ#4Zr!N4<*KUMUX&O_Qg&ctC0Y@&_Tg#2U8k79tJ2_%17=??r z?s2g=_1LIFducb32 zOAOER;-R`F-!xNrFbXF7=62@c>@b={T&l7y*i&^~SzJ_kStNa1p`G)U?%l;u`#>~E zEyAtKkK!;4-?3VckW55PkqxX7?^8FWDvg&N>en1;>j4#Q>Zo8f?;}HPsbOGr*fZKk z0>y;aO^#WXSiaO(kY7D53WBiy2PwzUO}C5?Kw{8)V5a{Vk*60=K=J$nwhmx7otqmE z{K3pa!j}ju@}ZC9;4K1*eZuy^C3WDc>&SG|dF=yTF-qHE8;$z%-+G_B+iVIfL3oc| zFX`W9FqKZl1QCSvY?XBC#9S4!DN}X;;)PhyQVNmwGR&w~7XF^Zdk&bQ9{Qk0cH`eg2Q~xG zq(+BGFbqpGs+oTy)i&OT5njE7p#vU1b^LI3B2+6W%Xu`;RCY;-S81;Zo*SE&uL5RIus_)fTTf<0LuJ%2FXw(D&gp=%^#e0f7AsBL)Aux|KTG0J^Fod+Wc z+-|O=t32hcJw3cJE-u8cVC)P`1~QP2)i4f0w*b_F=CPaPcmTSEckCTxkDQ#n8X$2M z3h`NY;rBi+|ND>#CM}ODpAsLS?nEc|31T~<94hH$Yp{sb$4OcG`kw)Y8m*i%KAD3UXZc*tA7BKV<*-nWwv2^at%zj4EWSN)I za(2UeFOjhMRPBZn^kf1bnHNbwWkS+xDuqb-&T{(v@p&oM{_0uAH6fa@PKQjb+DWg!FutCXV6h?4&^;( zJC$*w&Wds~MR_1<@H6rBm`bw?TU-Dse~qNht!FkSKO@l22p=R(E&X>sg3wNiI`1n1 zW^StBwbJO?FZx;|Z z2p}?Hjb-ECZv~HMs%9trNLgVMH#s?~>!w*8oBzCMlZJVOCFqUEL~k{LIUF{MUpC=x z%qWs2a)`E}{~`NGLlraOt|tZ#7fT@=Fdmyzhc-VFT~JJ@B@$S}Xp34>Q6;>4Sx&R{ zZOSRi64;F+_yq{ce!vDam~`M{_Ma9cXhxhF*xs@>yAS%N6P^q3$(X5O>p=smgL#hE zUTj>ZlSniNl0;b3rirYsxAzcCoDI#`aYK|((JkY0#hM|w)`5|3xuOsYEIkPAoK^YW z&^)&PawZ)?LHj$#SE(nqw(jSYBsz*a;*R%;xb>g{*|Pj17v!4QDlC^l9ZKReEBN%? zDm*x<%*;OfYyrQ2l{I61)YFy&j*YwT;gw4Dcj?clGC6*T0+I2q7au?+;5%bykAE$l zS{_RVnggDDX*&+6eEUItc~?-(l>CwC4#JlZ3Lveyij~vHn?f`mXdZY1Mc>Z)zcA%h zY^eS}4ETSP=FG~$`F}Cs9UYy7qp>vqJp-bFKn)OW+`nNNkNmW@cL;xldYrEDkBRai z+eYjc(zZX|%0BHB}Sb zu{W3F+s%mmH$U1kabMiOl=$1HH#kUqlDxW1-ICN&Q`f9iu3nMGY_2L$@2~ltl)#hE z1}9BCeD?jhcgjLjL&7mrYr>{#khz`!{q&Ksz3k#+E9@IY&6s1iFc6Zy}*q^Up zLXtC^c>zABdQ|lzH8zsaf~#gk%gbWGY_z0z_d{@?h}Sz-G^9}Qgy>pbv^?GBaMST_ z&nM77od~r2WXgbFQ9-uk#n_&r33x$$MD2s|dhUec)sL#@`&_RdH>(^shjJXo~qsC2rkT0bd(NY?|*HW^53 z-VFkj%-y5M>7m%TQ8UHu1uIHwqSjmbU(M$^y;hN{lQ=0WbF7?wA}M~UkaMAynA&3& zovcBpluRo^l6N3BUqIzRqW$+?mn;MIhYH_&#YsG*ix0muE=_MdO{7Vw)DC4` zNU<(kjm81_*-OtY0wwSd(?(bKwY%9GpKkoPg*}z^1@&76JxAj{s=BhG?rg$|ygO`O z1V{$^`$r#bnUw~vM+O}oK|SN2;qw>cuCG!s7c`TE_%rIihQfd0bRzVP*17XV^r-Gk-`#Z|^Bw<2Eb z5?~YZ&c-g}E_8c1b+Po^msL0*u9b}T7U}Re-X3FWc-9{CHjZJd{AoJ=?P*I6Vn2DB z7y#6qZ9QM`KxbSG6!j^LaHtEXj;=in-imiYsB2rZIZ^jT(5Oos+yw-dYqq+4)<35@Cf zmQ}9fR$6Nw+OcBiRU7?j@_wuQtIEN-gDM5;NNvfT>SBjpjcJQ*gJ#RGE&%)|$>~2u zTJFN;O4*E~r6k^D=uj~#G1zPt^-}J=%7MR1T(&8J(=rDeN06Xu#|JzIOH<)OF0iRd zgzu(){0QlSST^uhq=Y$`r*%z!c$aD_6`D4TuAT|gyq-aRhcC$L zLdomE>Tbe@hMvJ{vx030!zTqZoMvY-Qd&1SbXI{mIg1prX&A`dM)-Z`zluZ5g^7Hf zbwgnhgFHn{m7|-Y$KiMxj2JC=Hfn&&saRukK8hweUUYw9OE)q}btfnmH!}1EZA?W;g^eL1f$>CVFz*Mi+A{-|*vN<3 zHrJE!$Ca5o#jY<9C%kx1&t$@a45clprl#BQ9 ztB{Ma!u%Y-2HpXZ9#rwZ$9Bh#pK~G80(jKLj<$mCTc~0Qg}eu%8?fkwc(vP6rLbc= z0xm5;j+~Rop6rgvQGW_0I|FGVLekxV-14vR?j7{Lu@xowhawfLz8v7iJI^<8PeBA9 z4-WHioWKA>HZn6d1=aN=>*jGp$c^H2g5brZ>Ehn>hlB0@ijW@AuVie;rq(ypF5{zq z&w(JHf!xmiaVJ=9{g zr^ZQXmfM7Js9&6>z3yU~(L;fvd3UofJ{iTb;2=Q@x;9c~d_N*PXK2WafZ-5*sBH7Le<0eHs$k~I$P>4V`s0}{$KizxT80>mY3cpu`pm40HS17f<64$(=Js6!amO}Pu*5DCgENgBB55MR<9pd;&*kV)En3bGF%f5RIsowMV%UeQCmrbiGJ7~2a zX@_pu8mSM&&p%FswqSmeKn)e-f@9)JCz)4%;pLc*tkOoIzPCSKYAhZx7w}QR*&+b; z9dD}6n7D@%iOPE&kd1^ zNF|KcOD-1^Nsj03@0-T;(3ZHtN=II8fI7(2Zx8s3=t?lV@P8W7q-K+3A~nsYxS3cm z*Nu7FhX913R-Ajppl}T8O0e_x90weiasM@fn*1o)5QVey8)Vvf`J{^jG~-vbA$|x9 z@mdpfFA`_s96bWadh!_Pg~Pks_3gb7>z#g^1^h`NB<`1Uu>&M2e?hBD%hFw)xs@&& zdFF#g(I&?y6fA(4-avb<1Vpmf@8E`&G$cRS2$oZp=?ED^Mr);PYf#6GSYKY&DRckM!ONl~#t@`KkimP;h!pv@%*h9Zh~MX6logI&M7)FgWQ$jHnq(iUJyh zSb~HymL`=ijXp?x$&+D7E_zI$9QG}`#-O&hE^BnVYUDnOnJGR}{B+hjbo-!fY@HZK zz}?}}gf}*?WIGN;U+&u4-7jYA(QIxsd7N@V=KFANv{ottbWJ@2R9yY%J1_OP?TVVY zt&woxc@uuQsC<^44wkg`BvKEeBC^4+BiBx(=jkoXv6l`>6dmU?YN3=&lxj$p+|2PM zZA>i!e93V-j>&>iqItvNtEUjlQpnl??*aaq>4dvk+H)ipkFu!*th zlptAQn!c(dcBi^VuJtD$;7H3cAA1-Tf*d@Y%Z*-H)o}rO)JwT6B)11DVJ5%Mf5EDm zi=`o&r7lg0ObIdvIRi?#%Z=7(#$|9YSo)TI8MlV?v7;U&sq3v%S4NGYFl;l|?t>V+ zDTCg%m)YM*i#g>)J)Xcd!t(<4`jtSgRoT4(iOd`w=H`xh&8_<;naBjJY(im=jQ!Wgu6l2vS*g;rikdN=<@-NYThazmLcW>T|0%R(gy5YuV1* zjLG2$YH|yKW8A6CYukX~^pPRtce^aFz{<%LIWZW3Q4N!oi>(7=3Nk zTK!J;BEv#c(^Yrh(lStnE|KX9rphu&AuyrjQL!cH8A`?_jS~Sx$EM6JpG`2^eKs+- z_b5JFSUxxj03mUad!(ruVNDc_Y?>_2VGDlQsRoSwxFH;5H3sc{>#BH~S+|CH;f~1& z=h9exW<80T5gO@W_i@fLOApt?a6@{857{u>E)m4^UC;l@K(Z_mB_$5ss~-E@?Ik2) zxroa1f5?g$W~b=XR87;UVEwhd0mlo3tm_hI{9fD3Vv6E&}L7 zEg=dd2bewo0t&G8_`@(T3V^$g(W}Sa#vxFKPzZ8Bn+IDXB7g4%*?QWFz%Z+3oPjqi zF|EF(DzJCc;pZ=rkg-B-b3rvd`%0Ka;GkO>t4hNsB;>)U)=fQ!Ld?2g-tdYju(uqa zjwi96ud`jkiombUXo0{BqB`f0$x;L}bI5sBWUQuWGa3;QkhZTt{`+u5c$LHf*R?Fq z-&UdZuQhdle_+_e$j6Q~X7Lil?8KrVaUqzQoPCp%pQ{RVlSnWx94m;2aKYy`@Mj}= zAZo$yCknniG_iJh&h!>^ainKIOFvQf`GSPUMJHtx)iM;Y7KdkEz4|6ZVS}}lVMwPwTzZc|8y{SBa*rz?b9&ln1(D__d zSjq@~im&5kd&)@>6h_n@@toTLPUcQ!=36{2g;hRE`7`Cr3tbmqK8!Gx+-6F{3|%%@ ziLrM#yXt1mh=%(6SAaZgLLinvm}_=KDK?@#TN645AmF^pF7(tVCj`9DqLH#mIqB>7 zRqg}+!8VOm^x>eJI`1kI&P1}AJ^N7PRxjbGcGL~uLqyB?X~>mv2FfMTw+Ql>n4sqJ z&w(+HG<81%>AJ0mx3;|9@!H90NC^i_gaRqvdTnQ-r!t-Lz^r+{=m|s5^rUY~CZLd3 zW=abKr8ybr9FgGd=cEk$88kvK-fz4vw@wP8wJst8U(A#DFQ_bTJIa@dk=-rH+HLNXKqCLgum-9SXd zr^s>|<6?Jr$8caCuu-CmViL3Vj9vS@YLB;%PCh4Xfd#$2yi9S^_+#EbKQa>`z$9}_ z8^yvfYyNKkGQ4B^z@2HPj{(`z{XBzyF(cPQo3&Ht?{<6IztbB8O`Ja(2ssCJ`AQ-cWR#)7O(Ahu~lEmz8;6CmJTI4SDin-;8nREAn zjlX-bqs&3)(UrY5uMBb1rs?6@MI)>Lu>jQpf~S4y2I(@?Nkg)y#qX@6ksai>gQ2IS zflqg4*2;Z2zdO_;XewXwT5SoG-j`An9aP31*=4UTR>s}S44Sa*-k2N3Ok5iY!T=Wp zY|=%pko-plJdwhA)8$cm4|Or9ufc);q3KU2<+e)bTuG5Ec@H60N_qIz;0)uOr&Z>? zvgk3HW289nWT-{QR&{()owb*Jw?>?~Zv6{2Lze_gdwIz>lbA=fGLDEZDWXeJmCH_> zg={1ps$&4wr~IO`&gwc$g;FQha%0z}N0(A-;H1lx_uCg= zJ_a~#2Q$n~*su^TQh5~qaKG)D;QjB3nZUX3a~;1H42Yf`Tx@b2oo_!6IeNe=W?l}W zw2&0H{R3-wKD0dn%nCv1UpJBCfs63xg(zGk_Z6DkTGfh~q4IN9f8*>#?!~UO&773- zjtH8drU-ac)#GZ_U%O-~@}Ohzb@rV5ffv? z$K_N3ZvzWW_&{VSLf62)7>=Kq%kZ6Mc7mcKb;yFa)Mr?cVL-dQOXN(riEDmgvD`;h zW!qc_!Dx4{dr0*EhK>a}k!FC<9+#-akS0UKAiuYQnvSSSQQ_RT%5OPz@`YT*YwNp; znO49=ZYxZzlUtmx{>T6+6H3DOf-E%IIVgr31Kr^4T%IEKWVE%gBKmEHRAax1sNAiF zMy&^SVr6DukBT&FXUDf+(tY^L6<)Au@yKE%d@>b(|F zMl(VudrL3&73e-m%?H~E+Oey^Mqn@{&r{ojV>F-!$U+vzLV$~hYzrwRTh3^Um{K7^ z=t5kBB6|@Yf+tXp5zvyk=3{sw%P>Ushkz}rphflqs0{P2NeU$QVB3Yjx;sgqLIE*~ zY9;3t!6jIhF)+Em(ZX4($uYvTG(bQS7@!t_m>?5Nrp9mBgu>AQdh3BK8ZV5ZZb{HH ziuey!X*x>bV;~FRz0Q3l=cKCV0-)I3EL^SL80g~gCQNVx3hPI;V)A@k`bX(VCEO<#}oN3~EeHR`8OrV^vkCA%P22kP66P<{q4)&2I%T?dJ zEeH-KoVMD=UpTy+EWy(70!GtWIEYwNb8&WX3i)+3QH4je@3#ph9oXU5!ndv1R_nUD zKgdV9B8TerXtrx*|Amcfy!GAr;^xqs%+U)tezxgfxZEDB5;K_OxY~(;tX{s5e&Gh| zv8KN)J?EF^(vG7_I2!^RAlhcong{E~DU5?VUd7cL<~#=bTy@dfko2uvkOLXIK?$bD zE@l?Oon3;GJgCl@Tc-W=LjFz_(0`>gzPFU5vpMIcT|RBI1Y46MlqQ|dWlU%bun>_8 z&j-Uo2#WL6sc#?PZ01HTbBb!5kccG8wYuZAYmmnb^No&P_-T1YHiOiXGNTp_bV)fm z{*5$i18(#seT^I*BL+=IgH?HYY0kK}pvj|%^ylu({fxc8TFx)#rmh~8 zTXsj0BF)6JTC{hy$fhUWgqhGohEFq4Ii-)?f(lZ>fP~)x@x}aDEBz6o%EpVO2s2kU zNRm60>zs`tzw@|dEg9qsS-@u70!%Mt{XORR@9b=Q>~jptBcs#+d>sI2`fo;Ra`}j= zEjl7y4E(E7*nrin7zoVdN5~OkF4GKLR(O>lSmOAKHn7hN$P&3fO&j9`0MEh7$VRS= z#u?Z;OtxqYVZ#KNAUbQK>B1^_L1INf>~mshEseM~JXcg8F+}54Qp!|GJ4gEH_F-TbegTL*q&MKtjFjEDdZ6Beq#lFUIuFX+KT4N{?_Ik7q@8|p83Z;Mmq zSQCPHvS#KW&#p33*jd65d1L&86sxVv#sCDJkz;mA*kfY$;JMXI=uwK7rr{G~V5|TS z36+w-RQz%j(3RQ*+zSSH(`how)yxk>v39!ZlISHNNQGE<2E3lDopjAJ z3CGH+Au#`$cOnoOY9Zp(6d@3xFU295mooL^P~a_V&@atQFR$J$Er9&R<>^DNL&W{a zSVIjqZ(}{&(ceEr1J#yKeRdJa97_e1NLmo0`lMNq0A?gudJSD2H8O{#zkK&V_2T-t zNH%B+KB7u>Nq257IM84t#eAR4(9-Z4h_tKie7*!taOFi`7u7dgQOcpInuc#Ff1q5y z?-tp`l3`MMxZE~W#S()yaR}Qg0aC^BkyjcX4RSL;y#`@#H^CN!a`SP;Q+FJ4JIlMw zW)&22ePKXhI;_p7?r;9r`_sT}o|Gayb=}6&YOk8d6ES?08D^bzu7)7OIF4vCIIXCw zcv!%6C$tD6|9TBdsxk?yLbtwu1$n4iK!XYl7=_i!>4ZFl9ykblPT3$ZezSkv2a#gV zCWEWIF<}IbcZXKMLtE-RzVZ{LRYuy#@Bp^m<4dtg!Gb;|Tg{9eIDfX*^gxyft~%w))6gzR|RZH9(=&xiRg2cNv2FpQZ>~}9TiJ)zo|8Al^fWxS6jZtcR!|a zmHlSPe}Q>Hww%4cqbHa(Ug6?-MV}<^bRzxMv$JfYCg2tEF_prMgcQU=%TB=6jPU=N z+A9d@(Tb;o`;;=QyPE?LD49^cNFLcB^M3FnB)dtZuWoLMyB3-PdwFw;5l{s15u%OqBjl z2>N?rvdF?Tnx6l?KZE0j=;$=qTVp3uPB75k)xq$Lxt=)&!dcQx5(k0W#b)y+4&|r% ztzxD}4<15xoy^Old8;{H4-+faFiAc!t`6p1V@$sh)%ml@SL4OBj=Teem*$!X4~rZYcT4di9VAngD}67 zW8Ar-G>c@=KqR5pgO1pJ`fJ{q3q8D0gf zCRjXZ7mk#$b6TxRTe0w2`D&JgbB695ySD13r_4I}t!u;EY!shQ-s6(cYHS~rD1KuRF?tMW%cfj&yzI5&ivfXXHCr+@f&ade&><|qFp6j z{3Ks)=S|ml%;@2NBjrBZf5eW2IGYHMR_V1xINIx;wznGop4Ga*kN($lM)@r@-a!_w z^F{u&DU;qZLp1GLH!A{>JT;P?5;;Eq99=0e0QyGAiTfT>yso!ba3IJ#NB4&GL7aEi z%7$D_bAbjc-6aqk8J$P4a4EV8bgTlgv@F#9yZoMSKL}#M#=(hsSC--_ClscZyehgQ zxWp4*(G=_7#NJ5S3g&W?tJl(#WJi8DjWm!;wn4F-Q znC*+bXwBTCtDvqP>o{l}%kBKVipSb!*b0k8xrl(3ck{eCPwcV(`;D+y^Y{F|Z_@wN zpK)nTi;ez2i+WP`n-+rHfZ=@ZVuYlq5# zkIdJD>u~wi+LQCXWAvN|Lg7LDX7O`$-`@Xc!(>}WOde85Wq0&IN3VXT{T~TpG;%q> z*9XzNw7UqwQ51A9<`5ZPn{Wt%*P_34-rz6~gNN69kxg3%W}$i!nI}x$2)Jk}n7~Y@ zXU)DNV{7*ayr*l!7g4I%B)K%Gl~gJuP%4;{coH`JhESBmZ@dqDkvt*KSv+&|g6a=q z)60{E)KE>iR5KmBPWwv0`BItN9t~C#(Qq1d(=qHvL>Hph1!5fWvuQ;f;IFUXSMYK) zOpP9rRvWMPdjmcVGA9Bs^qcc{uunz`$P?-|CdoJ@tf4qc=8&+FuY0Q06{l19#-gk1 zlSGmVMJd?c99MM!9ED!qUjbvULsjERd2k(PYNmA!1#ba(Z5%biK!+cawzwbmHE?YL4O;ECLkL!Yq!#n?G?2?8xixNO_DZQHhO+qP}n zR+nwtw(YK(-pxDb&0;q97u-c=M#LACjDydelmf`mFjB_j8+Hr*j2^)ss0u~v1rveS z)0>zW+))cg$p}W??JKt~HQ|-UHvauc;W(5=UxkA8iPrIB@f)~KWpT94*k%eaff!Ra z%yip&zwZa9kGPv6`OX{>*aGrzF@@6PgPtoXQ66yj=1bmmK2Q$`;4QDeG7;{#R17E$ zAy(*T=qoN{c}Hz0pWdG7c~5jOL{r=#@gC?w&SgX}FgQ`P^O#UzEr5S*V(-JBf0t1d zKneItcEpM3cr*Q`{DZJH(q7B}^Kodu`=vN(adVHNMRos_u^qvgQmU)c>A}@EzT-)m zhAFjYnSjO2$BRNH0z`SBVQPZx=W=D>WDFly_X`{H0{~A+;{5$VCGU?7)345v@){du zWvp~f%1DARTGeVDX`;9g&*BIgjJ;9#xS^ZYk{kWUCLLT?fQ1sBy17K6(O79L;yWh= zrp?}4rMS0k{4+h1A@I`e&@z%jbEV61j^f;Y|A@aJWc@9i1tkeEG5 zLtmFR$ije?7B@P@aQvdWRtDM2E9URKbkc?$88iN#5v|d)owrNt0?pIa1rn!cn}2|0 zBgPGW#IW?B{SAMf>8n25=fL;i?K!|VpbQHi!Sp$M{~m_{%M?z7{pBa+5rKi01I_aw zyV9@R%2rV&C*r3huTX^&jcPH5Afz8o1_(ZY*G?>h_4HGilId_Ap%w_@^#^*-qUMF% z+5XX=Pq7O4SMZkdQ==3ZI8lKwWFUZbdt#Trmn+nHF0p-THOmTjVFQXyak@XH)N-Ms z$#fVJOt$%h`y>h4t=^z{XV4r=v&w(oOzjn-O&1>E58%nDc%mLbISy?Qh3HxlqOY{y zj>OUB-4@JoG*EHz7IZQtzI98endxOM{4KC5(EvvDq( zm1R7PB2@LVFxn)B()`nA{L*ndp)10ApFZYa;HM}6gmh3#W@A$Z&WO5>Hm}M+<3zTn zW$V+RHu;@SF8}iL`MzCxb9Yp^bfHup%`(x@H#uE^8D4*{U$h+=H6Pq{3Fp3vDnBI* ze=M_Yu`%Ne1G3Lv;qUo=YQ3{hLE9zrP5(uD1$sZjHeHDop&)7>W8$Gz?4sa$Kb{xNs!0o= zh9kv6WQfG=?i#$b<4oIS3)caKuy$GF@>R5`qigQ_`oaX019z3^ToFA3Yzg*i)~>r} z#z~!v$kz-y!b8l8=xJ$iZ1pT0e>y-N{rS%pmSxgTXYhFVI)BAVzyDZY&h2&XrR!SV zw9)z2PoeG)xZ5#|wjXJI=H_`N?(zuJAMTyrb_d7p?zoi8ZSzveY-ftilVqC%MxsqY z0ZvQEW>Pu^`#a6%zENs<4b5G1=T2=O zVzpgp{f=)d7_y1gT{++YadE4jot~+z(D2z^v(NjMU z)Y|*at5PU@S?k?EOx9b|`>V;p6hjdJTRN zx-ODMGXLZ*Y(4u1+d;tpYn#^Sr_6la7ujrE+tAvIG1h1%xt2IN zgzL^J2)Y8Hh+#{GN282~Ano5Ckhe>gVZaKg-ym_{y$m!{EWK9WL}XP3$(ee7oJnvy zzyd@b$E3kZq3yjrZ`~ah_tIMl)XwgUFRMNv^d8O{t)#&i=yMGcy|hkL>6eks1Pt4b z$*5kzL&K%tE*ZofyiVOfSjH=0mj*t817kJd@}{jK8Vmrv`+k_%-}F+L+!_j7_O}G7 ze)$2i0)siA9;#~R8F5I*dZ6tu$c7g~c)RZPGZq+HROjINPh##P zZKsd^PiMTFPA!3Hn)^w1Irw$E-M^dgc4<80CvMh7a1oF|_o@pc7gqc6W;gLF7$pC^ zr}A-O;6!^(B(*}e+O7KivCOB2H-~$2nCrFB8mFOh+Mi;reuK`c+o;|MGGQ8|sY?+U z)$AC&R1Kj7jyq1;;dNwd7;3Zl*<+n7`!08W8nDHjHMOIl@`u$JW&zD8)bb-3`htXM z!b6z=Q=tVgI&KLd7&#A^_~-uJAy5VV!Kp!{D%S%#ZGL8<69mmcC*zOQ^Ue&xD;IhJ zzdtUdghKSGbF{3urmI?4boSAIS$MLkX*vG}u@j&&P;mPnT)G*pJwte1 zlVy<@pa;Z;$OkEUZ~l~WtEh(k0k9dfJwlHV82>3HD5wK6FV}vYn19m*_OYMoFZ%Z} zvpR6W+)@0Ipkc?_WsS9No7$MVS}#Cogs;2#HW|u@K8sjQoBT6G@4+(VK?!{46>Ky^ZYv!;2}x0%N69oA3TNl>DFvWo z_6q~3AJUdPNDspGY!+tP%lP#~^svC!^F5G~jrIE7N*IWX9K+V3gGLzkN7EmssRpH! z$k8RqqQAW+JvBsPqXX;)-7r_Eq8bAcL+A~&Pf-*uVPbD8QtqN|0#zs_bJvgVUgAIi zN!946cF{?r2w!j^fd52}fP}_jM3b5{jt!ADYb?rzs2XB{x6)I0g-N$>R`KpBeONSq)37I|~j{ zfJ&blkVbc?x%xiH91DQUOAIn4 zGrg4oaJA^b%Q(Iv4A!Vw%c=*Mrz~3~he&4h7hNRdM1VDW>iipooLETws?D6z%5d`9 z=r}dqZaKiVIg7OJ>V;F1@6kKrTYuKV+o%aO7r<*^n639A+doq@~vgta1uiAp$}8DRHjdjL9ZNE__<~X~WfcZ3Wlwqqzfl!KM8%Tdw4M*k$eQN@0H1I~8S%2qIGeqsj)u00% z+yBDx$eX#9^e7^>+q@xr2rS~_82=#(L|RvvO)pMl=3EtlXwzWk!{c&Ei9X64=S<{D z2u3jskXyyX$*AMDc6S|`5S>*6kF%*})P|mkNj8sw5iJiUznp>R%mrAU&gKo29&D;; zxBLQmN^Bt6TI?u`StgQ)wPu3m!s7%|JWdVxHGzPP$j@l!34-{s$R#BVG&Iu{Htj$G z2h}lP54qb4Wnq?Ifk5RL)TppqqUlMooTXonqXQHl^-(XVG829Pn@GehyR)R48v=o2 zLvReN>jMg&i)<9Px*dL7-o*%5D4dU4N~okcd}SRIm=#Li8$JG2%S@O5h|_$AA?WEn zzJsS*1JqpbhYL$EGzBpbimS!O>3)huZ~#o63Gl3XhT@Sf1*;UE{OXSmn|5z>p=Lkn zj;=#y4_q=4o+qD{Zir3EzIAM$C8-vH3W&-&JBEfKwIZodvC=vNB5qvWxts|XtpL=~ z`9AYu;*vx3Yx;U>p}`P1MN+W@1jK$h2>sHcU!@R`42IAAGa<_dz2;ux7j`ion${Z=8%v8C*Ukc;!1(&MW!rp z=)uzZAKvyC`Y#(nTgg=AH)+X(RwC z;j9!8kY_$VZopc6`5=e0Os}CkcpZICIplu0M4md_Ly8lGX3es3ChBsj8LxJImLiQ;kYaxhHAZ3xT zbt=ri=muZZWZ^*XT)$&YouZXLcyCnQPDA7p=tAP*Py%Io{55MdvOQ?Kt#RGPMOy-0 z(LyZS3~o9C`WttzUb2S|1w+O`Q9v?|_QGEtWPo^1a_ChN7c}lK=FND?eZzizLaLae zjY^SzNdj(_Z~j?QeU7!06VlW}P7^9(mdB%#R9z0tv0-AD&Ie!rFqQ_NRlZ2hT0Wwy zbj>dlJWd!Y&v?i^%!R<%wqccKHBy{uBtrP?LlO0*=NKNKaz28HQzqm1BM658l7V+; zXecD)>IF?DNWQ{svH&+WXMRB$5JPu1Gq(6ln#>R=|IY80f=33f@2kfz-4nEX3=R5~MbmRm5z{DFPXTAXt$CNz;4)-Q7$n`6I=!l?z3$;p_a)hlxP`3I#4Q z&!w@Br1BFwCx>2YwO9Ts0T z(4=EdT!q6CbS``|Mmd&0^O|1MYt{|MuVIk?rfaEQvXUWXzV#T@>H4&AVkVvFduCD| ztCNrcgfYlT5$>^K6S?&+TGYOsS8u`5f zp#4loUNO8G7KiNfGQkjVKKLO_?_>A7g%_5DX_X3Wjw26QR(~5LJtx|q)5Xg zk?Y{o0|3U1eq6c$-=sJ)!sD`QbBEd@l_O=iAbZV2MAYF9q6ZJ9^DMk(oNy4sgCE{~ zywZ$$D(8+fmLglvKOn+Kn>&^IVIIf(4rfDT%LNOUv;9gRLYd9m89dwAd~+6}MhA=` zDDw>DSinA-scb$2*0avoFFbS-n;K{~GEOrkUU`uJDi)vq!QjuuJ{9S>cze(Cl)#f@ z{h)>*C0~E$pvL37fTz^MU-s%49ecBVm0|(u(Qdvtm>pz!<91%LZ^YdVf;U(^dpBYU zL=_+n3U^UXo*Ab|A&$s56-VJOTY)cc`9#*D@7QH^6J)Z_pW}5*;5fH#rSJoa@`z3!Q&jD^gsj_cD7q7;&H=2e?-md?TJf1CtObMMzdIq z&AeSwPH(0rHO6o7DbJKk2(|vYHT;$^kxGOSMSw5{fuPULixI-?veAH{ zpMB}qah~P5iX6N9?ped>f0f45>6)#_&^qGAUUsKU4_Q9>7a80iL1F7F{eiWBYw+El zsv5?e3d4gpUbBc%@i#vw1p@hb2nzSk@{tcH8Dyv$sls3~uFlk8y=P{ZI}CbtnBU^lyF51 zEDj5TN_MQLHS83Yw&B{3@Zlv;qXV%QXpDSHQGg=sYKgqXf~vowa5#LgUl1vtzVPF& z{-PB3TB}It36uJ~j0KMHI*=o1H*%7Ljq;qssjZU@JxZYw@|q*dYXSOaVvJ^t-)vyk zRam`Q;cy=sI%}Dm&C{iqlfdYF=HZAUY4Prz{Sd*>^a{F;5eaJP0(yXfG4t2uv&?md zx7`dRku3nz;{4ej+-GLzk^X}*OLIUZbx7F6>(3FJD<99eVymRSrN!b=~Z^xPlKIt&_X@IBf@Fc&yW%*f|ccMVlRZe zKIwSCeEkGYc)7F`=EgFZOhc3!QQPEu{qHA#r);#&|8cwhe`Q>m*%?{?^LD$VZ5zME zhV^w}13{(mKMwG!5snmxoKdMh} zNNx;wJrriiUC|KCgjzZ5`&xM;X@HnFPk&@7;3J7*E+V<1IEOAXC?wGYhVM7y{Dbe+`E_ zv;T!2IWw+lB+0BNR0vTfXcArnT-C7oP5?VJyAZf{9Z|iFy;0pL2}*PezGI%UBV#%8 zCs{FLu{2zg@i!7~cM+bst$lo`rFt0`NSau7{UL?gZ|zJw%}=mL(uD?EWz^zx4FgGq zkyj$ataBbO35-LA@ND1-;n_TxItiZ+$MKuZ)2NG{vpdf*c=J!Pq-5nUKc%mr5aeN! zCO8v?6?x7ojuyV5#45cqm8+HLw#M~VSM#13d2+y6&Rhqwj#hST6yak%q#8s;S7jo@*XvomW{G`P; z2rl*Q3F8eec6R7Gg~4NN9}`2XITNGM=@2I;x_)GNI#ZChBG-OeYwv6L5{RCYApx65}ixR-#y>gd@~ zUj5xGV-eVwgj;IqVz4g?^9uOHW)zy4^vI@l>Q00sBD~QrkuGbPb3vsTRrkqMAga z`T>KEFXZRB0?YnXsK0d5_-oBcewvJajc+K0h9v z1hIaO+WXjw!-AVOVg$n{2;)Ru9Rmx~;NoZf~#w>?9ChyjBlYPK%Sb-EeGSF?3wG_UiYz-7`<)3iY ztiH$l>$ib-I|a!-J0-h=5`Hsln=@H6aZi$T%rGyjp%Vf<{!)de6H}=cQNFEgZoG>Z zJ=&7-7KGN@QM#Aow_9j-da^KYkX^kIe=Rq}U7gt`>KzH&VO!VAaDiUp?G*ZM|n zC2THQTUEA))qH0_!0K-1zT|pVhn%FH_HqjRm9rG>tGqmO)Hxi(xgYEU$?2@6IY_Xh zb0VHCMvk-5hz%pVp4r3Ms(^9#5Std3ctscE4YErhTlN}@X%BSVIWxe*cQ7Zf+qxw* ztau%TPn(j_&fJIWyq+vq$Yk&U>OA)v-ibB$1;AK|| zv0U56s|9gWPCU|z#jdm|SknNg$^b?x)fniYaEN@@g1Uu35hW{nv$g5EiCGzW8QNzn zSb@~`WF-q*gCV##ZN*ur0xI={Tg94YMU?iWyMrgyF5Pz2Dj3yB(IMFXT1EMK9OuZ<3GWm5h()Ppp2Q#)<;P}W{w&-q2z&x(M>Ndd7`z0+5#ZO2 zc(krejCV8ARYVuO4B9d!yjEfcb4uUXri38GO-l3=Y|0aQfpmfcaLi-(GH;qG&0<+C zk)$^~k6DL@S1WSh4(|pMN_i^Q)>X8(D(#FpeJ5(Aore$YQ<2!L11=L2)Dabj zt0xa)0ZZ5=ec-Aw*SaR}+k2f58+3f;kws!G%x4`Ik1-h{WP#q4%r^it+rX(h$h4OU zc?C)M{nx$5SHVt{B^x-L;hN*+#G?x5a+r`ZpT<%X+e#S*yQ2b}MC$hxb4((jg=))5Tw zALP|DS?+*O9Zm2)i|dg|m?Uen7GzgXxj(iiv%-la#NaIfvak)Go0ecO=_h^ajOWC)IUlC-I zbR*{w1;rngmOw=4ROzrQKO};h=l)4Wka4TY;r@e#&s9NU#HZoQ2HHzX_0Ni~Q(}8& z>%IL)vW7heh`p=Jx6e{?zzdId{$y!-tZK?q(B`626eot7hWqU}AgF7w*!ssvo1Z6O zh>=?(3_%U{1xt|fu9@(_W_)l`sLVVX7gpcDViulBF3&ZqGpM4cbsl3@3IP0m0hLFH z4A^)y-8z^}=b&z19FIuDD)J2#c;sbN)*Qk(8?Zd5D*l+3-1gB){lZzXQoMr3AzmU7 zY!AW_$9@8G=9F{Xw|&gn!YBAg)*^oH@XL81f)HhLPJ}qjlD&t_LBbU6xOVks9l~^b z@6~r6B2f|>(Pl~<3}T<09UL+3pgY{OtU(oLUXW4#JQD$#ZZ_1-q3JK_1U^~%+~1&0 zTVI+r5$}K?q&-Qd8cF+@_MNRyjoL8{_H;JdKocC=0{*DtGs~7Oe_~_2O2}^Qk_;fZ zgPqFurE=6*>_L2@?4zEkZb~b!qRlZj%C7`R@b7(t_$*~Qy9@B>H@a|BNeHFJX~Kzh zirSz<2o=rii3`U5<%ag&QvG@Wj$;+TmMnL0E1p;*c(^uJ&}TYzRW@T0(6WSb1T>d- zp5Z>a!FlfN5+qP)du4i4giDeu)DC%W8LoOV$M>o(t?rq(8%T|Vr+@k@+?#L)5YsDe z-dm4E1j#KAiLy(6?gV8uFtnbV@NRtsmV7$s?mXCAWm287gD5r*Qi^UMl@;Dm zK0?kk+-O3_#6t%re1Gke&j(IXe$xW{AbpDFD-#<_pXDX?_}v}9Fw87Bm<_7k^=)1r z`xr2*9js04&R0v{h)JYUe*>4H_bfotJsZsfMWP!W_N0 zzY7SLvF)ymFPx4o>Vbbb@ZPd4>AcS6UK@VPENwtIORC|4wwa{E6~0_h(b{`2ZZL}8|HR(83JM(nj@c=CtBUjwuX|i2nQl&h*yXWr8oAml+O%7sUqA2vaYpJvFPt9dnAPnK9 z+U~|=lC>dgn)y$_Z$4~O+7`AY!CIftyAhoDw-q4+w=*eZd?uDQ{!gjs+Zd)yeEISA z7#hjF2ESQkd$0ZPBfiK~`}}_|vK-9+YeG_XHFEKEFr`=d_b>|6i&@&Zm^#sm*%-Q* zikKSPo0!tenA(}USP-zYaO@gHyd$(i(h8?hyJ=-z>sse3 z;#fhCGRX6tSuRinA_-ucBq^|Lm8;e2YLV+Dx25cDV{L7|)y$@w9Um_{sCC8v^Ru)`WcF+!MuNoKV^`Wf_u1OaUR@Mq{Ib=enB5x9&U;r2 zp$b`D|Co&7*A$V@ly^{C?2!r0@5tgA>+@Ip(VknUoFQh48Ts5<<@9Kf`4RHdB(FH$ z9Q8Klib8C-BT7niO>{3JOfTaf9-*GS{U(dm`!#1}^Bg3Xie{U@-|*pZdduF z)@kf%Ei>Hayp4Vv6E?=(X)`nEqahp9HpZMYtQbYHH^^NMgfX&ok=UB{2?sr_?uF>5h=*W6<(=N@uB(p&UBQ`l>F3o0&#QCFD> zjR%dPe}>o_ytj+@+#~6+-*=5mkUH5sGW%XnYsD5`Ply&$Qi$tw(V^HRa@(?KuJ7K8H!sq!8|@WKRGwgc$!IwBEmN2 z=ElyxI$Li|P+U;0=M}&}LrT8Rcq7_EbBNzY5i-nNXNzJUD}a%nTAd5|kGGYpK1;Bn ze-oodZyOU%=TywhjK5)|QM3xEXLtd`c|7)EY3^0vz6_k$2c3DXqt+{lgk*j@pZ)$z zuX)p?y*i3`DSj*KUZsL*he<+ldTRd3)I9Bm84oFmGLeRJ8giHQptNs{<{OaKy|4D* zJ1d8H*>e}+bm(qAS`R3iQdfIIRFB24=Wu)%l+|yfra#Fm3yZLM>rp^$=jQ2kBdf=-Hc~P2jMu$NK%LuHxVfGB29`JW z7Y`gxQ_2e(F*r3NEGUxvEgI&(er8e%m1^cO0h9`x&ai&J8B-2W+}2rUT8J*+nHwMe zI&?3EuAaIRk-Jq_C5SloYe^KI0sKK|5sx+1bbH1*Lzk1|F9&p8XuS0%@!H+4I|Y-z z77(d|mV$}`1Smk&)bi3v0YtnOr*sFed(*CS3^&&SbTD>@T`OB1i~Fgk?{N+SoN4!7 z51)_5l^(*qj{-8cEks>m9slK#lzISpK$_Z*#*5reKJg*$oxY`kYKlU!M=bZo*2=o( zvs?ACdyanO1L~a~-}b+JFxPpmfR-E1Qss5CzYXp)It0mq63S*4?Ze>bMr|Pk?KIv% zdi!x_x~U*}#8*GnY~CeFTzx}`Jt~DtRR~ESKZVMjk(VVoX#tIrn>nE3oX?H^KCaC(Oatf6#}Azvp`D^dl%4WMzZ%WQoB25ambdJNJtQ z*8adY@DX^p4#%yZ9iWN?^_6iv4Q~VUKz|wTo!#3EO4h?sa^6p9WZ~Jal9XZq+%W3( z>oMi@A8xds#pgG8Cub@FzQXku9^jdFVS4}Bze?etBqUnHXe+AduBd2eT+~#99oIy? zqQ*Yd(7?$*PmO>6<^bpJV_1N4NAM3t9XF8Q#f}zzXUV%BymG)ON4HihHQZJ_7lpqgCR?s9vTm)KzSw5Qwtqxtwty>K&AB07pB*Wa3EMuC|PK zCeadtBbYEx@3Mj)0CjPH(q1anYcQINvy68dMqUr~AGnRX@!;}Ae%#WYq^qWCi zHOI`9NUO65Ut^q55g~rVpP*lu1i@Mq&gV_mOkPbt>Wug1!nOg=79L`qj#Z$A^w_~bTE!KnpaDboefH_# z?cRvz6#P2|2G8!B8j+Q{iYOXA9l|IJ>%sroxFI_0-Y2W{tmEElpgvFRZvCWd{M?k! zgatufM9ohWH1&wQ_d8GMs!>C3!l(Q-%#^ca&SqEaHgM{E_p#XoC-eC*t5Qj0q36gOKV-~*jP~+{8`ylIKam*NT>Cp+ zYJh#H08Sw3s~O_3!6KCyNUg~Aa-W?&JhzuTlv#CV2R)3|Y-~RMw^(sG;7lQU#@&+5 zQNyn9^80*H-hKB6+yHWr`pi(Bh#Xn(o=ACo9b39VViOXUG?q2cWVX6K(Gn{p`$3g% zFC{-WJ3Z5OS`zHowj)5lf86OkWE%2oMis2O%?Y*}Zow1r&rduRMqXrk)X&#rXnpLf zGR5~Bo+2Lp`E|+fRY_!!lr%hU^O3Ch_MES7NV?dOW|F=GrQG4ZnCDDs9sL~MZJ_me zIO$#TgURwDZ8Q*^sEaEXHdm>^TpP)1nc7*J5UCS~t+*WfoiR>i#fwXTxq>`<7_1b&gIP2TIcA>wcri+?uz= zhzR>~eMWXxH~o%sIB-sld}ARLRzc^s=386q6!2KoR+`oz0otLA<$>ph!KD0zFqcm_ z?&OHst#!^5@kF7i6CQ^v?YDintK3_-2chVL5zz=W$=n7-8;KgOH*Gl^q+2l|F?3h5 zvbrfjq*l0n<1$^by`ehhGq(3&_7Hq^jdhI#Qlz4`J^Z#9{~WJ)ngn}t*S2A!$ls?4 z_KAwKpto+UgpNM>QS=x-quSdfDWdvz5I*|gN-J7V9(2Xj*fH?2mMFP>R=VJ&2b|e4 zZlS)l3Q1LSOG928tF@fP9Jvn!zXFWDn19%OI01X1`Xcrgy`5eh&PGbD93(q7Y(m_m zXk%($oUf|ag_4f=X(DGy0?1M32^burF95o0{-p({`+}Zd5-hfsx?KNtG1zhw z)RO%zdQOP>5&M~9s*8J8H1R{r7EtJc0g*WJ1QJY6`ZSr<#`I>ZerqHk5dFBoiqkL;>1Q{q#*Q^&1_Kl*7N5Bd;{KFTzeKYKDg z)Fa(VSzJ0}{)28u3hK2SPP*M}GDqdxi7hzuV#coZFy_q~AiRZWK?=Y>Okz1Q!-a++ zlmZ-?H#Kz}q$AP$A(N6%@yTSsFmgZneNQh3zf|`gpt_Ly*JDJMx>h>e30x+iQ($%{P(_;E4!Hp~7M6b^h6 zto!qpt6LRM@p2V)sY0HLmZ2j?v{c~KmujuyR4Y`VH)z@4T`;heI~P04%8%LY56w-6 z(J<0%VT*V*E>4|5&XE&Rvrl$->)>}hhu>{EkDOJ;Z)a=A>BsmJ)w2N}5|gtLF(is2 zM}aFeJ_fBjC?y|C0W*PnhQiE4mobRSJWLe3Gi+QBU`YDa^pfDs>BTmVaVMdVNZ$DOV!xO*UkLl(f*=GL+Jxq%w1^50#8*w?(?MMo5MAJK2;H zsFANdfwypJU&F z+>NUFqzxrQZAN{<`D|~$(b+OFWB1KhA1DpwJW7R0NY_reLGB&~siW&e~;6@E zu7YaByo5Dx{h~d%d!Tzw*|QQo{xx6^7uPRiJA2+qXU#7wNw>}fBW2_a8+aHglE;_D$GAvvqswV;7+|gT_+i~GNh)Fn%1V` zaFOmG0C{I;;t1inhYB7^+7-_BxaXvxWkX9vTicS#0qDDe$e{{`oh(dzXoRGiIrS8} zk;nCJ$ral1rbyXrA^i$vf82@_7iGk3%*2Fb*~yGPl14m%tFJ9m8D!QyQBOmB%cS98 zg^bsJBNZ~X7mUM*Qf`0!qEEI1<((6Synstt8@ZMk)=%hI+1xz{cA}8#&G2MmByL7O z1Ek0A6Gj=T?bpuMmXOKLqjwYMI_T)~U~#5u$&bT|r`Q&{#qtVk!NCMRwjfxlB4JqS z0Rv7wbDAE4f#uQ!Ej^RkGWZ0C#-HarY0r-t4j1{`Dr zvnUS}l~Z(g59CulXF4C+H#`uEwPgzUyukHZOykW{gCro|-PR{KD*M(KKra1oM&pu> z*RrZXWjitkE#5TqA`>0?88F|f>FT`)%Yo;|c>uN@as}?q{mG4OH{{ELlM<|K+0~*N zA(|@pE!f72OhiD*&dQh|F;0%x(*Ci%o^K3TO}?yM%>^x8(Alpgpt-%&NX|^nl*8>X zG7lUt#>n>Q@2pc($)|dH@v80Al&fz)#(h(?rvh{V%k{AJXO$%aL|M6MTT9mSo8TNdm zcr(?~+ug{4@SZnI7(ISe;UBQC$31oTwY63>PpCRed=y349S1aY;yAxugu`mJ z3v}{1uok6he!lZdE9@%cBZ&s<UENzHM#Fc;dC^n=%KQK>fw@J5TEYUgO%@&pWj)BIN6U||wAS1Z+n&5BO;UaAgWG!Bnp(YSe7}y04gA)+P#?eJz!_^lcoAU{iAzq~KzDI82c4+(f zo-uTs8(n=eo>K~}r0uEbXOd+4nHC(7Wocl!0tU91LefM zv7~4c+*>_xanrfK&?M0X?|NPUY&^&uxJTMVBM?XgX;(mq`|U;)&y%xvv;X|E_JDfr zPfqbq>|pl6;ZNn3%`+k0V*q5cwS-VL-Y$A^HYuo5TeF70w3Y>as22^VFhB>vm*}t& zNV1>FUR>dV5Yx7=Y~c2LxhwDOJMy*ZbP4v5^wC^8C=1?c`uPTQNNd~RC0)z>3H!w3 zfZeGKZ5RTLk=*`RFs?2tDWQ z4?b5MoFwCfAO0vfB=Z#_0V!q@5V}7N!_Qora$qndtz=!047P{$gRf)Y;gR59-C18> z+gh)_f=7XzBrd@Gd&M9*t&pW0Zm9;%^7rMi-HE1>rNlD7g?XcZEo+jhR6$8aO-nM_ z4;42HUYwgKg-bVM?qgrRWNqK&AE|sZ?NP2n0J9E5X9%>4VhGLE-`-o>-oB$>fnkfl zDWXo<`ymwwyH!vpRXtNb{}@^Txo z$7iZZ+7Q5yQTy&ajHgFmJq-#oFCi~h8&Ps7nvy}xEghl@ z{uY0C({kk>&jFUHoGPUD0*5lBw9S65f&v(7rU4)l$&jU>^@q}Mz9Y!S<2@r zvE?tvx4R`X^&)|V#^x_zMoZ}jegmdfiWy?VheQbNe!1!=M6jY1bDzq1GxGzq5E%Vw zF$d8@mK0gCd0%E&{@`|SxC+6?b$CcJs)*B>wR?j4xDJ#}{(HNHsMp{A%4i@y2?gC8 z;rL(~^Y_NH3bgj~!t};;lbigO3X~R%$>RUv>>Qgz0lF=noY+oIY}>YN+qP}nwr$(C zZQD-fe7o<=+^M=Xf1p2fSM91^d#&}TOo{D^n%30S`SLRW>upabEn55e5Zeyx9 zf^m%}K9(%V>rGs*s}W)PxoFC=a&U4;mlvWSx{LORR@0Xse-z)0gn0`DW5irQcYc;8 z4Ki0DENxYwy_zVeo1J5ZsBC~YSzuqC94uM1h25#VC@I=jCb5vED~mVJuRMYBVZ%sk zGmL-vC|A}x!I2esugY_CDQ>)}Y6wU3>YW3jdF95ex_MA#CTAsQL#oHAq$w5DFd#2F zF#2}C5};Sz-ha~HStupK63Ce;MfEztygiS4$21w$D{3jLYN+sQmOOM#WQ{B>o{;eA z>U6vXL-T-NKXY`v#H~PBmo<;ClUw_VJodMByj1{O`q?$6egOVd%aYyFym2kxx_2p& z?tCeP{f1zZeIU0F+ML0(lXU;t4)x@jS9TwwOi(xgFrWVHG=96TRnI@^K~tZV#J;)Z z{?m-WOK0D$B6{y>j5cZ34?|Yuk{?em^!0Q#9^WM#Gbb)n9m8k|2tvBKo3LLS7MpWD zP9}?1kQbzF=dfD3pt!_)DqA3W9j`q^3J#kljU8t?4$PhAn81Lfk_dxSyYDx(!L{-0 z7SP-ha)TF#uRohprf;?s1OA5?%Vz(GF5^-ttgl&>>xFirEm^h_Px|n_rIuOP}LM`z=J%z7l!Q<-tV;Yy|_@&)B7Bbp?7xH@$ z(DmfBtR+a@|NGsI1g0D31iOI91l z4~UG4jFN<$#7ECj)LNM|PY>B^^UQ9Jd3|$vadC0IeXV<_{>6y%8LOhgf(DU|+H^Oo zXxJTq=AdEkAtWN8pe5wD(F7bP5|#LtUIQaYl__4*!4j6e%d}RTKMV4cBm}?NaYlCAs|ci^W^wOqCLA zSxrGxNrB0W6U(h;Q%!;M`}_q z=Fdr7D1DrbBOOol2h8tb`wH)5zuYtaUYW7Y43QGW^91u9luK;=JJa+I*?%mp9#rr3g+NUn| zw7S~V`x;NmuzpfM$hil()^#f$?e78LdX56%x|b>ZRCL!84Dh%e4DW~{co6ZS<;@D9D{J;qa7@%A53a`19W_7Rgp^ zRP{9I=v6n}H1=41hd-Ci396BPWy_S#GG3;VDHb0=bKP-jH z{W(w9p4CdFN;%p9+4Sac{GQrT-!v_r)?x3oezBdk$o*Enpb=(}T}SzRe>u(BeBPQq z&Gq5=exFBML(8e(mkA8_Y*)7B$7ycajn301ETEj;Vz_7aN)5S&=-5850u4@2+h*@j zV4Ez6#i8FlD!Go|cK~f6BE1oNUv;n%fasbeKmt;h%XZE{nUJ0m($v}5-F8C8=&P!PpbJetoAj?{N>+WowsezhLCqTc<`RGi#W|U{UCy5Uf!()zOklv z!JkcyO{%p(4?jbEIjCg%%sQL69Rj5_xf$#G zS>E%Q2C^2(od-*Mir#>#X*BUZig|R~b|OZ=RNeFQoUs~r!22bm%QJK7FVT&~{bPc^ znB_VCGl4{mM+R#Q+hI8r5 z&K{3V{q^U;22j(R6D%ZZHrDge4&vb+*@uusIgY@dT-xRftBWoi$NBS_hNQK-prf&~ zIA>D>t4pq0x7B2du|nJ_*m7;6s6@`@`3=D2y1S~DPrMH}n&qvQv2cfL3rsNFV5ePA z?RiLv7~&C9`D5UAUF&;au|HIE_O$5*C^2()$^nx5)!Gz+{TSy54ci2ctqmrY)#GE_ zfX%NUWw%SrC*|ah2{uGq}z|FWP+?FF4**FD2jheFw$S z>3!ZXyjf8dZMVd?m-~{AnSRXNCSN!+hP+a*UhnVw{lk4_PtAivB3!nWPu8l+b&V#} zlRl^p6n##V7ZGsCZ~-|Ao`rFYwix&p!9P#S*7T?p;D99Q6N zvcU>Wln8A35zZ*-Kk=3$uLp5qcYAXiVIwCU8zU!o90^)w>Xzuq3ClPIuPPX$Up?w> z(R`8_EjCLnGIEbF6z?OD=;H;HZlfO`oVO{4CR-E%DqQbtH!`(}6Z&ir-^bY3gQlJE zP-YmBU1<5IcZ>^7cK2iQR6`C_uMN5_Q7x(jNm5B8EMPu_Q7^pqf|Ld+BF|7 z0Ny`i5sG=H!yQ_>Xxaz-b%Wwe8qruh-wmHMTdkiz)1I_q+WNLdwX$-4BxEc348jYK zvoR@`eKA?C>NJ7OV5n5O0j!3m;#{nWrIpuvJiTU-Hau~=$l9Wh^P>k)a@qgF&Fl@E zAS{iO+AY3T6%!4k_&ZW815F|goT2F$e1*f zu7E*j+*36T>xD&cd!1#Uh|!`BhG^|)beMc>r_|f9oA;HY&O$xSmJKm%y;Y+U&6A>f z5FRX?98Ph;=gvM=&a7{M=p9NY!FfzEJ9(0cFqq{(lE@4w*rP$za^qpq<7`qB5>sfc z{d10VunoLqZ~au)+BsLN<3$9k=!+=IG?f2>Se3Ex2>3|3i?aM-y_C(u;?i)(xfzgt z-s?fq0is!UlOU#TZSil!L2mL6ci?F}&=Oe}NZv9%jr(v(R9D}qAoxF^C zV5mDA7u6*s_plfYO^bKuS86Ei!kHFc2@{CCRP*}l8i|{|7G}C9F6v_Av3*V1BnL|R zh)Gp4(Qp#rvWHHh-lo|KlC^nec|f1xGT=Co_))8zlm)_xEMOR0PaCXicEtPf)n<$# zVvS)nyS?p%NnX>rgR(974ZPjUxxE9nx)#f#lNAVJYokpRY<)79i`GyeOW(}E@VBL| zmU}HA$n2uKl2>A1rw=~o1s(IT;B93G_<0?WWTWeoN!J8Xk5mKWc1T_mUCdU@(HTQmhU^EhQ7}V%GQjjn*AEg3UCR$=(Oy1;F&^b6T)eVZ ztJ!5Oa9&O8v+fsC9svYjF^Ebz=PxkXZ!cgO%}tLQ@=Gpso21!=GSg#)pXEsoYOHb3 zwDE#l$@35#<_!6{>Qf)|ZCPgFq^RZ`~&JBB{o>Q#47oDc=nJ=cPy(-Z>6AuGwFLo~$ z?rFMBsR)kA2xCVPKj`p!aXWFKSp#tT*-rbg{)pY&2EUNiLJqV_^g}v5Hoxgm9_Y{k z=z=R1aAQZJ=UO?-Cc(a8(!-ndP?!!SsXi4$d3t=(0kGO^3_E?DPN`=6NW`DzGo%xq zt+FetOS5{c;6pn2MWn623b-m=)mKX5#`@C4=c5*tQrUBp1ee5vZrx7AKT2QJ&?Qc0 zWABd>9=LYEt!0Ef6?tic;EDUDyqOuCVl5kL6M?f~4P!+9$e+lH;e|xscQlg=i8Egqi5inqH*vB+wM2Ueb$(^YK;cK zeo>j1RG80OF+bw9jocxn636rv2eMzI$MO?F{7@r3G0b z%~2Z&5#%ym+Kq(l0Drv2iB#HBHH%i`za3n{C-wncquT)La!iaSUq2U?n<3zXN{j6) zqEzo!A&icY;mN8HpSv(X5&sOwgOUT@j!RXg6>c;$h@_ByH)u(b5u^nl#W0gE$!NDG zn;$5#=e1q1V1d8JTGApNeEl$SI;Um-VkhqR9qo4VE?qO7J1Fr9cJSVxsj!m z`3K-4J}KXWkKEM@a0#&>L`NR1%YiIK-8cRAr)X${pwW)C|(jRJ= zc(5G_!7l_{y0xk?w z1vUCOeyMC(dMJ{)#o)SmXWK7{DE8RA$xYB-8x$1pG*Oo4vh-N!pkYv#sZjR6xF|Hs z0rCq4OUV6eV#9?Z4&o$2a{n7UF%BAsa+6S_IQoi|ZHWY>I^(kezp ze1C=>`8xkm~}#M_RO3~(AyW|fezd6e*AVDqAa8nss!xXLf#l^_p4wf?{WqP1A2 zgg(;Z!riK0z5|+^)|Z=Eqt_>wzDm~`oCXflKP(gTa+H5?hr(g<6vIQJowzF41)xz-c9@g22hA*9%MLGfH_Iaq2#E^L8KA zVqn&Q8_<&yokNd3dH2f&Al*JW>kdb#N1#aV{q0ZVwgZ4wd1JlBhW_&3BjvJrYq|Zq zxy`HOoN^oNHdS`PPwKY!^+O*)*oPhkTL5UU?cpwhy%owB{LPG}$wAYve75Kk%=`gO zWt}Gb(2#|ijMPeua`7uxM)bXL1fg3svvTK83wcsdbs0V4!F2iXgQn}a5sJQmW5KAb ze&t4e=yTN0;}hi+blE-6Zc@+1*+JI?gDc{m@G6lJp?Rr!G39Ls{o$OwAN{a{;>>j_ zqT!@8y(LZ|^+xFRe1GZo$%62Ck`P2J*Br~98jN^PjISXQxi!kwE_x7B@P)%<_%`Q7 zeiixfqKFIr$~q-|DxG&7)@0Yl+GpT-h_LX3moa@!!V^}Ys0;y74;*9QXz-{mUuj!z z>jRHP@a?8o=1Z4O{W?(Lzdg2bUh=iHui@@b!wQ*nwc-uNB9`iRxb8g`qC79XrmH3# ziN!cQTug#@mun$I($vAkEzUhuI8_SglhiL-qgr<~@;JByRCLL^x$3O$f&8&_?emZE zyKgK5Trip#@oAb1fR6iEh)RqjX}A|16olgSgi9=!SV+B}Px1>LXjT?cK9abGvEjFa z7^7VP^G2-?qa6smY+?j+xW?TVodLqjEz%38W*AoC~%iQ30BWohAvT+2D{z z*2t6?$6A@5|D<63iehkE?8s9{S8uI+DG|TyvUB*9^HIjg=SYdw@|Mvfbw9h@@m=hW`oJL->KY#rC-j93UEdXy_b>K>;7%5Gz5v zc|Ea|oQo++TfQ5(JE}#y-Im;kb=0PSLoFWt1EYU>yUPxe*_)z}3nD2`He7Zo?gN9k zzcw!{h+(kp1Eml-Q@EDS+odCK14_x9fL4E>6tAM>wT#VV}?u@r4uI+>Ptc;FZR|I}l3i*V#t|OTGYV}5Hgy@w*I^9`M{q6zZ6VO%#HMkf zWco=$0_wSG^V;nz)L86~C8>tvYDM%_6rWEaTAl6?VM#O9itfv&0*P8BRLX-ZU#5?M zEPN#0whCEB?;wGMu>cn!7SqWwh*k+&)jc6JLlJoIORcydu)N{n!=o)z>pl427hV zvvhH>1SmUtJNn2|K$I6fzDg~-vC9#!h>WDVS@C=ZxrkA2CN0BXO}#4ps%_z+x;9j zQ9Ey7;om{XXmG)-_zaW;H*sjf9jxV4UVYwg};K?8~u*9AK%J3CFhCO+0Po9unk)iai%d6 z5)Ypt=;skW(3CgorHJ)&Lb|^=Q+?`1%A^%_NTiM(ss0ftz5HwMBrS&e?vZyM#x-DNP zT>EkB?}H6=Cwwue)KdoX4BXDw;p6)d`m*a95O|viAZ+Q+dya+lCJf}C&R?QFNl`;y z%~-=Tb;TasEVHWF`)3JleWM~tht!!F8ubuW(NU!~B|@#(Y7qgAdb}_5IZH`Z;St`EV*wd~hldq=e=USOo z>=QI|qSK#S{<2j(gj;NQ;W7(mi0X%Aq6dvKwY@~%pw{RBuiQPo%g_!4gVs}=8o^z zNZ6TaQ#>cp2$Xg(2BOcfI&nIPo~k1JDe)i%D9Z&B%VW=cW?HLbBIJ8W5cEYBX9Qws zxe}ysaz%c9M@2oqGWLXS#@<|u{H6Vd?p;%_kj)mLy+li*M2%wv{(W|)`2>S8YJj#g zKUKUv2ax3=0b&7+hjjufQy%Mx%7~IB)B*U<(2xk|bJwY4D^g087S0i$<=811S}C>FX=tgCUUNwIOU zaq0T;Wg|@700{}kL9qo4B=fw^dcVnSq8rZ!p?OuW{quvKVNW)BDN`h{Lk|S*!ooJX zaCu&Ekx(@Y4sWD2(H0~Lvm_2~#u1h&TEah~aN}Msp-PcX_~Ti@)PYpOGIj9AcSF4$ z=vEVl=#Qd$K)C%>IBifP)hh~nFRMz;E|%m)R{Gh~6?uOl!qM2TdV2fmF!B0(x-hUW z; zdd-$bIB?n6zXOahr{%VLDNNuPyn@BK>iJeYA-OtCXp;9;w7McKV=tMyamd__ca~pC zQl;>*T4|D4#9VF0*;n9iD?-m?GQ-`nAyu`p+CBZIgs|kX)nWYjMQdwXP2G$E1i0Y} zkZq>JAn77x04>vHq&<(M2+47KMtZ`uqXoz!PIUq5spXUhn`6ULo_f(evB*EfkOe56@xMgtUYyuH|QSECPQVK1qsb0jGA&t&Jqv}#3L)-KiU<~@O z!v#SKBh75&xF2dam%t`KB+1I@6^o?86+nl9_S|v#N zyrPBeQjOqQ6}2+7dh;!=X6D>^w4I6*5tnq zm7BHD`OC=4O3mP1=Vk*wsYA3^m2olfGKA*6TcEv0GpubNJkB|?$t~b$*97UUP>Su- z!1(PUIx8z#8#Lc=VTPh5&y5UZQ4utc)dNT@R)+gdU_w8aqx9`~y1sd%XUVnPT=Xq< zA_}LMpkA!!oP_Y7yPH9l=?n-K{2;S754Qjp?_m=bgp%bh(a%4pisS!l4P+5ZM1lz) z_h`SwpKVi8d7{H#kAo(UOG(eko{6Z*9WgySmJX#63(c!6c}}VkLqtKSj0Ij%Rs4;C zc4jOH<-OBQdm5xcy_t5fF29aOXmSsdGMPX+K>pzSa}GGi>IOk*5p7IUigKxs8+nK> zS7g=wSt@bU5EGY%DQcnNBW{m9USTwT%9g*xIvNM&A~Hrzc8Sma zlU-Cm%eg9`7UhfGXjhvFT=s76)F%Wc=%juJ-~-t%$s?jDuBoz;U~W1f$t0!dk?)V^ z=H+E1WEX+YBZ5}lEQah&Z!Pzyn0TZ_iXbAk#Lql{LHqHi_AU}p>M$8U^m*Lh+y&z) zAzXgwG-V8L){LT_i*GSp?}a^t3h3_`S7H{%nCxMZH~Kye0us3!?i2wJnPgNVzeNZ> z^*i$xSwHjyV^T>~_p)cybyV~SZaSG|tC;^74SVN)|KFsID*BiNbp+gkRe28d8 zv!34&m3PgCSc$s=Sz%$aKQ)4P!|UE#JL8!!!^U)CQWG-DiqRuB50gD(Ch!)fSqdA8 zXzwh1X=L-_2q;kpl2H6YS%jyu*}yKNxm!eC08N`QgEkR7NpZdVl`=!86d{T8$7Xln2KK>+l{?>691;F3Q+ z;6%&I-a^RV%v~nmuAdV8+Idu)SdexNvvcIig47?+i+?mtxLmV|S!$*fscetkG}WwB zR%eU}p9^0A0i*5Q|Hik$@?XjQ|94cMf$@K%hiJ?7OAOJ$cb`(8v=J;A3CkAal_wJ@ z*LNN;OP4U~CDKX5eS2-pwqrTE!xU3OTdk{_sb?g-{E z`dv9@vk7bL-5Gg>vSPl-fR1+n!kVs=6lDv(4I^j+&qNSJBc*0Gv9f^_$0uRN9cQ7% zsVeu+rTcbk&p&^NB1{fHP8_`+3m)8@m27-zU!SL&ZD;1hVOdXB4_@X5z-3Yl<2D#r z>;5;_#r9v<^*^?`FflQ*{b$iZv6{P=;&KZQK0KTnB<#H6QkDWR6a#WxC0>9UA40r1 ze=b8(i<95Xd7O+mV~$aFqPrPK@%a{?WIC8I1p(e!BEOJuy8||St zI+rjvIARJS0#320)CgBt6)g{kJKs_0j?e<4*&Re}TWo%ni_rM_XP$;F`_<|sDS~(y zZ1rl_;$mBsk|R?SVq)okEtE{u?6kVdY02^BQL<9lxV+9MQ(&YcvXT-R=)AO~-E@@9 z#H>Ez5e$snq{8AqYgoMuw9J&+Cc=I6of!1h*D#d#($8G_T8 zO?(f)&2D26{FOxHf(=CMNeNz733f~17HrON!wL{fe=l?_u$*z3b27xF3rG|d%ZMx} zn6o|nGWfa0xP`byh!r2pL6!tA>@UPE=%1OMsgmi!666BO8;!DzdV*5Dn9LsXREy6S zY{7DI8zfs^OH$8@1xI1InXNYJTy1=aMHs#M?^q4H2~VQfcpN^)BwMN4k1sWuTjyKh zI=eNF42F#_^xQ5`$8T7B&fiXwkeS0GkKR(UAl_j62G2ibNp$%I|IK)@e7{*mIFswi zwbm=`HAyg_>h1^|a3xk%kR|^Mj=aqajZ1ZK6*We!%5G&jz$ZfV%hZNDtJR6Q#im-d zotBbMb+Hn(;=@o<-bVdbcaW?z!O1!8%`>%o1{rgaSdby|=`sd9_jhUe`w}HB_=p37&*L}7=*TJF1M)@k8R`p)-R`qWKX%0K5=tTq7HU7P7 z6@I%WjqlBQH_++2!um5BzQVmgI&ubdax4zOw%d2?RxZH9VL})(CCJh)Ujn4jBw7@h|0;BQ; zOqjgr#kOXhU=-jo=Y@KG>Y%xdxHddI{@B9IqjK3IpD#Bh?ux$N6Xuk zG+dq*8PuyFcQfOW}wKxAvMz#zkpm1&Fff9HJtq5C538hZy|y3wSYG0}nofwLWOt)FzfJqf$nc~`a8f5;%41WoD4=Dt28P~0ynEMWq>&T zt@fSnRD)!q85nK@P(4UD`$P;pu4XO^lZFCHmod#@UE}aGa15o& zquhgWR$d5~PlOu7RM_hk7=pv=c)VB2=j-YOTcR#@OcivDLA!?B3?D3lOoi`+4V7Qb zop9$^CPA}SEdwuR@4f^{od#nvn8;>-h~!4o&nWY7u!fpft@EvUE6U-o0Yg!QS^rf{ z_(s&clKexM!ZTW4q7iBQBZq@Xh)9#stUf;k3C*sScXP-ufctZQP{H-Z2+zW5r+kK)Y^iQ5if$F}f= z2oP3g>eTV^%xtvyoVgEg?X=dj!W8|R^SXSIi*2LFjKEgzu5`tp*oY@BIDukpGQAe0 z88yy)^Z?WcAbv;2xf(Na%X}+C9CATBaF?BXmJ7YP%LZgFN%naq^~Un42H>*^-&<&b zep7fzNwIOxA))qRiwG0a-LuyYIfjnNb_a{l>dlJ-ZZ4(;mYH%zWHbk1baQli2oL6@ z1g1Ji8OgaT76Ftpha^x#Bjg)Xnmk7!rIF@#{Jj;35Z$pkRbA-@08OZ9)ZF5ot3AF) zO$HmX4^DqJOJD4;MTDl6cev%3RUCmsglRksFz@$ykH|@N+j$G7RV({bo6DKK%KkK9 z&ueA>rYq2%4efPFwlwePK>Y((BCx6*N%!c6YO$r*PlQ@lOi`~#RqWJTlZ zzQDqLpr-VCVatG6g)-u0EBXt{hLWm51}<8S*DY;SVGP%RXr4m{ z)cC=Z>6Fx1I9Td4dw=T1SIzd5iB{wr6t4XUUXX;Q%0PkohtZi*7#Lu_B<*mhYxPs1 zI`DvNKt{u?qox5&JYf(CQvaH^FJFP)D&rE$RS#e9hi(vo^}<`{N(>qwAUQA@G8Npp zb)lagDrg0p=Jy}Yb(0~X>`ZFuYcRi6J< z?1xt3!63d#gkv)NxNV61euq*Qqs)#^x&PWI?r<(Lk4_m5$!8OzdjXrrSHRXr^XQ3- zV2$J*AICZ~Z|HcZ4wCqQP7ok~Qb&PWT!4gBvCfcqflbMOvUqRhcX~8`OX*>HChAf2 zZ{3BzlEvINw=_LXpGgbat*>d7BMh`j@!n};NDP+PHVIV7E!_q$W-5_`Uh>)ctj=Ak zCEnnCHu2u?dOnuOYkcmQSPVW zNqM_xZ6t$Y7MsJGpg2V&mS|2)L#REYNBX@WgHbj6`+#6F`_Njc1BS6J4O+>@ln|__ z&YkRIELm)=Az4p2Scvklt*$*M>+80ENK1>O8w*CMmnaSby75N?_q?-fd4-SQ4PJoY zCNa!$`1W{4CUel#8A?+eXKwd~kz0U_p*S6qilmVPQZH{Bk|w_fh;bw24uBnkWpR-H7>i7Ku=Xz8R(ENtH9i0Pis z^sg8WVx}Hvp5hrmM`Pse)zwv)_*sCBBk5dDSrP}C9#r@fND=;Jm7J{C0*>|ALGZ!J zotXk$(_6~d@vE!mCot|mkjJSLx;2L|KTT?Gc5^rzusH!n>+rZsTBb!FFa>2~e?Y7- zMf&LQ3qA~W^XK=D|45-&@bea^96x$(M2!IZ+|YO-it=m^XxECy8~>5%h=#?X1q5v7 zwzA}u>kj}l0*$yY25nAK>$UZ^%fJl@6F-tQV?2*q(kP0F^;8ewaDR)2hjSdzkxYK& z0Cl|dl!&}7MB3)={vig}kPlN=xU}@XLh@vis;e;d%Zp69Y&%*|R{dOS*^UR4k12Oc2x(RCzVi0VBD@4B9I+vObK{P>eFaJ~>7Vr~sc z=;#eeP>XrMK~bL3DSA1ZQdDx~hXdzI0u@e#XoBQ!n|LT%_bljM-t`)~s)X%}&@o}O&J-t$~@ymvzuwJ6}z zu1dD~k?jPa(YTW;2$WFi7m;4I19=tt4DI!Iu~yj*aa8T< zmz7w8bm$0devp6sr?E`G5}8^U<$;Ec6w8Wd+nN{B$HVrcCRi=vHrFq46es)p85iLp z?4?#y?e0Fmby- zz+0N%nI&nC_Qj~jsTeNWSyvH>&9L~Q{HipUkXrZFTt9z$-etsvv&(g<7PKV_+7=^MQAUV&$l@sC{utR4tNjFV1 zA-<#Wjd4f&9)Ni z+4qNd?sE&XQJrk*^xWZrZaQg(IoYDI@2{(~CpeaM4fW6J#~j-$tfe(?1V3Wlt6L+3 zn%Y0qS8-*NFbTu$BJGO_4qj2T<99^UgmUHyuVWySYbEEJlNoEecFde@$&HWFJsDT4 z%^`&(^=jg(##-074!GhzFKQ}fw5USI>Hr3r0;;5g#L!9ZnNy+~CRM8o6 ztCX}&>=kK)(UvI%sn6@x%adv^CnAgxSjpM6KcmG;th#a%h?BVV^~z;4mL?W*V)u?l z5>nDW@~w1)ejqON5VZ3M^r4{*EDWcoVhb~;(*z8$cJd9CpkMyMp86chGzf=2g4-a)`>#tktZ|I{ z$>4o}`htm$ZiSJK&S8eOjB4Uva!h_s zY(PxITXmpsXl$(DyD;!}bjcg4=Qm^4?I@JtF*fbiyI@!S377KAgn3@^ zYx-g9ZPRr{l(uK$9)@TX#P_cVmU4F@)>YsQA4W7L>*B7Y~WsJC`Pgy0+fBKrDT7 z$z~L46^g|PDNz|`6{n!6PKND~3BwG6tU`Dt?&zFH+sM9PZN7%@C$G-X$SE!+O9>Bi z!sF-fkkQM@jrOL~g@i@rsVFi~^t4U6!!ejmPp;f+Nl9%!wx7-!Q;x@*A;nc%B$!P5 z13(J2HRgFuh3pNAi6luplDm+@irpWy>ZOE8H(Rr3#Pb @W$+%@x@a4rHj3&|xh$ZAR#K_S~ zCC5nY^*hvX($nx0B2n5Aixavci6rDx2&798O~aJj3BswgD`vvNNPih0jVR)^;zds! z$D!FN@MX&tN+^jOlEcKL!ZhzIV=Atgl!a;(1WJV6(~XQpbUHW;rh8A6%n{QSiV)E% z2pn9(h0zgOLGcK6l&!b%hQd% z85#`Pn*%h&s|h%FNj(qCot6GH=SUQ>$cZa4G{UMZH>|4>ad+Epl3F892Jh<R=LaXw6;zz9|4R>fZ(=v*`6QdgeOgfJ$RoNwf1lNA z@51Es!iuZNR&sRKYnO1pq^sLY>`OeK%Jv#Zgu(rp8Jo-2IB%k3guRZ-?izsm==Li6 z0^0|$U3j)et!R1lhYT|9A~Imuhcb=7Cc$>1gfuUd?A#FbLg1JyYm{6EDf%c2q6Gd@ zcK6W%wIF?U z)r{xa0XNIXZ2mhJ7ualH@LJe=?$o!88*~h4tcw?~8yGe46GQZl*qcj6U&<)wYR|Pb z{@&@reZY|FP^F7_3s82};wv9G?`t9Tw=VA2k=>|$9^nlX?7cwMVxM3(KQ?J|UIRZk zO0n~?bdQ5>pXR~y+?u`xv(8ag7NXrynfB%z#nGbdAAt?+Nj ztvxG#maysejwG&jp%yBuLswm%6&Hjv%u^O>Gy(t0IU6|=Tir$r_L(6H9|QqzZ~z+TX1Gm{|Kk{ zD7PQ?)3=cN#SZc(BUCKP(>{p;gq7`2jj6R*aV$gi79#+iF9Cv#dTF52ur1 zKNl1puyc=08C_RUQv-=~@+grdIvQVMcEGSI&$qPBkN-H)wK|z?R6dSB1q2w`7#{Ji z<($qv@H_HTQ_07tw-ks#-6W4wg!s@c-ac1vJtZ+0KnU0fO^muNvJC`)9HC{>ymr2I zh))sB@n<~re5{|v*+h|+>+)u6c-)W?Mo=#bkF?lEt ze78z#Np1&rT)Le*CcAni?W=KM+Q-@K01gs`aoThw|p{T4OkYcDLDL3jpSsGGf_z_fh_A_4D77!i)De;IP5{FW%lM z$hIhr(#?~$ZQHiZleTT!cIHXjw(XO)ZQFKdRd;k(-LANCqaJSThyAeM*4#1Iod5X7 z7~UbWho_avEVbbDNg-lYT4}X5`d3Q1Ehty!sDe#Bc~6A)N&(G=VUTqf+qveYQIqJQ zI(xf}ILC(XTVa5MvP~>-a=57XG8ls%Bz*;C?IPel#vl=j}Vt znc&V&Elwe_LAHGQ1~1NN(%i2?I3ePiY8DRLm4Ey%X;+ zo`Dnm^W8kUHuW2hBwh?LrNQ$GPGZSs_u?gMrd9Q+c8AxsXy;KaTc|sc3>fgH z2d?+olJV9a!*4jT*T2Deirj~d0Z1lg;*>EoL3n1A_+C&ZtA=a~GCF~-KuL`HH8XBg zQKS+bB;ue?tduot;HPjr94-{5gA=lNsu2z`(@JC;(7%9KBQVn^-e+*^{4uF%3$!nr z+pKXz*N2AbpgRh(7|OZYfXp~G!jkhimKEQ(kf=jtI`-_8_#~+o2a= z%@h6mI}K?%*dX#zjfW**&k`kSA!IRW>Eau>;gL)Uj3SDT0E-8-EW(s%E%r32M2ohpeIuDvir4VLJEC+E3 zE-rMqo-du`C%~>4(UAT64C!Q7{g3af#F|(zf{JnmZTEn= zX{t?+jdzvVx7;5+6(cm^DEB}{xix7!)5;cTe0yBBq%>dx!9NbR|oHBe#IBU<< zD{4?+X;xWdc`{`sgHrgjP?nZ9AtstvP97>jIW%Oma;9o0BVF+wR&ym$MH74P`a*bfg#ul2Kz71r9t=h$3E@`2B>|4XH8QUgzavLmIvI2ZgEIK|?#QO*szH59?-_~&+j(DDC z91xuvW-VMe?&~2V#HaP0fhi$|QTQk5DS=vA9~jadt4TG&fw zHuRqTzx=__z{IPnX&c~^XgJ(HFs*Wley8D_aXIckEOzf(^IwbW3BLf#ptX=pPHAQmPf<@~z9VUKmW zZCn1>odaEw!b@Z`2Hy&0IoBETeL)@#TN^C~JjQbxh&1J1bOFTf7{0QX9+m_5k;`YJ z#EjAHdU;Iiwh7pVOi%$AeYdG_0@LX|=A|!ejR@D+>-t9-_zZUU(<696Xj+=S%DDG} zS5EX5<4;38rtWaVO{P20s4hfJFWDv9ly2L`J%G)uua57RjDMY}^*SqCJCM(8y)g90S!Mzw9r zT97AAL0_df-m2PlWG1WQFuCAr`k3}$ZNJsrfIK?zh_0|&Q*6bT)m6E|FrHEmg_v!9 z8?*?6Ua4*qlfqvu+Tt3?5S`y8_Nsgay1j}D41KjV2ej)7zukhh42FPe@VYcTE1#df z{Fh=h!2PyQ*;3GwxCHrI7MGlzkVEc+DF2+_A9l>mAvv86s(&<)FCS`J)V8Kz8b%sC zy-I;6o`hI1{o-fvvpE!ue-s9$u+=vaB?^Zy@LuxFIOhq|IW)i7IKh;4m+94o@XVdU zaG23D53E{J27366Jhp+_{oQw4g*HIuj58y64Hw!W9^Y)L(k--X*Gd=khrT`4kfxb4 zvJccmH!tTxCGp%lC@aNg^)(yb;V)M#DV}WEBq&A~51VNyK189I({@blvH61(8B~@D z>iyPHBk%1S;Yt{dV-h>?6F8q5lfshCR2_LGD_3eSkWEb_?g4dwl}+UEVf}!HL)EHY z(W3s}(XQH_$9-F9y}GFOb5O z<0bQT2sY-BY9F?_t&r3fL28o=43|{*=$2tkSwees%E+gwaxMAbl70`)X;UgB<~}yQ z@33O0M;6@>n|7ui64fd;WN>~o`lZwIXd<=~>k4`_IJbBem4Y9LUK3LtS3GZ6Q@S3nmNi^?WTME8qcwK&5?a`!tWKJFJ1gaeQX??a; zv&$zyCiH)0B^a}#w#NkcZrdO{H%MIkfwXKwQOI`hy_TKl<%-XlU7JtiJ2t?Hw+e64 zGbN90LXjqH_G_x9K^_(XBHfdmj8!80Z5YZOqD>To#LBI!t{mig74u*E7NCAkKd`uD zhX7;RNimd~X&u~i2f+Qi`Itp^U||g^VbXXvku~Xp4s%_S)VG)k3A*KcO7a5Lr*$&$ zwHMmUA!K)GJG1Fp(M!N}F7zp>Kp~8Z;V379T{;PPb4^fOx#juS1<6x(G^>{Q#aZa=6d?{bO&gdnC;)UPqB>qs-Ew0q}04 zFe+tB`eANLOuO$%eIx?Sy1!;Y8~v0d$cFdr1CXe*sozB#`VH++kmM``rM1AVq3nST zS3GrIhg)Z(4XNptV#3)FlIH0n7sNfZNS=h5XY@e^A<}XY4a6#kr%klixwV<|qSy zN%S52+Ne|zs;9Mhy8j&Gg`(FXKDNY|Xe%27KJg@)uY1+ic!+Pw!>&i!91-~i>xi)n z_$u$RDR1nr&^0GibRvM!<9v2dzW&`pt4P=Lt4*qhwFkS_?6 zZixyzC&7HQVzIyEG=9p;0yCw)zNWrrfUNm#NB+Z;`)`t|7?@b-|8?r9t>`JOsf^x}lKcb< zd0%I9p}*e7A(~l9U+*tupGK&dDsLi8%B0;k6VttfYT-W@qoDDYl$_k`S75p7B~4Fc zmt}7#n^5L!X7Vv?9u-SHE6il-#K6&lor-%>FLOQl*`49pbNzw4&B089tiUXDyv(w& zyr9C@*;4&=h0+&zNSTBZ!Q-)1sayda!I`Aoe>Uo1kt8c`t7NHOpI(tuL&kZe%U_IR6b4Vkf2F5lU!q2>s-URlbDSsC$3$x|`Xs+cL^8SSCS zC0|s^#M)p2bc>&oU$}s2!P68+S>&Q{sD!d8h)2-eDdK64$HV__o(uGVk5j0V2P-e5 zq=PD=C|DkbS;RJL^43u~T+i9ijaG>$4VeRP;il`u>CegWCWZ?!7hI~yneAsZ2b*;l zj3HlwW7P~CH7cu*pB2)g##5z1RtErwt=FaDpUXx6NU!rXW#Cr?O+G{ zJpBF&0Sy8r0o|T4)bkYF8z`TLc^PJD1`swJPfIxWvFfw*{H)rf)?~#1T?Jz!ZrPi#S z#D4z9Co35RUyXYotcDemdQ&^ZAeXvyBJuG8Z@FIIo19UV2+c3bw7Fr?W@S~-YF_PDBHORZseuK-IW;h9OYANZuX@zl_&yaH*r;w*N=xJ7e!rer1 z7|C9^y<>vW(9obZVC$*8qPX_3lcM?Qu`vwqUSro3JL83tiD5p50ext@)H#vKE`_8g za2&@xdiNjN$Lh)edq=}?b{NY^JYL_#We~`deyDw7HBr#Fl8gt*)}70(jNPENbMw&=REJ~Mc8r- zT81-krT{We(n%@7Rq=F8|sB^e1HGcp>x|& zLpS&u-)dWOGn^55X=ZGdho&^kFf@Hk(QfD+2qmBmMW=vS2*E7zpnc01{$vi+7_gh- z*ZdBq<$a?rerHULqJouQy}mgQ?q!1}tTfS& z@O+*)P3sHi4{-&x);-y2JP)AD79gse`n%AF@o3~M-QmRQhrMb1l;cU)7|jao$#elx zQM@}<)~qyqYG~U*Raqr-!<`VjMNoNk9xmUO32-Tn*KSgtvCde)Mn#5}8cbMfe8 zcZpQ9357N|jg-FvCw0Ur_;5^~ju?x&uL>=a#!^k?lvZ|b$Qy_Wo>^p*hewGqj6dR*bkCuI->+u;=I5jVHny6nR$1^{6%EUz&m=^ee9efdi0)k!~?=*=iCW0WT!oL>VIl zOYm*NxOK@RWJ^d`VbQ|B>cUf*0`(9m2Da4#FdR*x8|GJQO1Km5rY68mDUMHIq(++> zjwbddee|AYv7l_Zb1Ztwn5}K|=kdoC9Xf&#nZh<|fx!@B!}5e#Hp?+EZZ*I-!Mb7Q>s#c*q5o37wmnnrOomS5-l0 z9{00KZ=VF>ak?WW3%KR<7-^&`i+=$ojQ>y!BbwcOTk#syh83?|UBmY)~= zrvgckNC|5YpMELB?zapSb2Ga>7tRm@(OWmHBN#b9kcT#vWB=lw(<#$xg{lBmC2J&X ze8LtGkG+0LsxIx6u5YUQ$VOG*FAHB9fU5Z&iI zj9u+_U)NG5*aOS2Lu=-f2TUQb5cAZK@aCwxsm_p#l$Fx=F~I6sqbCE{D$bIC_+Eu3 z{uF^4MgMsjWCuI$~@!%M+GfdQGmobT37Q0b9Z!f9qpNc&V} zx&jj3+CZm0>3?bVF1^D;vZnIb)6MS#NAWOmH_;>kI2e__p)i}nuB$Yus3|v;0MJvk zF~s7=Kkb)BTjHLOTpX0|Sg%y9q2LCH`ARP!E>AE$jnjOA?g50DkcGZKUWnMQ>LYz0 zR+Vg6ia@$_s&tye(m-VRO%}1aC4_R|>PU5Pc-ax~!0@6@iMX+k`p-}05FW>T(>io2 zRI3aNPruO=K1oYfrFhpriNdk^K*}QuLiZ|}Z1eKEtcV2_Y_Mt5Vc(}kybWu!xa)oF zND{`lo5R$3zZVM3mU^hDwpkwr5RYf&;qTpbd9at>Rp* zKxingY#@f4Lqng@L?BBM$q{d$NR-d^L_>&k_QVg|yt>@HrtP8E7AKVTMLX1*y^xa8 z2*i0hx{iQXM*dVP4%8qiqvDyNm&o>?*mCC$q|vvp9`rSIzi`HmYEGq3HO$)j04Ggs zs>&6$u=(Rws7H#S=5Oq@vp??9V`K9@k^T{z%m`*aNM^h^ zs%OoX1z~j9KZy+G#S>9FZV|S0w$3M$YE}YRExiXZ2iNT4dFTG@_=XkBgL+bW9PNDU zM$-eaqOPGFj55&~_kTx#D(c+wO^&A3N*ldY2y**)3-L&6VOTHGk$3s!ruPisAG`!qeb3L0d9KbIEQQ(1r;uWaBzc6AAS)_rxJAdW&X%NGXeH3{oH(6JW zwKt|>tDek21XRo*IkSHdBwM9=l(fKlVM!ti*k7>cZ$SJczXWmr3 zZNO-E#*6@qGVwGua@ac{{)13b3F;Z$y*MLVoL7^0OVu!>Rzly#^+nQ_etyv2BX=cN zT|QT*+vp!Z!K=b0`uNvH%t_a-o?@_W>U}pwR;D zP@<%=GG=#3&Bh#UwZ4tq@@Ny_$qQWg1uMs!%djg>Dm&akVI99@V6M9?dOZ^aPCyh%^c1uoSbIiFfee%UlZY;lgp5h40znkYM1g- zCCAI%{Z~A=6261>`0|+TK)zQwPl}vitjV|T^@guooTUIitw6d8o0*XyPwxEqrVj7)W!j#I7T<$ zfRR#}*m&hGUxtA^1wavo_;rdF?Fh^_)`x<9dXg@ILDg=eu_Yt3W~uEX3Zj4uPjbtP zoUi%r?FW7L`pze1!P{{Hl4f`=p!W45j=h{1J566B+M{Z?LDCHSWF<)-vYKUJ2?$02 zQr1|mxSSkT&S07_#J1%n2bAZ|l!QroIMq-Fw9iA51FLHqr{xN21+ld|s2wB2z-kC0 zgeAn0${Bihl@dQYEtZ%)!nZpI1&rT&mS0dN^yafDuVw_m2iVC`A4ED|(>n{nlX40! zqAR0bN#}-8ztrvWzAZaov+M~WnPdp$6_v@A#aH&( zb2m#nRKo0~SwAI-0vn|W7CPjaKOEyqXG5_Zd;Zq15{JcMoH zR0-R2SFTB@xsB5-QtZ#Z{D{waRQuRH5tf;YICq<1 zrvyVy7n@Yw4N_lYdYa6U@V25QNNCw$#$&RKp3P_uF-7*dB&j3J^-P3 z9&y<)FK>J9C2QY5Zo6N?_xnS8g**;&3zTN^Y*m+SY^)IWo-7%b9+lz3FA4E;Hg3$a z#uoKUZZi^A_pUbV7ZyPD!z{dU#E@OW_x`5vFf=(>TwQ!MmJL ziDZBO-5UI`k3l%X9tb~iXZ$`hvVq>QkrA@B1>P$Brc!(sIufQfX0p1G-d?ro-rf<8 zp4@tJCY}Oe0U~w9zL~i>EwR3z-xT^7*qX?hI2h#m*y@OhSlH@-M@Hqrr{t!jY08vl zYDdW_C~D`H>FDT|r6y>W#_1$m<`?A2r>RGOgGa~CMFl6QoJ0M6rvvf${I0)BVNNuMaNLL@^QNP+WcAd4&KD`#T)gXXt^c1apxeg>lvqUn+x~$~CLlp^k_M!Vtm-2ER|d0LTYt1OpEeKsbgF!wz_} zP;*nHM&B?UNS>>M(VP7rr-FV?rP zE>6$M5#9lvMMg*`t%OFAIk8Gt>|VuclrUMmHks0-Sm!v%tgOO&T$Tf7Vre-_7p)s) z6Jm(F4U6^y;4MXP)xqOa7fP+J%yoqD32sFKk`-J5}cLQcsWPkE*f1Ixl zb?J0!!ZdlwkjwH|rO}D5d>kk;6k!mXrB_OFW7%gSk9leq_>TmS3@oW{L70*-1){$V zOYIYp{)y2M$N$4Zo3=nbmiA-5Ejp6_r+L-_;U78Zk>O6R78FfUZ$Fx`hHOSiB`godsknNq+S>qTmJZGhdL*Ub!E=CQi9?k?NfPyvCZ_I^7Cw?P#v~f%UD^ zj>>R2^J1wbqi}u8CbnaK^ZugpQe{>9^42~!FRL)=%HZ8ReObQ<@W7y!wcVrs_n%CX>He;sbw5zRdjDfE_*mYT zI(k=ARAl-N1*sOWeHk}^sa=5eEOgzTqm9fCDoVey4f*!S>-eT^y=a}haccYynApJ* z)n<0K;9o=%aU{3!2^pP4I5L9)fZP}xdMm6G`*2wXi17*pArfQB<`uJ~%m9aWPpQko zxr7^H`+iFWd66~YQKh^I(e}D{I4Q}0XyrgKkQCvSm7edo>*s8FTJ0m5$$LvQ>Qlx; zvd*$FqIk+%Da;r-)v-$QAX#g0&&~(ohxav|GSLV2JVy2}e?20=pL$o$c9}F2lrr{; zA)>cdnRssgvbLr%#j9bvnuC(Xni1H`(sK$IkqXWIT7g`L?#I((>wF-&imm)J>G<%= zVZRpr+JMhdXmi$%dEsU~D>;MchMs!^#`QaDqfjhyYo$u*w>OeRdL>-06DtF1;sE$} z=79$6Wp5%E4i2clfCl-_EMYr86ibsn{A35wS+@H#hKI@QnVS@`xk`H&Lv9 zLI@A@NL|SAF>;|;))Sl<3~uPsOzM2oQh=s9$BkIFENjlk$F9h86&lxV_Rw$6KcB^C zZbVu}xYBK*mv*sorn1L+ui-4Zwx+^8Tw>H;*qLc@a%B0uBfFyorq~?E>_gdOf-^tdzY|(Xti|$^50OwUdkB(s9lgn1tbSck|W1) z5u(ki8rQdaYOdZM5c?7LYScexw1S?RwEZTR^W%gUS!Nu@TrAbf=(r(ID+04Xc8)^Q zLQ<-EehjMHWovS6nFr%K8Ln1tDm#6luZM}!s1r=RGXcdl*j789wNh6%6aG(_VVHB6{GOcC)m+*lC^sUx>&-O9 z;TA}#+zB~o1V)=*v?vj4zuC{|HlSx8-*YR2Jl9@1J($F1x)CT;=i=bIr=6gd2`gn* zFNd3)3k;>Z86>LAU2A(z5jvT31r97MnXyjQn8SQ(CIm+CjJw55@EVFe*O9-Da4;=p zLzR#0AaODOtPB_sCu`h8ScwgbhuNSI+h|+_500E)w%EqYt{4ANE6mM$O$-uNh$133 zk+AJzIn`4-0TI{Zrx;iR5e_8f?1|{yhGpQ+Hb`EZDPA$z!titw$wg)!z^MmE^hee& zX)Tc<&mYC4VcRm*A|nW(X}mZziGnhHIZ!jtIJRV2ryHy{fUB zfrO`?j-8*+<-cmguRP#wfHa8{KBMn?fr}6kyGNA%`yeJcQ8|wYdrO+!hESZ@zGQ!@ zSEv<%5uow|wtcgA19qOc+;N-iD@VJs9@-?<)a?OKP!VvvF`l48o+l%h8# z`e{^T0SLI&`p;Wq3D1M~_-AY?+Un`H9Bcy5kVz!pn&9V8Qr6 z?^!e}03KYU_MrQ4(;d*vP|_ER=Q19&G07Z$s*`+&O$U{J!5iQUBE_KLN>6tDI{u>M zUr2IP2k@gp<(xcBNd#Bg^J(y~!1>jnwnAkB$kCbLk~Z^lG2Y!aRm*6yA6l61z_s62 z~5cCN6~-`BmTa*?vz1yZ@7XM$sd_}HBi#xRr+GrXw(BBTomn+<71E9Cp%%6kFIAa3F__z)V7pixXYqQ8DqYj)fIg^Wy_pmpzZ% zp&yo^_OxfEA3VG{MVtjE1(aJ|+?`&7{JmOJ}h3?n(~c{1zv>Dbi| z(qF7{U;{6LUUjxo->X;-)O#`(rjBTk%tm}6K4$bg_a^z+^(8?@NKf_;X?p+_fObs# zFPE~#B8=BFom_Ro2@iJ5eh34 zEjZ-&^{)%Uoq>G6xiX2t1wbGf)7ra_tbHeem z_C~Z%sJe0o{r+lHMX&$yTLck;M1z4CGg6ndI`BZ#E1u|j-KW4o%!gwEBZyOtltL~% zu9`tsZh;1+yXb_5OAqt>UEStfypV;P*UjIkX`Rn1I5t{A@sGQ2-O}=aQigMs^`xm8gH$v#K zNF3;n9=r2l(0W$)R^0!nE_;h?TASPD;^LL)T$Oga)gFaUXTpRMP8`eMea<1HR$9iu zIZ-x2HLgi43r8cn6Du|UuJi0}L_>}jcd4B9MdUa_iWPv#B%6sKUPj~#BrXbE@pd+R z7an-eOF}j?5>5Zk&E&44sFd#Pz5LY^7W1G)Rdw*{fffsH!iy8afPjJsQ~{rsk5^Y2 z65~%&TuxJlfXfRd*Tm}y18#)l*_mWhFvkp_G*x}@C{`95jOF#fTAp6H4=yod!z{$p z;8_7vo5;=~8pddD(X~L6L4k-XN8A)!o0Q;27s6yo&NheLk(SR0BkXvyOKa6NwBDZW)5q;zW)^nP0j)lml%wJs$>n-pc{+@7-vt`zh5b~Q$Vyoe-~hXl|mfOAC0vX4RLKLLwV4UhLajo zuCKs)wgR@KJ?*u&PNr@t0Eec1r0Mz8_UaFR%y}x&TGF)*D&9+RU!)Zsvu?MwrDxb% z69xEr#xoo>W1+4oGxr1Rar&REQ-n=1zH*v%NmJY*5}roJ4P>Lc!pS zC%M6b@`mTT38CZn1tW!J@S-lkpI4fpU;A-Na~6aifM~!OhCv@!K5w$kD&9IQooig&1-*D_A!{1PD`RMIDK*GAtNBm?^y3F`Aqwu=O;&1(R zJ#h>pCKq-aV39t!GDlVwIlcn0*iSS)g$a=x8DpG%jPHnN7BS=w%%)UyTl#nSgD z`6)H#Xommm>zNR#r&Zk*Vb=8?>SNcvyT%sA|c(jck zj)x9^_gID2)IUn%Mk0ECkzq0)Lc^+A&iYQ&`Vn>k5V*Xqna^u&*zo!|%->9I?l)+) zvH1%aLYwso&Dt~G&>8!-e?Xm$mi0#%Y3PkfLRCoa$4DQl_8YVfBZ~gOoiWJ0Dkclt z!;|wg4*1)Ip_QjG`0N`?1?=9rIuC$Z1Z|{~@y3!az@{`4fq~M-+LzNCEr0M5+q7|# z5R}(fTQJ(!nwf(c-3v^Jgb^!D#Yjm;Qzm!u9WN!f+1pUQ)ML zpedmXRYg07!LXo`Ix&HXi5dRtjs!WWuDYs1vMsF+5f%Jko{oi%rLP4ZBXFfds1~3F zHL5~G#@w%p#*)5c8Stb3bFU^o-XHehz=+E-XL*C6iISy);YoN^`(fcxjgBb3vDZJJ z(4t2xGgNw_d=h1CILxP6Xq0{WrUk_EfwGZKG8x4I3O3Je$yw|C=w}DO1 zk}`ai&Fk*y`#G|O=84HwcSLg+im@50Z+`=&>||p6r>W1saBTmdJ)jJXtpAojCzIeU zt!RoobX}yYI*`AR(MQ^%eFRFrggVst@~bHlPhPms3{|*tMm`9Yx~fP+G^r*T={z%{ zzb=_GRRg^?DT0V`7&%Km)3Nmwd?GC@qy7b<`8io?!X}OM{a9kkGyb0UIOTo^JtcCN zo!7A6#2ZrH*V@po@#C z)VBUFRBUf6yKL$cyHUMxnRIe4C2bu#YqgOqQQ`lL;G7qM}K*$(V0XG=nfY5 z_@qIP0;4XCk~nGiWZANEHB|#f3+VX9J?>}$ya&wC$zh|7!4*bosBx335Ci=cYHC0v z{RRfa$iL)P6wTOBPQ*?!o^r(p6?tZ zv6;jhb$J*W@N_uAWZLUs!p_##P3RXpcIsadT_ePR1;0oSy^pj_NRe&@K-ha_mGCNg zgmHr3`LqP{ZGVq24ES$@>+j4D%A8cfd0!5hXU7WB@q3CdjM>`xuWN%I8Wf{6-D3R^ z^FrzzgN+Q<0e6h_8QL@WqE>@+p^Pn9S}=)!nlrfl#o@jd!8y7A=Cu?EAck0kG`7^^ zQSizOaaOK032C&}n_y7;F9hCICo+u&9y!i{^)Ng0S#{m+;=Cm3-K7aI!m%=I;;z#6 z1GNIH>hIX)K*@Z+iZeC#`3<6YG9xMI;S~O*T-UJl3bDb0|H-2&;_BL@E{NB}UO-Bf z@c{*kQd(ORCI`+J-!T!(vLzsmg}S>$aVW11+rH`q2{8(5-vEm(fY;?60LRF7&=)kJW5-&Lb#7AAXjC%(W4{@0 z>7Dh-AC)bk5~e@uDaBUAkdc}d$BMO1Ln6H2=;w=AwS)1((q5kFy;MHNqL=Pq%ju4U zV0Hz8{B|Je7=&lP?H-ZLE$&WgK<@+^ozS22{5`UVA=^p+U6uHdn3Y|xsMB(+bgfdW zu;mDZ=L#_Um?c3Q0y#XS&Cy;7*pXLVU%em?lH{!HTpJ57vu1@SK(siCccq0gJfU7T zjbIzPAbMH!gjoaDP0T5|@izDO4to{P566gH$n}cFr6AeHu*M9UCck0yV!lb2pXm0J<$JQPLQ@IZ=)kzh z5+XL~qwIde9UUK+SSxXmn5qa_4c6-3wyNfYnfaPPc$)Prfsr8D?ep}^e!!Fc4h{sT zxdGExeSWLJ(pxwxU3hm~Hu>>H&pVOPIg(Jq**GZRYpKH3Gr*i*o|pJ>7Yz`K{TB)* zVQdfd-vgqr$bLTBVm!u-Q8a|Q2(s@3OT_&Ixvj?auHQ20WzC z4ayf?SI%`A=nJy6la0FL+j|FjW?i$pnR3y>8oq;FCaxHW&OEka&zPX#O}o&^T*0=; z4gL9<>xOR-y##&5|Fnz#H%u$@zr(@*NA9@4bA@o6b9s7vZ2XZskiW0qAJ0sY&f&~M z@ED``jM98Y82%q>M;3mV*CZTa4%9!@4tDJ@|KPZ|g2wXs|6lbDF75wY-@q9}2XM+J z074=n8jr`r@#2%|SoYHGy8B6ciQ?YO+1}h*T$F#-Je$@CFK(P7O|glI54nnX>K?+lyKhH(o;u%UpL}Zf zG#@Usi3w-wIG(ILua3VyslKQ_qdC94IM1Hr$#XTB29CK_;ByI+RbQWPDazRALR#Nc z9E`c!bhZ}lBVb{fElc0T?r~mu8AJWB%jUFRdkz4x>Om^5AGfbdoj%^p9Oz%{@9#=K zCw;#2!Ogyg69HJ4D)#tK|A_wvrDI@Z{I~vHk~mLkMOWnE>mn3i!v52xFnL=#sZ2~9 z#!V;sP|ehM^N0+@(33`F$;dgd!{#O$8ss-WT*!TI5*msS*>{`)?UJbs6HCWKxNbHlyqwUcTuji5pvLq8H-u#-ZfQ zOqx{=jtnRzi@;(@QqvRX!}C*BN5>ZHLbl_BgSl(+!u0YxMiLrI!vh5oj0EiR)6eod zN(#f=W$P5CC{EjX(k`7G?c&l&6?3E4bGErRN>SPCge6H9PIHhhsUoA3e8%8JpQG&0 zC=kkEjd5|7Msa0=Me>Ads6r!_JYow7Zv{s*dGWBYE@1|FL279N@c_Bf(UcMXvDw6J zw*)IkP%g`8Lwju7umq+CH;af+%UYEimJOR8>~7yNN1T9@dm$vvHWIBbR2#9Ss^wb+ zt%uJ9zn;8`a0kC}W!07zyXZFb-`RI-G6XLgVqC*F@W>h;P3%k~<`F0tRyqu93<#Y>_&WYf`(AjL9r!2q-SYAna}~ zK*lf}bHOZbtixIs%xYOKSoXqdw2WdsioEFo3`Xs)rCV4Je$_-FEigyr#6D!bGlrmb ze3A!h8%9qTX%Sv;?C(-JVu4iQx`H(diC>qy+J6j57mV*oLXznIz}P$1S1y7)jqEa7 zLwrLMMa+Q{Rx}8eqe3dzkrm%Xjn@|Z)l0)fXaiyWeI{dx|?Ye z8J5kX8^-sExx7+BOKy<-_n)S8D8%zZ7_gVc&*1!!SYEPxD1ocCo_;?J`@3P!4CB{9 z_8HN%lv!C+4CFPLU$C%7%P9N8Ux;Jbg#AVU2lEne4fU<$S$oEx=Q;>?dU&6X(&q4F zUE^|pqs-0t?H~w#9ckW=&@pQQNJNXr#`K;ok-YdcNsW!n;TR~bdeMz?}N+-AAJs0za#}r6GtCVIPD`55Ag5E#fCZm_TM|F^`E8h zxAzau8@nNFarw_8gA^S3ka~O=84nZamVN($8ImiQ z42|7G#+BW%1=+HzpN5xAj7qGgI8-^o&K|-o2V3J@`yj*9#$bmaKWm~FoY9D3DIH&4 zOTfv6aUH``=(0nZ+PqU|0xFijCE|CR2I0Oxb&aQulq^z$>YK5CWtFTFA!%47Lmlbl4!jAa?}kF} z!>JeJIZW^gs~KUSqw7EgjG%qs--QE&EdU}${St(xj*7xIX4wP<9r6GL6`gtoNfJpr z#UO#i98HG|NlrfEgcJn@O@@qVD-%f@4R?p^4ky7VYYQjgmyHp$^Kme-kSGcWkr9=U z@K5UYad&UiF|euA@sV%ODvl9!H<1Vt{B2=xA+K%`5uoX#6CR*zU{ek_u;>9l#=+N; z+eq9b{$oTVX(ER^p&kOr1%mk}4S@I$8bE8Q>-_zIX8ZZV;G_CW`}FvF64()w8xH%Yq(W2Gn3Yn$jz z^XxtQdd_OQsr@obyZzF~a{Qh}`W=D(&~qc8Y<|UOSk_>B@G&de^Pc$U-DWlJ?snL! zHgSA!_jGgf36J^<;50Wd`9CZr!+%2#Ffg!0n`S7U z8O;uoH4chByk?*ta@C3WAEnax0r+S_W#l+wUEEIQF!C_SmvWleHGsVtHRrKdmEkos zm-0n(Q>TR`&h0(tH65q5v#BXI$7|Q)OvmZhqxvYpzFKOmxHvO$2J~%l#tH3g`c?*O zh}&5CaxlrnIWtkr{-2asVgcId-%%}3h?X;L9cBhSX3ew{ zOkJ!H3{8$Wv71}=_A+HnD=B*Lmn?L-EH{8>(~d__&@!MVo@9=&8CEUq!^(=i8eWu~ zF6PsXCR?nXQZMuS(hZw6OI4-@D)lEOJcbAx#_U9w}h=oe~n257gR zLv@o-oC*h}E|Rf&c^#eH$^o-CHY^>wzVqemCGV^{WCwQU|CY?ZGrraM!1_Cv#EUAHCmNFkbbT_C zk60Af@f(EWjf_koD)alDzw|v_mx(p=ulfJr?Hz(Vi@GlFv~AmVR@%00{WG)Dwr$&$ zwkmDgwr%^X?vCi_x8sQ(yn}CcM|W_~xo4l)Yp>seY#@$C6sGHrOgP1p73N67ME zlP5wplwI$P^&3q9{}GC3MeE-nJUzdDK@i?mMtBRW#+<=(CWHR7KSVJ^D2M-gvG%k9 zdG!rS&yB|o?$MB4l`g*cIDf7~V!qp85HF-pR4P>%D>~4HE6}eOJq*D5c~%W4p7X3o zF%4#>3H}o%M!)od3_e&JkuTFnwv^C+4WAY;!sWL;L=~NWA3izH7_^9gX8nfrG8x2O z7KY=N0~=|?`g1sN(p)0xQgVM|ZQ8f$JWb}|4|cmJ6b4jBFSw4(X0A=K*)A|=@>_$oY)}z= z`E9PvU3;S0E>OQ=@`K}DagIim5w?H;1WYh={Kwhx_B?mo3_|D7QN!togZ%0?m1+DL)TdN7C{|IJVjuKKUrM!B%7Qh9MyZ+zE@3w!JO~YO-MGQ7lX~mf z9!5`(SZ#_o+dPj^w%*MATGU`TsMC1vEgn?0&EY@1s4+z$I(ZiTP_eOW-l& zJ;Vu?warweo^@+si)@za%*np?rma4UqWJvVWwck$kJI8=b*6v;*zZ7c_-ET@A1dyC zAGcrX6=K?8@5G_COc?ukUOHbbZq$;HGMPj2oQcO?OZlAU#iPrmTiXDADhJVtqh-lv zV(Q`8sxW4gxy(o7{wssS=lK_Zh+W64`_=Ac+jF@-}R9r|X+L*lX^LsXhN3K(=Utfu9DBdj&3X6LjOTSRHg zrPg;rrZSRBnl>gETL2(VJSZWI9l0M#Ot?$@Y6(z$i2MTt19kGBiaPUur<=&e#Qa~X z`ahzas*1&SH!`X%x`VW=S9#9i^0M^Ea`8lKlT|gWv6vDV@*f;wsDMjYviO+aVEW5o zAi?BBWP+s2*R6r(LPi5-~p;^Sg=93bCq}6z7l7hY3b$ z6Q)VFOidUQCN3XCPvkO=h~I3Ywe;(sc4uMmwSd89OomqThR*mil^70wh)^#`r2?Br z;&e!(?D!LeHfpwJ9HYsg!br+lA;n*=4h3%PELDaIg}xV4Dixwgj$XNRnuAs;l@Q6e z0M#fJlZ`e3hGOHY$)`P&Fv0dmm|WsRxhY7Ld~6A)S&&>EAo+*c@?rUd!^-h3vrfMm z%O>d_&zVJsVK%huZ6-IAF|}MOMzIw7Vhkm5&A3^(e}%M5GFg~BmszM1V?wmY(l1jL zTFhJRuV`6r0&eWM-FHIILuRd_#(9@=m#;3Wa=>M zH0tD5&40skqsG|`pwwNhR<0t@A#X%n^(WSCX|l9u1r{5B)UnawlMe7^g-|p5%dzB| zQDJjhkk42}xN{d;xISKz$}&8FRSK}M-}83kAo4L?1n4-yIZ>P(&vaomQ(FDrKKAH4 z9VzlEqQ<-(;7htqkb+173HvqEINc}lFV6KdY4SwV%@Dzf@4|ySOp4X-r9NEk73Sfl z3hf>F*yP3FCt6*=cuSm;=)sL}r$Jp1n9>_z3vnVQl(@^!syq*vPX(9jv7?8K^`>FB zLuG6dTcm{Gnbisi($_8hNjYR2G*n1WEXECvp@;#AwuwiUO<%QyIBM`&vNOVrW+!nQWAkYv@%{~sec0hupOpPg{$QT-xZP*2 zlknrI}k4b*QhZnzzpQ1cp1}AG}bqSQ)#fgoLDM z4<9u;D9cE9(g>>sbGY8l4-(Vy)#J_E*Wf$h?z{W0z7rTY&gX#kb2AFhj&(UL3ihVI1X&mLan?K zhUT()EhVHE>=k075Y86dli0<%~&hJX=ZdYknjjhYDIk-)A>N81~EFmbapyR9*UWixvLKls?ik66$oX6-W(D_7SLz#de5u^x_tL0W_0+Re1@_r7=HpZi(q`0ht`$ND!_euHMXu11QRPrx)})h zR7E7fCgO?G=4v~n=x;~Rdf)7cO&eSm)klv_uh_97fo7Vh12wa2`L7hqh7nM$EywIk@6<;p!hd zC%-)k0w)f#sbJNP#?znyX`Ca3x$nyUAWCds@NE#nq=tByotGYWPBD>KJN`?FBb4PE zz0b0qz@#a-rc`q-W;qiQ}uw$|>sBICLJR@6Rr)kgI z3x?f&zaia~d&QzO$r25^#!E!VIS8N)@Df=8rS#mp8S%=PQSHylG4fz(oHQdRtQg=e4e%7@} zlTCv|{o`lEui;>E(9mE53Tse-5N|!A(>G9GtX9j#W<>8Aa(DBz_{M?z%PyOEH0Sl! z#lK}KLy2oGzTT&6Qs3NOTiPM{f=xiwhYbOp56*MT#%jg_`4%IkUu)W`YwgMBbde~$ z+KtDCS5f_%#$|PqftIQd9e>@>&P(Fd)D?i~MmRbf+3=RpPPdlyN=rwZ#}zSNq9!kJ z8ZUMFOD`2@m;5gd+RAujMTJ#dOFOXpoxDq_bdCBLi#T~&>w#e_^&%?=8t=xB%E`#~ z!HFTgZ(K-SAtFMrn(ruV- z>h0bXzaMD}MC$|}Lv3ZvCTwhg349U$udD{$OT-+AG(jhiu10LXsG)E0BEOv6nY&cK zs>d|vESx972tw|%%{snDyFl1am#M4o&``z74m^vW-$Gh>z0ghNgSH;gEjtZLfS zy+@(X$OKr{ZJYe_%RAe6P$#`wm3f&zws5~goXD9neX^8^4iU-EVPwm2k{8>Qkc7n4 zw!~Uf40iw>m-vu2uh3dAz%@{#vLxlA<*5`suZ-s&#$84|Hr4nVqMkJ8g7CdTa(nW| zF4)=Jo@>U*O#=GE#-*-Nt?cTBwEvY?N4+3_94q3vUMZKI6EY`c~Q>s z38oQ`ZsQrWkut|c#!t8WLewM;!nMb_mUdO|{nyGHCQ=vT6&a`#M}cjb+pPO#MFi}e z%;1f+6*meLtri<5ixO!!343D@hj$%E3;&`@mEckG;i6Z#Ykv7POA8AV80TG4u){Xr zl^UX0uwfOhyp-qz*O&Gx{**H|=?gpLw@QXNuJKX@f7IksZg;P4Gbu&6&4H*L0Be;OwntqSUh-UUn9|82Dx(pr`W_po4iMif>L%cm?)|Tl;oDCCv$z+q(kj(iY z&;3yqx1Yw71!6nMY1mMeKYQfc3P@~r;M}^QU_v}g^X`#j9fx8_tJUIg+F<+q^xV=8 z%-u&XA_5Z5+;@Aw(1*g8VQSC_?E=2F<*&=>`Ky=7!JhJE4lx2r)EWD`k*LfSf(c3d zfmY6X(P(h}chrt0E_CUXC8Lw2D7f~ZY2{W-`XzY&SM>o?s-fQz&p9?Qus0o}b3860 z3wR}<@G6-HsQu1C&3`5Zv|OxaEI}q@BTY95d$2MT1QXx#ORu}B&%=WW=17J=QxzSA4U_~v{(bpOOO(9``FU0r)F?e?X1C$67(KA z@9JNaVS+5{kGpe_C%JnC4Ed0x9@i^r?BMX%Bo8zUKI~TjOI`_GkF(b|%ABi|bS^LA z?s}oQfR#}0_AX{l-m)8~n`_Lo8+gABEe5=~^;h(+9Yy)zW1zb>VQijV;TfR0_J*)K zA$J9Igr^+`*~eOXOWi=Js2Y10RIlKUSx#_L22{(Eac-s?I%JR`Xj*KB7FvemNtyOD zgq+r+nE|Y_dLkhN_h$c zOoEj#LCEcu?d_2qAYi43^V>xZm5x)CA=o-trLUsQ}yU!H# zlBbSEZ-Id@fC$k#k@|?EqYA9WdB8xBdA}!6d-ogEU(`oDb(S0wnbk%ql0^*SlNeV8 zSj1Nvz8Hy~&@^a?wxZ6}=DB3k38+S|ezhC0ko;BZmT@p=I-|+DD8)8O#4>ru9gwbo zyj|4yE(CCC10k zh)MOF#_XR;cH$6a%j3qV`#l@sK5a2ZhE#Havp~f4 zTFYxwE%NE5;W{t29RYOg=NoNN)aL937_0HEVtVo^t?sF>#|L^p5S0A-il+*YHjn^A zNbZ&?y~X^Bg(7C{>f|GE^q|g%-S^cu&bh&7yufO(Pf)`6r{e-CoqlkumNu0gtE%P# zKXzq4KU(~M{Pfv=F6m!7|5Qdg$KP(db0NMoLx_m4TAN(R+e1C#d`Q`#MaK}}{VR#hPR68pI^F~RG`e+^1_$7u}+a|H6v{b(I zA)Xw-t7!=@H(Nm1=VOE9h$+lm5+b67{glan6l}TF2*3mqkpY>wWT(WU!o7|me&iL#G z?+o`Ho5GUK?+nVdFSeUM_k+73$_j_3s$_xQYpulKTFnLItHLX(2g)0yWr+F~+k2e; z0S!w~%n79ru3+K?IV4p({?^3~^zD&dxvNwW-*Y&68wTA(;%v@P39X7g^~nV%s#cKP zM@t32Ryl|F{wQk;z>m_E?LO3hI;}KW>ASJ~jC4I~lE^P>bY3|NKVESVBX?{C3k%;q z?rt)%RIsm}U%Msuj2u8mzpv?CrE9jDSIL1z^g(Ac%3}IQFE~;QO#RUKsNaT(Qk=R5 zq1vJ|v zw)?59gdVNJzjU7RGys+0TmqzXytFjgf8r@uCO-4+f*tb{Hcko6hDskA#UOBVvvsG+ zne>3lh$dT$I37BIr$MU2qV2?9LvP9cq%~p^&uFU7&v}*Bfxj1g_rB4oT!M3fiGYBS zi&mhAvG3J^cGp)xZaDT-hIqCg01CG4frPt;yW;sAg+8SYgD;o6`eJ{RT~383FEnCv z4d1&r6bD?s!Di?y1JFGud)@maH*$`1bC-bu!OwI~iRC@7XLVvZ(7W1IFm%U7{! zd+4k1BYP-^b*-72UyqAdVR360 zv(AOjfEZo{a~VRAhCcu~+0DD#waR5R$1Vk#-y))N%A01_b|qzC(TWlWEJFY0nW*ny za#_(1)n}!J;9{GtF|Z~gAts_=7R! z3mRSZxIKREcxE=RUrUGxC_@$aWz7Apk;f`5O?9j7=Vd&@!9Y-_mg82D7LqrZB#7~% z@)kFNfd)+g;_C4Tgu;L*vIFP5fHj%43}JiDF^x?rKQBn>{>XQ6cc${Ld^rh&tFl?^ zTsQ%oaX`?oX{N24Wm(OoK0&T{*k0mV82x!E-J@*^nx2%JP&A$sYXq8z3|Ve7d7uk< zel|IJepqYsHfa39n;tq6fGqu6`=1W;|IQc-E7yPV1pjZ97TOU3>H&87pO2db^lob0 z?`5p6>hu5BXa}|ahen$V6F4gIPovER3ReY3i5~l38Z9~c)&H*=?dX&zQUCv~(UOUi z<{t$APmPww@1I8dy8ZR|@o)pDz`?BlJ$>tQ4tY#?(ftusQtz_N;knG#)THgQ;_CGI zZ1KTx?{U>YS*lT;tlQ=|(S0q0w*&6isQ>ma%4=~}meW}0qKTaLxW+)UgtH>$f%e&K1hR;rUm`om-8liHU}W8^Q>p8%K8oxIngr&rBY-2k^$tpyg#&>$O8| z?%;rv9Rgx49%j^=H^DR^1qsknc#QOaTHOC5Jo*2@uw-TZKONqBigMNiOmJNv8mFtw z;`}fhLtx_VaD?qKZDyCJtJ3D0zl(l_eSR$2o*F?yKR)LEDJ?{Y-1{B8AJmzOWmGqf zH0?_qGkXBQue4%PAE_SKQlP0T?edyAWT^(aAt=3%-p)ZHFSa&g89W&;8TY z$3+$MXEQuH%?z~j0w;9(^TH<5cUB-0YlqTY@A_hLYxa-yv+ebxu=84$Up#*C(emDv zTwmCq@jFOcLmS-x@VQw2TLDH~{{@crzyAdnL^t^7$N$MRV1tBSUH*JXHS+j$LjHd- z4NCtp4f+vDcK#RBAf4uaG7W}x)OIu!lQXic3=OL+3=I#n4HUJKGjf#*^AoDd4FQ&x zHARM|hKA+4=~>#z8QJNjyIGov37J`%Acu!kU?-FoC8?@3fK}7vWHimQi?p<~3-V)( zizCe94a;+?WQ$Z&DzIXVQjC=15~|jtlN72~^5YU3f1i9vikt^0SZNvMC^YW}73HXV ze|ZyQLna_@Cu|cQP*zgXDE0L?gl+?norz~AUyE1EzR$*EuZ+{np{=O2WM_@BACEqD zy-hvcrZTTs-kp~Y9gI(^?3vaNFM)>_8v#dA%9*o_O}+>dweNkrw#DoB?7!zbD1N%+ z?c5s|?H#|)HQ41-vlh1&cq}m>4es{yo@G{jCloBw4vce4-t)qoP!Kf`5nl*u_%^+c zA8vt|Z(hPkfTU)^G5^!r{&%8~SlC$q%eY>#I;<+H22L(96I`;f9(U>QYF3Aq=I&xN zohwjq?KNnHrS2bHp z>$aE*SJ!9PIoCFBZX}9;t7|T==j`k(CrY>OpO5dKjvvCr+h+>N#7<5Q8Vc8o*de94 zjlb28yqZdZ@SnS!IzF#lU_VG>Gf(?k$Upusx`9M#(-`XeH;ud(gx3#M5C0^yV~b!; zsriiehWPlAw2*z-&qU=!nx*qW*j%YOD>VgrgWQ~)E;{`htrAtrFh*UHY5Ap+aD!y4Kj(Coh3;L- z1?YAWX);4@zZg3j#i@@fWUE!_W7R37mtX=ae?T0|S5yXNL_6IYG&19~ljTmq`h9~1 zkR>Ncu1SxPj6}(h#j41ckuM@n!cQV%2F*z_SA~C*8;4^CbG9HeqHaXwh|=A~V+x_@ z_a|5x$G3{UCnX%x*E%z%0Rqaz*{-*aSZ63lPV!WLxh%OAF~0gr^ho=iwXD_7_{IF3 z1kmdA%>9iA#OMr4+!*GU=eM_lFt$|-!S0!0mErL0`yOo0eJ6!gvt!Vz&cTOF;b6xO zAnse~1_j^PO4vAo@_;WKU=;Y}zhJ2b9n1!?R4e6;2n(gu|7}&^BfJ zxz)wPE8Ngg1!;5E&*`qlg+5VY#|_tJNV>T|cBlG3Y_N=kkuFkp#0eCe&;b z6m*hAi#iUUB@{4)nCDg|8*f6BN^7vlFRHYNgkgL;CMap2gHsgYX|Ul|hZlQK1o`Z^ zAqjdp$^?o8$VC-o*o0-4jC-zwMykt#7Wr+(M8|pU;7^_I8@MHuGUVsNlKArcAe#gs zU{{89eZZ(V!+UWL1PmWG?iJv5uI6a!xRZ4^y2}a}!tpn*XrGV%9&auu%GvHdIb#3K zYp7&%mqyW1PL2(rE_1;{xM%lo$Z8f=@zC3`Z=Z0-104F@EhundP4dxPBs`aD2+Sp_ z7L_q@$Xt-?G076*`d;8gc^mlw%sh)L;LrW9LBsSUvta4Jr%pgLILA>c@IyS)t#LQDm^Zf*pn_^H8@U3`h_cu#wTjTbE8w>uj`_w&R z%|iNtZ*57B9{f?%w5`h5$g?1VZHnNcZWmq9&o@Ew+}_kBwWoP zuS{g>a@4+~FR=E+i8kmWtfEz1zG=!KR3J82VF$TxDPVtl&uGM{PZ+dDY2W+m?JV#v zw&IS;IrVjmfBhMnw;zkC)ZukNt0p`o2AjH1g!j58d^nU7C?sJ4Ssk2B>>6#wOGPFJ z-lhxEjKkO4+4_;eK4ass^}2Y}?tFYWZR4g27E=gRAg`e`fR5aQGsNTy`B>l&@7E8P z+kC2o7k<DZ7I}M4^(^>IrIB+Azz2kF zUH|%Xy*gHIsGuY*Zj{nwLpd8oLel?BvQoivifTQp7KEItO1hF*zbHIJ^W8!8H@Y-N zaQKT|oxRUfc&o5&ER~AttOL{U<6dojoAN()1V*ky@VWdx&bNc3v$vi)`^c9PayMb+U-G~T44X-Br=;fyBhZ?nZIYtCM8YFV z;3f_EK!;ZgeE9f`Img>Vd*tr9i0$kJ4hwAGSvywxS;#ifW*6Df&+Fvp<6?(Fgm+_# za$lF6@U^$$2kamrzlGL#R6>UfSZB@sD*GBOE^FnBU}q>*<0%CMlTS$Zi%;1WAVxo! z)ghomqZhOF&4Pph&<*>}Y7Su-aDt^uZY)>+3^wG z+80ecT2(LXG!${VuX(Yut49JJLt0izpSS0+pFQFk0Dij;rAj1)h$zn7E=fI{E&IjIvYMJUs!g=l+n!tO4P|6>*0EP5Q?Z>iOL7^uVP_ID+~` zzq2sVjQA&!6V4g5!;y>hj*lbJw^BY2)+NW9oQQzVyWxTH3@mgQyoljlButG#A54APfj z6yW8WDy{}wCL8YBuZ|?+<0&hMKA)v@hfl0umN-VP=fuFZOy)>N*=-5F2t2i-3!*8S@lvll_UvBRYe_Btt zUWNWOB*t-1PNGjzXLpg_9{aucGL2W_dcaSJssq0aj4k72 ziTVU{{(_xLyqTA|xIUcIw^C93%mVr84MkJ+SLJGDNO3V82TMs={2oTCq zc8wV-{Wr?aDM2J~{|O|A;un-!>U(6nHlU;fJ>Q}}d>(cbFFDgGGqOTvqX7EeY`wXy z1HOYbk5L^~SNa8$lagp3{1H$$1*)U3qptxeC*avLm}tXTLvw+&En)oWV(*dAhBlth zc5o(~!_O{~rK)X>$l+qP&L&vd;NmEyu?gjdF3!;6BMFIf@mhw{R1(|IrPj^h_rh1> z59HR{Rn^lXkG(tMw)tb{L3_;dpRRUn{|&AcJKKL5Nm5gg1E30FH%{$3Y}311cGQ&m}Jgylot&xorH^Hn4E6!!nLh`K)d|1kJ zDZPT^2zb)bpNmqrAWbocbfzx|RLu|rSIg8j6a$orilrVULiC>bLDX-qa?Y=g5-B_GnQ4y< z!AjX1IMi-Q*(JZR5UID2R0aX>UL5aNFy)*w zjX4j0zCDy=PP4IQ*|kqjjIB%=)lM!t*7h%rS=R<`x@X!Z^0abM2g*`1II2-~VtU_8 z2@3W;cIo>r9Cb59QOdEy+pDx@+B)Am-9NcGvguK$QOmB}JUOyl-FbMKMwUP*2T_q)QJKlY z7lo>jmL;Ic#1w89$(GG6ek@`|7Ri=7432|Yv9RZ43QZN7t2j|RS^WzjZOB(wt}I*n z7j{7Z#MK}qinI5`|g5FgzJNiF;v>$cB71T+~` zJ|Zk)06r9#3VvHMn;id9wLQm|{7Y1O8qUa-bv=L(j+g(mOQ;mW3xVIRwZG00iha;z z)UxP@#glo_7Lax4exQADGGH9`#o>K@n~9u=EJph6b(otlvdVrLqU1=bNcv6jeoDbY zj|`n{IQn;?ZOXoG4VuIqlN%9UlJ77JAFgPNc$<`&1Pzfb_5&s%aej1WE%kIy>Es>-@*l-U~YjR~+@_2H|;_zWbGR5)8&+R2djt^2MBDcsd zJE;SOS&7RyzLL^e@jQLe8w1e~#zJOHu^7D^y`BVtdc(Ug+RYZS)5uf*}pH_`=<1##357h zxs-KN6O^h}8z~i$r%Z5#CbpfqWYo25G}le#)N?yZbgPy34p6E~V_Kq@KfiAyGqU)abME0J?{r*N=o{rXiN^!Hb+? zrQodfOo-)c{K+qOuk#yp$G77C;^DIhY{Jpk*%n!RmAu(qe0*A6<{Il!oCV3#=&llK^k6CoapT(_0;>F{a93R z_Cc=+)Z%A~RJnU#&T+kog7c&<;KTuG z<(ELmI#)GZyqc?;T&)-Wf_FHkmB?G}%cz0~+Qq@ zQqtqf5)Lqad{+*Hp>q&04$uR+b~SL&1T*%6`upscd;^KTgJ9zP;?yRvyf943v8oGXv`unnH~cQ8ou>&=^Vr2L?8u#6dGpltwHD3NEE zj5iW6F1mc?LS8~~F#ch@RW>?41?0bu)1mr;#}XKq9n?|vHsFbKV%GXy(7A+`vdD*+ zgLyNrud&?{!z2F$57&Cw3GoT2~MOzdrK2&0~Z&g ztJX`NI}ds{xs31L_EVN0JRs}n75|p2Qk-d3{P~m3Z%P9y?G1ADk&Usb7eEGBWmJfN zeJV_ICptlRc5s!3sJ>Y1JSNF3Le6Q8i-bNNm~-MFjPdopfi@KeM++VtzKkQ{E22_e zR82u!A$a=5Ll8&YBXWm;7ceN}Cst^1T|g|!@&L`T%Q3Jl=!HoEsbEaZ%8wnHUfX@% z`gJwdJaOXJ?eI1t&89qrEmjr+5px>;{0mxLIIvm2(WLwWMCUh}T$vpvDgwilTm>k& zE{rk>vi93W<$mEI*{FrHzx^u*z6uhYQtz)IlK35ZcrY<1*0Rc9K3~~$g}NvcPR?a@HB4?NTYaIc_S=PTcvYQE7qX-4y=GZPz^rFybSU??rEvz1_I6Sy?Ih+(eAY%w z&qoo{CM~uu*PDOW#-{^E3~=c*-QM?C3|#oUo!70tGF`GTZritxbw%01xaeuR88P_9 z&3^wPs0SR&LP2KG5DAd8lcU3AH0-iuxC!R3)OcEp7BK^;Z~}fSftA?G_a@#wgzF@Z zu*L(gAe1`B5t@_CnyRBGp$Tgm4-8jFVXj-%?DX6pxm=^*t?IUtQ>&v_Mw$SrItcor zm;lLR#V$FZHq1UG5q;AN?bX74OKSrjrZzamrus4J>kzt38?&J9xP-Cd4CakuMXNKz zwF@}iX|mkH0r{|Vf^@UU48bUfLIed_D=&+ulrg89jj4`bvv% z@UjdykK@vvH-$1p)m2*I! z#4JLIBRf66cHtAe@!W+171#*w9fZM{SdV|?C7tJA>b?C?EBWGa3;jgY9LG-t(A4_+(h#_&hUZ8w}&5PEYW=(lBMm72tS{4nM z9zI#&`zdWn%RY73U9`j&eoX3K3!gJEzYD*{!@EJX#66j!oULY#wugr{1Hzx znQ%RGXu`C+b*P{2N!JS--P5L5<}wN0?*gNwGtbkgCzADsV=1WrCBj)1V6f1rI8Kw| zqAI57DbQP}ZZSOWbXO%Wr;3PlO*g_f(Ta*ocrtT4YXB5w5!;pcR)-r#!^MMY`Mlnp zhgf;jDeoT>tlB7&071J&F@giu1(ERbI(}NVP!SDU`2Y9;vO=XVeqVDcNRdoCreZw)A z(ueLLzU#(aUU5ZB&fT?K^7>-%8bkdMK2D+`BkpTWnv@@j$4K;AqGT&2!(7$|7rE={ zS$$d`wZUWqA^!W-yThyNHrm5$K_Vq?eKqH=%Y*xafrBV31P$=J%(U>}H=Z?UEKUm% zXaJJ%VAA9FB1+yxiDq{KN8Z1z8{`85OPx%-W^R9FQ1Ogn33*&IM3hJKa`&_L)zUR@ z|8|jhibq{&qe+_@(o#?2_#o6l0L=idjmH!-%JACeS0PUaL{UwCvf z0cTwXOHC%yrZTqSwSwn_jc;)m0H<5&)t_;|f%`$pTBPb3NiBckgXjjjiSRNlKf}}W z;`zA%VOL^pO=GeF1QqZroL#7NO})6?B7D_<&BHqj+L@%;yXc1n#v$$Ug_WN|Cmgh~>@<2C|~mpU=vYVI#OcmqmQU12^EHq@yCngiw$)^)Of}Mr<`v(ny#QZ8QeMnqo z|KwYvvwQGWjY*LCuo7kfi- zlRq5^VfSRn7R24_g$4Aj;A76F!ji;bXjxy(L?ldjaAdw>=HLlik&sZ*IQcOg3d0=! zF;@>+7Fz;-(DgbB&Ngo=*c8=Gb7*9w|;=dj= zd`8sWll3hVz}$?Be6w87A8wi3bZXfwqhBc+2y?^^Va0X%_KRvrQ|MVdpB4zjy*Ev`Yswo-P-3wm;p)KYNtZ>~ zfnd}pF+t!lWL@4oA5I#OIpz-0K}0zBZ)7IFTf6V>>#K6J9S!Szj6O%LvB(ehuh0KH zBPPkpb@6>eK<_SJxPK|B9t$-uY3P23yoQu?RiREgRD1W}m502;bwtZ6yo%5*KX}&q zYV}!mbKW_`?WO-zojH1OpyGROzi5WkXm5()N9qd^6cjF$=tqDo#$}qwZ|t0T?I=VQ z|B1QeT6n;?angZf|3{?zqUG5~^IRv-+t4L#`tx_G#63k2nW^Pl*V5`yoRkK$bEsc= z-}@)PB0d+I=FCBp6~U4$mm5T^17x+1QAZd5g0l z9y9C|J}=*GZy-^WyPM|F+1ci_{1wE)KHzmW9v*pmkLSQ=jH(A7KQ;TbcAHlW|{U=)f;WMA;&hljm&B^Jc=6Mt}XD0*<@W+(UT z**^b6RA{l}YN#Elf|#9}l$sG=8r#tk>X=Cm5>?UzN&$8vPC<@nw%?U^O_ZOqpPs?r z5LpG)e*UBaeqZh$del;?t^8~)eg;V+$eA&0)R__TJeZ%oAXGG#f3PRB3`O}_h=hy= zBV;@op#f?q_|Z5iSbT@BKmR^o&@PO-Bk5D^lvm@ypE+_ABh@0mc#RD~HpvRY64DwD zzByE+RR}`JaUekWZ1e}*9A5i=79qqwD`lQYq; zU!4D*emk2vGK&8;ayAn+GqE!@W0W+X z`MEtP3==?BF)*l23n3uhQVz?WlzH#%gNxI8qC$EMJcgo9RZA^{bB51tEK99nH=5N* z6H2yF2y!5~w~w_NR3#F7CgXmRy06J_+q#4tHjvduOVBRJVH3jqA5jd(90ZjH%bXm{ zdZx;(f3ICc;07Jch^9EBQFb3>AzLbB3>_cOboZ1Pw57Q(0*H*rnLy&b-v!nbYWIk+|F9s{LA!#+)u1<)F{UkS4R2ITlGni~lQOo4P zrXhdWM`|zAQL9DkIq>?yq6v(#n8Q@(z~Cticr+3S?;LYoeuk1DD$roy#%>?2IdXOp zuoZaPe2zGK1Iv5#G15)xa1P#q8B?y&~{*co=>ZkiLJ)Q_v-Dz)};_M0t zJOc#5==LG@A!H`FvW}pMNIshGRHE9RFu^W39cmy6CJ_S&w38?VrBgwHAtrERAV!1e zA2A6;kqIMZNi<%dNFwyvm`hQ(0OW!QB(dDe2-ir=~Bts~5VfE_=L(kW6921wc_8-Cb6`KZx}+c9 ztmqx@ydJ(>gGtJRpaDLLNFIq;j25ay`5DCT&zD0M8i$~e8kXt(^dXSqCt=(wrg0o` z++36c9cd?<2TDbaaO(`v=MtL&1*Vg1WI2h5_n%bO_%c~B*ZjjCuv@1&&4>9`Z;G7b zUWz)DO)kzdXAhcH=L&9uedX1=S@V0{`%0JV8rPV61+nxXMTHe209EiBTs1z zklqfMdG<6vI5$1Ilz5g=QOGnx8Ym^^+uBj@ni;p(mRBeO;%W7T5%|GPiAJMBg2g)d zkO&*qTPFL~8H5(cReM^3se)LWb-fW0Hju08Mkb8(`6vXqB>d~w-aQZ0U;o}Y z(NT{JwavGUXoEcQH-8>owY`{HXw?{yAUSSGb)_Zz&bytZE62!j-R)O6L6MG(60l-? zC_?S};OofT)8W@APjPa2ZS;L*6dO%B8NQKVFJLk-VW9ln?b({W+0*(y^SH5;mUVL( zhk5#^4dps6jS-f3p<{zSd7kmFZ^_*zj6_hI-_U?U_RnKOI-`|hIf2W1XS#uRSI6*` z!tm?>w((oJAB{Q9KoslxeC46Y${%XaIi*0JbDl$$n3HDju<_o0W9O=M43~-^)%V#V z+H)zJc1%U~ZAIV7r4Ls-gKYQard>Bz62VTbCTw$P=@Tw5xMpzTvQT z128E;X*HKkG~Vl4ypOlG7|BmZ<0zhj!Xu@^fY*9xDCke=B_&8M2uL#wU z+9>2MMJ8sN&L%TwEE^c~)Bmrqa}Q@i;p2F3x}51qDAOs-{iN2+u-rT3GD585geJpw za?jj`tcOSvV$zvwCasdlFsw~TV#r;_T*BI1j~S-99OwP(eV_Mz-uJ)X^Lu`O{+{3S ze4gL;_xnn2LK(X$d5YbyB11h-s6x9KMhmam z*q+FfwqE-dJgpG92xZ{Kwh^+fi!(^B^*AeiJAb3@dJA>rErh{w&-W0Y_#EcjqEYmD zjhRzul%Aoyliv?O$xKuEgIg6R8NX&o%TYF{o7@|~jmzm1H?kdyw?$oxvz$TUx)Ia4 zwD1&=ia1WAGNbCm>w`N&?&su@UQX8z3g2igp1LqP<1Nq`H@^Qu8_ zaoar<$dAuIcjTH?$bL^`^?mD7U4K^bh1k$q$giZ?rp_)DjK$RzB#;2Bt+1j8j>E-B zUZNBF)c5X*ASTB+uBM(wJB!b#`ugh-sN+4-%B@EVvtR2BCg98QP?Uh{+nzTK2z_yW z*mo->{rQd9YDPCLI-p6J>sM@_hCr30xh>qLk9n~BW;>?B6d5Pbz2WLVTk~%|o<6{& z%kvI{g%U@zJVudFP%?iZWGfu#O;R@S%kQu8Ly#-9X4KC@#O%R z?qm4o>9FPmBShoOS2FP4FSRc{UUm0e`B7V{wWll4WGY81vo4D8Zk1r13kIh6V~(}~ zjq*#2v;epZ{oduockZ8(qw^OZd0WnfR-rKgyVD<6#0rydoE~`k#RydAuV5l&9mA=} zTS49~f41OPP@Y#?jCYx|6|!`D=<_DCr<8L#bbSL?kg54qVIZ4>Qm-pjbq!K@pm2%d1J!ld`VdT~{ zp8DkRos0f6M0;-3ycWf{5o6FXlI121A!UXI2$}_1Zi5H7sphZizufbCJuY998%!KV z4A`%o{Dg&v%|Ba!BGTT8S2#;t8L;F@II_&W^Lp^!9AQnCz$pcjas}B1lS-d#;XZll zV3ImmF-WO!i8!yc)?YJJ=KZ1UwA!HHe5J%3w6A>tuv9G8cHY@xa(_NI?&eo?&hPh- z^r$GGcG61j;9!ReJyz%H*|y!0gxOA(Jfez9bjp-oJ#{8Wo<&B_PIIT)?ozLXucv|- zRqi7r55HY`>DYjR33IAH?(GCA2B%A)>!aH*#)zvN&hJQCTBC_bg8!` z{rYKVsKUlL+Y1$?sN5T5;r_>AU2j4qUO#c6APyfZJF;Xh(NDlQl3~?faF$6YdJj!- zKSW+YcmYMWCH{y*(zJOR)h~{zzb(Mv%@Rk2j-3T!T9Lw7{Fn^7}#D0W9W-EfSdjQ3KRtC7lV<6>gzy$^MHb2AYIh|03|)$ zf4iX2?`7E!pcPtIzTI5*F(Bol0$t96skoo?A()(8RA*mkRl6=rtfBf){-CVIBpCxf zm?=N&NlrS@>(cB1*ggzKSuK7L+`_I-v7#qhGbRGCwe=*l{9C!|1o<8;+isM1B1Q^Z zj}5%rh`PF5Jd8Gkf}h6nrPzS82%g^Er4m4HVrz*}*R!O`91M7r_YbxuvU1?}f Date: Mon, 16 Nov 2015 17:47:03 +0900 Subject: [PATCH 37/44] Fix process leak of CR clients and FLU1 proxy clients --- test/machi_ap_repair_eqc.erl | 109 ++++++++++++++++++++++------------- 1 file changed, 70 insertions(+), 39 deletions(-) diff --git a/test/machi_ap_repair_eqc.erl b/test/machi_ap_repair_eqc.erl index 85bb697..574e782 100644 --- a/test/machi_ap_repair_eqc.erl +++ b/test/machi_ap_repair_eqc.erl @@ -103,6 +103,7 @@ check() -> check(prop_repair_par(true), current_counterexample()). flu_names, % List of FLU names mgr_names, % List of chain manager names fc_list, % List of FLU1 proxy clients + cr_count, % Count of CR clients cr_list}). % List of CR clients weight(_S, change_partition) -> 20; @@ -110,10 +111,12 @@ weight(_S, _) -> 100. %% append -append_args(#state{cr_list=CRList}=S) -> - [elements(CRList), chunk(), S]. +append_args(#state{cr_count=CRCount}=S) -> + [choose(1, CRCount), chunk(), S]. -append({_SimSelfName, C}, Bin, #state{verbose=V}=S) -> +append(CRIndex, Bin, #state{verbose=V}=S) -> + CRList = cr_list(), + {_SimSelfName, C} = lists:nth(CRIndex, CRList), Prefix = <<"pre">>, Len = byte_size(Bin), Res = (catch machi_cr_client:append_chunk(C, Prefix, Bin, {sec(1), sec(1)})), @@ -196,6 +199,10 @@ num() -> choose(2, 5). %% return(3). +cr_count(Num) -> + Approx = 10, % ad-hoc + (Approx div Num) + 1. + %% Generator for possibly assymmetric partition information partition(FLUNames) -> frequency([{10, return([])}, @@ -273,7 +280,7 @@ prop_repair_par(Verbose) -> end, _ = cleanup(SetupState), %% Process is leaking? This log line can be removed after fix. - ?V("process_count=~w~n", [erlang:system_info(process_count)]), + [?V("process_count=~w~n", [erlang:system_info(process_count)]) || Verbose], pretty_commands( ?MODULE, Cmds, {Seq, Par, Res}, aggregate(with_title(cmds), command_names(Cmds), @@ -289,33 +296,16 @@ prop_repair_par(Verbose) -> %% > eqc_gen:sample(eqc_statem:commands(machi_ap_repair_eqc)). %% but not so helpful. initial_state() -> - #state{cr_list=[a,b,c]}. + #state{cr_count=3, cr_list=[a,b,c]}. initial_state(Num, Seed, Verbose) -> AllListE = all_list_extra(Num), FLUNames = [P#p_srvr.name || {P, _Dir} <- AllListE], MgrNames = [{Name, machi_flu_psup:make_mgr_supname(Name)} || Name <- FLUNames], - Dict = orddict:from_list([{P#p_srvr.name, P} || {P, _Dir} <- AllListE]), - - FCList = [begin - {ok, PCPid} = machi_proxy_flu1_client:start_link(P), - {Name, PCPid} - end || {_, #p_srvr{name=Name}=P} <- Dict], - %% CR clients are pooled, each has "name" which is interpreted "From" - %% side of simulated partition. - CRListCount = 10, % ad-hoc - SimSelfNames = lists:append(lists:duplicate(CRListCount div Num +1, FLUNames)), - CRList = [begin - {ok, C} = machi_cr_client:start_link( - [P || {_, P} <- Dict], - [{use_partition_simulator, true}, - {simulator_self_name, SimSelfName}, - {simulator_members, FLUNames}]), - {SimSelfName, C} - end || SimSelfName <- SimSelfNames], #state{num=Num, seed=Seed, verbose=Verbose, flu_names=FLUNames, mgr_names=MgrNames, - cr_list=CRList, fc_list=FCList}. + cr_count=cr_count(Num), + cr_list=undefined, fc_list=undefined}. setup_chain(Num, Seed, Verbose) -> %% ?V("setup_chain(Num=~w, Seed=~w~nn", [Num, Seed]), @@ -352,28 +342,67 @@ setup_chain(Num, Seed, Verbose) -> {#p_srvr{name=Name, port=Port}, Dir} <- AllListE], [machi_chain_manager1:set_chain_members(MgrName, Dict) || {_, MgrName} <- MgrNames], - State = initial_state(Num, Seed, Verbose), + {CRList, FCList} = setup_pool(AllListE), + State = #state{num=Num, seed=Seed, verbose=Verbose, + flu_names=FLUNames, mgr_names=MgrNames, + cr_count=cr_count(Num), + cr_list=CRList, fc_list=FCList}, %% Don't wait for complete chain. Even partialy completed, the chain %% should work fine. Right? wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, - State#state.fc_list, 20, Verbose), + 20, Verbose), State. +setup_pool(AllListE) -> + Num = length(AllListE), + FLUNames = [P#p_srvr.name || {P, _Dir} <- AllListE], + Dict = orddict:from_list([{P#p_srvr.name, P} || {P, _Dir} <- AllListE]), + + FCList = [begin + {ok, PCPid} = machi_proxy_flu1_client:start_link(P), + {Name, PCPid} + end || {_, #p_srvr{name=Name}=P} <- Dict], + %% CR clients are pooled, each has "name" which is interpreted "From" + %% side of simulated partition. + SimSelfNames = lists:append(lists:duplicate(cr_count(Num), FLUNames)), + CRList = [begin + {ok, C} = machi_cr_client:start_link( + [P || {_, P} <- Dict], + [{use_partition_simulator, true}, + {simulator_self_name, SimSelfName}, + {simulator_members, FLUNames}]), + {SimSelfName, C} + end || SimSelfName <- SimSelfNames], + catch ets:delete(cpool), + ets:new(cpool, [set, protected, named_table, {read_concurrency, true}]), + ets:insert(cpool, {fc_list, FCList}), + ets:insert(cpool, {cr_list, CRList}), + {CRList, FCList}. + +fc_list() -> + [{fc_list, FCList}] = ets:lookup(cpool, fc_list), + FCList. + +cr_list() -> + [{cr_list, CRList}] = ets:lookup(cpool, cr_list), + CRList. + %% Post commands stabilize(0, S) -> {ok, S}; stabilize(_CmdsLen, #state{flu_names=FLUNames, mgr_names=MgrNames, - fc_list=FCList, verbose=Verbose}=S) -> + verbose=Verbose}=S) -> machi_partition_simulator:no_partitions(), wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, - FCList, 100, Verbose), + 100, Verbose), {ok, S}. chain_state_all_ok(FLUNames) -> [{FLUName, {FLUNames, [], []}} || FLUName <- FLUNames]. -confirm_result(#state{cr_list=[{_, C}|_]}=_S) -> +confirm_result(#state{}=_S) -> + [{_, C} | _] = cr_list(), [{written, _Written}, {accpt, Accpt}, {failed, Failed}, {critical, Critical}] = tab_counts(), {OK, Dataloss} = confirm_written(C), @@ -427,8 +456,8 @@ assert_chunk(C, {Off, Len, FileName}=Key, Bin) -> end. cleanup(#state{fc_list=FCList, cr_list=CRList}=_S) -> - [catch machi_proxy_flu1_client:quit(FC) || FC <- FCList], - [catch machi_cr_client:quit(CR) || CR <- CRList], + [begin unlink(FC), catch exit(FC, kill) end || {_, FC} <- FCList], + [begin unlink(CR), catch exit(CR, kill) end || {_, CR} <- CRList], _ = shutdown_hard(). %% Internal utilities @@ -475,15 +504,16 @@ shutdown_hard() -> timer:sleep(100). tick(#state{flu_names=FLUNames, mgr_names=MgrNames, - fc_list=FCList, verbose=Verbose}) -> - tick(FLUNames, MgrNames, FCList, Verbose). + verbose=Verbose}) -> + tick(FLUNames, MgrNames, Verbose). -tick(FLUNames, MgrNames, FCList, Verbose) -> - tick(FLUNames, MgrNames, 2, 100, FCList, Verbose). +tick(FLUNames, MgrNames, Verbose) -> + tick(FLUNames, MgrNames, 2, 100, Verbose). -tick(FLUNames, MgrNames, Iter, SleepMax, FCList, Verbose) -> +tick(FLUNames, MgrNames, Iter, SleepMax, Verbose) -> TickFun = tick_fun(FLUNames, MgrNames, self()), TickFun(Iter, 0, SleepMax), + FCList = fc_list(), [?V("## Chain state after tick()=~w~n", [chain_state(FCList)]) || Verbose]. tick_fun(FLUNames, MgrNames, Parent) -> @@ -516,11 +546,12 @@ tick_fun(FLUNames, MgrNames, Parent) -> end || {ThePid, M_name} <- Pids] end. -wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, Verbose) -> - wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, 20, Verbose). +wait_until_stable(ExpectedChainState, FLUNames, MgrNames, Verbose) -> + wait_until_stable(ExpectedChainState, FLUNames, MgrNames, 20, Verbose). -wait_until_stable(ExpectedChainState, FLUNames, MgrNames, FCList, Retries, Verbose) -> +wait_until_stable(ExpectedChainState, FLUNames, MgrNames, Retries, Verbose) -> TickFun = tick_fun(FLUNames, MgrNames, self()), + FCList = fc_list(), wait_until_stable1(ExpectedChainState, TickFun, FCList, Retries, Verbose). wait_until_stable1(_ExpectedChainState, _TickFun, FCList, 0, _Verbose) -> -- 2.45.2 From ad419ada50177264049c8d09bdb63bfa3fa425e2 Mon Sep 17 00:00:00 2001 From: Shunichi Shinohara Date: Tue, 17 Nov 2015 12:55:12 +0900 Subject: [PATCH 38/44] Refactoring, cosmetics, comments --- test/machi_ap_repair_eqc.erl | 165 +++++++++++++++++------------------ 1 file changed, 80 insertions(+), 85 deletions(-) diff --git a/test/machi_ap_repair_eqc.erl b/test/machi_ap_repair_eqc.erl index 574e782..e9d576e 100644 --- a/test/machi_ap_repair_eqc.erl +++ b/test/machi_ap_repair_eqc.erl @@ -47,6 +47,8 @@ %% - Operations other than append, write, trim %% - Use checksum instead of binary to save memory %% - More variety for partitioning pattern: non-constant failure +%% - Stop and restart +%% - Suspend and resume of some erlang processes -module(machi_ap_repair_eqc). @@ -60,6 +62,16 @@ -include_lib("eqc/include/eqc_statem.hrl"). -include_lib("eunit/include/eunit.hrl"). +-record(target, {verbose=false, + flu_names, + mgr_names}). + +-record(state, {num, + verbose=false, + flu_names, + mgr_names, + cr_count}). + %% ETS table names -define(WRITTEN_TAB, written). % Successfully written data -define(ACCPT_TAB, accpt). % Errors with no harm, e.g. timeout @@ -91,25 +103,12 @@ prop_repair_par_test_() -> eqc:quickcheck(eqc:testing_time( PropTO, ?QC_OUT(noshrink(prop_repair_par(Verbose))))))}]}. -%% SHELL HELPERS -test() -> test(100). -test(N) -> test(N, true). -test(N, Verbose) -> quickcheck(numtests(N, noshrink(prop_repair_par(Verbose)))). -check() -> check(prop_repair_par(true), current_counterexample()). - --record(state, {num, % Number of FLU servers - seed, % Seed for partition simulator - verbose=false, % Verbose output for debugging - flu_names, % List of FLU names - mgr_names, % List of chain manager names - fc_list, % List of FLU1 proxy clients - cr_count, % Count of CR clients - cr_list}). % List of CR clients +%% Model weight(_S, change_partition) -> 20; weight(_S, _) -> 100. -%% append +%% Append append_args(#state{cr_count=CRCount}=S) -> [choose(1, CRCount), chunk(), S]. @@ -174,7 +173,7 @@ append(CRIndex, Bin, #state{verbose=V}=S) -> {other_error, Other} end. -%% change partition +%% Change partition change_partition_args(#state{flu_names=FLUNames}=S) -> %% [partition(FLUNames), S]. @@ -197,11 +196,22 @@ change_partition(Partition, num() -> choose(2, 5). - %% return(3). cr_count(Num) -> - Approx = 10, % ad-hoc - (Approx div Num) + 1. + Num * 3. + +%% Returns a list like +%% `[{#p_srvr{name=a, port=7501, ..}, "./eqc/data.eqc.a/"}, ...]' +all_list_extra(Num) -> + {PortBase, DirBase} = get_port_dir_base(), + [begin + FLUNameStr = [$a + I - 1], + FLUName = list_to_atom(FLUNameStr), + MgrName = machi_flu_psup:make_mgr_supname(FLUName), + {#p_srvr{name=FLUName, address="localhost", port=PortBase+I, + props=[{chmgr, MgrName}]}, + DirBase ++ "/data.eqc." ++ FLUNameStr} + end || I <- lists:seq(1, Num)]. %% Generator for possibly assymmetric partition information partition(FLUNames) -> @@ -223,7 +233,7 @@ flu_pairs(FLUNames) -> chunk() -> non_empty(binary(10)). -%% Property +%% Properties prop_repair(Verbose) -> error_logger:tty(false), @@ -232,17 +242,17 @@ prop_repair(Verbose) -> Seed = {1445,935441,287549}, ?FORALL(Num, num(), - ?FORALL(Cmds, commands(?MODULE, initial_state(Num, Seed, Verbose)), + ?FORALL(Cmds, commands(?MODULE, initial_state(Num, Verbose)), begin - SetupState = setup_chain(Num, Seed, Verbose), - {H, S1, Res} = run_commands(?MODULE, Cmds), + Target = setup_target(Num, Seed, Verbose), + {H, S1, Res0} = run_commands(?MODULE, Cmds), %% ?V("S1=~w~n", [S1]), ?V("==== Start post operations, stabilize and confirm results~n", []), - {_Res2, S2} = stabilize(commands_len(Cmds), SetupState), - {Dataloss, Critical} = confirm_result(S2), - _ = cleanup(SetupState), + _ = stabilize(commands_len(Cmds), Target), + {Dataloss, Critical} = confirm_result(Target), + _ = cleanup(Target), pretty_commands( - ?MODULE, Cmds, {H, S1, Res}, + ?MODULE, Cmds, {H, S1, Res0}, aggregate(with_title(cmds), command_names(Cmds), collect(with_title(length5), (length(Cmds) div 5) * 5, {Dataloss, Critical} =:= {0, 0}))) @@ -258,35 +268,35 @@ prop_repair_par(Verbose) -> ?FORALL(Cmds, %% Now try-and-err'ing, how to control command length and concurrency? ?SUCHTHAT(Cmds0, ?SIZED(Size, resize(Size, - parallel_commands(?MODULE, initial_state(Num, Seed, Verbose)))), + parallel_commands(?MODULE, initial_state(Num, Verbose)))), commands_len(Cmds0) > 20 andalso concurrency(Cmds0) > 2), begin CmdsLen= commands_len(Cmds), - SetupState = setup_chain(Num, Seed, Verbose), - {Seq, Par, Res} = run_parallel_commands(?MODULE, Cmds), + Target = setup_target(Num, Seed, Verbose), + {Seq, Par, Res0} = run_parallel_commands(?MODULE, Cmds), %% ?V("Seq=~w~n", [Seq]), %% ?V("Par=~w~n", [Par]), ?V("==== Start post operations, stabilize and confirm results~n", []), - {Dataloss, Critical} = - case Res of + {FinalRes, {Dataloss, Critical}} = + case Res0 of ok -> - {_Res2, S2} = stabilize(CmdsLen, SetupState), - confirm_result(S2); + Res1 = stabilize(CmdsLen, Target), + {Res1, confirm_result(Target)}; _ -> - ?V("Res=~w~n", [Res]), - {undefined, undefined} + ?V("Res0=~w~n", [Res0]), + {Res0, {undefined, undefined}} end, - _ = cleanup(SetupState), + _ = cleanup(Target), %% Process is leaking? This log line can be removed after fix. [?V("process_count=~w~n", [erlang:system_info(process_count)]) || Verbose], pretty_commands( - ?MODULE, Cmds, {Seq, Par, Res}, + ?MODULE, Cmds, {Seq, Par, Res0}, aggregate(with_title(cmds), command_names(Cmds), collect(with_title(length5), (CmdsLen div 5) * 5, collect(with_title(conc), concurrency(Cmds), - {Dataloss, Critical} =:= {0, 0}))) + {FinalRes, {Dataloss, Critical}} =:= {ok, {0, 0}}))) ) end)). @@ -296,24 +306,35 @@ prop_repair_par(Verbose) -> %% > eqc_gen:sample(eqc_statem:commands(machi_ap_repair_eqc)). %% but not so helpful. initial_state() -> - #state{cr_count=3, cr_list=[a,b,c]}. + #state{cr_count=3}. -initial_state(Num, Seed, Verbose) -> +initial_state(Num, Verbose) -> AllListE = all_list_extra(Num), FLUNames = [P#p_srvr.name || {P, _Dir} <- AllListE], MgrNames = [{Name, machi_flu_psup:make_mgr_supname(Name)} || Name <- FLUNames], - #state{num=Num, seed=Seed, verbose=Verbose, + #state{num=Num, verbose=Verbose, flu_names=FLUNames, mgr_names=MgrNames, - cr_count=cr_count(Num), - cr_list=undefined, fc_list=undefined}. + cr_count=cr_count(Num)}. -setup_chain(Num, Seed, Verbose) -> - %% ?V("setup_chain(Num=~w, Seed=~w~nn", [Num, Seed]), +setup_target(Num, Seed, Verbose) -> + %% ?V("setup_target(Num=~w, Seed=~w~nn", [Num, Seed]), AllListE = all_list_extra(Num), FLUNames = [P#p_srvr.name || {P, _Dir} <- AllListE], MgrNames = [{Name, machi_flu_psup:make_mgr_supname(Name)} || Name <- FLUNames], Dict = orddict:from_list([{P#p_srvr.name, P} || {P, _Dir} <- AllListE]), + setup_chain(Seed, AllListE, FLUNames, MgrNames, Dict), + _ = setup_cpool(AllListE, FLUNames, Dict), + + Target = #target{flu_names=FLUNames, mgr_names=MgrNames, + verbose=Verbose}, + %% Don't wait for complete chain. Even partialy completed, the chain + %% should work fine. Right? + wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, + 20, Verbose), + Target. + +setup_chain(Seed, AllListE, FLUNames, MgrNames, Dict) -> ok = shutdown_hard(), [begin machi_flu1_test:clean_up_data_dir(Dir), @@ -341,23 +362,10 @@ setup_chain(Num, Seed, Verbose) -> [{ok, _} = machi_flu_psup:start_flu_package(Name, Port, Dir, FLUOpts) || {#p_srvr{name=Name, port=Port}, Dir} <- AllListE], [machi_chain_manager1:set_chain_members(MgrName, Dict) || {_, MgrName} <- MgrNames], + ok. - {CRList, FCList} = setup_pool(AllListE), - State = #state{num=Num, seed=Seed, verbose=Verbose, - flu_names=FLUNames, mgr_names=MgrNames, - cr_count=cr_count(Num), - cr_list=CRList, fc_list=FCList}, - %% Don't wait for complete chain. Even partialy completed, the chain - %% should work fine. Right? - wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, - 20, Verbose), - State. - -setup_pool(AllListE) -> +setup_cpool(AllListE, FLUNames, Dict) -> Num = length(AllListE), - FLUNames = [P#p_srvr.name || {P, _Dir} <- AllListE], - Dict = orddict:from_list([{P#p_srvr.name, P} || {P, _Dir} <- AllListE]), - FCList = [begin {ok, PCPid} = machi_proxy_flu1_client:start_link(P), {Name, PCPid} @@ -387,21 +395,21 @@ cr_list() -> [{cr_list, CRList}] = ets:lookup(cpool, cr_list), CRList. -%% Post commands +%% Post run_commands -stabilize(0, S) -> - {ok, S}; -stabilize(_CmdsLen, #state{flu_names=FLUNames, mgr_names=MgrNames, - verbose=Verbose}=S) -> +stabilize(0, _T) -> + ok; +stabilize(_CmdsLen, #target{flu_names=FLUNames, mgr_names=MgrNames, + verbose=Verbose}) -> machi_partition_simulator:no_partitions(), wait_until_stable(chain_state_all_ok(FLUNames), FLUNames, MgrNames, 100, Verbose), - {ok, S}. + ok. chain_state_all_ok(FLUNames) -> [{FLUName, {FLUNames, [], []}} || FLUName <- FLUNames]. -confirm_result(#state{}=_S) -> +confirm_result(_T) -> [{_, C} | _] = cr_list(), [{written, _Written}, {accpt, Accpt}, {failed, Failed}, {critical, Critical}] = tab_counts(), @@ -455,12 +463,12 @@ assert_chunk(C, {Off, Len, FileName}=Key, Bin) -> {error, Other} end. -cleanup(#state{fc_list=FCList, cr_list=CRList}=_S) -> - [begin unlink(FC), catch exit(FC, kill) end || {_, FC} <- FCList], - [begin unlink(CR), catch exit(CR, kill) end || {_, CR} <- CRList], +cleanup(_Target) -> + [begin unlink(FC), catch exit(FC, kill) end || {_, FC} <- fc_list()], + [begin unlink(CR), catch exit(CR, kill) end || {_, CR} <- cr_list()], _ = shutdown_hard(). -%% Internal utilities +%% Internal misc utilities eqc_verbose() -> os:getenv("EQC_VERBOSE") =:= "true". @@ -472,19 +480,6 @@ eqc_timeout(Default) -> end, {PropTimeout, PropTimeout * 300}. -%% Returns a list like -%% `[{#p_srvr{name=a, port=7501, ..}, "./eqc/data.eqc.a/"}, ...]' -all_list_extra(Num) -> - {PortBase, DirBase} = get_port_dir_base(), - [begin - FLUNameStr = [$a + I - 1], - FLUName = list_to_atom(FLUNameStr), - MgrName = machi_flu_psup:make_mgr_supname(FLUName), - {#p_srvr{name=FLUName, address="localhost", port=PortBase+I, - props=[{chmgr, MgrName}]}, - DirBase ++ "/data.eqc." ++ FLUNameStr} - end || I <- lists:seq(1, Num)]. - get_port_dir_base() -> I = case os:getenv("EQC_BASE_PORT") of false -> 0; -- 2.45.2 From 8d3f631d84cc732e4064dcb65929bc3e802dc4bc Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Tue, 17 Nov 2015 22:07:28 -0600 Subject: [PATCH 39/44] Oops. Accidentally left this out. --- include/machi_merkle_tree.hrl | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) create mode 100644 include/machi_merkle_tree.hrl diff --git a/include/machi_merkle_tree.hrl b/include/machi_merkle_tree.hrl new file mode 100644 index 0000000..19e6748 --- /dev/null +++ b/include/machi_merkle_tree.hrl @@ -0,0 +1,20 @@ +%% machi merkle tree records + +-record(naive, { + chunk_size = 1048576 :: pos_integer(), %% default 1 MB + recalc = true :: boolean(), + root :: 'undefined' | binary(), + lvl1 = [] :: [ binary() ], + lvl2 = [] :: [ binary() ], + lvl3 = [] :: [ binary() ], + leaves = [] :: [ { Offset :: pos_integer(), + Size :: pos_integer(), + Csum :: binary()} ] + }). + +-record(mt, { + filename :: string(), + tree :: #naive{}|merklet:tree(), + backend = 'merklet' :: 'naive'|'merklet' + }). + -- 2.45.2 From e11cdfe95cd359e0a6f9e40dfe0c4b1ba43ccfdc Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Thu, 5 Nov 2015 17:25:52 +0900 Subject: [PATCH 40/44] Add stop and trim command to eqc_statem test on file_proxy --- test/machi_file_proxy_eqc.erl | 114 ++++++++++++++++++++-------------- 1 file changed, 68 insertions(+), 46 deletions(-) diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index c0625e3..bf43e5e 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -31,6 +31,7 @@ -define(QC_OUT(P), eqc:on_output(fun(Str, Args) -> io:format(user, Str, Args) end, P)). +-define(TESTDIR, "./eqc"). %% EUNIT TEST DEFINITION eqc_test_() -> @@ -102,19 +103,30 @@ shuffle_interval() -> get_written_interval(L) -> ?LET({O, Ln}, elements(L), {O+1, Ln-1}). - + %% INITIALIZATION -record(state, {pid, prev_extra = 0, + filename = undefined, planned_writes=[], planned_trims=[], written=[], trimmed=[]}). -initial_state() -> #state{written=[{0,1024}]}. -initial_state(I, T) -> #state{written=[{0,1024}], - planned_writes=I, - planned_trims=T}. +-define(FILENAME, "machi_file_proxy_eqc_data"). + +initial_state() -> + %%#state{filename=undefined, written=[{0,1024}]}. + {_, _, MS} = os:timestamp(), + Filename = test_server:temp_name("eqc_data") ++ "." ++ integer_to_list(MS), + %% Filename = ?FILENAME, + #state{filename=Filename, written=[{0,1024}]}. + +initial_state(I, T) -> + S=initial_state(), + S#state{written=[{0,1024}], + planned_writes=I, + planned_trims=T}. weight(_S, rewrite) -> 1; weight(_S, _) -> 2. @@ -136,11 +148,6 @@ get_overlaps(Offset, Len, [{Pos, Sz} = Ck|T], Acc0) (Pos < Offset + Len andalso Offset + Len < Pos + Sz) -> get_overlaps(Offset, Len, T, [Ck|Acc0]); get_overlaps(Offset, Len, [_Ck|T], Acc0) -> - %% ?debugVal({Offset, Len, _Ck}), - %% ?debugVal(Offset =< Pos andalso Pos < Offset + Len andalso Offset + Len =< Pos + Sz), - %% ?debugVal(Offset =< Pos andalso Pos + Sz < Offset + Len), - %% ?debugVal(Pos < Offset andalso Offset < Pos + Sz andalso Pos + Sz < Offset + Len), - %% ?debugVal(Pos < Offset + Len andalso Offset + Len < Pos + Sz), get_overlaps(Offset, Len, T, Acc0). %% Inefficient but simple easy code to verify by eyes - returns all @@ -209,8 +216,6 @@ last_byte(L0) -> L1 = lists:map(fun({Pos, Sz}) -> Pos + Sz end, L0), lists:last(lists:sort(L1)). --define(TESTDIR, "./eqc"). - cleanup() -> [begin Fs = filelib:wildcard(?TESTDIR ++ Glob), @@ -228,14 +233,12 @@ start_pre(S) -> start_command(S) -> {call, ?MODULE, start, [S]}. -start(_S) -> - {_, _, MS} = os:timestamp(), - File = test_server:temp_name("eqc_data") ++ "." ++ integer_to_list(MS), +start(#state{filename=File}) -> {ok, Pid} = machi_file_proxy:start_link(some_flu, File, ?TESTDIR), unlink(Pid), Pid. -start_next(S, Pid, _Args) -> +start_next(S, Pid, _) -> S#state{pid = Pid}. %% read @@ -270,8 +273,7 @@ read_post(S, [_Pid, Off, L], Res) -> true; {error, not_written} when Chopped =:= [] andalso Trimmed =:= [] -> true; - Other -> - ?debugVal(Other), + _Other -> is_error(Res) end. @@ -312,13 +314,13 @@ write_post(S, [_Pid, Off, {Bin, _Tag, _Csum}] = _Args, Res) -> write_next(S, Res, [_Pid, Offset, {Bin, _Tag, _Csum}]) -> S0 = case is_ok(Res) of - true -> + true -> S#state{written = lists:sort(S#state.written ++ [{Offset, iolist_size(Bin)}]) }; - _ -> + _ -> S end, S0#state{prev_extra = 0, planned_writes=tl(S0#state.planned_writes)}. - + write(Pid, Offset, {Bin, Tag, Csum}) -> Meta = [{client_csum_tag, Tag}, @@ -328,7 +330,6 @@ write(Pid, Offset, {Bin, Tag, Csum}) -> %% append append_pre(S) -> - ?assert(undefined =/= S#state.written), S#state.pid /= undefined. %% do not allow appends with empty binary data @@ -347,20 +348,32 @@ append(Pid, Extra, {Bin, Tag, Csum}) -> append_next(S, Res, [_Pid, Extra, {Bin, _Tag, _Csum}]) -> case is_ok(Res) of - true -> + true -> Offset = get_offset(Res), - Expected = erlang:max(last_byte(S#state.written) + S#state.prev_extra, - last_byte(S#state.trimmed)), - ?assertEqual(Expected, Offset), - S#state{prev_extra = Extra, written = lists:sort(S#state.written ++ [{Offset, iolist_size(Bin)}])}; - _ -> + S#state{prev_extra = Extra, + written = lists:sort(S#state.written ++ [{Offset, iolist_size(Bin)}])}; + _Other -> S end. -%% appends should always succeed unless the disk is full +%% appends should always succeed unless the disk is full %% or there's a hardware failure. -append_post(_S, _Args, Res) -> - true == is_ok(Res). +append_post(S, _Args, Res) -> + case is_ok(Res) of + true -> + Offset = get_offset(Res), + case erlang:max(last_byte(S#state.written), + last_byte(S#state.trimmed)) + S#state.prev_extra of + Offset -> + true; + UnexpectedByte -> + {wrong_offset_after_append, + {Offset, UnexpectedByte}, + {S#state.written, S#state.prev_extra}} + end; + Error -> + Error + end. %% rewrite @@ -415,24 +428,33 @@ trim_next(S, Res, [_Pid, Offset, Length]) -> S1#state{prev_extra=0, planned_trims=tl(S#state.planned_trims)}. +stop_pre(S) -> + S#state.pid /= undefined. + +stop_args(S) -> + [S#state.pid]. + +stop(Pid) -> + catch machi_file_proxy:stop(Pid). + +stop_post(_, _, _) -> true. + +stop_next(S, _, _) -> + S#state{pid=undefined, prev_extra=0}. + %% Property prop_ok() -> - cleanup(), - ?FORALL({I, T}, - {shuffle_interval(), shuffle_interval()}, - ?FORALL(Cmds, parallel_commands(?MODULE, initial_state(I, T)), - begin - {H, S, Res} = run_parallel_commands(?MODULE, Cmds), - %% case S#state.pid of - %% undefined -> noop; - %% Pid -> - %% machi_file_proxy:stop(Pid) - %% end, - pretty_commands(?MODULE, Cmds, {H, S, Res}, - aggregate(command_names(Cmds), Res == ok)) - end) - ). + cleanup(), + ?FORALL({I, T}, + {shuffle_interval(), shuffle_interval()}, + ?FORALL(Cmds, parallel_commands(?MODULE, initial_state(I, T)), + begin + {H, S, Res} = run_parallel_commands(?MODULE, Cmds), + cleanup(), + pretty_commands(?MODULE, Cmds, {H, S, Res}, + aggregate(command_names(Cmds), Res == ok)) + end)). %% Test for tester functions chopper_test_() -> -- 2.45.2 From 9f6b53fc151c72b675ab23db66c49f6a8069d546 Mon Sep 17 00:00:00 2001 From: UENISHI Kota Date: Wed, 18 Nov 2015 14:11:50 +0900 Subject: [PATCH 41/44] Make bigger offset and a bit code cleanup --- test/machi_file_proxy_eqc.erl | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/test/machi_file_proxy_eqc.erl b/test/machi_file_proxy_eqc.erl index bf43e5e..00d470f 100644 --- a/test/machi_file_proxy_eqc.erl +++ b/test/machi_file_proxy_eqc.erl @@ -113,13 +113,9 @@ get_written_interval(L) -> written=[], trimmed=[]}). --define(FILENAME, "machi_file_proxy_eqc_data"). - initial_state() -> - %%#state{filename=undefined, written=[{0,1024}]}. {_, _, MS} = os:timestamp(), Filename = test_server:temp_name("eqc_data") ++ "." ++ integer_to_list(MS), - %% Filename = ?FILENAME, #state{filename=Filename, written=[{0,1024}]}. initial_state(I, T) -> @@ -247,25 +243,19 @@ read_pre(S) -> S#state.pid /= undefined. read_args(S) -> - [S#state.pid, offset(), len()]. + [S#state.pid, oneof([offset(), big_offset()]), len()]. read_post(S, [_Pid, Off, L], Res) -> Written = get_overlaps(Off, L, S#state.written, []), Chopped = chop(Off, L, Written), Trimmed = get_overlaps(Off, L, S#state.trimmed, []), Eof = lists:max([Pos+Sz||{Pos,Sz}<-S#state.written]), - %% ?debugVal({Off, L}), - %% ?debugVal(S), case Res of {ok, {Written0, Trimmed0}} -> Written1 = lists:map(fun({_, Pos, Chunk, _}) -> {Pos, iolist_size(Chunk)} end, Written0), Trimmed1 = lists:map(fun({_, Pos, Sz}) -> {Pos, Sz} end, Trimmed0), - %% ?debugVal({Written, Chopped, Written1}), - %% ?debugVal({Trimmed, Trimmed1}), - %% ?assertEqual(Chopped, Written1), - %% ?assertEqual(Trimmed, Trimmed1), Chopped =:= Written1 andalso Trimmed =:= Trimmed1; %% TODO: such response are ugly, rethink the SPEC -- 2.45.2 From 4ce7a87d56059ecce72a03cdcf3226015cb509ed Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Fri, 20 Nov 2015 16:29:17 -0600 Subject: [PATCH 42/44] Remove merklet --- rebar.config | 1 - src/machi_merkle_tree.erl | 59 +++------------------------------ test/machi_merkle_tree_test.erl | 14 ++++---- 3 files changed, 10 insertions(+), 64 deletions(-) diff --git a/rebar.config b/rebar.config index f03a43a..2a77381 100644 --- a/rebar.config +++ b/rebar.config @@ -6,7 +6,6 @@ {deps, [ {lager, ".*", {git, "git://github.com/basho/lager.git", {tag, "2.2.0"}}}, - {merklet, ".*", {git, "https://github.com/ferd/merklet.git", {branch, "master"}}}, {protobuffs, "0.8.*", {git, "git://github.com/basho/erlang_protobuffs.git", {tag, "0.8.1p4"}}}, {riak_dt, ".*", {git, "git://github.com/basho/riak_dt.git", {branch, "develop"}}}, {node_package, ".*", {git, "git://github.com/basho/node_package.git", {branch, "develop"}}}, diff --git a/src/machi_merkle_tree.erl b/src/machi_merkle_tree.erl index 0b681ca..f05b528 100644 --- a/src/machi_merkle_tree.erl +++ b/src/machi_merkle_tree.erl @@ -21,12 +21,6 @@ %% @doc Creates a Merkle tree per file based on the checksum data for %% a given data file. %% -%% Has selectable backend, chosen at open. -%% -%% The default 'merklet' implementation uses the `merklet' library. Keys are -%% encoded as `<>' values encoded as `<>' -%% *or* as `<<0>>' for unwritten bytes, or `<<1>>' for trimmed bytes. -%% %% The `naive' implementation representation is: %% %% `<>' for unwritten bytes @@ -35,8 +29,8 @@ %% %% The tree feeds these leaf nodes into hashes representing chunks of a minimum %% size of at least 1024 KB (1 MB), but if the file size is larger, we will try -%% to get about 100 chunks for called "Level 1." We aim for around 10 hashes at -%% level 2, and then 2 hashes level 3 and finally the root. +%% to get about 100 chunks for the first rollup "Level 1." We aim for around 10 +%% hashes at level 2, and then 2 hashes level 3 and finally the root. -module(machi_merkle_tree). @@ -57,12 +51,8 @@ -define(TRIMMED, <<1>>). -define(UNWRITTEN, <<0>>). --define(ENCODE(Offset, Size), <>). -define(NAIVE_ENCODE(Offset, Size, Data), <>). --define(NEW_MERKLET, undefined). --define(TIMEOUT, (10*1000)). - -define(MINIMUM_CHUNK, 1048576). %% 1024 * 1024 -define(LEVEL_SIZE, 10). -define(H, sha). @@ -70,13 +60,12 @@ %% public API open(Filename, DataDir) -> - open(Filename, DataDir, merklet). + open(Filename, DataDir, naive). open(Filename, DataDir, Type) -> Tree = load_filename(Filename, DataDir, Type), {ok, #mt{ filename = Filename, tree = Tree, backend = Type}}. -tree(#mt{ tree = T, backend = merklet }) -> T; tree(#mt{ tree = T, backend = naive }) -> case T#naive.recalc of true -> build_tree(T); @@ -90,19 +79,11 @@ diff(#mt{backend = naive, tree = T1}, #mt{backend = naive, tree = T2}) -> true -> same; false -> naive_diff(T1, T2) end; -diff(#mt{backend = merklet, tree = T1}, #mt{backend = merklet, tree = T2}) -> - case merklet:diff(T1, T2) of - [] -> same; - Diff -> Diff - end; diff(_, _) -> error(badarg). %% private % @private -load_filename(Filename, DataDir, merklet) -> - {_Last, M} = do_load(Filename, DataDir, fun insert_csum/2, ?NEW_MERKLET), - M; load_filename(Filename, DataDir, naive) -> {Last, M} = do_load(Filename, DataDir, fun insert_csum_naive/2, []), ChunkSize = max(?MINIMUM_CHUNK, Last div 100), @@ -117,16 +98,6 @@ do_load(Filename, DataDir, FoldFun, AccInit) -> Acc. % @private -insert_csum({Last, Size, _Csum}=In, {Last, MT}) -> - %% no gap here, insert a record - {Last+Size, update_merkle_tree(In, MT)}; -insert_csum({Offset, Size, _Csum}=In, {Last, MT}) -> - %% gap here, insert unwritten record - %% *AND* insert written record - Hole = Offset - Last, - MT0 = update_merkle_tree({Last, Hole, unwritten}, MT), - {Offset+Size, update_merkle_tree(In, MT0)}. - insert_csum_naive({Last, Size, _Csum}=In, {Last, MT}) -> %% no gap {Last+Size, update_acc(In, MT)}; @@ -136,32 +107,20 @@ insert_csum_naive({Offset, Size, _Csum}=In, {Last, MT}) -> {Offset+Size, update_acc(In, MT0)}. % @private -update_merkle_tree({Offset, Size, unwritten}, MT) -> - merklet:insert({?ENCODE(Offset, Size), ?UNWRITTEN}, MT); -update_merkle_tree({Offset, Size, trimmed}, MT) -> - merklet:insert({?ENCODE(Offset, Size), ?TRIMMED}, MT); -update_merkle_tree({Offset, Size, Csum}, MT) -> - merklet:insert({?ENCODE(Offset, Size), Csum}, MT). - update_acc({Offset, Size, unwritten}, MT) -> [ {Offset, Size, ?NAIVE_ENCODE(Offset, Size, ?UNWRITTEN)} | MT ]; update_acc({Offset, Size, trimmed}, MT) -> [ {Offset, Size, ?NAIVE_ENCODE(Offset, Size, ?TRIMMED)} | MT ]; -update_acc({Offset, Size, Csum}, MT) -> +update_acc({Offset, Size, <<_Tag:8, Csum/binary>>}, MT) -> [ {Offset, Size, ?NAIVE_ENCODE(Offset, Size, Csum)} | MT ]. build_tree(MT = #naive{ leaves = L, chunk_size = ChunkSize }) -> - lager:debug("Leaves: ~p~n", [L]), Lvl1s = build_level_1(ChunkSize, L, 1, [ crypto:hash_init(?H) ]), - lager:debug("Lvl1: ~p~n", [Lvl1s]), Mod2 = length(Lvl1s) div ?LEVEL_SIZE, Lvl2s = build_int_level(Mod2, Lvl1s, 1, [ crypto:hash_init(?H) ]), - lager:debug("Lvl2: ~p~n", [Lvl2s]), Mod3 = length(Lvl2s) div 2, Lvl3s = build_int_level(Mod3, Lvl2s, 1, [ crypto:hash_init(?H) ]), - lager:debug("Lvl3: ~p~n", [Lvl3s]), Root = build_root(Lvl3s, crypto:hash_init(?H)), - lager:debug("Root: ~p~n", [Root]), MT#naive{ root = Root, lvl1 = Lvl1s, lvl2 = Lvl2s, lvl3 = Lvl3s, recalc = false }. build_root([], Ctx) -> @@ -189,19 +148,9 @@ build_level_1(Size, [{Pos, Len, Hash}|T], Multiple, [ Ctx | Rest ]) build_level_1(Size, T, Multiple, [ crypto:hash_update(Ctx, Hash) | Rest ]). naive_diff(#naive{lvl1 = L1}, #naive{lvl1=L2, chunk_size=CS2}) -> - lager:debug("naive diff: Our lvl1: ~p~n", [L1]), - lager:debug("naive diff: Their chunk size: ~p, lvl1: ~p~n", [CS2, L2]), - Set1 = gb_sets:from_list(lists:zip(lists:seq(1, length(L1), L1))), Set2 = gb_sets:from_list(lists:zip(lists:seq(1, length(L2), L2))), %% The byte ranges in list 2 that do not match in list 1 - %% - %% We have to decide what to do now - should we filter the - %% leaf nodes using these ranges and find specific divergence - %% between Tree1 and Tree2? - %% %% Or should we do something else? [ {(X-1)*CS2, CS2, SHA} || {X, SHA} <- gb_sets:to_list(gb_sets:subtract(Set1, Set2)) ]. - - diff --git a/test/machi_merkle_tree_test.erl b/test/machi_merkle_tree_test.erl index d4020d0..edb4ff7 100644 --- a/test/machi_merkle_tree_test.erl +++ b/test/machi_merkle_tree_test.erl @@ -26,9 +26,13 @@ -include_lib("eunit/include/eunit.hrl"). -include_lib("kernel/include/file.hrl"). --define(TESTFILE, "yza^4c784dc2-19bf-4ac6-91f6-58bbe5aa88e0^1"). -define(GAP_CHANCE, 0.10). +%% unit tests + + +%% Define or remove these ifdefs if benchmarking is desired. +-ifdef(BENCH). choose_filename() -> random_from_list([ "def^c5ea7511-d649-47d6-a8c3-2b619379c237^1", @@ -140,7 +144,6 @@ torture_test(C) -> ok = file:close(F). run_torture_test() -> - {MTime, M} = timer:tc(fun() -> merklet_torture() end), {NTime, N} = timer:tc(fun() -> naive_torture() end), MSize = byte_size(term_to_binary(M)), @@ -148,12 +151,6 @@ run_torture_test() -> {MSize, MTime, NSize, NTime}. -merklet_torture() -> - lists:foldl( - fun({O, S, Sha}, Acc) -> - merklet:insert({<>, Sha}, Acc) - end, undefined, torture_generator()). - naive_torture() -> N = lists:foldl(fun(T, Acc) -> machi_merkle_tree:update_acc(T, Acc) end, [], torture_generator()), T = #naive{ leaves = lists:reverse(N), chunk_size = 10010, recalc = true }, @@ -161,3 +158,4 @@ naive_torture() -> torture_generator() -> [ {O, 1, crypto:hash(sha, term_to_binary(now()))} || O <- lists:seq(1024, 1000000) ]. +-endif. % BENCH -- 2.45.2 From a5a03699050909bb8aa66f03b19e2ac8f3f4e1c5 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Fri, 20 Nov 2015 21:22:41 -0600 Subject: [PATCH 43/44] Add basic unit test --- src/machi_merkle_tree.erl | 4 +- test/machi_merkle_tree_test.erl | 129 ++++++++++++++++++++------------ 2 files changed, 85 insertions(+), 48 deletions(-) diff --git a/src/machi_merkle_tree.erl b/src/machi_merkle_tree.erl index f05b528..effcb67 100644 --- a/src/machi_merkle_tree.erl +++ b/src/machi_merkle_tree.erl @@ -148,8 +148,8 @@ build_level_1(Size, [{Pos, Len, Hash}|T], Multiple, [ Ctx | Rest ]) build_level_1(Size, T, Multiple, [ crypto:hash_update(Ctx, Hash) | Rest ]). naive_diff(#naive{lvl1 = L1}, #naive{lvl1=L2, chunk_size=CS2}) -> - Set1 = gb_sets:from_list(lists:zip(lists:seq(1, length(L1), L1))), - Set2 = gb_sets:from_list(lists:zip(lists:seq(1, length(L2), L2))), + Set1 = gb_sets:from_list(lists:zip(lists:seq(1, length(L1)), L1)), + Set2 = gb_sets:from_list(lists:zip(lists:seq(1, length(L2)), L2)), %% The byte ranges in list 2 that do not match in list 1 %% Or should we do something else? diff --git a/test/machi_merkle_tree_test.erl b/test/machi_merkle_tree_test.erl index edb4ff7..922f0e2 100644 --- a/test/machi_merkle_tree_test.erl +++ b/test/machi_merkle_tree_test.erl @@ -29,10 +29,92 @@ -define(GAP_CHANCE, 0.10). %% unit tests +basic_test() -> + random:seed(os:timestamp()), + Fsz = choose_size() * 1024, + Filesize = max(Fsz, 10*1024*1024), + ChunkSize = max(1048576, Filesize div 100), + N = make_leaf_nodes(Filesize), + D0 = #naive{ leaves = N, chunk_size = ChunkSize, recalc = true }, + T1 = machi_merkle_tree:build_tree(D0), + D1 = #naive{ leaves = tl(N), chunk_size = ChunkSize, recalc = true }, + T2 = machi_merkle_tree:build_tree(D1), + + ?assertNotEqual(T1#naive.root, T2#naive.root), + ?assertEqual(1, length(machi_merkle_tree:naive_diff(T1, T2))). + + +make_leaf_nodes(Filesize) -> + lists:reverse( + lists:foldl(fun(T, Acc) -> machi_merkle_tree:update_acc(T, Acc) end, + [], + generate_offsets(Filesize, 1024, [])) + ). + +choose_int(Factor) -> + random:uniform(1024*Factor). + +small_int() -> + choose_int(10). + +medium_int() -> + choose_int(1024). + +large_int() -> + choose_int(4096). + +generate_offsets(Filesize, Current, Acc) when Current < Filesize -> + Length0 = choose_size(), + + Length = case Length0 + Current > Filesize of + false -> Length0; + true -> Filesize - Current + end, + Data = term_to_binary(os:timestamp()), + Checksum = machi_util:make_tagged_csum(client_sha, machi_util:checksum_chunk(Data)), + Gap = maybe_gap(random:uniform()), + generate_offsets(Filesize, Current + Length + Gap, [ {Current, Length, Checksum} | Acc ]); +generate_offsets(_Filesize, _Current, Acc) -> + lists:reverse(Acc). + + +random_from_list(L) -> + N = random:uniform(length(L)), + lists:nth(N, L). + +choose_size() -> + F = random_from_list([fun small_int/0, fun medium_int/0, fun large_int/0]), + F(). + +maybe_gap(Chance) when Chance < ?GAP_CHANCE -> + choose_size(); +maybe_gap(_) -> 0. %% Define or remove these ifdefs if benchmarking is desired. -ifdef(BENCH). +generate_offsets(FH, Filesize, Current, Acc) when Current < Filesize -> + Length0 = choose_size(), + + Length = case Length0 + Current > Filesize of + false -> Length0; + true -> Filesize - Current + end, + {ok, Data} = file:pread(FH, Current, Length), + Checksum = machi_util:make_tagged_csum(client_sha, machi_util:checksum_chunk(Data)), + Gap = maybe_gap(random:uniform()), + generate_offsets(FH, Filesize, Current + Length + Gap, [ {Current, Length, Checksum} | Acc ]); +generate_offsets(_FH, _Filesize, _Current, Acc) -> + lists:reverse(Acc). + +make_offsets_from_file(Filename) -> + {ok, Info} = file:read_file_info(Filename), + Filesize = Info#file_info.size, + {ok, FH} = file:open(Filename, [read, raw, binary]), + Offsets = generate_offsets(FH, Filesize, 1024, []), + file:close(FH), + Offsets. + choose_filename() -> random_from_list([ "def^c5ea7511-d649-47d6-a8c3-2b619379c237^1", @@ -52,51 +134,6 @@ make_csum_file(DataDir, Filename, Offsets) -> Offsets), machi_csum_table:close(MC). -choose_int(Factor) -> - random:uniform(1024*Factor). - -small_int() -> - choose_int(10). - -medium_int() -> - choose_int(1024). - -large_int() -> - choose_int(4096). - -make_offsets(Filename) -> - {ok, Info} = file:read_file_info(Filename), - Filesize = Info#file_info.size, - {ok, FH} = file:open(Filename, [read, raw, binary]), - Offsets = generate_offsets(FH, Filesize, 1024, []), - file:close(FH), - Offsets. - -random_from_list(L) -> - N = random:uniform(length(L)), - lists:nth(N, L). - -choose_size() -> - F = random_from_list([fun small_int/0, fun medium_int/0, fun large_int/0]), - F(). - -maybe_gap(Chance) when Chance < ?GAP_CHANCE -> - choose_size(); -maybe_gap(_) -> 0. - -generate_offsets(FH, Filesize, Current, Acc) when Current < Filesize -> - Length0 = choose_size(), - - Length = case Length0 + Current > Filesize of - false -> Length0; - true -> Filesize - Current - end, - {ok, Data} = file:pread(FH, Current, Length), - Checksum = machi_util:make_tagged_csum(client_sha, machi_util:checksum_chunk(Data)), - Gap = maybe_gap(random:uniform()), - generate_offsets(FH, Filesize, Current + Length + Gap, [ {Current, Length, Checksum} | Acc ]); -generate_offsets(_FH, _Filesize, _Current, Acc) -> - lists:reverse(Acc). test() -> test(100). @@ -112,7 +149,7 @@ format_and_store(F, {OffsetNum, {MTime, MSize}, {NTime, NSize}}) -> run_test(C) -> random:seed(os:timestamp()), OffsetFn = "test/" ++ choose_filename(), - O = make_offsets(OffsetFn), + O = make_offsets_from_file(OffsetFn), Fn = "csum_" ++ integer_to_list(C), make_csum_file(".", Fn, O), -- 2.45.2 From 1f56850c2b29d10a9749f7b178c798d451137ee2 Mon Sep 17 00:00:00 2001 From: Mark Allen Date: Fri, 20 Nov 2015 22:34:41 -0600 Subject: [PATCH 44/44] Remove merklet from header file too. --- include/machi_merkle_tree.hrl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/include/machi_merkle_tree.hrl b/include/machi_merkle_tree.hrl index 19e6748..e3d0feb 100644 --- a/include/machi_merkle_tree.hrl +++ b/include/machi_merkle_tree.hrl @@ -14,7 +14,7 @@ -record(mt, { filename :: string(), - tree :: #naive{}|merklet:tree(), - backend = 'merklet' :: 'naive'|'merklet' + tree :: #naive{}, + backend = 'naive' :: 'naive' }). -- 2.45.2