From c92713dcb3a9d68c521e25d8772f8aac153f8f0c Mon Sep 17 00:00:00 2001 From: Denis Otkidach Date: Wed, 8 Nov 2023 17:50:14 +0200 Subject: [PATCH 1/5] Drop support of broken LZ4 in kafka < 0.10 --- aiokafka/codec.py | 53 -------------------- aiokafka/protocol/message.py | 7 ++- aiokafka/record/_crecords/legacy_records.pyx | 14 ++++-- aiokafka/record/legacy_records.py | 14 ++++-- requirements-ci.txt | 1 - requirements-win-test.txt | 1 - setup.py | 2 +- tests/record/test_legacy.py | 13 ++--- tests/test_codec.py | 14 ------ tests/test_consumer.py | 1 + 10 files changed, 34 insertions(+), 86 deletions(-) diff --git a/aiokafka/codec.py b/aiokafka/codec.py index 2e3ddaaf..eeed6a85 100644 --- a/aiokafka/codec.py +++ b/aiokafka/codec.py @@ -44,11 +44,6 @@ def _lz4_compress(payload, **kwargs): except ImportError: lz4framed = None -try: - import xxhash -except ImportError: - xxhash = None - PYPY = bool(platform.python_implementation() == "PyPy") @@ -253,54 +248,6 @@ def lz4f_decode(payload): lz4_decode = None -def lz4_encode_old_kafka(payload): - """Encode payload for 0.8/0.9 brokers -- requires an incorrect header checksum.""" - assert xxhash is not None - data = lz4_encode(payload) - header_size = 7 - flg = data[4] - if not isinstance(flg, int): - flg = ord(flg) - - content_size_bit = (flg >> 3) & 1 - if content_size_bit: - # Old kafka does not accept the content-size field - # so we need to discard it and reset the header flag - flg -= 8 - data = bytearray(data) - data[4] = flg - data = bytes(data) - payload = data[header_size + 8:] - else: - payload = data[header_size:] - - # This is the incorrect hc - hc = xxhash.xxh32(data[0:header_size - 1]).digest()[ - -2:-1 - ] # pylint: disable-msg=no-member - - return b"".join([data[0:header_size - 1], hc, payload]) - - -def lz4_decode_old_kafka(payload): - assert xxhash is not None - # Kafka's LZ4 code has a bug in its header checksum implementation - header_size = 7 - if isinstance(payload[4], int): - flg = payload[4] - else: - flg = ord(payload[4]) - content_size_bit = (flg >> 3) & 1 - if content_size_bit: - header_size += 8 - - # This should be the correct hc - hc = xxhash.xxh32(payload[4:header_size - 1]).digest()[-2:-1] - - munged_payload = b"".join([payload[0:header_size - 1], hc, payload[header_size:]]) - return lz4_decode(munged_payload) - - def zstd_encode(payload): if not zstd: raise NotImplementedError("Zstd codec is not available") diff --git a/aiokafka/protocol/message.py b/aiokafka/protocol/message.py index a305f419..c981be90 100644 --- a/aiokafka/protocol/message.py +++ b/aiokafka/protocol/message.py @@ -11,8 +11,8 @@ snappy_decode, zstd_decode, lz4_decode, - lz4_decode_old_kafka, ) +from aiokafka.errors import UnsupportedCodecError from aiokafka.util import WeakMethod from .frame import KafkaBytes @@ -156,7 +156,10 @@ def decompress(self): elif codec == self.CODEC_LZ4: assert has_lz4(), "LZ4 decompression unsupported" if self.magic == 0: - raw_bytes = lz4_decode_old_kafka(self.value) + # https://issues.apache.org/jira/browse/KAFKA-3160 + raise UnsupportedCodecError( + "LZ4 is not supported for broker version 0.8/0.9" + ) else: raw_bytes = lz4_decode(self.value) elif codec == self.CODEC_ZSTD: diff --git a/aiokafka/record/_crecords/legacy_records.pyx b/aiokafka/record/_crecords/legacy_records.pyx index 2406ef12..0c6905fb 100644 --- a/aiokafka/record/_crecords/legacy_records.pyx +++ b/aiokafka/record/_crecords/legacy_records.pyx @@ -2,8 +2,8 @@ import aiokafka.codec as codecs from aiokafka.codec import ( - gzip_encode, snappy_encode, lz4_encode, lz4_encode_old_kafka, - gzip_decode, snappy_decode, lz4_decode, lz4_decode_old_kafka + gzip_encode, snappy_encode, lz4_encode, + gzip_decode, snappy_decode, lz4_decode, ) from aiokafka.errors import CorruptRecordException, UnsupportedCodecError from zlib import crc32 as py_crc32 # needed for windows macro @@ -141,7 +141,10 @@ cdef class LegacyRecordBatch: uncompressed = snappy_decode(value) elif compression_type == _ATTR_CODEC_LZ4: if self._magic == 0: - uncompressed = lz4_decode_old_kafka(value) + # https://issues.apache.org/jira/browse/KAFKA-3160 + raise UnsupportedCodecError( + "LZ4 is not supported for broker version 0.8/0.9" + ) else: uncompressed = lz4_decode(value) @@ -437,7 +440,10 @@ cdef class LegacyRecordBatchBuilder: compressed = snappy_encode(self._buffer) elif self._compression_type == _ATTR_CODEC_LZ4: if self._magic == 0: - compressed = lz4_encode_old_kafka(bytes(self._buffer)) + # https://issues.apache.org/jira/browse/KAFKA-3160 + raise UnsupportedCodecError( + "LZ4 is not supported for broker version 0.8/0.9" + ) else: compressed = lz4_encode(bytes(self._buffer)) else: diff --git a/aiokafka/record/legacy_records.py b/aiokafka/record/legacy_records.py index c1ae9480..143576b8 100644 --- a/aiokafka/record/legacy_records.py +++ b/aiokafka/record/legacy_records.py @@ -5,8 +5,8 @@ import aiokafka.codec as codecs from aiokafka.codec import ( - gzip_encode, snappy_encode, lz4_encode, lz4_encode_old_kafka, - gzip_decode, snappy_decode, lz4_decode, lz4_decode_old_kafka + gzip_encode, snappy_encode, lz4_encode, + gzip_decode, snappy_decode, lz4_decode, ) from aiokafka.errors import CorruptRecordException, UnsupportedCodecError from aiokafka.util import NO_EXTENSIONS @@ -159,7 +159,10 @@ def _decompress(self, key_offset): uncompressed = snappy_decode(data.tobytes()) elif compression_type == self.CODEC_LZ4: if self._magic == 0: - uncompressed = lz4_decode_old_kafka(data.tobytes()) + # https://issues.apache.org/jira/browse/KAFKA-3160 + raise UnsupportedCodecError( + "LZ4 is not supported for broker version 0.8/0.9" + ) else: uncompressed = lz4_decode(data.tobytes()) return uncompressed @@ -415,7 +418,10 @@ def _maybe_compress(self): compressed = snappy_encode(buf) elif self._compression_type == self.CODEC_LZ4: if self._magic == 0: - compressed = lz4_encode_old_kafka(bytes(buf)) + # https://issues.apache.org/jira/browse/KAFKA-3160 + raise UnsupportedCodecError( + "LZ4 is not supported for broker version 0.8/0.9" + ) else: compressed = lz4_encode(bytes(buf)) compressed_size = len(compressed) diff --git a/requirements-ci.txt b/requirements-ci.txt index 3d09463d..373fe6b0 100644 --- a/requirements-ci.txt +++ b/requirements-ci.txt @@ -10,7 +10,6 @@ pytest-mock==3.12.0 docker==6.1.2 chardet==4.0.0 # Until fixed requests is released lz4==3.1.3 -xxhash==2.0.2 python-snappy==0.6.1 docutils==0.17.1 Pygments==2.15.0 diff --git a/requirements-win-test.txt b/requirements-win-test.txt index e2e51c61..e856f0c0 100644 --- a/requirements-win-test.txt +++ b/requirements-win-test.txt @@ -10,6 +10,5 @@ pytest-mock==3.12.0 docker==6.0.1 chardet==4.0.0 # Until fixed requests is released lz4==3.1.3 -xxhash==2.0.2 python-snappy==0.6.1 zstandard==0.16.0 diff --git a/setup.py b/setup.py index 07d9710c..c85d4f48 100644 --- a/setup.py +++ b/setup.py @@ -122,7 +122,7 @@ def read(f): extras_require = { "snappy": ["python-snappy>=0.5"], - "lz4": ["lz4"], # Old format (magic=0) requires xxhash + "lz4": ["lz4"], "zstd": ["zstandard"], "gssapi": ["gssapi"], } diff --git a/tests/record/test_legacy.py b/tests/record/test_legacy.py index 26f3d4dc..83afff2a 100644 --- a/tests/record/test_legacy.py +++ b/tests/record/test_legacy.py @@ -44,12 +44,14 @@ def test_read_write_serde_v0_v1_no_compression(magic, key, value, checksum): assert msg.checksum == checksum[magic] & 0xffffffff -@pytest.mark.parametrize("compression_type", [ - LegacyRecordBatch.CODEC_GZIP, - LegacyRecordBatch.CODEC_SNAPPY, - LegacyRecordBatch.CODEC_LZ4 +@pytest.mark.parametrize("compression_type, magic", [ + (LegacyRecordBatch.CODEC_GZIP, 0), + (LegacyRecordBatch.CODEC_SNAPPY, 0), + # We don't support LZ4 for kafka 0.8/0.9 + (LegacyRecordBatch.CODEC_GZIP, 1), + (LegacyRecordBatch.CODEC_SNAPPY, 1), + (LegacyRecordBatch.CODEC_LZ4, 1), ]) -@pytest.mark.parametrize("magic", [0, 1]) def test_read_write_serde_v0_v1_with_compression(compression_type, magic): builder = LegacyRecordBatchBuilder( magic=magic, compression_type=compression_type, batch_size=1024 * 1024) @@ -194,7 +196,6 @@ def test_legacy_batch_size_limit(magic): @pytest.mark.parametrize("compression_type,name,checker_name", [ (LegacyRecordBatch.CODEC_GZIP, "gzip", "has_gzip"), (LegacyRecordBatch.CODEC_SNAPPY, "snappy", "has_snappy"), - (LegacyRecordBatch.CODEC_LZ4, "lz4", "has_lz4") ]) def test_unavailable_codec(compression_type, name, checker_name): builder = LegacyRecordBatchBuilder( diff --git a/tests/test_codec.py b/tests/test_codec.py index 9ae53487..f047d648 100644 --- a/tests/test_codec.py +++ b/tests/test_codec.py @@ -14,8 +14,6 @@ snappy_decode, lz4_encode, lz4_decode, - lz4_encode_old_kafka, - lz4_decode_old_kafka, zstd_encode, zstd_decode, ) @@ -102,18 +100,6 @@ def test_lz4(): assert b1 == b2 -@pytest.mark.skipif( - not has_lz4() or platform.python_implementation() == "PyPy", - reason="python-lz4 crashes on old versions of pypy", -) -def test_lz4_old(): - for i in range(1000): - b1 = random_string(100) - b2 = lz4_decode_old_kafka(lz4_encode_old_kafka(b1)) - assert len(b1) == len(b2) - assert b1 == b2 - - @pytest.mark.skipif( not has_lz4() or platform.python_implementation() == "PyPy", reason="python-lz4 crashes on old versions of pypy", diff --git a/tests/test_consumer.py b/tests/test_consumer.py index 5413aca8..5bd51b9e 100644 --- a/tests/test_consumer.py +++ b/tests/test_consumer.py @@ -514,6 +514,7 @@ async def test_compress_decompress_gzip(self): async def test_compress_decompress_snappy(self): await self._test_compress_decompress("snappy") + @kafka_versions('>=0.10.0.0') @run_until_complete async def test_compress_decompress_lz4(self): await self._test_compress_decompress("lz4") From d611aa8dbcc8f390665909593fc2b0204bbfcc3d Mon Sep 17 00:00:00 2001 From: Denis Otkidach Date: Wed, 8 Nov 2023 19:26:41 +0200 Subject: [PATCH 2/5] Replace python-snappy with cramjam --- aiokafka/codec.py | 32 ++++++++++---------------------- docs/index.rst | 28 +--------------------------- requirements-ci.txt | 2 +- requirements-win-test.txt | 2 +- setup.py | 3 ++- 5 files changed, 15 insertions(+), 52 deletions(-) diff --git a/aiokafka/codec.py b/aiokafka/codec.py index eeed6a85..bc75e3f9 100644 --- a/aiokafka/codec.py +++ b/aiokafka/codec.py @@ -1,6 +1,5 @@ import gzip import io -import platform import struct _XERIAL_V1_HEADER = (-126, b"S", b"N", b"A", b"P", b"P", b"Y", 0, 1, 1) @@ -8,9 +7,9 @@ ZSTD_MAX_OUTPUT_SIZE = 1024 * 1024 try: - import snappy + import cramjam except ImportError: - snappy = None + cramjam = None try: import zstandard as zstd @@ -22,6 +21,7 @@ def _lz4_compress(payload, **kwargs): # Kafka does not support LZ4 dependent blocks + # https://cwiki.apache.org/confluence/display/KAFKA/KIP-57+-+Interoperable+LZ4+Framing try: # For lz4>=0.12.0 kwargs.pop("block_linked", None) @@ -44,15 +44,13 @@ def _lz4_compress(payload, **kwargs): except ImportError: lz4framed = None -PYPY = bool(platform.python_implementation() == "PyPy") - def has_gzip(): return True def has_snappy(): - return snappy is not None + return cramjam is not None def has_zstd(): @@ -128,32 +126,22 @@ def snappy_encode(payload, xerial_compatible=True, xerial_blocksize=32 * 1024): raise NotImplementedError("Snappy codec is not available") if not xerial_compatible: - return snappy.compress(payload) + return cramjam.snappy.compress_raw(payload) out = io.BytesIO() for fmt, dat in zip(_XERIAL_V1_FORMAT, _XERIAL_V1_HEADER): out.write(struct.pack("!" + fmt, dat)) # Chunk through buffers to avoid creating intermediate slice copies - if PYPY: - # on pypy, snappy.compress() on a sliced buffer consumes the entire - # buffer... likely a python-snappy bug, so just use a slice copy - def chunker(payload, i, size): - return payload[i:size + i] - - else: - # snappy.compress does not like raw memoryviews, so we have to convert - # tobytes, which is a copy... oh well. it's the thought that counts. - # pylint: disable-msg=undefined-variable - def chunker(payload, i, size): - return memoryview(payload)[i:size + i].tobytes() + def chunker(payload, i, size): + return memoryview(payload)[i:size + i] for chunk in ( chunker(payload, i, xerial_blocksize) for i in range(0, len(payload), xerial_blocksize) ): - block = snappy.compress(chunk) + block = cramjam.snappy.compress_raw(chunk) block_size = len(block) out.write(struct.pack("!i", block_size)) out.write(block) @@ -205,13 +193,13 @@ def snappy_decode(payload): # Skip the block size cursor += 4 end = cursor + block_size - out.write(snappy.decompress(byt[cursor:end])) + out.write(cramjam.snappy.decompress_raw(byt[cursor:end])) cursor = end out.seek(0) return out.read() else: - return snappy.decompress(payload) + return cramjam.snappy.decompress_raw(payload) if lz4: diff --git a/docs/index.rst b/docs/index.rst index a51a9b78..7e507d85 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -110,33 +110,7 @@ from http://landinghub.visualstudio.com/visual-cpp-build-tools Optional Snappy install +++++++++++++++++++++++ -1. Download and build Snappy from http://google.github.io/snappy/ - -Ubuntu: - -.. code:: bash - - apt-get install libsnappy-dev - -OSX: - -.. code:: bash - - brew install snappy - -From Source: - -.. code:: bash - - wget https://github.com/google/snappy/tarball/master - tar xzvf google-snappy-X.X.X-X-XXXXXXXX.tar.gz - cd google-snappy-X.X.X-X-XXXXXXXX - ./configure - make - sudo make install - - -1. Install **aiokafka** with :code:`snappy` extra option +To enable Snappy compression/decompression, install **aiokafka** with :code:`snappy` extra option .. code:: bash diff --git a/requirements-ci.txt b/requirements-ci.txt index 373fe6b0..671bec23 100644 --- a/requirements-ci.txt +++ b/requirements-ci.txt @@ -10,9 +10,9 @@ pytest-mock==3.12.0 docker==6.1.2 chardet==4.0.0 # Until fixed requests is released lz4==3.1.3 -python-snappy==0.6.1 docutils==0.17.1 Pygments==2.15.0 gssapi==1.8.2 async-timeout==4.0.1 zstandard==0.16.0 +cramjam==2.7.0 diff --git a/requirements-win-test.txt b/requirements-win-test.txt index e856f0c0..7cc1c9be 100644 --- a/requirements-win-test.txt +++ b/requirements-win-test.txt @@ -10,5 +10,5 @@ pytest-mock==3.12.0 docker==6.0.1 chardet==4.0.0 # Until fixed requests is released lz4==3.1.3 -python-snappy==0.6.1 zstandard==0.16.0 +cramjam==2.7.0 diff --git a/setup.py b/setup.py index c85d4f48..c38813e5 100644 --- a/setup.py +++ b/setup.py @@ -112,6 +112,7 @@ def build_extension(self, ext): install_requires = [ "async-timeout", + "cramjam", "packaging", ] @@ -121,7 +122,7 @@ def read(f): extras_require = { - "snappy": ["python-snappy>=0.5"], + "snappy": ["cramjam"], "lz4": ["lz4"], "zstd": ["zstandard"], "gssapi": ["gssapi"], From f35aaa642658fe7d63c1d794120bc3b9e4da3d59 Mon Sep 17 00:00:00 2001 From: Denis Otkidach Date: Wed, 8 Nov 2023 22:21:00 +0200 Subject: [PATCH 3/5] Simplify LZ4 code --- aiokafka/codec.py | 10 ++-------- setup.py | 2 +- 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/aiokafka/codec.py b/aiokafka/codec.py index bc75e3f9..015c2cde 100644 --- a/aiokafka/codec.py +++ b/aiokafka/codec.py @@ -22,14 +22,8 @@ def _lz4_compress(payload, **kwargs): # Kafka does not support LZ4 dependent blocks # https://cwiki.apache.org/confluence/display/KAFKA/KIP-57+-+Interoperable+LZ4+Framing - try: - # For lz4>=0.12.0 - kwargs.pop("block_linked", None) - return lz4.compress(payload, block_linked=False, **kwargs) - except TypeError: - # For earlier versions of lz4 - kwargs.pop("block_mode", None) - return lz4.compress(payload, block_mode=1, **kwargs) + kwargs.pop("block_linked", None) + return lz4.compress(payload, block_linked=False, **kwargs) except ImportError: lz4 = None diff --git a/setup.py b/setup.py index c38813e5..529c63f9 100644 --- a/setup.py +++ b/setup.py @@ -123,7 +123,7 @@ def read(f): extras_require = { "snappy": ["cramjam"], - "lz4": ["lz4"], + "lz4": ["lz4>=3.1.3"], "zstd": ["zstandard"], "gssapi": ["gssapi"], } From b9c8415b0ca4d04122cc646b5fa5966bdf4fa722 Mon Sep 17 00:00:00 2001 From: Denis Otkidach Date: Wed, 15 Nov 2023 14:20:09 +0200 Subject: [PATCH 4/5] Replace zstandard with cramjam --- aiokafka/codec.py | 31 +++++++++++++--------------- requirements-ci.txt | 1 - requirements-win-test.txt | 1 - setup.py | 2 +- tests/record/test_default_records.py | 2 +- 5 files changed, 16 insertions(+), 21 deletions(-) diff --git a/aiokafka/codec.py b/aiokafka/codec.py index 015c2cde..aab1975f 100644 --- a/aiokafka/codec.py +++ b/aiokafka/codec.py @@ -11,11 +11,6 @@ except ImportError: cramjam = None -try: - import zstandard as zstd -except ImportError: - zstd = None - try: import lz4.frame as lz4 @@ -48,7 +43,7 @@ def has_snappy(): def has_zstd(): - return zstd is not None + return cramjam is not None def has_lz4(): @@ -193,7 +188,7 @@ def snappy_decode(payload): out.seek(0) return out.read() else: - return cramjam.snappy.decompress_raw(payload) + return bytes(cramjam.snappy.decompress_raw(payload)) if lz4: @@ -230,18 +225,20 @@ def lz4f_decode(payload): lz4_decode = None -def zstd_encode(payload): - if not zstd: +def zstd_encode(payload, level=None): + if not has_zstd(): raise NotImplementedError("Zstd codec is not available") - return zstd.ZstdCompressor().compress(payload) + + if level is None: + # Default for kafka broker + # https://cwiki.apache.org/confluence/display/KAFKA/KIP-390%3A+Support+Compression+Level + level = 3 + + return bytes(cramjam.zstd.compress(payload, level=level)) def zstd_decode(payload): - if not zstd: + if not has_zstd(): raise NotImplementedError("Zstd codec is not available") - try: - return zstd.ZstdDecompressor().decompress(payload) - except zstd.ZstdError: - return zstd.ZstdDecompressor().decompress( - payload, max_output_size=ZSTD_MAX_OUTPUT_SIZE - ) + + return bytes(cramjam.zstd.decompress(payload)) diff --git a/requirements-ci.txt b/requirements-ci.txt index 671bec23..678f9fd1 100644 --- a/requirements-ci.txt +++ b/requirements-ci.txt @@ -14,5 +14,4 @@ docutils==0.17.1 Pygments==2.15.0 gssapi==1.8.2 async-timeout==4.0.1 -zstandard==0.16.0 cramjam==2.7.0 diff --git a/requirements-win-test.txt b/requirements-win-test.txt index 7cc1c9be..da081032 100644 --- a/requirements-win-test.txt +++ b/requirements-win-test.txt @@ -10,5 +10,4 @@ pytest-mock==3.12.0 docker==6.0.1 chardet==4.0.0 # Until fixed requests is released lz4==3.1.3 -zstandard==0.16.0 cramjam==2.7.0 diff --git a/setup.py b/setup.py index 529c63f9..112a1daf 100644 --- a/setup.py +++ b/setup.py @@ -124,7 +124,7 @@ def read(f): extras_require = { "snappy": ["cramjam"], "lz4": ["lz4>=3.1.3"], - "zstd": ["zstandard"], + "zstd": ["cramjam"], "gssapi": ["gssapi"], } extras_require["all"] = sum(extras_require.values(), []) diff --git a/tests/record/test_default_records.py b/tests/record/test_default_records.py index a79f2aef..0037c76a 100644 --- a/tests/record/test_default_records.py +++ b/tests/record/test_default_records.py @@ -15,7 +15,7 @@ (DefaultRecordBatch.CODEC_GZIP, None), (DefaultRecordBatch.CODEC_SNAPPY, 2171068483), (DefaultRecordBatch.CODEC_LZ4, 462121143), - (DefaultRecordBatch.CODEC_ZSTD, 1679657554), + (DefaultRecordBatch.CODEC_ZSTD, 1714138923), ]) def test_read_write_serde_v2(compression_type, crc): builder = DefaultRecordBatchBuilder( From b3d8a3c445bb5013c525e87fb37c83886836b9d6 Mon Sep 17 00:00:00 2001 From: Denis Otkidach Date: Wed, 15 Nov 2023 15:18:53 +0200 Subject: [PATCH 5/5] Turn on back tests with 3.11 on Windows --- .github/workflows/publish.yml | 8 ++------ .github/workflows/tests.yml | 7 +------ tools/README | 1 - tools/python_snappy-0.5.4-cp36-cp36m-win32.whl | Bin 27995 -> 0 bytes ...python_snappy-0.5.4-cp36-cp36m-win_amd64.whl | Bin 29776 -> 0 bytes tools/python_snappy-0.5.4-cp37-cp37m-win32.whl | Bin 28128 -> 0 bytes ...python_snappy-0.5.4-cp37-cp37m-win_amd64.whl | Bin 30053 -> 0 bytes .../python_snappy-0.5.4-cp38-cp38-win_amd64.whl | Bin 28897 -> 0 bytes tools/python_snappy-0.5.4-cp38-cp38m-win32.whl | Bin 28183 -> 0 bytes ...python_snappy-0.5.4-cp38-cp38m-win_amd64.whl | Bin 30111 -> 0 bytes .../python_snappy-0.5.4-cp39-cp39-win_amd64.whl | Bin 29086 -> 0 bytes 11 files changed, 3 insertions(+), 13 deletions(-) delete mode 100644 tools/README delete mode 100644 tools/python_snappy-0.5.4-cp36-cp36m-win32.whl delete mode 100644 tools/python_snappy-0.5.4-cp36-cp36m-win_amd64.whl delete mode 100644 tools/python_snappy-0.5.4-cp37-cp37m-win32.whl delete mode 100644 tools/python_snappy-0.5.4-cp37-cp37m-win_amd64.whl delete mode 100644 tools/python_snappy-0.5.4-cp38-cp38-win_amd64.whl delete mode 100644 tools/python_snappy-0.5.4-cp38-cp38m-win32.whl delete mode 100644 tools/python_snappy-0.5.4-cp38-cp38m-win_amd64.whl delete mode 100644 tools/python_snappy-0.5.4-cp39-cp39-win_amd64.whl diff --git a/.github/workflows/publish.yml b/.github/workflows/publish.yml index 77f0fec9..1fdd5372 100644 --- a/.github/workflows/publish.yml +++ b/.github/workflows/publish.yml @@ -74,8 +74,7 @@ jobs: strategy: matrix: - # 3.11 excluded due to problems with python-snappy - python: ["3.8", "3.9", "3.10"] + python: ["3.8", "3.9", "3.10", "3.11"] include: - python: "3.8" aiokafka_whl: dist/aiokafka-*-cp38-cp38-win_amd64.whl @@ -139,9 +138,6 @@ jobs: with: python-version: ${{ matrix.python }} - - name: Install system dependencies - run: | - brew install snappy - name: Install python dependencies run: | pip install --upgrade pip setuptools wheel @@ -236,7 +232,7 @@ jobs: source .env/bin/activate && \ yum install -y epel-release && \ yum-config-manager --enable epel && \ - yum install -y snappy-devel libzstd-devel krb5-devel && \ + yum install -y krb5-devel && \ pip install --upgrade pip setuptools wheel && \ pip install -r requirements-ci.txt && \ pip install ${{ matrix.aiokafka_whl }} && \ diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index e10b33c3..fb026548 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -73,8 +73,7 @@ jobs: strategy: matrix: - # 3.11 excluded due to problems with python-snappy - python: ["3.8", "3.9", "3.10"] + python: ["3.8", "3.9", "3.10", "3.11"] steps: - uses: actions/checkout@v2 @@ -168,10 +167,6 @@ jobs: restore-keys: | ${{ runner.os }}-py-${{ matrix.python }}- - - name: Install system dependencies - run: | - brew install snappy - - name: Install python dependencies run: | pip install --upgrade pip setuptools wheel diff --git a/tools/README b/tools/README deleted file mode 100644 index 6589511a..00000000 --- a/tools/README +++ /dev/null @@ -1 +0,0 @@ -Wheels were downloaded from https://www.lfd.uci.edu/~gohlke/pythonlibs/#python-snappy diff --git a/tools/python_snappy-0.5.4-cp36-cp36m-win32.whl b/tools/python_snappy-0.5.4-cp36-cp36m-win32.whl deleted file mode 100644 index 42cbaa28fce272c51d8e31f12e9276d119772069..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 27995 zcmZ6yQ*Hk9+*$gmf;R~}fmT1MC|Ihc@LleGU_9O$y=YQ71Xue1$jDvEo$&?*0-A#U z|H-g7ws!cxWRz*h+io)<^+7%mhRHVKb07q_M}n)0qzDm_7rCDmp%raB=!~Z8Nh>sm z{k$g7{WZ*4h`1TknP1M~WX=8~ON|re{O?>rB@WqEhnStjQ)VTveB%cI*aAFabXiBM zxp@tZHmd2GCzb8X@ueM<+IMuKH52O)7bofZ6oPxb(a(!7%T%v)rHbJ6fA@u?Q#rUwYhGV9H>|m8^i}KG}X|r&W@~SkadZHH!@CYo@C~7(B*; zMT5gX6)%@y|MZ>!$u;J9ia*gBUz#y%8x$nn5fLySj z<{w;K*~z5>qc|v?!Am1DbGy?o6L6)KP_JmB$S&#`nn9*aiOi)&U1H*Gcf8D%oqX6f z9%~to&%@8r$b@kt^H}Fn*eM%JX*B&d%uNPaiQ>n@M^6~A=&{?Re2~S`EX95DeAzQ0 z6#f#+OUwG0A==CnLsgUP*yHVx{=l;5b=cyLT-Crc%5p3L*BR{9Y2WmODz2}Xy>#Zs^=H~2< zSV3k&2rNi0_!3AKF>{wj?-4s0@0}<}=0WtdxN*Rs9l6SuS(qR(zNv#Bz5Ut=rG0=L z0wQK>(>F%2j_HiFX@B|>8hGa~$q<)Q(=099uZ=T#8#d1^;ov|{4l^U9O+12H5JD0( z&k$DwG#Uz}!3a#3*%$Et9Ml?W4}}V$ID-Sm|BOZr2nhdw^PB(McNk2aSUKoDtsPid z{&OC)Yd=^oRrU5e1=pqgc5Bx`4Hz_Tp(GJm`TzDaJzaoGgt?3nX;W!aNavFUkqQPX zA_0CPZU&BL4sI^!4JR^7cp~^iLAO-AMniZI7#;mSwHQN2&w=_x1PY z_4hUZcFVG$?`89{ZQbD0SwY3=l!Jsaj46qaKx8w@%{)19f}E7h?LoHaD15M^qV+FE zKY!VVp6PK8U3?9F+Owa!xu|GKyrK00{R>2YdGLH!NmPSlEyq|y7+6&wSv#Q5QigHe z_`VErHawj)$?0;W1y#ME+HU*4%Zb%m-?Op3%`}|X;cvfNHx-?B4$>KN;Wd39E`_h zfs-uwhPr=kzdtssSkG+bQI)`TRzaoCXf7WMqYLiEeXsy?AThg>;MkRr!gpJk0}m7X ziLG~H?5v+>QBExAH?c3bz&#pTGzd0{IDXORBc?T>UrlS5xr>F0rV!?}k~jqBV$~2K z{R`7sN3_Cm^g_7bWvbawUDb|~92!KyxI2uy(4s}JK5CmHZ_q2wIn27st|84KqN|J# zCpet*xt#uX>z@X#`E;v;mAAD~wMprUJ|zvymAy*35{R9C_Gz>%)T$mepnQdVza3z4uQzU;EEv*?t+0)ohA~gs%v~j zhw{_ZrrK4d-u@1G#cAv08yG|@yN2_ps}@ygFJ%pi3f8gh72J-ewW?3BHO*e1#I^G~ z-5n;6D6j{Ms23dU3n|BGt?{K_X2T#PMLmvJm6mfHrx&aHnsa13PlUx;m%N)tK_KBU&ZkvOENkA&gK z+r8+U><{VNlnV0&ApABhk{%s0t6-cFd5&8F5d#DI*v>l3$)FR(e|F!vZaG}9Y{Xv} zP~&3+n<;;XiNUNOczPZ8mAMTHmEQ0aS+^Q=WJ9n(sLN}LB|K>wE-o`6Jjb2kzFY6+ z<`k4wE!ciSM$TP%t3R!5HzAzP;pEmhjL75pguX-kZM@n$gpo+^MO$KTK5iCF8Nld< zLz@n5m?}zmjv(%N#uv4+d(fVzEFV$Wq@J}>jmN1dX!X9UMeL?B^XCISEft1?o$@K| z9Kn6DkS-Nzf}gHw49YdD4mp2IdtOhX?Y8l-6z>{1 zVY>I2-c;N>*B3hzk>F)29e!)kIRP+MOO$&P#43qTjaIWtE>h5jTxsnW$u|?~ z=c+w4Cq@|N0ybBhB_Xs@XsXmiEEkt6Hi^+;GtFpJ4&+8OiIHaiJnt(}Z&Rdp z17-}lHE^GEdgo+Y5^VJ(JYm#n9>bIof8z${lI}cgypONU5APJanj5<~&B%Sq^3BN0 zcaBkJy9;dmogQV4{dferaJ)!(DcHn-T)1B7-s4)tBL}GJ{-xN{SJR;qaG@|gTh}HP`mau>Qczq1P0qTXDz93 zwKr#BZXLfstoA=%8-5;h*BetZ@DObf9Fniw_U(@+18*xkHT2j;)F`2Y0q? z0cm%;G=%*6is_Zwa~IiYM}<=m!P$nwne>#Y&7`R{?Ho1H+I|lrh>(C4ffBWxR!S~v zK!c9vQPT*m&D9+n`dk&nCK|`i>6B$j#_~aD_8`@b`4iF!eB4d~;>rJ5E|642#_vnU zJAGqO1mRNKg<3e~0t z4C&EwQO&{$;<>Hx_A60ontW35Al1Ywj8NP$i;C98hK271&baWP%S1%LIt3<0_fnJM)R!2x#m4Vh!=Sq0NSw^p+6N4l} zDf!@7c)mCaR6~v04o4FQ&zKa@+X7?(`CVImC|PKiJ@`t`!r@NJcOb3iu5Kc>)243(LXOi@H2L!xz&lVP zljE+2r*`-kuJvI!x0P=g#)4(63*D)G`V`w{l6$sGZI}PZSer?i2TRbNii{DZZYc_9 zc}<^4G1b3OD^M9Ktu{G^Ka$)$bbmb-E#X?>rNIS{Ig`tFH)DYg_Lyn$7XNkIEd8L& zURk7A?WY*_)+AUyCImN!RK%mjG6{O0-M4@%&KncnFWa9o={AcpN2);z@HY+ZL>j~* z&yZQX@1iX!DLKqtQoqZ~L@+QnFo=TqnIqZ-vG<9fjcs&kXi$krV*(YvQ*O2H1qpR3 z6A8%f)ol}`ol(akz72=`?SUvh!hDjnKQyi@8ql|f4jBvct5*To2a}tp{#1U~&glck z4k!FfAEX1hk@xV;?P&MG9y*@eRuIuZL6jFPzHSmdK{+B+=EH*L26K6epBiY3S9n3% zo53O#3_+8Ltcd<2`e6{t7D!6(#E0=U&qJyLj18U|zySBD`tbo3P;=fRpK}Yj4T=70 z2GQ{_vDv5Ugm@fe{HS$8@kE?q(L$Aw{#wV0lN%HGmQD~5OOTjm7L9(R0m|75Akasx zp>ELH)U;MH2wYN6F~DBM!H}%onnwwol$_B|A?`(qXbm1KT$r653cUh!TcUy?awrTEv+Xl=K3s(#+2!s|efb@C#-9kRIdjuov z{qE$1`_CP-piOSJ$(^8FVWo0jsi|+@ZC-f{2%^O8cw{R0A&N&HNX`%+C|&w^s5bWE3>ZD}}B64$8Qp#`^O+{%R7vF1xVdnuV3zL_+_)##Eo@__1{ zi`T|FaH7TZ{6yctG|xM+)%^XS-UaH^Tocru zqMlPx3a#WQiUxJR!CY)0NUH6i{-(*x{pPe+Lb=k`@Eh!b8WQSqt6HC05d+BtojrU* z!(@)^!4K{9&&WT-O2E&IBssG&a)6AgZASCY}pzkEPEQ|3X4c*r1$lo z$(>9$?0e_+Ph%{(EMRz5d2&V+{it(Y@$EOdMlf+ziY^;Ihp{TH9DvgDce>02C!_p< zfza#HbO%3m1J`ul`S833X50mC{p?Ejypk#6H=E%T*;I5M+EKuHVdy(8-B-8}0f;fL z{YiL)!HQFWV9s)TnD8E!lVcJMs{vk;E*QH#J_Fd(@p-m8dtnX?Y10%qH^#9RAP+pH zeHE_!!*EOviC~LIOB)so$qzw|RvPT*SM^*JAL;3W%&l74lZz5!KwR!pI=36*UOTrL z`yA1|``MuA{q^VHHMEcTtcD~E2(hS_w0_Bqx1g72^KDMB2;>lLKP*be=b}AjnM=RE z+Z&F|yIT+^AFL9e?A5V#YPTzCn@`0NE5myYhMQbU`;_A%&eHGlb#ib%>EDA+1Zhd_ zPWNKS12V%SO`QuvU9eo|Df_pY#%q;St_ej=T|&1Ewr!CjJ@HNz>UE2;Ye?3EE``L8pZQQx{`FM%)hJ%EZYrW2k@pjt>%Mu zr$p?;w;;S^CKW59r==?gf<-R%89QZ*CHgB(_FdqsS1R5JJv-QW7NLICZbsWps+6?$ zG4>L1`)w?MjQIV|HpU3W96)#P#3bq2`H)$`e%4o#>_lG;O5+i&Ktu~>NFz5BD9eM2 zoqB{Br4`&LS!cPTv^(2#8I=0yy?JD@4tAwG?+)=yr;6M-7wxe@b-eq@hg$b1`O*{( zSyaQ`V7Mr*O`)7*;am;v$lCl}B5Z556x})pb>*ve>lldJC@*WcdnIk3e4>~cUWWg1 zF@XgqG6M!mMg7Dg&V%H}yPe8?DgpptH<+-9jzZ8#?jm7A4`&QJeNe?2nw&b3fJ&KFz5U9 zh~2RLBlXG%a&SWI5XFta;N?y(lnWVrYA^31g~^YY%S0@~Dli8deTvC|GO&l;$xydO zjiLCQ3F0fX@F9RsCg1sX9j5Fs!9!T*U<6mg`TT9+)q z26^2c4CvDqv==nrD#-VL`6pX|k=9!9?7Dhq{?I}$+D$0AqgdI_C+psTTlN{U*!`O( zx&!H0+cQz~2|28LAiyHy$$gPOi}vp3=_hKZGr`HSW!4+s+U4A1Dgg5OQ`*iy60U8} z8v&cL*$$7{iJZ=iSa<<_fvdBy2P1<`W49=R$0bAgmirS@ zXO>o7B+B(xcM;d~eRbsF%#sS|D%2VFYqSU62l6`$v8gIX&j+sb0}ZZG>u77}!tN z`ZVI|ai7BA$sLH;hchc8cN)Qj)(m?zp98u-Y$2F0k7uH!4hQQv6P2oy;S0IHrdcd8n#w!&xN8vUu# z$O1-ydO)qk*V25ev2!P&Ws6Xd+rt{IQU*^SknISa(sFq-ebrBBxX2BLI8nP~w1&k8 z4$Zt9Zk4SdIp zg8P#&O6Cgtu2PDAN$#W!cy`Mz7K88&ZbW%UJz?T-ym^{3n8l(Y{%o?steI~-`xprf z7bCwtSWFg5O&*Z5E#)))!e3&l)94LmP_(Ywt29Usf3cK$M z7c%j|HDDFVp-tYkMX?OF(!ASbei7XrpTUtje1u$-q(3Q#2y8?74+{|nY=|apFfcYL ze=Hass#F&VE>v|gNW_1WCOj`Z_ATD5s2*&V5Of~h+X+krgY4-PB)Zj6HCS@%rqVgg zPetg3d(DX z127$Sop4}9z-zKIwn**3swIuccha_EMJ^I`(dM>ye;(3buZtk=?XiMD6tILEEA5&# zK`mKGgHH~Pt<&g%mj?3Djg!v2)rDrpy113#8Hmq!WpYR8 zs8IEP;jg6TicuVc5kT{&en?(1a<+l?S2R2D7)DJ2k~ov?m*H1-xL^$JGm5OC0l2shY`I2VbfvWLt59Jv)?g4OJcrH8=Mn>@v8`NH*!zw~WBV8Yo=e~Ji{Y~FaJ)95~S(myR! z7cY;23Dc=xr=SZLWK_OPEUZT?61E4T=jxXKPHt))P2s9Sxtg=grTtJvCfaBcz{`kK zCXIzCU=ItH1QzPU(@*hvF4wp3^?(MxQ!Y4Iu)>xv1e$q6UgRN7KaOJ^JEnrZ_BER` z1*b8JdgBRF)pi$)tV<9%gXV9zkt%WcYRp;C zp^KVIGr%TJjTjC@t%RAj`m6O+BGpg!hFltOx61?GkRZt*MOUV~uLlCV7 zP!OJQm`yA}_gHZWM9>zUy%I54>$&nL&~Ez|-KYlC z{dm(Tl;a!%3ih$&wgzPif2z7gz3Fe_hw6T`uS4nhc3=6&N9FxtUMm9XI&Y%2^t5O^ zAQsl=t&LnTU0Ip-@F5$DEh-aiTquOO^&YV#c|mo=&vq+x@b!4Lxd{1%F17oX3|E)p zVdXdv&jz*0Y~%@5mIzxTLTR$=rdHI}ZVI?H0~a3ZG&##`84f!`?%74!NG0(J;e#*EF?Khwc zqZ^r`7MrRyH=Pk2M+hW47F!Xgj{E$ZqO^uzXsIm_@;K89eHEU9E(x+|=R|ud+|}iKG!B$lus2$qGP3#=aAw>B*!Whxxg%Ds4xlpbjmK}A=Z*I z3{y;K77>N``a0jUIr9xhyt*x_58dMgM*~vO6bF9MDmcd9f#PLa5!5|xjxl9zrL(L) zA{6`^XY0?5BmS zs5NnnH;tx6+tXa%9{AVcrW*pJHTGvK7dxR}ZbG*6;bBv; z;M!9eHv?^5Z$@C)0xu)~D70XZF3+dqrY8VHhO3iCQ7>)Gg#g##DgF z5AGNxhT?3|AxGn|_c~vqeaOhj4(1ImAb_b~UoPSZ^-K}9upF)$BO@m4zB{f1N1|Q@ z=c{PpSCHV>JXRR6q@i(@oan$MLV%kO)*SC80LC}Po2 z8ebB&94!)LyC88|f#<4kQqT{#iR@v|LFSpxpbuw{Oat}rpfR;>{JZR_-Cw39F<6Zy zGIGz%8k=U$yu>N1g#n{YNhyVtxhe6im;pDyiVK0h4-FZ|xE+h%AwP0$q-Fg_xEFGM zJ=O^6@I7hVnMkBPy^4w9a__9F(T+g0Aehijb%9Jl&fL-W{Ykg?^KI+Tp#rM<# z`Jr?9;l+M_mUx(kV1mx@eg+wz;Hsw_?X=b*h%8X53$Llw`t}cUxBO+#C8mGIIq0%< z0{uz1B9+WGs;M~FMy$q3FkBa7`)g_E*kuE6wTg7C9oB7Lv?&sbJ4GqW<%4f{cRf*X z{>j8=yl3=N1jM7%|NSihTq7IwjR&G+Ua!pQ(^|jdZ5ukVLcS~0$=}fnDg3F75I=Yc zo=OlB&Da)r`}#7GgIko9G*FNkQt9$X%2N`W&QGY4DJE%ru0>kZy5O^RvJE z_f`K7EcEN~tP1WI57+=s6~qQ2Ds%@+vJ*MHCp%lc{hs`+dO3xzyea73HMa6Vb8~h> zZH;Yhbt(;HLqm%e&H6ojd7EEem2umwAOCNJ#I9Kr+Y*3sg{`)#wx;UD7AO1at?ozW zvs#@3*3s7PsP1_X%=t>dyElGqmV^wm;Jxi6n+Mf!RSRIotX;TnNXPYcE$<5yWXS_+fYR8kd;%M{A>Q(1$zMvq+_M!)_en-QBC z$pYMXR=uvImj|Tl%Pjr*BSYk0>f-E;H!b?jrOt7)rjVhy@gimt&u#a8m6V_5+kWlirR0-}Jf%e9{i&cxoHqmep(%6vzKgpVtR?}FTcbAQgBy}RMcb<5t}UdcQU zRdG{IaLtUAyMHC0@~45AgfxyUcw83zU+d?Hclo3AJspG_?~#xJJ-iROT~|4pH}v(q zCw_T(YCkcJCk^kj9sSLFks;!QzhCM^Sv)EFWR{MOKph-_vRChm=Tm~Vjq;b z`kyn9GNTaI@VSgiH5#gbcO-ue+(oz<+PsU-Rtq1OGPvWVn3w zZD;&&Tr(0j^gNwDG{uw!X8AtmwOHowa#1{&!4J^Qcxp_g+ke--B&G>&bs3i()I}0> zz4hQDZb9N0WVt!6T@IZ7v=<7vTK=pr#(Fw_KUwmEc5KEpe3`oc>pmrDKN0xRSRN9k z)wf8(I5yfQeC5HKZ=T&_bWXzFOYy4aonUlYs z8uEX=3v%)|A0SUdSiGy^5=znR+UWAndrQB+3J$P${W89{-+T7gBIsoMP3%2R>zd?$ z9$hB7ni}}>-Zl`eo7bRI1`|TFrW1qM7zP0B?;EtO78{aF88n;!` zb^Y{4zwIgJieFQi8r=gI953Oj+?p(oFqhN29)MBH{?E^zK%GzLZs+NF(+o&IXP50& z%K&T-!_yd5#Gk2;E*mFSUrjyf#?B|}`O!hldVy}1+c#8C$=pol{VRJ(hXFiQ4)RXf z_20k2eK{7k&AzN|HyOV9AD*#`5_U+LzACQ@~kExs5&+WOs z$Vku}*m&^-s!9-G{JD}NsC_J0L#cf~Q32o!JS{_H^y?(``Kl?)hsHi9HXP6ooYsu} zn#b>4UH$ibgk4wf22h)YBz(^Ia=>%tbNC+1e92^V$l-O=yBj}vzY;v{^d5daF^O@zkcC$9N_fWy3P6h65Rq%uqP@qy_I7Q4WQbOfrdK3R z50ILEwrXC-!vpHME0T5uyUeam@UbgwAO~U_x?T|>32psfLjjP2%!g&A-T%REB3yni7w~gAA!lD;P*hh(!pBj zCLV&rpy7@{-ieB#;0{9Gj1CB~X22h1P>?E*QvINeK}9=}1X95uB3z1kW5B?xxzOkT z>tPWpY!o^%D*En^) z?e?RAQrVlpk2B{v*4b7b+*5=CN$!5N!K@k6|*$3`d$B4W~s z6~JLEYT?@Qu7!eyGClUt@ux6pt8>c>9vWfM3CBP-Q;ELf_%MouEg|;MeN<3w;{44E z1qo+<>=8N)UVabdO$-Gqdy@P?nv9AqTJ|7C+R2I~W9Gl5rVW?fRf!eJ9IJTq-)o+8 z^A|{fs9EdbFq&$s^JL6TcM7RPWDHG{gs8c8a`(m|YM8rZ_4Wig(T*d{6%r`C1G$3M zQe$5bIVZs-y#VPQj*XGNsRFr*#^{N4+5-_vfnb&Hk^ykkb6I`Zxst*?0gkHowY|{o zx|f*3Q%OweDZk&afE_ma|#C{3&DT%{=pw8^@5iYGf=wm-dxJ!R`f?Eq3)TK!d zujrt4Km0rmD#yp+(SpL||HcG^u8BeW46T?`l#&Y~c>mz7x*AHx26+s%nczfy#YDKv zjoA3dKgr*opCsDHYM}3+HrhHL#=CPtzRIU?rz5TL^$7eo#tPpbrSPh-aWK$RIfa?km*Yd-K4r< zL(`_;K>&{<43Y7vK=~EAno^sxe`|N&dR{b9NMn8V`8+!rjw%DFNXYE4%1gct*sA9b zkRyB_B(sRMNRGQeOdt=u-NU&=u(CSfEUZC^ep<;r$~=7>k^^`dz7zEN5{5aX1-UB* zur+O4Q|Sf?@FT|l6m-^mf48pZd`{7lk_A4eCF*JZE)G-%2Jr3g;xeulqf3tY1w?ik9f-Gy zN~>ZF<0Y}#V;%i0NFJBM?xiocRLVG@8h$u8bEWX>UuP{78PhNP&y}fegIZBl8nTlx zF-g-sEpc^MOJ#HW+RKG8F}GUlv$57@9V?r#{Xch8#7Fq-$%QCYAA|wVX!-iUGaMiq zIucTRgD(kT!7ZG?cBfEUtb-b6yHY*Bzw`5RK}D{{pAK}@dZsiSx^ii&NAO|VbmJr` zby7TdA2l%f_d4Dt2_37E? z9qbx6X>bcY(uS?HEK4nMmCPHO|1e7I1DcxGz=T-RfP1?x?I>!ToBry!CPw>l$7R5a zY|1em6udkbnCay3`sr1W_Lo!Dk_l^wx+AMtmw(;EjZf#p!7b-SUVxaRM5CPr!#Opu zs_V#yfq~LZH*Z>1wra_by_E9=`R}*jPg>cwzRgHxul<{ehwmQUu;Cl=l~Z0as<*4G zXH`d(&eBx1WF5*?Dgwg&HG7s1AW~`HF=1xkfJ+~z*&eFZM9ZOjW``%LKosMcJk{+B zmyjjEns5k~NkPy4BUirQ&|J%!NmtVtiBm22Fa8Chl;Fkm;;}}06socm5$LZa6upjR zTh>La?S;IyM$$r%K3kzp zyk(isi&ctP#VPQvSj=OFus@3Q9EmGA7niu}nNmE=9KkH=URXTgLQIY7xRDAL#wk$2 z!E)coy*S+?OPvTN3CulR*kHDJ=Y-3BZ|&~#6sGbc)e?mLO$cIP$l=>{6kak!2E`Ip zp>+41jCGJCW&dJjG0^83Cs)U9$+EOf#HexQY-i2*swyN$!^D!*F;Kl^Sb!DSIhcI; zrLhAzvW=*XTeD}z)mtaxrEK_=Z0xkuTX4`gIn;E7d`!#CpK)juXOo2e)ypH;+0(}( zeAq~(@xm83jBFy1Y({Q^D@EckmTi=f-@I%!hSoB1#N)HGtFPKGapcEa6mpPr;f>7j zQqMrE?5eZ$9mlNraW5=&2TKOAE!M3qZ7pRXx)4U}s_k&J;Zwqqc`4^rv>m51YGE&9 z_*(tdm%DK%4t~3S4%b}|4d;gnQr_2KNKX$h8*c1wKR}X)dIYdD6*~KtBTrxCU5$T{ z$*++K!cIA)Ks#iPLGS$B#j?PCvXa`pr|a(Y;G$pCGc13-D3{tL3tmSiqVj z4)qQBo9j!9rtGS_r`7o7YRmWQaSs#f-{*TANU+eE%4GuA!vdSk?oP~id+|^hAdRer zeygE5VLJs!+|3sPi|Kw{zcme6LJDdmrG7ivVH94$IfEcsxJYF_>kPMpOg3{*K6rLN z5h(p5I5b#8IoZ|NHAbpw{UxNGcdw4H`RaeFX)ldzqzjtn%<`BiNy(_{mTcHp8)|hY zK!YJ{Kj3=Gh*x1i%lnsjdiu+!maOdc%yZyT9c;?l1U%wBAP}cBvelV1GZKy>N*x{- zLxz9BYrX~jg~QH+>C|;=zUyA(?Ry;*{ro~`oMjI>qG^)}WIjZf`b+Ce52Xpo_2oW@ zqvaW-+sPUfce8s*-@A?+lg+6aEBx}3)3bAIr08Y!bGnIF|6IuVQ$6pm;8*pi7RdWE zw9!95uY^|JmJ-N+b3NN#bX0{_t*dB%YkjNv=LXaV1n<@YN`JF{JU<4`4Oa-40!v;M zWo2j;YUD3vwz7X*e3ZsZ;p8QB#)sn`;3UUOumdN(SWE*Ye8-QtnO$Q}=yFVR9y7Qj zPyCoppK=>c^s+(&wIY8zIaNzmkE}6lY1me?b!Ks9eP#*S`m%k62Pj(u*FCIyS`Dok zttQ6gqo<=#58B7B6MPx!DQ~oLl`YrU&tEPPE_T=c9;V$N9+S^;0R+EWZ+Z8MW4F&+o?!@ znCOXVgj13-lhTPz9sb*H1{=O*Xo=@o1z zf1y2%zWx;?`z&juu1>v6olLEw?kF3rC@nWDua(kHQ^_!q_U1Wuq;4qfO(~Z5Ct)`T zR11^{)SfRpF3nn6Igf#}m#`Se^m>2;#T&r;&~p`EE_{{^5a4DocbH;S2}n^%|M0qC zt-7uFI6p6V1D>OXuYg~cu<@?b!7YnlJh6IggKeX2UDavPC`nwWyj*_D`oQz=4=CC# zWL#rhWL$4p&=d_^KhJsM@NWs=fBM4sR_B*CT`pcSSz20RS$O}X4JYr6FBDe7s`O+Y`7ASFM=Ah(~)-@BW>kK14OWA2G^X=}+) z`d0c@UNAX;?_2L<)y*6t134Wz4H;A*6;VJf&vDK|ybiJ5;byF=Oqr z3Q<0*kv&JVnnR`3^_7erqaZG?;4pNJ(dURD*|!_X1~??c&cNXW`7pvzY&< zvAyp_RZ5=JjfrNN7#4w7_G>(|rg@cY zAkKi-^Urn8TgG&ts*!?XJ;31SWh?N25HR59{mjrG@C~SWtEkDyxDD(N&gT35X}H3D zxqf`SRZu**%<qT`-|^RP2^iWzhYCVRziB5zAnR;lW`T*Ye({V z_qFy{%7u!`L*6-X|FB{@eAf>QD=K@ffLKX-kemcDXbQxS93H?*sygH*G0}sRo{Lq$ zWbvM-6jyyUKZFM3ODZ;0YT{d^q6C#@{C9L5=cL!D3L}DgmP0)~8ife;+6Ypgxh9V) z$>5?mbvx#3FzyRP;hM(XQse3(g$j{w-y2~iRH!cdAv~~p8z(U}##t)(5k^W%YI2#4Vk>&7g0~Q|E5jP* z1R42EGdZy0zx46qbX!amND$f=9c1>sG^J>#(MS!H)Lts|`K?w4=D%i8SgEly zDg-Mab2~iVU;@G!f9XkY0uDH3~{wj6kmoD})ZB=k~n^ zdX@AuLF`(Udh;Gynn<8gv&Ghz$R|`d(_yl_BC4!`J_i-GXVYGT#U(FbaFU~IjT||3 z0yb3``%$Ab4w5fw$N;deU_}V#^PuBVSLx7hGUDy2_?a^P&cCtH1mmDiCc7Y#9f*+U zbw%iKx^1&3q&q%I$7%n(AB_i4FFSOiNd>%;il>SnR$#u0b~7K!x75#c&KASo{d=X{ zoUwvr=G#j&;%EYiyh6QE;VTvuN#0W)sn891|7f&r3sK>Xq#GlF(FPIe!V)w6i`m?2Y$8%dU)POhdKs{2}?+dB+c ziyrnP`r2FO|JC#CVxYrnEH$l`rF00Zni#ZhhM`IRdyh6jvA#AK0bR)%3vBvd$ooCT zxf_Nq4Mir(*%7cAS8n+4qgEsI^tm)XvBYNYJLvczO41$QQOD-69+bQ*YAHh4w6~)=)>FhLb-D$>F#og)CRoYc`D&tZ5v*5xtoM{iq3D zfjt5S4Msks673Gw&Xi)K%O6Ju4J$UHxK&I8HiiN@ZJ8{!W3rWSq#3;#`lb?48Lz^zr_>SEzv;x0Yqu;6zSO{6(&id*@@^ zMiIhD^MkBmx3zpy-@R{M2Kdo7&k>^~Oh_i_m@qOhOYJmEmU(e55eB>meI`6m)IF#e z(@lQku=UsIJ?k|GxT{tTY*tas%?W$z@d%VBW_3lKQt^?uWkOxskhPHDCt&fqyZGpnQ-;*kqPSpli>_e}AjQ z8oR3Z&8Q9m#?5Ac0yt8$_fqUTrmSc#s8B4_mV#^EZPvy8*)U81)DB@VM zx32lS^a-FC@ z4X4Uj@L6h$#wKG`$$W)e^MEyqc;ylfrvZK1hbu)wEdil^uO5*3vtP{d#gUCB8rouR zt=EU|;%9Ev&%$RK%5rxR1xmO%X#gf1u!*()XwJi^B+NcPqR!0rW=(V%Sd;eAz4pO{ z#(DfaSkGX~h;?Z0q-Qk1#iOg>x@@P2`Aut3<7Fr8bK#ILTD91^`C;%*glhIMY11B{ z2PQK7p_fAiWx%WU4cv~7CKOzHBN)^a&A=`eoWu9Il&}z=uJ8Qw9x4kZay;J0KN939 zd<@vm9Pwq=Lvr=AP2H;ok(FeUavBa^(9_yKJKtpI_G~WX{SjG@ov>G}2RD)|63(5I zFm&KMDgK@t&^owDS9b&@)s!4S6>?%?Npmg%2)TPu@)&T9Dj~@o_Q~BEhi&Os)m+!> z^KR2}ZVR|&#jqejKEMVGLZTx{yC9JX&M- z-Xx@K*M2%`@0(CvM*PNObel!^*6p2slHzt8xEEMlO z5>e$%dr_cg4Vcx@CdSJHjuZCQ7}SIZ;CbMmZhwa!#~TEJ{wFTc|Ho2>ZOFIFAOQgt zvjGA9SN>mi`+w!6Rb79_t#RyML81N;bzqw+b2DD~p27HLDi&1NHgd(}6bt>r#g)G% z@_+CW_bw%oQ@~&$ zY`k9n%=ctxFc(CU%nx6ZAf{vn8)!hy4QED|PJ%EL2*CujTf~y-LPz?;Jd(?@ymL~! zgfz>X?1QMGz(u}eTT;(!80SB7%eEYua!q9Tou9rTLgrG*2!!9m!zOzXED;kU)_Db( zczu;iWE=e*UO*#rwPO%N@Ngk)MN#|Zd6$ya;Vi2?AVUPbF;b<{0!sq>JB~MRYXn3$(0BTsE|1zQK=;Xdj*fL#n ze{4lk^nWyV7Qk^VN!}OBVrFJ$i)Ar0lf}%;%*@P^#mvksjlg1N23u$`p5E>5y|@1M zyLU6uF)|>!Pg&X zZ2IxpcMMOq>revbzM_co!`l`=br7TVyd7>Wm~V^Klv2B}qGgNRD9#_M*zG9zkUM|P z2#+0G>~i(8oJrmIfdI?+6kgO+ljoo&vuCyMT`e5U{Acq{H1V2!H%=p z?YZR>*uTN$^uJUiG^!=I){9IRcY7#$3|rxqi5zyByw0`Y&(z!t-J-~4hNEt6OH1jd zzYjoGsY)4FjS%0>Imt}XAAkmB{g8&FKUuc`?Q~5h=8Ho}{yK`ZMF=T^>$kzpK>&U$ z_`r^vY4ZrxN45l~W}a|RDJ+TrCysk^QdI4}XcGg(q%w6I}0%FQPQ4gxDTJ9R+DfQbOQ*FCA*gWnc7&ty6Ul9Ux% zKdN?vUfrknz(H>)e*}UF2vfCzToa7f+;5u%-S2N51edTmyLSvGbMcRNr(PfKi|y*| z&_K<-aGg3Z>V7qF>UP;#3ifsU#yhe%mfon=S#E)emS6)9PyfJTxzi=82|25&UR@9T z8A-`**Mj2_(9xMhR6DZD5ypIae7>R(S|>)Uj($Zq{_+q~(Z8R{2%j%V$jS+8p`gpn z){EV0Di$+HHTf&vsa^#$w8B6noSMJAILW>1ds}I1b|rC8Y&|d*3LtUh%^B=SU{(x5 zHg0-=A46+B0Rd#g$|J-!X_Q~!{sC|w=RV$4F^*vr zvzt4CVA2{XwBGJ#d`pEG_Sk2dR-`5JVZ_L3w#)BC>6-%a4ETK9jQOC$@PR zHKKaWqaF}2{QSP{$f&kBVR$GcO+cn-6L>$+@cQ;fJ}ECn-!cf{v_c?o8V~bDZA)nG1IWAeJ-1wj?(l~SV26K6XoLbc*DfTCgi8bL}G3_do!kC}+9j+VtmoLw^Qi}EQbz0{OcDq31ITe|5 zJbcaLx_1l=;~nRblY>VHpE{2oOXjyv!!O75E?rF-_(A(f`aq;8`&hG9wIC|lOOgVKsHZlCTSJW zM1?rLDdRFMIigQ_fTcyt1P8XTCqDV6)j|!AB2p{7sk&WhkdLk4x7Gt+z^I#OEMd?` z&;(lKL*#Zs5i*y{AxCuyJW=cp8(hRA^M#90HX15Sn|7Lyvd}6UwYTU;M3PgaD&-NQ z1Jc{DN)(3nA+bT;&D9KXV12u`^`JnsBho>J9Q4S(?mO+nA>|c3;1{hX3KSeG+UlCI zV0H_e{J^mlc|IvdsfNeT)J3asNU3etF$zhk`A;2YF~NJtwg}^ z6tnNfxjMDR^t@55f3EVtk&c}F-g>L=WXgsB5`d(Rb8ml&q1ta!rw(_t(nH_xQS%kwQz=^^`V-x+AjXI7BI&O)0`9s>db3 z6lf)uW2$ED-bC4%euP0kA|$?BQTZn6RZ>q<|0qWb5p5A^fsHmk-%8pD2?SqVD><50_1{bg4UtBThoNi0KOpu9-%t!oyya? zusCh=DG^i?f*p#Zg zEDFIPxB?a<+D|rVFK0f%mdpp}v{>A5Y6YQbz15JJMF!czo8R9^YejH8hSP<2fwMlW zYF7o_lR}Qq$%nIEl4k|vUVV-0Z5Mpjm}I+V@@0N^i1$&Y=9{o={n-Rfoxy!feydL0 zSzNekS?3uI&Y2?9*Ig#Ei@C&-QrNIC1T>v_@?C?-VupOPlewx(v5ahXB^7L6EDDz; zDC6F(CB3iKtr`4DtoqilSfpWE#6^}St67A_LMYv%@fr;>v??+-m3?37T8()+JnW%> z>}f=fXEAn`n^U5h&H5Q6m<-C6PuJGdCn~`5!s>v zTt+-@4z`fiGZS!q;x3%dU24cvEt;^C+MeDf1R}Ht)F&0V*wBekClC}8R(J)55W9HdDNCx%0J4g_a#Czr?1hv?5EpxvrD3aRyE2!4y!`asKw2DdE``YGJ5I2skC=b3Q7sQi#J>J%>x!ZCuOM}qpD7zly{XOsmK8-O{(cO zSs`AXrdfj23XQ1^vi{q%xbR2BTXhb2WM(q)RvynfE-Ut}`#$Kx`Dz4|SW@4@V|=9s zVlV?nh)!fmX$i)Ekb2)zWwVp>bxLHgnPK}GR5Q%(j|&NkD({Ud7`} zuKhvoW?o7ATdL!*ZxYVCegXxcTzRs2^XBWG2d1WgrTMwATn8ykl5}Ay*&A6`;-MHR z#c7xOM+|OJN=~d|a11+>V$Ml{R&l>FDb8v5@|_}ui&eL}uzE8^`tGTra#GTyytq8O zORMI#|Db=7OmjeG@rz(+v6=5f?PG(<)O{48v%j-*lZfqIRpAoU*re*1D4zH`3Q%l5 z^P4h6T_ofA2|FxX_xk=#58wPc$ynNU{JLt!4;i&)?$7U}zoOhxZKf6;LIBk#a#p$aDq z0t8h4F;s*0d)`6U(8R?2L%gX;MJ95M5w#QYg+SS(R(rknr%p;R{%U=RAoeG^B*dWn z(dgckDAJb9wnfPRIP!DH9VgqcyYYu^;<=x@PYfehiTFLztTSHmA)%_kXm$r))aFf z`F5b2UUanK+cIY>{@jQ%0Pb9M-o{C0P1F@bDuw{vg6`(m>-(-zUBtPS7hU zWI~;wvM@)S*Py;@Et{egbdC2cWTMf4jn#3?;2Cz(+W4@mJ8~69nWMMX9=5aFvKiW( ztJQKb$Co`{3j#${KI+JbWZZzhFHuUc0q9~+0-$;`jlrv zJ-O@yH|?6V*DbWNbj?X@E}0u9`Yq-oPn`HqbpK3uLFastcDLg%6Z+8i^F;qQ>9BvK zyZ*b`=^8kjTNxQUd^Eo*bxT{E5wur&{&)Qy5F)Dau}*>}x?Xe3-clT#XeGi=S_b&8 z`QVg_+XHopr#<)Q8$@CV+>(K_5W?r*(;W{tJeBGNgZ967QCa-3$6f|n+JCr52 zZuf3e8Fb;6o|&UKI{ab@^tw*e#?Lzq3DS*j9)DdagywG4%T>T`{IY4#p0nbZ`lTOn zQu)Rsg*E2HoQ_Li&8H3gcC6Rd@zV#-l@zYKD({Yuh;QZ-{+&3jZF4m7jIX)x0am@F|pMhlgwJ$-Wj&d`W?qTFIw%~25`o}=3gPVnfp(2TppMZH-mpxhb%Nx$#vMLcN`SIn?t29 zu4VIZEaAS3$J>CFabtNe!_DBmvTchYvLy?pt^jR*8%viRrp=nXzZ-r|oQu(%&V4f0 z{8?-GWzCA^ySt0M&(UsvYzqS~c8Pn!cY1s{E8SeZdfE1#PWNY_7U~Vkbo#E2V|KhZ zgvg;&Q>|Nw62F}##?k`Oa98$PN+GZ2ZzI#Ny)^`8DlH7bt$qZ-Y!`KBuUw08cLz_L zLl+`x$E1k^uG`F-sy{EYr6wb2Pn{4X7a?ukv1}x@In;RIwt}H>XIGLl*~lfXpf{5T3Nt@ zD9VuQCy8cnlrfGj#6=15BcZD3vqE$+xuZmmE$o`@fh|0#?=njnIABYHt)w~7LbZ*U z>~WAUo3-ZiTB8C#U(ik0Rb2JM#sd3rS8ARztW%72F z<7L_7#ASadp8)|a!-{4a{Y+UdrKD`_l~+y-)p6BS*#i(aRD~D`K&ylSwkP^hhEd^A zBvd15_RuO8TV}x}44*Yi z%J3moD8lZ}syApf9p7wc&>tq1>X(MgpdK-u)|k@zg~b4R*`b%oAR@vYABKs_FleMK zlQru=z!*ee8+Z-hANt%-0}7;$vb8zizcyKHU0xz*RD1v#Ft7ohN-v=!*=o+TvcF(Z zTwI#2Hw#t{;shnt8XTwmy?|NsGs>=NSXbO?O$!qkei?LDKwfbEY42;iX|+Q`$S;t3 zBE)8~*lu#fbDK?5Il^#HD#$JV#08hMrTUw= z{~}8mrPRL#ryRk?QC?Dp_(qK~B0C@=xirUy$CAye4 z%63fxr5Gj>a#d~%7NX|aV@sCska77;SlJxAwIWFd?@$#ma6h zM5i>B=;luSW15}nlrc~s`f9(VBOsvl$k6-EAbA1<%Gs#K?6dp>62NnIjS|)Fw`bgL z<79uJEE7hMkn16PIYdkBCJm`X;@k*TE-sbjX@^TJ=c(UB1D?!^swPPfgD6hw0NeEb z#*&0QXFKt_tcd58Zo3uMG*l!;)~3e`6Q2PC>jyjYPF_y0Ly$*%1y4>*A!jySq$M!~ zL0Wtd%H9~fJRon+>=ch*5H>?K5Mmc^^j@4gIf&!q&{Y6eO!j`Rx(e;5qxG7ub2}<{ zzN@zz4WPhp9xraqzSX$AL0|X?M4Kb`8s~PV8cgwWo!?K2*~Lwqk8i6!MYUJI$JW5} z5=2{TO+3F`AqM6$Jb_0&LChyov@EKLA&g@w32U)vLK&O$*|l=Rmrre@J>oNgec`)U zw?g0{eRLru;Cr(&gLqKOrHUCye82Ezncbl2SX*!>*;fA1Fn7*4qo1&%(`&@cd4}$- z73z$TdBnGvN))>}7nypctK0tQg+;H=>-*-#oU0q+!pMAq1YX&y+P0@0h~8qmd{M0k;8bS_{JX$t%Q|IA z<$iZN>~?F%8--&hO9Pmt}YP`OfK)#RvBsO%8EmSukPGlWAq3q>JNJn3)76 z7@-}!-p^=MgxK-+PI=f+3#TdEklPlW-p{?gv{4E`-JS|*-smId3hC^tuQ#^hl*W(V^2WG=O86$XZYj&&8^|dM|a7b34 zZ6Al0Q;5uxd7dB(SbmaQQ#Vh`*{AR_lM;H_zEsko#LzPO6|P%KTpa)yveuszEtr~? z&r{;5IqLv$esV95vc>*(kX58bQcFXeW5Z5U9)#QkK5-a%a%Vq>js-UR^h{w5!q;9E znKxHS8RSlu_2ARw<&$2sbmh?Pu0Hu`kAOlA!`#bQ7-ON2z}G4xH^rLmeacIX?J21a z<#kSD$MsxT(3~NO3?qT(Z9O5#zftwRtns5b*8_|gRNqE2Zfx3yXomHrMu`YnKf)2r zmgh4B@EHvk*`SMgjPy38H7?T9EPJ-am+6x@8k<0%VlE=V{6c-f+1g=^AiXq2zx|VuZj4Nd68UXjJIrlDfcrCM+hmk3lGPeGm;>)B` z$8Db^gb!j9&@}{FylL%C<)u8wR*-pxQauNX_<}TNrK4KuJ`i}SXX2hdi`P_BSK|u_ zEngUYMfQeXq6Q5z4NCh>cH8x*e)Tp1jxGt73~*(V)S#XFZ!%o08_swaQ=10QPIArn ziLT3dn`*rH?rR8JSgA9i=Ox-cY58BWA76^yTw}CNLyy;HmizXCn`Q!1oglJQG)#FH zi-)kw-nt$VrT7*reO*mydO$5FqOB+*xMwnGvk?S_WRtAqTwmH31z$$ zaq+!*n8^2jL5FrQlyp0NvICKmygr2M(-n!~42Zl(5DtxX!fD|w6`Icrb(>#+@_9WS ze2WQKXZb8fH#QS#wnpdt6L>-8vUZj ztBFA_dJ3dO?=AHQiogLupxEO)SYjbs) zuO`ZE*y`W})!xk^hSCnN?Zz$YK^=!?Hey5+AQx*I&C3~0Ctxn8OP&y5IIG1AjT4<+ z0iiEJdfqB2U9dt{ibsk?nGy#gq+C#}b&$lK1C@9vV+q;_+MjvEKWkjY2m511h*HTy zXTnaO_$2fYVe!13I6{3!k7J>P1xd2PbtEzJUjeihWNjncxeeC5m_aXCZ#@`JvPKH zm}x%PyGE+{#x*H_8-=74vrIqQa#FOK*a%_UPwEEz0I2JGZlM*N7XBF}$69lT!%0tD z9f$W9^!23@EM6Da%kimP-q$dK9+Tc--eo8kUUdx3hpy0waKwYnCk0k#Ygz1#Jv|Pk~9bCX}_Xz^7)C!DpO}(`^(lG zy$=c)>d57E`8^O> z_Dr@4$Ex+6otoo%IF<%&ekC`4^L-Ek{hM0prt48(jH9Ras$45VEI4q8OLJdS$;rl3 zE2CgH+O@W>v9?uVKph@!T?F;_)=fsDGdu1K$z5JtQK%ILKV^hJ(2Ar@TQEOFf$L02;VT>dR(k$x3XJt6k8!S~^{Zt`O)p*zkUtg07$!FFoD`pc^b#0yc;^9Y&)r>xVXC&fa4z7szDcLJb$Fz3Kp;OIB zSBsAX3B%`}fsIcx;0Ea(rW}poVk>Q{tek44z2RYPQ$GtFL}SGb99aQ}YCMa~Z7HGe$n{69h^^1pr5$%rTm z2n#3+BqxsBWHTa$-1|hh`vG;4M(?Xg5H~4&m&vzo&jxkuMeZJDqc}LT1f?RfNUjgK z^6Gh=GBc6Fr@8r%u=)|CipyJgu>wzK&)8+#D48W_=y3<>ma0>fT6tV3yynOOi`uwr zCIm}i-Nz1A04+%uO@@)oV$c-Cpt(d>SEH$DPzyiBx`~rx+;l19Djmouq@r;}^Yv-8 z15pFsT|6PMs+p8lb_7ZSl@1a&A$KGeH{OkGEi54kre*`x^{4+6g*D9xMGFSQy3Tw- zpHL+4JSnHw)|a1mvD_|e286N!0pZ+SN)^$h2j4D_xgoO#$Vlx_usisB%s&#;U1~X% z#P9c3-{jt)=(;+e-T`jiSgW+7uR@F5a?EcB%~pdNrCbm#jwdi1^R;0=BRxl>>z<@l zqJ&^_=U^mgkXJ^`urq5vjhJc*<@?Ql`6nJ+{&`8TJAF_bVS%C=6!!am7tOHj=-+@@nEDBGNEZ z;xdyna8h*C-zTOT6dC7PwhtWSCZ*^kX#ouNisF(0TKaGXNGio?#%UI|MV9&f1L(0S zhM8O1ML0@2$w`3p$B(2YsALZgpe3YR6vgQo+2+8J+S(7&-G@axwK}!_QZe|4BX$@r zzW>Oe0zrOkG=F=HP9Kvltc+cZt>~THoM^sF(NdP~gZxH3Js1U4e7JSDkM>Ubw?st| zAvs0i;5muJe05atZSObq+BE#_d~`}VU?dTKf4DwM!pk&BUvFdKpt+YBCXG-YVaau@T*sewiN_v;X+H)fV0<8fF@?HjL0f@LGE#+_`1LQL(jV@ z@f(eWTiyf$m8E$=SUV@k%P0Nl>W&dhWV}=K7m6yxih2Bc-A@YWws$ti>B88VgGg_q z(O`cSFlHLD#SA6UOnInz%WvHvdiD}ab5zxB14jXV6RwVLt!{stuDSJ#ps zoR*xs+x8%58^xgpY9Uv!%#fhC9}WOzKE-Ft8x&9;3$uzVF3^$d>hW}Z5E8scX0-^K z&Vazo$0A2h_!8Dz%8I~yS|q1%YYLgA?vK_GQC~gMKrO%rHl8Glv1e&*pUEZbwus=A zB|Q*5Zna`%gN`t*<=o6L+w6%Xtz9b!d%2IJmE0oFvJJDfo0>AaM?8e1del`vA{U=Q zxw&iMU@_NZtU)I%+0a281qS;p$|tCV2(ILI6{xLc>dq$lNQ6GSe)Xh~1FKrj;g1z( z#hXnNvC@Q}p34Ju%>E)1An(00=BS8vV8;;!KziWiTX#HZa|Tu7)a#&1O99O|EvK1_r+BHiuzBJt^Pzsf7BEI6ZQMltG`12Qwj2) zkj^jv1NrCS*t!31{rf=TpVpscf3yDAu;O14|7lR@Peg_Me?t6ai0H4VKTgQ+8IC_u zRGPn|{>*p$mGYnZI)74(b^eb%pWl(cNAZ6mH+BCG`OldCcjWJ3%AZL24#=Z`Elpbng8+azW`4l&2az# diff --git a/tools/python_snappy-0.5.4-cp36-cp36m-win_amd64.whl b/tools/python_snappy-0.5.4-cp36-cp36m-win_amd64.whl deleted file mode 100644 index ded5f9d180886afb5008add4cee72672b93debd3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 29776 zcmZ6yQ*bV9&@37oE4FRhwz=Z0*tTukwr$(CZR3q|zJp!=+52i{ZlFqsi;)D80s&1z z{~s`H3@vQ`9~dR-a#q`ne|jJv34&$n@z@apTEoCqgp&jb$qHP~3s4KzAGL;4b)@7Q zgMR?Av=jPS^P#sRT60TT94wg$GE~^X4jUKp%F#$xT10HbZqmy+rRzU?ds}-?=p7aj zs?P2M!}Y4#Ch;W)vb-sW#Wrp2sEtHgM1}F%p84SJZ*+5FOVX9g9m&Gjz27~7sg!n+ znGTUSgYKwx0KlKGK*DkAQbiFLG8nL`u}Jf$WV|r8lkf*fhq<>=Xd%RCLR}>=TkJuv z+2KSJ$EZ!ZKMu~z?r#f9Vpp#BrWn!~(M8K(+0Ry=nJMMOk>+$o1C7E1x*Dmf5PDBh zU=iT(&xK1xSU=sTK(ck7nvQV%w3Q4+V8IL)j?^5wA|1UKbPJN|iYYRymgMf(-mWh~ z!hr)CQ)f}*-Tx4m$*5mv*_EB?-Wugmlu_FQSR^hzqb^kR++f%b@X@3`s*DoREy#Je z&CWTHDV&HIc_)WfWb5Qb7;G$wJP1U~oBf|^2L_fcrE^00`|ZOI2);17e#VZw)dn`& zof5{Re)N(H+)T`Yly9)@ZRw>nu9jbn99?}KJiSs5qOR)n<(r@x>%gWeu>H$4G~Tvu zW@cX2uD_-S`sjkv$1h7vdf?W+XzsR~E$_d0pir#B_0Un`j32g`2i+$w|3-54t77|R z_AYIumWmMQ^vA;OipEfyV^4N5`V!f+_FyPMrgew3vhXW`!7m0K8n`C@6k>9}^pOel zQg6UTmz-Y7Gl+rG>b=$@F}2$NG6I*I3wDdd3Gbquqw1wg7fD~K*Tlu$b;d|vTgwG+ z~ucQscBV5FpBvNiDf^f__H=Fe^AnMCOH}YpZGZBKTt}!Y}{@gobFVpFg9m) zMDx;P17Sh3!52X?h?qLmyN}sOdG19((hno1#0-7=t;v+PjDrP;@QiG|>1ujCxZSQNi1Pi3d3C8)j%&e=QxzTClk92>Sc7vX~ekEn^T={So4! zxd%Awpiz-2^oC$MjK6^Yhp44guJY~rdUO-){|SvP5D?M-PjCKT-Jv(KXJMyv`;W4Q zHpc9%|EZ7hjSsY@s>G@}+YbE9rbY>xPTeMDByk`tM9hxx)y+C4aXTr!|Gc1txgOSTkqkSV!CVmi?j<_u|)xGWGG1RE1(Q~WS#ngLwqFr7@F)8%fc>D1Tn z=MP{P@N0K_cB2AVu23nHSIgmcxpbWAwxLKs5k|HD^bW=)JS-vk{ZO8OiokxdMD^f$ zzG;9oE^Qk-P1d}&4yoy0Pt_kqKh~}}L2%8{h` zqo7x1=lKH60Jcvxf$HbCkEem$=PJq1Kw#n|=f<+RPoppFgZZRI+3jnGt}8>0_|D@pKMLC5Wq&mpJ}`^*85w;<7(8*a2KV`& z$$!Ac2$;xm4|LX4U)_r&{32wml=d`0k1v_H{~ZT$y8WRL7?{XP73^VR(9Q${6(Afk4OGl(_9#~xOd@Em%wRcN zZR={jY{YD^Uk;#wSfPY)==C(xYN}Zp_rm!gLSHs&UHCbV`_b$#nbND(N1&cKbT65O zsZrpX?qF%I!1b$|9M(T{=No#!=m-uSg`i8SlL3DQY^A6(4DiWviEM+Un;zny%nZ2dmO&mXY^@PFT)So5be>QeaC*CTv<6g zm>6eJssO;g?18=S9_t9O;FLLh2}2YIXtB0Bh<_*|B1X$S<#g)IdY~q8zDw*?a7;zt zhE&F4pUSw^AEEfie8G+F!H)PIFZ#BgnLYw4<71c#5@f7Y?RsJxA7XI* zCdd0orsW{8cq89+2HVEtj)LCFn6^n{4~%j79lXiP%}OSH?BFzRRP4Oqwa=BJ54H(4 zH|JV+#3T5^uJ=^!e4-ouP}ZILe~K=Wy|Kf$8?q7S!1yWvUN4s#g)DC(SeM^n?Vd!u zD!cs?+AMF`Hne`@tUsc-(7^bQ=p=s2>3>X~Y405@p|KwwJjeD;ABMMenH&w)BkrSE?vAmf#A1&_v}K+Tcy z1fvXqxSBn(%}y&rf>F`X-pATJ`y!t_7}kdT9hSyUKVAv-eIWD^y-znFoN~1h%;ncPzVgp{H@;1UjpT#(7rY+@@@9jZZcvX+k+cq4=@R+HL~}7xU;_% zqZVSC-E)ajz0okwEA;*9L6!KlYiR}Rlup|+8a-O65um4Agqm6>J2R;5&86y8zI)Pj zgwVCzzs$;bbp~fUym^8SNxV6pbB;cFD;gwZ%vjN*JpWiYgF}iPg4rrS%|#f^jDuNQ z2CC|b1&A7L{?uq4zre5(&m&Boy&AQ(Ev0_kFe;8}~H94h{2K_tqUyG0C+7)M-e_(3{#R5R3`ZWROJ}DlCWS zK2XFqoS-{DsxLlNQMdrmpIpIVC)5IuPl(803ZMg7)Ri;R11~ZY(9H#2qSL*&cLp&- z6MeOC{t)_&w(s(L5Zb6t8-AA;+fTK5)k-Tp?#K&`rL*-zO2t^OWz>FHssYLO*n7O=((Gj$0eaEO zb8Ek1VD24;a0HY7hN%YwW%Wk#Hu}*VaG%kN)VlnfQkKV@FPHPy%Ov?1Tsd^f2CUB; zv2xVw!mCQ_BV$7W`Ac<1DJe#nR*!NauP8r)O$J_Qh{e2<_4$z3$}ch<{tOo24`brv zlI-Dbfhu#4hk|`G){Mm=0y7VGpYl?Xv?KU8 zi>83}vg`Oc$ol@FU`Of@B%*)sD(!;Kagem@y$@;q9#eTz7OxhzxngIX3Qb_h{BqPj z)}CytLfy`nj=k*(;Z9lk@@wzQK|}VeUu?Kz9VEwu2t#cY1NVByz|Snse-0-P)X8^a zhwzu^86>N|%D|tJhyL8CuO^2lb4%XU?-y%AVpg{(b##hmN5xDMQ@H@~N!a#aoGob4 zZj{G^zw|#Pz|N>jgI`HZ0&$JS1qYfm2?-1DWDQlo7w(bPc$#@<9%fL^%d~SyK42+# zXH!7hJUHZyGO9EOI&o?gu%n4G21uO}tqpS&XeAnm5;lSm9I9kqI9@J+i7@FGO<_0# zV3+-uN?Nr6xPDN^osw!evmT9VnvBs-d&P&m+0yMXo9HFh?59&*H$+_R9$7`~902LraHpzHV~e(>=0>rt)+RKfJ> zLplPJ{h&bLeuc?!B%8Sahv+&mGtw#nlI0%}s26baoh&L6NVqJ?S62r=h|y0mH~Vr=vUT^DNMl8H=^-P$TMo5k{V&#(CuAF95IK> zh+aIATzXDSh}X?k6JrG2<)HMnt?$rsF3d+2Uwt4Pn-5CIC~G!&ivKGstdI|Eywt>Ap`pe5SXH4p0aItTfXA< zR1V*ebfI=US8xZnOK@}ry0AM$i>fm+Lz>gCEC?g5Nwbvp?z19DK>Cy=1Ce}ZZ8VuP z9m{FQarRW1DkJ%i3TIAb24}uIXK+I=4jjpsibtnYDDTL}uv?1$r8d)ZY_w%gnWLZ+om0%gZxW9=3KhRRK@AG5)!fy6+OQkbXay%eq=rz; zSsMfI%)5gKfa4O=4pk{%M)e!*eiJarO?!)*(gc~s;t5t{{l2Cp2E4wioBPF!ZmdE_nR?0e<)pX>} zJ~UJmaArzGcW9`gZjew5_mJXvnFb1n9(kGftFllF@#YSa{f0jpXS+S%POA6Oiv0x+@X%%*pRWNm|D89 zk@x&(mi~N4W33FHTsH9E51qQlYV%=h<$e=W7=R&Fi<=VdGR$9N`wZ}O395ZRu{!iN zic7wFa68pW^(guE)F|+e>1engNeDmXzLUa@%lh*+okqe znUHyS?cOCGJhISxmamHFdlqeaI%e%0s+q5q==&t+uS$Tw?(}{Z%XWswJ2@cpZfDAu z+5h72WCyWdA53O}8ynRj_u1z%mhFk^KIa?`ElKQ~hsS`>Fq5kuc;LZgr-FK`|_+H+2~L52E57hrG`LEt2R!`nJtRTO8M6tJCk6{Uy|z$ zDDh3Xe%aN*6~9r(1ICUtCIdN4oBM48Gv>PLAmmzdwW3Fb_$N@^go53@?#{t(+?p(2Nm3oh@6sqzp$qkIy6 zd1ZYz&j7KHPv-9(1wX|3y}9!{rVp>|pHYhaJNpkDAwR!@w^^VMw~IS%VDH?F=)&cR z`MPTagYed$6@v1Xa9&@U+5U7ehb>qZ8co)NWym<*_;8i2GNO2fe?L-829y#tLxK8o zuV&r1bK#{}V_nR#?<~DjW6^yIqm{%A$4fgrn<^-E))Qr0i3-4&J_hm*PxhD&>D-g= z`Pm-KCEEhg^_F{z~}#mEzO_={^DS<$I08wG}l) zc_wxo91jYG`6T6k+=r_4E-Nq!{MrVAmlnWatT}!oA+ImR{B4H}iyFmHm?5{mDKIt( zg2tU9O;@Z3%mP!#mxCDXvdHBseU?TVzSq8`>h%cFE;Xr^hJkaGROl&I0P+~`Bg2@L zyTvuSrr#req@!}}Ks=7f_VC3M6`tg}+rfboXv}kfA4gQ)WzEmo5a;QEMedfi(J4te ztH8PbEhnR>E6;5KkE;KT@%A~V+eW(sSbCg%(O=67xB~fjb3?y8+zjJ??3ZI3h-3#V zxnk=AM8N{eN?rLRjS7Xb?-U{KR=PIX=Z3;-XCN|O6E-SpygE|EsGfpF_xm+@XVuDqxB1yIi_XCxi#hEAa`Z$4Gz^Hxl&>j;sY(^*y2^wK48!T8T zZ~bt~);!Ycj;acq;u~txFsNvgB<7BGStilFdtWJ664{5@68JErrkyT0#*{a0KEV-vC@Yhr zIWg~SDY_UroOmE|c3vcX>HKv+LSamNmM!M>h!`zX?0l?gJ`Qg^P9X9-on7d1>;R?M zH(4(JLn&V_&RS3G(_tHI8DLr7X%wycp4XggP$$Wlid<|l(u?L2?Xyjs(=qS6KjO>u zPDvs%_Sa2o^4b2{H|yAlAL7Ir9Fw?jXmhAbO*Rrki^XHCDG>^km1W6i`Sl&}60 zc`76Ra$GuXc}G4%_A*IwiMA_~?fFQQK7#6Sp7^0|GR3;iQdzcQ7d~LX;~JyTWYdkA zB~z$A9sEGpyn1*_-CGBwA#Y$@{s~gG8$_982TX+fV6dgoFSwf#Pzrv{rEgbwx}CqP zpEoPfuACWNTW2!cn!63MCcqcouRr#r_7iRJHh5OmX+RPqCd-^*@{^t*v&CO8Fw4%D0r%=p6datdeT zEgmDqv*zF8D~MR!pKm*?LZv7L7dGafZ_RgYNXsey8#^>?*mLSd& zme5Z{UxG+}!o2oH<*xxv*4&GQ3Bcq~V0ac`Fy0FG3kmR0rnTvtHRSl3qBkf0Qg<5ZL&xJ-`S;=I-5fOf|5SXEV0_ zDxe0wPdTW}96T0G4y#D*rplWXC4Hh|=FK>&%-y5#qp*}%u@07_5Hj4i%Y`uNSW9(V zLo)JrQ*P)z&%a}m2Aag(aB`Jlobkk=-%!MVY={WJ)4`%maMXbBROZ`A?9~OO=npv6 zGo-6T-Iya}c>bxD4GD&KOrRu$C$Rg5NMlA;AMZINIR=$FIyvW$$RL1^ z?G+44+L9wSebSM52IONITEt_DNxr4`f*Tz#FiCfvLzT_&(vgY@X&bkHo-|k$^sJ9$ zK=;iXMlP9JAgx4S9roKrpKB&9NzNUcL$XG?YIAF z@`%-kQ89g>535tVcYFu8RVKTyRu#HwU3=lILXQKX zPWq%zCymO8gfN1hD8I3<})9?Sli&ARB;>|z6xEE_P5>-l+}nirknW4+2lpE>9a z*BZW*fxTcBmhp1__+0pKYr?^K%k<=f>9dLK4o_&eH!od&>IFn(7(xrnUus}u2-TTl zT%hdIO1sfCvYZG2)s5taTk{b_PtmxKLB0WApzn@&(AF|$hI(v-*c=seu6cSod4p-* z{FB-SKg79hPkCK4t}eH|gej%QMEU%xH@)m+-TcAeX5YKt&^0%+*nv}J;G9j+-Cr)pu2@jy zaT+lF7AgCvt^e2rL;xHt`pR|ROfyVE5L>`e_VM6&)X=ezcTZvF-#K4e28sY^Slf{$ zsfLn4nDgh#s-y&PSV%Hw!R2ii1$8X6QRV=XM}mA)p}Fk31|Ot#s6U!Md;gv>OVAf$ zWeu0>@RV^fFY6{)j$dxF6}%U!z$D16iw)s2{@WBVEKzvF4(XgSy@4bPV6)cMLN9zG zh~O0tGF(>8(!A_C31P7vbp*xkQgtK_=sU6STk}|Ij#Bs&2Jd4m6-6O<{YL` z-4V&VYB{(>A-E16$*u(zE9^^p73!-R2zAK5BJAao%?=uF*+0g5M!h9jR&oagu7$i^ za3B1NwfE~XS5G=G_~~a7OQP_MXSxaBe1VDy4hZNHOC8|2(9LrV8TH#U)vo*=u0VQN~Hh^-xp6t>^1m9R;7UJMSNjk*$}@HRD^#Y9Av)h&SakM zZ0U?7xvZYIKTtjv4X9ogJq>8h$M=h{hHyu*EQ(tjg@E62dW8!U;-@4SKTuOXT0AZN zzDmzz$I7RhJ-w#o?K=X-7F0(F&DxP-EI{{dZpvX(c3()+wYD1qg=&EP(8|aqktr+u zf);8B_(Y7DJ=OZ)2fy%7_$D#L52fioS6%_0X1CD^b`p?M+_jQC2bi?S3q?ljfcdo^ zh0on1<9)toz^smfM?XA!$AlmVU=%X9FS}r+D9d%Q0Cn>os;X)-KwdPzr=f3CuhXrR zh`nuko#%rso}b?Jq6p1MQ#F^CdQJ-s@8o$V2fziF_^JkVW_5MA_jv~`s5cd^pR`ug z&B;qh@VbUcT)>1fuFV1hn2*{rSlYIHrX9_nSR1l<6VCsxQd5-N$b4d-wE|5ET6-=Dk6CQ@-h< zrK2kAtYop$#h-?r;w5KMwp}l&isYCV)^_@~(-a*~rLH1$+w#9@5G(Z;0!Iwg>aeio&(e zbWX7?qnjV?*FRK%NHh`OE^mK#q<;wE0Cav!4P~#oPqx41zlSb7bE%vTq#a37Px=D8 zzLY&u@c-KZJ&i7YX-`6;U)@=qR)uGH`R&j$H{*j>qY0Oq5h=52=QSqlc? zD+YH~o4otP{@4~Mn8v)oWWjN$55Ns8Fn%ni30Bs0cMlTQcJ4@-w7@)A(YKh1_hPAB zLW>n5|H+Zvt~}Uk863AQah-|j2bB6aFoYgay|&)#;edF$d^Q2AMCf&z9KwBjdX>7% z@A-a#nk!;xwJ%;i+04nJJ?ULff^zkSwuG9U%P?T>GkXx@XxY0NmhY#f*n6AiG9pdh zism7_CMZUxN?PnDR+5V1e)<5?vdz4x!#8?wxO*8}`S6#>1QwmaOG#q)x1@H~qG6C= zPKbG_2G&U1B?9{r^5p`p$jX!Q)dQo!7_`OSGQcEcUcl7i@z)$LxD*0Qkahl{92_k^ z`9k9-?R!9%N;_|PKE(8jRB8lP@_3%+rR55w?rS4^iw6Q^>Qw_}{_ADX2uAUI_?fx9 z+k@;~DI2T@-70w>2o@pAp&IyroJ})Og4`@!FCB>ZSuk8SIj7Ls?QpnPMvhh7r}E9v zX}KEtbr|+~yw$4aUF#_Irh}9kje6h#P2W`8mTK4BcsdE<6f&et&laSLAlOIW7eB*5 zN5?RZ5mEPpoimf#7txZ*(O6lgvrt(7F8M3_W)$15z(fJHstKUA>d3uhbAh*HYbdvc*oO{9#Kq{5aK_ z+rh%I^4 zw0aZ#3gUNzv!UnY^>Hy(^^T5zQCv=LHfN5UekMN~8P|?}-MS)iplF{C#BX}i{Z37? zBWbr1ZtTts(_uTSbKJU~_e=;mH*zPXr^b3$p`Y4-V6>AHN4kxxxwEY;0S=*CO`mfK z@y*g*-xN0UshgiUE|!l&>HW$rG+v0ru8-MC#?9DluVx1J@5V>7*-oFG2L<0#GSyE^ zgq@#{|GTNtuA>p-RM$ z(ZckvIKfbyHcBq0p{eW~Tb?za8s@Cnds@Wp#E zLc$h?K5M~e=>AgjFz45OG1BsvW= zOmpD-oV|60SUk1b$bO%-t&p_+IJ-t)yCrecv+WXYKjQT_=Q<7Vz9QPJ*Hm!m z`#89~BxCS*+&36b3he2Z;3&QGD2Zp2KT~y3p-Q%^`=u6CLG{b^VGEom zGuJc!twUMlpgCT{)wlg{4)J7G_eXkwb!Ey1Yt2QZaBIoU7QX@T+1K}UJsNj*V%sI) z;4+RufbeN?7qpOt^SJ5CYy7L@;}T`*^Vn#Auv+FrBCP*ovDO(<5a6{vrw|)RiYuZ1{di7@YSZO%x|lwKZm44b!1p=m z9f!0{iTgE?zI`1)=Z22Wug@){98L58r z{t^g?rQE%Eptgy%5%l@3-}rTWKJEP~?y`TyDe!(?RH%5Z;bC@t026V!)lYpsLKr>M z2to4FSk~ z@U!YI=Q(N)gfP@kT&T|kir{@~ex6IR2;83z8?Ku6uB_X#TkU+Tbyg-a8yF==lJ8X|f+`<;Ky`@Wj$;e^7=erRt?;$Nz#uD^X#BSCLYuBq}uykgB3hgEbM6 z%AySh*Ha_cDl<{DCsPWGWGJ#nQ;H0wGqPucg@v*lSc5@B#~2W=OBTY!7!z+$B4cBZ zh&EUvqgR?P(~1m=V1*h&B8jaNn*QNQryPP{3EChc<+O%k363ErGF*^lts<&qgfB7- zOPsGSm{xSfgoec#g)=gebk~GK85%_vbEid_LZFzuKbW?61%pN)7>G8INQ}Qvn3i-! zRSJpwC*DvbW&UPv%E}WO91>?ZT2Bj&Xoi)FCxeP)l+kd_Ss0Bu7Hu?^v>*O3?XE5( zOxzru8beYU&Pg7dBPdKl&WN%eK}!3Nm?a^G;sEJEmX{}jl9+-iX?;Qziz_2znmm4g z9E?05M_8D+j8SF1LKKZFC1a92aeiEsJRwJXkhqL#X`K`qKVz7*VJU*xlA9$*j(oD* zm^?#{Xh4Zd7KSG%Saim~w4Nn`+LD|lDMw7W$dsvReS?(NlAR?>j_g3GLYA5*jB?p2 zo$DH2w4yhlBS4O1;F6U*CPzki*(sx=TDC-a&gis(nOxO}#EFC^3R-2(?6irQU8ly# z`I_@Yq@mp~7U2y7%2!xaTeweUkxe-s;a~{Z`3?K^=sp`maO@=Nr#Ker;G+G+_!k~$ zyZ;X8B)K+di&Xjf4Sm}0L~Z4lzvu;B3b75Y9x>qs=(%!RSaL_Ja}D!e_@mVSYgMm# ze8}(JCNwtO`?DffPmdBM?p9tutqih zW996y)wK-tedQQXEQ0^t@+-)|{Tsrp4t8Af`f;k5@9h=fX2YK|mvv;kn;PiMNCzaq-V{-;olp zOEpn$SgDh=za|C$2X$fHPX*K>u5+{jcf|b-jlqg%1gw&8ze;OL-CaC?=;ZsYA{&g7lu? zlOGLS-*3nsFiIbE>5izy)1m`3JHs`jnM8|6AkOLC(;}t40)6(yxb8}nFH#X5%U)WS3rdFHW@4Z&!HM;=@V6HD~` zJ2Ii$DxnLb5=Lb>_W8=h0h;fJ!8tA6HVx^={K0_n8yp*!EzTyoE)(gT!Wl$nvgZ91 zJaMsi8idJ&un8&ZXt`_&A$3+E$O|#-&?|US{qa6b1_-f$UTl1J_RGFY;N3BfX@Kzuzi z>fqDGH+!cWM7Y3Pd zF9V~6ft|eWabP?dz0io%X#Q8!Of##JY+Xj;&$A-@Jd`4b?+TCyjOHC2lh2TB!*1}# zVN*4ApsG(D;mKL@v8x0o6(t;5m7^*rf0D5*5ScwV=Qrl)tr>7|xvl+?ibydkmcB~q7iCTeC zFSh-z_`ATfM3<=c)prI2NP+*(E`d6y{Hs88YzS`=;=%3JfZH)OzqOknY?u1xE=1o1 zZnAG>FpWdu-%C0+-f^b6$k?5Hi9Apo*tKn2HxlUZk-z0qTP9R(>1K7!X&AIkTeKr< zPK8C|DabZMPFfqiA@4yaryqsQD!4TCH7U^*7}_@R-qpw1Xq=kZPZn7a#~sl*$1(WM z?eyHbu;PuYLC(>w9n)_*WM*8aa!eo9ICQz8E*OI_UYb<_$WP)T00%vo2MjSoG3e*j zLF?njgw;4R7Uh@XrmQ<+YkGTfk_1q`@_a@;U7YDHe zm-PXV9>`9q+SUkWz8B0i>7e_7Y{!fi(mgFrOZsceGhvyEAvmK>Rp=g816S)mtkgRe zO&K+@wAvoPqik(7PSdT-QD|D15((5D3-o-+Tj+tFLaVe{8wewkormaU8?97R)-;3s zt>}U@+3Lsq-3)FR|gnHyOwXa)0$#ksOJ4Xy8C(>Se<5RJurKH#~ zCjBLC1Ho4Dm#Csn%*x8Os$;~^jHCtcD4ex!pAC#~K|OWNfj&b|4(cE~`q+mpD6%=p z>d!nY)ld)YKM5dVkS}zzu4FvZsunx({n%FilM|1lMBRm@JhhZYubpT#%_^IhKq?K_ zSP-rLG>VKYGkKoCQp+mF-h?F(2=_&cpy-K}v6C~)mh@iln=a!RW$t%IT!p#+-zREB z6=`y+d1sGj;@DH~1~!ABqpDoWGikN0U!SZoDHpYO2cep*IhEG*#a-tzu#l{s(Sd^x z@GQYzkgAz{?0^4h)5*X8&&l|0_rT3y9y<8_K2iT>E_? zUeFgNaKPS5>UDzy2z=N^*#4b+tCJ3N={X#=K`C*68wNg{*iiQ76AJdtl>cwGAo=hp z#rKp%>OBGUo`T#fA87Ny_oPkgodo96^O#`*I^5=^JF819tGk)R@6hY(k+!pV@iC*H zrk^eYpM+42I}bNpHl_Ci%RPy>moWj}lt$D0?~q}UfUow_vv`i)Z4+jidw~v}<~**K zpwagt^)IaSH2Bp_$PSuj_0_LSN3ZH{?@tp3C{8b{cr@fCP;J^G8W2I!Vhd25^hJL= z`rs;2y=8`&JFp1hX)Y6OFu>BlH$&`=a+@a}u^+{LP548^pGidQ=K-jx#2KMqU`SV+ zbkHpn7|a{CzoZ`8S`6D?{7V4Ke-xGG=NFDAQuGaJJB(-tg3+^m9OVGRo&s18$erez zL?vqY`^Jbw6nk6yhqA;$z-vm9Nf+cEtDVLFhs%F#f0i2gz=KLo9EsU1nes`;A59tC zuqC+`Sd*dzsh6rG3rHTDZc+ez|8jvC-!LRu+}*yJl%oHV$pAKiqQ8Nbh=wR;@8(o9 z0ir-xng@to@?r>Ru90{S5qLE^q(?CUDi4q5P#f5nqQnoRoIqiGDt;Eo{nuTojR)A5 zuw)L{n;hYFRP6Al)0FtUBz)6x5e5JdTkcH)^QLJ@-&9 z^NP1HEBAa+NUlJV+rezZe3Jz;;w>^PpBcISz>UB2B8c#|{Td*xcyN%)Z=OuuEjertF`*B{Vm30R-KWYQrz)CeYvXqL8gG4r%VC?*74 zebPbCcABW4y38Iw{YCK7%pVMfsQq4O+b?R@R4@%md-#EGKwsVOds_6Uy3z1&MHwVM z07UA0Uj*%t3naIJZys^1DT4!jSjQ5+)h5oS9bv8=fBs6&qDy&jm%Ej*4WL|W}Bn_Zf~Lw+9x$Y?WSs@ z&1tXJr{(kYwrb1GfB@jV`$RT(B)I^*m$+HEnbXo`d1Z;X%)ZjP+`0h90aap(C>&0{r}GmXYWz-SKDce9o-iY{R^t@5x-99cxeF`=(?=V;!d*OY)0W} zN{bEyO~?;U2hjwHF8cb&bplG2HJ`{5>NJ+5t@_dXe)53T$ES=(O!>~59F{GcJZS7aIQ71gQ zuH8ImDe4_RoQQSL<;pdSJ$pWWKE8A!LZ(rUpJ@v`d6d`L=L!<2S4dmEsz8*>sAtR4 zsAN|zb^?&Hq31>C2Cwg`Mu>=lGv(~dcMLG&KQKsABsvmmy@u{WpI?$ZJ zZnQb+j*6iJ-y3Vera*oxDqyE@QQdmisI>alipM?1THw6D+p3-1pYX^}6lo`fmh1y@ zxgh>`#fJ1=S8ln&}{PsKo9p>|L2=XJ~PkYKOR+vA14*Y|s`>aDCQE$z;)Hz1Su_owa}=k@04=}unZ z@G8qqBdu?kqi;3O_v?KP!!8_LQhR`s&G-AY)b}w*B{?!04YxXZl3ukX&KsG-5eGI! zy7mwTA2=xL6D9)Vi9l~I4Q>L@g^JLta&FnOT(yYgBy~-iH!JNryxW@i^ZsjfLei0v z(pApE@8GCxDrDCi3^P1)HIGP9s-KJ)(SH)en+$%Bg+yh*S$wSPPii)19;4}dj$(A> z_1pj|j2DUMK=D7X3S~v86vK((QS8%hg9`Lesu^~*)Cgoklp6y`U8brW%6PrY!sPA9 zum0#S5cwNw7c=$i%Opxf+5-=SW#fpTruJ^%4zz2|#wX8MC=p7o-FrXe88yD0H8xS(WWPfBw9FmunMg^ilD5tSh zS6s>sp)ix9rj_xR3lw1H zM})edrz=tAxnIfBveaPhRTb!I_^Csot0CRYCv8RG%c!X70{5%iKs?Vp~`wNR+!Qdo@SL@lcYWOWH&=10g zscj`*Rgw0I!TGPCu62d_}V9;pz%jR?f>n9NVFkBUepwz!D+Y5 zoRaK##-F5Ycsv>Qp{h?YhH zX`7gn!`u6_n}ZlC)XcXC7HZ2cPU8yS)izBkA3c;PHI-ONJ5cjAPrHBQyBaa*P58CH z#P_S?)T^xi& z_#$qTLDxhU2zKt z8o~BpWzRAc)3t9d3zQ0$q~BV2FXM4Sn)5SVPSe+hI?x9?D<7b`GMwsXr2J#pUGb+l z0Qx!n^s| zs8biSNL;GwxIx)+JoR`QlYwLyF!Ho(F(sRA$ZcQ4**51hihccU%8#I^n?=B2jEMKw zrk?lKN`X0cY5SvIB?63-MHdCI-xhc`$EwjZO^+<+YME%i~Z`!sQ_yFae)stc4q6E=GBQuM>SL%wHbOiB5dW zlb&1WpE=Oj_wKr@7%UmFc8u+`^tw2>zRNn!Sjl0&(dv}DSqXR?+hhxsPBkn)>pT&m z8eI=rH2G-3^6FRX4SkJ}fi%5LlI@U5g-q@bF=XZsCo)Yaw|omR6fcvE-1zol>(gxm zM=T2pdBSAK!x1q{FO3SyfLG?}yBZco$T$5+(5)tvf}PJdj_+|IY9caH)e`pDz8&oI13WwHLO1$BsvmQ(($(-snscbAy4D}@Qb~)mr{T9R!{ds zu*^)Yn6;|{hAE&Lh?uqxW~)@*KDP^*`#qvOszUD@-n*UYX<(r7l)Kv)_zujwz?`yx zHB(`m8*}XT^XP=tiWghe%uF*qmlntvDvtY|K?g5*z4N96pkFslb zHL)PallhxBL{wQLZWQQgT}G9(F;Oxbd(oTo49Wss@Lce>7au)01J!&$f0#@3pJ@Yv z7G;_wk$`~mzW@P!p8lDpzJzKK{-=6<~`}Vvw(h7pDi&17Msm6M z1QYGtsab%5OdMW}bcF$-PuGD?KdA_{Y#VR}(k#lwxw|jl{ghhBk0FHWzQ@=>LXFn( z6p7c%9@1CCKr(I9tuYqUv)sOHJbXg#3c?9ihDDM1E2f(zN8NXy()Wtl8JpM&9VEYF zF<`f`0;WBySoE&i1K20h?uQhG_|+i)jVSkZulumRcNXIUGls<*-J2AWJJ3$a?eCrI zn6TK%ShzY6wI;v3Py{Tb8hd>x|_+fmSOd`m>Lx*_7 z+>lGLJh4+d1k_3HZ-FSIz=b?vn^I4x>Sy0@O4aWgatx+;9UVL(LS|A)^8UDjhmCh5 zm?kDhtaS4+aCnTyGgT;aY z2Jj{Dp0>hG*VcA{D#O$*>XA!EUPL!p6ntJcv3^n7w^u z$91oxEhFYOc`RpFN1g>``L^Y`|L(=7VwK|Z{&?j(Mw1FD;&eLtvV zYv;U9SU*yJwP!|>hwNLtZe+Lqz_2>QUfzG{zumxPbexM2jhW z_Zb`tL)XG}H*l_ZEWRB440#3O?w{=#BEzueFQ&Viqm8sd*ySn?7CYbnc7o2PAD4B{ z_-wlhEnw~|iX=aqE96s70waA(1CSEQzt(t#Z%Tj)l4{#eOwN6ClMelsOJd}y)F z)y;Y)b?XNLEaUU@qPmJA8!d@FvvuccZf_PBo_q3)TO1_u-ab*;O>$FI2*5VWjMGKg zLWW4~9MZw7U6cJt=@93zo%6;A@3=G)h{J^KYpy_218fzZp#Cjb4YTpRDsX)W~ zkzbK3gFd`{=+VmL@Tyt_1)VdebfaNSz7=Edfk%ia|2NO~94Z5{?sI8aG28ZKq%9MT zcurgd{Qj!zmkspL0XeGmX9W&ID>plJKzg5v0EX8CncJh^1~|`H8YQxn6?!k4cD-IL z)JI^yH?%(@;V6WuT3?O{=37qQI$`I>dmG^;T=w=oqsdI%)BUm6r}|>MdfzutbN{(Y z6Bzlh95`{e=qv^IHgfA7(H+BJRO2i+$4p1KhL5kmXR+As64{87*;uEp2mXSrWVda> z@dRjVPb97xT;>R6IX*mJQV6LPqf^JYq91vE3@-28O<_XF7bN=131^|8%gxq}(_$(X z-A_Gs6X#T?f)!G3AQDEy-&&OD-ZB2Gq$R6@Bq*j17#kIkF!=5ab|5e<1|b_e*~gEm zwHl8IvS#HGY@0a5FYxdPcnESI$Fyp?{!O1_S=k6E=UR>>VD3>Vts8nppTkxjel(6w z`ugL3myYOcNq!bYvW6r=qiwB-o>8Y}=YsoGSNh%KJrn<mH_i7BSAC(QVSCt{i!%~Z4I_E#WsExbyN z^lk)wdn2FZ*TTGXB6zJ}2;7GKd{NtC+J^v&PJPb}m*4jUzYE|7ZX~kpeT_-b+{CMV zMNE#Z>W~$C6Nkkb@voS-mC0bu&wBP(4gJd&XIrSmdidI{^94IyVDOy^O*tOF=W;zb z28QyE@W{y_AcjqxM-3UD}Dw_m-tVht=jmcs1;o203bcL z4PX1}tLxWuqTzWXXft7xwQA7@sO~V$-oE%#G=2mN%TDEoJ_?Y}Qkh9w1vFA4O|Hwh z3`h>@Q|)1E(J{k=&FzSfy=yhoAfSrW2(PPdRv6^tDEO^(Arvs_CKyW?^bj_J7Wxpo z9Z?3)Aj%PWFI^61a98((?s1>$3IT9Kr-=#;{}p|C#aJ+_nN7a$ z(263T46{VT<1}U7DvSzHpGIa^w30CoS+5a%i|ru6n;~F1)q8LIyk-q1KQN2}*HFSH z!GWNp-zTpX#+bhSj#pGYLwe_N*b%k>%H_UkR$N5M`&<2)gKsmLx>_(i|9nG{@4(Czn{yrU!}c%MH@haDPps>Qb2-V%(Alt*cml&1+Ii z@|;^%04bTmv-y4iXmXM)&c>!0_9%v~fS}(~`*cIzxzOf%j}p}H54ZGn#>FejA8(gN zlt#NuoNIK-{<5;MkOACdBP=Y2xPe-<_5DgwmjIrN;;D0*{h8rOyDR0k)^m_@)v`hl zV5C>y$p4`F8j&N{7{U2)t-Sa^pj#0kv`|$?Wn-v2C|ib0oXpvnEL^O5SPV>wUTit6 zYS!vaoR#566m$|UF>Xceo2XYoGe&b#h8`^X8LR^rWqiJoxE34;zPwUMk@GxSB-p(# zkfO_C53nNQhEDTvwO5i^t`lwKSQhX(E+@!MbZT8M^Rs9;E6hiJV%S1vOFdQHnJTqe zN^^9f{7DavoOxz)keHHmkI^vze(=s>L{V=VOSw=|M8;``4x~w!3(cwoq z9A=cCY~oJ#Y`iVG572RuxZ%VSV&iIyAq%SviiJ17zme9G;7By53-26fU1;UD3Wg`8 z9G{a9XPqR^63B!43fKEK__Q(UX7$+X?DlWoCza|vVcEK~QQBIAhv@tko!GP3FxArb zGg{m;Mdt6@%;Xm{3B@IFp`nQAIqBH`4hkDTf7)H>^l1$+>%jFFFGv zN_(?HD$!qp0ZT8h9M{_ShZy&<-7Jy!`T(j!J|sjW!-YwoUc2Y<$Fgp>4nUdgSQe~| z?012B=d^@ydCd#kGjYj?!y=A#9cE_?S6>Y_mt2r_c!tH<-c_n7DV6#;mG2z+RuFuq zY!Ne^SV?8`{So3UgdWA3&%CC^guBC6@$s8Mv&hF%g94v8F<<0E#;Wh+Cahsay%TE1n47+9Gr0cxuCVQs@okHVy=Y{ z!8W$B(-&CD#>8FYZw!EA4i!6i00E`JFzLdv=nse`cK)Y=GHSDYAk#e8ymK4<#CL z^KKqF)!X!LdT?s(?W2McLhqvWW_|O3InOa!8pp`WBWUGqWk_lYKyssMnoVY~SG#GZ zAdNypO1-TA<}@C{0m(+K13tN#Oq`X+i;l~Zeao&7hH$FI-41epFtJl^zMj|ps4cB;G|X6HjvEEUAZGUka<_XO32o2 zwHo2L;gqN$B&e!;UGPnykkR{LS?FJxYN=BQ><>PL(tr%xcwrg)SF@yCfjD+UX|TgD zbxgGf_$pzVN}l98hI&z7e^t@d%bCGJ9p?J{PBW#dTg~m;`UPfrUslZF1x~IL_{^-O`QU2({xrTnW4Vr0vl)If( z(#}h99PmxV{m_r61e7U{HLc%%-|@iG6tFZu7nW-ygH4qFSwj9!-jT2`Mn-wu;r?lZ zTa-|cs2Ci=O(mOiQlghXERBhC8orKGCUbq&t@>HDo-BR;TwgIJX;M~HmerwEeb>9! zJ5R3Jr!xOdFr>)L_p#=w-elq-lE~TL*||}~_P+Ax0`%~h>aZxj_y;ObOg_uIGDK|z z)A?-MS>So-kYWj(cT9fyL6&mCBTz-yrjFAM?%RQ9=4 zgZ@{)gRY^8iTP)a!bTODh!rNZcF0#kWse%|)tXbCQ1ppNLHR>b-N}(; z%^B~{yc+R*DCGeCaw&tSj;T>Iqa?|BsGYj6l?GU(I_!$r!uOi0mHLa{C?kv%mWRpb z1HOzw&(!VaVZCFNY-l5mb8C8IB%As3?HI5RXRjfBH{K<9l z=+~f75Df{NNoOX~Ai4pTEu@)vw85sN)9oo;!Ji5L0X4Wa-6bjpg*~H^b%nZmlnU z&*wemsp^==me&EGVY51%5G4F9Z7s&Yq|iWgl3t962I3m%XO-|r zfG98R1U`^oT7S;T1zSjtXVAyzR=8RwmrR|bT({q><__DScol5wSYAEKQz4#Q_JQkm zjoPagTA8}$q&Al<^`pHOvk^y5{71TfxVvDoze&5>@s|pHrn2?K00P4Q+is_8;B0PX zWbE+S{VLThZE*+D-x&Bm^tV8WsYiy}2^;CV%`Ll2aB-uQh@i9#2wd~QsT4Q+Y7>sT z9?sW@#SpnA1E(Q`&&Sgo_t!noM{RuF8#Qg)#)`+Trmhq;su|jpB{%MN?ot?a;TK+5 zB01XpVhHs*j?_lZ+YJfRjBcN9t`tIY*6QTS;nr?!>a}OAI3{lN!;dQ7d8Dw1omkTF z2(9_Ff!`1Hezl$Y;F~<2uVC74yfgXUxPsNr;Hv3(zf@yZ>%O6XbiTdgukW(k`rN#Y zr1-uZysl-i-(0+3ZQoh8Ge*Z$wT-c8X?v&JHtDym@Zdi_M7H_-N;2p?o8}s8c7m-w z1vYnfZwwwYNmm_#kNyR9l2E0qbJkS6!*b!n_QvR!Fctd*lFVn#*vaIOX(6=z7w*kB{u=#h$O_tuHY?nu7r1c-CRl##j)N*b1X>EI-StFp+=2xSHBV~{zpYrYj=Q_q z`y6cN$22qY;uO2bk24U!Tj}QL)ycMYwY$FvHPft7r7?809kS!UBS!o_Hr2X=DE8Z0 zU@9pP4Rd9$p%U_H${U=7>#im=Q)y=W(c(w=gYBaB?2T(4{(kS7^Y?{F>LFP|pX(-z zrt0Zsmeg1{-LVs*2vz)Fcj}6e zr~^caRU+Z(@uk0U+f$a2soF^L?cw|5oivoW{gTtwGK?q!FUevscfcNGbfT`!cuJXu z=3*m-_>s|6^uI!MFuS8h49{(w?SRcat8cSN893lbf-R*w&_VwiG}-PjZfC>3^+e{+ z-Mxeif#}s_(^_Z(r}I==(U@O46uhqS)jvw6sOHj>E`dv{Pd8I#5tlwBQgJiH`o}F7 zpei*wB{q!D$JY%YxEJJ_C5Qix=SpRApv%_$xlV(}j=?C(AS#o)tsE!I9xE<;qI?Df zvqgsjmz+5`a+&1#FJ?BoCm$qe`eo(C(m@&o@tt zLHDCIV3^Gh1XYIBD`$jZ0-rInrRfFA#Pe@5xntd%xT0McT7UQZy<@>W(p|$G%cFod zUv74tNd@6bViBJFgqKxF!^Dvh9$nUVqleNIVLwOIObUsRDeDoRLs`JjoHr zpu=XVOfxJd52H?onBS3jH*kCYC1*ww5iH2N)r9nv9XMJ4%Gz{i|NT z(PUhcok4G?REl3J9;15rWNJfl%Qsd7m_>(fW`po>cLG>uYQvzx(u}W}djiHF0-L}q z1pYARh8oZywNx!l`TjLYVyp5J*+b%eD1g2-@Dv6K9my7R=B3>^gQB96G`(rCG7u+d zsg@tH%HstrnqN@2RYN;sm#dqZ!3auWG6Qmdy)Da^!iN$nMAf4N+ zo5~S|c~V1e@GlnOCsP1+mWHFqdJ28yGCJ!e_-l;BLnVbpeo z_$g5Ui;$?Vs0sCc)PVk76=fY*gI-HF4CE;2nQUP_FiBvUU2Ok4MvQN=M3G9p8}Q2E zY|M`FgDm&Tn`AM8fX7qmYa(WzAb|w6myc9=JZvYKFXyOc&=&db99m|8^smQNq6#c# zb_!CG5s59iE@PE-l6n&O(ua+8@)2K#1BzQ}-LATObCJY%nhqVziFky1^f-pSU~mu< z3uNqS*Fgce@uQ$$+?~$~uH2kNm4)(Qzb*79ruI=3b@Z-qs?;G!xxyD6SM0v&js-maNq zDEKRyWL|h7kVwKf*qle=izOA0aH1v`g?+=Jx+0EH_@Fur%yf~m8!PcKZ3Tw8Q}3{5 zyE;`gG>E?1jdVB!j2<~guNfpyU_cog^{{=We?UBV_O?-i+Qa6Q+g+^e3F;zII4Ol5 zikCx_#CGCul?dEhp^Ev1(p>E@iN##?+bF=ZSz+ZE*?th^Q7vGT!QWVti05o8PWLO) zd4=0{xiu{{sgbqm;oRsKK;P=#)~u74)7x*zgPnqBr^etjn+~$#Xu=>ZzDH$mOkN(4 z_ZN1`r*DYsAsUD=^S64hPVF2daj_UGfGcKuKUZCa*5jc%P1l(%6@1_2yR~{y;CGK# zw#he1xJ+5jzbtTNCxB_&LrWM@8)7CeDX@-4uUANc zIgHQXk^7&dB~nz$O!q~ zt;`@E)pDq#`w~7bd|9X0Xxmoi+(|c;PwHpRnWpsPmvp*~SUArxytP7{5i<_>=2M7c z)@LG8u5@)Xy;yQ|gMB!2j*fw8e3nDo686y8SF#4iSX=0h*{Z?7H^fpq2 zUVU`Ewu|$}%lM+~Hb37v1B!V6j-$ywUJNT1+<6k6477AntO^UW;Ad)82QPU$Ch3b` z{?bRJtgdhCw^z(MwmCU!j2TaKIu$y}Gh;Y5c}Ru8Ndg*KvXZ#j&-BJ!Aul_Ll*ew< z?Q}cAyBJ^Clak33MIBv&*h2rOr%ntl5tIxKJOUEfhZpkuB zmsh|pSC(nHJ zY4q|*t6sQr=yX>fgW4gaRKqm)G8V?1>ml^DO3z8QX8V}%Qe%5gtVMmB(b#f57Zx;U zOeDvQ=Xqa^5AttNeJE|{E2H%QBLUU7k&GRlv>~4Q`dY0-jG`az2xiOk1p@etmW#aK z#XMSilgb(od10D8%i`PQQ7o-ZAW#t(v0(mBeZlFP0gWKNR7Jo?NsDxaHLlt+sA-0L z414B&kc!yq^H!RcdPlS?WXTEu@4GqoHcVIziuuJrYp7oJUjiH*60GUq%A_elTMv0MTwm9m@h>LU4PKn&njR8d7xCBCcnREB z5I3+>rb5n(wS7|azhynW7P+}bYnz4~u1qcV?EGk)3P^E+$W+lV<()73jZ^yG@t7dR zH(%lFYD(J$YB?(HJ8-Uv@0pZt-(+SEBL{9R0AHGLdB$_iI;sH@OB*qE1nX^P_HaEm=b`IL-?YRFvI$)Lcgv1%D z#;WMhj<`Hma-n79mZu7KMtW1Zt>r2t&{Nd3v@_Z%zCf!}C2$hsqRXq1Q7&o%q*(7g z<%H=-OI|=CCED#@zQYk#de-<_0;M~# zOq@CVBjN0B^mKHYdxmRk3pgt_T43zjotxgHtcvrvT%?xj-Yg(N2JXPNh4 z_xf+fk~Xi+h7Fp29fu}15+qb07i(J0%PCDKU@oUio?u{jtN9C!Bb{vlp>IKY-YUr* za6(p!2a1Im5_=+KT+m-@AxS*@D)3Q<LQ z&GUZb2>k^kmX!()B+&}bk<`e8H2@VSX8IN>YO9x((!3=-Je^;fdYA60oW%#6Qo0Z@ zGBLehRZgvEUZA*VEbIINAy;-M?;BLv9GTF>8G3aPSX)$+kk z!*M+jLyJDUloPl9(GP)B-n|5t*vXU zZS^yt7N4#*oMyabor(C&jyqj)n-@(u|t7q%RQPv%N79&4}f3sn=kd z$2-MenmN?2|KWEsfbc*KrjJ~U>Dx+l?~jgX7ECJD9h4e#O{bq>OEB=7Rb$!I2yRtL zUP$1Kdd@)gd$A+Wx&0=VdbA?}wD1))mrX2^6*#0<+n_UxPyPg#8--jzABwb9U&X-< z)KWmEkaTK?p{iJsRCL&Wl(1nFfhbS&Q@ix)rNwDvdYDTkJn+U^F4`yAUQvDzQ%n{5 zS%*=SuWy+-kZUp=+D@Go(5lclizlz1jwX0Oj>@gGy5p4z{pzm#S>i`WEJU0Wu;dHB&`H(`w28;LlWgDc`gC3%JFnAU7Kw5u8EYVna`ei3o2B9BE z{fa2`sSqcn@;5owRFjwI6<)$S{J)R5i0QuErq5TM|EH-$@gHAxG9t%R%9P z*+fPsI|wa-Mh}S>pEDSP7w1O45*nWfTfK(ndg}j7X-zvw*^J4!sxw>ABNV|qOUCK7 z@$K|JhTCPufJinVAdH(!sXU5oFYoe@8#1$xoXidtr;WeM{L?|*p_W}i^6_AmC-)9b z-_id30dVWYUZxv*6Po9iV|m|evg+3;;eu#(Jc3=DtqDDi@EnS&eU?^<6oSo}ft8?> zSMdk5NygGZUKz2#O|6_7F<1YT?{$yR>54z=6FDrn`;|$b{9^F_^i;Ho1f!`4S#BAj zC~6Z}cHXOpu*5Us`FpU@%cD{A%Dizqotf>oSYDZ5Yf^az3x+FV8Pj3Dv!B@7 z4|1$VR6fQ#sx+7HhgO;<13sYt-ttm*T)L*8EpG97kp9P(R}&Wzk%pZRml>Oam!hW` zADyUIWSV8&+;fl{lcJZT12EPpic13M7{VAKsTC)gCRy3$S!Z|mV1_3cr|#(H;i>2) z#{klwf07!dmfhQfk&tdy6lY*!n*m4u)w-AFeiinM+JvHAt6l3)QG@>|Hsml+^zi9L z1%muMX#eAVI(=@uurhWrwqkH{bD|xWqN6I=1^E;4AL_l0j%T~_Q?+}2_Ia}ZNK_ON zl2a7^F$0Lt*F=KY^!^6BC>^3d1sfEor$Lnff*YW~zv(y1>#Zi->|B1)fn4SA@aAhI zdM~rsbP_{Em2LnPAk${xO`z!aMon{{iHzSY#gk(yYvHC~=&DSa6jSbWDGAOSUx63L z&4M)+(ELt4r$6;d4$pZ6><{(?mXu_L@l+#|0CF46?tUV9y6a#J{p|+?qX8PxBIEJH zYCrwh_6;S@JoJcHFF@)3Zk=u1#1n$ZN-iSkzU^+fBKN5#b#dxDA)+Q5OJUFqQ#hr@ zw#0#cfcy*GV=+y@%Dh$sXVqA<6Gw$sn38ImaBdn*={mWz@NAzCKx^FIw za}OT!@Ao0+fi|epL=WMI{gV8WvL9K*KNDosxf^?lMW^8gU+KZ^lhHR?np`T0#V_3> zPN&tfBJ2yo6L*}+Lze8|I%EAKWc>>`xLz$(2>?dDsR}$6Sa~s6*l1F@G z<}`ZIyK|;@J({=AOZ)26-`;o@{DruXo@zI`wo) z!lG@}C%YT3bG{N!Gu`Mn-(iCwLUM0SNvLI zK(M0&mMYJc&Jx`51|ly741xyqf3FYt+2{Z3)8zk;(|^a`{<&J(7wC3r79} zQY`#0$iG69|0@00CB;9a303|g{m%u)KO_EbdFT&BvfAGv{Gw|I&Df*XJtt^d=V{L8yhSs zitq2tfWC5#7&O?Ftno1XYnb|Mz}2eST8%{2z~4jEETbcI|3WDa$uZp(uY3Gvo)72Q zbv~U=ZYk~CUihhiHw-?3RyirlTc{@YTI_nDKi(X=s8>7$m;e9B$X?2x@&E+_nt=ZQ z$*?i7u>C(}l&H#BZ80EqLp~4&O9AlN5CU4lz?FoO1c=BAT+Ry63f3Mphf=jAWE+Bi zUSny-^|IzdZ-zBz7qi%zGZUo#;siUapUWynBU@<_v;J|DT*@h3``O*y+u&5Ni?_#%p=zgS)@c&WbEbRxY(C3*q#9cL%0Y+C^qM zMB)y(qt(5>B7FrCjZu}#3%iiPfK`n~nm;Dvhp`@q-$OdgzKuW&B1RMGD1h1G40z29 zC7L)!ZO|e)I4`-s%`1ppy55;$N@7MAErDe}S$$@vl>dn|r!DGl5bD=aOI3!@eT)K& z0Ed4nTr9%=={fVQP7gv=}ky^1NcgOK|eHIi7 z>{pvSjT-ARL|h`HdYNHUbf$f4kVREQYYSi&yYP%USJHKZVcRD_m-whMO2n`r=jJjy zV@IKI`oq98F}N&UCo4>EV@d2m_^-Ul|FO2ef5}oZCzP+xK760>3$ycQ^r%avf4$8q zVN~KrH@U#g#2iTR8pqz2PD1T+>DkE9)z`t(D`h|GvQAI75t^Y6Y_bB!zf4W-ZS#71 z`g!&0YpTDOHYk1UqQs;dZuN`$cB{$q?u#1=)hb*U12xY0ev@gyef&ZrlCw`4$2YTQ zaWl14m{7Yf7H;QX45c~FL??qUu}w=irUGPIS4aypp8^>CLePP}YvNBK7T0qx8GjGe zI$U(g$)zm42q=y23jmq1#r~H8xZGTzOE^ww2lWh1H(jzw@=~=XF7CD?M)Jy9CU^^v zxrEE};pb>@)BwOZ+_n&W%ED9}LAwQWlSW!3_wn%284N6Z>^vspZ@Mr|exEp7@=OSY zzrgg;xH_zdHuXeTQ7<+8c-ybjyXbZuJijen(ff?D7)1d2N_RUNkiv1P?51vBa~3Q+ zdF{EQzS0jPAqkVMhTp5lfjc^PW-H71j}H~#32ywz%P#wYTFPnTcH`i5tMnIhV|x2v zUV3aGEJ!x^0!RiiW4mhC5i2S8oiIrHLBynpfp4ERnc}8#Fh4QAk*zna&B_UdjjuEU zB4$(lH%5S_(UgQyPwE01c-!xveh&NkX=;{VO9!%MY_40vzTT`XMtVrg7zAa1gm`GK zevUe5G!zQmL6~;qFW~=4)TY0#viTh@SQZTb8I3X!5dQz>H~*LK&>7h?v(vg+*fKNy z=RC&O-mvayA}i)Hr4;9+Nzy?PU~mh4ME$`W<43$1>G7E2AW5rW!KA^)L=hgGhy-+$ z9OIb<87bQ-I1|eN{!-F=aV`W{wZPSvEK{iqrKeC%XEc{EZ^hbHUu5Hson*^>~;P2+G5QliLGF$^z*?IAPAe42EUdHFV(Kpli2 zT=sITSiVoBgG-CU>2dNnCJUF2Xtsl5az#3mO&Dkmpm(67Dx@{IS@wl{uJSP84VJX{ zyFBbj9Vf9EY&6*)INzq_rU{sLW@cVE5yboJZg-*7e_2)3H7UVY2ubFN(D8<12f!S> znnENMFm1HN%Iu`KV(!=Trcv#Y?#4DRd()l3yp0t-6_SbgE&?9RUr=@{v*B;MtGy2G zc@#9nhS+eMLaPPBVRR-H*cl2OyXRKvQXE$L_(o_p@)oylod;M)qz!n8!q zd(=}#czm&)Snpwu>_q9k^Z2dGpovZ@y*Gm?i(yE?c&h1TO}D3bjq0O1lApQ=3`)*~ zwHzW62xqaiN0co-{8G4UuMGpQ8!uy38=xJBm0JO-t*rK-sEZ@f<8%0oeiUs~6PVC? zsqJ8CF9JupHMj{}nwvFdDoG7c>-o~wO0kp6@aer(l$hFRPHo;IWTO3*C&9Uj*8#E@ z&cP^uwnl~M-JlxN;SFZATfHzqrz%|uTmA0+?VsP&giak4T31US~5+l>7#7)z^H=5bQ(kw>8^dgTuq_(n?CV0F6;}`Yv_@(p919`4Gd8+&g+mIj~x4L_Ubec{BSXx6h%pTu2Y7ddaa$$ zrX|d7d4N(fYfH`AF2Ycr2j*>lT39eFR`c!sF{Q9G?Mg__v(7PfA{^T?mI0DpOnpk+C+R)Zw({O090&E>ja*y>kYm28MJG z;IOU|Lj)2CQP)Lm1aUqYX4k?N>w62;xmDf8y8H@BUq;u+lNjkXn*wCi zu6=J6!?s$w4z2PuUaC04-L6q1*&UVIH69T92Si^k>C~MoB`yJ4>yn~lCeFTKeP zYVIC{Up#akF<*H%UlBYI6gckPDpD71nSa3hL;Mg(ief{C`2ax4_DPw|?KH-;^%A=kd;OrMOmB6P7J zNelJ?3v@=^`N)Qn`X^CM3e>>?@eAr1fcX;#t63`>(LYsAxn{Cn9yFBtX>&2$B~ig! z{$Lg7pyltSw}3jaAjcH~FEr^kVH>&^RsYH6V#wEzr5F;-BBRTbBse~uJ(o3 zA=2to(1+yKxkZN96o1IXDnd`k%Vw3i@nkjnkii*6X{ntZn#mr zw)v0acFKUuXE5LTG{Og37!K*;>N;AmD`{BbuwZ{mpUWiN1j+1=x3f`@JG1|M09`WP z>1}PSbM^~g3G;7ZIV5hC#a{t@J(lo4vW*23f2w)TYrG+;iBSS36|eSr;Wj^ou1cf8 zkgiIBK~7LoVh`E!r+d!EdQK2Ru3<5Q5Tl>=rfxt4pTw z>bFl?y&7G6W>XBE$k}AcmF*x-_mtw|C2=dmBVqbr^gfHw-XQO*Q0Yx z@EWo2sqMxgCFyWH)r%OoRdz}|$w-DYkjVLA5PM#q#K!2)_yDnC$Ei1TC7eJjN)H;6 zI%(Ld%g7yw)_PW(lLk6nt*?;^*{4!wD0DYjvJ{SWDZ4 zo4Pj}WYlgEs3j1-2bT7j-3EpH$?%3jb!b}0_1AO`*l4cf+DKiohUw;dP>eHVD|1Lj zMOaZKX8iR~1fIRX#tsNP5gLk8NHX3U0pDvQO00^%iG+q&j`UAYkGZq-XQE56EZhD8E<3jz`Cio&i?N5xOGBDqqV42%S|9iNQ#CJ=~HTYcLVu*taxXSH#zhl29Vd znpk_(K}-NTXyK|B`>J(^JH`-%1KQppvzz;_oj<8qHJH~?(Ot(qDWKPVK;{Y5^T3QZ zRpJD&hS&B5!)>0dCO| z%ZA-b;|F%eGnKR$pkP>8)lp1#HbulLW`HIyv3?&K37m7cZbVA9&cle6bVkSCg@bH; z=h|`J)f`EKu>c4_A;r3%%!a*uN8e~UAx6U;R3fW=+wFP2jY=f`bqb9Umf@5ksvl1}1sh0l@ zQIUjKMB^{vE(b7S<^(UgCok=k(JlBUp7SSwMt1AFQ%Gn2q_aX0@ zx2z0f1UjW8Zqec@g!kzesy z4=M9n(f>pyM1*gbU(r+eQR8uFqDwk5<3gw8rsD&WRlnq!;;^cH@t1U6O4vT7JRgP5 z*|ocUX-0h8C~P%i_QIrQ9eV7?hiX4Ms4=?M=>k)A@856*@W~?<8;d{z&7F6a{hjwO zC7buc>n>aH;_5b=|Frg;Eph|xrZ0HK@!S=^wRzH&xut#TiQQS9`=s)0UGQSwA`Ug* z6w5wMUh#%RS*q3y6m&y`_yt}YX&s=uV~qz&ZM2)qP!qGwhfD+*$hO!EI(-EkiyFN^X7F5?kg$ys@K0s1BAqOBvigjhufOYZ%ZnOTo z5AMS#efV+Ztka4WrSK1#GEGPwsFa(p6_W-R%tB`1;TnR+EFCqL;W1(6tR&_{coky) zT<46%+nD4j@q<_)PYkaLK!b|@Es)x#NUOR3s$1a%_adCEwuUc|>^0}=5zkh3FprIw zLNh-!n)sISKcH-)p7%BjwcSC$72xFN(^rgEpewigXV95!)WXIFM)W2*S_1@L>P%W1 z)8YcTH$#ONZl@F*<=2EH_WMZ`3lGh1rSQ4vB|zemj;EBIz@Jo=n4@k`NfG9#D}#Al z5?|uq&U@{%5-vcz$?_lESA@D%Qn~OxU;?tm81J-8OzCKXfM=avF!slbc-kQzzcns6 zXy!n@?~Us=;V*-*-6z2DMI}bbHuFSMkTX5j7(CbYmk8*v&hRmo(s~>^S8ISbR`iJm z&IG)19|NmbDPGpZ3u#BIHp&S)YFQikW_2TMmKeE6 zppLr6Ol8OSYiCG&6*o$(s>2DRmwg`pS+>Ob*t5;E-wln8BzsX8 z#6f7QK#TFKe+_;eil#wGD)?{!*&z_o=G0{ACM-2~GTD=M+c#F7VtsbJntgxTZkA3x z?RBTgjM{X97nf1#N+S$6EpuHN)VVN>k53bHP?^jHG8b{5WOnr0MKKhLSkUMAh_7Jcz<1@0nAKCeV~zi ztMa6va(`&1Px;!vN>jOg*Gg+SgF><6CVw3<_h|rb3I-vgH&WNcj3a)-tAKj*>z)Pf zmbZl&-9}VYqMJg8aFC*ZtoZp@kc4Z$W`p!{o7&oM&4sns=xP$zkc2#+w_vlshaY}9 zL?0wwIR=$#KQmB@9f^I$MQkiO`(ZUJND%$Lynh|gcQU;*=Z%R05q3e5_r88u+HTPD zoxRtYV;lg<81fY^iTAXwa+_~PD5IsQ2y)A)Q*~W(`;gNO2lf8cFwe)hQGAheFhCh#twJq>@|{OA*k1O{fp z5MEzM@)j3*A4D^5lrYre%YfrPS363_Y@SfFET1~awR(P^I`l~%5mU4}Ah3=utiC@K zdP|F&GCIV9P?4|5KLwpZ9u88B^U>trqkprWIjEa{h|=M!r9Z<*L37{L)NL^$&2L2E&j2|xMn6hh z+>D@BZ)35Y-wSI7bFbD{5#kMOG5{!w=Sn~1D{3djfwLzIy<3VCvV)A^_{!nMHgLQyi%Cu2B;iJ#_kcH+@b6#N+c_}7JK?vJ)IAW}f&}!fa zn>!@jXTGJKG!|0CP(uXp0gc=n=yx8KOt%u?q9N`%5(&khD@)nsI6}e(edp9D<%1ELt)|MNxQ(h#`VkB-{qK)5eRRN!!;CK7=*k#a2Z2Z>YFq zZp{>$T-h`P8Q^^*F^yJxk@Q)v2f3Qwz?m7Q(&=%a+(#4eEw4L~`%r}Ajuurg7<~M~ z-4E)w`2e!E8Wq8N@BJOE07@7{Nrrr++`_m2p5}iNB4~Pul9Yu=2nt`6Fj16;DVN#w zpMyzYB13mAs@0AQKO~ zclWexD>%x)W<4>-HYC+RD$~jt+Slm^r&JeJ%e_IFdYZiuf5nTjR)o+EC9qQ-r{V`r zoBb%P=~CmFhN*{-7++0`V4Yu9zX!QO~nK37;}MYG}}bZVagWw5J*4K1JRQ4PZJ)aBw^alRZT#FBLX6ZeA|&c<`r)G5@CJ6g1E)rx zL79`kWYkiJ_l5hD8@5iYZ=E&-=8+>ESP+XA>4BMFdPG`l61z?=Ihn8sz;Kv3IHn6~ zGXofmOAz;J#n_tpiz}EPI_JE8IF~UH7kB#?Hej6*w(ynw}n;0+`Xs+pf21@T% z^UsC6x4E8FN12d)`7o!)9+FKb5HW<@iaEF}yB9bGyFNbvJW}^Zl-BsDIRgbgLT7^y z_G0zjf)j1c7zI13zQ)qR@T=VQ@IRx&Szj_5uoZb0ZyPAL9NX87D$^o@7?`>H7vX{Q z&^(YVQ!A{$eKZ|r^;vJ1rH=eDo55n}1VY=wY`;&uoGPf*fx=KdT!pe2*i4|?_NQ_J z!b$G9HIe@g0|lgf<(b1&sjpLszW&!sFHP=-zCn6~kU*C5(;R6pj6&0WzQRa#l}+wK zM&F=lb?|)0k`p<@@ghEk6@3>yb%%G$6@7@YnbG*5?&=1G!Kss;a3}Lb~lm_I&fuspN=a`xjaI{M;*l)eImT(0XYP>HEu^zv2xDMw7Jtlkau} zVuFWY%HD#xYs&7D@Cd$HmisT zJ~crw%_X|d66z}ngZ@Vm!+V(Cprq0wYc%q=Qb04TTlp?mg}pDQ8@Y%%w=_MrR|OV z!QnRowCaIA1Nu@lS^wtfC$H)@+=jq+y#R%m=yK{^Vt@dVfn;4gEMRGc`|yKe4YJObPpC$Pbau?5PW!w${gYKu)X`Dh# z^7Y}w8ld3ZDcF^2#z7DQshoz1KFYMX^HxCj8yZY?`~EU7)qRJ%%`{gEjMXn%H?do{ z*>R{P8 zI9Ig%Q{3_~jLS9-SCej(| z%99@qJ)zZHeG&7>Hk`Heu9G%N54GxLx6o~)U?abiIk2(zglBEa7q_!}V@j>8wTaJo zY+{2AIMy@X9qr(Gv=N1%p42bC?F7)E0T5*4iG(1y4SBds-gHio%Ei^IO{E-irq1{p zWQSvf!|TfhjE{n`q6Avqe_(Ei;8b0X(W71*pK7AI1`{U5OAK*p)WZ;6v4rks5I)UO z-|ypZ0ipADxG|w1_t2i^rN#e%300`(@#fPXtg!m&V}jcz`#H*k5yXbQs1UCqyrxZ@ zDQfyh{sdu#b#7^Eiz-zt0b*xN1Xx&kcLXIM-c7%4*)v%UO@-z1=}*J@kVf8!TzYYW#{uG=sKpQ-rad z(-&px zHO(E5m@GQ%>)fmNVzTxnP~un74Vh4;*d@Gz*);Ln%TbrwxncC-U*^1T^Lo|xOY-#A znL$GsbbKkoxaG{dMwki0dL6((uQvWj{UheJih~zE2SLNkhr%7Om)Iq{5&gm$qJKl- zZ~Q{khK&xM3Pb~zfVRjFj%SX45%$DU#O)&pd%`IeEm}GOIVvO433GP(ON=AV8J{X> z8TzyaB9yB`v~L^oKrGHJRnmaGXAxhj&s*FG#1DKfBTnybpINXw9Ir{liUEok-y*O9 zUje&d7XLxaD9;b92vWQ*pBr->Nrv-C?&W85|CmM`XHOp@+4fd|<}1()u3tuhbW^JZ zrMBQMXS6sWy3Y-eM6Knp(yeX!Q`<&IO+lM`?4 zeWJFqw378!SsH5EIBr;Bqn$e-*ZAd}Q&aEwx35~p+hn=BMYA!&u0f+fZ^w`o6y$zx zrrP@GUX4d3*URaz$XbXC!~d|hxd93Nsc-pJyFrqwjO1>C)exTud!33v&iXm!7FjYVaw z(3zry7DIVCspLIhKB|l*ttej6RR4`YY$TOa3R_F$Q6`RzsWS3eH3NlG7FJcn0c|N& z6C58o8&~Ko%~WQNG{o%Os;0Xoih{SSNjG>X)Z7%c z6)m|UF|=4Q&Bml$%W5jm9q~C2E5RGy_)_RldJr2sidi{RTshYD*Zx_;)YqX0vYm{d)pFy-Qr1Kk*JS%g^&5Rlmm$}0S)0B`qrq-i z>Y-oW3(~Eu`TG4>kBy(lSseGkr={fY^9X(9CBJjmuq=(2u=@S$dX=r0(ar1oBj4vT z|C?M89|zmv2R*l~-e*jU(brSm%MT+xO99&!`_E+x=BT&)_tP56a$uvJ>eRQR=2lhz ze_tbkFSb9EDe0~3x<31Mul)R{O(!8iHeODNay@C^8&OKXFY6bRvnequG+&$5S+}{I zhg081in>~#S%K@ZlIw-=dQ0EU4{4h#bShnsceixk7MG@4euucIehK$0dF_~|Y{hn2 zK6DNCqoK2Y?bCii&2JZ@lTvE@z3YkJYuwYHh@`DUzxyv?u6i6Ael!y6%JRrCyY3-@tl=y*CDc3~gqT5d9V23anYzcst~oZf|+Wp)}FPba;agM?i#30j z(#W?yEm8y>G}&KtXImml>xgNy!|G&S&1`MY;BAUbnIz$_jY#EvN+tJNT1R7T z=`PwrQu}#;maDG!`xrAd|Dsyc7nNuIZd%!h&gOeoJfyZ%2m8;8qb4gLx0|AcR9^cn z`4i(et=2{()9K;fYmhI4Z~u~8cFTpV(%DhJtmQTD)3qk6Vw@+l)W+|7E{T5D=PCkh zNB+*$$<$zmcD9%6n0N8NvGwP7*Fc@t!>(3u;Y-%;za>`P_qW-&pXa`!)|=&;wz%)K zYsuxIXW)%JNPO5@e1@I=Z#(|v z;nsn5M2{)w~(~rzvDLedIwQtM5f#bg^vtKU)Q@Jv3+dixG zh3wxR%TBwib~Jt`0b%?$ZnA3?oV|A_p48N;(dHl|- z_uoIc8;CTBKNb@mMm9;ubvthM?Z#HQ`cFa|?uVZ|P{O<)%>(KxJ)iUB_`FVs$u1wgW!`$5lPo#e>2YcvPbU$< z`HfRjJ!QtTHs0g?{k>kWL0C1_wyps^z}`FIw?WluB64Vo1Jh0zr8?nESQ@IH>dd%D z=vzJ9=pD>J1B`6{CyL)n+T-!JIMC(j`#lY`u}19iH}v7{?E53|zkR|!@VkJaRsML` zqm-dpz45>AL<1j?4uakc_pgX2fgb44cWMq)xg-HUc%%15P^b){xPx(Ik^c2t>ObTM zxb~=SsDwgaAVO_Oe8~Xd0&Pk^=6l4pjLr-~F%Xauhr>4p6fC5Xko!YuRE}KSSvjJ} zKIkp5AB+QBcUU)ALfJ2n!FGG!KL(%_j;QZi5q^6$TQy&#C`cR;SwoMc4DqM#zDx$m zN$ulbXod2!`=xhE5q1CV^rfhw@|R>PJuX63Wsk@l+L8QBY_@H_os$Z1=48!?#S55# zVfN1wNzcLALilkjfNjrg&xoPy=8MxM#{KaXRM1A_PXY)sw`X{>+#^Pw6uxtd3(Vnf z;dh0CL0E9W6AGvAVbO*DCv;DpE_D@USD&`*_%K}09uISL6qSu5H|i8M+|M2)t&bwn zfA1e%g4~~gehF=qJZUTyrnIRtq!6kQseP9rwUR?;Z>oEx632?JXcHAr+1W`BKNXy` zk;wrNWj9B+2PQuGl$nx8OatDu8RQwW3wcdYfDMpVnoygmxInfRNCoa*HYBDV)f4Cy zCFvkf-=H4cppzfd9+}r_=|K*OTF^JP`|A$|_Us$xAlq?_NBja=w9W5K;R^-b7UoSR zg%490p?qw4@^;&_-|+_cx5_E}bC8aDLDvZApr>69%WZKU7NT9yDt4cjD_RbFe%CB0 z_amxvF1|<^>>V9K&^|(ZKjS1}VF*V9?py~)Bt}5DkmnB)oQtLZL!!b{_-Pn=(K)l^u-~)Y7urPCC0ayEZw-#O2Ai`)?!oMwM89J%6zdyu~7AE}*XFQS{>5 z_-CxAA^&In3-=WhxT!Or?ttQLq=S~9xb6euYY&|>Bc9At=qlckIjSNRo|(7TOpTpI zF(cm=`F=nZ{s)7v1H-TQKa+$A`d-XS7PDeenbdD}n$Vh|p;4tDD}DrckzUyb=U@Y8 z3EvMTFhs7hgP7*Kf0G!X-5x-xGY;bdVR3^Mxr1541pqyM5fHxA7)QjlR+nBnrf-79 zIq@(|Qq9^u%=^(040PZ2d{(9%K+sqBEw-*}mgFW0ej*S8cV@jzBZi?%xnCeNK+}Wh zSLx>$S&UkKv_kCsa)z@>Ozr*RDpp@&T z`fr7wCZK!67?|WTm75>}Y3VzP(mw}(nx5Z6pZy}0voa=w^|iqDi+*un z(>92_4}XiN>7V#(Umx76Zg+KL zVyVtrR5z=-zfMcvBYoG9<-a>RVDtB^*Iv}nRB9Q}SJP!umXELorcNUNBsGh3DSIlJs^_dOwRQa&-CB}g zFt5?whi!Y+25tVydhvImwWT!LTwRS?Z}QXQ-~tH~S-R zQq_U(D#wZq>YOTtUgAVR_4(ASLXc(O0|-*7!{(xBDnf-53`8-ZZp{ zmXX3SW?oGvL`AsxbAL6m$m@VOKP}=AzyvJ@^F##cS;wxTDIE+3LNnOaH6U-%k{)R@ z{S3r?yNrKlZo}lYES9OJWtEB395KJ_{_%}QBAvBvZP|n~C8N^zs8K}8mVs-D8z4s?UY#;q?+zn#f+rt44VOX=%R+K49WZUI zYr~L{whzUsn!%K19V5EAmtYYSkz$X*r&hF{Mv)YJ@#CIqi=w z!#NZe5jm+;;rwqAbkVvJhj|Etx56Bvzw%DNg-Kt%dKo>atP?N~b}CT%1*HowY-Fnm z?+Ymbk|Z-|W0DEE+tIPEtgzWuJszc}fOs_^O$=WO5e1!l5D{h`!lHh{v9-wtgluGL zFy3{*`M%ERfD_@Ds!Ws?Q<}nR%wv&1Iu%Tcd{(dmQ)x#uE9%5OYu?FN5=4zO<%&-Ja_u0=@$7NtME1@`{iN;hk zc(}8U3vR3ImmeJ{2`|Dd#^_x7>HXraJ|6B)4CJAE8CA2lu+0W`UpQNA_dw*j328cJ zDf>d_eukz%?r9`Je->CMdZ6=|3364YPIzSS$DXrq*xual?RPLuHP)^}u%woknBmea z-2KwsaemCe9qw66$q@qUX=>;QQb=)4jVrhWF@Hd~G~xHuRrvo^Bkta~%a7?BERPRx z7m;CeO8Kj!sYtXU-Zj5GcqxffWht%l`d^0@zRb4cWVDsQ`a3Qqdgt`aa8B4N+UL++ z)(*UtCwNO31YgvBEYxePR&N!cW-c?awS%Fhn;p-{kFf&zr&LjGKG2v-X z{Bd3-lX~t}3JzYZs!3BCTJl`wK{RJRFV~uZt3;FE6e>A66}my?lvHAo;X=+an=C0M z{1lmf&J$d+VY25IGw3C(=;m%AxX)kE@|#AF@Oz7};*#0P@@kR05RiKs$FB=t>f<2+Ya)LS?@+Dtl3QcA+4 zpqHtYu$68h-%ab{Yo~qRKiVbZ&-`+KU>>I@@6ElEq}1S+=mWT0ZzH&Aa{*<+adBc{$^xUBxIZOdWO@zV?=le z%#2wlsp{|Ck_Ans3Xb3-=Ja8v8q2N4jD<7fIpZ*+l+nsa({S*pY*Y@m7H%h73)xG_ zn%~e(953>x#Z~)Sz}jVfG@HPrh22W1Rhc>39jf30J@Nmh9$IK~zK?c~9D#aG{=}v!dxjM{$gV99eh#lg&a3Y2sBDUb45+L$EQQ zhpD5rqoSkcL*kLi`1AP5QJFr^jyPX7UebfRWcCf{w#GI(ep+UVr6l}(-aPf z;CO6a;FwYAaIDOF78C4C+eO~;?PA_s-qP#&=QH5zzZp)lg?A=7EiWy#jqAoi6Pd-bh1J4J ziH*c&X6P?PI?PwuC{2V0Y9j)TzS2}`dJSXT=v9P894E39)rqvUp0wZ8aJm(vwzYCK zIUB31gSCsb(^aWD&p)7hiF9)9qLqKI6VQnE%5lm>U?|+1cy; z-=u=QM~#M-(n9l~Klf12%)@C*abTcz8eeyP{M2AAs2 z<*lfIjlxBF^Ifgd>RU4&?+AOI04vT_HPS#D}+y+iE1{}C2n z@2i-0;o#z0{gkY}-!G-U4>?N7kxKDzR1wjT|~i{xK_v7uLm zqC8ZJ!T8V!&PkVk1xD!KX*QMA2oxgJYkf!^#;P33c-@P_d<1v3d%>&%)Kn?;FoRycTR0)_-d?S zu90|slS3E7pV>knL>O^#@v%h~^38~aGVXlH_B0Ea6J+EwwbUSUfO%}OJW;LdU9b`G zK4fGqOHDFHyIM31s7J>7IhEWxgp?2&lJu}c@{+;`e>4c?6t*%m28^1tcj`!Csud;* zBnb73CNk?@ihP9qP?#!8au+4~>}C@k=RsNZ*`#}4VbKd1oY>F`fGw+r&$0qzKYWPFR_sL?c^9lbKn{ZO z-2ZsUNg}9&lz3|*W~zj@Z9EE^U>qu86;n4MqjrBFs(@RY_0W5`s#oD^%I562)wr*L9Ub!HR^pjqffIC#OjrkTKxAl1t*X_svZQKh*3t zsK0;-$siFKObTYEkz&R)E5ac}hxeevfCq}Y2Nh|w!D|q_`Wmrkv0@8%bxV8CPSZtz zGFI5`V};p>Tf^88AD=&9rmzP8JK%yo;ZM;bgfyTs(|3)W%5V)du-orih*lN<*r0}p z`u#5|qagC;gf;ni5Xud+vaCkF@W?}KyIHp(ZjHD?)oGWa?PC7zDlQAjC~)%q$Z}pb z*NEEzz}dO%JB9tgJMT|W+Q%elI8DUYzNhcAzgc8~UD5TXUxNVSY}rc=?7t7*$FXI+ zK-DkHIrjj|*3aI6rDzGU>5i08KH?sJtJ?&MIGpI=B}*2tRQN>-P$S25Y^6WaH;7;( z;{Y#9J2`CLMer}X)RpGio73sMbwi22fS4#>CF)7RsW9Mw7T=_@Ok0*WSt3(AU=Al< zx`e~2L!b2IND@_!MX24Y-A(`5FJ${-%S00nYBaIX?#6fYHZkvE;xP(hy1NJmCEOU( z1rzk$z*>DY;i8upWSt#UVPtu;AiDIcO8Mwm`QSifKYs43rL&^PIyAA@*6-!u(vo#r zw3fsCrq-=;w-)rgu+0^&m~Yzn(0wODHGUYgZ1L5GRBbEh)yks!q;fPwIl|}`n!>jc2+lq)F&Ph6YE74 zuxDUOaVXjqaB-#J(&ZSEN0L75mcBIz-qfk6x~|pX-lAsT;&aZ3WI}>`fDPb>L`Q;3 zx!?_!+M34~_A(iYygknNtO!8w@bWkV%g)x0+rBTNn+F0w#C3KvTBq|3y5GvY91|5# zmiRpKJs&PC00YgWJwL_4cVj*Wb3!{fS>-j8p^PxaEGZG5#ZlK+Jmu={io!?ATddT;$H>^n){YnnphR&c;>jK_}m zOT^$0lSz>|-P)txW7?l*&>qPHt)88|OEKgmFjxpn_m_m(&dfB%ys$sB1DAgglhOlp zRiS1E(j$t;KFlqe!)&!+c;K$)uRx+5b8QHb|cAgD9iGg}q~&{hd`c$bICL zYCJOH7)|p&KYc@l%%+s&3%Q4fO>`$%ASOnvarZTJ|0)&D)E^(1LnC#vru&EB>PXmx zqWsJCE-t0XUQ&HPiU@k6uSBU177sQ)f;Ver3~Yz=z8`6}y?q2!Lxw4g(3XQqbl)qA ztaO#ycOoHQ?VkN98W1)o6G)4#_0Bmf#E(*fQv3kbG95j993S>TX%q2{PBB4B51yO@ z9iOW*zerw@g|q@rfF{4u@vwT8nZy~86@2d=Tb4}&xm2B)%}iFUtv=xgkWPU-N!h#X3s@RJvBCEXkss9ZVPqrG?fJyKYM=C@w;O3?!}AS@hrRf>-9_! zMTZ+rw;R{ZZl>hQ#>MY4in>DhzZyFWpt!bWVdF5k2X}W#@Zj!l!QI{6-Q6L$L-61Z zgS)#!@Zdon=iPhXIXD0L-+eQ+Yig>dzge~S+Piyo_v+5sIkw}v)6td@bDKJmv#TS| zhOvCv_B?p?;&h=zW5Z#+*tuQ<=OMzn4^^7X<`~08oT!oQ*v1V7g~E*~K3!|OEM8*6 z=L|~qv(7atSUl~;N1C{-e;eVxFTNljS=?|EKhgo$&DFjeQnIyk-XN?Wt+?7ZBl(Kr zTe4wfw{g#~Hp^fJqI1yc9IC{|`6D+MjSP95LDX;&_kNl7gYd0xb{i~R3)k)7x!#HR zO2`xR6_~q!j$^0{!@9qi?pm%k@+M)It2jj5LjTJT3^x7v>^sIM+ckiIxvwa){NR?w zsSa|qp0~rT19cC>7f8|C>!6}ufJA8Py0Y2lG0iyf|R)-$OaKQK@kpTdip zYKk26WcIAq-K+WiIa~zpsWWbIu%tWtBxN_rEm0vL+Z;1a7i9|>618(^2e;#um$X6B z;1s2Xv>tk3SqB1@KItq?z12}C=lw=YpT88YD8x}VyFIsj0{d5loZjbZqz1JF*E*4@ z;!Y1mk3lQ^GLeH0lb6|Mg6W!jp<6V$%y9J0Eomv;^tV3fDpe`tsv(lQStprEhJEm$ ztRK?Q499C0;O(x-Bz$p5$zO(1Hi@7`@ch=fIS3(d1s~Y)GHo6KJ>-iBYUT<1mBOM( z2;z9h$3@ld3pU{ppeE;sJ{1K`zm@=43t&I2CorxKi~vmq8r~25id>oW5$(edRwhSR zH6o}OoVjJ24eRo)nEUrULPYrkJYRCD49L3ArQyVE+gFgcO*9fXaS`$RtFM1;VuTIK zQEfaaa1dI#*{K6F`b-2cz3$1}9{e^TdB)QzQKYOedeOD(^}fNp1@?Ob{E-O9piI^J za!o$GzEJMnr~Uu;*e2L@{Hg=;i{QTHo>lebIGQt&UM zH{Oxmu?$AF&T{k2bcE~p`1<=6OYJUEji_0Tb?SPMzfhFyb}Tp^fo<(c#I-{!9APXc zN9W55q2I*l)G@EpQ;+|QcdApt3au~@ z38&$2ElzUpnD}1Wl3hs>6k7+1jRs5{dUb|46qpf%l8u|{5TX zaQ^_j4|1RQVAXUzK%Z+_)d;NMT8SfI?olbL8-B)^$5tMBFpf$76nwWwM|8F8sA(YhBNY! zwgqL8Vh}lUitTcOIDJDPo{@l$n<*cBP(J9}@C(P5M~$dn)365=96!HrD=NAzZWulq zX(Na!#u&a(EXFN5=CzT;G*kHFhlIpOVwzgbG_&URXAp8Nyef^1ZbW^1BcGJ#qOTc5 z2wEXfxD5ySqP8Wp_W=~0`ktFE19t=i1@MDEC35V2jY-hm#H)QpOirxoP!xNUM#LKN zub6j~$>7Y-dJfhM{mYl;TByW&_}Z=W1v_0}@tulHIUc^`aosxxhVhQ_$jKohg-@PG zk0kTkrxBE6d6%xF^!=cFBzquMl=1Q`^`Asvv++q)E4a=DLVNBQKKIqv)Nka*An-=g zX2GRseM29ly8U4G^3k87@hwDHb~-=oL4bUY%1qKKpphDRYD30lP;yA0Y9CvRju`=B zeph__RjZi>5ly64ctdro(jXs4!Edz-v4BZ8(OANuhp-X6$cNbNm@;HGmqU*F5_GKC z9X_~-N9HpZk!&;|Oq*_skE+lr8@;>eM?{iSq$<@RlLN}@pi0yS?E?~nyql|OlEAuF zZR>u4Xh)R&3^~}LJ>56DhXblBMBq<4O*8<)CybRP58 z;Z(r-bTYf*)yxIxdX119YzGP6OaaU3-aFf;b!!Ou!4XvWhEg^O4n!sWK6$Nh#*Cd; zyyBW!(p!(Cj_^frF83|7l445U0re-6Td1mdPW~PY z)VMkWy5mgkWbS-Cw^(WfEDFd{qL}~-#&f}Ho z0op`Veac1-j5ZSNtE{KZG-0a~hy>cUJ77&zn-p$*zMoLSS z7p|#aAF+#Z8>LJCwxOZney&cVF*Rot>z}LKcc>#L z|G2S%IHv6rI`96a5o~=9$!4|c@Q@z8{R`xNb8cB4d#+3W?ww2e>erDYjgr}A36Z{Z z2gcy$I6C>1QtP>lAkhQ4A(~q5Pw7-$D)YmP+Y(`Qm21ymo79p$=T{WKN~iH`z8nIZ zoFq$fuxUm-N&uBm^!sXj@W;0YU~ZGd*edq}^=}k)n%~oC)1bj{^2;NitTQ|!6EE>*=@=+ccwo%y9PE~iO%WRg@9bG63=^;_F&P)!I zQj;%I8@U~S>~)3=NvEFnzWCN{3vjCEIgKI^-dp%YZ7-<+U49t?VG-n0WyZ*aG0XHP z0!T)Qmd!fSJ=h{w)bqYK#gP)d*@RFz3*DQRwbCZ`eQ@A_tlpMC^ZSmQaOXA|1J+30 zyI))%C-{^f0|w%PVS>@7x>M7L!w9)90~w(@>Yd8dGQTio1Ct1@`C-3mTjl{JrZB^Q zZIa0DxG(ywSS05LAF3eM@kn9gD7}$cMo8EjkHnO^yetaIA-DqmL$sf4(r(UNf-SiZ z$Vsud;p8$><64U$3#$yOg*U&yk=C-{Xbh(d?>uK+Sk;aSrYEHwpOX(~og~jP*uDCD z3eOJYj4|m}&G_@&&H(SDO3hbc*}Aha+HVH;G5IYzac6Pis%7nGw76%A%wKkx$uDLT zOG@Fx!jLd@<|uXyB8wUG&5mcQF2ypk*_Bjqe6cBA76HcHn~Qp1tXneplRoKN!()?$ zX^|9Jnyh3I6$_zt4##WM%h0LF*i`m>p>HwfY4flLfY{TD9L;>#UTR8-W-;q!lwdX} zTRK^z4|kH*GE6LN_eV4YqXo{gkCwTogIaQDx+Xu9wgMMz4WWK>j5 zXdU=NP59VumCAd)fz+WK5h9V{!e-2@-|_fkS+`pUpiXtH2v$Y)yTH71T0*%ze+}O^ zamkFwB93z%VP}j`UkfppT$FWqg2UO_Q>rX2lPa9fcaC}~2su->h@DBQqO$q&0Cg5h zk7~_lUfW{A-QlbF@Y$eQ*+Cs9&r~==MFX0 z$!1OXacxg;6G9QXeVXG6JRI0Wz_CRxP6YmZ(0Y3H%`Pl4*J7w(8{7ElFF47@q&?%$ z48Ri(6*~kV0j0qR>EelKFw`onQ69C$n!RH91kUJKLL)rHm92q z{zk8Sf=F9%!Q*<^!(4gB{ZTqbj?1!r%bpLWaK0KL6}HsZ@EBjIz8I{4A>w11QaZxXA7tLQ)YsID3$A`u(5*XawZicusnNs8Fx7Xu zkXs<3V|OF6fbUsqY14@8_dZ3^z)af&VHx{pv*bL1cy>c+h@+2nOy3UiRl+ruJjrzo z^`bw0ucoWZlk#d{N4L`D);|Qa4OXu{9iwJ{QY4?P?2bE>$Wes%=rD-j~;HFrwAh!Li#7m^Qqy4$_)`^{}{MLnY4S2E*nr(fMyO~qc{+jAI=$nN5rk_9w zELR?H+PL|$>w%>yU}=6XEZ0T`mn2&DK9S1?$D~a?cMKPAlK|uS@$O7#UX`5ycmOLj#G;XL(hI`WDG_e#{Qf*15KK)5SNp zMmmzV6~CsM@k2(f$$QcY9r~INcPz8gP^{;$dd5qe*<9&oat(m}YOZQ{?ws^Ss^d1D zRg*u*7_-0MqfG&|)FORvfw}0d>o29G{_s2STBsrjgMomQzlUlte)l`*8k(4xzt@mz zRFR2XWkPRR<{zx0_h(nw)tgnS|3<2YcpZtpABD`x4A z=NIF-V`>c|tg0<#0<7y@dd=K11z4wCx+OT{XaQQqdefppt{)G64GIM@&~RCFW+Dxu zn-Iy)1!)V~mUnCV_S6osGyL-{`m-pnb1CFXnHkFS^PS+dD3w#l19jlcWXgKWI98N? z+mf(s8S~FbKTH`bdcGFjB3*FfAl}xkrzqshq^tm}6l;?KYT3|Q19}w@BOcEN-NVqJ zJw@SQIwaNKr$S&(8Zdfpkzc4ge>2 z@uL|YoiR0kLJZu?JxGh@|Cv@2@G_z3L1xhgUQ^70;@gI4deQcsz?LOj@pL`vm_go& zul)+-aZukzm2mL~;lq#A`27gGaRZGreV_Pp1VOK)kTG?_%EBCRUW2-h)ojXA@KxTk zkg)~>woi^D2EX9Pt&I;lIwM!$lsURMHBKeqYu^Yy zFJ^LQIo8lvmy>(gtc@fF34cysk2NqUG7z1j7bBv9x&|q%7Jds5<)xj(2k}eq&;4=1 z7MklB^mcoe^^M6TOXoPxZGhF>VFw(ql1&}Ut4Db{)RW6TaKo-qd(A>COV^y#=8~m; ztk+^L^4N+0SohE9AZ*TOX?HvRGNJb}gPxfGR>ABa?ymoycDe@6=2k|=4)4>iO5M^H zcL?K!f&Wc^8;qEGbflfIk-pp9vbz))H(H4ZM$3S}H6M~najWlJ;z`&2`8u%}61QaF z43zNsM7ra_hUfX1jjwy7rfu7J$@taum4ZeMLz}YX=I!onDx)sK;xCpcjyAtoLcNY- zwbAo-L&9{Uo5!D53Zc2{b#fK(>pyMkwP&q3CV%Qj99O>bNMVmSv83Y>TJvdxz8>j) zZ#(tDH+eW;{b0NK%H;dg72?}0u9}YbubK}vx-S@SoiDHW8++`wK0jYZQ+;zYH3Tp1`-WW1& zlA$_^5c3`8M`E?E&RJ8*F3W`v+Y6&#;&j|2SPGvtV<(eCmZcoXBRgd~XRR!ETZMD; zbL9#&He&vTf)@(Y#2O4?)~~y*FSMn5&M2wdyU!cl$39~g&9ax5Pq>%9@co3b!yUII zR?SNrXph8nPk2psj`8+tu$6DM3(#b-sHLrCgTIWsvVGKVAJ3y_7!}qSnLcZ^avLBR zgPMPV-eT!J&T)BQM&1ZMtqz%QqLyp3PjA~VelrK8FRW(sa4h1ziO1VOlyPHwFCk3h zzp!nIA+se5r7nYPd>u)b9i+<|zrP#&l{g!tIhFflta(~%_<7Zeb;8}n-sf;9Kem~X z7pKHMVS<4G!Adt*uTHkLtKI#VP&3UsRXRgQ+YvkdD^ldZiK*5tREgjAB2#IBXt*nT zEtQa0)7POX`0g4)GnHn>;1)l^V77~IXD?g}2zUEWoC6mkX-8y`Az>mk0T@caUdK=3 z5hz6spfU?EQiLO!1qcTyp$gbDpQA}MyXKgUt0bmCR-`o!E3+#oxa|-6g(AL>tJEZ6 zZW1nWBwyFiPbNbhz-b^kQbD9`c-^C+3+5jc*snws7tJg%4dKB1CD94|c#Ku(wjstK zy0539lPtyYT!T0JFZ2N6ActjqXr7p@kJPh_Vc|Zkz;z8w67N3tJZd%D4PalyMQQ5a19IIy7Iu`b#t9age%qx@ALQg)C~5P!o9u#g_m%(bV`gqED;L`GwmgqZ4wkaUZ8 zHjEH}nxOYjnq7mFBErXJUqhjCL*s9Os^erbg$Fv4ixeXOe*%XLcz~?(Y4e>xr8UtU zOF?o|0*m}~0}4YyXjNho5Q--~1n{+BO|B8h151goL8!G(P^$RfZ`Bo{(FTc_Xhj=tN(e@su$Q&&Ndx@uQ%t z=zoIhV0K4~9GTxS+l82aQr}^bGH}3=gjh~3@kHUz+q;Agh3eI0 z(^_nTr1Ml-)mT_Q61=YU)jv+5sNvF+E`?98&oEPE5tlw9QgJiH`eVoiXiAMvNevSV z33Y>r?ge>fDG>t+Txm=WbUB)Z8#H+An2e$fqB40q%JH)7apJN+l+Qpwmf%G*jZRaR zN~x$?y5*Gs=Rd#_S^;IE80KltFX^Gzry6+Fgu_h2|MCKtFl|rrE+^ zP*r%na%T7k&@*PXbiF{C1pX~1cdR=TSM&=*>o0x-yB6G|-L=edJPLRV6=v62R8X!Y z77;0rc-ciXOdOdJG39+f^-!B)O$4nK`T68|>FPP4RY31hG80QqrZ@r_bl5DFX-354 zVb#fy-PSe=Nswc)xQcP8Fis3-Gw2PI zO7%;_V^oirN^3}I`OInnyX4T#Y!DIQP5{SDZ5T9EmiZ}bU%(hlU<-7Wz#sP9Py+z= zjjE+7-@i6lY)xJwXIQ)s71*~9naUucBiUlkyu3GWP+VM^t~Ucw4(0@qY6*^0o+x0^ z{D`)r8rBiFQq#-~K~M&p6_6L4f71ODZ(8k8AMz8dju^Q~EVh#Z`P^p1RE{XzlNx%H zf2kNhg#x&{JiJ2>?86tY+Y;B2SmC_LXve`aP#hENIm=$4gk&NPr?xA?PmKmzghqct zORV>!2KMi%DC@u(^jf-MqC^8`bAI>8F+a-^MJe@eA}B|&F*_ysKAlvHDzKQ@DM(31Cbi_b zj91l3>Pg^BA2rs=M}8a$C~5iTcGcaRhb+F^bmU-8#3R(B$1&msi-VL@AY=D!100B( zFb4k7-TAEG%FRhsStuXw^J0HeS|3FTE}GiN7bW_b?^Latgi0~Yq!g;$7OccgGe?%J z;US~)neehXcB@5_04^$Ny?Smpio7OzD=gwg7gfQzE%BUDK;U^syJo7P;P)7k1>wa& zA_?OVa~_G0mQ*~#Nt#>~_6QpfRFnzV3 z(h*RwdgPeBX3#u=0p)DeBlcPT0SS;fJ4T6W_gm9$w{fyR(3XfINGbGCy&R$?c9I5E zB5`koDi;>Z^0dPxmh#kZqJd9lMOEWu2SJp_-+)^T{>G9-JZIbSx}T8GE8TV~tZAu9 zjjT<_52!piv9+bU5@bZAY{$i(m{EV~_ zs(}={aHIF^)XqT?ABU*|ykfTZbJbO7JsGakbe-K+!S`LcU9SfRef4;DYx1qe;|==E zM=079x!W+iJy~yxpX>Z~T+A+R;(T;l1rya;{T5q;$V(V)tu^-R`3gBOm+=WQ>IrHt zk+OL~O$=!iTS-`pRTE%r&S%%cjaWXph4Dzh4Dp%oV$BMPhwRaXjF9iu$_(m3Etfi` zFY)cdmvv^HwrzFZopek2NB!(M)3koVvQD=V3+EZ8w^pb#QsyDw!uuw_joHZ5D_z~z zM=xv!eO}*JFP2>05FgH5xf5Cg56Je`_Fg<`)3@IWJ-c7DyFo#KfRw?3fMESUM!$)z zgSEbsKNvVcamtR!yE6TNX)=&#)*$`iMlcDmh=JY^CE63Vs5L=Ho@j_aJSv10 zQlF_i(g3nUAhCqhAm@rNq^NzyWLhanXXwDfv7lo|>~ZASaA_d6X^A*Zr&_ zEz(+Ak{lcM_fkNpjgVspk;ixTvzXWrGf%%Lt-<(Ot0MDeE2)Cq$+I4O8ohkdYZk8@ zI^EUBVRi{A)jpVe84G`y?;-TH%E(QzW_z3TQe%5c`iAy0tFi5RE-Yxym_+^|f#-EC zA;`Z$^}ej3ubkEcf&^UOMlx<>%7%FQ({qgyF{*xqBZMu_M<~!US}yW_7xNhDEh=j~ zl*JkLY>UrR$8oeafgr_P#De*S`hqjHgBn44X^OzN(iZ7TYh1MzaMMirSoW-gAQiE- zr|ont^^O=<=+adn-WPN39oX<%7B3GYQ!Zs3{X50yaizA~9!V%4^9rRp4m9z3Y0gSVwbDHh$W+h7U42%s$;OU`XHq)8Foufkb-hFlT2xxJ)|>3_ z*D$^6-vu~2Bv>;bl}S^Bw(q~naD7^L#=n@{F!<#p*L0ugx`e->#!KM7inNKHIvsjm zqV1EG|2g~dx!BD$M%y&>Xmxt2XE(TUIv~{vDoaJfly{+c0H^G=;~`OsZ=urH)s(gi z+;U9Xcko;j-!oGOVTzRfyQFnyJWI`9kNaUXA|*GGjJF~lfj18`#qLko&^E@BPA3>U zFgeNV1B4!3kr>W^$a^H=&{!wjX3kQfxx7%fxp{!k%Sr!hOu!oJ4-#jXTC3tCJK~Bw z$;Fn@8=h*oS?Mk1ww9~VKu=NAvd$Q%gaWNjmB1;?i!QH5M!D!ouoAu3)E`X8TJi$= zYTpawW?O7lDVv^K;ph`=gCji;V;=3(FD&<7klA*Auszox&Y5eI-%jWGo~8VBW`5b| z`g7&h`0g<16f`Rs;KZ}r zvD2{??pdzwZP4tv7=iI?cW!!*@@meL3XyMAcV+>JGVn)UjZ|Ck)xim>-5UdprEOkY z4VyImIu1>2B*6H@ z$4UhcmSlzJNNVK48i0lqJ9C2^z1_=7Y2K0%k-;xby+`+0!Qul+DP05{ot!zSuAtU4 zFHqb!mUaFGC0Bkc?;BLr9F^F_8FqCT?L+yJ5kAFPoQMybuaV|Re~653WOB0bUd;5BK2D*$WV_Zu_&31X7c; zQCtU@?x-fej&zTXcV+{?n4XrM+4@k~djOz-Lzk1Kw?I_cGubL!tClx*8jkD1SXzv^ z<=psPt`?(pJ?0+t#5lsOt~FU_cVOZx)xJ=+_T(2ZCgmU|6mc)U~nrJ2L* z`tJu)fP{x?uzloOOrKX{dV@P+SUymx?xNP3YdRH%FT)~eR*&aUBf3>5dm%$I>N$hd z@5hZk<@K9b>d}q{&>~dQTsE;tR^pIe?SRiNJ^B+|ZWeKYyeZOFe-ei@P)h}yM%MW@ z0#nV3tfIpfT*`(`1g1R0PwmpHmmaT?3wK@{&{_H(WHvzsU2!X-k)b#q=aD-pxtkXBQ4#uNc)*jCG4_V(gnfNHviy z`X&nBdwro~w%(5+eWD{IE9KukkU1A~XA6_KJZ8;uw&DAB%=-BaZ%nyraOm?z@>#UX zirEBJU0WtUd-%~|H(`$68HqTULn`9KBzuMFnAUDOw5u8EYVnbN_&7+!-fYZ)tfX&a zEI7bkv~)>xctZy;Vf@(Dw+Lwenk}aScH>W zH9(Fv-Q?wYg_rn>@b7nAYe|84+axVF6`<udbI#GZQHS+MBdzq8~x3 zc)W!d%ZTLmOdYljl39X=9(T}gsX9fel}ClbtBxG-=nXq&LhzK)o3aj)FMuP+Q7{*ZoHInl@4SQQqj1g{qi*2hOB|;E}jrr)kH=oI|PtG zr-#N%$Q_Esi+3Ym4NFLZt64{PJ@tR0w5A=RZ2rKwrZZR2BNWLyN5<*3`T6uNmfL03 zfJinVAe@^^sUn(e|Lf%uH*{7XIhh?AP8)xh`MZO69gFfK@e&tI^#jp(DueizkLHZxByxRNTHEFm>ahdUH1Sxu&iLuFgMW#8{ zt$hc%aVdI9Iv``6qPQfGjv<^8np$y+X^NF?fpub$D~5)Q8rT66JnwmzD$Gk^)?nxUe;L0-#$6*bR@OO`U+Tec|m~?QHm0% zHCMA57J{wi6It#w6&^Z6n>@tU2GmHN%8pcC+9$_s$b`g_waE-;V?~+ttX?La)SA5} z3})T<@dtQ?TuZV=I;|B;fIqY<`RU)mP1$R4~SPcv&c)CvkCl6F6FhAkIt zR^#9ZXgenD>7k2UzZF1ED=SN^_$qO0=!4mm(q%uxfYJqRdre2?kZyp>76$(4;Ax3A z{+!?22Lj#QH3xs|%z?PcUxz&#(L|{@$Lq8_JgRV}&XH+PDZOV+56qn4X$gZt11o?a zXm9+HR&X#$XlDSq|ALJFg)?(OX|hQB-2IzP`>eT3Znf5Sxr?i_n_r5F zolv-ow^9SbbU1T&BI9L2YhVUs?gJvb=ZJU(-i3M~(g_M5<YtdP^u z9$c?@cP--$T7!s0!vH1U_jzN54;89XWr`{O?6N282FW34>437kzP7w1jHv$CJqiA< zts2;%A;wJqW&V7gai}J}UpSkZtBRRXLmes9vD1P$D?%Sr=Jtc)A#MW*_*@l=95t;6 zji1X~&hAoZtoU}Qikr3Nh`7aL4-^O!umQIdt54y(YqF|ihHH^vXy+5fLHf2`?^7KV z3?1bE-?Qm`zW?{D$^Va!e*_}_+QI2}`QMu*{X-fAq{)Biuk!!XI_aSJdBawfYl<{^@^Ezi+(yE9CDIkpF}T@%|6wpTn_p|6Tg`Fyf!m zZL+^f|0}5YSH$0ig#JXZDEuAbF9D*zqW(A`zxy}-MAd5kj{4Kv@mI>lY;C*(gZ`rnbi8!3Mx7vDA5|Gql=m&x)w^LO?4Cv)2PKQaGX8U7Xc zce?6Np!2(m{oe=qZ<_0`;J;H@{{lCF4*f^)KNOg}6eQFiQi$(wtoJ*w&is#W{{_O< BV(S0^ diff --git a/tools/python_snappy-0.5.4-cp37-cp37m-win_amd64.whl b/tools/python_snappy-0.5.4-cp37-cp37m-win_amd64.whl deleted file mode 100644 index f69eae28cc96f1ac959506b13c94b5a69e560223..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 30053 zcmZ6yV{k4^xC9v6wr$(Cb7I@JlQ*{Qgjp9r&>uC z6buar2nY)3i26uQ-F`zW{lA?ZFc1*Rf3u6dk)xv*qk(~yy_Kth0fVELUA!>DAQ6)I zT9PAVS{4>#VP5~4doW01iR(BVowv`zMp{%;*6+jbQ9LOeO#gAp1PB!&iHgd{?7w4E zqh%GT{he9RSDsO$M!V8~d`x1E)1QsFI#pY1QK(w@dx+X)41}IvC?%n}=DSjLkKe2d z5q!I@r!y(7fBUx=e=6XO0VmMvCuRAI)f7HU-4Be%o5L3kDu>`o{~s7R%Q@3tpg=&A z(EmRec1BkA|0j&nKk_zPOh`SD4}<{OdVCIqpw@73Rgq*NBJx7FvqH4Oe-GNjX}Z!1 zjewumIJyag?D?>p5$(C9Y);m!L|JMafYbW9f=UdsjW#hmiHFQ`?%#huyStmaj~E?R zk!r4<1H<)dI%Wx_`*M7#2PJlG?P!g}+QdZ(I^G4~o^SMX;!84>%N;2qIKAIJ!D&Y%$A5-wd*^eXcA)V&lMxlifV~F&W!R&Dc zeP)M~%$%b)=#iXUmp$JWl*KRI@60h}Fk_0B!E&B#KC@EGNun(2iw7D-2J|%2)FAX9 zqroD<;h&0@im`vXPk`j=yfvNS1n4Rmi@^YlR?ak>dZHb@=kyCwf0R;XSFI^LaeUpM zg++n~G^S3Y$GeRYm&s{fW;s+`>E9X^P*u>{gIFamyra)m^*vxX_6g9XKdMZUFsvwe zc`VL2Q7BzVnD{1#R^;jwL>cX@iM9PJsTH7=K*O`P5RoxFWg_oFZC3>2E6nd-o%DsTeJG&J5e zuV-eS*RH;%2m0tkGR7}T&3fS0zG!c^nyv4?c%e{jBJ?p(<4x~3Sq42PF0`V!`_*v# zvwD{{)BcJQ>h{OM?TE!vS>jA~G5HhQwf0~tL#B6!wz3K+gTXI`92mMM{S;yGJok|c z_R_4w#gv|0Dlm$J(&@j{BQv);{xSiVTMBiH#*6Hro}uYy$P~+5{;7$NzwL~bxw4f9 zY~it%@_0Y|91V>b)iaN@F9J^4SV|)4w_tA4$%+*}9zMDNz@o>l}+O6Nb6VxwLpXf zXr2MCI%qT$O8p_24%08-|1H#dYIg9rrUuByiB8@x18wHKALV}&9-x)|~RxHubf5hHuEvXfOyBLGM_EknGa z1wFQTcunq_XV&$7e%6&|X`^8&PJPn~^k495Yq?&1cC*_Ck={H5L(56PtREv15)bR{ zZT9Z#=kM>Me{W7#j?YcUY2N<~{CQq&S7a(85pec>`3az6?QBPc1~Gty4CZj-aoBx_ zIDR7LZf=ggEn|zC)(aLLWpW#u%F0Uj`VsThlWJpnb=wSTiIP9-u4Mo}A#Ug*5aDwH~^? zB{N=bkWqzDPMNe9gtAD34VRsu7Tz%meCN?57H0k$N9x!BK>NEP6NM)Z*y2_wkajda zeHmHm?^6SgeM7rbrCp@yg_z5>Mn^iwGOAWfCE{p%9j! zqK(pUgWvfYAiw{88M(IyQC(c8p*+IU%__~gq)mGqcX_CPKs!zQ`bXv@hBxT-c%5is z#CpmzU283{3i(Su1oYNAaR2WAb{`=a;P`d=xnHdC5vCjBuov(0rR+DUJnJTHq`2tQ zeA-dXalHK;Sc_Lo>@5Ir33psP%z1(;`q#T)(XvkO9p9Th#nYRJe)0U}8@jVH?0EAI zgFtj}R^TrRiPQ}R#yJW^Vn~&M6&OS6X_Lm**x?#=<_KVd;|xzTl_7~8F8x;y38 zpK|=-W+?3-*O(y@4z%tAN}lfr%{dAO>t_q}pLJ9Am#n`1yo6rAQMVVQ6OOD6a^f$p z75{eZtCm>D-Xgc2-ILy$Hu7%LF2)O45$iCRe9z8LqPgr1&dKXjF5J4c`QjP=#X0@R zT;~cs#ANFKkNX>AmzLy6O|h~sc+FZ;9>sC%hgI9?@Ux z4^;zWp3ug2p=bOL8vR=_kM2Ea5@G~OL#M2j?Ym;S@9FUZROb4wKR;df21v8VN`0fB zxCB!AnDb;$uoMQ3;tp+u_-N5@cn>>x`kX(fxJ2S@;Z7k&|2yk?HTsB~>^IAKlQiYa zMko*T!9%|6Of~Sh_Sn~yU|hOmw@SSiD}9RdnC6w!DYkc<&x)1OZtnQ8t;sjp=<8k# z1%j=_&ie=6PbOxEb$8CJ#eV+yo|DB(D;D?EklQiedAaO;{TA~d!nj=R6Q48sNkl-j zOP%tYz09z*GvE-+%{~MM6tQXWeh|&6o5>7<;BWXLd;uEvn6JYgpc4VN#b$L6Wq+$` zWu`TY9SNo9f4}Y~g-=PDG6fU;_u z43KP2>UO%)ha1d5kUpwBS*t@&t6Mf%_-=^6FsIXgHc-V^pdI);Dy0)~rWyHG(Hv87 zBB0Xh`dE&T<_cQLDOL!7a0QFnhmmq{?Ox>5MWW}2uxq-bGdkzPc;}x#3LBQ=)|*VY zVa{TV{F5EjoVb(_o~26|;Ys_uOzkdcy?TeCR-z5AKVqk-W~dp3raT^1E5}UoS-}ET z{Ep9)@l#Ry-DwGrv&6b4q^jP93%7!|aNN7o>;y(UW8>w>w{%D`9hSWBs(LbpBN~6n zN@g#yqRvUALM-tNVE!h%d>?rLK3e-B|vFrN+Gu1o> znWxO@vH%mnWJ_7H$OB zs_=vGc5BV3bU_YrJM7Jw;oW~r2dTJ?T6ijZT2ZOi2WK1c195J!&-5-Znec90x~$;L zM5AcDtUi}JxRtZgKRjQ8JB(`(NSvNf7|4>{I54^HCMxM3SE4o{i*bV9&BO{Xa3}jY z1LK1gZICVt20q_2OSvw45A@8Je0p^6^Wq>>T>G1C(;y`=(twPt&33Rviq72E2$*q| zqB-mt2%?F|pX#W&6kC7_*Zj5{2%zf<(sa!x3hf(WHGVQQZ$#oOkY;=8PFI_caqBA- z2#p|O4HY(F3<;Td4XC^Q|CM366~>RExuZ4S02G8h#ae8F>> zV)JfK7>80;avfI|}U=WfsA*cM(EnHD3+A%A2RB+Npw(Q=*Da`jompgmk0 z<}Ny-_X*i;UVeQN(d;VJr}n7#LD%w7!8SX;lh4W#@xWiOC>U;=utZUr!U-0?OeTlc z;xh-q^jQmfzPtH{Zyl%eu=4{X!Jp5i!qRrlVdTPn{>T^#EOs7%wEL&>v-XQ?SdNBk z!d`+=ltO59xXTY@o$PR%p5a}*M}I-qdkqyiP5Qx>x|$5dp4Je7k4;>4zgpjI94Y>H z?NzHYLf}3_b=t7deKh&zI*6fD5$Ozyv-ug3X7->@=N~kWdyYB13yI4V0DLiYUsmiV zuUzn`Uom-}op%e2>SPO}2Avn;I0BNAG^f0G}!2B z@m^6me4)Aoozs!fCU~9JaWZ1R$#=`_OlN8n&`=aAT?{Hfn2rTVIN*jaoHZLfUutVX z;a#0!tN8u&UN>QXZ9wGisJJE+n>veO`jKu{y=WE;1J2OdtDOC=hANFMgO9G+n1cEc zYMbV}(rJEr_uwJi);|u%vNM+8G6!d2ZPeKp_3nig-HGf}t5OIAIp-4s{K^Vux4|)` zh|im1RW`)V961D?MH^VBUUEm;s=pjJyRPu}pWzcN*vIWG7T~7=oBt{Q6Z~U6WFa2r z_R$~bj&}I_fR_g`9SArDN6FVN@WG^M7+Jmz!L{sxasQ*Za+^f6 z!lXUikqf%|mUWAYzY_8ePXv~r$nR-6{&DFe-}@q0pum3f=Egx{+= z!C|Z$0U~?MF&3XLy>KXdp9QBQ#NWGc3MeyXrrJ605Y}v}-z^(b3hQ#6H?!Ur^jREqW4u3g)Bf%tN^nBUE1Rh zB?rzy7mf_8Q7<9U!n3-P&+civcmT?VXtM`nwU8ijJ@Li6Z2Ns6|Mio6ku6Vnhkp{6 zNk!u68>jH;S7qCxQ}_Et>RvRqxO%k^CrQ17JI(@j0ZYn~a8NjoT{9HxLI;$W7K2jy zr?bciG|Cva?6k+<{WRINOy@p66r44DFK7$Z%uQX!U+C$ATQ{bc-n)4M=;8O;aVrYE zvmDX=yzPaq7b<#ryVf?m?_JeNo*0{U2I&-!Uu-+?)_yCM?QjKl$1X@-Pe2-X;-{cd z9;V~fyaxPORV@aRvr@=%mM;FIiZ-`KRX*?fq#4`(df-St89C|S#aUYH`m{;ScM-OI zP?BK@3t>IP1Gucn*gV)fw+;{wP#(W5^%=M}}d!{v|a z!#K~MFe=eM#q=NwFK2x|cAoW$7d-HjKzY#UotGT!9iXbn1xJNIqUEz=uj4t}PLm$m zQ9DTE49=h7vrISk>fV|4?i%c{TrKOU1%NBA;{-dw{TN_HCiol#pa~Xbs|vlB@o1o$ z^;9nOhU^Li#zVfja(2N1rZ)fM)Wa>9t|y6#`hqvk9XWxB1?^p%`h*}CMg6uHA>8?A zyYz#NxyzTT*3u#c!INUi&F~GgW%|I_`8paobdG-RAo%$uVH;|dTMDSwp~Gb8S`n1R zVX$a{QAe51T6KyHUDDIqA^QYm?Q;J5XdLP%)QdZ*i%*`CHgHP`VowhV^lFH}_ja(a z%1z3U8~v1ma3e7^_|pY9T0+}Vw(#5?L9H|SLyc|TOlv1JDgFKZl3tB-tqWy0+_0cl z0*S6Gos8h;cWDci(A za?om-a=yUu;H~PL8B~@ys#hS34!)0GmLu2>6@m}GtvGm*f&o0s0=_5loSsT?{XUrl zrD)$x8salpcGBzjJKjluQl#W&r`Y@kT{7GBcgw#6)8(S40G=f;FM(#q78>m%?u7p1 z{WKfm((i^$H#oPJM*4@ygw&;|67|8+>`2KH80%x~0bD`%82g5V@w?fiF<=5)<(Z%1 zDOBaKcj~vUoS)i~zP9<@l_wAQZ~BtHktYw<@1E$N*2P`Xr?%i90qFpbg%|C@{1}K}aPO2uAYT>Ytsltn4hfUav4*Hv z*=Hc`6c(Ey=L^{an+8b~|HR-}VDS}el)l8*i642TNj2}4n>TZOpiO>{xpiL!ZNb&; ztWZ1Ut?FAYJ=FNGRGKb0K*&E-H70|<5@@zA3iuq%)KjXL|3cv|DqP@BD%K*Igms3d z+;%etT}N^?5#;XHeb7+%dzYyNTf2d|cS4`Vbh#N#q7$Lz8jOX&R;xNwY^MW-hYTQs za*giIeq!9f{-?suD2_TG%}-WO&78UugW$Ef4vKBEsiGq0J$f{U0EymITD~k>DD*%bSvl2UCqJ+ZT?m;HND;92M4TIg7pimNZBbJ#FCCnJ-88J|&m!;a zB#Ya3EOG*bB`%u*d;~i!QybhvusPuDacVEtj#JHQFTo`}LiUL1tY- zrGB9Wd4Y-YGa%t%fI!TE;P0c+pQ{VmUr&DeItBgLEbuc2!-Na(x~&q*0Wy)hMmn*! z5-d_zq?iqNA@W_N_$%H1MPj^A%v6N-pJ!=B;AN{l zcW&OoGoex4KTFe3t`Ia)(p0V%9anGQX4cIVt#5>pxuf+3#Jl~5Sa$a!<^X0->UD{ZJ!I?9gb%FaP#4g36A$zZT=*;qdj84`z{uulm8;+6W8U9 zN<*S*=RbvdbtX&yFTdX}q%UU9=Uuh8FE2>`@I`r;IY!~Un#qCaaRIYaJMzw6#W zRFTI(n6PTv!uQI8WMN+@W#61$?aQQcS#UUfKuZF}Gi_|YHrj$C!JDC9?w|6$Z=Q~W z(%e7~W-&>$lZKn^kth94`8}RM8ko@%zB=z-TqjX|@#sNJJ-^Pyqn_${nv7m#a*oc$ z<}-JUNs~8%6I|iW3LSN)jS}M61yh#PYp~bcQBn}>gfTdVQ@eQFJ1_hNj>dCTwS{k$ z{f9%$ZCrqnZq}lJDKv%%t*Ha&eB^NtW`9KEw~{GD2}L zc^SGwn}3f_YVc^|h7o)o1gGrn?D>2b=KIr4>9&#@FHXk_3nvZkdqzmH)xE7;S z_T)~v3#0Yg3gZV94UV^N>-KNflsSABP{vUn^qb2X4G*wo2|tx+^F(iqV#SwiM2PSc znLS1dkJ%>~n{`Oz=)8aZe|qXGU9ebRlRvh@khA?0%$Ku}c*KVZicwQvjc`CTxzFe< zQJr{Y?2EvUO(oM(iSSX}53oVDzZdm^b5s|dRk_t0ZvJCsr?)Gq!x{o;>6Fx5G*8i> z*go`~AyoA`j((64^&$tKqD4J2gG)qaHWOp{vGvy9|3taA|o4=XX^sv973(^1s1fzb38(GvqB@P zgP%aaW?i=is;z?pljd+qyG8yFtS}EQeqzTO%;jG(M12o|U!7qpjeG*e7sa~lRJED- zY8^^c-{gO_ApBK(-80>}Jr(mTmRm5b*y=HI<1vtcHo#`T%R6JeNu9Uhi-az%ncf&J zamMaX;l3y&x>n-#{)L~c{?5qc!+WlRo^>nU73maNUtZShPPMUp3Vmd+pV-InMj!My$*@jST+=@5Dh(F!5^p+*!`Lzwe;) zI{aKmAK%uvd*$c)&bv@MImuL`VuTxk|7q4D?S>z`Eq$aYwpY6=WG?lNqo!K#W&3fG z^gi)2c){BZO0UjS{5#x)xJzBqADquFnA0k|R?c}6wZBy4!fkb6CMGo{IDl+TD#y92{Pn^k&rwOCo^bS>Ft%pMm+DG%aE@N8-eIu;x^( zVE#gfDQ*JFX}5Ddq1&_5sE9aZK^d;R!l(P|H1-ag_1?*$NM0{bQ(BC!r|{kdF|M$Lh}A18&Ah_IK*@xy`j!4Lqa6-(G=o8~ra4b18e zw}S81-zL(|8xTdY=l(UGuqp4Q%*Zj{lP4u`?!n|#UYZzsnkZs{dMCDWM8`x;!{rsN^4Eo$l@u6&PP z%hBGU%3_y+(A=N1btwMdv+;!k)=TkXRsE@pKPRq3g`zNFH*PTh{V>0vG`}Vw&_|R^_KFD2etJjHlHVJ+d7mLRqB%X zoc*#(4-WgrvXhT$`Ad0EZlQ;t1s(lX+?u}h0Syb38of}phgv_rwaux?t>;ZU>N|dj z_v}M67dQnWtQ|;}^YHUT{2E!#mCO9rg=vVnrKGcEE!3yad7I>U;Qnjs=2MOi zk`b-?AGv?D@Qs=Lj1%(0dg!WQTeG>d$E@E?+;(`wU*ABVjeaC%cNaaCl~1>wcZ$EE zaW3CcbIwsSkTrFM#!ttH_P4pHy}uq4+n z9nR$IGh1g*AIm1qnH3;j_^}~5Q-FWM6dM*Sd(NT6xguER3{x06soiQ~(@Q!QN%fMS zwdh~S{>A(3{Hh7*^i8sg;f5U3}7E zJcJ#SSzDU#Y`u@l8*8H|y+F7vBi864j%5U(u*GtyuT8SQTP%^_aGW6wXL(7uk=9~6|v}4B&TO!~=dvH)H1zE{;DPWn*qiZy= z0RIyReRN`7+(}xZu0x1zSSnlU@FlmO=;ipkB9kq~LuBQ*y7uChX69e%`c{e1&QU<% zkaqI7dnT8!>e2GMw8K7gp)p1&CD~$ttLO&{ei62SNeR0Jt68E$54PEbo=4d)T^ekq zBpHYXa$_~~x9RSN$WO+O-H>SzNQ;B<(yvQ7-5EPo;Zo~Nas~(38eONeNkHyWaI0A9 zv^h_=rCh~(>0Ce6n<5yBaj+A?VlYcjuO4RF(?zLvOg}5jGn0hMfBm+fZZv?S>M(Te z`Qgnzt`Dud&yQ*Er$w|4f{6yP`eA%O5na*9wIG8U_F*^7{W(0AQ+ z3TqTUegs&lK;HeiY+%f5*YsOeF1;kKtVU^u?eMKyeXc;FXjy;KqP%K*MZ{@+fq+;1 z=WtEPW_t~flUF}?`TwlquTj}-M+%JK^5ljFPR_R9GAMiIV0R#X0QR5AV`>cba(Tn& zbU(tnYx+u}OJ}g9ITp!a>67I#r1hF%D&O6sKNs@X1r|aHwJAuu92ZrS<@aWw z4VgRrXirc*_R!F1fwChVqQR@O2?j6JF%ob}9Nu}Kt zze12dL|6)%fnn8uW4U>LLUO%To8kl~Bu8v4sOP^pr=FI-^1))@^NBa8`01R==ai@n z>`PWD4Zq_(n<=f{QT=vKXGgxzE&tLDU|%~de)c3l@@-a72=KallsUR68lQ7KvcP%( zrF%jI-r0);4cklcr-yvo(HLZ$2pN^Jlv<%*c#U`e0rhHG5=7z({}Lw-z&*Ppg*{`Q zhO!sv*J4IeT(76lOz5ASEw|E9S&Vcs{s_JSH$60>u*HstUtk|O7c${g5v za~%X~7W6*oamu0p^vN1$`$|J-dqp^Q`|6wf7!$P;5_X^7Ud@?O`wEcX6ym&gueo$_ zTIyX&a8o(wLGr}G>UL?1?z*LkdYEUYV(y{J3^d-IZ_{4U$qihg^{zaJ*onjSR3Nb3 zb7}bs|ImVVUkO4Y=~hAcswUJ!g~l6d^TDS<@J+SRhbZt;PU?FK>u0>iq}t@u&yEbZ z8zp=dPZn^9G_F<|8&ZA%P1qC%4G{Yz4JsaR8dIXLQ$EU!X@tFoICq*|w#Ev;YYoMH zw zdD3(GdOJRab|K<(5s2uxMb0O>8#QEDH-L)&P61))`o~|3d+ZebibBRlGJd%@>g7!lVT!dLXctkoMsvxOweUslE?SK_}KeQ^%Rzn`_&e}LmqjZ&cI zG+^)ekvSsow2*Jm3Pk~-B)UX{uE-3?=jxG6OB-#!lp|kLB(~^A;ZTfGj^p0I6p;z2 z``X##;WdogC=EZt$l7bKHGK-hi0J!T4ZZ4sxu`p3cvOx>`g+LsY=6k&ap`os}1{>eU^irmZ}5D?x%i_PAhu?d1Ua%JOm!bm;NX zL4*FL^(c-8^^;iJ(2mOKAP3RpO6yhICiPd#kWy<=<n6RrS<|{0?+9V@j+VL_oLQW8>v;2y zRYT^wbs6Y~`85JA(frRSYLcqj$iW)2oH#m#g?Qo+96YG&Ghz2 zhJtKv8jm_&K%C|*MkHLlg6#~?<+%0s@E^@6JMmWe3bej-*uw9|JYU98A^}cq zykpSCi(~bI;+lH2b!8f1Fxr{8hl?a>O>H_UWD5LH$%-9_W|hd1Uc6JrvNBmGU2uwQ zYSL}7vD*udl za#&{;^S@>C#@FVk)`Yz5{BAVaO=K3%65a|021W&9Ko!8fuE4pdQ$CW^s}aZE!uUSn z#gaYC&(^krohK-uD*%F+AN-fwGeS2eto`6%c=K$02t6goSonm_)!aVAnUAN>+ZfTY zua7qR;)7ZFSHF(WkJ_Z-+wPu+e5_~n_b3O0UjDuV zzt^g$g|^;lhc{LYm+z@)|H-cVWP|V6Z?2CX)CrAS>m1LqHSAbu)Lz?<8e#S!78=)& z_1N99UX6z`;;9tcKF>zOg(Y=)~Iw}`J3i&MldGv|j)UK0z#&cmOPoGz(G>oE(PUrkqyA^edC&KtkC zF-Zcy@pZlDhqKL=I)PcfzH*hNe!t|#qe%SxofV|sp~QgqkVf?EJfHE}&Bw6kXVRut z#ow9XuGf5rI<7=)^&Zdj70DN=z8&^f-{}cN6wJ15-$T?3LA^Eykw%xFhHec)|IZo^ zna0L8hiTvA%}(XbT3;@Shs&edNBSne%Y3WUN*T$-@Z!Gr9pB42%-)PbI{&@rA`%-_aN`$dASHH0so^R?)>3f9~)zhv9XEcuavB6!nSSyJ?6`v=pDB0rH>;&vpKah znZJOZ_QzuMKA!@^*#}H{RWwG|BmGsl*vskl4znZG-QVZyGEV;_z4~vBimIWXpM^VG z#cjg2(vJGWJjknjT1I>IYFPt}^R9QqpTvDe-F5Zuf^Uz@L%|%TgR2w|`*t>~k$>Z9 zF||wKuPX`)Q&!p8`+n#Dk^}UbPhVLz4-eL=xn(h8d~f#J4^Kq95Whc0$t~JW@pn51 z9-hCYZK@dj?%qrS_#Xbb$!p=-tNLDco8t7alaQKc`y8i|a;zA{<#nmDa+0}Oz5Lm} zdHwOPd)!`R^@#U73xynm=hegEx!3&r6(7LmeUk=THke(Ywj zi}kUxf5|^X-}h^5J^Ju{xVZD(em!&=Hnp+D`O8(|w76R~ZNSC%FzefMaO%aV7g^yy zd7HS6vi81O7gqQEeKHsDDC#yAF-GX~v6M`iPb0wO{_NNL6im!;3yb^Y#Q4K>6P=^9 ziI~+1s80$(=1iYq@csxQRYkn&-cQ4|SKNO0oY~2zxzT!jO7jtX@s{9Aeen{Rto&EE zNh;c_rT&jT|CQ41Y%2CPx9_}nZ5F?a?{WJ^Van&lE>v_g!nT*^5PdPNYFD ze@>UxO~~QzNpHk3pT_pWm0Rc*!68!NN>Nd!DyJ zN#C78ANOs~x2cvshm5burXv}Sl(&F-hTo?w2OGv*WH-d~P;!9Q`BJl$u{2ZZy` z0pj85VqdA$WHA7EVrOwE)Ik~Pa*;_XEo^AjaV%_FN|?}`#HK}bw~t4Kn6km<=0G){t=0p<>tC>ducQBNvuc_d{{#v26?k298ND2^m?W@F9j zi33RLkLixZ5hEv>WH8gol8|bJnT?@nEo++7@y3l8k1`+ryZ_JD<73TJPp+q8FE&Sw zB%w)^Hm#%FS6*XXYeJP0TLv=+$rd9iPRbzT$Q&<9&NOu=Nt9GHrD&ukW8(>xHlj?d zKa-v&b7xT&Q#7Y&svzUzNtHG$ONcn5o<`${6_Y4qwz^UyiO!sqGh2$}qsW!EV3o3z zgGe4wAT?rM%CNd}EsDsTm9wx&xXZwvhI6NgjL4jrHf5DIm7_?Wm?dF3rZdOj%^pu} z%3Qa_fL1qW*1Ez&`I|E|ZA1bHUTSUEojeK&Cs(1IV|Gmzp?6<#21M5`}1rEf)I@&jG zj};F2;?#}W2%5vLi53_1yC*tL2tXAK(Hq#-Y6IB?f0BEQezQ%0ORzR14__c2WHw+SfR&7D6n2IjYc_ukJkgz$lrkx}XASX{M)0zQFKTEa)MpBS7y2|tZoyvD9q=5deOO@v(&t_zEa1zS4FdKY zd3XTcApVsWev=koiz^fifV;DxD{8&UipCc6ANBX+gkvGZ{CvbpF~~@E=yzCdF^k%* zQPo&v62+ywf(~MF)5bXuYsB=PwD#@|Nw*%jovkP{;jK4%O7^I*vihsS)zBtN&?lal z)kJ2{!y*~Yh3_|IKfq#;&%!@)d7~q7&+tOcs>0V;qKG}B`b0fd<~fa$$XOV3(4ce@ z8_aUA<7>G)D+TJs&i}TbSI=EicGbFRPfI;&2pFY%F>=XR_NKoMxm0zBy&l5SO=H}v zxl*OrLl?pR;dZi)?Uy~#-pBP5(U%uA!+L8qs?_QqaKJn0(9a2RrQxKCzNMckcql(( z)mS>i(?EHst}e5MNHwGXXElu4#~s1Z#gqW))EpEy5-!rK>B>9eKm*asAF+!{S8~|6 zDR;q!BSUSA)Jsg2R+=2}@E;c>%s(YG13a;z`wVu7>EWb(ybABICmKx9O|M>Iig@r8 zuq5e}k629&=ooq z2lD;__v0RntR#ym{~L;;?rkgST~xp7Hw2}{oHPF%v_?q$Euy547tZ_#TPB4ZrU>vd z`6Oji@QvP&4~&ZO7xT_SR2wrTeZ1c#Us38ULW#vVD=DTP{xYHofqwX9d=o#=MuEt* zTR*V5kDOQn`kJx2Oi;QV00?1_MPn(dpQxr%lm?q5PZ28#*R~S0v?($p^Va5yR6t}8 zGg;>n0rBJSM{&l7IydhDA}2;cF-H4r2JYbkj|^cRJq}OgD!@&j)g-6k9=}3TmPSZW z8pbao5`Eau!;lDri#(5Ll)!*E!pnFR@O>s6qodAbrw(=HMA z!h+$0112z*-hv+G`GYKiX3;M!M^WNWa%Pgrv^L-s33I|30CtxJ)IFM*VIRa=XvfE_ zKypO1WoNv($CpY!txVV(vItFswe}tSQ>6VgH#RyExJFgfZeZR+VA|u*VmM?Yj^7Ts z$EymCO&*9zNE}&u*aBghi)2|ygen0@7=L8Rk}OV4z?Y_w$0f#w4(}_!i9it7DNmmj z7^g$!aR~3VSB3h%9`wqkeu^}39%SK?On5_Se;ktjVgrvzIR@lg5D>X5jvJBQzrrje2oHLP+79A5ayk6WutV!) z5YmNo12O;0yGopf#jL=t#cHmwV_88%RY{l0Ouy1u0l?xsucC_!7Lv<^_5NF23z12O z@zUMj0!s<^)FV)CJK;Ug$#)6fA|H(ZOOoOSw!aNzb}hkj_MCqj2=dkU&AsQPM6(@M z6ct=5jywH+%;BO8%^hUMp);V;WKwa5bb%h}lw8gFtf1JJ9nwGUrU^j;JgyARUVMj zpLL9pJ*1`9Pz{qNIPW9TTV2^*URg4zyU02Vo(2<+?OHE_(8OlxCISUVGiKG4S%oK| z(*k|?m-p!1T&J`YMgoV5q(!AiZ(3~=BhWKQqj<(P+?-6Z^`BL%9ZQy7Dc6`f1y*Pi zOhtTqT4ygn7gLQ!3JgY?GL2!|)FXrTkv`SD2A_+~Bq}EHNc-vyPECjdN)gN@iiYND zm`psxg1RzrLdUxPA3h$8C#qw%4TF1HYNj191Lk?!RynI_2`o!IF*rCkJ3Fkcc&d3^ z)F=dHFm_2l3@Gcqi7vP?+P@m}rAO5hfo4Uxg295TcD%NK3cwLwpWqdPD&WNw>S+hWrn~JPkm{c}; zhh%=8e(d({bf@wwf5hz)zTp`6w+RP8__b5p8hriiZ?2XaZ}}tvfj;9Kb0;&7)+K+g zHW|N93SUf(JU}2fO8Z^*7{3PzeM0+PPBaGG@Mi>Y;N`By=ZOHa{GA;C92H-G^d8UZ zfyikuw(+^;xqHRSFIYlUGcwRn7f6VLVn8`U6fc=f{Qwh`hTqPj1xnK(>9a$#VM7sm z7|;ADdsJJ8BmrW0^f$3vxg`eL(JXXuk64AouM9pQeDi zP{DYha3zKEmHe^xT)8&!snBP0bR?ooz&y3(4zXQ6s1hLq`+`52_HM!25pP;mkfY97 zfjr_Muv1{}-WNdT@PPtQmNbAA`V^!rK&p|JpxUC&uYvNBm!NP*o{R!7m;uA7&`bd_ zN(zM;5^q~rL8q8NJ41umC31n0CP9+@(jb;tv~jR|63lvtcf(xekY64^EQg`@By15@ zr2URuOXg|N7MsOuqQ7R#vx#rejyLqht?|_%UEMJoCc(P{mDI(p*|u_F_o>gZ>criI zFF6*IU|bF;XVO}`8EAv4$-;`0V4lD?v}SLFT}TfW?ZtN;w`g3a1Ne|uUnkqd?OIYk zz&|{(3DODnK%Qf2cOpbRsC#XWxOeMJIlN4rx}fK5fumOam?f~l_Sp!&Bm(_)tk7OC z_ZG;;ql&%hud$C{V}VHCO+mHdfs*^HnqV1hm>=~|v9CnF1olW34#z4)i&C z#M=;et?z)ZS&@8pR6B&Tf>42QiWx`3)_IFbQ~oWxe;9_r2q*Z9bp74n^$0|IP>!Tq zru5C=XIfA`>)OOOp4Np0vV_1n69Zp0=*BjT2QDcsPunUF*juA4#0)VF+a)T%ahOX` z%(b0WYJr#gX{q10>utZfzmw#J4RxNxVB2MQegnh2Yw3_$Z zgICb1s!J60Cp4;V&^ZT9lyp-DOx!LY(F=g2A{Z`jQ^kL#t%@0(oIJ(~iQD$IbhZD6 zL4s&}Ucky~tXuK%!X`;Rk7S}J{@B$qxXE#-Ys@k?GrjZH zbG1{_W97s8YW?Eu6YHaKL%*fdyVlue?r^>H(Zl=eD)Zi zLG~-C_Y3*I>A?R9}liVEDhr z{DS`RO87_Rd|~xE&~;Ir&68}q(TvL5lpYffnpgnH0MP`ADgOG%a{)@)lXvgD3>O2L z`+mp;wdTm)z5?64Y@P&9{v^TlDXcb%+`%kmlCXA5i`v8ON_QWNIS_X;Oii@*@qUOZ z)5d%@`a8wnhG}bIXT&f&+uE;W_lM~4s&?aywYYcuU?R>lk2}vI?)2&C>FC0R7=>0P zVWutk_(4H;k2^%LUNL?3vI0>)vz|R$qmo0V#O0NY9V0&`H~%nXmC^f%Ajzi_$r?B~ z-B#cJ1o<$`Ooe`%yME~1&##u&XJ2yyr_t`XJ35vge0Qt`hZ5zjxR8U=O>OgCqtfPE zI|1(qdx7ijcC&VJZ^Ek}Nwl2^TB;Ak?VQB;k{$WIuH1?vJ2%g?bf<}HmPRL)B<}L* zK6b&&O#G^s)GJq0=MOYK(&Z#(Nr(098W;#oAr{pO?*m1H#=tU$19|UjOgi zs<*PL^z_?+-k>bL-=DfG+?VUe$6E!(gUf6Wjr6`@&c4-r|F8EoOos??DV+f-cK`2} zzy1%oswq)1=y=sBlMHGt@xCaW&bY9tGPMUV1i&HDpD>Y_kA(Vj>2MSHZq!6Rm2=D1 zVpLbuY6H?ApRPORl0sDt#Q=vP)U|11ZtNFxA(*5Klh=G$J zzU1(`tfZ;~t`cKiNNG7(`Ap{Txk@pWS91esFg~PW10}{j6)H+lsYVmSqc|tsh7}lL z)H59FX^|*IsMm&&ddyY1R0;YQMJZcRU;Qy(APUzsZWe#8E|RGb>Gr)4mP3SUvK~SM zDz|XrQ=*)tV!wJ&;Az83R4S+*L9_O1|*eEt57t44HAUo2nU`~*c&ot6PEbA@fN|cCd-R}S*q9z>kzUcF-9 zP*S?7(C0Rr8JH(bp|Db-r&S1+3l(AJM})herz_DEcwWfUv(;f8)f5?M1!%&ct06rL zTB{V4HW`6l=a&iXMbGWJ4|L1vrvuqF%XR16H8qexBW8-ME|E{DaHasVydtWs0p15? z)n`+l{YAwuU~rPdtMwfJM`P~*WJ$LyY`1OOHfGwk?P=S#ZQHh|J#9^Q@3y9Go72|U z=YIE`_q}t^|DSK~sHoTxJD;kkT2+}VGgoF-4WDHN#&-B1jji~-3i2k{j{tcH=41c; zK_|(eb~56%(U|cP-ZnrKG{F$4z2OFkcq=00aZLd_+_&b51JZTR`2Cb+k6VKt)YDe2 z2vR=x_`=b`n|YYmf{pZ>(pA+Xt)rRXC!bCUX9p~Qsi~GCwP+eYBKIHo}QQ^gyEISYjnzU(M?@-PW$}LPWne(d*V6?}xVA4+c7{`eLI>X-eDR zis4?11{j(|z%|-1#o|J5D0De{6tIz*G{nat&Q(8jaS$?5);gc%kYZhd{&YR`*s&x& zvDixI6X;McO8gbiUh7J~Hk7OrYB56an1{Hu-YZ8te;w~j*!O9sh$!7@w?2pttiD!1 zHG3ZtlHDOY3Tck-51BOfutw4J`t-)O^n-?Uc{T_bG#I&%3bgB3>!b4ZjtO=QYUW>v zqJLrPeqku$?`4Bg1-A&Z6GT@ymr{ftrsE;Gug@LcV5UP_1b#lW`G}Tb{o*2 z#e`&%j0`3NGt*2lW0?}=5Msc)(P6>^McsmmG+N;`2wr@M*s@r#g*&^XzhA0|GBK_966Qg9H@V@&$SS(D*m=X9TD|S zBr2mI^5WoY^8Nsn8)ju$jY8p`hxmH4ZbRG>afO=GCS}{n%;Q;H7Lrlm*wdcnj9jh} zw?jQ=$1mS;>>J)0e}d9p7D2-aBEBD6`aavMMHbiIl}4EXOPR%tBLekqvDldJErh7-@9 z!r|1Rk9l$=iK)aQ)Na*oroV3&vc0lpq6r5znpkLe;X8Vpn0K@A7zMFhorHrDu8in{ z3Hq*JE#8`NF)9dtof=SO{_vHdnY}rfKC__lXGA_-4ei#aA1a&!A3!pChv=#Z?7#Ju^cGmDU%ToN7OvM3@RuCUZt1cT0{gP|Lh|{ zuexv=b|L>HzUQf!spwd3Tf$S2G?dW(P#5ojzn$Rz=34rI7qd2!lebm!RwanEIFp2Z zf55c1#`e+i%2!UehJ5aKp~a|S8|7MXeenXp>`5^_Tb_f$x5*xjor_deJ5W+}@g7tG zdnT3?hoVgZ7gtIyU5-HoB$?eVnM;GFOMUz>}>70_3I*r8K8QIxQ;Gn>vX;W_ephz@Vp$Ag?q7jE4WQRR%eQ=n(`7}wCo#mH{%$8Ie! zs0jAJbHV?<{OG$Ks^bUx%Uq)WET#~=B-Am|_yjRc9*~VAvBKe<)1G`TYG3{H&qxXD2gnc6IeMnPGTni4^jPcm; zzK3*Lu)sdOToEJti)qhHYn3Nu% zs|GdMpB_;>0>Y3d0OQ+Y8b$UEI?M;=mRy?UiJjUputDlz8$<;KF6;^0jCxAVAorG2 zx^d5lV>r$G`0x=CGMh?@FXS2?Hqo76mY5i^#@*M@{k2pqQyPXmx zqVmD@Bq6QEUQ)e7h6sA0uS}%@77qp(!ke-(2DU?b+Kx0^U*7|&A;%O(Xv@JQy6zD} zRz6GZJ&;tWcF%qh3kaK*4W!5Z_QW|Q#E(*fQhWo|G7Sr@A0g9z|q}r9ltXc?;gCk4bQR*YOgzj zCnx+ z*2|rnHBcTRtcNh=sT_`RT!hIwxvp*8FklGW*wVALj;qooHhj+DG=G~sz6r-JA~_Z36+)xME;OIqVMH3W@9m@VWi3wTMmXiEa&|)1}>>N}fa3_!XiDU8Zky ztpqc555jk-@>vmRn_Du{dKn)BkTq)3CN;w(_jAs&Qw;l{!P%!WkPOFbmY|((DJ1;y zh$-JjkT;1SMe+RCxj6{I?}Q%N@v>~6p!>)d;ngh?_p3$35aA{8j*m-fJr-;uz<^CJ z4t=YNT3(i*u@*p1Z6?vL4~(H(iZp#51(dk5=p#Ev9<5D}uIofm&^hxeHk;NJ+A;PY zc!Y@xfAW0GqcSAxy^w(wx9eO%+BVfpr@kLD3H@K~r~2F4AysW50c(dgB<3>s{o3G1C#Q zk~d#R{u36pf$}XfI9n=$iad-j-8M z5**h6jExFN8h&>HI~1H1hmeb(9uUCPUQ0v-S-18KwM!lm5PWz9JOq18Vp_M{{G`vb zs%ZvPajnD?F!!leG>p8W|H4)oeKd(p`5JP+M@MwNtS}EERZkM7*|A z^AjEnnPDWWL)!Nt!AvQhaRjTIJCSh0205(3{w$%jN*rhO3vCjW# z$r+}IC(Oj8Ct{jvV*OG!vB6#gk2;8QFLNU8C+J``j zZUe7P*Pr(UKa1doZY6Ua{7gvD+$C!LL`_eu8<3UylSjpy@voV8RLEd0&ifA5i~=f` z=G&;m`}jL;3Wd5|Vep+x%s3vu<#Rna1%>mC@yN>~AVy4G#Ehm0IHVI)V)>M>qz;_Y zJ&`>UE6I9$l?P0rt=amfsTbYk0U*70j9v#C>l!!mV&VCsXtQBcwZEecQQcvhzkLay zX#NNlk(((De-tF2r!tqa4s51In%H!y0!XwMeMI;vk9j-$+ z%}-TqorBg}avGWJ9HmBe$mEFpKBO9rsdGSLnE(5Fh9sz=UB_lnFvbaaKT{rNcu((x z?(u-?8Ub)ir-cd){}p{@)kG+#l}%yb$eN;%46|I*^DJ$_I)Vz&m_cS=x|+2B*{B)% z8{1KmFH6vBrvKjVdEEv~VQ3TuuBn_&k^@26U_e1Tf-!UF9j~-*j`Ys+s4HR-l*?nw zysVUx@2AEy$sI&Z0;fQqxxBRh`R5|;z}E$|1q?p0weM0xwJfgK!r-1g=5o}??g?q} zSWe68saq1j`Ba5+N!lxM2~{)uDA+4SAu5bGWUMSm;X~Rsx@qynK7qBfg?J5&l29&Z z-U-sE;fr@Q_E44^A);!?${E@&4C~LdLDPn)(Mi;X#NEd$Ge2pQPz)%WInX;uu&=Y9 zH`7I|PrwpsJMIB>(H+vb35EW`{U60nLLlofiMgZUs${;u0eY}4g9ZpajN8aP26s(O zO()ms>8xcP=tVSsp9}7j07FwOe(M(+G@8@%#&H38Dg%eQ@(NEI8wle%zG1&SzBPla z&m-Ed)*K$vqji1*-*3&U$Y;-U9o)Tl%~&lMJ<=?jTb2|Z$Z%u~X^p2-NG-RS&kPnj zkRPV0=l+^O)uZ}rgmGIkyrFvSwV*{k#p~CKB1riRp6$0oK#Q|fSuQrssAm~;H3a>> zIzQa;ohxm=&p1Kj!D!n+cS53)!pTlWR7I@o)P-iZ+z)G8OIg4@Hp1dkm^-Lt$H0#? z4N2f%F+2^<^TkwSl5*KMox07iO^&B70=uTgpO%~6~WH!4dH1bWp`!i%*H zRJKNX!*Z3l#HpOksUl@+M`gg2=w();YUb@e#5r01M8T(#l9Sfde#!dPG!rzZmFS^j zt)i`PF(wzA$?Ks(;47;o6nW3%r9!<2f@yjz4ghN+Zs-h8HwR_el?Jh9juk=QlPZGz zWasvcN`K3y^O8d3Cx&fgw)8W#-I)s80r_;Ue&|#Uh zv;H@~hHXJkjeO@Zc)|xu-{|cnRei+%G#(1IU!dMU>S9DYK z4z-=SW*kQFby@I8wK1PGp0-~L)3#7apjw#wHQTa}$g#zl0c%r4_QwM;=cS^#zwsf8 z;+&2YH;yuznPr7VeDFvjsuaF{Xva>={7^NDukzCb6X5=K+Yh|OzlMl7td zD3(3~0mjADDQ(!05+|IMZ?Qy}@PfejosS-3VyiIwR}Hc_cCYWGNj zW}_^fs;q5w-#7X;6P^xF2WTJ%TG69f%M+UGJca~eqD``74WhMwijj4L-W-zz%i)N7Rvr4HdH_1+!36{gM(L%L& zm_Iy1B&a;D<(`y}g;8}%dEVU{eCat5QO27!Qn|r03|MAW&7}500K}xP-B!7R_Xkh| z@)02-87@rb?D{=V0G3UsO(4p2*NRY0^q?!$JEs+d>uW*8zNu?g0v2(++bBC@q{dpP zh18;)<1;MI&Yp60d4+WGOrcBkTT$q_ie=nvat)R3w?~NcFnSalevA4xQ|>N5rAJ=F zR?&~;CPjV;V*codtTn&sEj}~biM^vLZ&->%DK9JYtuOGvCkOH1a-1mzJ!n(ENtXu9 zzUqec*g+Zgm`w9rSYjcu#Q?aBdE6cCAZ=#G;rk?9IbFKcQKnk8;3jmud`t;N>Go-k ztMG7OlAw<*^Kc^Ze+92+)c)RuA?8{P6Y5}_ID3JWYEIrW;bj1vaH!hD0|+P$M@bh? z#6lpJ*#(}8Dyhv2fy@fr_WS*x1U(q(Q-t=x{Ka_G7ndvRR#WyQ!?LDt(7PO_+x2qF zqJvj7D?JZtLNgSWgU)dkqwfYc)orq=yKp>}b^HZ6?mL`+hYB=%7ZOC-feIZr!W`x) zFz%1hQQM}P#F%lB>W{{c)7e9>e<;&PSoHG9tNqUGr3a_h**PvMC-f=ZXf?11{N*(v zN8=P-a}2GrqXJ1y0Z46D%dpK3_3kvw7NSvXN^6u0*qX&dI3(Hp?uburE}LNO`J(H( z?9jI7iy=~|PDq6cgytIx+MIbW5?=c;?-4<&Pz;5@rb6I`X+<|rOC_X(S%)u3@xZB`IEZmaQ< zsP1WRcRsqY(^NisaBiTV?SkjpALW0~E9(@bISu(G<9-+GJrr!7a-vNK_4v;AT=SI4RMq9+oF0IE`K>DO0(= z>eUw4ZlucGKQ~rSNSRiaR_1hR*WLB+_b-rZ4X7^g3Wb%L`#sh_HJVO6L=(9LxVSWn z+TGU_FG7z_sEvx@OMIXL#TBx=t3Z5@V!AkHhhytr+xy+aKfgvgn!c5=rj~gstKQ-> zWsL@T!;d?jRc$2RcUU{?t;1}ge4A1S&3-*!Gctcc`dhl|E`wD|AlC$AaL}_u5vANR zV{d`EEB)u8|BchEC3HMRJx+}Nxt8@0-W)(QDasNz|# zvsQnmn;Jr}(oiOZ14W;V6kIqG)0-Mi)|&PH%%_>ik5UCNsFF5(>Y5ohH%^gSfZA>N zT4RVss>`l~Epo4=R%5WlOBrRXxH3w<5cp*Rdahxw0P7v2e8=mX$^0?3CJ|Q6mI?vZ z%`UxG-nb%+a~|CitVxU@tx}^|Nio;yqkofPQ7j~EHl4X>lh`I$ic3-Yf{xYwnt=ng zW8AF3uQr1@OWNhMD*%MQ@GQVwLl4h*x)jvoYeEICSN>(R#y3eNnU*FaB02DWO1i>HK- zr)dfMk@gdYn&$?-36=0d-pQfk8iduwxe|Pa4PC1_l;xnSeCMI#O@?e=okk5`;3jNL z4!XLdR$x^)dg~qFI=ih}U@W-WtQKxU-$=XlyFUJ#E)U zlY&KfGuGn_O-l^Lrs>6rXdrHYifcta0>$`fKO@!t8H0JJmuz8qUcsM<&Dr0XT(fnL z^WA^4S~%{2;#IS0V0rhc%!GMyIRtIkH|wlfYG>-RM&PN?P3mohH0ra_V%*lxqgl(Y zW1?*0dgfYDvyP!dMQZbI_b!c54}S55C7PqdKaNnp>sWp4qSJ^l!}#~p?X_Z9-g<+4 z72Nu*ZKKYdHOJJgLF94uJC8K>s546j9-$4t4)FVt{*R6`UwqTYi&ad!&37ihTQ{)p zbGYidJ}-5cb$V~;AKh>7_#1odcD}c7V`+Xbhp+2d?6;Ti*E@Gsos6+@wH*^I+B!a& zb}a@St33FR578aIKT-_4&u6(NTAg9*&VVgkJeor%Of%KS;A4M4ohH@l>7KWg?Xq0@ zvb{0-C(Xn^fu!==Fm^LJW?RVvJ+V`Ea@NancT~BwzE-b5Vj~n@DtaR`O|C%^X20BT zeWNYkb3snq-hJKZJ@y^9Y?ZsZdd9u-gBv7_8|k_uv2I=3Kz$;nd&X+jhpGOk zUw|ZwLn&{s7 z;^BViC21~JYdY`QMC+{Hh;MYKo#9rqp0o73?!y-hef>4yudtJB0qYz4(02LNMlqg3E3jh|nj4E)? zVxA_+{DxyLzM7Ze3Vx1jsPtbC7y;yXSzBdN7dFdk}j_fGQ?a)Rp?DC)zWFNXTFj zQg)G05P!!Dw3PX&mFGaA1t~ShiG<271vWhhCgq;sViYL|F-aegJhuibO@xolzJ^TY zjw;Xw(ZI=M1_y8=7cGSc+yaITc>t^m>A$)G%4=e|Rzl=v1eS#vh7?9ZkZQyvKormV z@StlUT3n;x2Ud~=!6^045Nh~8?lhDjQHO|c-rd^Hea6{ECkj7C=T*HTVkA#q5C^HTrAEd>8E3v&IgK zw4o!86xedQBOUaQVbh&1lTJ45-(JW9`FmH8VG#XVY}$)0;B;QftC|bTM?yFCeg?;> z6m?wsGUad?jhW_ZED|zDM5^vaSpPWXB2?vO=j5izg~Wy-1dpP8^VG==w<3}UkRJ1Pls?C}zErz+<_Kud69S;l9nOXXBlZM_OANnyHfTB^GMlE#`) zV?h|zaKP4hU+NGVJgTHx6zwj0)k5p6IP@S|6NdT1P;gB|qe@l;Ch$2kTZVp+Y@)yx zlLywlsTPk1>cG)x>>k+GEnxB4h8 zai&5xN&@@}d~}T*kgCA<$XQ8cC(|5347zMqDm0_w3NRXENbYMJ#Uw~^SX`wzROn}s zH1RwUu;1C@d4LhJXUP~Tdt*e|J-+G>8c!#**cjS7d$7-WN0h5!?b^B?y4IFw%qu`A*f=QW#L5BEF^|nL8pefC3m; z2Tx;=)Rk(pU|!z)WmsBTo}oVrRte$^E!`FpuQFN0qV)xJM=iW7exO0R_paf?kj-a+L!B2|;Scb)XMNMk-rv?n}sjBG08unYcV<5*s z&*h5fgGm9)?qLTsFk~wU0Upm}Zitw9f(4V*Up~?m z@UWd_zg(c2Lt7SlaA=zYGQXbGiYc;~+bc>-MJ2c8yH3Q> z?tb0dpN}N5+j8V+LBu26r_V9!4TFQ2TqJA%eFGGLn>Y^o#lz*i=-S;`Ohvd5mUnS5 zIembl3>Q^>^qVq$><_ATEkfm3W>N|@ZcA3;mf0gK)`-wCg)BI^T>I4$DQGS#8U03X zcZ&QLdTT7=MOQVU`7MduG3cO+%ucN|BcUI$Bnu*oK}3=!p%y%nU#zHjM3S|*C>)xO zG?Z|JBZoC$U}j5I+*yfFXsaL z0|FDlb9aoB)E~BH-0$M$PEnVLB1tLqQM?^vBzKa3sz%}d7Oq}ctjO1ikX*{w_#Fdy zHZQ4}AUg=AJpK;YVhAviBH}sUPSE>`bW!cTQ)NR-O=@gocJyof3t(Vve|z59+xhJ$ z^0W%mc|K*xBA`jV_F1Tr623;I&H`5FAvf!mD9$# z3#J)^#AV%HV;0VH3?J<<7sRYX{)IH+xQ)4}v}--R_9t&_1_M67cW;(Fy-;7yJoyt^ zLr?I|_RfAh8MBW+S}b^6cDh4=0RdIO0s&$DFYC+H&e6ue+3C~p%2Cs?JK;d`5#QmD zZbJm8K1UST0Y+catw^r)zu&Hml-WUw(r=7y)NytBc$r+1+Y#WuU_g-=+;uWNz>8zW zg1bnelZBQkjaOx17LsIwar9QOXOg-65g>C+%IfyE@%x%t*DfzloiXc)PPbY&b#4O3 zwg9OFI7Lu1M@|Yi_nF?LC+uZ6nexPawv%o*bPwYTdrB&KvY3-=FkATqn4-U+lEp;H z@dFS)0#@jL1SJWogeY^0A~idY;7P*nYxFfN$<+2EPr|OG$53TI(NBG*qX(9nSR_)= z{u>SmhvJifb17Ao(BTWeoA%NJ`LbN{RZdBW< zMEtAwq*968bzMCWZ2>~h*5x2%q4qfxinCWgUC`Clb$I5?CB1Ix*rj}r2V@_ZEz3l# z(1o4(odMUkn%tnDat3UBxOAMt^o0KiWhg9)*sscD6LWuCh84ge>VM`g4fPQiY5i8g6HElI8|J8fk!N;CNQLDccR z!yE=S*zEHQr40ywdregSTs2j&2YL3RZ?m^=M&07IW4DLK1k^4er8=gCw}}YmuRcOQ z>&(1V8@7)rZ*{ikt< z)3(GjUtjB#iBSw9oxtpPzCZw<({hmyx?04_Y*E?ZAurCd=UDPiAIH<$1_70F5epR- z8wkzT4`~MLrz-(I%G+eBZE)3BK+Uog;@Gnff>p)Wp0_i!HM(NmAj?+)c;76zcVHsw zS-d@s&A3!>4DOX)CzLzx`lKLyk(z;SAkY)c>VH>XDR68CTU04GaG*;3lHsg&QZL^F z0#EZw+BIPHo@(xDdL^af4`-;#S=Ud}q(z}cZU3F~;|8i<h-_6&Gron=pEwooU5`o9{0r57Zf3MSpjP8DenS^p_+DAM@YAI1KcsB3 z5?Jc?`aBL}5Gc8cWPOzI2z+>$DRysR!a5ksx}BlyLFA=w4&eLrL}NJvqaF}N!s489 zTRF>x=kvqd=YK)_zMTxd#|Eyko|3pg)mxVy*%MdgOD(pI{pP8Kos-#8>1ewS3-S^( ztLTn(PAt;yRt=iQxa{$6W|WVa0x8pfPdjBg)>aTSQ2$XRKi6ivO4;(-4ojbC7ZT-l z82jXqacQ;phQzjW%Jy1^FmItlem9fvcbd-K2~S-2zbW~wc?+K@!G-i@D(x{fVuBuJ=0t~Rt}Lca1&!7sR`H9Dz< zGyM86#+ULfGh&*vGzlN3P&3_${tyYx)`?yHWB2%#gq}O2be90Dbeo zExe4|Dlnt$RBz#UFyUpV>-h1Cv9?%-&FAWNH8z#U_ZCjrW7<2!w*>9Vr-7;U*cH?B zq1_$2f8TvOKxok~M;VTVx|Idb*wQ1P^hL47Z4Pg3*TdBp%sS-OxG#q3$?3?t)5F;a zcVp{lfXU?Qt(^tP;{)hx&H~uG`~KV$f%Mc|G}i%!2a4&-k>1hq&Rie})AOdjMIB=+IOJ8%@@%nQclTbJMjgFp)j&*V1cYM06MMuzp7(yhf_qUk&m^orit!?Jc=TexzhvV37yY zc}tIj#q1O>O63nv zB+kXWx#DCl&pGqlZMcCQ^FaZl-)3BOIP`_0g)G_?rEEfKZf#S%p8j;$Ef{0>#-ffE z;7a&VDc<3_X7!tno$AJV+We%LUxtX-TTM8Slnrc6gnqJ@EM3tY{-%RAW&F}Junr{) zZkW+w#?cf}NewQyK!Vy?frK%uS^UXn&`_5}gLC;qet;E+ddTFm>5ANo5NedEP_1 zr|FiYRUZ|LtU7VPp*8K83&T;`^s$2#!AQ}^kYlE>8a4+rYAw=#uhUX8tVfvox`CT( z(tIWBCKJRYtg3lU`|Ww814$FXLn1M#riF}7ZWvk;jUEy&F>g2yFTtIBH9RpHwr(BG z?JVG#(uQ`JvK5nYO?SSiPdJKio{ZCbllSaCj@xzBkVq~tFoK&)xhjTizu@YK8!~%< zoXj2-r$eB};?qIhrJh?&^6_9@ApZ_c-_`m20dVieUZER#6JFq!XL;Xmu^!Yc=YnW; zI)+`JuMaV>^5&^>et{+p}5g+M;LN&ICDq008d3PH35+M{FC%JwcP$bjHFDfk^}=2+afsfkM{ix zkL!pZ)TR`j+MU{e%NqQz7ED8>51(FCAjr>+_CKDd^XI|~YZF%!YX)a`XWB_=I;!$L zkUxl%?(@^tpQ_#abIgeOL+ z6v{^O5`K#0_6`evX9pc`(itC#n=unkz1$K6(eJn2ie&H8_)gOI?-h9M?&4YEY6W`$FU?DbQORrC}Rw zeJ~)}y3IeV+M*y#5UOqB1yQAtEuFM*db0lO26%*2h1vWsGjLIppM>yyIoADLG~^5h zWD^1hZq(t8)sI78Q>@?x1E;)5=wm&o={K^5&nQD zJ0+&K@FEMDI}~NR8P0DJ$Q*Cm7r*J|8+i%w=nscN1u1fm^|4wY7QgG^-WrTuXYMFI zg6u43{Vi;7y>G3Rl^YedcDoaGEmyY8v!TCT>FVXgfkE$b8A@{gT&j;7h-yS9-rehR zN0dWnF@V^%tSF{+;0_DNP!92$eUyPw{VqA7)9!#*mK47*svev`2daBSm`EQUNN@dm zl)5OYQ!V8aUefS8`?_EfL&1`YiB##l%uSSfD`xP0M>5s7Zxc;&mVK2&9oZKj-iE-3 zN+v+i;fo)qO#Zo*J^^|&uD^J37oD#m*>&a9)g7h@SINGuMrhvMWmr+k&C3)=yBEJh_DdML6P7o!zEoHvwc*n0rh+su2pZ7;chkhrA^%^WmVkfk z{&x$-Kjr`IR``FVfq+^9hW{@Auik}!m;ZN7)c%sU|9k@fdHnxQ1ODGF)&7qAcdM@c z!o~9Z3-{+5tiMD4T`uxpkS@`GLH?DQ{HOGv5yiixzpDKq{m=N~-x2>VI`kK!TI1g# z{uV3xJL(@7O@Sp@ diff --git a/tools/python_snappy-0.5.4-cp38-cp38-win_amd64.whl b/tools/python_snappy-0.5.4-cp38-cp38-win_amd64.whl deleted file mode 100644 index 30a48d28ffeaac86a73d0eb61d62e1fc0bc17308..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28897 zcmZ6yQ*bU!)U6xawzXp0wpVOpMJu*#+qSJYwryj@=9^@Hd+$^ApK~s{y83GNxR_7P z9@Rrx4jcjl1Ox;I@*7nvehK7ueo_6sf zNP{FO5^G70&}msXOoe%XbGIO{#1fZr1bQ#;h0V06=B%HGpW}FP1USG+$^;kPf;}n)mDd;gUnraIk#;6dv9-R z?-8rhI#S)mV_>*JUDrIJ^gy0J^{~XQy#u3(REM-ELD#DQ(&LR`PGU*6YPmB-6tC~A zHzkhgRh46~&nBY)Czc#Vc9@9_t1MmThEn!vq125eLLT?C$Te zkSo~VVUb8rYi9Q z$~86LwytMpp4YB`(*yks!5QNhrRKc|Ye2f&?G~FmARi36ZG-_9dc4{F7VDtL#D#Vg z4?rEyFRO29E3Hh7SPu|~uqz%*V}&=_&Fn{N*Vc=z0-fFy+QufR0)e;~d}!pB^j(C* z``k|{)JMC45L0@3smLS&PH*tifXdS5_`?iZVI|xn7B9MsevV;~AzLhasZkprf7=x+ zd-X>lWSfAkl-KLw`*>)~xPfJ)V=?56owX#AVH@ryouXLjnHYv}k@cl{ZNv~``iZf!QEue%c0j*>$^ANHVMo5Q{~2v5nh5%p@pddQmHSfN zUCXNWJVbHo+G|&9bpTFA7A{AVu-}jye{BBTUXevy5WT?*((F-yOYsA}jK|LX#>x3s z?HBgu%#L_|MqCg)SPtYOSSBe;r$)~)2L<1q7+A((XuoE5GkRly$^%k z>M6CIpF9#Wc1zd=|6jcMzuaLob!6ja zaR0AmjqS`h+5e-D*|iU>hq^?YqpO#U>7T!Wl3;--ivfWs=Tz1mh=hnJZWORG z;o{9tPhcV<;0lsiw((BcB2tz?c%x4EE*>y3afCq>pu-?hDtd>zER3M363c#P>N|?O zq}-3+x4pN$KfRrXHv+%Q6?1uAFP1CSTpf6A%@GsPMKK(q!9uR6uUHfOeMY8WA_<+C z)2?DY?2i31Pg)asMktYXE$X!_=a`P*AEu`0a)3H?4(_RslqWu?3eoRPQKGi0^718$ zhX?$Q_g%-U;Y~rWPn+X4zd#&znwOHm92o$=Fa%8&RNg8mQwPJX-}CeKtQVHL_?k^X z1;2sh>_5wo94)}gGvX%svId&yfxRVQQBsh}pIaT@DNanJCRxc67aUUBN=1|;futmBr#yPCUodsE&1N&Sq8i{pG$TqfTbNF9 zU2^eS7}Somgj|}}*bs!zP?ly9p2ScRgpiiWIuQiTq$Qny5L#&zuqoR+@%tm_N$F-x z)9$Ci0x=f+ut;|)c`gB*6fCTVn#Ig$^(UGO?@51R5?)Y0WHCFtkp0}kdjq6-L3avn zCN}%dYf3R8<)td9?ujnhPdN}wvI{6V42j2zr__s`tl1G##h+49MPz8LW$kv{+Z?Rmv9g&3Vq z51GX;qRv?#IQkS0sYKP*F3J#L2vd{@{vJ9UHY!i(1qJ3P8Zcn?R1Z=PK?2H*X(a4u zsA_+6=AZf^*?jGb>t=*#bxujB(>(GRf~2r|Na$?HYLVM@f;C5_-)1q)$?KQp`m4GAKu@SPn@C@B)OFm&gEiGJ|11eW_ zVmVKvinQ(WpVTRoHJXU>nZwKPm3_o%gONXShZ1It5sin79wI4_O$WGi9?p)#q|mZk zArl&Ng!iJZ!p+x|1k#?x`PHqP;@2a=Rqsx@;pkcCd$}Q|HO1(%ccW4lL1QA;a`2 z&Wgg)wmJ??5!!khI$vR#%0^-K6hzO4F@upo56~Qy%O36D2*uQSDEzj|eyL%2rVvQ_ z-mn;2;si8_hVIiHJ_SWoc_j`nx;$L9Oq}Qw{-bUlj&}vgZ6?*1&3fwwVA8~^TxOwEV3gu zQ{CAoK(CC<8Ea99+5?XqJqwK{NnNZ!fHrJikbp&c9b<}G%`}b`T!L#us!-P%9amI= ziZ&NL)fJ|{WV%lJFR?bDgh;dfA)n7V9R~gBhv=2}?I|ne#jW{(&+%mE; zavT<%Wa6{E0ApldYofTk3Ff8Dcm4wbgfq7TDpR%q{EKbcyHVc&%0F=CbjYC>1>Wd4 z(4~Bu;t!FN4r4`>a)o9l+^w+iD-vGr`67?euZ7%0mInKyf1JhVeAh~R$61y!U1k*@ zos)G=Tp``{cX8G&I%8}WECf`hB)SZoSpu}p9+G!pyZ*OIfx?(A@;NLPPIa67`X{Qu z->ax;f74c*BUo_=2ViDatX?=s?0MY$!=QE+Z1nTA^&soE5^b=AI8X7`n5p5tq%i>> z9_k?-D5HEqh(3rF^sNUSBwjG|U)SNE&E|o1&@L;ed!S1%N&ze#^CZrr1Tp1N{yRU8 z{-;ZYKe0QPvhmLpCExK210Ob(-;+PeUzT-MLdQbt{%V@Oid8>~{f1QzT1_sQEMpxZ zn+dPDxz}hJJ84X|1+MC*L{%zx{6)56Ar*Q}dQIgQ*9Q?EQg;9xZ6mLuq!p}2213MD z77}M~eN0sk!(Fg~9r*F{s_T^qwy5ySuY*wrArrvS{Ga;WL6~;1bFFZ7-5~WwX~=s* zRXcNa`w~{Nl~t0|G*wij%Iz2%d{smVTK5E-^_L5~nuaVe+UNuinLFirnz7$QpCl4d zV+Bl9zJNo|4Jgm#Z#sB!gi7s|e5DR@z)SK^SYvy_ zOCPyTLOy1j-T-NXe=ux>GgpXiPqs^2+<^89sOB~(g7=rw{vUJ^nUxCIXiXM%e>lxm ztv@zDOG=G$9dDV+KN1=4jPZgSU@BSzl!aZmd>>sxhQUOF=wi)O@EC9wDi(nqP~GeD z-7c^(8FOfc?Gb`cLzIut*`{ z+yaB8etU$=ljCw#Gayc-w|<0731xl{q!RCHLL>_8%ezu0INUl`JjU!%7wJe(7}~UF zql4FdW=5cF&MA5<=pbLB+#u!=#?qwf^u%u24j=gtErJEz0m!FRT#@<3%2Ad^-?)_&1&Iw{_iJ`PFOl9UtH;b$E z$w$UcjsRgoq2H)d!xfc&@g+MGy${icGG{rN4G}@VxVyaV%4Loh#^0En?UD+B)~NsW z7yw;GHb*9;neyhAXt~Z`T#;o@R#zSyc{a1VWxm@93bTJhaBK_DW}ohFyR06f4Y%7q zE)+0A-ZOOG$L_o&UF^}b={{Ca=kdqeqrd=$ebrD^^Ye!Pmct-$(&9cv`K~YEXbU6; zCXjpb=26854ALK%B&f0*tgoR_pFzr8Gd{kLWc!aQ2jRtH*Q6u#> z6}F64naMUXDjg=Vpv1$(5wf-|{t!7$Xmc<(UZEroTwP4nT-4d^zySD&(J%Jpx~iWA zdJ`Awwbpk*gwG9Z4P-)gJ41T5Zv4SbVhMOR!`T+4-~%8F{I7oH%*(Ci!P;=2QHH*J z>y(WU0fd*i<3zFFo%`RSryp1sc&o@o3@w$ZhQ%_F(OZ_H#x15f6mh|pMFA)@Gjq>f zZ4FdMFBJteggVY&Qa1I2(cjxNLKOF)3fW72f46m-D1>8?a^U4#WtrpGqF4uJa=r(T zIiR9g1){E!C_|ER!W1OYHR;=1^1KlkmWmxwpSuL_$Pyok(%}r;& z!Hm%fHyciPel%v~ljgEJ{=&s-H>?%TlItZhROXCt z1zRNq!SgN{ z1nssj5^#ZHCcWL)vmb4D(-mb5>DJarX}jCS>}}mHcxkgYDpl8vv15yg1B)MR#>&Ti z(>x}mA1;yr0#5I7B5koKrsbgm=YUwpLlqqRR`P)>jy&u|9d>s+srv!@X|myJ;~-!O ziGw8C=h|rtmQ>`1?G47FqW!*hr*=*;XE*Wy8iy1Eh&Ud4O1%SkSbD%gm=*~H90x0S zfx|4AWv&MX3`3UZjP}ZmB13MzGy5mTR~y|2Y@nC3PIK=P{RiF$$qV##C({Sfx0;VT z%iHvy2>ooUnuFi=1mH{C$qP`4ReM+3+m){(b&xk2&F)Uo8{Gvn^&f&L^H;Vb+^j48 zDskkI!MVLvq%H~{%6E94dg(^x^Yc5qP4h1#8XDQ8v|+BJ)d=TaZ3q?IR0!<-Ro(BS z@K$3DCj&DX)C42acj19J9P&kTLt5jh;jGV5Yr{iw5Dxc@F^!HdTv?J0D7HKmyHMcj zxDNO0%q~BNlE0Q&cTy;eXx@3d;2}2YVW`rq=!urj9R0yxB?huX-x}$gxD^De^jLVsc;s@WJ!IoRxdolIH|M1=*!ZHQ7d5N z1fhXiKT!@Ob$dA?IgP@9RWj^9%J}Na6{6%UOoev`o~Oll8lHtHO^1G2?1V($gYht8 z#m^(sivtP7f%g{oZZOzJoDsyYI&s$g(+7jN@wQaN+xYi?pg-G4(g3?Rzt`y3u_L0p zUqV!HOmMqiL!dH}@<#6e8>ZWta3s0Hy}LNW;kg?L-75|3R$xTYzRuy#@Gtx0SfS!@ zmiksZ1YAn{eLc$FXos@Ky9e}^4&z)&T`)H z=tTg?Z9QYaEnK$uwTAej?E;lgKXME&Y=rQY;lob-SXH+SRUbm!oBWhG^*|Z$8&x3D ziT1o4(LgqfUXl!8KpYBJFB~lILV&f!3{=34^W)~EBOGSK!85g6vPge&UAQ3kW!(K* z$q`~2MP`tsq?H?bfcDTIx1EaBDmJXy#X&UrvDXOzD%w-sMmO-tS00y~F*$w6lhb>M zY!p+`e`S>7A-?&{P5K}%DOylF_b4ROHvQT-mc)#xGJKqPPvo=<2-9-})B+Z1|{ z_uO?vP0Db(`yP_x{)~Nb*;y3*90>5#Tz`iu?8~$zt}4kk;SUa>wB{WqnT7B{aQPu| zmTX#5;E#O@7a3nDe9A`}tGp=YZe9;+&=00IT@joKe%f#H&8WGIS&QV|gJ`@{Eqi)w zuv_N(ytd(gb{ughJUsLeF{4oy(_PyCQI{BxTfJ^jVC|uCE>|)PQ7iXi%H#bt#$e)r z4xBi6x;m!%5l~6!hOBzK{e6(KBFK*+5(5M1hb^go=^TAGR8#p&H0fY`Qr+NF36`PHih%{_6LK#Zja{r;CwWx|82Kbv%wkomgP;15fx_M=ho z`usgKm2wLON{vzj);(4{>29#Dj!3TyE+KQU_7#SI0BN=0$)fz)Xdr@n*ro4vuJRTi zv?%8W6nc%$!djr8@JONuSw!wl1iTfKD1fPL5Q`ZX!F0>O3&S(y#_@{-1B8ck0H{WSc*t-cqjSOxM&AiwOd?st>C~oNn^!(1Mb>N63 z@(g>uO?@|5E!!5T0!Hw`x)Eqy%4wIj9w=Du7&Fuky?6^X$!U`Z8^*UPe;L~^7YS^g z%AB%7!K0B?`A(*0Ua^BKOH{-1u`A2F-*Yb&b;&i1`S^!4L0(U^{^4qPzz!+50~BsX zuaq+(31oHr46<(5ItL^x{MM^^DQ^fg&}*VBJ6@bZ51{*_$y1 zVfh_6_q8EDwBRulSV4Z#7?#KOfy(!4OdsTuJ@L7OXK!X+(M}|w_$p_N1~K7VQFzW7 zgL&Pi_~DNKbR931x@6hUi^QWEId=uMW}_k@^cfJ%^2u760w)*|GQxLSNPw6nqUTB3Zw_qib=m6b9JX|R3KjNPG5dfQmEKcPKHXD?FltHM7y&c-69%O9K0wwE- zweI14Zheg^zR8?i;4u1YU-VHF#*SCosig(&Aft*?F_o;15zgO8VK(b+C;h1YXm(YOHlh}AwBR9*VuPJm>0GL`}3fuFCs#a*1;cKxeytjhATIY)~ZAe z5%n0EvNnnez7$cNWH!ayOAPk8F`R(J%S(*5o0~*ey_C~lf6i+B8-GwezcHg9t??nW zXXr2uVSb#OA3`&dvY{$1Xsmq~TwO#T5n>U~ouM@In&rLzRP(_ve5ZpS){;JdLL;DU zcP7y--)NC{jp9@E^>^-p3ntNdADQ!W@hqN2I|(OL3-!6y*?F4j{EDW>iXXw;dBe^~ zN@Oy)IsR#LqLl+g&JW?J{y%?8Jv@@id?vZ~m)-=UN5f_%Psmdf*T+3ac8*NMtyIE< z*DWX5=P-9`l3BwMR#d=PsO6O56^s|!gg<`O;;Zm)vcB5N(V3vnCVA&I!+9m3wgja@ zh2Z64V9Tp9cyLAn&DT%{vSqK6ux@Hl#n-0rI4JmEzGM6B*xRTqu)%*+BK|PQs-8X+ zR@}c*eGG;M5SE|Rc8x5kYc@I%2)+OW3EZDTRSopsBO#%Sxdzf_E+Eo0nIw9t^P4cJFv zcSG-kdLWZ%NotxrisRw(xN~`~OV|u@i0x#m+ohtMmmj?3vgYET(P0UR16YJZL{e}1 z6r?y!Y0+ZzAemz}eSFeq-w7Q)v1wSUKzAhU_%7kFZx-Xw*0=Rn09I9t(4b*g9$S^` zZj+ess5=G~VEfvoW-Z)N%tF{;+QHt_(k1o7 z-wyZ;rQ$nrm_Ffa^+r=lU=PC(@t3PGlU-erZJ}(VYOyl~BpBA)N`^m+t>t)1F~BK| z9S#8N1BvWZSn3Q2b2t%%l|wA#RxicSZ@TQ>g}W?X4rF#C9e^ZpoOvxS*yi4dDqS4Z zOm%P2_79xC3GqTJ&A$PjDumZFQg7kgyLc(n=O=eim#1)PTuczgXT+axkw}+t3e1`JA4{aUB z+cB#S5Y_cfOcQWlw${sNR1L=cMrA9PH)SY4eLW!EBxbQej&_@!Y4-N2XWc z(i)rd{ul97#$3Xr)WQY-B1L4}#D!~uRbUbxh0Y}e{Dw~YYHUVb=dA4Vc_wSD z!1^Apjo^=l^W71Q6vi@qu$MpdUvi^Ef;LI>kmWhaQvO(9+b{_+LOB!!=WUoPGX~Kw zsUDP-pn5&ekBvk{4z&|m)YNiIK#rX%7cH^~*jJ*|zu=SuQSw`YefvZFRR6G=BrKpc zRJ044qr<-vwb0Nh9}P`!M%bxwxDryj`|c4(YJ)CPsZ0hy_fAw}faT{*~asyxM03uP{nM!S}&pQ7iy&;aK;! zf68^o^N0E66)Hh4ScLgS70y8|{E7(cMUr|%%MVx;ai^?{V6-fFb>#^)ICDw|-#lo6 zz88JgK7+}40JUQ$!`o0vL{;IW{d3ql!J+gYl?sD+5Crhbrc4(Ip#>KU=H4=uE=>)&ZK)EOq4Pa z=0N&{b$i$RMMCpW@Q2mrMZ&@pdm^U6X`XB4b0y1{TA}FhTfTSh%09hQklEl5OOyiW zHB0MpJ0J;cGkL45D%8BTd~7eqr*~RGIF*MP#P!gcVDiTDNEGypgAE z^0U1!w3e58K@qYgvPFu8r@8XCuPl#IH#eiV>(<8UXc%-8vNBc2lfSS~+<6K5rSs&Q z{c6CO{Yzoy8S2L&U%?K+F6EG3UYo6(1g&UTBs2^8 z5HT{ItSSJ1bd?dt?*XUZ#I!~a`Wmk73eYR7;WKV_-fz3XE?~V`FO-=0C7`UFj1%9f z4n?dwMm*SGpx0Et+oNC?W>ue+uW{5Fs_&`% zq8@mR^VK(|4{Pr&UQ9~EE!^oo=dU_wWSTGjTYKWNSpq`@N6>7Q7U))UPD(2*qG&Lt z4kT}UYz%s^_TyCr!4b3{D z(OZY-&>4x92tXi-HrKrOXtVYGs@`l@aWZHIR#;@-_7vdy!@Q7TIGy)cgf>CxM_QnugQpxAlU``Jhkrk|S@^2l^wQu`YJyvwWX%OqHtYO~12o(HhutXz zHXm;QVi7OYI{)s;PVcM#nX?_-)eGwll@$Nofr-H)}VmSC{qDa$(9fG=82>OKi%Dx8d1;twk1hU+nKJ0GpUh z_h2+1`JnzZqe6%?Ip*I%7Y06gxK{%nOpw)S+{2SP<$+!57>LPb*iZ=w%|iiEh6iVr(% ze8g$dJ5Ett#DU4178)w*)W`!K8gAhMJHJ#B8F$>#C#r;*P#y7JJ`MS&^Vd75I3`a3 zK42{IZk(GjZVk<~ByH&dTt@s@*q%fvLR=E*5bFM(f|fNe`V1OfV%f-^Stx|2ILuM( ztxz13BL#mFnfyRysKXfOSQLjhi+EN~j;gZew02J}ZMF6ZqL6dj%o+yqgkc2-G zPd0h3vQ6X1t|AeCD*4EZMpS#`0{ri?L_@I$_0*^fYtCzrrZQ$vTK4q$!bL^s9T0v5 zD#I~HqxVOVtQP!Sc}f&j8RCqI`ND&yRF3Fxk>Z4GiP_`Si4hqTjFI``WTsS3$!}(8 zn90+WkI9i_Zme9HN~DwP7>uE7=)Y_7mLAolJ+3h9;e=8vOw0eOy<1a|x#4le_6LfO z?gDlIWr!RiWph{wq5+7RBS99+1}WT7E7G0f%0f|nG-X|<~FTVxpHUbdLb5A zT+!(xQZ^RO&7Qc#r7Vt6ekl*%11^?*En8HM&^d_jJt)y4yxT?&I(RmGM)`98e&U}C zhCcdGc(=LR8Lz*&8c7I6?X{Ks?x*Tb*U}+xkqgzrTDGtTkss4Z@=;Xh{L!5f07-Kz?8FXsghXjJy7Gd)WM$xS*i)UPYg>e9j2a9YYpbPh9ltL#15KEl5P7Z;;~ zd?dFYKnv`w3ymJ*P+G~gmZE_FsiC6jRvKG}9h z6i}12aQdaIB|%`K@>;VFG>pGmA0RQ!IlEU5Xnq-T1rr~U;7k38c2@KIGwkPJ`uGjh z>ruMV86Nx%lJ6O^h7Kwqzp*u*rIU0k%n-^<-#i$Attfn@Q}&ygNS*4$3;#}*j54m2dlpw;LH0|>%(d(41W9o0 zrs;uFLOl$nlfs$C>26t`>dyI5j*yvM*kVulupwz1%sWu9G+??pP4}9cfLSp5)Z*Mh zS|$&`2e<-6y-)m-Nq&!!z(s%lg{#`VhL^7wmN=Ol^pF+ANq5lg&6Fzb!NzuWW1qE9 z*KL!$>r)$wWy7M6(TjhzGcRPbuM^|!yrz4Nq%3J)4}$bn^11qmZD&pF_iFbNfZFff z;iR?QS@}KZuD_?^@f5q3-rsxlwt-EnhA!B}acw;(#!zo*mfJ?r!}^jsY3afST~leL zH{?VKzhOFDNBpX&)=GS=8 z?Pm#>f6*`A3~b)%gcl@aI}xtAKWoYmL;k*hIM%%VJ_tR#@z@1AJ-|z~6KC=DK9~I4 zH$s`Be2F1E%W~$wOx^4AJB%G{ zd9uS-{rOgSZm1Q?BP9LY>4 z?2ifk{^GoCf0>C|>Avl_49g*$b$#T=F8;aRFeT^BL(d7;{8qs3BzrECW2&jky8}WS z_B&+Vy{?^ppUiBq@bLL#ey+CP5WB>$>3yp$ZGIyfe}3S~vI>1&KAGoJGF9kQU5N}U-$a_ma==L_o&^!DcgCi^DbLf z=l%s8$#PE@aPq&IE@i@!SG~2H0uC^f)Cm-y&K4^9y*ktj^&WgZZQJ$B=n#JY>kjtw zdFbol!++Sk>D#++JYB?kuKguf1f=`Mz`<*L9m};nYZJV$erq-j_hI=L`<_lskB_@j zar?G^msb@zfy>bR)YnhA_vHXjRGmF)$>g(=)@XGze}){xYc0@mxqoY~_cYR0ldQX% z?&E4W$Z(Rr_dZnYZ}0ciC1!o;pyq$nrN`_eS0d)T{k8p-)b{{RyZDB*v&5C06?V{jMZ6SKIUU^4?A?LjZhO1SDwxV=?U4aNE)3 zs_NtS>1{7;ylI~JDE*YA?0+|y+Bk#$rK85FpP?*tuq5MriWt>JK34N>{j%p5Q|rOH z|GOr}R_H!kq9_cvd~c^-0Tw=N(3QH^dw$G%?>>5l^JfC@_lJLf_1VP2 zeR};iDF6GS#=9c-Y&pB7H zB-=uO`NLdO;=TQ{UwyJIPHWMBioAK#zVm4(C28x&<9Ok^zW+VEf_jvw=U^lj`lXM~ zVy4^sEH+%IzqHQnaL+wH&sKYxmD&B^aAuAbc|U)~h}t!NU{4+ppHEAAKBbND%0HnE zJhPYVVr;Fw^;N7^iE@uwTZjI_o^Yv4=%##ikV`n(W? z_Js5dZ|ncU-nS%LRn3~kE?f&cds%8u248v4{q zdgAT1K(*ezB=GSCf4$i-ONMv!W72&eExM__+`R1;>n6k0HSY#lq1G93gIVWU*k>y` znXl*Kx+P-Z`>*=}dvg!gm64&yJHghVW9$ga{VW;VP&bTZu0im)14C4ZiF_=mKUI}O z<>L!OA9siIcWJYLB}%!RB2{aEcP+f;#RT#dJmI&N{~6>y@=>la6XKcv!S z1~~Xxg!V^=`r4v?Ffx#0fL#kwFxKzbvhS{SWUx=tNt3o)&xPb|PHbD)1aD7jTfmB% zGIseU;+9;4?V!dCs5Wc5(wp$33p;VWqINCWTbF5K#RJw7ZW~IQCIqo>2&3%9E|J%H zxW+dY^}7Y1`B7u&cUY^RalVQ1{%d!bw>*P@w>rovJqOHzz+({b*aQX41~d8pa&VMOQp5tB}@%vo!6H21tM)Ek=WsC!j*c z9PK~Ruo_P?ZzyZ~Yr9yQWp_!OsI4D+8t@Xc}Sr@`@(JcHC!CZhrO1vbz#BYoNN#f@pDAQe3_x(UK2=5V%SW~|8f#1zf{6d z?dDi#{5Mgo5Htt!lav@Ur%Ary)61Ha8XBME1Ei5*WJEBRmCSqzA7@WRzL1kST>YEd z*k`4y&Pqz2YApU98T3|QNIWpZ1inpziTf0mWCizIJQH2+ekg2^AHXVFHyOT~DnTnv z5hG?6HAxj6PjMfrLLmsfp8|uHkG8haAKQ!=7ucu;{}32smnmOY6ezzs5NX3&;|MbU zO1;2T!S~w^4*x>Rj?ApIg!qX?#4r@)xmj)M%?g^;SC9CZOIl=^iRR&W!+2ujD8m7t zM^!=64_hY87fqsL2s~nQBcI}x{Mg;mc}hE9^f4HEQs&L{uP)Acbu@IG(0T@MI5`J6 zOectYoH`df(^7?S{DC+)??Q?AK?~_ui%pTMBnrcRYrtCBldWy$H`9>L5e@$a%ZyCVo;pP7#?{nbkPTC!mdcD}cWy zoV6E@eU1u+)dVZJC0D|DL)Ckti5Yj`dkRV_X@KO@$jX~}x^rOudmqCtn_g6s%t+px zd11wkw>Zh11r81Th$y*eG7j(#&58Ht_Fm@v$5==xT4Yj4*JcgLwN-Y0Mx-e9nwoal z^s83HcTfQGs#$2sGCiiSZB6%f>BQ_KMXTsd`jA7aX=Tul;vr>{Qx);i0!}KaeCPx6 zH6%5OFyt6_4pcmODc(v^!ga+u6SOTKN>H;3J*(U+(J$1tGjHIFuKvZ8+l}FT$~$$z za3{eewP2>0Zt`+SZ1bbw=vi|K@`Smjs|(>Myyv&?6k3nl%R<~OHosfK%{(pV%Ye7g zv0=~_bUUa<+G_>>Vx*dCph7;clbDRBy`T0$%P*O>*iZ`Z;?ADoOnm<2O=o9!{P>fg zyqG_T@CM*cwJZQ$yr6E74q5dtRR(2u=H#u7+m_Ifny}M&P~*~vinwYS{KHQp^S9VFxbo9n3Jq36Vj+`%^kHyh zT%vNY?fzIBEv#a&>uq*u7Tl5*f@j{;%Bpuc>$YK#24vRCFg-ih!ClK)uT2%;OT0`m zeqPf6ohlPlFO~#%Z#QZ^LS|_#**EjANmqpFX$^NzB&W3KP$S*0t~!TEJ9{$sB{sw- zf>KWIN>S7AU(#PkB)~1@Nz%h1@L^EZ)>u>MEG>n%cE8kKb1*}{*~0_Jg?7liU3V&X z<0T4=f7xFv)3h}`HAPG*nC37sn^3iEP!M|=zR6;%7O6tA2@_sl<8$;+buWxmPY;OK z0j<={n*Pj=p0s4BQKlq_3@>W8NAZv~!_==|&3J?)g_vq)DfG?5fet1(jiJ5a;~LBZE&+MAb4<-}>bY)O5*A6Uw%VW_CP{IXO;OJptng|` z)#$H`%G7Jd%~&ZIsTjviDh)@~0+lndoVNBdo+=Xdq!|&tCZ_hkT)@_pHFlrl7~2cL zws2fA4XUKlEa-Lqn#S==v3tt?)g+b^=2>U>E9TC-7Nu%DH&u%tHDRPWN`;q+cp2Bq z-5#MM$=DPD-4*pvl;m1awf*43cTUc=*Jm$4>NOhwOHj*L6AGPo8(PEnCl^&)^2d%brr{Ulj z{Nm^NKY=N93ZGq$NjtP}8Q2e1@3c<}eipXzS6a>NpUzu@?~S9qD+Ay%8v7Hk`^yeV zZ+L$p^=@nOfTEg`&*Gn>#Su(K@4yv65qt2#rVHgs`%B41Hqsg$d7tiWu#@4F+$4Nd zEo8e6d^erJimKsr7Z7#+>31JzKXZB)Jh(@p;r`M|{x>M$N%-MO5Zk1A{iIkG@bbgI z(`cwA%k<2QGa!GcRtwHE;GiKtKAaueu_gs+=ZqYPd9`&f^@8$^cQbt=@cIZdW^4S@ zkTam|s421uGS7-~I>nly3qrT=XM+3n9eNeoZG-zY8s!Wu*Y85AU|%`T7paS(f=A5F77xTm$IiXw6To1aRug84CCvGz4FRq7p zi%F5}!1eK=EU5;`3+{$n+ykx21^Ln$FdBjw?MF9S7EghgqP zg*|tm(ID`~z!mVVq3o|@5b`;O>4He2QrRF`Q*w#6f$R>H?Fvf_`NwRi@xo6n!tDa} z$7@sV$S(F^+V_7wY~(#1Ah8Ru6&1keI0Xnx|?cJ()$Qy ziTQYbdD1&DFjdfm__}p-Z@F;^ti9y?@c5jW3%|6AUO#O1 zY#we-ZdPqxxANKA+UD2n7+zM}RoT@ltedZ=_S(KPZq?;>`*icVQ}R>sTX_FDGyUnM zeeZYq;`^Uy&i{@A)(GSYeZYW#Xc2*cVE<1v$I95u!NJk+{|V?gde&;|I&X1e`Mt*c zfNOXrtke9xuznrrzNpFOP5!glg3i~R9up3pSP+r{rUe#L4E)G*21(jiaO=7Z7YCdB zddLO0;mqE-g4nujnS@OKB*XSDtTB$<#V%!*v~f*~+Q;uscN>d2lyEXiO|ot}|)4Wp7AFtoz`TIKiEq%&g%}7zHZqMNRWy}ANc(32b^O>>F?`yC6 zt-Lxt{no!PFpK}^yZ#FQ<@)jQR#EBjGTU7aie>L9^_`Zhi5CJKzJ3zzX_w`cd z_mHcW5*33>07m0V(+_FuD zdNKJ)+L|nXcKTIB&mXeSJK*Ys^j{hpHw7pEgQN1P&|M!0oQSN|d{Si@03{i6z$BOt zCE^|%x!Qn>%QOj74|7h(!}03B7wUYml|sTl(Y?wPsq zk5*{|N=}p#MSjF7Wm#oZCK;UApb`|zzkrirXPMwG3exS#*y&P% zj)`bkqET>1lU*>Wc4X*(wS|}nx~(&(5Q5=L;e4zCF?|PB;Owb8W>MF|__99>JdIh~*1PWbet3G?<2bi5jij!ZZX! zX~rlJbO9u~@TANW+$3o9ZVA*evgly#6LSiLdnmiP$YCNa0*eq~_JWdhZiwCO(-ev^ zLrF4INmcX%wZM7${Ug8C$Uz?x;Qo@pkDhxcBRx(-iD{J_wS7qC7{Iy-jy7rH9%GDZ zZ50p(Tfr3#YHBG5^>c=IHwar2j7pNdBWN?KQeS8^-vB%HPlk|GaX!qA}^d%!Z8g-zO<8FM- zqIHBfjbSiiFtcYEHlfeAL&BoP%7a#>+rinHRBHH}=)kC9#ZD5lf^ERgSS$qKg!>I? z9qu54u4X8014S#^6Q=H6g=V=U;I>AsVoNU2A$XUB7v5f!<93y`Gv13i+E@Du)1U3q zyd)PG&*e_Iz#Vk&bb{X?N*r!}m^tXOnn&)v|Ha3MINa(!WVDD4%`6iYLIGi^oodNC zC&4YsNbsP~OaP9)4-;j&DPSD3_8Pfwy=sqeb<1$iMc+e&HeS@}Ym42CU(3>zkWesa zsj`mvGw6yr89?18iZb|n7I2N4#(WJjxHsTdgi)RF*rbV!{v{rrSr~P5%8_z11mlif zRbH!HbnGd$(`wKZzfM}I;k-xPak21r6`ze_8Z`BOY_p)4XUgZ)z|+0rH;wxsun<60 z24EF2nIRGE+&A()*ebTht?YR-szrixvFWD*4LE=VaBrI}(hex{%s;?$4sbQ$sMLDOK;KIq^~HOFH>qBvPFMA-f{ZYdHqBE%W_#@(VVV@^f zxzMutVen3ZZuT&4)8?lKFKAqEG|myN0Bw1fs@SWL0i8A+ZptDYMPiv;ZTl5&DqSTT zvnTNB;0LfrAXNm1zGOD#=Z;=xkVgk+B&hQC-;RtV7Fv8G0%(e*;}!|c5_(-qT1ZUQ zbtJwA%fX1AjP?o)1vrSD>}_WZd9&!DIQ!V9>{o%wNij=14hGKaX&s#Z+vMPJZz|yX z7F~-Tvs0^sG?FS5$(fZjwC6u9`kL+6I=o5y?Ep@$DbQuZZ?CM6%Yrs9M zj3R&3D}QSovh}|jI}4ylx@y z+6!sBFK0zKuk=G6P z-R|rxFwkVm-E9nfC+1y1PPzZOiJ+x_may(hfx zMe`xh&zt6oLxbGsI_>B@q2cY?Sdf$H{H+@TimWkLGW4`=!z${SD5=f;=&c1hMZRu$ z4*1*4kKUW18Xlm(%q3b%XX(Htsb+B`AfWtjKtP|ne^yTTm!0%W!^>uM26`{7(0xw);5_U_WHY*_|+hQM9XA*g307Ow?7*f51+FVe~OuINhtn` z;bz%E=iR&Py>foeI<`_9$@fGA*ma_SVc#+qy}RxZ_KCRXAw@2JEzoZ>%5B5*KCJ(p z$*91TZs|tnCWYt@v`c(PqibX9R0KHz3>L!N^*(N@Jw1gnCxmFK@018JA=OV?8EUdG zHLP$1gf52{#=F@#f<~tlc%;yrfz97I--i4Pn)nwUdNvxS|yH~&V=W*aUr_VV> zK@#un5*1y=w}b@%Ec1-m-Q=w#h?GttotzF=o|5{BL(}AHlDcRCWu5R8dc<>74VFh; z><=5Q{eBWS!eB>PtahBT@vM39*?q6oh>a@oF7-mw#a-_5?n9P%WkLs?#&2^iUuS9_ z1ny8}GQ!X{w5kXTK|5TMh;d_#atuGpwJWdr24JRm|e|D+PrS;YD$ekBh3^7Oca-fQ>H>y(_{$;~?M-RA0YrqK6L2P;5NQvEf^~+NuK5`;GZ9JReA0 zAAL8$xhB%cktHnA`_Qx+bnBo#0tURG{SfiTAxu>IbBrMM$t*%gYNfe2+o&f=Rw44i6Ye6*NbWCwQF}TInQ(?-`(Fc z@jlAt^fvjwyeF8*wDR~Y=;=tzl*Tj?5d#x2=v*~6vUAbO$yk;;2PUAtouuTc|gFji9p2w#1 z;QiOZ0=S_Yv1~gZBO)|c(P|$d;}gqzWcj|tQIRIRE5;o~5?HhI-h(v*zw)K|Rtk|` zo(`*g{w`-2JjWsvwnvRzjt7T;Q0_4<85sn`u&Ilv(Ij5G)UV}Vyh>M+`%h_}NFE90 zr93@K{ie{?ti4lI3a)bikRCe*ul)@*4I4Sp@Z1s9nXt*4b!bBrcbKMc-~7m$K7s|M zXYxZI`AFv}OvNqzn<$Z{H>8}0#E11L_OUc+7~#Qw?TSvkYqn4!pbFIrZYXb6>gQw2 z`L1>&6fo!{7>Vik;x~a7c@w%GlLyb`u*p#V1|BbVg9|F+lKR0xARPrAszo!+Ls4j% zh1OGa8lLDFp-geeV2}Jhq!fv%bwH$_dwVrQ6j0x$Wi`MT<$%1OE(0^Xr}IJcctCN5 z0Jx!1M}>y}j=r*L#Glo|BHMptNtRE7S*qrKmaw*r$L8byI?EM*a6Lr~D`m(>iTPv3dREv}g(zH>k73|j={aN9C1DJJJ0 zRDCA8gQ$vQ=j}C>k?=hyAtm;IT|isF;09Z(6CbK(a>fz>ckecprbKp)ONssBu&k22 zB?_ENkuRN~xe^msF|&_?wNenINRLgz%!Cv=q-mv{5?koyUpQYPrs+ht{!AS(t&bX+K&emIb-Xe&NS%P9N8ZGS-cE#dmHE7xDrk8E z7Ej%N52%T3m%xe3_Z8^-D0JWlS%-`lpnCCW!7z~ty97Ng3hZC->~8NLL8r{Q9gmXtn;x|LKDRHxlGZ$9Zj$`on59Z#H+e=CS=NAJ247_ZL)HsPCMw#BEKhYS$vT0|UJ4?ihYVObGSYmtG5efd`YaHtK|aOnWx)>TTI`-=8?~_pVdXAWf8g`^62h{LlH(AVALO#^~+JJ2g$% z^x*4K;Ni++UMXCyzZRygp%OsVG54#sr5=%^3)B79rU-10`=ic_g|ct)APQm}j^s9u z(wZ2h1O&ZsiA*TV%OVl&gDT)KqkN?kceCf?ZAiU=PKrehrj`+#)>;jin59t6y?Ff$ zHJACvqS>9ff3eqxR_!QZc#zBRIC`_!i*qf5JgBa6yzhX|8WC^ROuWwT401my)#M3E z*PoA5*Xch*=eKIdp2voLhWVXS0lkMn76w~LM9?w<% z7D>-yRZznA!6J8Fgf{BgT-4RDYE9=&{H|vOheZ;qNmOKEypl;!EP&cI5~tQ6MWZBT zUD>Na+iJws?rsMSWJfJ@G>f^t)SMi}WZFkB#;9Mmbh1Vp<|wIYkWkp+hhP9g4VYsc zD|1T&w&2WgNqQx2gZ@m|rK>VlN!|?PRCrMfuzpr7R^lYuDKYwDw`{ObDH7rfj}Q(j zgJZrY?qz0BnN*r{_Xb~l4n&alW{Fg)w+sW8UQsovdEf^z>20%BD(m?HRF8awk4S<8 zlRmqC&*k^Us>8}3Wx8{PzbbOT8S0(g0>b$0MBz-nQ{-Dg@VTOS%xq#6g|)^b#CZrUiWQGpZL2Y7r;q&O5B(OQkL5-= z9#KM`$cKzIpXn`b6YGh+qY6)0vUqV13)8J{@W3YrvEb6|NqOC9Q@6xR{U+bFL%MCC z^t+9wxh~AVAhARNI1IU5?QI~fX2#)rMV;B5I#p1nTGZhtv^>0w@r7visg5gfv0)OR zkIi$i!|{FvuBTPs?!pjqEQav6vrL@5z=}5|?iu}{1Dvoa*}?<9lIxEWFP;boK`gWK zJ{6Qxn&tzUt7l*H?fGK-arr$a2A#T46DznXq_Z%lw+SaElF z+Om2FNUCs8=2!+~!Ns=I?$wgLjc6 zAj2kJP|EJrG%1%aj@3XC?C4uPL){^sQkc4e2dTD!Zq)am)im|F5}u8$XqGyhdWX>M zL)EL#$0%9fLM5}j#=SYy4Ln?yLslho- z|5icmAASdJb7goz5FnuP&!rmlKm86m2FAu_pJmgUl%yh78PGZ)U-1>)Yqi#D&$N?+ zzOK}l@MA;KCL#sqk3{t(N0PK;ygzfR#q*$40Q4#(^q)Fs#!U^A#227;>%Uj&e<9Xp zmB$jiS68mmTlzsBVJNpUO1j|xZ323(elPFKJ4We_hlbJoF{K*8m#QtruV1crY1MPa z?rMHW_f?L z>dhg)&nJ^AWTY$p`qc$Wja)g6G*}PHNTR5_jBQEoyDbjGlK$%z@zjLAqBpPT4)Kx` z8{w{gJy|YaDtQIkQoc3`S|tlob5OSeY}EZ#zh?vzxVI<_M4PDk=X5aCNh5mioveO{ ziIq>XBMYkyEuXJ4w3u~`$dXM}*goP*RT-ka1C z|F=nbcM|h<(3)a4WS@2nlgsv>Uu~GOY4kLhF`c{;9uo`&?SmGKu(@gGl9;`YOB zC-l|M^}OTC;rTrigU40zD+{wlx%KNiSF^}VL07rYgU1{7S-v}r>c7BESQ#C3c15hf zDzf#|+QD^nSv13#akN@2Wc#q@YeJw3tDT9#)qZ^av6#V`=}<#uRZi+|y*8Q3zz@jmnUh+{|IW1YX;T`<`{B;9Oz z%LG2FYkFV+0pb1IVW*?-WM*k-WdAw*s#Gm(aE8&}=y*T$wm}Ff$3{Ewn`nE?EP6_D zaH13lpfvTry5xgX$Zz%6C7g6WT&xp{AaaTa%t8oWOr|*;Y0KT zXw6x&P2K2)A6LF}NnnjSGNs|-Tk&WCzaQ!TY(MkHGk&~S#kARcXYjdk0jrzCQPK8# zsllw#c|-r`dV9y)*kiTvzIhu<@p(CXUC&^>xqQFcxwGh?kB+HspJ39|@=CX9)@xto z!h3v(Z1?_|q~CQu%Q4a72wQUoZ06+F6g**^t~>@G{S)dmp;|}#yt!nT>C&6!jovq5 zCiV#=na7I0i@`qALI&uGmAr$!R+_WD!l~u8as?6#A^%d&6PaOh4GKT=<$gads*&l^?YWkR&lE zrEO(HFGk&2-l}&`7g4kHa%=PqKQ!Ao_2G?x%`_mlnEH;hogW#IHiFKogMT$s%Cy_1 zweJ^ym_es4tY&esE#iKN##w`vabkHb!O!5mv22MTu_OtkECX%ijiyNt(PU0M+z-7Z z%tfnD=R6y!pVb=tShZxHbaS@zKHSNVX`$!FE^&*Wr27hQsgt8yFWuJN;r1fXLbXnj zM%USX#ESQh7%_NaqIm~V;=8@bP+A}y=E7P_A>i4ZH#`m3Q-g1+)IuNB>Wd%5a#?r& z#<2i@zyHiWcqx>6M3T_&vc;sXeD*s_Vj`U8#1T<^0n)||%UWEEO@#}7GYA@&rghsx zneq$x6_7ol$2p2$WdzZ%AOR{L6gl)>=MCW~guEI+kqHnX#FoSafQ2rh@ZU3=r%Eur zW}AzxBqT$Ur#1>Lvn?pN8wh+s7Tw2DXcja3tiW?5Ti-Z9B1P%Xt}i}XL7-)D-K(Yp z;uje(pg<5C#mF}UX3wpW;0Ss=&Ma`(7_A@G-`m(l66qAaM=ASL@YfzRMngZBf~gP- z&7ov`D>HZ?c^OLm1Q9N;ZqPor-DlboAcVy9Dwuq?C@+`7f>6{82wYNwC6jlhF8@1u zYaft*ElDGa6X7hTEE1(HCPD(JS^5VWC~YQ-d2cu zb_NqTfCH&eF+AV~Fs#o7V3tk&)df&k6VA5aCpGzMo}Z>qX21`rOh^Po_N)sJx)!9) zF$#WQA(j`2QtJqzjQ8_SRUQ&`h#;|AC_FvBY!Ig-Wd)g{ohaW9ejwgaO_9?#Ib9{g zkSyRgNet#L*n^aI)U_#B8Nr*z-sv>zV8OceK<3Te`wbZa(WlO$x!4R&oFO523-MiYZ4>@1j>Mw9JY?51=(-m@W(jR)sYvW`tpWW|6X_=>|x}^KLP?eYrPw zLAx}t((oPJHRl}bsb!4ilEYo7Ful&CfN&u)4^Mu=%_^c|V9N-PF7Ln5MQM&P=C_jP z<&ov4X<&m?0=`GiNGLg(W(%OxX0cGD8WoX+Q6)igUE3%mLW=pqQH)K2eilv@%M}J& z#}dm0jF34?LQmckCB*9XU3b86I9)jF@8uBL@M-c!YhWeFgnB!Gu5>x$t?9yQg=MtmFfFKMlFz3S=9 zMH1a@KC(9>;1cN7WgGQ`!A49hkg~1Y00rR0kAr@5b2=}$a&;6|6v&7Du{e;J+D}%3 zgQ_yBp+Fn`lcG%>Um=>2m`s_|oSCqB_Q--cEO<;d15P^IcC|^4J35 z!>TYav&D+8%!DV@l^AA@eWU6fsua=CAbKh{lHm|Ax}+F=rjT3#{^cx`qjs5o{_)`1 zJBA4=4_h;?cd^o^s7nOl#ALcCp7v2kXj5@9wXz%|6w*+<`xM@P(Trb{prmry5M~a-2Sni&;gDosRCR zpd#CVV+&5ZeRW7M!cnX+Aw#)Fryd0 ztleYC#D0$9r5WOcm~qInkU|)pfCT!`ZBh$x6*TT_%yt-l(lS5*pR$Lc6cIN5y2_X z5qWoj(U-K#63cz>x68vNcaS1<8=@PuoSi;iCYPjlczG`9P(%lI9gGifW0=3dT_n*+ zK}#0LDlswgi!s31d&=4}NM8Q*lRPG7c6r;ly<*h1$;nZn&v>HIuGCJRo4~ftLn;DJ z;#13#7RSkcrZwsgdD%@QKXIMypxF)H!}!LUluVi^?BE>8QaS-9=gTK=Hc@o^0K|jv zC3ruKoCsA^h%rfyl9h|^ByRUL@(Pw{YWtBZZdc50sJxG0P?zE8fvGAQi5Rr+nhnCP z@Wk(2TuCu__`;|AkoZy@AYI~s$KnW9{Yti3>OFl=>`!tCAaphps!h5Z)wC!Q`R+L> zUnG51Qwv0$htR!sIRKfjc@Bl*=+Q?L@cZ{wX!^`0t#wPUg(63SXp{%jLoZ20X2vpBnJa%;2nsDa( zYmEXSie9(_m<`uA2;g&S4$=Wd4vDDT9K*bz{{P~4?{Ij(~ zYJs|`@_>)hR>?{$9F-MNlMLAy*35%IC6Tq~?KDl*&S)3N(p3Pih8gD$Ojs?Gr@NsE zha$G#z5MHhLi=5>ID|J+6VNpTdYnn^ZRKxSw#`7Z3Wa($RMB6O?3E5GrF%f&DIN*C zdd!|vO`VOe#56pibQM|ax(RC3DAcHJw^={0q54#R^09S_F{gto5~l=iKjcYqd|!9M zyPVq4e{qy)en@ax!rM^c{_3`hxQUfA6LL|a<(-=UBkSq4*wrOk%OvDzb!Mq|H>hdG zKgAIuQ%TK)d!cv`yX?L5F+qZ7q0-02gt{BlVqDT^=t3ROBSRa0nwa&cxK&0RQ_Wtl z+hG&}IVXXXmptxQFD^#1-5Z#YcKVVoM<`nm8S(1__+A~MXmW5gcZ5si>+g~T-C61l3R-Htydud9>OMNUD1y5 z1)5z-0n-?l-JVVKGEq|?CA#k^rwqrMvV3|fKMQ2$TCG>fn_t^tY2$5zB0LVGpX|~u zE%x4!SawcXUTYBM&9q4GW^#SblW&|D-!{7cnz{X>? z7#QAi;Zp5bdxuZpN1(2kQgSDpfTjGQd{Ktjz7Poq^!GYQB9Hz`Jk-&6EkvzvT%zC9 zuHu6Hu)>8YrC~DQrjNbjy9u$l-j5xizhT5OQ^0{FTH-no8@e<5qhiO*-XcY9_c4>3 zwWf!s^GZ_g(L7Z!d4rQn76HblW)G?>D0R&WtlrgFI7Lh-wedxp4|pq;r@G1VVCqq;vdyMp)c zyKegN&HALtL%*PIWx&%ncgrMvlWTUF!(H2TbM^+a47xGwjbeCmII`?;b2Px&*gEQG zF#3IRK>nN1`{gE+EZ}l|#)+2Da2f)3iUrO(+X`{2x4ZQ#;*A35ZirCg9H6`HirUfp z4DY*ComuUTjTy+}1L$kk0@%Cj{@l}7iK)3rjspxg6yuj8oulKOIe!p_=Vd3BeiYVT zXkh3==aZ$602JwS=_(w{)(=)Hw(Fr7YV`T#oVbmT0SJsdmC_BD!`|r6`pGLYt%xz; zz`@SVy-g*@>(8wW{9WkRS~^BrmWBRxcrhU* zO)0rc`vdSiI+_yE44EF6`}9APms9*C8AELc9tM*E_=hSm{iK=ViR9| z=6x?c`F;JpS;PVKAx~ZXT@+kjB?V*#NxN$u7y}id-X1K2@Nw9-}y4&mwau*LXCvgEGy(O|EGkS5`G0jsK7ol~a3d z*E19P)lUWDHaUrOn?P?#Y%oZ_c1JX{O1#Tg8S+Ghi*?}bEwMmmq-dOXkqgstONWig zwr`A>Hw_QjmJ z!bA@DIn(TIxc(i}0bYY!6OI~e+I*pWCe5;97Jg-y)~O%vzBE|P7-RQ_LiT3h@_0~5 zo}t<%wVU=GDuz0mJj9sah6q?&jM$JA^sJ5e2U&}jep4OZ(m)&2f9vjFhmr!ghe?CEiZoA3_*-=DaM+5X(-&sUw_ zr>R8tA76D+LW+EXe2RQY31ik-3`oHb-r;V(K%FE}drD$NO>&b``BrUNpbkAKT_Y^y z`{x#*l!WF<_5N3$-EUK-#u8ttZ&P0hP6L&3xeG6s5lHPAI&B)oGx-hN?;%}Nw2M+I zj|v4>9oXQ|8h1Ab2>AnorT#>t^j5k<0}_xp$wGP9qQ#1<91 zowwWU(?Q*-l3hvk@nD%J^A1hh+41}VaP7icp&5A-Sm2aldf#uh98fFefM{_zhFzbp z4LytS7>TNTmQ;!qfX$hM6{C?=@&mMs$5KIF88X4mtezP%))dP2xkYGq$Dj8L9TnXD z%%n|z(SLt>D&9hZQJ06Tu!xWswhk!2=+i~mW83Aj9=;PF9B%S-Z&JT9YuZU?WEm96 zEB9?ns>oo%a6v3*ILdeO6hH~ZX( z|Kpuk5fu`Wgq;$VnwWu?prx7|pK6e2m}lPFx0jiapcSV9(AUe0iUVlq!ssC><);~@ znOPQ?=l1qtMyKd!?r0X^DQLwf0Fs}7k{G9y-rt82lWdU}rDJHG21ngbbGr)rNoh>h zq1mDNx0u0ysABt};)hQkDiGx7M*Sbp((yCx!qUju$db;{)scEqf`+1W59ALbI}#ad z)u&?j{2cKl|B)y!Bp@R%7&Hfn&sRr+*z)=TyM(*NK;!NQ;%}5E31ba4v_oN=3YpUt z)w+0c(#6`RbkX$@Gd@xcbL#$t3|7kTg;aSGbaWjXye1>&R2*skXtkpQc3&D~LVFKk zH6Pe2>hV!d^+3{&u!N)PzrCgLkT{B@R&C4+ z%q@Fdg1o?T2o+DN zJ-;e3ci*}9Qx>4J?f_Mf!E45xWyO>-xUC7GZE1AgvRA)%f{Imbg=>+Po{A|NJ%;RY ztn}9BxYOC#Z^pzNa3&bf*&f7|a84|v6J%FR!Ws>=tHv9MTTAi+cl10w3o}~ohuMf$ z6iqaa0D_r=a?bz)^;6Eh_Kde{2Ih-S8sL}McY)AAY8OH8d2I1C?6V2@9XsDn8?;VX ze@-7rGw|F34jRK#mVi$u*Lk)RQ6~o6egiSlKQaecIC)AjFS8y1G+Ome%qxFca)4!;(5@ytI4RUia4EO3r};JlV5_K?QG(4 z280gU@7BqAt1|I*yn)C{0E3_b{ofmweUAD6{b~05$L@c&FZ)ye z&rV_gM;ZvI*>Cvo^8e}^_ILSz*TCp6dE3uN@Sn&3uPEUE+0N+ixPO;)^%oA7?O(V* zQ?ULH`FEkne?d6-{ssA0VDg{Re=aEgC0#G~hx9*}7ypjG%s(qV*^0FTcm%DgSQH^A}}T@BecK^e6I9YyK~!ron$g{+Dh4C-P5o=0~KNaC$%n{T7#QfiC@lW8NI_oc>^{2A^f1l)k(_nvs|5RQ71UCZr e{73NrQDm|b;Ga_~ARvU#FYf2lS7Z5)U;hg+TJ>1~ diff --git a/tools/python_snappy-0.5.4-cp38-cp38m-win32.whl b/tools/python_snappy-0.5.4-cp38-cp38m-win32.whl deleted file mode 100644 index 3e0fea235838b3633bd3c683ead2c50db0923c6a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28183 zcmZ6yQ*bT})T|lXwr$(Vj&0lKj&1DNwr$(ClQ*{QoNxG_nX9gK*%#f_wW=OP8Bj1Z zARr(ppd;!dSvC7jt+f9xJ76Fnl>gMl-pJ9>lhMGy%HGP=z<|Ng(=JXJVUP&v&sw4* zWNIcBV?l2J+20_LgkslmI65!yh0WBcrp(`m-=jEEIGFz9ik=wX}a{@uvdbIOGIc?W8Pkv6{krsr!NPcx(8gLHQ6|@&98ZdpUdB6BGz&68isV z!_LUc{{OU5qAq8%&4kng`9K&VQ;*Mq5ZD?Ht|F2oL_}WTc2PPuoT@7&-x%`q z8cR1}kToB6Gon4Wl*P%KnIJ=r6XLXSF0UMeY@KZ&{wzsXE_Wo0;Pig?1f^0rL}fZf z;SPGB)xEwVeFYJX)08TTx{de06g znmI>r(jz&!E_=K!DE+zoduNU*jTuw443_<5^O>1aP7-BFUo_AtGN7lCstTe17!4K) z4*yiRRD}K0eF7v~=cVZkCqP%pSOgZrXyr`9sVCaedrrR~sjiqJvuaJ@f#dV{Sy&`! zKx67Odc4~hahaUvWtKzPmHw?!9#t8wJ&;x6!Yle*Mc*BUW1j$B>Z8ge5yOgtm&f9a z6NS=+go$r*XhpV8UX;E4G*cbeR0U2znTE#O*7eNH z^V-$d^gthdaQgT~iCGWa+86EZcC+=}7cUg5O@uxMYMkl)7R#W=#D!KAcfTr*UuN&p zR%)pzp>BUH+^*O^DodQnE+#)>yVf2|CCIey&{kFfB{2BK-~+?Ii9dx{JkNdPg1s~w za4{t(m-38%KXDgS9e2)o zmhgBz{2UF98Pzk7v@eF7vau9L(r?4uq>&XVd^~(~g#e2lyN=5Rm@m#y+$YYJJQG6U zFS5Ket&JF@rf^-Vx@%h2oQ23w zU3=|nt`5LRNyB7o;P)AD;f~Fp*~>GF382<{ftx<^bIO09mU7#<-#EG4s!(HY&g_Wg zrN;)rf@Fg)f@Bagcc^zCv6J!MiGrjbL{9xN^6R%HSKcxW5hTVpvG<|3TRow)^OHqD z#B6T(#t76lnU*r?OZ@wIM1LwsVWYiHC!`H9|Tnfj%Q*l0-u?LW8Qy2{-1jX9L9@ z{mXGvOK~%5>L%B?!S}+cg(}C91s99VE*{w#>B%4!V+G>0&>l7FcEi2G+HL%B>-z=x z{`uVbDX*tq@C_j6(q5h?4%kbZ&aWwq<0 z`^~W$X>r5o9og>Q)Lj|Z&|qh0*LhE&*tNAexXgUJ8u{a6;_{E0f{aY^nzYeuEqnu0 zp3A4AzhOH8_`O*o&izQl@6sXG-E{uTGCbkL!JV$~y&)EFT)|FoH$h^Hz@1NMDx8PK z%5PAo&Pno}x4FBxwiDy?oj3!IJLG5zt@v}7cFMP&;DD1}6Js&~i)x!;pjIp&Bn)k2 z!9q~vtH;iVd*eWjp=LIIK|Fg|Y_cSVmc*3$s3pFQq&_r$Ad9&j_{XvT0Xp(4p<(}V zEQT0O;0O5&tEyrJ=6%bh2W|xJkzXt+9-a9llg0*}cmkC?(-ft+eZd20HX>nOG84p} z7HS3lfe&I1-J~_WDpF$=gK@VoYo0X=C$p-j(c$jF{EcAuixBz#*MPHwj}avmsR1vn zV%qm=yvh@+P8F?0fYgL&-M~WO_yC>za?O6v9R4 znlsv#UqlJ~6{O}qZw>85=1as)GL1w1jtTtk@bF)zLl!s6>0h+%6!fVO#;Khk9dELW zfYo5hoNIb#*BFC~oE6LX;vyH5U_-2e0LSZP4Usi!CHMYP7+JvG4#Xpuw?)7=jl)f4HC3Do zJt+F%5WxnERN9N5A7)dK<7RNIty{5TwR##{Ft$^)-N>vdHbIzOyeY54f=3CRt~##OuzjoSGdYGC@Bx znUP2eS{WOS#saHc*kjDdWY|uHjJ+8(52t;1ij$9ErMLbW7Xw8e3LOFJwW>p&84FBn z7lRlt{XXy_C27#uoC2DNOqyJfl+i3(i(qG;T}y-j);!HzlO&mCaG|{I&#%9*#O+Wv z*2;G)Ztsml$&OBN{pG9o8fdq3Mtq&RMzT#@Bm}lR!q6e15}6Os(8NJ~8i;q( z0>s%mUOhRn9i-BE_B2ebTOq zx9=s(p)V}hV@_BC#` z;XY+``~(->022o+fG>&|J{1U2L zx{4-9YAjb@0g1~lEnBeAls_C1@m?etprMT%Y4k#u7yMu|!{9Q#_M>jajv* zU4Hct>L`^jZSz)C^+clG&p)cZ;0K{XRNTU{_nz%5FBV9(>LFQg%_x$=;Dau?6F8}-7Jnv65CmmeGlusNP_*NHy=Rv97LYg z`dVFt>x>!I*ldWcNg8h}Lbt_HGFT18Zg@RU6Z)~Gba`BOu(k&O9ifh3x-u@~7O zo)P5r82%Kixc=&mRchDRV~HyW7J@A;=DD#1QlW!0IA>Q-a{W5n*L78DucKpNyxb+r z)bSVzbNEK@uWFcr?91p-9VO})Pb7=gfT6#w-;I^m&{UFp-)E{md*NaIb(1~oLbDt_ z-;H+C3BdbZM`vdD=fIX!{41ig5wb8{681Vo>;~MjE15)rjaI+D)&=0Mjr{CdWEd)6SY zxI>!q5~#b1LI7L83TLu*A&TBi8@cC=I)9F>@#ivtx1m^M#9R$d?u_+=bfPh|lLGXj zg4GuBu9RK9^DZ(PF4>Kis?Ldy?jAe4r!ICQl_Qeaa(XaS?r5n_9dXrG>pLoYu}UFRskU|m-60UcuI zd+qSnE)3kxwcdk81z6FDVqT6XPoOBG2@u)fYc1U0o7tC}v!wP`TrnjM3_NM~X`GIE zwY)}Vn{iwAm~97QT-CfrupVyAsxP4+Y*ePY1jIS~x_3s9cAm#O(vbKAFl}=;V_VN$ z3K8`2n;dztTpm14Lm!;7tz*2U;TR!OdbKU4Tb4ibar`y%Dya zpKlUUWGKqE^V5WBjm=)hW841){R8K$st3extVRQvlM}WExaHH*wMz3h8!D{1=}u(+ z*w|-R-P(7;xNb6}YA2H|EGCC8Cv`e{)*H{1C9vIrSo3PHqLQ?j23>%)cU0nY^J$` z9Jq3Bs9tBRr&?B^xc$PI-O_(e=UtV#EjWXV{xbei3d8*kvOutr4ri)YS%W0b{MQ5d zK$jC5_W*FL%2^3t{vlLzsm9hGPF7<-k6Nkx3xAn1%t0_D#wTjo96Oq#g>CKc-zb&& zqD)5>WO!FDrb3KXe>+`~7OkBTgU?+j-zE%9j{Kem6>|JlmXE!Ycu z%Ls^j?Q0$MT{*E+Se)+2F^|)+THO_CD#NjS5uyJt_x6B5BaU#VW`|HnN`b48aE47r z2q@x1ubzR0-q2Nn;Ov;O_S_*LRz2>y5Y)tllt;cWD2Z8pg*V&!Yp(;{i;f56{6i)c z=_G-mhTi?u!eUm7d7@A$CTg=S2@Z^qQIkv-#YSIqGMWWFQy&x7x8=S)xz`#q|S+_c^d`WIl_O6DVKd+Lg39Xl*m^-fQ1RTxk<@BLy()7w`f(5+g z5zd@BA*`Os%e!?9i+;)ITp6G-A4ChpN#;JF2bmBNt7eD;G=kU2CKVjIFcdSmndXWXyS~hZc|`jL0dbn7&-Ce`m+(K%#_-@o zroceSs9RjZbr88%l%@zD^QX7q)qQ9{_@a5QV=It!D+Jg(w~A>Zo^$A&xhGnp*OXt~ zY)2yRLS`d`_aO_N(usNJ%K2CAZ$Ml|HPK9m)t;NAG+J$>WdJ3X0Mt&y{@p_d!l$;8 zt3mUZq3UI%IZ$`fqsFYZ*fzow^7=u@)z1D8A^vz}G$SCPvfGS_Lf{$M(pL{Jg`!tD zues7!@TXjbTL`bZl2`PnU6I?Y^PaF>-m{*6yPWf%-%q>Jx9aVpa0{K{Ij701Kd5Bo zc6x!rYoP=FNUitR^{_thWC6!l!aR=`Qg*H!wfg5K9DFyqb~rSQV~ytR-M~`|&2%B8 zxky1RB7=TJt;2_>3DCrvdR?eUGVWufrPHE@gvkLOE#631o^yVVfs}y<9b^HNz)CiP zsnRWJM0Gv#u={hMW}m`t;Ih8rXO&>gv=!Onacb}lPLMU`VDKSD=S4z#PgsjLw+-Of z{QKVIM08-?+90~-P;0re)ot*idy$Wpd?Oc3j@t7LglDU}gx99aVcKgx+Ic6?0O>7b z{_F?((huo#ETTgQ=E(!$FlLkQG+S2cf9^zZF64t65Q>LBrY=0SzC74 z(CcJ00zMM*$-EvxySHvKY(Du`R)ZPzMV;%*$8!PU1E8Kl>2VFp|CV#!7ib%{m3_Yk z2u+*?{j!c(6=^{o)3_%Z=$u;Y4*Zm1|6^{?%*)Bbu5R#qfvp=K##vyGN9o*Q;PU_; z-*FH^QORMVp0}b|h`ByTi~t8#OByVMS2+0du>)T&{41!P4Hc4+D^c%1PpQ>YOm}Ct zh1R51!K({`La3t#qMyrdF|23QxJgjvz&iXE z6kX$>Oo+-LH1l{|>m#!@B9IItexvKBr*V$8AY;>RP+P!vnUUJA`;;H1`Q*1ZlggDp zN*jF(Z~2#NX1E}$`VVqcA`fVvMAzlqD2>#w1#88Rjrh4g!-b$h&BpI`)0}|!F{=eO z;Jqd9Yv`8taVlZ5xsp9!Qr_Biwj+Xrr}DGEj5iymDTj$2U!5ICqu7OB+lfq@Rhu&x z@220PibsSFd<6#nB7#b)U^`x<$f0yzV*)DF93WfJP%cxsf>fy}N)NpAOs>@81?#SO z>;;?DLsJTV<VnUnQ$Sc>0HeQcCw5D_bJ zJ{~URwxHc8z2Yjw^;C0_UG;cbk#nF*Mc7xaJ^5t^qzbx6im7H6Mc@IE|3D8`tGu(( zaiX4GB5ru^h6i&8pPv`S(soEm2S-Ngs-6t*=5eg2`5hat)|u$X96Fc4}WAWEZ)1#S18; zyfg5=2Ue&bX=0(Yv30bRcK50nT<6!Z?sgFZQVGHIv8ZPQ&#jO>VMhYq&V{pAba7QU zsxv@LVzebC zq(i1=BLfJD1|DOv?8BZ8FVk(_5GMz!S@`S{C;Q5P1y+NaR1p-laEN0}r5_Im*J!L* zdt?J8{#gR_oiH}+K*pXhw8)T~CzQkk-=})MrhkKfd?v&sHfa7igYhXUxU+I26yS*- zRwulj21YXZgex+O1Z>RT_SV${EZf!?hK|DLj~81aG<{Mh^HWU~gW$HIum_o(;dMY! zs$NlWAA6T~^Z4ix!>kd&2eoqd^-%6qt$I_SVo+{{5GkbIi>i>NCcApE5)&El#HFzj zV|u(1uw`DPCgPGC&!ZDA;~SJ8h|HeqlO?NXWNg=WbLOfbYd!dIKe-JdW&K|zm-4P# zd?dOON0u%Z_o<0|&ZusYYhj|bBnHNC>k6BiVlEw70#1FjfM4}}61po?Itq&L14S&# zuD&at8O|D%rFwL@COQ_RB#8Wigyh!WR{pn(qdyA zKDTvl+|so+MyUR)N`9r^5tY|gx^Sv2gW9V~yNo_StUa`+Si~%anVaI67vs3e50h~N zC(Ze|tpFGaPj>@&`}gSKMnvk~xj`F{_^Q;TLHqeO%yB?tEcD+#CZ$T|E z7SNXE-^KKW7=j4MfY1$MS5it<+4!{yGMtME6y^_-<>c&vmN}^{Q>*s}@M^v6N6(He zB6~t>O=oecGh|a-9j)M4M-FgXQ6t#wNZEbwC>8Mm3K1NpD8l?%X|IqWrLlW5NSagD zPToUUN6<=pb`mq?gY4IflU91w~vXerqQaj-nl|r67p; zEo%O&uus+}OKQjitFPbY6vRCu6MxjOo+D>eoNpkcK$(?7ns~MYv^<;LHLo!~5m(-z z7SlsC$X71V;5CVOxuKiGR^h!01mIK6f1tlL4-JEaDRHQT4OXY+OH#n86G?M`8} zRU$U3=dTdqXF|k5^@qM9>gL8qb@A88cdq3rK+qecX9+20sUY3g_Tm_<{ii#OOi!zcCafp71=0x1oj~rH|a<-8!Xj_HfqJ-av=80a5WvXNX;~H3GMxi5Qn{ z#NM+YP)y@B)k;YB!zh3oa9XQ;5nTU*o1g!;=pfpDNPR9pxiM3J>616&KGi6)E=lJAAl)rQzBXj6{Bu0;nHP1zV-vMPJ zHZC2m5KS9_Fm2tP*-|_EB4XC9yoHYn{W3;9p}d@6XHN^l3u&v2pyB#HIUy$YaSS%W zG!*5`ITaD4o@@v>9OQ>+rFYk~pg_h!f$`eYfw2L)2CPu|wNt;GLfswIng`x_7Kh`+QSys9sCtXF!@|9!oDqs6Y*SXN`gm9C7?PZwT5>K`uI zKG{p&Uvu)_r{Z|9Df!)oVJZo*@K$<7d8&9e7XwJcNvk(G1At$8xj3rlv2IS33t@CWhfyQYhY@_lj?O46wiS#)QPFV5kCwAFA__}jL`RT$|fT#GC z!n30nR8wH*cf%ljOJB%RjrN8#l)SIwXTtaAhWE!{PQPx;7Xp-1N=8MP$pQ$vtmnZx zY0hC3?@o5VAz1C7eu`Y4VusX!u7GQm4eBzb1KMsoKG{{Gv-^r-71wSW{w3-Dl;s79 z1)p`ey;;GYMx+>UtR^Wlx8(IM{^3BVAJ)8(#VzA4XxutK`Ex>-3=R6?f_TQ?Rju;;DiV&M-r$(7 z@+GOh2#8;Uht1?wz+=D~G98UUQ-xE%NS(_QwK^oED5u9+@hNO>9u1M##SFgyUspV| z>(RxZTi(zdmz~nzqPBbR-xl!aitQuYx(qjKox9(N{ndl^1%^at59g4dB+H8Qr&{w8 zsOaB|IQAZ^mnv!4+iO7)uDuUiOAOb)ZziX`wE+B`d&wV**b~BZNaR8_BH)Kx$S+(o z!T#TwFUDv_1H%sb&WtLhimzcIuoaK*Q#Aya-_D?&7-cyB}#o39Io9i*Uw&uM7yMP-bS~XtX z*!8G~SXuEiU30VNzGbw8K<1- zGlBZq{(qwB4dsfaMTc*MQwWYvH8I^w31gBa#yB;a;e>y&#P4Qf z-^x+n?-OsE#pfLe|AmR%Q+t({7K;HBs?jXqt)xG6xuf;5p&T$k9_FL);Gy1D@HOLJ zvBxdsG=$+q!x{74I6pdK%ate(@$kd}FDyR^L6Ku0q~T8tt^bT&AhrUQL;9QRvWFb5 zmS=S*l9fT|22wunnQ8|8n5N}kGe(q)`k)9+#q*8 zNVtT!3jqHeKc5l6nY_ct+4q|tAKmj!aUb6!sLsR7t3x9>A&(=0aLsk`vqEw@>^Q%( zxrfP3&)szZ*M8<$5x)qcP03o{&~s@0-k*w4L>YLIy0;m%P9rN9EmWb`1g;xusr=U{ z=gN08GkZjaEP3eL2uetwa^YCA^Su_hv;x&$vtHD;hM#`cru5!#Eg1J|cZ{c}n?3~E z`=Fm)bgYo`2N~*ZH+64KJ}~X|-y@X4XLa95s^Ye)FX%-h zNs{8|_+_(@;bf_HcRF%Y5@)_63!)Ape&};y$(Hz-c$RK1Oc|(mv_&P!pZU|% zkr3VR;*baY_(h@}a4T>!*5W+TP2h94{tZM=_XaVG1vY@c=GWcvB<1$5BQ&C5@*raQ zI>?|ebEKpDj(B@wsdrZ98pnOSIe4b;S7uCL9_P6$+ko$YmF4lO!2xhj?Sj8wwfCU&(Nu=JLUy2_u(npxX}EIakR z@HWkv0HJPQb2aw+H$tHmHTad*Fq$f8oR70~>oD|(+dFj=oGJV#X3>i=jo@!{&%AdU zM%He4ox4?SR7SUc&gB|5jf}OCcq4l#YHTk8d+wmM7Pr;StVOJo==kZdP8wKiGq476 zR5*J%HW3YJZ7@;cW2LB9;2bPv3+97#b9gsMXF6sryAujn1KZY&w@$U@TjVc~@SfR0 z6_hb5;JBk;H8}JLPPO%(eilz?O#wC*}s9 z)x_$K&Gu?S>~Lx5gPOSe`;5aTDMlYnYlnZBy^1UwM)w$n*RiA6wbNzl%wpcQ7ACy! z%m;#Es;H=hsIZ=9$2dtFAW`MjPY+7mi)3L89pAgr8rrUTdLOHfQ_B+mTj>>23Yd2!+K{Yc$IesQWcpnRBl-sPOpQmtYWVHOaEeKYyMO zobh>tzL|69G{4ey6tJGSk<0t`vDl-qxo<(#NUQaB$|65~b!fxmNnA|5%i<;QIzGq< zOPZKG87#z&`_})Nd9i9yQSjwzs_(L0uzzk+d6~{M$?~t>n~{*m6VxkOYO9g+J0!eL zK-B-Wth`#__QmW{9?9eL*CgnrN#sLu{z%<0mWi- zT=z+10@9ys#&h?)A6^0(Dzw`hm0e$Z3a^5@4|hAasty8QPmCIZfOr)bi~e4E!Zd;R z%SpDw-V5H19{!Cr7JKN+naziOyE$F2_hA{wVhg)lvok&yU)wg_5P9u4Z3U94hw-}#284`i1&841QOI?^4+QprAMi#2Zp0o%{5$|gj) ztycHTC!OEk$=_I$i73IJ{j*T-S|7*zV09z1||1zgZ&gidC#N8WLcYZep zbsHbE&J%|`SO2F2uoKzs@3bP`qP}hyzul0%Jak6WU-nzZHPNiRd#()cL9e)9W07x_ zSO5Dge~wpKuY>zj0baL@jFitZ`ri^P4es5W%`S7J^^9>DMU!IsbET+QFsYN0(@vd-mHF>VU$m7rhmrQ{&Z-4(-TpxA$yM<5}T4 z`*U=O)%h{a6`-x9+57`uXMptp!}*B%%K*oyTe7bV|EmKrhhi3DWR@NCN(!?Y_jq_b zoR{L_oJ$N$H^kf>R9_vWa`!_P|9txO4umoc;Rwtp6Qq@D@-8$82H_m&gQNro;Uw(C zbdLyk668q^4W;TVEf7*4NURG*C=CKC+M%fbCkR9>JL*)@k!Z~6i)#?<-{I)}VI+wq zA7^e~=o{qD;Lc!pLO?zfLkuFNts)oZEI=A0*#6l5m@+;;A9@>1AP5Zdk0bJz*dUm~ z5yeAN1c?Un z*y9pZQTC|ZVHs+uGY@A@p72)+Yddy#CK%O9c4nTG=CQ0jg+DK}>hA2$>>P>kq}Dc_ z9|yGh?1}lK3lu6#c-9a=7>(H7(fv+oSdc6#`k+WkNRkb6W{%u;T-Tm~P!vSexEFdVz$}|_i)(Ed}0nFHF z$d+SCHRf$(;U6$N&f$R%(VdCRw5_CF&}{@pKb)`=qCSJA4faCeE#9wK#z5sBS0K^- zJI1 zJU&?PGGHpL>rssT3j8q#y*3S)`vLB4u>V9H?|P>A+CyiDf9dcN`71g(MN^u($g=kN z`*0z?jgW{@t|RVKzgqCy5?3+gAEdG5kTCFb-d8y?eD)%nS}1w+n8B9 zV`PuYNANI6SF`*3F7eyogZ0|`c~hkonKc+I;vi^l)8b`Baqay`wa(vsOHXPvg0j1+ z#02fY+pleuZ(=!S{m}&73s8z+mYUwXrCx@M1OF}wG690c4YNXCRbjK9K0ppTg??of ztrx3ILuX1unxmzCLmQmzHdCDP18N;OiqSfUf2UuZBb*0tE7}-SLkHLqhW&eg20QQE zZ<(=oxS>_ZA=)Uf^YN1X_WzRnK7ktK$0T0K%O*RXMwu{hjF%7<*)yv1f1E+YJB*Pk z@&$}{mQnsI?Qo4edbCN%O=Cc>y!f=EPZEFhHV5k~D?_@Wm0Uva%Z=b8(0}-Tk4&4E zK|LX^aafVDL-P9;w0cbc`Fs}mIl+|j=uL(@W|PE@=a?ocu7+yp?mA4rz_5GRZgG>_ zY)?;hV;`=!H&V%IS!{D}#XwxjGY0dXjkle3$CieE#x)|nOEw(82V#v0@I|P{2r!EY zU^X*U!RAbQ_`QDYk;y;8TmR9(P^)1^U(1wJV?W{#$(Tv`Lt&HPiuIg2UC+?hVdZ94 z+tJZbJfYmyIiVU1)VLO%wPkBx-H7slZ-_B(5;r-WiBsXJ?vga3y#m%kNKFHONpsE( ztGZEHb3_(unsxS0*uhcxZ0zL+*=Yo1@xxO!q1-WSTG2HpR*QIO20W#%g7iVeYkuOU@4DO{0*29@zNk@U@6kwLA zp@CC5jiJ&|Qvz%V_+W$P)6SX^Fhc<>n99QE()H$3Syk3pS6VTxz&T~;tQoQkvstPJ zjQZDm1rgRKo3Uoe$*Bi-bl|*-W}%NzxR~gn-kpOcNFG6zlOY00F^WX5YE_qx9(0PUl!9}y?W1Op zOw~5+b(Dr=Gv?fx4eV)4w%}2^=&6C0_fqxE8tVmIcmlQ0aMU!{=-aV;ws-SdOcZAO z+ySea;8j#lTieaBYp`cxD6XUvZ=Ok+!9uD38*5nt8Zxe_z~eCptD$IzW%>5L!@Q5SX+@5} zYD!I}qtVxlN5!p6c(afqciu47){&t|6rY%vVhJR;sLuZoPh|PhwIV?_P1!IV91$%U zQPt^)iuEH;%!EqL%u-*eff%68s+{ikHB0e)#+g*uiFfu@D=lSrJ2@#8SEZK13tiYY zwFV!v9w`7Z2w6<8v|D9HN6idofzF&HL5f*R5p6TUYfWI6Z$9{2 zF>OBJ$7a)>jW}lql|`M>HuQ)sxno_io%H;T<=u3h$FtQ%5s0?0XK@9FL%Y1#1dk#2 z=8gG=$I7_GWqOH?U9?AAWld92Vcs8urNYUUXRj==wf~tZ&GaUH{v3j7heq&mBs)xN z-I`G)hpikTQN7?{QYEpP!lt&CS$WI80*6{<)A@SRUS#t_%ZNQNw?JBKu#N*h_MCP? zaVG@f8{J&`eJn>+I%8vJyVrAi^`X_koT=4aQPJ4}VVvQoF6DHyeWb-FzMfeLk>3BI zbt#*w^|&Y+v{*@pp>jAG=EjF)*?4BLF)detp{Q+=VrFlm*V3`E*geUEsjFd~Qo?L5 zcST1KB(g(u=QWzpLtMe@+nj%AP|*52iy7YMATdrhqnG)~HfKF}aJ5(t>B)7RkUXZ6E9=Q{JV)&>nJbfDDLGF)yII*OYr1Sb%Uyjw^Kx5r{joY+B3Udg-bpBx z`V`&V`S5<`eWH90{s?%II&UkZEIlnrE6Z9Ul>YtmvHDcD^ja)2oyF}yoa=QT@^i6E zCzo5Qs8ZKpsm&gA;+{##{>gT0J!|9IfZZ_CKx?u#_IFr(bS5SXR|B(6pqAd=VcXyk zVI(K^lkCQ4+i?3ImpE?madVqBY0FYeQw#H^hg~IyXjawC8p8s^in^su^YdnSv-~x~ zb<<{AOPv*9kznOV?z7;Y-;B$^YA?pNAWB0Zuwv_VL+az(3||F*GLyC z2QAk@AF9y7PtG$EZ(L0JY&m(8e5$f5sZ@edN+3(Wst19u0Cv4v+)_4jnelY?1bc2u zAn)~cWFl@Wud91;T0!e(u2*&wag)fvXIIjz{!9A}%>PX+*#E96YMh~IMu&-ufs2fb zjZ2t?mxW;i^v&?kZprP8W5~mP#cu6-{(70e!dv{c+dj^|&c5%)El0*((OuJBcCCiD zhPOy3x0R%Z<62YOU0Y#uqxpwToUk-*er{Fb^6>fnMYir*POqWIYZl0FT zzt2I>ulSDnzVHXw_WTuW-EIA^gsu$fB25mHos68C4%wVGoZg*AoTd)dhgXK#hwUP^ zklR>q-rG$KarvFZqO-h_^297j*sW)5<(eteMAJ|7SOEQl$ac0bC{?5qP zckpKpWj5uPY~-hetzS3%x|e>9U-|?658Cj5pj6fTxq=^1KtP%VKtP!PgEm+inL0Q) z8vNgYgQI7SmX6C72ZrBk%rB_AXZ$*q^M%#xK-Wce7EhAxW-}^pQ(8lM;gFDnq`GV0m0G%7iii(OvH*fH{A za`Fy?R~fyI2ok+Jk*tA((roqZPmmA8%#`VOxa)`BeSK?bz4tXIa2oB7yQBZng9FA| za41pUiV8R=-Bh>UH7aeswd3)Quot-QZntVD_a;2^6GhvJpe6f2+|EgiFWHgb>&mS- zvT|}wOLm*MW@&U%NMbLa?*A=#n*F)zCH2hF)KQ1VN4lKEO!n_|)tJ=kfe=o%NPB<*#BWuTTrn z|9RQ+KOhA3`FK7v^!j}Rs@}?~($a4Idjm81et+t&a9^$;A8+Lq4lc9YHPZTqIr~=g z{J!4TFdZVmC3Oa<*!{j=O8p*kRFb1&(DAC1CmB>*;(SmzopE7Pq-zgg2!Mm5KVc#< z9|`s6(%>fW-KdGYE9aK2%TsuI3RdO7)YIAO=i=_>jW`SV>g|T*b$_kW#a;@|euua};ALujU5OV7y7i28xZn zE0h(XQj8{sM{!QN4J$ChsAo9TQX^4_P_GRk^_Z)2sN(f63X`{^zWQUnK;*A!+$_|u zE|RDa>GnMlmVV z8t0lwHZ(hRGm*>{0wKakN=l9|u~BSAE|&4;Lw2ND!JHr?pJ}89Th?2~7Aq3f{=Ew^ z0p5pM@jCcLZ91eW?-H$g~CdXo>nGUE>M7(9}(_?o~}fb=XoJd z%Tj}NR8?T06`%=&u7>o;Z>^G7++qZJonI!j7d^M@KF}?vpAKNxEZ3d;tEqtm8ZlF7 zb%}gJg)hE<>R(&?*(O+2f0tP2ByjstZRU=?sfw3PkOk*$cqKXUv>j+eU zU_K8x9(Ium?j$4Lp8Pjm!rwj-4NWi#>S(+RBGHBjd0tb14yV&Hb3(e~6@Q$v;rVFP zhkDtj9Z4$S5nni2_^=G~Rj`}>P`agdrhT>$^6uL$>FR_PAU)Swq#i@#PvjBoi3(q~ zpg{7T^hkwnz?-1nvMoe~HiT_vEy@P=ctG?Kz zQijq#q++b!su6}JapE3rjACuIKMcB@GaA^$LI&dZ6z6Ubx;Pk_C~HT+dQ_>dz;M1E zdg@#XpICge`yF(&A0_^d@2G8aP!~$h1+^F;Wd-S5!NJz-jLqZo_^SvF3%1Dg9alPQjvBCYiClS-Z{a6 zLEVy#C}ssypN*kNu%81)4csc+K^RrVK++n5Mx;AT)w>eKVu%0l8l{pADSx}bT_R3+ zTVeL!tIVD89`w=Pnpdd4EZ3$bnZP(scl-sepnIod+Qt!QQUIzH# z7WW~;MNCL0si+V#Fbk~|3zoS*Tp|p34|+^^ps0ILQ6`)GMj>mjk$YCF_Hb9X^!J=} z-2^D(g&n>&m`%7f%#HE!`GXcp>+rvWZs?N%l&vC2gKD$=*T|_%*Fb}Sfxm@lRq>CF z8i=UhV$m4|Q8y>-$;U%b?wFNjHHw8to)SAP`i*hx#1-l;0Lu1@g}1A?EF_blsrMu6 z1^HYPUZ;BQt`)y&><9jZ0D{tf7GdKVB7u%QL*MCbDL{D#d^1$W7FS2?K3QXrV#v?IoB7L9efQBjzEZb*%p?ahkl(!VO@qjU9x3yt&mxxbddh7s$~%u&~{kBdh~ z-et*F0rQ(yzskc_*z3YRSF~cGdGka6oe0(RVcfdaPZw6esLpVlJz5UZ;x0wLM=l*Q zbvWFFStyFgBB{#eJKRLFQaT2}|LNe@Zx2T-4+?q7WWvW4y-Y8Q3d(?2>FvKA8A&L( z_(ssLA)1C=C^(Dnbt!KCXR5Y6;XPOeO5}L7hkq!*LHHQ3oj&Bvtc&F0W0SmB2_hrG zBD{f%VcT)I0+oySOld9$bN~$5z zhbrXA#FFAv1Q2rjOUa|pHLQpvd)Om;YZS7jS5b9ctH--d%egJ!ni0i<1o;3PCVAEBA6t zR6teY`zY{yxVQ)mG@JJP6bIjf`5c^A8MI>|X8&k~-F*|Av|at_pq`y=Y52DtGLDMt z<#5c|2j1|e^%SV%v9;>dB>$s9Keph{*nWLH$i;l&-XjrJ&XflQdPbjV4Q<>%Ilyu3 z-YSEta34Gm{L}64z~gA0;Quz4=&Aq8hpfxBN+SUQ6|w;V{U`qqdH(W1Fp#|X7LXVavuykvxO;N+0O-?(SVmASZ#rLRfpeB+PYXrZMM*lgtfXk{~9f z2kNUs%?_qV7LS84AP}wkEc1+vdc!or#@vFtKe+>|}zMbMc*X=iT?t*S-F$SFhFk*R@J} z*WOjT)G5B|g`&iYV?0qXBUsx=_b`nt*>RE5jE{^P7~2V&*+A{wj%U;Yj_z)?eNUOa zx^ZJSJW9@~Jn!%$>2adyH)A{5O_f~OxcHohQ5OiZcaH72?sc?f#N4J%=Nuyf|GbP}#6qE_SZhKzRr;A3~KTvpB|a5PsCic5dT@0z=@$6rZlOUluR1;c*6~ z`dQ}~6)c|i;UWIGYhqA6BxpbKW3m7^}G2HzUqN_AS{k zvfFr|Uz?>j1JXI@at>8u(|nww3E{=v?nad?n-= z@(RS=Kie@>hJM{&Om{6u8)=iE+f^JaZeif{Cpw#ceAYd~v+WwRfVrp{1s_Vs&9w07k;M*I56hXnN41!#wBGBIBqV)EA!&_#5NJX)C?UDb%7 zpmXMwZ8omUw_)r*@CXs+5AuA?p)?@vIhTeNv+YqhiwTT`!BuBaN ztiVBFt?b$j&N1m~GZqd=CjLhnP6)b>(4Ry z^qP~mLD2R7)=qE^t@$mHbEtWc5qMA@Lo9fl|z<(ht+3i?xJOSD}l89=DS2)6$Pma!) z6+-L8Xw@;U=*C_iLn`|AQW+8Q1qoR>;Vcw%x!HQKTTR7c2B;=(;+^VMFheU0M8c{0 z+lrIiJAZsHZOy7A4vMV@#zF-o4!=2r9SY2dLCD5U_49wyT1!9#S-0{Cu}vD`7kGFC zJOsJ__+-_5JxG^hS=9vi*l`wz&(y0@RzLEBK98k5`e+=J%o=>ZM@x9NEI$V#SxX$L z(Y{_x$DmWYd%=CGEB)s2mWlUXHmkoWi20UiD%Z;AyP&TtHC-CpNK67u$f$eO*vQGl z;5!u_ia2xU+T!+tJqQnmOh1y*F6HwOZ>ku_FoM~|oj@>QjT~BUcN*VPA%;ErnWhza zk$ea#a*FNp2T|IFKs*CJA2(w@=#YF+-N-A)mPd`KUh{|t1S~(lZyO4lElwC7DoGQN zDf&2`P%Qc#8pgGe#5800)2D>QCnD-v&2MHc9WOv+TDVmj={*Sg_C`J_FGYFjgz#D+ z5IBto`J%QZG!FsfUHYDzE`#^@g9UIyHxk+QzQ)98ZsOIxA|@wR^~j2SNuy#-cvnn2 z%A~O7XT1k&hW_PCbFGwOy?h zXw{(&QQmzrd;RQB-t-yPq~k!Mau*aHoq%A z@ut;6jesgrE4-n)RcVlqt>CxXjZna-n`kUy&`Zz+TI56Ic1#g6o5LYTbqPFP><$-P z#3S>Ci%>QiI!v2(ijT6;DhsWr=x0QdQ=}^8A)^EG+mK4sC+!1bgWTJzY2v{8Hf`$x zfoMnM{d75);XU1V+Q$RRD+Is|ttKioJS+Ojs8DZ+kJHozt8hv{ zLmH`F@oL5bWP?V?EtZ1>Z-#*7bl<)0^SU*d{Lm;0Tw^Jl1P6kWe!sj{I79l*8*XvU zEXke6QD^ufD3|+|SxGSk@1XiK@f}1}JSTsznVgj087Ub_z{>*K0tPSGTAk!jHM0wr z5V%LTnJg8uTYPF9rsJ|&%9c2AE@i%KqSi`me8u!W3f4+NurdQSDGM`F*pQaBPHJ4C zcR=+_K5jjO1eEisSG*Kz*y3H49hAjJu!t(MQks@C{rWRa;FJMsR3eoDQP=Ux^dLY>7<~L!h6R-rD_Ip50RJ#;Te7>Jh-+Q5>AjmpQLiT8w3aQU6Ko_=o zKp&x-VH>$y|E{sI@#N~;H4G%w2rUf`z<+Tx$HSE1H1PwX{&joM;axw%Mv2}X$}m* zEpfE+DW%qP=|Q3ga>LZM+^lJo-751V4BHZ6^_6QcdCh9cp7SdTAf?l|HeU|`%}$af z*;v%09wpF~5On)$d~nBiE;PB`qycWB~VA z2#ZUhZlD(J{ohm7C4lFndFq|!3Nt)u_N3g_dk<5uT36`+40P(7`R|mhkvVcrk(>|L z%1aOUx|NYai`Dg%Hio*xvgJ5LDV$9y!X>IlCBPKuC6=SAW^LX?Ss8wWK|doTeppfY zCh1jDPf-6XM-LHg5ov*oHa_1>S`P^XUs)|8&v_m%7VJ3?NY!Pw2UrnuL#KJT+AGPd z)QdK8tO)p=RN&_(Ikjz+`&l%e73Cv8(QhNOeLGd%oi4LkPIGjjD5L{N&O9?YOiE3@ zNNwVF{JGZ^GAx~X+V|>Pzb(M2p6fIQPw-&j6Scjh0&w|d2!Mf?Pn8)b6~-vj{}Dhu zM!0O&ndZS3xuTx?y*ZAA@ZBba(pl)iw5*LLvHz0;2YB_i{F&c(oP>L~$rz9(s=oc= zhB(3J{1^}*7jzT!cGaDlCTs@qbs6vo)iLi>p4Rz=DI2InP|Z*KRogO;$T5ZK{%ezj zcE|nEXT>7fw|Ec*v5rRy8%JqPOfo{k-nhi3ROMw+hz`LOaG#?6WRrHY=Mrqme1J}h z#SJHy5u4Up4VhVFP%OOp{f)Gi1;=7IU3lj?>%*#cR4_a#OmO&oWSGnGH zz-Np}wrVC`=5_{opHyn{gk|f`#%byd9%Axab>hzA!d1&U&S-GX6q&y6Fp*u%CYF@K zg@qxa>&%hw7(^B`W9{W zKg16o+pSW0uXmt&j0Fg&K1F`r~wzKH%?0kmzTWoeG`|A zcub-=*HLzc2=%oPbIC(lV*S>3rv?*Mg8UWsBIEq$*0Aua6LCp>!zL zeCD;SCfuFAijQ9mK9W)_H!ARn6Y)hoWUTp4ZSk7gOza(1c)^kqeWrC0Xh>W!3oz1VI^e8&C0^TVNuwMFY5uc-$OpA+4v!;d{khIGsDyP$pY6 z;U=^_y-f&2X!og)D{!%45}}VRa4ysPJh8lHYM#Df1wAQ zaH!b91Mn#fMoAV=M1vug+4-Le%BjrqflTvU_xt>w1l$?uk_Gp{{6u-w7MIIwR+IN6 zLNlhW(L3#@+H|u@qJma5$~_LMLek`y1J7_2qV5JZ)vPnAIxY3-ob-<7Dv&3ky{RBzLJ=)kG8ca956 z3A~FpTJ+5W<~=85sU4%Lj-i!zlp(3e0Vz$YX*QW5ULB^Hg47C)sSUFJTQj%_hs2w8 z4tQi{GVxX(zjR!d?OXSJFog5f2q>|n^1@?$rTSwq1BQu?WlCuY#(t7|-%(|;lk;^* zq_de}`x#U-%w<3q zg^u5k%0hq7RQooK!2aM+spLtfW2hI+ z`n{UAK3B@CksZxSms|f3x_zj6_4ybji&c?qwz4PgP$F9q&ZE;L-bJc)mi$3upP*4n z6&feSdIh2VrV=-i@}A~q=e-LnRr$Re`x^S$HfXl(QSNq5NjopqamY6b=UqR60#L3z z(Y$f{b=Lz^Q^3;vTv)E16gEk^u$1hLtTXXIjFjS})BVE+w0D>TS!KAiE4N<*>U!|E9?Z6lq|UbwOh|xWq?Vd!>))We6OimrN8urBGO1cyWqi!*T{_L2aRnHs9NHyV<7fdI#Rk)&Lav{WevJwRF_5sCv}PiWqMKmJ z&IR8Vv@P$~^zEq}VrTg0TlHs=-{w-tlrqwl=jXdXX^<*81#%l0{0e$ zgXj=ff1e70I%!1jy^}WxHMRClkzQ)iSV_NrHC$=v(fZu~eBN8}P5sk}<#hlknTsFw z$k>dj`7=V`Ue3X{X#SgTB>}HL6g@~S+CgiIIgowZF-$MozvJ67XDObpM;+74JMnc~ z0X+@r+o%#O{v>$(nHs+zVK-r*ai;GRUk)$ml@v0rPEc8xEzWCD-?^GaQ3|@sdloX@ zXu!tmIBM_qsy`x#+<9wav|H7Jzom~RaE0t0aVA|8#f#quc2kIGbA;89RJ*zbbW0TbyC^ zS9<<;{cR8;sb39R|Ff!~hwzPF$H z;F&z0uYR)Kd}H*zaRsZJ#ZlAo{#Em-M)wu{z3cT2Z)1<$*5~GREY*!vvr+W#>CDcN_PMJpE*?z>1_l6ibcw(w`2T|g;y~tQvAR6wZu(&iP0YxVkzyn~B|2E$RP+AkswiF~Y z#ka^$Gaxq6c#ey6Soi8@4>R4o#bo=`T3 z(~-J@OxaGHZx26^;H064QFHm${G za9U5LRgHz^Bf;xhU;X10@)|BZ=~B3~hIBJkW^w5wLKQbd%s+};fU4Bwl+^fRA)$T< z!Mz~YEG1$vf$JNi18ufu;RZD>I|hR&y{Js?j&i&#dz`rJPvtWppd~ob45QPOrBX`D z)*gA~#84eqO_f~$aYI#zkpPTJ7+`C>H)RM79#uj$l4cjZVxeV540-^q5yNa@D5xsD zK{+G*6Yv=mTbf>=OalKFqdVrki7VQLq4ih4!CedPv7TC{I35Mug$lFlOiBn>VvC5B zC)}(eYDSKXh?w&J8$FcfSQ9~OMSebcUfKo@NEP6FNuWo*gCd29$TEDOuz$`iRFd0NdxZ}ezQ5gmemu0YK?h6=$2y6kb;`_s#8)`s<)KRuJ z=lj=8^WRHmVqX7EX!BgoabR=8NnU?qF4T_6P)AVM*%0ZldwZ$qdL1&OhmSjW?}!Xb8ChsV72e7K`m7M>@CJFqI<=_oRZ{$k*pBvv>t(mSv*4HU-(d(W~KD8Lzs!l~?v@KU1z7NOCssEG}J zRDgj!6=fY*gFZ_)4CH9&*=%7wFiBvUJuLrv28=JVgi%U;oAAmJY)p;`!_4=}Tcojp zfX7qmYeFWTAb~`+U+>@Kaj~3aKA)qSL0jazb7+|X(pgWcMHQIM>=dLVBa>QlT_&pP zCG{lmq>q~Fc4>T!&E!C)gM70B4tZGZxB62?J4 zyE~s1T)8=kDhuVqepwty`qocgf`h6y`c;W8<~wDZCV^566A8I0w*?DP^URSYOL)kb zdWbJx5yR>*Ff+x;G1H_{Oh zFnVMdeP)n6fdS=gRHOEp{s9T#**ivwY7bk}Zg+9AKT($mBS^^gP`n(XC3cbqRU&b2 zg(?>o%W}2DC6;p4Z=(UvW<^yKqz6G1$8~@$dVga{LY}kjcwJVc^Gdg!3TqlF5+iHV zqxtdAfc~}p?Kvker`JKq!`*^sr>2lIn@-Y_7=j=zzDH&6PrN)JZ@<_np1vS%glZth zF5K$9ICXFk$H!r)0Irzq{akew+D=C5HC<=7Rq%XQ?$#SXf!{n{+?su>ae0Hj@DYeM zNA5PxZcjFt;^jEM9~ZNWn>ZicRY66yRlmp9An+1ITWgK~dbvUh%wc#2k9vlfOQdL7 zP!mHO!%`B~V$p;)Hs`ZztL<# z-EWsiNbev;>NUhPXuCMS|N5~c zyTi|SPLCo!u%m#8aP;>qntp7>n} z_o4DW!a+U8qX*`y7$g$VzH1H$`@$3dGf5TYkl}OR?n9CbNq}sLBOaR*SoI6}W~tBA zJxKuRA%MumT%c6L@4?Zj$pPtpCN$HXt>A*T+CynwI=@)4rv7GeNzOym$ph*TH~m#fSP8=$FgT01gVIvJ#VLJ zsdvV>LYA%qaKD;!@4$rDGJAO#nQ|#(>)$KBOenSA^-4ncAT$?QbvpFCMBC?E z{+Fz$mtr^97;V$gqt)r9-reA)>3~!xh)fj?Q{IK*LF}@(&c{S4zJ*F(S5um9P|I;? z-=T9&JkJar_$dLT@ys=Qz3zw62o&6eGTw@~_})BB1sR05|kF1o#%804ZSK}z)AQhzcYYsm}ft9>t! zn{BmOrD%R>gQZKb4UY6YjCrz8yRh7QMPl3e$@Wr%FlVk!b~l~tdzNzJ%=Eg^{pZT< zkCfG2f<|X-AL;Q$KAZ+*A9IJDyOWu%&A&Eu(lOGr(toBmGIw;MGq*9ZWe`wN7MD|0 z%2FJY?Po;l5If;J#cGO{=gg%PtfD5m67BJ?*yRW>J8OC=h0>i|A<7&LPCUCCKOJA; zp5@x!2F{9$5tz7k=ce;0ujV|d5UHcQHw#FVfjjbQqTGV34o*<**%)LfZTH%0+@v1R zacE{EMnVO0v8K_yoYr&#=5o5^2?2(;TDZ_S*4Ysd`Vyq)t&-9SCuF5~s92OCu`fc( z1&)cyh^koYuaUO@ zS@Bm*MMn%DN&R6U{x=mbEg>W#t0dB-)N8ZLhSUi=#RqraNHy2ED&=pZkd$JU;YV9e zie3{NA#4k!Zom(KzJA~qTE=PNpH_0LHFr3e@U+!&cz?lITP(rib#c8Mo6O;T4I}6_ z=^5f(f_C9m|D^fY8QuM^)fKXT-*wYZVBRN75r&Dnl>yJt+%1>*S)ti=7I$se-NgsY zD)`2zH=6Ou@yM#f-N_JVW9z7&(fIP@fZ~$K=hsapdEmwRv@;!%(G&#iBpaMfwl(5p zZ+9y`;*ApbZm3AoETFsXipI(36pvM=&b;=<)*R&V0rVwn0qo6ffA$GqYH~J;>j1+Y z#pKtK?$PniYyb%3^RhErKMH#pMF&$MsMw4f@=2PW;CE z00c&!TIq)CVQ-A1r}m0mD`G4-aEMEDZ&S(f`g1FzU>Ewewyv?ZRbfCK9&KF&^^evK zMxrx2?sUl=UR+V=Wd=WGjz-%{Gb-NF{y;p>j;16uBj(5DK7$z^?^J(jrZBsKhrtv8 z!J!&VKbaQem(`fQ;LaH4Pn4>=D7EIAPKDviFz}ky6WLSUK$}{{_E`54w@fw*P=2D3dym6LG_Q|$a6ko#?zX=uAV-)A>TV@XBnv8~Z zP^AU5DKyRD%B!cN2_BN6a_g+^dSybt_^UzOrX-VX6Y4KX3{S*xs=O;FXf zb@GdcA1zih#@M})h=Vz}A|6z-SD21z?WRMAnvt#+AIYcBLxk)t#vDjW`ZmUbgX~31 zm(+*1w9qCDpS%0lp=7`f(%MZq8pF$}z(*`OVVuseXJ~B@2C&ty2*aL=uv4oB$uOsz zy*#gQ6W`$fbHqi?^yfBzOm+SrrV{zzPIWRO$^yay$^yxWV>VfgNFfhC5$=9Koutuw zDiXv^3O{7>t=qCd9eYr^M%XC!&n!Wyh%A!p1FpQfUnk8>r0{8Ozr7It3{u7AExcGp zAhT!ev~85k6g2d>hjdHTDN3z8DimIICo|l+0q#6vUvp zNLN>*sc2A(Fv+@slWp8|DdQ>~$S9d88dl1HoN9A+V~MlvZ{aS^|v@5;q}d zI2JeFjche6Aqlo-9nJOB|Cz#?W|*So6T_O$TtTl;B<~z4r`P6})B9L%msJBo*?@p> zZZ4&YXwv<>%Oh^c%ziRbJ5=m;{%-RR2X&`fb|vxqgH@i~8#G;K$MZYDtqW^~cH~uP zfm@FGZNJ%SK%@?DIB)aZdS|v&dHfI)Af>vI|AJ8rtM-6#p#0)pR zdTPW}Qz+l(9;wrvaMmw!RB-n_lP=|#!Q0bQ@fH${rXpm8Wu&60O`FKeFw&m5tMMR`wC>Zrs1XNsDF%4HYhUAv25)-$W2Jm zNzwus>J`N$0krht43JccQ;btAYzr*Yd;2h>lMK^$v5m^tjZ?|)@54w) zw9{B>@(iqR9Yf5}b&59~GWyW-_^xHUrUj#XgpS zxwexKhotKW2Zf@uF(J6#Mry?zuCu#)15GZy%OQ#<_@Pxn03I$qqP8?|3Lb1cc;U>O zM%%YWLq@Ofm(Tg^3W%Ng{655*Y>40!#n&=I%`@0_jL0mPW764|{t?|+HRcSv$3B}5c26?z{P8l8=y^@1N+Qad9e2!#JOK@Nr@1J}zXkl0J z@r>;YTWQAgr{&|sDv=_P3#)NCUx>7BfBZ}mW^aq}kpk8#4Dj z_*cY#3km&+Fjx3*h`$7g{)+meLVove{E0f${2leDx8tvr|F+WklR~2Vzg+kHj{Mz* z{}Y+1_jkyDS@gdne>YP8L?Rgd9rE8M%kRwJ)!&~?s1J?!|6lR_LmB=K{9RZ539S5E q;D2bY-@(7DtbcjeWuZ diff --git a/tools/python_snappy-0.5.4-cp38-cp38m-win_amd64.whl b/tools/python_snappy-0.5.4-cp38-cp38m-win_amd64.whl deleted file mode 100644 index 18e0a50e0aae4b7f3518cee0b7dbc1cbe76f456d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 30111 zcmZ6yQ*b6+)U6xawr$(CtvBx29XoGqCmp+Et7F@?Z9CcD-se>P=j@v`*SeS&<6=Hl zbJQ9t^575{ARr(xAcwSv3Yre_I_5(Eh8g4kk`cUd#Z1wS%=A0KnwrWgjn! z)K7vUxsv1ro%RQZxgfXq)IAt1vDj@Cfx+8nZapok@z2lw&tW_{0$lG=$`}|8F`1g$ z`}FFOxyhoM%-;4i_zT~#NrQdKssM{b!{kQ;zJBHAN));-;V!aX856PRCt7i6j^&O_ z?ZX%QT!g@m+sRZ)OKI=c{C7E`DexFp^SCT;zKY6cq4S>kXk+laUhM!<<^Lxmdog>` z3mgPw9QOZ`VQ*sX@PA~KXerumvY>Q9-xCAn>j=4!f?C2M)y0xUNGJ<_pB7*gtlsMl zrWwd8H2}X~;uyvNS+il+LwYj{S={V@66I;}fX-`YN@_8vwtA$TWFB&hIi;)LJ3AXY z4_NKiks5BEeS>ux`W6W#dkO-n`^EOHZ5R!tdZdL3`ri4Fp0A8Ek_&Pbi|r|5c->!J z!D%#(QGcAH@cTV6YF}PZK7&a{=}J|^e^bIiRE|VhJ){tZa~?(9K|9a94#SEf$B-DR zLO9^{`%DieS-3>6Gom=VEqcDrsY+hB-&$hJVaF6LLS#SMe*8)OOBQ9tSk%`b)@P`l zrU7O25DgItiTGH!P=x#4c?_aZ>#gI0AjD9?Tm%7RwsxW8HWY8~K4YAd(Nam3U$&w0 z#PfB35)}*X)1Ekq9_=(mUZkXZp5{_>V|;B;LRZ6R3u2c(_l`bOH}Zhv+9Sf0eXlf2 z!m_5~=lgZajYjQC#v(92u%u9{B+hJaL+V8=@wX}Pp{B2I(MB#OOsLl>VvqO}yW@N0 zuv4>dt<5!YME2V#rNG0&3PkM+&&h#FR{LV{$;`#w-`U$Ibuap&7NFD!%Tfz5QH~c_ zrmg+DaWysdv~u}5+1JAuk}-N-V$p@L@=1TQ*<^G3$q$2W8)1Zn9&diP!Pf6NcCH)6 z+pB@+|EGIlBdt`N*q}EKVOt`W#tLt|gTAMhz z@2Q7UxSMVbA*ST`LWx-toWbb14wbdV>4ydMua!urc)Zv)`YDD{hFp={g;sTZ{LQ~u zxl21m;3ffk37_}<_u;^ZNgeA@+dS}ugRMA{aTD%3ouWwj{r=yl$Z<`070O7G~Uy!IZ~&aOA= zwAkxYTM~I0al!Cl*^u*KnWU`kTAhcS6#Td1U>W<76OtzWy>^sp8|FY^QbIEaUq<`o zV`_VU1tetbrur|eAU(55S+nl6c?`(5AF@6kr}`;+jvpIm%4S@?8{*!ctSnY$Xq#9h zjXh!s>b8dMUVv}W7J1dc&mj-4oO$0O2eQW zSr7$QS6i9O2}~`XnGNG^e4l3Erb6VUyre_pq(ZT|1LapkexYEG7cC1q>tG7`g!bSfX4X`=v2#PAb{ z8o~Ly`W6k|9=~RzHKtZKX<+EzPw1HWdwGS%GP2ZT`S98@{c84L_%%TXrzpF(M@p9{ z(KnoN*2<;{+3lk@b^?hGVyUHxuL)w$xik!mp%+Vwp%){Iv2OhZpH&g-++_b-6k*Dd zRjIgcoF`K3R}hFjN6(}kL!ZB|_~rq8PKRs0aQ0AV@4|_dB>XWFYmDP3!m77b4wJM6p8C zIQ#_l8|a|`jrGL1Ri~e)YlEH1(V!)nXYJQ1Cy9nC_*JhLBatEtUeG{Y60(;mM;{X$ zqoRdT`$^RaDK5YNQyYDd1EI0FL`8iFVQ_wweL$b)I_B}q2!^L zq#tYAImK|nrI7ghw(piv{2;*QG@dx*6+UbjnNWn-ev$o>$`?E!=^*|BJ>E6063(`B zyQF=zq?MldFSOec59iziXf}Iba_(XA2+GiXtWejM&2q=1puzi?jn?0l2Y3u`Y4!nL zY#N@xh@z193MCRxNEj_?5He$$1f0M{GhN>-(mNg!yp-rTizUKFN%|4%zS$ILi^Qh} zndyyUWns&+xrggC(FDP9=Z$h9r~%3# zf?5-vZ?N>$`3lW?8I=5S`d&}BTaYqOQcxD${B&1?#2xH2RYfxM4*mA8zYvuFkcOYK zE3mExyd**!*m>YJv->bTf(ITgQ}uX}DkJanQyy9_lj$zHgR^r1g(^gk9DanmezC5f zy&B2~ub~J|;9U~>RHxRZuxv4u?@0|}`&nmOqt;nQ~9zJvB{ zB~SqR5P7=zbRqEQI@Q+-YgE>2wpz8DpyVj4c|2swAeov&e!X9ozsYynJ$}e zI54cqaHgj~IcBy4%d`Elh0YMG2RJ6@Sdklk)a$qJ9RChot8KhN&E7uW{2@Jy`h#_1 zPB4NqQzsRf;s|E$9|hn6vrK?TAX{^83gy6U~0KaKU^K?TBBKC z*&7HVPNcE^O9aXYoDhJHa4P6m_yOR^SJn&$9gE;{{ zk+}I?)W=i))|MpzYQ()W&0AxkgEKuuc;rfC&WVoe*eJR_c$b-P!=1iuXQ*(3POS@J zw_UJ1pE?e01P1rHkHb`-b5P=ElZNNX3Bq|W3Bd`jj10DYRl~w^Jd}TIIbovI0EZH& z=B?5^L}GtEy?7fF1PDD}qyr%7|J}<{Nrx)W;x z{RMpA(ji6)R%9h2Iw*~ zNiyiWK^`Qeu3YZXd4#Lzr6si7-|8kz%TkPqyYJ~lUa1?RPIxd0<>wrgTKTts(Yrgm z2a2we-$Hvj(gmH)ORv%c~ zkN?FT@qW?C)jmMqT3S00KOpztYcG<a5+&BopI%nE}%oyVF{c#Dp>AqEzn64_0rLa%zRxmb1Q$8 ziVwEj)(Yn~et=&TU9fnRSb0qN7D_wF3Th?(8zIiS;sS{VvzOgUZ>*^9sRU7kakST1 zZ*+dc{(xJRFbAg-Y=z{)59aadSn+^E2jPj^Fd0z6^42YB3STA<9j4B!?fk8};DFh=2x}nD zqXBmD27xW7{YV+}Vcloe6u8Cko{wCJd|>)C4dB9|kz7wx6x^r_via)fY+ z+^((moSXH&Az`4Zf5|m%5*1{r%es9ESw5P>6y|!4lOdgOu(j!?;88r=L?4ZLf0eP) zXWWQWQ=WXJ_S=))zy{r8RGt|Dmn3<81+#3Ct53rj6UJ6(#Lyv}Ay+yd;@Y8)6N3{GVDZO&B)Ztf zLQwF_EkytP9N~o*6=yPeCkqMY?zln zLUAU^j4;XES1Q0~rmRgO0Ap3Y$&ry^h^McWRzG1Ze5zEM*cE!am@8I z%h+}Ml%1Amn_$T%VyHnZ-vkYvblV*WpSc51pl!UX?h!-V3j_IW(JXW3IaR_o@Vob z|NWS#q@-rbDc$(J%r@Gz{cpIgN4 zM+<2(xy$>){@sCEkcojAPOA_k!jVm$lP)G7P!10krlVf zxEQAGRCA(ne!((X4QN05t8bE8lLm(#6>u;G6!+A-@Is_wbGjZspW|o&H=J>&o&KsK za(on&wOUa2JY9aSN5VyfHzs|Qs#QY6>h!LIv{jKpJ=D!nOEF z2BS)G75?Q-uaSPUf-mL0bQ{qq;BMxA4=P`b8W6dM!=`@flzUN&uUx;wvAR&jSlcS> zd8rmR)5kJ?h#?307rw`Z3IW3O1WpiD1nKT@!eH1xSw^qY)6#>weoZjjViM6ZwU0NN z&m4t%0HAu(sr-Cv?w=;XEG(p6fvZn&sZA3uKZhJ^rZ{j8-^ukEK^n@>XbkGHsvIsMxH8w|5!W_%~~Lsyz(o zJ&o(xpHvN9Yvh&ognB%Ry=b4RoOFVgi(nKYXJ!dp9sZ?3hC?ar^aLfbYu6mbPeF<{ zj#c*Qm&_?QF_Fgr5z4ntp;z{aD&r0G}7`=v)@2Ee@!HK0Wfz_S~I0-ZG`;!q*_CL7n=rmF>z%ly^swvTm! zLEtQ^=Dp<~{s$2uyWU=^tiyznHbLOFuxqIi^Z<56+bIC8$Fq)6eXk(1`{*>znK19W z)4v|ZzrL#VW5bldN8I=E+z#0L2gz4;VGsYABl{O)Nzcr}4%GVx%h%R*I3Ot^t`PFo z3iWGxFJShJu-hC_DRWxFggKHwk2uHz?t%B&%yy|E{HW zpHZ*AvUQ&vFyyaSAL@?LT^r~SA)Fg#B-ULN2=(0odfJ5oH1_ltPBS@2oQA(DrBs`g zE7&Tq-k+wBqA{Yw=sBfNgBfatAF6$gnhDuNWb%5__=A1sSp>*b@Dp)zWchRc%>VtA z_wlLe=(c8VM>w#Q(;OR;uz=r%1+N!cgdtrBaV7%LOoptK!HRez?BDE^KDbT66UqOxzI>=kgPgl~*C5?g?$A{vZz^2Y zzFD)kVce5JGCXB0yL}o1LD~tOMsJ=Z7mlJ+b{O-I88i+eoEX9>yD#&J zaWkEnw)K?CI!Cv$`APSTAaAM{jM46nm1BtxdDx|9qbobvjH8bKxdJmzxurLu>63Qq zOqs*?!CKk7ajObmu%^7r$krr(#2GJ~Uj{sG+9jVKI{w*!zB@G3p5XDl`sCBAw)MH& z%`_Tha$=KUtJQri8cpXz*+gE*w&ljy`Ga-?0;RXbv*|a6Bx3QuHBcRu>1WfJUW~kE znD{p=-_VP9Rg8nJuPb>S=GoEHF}spHK|4hepVui0j0%?yK@k{jT^m%WGv^4q6r|rE zxuuX!+m?(s%RGFo=DW0YE*s3Z8nCFtsz4Nx-K_k7^aQOj+ z;VYlqyVO(#;`sxYth*Hd1~9qk;6FSio16r7+R_~R`=SIiREgY^3#9*m<1BxK;TJgt=*O>a55Q1!Ml0Bk~mK>4SN5L;2K2h2+WIyyGi11lms^z%T$F0 z3Ex?ivi-JCk9i2HS-fG^!WTvme>nwgf;|2sSX5Um%7yz}iR2k<7~=g$*^*dSP!Aar zKbZsEV3wI2fdG%>QGgI7iy7zrd}v;5|K@?6Do3+&Q%3EW8Jy%{Wv7ob?V&TiDPHY~ z9%vid1)Iq~Ss6TiLuBEz?acjgV@Y3T{?>(bC<>a^x0ziNY5dkIe)}iKa@ujaKovr# z{Bzzo1CyTXc*GPi+0G?4IL4WSyj#5MFymQNTltP4MTHlWH4hd2CKk%JxDOs~(1b@) z*t1#g;anxM&y()oS4K?8Q|y?O2qM@V%mWr5p%X*Xpo%co|G<&HnZ4Q3p<5aCB2d*k zEk{bB1Yx1Z7^e{#RLA{3EFl{0AR4WZhqW`JRQaiaQ-voH8b1xSU!hh4a#K%b`=@A zv0&SOn0wj5A!Mr*5@a zYUdtfqSGdZo(V`)SNd<~6NCT5Om~yA0&bb;qNlEjAOs;Y7eYV{a}qh`Ms4r3zd*AH zyK}g4_0q$XS3IKAg>6Xr1@{=|87U}%oP{4TK(#3V5ux{vX>oU@+vXLP`kOxEM~P9% z` zZNP;7wDqdje7twpjKzB%67C(x67=BgO^1RHErM+`TQVv@DRnsc^_GVt>oxbwxlQhX z2)U^kG6Ra9Ea7Ad1q6h-koCo{+U#58`py%Rc$NU>P*DsZL1 zgVAxN1IxqPDD9ofkjySHWY+G!ZbXzo;1OEM&~uZ)aq2wPnQ|28A1{dEXOw$=C+fd7 zXtb3MwigRt?3>^!{9;9mylPjb>)9pdZxA{>Z}CpSAH4|evoa7Bms+sz#jY?qkeq+; z77;?5!FAWX(=lT44#Rn1lbttS3+51-)#1*?XPOE4h8F$vfFY%s$bRo;x5`R@V-= zAnvRm!h+fPCQnyX)o##`=!3XnlP^mVYeATzrkUzOFwmP?qfG1%M3w3QMHwmsEySzl zA}pRvV#{kAVN>1VXqXMi1X~miJaw=QHhTKZ>b&Qa@PA1phTO zATyP@$AIOhEq_Sgwk`1<(Uv%8?oiM4-Ie#Z;>_1<7io~1IYJVhu~~_jZ$Q++%o)1y z{yaaAh2@B;woX!z;~j&j?-?bN388(z?D8Dd>cIw7M0GNtuZXWNcVR|%fagrt;o|pV z*pa@%C`@_SXuV463SpM!OQzsCN_X&CWZC5J;CtsqsB}9Hp=+VNy>2Lc>qp4l%_C-3 zD|bfYlIm0RB~PJgr(v%66LA3tdu%1^v-3fD5+TlT(0X1Q7%%2^b145JFydB=JL=Qd zoszvQX=#rdLKcnp@g$-{@PV`XK8G^NcxRWa6D6nv4VrMTVh^&ED$NpHv@Q~VQXXZH zvjQWEM@(8#Zm8S}6in^bbLl;=LOcuqH}H3`xD)+Ae&-axEZBQRb~K)_Le+^d3UUJG zBM9!VSSsyCCD&F<`|E!l(E5(hLbinu(=QO3$iDW*ZI9en)~|YMJswGa4?qLJ=iQj% zRfW!LW(dRy=ls1@n4<@2f?jBl$Ns=RRYHw8ab_$eQI0eXw>4;`@CQKYxq1R>BJ)-!*YNDl~` zy%PvfrMEvQzIuQ3NI@2miG(6i^ksBztbjVrr@{tB{0D+pj2(rxL+=u&Tf7H;avJfa ztGF-Obzu}~Td;CR-e^(W!7ifC+*Pcq1%VSoj9YwS%m{xTc4WDIe24cfk@NAL*mWo3 zm(Uvd(86V?y4c=o;AUMTZYCHI#qJM*7c>S56fnxpuVFnKRm|N^^$B6`Cy9di*_-dn z_LCmT+nv|1NI!cm;u-Ju6n6v;HQ9sau^V!Q*R$7MkMXn>Alz084NI9l4Vz)?p^OWp zsuEdiov;6?k!zke+fsizt6uihiTByT))dk2s`nH9Ln>(*KkK=by%H;VR-L9CsHU}# zJ3U8nP;7`dqB-O@_Flbm3u#euplfBg;U+$Fpr59uS@v-dNW2BA$Rc9+b8%Xvj&Q@B#- zCq4%);P;BaUxY@gh)W&aAWUPh_P2W3CEVH4a^$D_dBSSPcZt2EcBii;nJh^@5^KNr zm1V?ZOd}h&R~uv?mobqemNoj$u~ObjJI6LzwR)Ppktogh*~2cuoPPu2{Dd)!3T_KV zl~lVScvG`wj#W23MXhR%+e{?Lx`kOh6dsX}-A9zHGy+4yaAfXh6*h058ic zv;U^kwKq&xfO)+t3f1UlV&8}7yW;2l`8)qYUlS~+DEk6Zj6CbjAZKM7<|6yukl-0} z8dnB79zJK-3ts3cV59LF-XvD$0Jv6;wS#r}#9Z@N#_!#^%*nhJkXw z?ZbelVWc{C5}99efxqx$H7yq38oC=ZyF(aGQN6d&cIg8XqFkX~r?7k4F3oUaJ@eNt z2Ur%)W{8Eaa6jLY+Rw@^hCD6V6~w;%2nR!Uc}&3Y=Cdl(nS;rAnaQ>^TNfXbZdUNpbP2EvuS;DP*!1-D3e-mKq?8lc&72O=} zll5f8eF!>pk1Ho#;B24{A~}s)?AuxXjTFkGjJxw7=sd7U&Ae)`kDDdMdJ0m%-2*oB zTiS$%>`m6|(I*`y^5U&#`&WQlND%3Ty35`xU)DEdKkf~GHbfzQ=xas0X&rWy17E;k zPH>-W7+!C$%g;&H=n`gPUgGEs%ptT6jwgg?2R$54u;Kd@!R|+3y)Y%Rx0YDEaSqy7 z@4|EdSuN5#G$eLH56fX~RJTJzg2O-q>Ewuq;^_{5FVkLgU-6I^KlCH_?Tty`4?L^z zP>+1~SkHEEK%lhw<0QhlP<$Uda45(7mu2{NGY;|;Lz7i|crLej%bDTCMn2@_L5(-j zIcf{aou`r%@>q61`+9eBga3({@(kgnubtUsz@6{qS-{n3q0Q(+{io>kh>)M_>;vmg z@+>GdU_*IrU;RU=Cu;jUQk`vYN|M;-eItZ8{@aIz3GJ%V28)LeZ%{%@>iAZewC`p!7^0n-mj zZrQ^#svc#toM7_6DW>K%%QD%Q)xrADA;A$C8x~~DD}ObjPo%Ri7!Ls<%n^iIWi1us z9$9uf>uVV7K?a#h4J`T_vzN`ezvFuP#vfD=&2bH$?u=$?nqdQ_T{@`_2hUrZTprbZ zlka;j)xnw0-=x;s!lZ5)&2j6vKpw_UVE1zKp8sOFZ-l)6GD5Q5wjlayGsQ0o{+!&u z)VhOv>a<7?5%2CVjrCjB!^04G+EBY%ByjL_Jt~KOKK-~1N{rGb?-|xD86MsRrTw~w z?g>&Ep z)?~|(6MjZpd!Roe{PRzXw%4AF0kuFBXicUO$)W;0p@;C40!mA?DFbJsJ?+Up6@VRd zA20C%>>=7wFelMYB`_0`I?VmllCRvT?s^jb3}0vdq|4zEM>md_fF1Hk$9FZ~uf*V6J1Dbw&G6U-GE9Y343(40M7<|dse=x!*d)#CQC})EaEQ$t4#fvNBHCP6%)GZ zv2z;M?AouZiqrT6JtRE|Q_PIjEXVMp(Ee~UreiyUX;~F(YU*C~h&y3aoUSWc4h^Xq zDP^VAoay-TSD~fNn2;BWQu=7s%(cdfSn`ld$E9PyjETy7`Ki#V_RjgZfFC1cW;!-q z9eqhiTSjmxUr+ZPp26Ye`_J)l#RUizEmbRKPD zv(h6PtJD#B1Aqc+l5pUw+~2q)OA1Bor&QXs#y2wYDnkAG7BqBvxBj$0@C~>QQN-6kiy-SQ!PoN)=zgdvvVnb9;#_5g@Y}qR6O@E7#LBC;f6!{i#2XQ^<)R_ z3?sd#FR#=HR@joK9B5k`c(FPnIk8oVK3~zM`s^&qGZjp*kJI%;IRh=rx%3q&R;*Y{ zOL4}?)Z52`&q2Nz^fNN~_$j^MY#|%L=`G93C=`FOr&}l~Dvn`%5XuR+i$&_>9Uf?( zH0suOIzF1l>eOIQwmW8cH(hKmo0Ax-7X;}kFc0@4LAyWB@Zm%`IXI+_nL?Tz9>3ou zZd4ku3eVb|QvSAU^K57PI=Er7Kqh|4+G&o9{KvBJaFnax0U!VKb+vSJ#pC=u^rz`n znX6d%;?w?PBU85w@vc_MyZM-RsdMjwPH%;^WIZ$Q(tFL#MHwRt> zJ5yYb<(DsC&A*5r0_L0mL7%bV{$F3$BgBm_Da5!wwZ}`;nZnFW>fBB{6igpqaq$6j zD4hdou2*0CnpsAklNB2uQb!jf$98#d8)L`>ovW(ebc|J|XZOS&H>v(zw!UL+3pXQ- z_C2m+H<~k>F2-uXetnTu+5unB`T@)eGpj~9&#jIZ!*(6c?%9Xjuv^&rC*E8_D^#&e zc;fzq-%}o?9RY2hLkt#u>tElmkY!;0XZ+W6MhZN%3D9Zm)V3Z`;yZWCIM?d z!)@0my!w}hwSUE~6*>;aqirurJDskx;@#B0pO5>6yIl1>r)y$SZ;V_F)W7^smdZBZ z!kQrnVl=Xv^M1UhE0~kgzY2I2T)WN+#`@T~ci3&xxaS}Ct{WeOqIhqY53i?@(dAyP z|J$A%J6i7|$HTFDtbOaQ$^n?d+78AwoaKhp~Rh()QM47rwWs7iHu0`k44?T0rk>-*tYz*An~= zI4t+ili`fUWT~9I=a-FIC0$OTCeG%M%{lO9^GS=*@<#Y=kT_n@S>DxqNifRsN;}r9HLO_5 z&b6l7c@phw{|^48@&uZKab~q*!}a(f{PG6F@45GN#K_yNKf+ZqC|S99?C16I4&KAi z|7PDOo_hHEcZtX1XUwhRHbckJ1!WL`uxn{){ym(zw5&#ehx9E6d1I!VMQu5;;q8M~ zQC%pn#dvOX_QUltirB&DNJ?Tzs8-%xTVqAPhoJo%IZOEKN|=sMTJUYKkvLDNt2W>H za~3R$l7juc+jk^`)eo{uc9l z;~KWP?0iHXdCRqP_5HJc#&p>-=I}b>=X9`Voqn8I(~mOU4!_lOb$R5(qo(J63*n*b z=k6W-$+73b{4_v`PvGN7G2G}qa7OQOw}w)>$KWSCtUSQSW?yWe;Tg4EK7FS(fTriR zHtirZh*fIjHQcKoaI&Xvm?_pTvkF3{4Z#9hl>$Lm9FvloND>;8SnpD(GK$NKFB6Oz zq(Fi%A%-EUfKo|P00&uQ91_XElL2cf9wER3gf$VDHFlyw9}i2+-({hSL8Ok_b)w3S zCF$!!Re-3ZAnlWr8e^m2$S}d-N*Wa-%an1)Neo9Y3#%WHH6C@cN&ELhyp#KB}FF=5tuk|&L_pnzeZ;UJR1PJlf(N9521Jp=Yq0u6^N zF>|y+MkOOkwcc3*6>DU$zBOsLPCrwBO_qYf4vjrBmb$>?PC+0`#C#7{0gOL$bby*Y zdF{W>+);`Kbs3dc3`;ovRGR)WHuW_MG=khgiuwf^nO6==yetK=f#ghccfLePgxVkxbxt@UVJ z8s(OYRX60a=!fxx`@`AaiaDNho{d2Nm%rTco-oN%ex9pLi1(FWw=zK}Wfi!fN|E>k zrAP2}qJ#Viw8xU85p`NXLNX!tz-!Rz)jSwoRIUhLji}cgc!*Q1y&%YM=dKej&}VbB zo{1NG1_748L46e@N~(-c#FsyB@~Zr=-z9$KG~p%UmyX|Auej&;H#IKwmwB#dxo6aO zwqNHSNWGGH-4iE8ixT<#8a=VIrR?d~ef|wQ^!`s#o!?Kf;*T3|=wH`BuF8i~BZpD= z6R8q_pWpVB--p$k)yGSZH;6-zAWhTRXpzq#Df6dm9Z`=rRe%+dF9$f(UeN*dE6o~| z8AHD7Td90kHS?9Svf!@1m5f>4oi-{)O|_V@~k zUyG>jHp$i3*&q%yvm<^n;XUAHMcMCr$_!_e82poPv`px#G#XdT9~$a$iv=j8g`9sT zyeyf?3-zwp$-D1yuVK;88fK(}?`u(UXx=ZjCf{Yj*|oDJ&vxK(t-)T8`^qB6M9aIT z1Hu2HGP^jGVb4a&$u&m|MX#hG?#nWy%9x~f3KWDi5uKJEX$mI8E*_N^MzJII%3cwrcUqC| zK0>|R478$EQx)QM?(u+*SWyuJX32dmuQ`{g5KFS7mN~Mg8TvT8cp6uyo??rrq{}?lK967%l zu1rcOTp{pzJY`l6@PQx54~M~}KbQw^odYl;3z6{$CG9H$8~ckZ$2QJZ$X?;%Nz@{} zeGP7)&Okcx1XldT2tS%O`yK54kqjX(!}1CiDJF%1rAs-kjuz|!OF>E-l-fl6$hpUc zCPbbAEQ7FwHd})b^Ys(#M{|LP87s%mrw|bXYZww@6k(PVEi{f% z83#Xa#OnQj2IYud_|09sfc>0%k$Ny~A3TI^?_{(M1-B^fC4yrQ@b!ebrTwioK zssobPU!*CB9K?b)IL4Ip4|1Z!{GHajRSud}0g0J3WWsDT+aKvN)GSoF(n%ajT*R!t zy4(aMNuT^I8)9=UupDY}zzYrVdp$povN2SsM+SY|7$9m2w>J$-#mbIBK}A{dONaWD zQ%sR|4v@!g|{UP`W zdv*E&y*TH17FA#6=qD{)OXSBMJ1lw&_3^hR9{DLO>w=|wa)X73xx3Oe!5t853!p6Ao>A?nT5^zqK&JUgTL-x6|QV*7M$!em0$Z>+6BM* zw)XPM$PJybfP5(DjECfIBV(OMJU|yeWl2mo7qCHXGSmZRVbjCLo$}ipS+F!f_!q|R z5B&^o5BR~n=27}F(@ZVfQHqYjTj(J|nl3~8gksB4zj);e#ab2u=Fw0)JR_zB{do29 zzKc33mPic4E>l%q%FzZ)J5q-3BdbhJX`PByuCA&>^HI9^)yMtjrDFTz8ob6@ESF1C znw8?CE@}I6?Gxdn7cr?WTd!{2m}SE;6T9xzOU7udP^W_Nnq@z-ZDckBtLDW!ks1{$ zK;7JMS6bbujiB71%xNc@7R|FsQ^&WTqI)DmHuM$azR51LnuEP(-m*1Agz3?C>&7gzpwKj=%cYklKBkN;VeixZs4O@yzcm#n{+NqKplM>hOa2^YG~W|zzCTEdbU zxUov6d-RZo!$gY)&Yi}S$2~!E$)o`i`4c8;nT8`KJ7!uaiP0|ShG~NsdlI9*4Yw|r zMHYkP-#Zubn0}+UMaTBcf87ZRxAd44V4=m(^68EZ>k1aU$85>Na0Gu?B&lSYEa`Xb7iUFc1WrF?m+L}CnVOUA62*zjE;-*laZAj@p&C$Id@b8M?X zpcSY#HH={7S~cvze&P&4A27eQaH#h;xHSK`OdUyL3Vm*>;XwI&c1>zna*Enm95-Of zyFBXtJ7Z6Rybj_%tl!iy6f>30WKm7S07zNBl(;I^(P8Q;7{|BvMVZo#F&6%t zkyK+9DKG3CH9#1cW}ImSiEn>P+ouy#gM_mW+vwR%*OT)d?8*DBv zscM~Jqaf8`FERvkirff1)c#YJrt7SkWz)1;s@H2OvZgLyrko?BiE|7RgQo6Z62O@m zj>cAa$B63iFp z8aF`*NA7^@6*p%G{g!fKwavL%;Op}3f>FouLk?f z0YUD~F<|I*XZ^zM%`qdjWWw6b8-Ww>0PAEbbp*-wicU}h*P$y#b--NM02QJxMdgY- zD+19xfT_oXXZe+_5?3doK<;aj-TNR9LTn!7UJr6Gj9wV(CLl84>1+T}IpikTtBwu= zHoXJNh8}uH#u0I57uZ*KvZ={%DHy*4_l2P@EG?x3=7p`kBhA&jrKRx28W{=QK~vhD zr9mB-9rck%Njg}&6PgO+^o`NWcotv@bmH^}tC0xY8R%+(_$k5``-bqK-B%9z!rXk_ z*9Y%?a5`u$^8sn03PB**LUQ_Xt0R}`0F4V7CGIhdrLN{hS^6!)hirU>@JSdU?h?u!7p;C>MfzY{JJT}=jk=YYS(+eEYkt;t2cn*5qEL-B|6=+F&} z+7QuN)k6H>MZ9dj)qMaH%MH?R$rVDU!r!x8UN{*#MBA)*dR${(1NUGn_5nHno&WOG z|KseoBNmvj4SQ?d_E5*z+rtF5q05{N59J!n7z)Y|)?wM@^!!A**?e)vUt*542^|&c zsxRPE&5X9|!WxS*BUp<8>x8@+I`orFbiK1O(?juSM<>NYcC8&(Z+uU3)W^1Ghm{BT zv_Xk_aHyoEw;1yHVWU!1mdhRdGaic>T9-GS(M0xZZ?Q?1;rbr_LEZ#Cfz>*BjPI=F zZIATfzFJFNSM-bu3noRMoobEo$h8`UCFMxJEG;uO4bE0F(Y~U-(9WT|nW5a~odT;R z^_Xmxxcc!1QC9I`E_gB^WbwPUyCL*NYyHLT(&T0MMscHZd*W(pyTjXYYJ0uw$p`=W zdc()~=KRa+L;hL&x^@lF(c}L!{nhczeYw;6)B8RAb^Z09q4nZ%d9!)5w$u1C?tlMT zbAl|e^D3Yhg?Czk-xjun=!Hm$$cfmD_=MPjhllsf$-&izzmCU;xBv3@#r#IMtITC) zS}@j!?}cU72lnwzm_F$zujc3f%Kugg<_f>VfPm-_fq-EDPpr_&#N5%*3Gn}h3!S{G zb@g30xUl?RVt&B2yb@MvT+Xdu`Z~_5viOqi)|=4z8`ERL!4va=8DKhKF-4#6xvn5d zyNd4rF2W_iX1?xoz-_p)wk{zyE}F(6lRwC?eF~~fBDb+iSfp%zr$z1Jccic4P_;}w(mFZzWnUqclwqo1;vNvIxo^I(?vDYFwxU5+}WiRR;-5-nd%;nAf6?gJ@ z_;`5kN{U9WmN3;Cd~~m5u*(}FT&JAAd{K_9m|4e}rCq_LR_yvh!HJa@laqH4vdrv# zNR;IB55)#FINi?3;TZKG?7v@ITfB7xZ+?C?^geq!V|WerN1f5JjF3Ac&3M#kuSEr1 z)W0=0-n1)hzw{Ca4sqvrZf`bf#&^fO@{`2dNML1pz?Zfh(b3m}B}BOx$4&|O=fuS^XVB25_&$#<&b@2pFAQrq z&ZR|a?{i+BoFj8>s$C+1L*)7AC6-m&qFg=zugCN0`zq@-eIh{JSV_5N$LRZcBVeC+ zr^napiK*NFYp3$HtTH|QCZId$kHF7&?Ir&6)x*P$lJfpVmWOtF&mecta-RR^+X}X0 z1f-099}TDf*K?`=eU5rcR179TRmwP%MsvI`8n+8Re5zc{J{%EfNc0C>B=!Tb(M&qR z7~yYP5}%5hMVr4GMdU|mD{=x^>6a0mc4Qy7pUY!1E;Kamip~Lh2W1nX+rAJu5r3BR zNL6HeDanun$H9Cl5qH?h)%)C}MmkW^vT^cQEZ=feVk$0Y`Y_;p$R+xUO?}GMRA5p~ z#s-J+jysLZvBGGlxHQuu(MZs*jG+x#D|2WPjLr*FHlsd!V?M!@uIPUM(z-lPra@-d z^Fmq-5vl%j9~w}xi5H&|;pHuTEVkOQr*G#6q z$+?q-Y^D$d8BRt44a7!~zYJ1xYj&MK~0g`~#)7H9^#2OU+z zQJsR-t{nph?v=TAMyI?6B`-#aqA=u~vZy*N90Nu(iL1em1*fgxn>JjSW{Zsm4MzX0 zhswE|suJlm7_Nnu(n*6kv(d!FI%W=olM+3tMzmO<3^zL@+5tORfuY3rOqrgg3Gbw# z%tS9l_rDrD2jI%yWzomBZ5uPOZF^$dwkEc1+sVXEb}+GRd%_7`&b{}X|NqWAug=@M zR;}7qyMJrdht>V{*Q+}Ox(d=guccB>VT}>!aeAK6R`|rGb62~Je!~B&Mw#}MtA;uf zXxLImS-dAdRi~g9`E% z*pC2t2<8+2gFz?Bpms9i^|9!QV%|1DBs9S=sJ-DPhu6aNCnyrtc@}G zI>&fB1~v09L{STvx?dOy`Fq)5RKYDm?F3Pk^&~7IXoNaLRJTbs~@3yGMyXeqyu8v-SB5P0&g4+aO;E!Lrr$m`<)kaNWHe- zxEbIFo81QVXE7m}BqM^!z|1t0%~+;HIfNMSZgrUOKvB1$B8*mf4T6^*!?!IKZQ;(Z z=x^BRItfrl3x4=mVK(AcGdIM=<@K8>F2jHHyP%KxQ?>{p^{f8uy+BT3x&Z3m>T@kX ztBkvAP)9_46NyaEkGMSinshJ#<%U^NTCGs9?;*a?tlJQ~OkA$!v_;u=I`ecMn~7u; zIR3nEIU|>2#O+YW*|Fd|fqlz6<4;i1%OYqvNyPVKTi<7At_+<^aHVvWW!bwR;o zo?LyGHH>)v3=XFjecY2HQA{NUp=P^gEA4%!fbEqn15G%n(ZoW#3*XV(#Jrn@$0&&9 z`ZNrbaCKA{Owe}~Yw6B}i%~)F>(qcM^Oq+JqBFnBdkZ*op^*7BHdw7Qk<)`Fg=wmHJ(Gfk^6y3a(Y#Y7Br2q$ws?Qr`S#kv5zB!>o-rBma7512%bWCGZh`LX^Vdjl7RmVxwFGt}mW1nDtXk&z9$~;O%FR#_nZ` zsvRh)x_A$&fISmSvP0pPfQu_7moCSk0+P&Lm&}zx@S09}tD7U0sXjZsXQ{ud#Trr$rl~jsfz;APnsDI_4Ef-gWopQnFjefgK2 zwD8TxZfzL*gI}O~Ko!`s+{Bn$wxc(?frxI#D^q1i=-YFo zJ4h-@E!P2@i8POLb?M>9|1hH-`h66kcHk*)giy2nXR74ubsy=gQ4pDq+0G=3*+t$! z4jw)scNO6jE5ovA!VS~iij&^EZ~1%G{G4rEl`fM1sW`CvWFgambsTz6{SoXlY42mI zV#0cGz*e-!ruRemz&ndcp*h3yo!(t4$vtSd)b6+L&FwRBr_~NY6VFW*Zy(K-J`! z!U%2Im_#={V#vzpDZPi13RUh|zr_MVr)2}_vA;cYP6_d&6r&W~LbXgr4IRXV-cs3w zzoAo(kuicNWkbj1sLsq$lxHF>f)k)AthV1Soo6I+24n``xW|-c(Lv5vC1kOZS7~cZ z`0Hv=e$x*{iIu>3rer~|v6bm%9$mKQBBPxcoiH@97c#en+Wj?=SqC`2|E1%1&f?vJ z7rW_MdP(hdPY}s~8^y2{+r@6C?8?T)?=phANSL#GV$b!Ut0OD!K7A^0-$0%XW%ah> zb@=Yh=}L*phQoTbd%F(GLxlAhsyvm=F@cNlvsSKa2R9TL0yn1Qe7)nkWSI@0GdR`X zCfB%d>AW8w@#ppD8iD(<= z^O0eFj=>yA_psX~RGE$QEH@XG3~7=<%xDSsafKFBwfrB|5Rc%5i?GK{0xWKE@a+tp%^*Jef$_y|9a_-BPYg+6c-!(^7b#lb z$MN2h<-SB+IkgKrTCUig^75&g-JX&krSon^WbD{-m#dfcLi(3K2(YYg(N%2?MGjgr zdsh42&BDPvEqcAU-bz^#zLo)4eX|5}6Cq@Li`AUa*r?WyEB zY>i(ode~+9HrGlpQ~M}-@EIY+;02-WiCIiWE zvTh05>6T2wABULyZ4`Nn2vQW!e}kKY5d2=~i5)M~_8Gd5d63Fx4*a0!-nOlHeVDu z2(8`iH2@g{rh*vWk7Vvo{#)QYlj)Sm($?txXgZDh^-v!{gFes!h=damX6ggErkHQJ z`J05@AMYK6*Kj$z4~(XB@y`#Z-k<7=?dJW^P~D?wohB&iaW!b_e%VDD?rr>+Ph@W_ zgK?dU`~ou_;RZgw!GYy+r)yL*N>+1&hCcXjWM%tZOO9tiM`sdo-N-6O7|ZGL<%(ix zy*Ql)#tr@W>r+T&|9&bHLV*y`S57!fMLlk|UYs^F@t8sC$-8*x234%kN<-0b8iDqb zB#*A2Kg!y&t4M-l8-TG<0f{5;E?`H3v*HkPanl0=nA+;xgL8R(PA~icU zO6VDN>-Mg=&-G;9J>RqNKg#C}wuG?W6V2q?`27|Q^rUCXVw*@vfr*&(Zkn1nc^Lhs z!$T2g@7-G6UvY-u!H^k7GdrYxALGrG;uuGFb-xQ2zB;e;}DgYf;2(BM}Q5RIZ^pl2-c=!k zwYcazTsI1+Se|dA67S>hv?&nkc7?%rE;i$M`j*G_=oA#jH_jt3kAN6Hbs0UDEZ~qv zP=V!Bwwf|YI)G=KXNK+*gWA|f|a5cVWUK2K#XWgXZ|jWoR}>pCnoVnB6(txd-a54Nx;G5N0D zN`rtZS|_rpwq0ddfTQTY)`L*Uq?c$SY1l{D3|j0WIk^`F&V53RCBh#4zvI%?wFUL%WX6pkTBU z@Y9mAb}O60z_B$&0U2hQrssL;qIEbGpfR1y zzGN+P5wcM;vT z*#f)gwC+Ncg`-1q{2;r@>zCn1mxn1r0MFjX?&UjRMWmO%rA z9>yKy9)tU)rl!-Iv^3Vz4)j7AKLxL?M8NPAi{Hkj294(Qym4$muFAlXuDrtY<|e{~ zj&JCK$G2vXjd?`dwd$iIdbG}O;0LX_<$3J6u7i6IuIX#}W5=4Mb1Raf1L=;8-&^D8 z6jI7;<}-rD4&_H^>bSqAQ}w7Wj56*>hBZ{Jzvj27CwncdDuR^F;MsmV0<<_wmF8g6 zjCqzqS3%GpsPn^}+`H1|`AiTr9*(sQbjK$sDV*+>N0!I9PF-qt%l)vnwUh-sU?VIo zhq{AWb`1PT)sO^Uh~{ZYuFLw)W~xlhbMfr^o`nCRt32JHUhxFE2PR!kcnWF8~h9;87EpX z?@IS%i(J*n`_U3dO7vkHLgga-Xja}%n>c{!$N^rnqj2H>12^HpeJTc|nY#a=q%ltD zr62|b$Q9iby+ducwi$;Jd_xvILT%h9m8WfCaoQFt5mXEFpn6C42|1=HBVc`s$o^y? z`l3WM=NCRiVXV`!;^uLBGqbF)hz}l#8FfW@6r$tzN;u4Df4QW+ocRPha$lg+5(%TJ z6~yNCHX{~RSrkhjfdFIe6`}DMPFKDK&W5n+T~!P(N_l=~U(N<8o)wTsjWw?KUGP~G z((T&G*ZJKczGv0id=a^ZiwW9#!^fC{Hr=?3xNx=d&I?-H3nk`nyUgTQbBU#8aA9GH z=(_V1yM~b^j0NT=bJf@48QJX0syKewl&(wACcRrr`rmBYG6a&o8rZ;LlZ9!M6kD0D zW)YPLqjrzRYc|T#smj_`^?jpnGvVp*bbtnOpcOry#oSqLNr`4L?`M=`HY{I0U8fIs zmeDp!Eb0tEFan_k%(0J`d!z$fac8G|JDq$eNib<YdXH!u2&j{J_*TGaidL z&TWjHF+yWK#6oIG&hZ5nXLnz@s;pePXr{m=>a8&3Ld7z6HmRD*_S+N0MJPRr4ZlTQ zn<;mfpVAYrVXNrJN|PeL1TlZqW9GWw^fsTF?d1M(r8g`^f|Qq)`Surh;M2o6a5>K8 z{2sKaU!=L^pKT5yv( zUOuLTqI3r|CzW_OFp1D7mbo|)_zS@s={3LhV2HVvLWMfmCeMGvN;N0#oA5FKPB~QV z;Q<7chGV2lr()kBR@eoe3oEG23xLe>-46Qwp9MV_>63*H!2HE{)R$H&YS)tYB||f( zZ_&FPrrY(hOQV8UH7h(1t3%QiR)Q{Y6{GG4x72O2sJn1Hm390DIUYKke}xD%dlwKy z+JOq4G{PL^Dli_5(^1=|m_(a#k?N1dP0-mxZ+s}zNLcjp$gBOz=%ojz*4aHNEF<(O z*=#ki2wd=*l%sKqsy=~M*;Rq0rU0ZgtEJm!g?M+GWeL$JHl;So1#HjaAsmrx)jQ&o zo6E*qd;ZpSU2$mJ_r(w?P$#6qmd+24@sl2i!3rE9K9Mb>BOE^?^SP(aW~boql+0i= z$MH9;VVvC`7Zwsz*&CU*uGs~WE!vkqkq23H^RI?%-&L;@i62Xi9z}wxdC&vj1`3^c z7?Xqkk)@tCgTVgiTPy>}v`Y|?b$B&T&J&DhH6J&DzTeO7>SJ^0}(sxFg9NB{%+FX*Fov z6q{9qj=L(nM5+hcyWNj&>{OMH9-Ld~7rWrO_9yvY^U6B;sZPUwNw^;d36y{emC2UP zU*GmTv9tuOEG|XlJIG*@WQxkj-^sfY55>tSPrE!mZE(vn3KCVrW4M_V3rAC~o*PZ4 z9;1j{0$g00MeQD{i6Ou-Z^6gabvNgcMvb8r_=`Mb?`6{=%o3z>iW1FsPI^eD0c= zFgH$?T7=qb_*!jR_=r%jB_sCGOS6oAgxlPS#c59*^_^hVqpvlOTQ zf00$slIK3eYT>vGidV&^f#uz&G85{>kE4shy=~L27%=(m2s?IUjlA zEO4Utm%9rlhgZhKUZ7m~Gc&Lk1`rVbza4gZhAtM?#wL!R!>?My$_{q~{f$B3!(a!5 zn0kDyldze-*TSl|3>Pg9{kd8mMD%6|5!r(t`qg~%T6Q0bmL#o zcQ=Zmxf>1gm2ew(wv9S-)*MrJ1`#J!?>y4jW6mt;c!W0mI>7J8`ae3(eeq47F4r*a zw%(ci?%crY=Wx|^eSX(s*6O{Xe{{dS<8SV>+xgzTji>tkK6>5AWWT$5zuCRF>STj;?W#3X_}!n4j=Oa>MXHF zPxqpwbdTlAm+g(wKXE4R86<_@hOwK;G0RFG=$W0ele12ayQ9*j^|fjh5*wl5O3@pc z>E}8WVb<@5?QgVY`!2|-JA1F2y(hjCmaTHv*Dtu&esG_m;-g*nB-X9Vo2bvkbT4=< z_D=B*>M&Iwb&HT>u_$Hj<-@;Cda`{r?w>ECXBidO8JT#s+qn(lO@J-FL2k43pX9hc zF(Yk$Kd%W{XrY$xa7gbsDEY8}PG4Nh=HXbv`;dsY1uN&q_F0CX!GB}h7Dr-B7EWCO z+RPtImm8+bntXg1{+&1%qcxrTVxo0kXT-Z^&HB^B)xq~@w;;Bakq@WTBjG0l0lc+d zu6~1DdrzmwZ{b#&4XSj8u8w1N{CC92p;I&Mdx%p1oh7ESLa}f+_BtwI@0R?LX}I26 zLUYwt#_w(Zgx}e&>M!277U3TbUO0!YMAMGR5(nJ2S+vy7ud}5mBj`?@5v3L(?L4q; zrF1yddEmFcL*vo4?RcqCV}aiQITCwapafJ!l8lHDp$bA#LhpCo5syJAX#!MO0Fj~` z$t(a^=u)b{eT#XTMDtsYxwtA~3M3_3ldy98!ovH(;NQp+2e`^Dk`|`nlE(@SO@m~z z)PbCaQe%}wI!3pBpP6g|qJjpMiQ=M}1!uq<`MxDOgPu&V3g0)y7)B5DHFc9kxkT(! zEBp{y*oVew9N5p+!!i@68(D61QNYtB>7G`bKMB`ko9rfTpNqKnxBHF9iW z*L)9b;YDMYMcU92M+$5u&5;iJ$B5}}mq{lZ_Af7FfxP`|$WVxWEjI0?7H~Q*VFZuDJoA)@ zp#-ipCP%s)t)fjDJa!C5F$OW&yj_)eIrcaSxigguAfRP9u}tIhl;tuishX2RvdZ|tqH??aX7d-yip}H925A0 znJryENH#%Wo5=&~!PE`y%E;!M|InT#_jqp|a~zK%-eRTsZ59=T8;NB^$}?VeF%1(( zW<*TIz@0uyORTAojgkPr0v}x?2c#FG2_5Q8q8l?u(6xB`p@8It?@W)TTe zEEZP@4i)-&1Wg=IIBY#z91k!;)+`w#WpA`7yT@1kLF4K87JI|~FzHnPG(1L)i0QPZ zlr~;gLzrd9US`9H2oC~SW@@A0k@C#1SqFk9AcEV#YXku>mqwbE}$>Qq@ zk~yOi11Nxj4e(S3NnNQn3+9#m1;dh(vUL4funG`oXz8}^aVkFxS+u^O?y7}##jV!1 zGJ_G6!(;{KeJ?oeeTz4%acm5^18E>eY7vj^ra-#1-87RY3iqOh+!9zW!B3$8?5&LM z(gOzYB^tIRbR}0guQEDuund(Zg!(SB7b(G+h{LJvi}6#V0hXcBUr`eq{iy+i`>HCs zu!j9s?ik3?&~rH=`e0JPvisNp4U8DPazs(e{af%V5p2v(2_q~ID%)hSL4c=onOh=e zo?yX5_1_<93V7JgvR^Jy&7my|JUFz?0U2LUYs3^;%I9l@nw#i z8x$hHj0Kjq)w|#H_U9oU|~!168)CZ!Efl;WbQk9||7kNH8>u0^OE!%Rw{#%;+;+%kJ?#Tp(mu8;{Qmt((H zECtO)C8OWS?M{)`LT`;lyyU7TG`}s8GY%benbE10Y9#a{hGbD>DTqkYB*cP8@{1J} zk4Tah7llL9v4#?kaKwlP49skaiaRUuDQy*og>(OyR;LD43^a&=`khP!1dKj8M!z{E zPf%b58}*n&RzP3^c+ReIqWa_ZjQf3@+!^XJQ3NT4K8m+vwB&BmkZL6EFX5`irSd$T zaLMI7jbG7#7xUukNwULW%9DD)Hba1k6cNwGPQ2b%q{}Mz-AWr;YEokxv*U${FMxsd zgPnP2Z|Aom$fLc&7w6`X3)?QT(ip;EZT=?}A51itVSP4O(tEs=Xob30Rw zX85@-A15X35~eQ4_tj8Q?KK~zh7^Vf^r#Oz@uIu<`XGf7uCfP$FY?~ zv{|*FO)U8B+qe-brnb?a37Eln`LEWk5qZd-UC9Xf->uCdp44-xV+ImGuKZYMH)uQ7 z7CcC|Rn8jcE}3Qw5>|A3jafJ^FnqK_T@W*m_!m=&V>jm_Q*ZS2+Mm6#84URR-o06J z^+J3(bLCHI4L!j-+dKR5WX%4I&GNYFbcX-~0xE|E0>b*gtS?hLM;ilYr%%HxTTRFA zlmp2}e3w6}4H2CB0#RTW7=2l{JgLI}VW%QOW)~?^zcHp!$JOQI_s?ayT><_}1{8_G zJtxycyjWH&xXWZZS!kJ(I8_#AAxS0}M{fmtCYh@r0Wv3~tZr|cziyay?Q(O~88e^h zbgOhz<|c7$^O1^ylLa-i<)m|=akPfj6E5_575W-FTn zQ}h>9vY0GBc?9A|zzR7CrzAm@5M@qQq-N(4JdNLbjkE10?3s*;j=k|)x1({mHAFTkOq<+0f=2K zMC+38$36pu#lL$0R4SIcsjUN|%}40jz8Zuq(7u2|arWw`3%b6(3CozdqSq}Mzmo6q zfb0XaWtof-y0kODH{kkKof9-9XTY|POUEfpZpAWBm<6mb$*rZAr|sffbe%~Bv+PhN zSc6(?{LhTV!s$*Js zn}}d8^bz`5XXK{XuzgH(EmXh84?&JDN!AQ- z71Gq;oyUAxuCE&|_*YY#hQFQVTOJeLmhm^$`3OAL5Vx>XXF@Mab$rtbc(b2hOWfUJ zbj(7J*JhUc_P#gI1g1JeWT|SJ@hz4N;gr93Jta!>FIM@vnbG!uT209K4PR>Edu8gv zPm{9$kg~~)XQ|!q^Eiq|pyVc!^-;nj@Zn*m*t>%X?O-hJc80PCk(atXgzwW6jo}Q8 zd_)upjdjLt*Ux4;~I~{zF30!AABXNPMvo1NdC$7wsT522r#Zv=2C$p{6 z(RLFW+iNYtyoC<={Y;+UMarEE^V??6Un{qNl#A#VGQME@ zEKMW&=`v6=eTxufV1Ob z1SfAjxamDBYB*0TMeC^^%mWi;;f}qVskY&2z9*>lZVoY)b$D+#ZP5(sI<~NpAfW=e z+R$oU&uBRVb2(r0gaE@^FJ5V$=B=H)k z!bcrT&_UGs!XxoT^Ctd#0Ct2JwH!<)-1LcWLJu)E&-;lJ^cRdcRw_7Mb7Ht_383HoY`*hEhEWY5BGR1)LsoBGtN@{(JLZt%}IhWrM@)h?A ze!_VK+z7zLakn;nSQYiTE%DnrTk-M@VS4PVDL*dnd0XZ;^KYyWww|i;o#U z3+li?{Qoq(jHIxroU&-Ma-Z!U8&VhSG(X%!6ZL%4nsk7zVp58EraxT;8G3DOgoquK zhM@oe`u34qcm=mrU`E-g&cg9<(#uZQ@#7U^eW?_i&(-aEd@7gkEsU_ow0D?q8QPUk z15@j%E4t@HyF29Iq5E!t(4t?CG7JlKI}@ICX?&aL&|Gn-`{sx6hT)TGcNSR#?uh6Q*3ayIW~w>eLZaih7ix9ba^p8Y1|cx=)yp>B zj{0JpymVIO+Yn>HfkRwd`kG5mHeTA8gu2mhb@WVhtcwEc@#*R#XnwYBG7(?cb7x5H z^5Ka=uQ2+na5ULnn^W_Z4FutPbv7rV8M8dC^c&9d_@oBNFo)R>J`SY-2#?fZ2FSIU zc-Lb3zjwv3U{a~=q10JuITwYmz`$$OOy*D{xYs0mBY`vOy8tyF#Erk?4Vqf%(~bwy z!dKB;x3EZ6;gH_!g3c{J2M}Cu6>|Z7DACq@l>j$XPX(Dl(ybqZs$oS^)n)r$#)eG< zqB1K$?b@%O992HONzS9`?bvx1>V( z(c%e#B_2%YZ9NVav(vbEcW24F9vr%UajNAQn>M+`*bf7s8X`NiEo6SQ>da)e{x2Z| zVq+w$6+b+YIG1wgijufI=gf0<;0AWh2L+6NnQ_(P&=-gnuxOW;unDQTwN3GQ`qN>z zV2nQ)i#l3>E8#;Wdxz6UO-hXO_+uVGu{-hA8a07$>!Qh#YIC#oOx!FYz7z zf6utc*@3*4&qrOrr>R8oA0KtHqAG$Sf+~W^iQ~4}Oh_S*z7Zb&KwV_f`>K*8&5A!| z3vAl6L7jS0x<}b44=$`gsfjI<8v<{K3O~9T$nLIdQ11mkq}hfLPjSy0xgL~4~dtMI}(c*?@qoJ zmXHKnyMg9*9`HhGLpws*ipjXHJ73r*9LYCN#_7Gqd;Sp1?Yd@2Bo`PM&dsG<8BKPO ze|^jinKeL8W{--~A<$#->7edXB$c(l%!e}|^;>U{YCxOZc((v7|eFLKMXydSh! z4{DZiL9{xZz;4Xfg`G!wjYikM$f!mM!{*MxO42E)1^_yw;%Fdmj9K7j*3ONYYl{^6 zJtB2`5-tWrj|=aAWYMSmHhh16F4;zc(Ncn}w2D*`vkj`a?AJ%w=h)-19l4hp8fo_S zY}UH5Xx`0WW*ZXEukdeAuFPb?a6_zMIxcYW7vK0!j@69H&v;Lj?)vT6TFZ3U7xaJk zd=awiqS1lpWOf@Po&9iPFILc2-(@W6-7#ox% zqyTgb;f#>fO4Cf!tZa*{Gy4ZHV^fSX_jHT!RP<7l0GZD}Nl#GA9UQ<&%Css;Ffg@D zgCqZFKS=ku3I9QDO3|s^sr|R8!G9=Y$KjI4PcJGEL?uySc_oqWbAW^bEhLC-A70pHnNZM3-8_GN z%`vKh5PiD19VS{nA6y5w}Ba%!41w2;9gMN7Ye z#5ue~jcw6}f_qz|=Py5fD3c@@j2)cAR59P@rH%{&6*?eZYa)tltkxG;9Z;Y;_!Zx) zhzL`It$9RaI1=K)udJyF5%NO?M;$943Co0Z>?6Z=?L)HP86;Gscni8YX{jnKeQWPd ztdS3O%k7~_Eg)F@qy)1}hS+PWb)4W(URy*L7u2oA8Ouq&uS+5rJ{-4>Lr3S;5R#gmnwL{c142kHkIyBJgYo6iYbppG^a29svbR0xvyr(&MkjzE0`e=z zK0UL?>kcqxEkU=DEwX=^H|Av&Z#F7k3FG!9sg;)0{dx#YN+JUlwfmXWguy5#!|%}_ z6}QfSO_Djvea84WRR7!93X?i7rrVFBp^SIA(iIy={hbQ6qh&mSM^k&@cvm0cI_JI^ z=H7V5>%#U#xss+`?)778-fG8ABNv7;@z*T8M&Y_A%CBxGi!B$rLZNcqL{VUHzlA>H z=mgQ|3=Sw;+t(L9vl$m_P(KFD@y}-hXyDP(pvxBrI`WbZ?gu?@$@28BV@M=N79J#% z9WwVjd$k!lG!}~za9uf-2#jiMBa>dU8e@;LqrRb&hO1AG9OA&M||6Y9YPx(LV9R9a75Kv3N$lvAvRrT=i^8c><+F$bapHuLk*Z-e5 z;QzZA+uw2js2=&}Ygd2aMtJ^(`|~BNzeE0AF!EoJ2ho2){uP@1r}Up&ihoJls{JAT z&ke@EBmUj?&|ioHjem#u+h)<-TlxJT@RIAnN_JM z0}6%)1Ox;HbWC+Dt7^BYnf~9&78nQ!<-gV0&d|ZZlR;nK($3OFU!UH=(>7iRVUQ3> zY%R$FGA#>>p)e0{?iK`+SmH7cN9*Oiu$dOsob~(gdmK*!2Lm`snE;_AApZOJV|M+- z#Blkqv~3y+LV| z_EA}mQMiL1X!WnJNMAvO+4 zgWj{lNv2NGn{-HyF3TQo3rb>_Zg*yw(wH&D%V0TA)}L9a6~s{%bj1Tr!UMYMX(|wU zkI`U};P6jHOU2kfJ*Pmj^p}43j-3#xbTYpr>y4=!+wB|F7;7uoP=RX z&ckhf&VfSVOw7nTIkX~MFE7GiYenQqAX?E9@K`r6uxurr8^#ZCh&Ukl!tDMTJMK{( z*ywOh9FzLdODS|WwE+5ijpJZPFQtCD{A}#x=I7|;oq7;`S+6hO49!>%HdToeP_C~2 zwsk!-^SpNTH9gQz7o0JEQEJ)?xAsMIyWL`S_r(K+Y8|16ff{dezr{T0F>#?8#RX8o z@yqI4+Da=EA3&}DdqNh_&FXLGi+cQ=~xUoV`VOhq}ztMNhd8<_;~o}4gnT9aT%8jFk74Qw7FW+Uaf69LIMjGPiP^aI$C{oOJN5g@`dw)3I0T|K3+ z^^-+F#B6E&#t76ho|ZE1OIt((@AxGi;B;u5p<(^CawKcT=DsBW^k-)?F+f_yBB%r) zBtUZyaMnYkp-|`z!E~B@0sl{;%9Qsh+FV%9Vzd4SG-g0RME^g$`G0YT-q?YKi_ZPO zlr^+9VQ2jheN3)>U_H>pRxRXC6DOK^NI?&F0EB^pSj+(fcNs+3=Yf$Bk{-q=Nr9rt z6P$=p#4z%TVQy|4&!OG?LG-kjT{KNTl2nE@~XMqt{2Od&Q(0g%^t9csKRItkij8WWQV4NzaK`XA;XBf1PXf5 z-25i>(@s8;dX|~V%{mWhs4>niz9UHP6+L>G|i41?$bxq)G{@ej*bgjk)vfr zFG6GEs9c4g*VLx+pa2S){FWZjJI#QMxmG?)E}=`_Oy|r)f>u_gH}5GS7=rTWvvfY` z%hYr}na2}T1&ugmjWm+MF$+ICO-HJiHDB3Ia$#xwzlrv^@;lgrt%-#?vZA~)E(sGz zj$|ZCK&+j5;w3I%MAVgXP)WK;4wjCEw9AxWcGJ-lx-w9jFpL=ZsuA15l04V338Z3J zLu?^7Vo%p0aBr!dRe?^#up(GcmT5aB%sQ6z)~T@j9F`~|U9*lM^l=G;tP)a*)leRI zx_QjjO3R%tp5cb~Fl4u7wsU+Gt>JncKvPI~EWA3%1U1CK&6njO+G<1#|o_1>uF zfga}+l6Xj7i|({g=?0ae9U?Gy&q9v^ixn$g9|!-|#$isjV)yJX;&{vn)fL=e>Cl0s z5M1_nohz7QfXwHQq}PVl`5%VL1d2F@sC?=gdjdP=>LNBn*)RF+LFbrRrfM`ssn@-> zQ@<7olh8nzgi5G61)}GUN3Hjt2}>K0C9DzDy|Bcuz_cn7m1@5aP0LNeKR%q1%=I)# z&cNzF1|@=ry)Sd`F4gIzBYAKGZx~h0jVC1Y7*@PmVrun}Z3;)*zqL4qXc(+hVE~+_ z_icnTcAWS%+LdQ*ROT_bE^N%)vm0NGLYO?puR3Sz@P}YoG#Wu;LBULd;22kCxlypy z_@~OT%+vg*Nyk0vrFObtvt%8g*gsDRo~U+*p!qv`E+F~l4^Ig02EXTDvbY|*nv<|~ z_ z0)3bunL7}Z<#s=l-(!W!x)3OY_2#_I8|HJ5sPrzv6lyul@s03(fG3?5!{s06X9Na3 z0D^X*fi1gK!x|wzY|UHtCNHF>u#kfIkZ+rUvTxHU5JLGLx49Pp z&#(&>G9(&mjjbjK5SeA7`wUE=`joQ=T9|CJ3qPF>~gUz%E1G);VT#e{C4~H%W zwUmMK7Xp_8QYQ^xSOdXWpHuAe(DWHl_NHZ@vK-G0`C!0ZWqK$$-s9+g&p+Wu2q}gX zTiy{Bfw)%RM&zDX1Plu72enj{Pt;Xa{^V(+5r)auH-l zN&>`bPRCTHZXp8<>eg=y6TyT*?74D8DAyG!svFJ0g{Az_FB1)a-`?Q+WihWh$Jxjh zze|B>VbYosgF3AW4sTVn1GrmIpZ$G?fqk#@l#avcLAX}Wq{-U1@ESJa??1NY$R>;| zJ;)y$gI9GT+roPOv<+}_jfAnY&JFV#20hLDv!=RN>JXEGB3w|;23^I&o_ioGEEg5N zN1e#*X>fIJ;| zxPZ=&M)}jIt$bN1rIrE0>W{lj?p5PYSjWN_tIF@1-@l*b_7()+`d0e83wf!V z1pl6`uPggbupcXD-zQtm-{CROO_sBXMXJ`wt9JD=voX~6H7y0jG)WVP-w;V zYFAne@_Au+I#ODUm!QGnRXvUMXbgmP6KOCVlv|cSF^i+#n|2SOIG#?1{DaV9K)NPp zKjx08j2X||v<>sxr=jEk{8~b`YN9uC3qmL@E%160atnFTfa$IcD-3+hKd7Bj1rBrA znI3z(soU?%=))2H_n7)`?m^n$%vZU-^w+x2C`fUock%l^9`UIapHieRakAcy{s3Ghwz#%@mMQZlbo<0?TmEVk1f|4J z=!kVMNcJiak`E&!NAX|m8zl8}KWhvX#%(^c13aA_J>_ryKMeVv(4(-)%{GX#$Dhkg zlV9tFAlSWoS|-Q_!0cI}#)0dB#NgX_fr1ZFbh`fr_>;WQUb^@giSsi3R<5G)1l^ZL z0TaL-BzfhvdzjkBnD)T!4RT$tHXZaZ@2=D)bS9Mv`sKo_Z)ODvuLm{(8T#c$KeXBv zQ;DC?<fQ`~E@83>p1%@R!}#inRh?Y_AM11gXnaA%EUEYX3_5ta&j;IXtW6 zJ5;N9UJX|U7bh6MhMCEGv|j5V z^<>;1$Xd?=O=VOn@g_aClHlzWoNsL%@+0V$_|YH7a684qWF1Q8p6kv$18k9d>B@PM zS3pj$979GsoY5XX!KJY9rHWVMl11;4LnJYWq3-$R^u*TZp3A}O9Ig*FV-NA|1s-b{ zvm20jPxt9L5{2L&{5qqO@f7|(-L*oVyxTvbvu%KaQwHmKgqEzqVg2-m&bu8DOkK@C z-~~D}5^f1jP02Ua<0yDcqV3n=3YY)h#xg>>%pANX^8?qyhRGp6*HA*UeLq@2@P#XV zV>?zZIFaL(`?^d``%o=Dr~2GgSs`dJnLD()_7j8A!A`iw{wWaIaT8w=3XH>1o0`ez zicB|0^w_Fj=?g=!sq5)GCE}vclNHNuIy={KW!0~iX`Q|DwFCJz3jcGtOkvCDc_Jt{ z>)n^u*)IZK|E&T)16PgWgQzFk3!B<53*l1ZY!X2te+=azR93fUl}&;ZlWipYr?rVl z4MCFBNpU5ZD^q}nCdL#tr~<`;h|r)C#FVpfFvTlJ;JbCg9IkHsh1x0B)Yl?)2}zYT zz@LpDQw4k(4955@ppJ3H9mrF8Lsm*3bG!mS@#_$@Cww;9jbid}$I5 zEX_#LdmhVN#z*T$yV~g#v{zk`kuS(5>e3{W{pz66J+AfE38S)jzR)6#bK~CB-Gf`- zTXBM3E|kt~@JBHgTz5$!5U&wSL8-}@pXEInSq4iyW8PRLbAU0ML>fH#$m0a43#1F@ zbY^*ub>_~3&s~7xrKMj%8y7QAncnmafu78>WpLHN$z})VLZ*4h7qM&L zQ(tFGQSn!qH%9A5f50T+YO~GvTugA<YP-0+mN1MxLG61yu)e?o)-C#23z+#vQ38=@+UyOi!5{yVfs_n({Oo-aX3$bx?GgYV21P&t85SS=Mvi77+i6{|3>}?T$e9?)zGD63dIet|*Dc{pd zw*al%#9e7I=UVh3*lk0@YJuAIIj-L0y*X(lgfvGjtUcdpucv}7c?2yUa<@b9H;Mmf zcGH93_>H!Wao`N+x*EgeE}sTHqwObUEvPoEaXybF7U#`%Ne!#kmRzfgai8G_=AJG} z;M|ooo=_RY1h_Q%pH(+&4L;Q1`~jGSHOKvkZ~TGk^ukar@(McliYGLFN&kGzp+tF5 zAI6OK7bo3jM7R>*c^Od4RmU^K@I}-9aCDZpYF8xDA4~_ajL_DsWQNYoCu?9})Hh=V zQ5>1gg&Ijvc?nFxnj(GlUFX5vCooSHNd@Cf<82=zXptOk?bZjU)LEtyzvEFD)4Qsxj~I zK;lZi_R+1s8mhM`y&-Fr{Up32uzV*u{Ap7xaF0kQfKQ1UKE@>}lb&SL6nsS(3KT%*+K9Z)C=r|8>_*YlBzdD5>FB_a{n+}!*8J*)|3 zcGoRk6 zgtmn_UU@_{ZHRI1*#cck=O>-hXyJaigQg#ceIwF5h|XjlgiOaiz)<0s;3&@}h#0jn zJBRJ$Dh`~tNgkEL%s^@LJ)6fWO|SECJAPog%9HHBI6*j3oIr-uy9nijq!rO838Ys% z&WXTMz>avow+Vn&?X2KSML~qGg^ORI09wKb$sk;!dBSnGB{chU!AY zds2aH7{rG&?llP`v12D}KXjU8m8iAR~uG%?70gG_ z((*Al9HscP^>RL+Xz?)sQ1bT8CThN6!#gOw$ejk;qmeP#2f8VWU zn~~9_ix3@*O+QF(3(JLyPpETiUdTgGBfl*myJd3$T!9TM1jcL;sjx&p%p!Cv)i&qWrs#2HJ zw6sAdF4($Itd10c@OSRCeY9eN@N(eE!rg~s1)KYmB4Wj37x-o|NzsRqdM^CVv^ zZ#d_|vSXE+XPUNW#PG2L9t9$g0t8rOWg3kd~BPYhN>5#^}_3J7dE0l@8|2CNul)cM)Az;9^abzDJl8Sa!#RnEx95E zD(*6?FhWt7V+KEYnJ-sPhCqZTDjZAriFG!Yp8a6<$#W|&-vq!O@y#${!QE|yWuukw zr-)Xm)#FE+#KhcmK{rfWE&LOSp4dUPya$Lpq(fCm&aB8pZsgUCaIIn3P3Q@mKf-yIwz8@_HQ3s6hMP(CZ zYw(y&j7rxi@Pv`H_z|~h&cS|x{`PlT6oNlU^fI*JC9pUE`p*TaSB?2^*eHjsMY0yi zp2Q+vL;G3Qbgm#gS&f}u1)|laWd9i2lr437p)6i{bB z!J~V$YtFnnI)jBgHq?f>pcN$V%+lSrsz`#s4&V}TUxplkfC`SL)FmAoL!X0;@>98c zQ8_e(6N0Z!al|4^D&i9SF5y9>fsdKnH;l~nGiCgd0Ss}74--&Yv6J1Lk$|3@f{GI? z^orA`p}85s)MUm9-gp%z^(GYEZD218w#q`@{YYC0nZY+oy30Y4na&~n7aV;}+ow^& z!#-&>$8#~2b~>unWR<%p^q-YyE!om7(3BTp=`$}S=+%wNd7gKx30_%|(1a=wNyRLM z`*_Zl5Br@tnHL&icgBVDjpUDEY!v3&O%iwG><85vKr3m+TsP`M`Nrd}ENLN=2S-v- zN;e3m6wdN5{9VEatiaTfSQ*RNcZh(PIdZ+&-3#=28uOgtM%HwNN3CfUT;}LRXr~4NtR< zpf5;dCqN~Miu&h9+ezBX7!>2fVZ?*Oh+`Z|4CmFZR2=3tFE%@2ZVf$`PLKs=Kx8sB zcLs#KcDZ+^0Bvclqa7J<`O33!rOY0 z=8W8tH)K*7Q&RRqtpZdGrwa0>S9__gb{)@|tGoAC*c#tFAJ)e!S1a(bJ>miYZYCbV zE<#Npt+2?-A`vnv5piO@%(+{O^}|Ar!_>C_3D;F7C(O*?WANm&YKM$~>rV<3?XZ|? zfd+SxbEq?tO8y@a(BC$|c3afZ;gFFu@+685y+-aAdt=fjH`igBwhPERWsmL|C=xKj zKwz-E*9AdT=T1&+BT+-7U+t1*8eSx*Q&%&IlE--TX;Z95G=r6h#oe@_x9)TtD$P{O z=F#RPYx#%xX*Ma#1V8;a3ZFh@qt>im$Tso^crwc7fR;Oam)Ew2C;fG}6sULquFX=& zEJ~k@q%MUjvqMk#sk-Us*g$fFu{ znD)rME<+`MbN(RU*{#5wn#B6=lp|xn>%-FPsjq6V7pJ%YpSRQ3h&oV{uH~Fj`7@*HnYN=N~C)gsOn*g%b-x&K`Bv zsOl_!`|>%Rj~tnN&nK>Ye$2(Q-lxtK{@ao{;SXNn{P_7Z>JQq|Im~~LmLFg6zIEja zkiKd2XQDX6S4k_t`ML`9PXB2i+W@-0KxYjorRMvdJq~s9WvwUpR^Zmi4c%EeLsHh}7YnAau?SRs93^`mJ4gGLU7?x7bEL8aBLJDM?XcNs zTu_i=U@S(73);FcI9E$RA>PQnxEGS6g8jB;N^uM;&;*zPLeNo^;UM9#;RPi@h?~9G zLRQWC?6!7Pf?L$N3`A5OzVNTSs6SnidHDP*@n-oS*`ugpl-z<-*NcR%Vs5y!-95vO zI9I;@`>laJR>TGJVeEeM-zFWrOH6L|PvK zEKvfy!RGjL3N@_uyf`?|y&mSY)dI;;u- zFF=}A8I5_CD=c=$8y!PY>4g+Hp0BjB!I7)tjmW))oaMMfVF?-hgih_4gkpNwa}MI~ zff>30MIVk7z^T1KbvVkH_^_K^0Tz_iox&IC7EQVdTh1feEw@3m8J?mm%JWBzrp*IU zfblJ=Zvkk99oC&9nn92Ml6WIJ3ZK-QT`%cs*F&mlZbT-@moiVH zY68yE=BCPwA9iFOP|)vmpCcuqUO8fIQhE#5?d$%ht1joND_E%-J#3$k#1zR2JiI>j zb=qHX6|`-s?|>(P{@FMCzM%L-;dZH5a{5U9ldj(N^qzrOE2-~lyxe69)2I*Z0{3CW z_vmtl>uK0PbBfC4mwP_Yz8@Zf{V?{|g@E7_fW#69ktr*Oh z%o8rjpbN|sI!)3c6^ujXP7}J1df=1P%bVxNN~))7aim|ts9f5={K@|#$@Ld=;Y48j z;ijGVN+^GFwmjpnYe1cUO9)q}*63shRFR5A19DDI#j=yyER|%5)&MmpnNoFXO_FjV zWowkGV*-h{Z56-u#MvlV?5Wl_;OC4ClUanXhaeON`2(K;4=nyC0lQ=`rwC5e$n zM;+eCs3`Lzt}syvCfcD!Qt~KMP5MB}BGh%TI!q<;S0iu7NYVfLo#akZq|Drra)+#< zc(sOXI`$~&JK5JW&qNCgzU%xt%#o4uX33p>qR30>*Hh0VlNo+C0(AmPBTwd^&Y+Se z(?D)O90?W15KW++LZsAYNIVg^vPH%a&B0u8!@~}=WAH>#>qJ*bc9cq^2&UmqLlM+& zSUfRHN#h!H)hA)7u2&9jobIrZQN5wPp}0at#u5!A9cU=1*@I;cB$32+q&!KuVkwTr z>q7u+le1-sGbgT0#(Ow{o0p*5x3eySB3&K zdkY@tmV}>?a4=j-#~3;icVlJ1&cj!f!=AJMyl^;VtO@ax)#y6ab=Y0^ig`?lBHLlp z4~o}9xl6rg@s#+qf9xD!MApeywd+~!(GIXfH#wL1ROBt|%T@3WkKOqj>aTugZ*BVb z;ZNEno~OkneYi?K^EtZN^YGehUkOR?Eu!(vtnWI7 z+gyLL{c~@>d(0oj<7{>O9Bw61`zfx!2&8)MxT9^L`#L17#s+%VoX(9c$?K?>U{^OMCuH z|MB^>8$A~D_4O07^-}-&{byI3RJ-YYsr%>elj2O*$z*ZA@8OsL|940C%*!aCb+kRt z^p~l7P;p3KHurPzQ~|##a`S5P^Ivc(>}t9E>t zru%pN+*JPtcZtj0Kg6&(OC6U_*iB5xYaFIUzHg0LZG{|Xdtar`UCNpE8ZwxPJy}9JOxl#YJT~?Dc+=L!g2X zeBW;$$0xh@vQ)Lc7blhuu2)SH5a@KjVt?A36EOId7>?bl^{LoVW8-$3Pldz&JXs7J zzk5fGRq0#L{!AofEGM;|1pe^!|DHwm*Zp`%3q9K}riuIA8|LS`|M33{>>50kz<2c< zZLwF}%grM|)p$BH_^nv;+wJlZU&_Cc;^!#ZTQlCvb#;9TB4T>}SbcIIa3x)$giLhu=Y@MetkjUtJN7m!%%B)3wLQMr^yA0$*lp-uG7f*^8bpz(-nSS*2 z=#IG3`>J%*ss2HkwRrpK=yH9|Fe;zFL2>%S+x(GkS^xV-zmw%$nO5G)^NhpyX47xa zT;W@BWBYc){O4oh-x5#yEnd@;d*D-$9{#U&^|X|?AJ&Iy+T?w(q4C~$lJ7#a+OFEu zngn9}R=4Tup{Dgx8?W|ryY)%@_WK&7-)G|9ntl0Ab6=$lN}pv^WJgCygB;hjSnWmZ zCQJJ5+s;;c+_%f6w=JEic@E-__g1$jqU?Eo|ivn)yFjm5~U<2u#+~`|ACTp2XppLtUp2TcCS(B%0MSI4VO7M z7EVpG85+?|LT(aHHj-1d%+zG9j-zSlj-e~}lnssjQ5cWqcXnF0t=JUsS=Dh=)U9JW zSB*~T7Pzmod^#xg{7)EilB&tkUHx@*^knqTi5NW%E8Ie3nhQ?Lou72g4i~cL7>!Dh zrS9f`vpkD$&7{2Qq)dR$Mz%BRRwsGw|CHJ=8V2nfN~apB;%A;tq^&deV;-uI`_8Uu z)lHIGc|9@i(T65t3Td(!gU}UKElp;e#wjtXpaqEorK(quTGqd(Iq{F_D+{HpbwNfF zjr_eC)2&_G2-(R~&K0De@5XGI?Vkhz$C>5s~x~DBk|8dNfz{BwO z3?330f9ozg*0UCjqnn{v83o)i)0r8ZVud-FXBvc*2tvBybgB|(ls@N_zmXvv(HHAn ze^@R`E0vAA7h2OaD#%@y&vy-9<9<{jIY@LA8zwIt1`A*ZLHhu#$Ze*fL_B`hLKonN~!t_c0Jn1AbDsNqTJ&V^mCRDYP=^Wf*RNOo`MvU=~oP<}sS z4Ntb-_{#W%6`(So#=L*EH>hH6#)b!a-$15T&zUTafST~Pfs)mG@*}}dD zIfe`G@Kg@>>g>aFS$FK$`X}pf_O8xl$ln#e_lAeior_KT#rnGc{RmHsW@q~66ML`1 zo^QhwIcNJT9?@zF&)cumpLXGFv<5zt-I>oZtQjOVJO4(deAU!$+@d__GO|0s&5EnZ8tMuZ=6#1G`fKPAR^Jo`yJj$XsITAQYI=k>?? z@B6P&2-~9Eh;DE3&n0`)LfN`llfj%gLd0*mDw|FdxLOQ9dh+M(&Udd4F#kGo;*4tJ zy6i%=hxY)94*&S9FJsebrR0*N_h+L?rH)idsd8JRJ1hWVzzB=Zz~u?Z;M2x?kxZnf zUxZ1tGQ^Fx3o{qusrrWLnBx2CrzApd8zEnL5!OPGT~}8mWpfnK-zP7qkI;+-CHIY@ z-`=5ope2hbbcgWDrCuo0xWsh_eW*s`Q0mOsYK!4;_^S3N2C68~4j=g=1Dzx63lN@lVZ$=~u6nGCQMi(Wo_!JW~YC=YMR^BFc(i|bG zNH5N0N`2MD-3B_42eZ0@;|YD#in-Z*eJ#&ko0F{2(IM(s2^c`ifUf_B}F+ z)hkoZ;R;ky1*6oM)pv%_K-hfZkYD%6aIGRzNAENMR*_ zC*ucCg_2qZKGcW^*auq1us{ZNZ367IS_P78SxzOsk-Zq~pis3S3t3 zTtuz~!n=^ELOe=7M2PsvX$e*o#6DupB?TmIPf5Lvh}uN*Y7{9Tk3Y6mkQe<<#gC=G z|C#s}OX=@-GCXZyj&5UqTxZu=rio2RqD|waRtL|<%VU%iIue$5xTxBp)p5*EC<@re zl9{9-JULWBnIba>S;xH`1+?LijhhH8jK83uA2xmm|K-z_%yWoWM=49EoI}7La;@v$ zrkI#0xxW{^uM~CeV$COmCjCI0F2`A&@P98?B6_DjBfC}k9~nXtHZn7PO9_xuqH@~M_F`a6?%O(z`o{N&aUZi z)QgB;B;!ac+-tCun(GOvtL1xoG(`LgnH10^p7lMq3;GIo;2`}ba_n1ie4fO2u+ z+h-3w_&Dfhwa~^1%ZBSP6PY_2z1+Jc5_|i-@E#`++K~P4zk@wRuWdR_<*pGUD}i=I zkQm^tw$Ag&a8*p}9{p>nt;sl~q?u^%oL1YSDT5^BCgW845uXO zGM^h63^`ds@V8)0X5PEAb=;DS6vsh)Y%*w@ID9GVtSue%COm;OP(rOR%6<%o{Dr&{99iJOKH{nOeX0l}QorU$!r->@z5_!3$6bYpm;e1nNE=I|+9 zl{|Y;65yv@#nUb6XdAMV)ucGbSxOpr%>n6J)xn&d77qh3IO?fu2Myag9SKt={LOEn zOmkP2lnsLtBDDL-8F?z@e;L4s;Cjw@Y80#_ol#tN)y8slD~%xL+iGyd4qFKy1-O;j zGUM}%HI&GLN$kV3+D9&$G!K*1HR2Qw79(ib)Km1%hZ9vA8^n1mW`SOuQk{Qhj5L+F zWOsRGGM7e*0|>zEnro+aXwe86-dF|u!NcLK@mv|)O5+f>*gy4 zgfUH2RfXJB54=zjpMm&>lvE5)Dmn2QqGid^SEGy=Crn_(Hj4aB#U z1lg{xRR3hChfgHS2Ix)c=rn|$%Yk^L!@N@4&`oKC3c%ZlAth@I#v7f`iM$f?IQa^R zt8G;?ix<@blZ8j!E0rvp)dtAV1}DmlTh%n9ck1lGdkY1o59_)I4MVfUUYeND<+qu} z(0Fu={}lKZa2ijJ#bR@aqlvfUl&%tcd))t`uicw&SEAoS1Q+(-!M-MX@^YLBy!KA{ zzi_(ya(G!m<8%gBrF&`+X zdSUm^{&v9^w8dwv0)OG=B~RaanW6aQ1vQuw-gOFwTszon5)Og~1Lne6c`_0N9QJ{Q zx*!a2x4DN!khI`m)&3g?+FwKR=P0NFqF)gffIWX@Dv_soSb}2o<6LhFed57(nsZQ^ zy`l?=3GN2II|5eQBaXv7U?Cwa1M-0fw4haVLw!JK@S;KF|kW`Vy!Y%|cI9K0_O3p5%V?2bkJ zLkkf5+Xc3-MiqN!|JMr>K>ptbmud%skwfke&)~BqG5&$^Uz`I8OXB<)pw~F?4#^N_ z7a*cs$OdVORiU?_6hHX=XGcO%Nz}V}it^|O2@SGRraFH<-I{nNSO4COQO+NHs$S@Q zftr4lQYn)neH0TSR2y6En_P(ofv_X8tj{sic*u8zwM$ic{DA+u<)H|%Ym#yzSK zUdY$oktc73McA9;kwa|EJmpHv$U82oSfsh=t7y?T_-(o9ABxz8{`%hy8S%NCBp!j% zD?l+r#U(I7sDFge@GETpW#Hklr7HSK&J!IV1^H;vg@m)ZwxbE`@Q8}!;n6H+0&Bo< zS?{k30>XO=vUSZY@>1ET*nb@p?ppMzsWk^dz-bOM2uSo+&VGdo8rPUg3OWyUZYqtA zaTE=W9gUM(Ni7aXP%vYo4Y(-%z`}*E!=hvTl-!7_1(5!U!IM^-GLj)-GOV#jl!+O#-WDbJ*8IT?>IT0wyX87VzaSX zJa`@)ch+l*t%bheug>q=tEmZwBOY4Nr1}GEj8(MNlvS0Ln)(fu=GL+@GxM2U9Dg|x zIc2#?_Mx_ITiEuEcH5oPz123l6VD&d&qwY2-+c6mUxl?_kpJ-&{cj&*4PTzX2NV#H z20joF=KuJLEDTNT?H%<0KX;LXXRW5T^A)`D|D=#n+B$V{U6mKReq7P_$Je zJi4meJZCBH8$Xli=Dy9>z*I4Ipl&+ZiYD9rRP-40j7(7Ug19gX*a z#sp52?MY8`EFJjXSSt<%%3E<<;rwxBh*BS-ihL^;ftr*N>04 z@(PES+3xD;{lgsntNDIk?`xR$5#W;A1C(rj-!El;54p-IQ8DO$YEmZYRa)bHP&k}$ zVN<2+4q@@M(cYVOHBC=NVi4>&(WW3*j3XaQ3z454Ip)ys&gq5^e&20 zwxhlPF<&6^*VL}&YF8J@l!&wko(Rjqg0)!>q5f6dIPob_j*_uoy(sWBVI_YnDIY-Wz^Kdmq>UD(S!1F=g3!EZA+znLDn>dChpVBa^iZPDZMD!dO_)Gor9@Bv#a}K| zfSDf=>V}@KLX+oyAxqCzg>_I-pr_%d4uh_N^eAYnmRH&5i3A? zuB})thT5OdBiIuazI;J}_&xcN5?!AsQLS}bkn+z^njsPxO#q=bED_@bCm{;0TLMLl zG%85@#GD-79@1_uVwg}1-y&F;9ltn@8+>>BG^u>dP?FSCQWfn$?bkf*{*m8md$s*UXlrn=Wxec;0(HVJi%=cCI~k@ z%p7!C%_H&N|K_2GA8vIYGFZffWR!{uAq6wnOf_eo6XO)7|MQ^B_y-hqA1catlg}_@ z?KN`Wa@7v*>Xz=FgSH1BWxS}<*BY}Kx0b0XA)#Q*-7I2N6 z#&``hxHsTdgjSvK*rbk#`YjrrSr~P5%9e661m%udRbH!DbnGdy)2i1LzfM%C=DbJI zak21r6`ze{95nTQY_%YtXUyZ+z}3CtH;w(kw-A6|24EI4njz%x+&Az&*ebTfuIza; zs6~KrvFaxW4mbb@aBiC{QV+;;%|F1h4{$VL{k4MF@<2+g81;y_)oTGo97*!@mM05b zF8U&EP$$Q9YG*h$FpOj;;{-2HKRs&QLkK9p)RX1jpV#fabw^3KfS9aUBkW7XsWcRL zmfWJYN?%blT_#gMWQibJzJ$Z6N1yWIOcqy(L#W%Y+spVlC}RI&&q5OkZZ@^l>BV#M zF}3Jp<~0sxzPpG3CD;Ys)(?*(hLs)96)u*a&%D*yV{-F0^cZ=)Du7nmmkKwfX75@*CD0 zjI%|{L7LyC%J<4;K&B0c8#4(;5t=7gTYrZeOIAt8?ESCC&H^}&CRxy8W@csvTWm2i zGcz+Y%VL%+mc`6uF-s$6W|k~w)@kqF?*Bi(z1y3Kj*f|#{JNvMsxvDqEA!inoo}}- z9I-4YINp{HJ_TE&Z%CIDoaK&`9?4;K;e8+8_s(LHZ_coEqj!@^atGQexuey& zvGrRFIS~vN!qWXdezGk+l{qJjWU}`+31Z@p09{q6iQXUK1;Zc=Is7obO{S4#TF_xW zFgN5)37lDr{T@URK) z1T)0Mh?VZXhVHLLqUrj8-YGOPCu;^_1XoAG1{CEFt|xIRP4>cy9Wq4FbA2T$b+9-v zz#!hFl`*g#($jW?+4}k(P$fC05JF29Cec-wD6-ONO80?;e1&`Fi)cXDluRH!w$>Bp zq#z$kAxgnDRMU9Wz$v@#k&hHcEhvyoZ9P-Ad&$$ieWRhgWXKYm5qzfWe9bFFnj0N zp6gy$M@Gzj>O{`Ijywy>>TTQW;N6?kl@gT=hxKCTdJU9^2RUdTDD^w zHw+j8H>T)xt?jaCi4C7KIMv@K$2foSv=<+7;x1!=%9}N)2g!w4l-scq#+ZS(*7&6&2OXwwv&{t4)ek~D24j zLVX1F`#=XE5{^NbsrBWUV!q}4+#u}ycyA-Tgv;K!XEdFSf4V>M{#0LVSMLXgY90k^ zG(k}hD?yWYOD-hKv`<6@Xu2GFBnT>Vo`rt3fO7=UJ z98Z9@_9Wt(p%sqrEGI|j%L-w&Vsz>lSM;N=kD=wgd#Owad4fb=IpHi7^tjnR13%2f zV*07aZ{nTnRItLz4MoCf_*;vTJUS+Rm9%74kOarp0b`>A5{KShzzzjw#2{qjruz6X zwbv35LDsE3L+z4=`2`*x0T00*6PVUb*8}u9R+Wu_a;}v)0_Gl-(z@YS^m%OMkw=r5 z`f~W^yfjehUVA zQqv`|4J4$%L`-^D4Go+;jDAz$VTdz#Zq4qmI0Nut$PB|7ZBo7u@n(u~jKf%++zEu^ zHppRh_NVd9~O#1qmniPnW2y23&o<}p$J`vN@X#Oy7Zhr+L*T$>V zNbg24a4_~wc`f{zP6V$V3W3{jkSA(aO#2W>(P`kd={j&vFpv*7cq5VR;AcXD<}P04 zCt`YHU5BjLn=~TUh=0Yrqf7>Can^INW)x7iG}l5U*2C9slPB2e3WM)lXvXoVmCNaigqp9Ak`hF`P-KOipGynVcF@t?~elHb5!P%)`5-GNK+d! zu7i?822}gl+H}nDVDr1;isBi z5(HU?Nyr}gu0rN}3($jY>Nh~>V%$dVGPrAKXgIn0@q@Lv4LzU6Pu^=Y5imH(;D<}W7v4JqA;~O^bq16boK8I+#T6uU#kJhdQzTccvn#-Q!+P{16 znzs6LAFypqw_qvL;*Pl&l z$zJm-3LquZc(z)HfF@_j;%sc15zk`i3JCgrH9olGJ6GCVpD}{^gOQfL&iDjH`IDW} z$kG_s$#acP*Jq^7(L8m|a|Ib*w0lzS>ph35S1qgb z07iQC&Abn)uaP-&jgg!W*UC!|1bP*bLW@;(RJKNXL$YPK#3`JODZ<67N5#OD=*3ne zs^+ae#90~sM8Ur!Bqpq>{gU)6XvS%Nm!XG>Hj6aFMVp*&Cas4Cfv>CIaUOGPs#~$lbl;O%KR-G&I}hAJ?(w-tJ@afRL^xDg(rNl^o`nHQUSQW z7y)45OU4JHCfMv0cqJJLMaB3IONe>KIC5`EZ)Qn?5{n3cBDCiY=Ea)4KD z%b)rG!cDk$pNs)%r0(4>s*e+V&Wiy7az!^qZ&Tf=Zp2{(UzY)oP#yJ2gk99gy*f>gSWR?*U_Q4}Dqb@6rLUar% zhr^8amrdHuo=dPJ_XRpB5;vM$Mr>SbF=AnrL9z7V4=~nV795S?bmg7rtovTMqk`c@ zDaYsR%ULJMvkdZ}zRLB!13qIyx>Y^?I=3^x`=nC+Q&_g{Y>c+n@F6CzMK|s&E?l*= z{frj(Op#e@hnf6hHnF$_?)!H{blo|M9mB{X#ys=m*~&|?^elEI6&ycoO4mhblkUw$ zeJz`obpE8T1~zcmWZ$((3aw06GKq?WP&lFB(C%&&owA+$1~2CRh&3Mhg{UVgB$45ukFomV1&u7Dg4xB{_F*@I_}p zL}_o#B zQT?t^@0?Z;uCG7C_f1_h;<1S1+(y_LBh=SIEhHCZ9iL%wcJ`DiN=l^)rt@5)-tt4w zlr3Xtk}9cewH_hP!stlwzynYy_&oy5tcD^jo#rf)vA|O92LBxQRaD28JZ@)9CU`O5OvqTsb-T& z-GSq&q~kBZao^^A8_M74oktL92P$}64|A9!&$vHIM{S#85^csssy`ApMrRMb{-H!8 zZqdynr+S;-O%G13vvZtZLg-Vp(QIH5IPWzsOXCz(c?_+*qYOz+0Z3_7O|#7m^=>!I z6r@pTNUfI**qXsZI3(Gub;Kt(mx;Ibe9?7Xc4*o2#SqR@Bc#HX`WYVMC)F2&6*xqE zEK@>9IQpB+=Z-pyor14jBAv|~$KSAuab|B+NKjOHcWBDGY6nQBU{CH?4rIa2zY?-_ zN3BLUek3(|7zwKCUJrZ=C~WM0L>Bs2rrM8b1oj8tLTNyTU4pQT!>f66u0TAyku=!R zmpZ1}LwuERO(idKT_gSIufMA3>T;#L8`#mT^|%cVq1y(lR-cbivc4*k&sKED9ZF;? z!g+R>#=A=QOfsY2tX*sLJ5-BjQuQr*+u?0j@$rz(GR;ao#M+Xc_IKFZzB zDe3%7bsF?b!u>EvpahgDk2h`HYVCSrX$n|boD0jfk-;WO7nG2{lXoN@h>=mAba;H) z;Fcv6Br1kSaMLLkoRsM056k1?oJOw`lqpZgcyGn8siyyyQET#?OJmjK&o;s6@Aqs|Kq;|I+go5R{OI~y`nP}h9e6EO;e|nffXY6X zYS91mJLnminp%A3d}~yZiCkquYlnO#RQ9aVS*tnKO$i}bsVf%5fuc`B3eFpj?oNp! zYtDFo=G92xLn#LsluH>tbxe<$8z)OHK<(Cjtu(|U)n!-27QWY1tu$ErMj2_Wurflv z5cp*rdbV!wC)PVg$&Qzn$=oru1`$@}mNEg>^)9_;&X@v>a}M1StVy&0tzx}dVFB0g zNB;(e{1`~sOgeLs2GLEhWS9IO3p!T!YX%O~jXb2Fl_1buZX|R7|VNp7TzIVaN{7{)vc!}uiueluFB@7Dg( z_k7+{{zDz}#OgW_l-$*yW_WbQ%;Fg#XfNmBM>PMruxH^3HtiS3pmL2DYk%i@ymUf2YRpN7#SHGMpLs#+ShhdMAaBsS{Qd zWQ+3})^)6AQI>$N@}7l`H5jsebs8~zfg86mIq2w&T!B^Q=&o^qYwxsbg0bLgv0BLX zW6#ruKo!+Em4K`HAo#YJ!JX+;O=D9=?rFO=k{B%fEp0v4(6rD{bc$Y#hz8;ssGv&t zBT$r=b`l@RKdnFK_XS&6j#u!<-C1TWlWV5#ajyFStA*nZC|(7dI+k~j@^qLNmqXBo zeWT8rrFN#C1*z>NOZ`}{kGyk#PU+ylL>~GQ@_WY$npQ%B;Fo1yY|82L^Gjy@A zHa2nm?0%K%R(7~U=x+@C9|qeX#MGlB?Szf=-4<5eCAhfJN<>iFh6HYT;8coReYJ@v zT@UB$#A1ltl0h>N!siodP6r!a=VP{h9*vrIZR5q`SJPJt8r2MK%95LRyLYLKdhm-c zEKwY7{;`Dm9mi^;=j}#>X~wrtH&+T_IqP+D<#6jaw)Hx*)*O>J1`)>ti2BhD;o zc!V~5I>7Ho`oG#veeq2n&sQ<+Hs6{2Zrs3XXK~eZeO{_DtM%T{KRVyu@i+F^?R;4qw+Z*l#Z0uXgUN+8JYFtJ=m{v~_&a?V1eQR(bFrAEMfPem}w;k^u$it z&RHYN-B#|>{93UBiH(qVq2P_oG_eLnnE7(QrA1q^=YpKNz5BY+ee64C*(`f``HXw% z2iH#+JKS+cV%@y7f%-&D_l(zM?-cK#22=4-vj9mJi&D~BI{0GJmF25`_jDdT!>F*v z$n;ISmD>>B1lU3ga*L(+INSA+8EGTrv?_GIiCV7BA+2q{=)(d!ZDBQwhhq`%Lp>yp{_{06+OX6&d=2XtJiRNjI(YIA=)(H<+ z2j9b;yx3+&UYuf&gb4-$cx$~J{W{s!u6B`;5S2{@#tE%y;P~Oz^{NDiM`HH0xBX&hJ=Yw z1)wOQ_d0HfM<5h60Lm-&k|M3i_fq@({JZ|( z7i95$T%{%n3)66kBl)_9eli*AKu$x+k#Zs(qw5|GJ&=H?pnfHyxM*g9X)s4#twd+g z<1tpDyM`FU=)RtYPO>PMh&^igU&8Zy&=~c7JWA#wY;=dx2`#MP!IY&ab>k#>{Q4pL zybh9is(>&u^Q%zG-NK)_R93{|K0x5oT5Os8E46uFDdQqq!9c+xbZLJ643K8X_KxQOC(-1_y8=7b$`V+yI6Qc>t{P zKju3DN^7FoR)XYa1eSSeh7?AEkgCKaKormV@StlUnp`8`2UZe4gHdXnAyn~y-Ki@= zq7D)zRf$BTCzKB0wx_NjQ?-%gIl%WPIBO_#`=_L$NC*9E$aJT}q@4}>)(e?G zckdE145C+)O?$BkoX$&WRbyfKNbtJG&)_(PqMA!zx&$t*KHXfEMO^xbNX6X<>mRqA zkE+z@oYXL}kWe>>;E|teo)R&T!1aU4kuF=aV1ova9fMJnK~yGpM>$@WJx*NqxAGYf z&=QP0&k)KbVm#&@zQU&-PIU}+7WQrq* zL6^-+nPx;x9!8xE$$f33fCMQPi>nBS3jH*KCXOc@ww5i92N)r9hK!N2J6eR@jm1ca++!rtb5!eD= zB?y2yH`0IxsikUZ$_uDT7F&~-$Q~B&Ljm-ygQqe`=t{O&FfZ@T8x|Fnr0LIqm4P@z zOSOc=DNp3HXnsN6QT^T#w^H5A3`S52lNp#Bl6Uf%f5xoJu|D($q>dP=Ni4RL0_og# z!%U7S+>07=lYglQKZOFYyF9!@59q@euiFyWm001tNN>l%GE^KB>^aL?paf?k4yU#+ z#7~U|ScXM^MNO>trv~)zsVM8h8unVbV<1OE&t?njgGmC*>|qDgF=Bj^C5lq&-Go<; zU}JVl7-G3s-Xe<)0z95dUlTF&1Pdgpy?p$T$HR7(`ErhG4sDs|!J%yqNdJ0LC91$; zZm%FE8JX0Q>pEUpC#f%iFMZTlCm;D`B(S)p*8QrxHy25Kx9P~yf`~_`N1tQF8wLk4 zDPP9Eb^{cEn=l6Y#lz(+|H|E2R9Pqw_S<5A(vLohVq8?U5iKS9m|s+_nuJO*%%l{m z+?K4wO*2PUtl^=f@)>Zl+4if2lF(dK()#t>?i9IA^wwC!i>|7Ib6eusqtHR;>Ft`S zMuNX$NEU<_gNP(dLM?bCzF1N52q$TBQ8+XlsVm|LMGUFKz|0gWyR#CX&{kkrIQNcd zwyRUcK!X^l-AG43!03}>^qNER1O=9{QI9xe1_UO6XYUv%sy%E?yWhph{zhFQiXf%X zNAY%yme@%eP>IC76{=WREX~ykmsrYGzl{bwn-^A&lN|(89@hf47y?WriFnSo;c>#}Ec<^F1p2VDj>S zyuYwhK7B*n2-84}UAWbMb#CV%iI2lj0bDUV_`B&Tw4MytX}ZmBtKj>s+^yGx0>68{ zx;Ob%;qeB4<0BMpirj6O-JYyB!_RT~I4)urH+4C>tAvVbt@?C{bF?vVcKS5DvQ%~KPB@T!#CG_iS`fjh&k*@{fYFz9OOwj{@3+e$q<4@a z_3LBmbzEIOUM7}gcli0v8BoOgcb!ZR@M2l9;LeljWT2&s;#63e1tpkZ9KGf3nWQg% z1xO#0vbw!(++H#3+U4Y^F=jl`=~n2b%#P#O{zNJSP8QI}l9j~Gex^6+3VYd2qC9b* zX{Xx_-NX38o}5CSBX33#M(y`dcD*6q+Q%YPN02r__7#Gc-oRZH~;;BAs18_ol zlttO${M^qh)F!Q=CCRpBr!5OcX#^iTh&;Y`n8m;bn|Xerv;pC3t&Gf_t)L3_AkTdC zZS?j{t6scv?DS9{huS5iRKv9JHW9|0?;-TFPR~iPVf&c$R%3fkszrU9)!24B7Z$W& zOd`ij;CWw52o7jaeJE|{E2H%UBLOwAm5dvivL&AW`dY0-jA9Vs1ZKzc1p@etmW#aK z)gnfEi^>KMd2xn4%ktaQaU88}5Ks{pv0z?-f#6KdphmF%4@JO7NsDxa4X)Y>s9A=5 zEPLibu!`8)^LCoHdPj^KWXUQ3Ps@UP2PV9R#oN=^j7u5E;9l`{T&eA@M-svpsS)TJ z0zKZW=CuuJrYp7oJUjiH*60GUq%A~2m+YdiwxW2Bt;9pE`7``~mH9aJ{E#Ysd@e+8f zB5q=*PKTWr>-hf2`fs&g@#zzs4z=wyKV)q6n ztc|g_(;3PhL{9Sh0KP|0B!)9E@&QpeEY=yfnX^P_E;r15ZXVkA?WF%bCUA}QH;D^W zjdjtHJ#l%iMVEIYqg?bPNU{EV z>TjlFZFvC$wO{#ivn{r(lufU#u=ELbA(38(F;5O@7gl?3NNhX5*6sY5GJIh$ws3N$ zx3D#}V-!$P7MD|0%2FJa?PEe}7dzoQ#cqt2=gg%NtfV2n673Es-{lA|J!^a|fzq2? zAHN~vdvs6bEWY5B(uIK0 z$(e(ya%z2xe8qheS(g_Gxw1QXzu?N|sKh4D?^lP>zLan2;ZvMNiTE&i8b6%q50TJp zo!Herc8_04-XiV(d&6Hf79KHvR@8uj`2T5mX$c__StXH1r5@W|Hlz;NDL%OS2I{$n zRjB}5g`^bo41c;ZGW6=$2w^)Ybwhpt^z{R`&@yf_|Fn`*jfLaExR;%-X@339noDM+MS{M_nkL=gciNBl;5#Xw=&=vo4Vu@ zzbG`h&El=?dbs+6S%=&h_e3*2IUQNIdpH~6ZfqU(F_~PR98g{o`@Y;{QUqPBPrJ|) z8&5&NPO`z-X4@c6_H?xnAl@i(?}mva%>uewuV|fpPw~IX)LPWs*ja!)K7hVvEr7kd z@6SFFNKMX0aUEcIpqRcK=^Y*K%m#umJukbk^`Wr$Km$V`x}Ge31fj^D$yVZ8w|ubE za9j_@(xT5T=frP(^h03$R4dtVJM4*Z^3qw6Ye9?!2M%>@>S-)KUVm<366{33*3mQ3 zu`UR##iy%{pqXgdU?M)V=T4X0;l&e$US{-H=4i0HG^ge*=?lX5YHv(JGiG^Q?lqj@ z@ktGkX8vy9|1gjOAUsrq=_A)>`nDR=8`2TOf=Q*ii&A5u>0A)L3c7T{v{TVpK~pHZ8J= zu^$FNRYZ1Zo5*}5V^rypa!Wg|b7ICxySHy=(_WrJGR5}H~mJZsK@k>|VI+P5!VOpCRM?-iSHTbXu_$6Ii*j6m+d)M3{knJH-Gc@OEHs#};^aa16@>cjzu*05tP1V?Gp!w!}Y zBS{}kj+xAA*ci;Hxkz7It*K~OgE0AZ12@~G@lwW3I*3U~MdONA>v^~hNdv({JRzvE ziHuHm2wDP-9uhAhXDAjg-kp5)dqNUy^*Wl{X}~k34ebzRGbZDj?p%J4P$cgh8K?K= zx6}JrZr4>qBH6&eaBeQ8@@TUCpO;76kePktWcH{yZTwvppAPB{wd@L#j|c0Ya_`Xe z9qrE_0QXMp6}sU!p#^R^miPT8>wb+AE{JBQW7ze%n(wEPUc=F~&(bPULa;fruo86g zDgl5t$v7IwD`OV8>D5zX=IR3Z&&0I4T?uD>B1idmzcT4lUJT!#o{F}RU^EpW%dH|6 zMQwx1&U^I{_BeKVY=`b72ZkEGJsUNzEE;#xnb`)!ewO*SCYNWhV7MWcF&*W(_=~NF zkYhEX@-f~~rMYSyS!<*$tTTK2Fe8(U(|2?W@Kp4Y z;{fT;KS_;I%kJ;PNJuv;iZd`Z&4Hutr+Hk3|DrafXxDDn{#(r8KUA^fV9~>;4;2XV z^Pv5Ycj^4O?ZVo`)x?^?+1;6TLW+*6WDn#I;)*KneC4NN_x|khWdD($~i zNXS!1gV^%ng;ke+0EyJh)drshLzcG}Vxl0jPXo{Cj5?yAEpH{&&RT!S;Sf@La{2ag z*olOzfS*K+1Xg&s8&35jZO%wNTzkvUs7ZJRF)JvZ{|?H~N$w(Va0CSF-~ zQEt%8%r{Ugl4|zyNjnoryVH4NS&DJrb2VZP-;>&zBVU=FWz3-RZ*Q2bJWkek z$rswz?7WIu#>L`(fo%alO3B!3J^MVs7nJids9WWdpWU>4ozbdhLy9i0L5AeYv2*s_ zBk<;bLKb1l1KK$GCFgK=K_G4kuDV7u(waS12KEtWEj2B7kXL+7;j(^d-d^wUndQ{` zphVTF7TmwM%*i{=rHT^3LdH$z&)?OJcmiQTp?L}KVTn!3pja0Wx|RH`R4mN+Vq8r| z2EVDvrcK0s(|YnpE-<9t5#exxF(3iM*FNH=K-RQ$V^1)RtLIVvf* z&26GIl3mwlcLxSR1Ny<l?(u|NGMv@Q>5~tiAT9{GVmc{*N>eP*cFr-{t>R^z85Q z{}{r5R$ux{-u`n4{`30(6$Si1Ycc&D_wP1c{e>Im_!sWaEm(ht{JT)(zaS3+|APE0 zF!@jEKNl4LlD1R)L;9b~i+@M_yTzfu5C_Wt4)M38qQ9g5(I9_%I{rc#>;8%Q%kS}b z%D{;3H6VupO` z#{c)+?|-YsKY@ShtiOP?mj4O(-!#~t;6GK@Kfw*aKK~K?e-xR#6!_=V3J3_{^NaU6 L`mAmK@#}v997rBW