From 30486544841051038baf671d0c5865b8cc91804d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 20 Feb 2025 10:01:30 +0800 Subject: [PATCH 01/40] refactor(stream): kill too long checkpoint trans. --- source/dnode/mnode/impl/inc/mndStream.h | 2 +- source/dnode/mnode/impl/src/mndStream.c | 40 ++++++++++++++++---- source/dnode/mnode/impl/src/mndStreamTrans.c | 16 ++++++-- 3 files changed, 47 insertions(+), 11 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index fc1c95a3b3..def817377d 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -116,7 +116,7 @@ void mndReleaseStream(SMnode *pMnode, SStreamObj *pStream); int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb); int32_t mndPersistStream(STrans *pTrans, SStreamObj *pStream); int32_t mndStreamRegisterTrans(STrans *pTrans, const char *pTransName, int64_t streamId); -int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt); +int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, SArray* pSlowChkptTrans); int32_t mndStreamTransConflictCheck(SMnode *pMnode, int64_t streamId, const char *pTransName, bool lock); int32_t mndStreamGetRelTrans(SMnode *pMnode, int64_t streamId); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 247024b283..ffd271fd00 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1258,16 +1258,48 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { SStreamObj *pStream = NULL; int32_t code = 0; int32_t numOfCheckpointTrans = 0; + SArray *pLongChkpts = NULL; + SArray *pList = NULL; if ((code = mndCheckTaskAndNodeStatus(pMnode)) != 0) { return TSDB_CODE_STREAM_TASK_IVLD_STATUS; } - SArray *pList = taosArrayInit(4, sizeof(SCheckpointInterval)); + pList = taosArrayInit(4, sizeof(SCheckpointInterval)); if (pList == NULL) { + mError("failed to init chkptInterval info, not handle stream checkpoint, code:%s", tstrerror(terrno)); return terrno; } + pLongChkpts = taosArrayInit(4, sizeof(int64_t)); + if (pLongChkpts == NULL) { + mError("failed to init long checkpoint list, not handle stream checkpoint, code:%s", tstrerror(terrno)); + taosArrayDestroy(pList); + return terrno; + } + + // check if ongong checkpoint trans or long chkpt trans exist. + code = mndStreamClearFinishedTrans(pMnode, &numOfCheckpointTrans, pLongChkpts); + if (code) { + mError("failed to clear finish trans, code:%s", tstrerror(code)); + return code; + } + + // kill long exec checkpoint and set task status + if (taosArrayGetSize(pLongChkpts) > 0) { + //todo: + + for(int32_t i = 0; i < taosArrayGetSize(pLongChkpts); ++i) { + + mndKillTransImpl(pMnode, xx, ""); + mndCreateStreamResetStatusTrans(pMnode, pStream, chkptId); + } + + taosArrayDestroy(pList); + taosArrayDestroy(pLongChkpts); + return TSDB_CODE_SUCCESS; + } + int64_t now = taosGetTimestampMs(); while ((pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream)) != NULL) { @@ -1304,12 +1336,6 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { } taosArraySort(pList, streamWaitComparFn); - code = mndStreamClearFinishedTrans(pMnode, &numOfCheckpointTrans); - if (code) { - mError("failed to clear finish trans, code:%s", tstrerror(code)); - taosArrayDestroy(pList); - return code; - } int32_t numOfQual = taosArrayGetSize(pList); if (numOfCheckpointTrans >= tsMaxConcurrentCheckpoint) { diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index fe3359dc74..ce82522029 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -31,11 +31,12 @@ int32_t mndStreamRegisterTrans(STrans *pTrans, const char *pTransName, int64_t s return taosHashPut(execInfo.transMgmt.pDBTrans, &streamId, sizeof(streamId), &info, sizeof(SStreamTransInfo)); } -int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt) { +int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, SArray* pSlowChkptTrans) { size_t keyLen = 0; void *pIter = NULL; SArray *pList = taosArrayInit(4, sizeof(SKeyInfo)); int32_t numOfChkpt = 0; + int64_t now = taosGetTimestampMs(); if (pNumOfActiveChkpt != NULL) { *pNumOfActiveChkpt = 0; @@ -63,6 +64,15 @@ int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt) } else { if (strcmp(pEntry->name, MND_STREAM_CHECKPOINT_NAME) == 0) { numOfChkpt++; + + // last for 10min, kill it + int64_t dur = now - pTrans->createdTime; + if ((dur >= 600 * 1000) && pSlowChkptTrans != NULL) { + mInfo("long chkpt transId:%d, start:%" PRId64 + " exec duration:%.2fs, beyond threshold 10min, kill it and reset task status", + pTrans->id, pTrans->createdTime, dur / 1000.0); + taosArrayPush(pSlowChkptTrans, &pEntry->transId); + } } mndReleaseTrans(pMnode, pTrans); } @@ -101,7 +111,7 @@ static int32_t doStreamTransConflictCheck(SMnode *pMnode, int64_t streamId, cons } // if any task updates exist, any other stream trans are not allowed to be created - int32_t code = mndStreamClearFinishedTrans(pMnode, NULL); + int32_t code = mndStreamClearFinishedTrans(pMnode, NULL, NULL); if (code) { mError("failed to clear finish trans, code:%s, and continue", tstrerror(code)); } @@ -160,7 +170,7 @@ int32_t mndStreamGetRelTrans(SMnode *pMnode, int64_t streamId) { return 0; } - int32_t code = mndStreamClearFinishedTrans(pMnode, NULL); + int32_t code = mndStreamClearFinishedTrans(pMnode, NULL, NULL); if (code) { mError("failed to clear finish trans, code:%s", tstrerror(code)); } From 6febcfe839569fabae35e081e0dec965b829ae62 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Thu, 20 Feb 2025 10:29:26 +0800 Subject: [PATCH 02/40] docs:add taos_slow_sql_detail & log principle --- .../en/14-reference/01-components/01-taosd.md | 42 +++---- .../zh/14-reference/01-components/01-taosd.md | 42 +++---- docs/zh/26-tdinternal/12-log.md | 107 ++++++++++++++++++ docs/zh/26-tdinternal/normal_log1.png | Bin 0 -> 73856 bytes docs/zh/26-tdinternal/normal_log2.png | Bin 0 -> 65991 bytes docs/zh/26-tdinternal/slow_log1.png | Bin 0 -> 457097 bytes docs/zh/26-tdinternal/slow_log2.png | Bin 0 -> 47342 bytes 7 files changed, 143 insertions(+), 48 deletions(-) create mode 100644 docs/zh/26-tdinternal/12-log.md create mode 100644 docs/zh/26-tdinternal/normal_log1.png create mode 100644 docs/zh/26-tdinternal/normal_log2.png create mode 100644 docs/zh/26-tdinternal/slow_log1.png create mode 100644 docs/zh/26-tdinternal/slow_log2.png diff --git a/docs/en/14-reference/01-components/01-taosd.md b/docs/en/14-reference/01-components/01-taosd.md index 7456593ddb..246469f6e9 100644 --- a/docs/en/14-reference/01-components/01-taosd.md +++ b/docs/en/14-reference/01-components/01-taosd.md @@ -532,29 +532,23 @@ The `taosd_vnodes_role` table records virtual node role information. | duration | VARCHAR | tag | SQL execution duration, value range: 3-10s, 10-100s, 100-1000s, 1000s- | | cluster_id | VARCHAR | tag | cluster id | -## Log Related +### taos\_slow\_sql\_detail 表 -TDengine records the system's operational status through log files, helping users monitor the system's condition and troubleshoot issues. This section mainly introduces the related explanations of two system logs: taosc and taosd. +`taos_slow_sql_detail` records slow query detail information.The rule of the table name is `{user}_{db}_{ip}_clusterId_{cluster_id}` -TDengine's log files mainly include two types: normal logs and slow logs. - -1. Normal Log Behavior Explanation - 1. Multiple client processes can be started on the same machine, so the client log naming convention is taoslogX.Y, where X is a number, either empty or from 0 to 9, and Y is a suffix, either 0 or 1. - 2. Only one server process can exist on the same machine. Therefore, the server log naming convention is taosdlog.Y, where Y is a suffix, either 0 or 1. - - The rules for determining the number and suffix are as follows (assuming the log path is /var/log/taos/): - 1. Determining the number: Use 10 numbers as the log naming convention, /var/log/taos/taoslog0.Y - /var/log/taos/taoslog9.Y, check each number sequentially to find the first unused number as the log file number for that process. If all 10 numbers are used by processes, do not use a number, i.e., /var/log/taos/taoslog.Y, and all processes write to the same file (number is empty). - 2. Determining the suffix: 0 or 1. For example, if the number is determined to be 3, the alternative log file names would be /var/log/taos/taoslog3.0 /var/log/taos/taoslog3.1. If both files do not exist, use suffix 0; if one exists and the other does not, use the existing suffix. If both exist, use the suffix of the file that was modified most recently. - 3. If the log file exceeds the configured number of lines numOfLogLines, it will switch suffixes and continue logging, e.g., /var/log/taos/taoslog3.0 is full, switch to /var/log/taos/taoslog3.1 to continue logging. /var/log/taos/taoslog3.0 will be renamed with a timestamp suffix and compressed for storage (handled by an asynchronous thread). - 4. Control how many days log files are kept through the configuration logKeepDays, logs older than a certain number of days will be deleted when new logs are compressed and stored. It is not based on natural days. - -In addition to recording normal logs, SQL statements that take longer than the configured time will be recorded in the slow logs. Slow log files are mainly used for analyzing system performance and troubleshooting performance issues. - -2. Slow Log Behavior Explanation - 1. Slow logs are recorded both locally in slow log files and sent to taosKeeper for structured storage via taosAdapter (monitor switch must be turned on). - 2. Slow log file storage rules are: - 1. One slow log file per day; if there are no slow logs for the day, there is no file for that day. - 2. The file name is taosSlowLog.yyyy-mm-dd (taosSlowLog.2024-08-02), and the log storage path is configured through logDir. - 3. Logs from multiple clients are stored in the same taosSlowLog.yyyy.mm.dd file under the respective log path. - 4. Slow log files are not automatically deleted or compressed. - 5. Uses the same three parameters as normal log files: logDir, minimalLogDirGB, asyncLog. The other two parameters, numOfLogLines and logKeepDays, do not apply to slow logs. +| field | type | is\_tag | comment | +| :------------- | :-------- | :------ | :---------------------------------------------------- | +| start\_ts | TIMESTAMP | | sql start exec time in client, ms,primary key | +| request\_id | UINT64_T | | sql request id, random hash | +| query\_time | INT32_T | | sql exec time, ms | +| code | INT32_T | | sql return code, 0 success | +| error\_info | VARCHAR | | error info if sql exec failed | +| type | INT8_T | | sql type(1-query, 2-insert, 4-others) | +| rows\_num | INT64_T | | sql result rows num | +| sql | VARCHAR | | sql sting | +| process\_name | VARCHAR | | process name | +| process\_id | VARCHAR | | process id | +| db | VARCHAR | TAG | sql db | +| user | VARCHAR | TAG | sql user | +| ip | VARCHAR | TAG | sql ip | +| cluster\_id | VARCHAR | TAG | cluster id | \ No newline at end of file diff --git a/docs/zh/14-reference/01-components/01-taosd.md b/docs/zh/14-reference/01-components/01-taosd.md index 8dc0257cfe..d58e0570e5 100644 --- a/docs/zh/14-reference/01-components/01-taosd.md +++ b/docs/zh/14-reference/01-components/01-taosd.md @@ -1684,30 +1684,24 @@ taosd 会将监控指标上报给 taosKeeper,这些监控指标会被 taosKeep | duration | VARCHAR | TAG | sql执行耗时,取值范围 3-10s,10-100s,100-1000s,1000s- | | cluster\_id | VARCHAR | TAG | cluster id | -## 日志相关 +### taos\_slow\_sql\_detail 表 -TDengine 通过日志文件记录系统运行状态,帮助用户监控系统运行情况,排查问题,这里主要介绍 taosc 和 taosd 两个系统日志的相关说明。 +`taos_slow_sql_detail` 记录客户端慢查询详细信息。子表名规则为 `{user}_{db}_{ip}_clusterId_{cluster_id}` -TDengine 的日志文件主要包括普通日志和慢日志两种类型。 - -1. 普通日志行为说明 - 1. 同一台机器上可以起多个客户端进程,所以客户端日志命名方式为 taoslogX.Y,其中 X 为序号,为空或者 0 到 9,Y 为后缀 0 或者 1。 - 2. 同一台机器上只能有一个服务端进程。所以服务端日志命名方式为 taosdlog.Y,其中 Y 为后缀, 0 或者 1。 - - 序号和后缀确定规则如下(假设日志路径为 /var/log/taos/): - 1. 确定序号:使用 10 个序号作为日志命名方式,/var/log/taos/taoslog0.Y - /var/log/taos/taoslog9.Y,依次检测每个序号是否使用,找到第一个没使用的序号作为该进程的日志文件使用的序号。 如果 10 个序号都被进程使用,不使用序号,即 /var/log/taos/taoslog.Y,进程都往相同的文件里写(序号为空)。 - 2. 确定后缀:0 或者 1。比如确定序号为 3,备选的日志文件名就为 /var/log/taos/taoslog3.0 /var/log/taos/taoslog3.1。如果两个文件都不存在用后缀 0,一个存在一个不存在,用存在的后缀。两个都存在,用修改时间最近的那个后缀。 - 3. 如果日志文件超过配置的条数 numOfLogLines,会切换后缀名,继续写日志,比如/var/log/taos/taoslog3.0 写够了,切换到 /var/log/taos/taoslog3.1 继续写日志。/var/log/taos/taoslog3.0 会添加时间戳后缀重命名并压缩存储(异步线程操作)。 - 4. 通过配置 logKeepDays 控制日志文件保存几天,几天之外的日志会被删除。比如配置为 1,则一天之前的日志会在新日志压缩存储时检测删除。不是自然天。 - -系统除了记录普通日志以外,对于执行时间超过配置时间的 SQL 语句,会被记录到慢日志中。慢日志文件主要用于分析系统性能,排查性能问题。 - -2. 慢日志行为说明 - 1. 慢日志一方面会记录到本地慢日志文件中,另一方面会通过 taosAdapter 发送到 taosKeeper 进行结构化存储(需打开 monitorr 开关)。 - 2. 慢日志文件存储规则为: - 1. 慢日志文件一天一个,如果当天没有慢日志,没有当天的文件。 - 2. 文件名为 taosSlowLog.yyyy-mm-dd(taosSlowLog.2024-08-02),日志存储路径通过 logDir 配置。 - 3. 多个客户端的日志存储在相应日志路径下的同一个 taosSlowLog.yyyy.mm.dd 文件里。 - 4. 慢日志文件不自动删除,不压缩。 - 5. 使用和普通日志文件相同的三个参数 logDir, minimalLogDirGB, asyncLog。另外两个参数 numOfLogLines,logKeepDays 不适用于慢日志。 +| field | type | is\_tag | comment | +| :------------- | :-------- | :------ | :---------------------------------------------------- | +| start\_ts | TIMESTAMP | | sql 开始执行的客户端时间,单位ms,主键 | +| request\_id | UINT64_T | | sql 请求的 request id,为 hash 生产的随机值 | +| query\_time | INT32_T | | sql 执行耗时, 单位ms | +| code | INT32_T | | sql 执行返回码,0表示成功 | +| error\_info | VARCHAR | | sql 执行失败时,记录的错误信息 | +| type | INT8_T | | sql 语句的类型(1-查询,2-写入,4-其他) | +| rows\_num | INT64_T | | sql 执行结果的记录数目 | +| sql | VARCHAR | | sql 语句的字符串 | +| process\_name | VARCHAR | | 进程名称 | +| process\_id | VARCHAR | | 进程 id | +| db | VARCHAR | TAG | 执行 sql 所属数据库 | +| user | VARCHAR | TAG | 执行 sql 语句的用户 | +| ip | VARCHAR | TAG | 记录执行 sql 语句的 ip 地址 | +| cluster\_id | VARCHAR | TAG | cluster id | diff --git a/docs/zh/26-tdinternal/12-log.md b/docs/zh/26-tdinternal/12-log.md new file mode 100644 index 0000000000..6a401e8446 --- /dev/null +++ b/docs/zh/26-tdinternal/12-log.md @@ -0,0 +1,107 @@ +--- +sidebar_label: 日志系统 +title: 日志系统 +toc_max_heading_level: 4 +--- + +TDengine 通过日志文件记录系统运行状态,帮助用户监控系统运行情况,排查问题。Log 分为普通日志和慢日志。引擎测的运行状态通过普通日志的方式记录下来,系统运行相关的慢日志操作则记录到慢日志文件里。 + +## 普通日志 + +### 普通日志实现逻辑 + +- 普通日志分同步和异步两种方式,同步立即写入日志文件,异步写入到 buff 里,然后定时写入日志文件。 +- 异步方式日志文件缓存在循环 buff 里, buff 的大小为 buffSize = 20 M。如果某次写buf 的日志大小大于buf 可用空间,本次日志会舍弃,日志里记录: ...Lost N lines here... +![TDengine 日志循环buff](./normal_log1.png) +- 异步线程里每隔 1 s 会更新磁盘信息用于判断是否有空间写日志 +- 异步线程每隔 Interval 时间处理一次写入逻辑。写入规则如下: + - 如果buff 里数据小于 buffSize/10,不写入磁盘,除非超过1 s。 + - 如果buff 里数据大于 buffSize/10,全部写入磁盘。 +- Interval 默认值为 25 ms,Interval 值会根据每次写入日志的大小动态调整。Interval 调试规则如下: + - 数据量小时(小于 buffSize/10),增大写入间隔,Interval 每次增加 5ms,最大25ms。 + - 数据量大时(大于 buffSize/3),写入间隔最小,Interval 为 5ms。 + - 数据量比较大时(大于 buffSize/4,小于等于buffSize/3),减小写入间隔,Interval 每次减小 5ms,最小5ms。 + - 数据量适中时(大于等于 buffSize/10,小于等于buffSize/4),写入间隔不变。 +![TDengine 日志buff调整方式](./normal_log2.png) + +### 普通日志行为说明 +- 普通日志命名规则 + - 同一台机器上可以起多个客户端进程,所以客户端日志命名方式为 taoslogX.Y,其中 X 为序号,为空或者 0 到 9,Y 为后缀 0 或者 1 (windows 限制只有一个序号,所以格式为 taoslog.Y)。 + - 同一台机器上可以起多个服务端进程。所以服务端日志命名方式为 taosdlog.Y,其中 Y 为后缀, 0 或者 1。 +- 序号和后缀确定规则如下(假设日志路径为 /var/log/taos/) + - 确定序号:使用 10 个序号作为日志命名方式,/var/log/taos/taoslog0.Y - /var/log/taos/taoslog9.Y,依次检测每个序号是否使用,找到第一个没使用的序号作为该进程的日志文件使用的序号。 如果 10 个序号都被进程使用,不使用序号,即 /var/log/taos/taoslog.Y,进程都往相同的文件里写(序号为空)。 + - 确定后缀:0 或者 1。比如确定序号为 3,备选的日志文件名就为 /var/log/taos/taoslog3.0 /var/log/taos/taoslog3.1。如果两个文件都不存在用后缀 0,一个存在一个不存在,用存在的后缀。两个都存在,用修改时间最近的那个后缀。 + - 如果日志文件超过配置的条数 numOfLogLines,会切换后缀名,继续写日志,比如/var/log/taos/taoslog3.0 写够了,切换到 /var/log/taos/taoslog3.1 继续写日志。/var/log/taos/taoslog3.0 会添加时间戳后缀重命名并压缩存储(异步线程操作)。 + - 通过配置 logKeepDays 控制日志文件保存几天,几天之外的日志会被删除。比如配置为 1,则一天之前的日志会在新日志压缩存储时检测删除。不是自然天。 +- 当文件里日志行数大于 numOfLogLines(默认 1000w,取值范围 1000-20亿)时,会触发日志归档。 + - 举例:taoslog3.0 写满了,切换到 taoslog3.1 继续写。taoslog3.0 重命名为 taoslog.1735616543,然后压缩为 taoslog.1735616543.gz。同时,如果 logKeepDays > 0,会检测是否有超时的日志文件,然后删除。(该过程异步执行) + +## 慢日志 + +系统除了记录普通日志以外,对于执行时间超过配置时间的操作,会被记录到慢日志中。慢日志文件主要用于分析系统性能,排查性能问题。 +### 慢日志实现逻辑 +#### 上报架构 +![TDengine 上报框架](./slow_log1.png) +#### 缓存逻辑 +- 为了提高上报效率,慢 sql 日志上报方式为批量上报。 +- 慢 sql 日志上报为了防止缓存丢失,采用写临时文件方式来实现缓存(crash 后不会丢失)。 +- 每生成一条慢 sql 日志都会放到队列里,然后通知 slow log 线程从队列获取数据,slow log 线程根据数据里 clusterId 写到不同的文件里。 + 数据格式如下(其中,clusterId 为当前日志所属的慢查询集群id,value 为一条数据(json字符串形式)) + ```c + typedef struct { + int64_t clusterId; + char *value; + }MonitorSlowLogData + ``` +- 说明: + - 因为客户端进程里可能存在很多个链接 connection,所以需要将慢查询日志根据 clusterId 来分组。分组方式通过临时文件名来实现,命名方式为 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand},processId 为进程ID,主要为了区分多个客户端的上报。 + - 如上图 connection 1 连接的是 cluster 1。connection 2,connection 3 连接的是 cluster 2,所以connection 1 的慢 sql 数据写入文件 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand},connection 2 和 connection 3的慢 sql 数据写入文件 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand} +#### 上报逻辑 +- 读取 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand} 临时文件内容,每行数据作为 json 数组的一个元素,组装成 json 数组上报(文件里数据每接近 1M大小上报一次,上报成功后记录读取文件进度,上报采用异步上报方式。在 callback 里继续根据上次的进度,继续读取文件的内容上报,直至整个文件读取上报完毕,上报完毕后,会清空临时文件,callback 里成功或失败都会继续读取文件,失败时会记录上报失败的数据日志)。每接近 1M 上报一次主要为了防止文件太大,放在一次上报失败)。 +#### 上报时机 +- 客户端运行过程中定时上报 + - 每个 monitorInterval 时间间隔上报数据。 +- 客户端正常退出 + - 上报所有慢 sql 日志文件, 上报成功后,删除文件。 +- 客户端异常退出 + - 异常退出后再次与某个集群(clusterId)建立新的链接后遍历 {tmp dir}/tdengine_slow_log/ 目录下tdengine-{clusterId}开头的所有文件进行重新上报(这些文件可能是另一个客户端进程或本进程正在操作的。所以每个文件打开时都需要添加文件锁),然后删除这个临时文件。 +#### 一些异常行为说明 +- 因为上报数据和删除文件里的上报内容没法作为一个原子操作,所以如果上报后还没删除数据就 crash,可能导致下次重复上报,重复上报的数据会覆盖,并没丢失,影响很小。 +- 另外为了保证性能, slow log thread 线程把慢 sql 日志写入临时文件缓存,只保证刷新到操作系统的磁盘缓冲区,并不真正每次都 fsync 到磁盘,所以如果机器断电,仍可能丢失数据。该异常出现概率很小,可以容忍此种情况下的数据丢失。 +### 慢日志行为说明 +- 慢日志一方面会记录到本地慢日志文件中,另一方面会通过 taosAdapter 发送到 taosKeeper 进行结构化存储(需打开 monitorr 开关)。 +- 慢日志文件存储规则为: + - 慢日志文件一天一个,如果当天没有慢日志,没有当天的文件。 + - 文件名为 taosSlowLog.yyyy-mm-dd(taosSlowLog.2024-08-02),日志存储路径通过 logDir 配置。 + - 多个客户端的日志存储在相应日志路径下的同一个 taosSlowLog.yyyy.mm.dd 文件里。 + - 慢日志文件不自动删除,不压缩。 + - 使用和普通日志文件相同的三个参数 logDir, minimalLogDirGB, asyncLog。另外两个参数 numOfLogLines,logKeepDays 不适用于慢日志。 + +## 日志级别说明 + +日志级别分为9种,如下所示: + +```c +typedef enum { + DEBUG_FATAL = 1, + DEBUG_ERROR = 1, + DEBUG_WARN = 2, + DEBUG_INFO = 2, + DEBUG_DEBUG = 4, + DEBUG_TRACE = 8, + DEBUG_DUMP = 16, + DEBUG_SCREEN = 64, + DEBUG_FILE = 128 +} ELogLevel; +``` + +日志开关通过 bit 位来控制,具体如下: + + ![TDengine 日志级别](./slow_log2.png) + +例如: +- 131 = 128 + 2 + 1 文件 + info + error +- 135 = 128 + 4 + 2 + 1 文件 + debug + info + error +- 143 = 128 + 8 + 4 + 2 + 1 文件 + trace + debug + info + error + +通过设置日志开关的参数,可以开启不同级别的日志。 diff --git a/docs/zh/26-tdinternal/normal_log1.png b/docs/zh/26-tdinternal/normal_log1.png new file mode 100644 index 0000000000000000000000000000000000000000..0fc10d4f4b5b68712ef07ef336937a78ca981ebb GIT binary patch literal 73856 zcmaI8cRba9_&*#GLWvL}$#zioUKyEXuY-(Z?>&-aWoMmZ6G=A5I(Fz-$KHGIV`LZ6 z?|t<7eD246-{0@=uX>!rYh2@bUDxxvUcz6gDG=YLzJ24y4PwO?vKlvT;Bwu#fjy0X z3;0CRwNDfH2g^-ELFPtj|HG{tH|TFD%1Uc_8}Bp`B+}~oZv>2$8E!-Car+-#9lR$? zHmytvz*d{Y?)vy`RW=w0B*mx#qGAhs2PvYZK<g@N`(Vh=@w{GBu1MaTTh|gv%(}5*^lRcHl~i^FaMmJ3p>VH_i_4ITib`spl!?E;zoqxw z^h4i#!oQN2lE4w#KU`AbzP`dM5LSPK*{=~%9C{Mf8%yzXOZSq~(vFt3xIkRKe;oPF zRRRK9ww(BS3$=?*C$N)pgx$guKl|}IB|Y_Uwo`k& zSl_|IsDMX8V(HSb+xz$7=9bI8B;*rYTmtrAW9te66hoJ|^>?QT=5Ic_$F~P1ap+&& z#|lQ}bXg6S$;u=cI^*%fVhOq@rkgm#l^z{h)dW^1n>%_7l1M>BAM*~dq^P{+%I2Jwnz;545If( zp|X){JG1F2nZ^`cE5%PPh1gu$AWv;x<_`TJJ%9BN1k8~xHlm>qC$61VDh1dk$)@@N z2Ou3BKspRx&j$00Ig;2)t6zW~KG-}fX1So_Er}wf4eEBkU#oGz-X2jb%#shixZQz> zqHcb)p^=3bGEDacmG{@IHL=sZ_>H{35Cgi%wgl|Swq{X<5}RZ*8IVqA7ra5rX@-%V zi<8TTBEghv_U)|V{!wUfPiUMql}_TN;R(5u1~jRaN8*_&1tqL96eYql8Hz1A(BDU0 z7f=*z^l04V>C($|CM(bTHdNQ50Ry7hIKTZ&;@ZnzV02G?-v-eAS5W|-@I2qm!`UQ0 z->DhT(Tj_qno>RBq-Sz0rM2~(yJKKbtyc8TGSnxAmT3tPhn3%b?k*Fj8A@#%KWs}f| z=*UQUuKBm5kMwnW3^W`!{r!ugzIUZ;Q4NI5aSI4KFNhm?guA5-b$rst2#VYmil1l1k+(krAdI zIp=qpw-Tzqfau$MzOR0U`Gb+rcM2)JkWU3UlWEk-dVss~937z;@@GfZzNiIaj5^R2 zu)UQ=j*WiEC}v*~;1HcY6H_st1c1x30Q=%p!S2FwUlt$7Z?O9Hndea!$w=PH1JQ&{ zln~M?{wT>PeykAMj;=Nq1JM1xn+B6@RI3mS-jxXXQu1@9I{NwY7g@EWF*yD1HzmGS zjaRsTy$1lBS|ZpHSG%hG2xjAl^q7q|z6n_113)JO=FRKb=_?zbU!QZ=URX{C69ysb>SM`x*9 zhSfbpEcXo(TyyEJgT!ZLAlPKEY#7!1MU7)T5Fz#MDCVd?#J}qSAdr3ndT`CtwA54q zw+#{o9o_!qz-WdYMnKKb|)L+unV~yGHSO&9UDeSVn8o28L5E!BliXZvA zaI&u3_?(}gH+~n0KtG5g+lq8M2Y1_waFQx`d?@@*04p3Rc;MYZEbjY8+RN3j#`%Q1 zRpYL!1ldfwm8DhmcOCsu_5zZ>0-gorX+^Y=LE{&vjyXnGJP=ejjGz~hHIHuuz9AUZ zKqTU`P&$W)kxpv8A(TOeBxdKEDch2{SE`wnOx71~n}pkI-I^#q;PIJ>0Es0zD=>u* zZV|$iVsaUUP30VyQ^O);_4@>iL0nu^Oa}+%e(a8U%eorZnD)&9z#J61-Dl%e0^*L* zuP@+IfMKEOmVNQ1z8PUxlW;XHq5_kXT31D`hJRMD2UV~*bVJ@#BxG1zoL&%~jx1Pv zQ_$Ctx>Lrwy)@AS^O42sD|Czz8@Tom8x#0i$Q9;zT3TA_z=P3F0*^=M9QwO3TaN;0 zGq>FpvffzKDgF%v+|`!^W8p=eYF1DnBH(K&Ax6_`kav@TG2F)hvHDWNO33E1AYWkv|ry7)&ielk70sICWvO{aa|+||-@{95e)#}d!SciKc=h+jAH7MJDm+5oyKqbt z>+eEXYspPEt01>0PNWG()K;7#_S@QWw$O%a1P#qI-ntEuk=LruwgXfhIZw!&QKlYp zREpk3hG6ua4n$$VpL;DW=mDuG;3z!?v$I9VugemA@GVz&r9xI{5s+c|+abrp=$SI9 zvXXz45;8X(%pnYQRbA-u&hZ-cVo7Bo1z(3%+ojR!4?&-U~PI*!n9^UGI~f-oR#2)wt3qz_nMgutYW7=DPGRu4qnXStqXzN~4L5sE zX?2}(D7-$39S$1Xn2Ta^Y`@+F3*TMs-H~Go)rQfAUl<3*LK4HT^^tPQVGbOsM>_lI z5>Ik6^KI={-;d_%$Wa<8@b(B!g4k_5q+~eERnjRstzc)$l~^|%4hDC73JUHc$Bq+l zaLaZreg(A^mumi2sa|8bm8#XTp9j&WJlj|6(g3m<43G-#_ZMSr7#&>1-MhxTi0hqo z{31OuI6&5=g&4gV0iuHS#e95|DH0MHsU(s>(1`wT44+ch2vX;NPNMj{%n(H?zS$)d-(mtK}t~bg;$Y>#na}` zj|QG?-lD7VCMCJRJ8z2pZ6OGcPsFv-%PL8!3v&9t8o>oS)`1B`o9HB6M!y{p8O@3X zNhDQ+A-mI`26t96?1jV!!;98I0;HUVQy#-~9vFE@WqSc2(sASr>5Vv z8YQV{09TU4rX_^IxsJh=Nx%GDo+VDgzfC;tdCM8dU8tg}>ZcRKUILA!PxHRq`|Do=hGyYh?qxn^7*KV{PE z5wA}(Lt#ts^4cY;C_pT{Ggv|e2s3sa7q1??f5rpcb|4kRp<5;ZFC}W|r##Q~Ede+W|zJap1k!hxumWn5c|cEKrmnB+4%Ktgw4B`oxoc&%H~Q_4f$oX^jWj@&q= zT28#`yimT3dT-D#0I(NRZedl+=Ls+RdGL&dnOR&}Q$MJOov#ol0WSnOzC%qy`8Si_ z1}s!)Ut>Xw4C~|bK zd^;=i7n9K%mQ=2;ytfr>>gwEL^+qpB0d)gf=m)gGd9>V!wqJNfgK^ubYqcaf4Dc9Z zd{i2{qT9zF@w?W`l@{jbUW;_fA1i!jjTCBhEID%~U*99Kq@|9E+cKfKPs{Y8hN8MG zIip9uw0MpEYI3~ldwUeAWYGAhxx+;f%xWagkwD7i`XK4`Qlpz)EDts2L!0ZQN!k3- z5QbR^L81D}xy7qnuqRE;XBLH0-Us5hBtBoi#twJ1L8&vdcEDamh5Vz@amV_NV=Ap6G zxlYEtKW*UZ%a&>b=^9cYH-{|!+EW!$x-n(a+)925OWoi){A+Jh!g%?yau=`-U{ds0 z05;=%{Q8%us3YA}L=YM9_uOtzW*T64QrwlQp=kGeXR%d8bv%w!|6BeSt45U-c7UO1 z`g4Y>DEUe1enocy8}}W^4}+G{N4c6S@!8at)dh7b11f1qB-6{}zcE2d9JtV!+gu7i z=63b>78{FZB6v+OqTVChR6-;Y*1i`T80e!Rc&Cn3hCiX)zIw~O6k8(8)JjlqcTvIa zPZ^5%Y0&y01yi>ka0DS=&Xd)u^gIu{>HUyN0z*)|5(mt7%uv%3<0&_=cLAABUIQ2X zB75NXwmcl!@T3x|svxq2lbrA0udYH_bUZcMik~n=^!N3Z>4Dih@|LyF57@>Ara$qf zG#g$JMr2x??^*DFCB(R1GqB!v!)Y&&)V`C-01R@>raA@SC~gEEl9u0w^(=Dg?!Hd% z(fOp&ZT;rV85wt6>Eac^Xidw8rR8*K5D5~3cx)M3^}N@BCbs6F`5fy$6sfj!sT!58 z-M==&cTyZczI2KM+=K=DZ0?NHZOW+ynRYmT);#ljnA;efzct}on(kF?{FeDH=0u4& zJ8f&rIVs*C-df3qq^O+BOJmZE*X#R; zqwONyLOc7S38EU0%Bm{sFZo}T>ZFLMu^FB|duB@KvwH6}AU&#kdSeq-<0XbMxy8#Z zIEH%3f&i*n+PHLkKZ^@fXdJMzvh3a2adoX%Q!dX~tloA5c*dy))3^P8Heau_0vtWZ zdwW&Nv|zgJi?7^<>(*Sjlfw_f)xu-}j*9*}w(1qNAL zG=uvPNs{4s{p2)8kcOJ_F``qvzt~V0yz?f*D=5f{NYeduvr!*B>QkE?^!>@RDo>xN zqsmH4$E&DkFY=Rrd$#pA;GQ%*-9IVgRo$5T4KPn>qHZP!ZzkAWwx0vOt|zhdXPJAC z4^?N~V`656Tx*-#Oh9`*aUE_(*Erpq;-pxJ*O=fCSpW7-$`Gui?*&*z&%)*ILaNDc zB#htV>}O{d#GM{Z0=)>L1D(8643tVC5L+U6$tQU}bQdLt+i?eLgMcVDDj8QFJ<#tB zy)a#KiIqh81%4TLp8`MhpeFH6n=9Pomf0n4w>3Ip`;R zx)SzB{9bA`TE2Jn`_`9w_#U{)GP*kBQ?;FLu^F9?3Fl)ip?%V&d{P`==4`*+#-l$$ z`hO6W)6*g+i(b7ge>P40_MXJvYLityxmfi2Q~T?nU9!g<|JXFjMH-<3Cz(t4GTn8Z zXZoZ{i*b%EGAV?;DSL%_PHxZir`8y+3TpRx9&_n;0LtfI2%`^n!SUABE0{^ksR;OjTJLnjh{CFINg=HZAC^vXKNLkX8BUtN;qxBpvzz4=UwgVehbYl_eY{h zIsXjYeZ+J3aPMcYKUE)1_5=9%N@W zp~a8~4=vMgnkIXEnG-D1qeA!F(9WDkBra`!;dh^jAOVch_-9t{{NzqL!go&5cCNuhUYBYk)0 zeCxNyYO*#!x|uxnrYP%ZCyO5j zc2$IxxawXNW?wjsQ%~+p%tdl~ zDqm!5hL;-Qz!-_j8&VYdPd-G4tG?ACW5p53>ap zElAi(oQ6%5fjxgb59>_}uu{nHyRRX+*En)C*S36klp(9!@?y4330y7nZG;UIYPj&l z4vIf9J3S3Xk$m9gt+cfC6uz0x4OZGcf#-68d`lfoAwX9ua||Mgy9B4P1#8L!OyWTi z#eGmvwUgM}ocni^4|t0mqeoWq`vT+70oZ)SH|Vz7q%2EH5tJ-M!>HcJfnHs!i$>Prn$z$Bt3)rCNe5ooG7b< z%kYK*WTmQJ2Viegp}eAND4>fO8IX0IKD*eL&RB=_6!sV1vhDk?fQ+)DG#n3P-v zCulo>zJO62ny2P zDRr2sO>^s)=4HcD=LSKa^u=ELUq}v()IiG0nz``cojiD7cnBgBHOf=e`Cbz8)SW6P^TklJw z)yuPZKH1<@raqZCu2JK|nC9SPTgL_V^qnU@wz_zuo0$apj44NF8D7~fX|zH&*4KO6 z%-kQ6F+F<4>|FO2{g|uly>kks%{g1DlyTcycYp>SheY&uGF-295S5Sw@8rq+hF@g@uKycMd67N!d@nt=9#d zol(}XkZ=(`f7vNa*7Y1QTYKie9wWLBPMP}ARbF=jo$2j*aj@so{DT4tlD_%&v@*-V zbAL(`k(AlN@OTkGK0d1~{~EHpktKI4P#3Y@0^iYYaCiqgMH`;x{MD=to8fb>mh%?)h-42Uh3%+{NTAr(%F~khyse{;_4e}`wZXfs?Iay$*41-TX}MYv#)2MMPr6Ewh|G6bbI(D1up=n0?#prlpG9&dx> zmme0#I;XHWoU_ME>9nnFjpU1)mh&@@|M(%|cIlyTJJPKnY`da8jDc*wE859@fu%$eQrETC+2%>ZDEZ!feY6m972~i z%%@N?Hf74e`9{yQi*WsniaB$arRupp*OoRk2bE4$7wx7qs>qdjy*Qs}UG`6001u8uDLHY1WxR5^^yG6KU?Kapjrfu^#J^F^;ft^a_shg!!Hva=4^z zdU_cLjr%^{EQ!g@Q}9Ay8xnN7y1A)sxzZ6m$7k5ZukOZn|6a zv;Ay@_^Z_oe3;yDz7r-nJbC=id27?)KatoF!c244Ml#R&+~!UuHZr3W-gi!ZdgeAD z?iJ?hroL0I&($4ml+wUmznL?7Cao1jJphD487hjT0z-!Fpae>yZgnEvE3uWt4cO_8HG&b@66%FQ#4|HW)%NZ{X4Hm6z0XuxGK z>Q%|?(Qs=C?#R*q2E(7+e-1RrAM8~$F@QT5a%AN2KeK`B84adE6h`Et_E2Rxy?fh! z&Oqe+gl&4SbX$M}1&@?sY_-0UsSY10Eb#NbsM+mrSd^c8PFk}n;om>Ck$A{h(KN`K zYdz7&Aj`DGmg`}rMRA;CtmaWk=g|agUCJ@gh}n8o`Nk5D$$YrqbA(D%LA3WsVlBc*c1JahRXtJa0Tby5qZIQ^xeBh_G#hG@ z5cv7@*_HqLC(V4BG>*vEHVyfa-UM+S(?1EYnPL~Cxvi#ccTB-l5Ox|q%@@nGlh}Keu)cv zx0PrXSC6~3;;)ovspou>+)DjQ^|_`%a}mxZ6Z0ArKk@bgvK{s*A8&v0xj9Yh`>^in zN&oH`-^Tp3&MaXV4*@jQP|I_RCyeKVUn|`2jIPqOJrq={xa6Qvr)l?EPVdth_IJR^hHh?PMhC;(*!}_tGi>JF?1&&bznAOT)vE&bGJR8 zbI){6R@v|cHf(>P+~20MgQN1n{j>LZy^i~ecP!J&OT~Q`zqNwR@3K%;mZ*)2>mnyV zo6ZwK0s=|EhQTP~{adBigBXE~fU70EV$Z(Q2EiKKf|Fo3-G*Uu(7n2i%r6xi8BAXG zJFmWeTXQY0W<6_cG^$v^X=8XWJsWbgo`6)fiS+ZnjC;}_@~rE9;B-~E>3O5QV`oHb z;eNXgdXeY0VJm+va=s+~#{KF6hc@4-Hm%9#{UVCYHfDkRUo>);o0$RIBbf$WUV?29 z9hUpMe|gJ~z^GF01=iB=P>88s0okS;qH%>c>tJ~Z7nL?&)n`9b!+GKo%5UxW)a4{l zL}_;L_n{}U`6x!T_fy65Pu*U}Bp2rsP69KvP6H6mB36r#z(_a|m$^)=u|TLQ5ou(G zsDd)2l>#EoL3tK*aThwiyI`53>oMW%uNzBC&3iWM=C!G#+qf#$^2o1l!QBDOnl$Hk znmGQxDJgn4U3KJ8zeU=(!6v~gR0+{|b^hB#>@oP%*UDM^+7=lB2uTUn!14e>?mIPD z2+5`nA~Eu(ok?wXR2H2HRc-tMtEUznlH$?e7@n=+mm{%9?;%59WtB`Fmkb&hX3HI9 zpx?8A#GfnaBe~~YS^ZluaeZZ$Gr#mCNamR|%Sz2ybbT3$OMV7!sXN6F_B)2YoHdJ~ zYkKTHg<>t9iBC&9$gG@;Re!7pFMl)N00jGfAlOr>-2>g@8aZdqzpjEw%CB7};6Im{ zyA3PhT8^+Qun^lwcXQk}w|6MFS5#ewyKj&`;#9;=Y{YG2lz~R-FN#jB)*!;<#5O&7 zPWjN&X7%e14(a2Kg~@TkTz04ZLqjrPvk?PcLs0<^cO%`Ji!n zxd_KgU=B8^8mU~Hiu#Yi%uI6P*QC{E2rv)AeazEHvBdKn!MA%q&@mX*@Mqi6t*^n8 zzi*|7ql_<2u;uYlom`zoz3rSuB(ON$A;yjOI;>S2 z&zY%LKQq^x&Ty5!A=VkWo9MxoW=*w5oo@BT!qAMkL>D-r3p5Kt@%H_hl%9a0zws#|EMs<~ z2H^{-{uu%PNdexmswwQNlL>91!PNojOe@IltP=dCXuA(Y)xWft z(Mws`mMd}4=OlI5t0sMW__KNYw4r7Oeca%<^CQ`3bF^TPxeA&qTE?yoHqwvTn`x+J z8#i-y)kk{mcoGlSbFov{K`Tmgi$%}&C0`5JPktO96>-E{f}2}_G0gf6Y$G6g>|h(v z!RW#3(kT*P`j4^*BIORZ(wS|-8<8qPvjK}th6%oF5aTc?4 z8Q z&?J#S#cJga=!TGRzYlM0P9SxR~8Um8fK=w6sxNAN!NcZ zJ~*Lg@%ACeFCo-Meg>JQ%?Z4uq_gF%lQyKT*4MXWW3~_%Oe3#WUU;r3JZa4M0$gea z^_x-7<6)j3GC*LTF`5$2kSA5)R4x?U4MggBI$`q+m< z;4uDJS2@n2ldr0;VUfps;(kixBhMTi9C9j4ld8Q^NcXLKU8n|&5HYPDTs=3R6q{s} zv8c`7uO)W-^>B%YvezBcD9Jv(PIiH^`k!Ps-MQe-%gn~9hZ1il(HGNRMs@U8QMCNpvHIR{;W7~Lw#oHg;Ru05m9i`V3^Np4h zrGv8ka{hv+@XWUlm79c!{G_vMP}aiJ1PKh{Pjf(=Mx>lbQh*4hl{5MqBcMQxs9oi0 zzO%$*{*7;+DDZO|1bVXuSu!mwV(=C6Je?I*t6Wn&`@N{z`k|cO`53ZCqcIm$*TWj& z?j|`?9F4N>9%pY>2shR>t{btRZ>*!S2r<$e+u2i>DuOz9QFLk$-52#dE*w&-JF6{q zh|2-0ewMzac5~I|@ep)48O!ZUanhPU_qYfzQuE9)hpGl9e781T|En3~Zo_IGM04qt zZ7m(V3KlJGQ#6tK%-C~md4V-fH%$o}$Hvx?8UvWBzh!kMMF&=FE_GFOeZJ^FTe zm(P?fm!6i=tEzZv&f$W88=^)vF>uQ8g6QzA`gr%5Y^q1D8GL;y@E!$wTb?t2NjebI zXML)0uR%in+LUPiGbQ!5JWhStAu7L2Z7e9$#r!v5cZ{`+La$Qu z`MgMrAMH13k-OI|CaXO6d;(lqw2K?c5suaF{TSTguldgi%V%jI9uY@SRRubyU}>eg z$4lL7%j;XK>B7Si$Nw+eG9Zt`dCY!lb&rCc3;C<}+!7vw&uVA6rIw7;-*cF$X>#8i zDLJZTT5{ox$#B?R2AQGK=frL21$2LkaW0A0<+p8jFwGC~YsZ>Xt>(Z&N$(h%`>ro( zlQ1V!xb%D0=aX=WGRgb@HK9zYa9?WWd=vw{G!V?^)O3oogCSKdDxEpgm~;i-*yp^`9v88dRHly`)P<5V z9b5m_F8|u-3>Mo{ycC$KhbX7Lt`GQw1=J9? z@>}kZaAg?N-NM5l0S`ufMoPiT3Y4Lyd200BOT$Qud!-U$)4`Qubj$D4de;T^3F|Q! z=WjwW_F5~4H99{^(LiKL50686Mmd5f{4i{QF+ew-%5b;a-Q(g7RvuBi*JXf+B6}X8 zL}5y%Q7z@`J=*>U6D`oba^l~}@2!B?w?YQ2&N9W5DsB}atbxV`#M}q~L zpU~>E50#0(TS8K5ADGf8Q}Ld3%y|namx`GcA)~ob11sb8pqy6^bH7@=hI}xkOixpN z&6R+(p7s5eHPinQj=$|v5g6&smXsI(J;^D~I$FKQoVva{HppQPlt~1x|4UBaC4-TU zyh0fdK*{!cG`qadqk~3}*u3#N0jN~`wYXYmiX@^JK?&p$W~I?Yv8(l-`W_S;j)ut3 z%o55ov)T&W6hJ9g5-jJC__Tx)yGIODiCa`7HS8u%GYq~=#wZ4Awf6Ar(Q&YQx(@Sq z!HpA7+%maI)iT9w?mVKiOa0PJ2IQq|sDk`?fi}qY3!A@rsSKb;k$O&yAzg>!su1M~ zuv!8>RE}|eNM;%=aa3bpg}f@tAv90K%^c4ILBVrxsU|@xNAq5D{g1ai)IU)I`KX%6 z-fMBg*)jHJ6(B3CP5w$L45<_JxRWX7<7Nn_3@}1Ny;%@hxw1dS2 z6Y5|TAM5bn7bM?-3M1-?)*wKy;0ET;%o2QN0(a=^uRHWTV9~qnu73B@1uEnd$zEf> z?-R=uOOx%mS0gbZ1RG;7saE~K9m!!0o*e_~pnmaNPS7)}*~s+9fsc;sg$ps}VVw-m z<3^tys1;e)twf+^aOTA1o?q%r`A$K*JbG}hg}4*?zk!}1DexIkS= z*j}!|@-fuehV1>}{pW9L(F3ywDP4F9SfJr)q~(@wFeLZ;d%M!9u~YgKtMrOdK@Zs= zF66yFyZ*P50)*uZr$*4DG8xj$LW@B)c)vlJzu#Gk-(H<)l`mB>=tSIiXHwm5zv(Qr z|I^pWZpEJ)+AO_2>i!#?2B8G`f^?GV)1Wf2qH95LNm(YfLyva^>#N-dgAhXq(K=0pT_UN6h6hQ4hVP zhucVJ$P?Bea>M+i7$uN4#p7l*7u}UMidBzL_xjHFCo9W(P}4h2y^`J4HI@7S6fZ0` zUp7S4btZjc9(OaoAMb2|yX2}1eL%ebzkxwLP%CuYq0&cJL&Zu`?V%#nu&j!cq?Yp@ znaW!;!Icbj_709!rQ+XDeA^MEZP^n(?!Nxu!wjHn)Y}nFBkC}|b*{dk=o4DFiTjZg z>t)8Ca(FLLH?CC8lyXw!pGuLN+;*?5%@yN0yz`vYV|~qLHzn9)Qs|$&{B;wMxe*v` z17RN+-6#V9+m9)E=mZ}|6_}^Lf$z=^me=22V#n>3@`3R16>#fG9d(`o~^V)IDSMqYedAQF2Bp?QXMFK zQq=c2v`a5g8zoiZ*US7J)>_D?G&?jKQQpzNP0#K(i7m+(zE@VRachXbsG8VrB5AT) zJtwcfk&MN$hb+9hj1@Kdc|{iCZplW4kBKTuKxlq1y@q#LFQL z;&H2=4o=Dz?7`*b$4dwbNggHHB;}RBAUwia6;*T18PLR`%KfAcu!v2&rBR`0A@ohB zm3gIQ|G8*E!bx}`4O2NKBs!xby={6>B$}4aPWeMlJtWx<{34#6ptorcEA7N>etceF z=bE)e`uJ%T5XoiC^=^Ga#DAa{`ofB)oLKfdf&a=aRvfcT3c*eDut{3kk?S*DP*i+$ z>Nn-o1hymdDm-N{U6}Weqd4>-)XykQDK?Fz6`fwMN7v?714N{@BiBGijO6~g0D4JG zjOYDPwf@uk!^_u%qJvv!585E}4H;LMj>2dHh7m)@0BL61)8bJA;Q7LNV3P+5%1k+` z7NF-PvWd_$^R=F>acI!2_?`n2J5u z0EIx2Jn%daz}3ZTyW{?k+!9#Q*fY@uSL7B90R3qTGl!h!)209uYY-ptx3Mte2{W51 zkYpr@y5>T8EF>P9Jx}d6w!nfR5OZc^W~#z*v}>^|A*}r~IYnuzBoJ~iQ{dd`<*qOG zQ^Oj!k~2(h!s|QsmG`k?{Zx>R30ZZ1L^KLWhOVov{)Q&S*Tm!snui_W$-GIK* zV@2QjkKzKa30E`{Pku~JK7R7Jx_Yxg-IO;1UjMN%Gm;-xk~G)y{hw7Rm&2QW$WTYWkE%krG70f3QBc zFEgHC>M3N7l~r$wX-Y${6ob4>8XDj^N?wK+D z%2Opli9UC!MLjrOR>*;9xA7->1avUHMKcR4)%tw~O(2B}vJJVwn z+;G;SfyGsaZZG%?wGihN^ms1sb>#=V#@*T+{;%>YXZw}GOq;YNM#TB`v0}%o{Svwt|Nvpgh z{^v|*kR5zAdt_qvCEqIbyl$@^t?8oZ!I>Q(_|&knxF|YDw(TpZP%b*3CbPzF`y;Km zP(WIq8ZD@`hi*W_j*0>^Nm$z>oW zqvZ7TZ8K{uyfFmqi-#Ksy@CiI@%p2}-i}b`6eO)w+O^>iDq##?{6E9T2|>9H1Vjj< zSu!U_6{zoe2w?TBt%{bEw}bvfkP;a6=cINpdyEw_N7q%?+=KndVU44YtP8Ey5AlWq zVdpAyJTj|sJh%E$+S3(Rjbg_4!x#>Rn>D|Vdl|eHdH<_X)Cx%b&BDbQKhv}0iRWkX zbQKyvz1(=y`tQT)oaR;1TN)vOD0 zP4$@$@+<3nM(NApa{-R$u#(C6es0Ak)}b1e0}%f}p@HO#tcNb{6ym#+&ut~eJuJ%Y z`{ZfQwq|pIzJ4REy$)3FyAm+_R-f<{Q(&H~sIl5sLgk&{Oj()>_=hK}gE1NIo=VRz9~*WsVrfiX@@7Pl4( zOt8|Kn}qc1vi<(*vw6PZ&N9?kd(HUv!dyD$}xBB?Q?z!H56%X&i>1AN@2pQJt%If~f0Y6_YEndx z+spS$!P`VTgtCi=kuVuxYFu-a_^`QI0x+iUe*-&%6fm_!J++g2TyEH^f+1u-LX-pO zTYhyYH&Advx?PauqN(eNI*q+z(jnnc8wVEQmf&nlo`1GGX9YAo z_iB`lB8t#$DIdT0xL%y@W^hSc0w>{puH61Na05M`txeDW`WC~aTt-S}BZQ2UW^VGx z6pmc{ht(8ZV%ewgm&>Kqb>$RE-yb{n+5|X8!t|JxM+$p$J^UT?=e+jk%`p`JDRi?y zJPq=-nWh3x)M-bSZ&ZKRNQlgSVWza%UIwwVWfv0pmlAmljEn_mAC>}OD(G^~5g|q0 zwpI|PmQrF^Q-!4B0LGG&t=V+P$yn4@*VpORA|kByM`?%kTU@A8fB-xLb{~?<;5TTS z7Es^$78sOg`U=oc?JFvyc8BE(!l^Q&COH&5AeZ!D^@K*I#q@M{&Tg;iY<@7$U2)NY zmcA{o$m`L{^(#Ei>vEOlzjC$6nJURhunaGT@7?4qJQn;XZkFdm7(srjNm6`mFmk6j z;-Z0bOD<(DT(qe(XL&P;Q(89>H(9x~I-^vTsxp-9NzI71@tvX9k%rqEJmooZ$}?B4 zV}PYT0SLPmNd)Gh74Ytm1?aU2SAt8W5WTp(8A4LoKswcftD2&?W~&!U%G9`?!N;gn zRUB{VGWoE`Op!WCwtFqASARB3{Q{w~K(I_&omtx9uL6e(*zNsO2qy3)7}6Sohj{Rn z>H=-Iy4I=prv3FkTidK?eHpw741~)CI}XK<%# zrXW`k``yNpwRNEl%xaWu2GhkL2(6^=n6(Y*buW41|I$mItynp%@qYCIn!#u=MF`i`>Fy)M^6dQ4|&YU7lQkBw1PI4#WA1Z(+(J=nCLfM|;Lm?7-V zvc`{7?L4)Y-&Qe&yH2HwD!t)w0}t8n)$^827dLxw?LVaTP)d(cT8EuDefT=rz!mvZ z@e~+SAXOUu`{c?P$Wc#~p5~$f7sXbt?7s&z+KD7q8`52PiM*4mtuyzQ)Sr)wffG*wco2^sh?RAkE z{pFmK*IM3CkQ_?r-h!N3#tv>UO3hjw^PCHK3XJ(nESz;wWdL*X;Ax-ebHFk*$Ud$G z_W+2!()(ul7OCY>LhWCE3UtD^M-!Xn+>^l_lec?RXkZ{84%L_NK7-%UH>O$N)7PnZ znf~5!8Ilpki8WB;FoPLitehn^s-|VK9OJCe&j0b<0X<&gfL&eWq8v%W>98aQCO;6J z`#ZyLKoPF1rtYHJ%~ zAX~IP6D8dG)iOpQs%?r{h8bn&RHM@}N_K;O-FXm}v6Z_mrps?NPnl-5O`K0n&tvgj zE3M+`>>uEG0cX}4;V!TNA4VLrMxZM+@7=t-HXEE5YdN zSUSDxGnLjsnbbOk$*i31RBuWM?KIBgh8cN^FW{7nCIO;$qbzSn`ygRF?iMNEaw~RB zV{l4}mj-rVKtL2g9VG_%v{AS*W4^T*!Q{_ITnGNc^9!&Gp&_lMq_wclnw4x86a_*^ z-K@s~Ni~FVEw62mHP);#^=(51w>{4$)qWx)Z0&*S`yoG#JxtT#yn>HndCzP!U*Q=* zi)G*?>H^3tZ4a#tNuXcDi{VjdW(U+HJ(-No`n*p*mePv3;6*nS`2s_vwOzT@lbV@ ztOCkuXO9d!8(+x0)#?`^^3^!GzU-}z2R*&}mMOU={9HH-+!nLRF&}MQv3O7NoTBjY z88|b}oiy5S69e!P4vpkxHWl3C5P!~v_R9^tgw3$6f1~R{n}1*|twhGbI|02+xQH?9 zwZ!hfXRzbYP76sgNmDz;f9ln}x#<&>WBUgWF#z4j$vyDMd|F@D1VU6BvlDi5gdT)bAslla z#V`1m0s%zLC#6DNwN6K%>^Na$r7^wCfR2<0?e^O)W=OfNU71@yGuf3Gp*g48;8b0@ zcM;nsxh9I~j&FwEmC>%=X273eJ16}~gtR9GmX-aRr~&}%!BD(w`#~t&M||eTocQ7y zIo^-lC;(81P|K1cfH)sib&WeN#a`=SYa=3E_Uc^<|lBr%^v}di0V2G+wx1WHS2{`5%Dp!B6C5#3K zZ>10Wleh2j+qIu|Or1*hO}%+@tz%%TWc1^O+!8PU4+=CcZf?Us5?-^-v*SwBI)^?O zJc_V#SwsYjId2OAUwa_&B6ryYAn~l512oWb+yY4i2ko%JXX2J^H?xK=zP-hYl>T^n z*cv}oY8+Im+qc%c`Q``mHX%TPYMj>kwV$4zUiTRpQDJ9cd5J-v5dr?S@`NhR zQ`&8nJ!UaE`229n-GKCTvbu!Z;MrBr>(~xE?k>qHTorCGn(Qg-V^Y48@pv}1h_Uz|$jm*jRU#Ap-V z%+xSFpX^!l=2PZcQwcJY0box_lng!=E6u_z z{_Vi2CBJTBg5MoNDn=c5<7Sre>V)l;q(9_sFg*J){4)jHPW$EJ z77Js1?u0FRBvvtCGlj~T;itFNY@X5zc4WGbt4M~PF1a5!^=l4;fw=X41lPd0_IDB2 zr1D>z?I=j6(A2_s?ipT1`m}XKtTS>S97!;u)-@x{4bQu}Ov!#L{7fNiroIu%0u*pi zn5oXCbM5V*6N+Rsfk~x*+Wu(j)^*>ksk0x&?c?hv(GvN(j_FzZ49sXV%NEFJF3qFxH-ilAjmhGJCHMZI*iLdLNoBq}C`buQ>MzFyKsRmS9+ z;;rbTQX+Q2pgW}V#csNlDuwBCpZk$0Nr35oHcfUuvnA<%5!G!FtG=PQA=^}9gGrqT zLXlP>t-iGjzoQiMiLIOaOy-kj5cf87xc2ZMtJ^uIJIwBUlQ(9yVm*nt-S(}3@^L9E z*z1t1qYQ02<{a(CPkilNW{*4Pum(EZaU872%08iCnDqkXxxn#M$8$luCw>k7nVU8br?( zG)D2UBCa4jEFzow0TX-(%F>5{@O$g%FZ{!?EkUGkZ7ps6Q6C#>y0Fj`5#o~RZpG3$ zR3W*Zj}i5?4(H&Z2|m(EgC<%*$}`V|q0s>`>)zNx@3hpAjSJnr5{o0~@cs7Yj0w)K zhsBhrTiNS(o_y8;_PY|lEdgBcWK)_a=@8`t8E7xJqW|Laq z3Z81S^YBE^%U&MxAlkWtuKQwoAjhrnXfH)@WB>cQzTlY4wx1?Pko)Nz`+9Kl&z=s@ z0Is)I}*?Q!16rd6fT z7}TX?k^Cj2kCa+?OgPN3YLIJ}H<+m@!P5vrtbNK4@&!*TGN&fBkUR~DZrv6q_XQtR zS-;x@`1-&PVKHFomktH>D18)4`Aj_vYv4ldL*fKLZ)%88V&6r)gLg6#|9KgRhk#T9 zRp^xX$f&!#TUuD^>vUy1+0<}bL@OetycVpuM|vR2~PtTw}iA`M24 zi6tqopAfS$@2WF6T|yD85ce$)j#-c2X`E{g|tU@A23)SL`J* zi&SJTyJ@NZS`H;o$i)gyd*M~Oojhxq$({nI-|HsL3#P^O6@)5m^!bSp+6(9h02^WN z#=f=;T$E;7JAOIXj?by42NiYQ$gWwL$xV4lA2wY;4HC zSOEDZjjh!!8$AV*C!&Rd=`*NfQTX+m73GJg1qB%Jckogse3HwUs{!OH6-^#TNpWLP z;WLGzgFh~e|LKpo(vMV8F2v7s-s1H@iQqEGcRa>`l22%mJyXI%k#^e&?Xa)fjZGe* z#|hF=Es9W7+Scb^Q&}Xt*d`lW6)6pH2O7OM%P8|(Q~y9+N1fb_JBU1}I;fNk52J%t zH{I^bD|}k|9&HW0@cit@tAL+R+alC)X+VEd3j44>MF6I;8$CpY%cLMmRG(9}njU_< z6z^Vjv}s>W%mZiH-WhT~8wxo7@@q+hi5bRt@zG^I__-qe$Ww6zF$!q9MwmkycTjZJb#dC&k)g4l9={0yV5g+)3zZZ{A*M|}KwWBw0tDOspQmrJMy+$bYlc)z!*RL!F( zUjV}oiWZ~Y;Pvpj|5tBm(U-60yCtq%EbQaz-;C;??Tv?~Z17(3dHnMkD-e+)&c$8} z-9-gQ)_u$HhyNi~0K}aE{7T5Dh1y5T3JMBPsXz{q56oCwzM}u_pVR8;1lCsOkQ2Ka z^4aFNKM3F>?8#{8UqCLMP2nTQwcmD87N}#?U9r1A2}d13OC#vm4~T%wLYVH2AP5ORs*T^1;Aw*hmD)m)ymR1JBpYj(KX|@{ zz<&nW8?Z0{=?6NzP{oMmI#N*FpcMp~QHUIo$}$Z8Lh^HPz{^2^q--2}@BTCZGa>wc zc>XeN&|i-kAvW0$#XFuN%@{c?XocG|nHc?x2pAkwNWolkRXVHNUQ#|o8-_u zH1d~h;CX5_mN$sh64Du%f@M9ZmlpK^`PKQSpcTX|D9-}$d$LG~-EXZCDnrcAf~fIN z!F!AU0Bo5Ezp9JbXz|OarrrdD7-i(dyWFIH1g$KhMw^j;AOOD?yy(UHW!N^6d`f*JFhMs;v@}r1u&5)em@}m5)MwH&J^LYC|A8`j?AZ z0jXOFIBnKX=L?kY&wrKgU*-E(`G9Zxzp{M44LLA^*B~9zg?+E(0(EwbaAa}lUh{DQ;UH}%TuJmv7`Pb)OqJh_K zm0^p@bl^YXs!}w68L{6j388}UwCLJ@Tb#@218L2c-vg7PG+cs?(g3=~7|73$ph@}L z1Bm|uYCrF`Q{_H9R9VDDsq4?~{$)dcKU-r7dJ@QtUFQXeY=ET$L^eQ-|Jiq2V6#-- zD(D4<16!n$wxG-Q`}M!R`V`^QjsM%t{H?M7hr<4=?f>k~C1OJ4m$Uq@4wDOlAsu~9 zrv1k-es>x$*&t4NQvdSk{}}T;*cY)04(|PohW;T}ARj73(1Cq>;2-b(?x*J|!W}e4 z$Nl308;R+g=Hs*K`70qxF+-+Kl1)R_2+B4fq=Zc1b=u0T{1Mz)ms=r3bE#K70c(t~ zc^q$!Lq@65QlM&kl;}^F3tU);>py0zMjH$siy7<5WrQyQJh$F^c;hdx^|R$Ygg4`I zgCfY40SzPYpDMqJgMT$BbQ6qWP%Ai!0@*=tgi^oyJ;KXrwE$$yNp*g8-N+%3%Yko4 z%_dp?hu{+!QAu6*WOf70&qE!*?Z%JweYmJLcj-eU%i53k*Lcln*HyjS0$HstB_WT% zJBbj482;zCCU73v9KI{vFOrVl5gK-X+3b6|LN;Zfark=4 z$IY$HG@fIttJyt?4DSE&^PH^;S$)a}x^ zR57B-NO^Sv;}0pfrpQi~XC7vVYL-9+*>n)BA<*yb`jI`c@EAgY3r9ef6 z4PQIzaJTv#P=9f{qts!33eTFH;(TMjRoZyE!{F9p%?ig~{7P)jRy=XO621mD`;!v* zM1ar}Ifn_tK&=@HA|0hg%x7t6QWVQb?%ZJkj5<4{*Uo-MFl>Nj7fz>f_pOjNssNu| z<Bc|J)o#d(`CDj|G>sKg3JBX zd&z22_4O=Bv1yMb5s+9<4(LQhQK9}!a4a@@5E2r0)YpGexqc%hC#UO1j@nw|A4FF# zmmc|5z(GbQ^aTU`k=V3;+^$QPESs23e=dysCE$(p+-t0e>dxu?sQ% zT!02BuRjp~U~o5;OHCN&q15Cs!6$W{**WSuakSo<GxsD&CMC#+f9RaZ(dKkkK*Q zJ1a)~7x_lI15)W5`mp)FgMtRI-3EL>bhuu>WRV&%^>Duk&A3EKi zh|dqaGLyHcXY*o)Z=K#;)ArnOj-ph zMXZ}xPMb5|*tj_tN`o>oRq{uh%Vc3LF107%n(*O5Pv6|6d#kV<1uZqVX9A{-a`%sU zrX)4?Rwnuh4Fg2zE@Te4d-Fqc>R~7@gV$h4^z3o)On6jltE~B)Je6X$8gbl9S_5^% zj1)T>8rqK9+PwW_+`Yr>1^n8W5qaos2pPh5vJ+O;#8}HmbhLYPvjEX}-T`t!RQYi0%evZ-*1`v97^pdQa|?$#Er5pShb4g6^LxffBl=|0b0 zzVKK;L%h^3%YqG3ZZg2$20|qIb3thc3Ck=IotYY;YJL8ZgGfVI*zz5TnitrzX!k?W2GTOE@||AcLlzyj~J4PP>S?n14FvU{-l1x0rA5I_YxMrpXx;Cos|f zmhoDrd#~ds>u$9(^GW;o5u28bj+NKbs612^y2D6Q2Q4`(37uTJi4lI2vWG46!AbEd!{ir61}~V4PA%YE*S`t3)y=(m z)h#YIhSTao5u;onvs$Y*{*k!U+uW3gw|ygsHEl&hwMIq{lf~U8PKR!-ADpi@(dXSH zH}Fg=-q{*+t{JV;p>kd9GP{M(d`32O{EgxMKG$SPq3B_M(88-Gb2fzARgJ>lWh-QQ z@&r4T2~cH@Liq9CdRU{r-W)w9NkZj2j{UL|VD4sp1lv|RhaXLUbDi{?(h_}Aa^ro9 z(Mch^Vif8y!(M6DpL49#OsnE0HVu%0W|%5y^^Qu}`#J?6tsMjW8i_JtB}~rc1&K+P z$X5z{_Pm%`H6gKaC~yCI8A(Bz{1`6ne)a!e6=JoP)7*K9bFpYlaO14xv>R>8-0cXT z`QQ!pV>wGk>g(slf)#EH0hMm4>8Ix!D#GWX2UECLy{HfErtIki!NEgE45YmOM22Zi zOhsxE`0Dm;JokBZ`iu@1amkCOLPkND!cjA2kkdxR^ii8)g0xewi;d;`svc3J1?|5T>lfPWi zZR{w#7aKAVk#(^oxV6+#Zm&6arqSO9^KP?R*Xk^{%EU_qyf>}jI%3|m?RA<~-@9Uo zIbE5?UnBjA?N}%s!gvoq>Zo_N7s1Gj3oGq5KG5pC2?x~n?4S~hYkx=N+~WN4lHl&g zC1(PE+-vUUU7V(M;ScW693BZe**=E)i=XA1x{htx&ts+>9{^vM(Xq-l98^CLBj`ra zyTE(L9uHG|$ewi0ONbi;L*cJt?@*(ziV0Qso$q9KITHjPn}=gEm()~DU+pr*V|0CX zte*Tq(CLVH|3pI6Zc}@G{wzRzSHjGZ$X}Y9%1^pvbtdQOej+6R6eH^=Rzl_ts>O83;O6TRC~vj%-F49Bv+ z{jsI3^C~yKWIYO8?Q&!7=0v4OR2_Ql*BW-YYX|_O%P~ln>|>yieAYa%lC!1Z7DEPt zjI})DC|Z!fMo=J^JV$g3oTnqoHjw<_w)PfM0b39n<UCxg`c@0;Aj#QuBm=;LL zJ$x=Y%W^aXpyQPTckZFG0p{>x|A<{kKGQQK*zn_J4UhU~x`?U-6u_k6ME;|Ohi@wJ zoNv5UhNAEL$1BD7?SuRr90u2p>n5taT+woTaUp>9JOJf+#*0Vc9!3nJqhZNwXb-o3 zm+-KzjGMhvc@{gw0b7i?njA7+VYp_#e{W89BZx#m2VwIW9^B569ihd4+alfVWwCNB zrR{=k@fL(#C#s;>QNAGZNCjjn5j@vPg>VCq>TO136*0Hc(hS+Tsu>px2~_Sryz8oQ z+O!e{%cyQD?ud+G@#vkkjJ3){=VRvuW zM`#I5`R6FhN!zM;B!18%1E~Yhi5NsVtruXAxx>2lToa1<-cfjXXlRMLNCw)Z_3?$~ z$G#;o#XiH}N9j-GsijKfW2G8u*v*!E<}k}b$MSqK+%Y1YKI}#fFY>YT8dDXuvdN43 ztU~8|T|v~Xw_XDo;4jWH7C{u|Enc`g>!&&68$rmoOSCxOB<-zgP5~WsEM~Pw zugCjPD7ZB7Va5DuljdTD-_-Md#2$E>=SnB-5$g{cTZoLEU>$kXLLA93m?nUIgU z6kL`aPeeXvE)pw(dPDvphlgMJ@j;m*2D%oE7Zom3Rew%i%_6FHKe4xq1fKfpRz6}A z)BVhgsBY`1SAFr6HkMHJ)(qQG^h-PxTcDD~6|EqW9^Poj0Z9OM(A&;(o-Ox9EYEbj zRDfW(s)<#fd4pzXn{YTtfykfVP#l6+5V=k@Ry?JdOQzzs91xj>I?*sU-Ul`u9rdpU zgDq{nB0gGqDZM6D2~JOqjf#p2XPh~Xkwnir?&xhheN9viU@uw}A0N!HiIS+m&f7(p zzrPvD$T(ywcI~a0TQ#mMQf(T;M9T&V?>w+$E9SvvSyC@@4jM!x7v@EgRU5j_o@s(r z^c9t!8oDYuHrp#3gQ+YJ#Vc()qDW>Wg&n2NHWlTSdZJ}sV!1%j(-wPX?QQY zA2Z&KhQ_aSkcxZz>0Ht0@v65*Vbpwq&%ayyzA>*pR+VOP0dY)$`rU#Olfp=ekZ-&X zT-??Z-*zxp&n`&1J~&VEekNG%kW&lGt>z!vWE+E&7vSCN2fbwjmAi^>MD9Z70-o(6 z_Q}`54^56r-wD1rgDB<-N)iF2ZnA*xW_xq91>WeiR{&YQ>^9;4l-pu=!V99sYY6tMdLz4Fm}uXu-)LUoLI zT-&usUD_4?W3ZEb1#W8fY3vVa>=^CFYi^4(us*0h4bO=N0n5&;9N2IRnoGHzCO_=i z4zDh1If>EHLhRasnmaBMPn{+Y4T4NbfOij}sY5Az`MIduWaZM`oWS#QklomV5h>M@ z;t#ASl*sd5bmhkv_r~L1shmFQVxMyV{&@FviJz-ifuD|f73{ZTLHvig)BuVl348{{ zlK!|dgybwQt$I#G22zHCp1m0GIFCnMTPiBvr7oD-8R!Pq7gz#&heOEHaZR{OFT(tB z_g=S!KXouspVS&}xhrB*Wu%kN}R*V1F}6vazW>zsP-8|3>8;6iwM9+OT2~E8F1tmYB3=!ITL%U zG(*;f%xjV=uhLUDz(n8OttI5^Iw>AK1cCnE@$SW<{i=Ji@oeL)CMA4Z&0N!rJ+pe! zz!H$FcZ(s)kDxD+93YA7jX(?rivRq;(zgSPDH>UZCb2N{53i&i1Bb7S3yte)SF?3{ zu|)0IIr(2LE=J^A6{|i!{+vACM6Gd+EP8M1z5B@2)D!`)T>QwfMBdA9rCDm_gD>s` z(5FZT#ZbfkFz4_O!4MfFj{p92;E{MZ)m-A`dDT29zG4;yG6rtWfym_LZIFrrRiECu zK92`o;l_AG4HYv9g)P*czJ}}SJ4~ns8Fk01wP=S?(PSQO)9{igRs$fiNtH4a;1G-xwtl98Df5^rS0jj4BKNITv5HkVujA)VJPsfS$O6H@6v6~;tM{E-`;kyH*3g~PhQW}5wA@=e!ML`n>5~~Hm67LTm-FC_{>PClsD1Q;cUryM)mlc zF|W#b0!zl)Z9zHHBVSLN@NH*9mBEIya=bi*r6PaH2X-s$#-OI!sT(-3^$;pT-BaUN z{v6_~$NOGKbqR=qL2s!CQDx5-B?`DKpEghHjxw)KCkw*1n9tD$>kSyp~|A+h2Qa?)kCnDq8dZtI9B~&jB2kdP#kUzvb_> zVUliZ{+XUu3TiV6GJBd^HsH$qX%N6|2&_j{)^1MdA3U7=8y&Fy8V_31V`061)Ui1z z`o97vr;47(}^xVRuMpsqldsR9jPGAmu6c5;)k1qyiRkX3N zqN^f`OH7aylGx3N%9$snlP&ZXKP$t4I)w6)CG!X|^_?{#=E?|nd1O|WBI#*ELYoZ( zy$X7(Z0!@n>-@`5!iR8X_qe-DU+ z_AG=Cs&NXd)H$W-UO9qn2Xx{-*+V>$4#;n`;xL%Gyw;^mR< z9;hhP=|GD0(Y|tvS`|2*bn%4iKx`l`w%vNE2hJYCf`yuZE@Sc!nZV99pf_T7|354b z!#vlh)sFCWtDqm*u4th0!CAJ}Inbzko^03QhypQ|C(%eEklUi_D)mKtOGd0uXaRmiLGa?Gm`95+aR!7FGC%P5II8(R$7!2GD~Yw~)KB5S1d zZ0)t{F+nuA`}-;|kCe&E5|M2L-wpxG=(0pi$QF?i!de?pzq~dR0xU`(FS1rk6uS3y zI4&ABeJhL?f|`sfUi#c#&P1g~Ur#R}WF^)AM$T@1x8A4xiJRkOPgtHMyJuaf0PC|l z@N6(#d+4Y(m07RV%ErKg#VY#oTF;+Y8_z??<^wvlFku5GKg!rXv6p&xe zKDH-g2OHFMu>?>xqcP%!vvaCL@SkTlna$>j*6I~NvDLEdqyNh?F^aR$ANxcN`Fnn| zHuRw-@5#ulb#!|3Il?(pGldhySR7v}n{;GfvAmDWXMO>50ZLo#ONw|QI_Yn?%jd_A z(sK7MLMsNH{eK_~o~~p9$ios|t9%qLc7vV#>y8A)U3VevsOoYelib+UwSnq-aeUZS zrADi%o6xTh@r6_`)S%Z}8cB3;M(?UwA01!q`rLP6IaA(xRGU6)&|1is-rm`fMRoN$ zp7>wlApzo9U=Q7el=DAp16py0USeR`f{2Q$d4q|-X`%u}Hw{#*B2+8wQBV3`wUG29nAlEPw$BtqcP|Uqb!e^0i)1fxyM5MSI0h^56B9MxZancE0$n8XmLJw)|38`ub<_r zZoF~U?lQDghZd>7VSALlzrMddb*?EJx=S^4eQMHot)%2RA(J+*WkzEv{;RFmanmzm zOPdM#SOJaAH*#(^5e8Tjk>nRJv+`0Cm+CUXaRW`=7)g^wPyJQj5-fbPMe5>9W-#PX zZTZij(wv5KSP$?g`{Xa5&#>JhLWK2vCqBrURxa2N0By@ek?o&>sx;9dW8OsdbA!P$ zo-(tVc6L&Y>3ntxMGQyfojme+3%)VMVatUxcb*Vh9J$L$h*{njWDDQti}QqLV=ghj z&QSjNaFjSb%QP;?Ht`nnIPN>h8%nu)SOm)YT54k_dL>0IC=U zoRTt8Oh;bKSFe|VYlp?`3;H|l9$)tzl~7yVEtG+WTakw;)&p1<1jVoYbNas;kM+Nf zdnM}dMI+))?6z`bz7s7AH}ZQdphV{@YcN#sDFT!%2$*7+3hWZ;=^>4h1d<#IiOSj! zY&mk9yYac}sSPC~N%&M#T6owZuDVzlJZk2EER^_Eeye`+m`&n=7K)v&1{ z$4rWnMGwZj0LF|wt_5R`@)5_u7RmJ2?+(OL#4^IcBM$f>l-Vw4GZ;+qJ}@v)64Wm7 z7xz^nYD@`pb1Ewjn0P88&E~qKa&vVs3hee{mEL~sxCV47UmHIPS^3Hcy&%M*#Zjq) z^pg@&>dnu|WlF+sOwOj01jrE)(o+*jE|Wqmsp?F|kKTNe@8^-la&cTEWY+mcrwIRK zo2>luJvFSL*_O@B5(oaUj>2qpR#nPVC<7c@nC#>0p4x(nr%Fu<+7bv^sRjv6_=tCw z501(CBPvJNKxscPLZu{9>*S(JpSOb;ZPyy>q95JCh<*fR{2Nj;Vs|Ns>AkbPLLrgS}bE-b;7%|RFQFj!ir?RU=mLF zfrvBw)~ViHBKYom$}3q~&X}3w4Q<(RMtSJ#VW<~B@oWC<@I+j!R72;LosCWVJ8`nRy_Dn2+NQDs8&ih2c-D~rRg z*^%^o+!*ZyVDUnhYUL0$wn61eoLmZM>vEhR2lfrw@$&NW#S|>%r(!L41*-G&>3)2n z>JM(87u|k9Z61UgL2zJu9d+?8Xt}g#w!PR4w7gnG;g0|if@T|jjAh6xK;91B$7rVkBhKDGpkzn1#X^j(j0b73J^3E|O`gVdckwDK z3+=_X0bcw&_~h82iIxb&%s$#mUy$BHNN80;>mddRd~7ETkrnJ}#_zubC9Lj*KhA9Z zx=nMi^`OczWTw35B}stY!18ir@Ku-11ifVJTm&@$)^k-)AyTmP94~Jr){IkkBZA{Ho)o8XjE*TdC#u`Nf5|QV1#zIM1#p^4NZ=wOonwcF z3<+-E?>0gxd9cDFvVi_YM+t#S8^BCBqOoxy%tC&UP`ZWVd~*UW+|vN( zlPqd$%~^R*q^o=+4yq9%f!zoo9IBxHs=7EcnHRDw7bZ#;IWS)XIvFhJmzL!P=+`|W zqMLa>#l?kb&)(Ljn@pd_;y?z-r(c7CVWt;gTuB1eyRIUHP5v_CvJWt&s5K<`q=wd0 zex`RxYZ;Lm1W>7{&Ubf_PlAv~DB(KdbBf=51dz}Efg%F*iGaGv7CZGFFnl7@D+EX_ zM`GNFxhi)8LS~0D(-M26t$Ztm|zMts#64Nb(?%QQiRAb@>ybq>2d^dI8`k zYa{p@F0|$&07r4_|AN(eAaJH}2;ij&c)#H9xKIxSO4sfmuvG%YNvh7@aG{axi0Ntn z1*`oIKsvn$)UOBPu&n6sxX@gH((U~RY_(elysh?kT&Oib{Vsg`4Xcd<_66Jr;=;}a zFxcMzz=hfYnT`GfwyKIiANc(Zl&pooUVmi&1*@$AI8$+H1nRd4G(7ruP;w1G>57;C z1Gef1gktjbSGjK>*>)l@EbHH}S_gm-TS1_HYXA&ZTH2j+NHz;}W22daF-6dhZ~wXEWF(4<k(li&_k(N0@Ie8G{hYtw-qDMCJ}P9m%7P469Jq z$kT0Ek0#Im3Gj0OR|Hmw9!>me_^(Crb1|%dFZB`0fKpe8wfbLw#{LeBLNB{T*{_=@ zmn-UT5B36GOJ@IhZ}aDZ``e$%r$M}uq(5zQ$!qdIeB}#>FeLP!-u$&%|Nb(fKQ-(? zf4XW`u>Q9${+A8;M;|@(fojEh_FWRS{SRII0B)PI#o}=O^UJ^6=>kIH|1*X6f9oQ7 zELdASlt%wU+m{lJ1444TCT0IW3?KQ95D!o$n`g)aiT|DhFf)8WNP9OvZvJmw^t1tE zT=7r6Ea37#eCA(804MeTD#8`?e-+_hML-(*e-+_hMfg_{er^)}bqyfw_}3!*|HUF0 z7|!f~GBHLrHvQ~uWnLccD0p~yMt~g7y=f0om;PsDs}&%W*z?lV)O2t(@&4tn9)Uuj zPI=~)zTVmeP_Q)xBu`IO^6yBXqR^m#ea362%~JgK-e}uP*;_WeEi1v6ObbkJX&ZS65eMo;-QpUw>E< zDqw)PUF#tU0^x^-5e5tpXlD$UwCMPq>!^Cfo_LzyQTFrW&uyGkaf#C^D_x5uLyvwp z>Zu~4<>CSr&D_a7t!1uwdbXE>ray2`|3r7OurN*2VPqw}+}ZOk8JV1*!(RESfv%+f z$!QdTNp#;uojuY7mwWTfEN6CFEouEpm3!yAA3MZAt zj_-tIO9l?;+zWI#rwaO;ng2?SWa5F;|4z?yQU%bm&)qwBUM|HJ%bSiCwD2VIaFuAE z?Hf&{Mp3i;2#@ zSqqZ{#S~P)%o&o<-9+ZRk$cM}R8@fAejjl^Rn2}Qjy(d-Yq$QA*M8GrlyWK7K5n$Y zFn`ze37Q$k`dG2F>2PjUT4;{@8L=5}k}dip#76dxMb9D*#4lv7Oc9Z6G5Sevft*GU zS&(Ia?|gUyDSMDx;2+VoHIYHUtZHIvm=u3@Lc?^IxpW@acq77exypGQ<;<%4y)n|4CQ`Jd^H>Cv6YzmzZ z;#5i@5=NVdE7L2WXyG8B35~D?N;eqXOa6OeEu<3gL5?3Ur();&{%~Rt=Qe^ZV_bIjE@ z7AxH9oC@8is)**I#Uttq)ox$Y4FKC}&M-S-VPw{86=aa|s*LNJe{T>ye1$SF;50qG zMcCA4%f{JiiCCnMSOT((ic_N%f3b9XhNZYQZ|hOtf|ez#upp7U@e| z%7u3J&dhL3@3DbTwg^zPY3~=N-)nvPEuufhgPAfRhTN*qcM;mP@dkpnFFxeyb-e9A zJD8f{xF{UTQfQmphcFXAi!-PG@wsF;5gp@r{8=xetENyY{o#o_5yg{~s-x>Z{?nvu7JnpT#k6X#h{B|@(#yyM*( zR|;pmEL5~`T%o&h&d5v_+Zn4@VDepWIu!PSSH2VvO zvZHW#`-&c^>NeQ6I7C$L#u{az!S%nQuWxOcTfNdKaQHfUa#SgjJbN0g_*IKzgKK2S z)p%$?xL~}>wS2VDcxf+MfoZxaE8BfvsY@<~EA#{Xank$uGZav>iL$Kk)>R$M47Evs z6KE{G++t7_hb6Z4j+?*gZfLtY-EVb6x(ja3}7}v6$LvNC& z8N{qjT8hE^TkZ-IRrPU+4U_6>CdQKjh4&mYcWf+~sEN9k1l?11Z_HY5Rai*a22`f{ zNz%W{4!xS_-!xyV@%D@$`>RwznC%o`AElo3;^BYXsH*QLURpeEc|syKJEvjQLQSwa z+#Qwf+rz4qA9A z(w@55W2ns&>D3+IWS{jNltH~wjYXp^{!~0Bq7+u;Mq~WaFkg!GiR4syx{g^iYAKIO zO5=fuOIDi$m)?lAZPA@uO`zH>jG7@;j%!AC=`KWBqrxC~)TK_3mX$+-Yl&wfQr$Fw zs!cI|musS4chV|fvuy8l`mJG2>Sm#5Z79$y++kQeT&e~hT`e%fO!@5MODcjNB z13VjC+FLu$O1h?!q+YgpQUe`ppopNs^$wb~b^3zWEw6_Pm80ze2ktKJu6#A578X(% z8>x7ZwPtN~82PlVY)aR}B#32KHFEEPN#D{p-Roi`?*~F~Ha}W-eq~knu`s+!8SvY0 zU&%pTrAKu8DSGKfQCG){8p`Pg9_d-k2**$6##gAb7s8b^)2?CNl;EPXWHKKq8;J6A z2r({U*s92@kQj8Zbz0=G%!BL5A~Ix%AB&_hiWqYZeCUOBu(gIH}yQsIawg+*#>_02Rm8w# zwYlc>6++Gj-KB5Ep>0{kCUW{89rYsgtOrEj@2b$UatWjWu5a_?8|#dq%ixkJ^ev z^Xb74o1WPQqphWbQMG>Wh>&{Wc(dy?)=u!p1vrpO6IPzTZiy>m^h=i5+*hS3^}&?D zh_yLdV5q#yyBrrKj7b-3KnKvQ#fnVML*<#-)f^G?v9vnb%Bq7|R2W&dUpd6t4AO{| zvb(J3AB0 zbel`M-or-sO?GP3R?5mscP0!_qq#P~;13mS?-vyb|XgK6YVd(BhjBOXfRp5<@LKLsC5T$!>i(dqO?Cq1GTJ*<@R` zNrW%p&>IKBQ1Y#t`gg?eKr{gY&dz$+R=7XW7h9-6Lg9*sG5qgL9S87@w5D@a3k+mF zn2st!kC*LO`FRJKHg#;;X}s_WQDk~@6lFnioII0H*^^V& zu-jZBjrV;$2!23pO6r0*0{{-5J@zlVkb#;`0FhL5hmjuJc2O&o`Mkx{!U)%tSHPK-CEg{fb~13ArppdjH`HJyTIPV)7g;nI>smvp`)BkV)kuDu7+b%N`|=(t*z_zDp70E9gmE*J9XQdWwU3iU!Rl3u*Rl0uFN+0 z1h~QrZ%!QQ`xzLrh0w?B-I-xq+uVV6`Ba4sK|8tL-}p0=@pg0|D5kami=!LL*sp^`BVd412K;7=I*SwDuZoXA`iS32;h zSW`-_x)s!yF02u$VKv;X4(?5_^Sm7zB|ylN8D;goII}4(o~d61+thhXvB8|^dxfP% z;)lGLnXzG?)SuQWWGFiGvPhaI)nysa(a8QV23y$(@*yOW1nFikP+hLWA#B3sd6SIg zxXGo*MHU>Tv!j9Elp>V&j;ZVv?ixL9GQgdJ3MLZFnBa~a4`y5cU^Km}4??C6ZNQIi zG?X1HNj7$_p|nrYxG#E;-3ArWl%z?@)jrTVoRQCDGH$5fxK%Ot}iMH zeweN3Po_+0Cu*Xm?LTv9t3BbeRcFCSbMsUB_7_)ooA|X%oCP;fD2~)PE=a*QLSPdR z%!}B?zBM+%fMdL?YHO@zb(0Va4$XsRK}*@Tdbu-# zELLtX-TX^d`lwJ$_-xWZb&}?U`-Lm};!7GS*WNANc(dVc(Ws|P@li>VN7(5EHhm8* zNrv%BUXe6##WpWe5Q^N*|Gvv$t5GU1syb*=xU~h ztK=u5NMRlYl3jjLhh2Q}JcT*5=r~ssK7E2w-Dtnn+$xo&Sq4Ip!R^C$7*V#2BC8#W zJ-r8KC^LD@o4Xd0L^Eu3)(#3$)izv0#O9(QnT#Tjpp{02oP;(Ld9);98HzbSLYVk? zAfd|YDPFvtI5WL*~O%)6!86YY8~z;xGE{3V)9TUdRe(PJ*tw&-Es zCN&vL-mI>Q_#tiA{J;^8L9prmLE0zV=x-2JVWpt*;)H<72c;G^X#RD{NyMj({-Mm> zyK-!CdE56wbX1Nlx*qOp`aEJ>tI|HZmc<~6KPNhd*V6^!w(ifrsj9F+xe7wW?&0>( z_bS`9>kY5jsZDU+D_lrrd!-JDSrR7lPH}9e4{@~JSu>EA<%-LfDF~~yY-3W(qFT)o zJ4mCSd{7JH?>r+)d_Vz>HhZX`_rN;XpO#Lpi21-o?gobCLTS_HGEhCv<;e3w2xsMfdH{1Bf(F%@j;I668~ zQejSRjCfB9x`h`iQq3r0%nq|y8Vf3w!_lEB^+MdmdM_vTP#Ouk(ls8+Q>$K=1B;8d zr;gLnM_lfMA@SxgEZG?Zl&PS`Zfcq@TZ@B)Cc(;dUYo!)^@@aor@18BZ1mb-`{6ga zw`8wW4nQ76@n@+=WM*sszLs2x4Gvo8ZQ(Sm&mUK!lCeG~b+O(a>Bu5SFRHzh_&JF$ zU+~BBE6FyQ@bb!_^)Znj>Vw7-mNuro5y|sy*|ub%3uS9=Pe3s{75juGCr9tLFYPl5 z^(r@HR@@lFxX~rQD!J&DX2cjcE+{CadU|>$PpqshYOgVP{sS@LjtW*LPjJfmmUF0q zO86Z-P5iMgQAedpEXb;36XU+xV{8*2)@0uZSlozTrm;hSAK@KXW9WfusG?RKCL0~c z$2Hx0+uZhNCGF+J`~zL|(82welRx*YHy)cFZ1Ve{(?mTvX$~{Bl4d?hOvm>3eloT% znX&93IkYiVEWJN9o`Xfg^974RL!=-=Q551!Y6%7MoL znlxA+MfpDHGPOl8F>o1TDy_wZ)?c?7E0R_<;IG68P0>)$Q8<1_YrMQe(>!IWk_RJs zF3bP4ZT7a$#QQdgsZm0@{su+9h2eDA?UbRyp)|VjH(MuHYhl6aD>Km?V6*3{{5Ga| zQ&N$u1cfSOU47X`B>lRJakRWFjkWLBk?Rd%H_ywb+^}cz@CiP!{z`4JEMbJ< zL)72#K^p!f#)MRRB3xN=aw{M00!io!_h(2%YVuKJToX3a?xsh}u|mTOj(|aqIAXKm zd4Bh&IG-W^hp)H(Yw~^n#$P5Ph%^EcFH!`CfPj*s^eE}rK)M?oBSn<%Zlp$obd3;@ z?(T-cCXDX*Uif~0?$7s!`*HsR+qHA&b)LuZJf49SVp&#}fKHzAp|gbmzSDFfZHX)~ zNWre#_+aMM2N16aS86k3cg)~CdgOA{9QUQ)^v{OrL&AyAU zVMzt2I8;p|Y+-AT@Iwby&>-N66oP-ENIPq%`KAdG4%JOsPYlN+5>>i^qpN|qiw(U9 zf9b^fk|!xq__gN~#j3#+oFLxIb}P)rbg=!Fb$&0OHSimfH5bOz|IL;(v)~m~po!X5 zqxr($wlyCFlvAs1c#=WJj^Z&H9(yT(i9n76L1RHsTuUUP&O(J~0+_^uOn% z%l-Yw@rX5Qlc0Qgzlk1UKrYmUc9QRi*{F`xh&L#t1^hL=4{FWEwSXFq)MpsPx?7*(p2eZ=(T9xnNQXW58vnJ(Smz0>! z{>9?UOn34+8kWt8r$XIH+?w+wh+syh_vOxy-uDXVg-4|DUSl5r<1_L+MeSpGqx@d# z@XG!uRv%Iz%0NN%f7?gEs#m&p>~}ePH20M8(Hgs{aoqSTZTxe>MIG%*((3pU*BgA; zc)>URc^wS#$b{?~t|S<@fA<#QRS^I7lSgAf`?x7z5!i^JCn*JTk5ZVdDTbYU{)AQn zA-|)K)Ni|>c)>F6(%BCxETD9OF+rLgga9r7!rJN70MOmwmrHa6Ki}AAd6{;FzdX`e zF!UdC=xt~76HFg&PYKJ9jo*XqG*%3KvF@1Qdi@2TLj7%rpva{0JHD)-9m>&MZtzTk z%fyfJbRhI*n_0+X=5);O!W}LNhD*&gx+}$5sB3f{df0xL0|RcqwzC`7O3ch3 zs>#B3GyG{)a?>5^W8VTS%<8LXp^^rB3r1zfDD$mV1>@Sg>JOU7x4#H#rIBCBG5ubZ zug_-$=$QlmkvIHBHU#f#fA5{OZBUr^aPl5~7pya=&i2TZ8M^*V7g{;CO@_<%6MnE! zbecLjs_nZ_Zw0e~Ohp^MLWjtt+yl$Bs|v7>3lF0B+Z%zHm2BfK0EN;(r8p_(+YqH{ z+S!%XyPWJ$c7WkRfDs?KPxVR=FJ5re749C*ssSE}A45GEC%v2i@&RgdoN=lK<Xe`O6t%8PVjJ?d1Pt|_7TbSTfBaP&7Xw*Y zpQ&7I)1Fl-m_%phfJ`mszj4l2WIegl+i?!2Mqp*e)7)3eK@+fHm71#1Qt>Fs%uuY8 zdS=cXD^pEZOG``0(WY)gN##F)(e;yFHGl*O!|mN3Q?DJH#}7_OnOxpbSMLDLYgl{_ z2|#e#@}v&da}k0#7lqCLm+Iz9AE93BF+00ZKtLz@JVeHeZfBu_wdN>Pmar%puMC9v z@zkJjToSB1m;kErjGbPt=<8+DnIQ}9AA=jy@d31wHvC)3CkMAee5K;=;9N-|BJ&253`xZKEyt}Uo0#{U2I z)b5m=UkWMjXq-74u!AdhOb3t9Ob|4|DuZ~{s!yHg0#e?#e?P0$jOWKD|45z7*w&9Q zJ2+go!r^Jo%nb<{yXdFY%U$9l+Xm;7unIB)f8;24#|ByJW>T@`5=ut3I2|yx@>;;h46>i zC&J9!3-l?LN8$ouv2ou`Y)MyFSLLp^{`{$I+TZhKu`~UQxhH3?OuYMNs?1aBn7dA* zeF7L_7JhHU!+V0eTu#Ujx?Q_Zv0r)*2GH*;@Bzz%CwJmob3i)=^Iq(lv5-+?6HQoY z@n%Ocj{!LDVBql5y!U*dBfhqzwG=dA1wBEq<;N;47l)94bWTs$P(Q8{AvJ*QOl3&W z4UuUs5`F8%;XSB2$z|2}adVYSsps{g&rT{}s zI{tf|Ua&j+U6UlVZIzG+RBXfk*X~(40C>cbp)Y{*CZRd*UC?(haNQ$u#i8P_;f`c* z#U7y!4Z^XJ4KhXg`?h3oV*div~zS++$-c_bH$@0F??t9 zzKum;(KUlp-RiN>m`#nRqE#Q$iI^@ouZxp3Fl%CbuCEDzPdtgBecyktCw}{1eB)3` zr2dW!q<;Q9MQ>e7+8>=Hw%AjEv5j(ZCW;rSHE46&4(;C1eR1392mNjINg0;*k?gPO zZ@?-zfLQFyEWOZweX( z4JUutVrtox7PSY3Qq4hwqR9ew1EZ-G^@3US@!imJMF}%p2on-Aaosb<8yLsW`JvaG zNOHr}ErY=(HZ^|kq@?k4s0KyEF(6?ot?%f*;z}YtZ6Og;Q4%{E{b#VND*(N@`QmHA z=gB@G7kP7dCiLa%mjWP+zC(U^UMn~tow7_plievRAt6E8S2`?jbHl>QN>xNe1n$qB z<)!XmzmG4xgQ)_`*8XqgX%=9LKcuzIIH+ z|5U~~+T@OWoO}sqXlvQ{INuvQp?uX?tH#6BBV^&ocM&}4dESzz;6)CjQSA{gFRUKw zt8-kLBu0%*r=CZH$AYe0fZt{Y@>0>v`-VJAxAs|fF%tOlYHRHUlP)!V7_IQW-J(hs zBO_pm`N%C#E!I9in3ZvqvtXoD-@9?9+56@-H=SMKivCJxPB zDtUpFJEQ$ont+|&s3nk$l%J9D@Al(f3NbygL5-onTF1tx{|aEU*|yWRfUDBO_H(1Nlu2R>2mj5TK~E4#uqq){qhl2#|)@*Q}JJJ>cVE}P%-nK`8r z>FOpRrPZh-r`gP$;DeH#%$%PkR<+#oPPWjZY|r307RFbLnK=PL$#zqasmCKsevg$X zI^_4<0|OZW456VU*G>HqfNbS9UTxc1@vwd79Fau5^1k*MH+Oz9%%Wdy9C*HYG+DTi z_F$C#1w(dw38|GFo5RT+*)CoO=l7wZVGq(_E-v{}Ss%@DrYa3NUw?@S{-&!E0gcCb z+L{wgmIDMWd;$?YqoPd`Ep$FOhA;Fp@AQ$~bxQqWvs~U|oF;2!LCgug77Yu_B`Ysv z-Kzlp9zm>*I8bx=mOJ3t%(ppbEbq*@=N4}iR~pUun{X0m`5tiU$aw?9ab8Ae$?2gx zh|eY^j*Gh2v%Pq%8~x=8h>*>jy#rtLQnVOBQj5AUp6pYX0^sBj_``$ac-j9W?;2$H z>2~;SHQ^y60<`W04Gq#&HA;`87<5YXo5eXHwaWQf()Tod%t?KQqExOwpYAR%Q3b}+ zx1y#_YpIfgT-bm`jfaEhC3|1goX)KwCTl)Jmj(E=wf=>eW=QJ}{qG-gLo;>Fm+MLX=%c0Eagr9@$Lre7l?WRj){Anbc>aSz8`!oJ*{Lh?|=ALDZjxyk<`2<#5~oXsMUm4=hhfugZR!o zoFerm@|QRtw$X7l@zuUI%pL(YN>pi-RpT6((4PmeuVP)LPzx z_+nTdC%)MTWY!vo!wx2N#}>V|zzV7nsK$a5y%dYN%#Wg_y`ll(#kRO`kAS#x+IqhI z+x-E6aUhSy*PdoJEE@{-SeAKS(A9kK*yiTu;W}Vw&zg`s==nIl`l(5(mgez4j|@MD zhO-5VQx4WSFLmET_Y{^Scr^^q&QceR6yA&$jvq~yPI@_HO~ex4E6SZsaYc145dlBN zqMrfR?fv9_WwA1Bc>j3TPLtSLKyS@PZgHkbMciX_NM|` zMTqrbln%T9CTf724mEGBPqG2<5jJtBh(Ruc4C=u=)#>3ufBsEnPUdR+%CH)$7cE&? zbdJPev6}jgs5BP5IzgQZ^syln(vGpU*#cLRZcCNhN(Xah6`#U}V}a_*CRjd6cv#D5UqqhRIM)S|iGl%ff;FS{#+pjAY(QJ3CiK}l{o{6hl_$E`?&@d^B?PQi zM=14pgq#g9ds0R{2oN%-0O^-GnmC(u|ZlwOlgEA z^9snn%|MrxSS^o`+E@K~qD!H6;YZD%p979ZjF4a=GnSVvY6RvVFz^JK1ZuqtBI~B} z+NC<0$9ll(Ewi#R7WR%|1%)N`FUJK{IMlNTUoZ0{Rpu5qLXVtbtOJs$9R`Zi)0JFt z#UQgSbg}Xqma<)swPES8{or_kz$sw20fAe$%fYC;h9r6TF}QcQ>?5D>YXFvp->VhG zn$Fh0qI>j?p3bKnfMru=>uq!mG&?L*N?r471p(G*n25OiX}l`+=MASRo)i1lV^6XF%Al*Ai@(8qyj3j zj2VQy0@Is=eu0z125gg10GKik2GT`nl%|6<%Xp7HZk^j4suKC|?-xDb8>dKi92oh#jYyjFBK(Z-I?<_UBx zrhg_j)?nt%O8NyUr^`xlusuNu@~z9F_VU)ul3Y`;5bTe#;S?zs!ZXZ1@g#XEzX@-d zryF1M>!vM0w;7x5ul+>P@60m6yh^f)-fSPHm&!KY42yq75U-`iKO>oE{q%Y`mdkSs zY>5Gmy3^BdRL@gF_Q)RI6?I(unUwH`>QaFq)0~?46z*&-5sI%=@71f-T4N&%rdIe8 zVLY9Av40VS7x~&tRTxZEXfkipO!T;u;VP(5M(s3qFjGv92qZGEw)yZb;qLpEV>k@B z*jn1&MPWG-$_a2;Ro3CC2Q0S(*(w%gsoF8Pg*x*Clhh3aI;tnUA0y_q}QS6R4IvZ2hZ zo(kh@AB+GFvB!@{$v`f+A_eFDDNH)&&e}>;ojddHFSVdf@xuo@{$d`G`SNYkqbxd6 zTF5J5o=hoesmH&Aon00Zr<(TFOVz0Drn24K$@4M?b4tirY;D*E>jAiER2^0~GmiV& zWfyeFrt|n!YJ84Ou!Cl1j#h)2!{9&_H;)?s@8Wnyu0HuK%w zqQ{A0(Z8t*lky!b!(&POvB}ORT@XO1h?{wUsVDYDSZM__Bx!|wYA$8(^tIzb{ptZM z@3`7%XDO7RtRy8wfNhUCD>Kt$f=?`YJttw`dzD&Wb$&kjSB$@(-?+c%;lIVFFDGK7 zWYr1Yvl&9!>_&|tHZ}(_0g;StZQ`bTKbh)${CDtF1d`~*VXFB&4`oL}#XHtVE@uzP zne%}t^iBbiBw*!Ihyz&cqreE3({a6$;`0nujw^gOCOAizpNqs7| z+$M@FQS!t#EL!oH22#Z*q+ts#c{-XIL382314?E;)zkz0?Fj_$F`RK&eO*FwI5XsL zpQf=4mxq`X77DXkTt_r z!>7e94+qQ2wJ;|3{Wg>g3K8$fWdBdqsj~J5jn)X2YTz*;EJfoa{7HU7Jz-T$v%c*B zFpBCHyiDh>9RqB%n)O5uhd%IvWHCVSHM_~JV;=wQEg|pOi&wt;AffmG^oQMYdSrf- z#8{*ljjZTSs|H@*X>s~e`+Sneg~TGxM)|E!r~tvNo-Y4j1ByVTRP=J$1-*>*=@r2`ZT0Im!xsxp}Af&O6tYgPt`w}fZ1r> zX?@8Zm@_~%aeOlV`PhD#oxf;tDCSH!C2N3sWJ8aKk}oiJ9pEFN%6BDdE9-d3XJ0Zx zSWLph8WvdX<^G2Y;D>`VJylCZO@~K|Z1t5w5Ldk%Q&6B-WDwr{AYf2t{bx)22Q!JM z6~$m2RWrRHD-ow8(qlS`89jLOXRyjfdetlNdyrNEQ>;A_*h3Dg5{hgYt_AK^kne2X zDl5!opOKgQv6g51>*TP_GDJ_4B*57$=XecFq1qqCyv3HM&{YpKnCrK-waK;jX7f%@ z!aMzLK_5a^0DC_)$cb_J?h)`^yJN-jdC5Qn5L$-Kd+KeRO7-kbcG&YY`7o+D zk0U#{B{|zyo<6b~YM=Ps zsVRuMK4Yh|$QxZFi7VXM5D5?i-C^r_L2&F(JvI6mtr$ECkXrg_JL_1ll6&1}7e7D$ znEPRI46XUX$}F*)Z{)ugyQa{I3KPB_<+n>aZ;wm<`X*|4W&hzPDqy{GUR!mR>*}5` zs?EeS{2H`k|b8=ndWW@)i&I*CZg4n;7Js8)AXeiH7)JOtSrV~uU+=G%0(CJ z1hK8Cj#u73!fl*TH-JIMuD3D>Q%{4{?^?NIkriwL@8TNQ3lvj}I2Dro+3zt#|1ck= znP219mC%o<{tQ5AOxYihWAl04mEJ@ksAmPZT34+$#x-33V&+Pr(VwM-CNzv*FRmmY z$3WLT*|JgqL4sxq3=`ztdL<=(sGuA5jBPbw!p6!@Q)IrMQZ+|$&RB15y`=G2@MuH- zJm($#Xlx!vFKImw;;l6|k;{|LI{tL^$6Wal`O!L;XMR5B`0 z81ha(5eU4%QTrV58WDfD@_t4T8?`$lie2&iZl)L|@C9?A*ui909O83pWI``TA8en0 zFjT-mj@**;CNm0T6(B-L{#K84Mibke9 z+TON7Ke6*rB^>Ed%X9!00!tZTKFmEs`nzYL&UqqEJ#>(1x6b4EXxStmlNDf|XgtPr zLRt11f;4xS=FDgmWm}Zg$YL370eFiX+GkS1kK<8k7wihD(gN;>%Th1 zl2%N=3t+SVSW+}%8C-T-e72g_Y-Us4vG~``Y>8i3nsT;&jEFqnvW>#JoivP(YeH#4 zyu}BNqtaXh>nE8_@ZvG;Xtfm^IoaM90z875JM|cl%W9|S#Gx|euv2O{HHx&b)pUh< z{xoVOm;CK3LQwF|-4bBzo_W9hoSSsgIt?ZpV*~67FGO2{n)5<;o0+sbqs$wLTTUqO zrb^Zx?zN;>^B|t>mU#1GI&uy)dM=(XA0Y8KXC;d85B}-$Zoav3Yu2fA%pvmB;~wgO zQwV$}G9tz0toFKkc$68f#2=;(TpGeyXY%gq=*DbM=@HRzU0)5Nr~J>2JUI4G81CxN zq_hosGVPTQ`wM3+0&H!|&amy7{B_xH(Xk3zgWs)OJ@5gtw&riutIXCMEr{rDi`YY1 zQxI~u_1zL4=OgJ(Dc~?Q4(Z6Izxi0hn|frRGqiMzhrK5I=WU2|;x`*tW3A9I`HtNn zU30Yj#)-!UZ$XWjkLBO8ERj5EB&;^DS2&v~Fmivx-hNjr>c<)MAXW>J6tvSRwaI`* z_e*+kIMx=|*(yZaKc9${H;ONXZ1v@|jKLDE0c_weW70=k>Ybiow{OShR6^3Tbomt1 z3nci1=>c-|h)lFg9q@7PPyK7Zmkn?-dRuw49fAo3zVr34lap`ho={%{0D`$(CR=mH z=$ucXF!ud&U1K9u_8B*uVcu>DtdX5acH|%xfb8B@914$D{E$xlJYym5cR>3E7}#9Cn>2l{Ry7tx$iWwKHIiKUd;b$Zqte#)`AT8&F$q&a>}es zH@T2U?9tH=du95k;2Mmj0!il!9{(ulHQpkbN2T7LyNeckyjgbV?N9WG4+8?&fFOYP zp&|H6X1MXq%p%WvP%OiavlB`iJpt;b?1_-J-beo0ovUaYJ^)`N`$2nugWu;APNBd; z5mg3XFv+mN01L>B8e57H_<}tE_0HwrMrhH?0+=rEl-H7y?wwJrY5asXf_vQ|v9uRX zL8jIk!Azl#gk0@luLmjv5>L(se#1|$I-5!*2;-)b_Cwe+1|@1^JRSv8aPd>sXK&Nd zrDNK!3Lrlf4vK@Tat2ZtO!D|snTxh5Xc2F1=BqiQ|5Z`p0Bb8vN+}SR50FY%09+Xl zF$TzaEhE|H2Uf^+f0MUz=i`5nZhwGU^-ro*)JXWw9*J^AV~+=a2_1Id4J;eEiP zEP5}&u&hCDA+nm<8FVlS1-kw#8f@M?oHf%45CFK`T%GVKu;Wk0lIws$JR*zJqS%LF z&o$^SmvF{a#V)i}M+Fv616zJFZJmK^`XSyWyd(MV)s&KQcPM{8wgS^J7{+`^qj(~D z5-6)LQ_(_j0S+?B2PdgJr_^KCXv=OjKStrA{HsRP=U+;czJZh3&C7~`@N&(@BO5!s zqQ}&s1V3vY!1YJ16iv!NgKJs_c7#3sw>yKaY(Rc`^m}|j zy{s&b^TystQz@OOtN=6J4nKmW5ebG&zAT$m`tXv?id~DhXfw4>ykqimQq?SRD17Kg- ztO49jGxOX{V*zVOY195w{f{==+oRpe%)LidRul4?g}rdI^Le_*mWIkC8_Zo79;g*0 z$sLnX1T`YvV*M?a2J7mN#^|DKn#s0h;y-OnKpPsr)AjP$LiO;tnf7BR|9zen7OOzS zJ@EcS>2QaL5EEz;4?chE_~vxce1tlchs%YDAe#vh zSsRm)lX(dxo`6)iRk>|u_$Lue&AT5zvxBT`z}9w+7q#bbGeLjSul|^gmq3N8JZVH7 zC1_F;9*Op{LpFdHUa~j)(@F`nO+kRAE`KIW)8}^VszP_SSoV{%^F9&}yS%5}MahEt z9+0F0EOHcaiTWv zs&XnEE7=xIXi`8A^LiaNCpr+Z`2!;wS^$S;tgf&32$U#uAxrzCHP;TBen6oH%nzFR z!qDBm-KVG7gcRgN1auxtcaAX4Udxp~;LEyzX5wZYb@wB8-zND49TlyHQC75e-KyQ6 z_a7^U&F`<|#N_EFw>MkcQJm~=OeLytk-iZ-sT<_O4cym`t9dT{Fg!0Xzs?1Al6QRE z-TtQD(e=|inkzB=dODG7qoc*^iyp+wVXjQs?5aj#b$~RMjOZJVbN@`mHJ{9>DRv21 zB5C*z{`*Skk;LFPdnrR722^;;#|*MdXO`{90}ULw>&}HA*O1Y5C-Nrd<7vVEPjR@2 zR=JqXI&D9Z-=eeh$`s2g@rl}}_0{iqiZfDh4HFN?*@ji-;(upZdzt0dZ;|MZC6)^F z)>eX0si5Zx-C7w0s8lu*n7T$4ys5XoHm)gJ!-Nu>#AlUcn4%H61wkopN5u}9cOfpC zwnSsq0kAqv6!}V_WVxx$4!~^AHgxhBpVApx@QCu$B8$<`A(Bc#03GscO9+J?_{y8B ztaQUcwY$Y=t{ZgNQblc!!E5%K`}A=VyatrdsqoK3L{CFPLSCFn%c*z}69ELWE4sX= z*WqGn3><#U7+wGW;LOhFbYG=J`TH9t+Q$ z?I7!j?VcWz<&lYj9GXXCzF$&cjyLfY($_+jFGKve0@KxpBmMUhJ=0-^xH@V~9F7Z; z`Y+HL1AyslI8ECNppVvIR>lG12P*rL5^G*~kHTvVRv!WizLSUjego4#tL0gGf(M(sL;cwv(MCSS$`Efr5nWz>2Xf!-P@x#P#DS$$O$!i zZ{{4LBzE9TICqK?yGVS@NpsL-6Q~1|gOHxF>a~8j0PA0upO3NDC^RU@G`ddk7`)VY zslB2~u2tP538r)s#{6IhOViK+GL(IPI)J8{+QUbf_w>%&#g27k^qbbC14ZJJ@2J!i zT$Lz8qrl+#-Q6`Xmp%%rrbd=g)kPVNLMEr^!j>crUX~03CXV#CCXSUaeG0yV7c)&} z-~zx<*tTxn^ayrwf-?wJSrd*%(WUZz8b_rv`Oeomd>B3M-Jkoo4xfCSSnINAHmX<5 z@Bub*CB^zoWBDsx&1npG1MyZ*M~Zxdck+ZN_#s?%^Uc<1PJ%U6#UcBuUX@#w)8y_O zO&|j~Rb$D?VLomuR#x)99MQ|1=Tr&KI97I6s(1(v?$y@yy5@Sd`Qx)@Ty(Jd#^u>T za#DCx8Ofdsq;fS7ww5%MIcnU^ojUJgEnmF&(?AfMY|I~v)4%~*8TRRZ){3Kk=I)*) z(zQav@{lSjWKlZ20PBGg61V>LE8kxyKY6^oqJ`LkD}QE$=lNJ2>RN8}6f&Q^O6a_^ z#oxcGfcXXpdvE}Mwo8%#76kVZt)~T9(|vGxtCE}`t7q*SZ2P{LJTa*V`vGCU+k$Me zB&X0pi2vUL078&LnW8v6en{E2E`N3!nm(k%q7Gk} zcN!EylhEC`?s`zfab)I<0g%yw&3Chdi!72A$YUkRPCfr@alx_yb$sdhG~&QPeZ;+^J*Z6V)!vn->dDl(2Qb-}J88dHrgtl@dA%TfI zXbZ&BUGF*^Ek8ZU2W~5roWe9>qv}teNZ!-~7a~p*EzG2T=`j#58S*Ol%yqi1_o)6E zgQx^}i+XvaU3nT3yq z`iVqp4H~WJ(Z2QfNQF5#M&*^{y7Q!f7ot8d$htf31Tp7pWvJY;Tq}8e3V@kTrU?L) zyDwg))x)j42@Tm_w??1}zjxB)mfUQ*9nDNOnYtxn>=y#f>s&CLzYgY?B8>*k;rMC= zZ?GOZ%hQ*7Xv2Vd(zKv3sDR?)N&zS=3mmKzZ-8Sxm%H*%VFk2UQ>~QhH1Jk3{_URN z@DC;G-pr zt-WeEi%TAFpK^_`fP5Q?Z~S8d0CJaHJR?ck3Op%KpU`~V$C^jhL^szs=D3}2<;F&J zjjR4PONj#Xq05&h@{X%|_0ARs8`@lBjTSxU9eh@FvPXbOUpf29qbWvu)a=yh8k}|F zV)y{$bu3W~TG?6rh>6)1#D(&_7KSeb;(US7nGy-~x2(fiJ%b`%aFLjBWaPd_4hlOf zpi(P|Txt5SDmDiI$Ng0fAP`Yjx~mTKvv&%+@L<%aR4sqcebsa|fCcj_t8ck28h%F) zzQI1;(VwhlH~3X26{df^A3uJ)?M)=S2vGahrh8HFT#|8!Ht>H`t6MyeGNY|WEpur2 zz|Xr>1Z?-%2D1S!5^nj-ET|rTzWO-k$*{@>`w0b+CIy#XQ5@HeO_lXD&+IeAEN@*- zuV@5>0`Sol#0Y`g9>wFvtYs)jNS679h}KGS5d{Z65E1$1t5o@IXVLM+w?Nl5C-^%d z;UiNp2WZ_ub|&t8cz?7$DO&z@is1XWw$#V#XAT7@n4LABBvA>UQ+GPkIS)=No7SBG0cuR%y*oAc z*HR+moj%5IxrRF~1m=9YMB)vm!HIXWQ3*MQulIDG4RWK&bCV-e0*QAd4)DQp%}^2Kd9oa*Vi``r@?FvX6_F23=C`+j0oOJQETo4^p@TZqZ*e51^E!1#I_Uc zED&AN{gzcy7*FqFX0m{q-$um|D-io~KuDH~X*Cul1P^49Y%PT}Y|N;9l{viasewDW z@V#2^g$<91&u8ZLbsBxG#h9G=lQ*8a9vugA0eq{e?yOEhF>KX0+Kj<*cbLSL zrp(;klihZGaK?2P&oQHMFPa^9;Ap#>ogI4FFw*Ylw6Kp)c>{V;AFU!ceGSJLSYF>D zFfv8+Tiappx~zr@wNf-Z@cz>go)2i0i!jjy9otdj$B?R{`Gqg6J%NgRJpKE8j~GS*m_hiks~ zpL9P7o`YOL_%LHyjDVh4+bi~M(O8)GbWyVtO0mkQ^O#cMeBqvjQ_^Uw z31P0vT(fj%%yUb`NCXZ*HCyUP9oKn3L$;?9)P8LFI?RNCEErrS5vXGnR9LG8lOxLD zO-qsf*6RtI=k@#tefL7TSaM}ruFaPh@G#fu4zNFqE%jgyFeBv5XxL@TO(WKlVPRI4 znKd6CRM0nH)Cj9zYHR9@@J|k|2XM_y1RYcz*$4zGmX%lH`k3Aop94AE(9-ZWB}B8| zy;C+^B1knoD72?YxThkgsd)gWU>ONaqg#0iT8)TWixTm)1!S+OseCEwi3gyxIgs2- z@IL)9`Cm!3Jn=3kr0JH67SxC!xd%yu4GNzWou7u%J&lT&GUCSzrbu&+I&X|^&IRDY z_v5_lp343Q157_so%g#fua2n4*!A}IxArC+^fC7l^|&tuZ3Vf9V8|{GjJE%2GE4B_ ztx~34qz*u#>#vSylXs_nasbTF3GK?U->0OnTrR`<-vX}Qzh1*??J0skP4mXSV+DA7 zI;`kx&aY;ZF(x@aIC zpfpme-1C%Y=^?S&l`?nJN^h{O<1J(0`Z=KSXCLXlC=w5$5b!o|zt6FAMiRPj%r zi6h)=QYaDb)$L?y=YPzdIC_PJzf$OKtI|Fy`~3C5vfyU-GD2?_G4OK!E&qH?)&Yo0 z>8+?zG73uT_c!WhDCqt%_M-f}^XBSkD?=Z72^+K0JJ4ZO4ZN`>8?3>L1F5s7xL&Sl zZ!9AlFmz3s9i1%crI}}U72jOD9&HFTex&hU!Tn|C@)n!sd9HobB6bt)wlj@No{*S= zp2p#g?cHfGT6HP6XQmn#DedN6Els=FATF+NK8%#6Q(ZhiZsvDU>qN=Z=uh|B!F%Bg zY26P-CY$!c(cP@!+FZSh?`c|(Ek$UPlM%VthvP zgQ6EjBQ~=e^xIn#&1X&M%*hoSufy<*q?MKMkdXVdeH9sfLR7f&l%bsY?#hvGG-*PQ z=@-;Dt2y_OZWN>x4q;WJhP~<{wdq4>8aGLp> zD{+Gi6eN6bdAcVZBUm6Ow)j?iQznZ`>_Dt=XUAx8C5m-9()*$s0O>7|t8N_Hv*BV! zJua=6{);y?I-!73LFt_fjWh?ut1xMY8gs<&2_JIZ6amN7q_4l&$EfuUVlGG3NSt5h zk*6*f<6=KQm{Lg$?q89m+6|B%*xP{1C9V>EpolN2D7+6|@1#dO}MdU61={q%2y_hp;( zcm^sxe_JyTWcZ2v9hbJFpWx0ZNpivYMj;6?R5(5vKhGM^1vo*X1dnpKo+i{2>sJ)n zqzUFWt-W?>#Z%PSH@smL^B4>IlAL?P<6>52&$ctgpe4om&iQBi%l$634ZCjBq?m84 zxS!{Gc)RB2C&&YGvZ6>0NOQ(kdBmzO5Kns*LHh zzveHiCN1pU{Q&?%v}+PBbTl*sw6-}kvK@?1qldrY-t#BOA0F=J-e9{qD-!6>1KG*7 z(b2$XGD}46GQ3Q;U+G2v<6k-CvL{AdMLH&O+0!|&C z{QW!LwN%;X8?7B6NtO>KgB5Yp+S0stwzhbS6jAe0nko5u$W4m7T+JE3j4}(J1z-MI zBGE|4LUy!?0zY7MwsPf+;)@Q(0*hTpzvQB7pTzsD#7R&x23D{<_Iz#S| zQ4hd5!C=)fJi@T;D=(&rT~}!*L$uhsH%tKOrGXa@r~f{Yxc_Q`*Muk$b^gvnS#sfAgzs2c?h5?+DKQ&9@K%rOX*f>PS%lkpqU_`mMT#p@@QN%q zpQ7m7fVbE)G9KWaJKQ>x`%J(4G#sp7r06vqY$+7GJ1wmj!rY;ynCNpx zX}RFshuY-Txtx|yg6(;n+<2iD$0`PC2;PH%yG5}%MEGhQJeFvNgkXMYZUEEj_rYbu zVslqjTN}*X+q!Yr`_8U>npbf)C*Jd(%~aIMTOHS#(fWCMH-Z9X`UhcWXDFPp$(z0O zB#`dZY@yeY=H&)FpdyOzI!6Gh2pIik$LALBZyJRcc6#lehMZlmSm~KGa_Jf&Ql0M- zXLti}yUG*dQQZRRP{gx%xId}OECDCOs*Su(4A4X9{N_>BY{g&Mk07^UO+req=*wNo zv31@bJ*EdCfD9u7-N1Ap9rq+dS>nR8h$>W7@R)o2#%X975+YaT6$} zPH{a`mR$w9;G8-3UA#0q@>#sNx4mxXdW9J)77~S4S`bhES&Pf6FxL{Ed*F+&mOOsCHQqXnXmv9LG!ApXn&zR>=s9i24?XDN9aL`k z>>)=Z5ZlbC>8s}Vg%QX@0>i<~+%I#!B{r`&|Nf&VAx6ZJ1#%VKQ$kAvjZAzwXVm=B zP|I?Q8-h$PeX{nEaW+Ap^gDWza|0#jKLgcvdW%i&*?8 zIO93t64L3PSeCz9z8|0On;yVxP`uv7woU7!;spNPj$?t()|@{6-Zk0RcHxn=p6}IN zJfn44YWamQ2vV+I&Z4UZ^^h5qZ7GRmh#pMOCV`6ONaL!Y4Le!Cv`S8|7M)tnW^=FK zd!h%cl?GvZE0R7kvoA05yQrKLtcZbJE2&~C)?%XYhvby?3==U>=t0#6Mqwfn3S`JI znOrOl8EZ^-Q}1WHK6@OcqBQe5meJRD6tU^B%pEe!wn{9nZdS$SL`g-fLP8>)%ptDp zlurr7W&+?3DG;-x8S{MdZ`8p%cA|uSRO%yfIxYD3^F1Z6OufTQihODTC{_aBGUw=6 zWWt}$OTy;jP(SYja0F{M?#mx7-0bhn6!{FC?|!Y}P_MB*OR$q)I1k2)|A;ZN^Zmj2 z$e3PPrj`^>rcB~FH4R=7y8^RJIU{H+R4QVZHGh+XJ;!SdDiq>GnY1fi5sH}!>*)Hk zwWSfNHQ|ux51JN7at40Zp1UL5_EP!5UuzvpAS$;cuDA8aUx{`Lk53)Y%e%6zOtb5| zAOxEvi%2=S`1&ay$-MAU_dXTN6-&jN@Za~W{j-0gdv(DWUT|{R!Be$-9{y8|UKtFe zO7+r{KnT0^qs9`7UrNfyw3O8+M9SX(Z>p5s?u~3PIM9J@?`7L*%!tyUS_m<8_TzAc z{KkP!$#dWTh!$CaD4CQ`IUoP(WAiA1bfhHCM{W5r@6_6z$W+Tma7W(W*GYJnY0ow- z^;=y?HZR8bb%uLwd!)6UQgzYmvLGGlhuTQrzJ-2CjyJ#2Je32acyeXgA>?(69h`f7 zIsHEa!0@pLjo&9jGCsWXT*lK^2HdyMe<}tw=+3&N$8cFAJkxCM`0xPF(KEFAAD!+a zT;c1}RdkY73GVd?BH3029LJGQ_I#k>HV-ME(&v@-4)vmI*om<)--loJ^vdjayIa@W zzX=M*CJH@2a+VsFVwA}w>Ay@RcjB?w_`k+cmGzNk@aaiF$UgD~>$(3^z-x~E8tNDysFz@$pC2UYs(uH6xZ#)#tQG30K?4NtTCVSiJB&|e-th+r8zPEVDH3At z$RSO=ILY{M?`7pUzN7{RXK=}*$??#4kV^bce^>3ASVQ4rAjYkPOf!`G)-fj!$EVhu zGOQqM1xOGQImXWd#?!!^VtrrbdoM>nKEaa*WiN8tD`0@6w21f38`Ab+c}p{MSFO+x z`T8$hkSQTXi!h{QOjI2Hi^1AFJRdQ#eijWKQQ!d1y`u0@aX79Va{UK|tpNKiz z;?OVj5dA>zR48HkU(22^Tz56Tfr+RkonuBu=VOJqmEKI7^U$aw`-W&Bb4?V-pxxhr zUi8XrKCg#~_sU1sGY;NL06}yFocyY{d8ZLR?~=@?3vQp^?|1Iwzpu3Gzn>&)V0tcAf~{Yo^*fS4#g4=*z_)jzVbd5nqO(JLS27HDxfICn6lkNQ5})s_Ue_f9vF zBted@U!=@(BIFOC;oo~TAATsfHV#BN5DLoh-+Gk z&|V3M@f0z-O^~ElhTVlr_G*xFy~U!N@NV;x|7z-z?B5CtXZlG?-L=fXtI<*qO7;ne zi{gW5ZVY1ai$H7u;gy=+4>#BJM?(7#z_&cn z)VW=fp_ljNGfgf9lKP}iYBU1j-SEkmLQxKX1GVOF^Rrg!Jx{wny)3SVAAEh55Ph?R zPJJBm*eU7@VCPw76|Yln99X~f6%SFD;(dLF#?wFj>ej2f@YVG#D-19CSONGZ0f+Nj zs}+DuEt{Eh=@snXK)m_{p7`8zi=)Eg!5&bDi(Z69-ntJY^xkFy5@+!QsJ{j53a)7$ z2rBV-{D19zc{o(<|9_-XDx?qwPb#JCAtp>4$&wxUpluKfmjD{l0&Euh0B-&Y5%W`+dLO>;1mZx!-a;=_#$* zJTRDaUQo03_eL0_0==b~jR}36!bR+J+$UDM8ug`vOns2w`S9Wfk?=CNJj z5KZ0p%;jX#A11e6X~3BO2I8syqk@e4BLruwlo~cO;|?e{hW3(lrz05)6hU1LBdws2 zju&op73uBi3DFq*x=QMgTthiH?NBk62|oSWI@aaU9fL)Yw!l-MNr&MM%Sni+KK73y zhM3)GPj$~=*W@X)T^$%c#vl-K)xDYtp2dobXICM|cG%S~o zDxT+b+tgl2YJw0_443%byeY+J>PhB{dq`)B{R244SvbP%je9a6p#I42x< zK=&$#==>hE>+GD6z{iXd5H-H*f8KBW!B?3Wn5)Du#sk|N53i(N3 z#}~IH9rOEJ{UDNkz-I{!6RHGey3q&qJwdT6gMmVGE4CI;q0GwiM<;;KKHxlav{XWb z-mxuIQ!J~@E8H&ZsCn0*jKwpoP9f}RSTZ`RcH*1iT}p{L9>g{#_7$T z=DKuZT1vs@pWq8_?iD^jfxIeLqTSZkYTCY_t3uNwJ2$t2|E*Nkm#2!qaF|d7drBvg ztKZN^lI#gsR5aZqIVXP=6y}N)J4f8wq9=I?bmf4%w3m$z&~KH-s*m|XLa@n$_Ge(9 zPsOg#>zff884S~rRA~9l{W+*>#%Fv7v+oBB5L2M5&VWbUf^^>fR;cZs`qE}w5Q=Yv z0W3b*EGhKBCAbp|v%A!x?&IhQa9zH1Mq{(|FhHyaR8Pv>I3S7GMEX~$F=q7SPqsrV zR?O%{B$vt#`n4=*OKs1t$S2L$!F6|!FEO%oazNLGU~6-OwPO}Rn@Inv)CQ_{ZQmL$ zA~_MjLM{;t2kHYLFa%}AtjXChp`;XWeRQl^`u0zWy9T;iQm@$taeSRNMe+@|()Smh zktQihljaWu`tm(ezELf0E97sM~xWLLf*c!K*#UYwpIhc){FX_b0`E1#*nBdt;6iPIgFBIo9*iS&u|>GW~; ziGb{T@6yQqzSEqOc|q;ypuxO%A$DR6?^97h+ZY&`nArp$rv(j4HJ92Rf=-{G z62zV^_A>1Mli|I!ItwLtH=7_LXj|~)4kmOr`pS0bG+6@tXTF1oWDHTK{AdIrcnw@k z^lTM@+B~6;a4~S2Dkx873~6}D*>A}KHNwEy&B82}FaBib0dR^i2me*?!IUsDq5C%} z0T&a&#dGA>dJtjS*?`)<9y@P!(7C8+dEIe(BTRW*Y{*-tNA?|s2&)E!-_0ym zu^oB}x_Aye;N5vS13FDc0>VlPiE}UaFpC)(-igxO9JrOZAhxmc63-RrG!#6}aw>Y{ z02|V0*JdLOjNagZ`OL5z>=3mjfG{_Qv|-UhY{;U58UDD-!-5mJi3c!N319pPY z>A~ZG1({~C>Ij!{*d}gF%->D|YV*YmFGJK01Hu}z5?&VTB3!!Ls(l^ld_)NPf{`vA zYeUdkpaEEL^Zglg&|n*U6E|kIk2-)_8{>k%AZqKuNM0JjMV;RQm{UWSKX0OkSOO!R zRR24e&Vv66%>RYJY>A&NGqYH)wsrH_>j26}B54o)*kZsBcKU!DDtLa$%fJ5iIx2|m zV~=~rJed$Q=th|&#X$Vp5=(G^^#Tn$*i30LsT>i+^6gFu+mg1en__^bNsB)oVIKqH z=Oesz`F|gU{~8&Dla(;gmF%NDmo^psAJF`+4O_Ye-GG1)oxJ+`{O>St-SC@f`3E2V zzjritWF{qlv@Mi{i};B{{7PLUtfBgi6LEH^E7cTBy~BCCjhQYN62PDjW37zQzvLwN z^^QbZj@FGSTD@ssl8M`hE@$%`zPK+o@y=XQn{WR{(?b7p`Kd5jJ0|*=xPhT?L*`|x z{{jus3NU%ykTwfpBEbMocAt2yx&;bA&1?Z&YmF;Cve|+P!bKjG=YWj+4?46n5t*zj zjnd7S{L#d#fPBrgYZYfJX1+bjZk{KmC8^=oPs!;i^WI9o(>K(oqpWpJc|*f9D$dDo zxW(amqH>^;BDH2Y!z=Ne>&4ZMIFpiAyn=>|Uis%P?GKPAGGhkgX1l3$b*n?bNN+~= z3;q7h!fnL@Ytt|Brh)6F9$g6n8abuo_z4@@RKDqzq>XifMgNaYEfU$ZmJd-FmB&~3EUBvuTw5{(GHQCHk(%YE8! z_yVWAQ6f^i=$!Zq|TXr^&oXMEfyQ4K`G zC}L~ zXWnh#HduLkt^Zm7dS`Ej!{s;&;dyBrllXp@20}=~g#l#(|MrPjRt4D^_2;g?NKItFK6K?IeuBptx6!Ft7FD+xxCW!7 zdoe@}#hC`$HncB{_#Y`>AHSBQl|*f?D=tW@OndwFFwQYn1!`UhxaG+B_8prM16?FO zULN!pzzQiY%$VxwWaAbiRChXcN`NswmsH5>ZAG5S@Aa4-@%IUhBaruq?Gk&%3JFgY z6EKg$$q)I1e-r0vM#9?x<@M1z-<{zJHmn>-CmY&CS0D2BXPq3&?sdO;r5xOF;W#U8 zu)h)w{!N=2rHic^5K*;sjLjeKSb|5;t4Zq(-NwjIZ?_DgACp3w#lYe#R;Sn|Qbuao zJ~7L`hgbKHjHYY)TI458g*2S^66ZpW%sbibhK3Yw3WPks5+$g84|vUoQg%n_0~LI? zJvc{k;ffE9;%__D_*rdz%DGIY;>*jfMZVyYXM<8+qER#;%g=brLgeC3Vf)U{#$SLy z%i;v#MCm$zVwE0<$%=oBINB;$AVCqWG+#nXwVxb4&US$h;mMWdW{OFY;uHq?`w ze3sUkMg&AR9 z<7P#RZ@|NNUB^SApo@X=0}!qU^K*8L93Wnd|J>#yvDt8P7-Eu%`Xq{EGAM)+-S>a04h`jYqIsR#dMWoC4q0RQ-Aaa^+Bp5Iq*8=#u9$Q3g zGN@`BBW?9^TKW^T9O*lyjrM&=d@l^SKhGd?&@Je~cV;^HW55S47_wpUjFJgsz=~u~ z`)te(jW>>Sv64bY#9rijwMle6Y~5kgyn*fc;^t3E<r<8kYN)_T4kNGVaAQ$Q>HOVsJvkC~*oScPs`3gt|77L& zfCbB>D}gIBDRD}^lVhz5FT@AH-tcpmz~;aHuQUD=&jbbzn8~7eDV$oJudLRee$OxsNIq4kZt7*}GtzC7;VFe%9bS{uJQPI< zq|mA!w5_mkrMs)<;)gR#-Q5?}XTDRHSp#V|n~hu5a}t*50eUbSh!t3wk=uird1;f> zT^VqgqTA7gbKVyhn{Aqv3Ug2R6^l0z2Iqh|I8m{ZRqX6H8&NwzeiG-J6<}&Y7N3%g zP0rt!tL?|fp4x&_jye zz$#wIsAs;CwJ$e!=Pf^d1@E~_i(%i0X~3?nC8G3_Xuxt|$|x(B7Mo}NCmUmQI%vv3 zpE^((H`k;p*J_d$?m{=0ZV}enm|b6+%)Ss!*-jq*rMB*A7g zT2z5%BH>1ZTr|SC{d23)5DHC^EtIlvJZ8ba4pjfggjQzQ5 zjl8(LyS@6nkYI#Njiria_&Yx}nY{VWpZTRbHKRz4Lk6!k#m{+vxJi*K+U06a7s(dY zs)y=1yMiV6oVoacHeXTa6Cjx_haca|ANPJ(pZb-WZTd82Wl`&er16*ll?snb94j7} z&9=AylaFr316~!%H*=dZp?}Qkeg<;)fl1u2Il{zvm!+L}$D-gHZDSyAd+msxg4~z4 zfwHzP-Ft=Ms_LC#^~=1ohnG2!`%UzdHqFQns}4AV2Y#3jmkTernascYu*3ci-Z_t< zvw_rV{MQ-6pli!up8goO&dY!meW!G%Zuxx``LK`%PTU**giW-zVi+r6I2+%ch&?0E zm=i!C7;(A}OpmJ^M#|p3_I}uFEz2S8*syRkP>B?n4kka~L6*dpBH-M9KI5wq}br5p!bT!$R^gMstHh~_sOXqbL zCpWwfQ9`Bv8k|cNFOyW9pUk%_o6&XYuGeqB_+x7R1G->GXrlj8<-_swgW!GrlOZ2i zxWqBiX{Pa&tHZnQ`(y%%-4bLI((<*+?S@hN;DQQKdc(==cq+;Qqt2HqPDsx`G5JuTcjbGqs&WupA@hfGhym$ zd@6vmYpqT=u@6i8^i0mta9%2;Z7TaZru4?YkxB*LTRkIs>9>3>;V1g-%~!&168(QFe!^3GIVOqYJO$y=xzLiF2qIfv?s{%RYCdfg}?O6EmAtf z16LY!)?@B&VQuCN8uTSorlIa6ya6nqGTsWux`27qJ+99~5C_1mwO~YHq6zmQ!9ifa zO&TmI224JHO%=R91mnLsHjw80Z;tK1N!kA_#Qja#UwAIQW|y5DE-Ti1quen z73Omk?`Cyn+Dfw=PeD=9%WFdg_vmKtAc)~kzq5_iNM1?({zR2uEytHcPX6h2Cv&T& ziU$(Gd!E7RhjRR<9Ed4VN~Lw8ZoYJ*t6c;<;^eBGzxs}VrTnB+Zts=XToor2K|juK zL+ZAIoNvN2cLml3H@*og&xQ@JUro6T8A|PB@Nm^@JE?yLY><@=Rs;DGEuTUW>`1D|JNV1mD}$5-h;{I>(R36|5b$NwE)fRRUg0FUyw{^$809_5GEpbzzg zHdvZ4Yd!PttTq^7)!$$iEnHEZ7`5fFei9jk2nLRbF%0wR_qKkZa@-h1mg9$({Qkp# zdq4)9^X3`NEl&MpJaB;Hph>lyccZ^WBU>H*zp+lV__%3_+p(-nq}nc4{*PI9Js0T> zs~k_p@g7cPp@%wn#?`X$uoOEcLm;7!=B=tLAH%}b$LHqUJU@MSFw(ZLuy8`0Vn`p8 zAPd8WF{gD4J>kopA4GL(tP;BxRMi4T@aj{%HcO53S22RPW6AOSVfauE?Y+98M#R0aHw)H|81nURqJN{X!%blQ^J+jSM(zH2jNbzP#fS|S?s zAt9VZU$@HJv4FDfx<=lukq4yjX$HI|UsPONpR8+QV4T$yo0xc2T3XufmXE?zZ?QA+ z#f=*`+SaQrx-tY5-R?AuKCrzXwV5Qz-7`kP*OR|O)I5F^L=}5>!FDGDV;Pw8O~<~Q zWI26W3^P3ZB9WE-m35}^;gpjgQ>o+vHc2WdEh=@5C@JYK2)u=HJeUOqcfpT1S_U+c zYKru*&V#*Wz*`hZ-Wf>p^gJGpmfW#Zs(l```p_kB&xx*1kVJBD&=(Hr5;{55EQT!C zO>i@#QP04AB|x3Brz1&5WT^`c%;K<3xVZq_VlP_g=@;%a07ZOhwq{V335bD~uLd+O zM>M)xTg!{)fGxDwFh!2B9B~shW1r3*$&8On$ih63i&_Z{wO_r%2e^MlvQ#P(>^!4K z8HxObwDcQLdJO!u1_9-@(vq0qWVuLYJbMm>0kTAm@5WjmNae8>C71nc=ctAt>R~ z;X8?}ERW|zzf6yU#l~gHei`?yIXDx&r;>D~J~g$$ZhUTdLMne3D`xwy>`46S=$k~y zP6=)YknOzOLXWJu%K&7PHBKOYgw(eBFV2)>T%2fGj5o?5;9j`o1t}For;vD@keIl* zcv@}k;-Fhu>yt%=lf6681WjK|T3W_Fe?B@Z%+i3GB-G2(O~Ta!EdwufMQUWd7f<;4w%)OfWn$h7rgnU>j();5&b>!;kw(TDDjHsK z)y6q+gAG50koxh1udIuOz?`+Y&gYqAUY<9y+~Z_cb~c5FKSe06>tSe;xVWj7i!{VO2sYmTI&lsR44$|^r1GoQRNN=P#yd$`+N|h7ue{-livZVl5?L6j7g##-EdF~Ux)vB9KeTt{!nlI*n34``O5 z<}#b03{vnq(Akx)D=bilX3GFW5^pIZFu?XeKlpJhga|hu8EYSV5!f`hWQlQp96V$> zc7G+GgRz)#f8KU)oP&VE8U4^`Qe;PAp~vjk)CpC%-Mm-xH+o(jtj$xi!P>k$WYPDE z0i)JeFARE;cnItRTdJf#rsS1op9PJ|ZCp~>Zg_5HcuXBL;jG0J!B)N4GQj+gtjx>@@sf&) zn7439Ns~1gHWH}WjdJ=K&>-W3s9J$?eY`_NCT65Mz;jy<8P@$mBr{XC`4Pz>5$+|O z{?6pCl!`3BydTy$p2gdHC z^6_w8F!yAUnvQ$cb@rnrk+t*Vdf+%e=Kf>dD+yvbMU_ z^^yw@gFVjjv4aZ6_sduCO0Uk^QmWAQomqEFSNI^thTlL07I)|b%JrZcJ0O#N7aBPF zgYTPtfa@T|HYs6r^QK0_DYCiAD7(#UglQw`9vb| zWOdtAq)*nT0H7W^9lH+8^AJ193C(N%l_gEq;cOAIXCa^k#err}e8wnqKKg}Fo~sO) z^>K>VJoPLZSVRyqVRRz`GCvgFisl-QiH%K{Eqde$iz8p^{DNFRC!2A2>7mORh1Ro0 zEMh~_AMOY1r@dNsGIdw*#*wOrsO&fKrr+$J1%Um9SKR)3C!=y7kGViM@gIR}L_ej9 zMWL>rf>76jP1J*o1FGw$M&1pWmv$ z!p2~Tn)^hP4_L(;aPMWf#XTEGEJvN-c^eoKKXNu#O{-W4^Qg|fhW(Y}Ip< z<4TJ?iMdYJnWp1((_ao0mb}?1CGSBObF;m^4!ZV@-4-FRX-Q6DK5Z z$>ZnP!jW=i=pp&EsOas<1N?!V;>?6)nJy{7=>jl*RI<&hMby-H=PI28f-U>$5)X4O z1(_H@rsE>m@PUG??5tE~#yHg+PfZJ-J1`6U!#eI2Zqugxzc(FC>o*=Jh)FSnx)ts75(f-=@}LO!B2VO~V7TQTuP-e@?;#t5y}4 zyB#xrG(Mb>5tof4dWyw`(OB~y5js3&mi5YlmK7`_J^`225w)g?a=!oii?(w)(VN7q zB-CE`NsE(4uxA49@nFkAc8=eyj3yq zh?|4$N%+GIH$FhXEwqmUEAjWrsBF_C?L7VDu7d(c?@VTDyOZkg?qb!2KVODVBDl?0lc{;#7v6y}jw7$xb33wcsWH2!O!kbvhYmloepkx#5s=F|c^3tKkt) z{EsRS5e*~O?QD!pc?VA%!G>$vSrFq*7%bLnkISMTd7d5`<7DhiIRvL2ILO-!+|OHu zoRV=(#fY+e``rECs4*KIhEePf@mFfkdgYmGYB=ETxNh{{4e#Nz+T)ANkbmm_c~GjM zST_a8d&>b}WqbhlMZLc$9tX>khajypT$}TCVPv|`3H1)kvOH)uADx((-I*6|*SPqm zDk7nv%+V|V9rpQ4bQfxPnh$31AXd`Eqeb2$Rh16(eGUtuc7{T&G)no>#ip!*n2B(UuA6Ovv2b z9p3{3PC(j=pP((PVuXLwV4@m+IwFbkw`gyL#OCV42NnPaVFC342X6v)0-?sZ3g0kR zqhu5TH11a=f`BQ)&gZ(!#d047RHpS|xHSsCdb^q<$@zXDy9s__zR6SCa(ObEhhH`4 zNf>V#Ni;{7Y~LBjB-0GTtP{*+o5B{QhYT0-i22;1f}VMYjcq?HeLIFMh6Nk}8S@MuP| zlcy)qFml#raRg4V8)R3)Gk^GQAboFAcB)ifAy2!v z^8Wen3VpDH%<*6UY&dnw&iWW*Wk8?q6$lY8~y6pr&2j%*&*ixVFSq{*)^czm;X zW7>CGQwmS0JUkAwv|ChlCcPAx5GJo<$++I09M75>v207}nb z<0x$SW2pJXYa;)u=#OMXyPtSvTb(dGrx`Qm%tH--G=GPm;f1G?y6l z0$28IqdCVb>ok%?*E|$7v3;%pGc7%7z6DfSAQ}l}w&j6LEuiFaNj)k-h&87B`^79h z%w=TYUQ`p~DUl-n8O^lM8w!k-CUw=-7Bcmckxftj(wA%NPO{*dIU5_c+QndT&xa6U zAIo4fsbrF5RQ#&{eEEDLVLq03TgTMH>xsE)*NQEPJ+Zr%oMSi#D!b}AAkFdvn$^iO zL8%YSQG^V>0`PdNaGD6^HndHi&yVQjJ)CB?x^Y2@)1Sy^;a7q4Yuw%tts+c zroD8Vk{(2z{!^ySG;kER2SB1xM33OQov~>`2(vb((w%7b{VH#`b&86O7^N8-qWLSH zr3{D-s;i@Q95EbqtwIknBs|<(N<4Y)knWH0#4MI9Hp@n2o^uhb9jkX5?RaksyM_PQ zTGHK}7}d?E*yG)t@cHZm$h_mllW}DVL+1nKsLKQxAm*JuT`){4e5WBDBkghABNpeN z?q!1Q0ozmsG;8S70zEyxR(gD!`2s6Wk*}BZ;?F@bJMYA}yXSWl%%3gQ`~$~x6;qIx zSDw3uTKCqVM79bMQutwq$v5#nFBjR#9(b*=4Gj&vag3fU>L`mLK1ejVlYmlJG`9~#P+;!X;L0~aDCilU;kVxo zsg=mPW7zN+F~)|QqqzgEdZ2eHnQ#~YVo zVkv#$qZ1h^i(;dACio;YM6|{LSJPRO8HM#a6l?P6$;HTRayqzZr8%ew2uQ)YEO(G}IrBV#QIK4Z=w>@kpUvd%4hins#igObg`Dy(Qn)T5alxTg~qY3cGb>55f zs)Ia2S~eJ)w)XbQ>z#FlkU~Jy2-2#Q=xW(s6BSL!v~xqcSSDF%A_iWN4ubfUGZog|op>%GsUFL?EwOvX9bkQ8MA8+!m#9gJ!fA(qZ@eZKB; zSO;tQWc0=Gi-QbiIVn9>!Wix5MqOU&0^H|z2r)j}I7iRmn)zoZ|16;Hz`p8{^<|@c z<`bvt53aSuY^zLongmRm*&OhQ&-8V!(bH{40KtV89Cs_yYzn@@C8ZVm6v&IM&XtqU~aBd90rmdckxX_^Dmix|DsvEad+HgEOgW literal 0 HcmV?d00001 diff --git a/docs/zh/26-tdinternal/normal_log2.png b/docs/zh/26-tdinternal/normal_log2.png new file mode 100644 index 0000000000000000000000000000000000000000..12c035e7d69fe408449c430c2499bdfd13537fce GIT binary patch literal 65991 zcmZ_01zeP0&^C-KNQb0~z=Cw6QqrY#cY}n|A)QNicY}md#78di#YWmk21g0m5Oc- zBq%@Ix>5xJhyH>v-;)Rd8W1nMLmuFlOOccnJNrAkaU}_^t!Hasri)Pd#?>83ioowug!A9yx zz$h+uo%=}C6Qo0|!upkSX<%@V#|5?NWN%Jm^lUp>4!Yxa|4~o)3ewgiM#8}z--laO z023?4@wrezY>;Ia>Ed-`wHKzQXzZc$uj*qiCPqgC{2tvD#vtWM-S0Ev1Be@XSX+>S zbH8p+Y0nL!GkD}$8Bb99m1#mR+Vu2vBZo6lYO1zNRTXPiNs096*m#h3lQZ7Q@ccaE z>5e(;>C7Z^#)li>C%x}sy8$lnBjB#sp1qV0<07ioqrppxWjHagH6TUo4_nIKE*csl z8X67I4AMQ)YT}|J*)Ns5&2E zj`P-p5&S3E8LcW!rcCy`^RIM+7*K_|WqP-_4(TuI>oGVM#rwFgHTp{C<|*8q&+u<9 zPw;k$i1dg2%g@1k0;?ZM{~C%0Anr;-3J#2W+k(6WbRiMVI79j93J_qWN2q=%*0iRJ zOY$dEOM)inv13tp$m+AQZXw7wm4(vujkv-%9+xv`iw0`9tJ9}t<3dwarZs1Gq6MT9 zwtbAy(z0{!XT5@djYhKm~Ov0QL3x4d&NO>}p=3gUIcP(7ny|5z>ju=Y@xp{*3y z4#G4f>joPwB-wV8N00*tN$huj27~ItuKApbRVk|-G7AR2L)o|JIM)J%h1+Vj;pu6H zeFU|c9~>A0p!6iyuhrG#68&xeq+|`KAXp-7BJy6@wZfhosS<@<&k2UE8u7if zi#=O})?UIN*-GRP6&k6Xb~ozYFlZ z*$&n+QljIfSrz~+dIvY)CIi2AV|;AfS=luw%J^GJ%UTH6pr#Z0gQ9PMDW+Q9za)K_ z@d6me4n;5k&PfaOk9;;!w=`;~fPie^!rF50?P9dq2l z>aI9{lL9=wYdW`4htVhh^G3NI;Va|HzQ{C2ktEE>>L>u!)JXINK1hxl*p)RGuqVYR zIE3bB$E1D&{NCpX+qJcZLupM2OGO@TEyOHmzc8^4MGpoPdrB=&2A&q(>BTV8$QPrFgnCM>jpw(8XVi(7}`N;6PiYfHUbhUsq?#X##Ty0q+5i zIMnco%dbkTRT}VKf<+Q%nC|{mTTWuk5x&bE@?o(_A(4m@b$*W?hrE(o0JYVZl(%Wh zwA5DFrSnU2FASfA@IAxgB!i-gsTa)3dCXu_M1gFBIw{Ym4zl>V9_dFxRLA>&X3 zF8lN3`zVMC_@xym<2JTcEYW+v=^7z;eF)!lit+If)hRIz!nT9r8yHD7BVRc(K5>TA zwB5f~ZL+Y%>JD0%A6JIJl~H3oBYhi3t2ga)I#ppBKOreqs+~oe4SpU+xcwkfatT0@ zGHKk2Aqztja2ljh>u{v7s@V<{MnnEp`D&?5MgR+wJENy?hcljs4txn zA7Fxr{L0FYa}vl(Rj;Y8R_0ekqkWC9O=HS6Cc+UyK{o4an7bf%nLlqW&$wn;zBH0qg8^EZ&y9}4HTVaSLBDfKn?06h*Aa62wbqXcGqR{&s zunMMsh#&TV-=%^d;^I0b`*81jKVvUC^&Bw{+;mlowJL*@X!Zq)ytWyH*^uO@f0LnG zZ-ZSkW5lI1>udbV-ACBrEK`n{@vEQ+hB4a9&T7j&lC{}^GJ zV%?NBHB}4rgTr7)pmR*fl?VSmq&#W>{MoiPNt%N_R3R)3Cgq=iWruBO)wu*rMDr6P zDf+f@TF`xC@)5)h}uX!Wp3zU!e>`il_nyPpH;v4qz=L1i+#`> ziTVt}m=eR|CHD|v!ZLAT0j{glT{?rVu9jGAw4;6B#g{O7w;BT0eEnwTsL|HR*mmdM zRZGGQn#-LEc-6af$g4D#&5R*{BP>y>kfWUcq+2g!NC48YSEc%OAtG z|BV_5NkEvZe>BC3%<;bUGi2y^{mJIb7ledCaRP2|5iBQIBp4p?iHC|TL#ytnq(#L| zLSI^oYbq3hU)=L8rdXpBC4;Eh{ zeMh+^yT1hBkqCh$^h^9WVr17y!MTs};rAAHO$+#rPO8O})(vEH^WAw3m*_$U*+2Sf zG&LG0OZ?gsFg)X$0QYz~HGMxvAM@3(&A#mJvF(|f@c76Y#YivR{QlZn?JCuf7bIA& zT<_kANJ%ltz{T#>AJjZ6CBKIg0Ybom$~)6X-YC>raw0G`KDf8>vf%p_*zbyl`N7Xp zEc&$iuVRb3FHwD8dITp%hGmS|MII-%&3>cz2$>u}GrL-OMwPL9m6yNCq_v>+^V;<9 zko?PBg#x1;L3^DI_I77P-6+f<+sxDQ43EE}xb9gPhNZv^YZi!RB`)Yr^K~upocHpl z7J1OO5b$ag6>Mg-k}_Ha33Kye0#nSrufKm5MR1NseAlHu6p+2`XHART>Iv<~@n!zFQrYC6+ z3AcmU9RN#T0SifH9ctJ)ed$65$Oa+~oWY)>66~umwY(!Ne3<{e>}7&X9lCUiO7J9*tz;Pp&@GE*l;ey>Jv&gg~>dwpiQ ze(0(;1~2`cyHdd0Dvgpq31?D67cjrZp990wG${v{>cHRQJ%c-#;YYl@H^itJDYHPp zPy_fy5O`I;R@GGDMX4;_jd=-U2OuC$Xd->4&e0!h8zr>fm6aT~l0TQPoL~Oc#u~7l z7v!|{_&v>;hLWof_&JqP>Xv*khik%DZN&rZahI~?Bf2uh&JF8s?C*N#oDYHc3+yx` z9yA6wJ2)c1o`QiH;Pl0x()EagoLkKzl)(Y6HaLALe194up=tPi)X9`2_p60+p ze083tIV`D_QvwTn4Udg?uKQzre4cT7hSbLnpMHi}k%v+nb}Dzn8!-fzfM{kp4q*6G zVK$oci_AaI{Yw?V8HMx+a^=<1JKo6_E-sP2DU5a)WkK;Vo42nhGF966K z={lcasGwAbPi`c5S^Kp{dhxwDTmj8Er)|r9I&2IU47R&Zyln18R5-i$aR@LNIpi3I zeSv=O^W6G^EpN}R=k!D1fP9hqxvg<~lWVT$meO`DtWwy58d}hI&{v7yLCOwgNN-Q_ zjkA;%%32Ygm9*B=J?+M%YpS1qH#99fh=}jg`#bn|i$@d7j75xZn>S{y{-Ew-YZ+sL z9G-ECd-p+|4)}>Bwf>_x{Uevok#}SztMKje&{!D#ZvolHm{j)5l)t}``?Nxm_I*0hf%xMk{Jbl(_4%dwLI%)=v z@rqMClxT1&63F$KQCzWx!Od+-%0>YvLhyQ+7WvlV`9(EA;3E(Xg8ZIjZpA?lMrcdh ztFb{dLF02OzAoyOFO?yAWXdyQC(j>~MMD2{`|9K3q77`dR=S2bHp+C>| zT<;nd^75a@I>8{1iCAGatPg$jzKW=2dzep{7uTRo#7}$EJy_Kf2|oDlT?5*bF*?4V z^G?g&HWxRl01dr30R;{JmKf)8`JSaI9r+n^V376X{(ZaxLyloa!28B&KBuN<^7Z{m z=D&Acgr}rEFmK8WAqJ5MGa|VJ2&oi&8@I61Vu+Res>pSzL4oA(hcHW9!$l)9!QXzd z8ue^WUZfk&N7Xo}BhRS+#Pmb=5}oSR*f>h(PBenK%d$SAW+GQGc34cus~ZYq%m>H@ z;Ls8m62Jh??iF$<1)IIW6>2pIASixpB*_Xnlvr0n_;{__oNk5+qB(Ktsvy&6zZP7O zSP~PHi_piOxgV-4rFdKA5qu9%@ak<2rf;IQF>|YObaN-mG%twFcu+_bb*0OZ`& z?ea^{Z!W9-QIzY3AJbS(W3f;CaWyF)UAf{~XTmC!-V{MYswY zbtrA)s?B^>vplQbJ9@pH^_G>*(L;Pm`pJ{Cj}699Z6@h7=i(x|k4XBOSj1Awo(U4$ z&C0`5Q)|cxe*yTI7_dXT5-#I=+Gz~7->PDc4cOVD0+qw_)((=Xn!0E-c*?G;G#Wg& z(kd=I!gh$Eb^k%H$gcKWLMAB;vFlFmjL^m|5cgK*CbSp51P`W$`$ zZhAW&^9F`C9%26hW+ZZW2)Z)h@2!ZL=virLEXpEqrW^qlc6L>1-w1LNigv-o)m2pl z^ygUKmq>Qowi-SA4$PRBPO;bz-Wv= zBfHfCT`=x4LzK4eQzSax&%21h+^XyO2orj< zRs$l*6|Fqa7@Ph|h&%%aaI!pqnIUx81Rb=tVXz6rGrqS1sk6ODLZC8;*5FrR;Zc;{ zw?qqX{pDD%qUx^`-Y%BrTB(0V2-QqXQPjSi++r1CdKt**VOgJEq9XbKo8gzx;)+v` z)ZZx(;NJmrw9(oI-OmC3B|=bAS;cBqSBy6o_|_SJ^EDmA&T@N9oKr1v`IYn8-rOq- z;v$5n-k4ph_A|dLBt&QUXx{&}VcHAYe+QHJH)7z(6c`tCD%_8ZU?Zm!P6q&PaN{81 zp0`W7TbdRU&|!)g)uY>{Y8yl6m6-;0^6k~nc5y@(!hTOq{(5zRFfX17PjM3PhF4qY zfoYn9$>QwN_Vix#46sq3xNRCRaw3ZeXhoF6Sbiyfcj+F9V8j>V6Qq9 z{qf;<{1jA}^NR3X13stL%>R=3e#lP-3@N-KL+=AkEm!KyF|NbbfrnLH98B;Su!L>HK8@GjSuWE*c1>{1?AxF18}Al z0L+-;Tcr%_rU5cw76>fi(OjfIpuzv zq5WA}6GW(ZbF43Tc;Y6Vhq1bbp-zZi?ec5u!BF#g!*bLV0mAr41Z*UU#j{c~EcV&& zzi>{EZMw6Om!}Jv2&AN>y4S6Cr#2)pcT7fmhl0~f9l>S9bbxr_ajb3{QPW6Gfya~L zo*T`@bkRfB2aSCj^)||Y5-%tK-{$|!S)0D;G>oi915bs?;Cpc)z-O36R4f5Lvufz2}#^M}rL52ctt&GB22J z-G&Ev)^o<4&fC_Sk1R|~aMPc&%pxT0^ijeRPcW%GNf&?I- z?;*i7(u)b!tr}w4`SrALhEi-w9PK(tgeNaA zra&oTIY@8W*@NHYN#ZQLgK8dx{=fO6`SJaAN4p$h!nAnW52}BwQa%UXhTD)b)Gl$^wR=n zQ5EDr;ZQp}H<5VXrB@61`czZ8?gLn;5!{#i}8cZl5Am zc%>+Rq%qcUHS?;8k93m0?&gE94~kT1Xd1#qiW&jFGTM}4g- z46QGA^7=g1wZ2lCw%Tn-T^V8OQO}+-Pej?)pTt33_I@Uexb=BZYCD}&wL6}0Hr3cA ztJ++pE)Of2qJ%!7h~O8|oR~ZH%W@)RB)a4$biSymQ*oU76q&Hx(--%G>LB`x7)yp{ zZh*kX}szj9+emb^9t<;&*)9yiCW_-M)^s{1;%sNfSwDDks$_2 zl8@>Y$AeiXydMvRS&YXW+&F*UT>SYt=370i`{L@&&v)gF$lu^Kd7!oT4p<0WOyGiQ zQLEP2vM9&ma-c_Q8YiZq}kK-CBF>(E&S?{(?USC#N33^(3TOq zLYl)=xseRaJ4R3dro`0OMn0vY2(g^qjOKns^BDrg@eke4J=xYNKK7wKp%hJJ60W_N z&?I%Qv#F{zn^$^|TYKTOT&~2!+krl=rjSQiS@!cMRVa_ZtnCC*AUMv}OJqtxhEGEj zJCi+(u#JTNvY9=P&_a_sNkiTF7#h#t{$=FTi^kPl_uL@;+!LQq=T#$APAkZj^LjXC z-#&`|%wiN>jAm@K9#=8H8uD-!(R>Fj@cLF%8JB{;;GWm&k!#ccjS1Js?8LTct^BLZ z>ynltWAz?kHafz;MH9r?u$=vD!XIWTqLuv2k9IIElj!Q{2 z3`5l1HfQPFpuWRQ+VsoXHKV~3FBz!v{?!!%fpXCqsWgrH(!g3PL~1!!g)N|XrrL@9 z85Sd0xvOp}Vpp)&%M)n{5noO!R7(45V!_ImRD-tY9GnIoM%+Ss&aL{k=56q{r7Cr` zIkXzw5gJciDmA~enJ%OX$(fdcy6|70mvCWME*~Z1bO&>4IZwst3*WSKwq;c;_#z}b9=EI2Mc0u>C$$pV7l)YW&<=Af8<&_FM?#aROk<3YL zpby@{PbE{H?GHDU3hsU^#V*|El-MAO`L`cvUio33!Oz&Bj?1V|3HXehWi*od!=06f zqaRd2Eu62+OJ_d555L-0a&6%_%lvDAY`4zI5v>D>!~u zt=6)`4KxTXDPgqUWmR!JE2Iv#Z0QI#4%J6-`r6#tea@8Vrnl7mBt0|y4W$fLe19`F zw6FJ`r-3;HTA&8v7N8WQ>>fjErD=Pz*8BtOxih(GN!9LA{D~q2^Uer6 zTnRD^_0ux%`HuhLB~L+imBH_;H#Dba{;{0&-?GT}Gfd?ImNXGD-E*jV}Y}c5eI;?X8Ne(;v zGHh=IY@fQPul!CDIOvlVB{%{E%A&j{eH`_FJMYD+Xdh{k8U)g!h(StY&h0zzxVuf~ zp?-+zsm{~F(1Cn0v}kq(wK~q9oUt~jv3BAEV)K&vp|{tUg^?|3ISCmo3SKBXc#XSZ zE5Apwj3@r}0=#hYG}O*OF{V5~A1Gw+`@My87p5BG*btU%h8s<16T$4iZB@x|W=vyb zgPC-iH;bw_o7pGa-S<#I69rGeD>A{ST$R-d2^8r&n)TZ9ZSiC_QlZ*11>H(&Bb+;o z6$%d>(lK1#2oA0~FL7|*BM%_ddGa&DH-opia6V|6c}6@cWR6bGFgY=Fex+uT+8#hy zA)Q}qe%ReN+R~`!C2N0HkrP&4%5Qnp&@TJyeJZ#abAv@D9!~Jpw{k@rNy>x)bL3?kMM5x zf5>WJCfAzoxz&WV=1$e$^oeGhd!#AVy(J`m(^xuDIbh8{e!NVO`}0oVc*~cyw1GtnB6NximP8lm}%x3%xKv-=M;8sb8v!;L2&3UTpIh4x6Cnl4W== z#e(q)T^MO;L!QW9hN^r%0SCLVcX8(P-q1H!)%3r|>ffl;F!YEN^&TrI&)^(^ToM zxw^xMnb0NwII6guxA~k1bJXGlO8w{u3lVBkQbAAskjHl?hpF)BZza|2ysJIm4QA9I z9?4sDX>Y0^)&td zk4^qbujGxW9KOR_8k3z?dDRN6b49rUs1Z|0>sgDD_6|jg=b|47^>r;X-{G$tQWb5a z0C_9)FnL|QZ)vuc7s&$fq@LPv!)j_pycl6lu{z zdc+TEgAXI(89#CjIILiyL**U$^Hh5ElF)*&F=71U6U3ne)(5u*{Lc?G0ra|IQGU(O z&(H2GW7dz$_h60Fh?#`M;-601R2@&~p8z#bS_5oc*~?XHUQi;v}5xy51#HZkZC{YCF{ zIDSwftgu(l@Lf_(bWBNhPUNIWS1mwKKtFEbnIfcz0>Tu;98|Z>at=-i<#LNUd!o@K zcMXGLdzsdnb|aU{1Fk~w}LW6{Ql>NENJkw7gEMvB% z#q*Zw4nMHX>vGoBG8jhSonq6}(s5tf^g{A;dPPZ8vGv|=3>}?XPz0%rpue$pG0p{u zlRh@PvE>L$C};#mBh!V6$5=y_b1Q|x@8f)1{jJ-jrlAcvdouN_!jl|D;tyG{68vgi z_?3D}elEetRvj^GzG~f>xebdr#Z>K=z1ZdKXld%8Jo-`vhI6x{if;v>-9H|COz<%J4u;QEGZ#tCpx|rERGnm7^C*;K@OM9Bg zjsPx@v8y4uzG=3NblloBZn91|-B_N19`g!e$GYs*zuK9IJNawF83jm-v;O?2r|S-l zPAe1?gRV|{meb$Wxa&=%e%Q9)mVHOyJ()-S$Oi-62K%~}q8c^kH~>zOtF2Mc=~o^j zv#o#uWf-MH2`L~rZ^dMnnpkkGjQ4CS6U;WQZjw z7B|T9i8n@PuPcf#diUC`$9(oBRI`)DD$?EbtC=L|Pg!%zMsWpLoh+2-u_Ut(hdzaS z65HuPjU|z65ScZODJLa%Wo8t0m&Ufc1FdzmDqzE9vYieRL9Udz)-4)hMUQ5HT=4!< zL5gwNQ;}a3TBwD#NvL1gD3i!UA$M!4DTGD;b*fS1{1BN;VwZ>wOQru8G*6!ChMJ3bLFp&?gsu|D4=!nS24=wIfnxy-! z19;04xDcj%&itv*Qnu~Fi|zS?n}t-Ncnb;KZ8)pU8y{5c8&ZF6k9l}2CI7I)I~jA% z8K?ZglsvDTveW3e5?bqG!cxq0u6vvF`ohmJqSLvrdDN_P68dcA2UVyP&KsbU>V8`g z&8Gn_d+}aa>)LPo-ICRNz6^mP4{PRcw%#CwBg|HQStzw%SSW|#qx-A#6_Cmh*UV~C zXz886qtzvXsr~`^zZ!hdBR_}}*5k5)s%hf}d-i zPR<&GA84ur^Dijc-oI|D#QPiH|0&?#5RwgqQHh}JkGn6ekExkl=(>1%@WVPDYBm4! z&5uHXt`yoiY*B3%{qwTPA3_DVgnzs2{w?&sb&mty(0DG&ivPc7{^tX%G~jOgMXLW4 z{+|#3??DkDpP_n+CC~q!`JWHK+(0aSsIvVJA^aPvegOLX>?GPBJ+y=TC;R{NfZq}5 zd1Wc-lQB;{n=f12`btsS8X zl(MwM2T~>+8Sd&Jv;QHimlS|BhkaMdVE1oj1+{n>dUIT*Y(hGSSr3nv0kD8n)Pu72 z^w-475@LKX6Y-mh#`g@hm(>&w0TdD*xEugM=Aokb`YNl{x#)h`j5vQ>J%ZV`Y&G^^bE^B^C1eiJv!Y)s;)L7#FU@yDf*Wohr0{O2 z7;u}mFbn!UC+Qxtl>$J0IYIHE4+lyV)S_9ZuE_?ag?ii_>`{7=LKKlwIM*}}w*2^z z>Yekv;zNPZ!y&A@(asbUwrhzOT5G?*l=XQWrw;34yi^DJ%Osn9^i?>w{L3#`A~VNG zm4?_>{(uoVj0e5C*H|91XkP)q@Expoqe~C@de_8z;+|9f<=IX#VqU>uE+hu|i`h-( zr{){Q;jw8=`PG?P>#vh_w#Gk(GX@b!TxIQvsvykfxgh}qgh>4K+*{wil9fMyVEOasaBAI5t_3Z3h`&HDji*pjI0Ng;$2FbtT?s)(M1YeAAiGJ z16f~iZZ0KsAt^bzOD6tK^7^s=XHyu`iJCL(y=2sAu$ihqk~`qL9rpYL z!Mb}j5adR#w3121pkt1?PL>i{IRZEe z*MReJr;Q*>DZU8@30+KWj}<|tV3ynH38FV9*=2#o@-9icfZm0v_QO4{h0LaYr;-aX zoI70S5g=J%4U8UooK8`#yIpG?1YS0Y+}*C<&5(AN&ePfN1@kg$3xL?3Qd2NK35*p; z$0T~E=(>!PUiP{EB`R_Z5Rt+OeeUBE7zgwj*|i|UyVit#wzdr1oR32Yu+okI#{?o9 zM0mr*h{R@+n%n*UV7o{8c~^nNhPE2{i>+V3ZeL+CFa-px@hfe-1=wBdso6r;4If@o zllB`m6p_mmV)svVadBI5Rp!V-|;cJe{3ZNz{!`Dq@PX;&V5p8wS7QrS;T)7 zO?`HKG(;fPPTraGd4R2~6ssjkm5KMtWAqg3qMKWu{<=;i`6~tFC$?wK*B?C7(1n}M z-r=u6j3nFPVF%91nh_1>Q)>cTBP&^w+TynL*}AP?&39-+$IX{&wGCw(U}s<9Gu4e$!o5{%>r%sWleRK#K24%S zu@&^_bq@yh38;>4+IszgpQqM!AJTF-ekQI8k*V-Kp(w(m(6D={hrk-3F|qJ$&JG{> zO*^?xWEMGm1<>g2G#-~c@2(N@jCXd%Sck~Ps}!j)DBNan64O|63xAqet4h5{vugZ` zOPGI$L^a3@^;ALOV{{Fo9s+e=`rWa3_-!^)VAG|y$O4n&2t1({@3ERa;7R6C06pFfSC7j{T> z^|3U* z>;zo_JTrBGYdu9;JxLB;25(sQnF1%b69aOsI5iSx^+4t_erPFWSTLObs zGK7_{kfO!!>-fZsK!`vmK@r!EYBDooSk6r;Q;-&>TTkJ{M!<;hO}>h1>~SZ#0iqOb z!7rj5UW9(|NKQI8XCWYT^Rh z%ac;&7sWH#8gJJP3UHJc>8dDgw4`Qf%ja@r3)KlN=BfuO@ngpwp2h!IQ^7mG?qnZJ z8VL`kZQY6s)w}!#&weTfV#JaqIaY%3-|~bp>m4=T5NF@e+g;PUk9t6-uOOeaZA|3s z=d@2AceOe(1p`6Wyj$*V_4o!9*F{-t??z)mPNB;)-+}XuJvr}ZkF%V1$2O1FQNygE zr+Hj&5R;#%i_@cBN+Dg&GGXLO9o6X3U!uZ|SjK4kAE1w@aiX|xS{W8Tgx4n^tPhfL z3S}nmOn-j&qGQl+b^}m{ESk44t=8fjwTjFzTEETZaFa-11EQQLDxVi$_;Z+T<$Jc- zsgy!qBdZJCh?>^I8rQlb;iJpU`8BWlatCMvM zMg|S-__lRgJlF#x>#M981$+}?ZoS%prrSFP!=3gtU7CI0NwcB{$gN8Wr)M$aKP7VV z!W)qUTdWp|F6GY+e4yetlJ`Ee6n+A4-`N7PX(;TE3OaWXt6%?F2e3-uHrj6_1PIL; z&;pbE{9s3c#xIe~C9P+Dv4u_9)D^lUsq8n&nfho?2Ex;$J*D7b*Z}( z{RTY5V!oGmxj+2cICG}hjs!=BJ!MC&w#`r~E(0w^EsV=V=Stg6cE|mb(Xb(+)>+&e zWmkr?6pHEjYrEld#6dT;Q4CHp+MxdJWO!w?EC7V)FE%b1@)FyBqI$%)E!YxIoYUEz z%7q!QWItGaEw|ayQ}0qvi?KfC90Ag~a0z4}7u9X^LeRfBTm$iG=Bkp^*OlJhDIVZx zqOpm9NRhs^Cu;MU{YE%>M?>A#1Taed&oxfmy@Y8;F<J9RQl}fM}TF9ZL6BZgN z(qKBDPJiN?o-xC>VaXt!lDLuGAN-?S$G?Pd5`7953%8mYA$^r5=}7!+f*?}lR%~Ro z&r%3e$}KE)TvwkavB^0Sr;%-?utDiNgdjZqC;IcREx^1Z(KJ4(wiFw!iuZ_9TkuEfbN-TY~tab4lfZ zlDZdA_kB>gWk*o^i&{$g_2pnJJ!}`{LA>Jacqf%(LLPd8NUm6-H^NKulHh0lp?#_a zM6AK)`S8!wkX>zxJf@DTf$Da$FYe_(pnK3P3bY6aVnwvAta&F~zu69{kpHd#?NCJJ zYVpA)&3)E@Qq-ZVHFj{z=i5I@OXv;vIC2O<ZCsKvfr@X2U&l2x=MLKc;4zKp(G8`Z=%Tp!)09?L4FkRf91kVBRq0> z3V=;Qf70GR(abw_#Sx}akv~_ojyo#z0;KazM5;WB)_F@gUcigmFE%E)WC3?Lx2>Q0!gM}ll@V)VpNaVJUG|H(UU5vKgrW1-RjZW zY*!&4ea5x0KxRLGXvoAO4(*V^34KG1VV7zid)%b_=FiG<#`dlL40E9kcl}t#WraB3 znJmH3c`yMJn;(Mhb1(vLyb4p#M@&LR#k=L^gKok^`N-vTMo|NKUCDbl>6-f%Rd?m=PSPRXL{k>7w zLrL$X8V|QjwTb4Ie?NgRg%}!55r&|zhQaOmFK+qptqDmdiP0-EA=1qMAjlM}`!W4p zG}qBmG{Cc{vZNE>8f<%J+eGjU7!9+#8V_|5jDHqwi~o*bUQp6{W;-CSI5@mIp2p6? z*6&;q)&wBWsRp)}YR};%)ut7PWxHc(z6Q5Stc5bpx>+Bg;^?v-sp~!l5>rl6&je#* zsnorJ^`t^Zb`)kTWyS=j@=q3@fyN$AVHhgQG$sL5NV=~DWDYDQqZ=^PyvxEkwuW;btQhA2CUpa$i))ch`*E z6=5tR8hs8r^8;T!&YfT5MwuJHQx;Bo5E6HXs#*y>!Nd;@Mc1FH-qoIcs|po}a9TbX^X0HE*e*%WSgle1JS28XcrTk5 znC1E9eON?Eq-1{(dg!6Z8b()|;xNuWfKT}9O$(BxyS&s6n&=TIMq_fpSq zDSNib{EBDMvHfXlu&5)&qBi^)O|IwKlKNpO1y0xxf+%O|D4Us~l@n({`2OJ>M@x6p zKRQ13!GJ%EBX>cnIM2YEoM1c;<{JCIg#;poqewOXi36oIdP1tGw&SL!V!a zMYM>j7UpyH1Ew2<(Nr&<3Me|QjafMKNEqoD6n!;vU^O)J)dfI|X`}tVLr7CTa-9B0 z9yGX5J^T2J z_}6yKGIl3MTxMzGVUn>HHEOpbz~M%U>=j*9v^c^ql6qb}Msh63ASkU`E>v_-XB8I3 zHm?lT2T_``tB9c38BwCr5k!;zIe^#dvq^H;W}c`d?t_MWPSp(7`mu283UoWf58161 z(Stpc9h@FBS*|h(&gylibn~{fopBCzj2L!u|gyrzS^@S3e6XH9Q%-!sJCox#~^05{B-|m z%c+IpTBoTj{ZkqAx;edZDxSrDIYb4;9qXn$oCKP4X$16gl!bdrCSkEwXSlst3IMW= zYoyjJo%-pY@y=_%q5|s-8nrj|HKdap+0;Okk)S{tlPBm>)n7w2MmBtX2myo_AM)X8 z3zt^*7xU``r}<(a&X>7;kbe4ju#W4tU1d?cf5m&7S7Wr&)M6K}wVP3a5+H^RTg}qo z+p!`rA0G}eXjJ>u6@A07xTTUzSX8@U@zXJIZS;hfC-qb{6-~mTxYee8toIeb{ejrB zUEI%;*)0&QN|k)MuTzDygd?@RtY1CesBw^3q^nhl-&Gn) z96gc)6}NAR+o6#FsJ;7~3iwX0DyWx#<{SqTTMmU7%>a#0^{j0=9H0Qf8lZng4}_?9 zu=auSWmF?B{ZpV|rAr34?Xqd1VS$a-=ozh__PvwJ&H?$47!h+cx53EhoTip-=4j8t z)qZJU_w#U$s^LS6E@SK-P266)%!dNeC<5y7{;1G z{nrc7(RsW%!lR-clA0hk*nfMypLzPt(1NfGCMt~>o@pzHiN|RNvf^H(! z$Bku~v=(dXX%DGB9^=4jb`Py*-%Rxr0?WiSZGQOvHt_wEwViFzwUO0eoo{J%rK5v8 zAkl-TWX&LlF!yEmF0`cjsr+JLHQt7+tq}i{z3z?=F-1*SSL^+W9kp(CQ^+Yu542+) ze0CN0pmIu~Twg2XF;5@&XC4{$nQ!oO#px;%bkV3e5@d;c?>6--LQQ%{3s6@XgA%@s z*qxnmg}=&cjqg!fODPD0oAqB6LcT>K?7(EX+9!$rqBs|~8*A8Ef}7-`}J3!j@CpMa|_;u?k&=A-AYq?52M<_<_`ciIRQQ=b;k8CfW+%gWhao0 zMo2MLcD+1n*(>`fGE!tWxXX&cK>}leJ`!=FVpEfOA`+;A>2;d9zuS>$I_60wud9*# z`ymAWd8Afx=wqJ717K*_Dy#`jD`I<^P7`}B)*H8ueDy( z&lyK(IJ`sa3d6@ou z9?7q!PofX6)(XP^QVx^lSW#do$LHBVO*-xLf?IysTee_VQTH;J zu{F?M)XB$YIsnV3plZ+_KhwhRxTf5!u=*@sy_97bOY_`UA}&U+nZJYzcNmztfA*zO zs%noraLRUcUWG*fM;P4I+!aiK*uB#s6#ivN%8=O%pw0nJd&28udm5(EZuy!U9sd~? zv8d52tt?KXxlJx6TlmL)ld)9jZE0oM_dc!G3uPs)6^GV4h!0meS>=uT_FXEJV4bX# zgmh7F2OpO=n5gBTUX-Z6x-4$lMixnsV&Ar8yE+2OG2t%W09haW!N<`pKuWPd^h~RB z>)-DTK*X_I6Dj5_J3<~SJHM{Oz3o>R);ecqClqNuGiE()#W3X!&~@Qc68!rj+-9|+nm${1u3`8hF;Z>FG!~f3OASzj|(dE@)RNI zY?e=S)Ej2^xOJsmUEF)DZ2Y1x|`clq{xfjDo<=h+0yzZP* z&%8htf0o!M=$KHYN5l?HYx^-3_{PvalG*$<$#O&+37%9akCx_dK_6AO-U5xEZt;RT z>c>-zFfXFW^vEZ?b?UKcQ-;lI$>w z^jmIri3`;0;Emnv?Lcy~UGG28bclq+KU8=;;j^$ktLCF9B)H`;H>C5u5}eC^7qUq- zfpZxHT`xPB*Vlu#jeoscW%*h{tW$QE?VGibfa)!)cx@YU zs>$9zb|X`KPOm30S52m~E%t)dIg4mw8OaDEgoqMW(pVOq0wa^lEG8OH1Wzp_YkWFC zO6O9izY{Z@HTUx24RAS|b4WK9`|sx#^pFlpZ2#RI;d z(NnH$ETWKYiqx|dBgTM`9sO~Juw15aS6X_o_ zg{o%ZSc$B@F`O6>GGmFxOW!`S8N-&YLGjM{(^g{^dYe>$PCll}6gH1%lfyZwiNFHTr!U5*$ME3fb>Ln-0 zWhXYL!j0nO#Ea82+l(gEpr%e{6m55s8xD?4QvueSTNR1q&P^IaI1h1L10&)aY8WbS z>4cVZ$Du?yKY`Zr6<(e7m<@S`ZqVC^OqI{3(-T@Q> zUt^1$MNv2z-`PiBYAE9iVuh{`WJ2!T9MQhgkJ!9H^(**nZUH*Ky}y=8^UXT)h4(;T z($4>59V2!H4Y%QsViLZS!9Oh?HOi06d<@a=qkLOEb`-lfT7}~#x3+MkjqvG%5S>cg zc~&OV1ED~(0*@U1X(A@XX#l1Cto(`DVt0nH;;du4C!_9S77nalh58c)_93n~(4+Cg z7m7L#gKj_Nz&kmh0Oc=SN8VGR2yO?7j88>*?OQvohODkZZe8FcY+%p z199_D61WuAZ>Sd@^*$8BSv(sPQAg+xI+qn+va&WjOJliQq_tamYOUGTI`)hs;p7_z ze5^>Ft62pO4l0Qel|jMNquvr5_A3j#SITMSq0*;9rGRhdi%r6K4^wa#2W>@FPJNG#_%nF^l*q5W>^%WFTKbW3f62i;#_v%< zK$?MT)pvigUvDOAiUqs6fa=<1JYo?&r0q7WPRL&&y=v!H!$A>S4hu{R63g4;zFkBx z$O*($gDF{JAd0Ut?`7+^D9!O1p}{n!@0c7z)hE#85aiHm_-M;MucmB+&%|%i3*e)> zo5WD!@&EJ+xeQEqca9y~e1tAYV7MOmS->b0B55;nQ+~AfVm~Iu#U5A9owcRuSa|<- zdA$?uQP#eCr0{D>@G;rpNnd2VCHn0^hDe6pa{dFQ0 z#l}-?nFz#{HcSt^Q|b*GZDWPfBS6`+W*6CkSML>fNgCk`Bg+IRwNIP-w+8Kl@3U_083x91J!zLIECAPNxk z_&K@`(`rsyTPegW+P|I)^ikj*bp4rXRI*VmB#sf5&UB@u{rN8k{ZxICS^_NMeNVLk zc!~g!XF?8VA5p%dR`jP|kJ%A)obBFgk|31!FQZ5<{fE#Z%2S_t$9v+D>^1B++b6G8 z1NxK--**G$5REqMm*}LQl+G(q#n5bnbmO_2N7hV$SX0N-c}nW!nY+!-c`o|7sJKG` zmMe`N2H)rPW&K=)nAP4Hok@h0(aAPcjPSF)mk&(IRYdZu{3OZJ--*sb7ZiWb)=E99 zwzX2}p+B<-b+W;^(gt}Qm>BaFJV-@VbL5v7-{j60n^8cJs@_$}HX>iD zTCPErZt2!za|@fp6|TGTEma_VcAA zU>n$}hch3ni?~#2+{H9-f(b>UY)-everXUI@p3pzq7c&gx|){tBjn-_hZm9?tT#dB z1|IpPe*$rTWIhp}zdGIa2qWg0IRHN@kL<)jxfl?>yI2la%K~zZvvJOF4Uaqx9lXD~ zzaLT?!oZx+Pb7FKNWKAg@smQA*Jqgnuz0v_RG@~Zn`})Mqgw&#iJjfBlh3btAs@Bi7xfRr4;B^s(W(PG+ypf7dpy_fO%OupK4$0!f&<=G{&jY6xax@%czt zaINiM+)h7cZ*3BZWUotB9u_y<-E!b|6(C?4cBvRK|dhNJx&%< zss;vcb5cFm1eVfV?zBujdT@z@36x0EWOZ_1b^0y;m8ODzM*BLY<8yo3j~3j|ID{hu zOsNC%0z4h+9hzpN+z^=+vcouG{gt2IQWD>6QKN`Jv_Re^{6tUFyO zM)bwk;*#^>`Gg2EFn7Rw(}knCzW%WN@?(6vw9#*YUUeG}u;c{+yO+x+G8t~_W;zO7 z)ks!0i1H+x4sQfz!bVmqtzilTGjEZX77}0+AC%Y2^Fc43K}@%*irgZwMm|}; zh25}Op=p`kZ--{PZPsO{TU{ukXAM^l!C5NwVZ7ia^b)DWq0`GT3g=?e$8T=)jh zSrO$i>P<(};c~Z!n!0Ag$`$s+%NX1wr=hxW%e0;K3alvHmZ!Wy@ndBIx6x*Kn@4XEu4lb3A zHEwXVAJj)%G#lvNFFVn~xOLtKjIK(Qm@AfAzS}#5p8#b9xeN{u>aU_Gd6qS1-R+3w z{WPvsdArNmeY6Ok-Dg3|Kj9lh-Iri~1eT4<;(_3mo5jFQt2KW;cBMRbsTtSp;^PTEemDElwefv3Cg^=;3w5 zk}y=Q5gTOdluNWgXsb+gMnhUL%y1-ksK3OxNrxv9`+s$l% zk<&A&OY|+CGG&11H^#ELwteBokmwRr~!73 zQC~_^6wo}yu>C*1!lRWZJU@M5Zhk9eMI!SUZy3s2i*!y&#N+?4?_W?3G+h z?n_21QJ zC9@D>*0KHgBD)KSGh4**Oqa!50J7CtovbtgEQ^iubKR!%QF!&Q5zWdoJG;{xvS(hN zcMj5_t0Lvh!gx?8RV0e#tTKvTXH$aO4&lDF>9MifPOQ7#_!2eG1$|lcC2>LF6sR85@d@_T+FFmo z1fBk5dXieLYy18U7GHBq@yf>B6dv-oiGS9AP?RSdU?c%Wxz2kg;lNEBo@q_KHH-)ctyQ2h1b?W!~mD zsZ@~cbk+Rw4ihA&p>kbMG1?#4QYZTXSfCxdbE?uJa)6kQGZ;#j$&}6xWntM&cXMt^ zyltvSyTZnPB1x33Kf5{(b&66r2Pj>S*)>;+r~JTCfcMhv>Dl95+?_}kU2N~4helZY z)r)_9&mt)tObV_u%qVM57!+rFp|K`c~{ z&{|wQfm>j3x+$ljZJI7yj2p?ZZy>cZLKKD2E@-58w`H9QXr4c&Zk)WK0G(g2Y`G_S zH{dj{Hhpf_i0w=q~TGu}4a=SUyk@0G@CG}g4)=IQ}ePUm(kAsZC|!EB~Js)Qg|IGH8SY0^H!EU zs^&Dv8qFwWJ7zE7siGMdlL>KBwAJKFjj2OwEC1|q{Dv6utX1S=lZtFds4GXX46j$< zvZvAt4+zT$o|qxUS!l0utabiiQQ8dfFQww~@82XXRg2s()eI96x2tz6;LX(Dn=BsD zi$LfU1@#;^);^!l)h7(TSWkCw-mq9!8>L!VU9UUQePTk%3mZWIgxXT} zW!$O*j^XY}(UYkCNv7M*Zrsv7Pjj+~T>>(nhEnMk5=3lD3T-$Btlp;a@WWr8+>W7- z;p>eROsL|JS_r}0m?GTdQP%8+$4l-hRCQL)eq5a%@*Y0Gu!T3Lr;@lZ z@Q?Zp=VB~27}#I`esf+grHxqf(mQbro}AjXHa})R@f{!z|B+swNxx#3-YR=qbh(5Y z-_9_qg#sjN^xNQplzWL(yK%nID_4u z{gnES6rZX94J9lcQ!L|tTB@NVVmIwhBlf~YF9uB4Ljvk3Tax$yMTYFhhN1^WlB|EeD~>C#!4ZfbuMoPz0EZ5@PnJK?5&}k5FLhp5z6!mlvlh!MjkHp$zA_#y!GX>zC1!*k; ztOJ&OHa}K&qdBDg;7~!yB3QanPgrWYBQHZV;1}bgCO55qxS>)a{L~>*yXwRYmG~Er z(5+pth*;j}qcS)Dr2L>7IE3?&8s#yjdLX5{sj^CV8On3iDi<9j+Z|lb$1H2V{V?@l zcZc=zKa%u}(Zii?Cg|;f-ZQ*hydqVv$G-DuhU2+$HbV;QjC*3A6t!NCQGqy;t4fML zEC0@`v$Lm{m$w{1v8ZT!<>+A-u!kvP5v)E&6>C*B9mZ(}#dAP@>#ruUy~08v^AXA3 zVboNQ4BWVCjy~qR6|P^{D~z1+r{lIeqbU9LZ%8SMeWX8`CTOkycgM$lD6-=ZnL#btc~4ooS`)KW8S4QR;g3oiKy(=0+|&(bY(zIF3D74X=p$sGOWyH^l7OB0sjB z(phqKTpZ}_sp%!S~PHT=rPY+7werWj*-SV}!bf`^XBq$cK}!Q?+v5*A+6IZO%ZPa)5V zOg;axky74K#CYuOYkR&^i4I$fCpyes?jq`%o}8rizF*?{6$y3Lbq~rX)G@x8QvQGn z6e?fGvuLV}g6n%;-)$pt4gpytSUX9XD;FeuTR8*^<+xpoKuWbUbtki9Ta&W7F(7Z2 zsqPQ(iGIKP%=$+#?0JV7A@VKy?a$lc;t*k8;e({;ACp@+1tOv|##S3zoPjFyTu3xz7#ds_d4dOy`)O4=aP+s!jB22JZye!-3D{Qwe<}|3sWkxg@ZqmZeEg_qU{Ygs9R{uAmC7li#B?_REC7U0##3vL-yS|F=y9 z$OQc~CB}kd;bcDIAA^K$tELL(oz4Pvs?!7qGnUu6^(CjF64XRJ?8lyRLpeIUkGlVO zQEx^qOi8HIA+@~Ggiu{ZVZxK0dOQD4Su9UDAr;qF7xG2p;0F}A#U4=76jEmB(XdZR zpJ(GOHsevGdJ4kBU@Jl)UXm_li`XP&#K|CM`_2N4+JN1Jbu)H!1QU1!bvz`GzGDVN zoiCYp7r-@nH|`YIxM>$h}_yIbBd$Hp0QjKThPuq|GWWmgomw-UZakI z)}SBwbP5xtqvR9d4v>pv1v7l5M0Jd^T0>h8#WETEnD!ZeuipO0LvJdM)?se5(WDp} zW+HyiPb06hyBfvA{-%cTB+JNm^`-nV#hD$E)PO+Lg?NpajUL+S&7YS_F^f=Yy zAIhBu-c&)OKTn8)p(Boq1Um1?xbmkd6Z)$SadN#OPBeLUvCNXUb^v$`n>=Hi3Iw|#H^mj%daVJYW0D}2WnQw>vi8ny&`Qd;?M z@uUC+1%67IN6v$;4Uu;rpF7NtB;Z4`i>Nv9VtN8V9Z>?rUPkYxYALijw($=wO5O^F zW{^sd-+99N`Kj#;Yj+6ft(q!}aNTAbl+T(YSj#Lp*RDIU($wI`hEr~*pNiF<}WynybooAxdqhsxQ^_%fupw>r=Gfz6fYeW>e zpMa$mg6|uaI;<2%f?zyL&nki)ND9_@7NL+^Az| z`-{&fjjCT04m8@a7Cq!QL<+T46=PT2TX_yZQtp3C87H!}KBX`rHL=zBtjF$*p;YF< zfiiTf;K66dALd%m9Nh<=gyOeZNg zePCdQvn(GrHDoC?0wZ1ZH2lyff=YG9KaH!XRThCz2PJTYs@@oYI)vE8!rEjBg_<>{ zNj0G#YramX7M@`wYirx^F9`7TTZe{KlYybg)|})5V|} zGS)LZvUsbZsVCAG{ds)0V1XAl!^nH*tn=4sRcpkb;z=>4I^-|pa}$IQ%B@xcgS%G{ zmHl~ZVEv0{=$%^fov6k3li0gV^?#sF{Il&C86jfIbf<*cDzufV*6nzqe_>!Pq|pe> z!W!=OQnP&cN-n<2Aj`O-UU=u{YI>#QOiByL(QJFai9GWz?T_Dk@A&P3<{LoVFYi6{ zlnEXzwCn%HB_ek3;dI#J8iEm5w(#+lH`iLtgA^(G%7^EqMXp{P0uqji;KauJ>SS{X zU^q(a*i|?l?M%7>vB#Po`sT3%as;{5azC!Tkepkz$92BP6oQucy-p(>24_%qxVyPb zA9KhJWh!%>Uh=;hHp#IK;Nj35|2aUvr9DiQZuN3VE2BkqN7Kno&%D|Xml(*kL0g}h zwM8O>c^{3@^C1O|xw>9$%)bCj5O-YS!--l{?vX>G@4Zv2mw9>~$S)S9nhz*Km~>WR z#AtTWJv_tv0TXv`nt7HR{AvCXob}ATKk!Q|LO9U)pmp07KYVvl<)7lcwK6ga=~lN^ zn&RlSnoR5KOz`mHd_`}ng+2D&rjQ8!>z0H1$)X(aYvZIhD}cNAR`_;PDI~9} zEBo+u`Y4%Aqw)nGnB|MU#G3yXu}ASnwmIKJVJtb)ks#SBWhT7;UI5CR)|fOEe+pCL zQ$YO5#nVS;oUJQ+y9vcogiRIBe@B+VdR{+1(<)>sJ0Yuh8nS~!6%NZGZm#33*wB~8 zN3WhJ63HbF+Pa@D_To?pP_mRljr<*l9{Z7Hf#;*h6clZ#3>hJsM5=b}Py|{UKgRb{9 z83M4cKW8Z+pM~`Bsz7mg)A1e)$vlTR=taFPQKE5fYAKO;{tA+!$pCgTw;o7*mJv6e3^H<-iw zTYWA{bW}|B?If_Q%7A{8W6u(QaP-!|)4R)!$1KbP#kSecpP%GvOQVzx>OK zKR+c{V$k+u9E-e$YvCFyD$ab-ASALR7^EwfrK1V1SbzbGp_W9qoA74`8%U!MNZdn+ zxh)6DlLNh0YK7shW|hOQj3Se~B@j4|Ul@71LvLAXF0Vfg3c6YS>#Tiw@wLY!{WTt+ zA`$Y4Md77c-TD@(Bgb7WZ*%e*cVmr{T*p6@KuLj)YDGB3d*_#^F{2ngfV`o5jOq3t zd1EDHy&^JiNj!9C7+K!=BIi0NM{zSDcxDfF-q0~E#hR_~wA*F0$E#C0KhHv;&U5hb zUkp5I;I{)%fs2F+N4e}F?iqGXzpG9pqC2ahWa*ei3bpCJN{SC@KCBk{CMCwI?&x4g zrI@c}9^>{ps)o!Xxkc4{h5~%g;taa7S9Ym^_iwk5x`nhSnD9ZNiQ}1 zT+-SpLJvBC%OvuPsP6IFJ)TMG=|FapaUA4HTDr@f{NXg!-RBLFr{@qMy^m_7j!w0@ zEs`|u72&cbuS>pJ4Kj66XM$72!;}fE{xC!-pN)O-2<_mxB<+9^Gz*DRgS4lmT#^Rb zLX;xgUD|4|LnEoQ6DB{O^7yXO;ah*i>g{;fo0+v7Dbg&hPt;Q9k+-;kE3OION~W|N zAtG_$vQth9kII zfJh3D_t%Q7H7i=6Fv&Pf?gegTb<({Zo^{m!c`22G-u>zwBb~LzoE9;;$bXXw&D(8< zUfSR2=;Ue(;6~T-x|L|TXAaNpm6@^7*%-c(pvsRMN)`Yu_UcSD~&IL>px4-73|(Wi24JvC6&KE$LwbXnjH$bZkk%pHpk7K0x1}i zjfJA7ISX!ctvECW=^P(-FkrnzR`jba_880kVfT19B(Q;~T09b!K-VSAAUGaLL3^%E~=N#??=a%TSEI zJw+FR!G}fNPUUBhL4H&A!r`&jd!+(Ig*vlZ27 zb;i{^%d|bsI_FRVd@d)n?6x|e^#0WGtY>2Kmun%$l=HnsEmw&LdVSB!*07!^5L+Iw zbN`o^8)i>P_|i?so@ZvZ8j=EQX5g(yhaVjjH*8Md{jj;BT^+0FrYAO^DE^&&L%mMF zTxB*0GQ5dj9!J}6?2rhnAlBr2fO4W&d&WPCZtT?q23`jPQ3&B!cf> zCZ-Ha_H#Dylaqs)E3w-nIB9-JqN@vSqM_sPK{bxVcI1i-tvWHpj@4`T-pXS*q}V&C zE+PCbT8#SQ#T!1hel3QBVsj#ewM189j22qIrWd(D_zZ6K-t@_P!%xMNy|zIvHNqd0 zzAL5Zqr;c(3L_B=0eaphjxMHPsAhY=RLOGmcF|@-A~CV7YDEsWPMIW(|RMeVED zrxW=OG1wry?-i;@M~*(tS6PDLlJg}k@aDKev3L+{p=DwVe#0gX9GX-x?ItbxM0T39GAB74?$|GDVygZ1+}CQz5zC4&1hJA! zhMej}7~BrH)18e{y>HHoQ<=3U*qA*nEoT1|Tw|OD^UdXM6+%3JLLu6+lz%xHsur@C?|Vu1@kJ_oBCe-(W?$F4xGm9XguAX zAinEyW)@cMI{3L1zJ*(^8S)gKtCFlA4>HbUVu-KQQvl_WSO6?guNiNY1+)jW70p~= zrU$`i%C=@nfam~o0tV*u3&{IQY&=vl>W&?HWIyA2{K$7rv73mTh>pPgvPS8(Vxm7;rX*#>VDANG!RDfZZ+__N$5LU)1n<++{3(W3 z6-moCS4px5OGkz$~MQ>B(fv_P9V3 z1V7Xs!4odRwwDF#uCi*Q$%?#9M z50oX>57BR^Z#UD6zztsit&Y?e?Mx-jdRGQ96@=y{2I+jmc8YdHuzK`N0sm5d+(U-W zfxWZSZ{h5;@0ApA+ddu4tov15eBr3R@E&6d7fs>tD`>a=+sA(0tq<{G)dvSExgupajLnbnLl~5{%oCJWDp>79qIQBQ=NuO zn2zQCc++0@QcRWC15G8qTfFHhj-CKR3=H`q$w%n4- zw3X110Et}JFu!Av`zwek57Io?;$X7!(o@+x51KFmP(&Tx)G0Yayoap-3IP1>@Cx!c zd{(F0g*V@F_wqdEz0RL%Yy#$NJZc4AguD#jh`k9~Low&pR+6J$oNHJ0Dm~WQ>N8&O zq1VB03=~-Xq*BFmig%-Z*<^ecFse{lVZ0>HcX_8jmZMXw<23_vZiL}W`eyhtPfe-ywaXh?3hQ=wer3JR1E7GvQ;aO9sMogE`+t)86K$hE;8T)888M~;^l>|EvwC>=6tEyf|M#KwP3qnsn_l~eCY&X46K)#MgdfDhz{Le;C zxL*=T2wqS3$r>IKP#t`kFm$WE;qBzYt5M&o0HCoIw7L~@;3oI#M0P##C5eO3ZkMiV z1(YT52t6}%bKS$lUu~^&>62gk^bu(zB98LnWL#Ux5Qm2=8P9^dL`6mp1@Xj33nga{ zZsu_GO}6U;OZ5=M4(!-NINAZO6|b0z1NJm@EPLmD$c}5XMn*hSt%1bKG0UDhxGt?)c)^W6{m7}_vf13%YAJjHvcMSQNpgb zFP4^bCipyH9d8~C3{{XhCcWddX=i@7t$ZJkIx-U8DKQ?83MYwCyN4$x#=p14pFe$A z6OXt0ED^Le;%|3GexOf6hx){_0QxP_DzL+t_qu_tRV8uhrU3-=wrG7J{5N6?UoohN zLcMKUvuS@luMrz|!fK#1*#9D1HM)MXspO?neADSC_Qu^4hQ(qAZjz*=X8XhXxI_M- zCmQW8mOhmKN1m(aYsM3T*Ig$vsZK;{o_1)>gg_JW!1*N8=~7u#lLSXuPl(bGX2jE< zMreg?e_5YaM1|m2)1pj$zC4uQw%TtYgz~ldBYBu$yAoiz! z8c54{HNA*&BIrG%VS*sQ_&&4LZNgv{Cc^)i;!DZJYPBuoBn*FpP7Kv3`yI`?8C^rD zL;e-lVYaOBKe?eEoa)*#jsNiGE-TL_l#r@eC`vf=t~{%Z)zqBJ6wTRmq)uV{JvQ5!v(bgO*{9{hv_vzA<3sQ)ZC9_PUoy@kqm&j zPXG_Mf4k@2B3BF@rJdHotwOQe-%j9(MHC-SSCw2WJ@X_m7g1dtqt!(TsVT!6J zIJXi>x>5;}m4c;D;<@Fpwoq$7>8f7Ar`|x+IPa{eMb&IW0B}7&KGd%|s`Y>FO&0wo z1K7M7UXb|nkx9tz`M*WCE&SB~DVW#h_SjywWz!x282#)%)j!3s(08)#6xFd$8I!B5 zOI-d>ac=|JbcY-O5Qv}a>-!x!8sXssbx0XpmJ*;Z+dz)uZ`Z zg-Lyuo-A(wSZgTP!=NALex!LDd+{SKp^aydjwWRkl%6>N+s^5J!*yfnWhQoHSNQ(?%E1JjA@NEO4 zj|AW?5#do(vhWJpn#5^{J8=xWo6{ZT8AEa-@P&>!VJf%)!9`teuyTaI;xM?aV%#)mx#Pn zhvlZs)z01ZV+|4$52n5_3X0kWK~5wN7eX?KVWBYvsPTcod~|+Dk+TRZs#R#=- z#m#iNfwauF5K>lwLOpf`@5+<9@%I;(^9Wd*G`64nQ;mt=*{nVK6(rv~A@#ShuYre6 znt;hP2rc8qC0zBrRTtJ5Pp;4K?=APV8)OY1)&7?4G_;Ef0Owls=vA6#ho+l#GK5`S zXGZpY=!=nMj12A_{VY-n051e=PFa79`{wJe6TaIwtzkQ+lY2V==c#q98SW!VZ3+0Fec5WltETP!Ls2f} zy)$w3?P2>iz{27r*wXf*+}ewkCz5ogjhAKsb4*DHv@tEGDstHysV(~7G|D@@FtF+8 zkPe;5$38%9PN;DXx;MF)TDnQ`R6Luw=J2I6>nyBuX29mR$t+Jtv(6ev3wZXuPs=h{R1 zsTh4d_UE*vkq`Yyj>e7OfUKU<^+2zD10xjc-sdKV2coCYKP8mm-@eAW zd44y_WY2=)ww3FxMYHlb#aNrhN+$qfPp9&Y? zuC3Q7A$T$xF8^pnnt!-md5X6nH<$QVKNPiJSEG5Nkhx>8Mw*IGf!D^HIhWk1&@wfv zr!#h0{;}}*Vkp;Ja)K|dvyk?N?ahW1_te%j#Yk+`s=Fr>JnJlf@_+L|BY|eg`z^Te zT(C&LcbcG$Prn&_3jIYW#R8&vGuag&9X>j#x~z@!9NKG+SQQ^0lq7^0lbzI9If3#U zjCs>Ne13oP^tMO1GiTe@y$l39w&l8X4^%%$er#;))-sG-iQdrN3RG8$N8iQdW^$s?( zHSpbXn%4Y=Df5GIaGQKngub~k>2~>^R@OU0c|ZI)YMpjck%W-6F3Eby9k_9qDhCR! ze`20R^T`xG_c0kim~VQxCRtV2s?WTFhyT?o?r3VS`HW(w@|FI$!pN0DjUm6uGR2yP zgS=&bKQ;o_hJtI@X6D>$BmqC@9p&gy)1$}DvnlpnocM`{xvDFx3D@J^i0JJBkbG|_ zD4m(v+skK5XWjiA;^5W7|9NM+{=O*Xn68d1jZ<6zIt3g1|*KVB9NrU^XDb$slK~r@4oZ@Yhy=*d*5ZFwf z86vMIgpgT9;d!N-ti~s7>IgDEMefic)-_{F;~z>BSqS3jtZOQ&GtTnuDbC#rJcMHt zTLrI@RHai6c2ke#kU6Jcj?D9jB-Q6dXM*ythPJhM zHm@2npddTSvIXkU>-y?+ab(A5uSkeF4QaPYn1!HTF73fA(3`7pr%+oOUA~h3m~o4g z6*x=1ApN!mYE>R2=4s6*jX8afJ!?0!o1zn740B#i7xsd0TzOcNa_E1rbk4EfSY&+% zt=WuclRIA{R=tOxO;UZ_Ehk<(#r+%aRQ`-ZVS?jx=JkWTJ^zQ;8E|S-vr9(=(P0&}WBH=hXvnoN4GSKNW z>WZXBrSd;f*}0|ViT}$2T(wdFBPO?v$;kMu<()eFD?$;{Evbo>pI&3o zUwx|{$Jk(^Q?`yPYvYBEn$6rfkN)v+UQ5kd4&;2!F58)2i?poMvy9#INC9D)UjFAE zHkV^u`Tpv=ZgrKEmeHY?d{xVFHZLu`W~ljir!n!DjsY&;Q%u}5Li81|e%8U!kvDah z>FE~6h_MOWg!~LwI$46(6Fg5sM3Ozq;4j2`KIkFYvAZ#mnI-Hw0Q0`kEBHJMJA|6TItN%Kl$Gd9g^^nfCK%NZ!duOmcP zWD@uS%oQh}3p*fN34A6OQhI=4!T3o$7HV75$$IB=9{c|0a&XyqJm;6|8A@J)c1x0f zGj%?3R6>rLE(2H*{?A^F9%$$6eP{k@JW}fykn<*ErZGNsS5v{iF7n7rb@GTWDbkz4 z%w_tg5qTE{TT5%vfQaw+&HMo$iQLp&GGohvn1^$|gDoII`s}4XADFF3dp~hGPexVM z0XW_78h|n7HgRM&cV=5Rb`_i1v(Jpt4QG9`Nlg`q>A1OU?VCP%$2P0ha-(wd=ZL+s z`I{Lj&@^&8YXjd)UoJ`u=W2^}v^eTa(%-J7oWC&R;dCAuW>=5_cRxQc6IFcTU?(^0 zt2g;bXVU%Z^;hpeAV;Dfiv#@QVAv3Ge&S;ZA?etCx|RXHqw01TtL9sOSDIr`v-R;i zPhOiUy}sM%z5R-pD4x6+uFIZOI2ZQ~6O_!*QlY3T^;4aeBI@n%E;E-APMqbFzd~J& z>qq4&ZoIpp?v^9X`If!xVde_AZCH2<%#8x-ERVm6D-H3K5}V{tQ9C83k1S>+Jq%WaVUKmV7$JPOtG}f_#a?h^;I~<5(TpiIDOO_74d^_j5aCey*%+9m0 zG0X0Dp*b7AV+xSl-(<5AA=^$g0s<~7T~A7A#?D@fBEL*WL8s0L@wQH zPl>4qvY!s#2$OU`3hru*4)B~LzCnZxRF5n*rzR}y$|vYQOGG%d8#?BKW0WQe5ljtw zA@xcAl*ds*%xMFh*)NeA;uFi;mfIrck$y$12Qka}v#ak~YnKYYZW9L#s6C&FC_DZ9 z#=OfIUe3;U`!}5DWHy)@@*X?eTD6<^GLOkA5W&1P7CYv@2y)pVfv&B2fTYo?vc z0bS#0zL_$i>k-I#nemdQ3g2>JnS@jmCoy%a`WoLOWF+X*hVEqEK*}nxv2jgJQ*|}+ z(v!(tS3KJB9E5(?A-)cro^z&kLx#0=@|+mupP|?R*-Rw588;WR{9#`kf&$WVfD9N$ zXgct8o>hNIJL-UzJV{PdAet|{o;DmXp!jv%IfdCVllbOw9e=l!P>aBr2qKTf`~2J7 z>sKUabIeoLlX{911Kpv%*jUlTZ!HE=7UOXH*yms9ZB3>M7`ER=50(Mj*x0tMhT1&w znh#2Gybs}uk@npGWvQ4nVPMr`J7~dJn=yM6)p=I_l(YQu{b)SJj^FdkSw$d#IY(fX zRA^)l|7LfvYV<^%KWh3K=)!-C5+7>aB-RB?Ctgv`IP3(jaCL133S#aO*FQgKp1`8a znyUVUFZx#C2XP5aV67zh6FcQ#aq;J>I`Mog_}%tqb+7)3%UE~}_J;P*bHTNDKrS*{ zsQQV+YQ*T{2ke8Sq*i5@m9xG%#4q^fxYhHm{OkH8p1AG)5bfxvh2hFX)wnw(guSxVgC`JzKmE>#p?37L zDwyJVlgHlpV^zx>^rxqS-VIwHk~nvE6G-c3|9fAXss=W#hmx^BZ*bNb1#Ylq$DJlA zuCML#r32Du>9rJdzpRq~=BVN_OYdjIw8#NRmv2I;ue&c-K#HAe;!kJhWa^^cF#}#M z{sMEH_r)#iWUA)s#2nr3P0FrIq}%+|NxL|J-_4)8H!4`zCyNwM!3`xpyKJrN0873L zuNMz~nz+5c%0;qPtAWxNlbQ|EGy3&mv^AGj0q{$iHPBlxXn*NnE^=x z$}Vk$DqNd0%6F;HR3_G*E?AvZvMt4%s7PcK-SD#ZV-bF)F z?;{00Tm|sZLqb-a*>vuYTr-0k2S~&6jk}xdFb_K8uzVm)`#)2v9_i5VK6>V3veJ^5 zRGzrf&&j3xYihq{2glMO-uZ5y6ZRt9cN7(?#fKSNPQb)L2Bew;%0eM*QAae2$z9~AAMpW=M8 zwYBy*0h6GBbH;J~97FLu+yO_(WOB1yuYaI*8J1BH{klb)*tMV*dyI#cQD>3LFpNNS z^jD2Lu4^CQe@G=f`lx-5;9btFE*LRmeQ6zJvad&O(tk?#eDfI9g$| zyfjb!wnyAD6hkB^JJBs}^^TyMf5xC+w3+qmcA0_RaU{xAL^!3*@wAiOl?ghF!(o3I z0_T{dWgS*Blnpsaa9B|Uw)-B7fNBz^sjZBk=aBF@qk#R9WD@V`vD;?702cwg)j;Z48LIJ5EHkXh(+s?WE+{{c$!r}7IT&5p=oAuf>4u#gE zEREg!mT@h@GfD4^Mt?QCXC@60@g^EaR81wk`7L`4vCF&pAOK7=QJNv453T$$ZE{1quEV>ge^i4`(r;H|#P{QfnTZz3OIgB<3~6kV>aB? z!8MjLX(v1xlUP8a<7cKQJ-GI${cQ%^f@Gm?94!I}szMLFre7RCD?U%SB;hCb( zfI2k?@wsm}*3ArgCR5yV;U*VZo&v+;+n7C@@qn~0heI#?GxA(+g^7N5^(Wx*C9cXI zD{mtI|6}h>Nt(2`uma$d{*%Go0Dmz2gF?J~`S(2T>O-#a&-Ha_I z>(~Zkn<4u;V;lSMyL8|8_kNz|`+f0$^?&ue@xjb>o#%0!$9^759rSZjFrb!pWoAz@ z-sftytu#kP?yubS@0JL<=1-!mwXU$ptCU@ck#kMGElt+D{2nc*S$SucW_S4$-)+AD zB4;ak(}Glku{P`%Rod8*H4T7vL*gJLvVFx*rmkj|)Lp!zJD@JFJ^H_ue797`*|_limYN=?Ic za$0g!47@$Le3vh&J^fJ{=JAA!i`P4V^mo)?^M6Eq%F9znq0zZzW!Pa(LtJ-X^EzPg zEglG+pe_!fcXF-%MPIYGSdd#^Zwd7D&qblkPAhKe&mFpx;4VJ*$sJjs$xZoQ9hoMM z2GRPjnLIb9^gc6rRvQX&W4&^0dQM1->V|ZmdSsV+DP1R|dn{ppXDTKs?6mxYP0QX> ziWCjnCconO?e|=GIT7=mBw-rFt}asoKi6@=>+l5&1*^Tg5~A*PZd2O}D{( z+|Ps>la?Bu9c?5y>9@`93*lQ+sf(Z4$a2pjm(cH`#zDhd|C#F5F>o z(7GPh-ANEl$cn>k>!BK2u2jl(LbdbcbCXq_4&T^*k7Jk(m)9KczcRWXJr#BWh==hR z(1yM|*}2nqeSN*7I1dRMOtDIG%k)<78ofCciD>O8-UJJ|s)Li&!R`an9xdR8henis z)hU}ch|KF_d3L=;H=&B)2O^Wg?p)}dAXK|g(UfgTO4|YL4(N8DMH!c}_Dv9`+de*r z-ldg6g=(i$Mb~Jm^*dG$+pVSG1eA-^f9mq`wq4TF7oY!O&_!i z8&l%JGPmv%G zAOahd5*(9TLKS?LHX!KvQ|F&<{S7HzWQ;(7%abE1?m(%s9hWDgec|9`&!-)7Coijn z6yNN!I1#E(1GHauDaGFf-|S8ZI%?HQp=5G@LZ0P_@8~}-@`EyZNU6uo4J&?)F5P}cl_7HM&dvTel9l4wR0rQhz^?4?7<8J-3FXHOx z0gZWUmVL6j6s=w*9jMd4Ww5aG=1;Sh^bGL(hL#cti+K!{)sy_m!41F>x*Ib5lm8{8 zqfeX&Zy5;tGx!T@W3t$uzXY?RI1T5S>(a;Ll}idv2^e!f7PHS(-ubwnJ3~qA-#wjr zczt(XXo@s1AK(CAKKusu9IXE=Wad6vJl(BFx#~T1m`Tvf%Q$@f0I~a{wQjn1Tv1%6 z3+>(LjuYE`8{j*~x|`vQFUokZ;y#H1p5W(SEyzNe?FvL{}s-vt`trC z?sDu(zeU-AJ7O_awMbU=j#MP+uXrl+3BEg@6NxtNQd0+A+?8VPK4Ver6CE=#3?t)r zbqC0He$8+{oB9Cm%VEW_5}?_5F>@ND*>SClVHr>B@4K+9CXD>#m_;SuDdgdrJ#2j( zL1k6>@->KX>ZXvQHPYh_A-K}kk(+pWO3NKUPP4&O@{f}WWlt+C)+Wftl?@cQF^odQ zJ%SXRGrNcm^mfZXZ{}$0AI3X??-dJ1LogvuIrSyz{-}5LX|sEvK}{1w`$dZ={}n-y zzz3xlyL$pwAC$!c6+XZHpma3f5d2UHHe!E?-|$9)bZhGL*{k}&Q&*X7X522cIYPTr zJ&_L;l*}4yoG}gy2Mvl0hm`XpqL*2nPvR>Zl+7iImHQgpb!SR0>`udiS-lUKWr7~a zR~qLo5$f=N>crqWW)XYE{8y_TEU&nPbX(|2bdCVUWM`MWC_tmH@JdEQ z1DvyHza$kQL}ko6gnmfgD$v47DA4?il8MrqlK5aj1Bd)~z(j)|LbRcDvn(EN9MjPtt8XiX9o7syQw`Jrh`L zYo6%StLmU)2G~$z=WVa9wsz>Xd&h9u)#ZFHIWi+dVcSQCALLzjT%#n{hi)0@J=dWW zEfGF89R0#P_W!b2V7pyRNpJVJ z#_x3Kb*Bjw2&9}&T_;c3HgT`>3479#2lz*)OZ@G$jqI(s_SuEHk_LYr4)@5B9Jquw z=@&&t(ro6O>0MuL48HERb>HfYAbRW~9&r-V){Z`{Ontm$Rp9DSw0syEqpl>rOkH_G z%>}&Fb?OZZUvS(h<(b)%&cOkZ=FSVF#F>&XS3ARzv*%8Z9?FtC_w~9BPGY_M33%ZZ zz47Y<0-#!9&tY4bun7E%DVn#g(VXP!+Ev3Li`^%YCD70~2?bdic6}|%q@(czFo5i{` zR3X|L>TsE`J!48_sS{2u6vA`#6K9h&tM_T~HOna!!}1#O{_c-)L|*jKXmm`Ql~Sp~ zDk_LGuXb`c>B;# zs3lB0qxu?QmLo;Cc6+X+jgWa{Bbq78phMY@Tz?I<^f(<@XUC9&)(Ez~-L?D9y%Qoq zCUqfk%7Nn&GetZmbqCJ7DkkF_u-?yW>(i&vb1P=RYOUk>kK#HO+`Dw+ynfKcD;-CQQh~G!MP#pY0h+FxZIe~D=NZLqpxA3y< zPIL-lm*6!!uwGm>&QzNy?pnNLwNm}OT4hm91bIpUVfsrz=O`}t&XLHv`+5EIp(WJy zz-54*S6%%A7Qt}Gt5fJu=OB*9PTXAiqMLNRFwjB7&W12$mH9#M;JMkC zE^~752P3py*Xn7$3D86cZ90E#H;AMFbnR4~ctLXiQC~XF&B|1m)NwMYcB`S(d7o=8 zUXNIkCdw;)F$vMVUsJn+iv-t=+L+25e23K#;USe&mvz z=jk709#p>MUvVI1B$heF-p@svd(HSt-BC|`F@v(lE{V)P(!f|POX^-2Y3l;62o6WQ zB>YDeETr6(=QH2#|H`YHnMSzr^fdEb3pN=k*ejd=zNMT=&Zb-ZUY@gtkMig_vRt%z z-QYEpQVNSX5Vy}6-sCOR^B9IE&1;V*qq%^4q2${c`zmof4*nT{BAa1K#9m@Evfh0+ z-pXlWbs2*hwfQUd^w#g>(&CahYRNK4?et;4knO=YhGy6C!MtmxRYXo6X(w(TX%CwT zSQGK(7d}CUUhu8&@dN_l=71KmD>7}Lwp(-A!YZp?8Fa@`px)y}aHEi+VjZ-JAhUm= zJo8{%C+TM%mL;^5sXqp`ZjN(*AE8HL z2OO;<+Zt`3mH9K}A^O(hj2g5FslDfHQ(yGRwKUw)_HiUg7>RYp6H2I+6dmLJKlShg zjMT?e)(LSJ9)&gH_**$CROlk5cCt4y zr{u*B=Km-~J@tw=oGwBZeaorlSK6eI*DG`=+7ef!MR21>cj;HBQ2YFIn5}_s2xbRk zn<;1Vlul`=Mh6)%BSSfA_=3`{-(EXB%1nqfc9`&La-EuX&OG!G*KLt|cLgvTAFSqj z4HJn#ht9>sr{BX9A632UOVUY`(Z&wi4f~U9j@HdM6uQ;E)9+-Ld;WNH<_}tVgD!0_ zn(nr6nXR^!u_3a(Ki|-Hcik6j=-C@?v|Z7b+Oj9p%2>Ahs^+O$zm6o3Qjy?)hzPzH z3syF`su85t4l;7RZ-iIe`h1?4 z*<9g~f_Xg|UiRHM#)#C6S)%9P{w0&~k@l#kDsQ5LA@dU7_FJU{1>@(>&X*rrxt(I{ z4_=S5x9RI>(1tIh=7>kT4>s{<^N{%t2O7j7O2~tRs>3T9b^A`^zJq}ekRy~g0T$Q# zy83)lCoP}3hq6_)&o2G?6MKEmGpO4R__46Ii^{`j5l1m;jN$iCo!Jgl3KJR^57 zEx$V-L_x1}bu`r>ha31cj9dGeCG{xKOej#k^|{a-t&T!@Z1tXcjch zW%8@YfsVnmUD?E0^ZB)to{I3F1EVOla2?sJ56KU_vu!+O6oSfbE5Mu=4|F0jF6u;N z@+|n(Ajh~r2Wv#v2WDYNx^`j(p1k9L$gWnpuhaCt!|AwCjV>)k1?R zf!Z2?Lb${5Mv4p$D*{AW)eJ8^QZ#^q*G+m#iY%AX1f3UVv2QM&SsZ>Orqzitd? z;j`Jsd!iZlN^LvthbhPRarFoi3r)*BM%sW%j1E`yN+Q-%U2z!inPxNUA9%!|_^^by zf$1o%hjWX z@U6q-q;%Z4jN$@t!`|L=drf--wmqBX)(-UM6H6m>9>j51em(9ZK&j6b)Yqc9!>~ih zlrr`sEbZ^}Kf(%wEC(7?)gHAprbQJ53co(Fljce(^*WrXl5{!H1TGl0cpmWu=2U)) zw*NYU*$01}{tn)*V^-I6Ge0^C9tKLWa`l_wp=L9JfOI>rE_Yu51OmamH&On6=_3gV zoR5r*t{NpINZPbET0y+b9%(xt^^1`f_!c8rzX(jOKUNi-qd`%>w9}b+rYMbvnBd4c z@ev`!Xb{Y!7VK7c7v-8`XIGO}b47=Q*HK*G%LL6_7AS~|SvHmLkeJ_a9a!fw-S$eG zpMo@KR(MR}Bju(&S#V_6#Xr>0Qd5sV9M^rP;gVP&_6H zbVq~EF$qXq#*4jYhCgjIC$5#V*N$KHn&`XjKgw^fwf_aZM;Ooux~m>=6q7rk>i3Zy zFn#5oJ|t?IjgEXOlcLDqiG=w07^MVhUbGh?3tXLdNM*f%^VA}~@U;mbpY z$@p*8l-jIK4n^TQt?M-6c&VcC?x#hWsnci!Z{f_nz==h;&VSb=eOv;vvD+gk9p7lubzD;De?t&Of5)cHV`!c3e%Xs(6$4 z46PRCt1DPsg=?E$T3N9Q{)Xwq2IKn2s)#*3i@hW_=Fc{u9 zJaF}1_~TO}2X|#!dR!F5c>4>A^4;@h{Ir3l)KUCYj)wS z4c*}TJ>WPKqy^8AV~{N4mF&B38I#kl+_WvfPoL<@VU#u0jn4MTh`b!4A`FngAIWhB znqlrJr3`vTu$EQ9kdQh+h%fV}H6||HHWSLi4NF^pDH;!DA(OkFO`6He z%jXu=;de>i71CXHybJGi>jth@tERwKR`~k}y}#1)?1pag$d`Nhn4db&c;kw(N?BR? zY7>0uqNv$-JVvS-dQyr`Gk~z5c(<|Y<~D=LS3?zCwvn}9ujui0y~@gJNXq^ZAhO(@IX@y=IqvCQakeUpxmM&iAL^t$Q*Kt zjbmcNwcfzgbit)lRh5dGj!i`nN!5+6_BbCwMFRkNg*pSO$tyPUE(CuJ<%LYuBCLxd zo~KQe_i+Id=~23NqY|;0%iIN%Y3}hRdnqb5WDkdm6$Rqfk>8S{JHZip%G>lD?CK9a z&!_ct?dBa;#SFW0nBI9+x>e>QS{?}{(kUN3Jv*XM3isW zTmo-T0S60lmlMBYytL%!o0>l-h8eBY)YBUmuStc>DMpA-jPwp)lwUE99A68w^Hw) z@Fff@tCzF=#-BfcvSB_CdOEf@;wm8$C{x{-Raf5+#2*QqoRVX$jm%E3pbTSP*cTvg zvHGoORS?lm9ts8*u`h3kDAv{Wjn!gyg>}=y_I}JU3WiY8v~f4fA7qHgr zs_bhtjMK)zji;;PWjF?X>-B{v;G$u_z@=Xcdr-neT9C0hyVA#QbdV)PC#p;-(@h_V z(C~69Li0q}Ejez%b&rZZpS#3c?(F6a%{z_0irqAanw1RN0vVT8v&RL#$(l9dcl!Lxbin-6Incn|uPtE+Q!bEP?D3kr&O%4ahyHDuafgt-NI z_klc2ccUA@(j73S?txXqTyK?WlZ73yI<>UKIYCM zKOg|p=6l3`<&S-lSUa3_|2|HN)z#UV8_ce!jk@E9`-MHFjM?mXJho-$6hTideeSdE zXS*xvt@5XCaXkJ_B&|6BktkVYKw-6QveE$ETZdA0gEF98W8t@{L(_p+z4pB?-y{I$ z9#}!|zV=V9p@j^cIa#Kj;V(`imRD3rKLXDv;|KjAq+1I#0YwlFI5gbuT;X# zx&9%0L(ekL)OQ!}eNwVfku2mR{|47Xq;bQwM5r4*MtGji0uWD$YRDXaH;9jTZzM!T z37}9OR8bKGkl5Utl96^uBV%LPC2U2-wrzgyXyYY^p4t?csw&70-`=j(oApfJ9oN(W zur>VB`D^S32xc^hV)umvaG=!b3Kovvl!~VFCFBaQWimmF$E23S90?}%M zn!w@SPrqK}%GS=e*%w;)`SXiApL@LLYL|v;R7EKkGVnKMx8HnbT4-`V}#y|yz%`H@8QqCdU-5HtRnz)o&i=Ji!29P1XZ;&eQLBePqTjHxFTvKI)m?!mS5TjeTFL&5QFp2zK}T{-)eG0R zfs9IW{K)7=ZyAq=;Fn;`S!2pUvo&3pxu$=v09R74ktGjI+5E z_5A()olGn!&=i{R+*Jlu0pOrRuK;3ByP-_68+~PbxSStAz&YV#Lgx==XB52oBv6mW=Yp=lA7b6=hN+(VWRcOkO%HR(ER1%o?btoq zyraQ>S;O~OPPd7h^Q`v~ql2X)u59d?mjR&x=}Rr8_yiGbhPpbKsrjqR#RoNJ1i}G1XXOOITdi&m55@?%}E`h z{uU-CN#!o%RQJN53p7@iTQ{lYJ(Fvf7Fqc2^bdcKF?6SWnd0+vV5*KM(g4Tsc$`SQ zkpKDIB%KO=0k!6KEBgMiDtmLeaRb@1`&2Wtc;RF>S{BOHax71Mm{x12VD>nEO=q?T zNR>-80jQc+u0bEfd-5cxP@1ohWr${pC$n}9DH6=6(EA&ekS=!u-UYu%5N z&}HVuE8E{GBj$c5qi5j<1`~)o{qe*(yTy`>jx?Tg(v3-D<|&EC$Iy0s3^L&gq_|)S zEG+=P;K$#TfAAWhK;ntBXC{E;Kt^U%v2`S7r`&cwugSpII4QMalP4Jwx^N_lwq{rXaSt^Lc`f zOVVzhFJn|wSf7q)@E>!2XcGYSQrRrbq}o4Pa6fN^J!qTDHCw6N29)7wP$ zDX&0f#U%7MjY#5)(NXS0n-K_y>Efkhv>&gnOs_zC7p&Yzb4(FdzXj3E5@E8IQIHMr z8mN^h%Im2-IFUNt#&+}CVV^yc^wh&JSJWV!y=QTOr(`hw5CmmsA#Iwo^f~@a4Vq;c z+|_}S?M8E_){t&sos&9NQ!>8Dj%Sj&5S}_<$PRN80Q1e8y%1qyw`Q= zZYr)fB+YfR7mv+kO2!~7P&sbe&XM!Msm*PR1^FM|U8LLg@Lcjt^x&x8C)j$B0y}KC zL~r@{{4|4Okcs5LXT1w{GkK=AH=?bTqJT3T7?%5%cfTH9#ATXqThaq7M{xyr(xV= zIuAWyDz#gY-`&W}Nc{QpXL?!f+U&LAa>p`HLU_2JWtXS8#TfhSBn-L`QydEn(a;Op zFYt%&haFo68%QOWNdq7hWW3w4Q4rv^lQdZ9?Ze~KagqJBI8lJ!L<6wg*jt;A;S8-h zL4sc0w!a&}R`N*DBSf!kApQDR6c(ZvkY z?H>>Zj8`5^Kv^l4Tw*ogjG$Rl!~mbAIeQJ6HvkmS$C2XEf5u};hdD3?C!hMO%3*D# zv?5>AE^|XnE_iD=6gPtME;zt#i=Txcs@~)6-D>vQI~X@loE$OzwqI+$y)NRht+?Gr zIobyyzUmhMBun5JbjRxO;p+^8F{92`Xe+huhnry6p zSDV%ad}qchx*^->Z})!ggC|{|^S-B|*>mk%BC>r5|88opioh@Kc{B3xHUYc$4 z^+p6lHz}$W98N{EMt6J2XTjE_6{}b5@e)EfE7XX<5;g-e4D2Z)463WN1DcmrJw@tFTT42&3=tiNC zbj>$E3}f+P1)2Ug);EZ4hzxfc9}NNTR|nHy1Q?V^x=Lg(V0bL<6KtPCf{t(0ip+~i z&OckIM*seAXUnBdjjp&-O+sy045Zi;G4h-rqYjlE$XWrbTch4l=sVC#UiT!S2#QE znS13evsUIRdH;HhtlgA^B)JYeKw_OM@2OAYn??HNs}GtSOVeLmQPNyWB%hO zdy|Ofu4c(*qk2~D4QrS(IQMFAj(R)$s8oG>paoq8R6w}ou&}C#Xahu2l=kO&Tq`zvIw@`={lht8*w4%M=0Y4+{oe#huH3l~#QI|V1G`~2(({Xm z(ba^;XFz~V3Xj`WE*r(c*V2MUM@4{4u9T5ux4GPm;{DTn%moCt*Tao^Mj|hN2F*FS z<4B{GMa^v%{mXYTk`GO4BeVC5DBOH+XwLwK+A968Yk6;VGfBC_p$+ot+`BI4UdacNz;xvwC+y4fTnn+Pv0~v<}L_9pF zJm11Jcf<|B-)IRsq#G`WNhx0&f|>W+08Cl7IT%7)b2Uh)tMeB0qXvQW&X3R2osNz1 z`c2`mZk8;$rIQQvtOLCLOXK=YipAr)4=xtVWu6s6i#g`H0LvzGB0O3RRP_9c8geP4h^`EzB=Jgb;smKYsuG`z=Pn9*I~W zbrKW8baH56Ae(MyR_uu|GZOu+t*}>m;NT9ib@p}UT6eKZ{~R6?nb=;z?Disq-2l)v z%Zk4uh1x!R6zbqqKFloV$Ijv3OJqzxY&n?Z){6!E5Y$}FYaat?oii2Q!M|WhhV+NI~*GEGWztYw=cNotLzA?M>!FtTQq+5G#vbc@LYxwE9*} z2ZO&hh{^Yy8k$Cxp8vDlZ%fr6Y_Q`T0fK}Uit#PjY00m02 z>w9x{!gDmm zZbO_i)^!`7<+&Z=Ry2TjpXMv$<@zfwv0tYDeaYwTKa6 zouYDX#uk)Sd7;(0Bs*dfWFCxdRUG6?$2P)EHxE+R5ZGYvBug>Z(Quw6axU{(zcQ97 zzZd6e7*BBlFjgH^GxhcK+_awS^ZIE(6M$Zk8r^4_1_iam_45Y4cQ2*eG|i%} zPYXsp88I`z#IQ3~F_7VRa3;UF80zz#eCY708%7Yoo~aEIkL^%x1nr?_=z8| z5U<1&ZjAAJVq8F^@N+1o@t!{dofpW;zBl0B?+;2@yrIHy%YtVA z!cleTV$mVV%lXE9KR$5A&>#w3T2(1?T4eVWK&A2SnQy>Qh1Pn9$y? z-9Q#q&Ld<%%8z#yNxC3PTIt>{w@oJ!-R@>Qw(_LY{6!u61eR=T<#%YB0Uab<3}lJq zn~2;~oUz?0W)2eE9;+JLH92@=y~}cUgY7{G6$u*Hp}H276q1&3?NUgiU}zyt06#xG z+ga3Mjd;EQ3(N{$x!p6%)gYJ>bVqO$8C3oF)BINt|29S zkhv^8tHxMho0W!5(q#KS(yv{hcVg!^B#&_1s2O}UZ@F9S}EmI2Z1<4eFg_6#kUsg zTdVn=e&S0%cFW)jq_3`ED2pm2iT(nml&V4kvI6PDVwdP{0`|VXLTEL$5*$?tn2$_o zaZc*`$*8Xwlfk+EmrJf=8NjUGZbizI`8hZgQekizy3j2?epriLNQux%pIoL2p5?Ln zh;8sOTZBO|FJG7XF7Wpa_g-AGJTNr7bR61Ibf*n?nsjD0NV!IF5dY1s1{fD`8twDB z=Oi85gE#=X-*IHXraHy(hI0bz_yG09*~C8SV_hs#HqCavf8fgv`>E)MSMhdS1jkaM znV;*TZ;u}%K^mw+ae|Y{wawsoh55XmI#`M?-zOkAI&C5d6kR~YDP!Drt?$|M+Vzkq zg~zl^tFNEkNYf(B;W1U>yd@VhxG&;2y4)brz;#emS=Wbg#ZCWV=(5r}U@FNYyfdL2 z$U4Ai-A(cG_G%yDK9VQ`0kX%nO_o||8zgj`x*MYBAG8^N8D616zL0$!LU0GTGBW2U zpbSDnz%+jVvfTljl~MD&unLsQf>5pc@VlsiaS!GP zM~xi9A($mtSlI9~9cQjUzZzBO0pky*RZQKchrQqP_$2Z-}q)mHZcZ2%xos=rUt0Tx?+ zNe5%K;y^x1Pq|EMR-)SS4uo)A{pCS6O3DSZ_Mg6J7D`2PhiNs)9_!Gp-5>=j1HZgE6= zJ0UZ$V)BYl5Sx#H_oUDOh8VFcH~0{X?Hd|8_hpK1=kBcpi`<*t=uC05NgJ#q>~I)y zSCMouXzZB@#b7X!lpn>ie0KtgL(iwHpU13KpZ@UlUcpTLOhY2Wg#)-3)VnR)+|5hL z&EYdtvH}RWL-~F(t*U|58<#8W-Y(u=wKrCrqub4Rva3H&0`1c|o&iGS=7OEau24V> z&71}h5?a3;E7yp_ULLI^-Cwd)s>sndN+{zAhayI^q}L?9zxiJ%%g`kaEQkLwfYOGj zIdOwG{7}Z3^j9$1tk+qToF-y{aAq6F8{P>|s5tyDkLDg!-X%m&@0{Jnd*L+qSFTq$ zJ;i(YCt-**Yf&@52+tM!OtXbT5x&QNf=YO=q8M|yPsqqv(YE%IS4IxV+?MSY2Zt3G z6=7<|Uae)uff5NoaDuUcF;jzOwWfpdJ|#mEJ&P~epNy=RKy=GjFxJh*0p2-X=*?~r zffRQ$t{jjWZ0AJTWYtU%XN`>K<43jZ(JvY%byK{hmM8C`Q(%6zNW2LctmuGOz;N}u zFo*?VUDHxin=us6un#uGX8tqL_ZlyRex_E!Cv5h0AY(sEpaG2+5i~1o?=)73t-s+Z zUf;vr0ng4CP&%ByC;G~*TTJ@%Rn=|~%PPKd$6oT&vo{B8YZ~eA*B@TRjyUbr?k_;J zrz@{kX`9&FkDomU2Gi=A7#LjtUgfshKLXqd=oYKd1~k+z`TT;}k7GE1eg@melITtO zdli%=T3Pm$X-Bt`oY^ZXn%5kFW$Xy(K#}P;FT{=sRfZ;%y8dR8BgENrupK)|^x~V9 zEz@;&PTPpPcCp<4?_7Y73kr~xmF&?f=UzuGJKc=K8@X2<3}a+hrhuCp;S3d^Ak$pg zNncD0&;&tpKACkhhgRcrkyFTGX{YH2u=TGIkqoT~Wn4Yh?_J9c6umwx)U5w%oCD`o z3^XZ`Tx%B)U8rCk9i7r&14;Cop>yEDd#M}j_gPk(P1Oa#Ie4l}KB_L*oa{xwWJv%- zpsndl3;JP-;s1%=;jp#f*4WkVk zU|?;G*4!3#ji(yNZ;O8<1(7bz^)Jx8l~?n0HZ=SM5BOf0;P;xdhibfu&+P*WB`?&- zy;<8)UTHQohxe8lyA;VK!3i#Q`|j*~a`Y*Z%T(|fTw^|tSgd0$I{HF_J6MslZJhUh zvXy_53c!mw;JUT@%rD3S{_m6Hovs`W=GjH4A_fM-pCmo7a`j3YdwMV8^b zheL@~3;}kBZg_+BKOzK}$k_$o8g)jy3xonUFghOHTh7is#8@VZL_hma0fqS!qv)T6 zUc>14VvNFosgg>u^3li=+$YZ9`8FjH=}p|J$yBCv-hl0;*u%Zox^_lH?dxzCM@<6Z z?oH+Wd++vY%>(xNJunAL>C%GU5f)g0g+>1xSV8I@|1k(^x?PfW{7g!Cy@A>pV2p698f*~zYAVG3%8UWw=qH}Np zLonK6acK#T0>A*YnoRw0hFlscU$}$O0Z+16rJd6Vd-z)5Y$?CN@j)4eY=ekQ$NX(d zBgeeD5I+nND+;YEJ}3FZG^dy(kT*pC(ttWtlB1{d9A=2wycYWc{~lE$flmE~F!g`> z&Ta#+`moQ*wnrKJiEXLUOEyb`*N&qULfBd5_o5j$aeqYf)9_koTkTc)E_O_^%K_TP zCJC%Gd4L8giOh8m)~}bg5YZhHM2onA4PBB5rmyd;H>!sJUGC^RmwLpQEnQQPUVgUS z2Hun8VOr)>w85V$A&KZ@#d>rTgN>_Svb%EEf z%ghgqr_ICrnP*EVB^Vh!0vla78W;%ksf!wLI(-)))IP>)&?q5bs=}i)KIg?1CA$++Qxg@^aFOupO@$u0fzDBMZK?SWMy?Fm3#2)|OvgKIOp2O86_FVYOB0GPc?h@1VBn+t3z4Mn$y$sH5pSd_8ez|e` zG3>oe-Tg-OPzwc;%Q83Q7zyoQ!JCb^zP=l&Usl9;d$1}Q9}~ewM#S(<0U&|kOtSyN z3}Af%6H7%Tp09wke_qkV01;2|t`R3q$?T$G=IU`n(aW)QSM7n#`_k0-5}MJHK*SfY zJf{th-`v}~upDh}YH-nL0F`LTiFMtOyOhpHP4|qsn$KLr z`p>sgc^yC`DM9E;U6=4gklt6vlQJg(2SQtryBqzi0={${*Ju314In1;chpBBEK@Iiv!QU{O_MnLxFo&nzx_-?TNoX{VThK&kTqU{{6pyepAT;>VH)(-T3#_ z{`#JEF%`9ve@l4cZ`l9yD{R6*azgBzlmGjr!0U%ZR6@E5U;nWEKR=}dpkcm}P4jGUFp#F6{|2@^;i1;r7|HXO#<<&nC@n5m@mr(uxCV^pWzMH)owlnPsv|Y!k*(q|K zb{CL8um4V`1N1YX$$$SSF|h^=xKgi$K2?@Rb}`+K*0Xa9_n!a_CYrPiSpQz<&lQ1; zfkX)_aV$blpa(;HT2>_^_;B|T$*+wJ6hO7HT<32G3NvE%Y5)Bxs!*j+phxEYSzwM? zNSLW>TP=uj^{VHXgZIPG?eSo%<1Qqs8Vx|HQt9{8pdSh%5&x;WVikZ)(Kfy^FR^Ql zCi2fU_3oZL#Up8>k5NAO#VYC8_bJvsi}yeEQX6+NFc-e?$=7Ep8eU;6EC&X0R{`Z` zI;H09DOfVq?h%;9I3-RnRvswgmfdd7b-rz_S!i^c5L21X28Yf1J}Q1(F@FI@dZt zPSYLuF!f>AuO`x3=mN2rf7Q0Y1sbN$yu|ujNM$sSl=>Z)q1ZQD*w$mMQhmAoQKK8b zQeSMxx^WivU!Mw5F+I5q6f}pugo#=ujG2d0S$VWXC)!_}sEYlZC+N&2aBTYhLp8_7 zBG(h3+Pf2&QlZa193okhqZ=Yo^IW5&jP1d;tvZ(#*XQ%^{@2zTmg~-3RzEfJ;_?~z zA0MItL&Nn|DF-lk+3PurrwpFoXXsnoB=+djf8tT!iDu!fGp_d<$Y z!c(JDX#(=y_5S1g@gGC3p&hm|4iQzrPgkUBskR4+9-shzkXZ&TH5l66If=yr<1|Lv zR&)O2Swa09G(Z=*Yv^R#waI2lAT#;Vf}QDB7?qXjX%O7!#D&03gA@hWx*!{6;BHs| zFV+9rY^)bGJO~6Oj;Z3$0j-R(@LnGYI1d+P>Uxj!-g^ov)_tEeOFd@VRc`?P2Z-@Q zb|*{Ne_+Zc)JyxZV`d^ST@mO?q3yBl07`Rt!73=&HA0)3Uj=lkQ$4?T6=4NTC;GQj zp{9!yUIm&tfTVDp-rMC8$CWjtc>I;Eva!*J^j0spreD0z%YMyAzcn|<-13`0O@EwKfj=MnocrT)HyuUZvE?;`spmW6aPkdAu5Cu zUUiB9Zi~;qXr$JC5py+vUFp;bG+=7JH1Pc63S&By04ufrMl+tb$el?P{V_tPc}{cW z&A)*jj|kCi9w=07zaDP|A#}34^|UY3CW|2Pd?X#C80y;p3kJJQct* z+nD;FY&Wf|GHxsF-t1PV_|m_|vvTbr-Hyw7&XLDnp}QM%pax%?8@!Ci!mdn};eXNf zGbRUXJ;apjP{~CIyO$Z#j{KpgG(wCdUD+1DqB^d0q#=zt6vHuuRayw=HH(qnj=SB|JE9dLk;+}s@lXYd z8<@ZYg5)0AqHMRkem^(=n=;@g98+cPjV$M#_`6(UmmT^AbJWzQc z`zSxO4#)?_h-dnYI6C5{M9y45UAzSFISU&jTaSbeytDwsX7Cq%G+H%M47867t**G3 zHQlB4-MjQ@m}M{K5c>C8;x_>;F$YeJx}+WWX@5^6q*&o+ish_|iWQ@mPS``f7t<4x z>Bm`^0Pn}K^>iwMr_GZ}gg3ne1MQsk^$?d9l>Z{ec1;yp|2^UtZC!(Q4ItTg_r0_t z=0ix{y(MO5EX;$h-SPLHw9Hh({DEQ^@dsNWcC!nt?x+yiKTjjngS{tLY7=a9bbT|% zdBzU%JeX;9SoTKw^)oc`-Z;#u9`1Kp7}>ME*LmOmMz~4_-KOn5Xl8c{bA(ca_jQ&$ z!mQ(}tk^NWLMmudix7q{$O3~QRX)il`)<+%oOn8WX+|*Xn-Y)}As+O_@Kuv*kHKm6 z<4Y7ES-JLv7q}i^u^E@=rvqn?HI&VA!Zt*Wbe!N^%GLvZQ8CYIi#ip4h@dCCm~B*w z(l<@&o}DQ3ec|vwO2ir+vOkrGhN<#;Pbt^*o;G$2g8!&F*kJ2qK zu}c`&o`r9II)9xaa9i``m|eYX7Jd`UF?RItQ?qP3T8n|zGwBIlDA)MxxpW0sY1tL^ zyll|+;_k5G&3EG7f2}9j20R|8;P*S-^iKo9LIu(1^b>9cv-Y8jr8}b|761N3(3L)IiMRQDNuvj6MRkjCSp`7zGF|D>Wh zaRw-8!3+WYtUI!nc6A1HlE?@K3Cr4Ko!ciRoh<170w@kBtCC>KxzG3a*@T`w)1z|} z7`UVKSq63MCotL0(9jz@fTh&~t=zUZ6aPoHWa)(fjhz1Rd{tCW^yv%?1UI-0OAePj z4J*sbu*w?;s!8E+k3UoFe(#O~ps1Sf9|?c*MwX$sbVwB-zEyDpRGF?n*XDC#PVYUG z>F)_H9uE2!Z2&+>`wxU_96c&{z)rhYWqmAE?F7En_1HZAo$dzhhc|Lfx7WVmF6@ zgP=VNUU)?oDq`iRYYmEH?tiFG>k;sX=4lc5aR})5GjUh5mX_*Po0w^N6@iASz21Ar zvB|##a|)0h?;i0#e=>4_8`HmZ^p9n;GO_&LB7hFMJbi&eUg>$u;?vHxRu+KAkA?7L!bMl5&4mMU_#97ptif8F-D zWzDha@;}(#Vy-hz7uZ!Vym`W#NU?PL6rzC9*)Hg++j#p|O$Pnn z(wqa;y|StOsr>U3RXoENHWC)31uqjR7V#As7x(m+y9%biO~FbNtaG_1ceYiO8#Qt6 zAG!?E1J_2iMgu(J(qy-tA#eFDo*Hz9uMsN9HlZ&9Jc-y5zI62V2k`)hZc|^|0ZfIL zQ=G@nr`tS#%J^T(%l0eD&9z0F=8CJYTzwSHPcAQF+ZoP0l8*}%2_1D%@ zxg96=x3@VDc@3|*rg=r#hxCg3%@5hCoiGoYd99yL%1bQ6M-E5rs`#oGFsd|JKik>n zpJ3|lI4BzeRTOmK;A8)dPWz{(qdolg*cVn(pY;0ij4N#pYi%5XDhL@AVYKX8VcGFV zv&o_I%y*)Aur;6=(n@!Cg~N*Nbfxt=-i*BeBM_I=TMt+E%`Gy)e%uWBaQC*{4@w=< zcXmUr58X(Mqvl46ojpE3j(Fjkq$P(~VMwdlZ}k>%eg=o+Zn8e`np9`^EFBy}g2vz|gs$yBj~Z3ZW3Om&{$w+W6%KJv z8LV3;2rzoeII7oteVe4K+?`(!vVc3&UP!JQ_Rmz2w_NJqpd1A47FUL3`U%DJN)P~Kb5|LDDk#l+&CH84Y#*zf z;Jjt@sdz*;{R-3Ty6N7TOeoY3FZK)-r8KQ8>Y+<`FIzQ0S-TFm`Rz}EJ`MF3Taxo7 zt)h@~4?i?}5OQEnBYu5vE76fsk;6(g!{E(n1=`P^Lyv}JJP$>Ze=scG;&h~y0Mh8zgl@}afSWICx<%n$JWCef4#i;(=b1; z4KYl^BAk~=MkfBvhlI}9-zv$)6B}@_K zN*)QLLyj^_fd<+syKD3tA2R~0ebDnMCa=h==IPgq5Z*{p;XitEvCpaxG^gG@Fx%rF zQ;vVzH%4?;nDl7KD!@Wr9V45+H3Sf_KUV=duX0A=YD$ z24U{Rvfu2UotnP?6ZYz2kk{*P&z^dW>KDE^#Ylautqc3L5ZA>Jf6V&l;y{i;We!?p zM9*Hv)mDc*DAqqCj z`sm>v4azTLp9D!N{Mg(Z6Ngp0Wbv|4$8JP%f~tBbM-wU~7^&KPPwHGcdj<{CCIP8@ zSpX8O3T{_GWRI$>KIro~$8@BR=WA*(Ejkcn5yRXLA-5yBX0X<7ui|{rQ}xM@g4!Qf zA&1_3$i89dqxWD|R(h8nHK)_(wO=FK5<>$&V=?=auUin_f?v*n#185w@emjEW2H7m z?ix!KFd3A?Z~@NQ;@|{2AZqB@ox1CPy_WXe0Nkm?lTJjouF! zTP7Z}WhK3EPtrTMbcKFC4T|>bt+jd%^sH*;;UjP9qhlNBGi(Yydvt|8O^WJj? z0K|&H<$ZIdCJkkaHzqB#a>~muEqL`ny05n-`f1(L<8>M(yEF?#-;s@xNBv!Ij|^E# zHa0dcCrumYNN9veenwokIpC0{5zspw$1m_4a1gi#*%(0PBKlbrm;CRjfcdYqY%ap4 zxkWXZ&3AjJ{oY*kdM>SY_YePJ9ZIDglKCoT*J^g-hrC5d5A0(9MA!Va)W{do)j#i% zq3)o9*ElL$C*#)*wVck1E?_h>dQ8F1<)XuC-k-~@A41=#JpKgwq}BEL+el57*n5_I3Y!sk6@ zGA-BVE?v9+0-N{&XT8oCC{^b@yWjfZ3H_a!A1;GmIoP^A{{-(7tv3YWUx6&Q6FjF6 z7I@wP$exf=gXi&IlCBMAbZ{%kuQNHG>{iAN)Zce5PtX`V&Yx^Cb+~jV3|aD6eqjaJ z2lDN%#Nl`aOR0W|#`$B@F~4_!hO`+2K{KX)K$mm6vUn}h(@10QO3n6L{t+0`s;EtIfix&Xg!P#)wl>_{)b2s$#t<#rE@G(HqKIDK5WAOF0^FtFW% z2Y}p;XQ|&hL$h$s%Wj%h+;t_*o|AKhANC z^wZ49;(T=+hl^@XfHy|Y8SlzX*z$l}_vDw(QWD>M+@V}8i2dKeTMF&OUl%7(DRh4{ z?6Hnh571vL8C;~gY22QLT{Ox)ey9DVLl)hgQERXDd)K<&WfsnXOzCFXR~39J1~(wpq5Syh@Ym^0puDme+r))^2b9E!S2d z3-PX8q37C96&OV;$OJ)Ag-&6;I)K>@tJK#VI z{MR=@LNSn@rTMzc|ME>5IuoejYf8+ifBPoL%mq+s!%Tnp*MT2i0PYH>>!88;zl^N9 z1gw(NJt(oi73e?5`2TpZlF4F7BvQ-RxaK#QOy$F^o3{_3auGB*_!$QPIng~>_cJzI z_nt%Q_K^Y87j!i!i~T~})@%L4LK=T*w%2b%cpf(V|xycx656Jz?H^F(~ z<3}Ako<=f?7ajdyZF^#9cgeG}v5A8+@D~wVU;mg*66ZS5y6S0Q$W~@fW;IA|p7J-Z zNYHaV;frwt^e~=nc1|4|9u!Vbw{XzSexqbzri)preC0rBi8!8s8b$TXJ32YNt}hTJ z*n+BdDiGwM?h*PrI@I)$`}v^ySnFtnEx#TO81IN`^$*r=g6U2Daw719pNknH%spKG zv9_k$>?O6bIwlROh>vrcT3nEpZh2jZH8ZU)Z9~6NVqX5~i!QqrGM#=w;F=$fVf@z8TODqueyFyso4~XJ&pjmj8w*Jt8P8)!JSL#m+oC04mq}+#5j;h=Jtl zLYpsTP=%K%cp*oA-%NJEtrMghPB%He%b-8>(}~7*9qcE6d>zP$1XrXj!97XU&4JmT^SPB z$#dYBop))x0x?=(JL>wyo53saB~`^r(v5?ZSgx0SFBQANKpl}0H3@0JPo6AQXtRNPwKroUWajV${jO-oVlR4;6jUvB% z$j)3~XlyKx_c=|_|MB_AbVKTc@AZE5C=~qDd~@O;&dGO(q;-1=R8VL5A9~IxkXuKk zn;jGiWA2}{r+Pj?fM{=jjr>^AP>8X`?RIaT#?=1)$GBFp@uE^A~#^cVlRcn_M(XLYMMa;;RK=KS#m8?@5G@JqeBXhBkdH}2t}(b8i< z6szy{RpjvYo8uRHwBuThmkgzQ%Cv%AlaZOC@!u$8OSuZXb&wVN0hC?J$udo+O_l z$HtZ*pW*$F!egs==i!Ff9&(0ks?P3Sz^B*dD8&oK%-wPZ#pqeZ3;R5gqiM0;a|6SY zJjb&gHxc*PizpQ*#tcgj!6AiWX%ynKUJ;!LqoU*@U8|zy(ZeTwa%mWG>|2@wBTJ?#UzlfuW=33-Ut=BG}{gtcqcO+dImGuh<1g1L@Ct)0T!i z4)(69-9j?#ojIKqH!kaShE-jCMU7_n>e7z$ER|%qN?@SbOHT2PvQSF=tv6)oh-`ds zhDdeXdq;z($4bivBI}z=Ea}g7FT@ng;gl!F#CRJ_i__Oy`_89k z6QAgw4k}nJ(7J!(EqJOsb21GSOyR1NXD z-7wee^q%cxEfy@5^>?ABfa{WkTdDBymghY7smfWx{@)&Z;K`L3e1Bx$hP#Jc6b$>u@$}q+xc4e8Tq5-sTdTv$3(RCxWpf*Cl|O7D2)RP-y9n69dOCeS{RP z=QZk#Z~fnMpY(YRW(*^{*SQnsV^&3v8vmtg=mBzB0& z(cu1$$dY^xmcR%}X5~&Ya zqMeeSxS!b5BQa4t8&xQI8=v>Zybx9 z*~U`L&7=)3<-Oz7J{89Q1})6t=*Mj4i4ukpPkdE(I)_D7Ie#7wS0|+5# z$BADz!E!+ngd((DBkJUX$fpx5K1V{Y32y8N7uf155d;OUww5KKh_KQEjqu`TY%Ug2 zi)z$K5ybtK>G;K_-m5bV=}QjUHzW)M4jIuqg6B^Vd7^i3Oq-MrwU^XxVG6>(itVVn zoq!}W17K(}ec>50FbI>Z2-fT-q1qrYx+DdNN}`iYAoaPrEboqrSlQmj9!)rxrHX(Hj{_vdEq z)E-aGEf8WKj1ybn(h`s|ziJ5K$A6aTeH+X7WYyAB0p(*OdFd@fL(_QcvS3;U|5=A% zx!v_%ksjfd2c51735?{wo%)oe^uzW~H)w69qQ! zWPHy*=~d14u@quR*m-TD))Q2p42p~!UeYW{?45sxlw5i4Apq7N!U0L@#$8|@t%vpu@j}C^95m9;dX@dT${(8K1&Qt1Tw@jt3JTtR zRGo^rr)(BbIgEBmi`upH)Z{K$*3QldJR~3hijj6bl zM%8GH_p`lT^;c<9-n?hL+50dk;3%Z?NtDB2=bgrhV~rl>jNlWW2%4oRpY@mPQBMC^ zoTW?GToe(>(QLG+Ug^H9tTHt^CXsCzcU0Q{X=GUjN?FXP1aHgQ<{_S_H6}7B+*B$yeKegu}^k84DJ0n>fY;8#5``11`eYVNf>6~4|<3SG4bpH zOaIy5U@*h(E31L2g8H$#ok1_QxAM)362Cdd8@}biDk7wsC~*5VtLVn5Tjh**c+k7M zo+Z5`>#XB7+c&o2Y-)zyFGG1A&I;Wksx`gK*bXp*) zEI6ZyV-n{28daOlvoeIP{EE^d{FAjF{mJ{OSwe43Ma`wJoJfZoM*aqfpa~@d^rxsW zjazt#;h^*fPwfz&p`B;oL$f7qz?SXr{iAaMzsNlB9_E&h_q6{;St6#WwXO-bVb!cdoOSW{0P81HDvqZjJ0?*cHz^c8VKvB&e>WPh7 zUwZ^xM72hwslyquzM43EPC=VS-uRsaJI$wfmP8_Z;llhL(yB=wyCdTrr&*aZ(I3o~ zVw5+lb2_x#iCyv2@K~8WV_6~{M-r>UUcqOrxPD_;#P;1IS@o-qUDFu*8ln^L%e`>{ zVFAjm!YjgLZ3jnEvn^&0bfMbwG zkpgkbceU%u&3ohh3x?0iD$31E0tV95%9j`VPg>9S|1fHC`N+eH&h*61HAo@M_%2KCYs!{4pgXXeBmz@+mpIy!Vsa4^2 znY_aJv7;9*G1Y7l1=9fqrZo5*>g5L^R15!Lpq#vkQ9 za5BUWDPzlR=4tZDGbO43n}R8^uBM_eh1NtCk#e+6K0*zI?tl1A^t0WVm|JN%$J}!& zEUDAAg?pYi=} zN0gSxOB9$f#AuWd#Au~C;zT_-@{x3QjLO(`i$}+BS>cHzxzF~e^M@C@1A_u8QRT%c z`eg#y* z$5xLhO)X1`aQ-u_FRY&ZuCB_S>drP@k9UdgR(hb(CL_a5P2XFT111N*Eh~k0J?fDy zBp6J<{u#I67`-tCh>7N?RbLe8$Z18bWfgK&4{K`~7p6EEq@y>?goMbR2%#U<_%-ol z^ol*cB#~kiS#~rt)~h^l!?}H=VkJCqBWfT>W4zK1enz22SAtv_FR$Vx=l-I7;;cNC z?I&Y^6fW|fq-^*?yI^TXH3b%A)37&aLizLv53`?(?#QTXc~^;7Eig=r7`_?AI5W+K0cTu_D{DR-AONd5sw+Sw>#0S$`W8Xrpo64Dn@9qU}{>ibM=Ym#Vci^3{Mj5M*v72LlQ*B`0o z+T$B*ewqq5D*(5awKJ{QCyfu%plsRi;n0>3J>&RO1K3|;(A25jgAON@mCo1<$<{%Rxmgrx{Ef0&W4M^rLGsCW zpwvp%qhnxHB@Mb8VBq8si~yQXM2;QhFy zrgLBzIXJK}HiX`{Qy+O8`KHsEqcdmhTn{e{2U|PY-w^Nf(vWu6(r7NBbEQ%D{$;8g zBtk^B0XF3^T>>ne&@7-WBGy{#-~hHlp+=0#7t#l8i}5lZn(+zDW8ycojQjd!$$4Bh z-Qx2ge<;cFJWnZMzn%P5Hlt@}&(AwtnA_`wL15B7A2alUsW=Blw4@}v-=?akBmm4& zP+zZ_xlb2)DnPSRB;R=cj1!{)Gy1hzim-A%viqqaLBzL~Xxn~FdEpIuclGU_Z~eyr z?^$QF29);k&-k?z^cy5Z>(^s|*6sN^4=~()b+A4BN(eI8tO!26WRiZn>4iV5w!fb~ z<>x5Mk~ry@f$a@kGrektK;O&?dSSp_z+iA1p0}od3njWT-}@+~CUI&u&C1@lEOGUK zFDK7~@qNx6@Ca-J?5+M3FN^B^Jh}l!+*M@LzQQN^OzWeKSAKL~bVd{Em6i%(7JQ$N z^(~JIY7-w8NmON9a}dXWR2pZn#wIpaXy%Cpg*r1|@Mgu37HXe*!H3V^f<;X7;1PbLW zhvWYB%dsCCt98Sv`+32e!4Q%QvhV`91jp7jWK z^CiX3$q$qjV=vTz-$FmmVq$J?<*&`hL1H;~hx)U-cMeqT{V!k?E0tg}FM$1-=QK+$ z=XF!I!L{F*wN@)kS3_QC`pOh$W`J3SE)VhiZgmFyUPR$!H9X^N>8f}w2cQFh7dRyp z_Cj9qf-^Ft*#ej^NULvx(BCtTjR?I#bMCM2PMj;Tslv?oFKC}nMoZLD;yUo7K8`Q5 zbQ%n|C-e6sWS$OGSWz8nG<3lLwcA>V5~?qCQ5+~h%r~h05MQqj$%RpQ;RSkk>4WIp zs!4U{&hM6l1mxBpS@={vq1r#-Adav&1QylAKJ7=vWzR`qT-``_f|U2d5pp64TtHkJ z@f*qEIhcaa`o1->{~Q-quc)Ox4U2U;yQ-LQ~W{Vz)w|1&t1>clV4S5-ud~ zT8KN@srdP?_^(Ku>&!w1rz*UHjBnKg7&XanN2rq@4r zdfRonESQh=f0EHA%4wS`@WMSC1A7A>nL_*onon4@0Sgm4N6kL_UIvs#%ZVrXP!^ap zow3kHB@z|uOK_e`A|^&&(J+q{cKIuOH1+oYswGEzcuPRFVXg=BAeXxO1#q2PSde3d zv|5TPB1~8NQZ=ENFgdyXJyK0zY|}`~B`(~i#=`%77xfm939Mja*H&37AzPAQsC}t# z$DwV01-|WCeorC6k&qP*d{sesM9P1thEa$r1L;K3n59OMyO)dR?^ z!8->cM;=~rpSkBhlOv8@`1*(KL3m*R(itbIDB_2(y%=A28{qISXsV~@NB+*@r1JLH zk3bgSu1PxAc9nW>qZ0a$A5R%#NE1~D`v`dXkAG3kY^M|4@Y`8@r2vshrS+db1UGz+ z{`0^8{q67Nz-_{-ay9qAGqHaS!Kgg?-}C(XqMx{GSZVu-==}dJ9>VHN-#|k{%dc_s zngK+DYFIS#*SG$Qgiv0l7}DHQi}mckQ4B4W=l}Di{=Y15=d7DgZm3JfNf{dON8^^x K&64Z)!~Pf6;$p`D literal 0 HcmV?d00001 diff --git a/docs/zh/26-tdinternal/slow_log1.png b/docs/zh/26-tdinternal/slow_log1.png new file mode 100644 index 0000000000000000000000000000000000000000..4dad9ec8d8aa9015fbc7dd93bb7c8c345f08c56a GIT binary patch literal 457097 zcmeFacT`l{nlB27ih_s)13|z9ih@X#A}L#ffvA8W(LfGLC~^=5MFBICQ9w|VNK%TV z2y7(hBp@IlsR}G|QE>`NiKC=3KMxYO1Sj+040_ zhK6Q~n(80g@E;!y%|>m84e%FSLQ+2bN9&}ma*`&aj(e1bhL1+=j}zzJj3;_D?$~yE z^G-DzU7)U7k@xwB9Jwah^vU(@gSV!IRnK?Y9b?@PwCk{gxM#tm9w{?)^+q;*oip5GS;L4Q;vy$Y3)PivhymY?F?D_fBv34Wx zv6iBFFX88n&h2u(@*^|LQ)hkq90n&QCJwtXu<$AR(a_S-|93weRWVhfsVe#(KCNi_ z(2wR3;l%%FYDK@&leDz^!UO&%bOX2kx9R9qg*W~Wn8rlKxSsyOCFcLW#D2+n`(_5d z58U|e&(9t(la0MKIhl?NyGnzNrm06en~@=K*$64)fwp4Nnb%f_(Kjineja-5kF$i) zh3dih-lP`s?TGxA3yMXl<5l`Q&_yD(Ot z{XD`?Htxip$Af5z0$04z#eFP99yT@`GqK#W$cyT8u!nluZ|T z(M2K)h*ofzSBd6eiTe=??hs|9yQ)#E++uYLF=s+g!=&@4ii(OPiLlDS243#^p30&C zYNfnM^%g^RGypp&jl+FH@*GHffWCz=5c)L`WT}}5vwzR(7KeL+*)&PPxEovtKm-p# z1P?3in}H@lb;Tkf;-7ZULB6Lpr{J%06KJ$|jTAbD2Mi7#?#jL>|$m>zRk~_Xl(IDL+T}455EU2Tv zcIop2Ee>%RT&fIzOiYZCmqS;lpwDD~zt-$*y0OT_gl*9D71kx{iP_wp^1=N3uyI*Z z?tuuxAHT&MNDuib7?V4INnwG94m@Lcrl( zG#@A35cggRII%OVULm`vh{HTTVS#)1?hC$srZ7q-8c%LnF&Y)2K4 zJimiQ-4K*=V5UtRj_LfwQTevPO*Z4CN_Nk^0Y=h$yyLX-ga>7fK4g|i1ndY&-P&DY@$x}zEWN;d;t9PylwkaVmCKsrR#!*|eyE_hHnP3r?p zIyjJytBUyv%mnDb1lFenVkXc9CXkC`W!!`K;s6s6Qnc%QK&QGzzM*P#)cu>KIJ4+^ z=N>20H$Ge2XkT+_?%RWpACLKxCW*J~^9fU}KE5m!kE_^lTrts1WmS&)A=%kQJb8J& z#LyN~^D_cySWXyLLYn>zCcCE~J0Sr{S|o`UMKH6E{Q7k0BJC2G+3~DRh^_byU}jsG zKeA#{9RjI-xqA*%g9HfA+L2+4;Y}LtGUme@LN-(im>r8ad}*&eJT zKXLAmPt()xG^1)g!RE<{(iYt0-j6Gvjt_c-E(MHHV$=mDNRyIh+iqZ1tq#-0sFxXF z(_Mh+-dqX6FwTWllQibc*>)0{Kq_45?zg!F#ImG32HQ<;e!CrA=syMZ9=bS)uuL%q z6e1I&Om5rb*YZ~EUQg(T%F3{GnLr zwX>V(xt(PvERroQY)sgBpkV29sPfjWVRa>acl{I8Td!O8vMKCPDEaW=qKizVH*;9% zr_s@ivq>H4mag}+vsnTY%%qR+((Q|zUnyKP-i1OQU{R*T-mNkIu;NWimbN{96kMZPa-nd2N%JU zn2f(U)2T{G&ph8FyNg+A*}r7T)^7Gw-Q1jZYP9RVVE&|{o}M$8Yg|tR_ub)7Sn#z- zPBy!7m4}Oq>qYnId7}pd7Te{@8U?v|gPSr;K zOTQy~s79Rt82;@$vB+g(q3xVbrtw&938XF(E?kky@>$kL74-kO>(kWn^Ja3b}`ok^0MM7Y4o~*V{zX*w)F5MzawZ zk5FbP0ryPndx`2lTt$Lz4af$mQ12<$7uFw+^pvdHNf0Ox{ev8fp=WYyzIk}!DatPH zxLX~f2j#EFTt7mNT`^^;7$m=K>LuF$a2>&)F=(stL=->JP%P3R{p^tx15KEEL+;{{ zU)*hR4HW+!LUs$mE5G(E+^8St=aJl-PMV8|09`E7x8P=l`|QNnkR5}PUgO6J-q%oY zm#3XoFRI3OH`VR)b6!_Z=ABaGkO_|Qd)0GuAQeCQlMG?UdalFjhpDeG@$@yFyIHu# zqJYIX!qU&_X|g57=|s7Jt7>q}qYTei<3;v7qKWKfr>8rEd1CAvb4NF* z^zX&weG)X!tyg5&zh4Yx8P;H`ukjM^0}&|rBOC@;6^THAtGNylmikd5haPxCNUTHB z^{G0GZbaw}V4U${KRAdfU@-{c=b2|#bhL@<8$_dZ)`wO+cJ%P*<+tq3F3IPTiRs2K z-d$yJcXK3T~-dbPY%l-A}(Wj%uQGv+1Gk}ILlTEXAVVmHp3zO+2q1E*dps#X0 zh`X+SJ$!K*fu~YL;+3&4n8IeD`;%qdZa)+J0>KS{ZJ&;mI+th(EK=dr+qMg@^}G9IZiuft&pc^v|;#*O+?Fi1JUPP#-PNN_YiQRbaLuz%J4$Z+^ z_u|#*qqhuwF6sOFm8r;B3MHbQUki`M2HFnHZ=IXBx8qe(pXL>wcGspk=rf%6A~t`w z{Mbq7+NIDijZco^Z`$7G=W`XbJKqR4iXRZUqH!*xn+)OG#Wpx5(xa$1-L4Rn4>51s z^9?$!gz^~1RzmAnxdad_UozzDt0mR$eKn^*{w{ziko6E_-GSx@I!Np{dZnC@#~5M8 zl0mRAMjit-N(&)+zV;IG7!VP7%s6obncx!<3ukdlpB2@ahCf^Q z`jWJNSk5^@05ubrAw6+^Ogy2tRkE&+m+6fN$<`q#J$*btCTp;&aSudtO#l4CoW zl|2c#3qgt9vf1R%bxK+58eKG)PV>yxuRGIiXlUl?Y13aRkeuBALo5HKfVXR2wsvwh zq_nPYGBl>|4n{{upTE{TQlQvuH%H1we*XZ%kl`h+R>I7D?VW#>bwoIDavTvTJmM{e z3dc5#_VEJGVPt*kh~xiMt@$$bt#4?w&$7mXr|$d_;k_YX@J)APtHtHZgM03?<5tcMx?1mCzCX-OUkV@L#itvk|6aK?X81s+lokiE1{8yg@{)kfH5G={q>3wO3? zA(I<(6G^`d?7ha=n~gcP8}1tuM5=Kq3~xg~(So!VzNDUOSE#)4!SMb~QYRVai9DBN zk~>W_L(SrL{yFMGxKJwS$`Q;T-O1AMZ9)38pO#nev}M{tM57z)fx+voyDaBEsQFp; zmPc%e8Gwf;f_C01AV#=4bbS5Fu#rMxRPZ>)9r*aM*({YxEoyLYX|!x)Y!sFG>}y)* z`BIO`Le9$Dj?({@7ZDDTF3Oh`f|`8_s&x255RXY8)RX};6pIie8NE_1V@RdSh{vX{ z`w|eI^&&>}NLMVU;fwq%#6T`(K{=7~T27ft9bIpbgUdquilyL>wF(ZcnC{ zr&kSFyz)=lNgqr;;&BZpe{S7*sUnX?{IR<|aguL}Dp$!slAodv#G0JV#c%M7q}W~m z6Cn6cjMMIU{x;ggolUz!l^FvRr*}NL93D4)MLVXd@MwLGXG5#WHF-HX#vm1yR!^vW zFIXflKnCqdTyzCUZ=bwvpRcn%-h3o};j^X=lT^>}aXB1gP|%1|U*f$cale_w`Z`zL z0y~;f$?^G%pcWQYq4?ey&o_80($hvHQz)N7rrj|)2;5OI46*Tg=6z^;U2Xg_Lb$wwwEU)oM)&*|sXf83`V&#XoVZXE=$4mf{w>9@7C-yJzAoxgN@vj| zY4Y|?M}rwzFSniM^~vw&sX~+}ZwsxCu><4NyAUfVF~QqsLTbVsiBc0NV!`<<<-|~S z)pi6~&UkUaFjQ%ZP{xAuU$3u$#vE4Q0;ZGV7m+HVkOfTtD&~Zkgc27L03xpA*`f`H zVldBb^W=moH&%0ed;qoR%iL7px&G6N?BZ5ws*J3rN;OUv?uojXM{HL zGJSHB-IU$)aKP(&@4}@mhiiAQh^IK`W;dM^QT zH4ZBlh9DKLy~pW$L9@t^sP2y1T`pKuk?B2S0~M^IHDcn7ck+D|U_u76$}uAzQj!G3 zBEDSAxr%ivU?fEz%(+76n<9~OK^V{>Y4;r*G}^9jLvzrTyemxEOdOg#ZkJrEhm>Q!&+|P=_zFEU{@EHkP>{5c!pz$7smFB81_;4bOD^57UL z*({;c#ILurli85*qz0hf&ga4WFT3Tld!7yS4(!g9&+dQIK^1g8VKDPqU1OS3oi>zb zGw+Vn=JpP$EwNW_F=GK$7LkXMrp^_Yw5bDMQHZaebEIHYAH-K7eYAh4*i6EM?_VP> zduB2gxc`&KlD6p^(i{x*F*p5DU?F**JT;ZFnbAd3s%#~pPF1fjq%SP)o(27GdFM$@JwkgC<}fFc3R{aHtu=m?o`o$>WQ7975uKs1)K(vp{=8~x-sF;+$#pver(;)Ud-9P ztuoB$g)ET7r^yTGH*E2ux995@0q__E8N^|Mm;b+9hwo73RD_yAdGPwqu*;ewGcUz7 zFZJ453a(IXcc7*+3xVfcT&MWJOwrCOoZP&6ymxy-1}M#nO)n=+V#G24d`cD;PW^b@ zyhT7jptZ{a$7Hlc%_v5|C$D=H8k^lQPcrJ_N-nmFd%qCncfj^itj*Id4|y!=vNgMUGh{p{r6;N!8(EE}zf6mf^TPIZ-OO7`6n z6pbN2W`7hMcXvF@D6}BPzBS#dAg;sT>S&CUL;mLdJwta(p7R(bONx3qw-SX1JyEuV z)z!D(490ulDTKi-z^J$yLW@Xelk}(XeKqEb@9>qVUbK9KOjBaC*gy!Le8z!N#;c^z zd9aX%eF&Lsn!Iy$VT{8NVP4~_dwEDhJKnnh;!y6E_;IhpWid7uj>x7oZz>|XYg#ml z?c~-o3L0?Ab!sr&x-~U(9&4r>QAuS99Xvn0>a17<1DysgHCsrgh*O9&6za!#14`?0 zi@vDIKgEqyCJJ)MmOOg%ViaPVuE>OKTxPoxO6hBXu-yufTzr`C`~uecRuyry?2J;2 z|5z&3X`n{k|4Ut!?-HiL#8+Ob!L%tvWst?UV~1SUxJGjKCeak9W_!8oUw57i25+*z zWA<{C4fLh13g{0>{e}D_4F5B1IL`-S$c6{ze+mQ!n9~TZ9Qz5$_wLEbN#P-v*N;CvWmbZ4=6H-- zcd&v*i;2yM`S9c|d%O=m=~tR0T!b_FmZYv?Xvpu1is(nV!0%86VsLbx+%!oONLN2U8=#bu- z$grVGWD6E@18po$gY8^_US11fBE0SvjHw@hp=N(q#Ib!iSr2Pr5e$&+(;V-(=JoCmI>gHVU@B~-r-PAy91oMKnGpOURV4S%F^~le0WNSm$ zoau7@tE$AM-nu&1)>^Z<2v?yoTl+xppnDGG?v${(l6=qIjxsW1dNFb4_*WfCR*h_{ z{JU`Cc?O9eKBnky!E-+Pf@-|BBU7twyi zGGa?jX!Z}Pv@a8X`AACN*2h&=xv?Eqmat$m?RJUUHyUk!jw~Zy>nao+6I$07cb@0j zE1o!W%Y%X8uV`PZC$r6Ej>4J}>Kv=(R=B@NzG_>DxlAgmMsVD3@Tr8$%BYdUgP9uh zd;=#a_AQon7|!yDq0(gxYqPk#4{AeZb^&X%ydgcGMsZ;y(s*i_T+Rk06)f-2^{%{M zBJW>Grz(1jEkGqIDyluhI?u3ySf~DDw&2X)rJk7v&N{n#rrK50Deup1SRj*T@wdwoY( z5=#*jLbP=G&}glzY9Sb!@Q5I<*TQm$6h@mYWK+;{(;+=8lrct`QUb=P){ZuFgcPu5 zmE3#8-K9_&So@oDIh~d+Z2bfK>|raJtnSIcA{Cxsd-BRmcJuWN*Ikj5huF&$r^6FR za<4|UrymS_Vy=Hsz8RlPE}zn5+HT}Mu#k0GJF}fJrp}$lrXX)JJ^gJ_k<0X#mn_Y; z`LC`X{-uvu(8unrlW*{|gk)V*+YvAvHX=Qk2QnKl6tnpxm#Yk7{t64FIwYb;P@x9g zcmV5Wu@PTOfMmP(JQ7dKGjml#t+1YXHYCU1$ z;E=v^u3@9Crlo}^4!X~8+WS^Eo)h?-SwG&!j$`H7J)2rzUyO@U@NwwM6%0%^zcV%_ zwmVG8*YT3KyRU=PXTO#Sv7S9c5n2D-3*fu#F+tNh=?n>GAO-M^mhK2Lwecd4_ILN} zaG;L&Sl`D-5TzNg5}Mrm)Ukf@A>0U*k0hff&`K&So6djT4ZNek;T?GY#V3*-GR^KC_khyf$q7=+ogPuD)zMah^Q@Ve~5# zLf6wDV;>8t#A_XW-GrIR_r&)r3x$KdAHRz6Foi1F(&Bq8x}0}S+y*-6hr!nEu@%%O zK*v+ZFBswip*XCS^j}-?joj~;Ow(i={(?+Yw73v-%Uy#>U=r33d!yBr!V5SiZG+N# z-IEg*m_ju^SvnsV6yLJFyuMs?ha5z|%_qu7tp5}YOoWw2j>zDTDegv328=^BLd#&; zg5&K^uysGh2xucmjM%q>Yy|<4g(g2wqDX%gEa+-1!3c7ccyZ{T=dk6QfKfbwHO8mz zL|3$W+XUrDlloU55VPhhSOXaT(Cy;1{@G?e=r3o9HMogq6!0AJFi%Nt9}l#LPAItY zq1th0c9T0XPnH4($9kYbIwUA=m%F{}EWj^gB~Pew|H49{Qgbh?%` zd^>*oNomp~w({uz2xfhx-ffI#&45{no!!S#OooosX&F0D)VTQoEPb$W7(pySdLlHM zGl`OuAz*1JwkQU%1o=lsVsl=*O+Jju&Q?=tAu8((?&2#EWyaQrIV)uqJ?L^LN>DFy`-NwtVDcU_}*38 z2!$5M>EQa6UgICm86@YNJGOa-T1>~v&kwc=xGDr2J$Tjg$Xq3%qtkNkE~(rltnx8S zWwbmF*KL;CZnJW?6I)l@2Ql)|W6lT2D6OFf)HNKNwKxa}DZaadyLvib%S%yE@8DI@<5y+} zW5S-vZ6l$=j2mnvIy+AXnLkvXPra7rP~CY0^m(JMOC1>^6sUSXN4s%A?-$tl@#VM; z4jCd8qefN94kToVDqx5sX_GlrI!F$CZkalIaVsZWpKZi{#l5c*a#DcoHmGr7>3XP! z`X@)}H@ziz65pjsD|Ven(|3&?V|A4xA8ZN;*KXFwWTXdV{qJv)DIKGKME}6tQZO)R z0IV7WR&8|GzF>LUje9AqqO-3XLeC)o%SKCfw6c8p^4_nwj*`&^dFUtrvqX?bPecO^ zBB%jOJ%cvr%3XBfXaOZ(-K`Cn;x+&xZ;{@21~Vrd*c|?VIC`2v92~)>l>Z{8=dZA= zNf3KM%W!3zDyw3!koS1MS?;%AsaV|mg`JX|8VAfgZBjS0or~+YWcTPHCS@A&z|O5( zRaaOGdg|ClrqP&?EkeZp-`+2jhHW0w2HQU<5pv2@g06~ARfX#F{H*#8*8{u8zcrysxE>FlY%pK= zaX9ZtB-z%%;eyeFUse*2whghVS8*bwK{36@Xnhd`d)T4`7!&Dux(#!9Jk&kJLqAZc zXR(A(+I~yt9ZC>b-oqX%yxs{CT47aW11moe{Yc`O@Poq`E@}Br{ZU^V64Ef~%1bpF zxf?hCOX`{epnmmavm-D9-`hj5wt3-Q1vmjTo*?&ik=bGc7S-uM?}Y~0|(d!gqj`ib!e<)BhDLzLZ<1@!3t*E0iwEByC7!qORK3EnW%}GNM04qYDJtd=NI99pZG+A9l#VJ;T z9uXRmL;Rz>uL>=6dDeLNG^_C&Ba6#4Xa?$QTgz9#rTWBUv^!!HDcEEILPX_ zlmkGGpJFzM`&vwRE9?71Q=e&?V7o~|qu^O}jk3EmLO)eG?u1<7MfbRFOsJ%!BnxDw zU^u#9IO#I#QJ592hb$@;+cNoI%c6juf9{*Rd=LTfQ#!LaBpA$g{+l0Uh57ux>;9+D zPCYLT<6YTTxcDuoKV+iY} zJx^XbW*ch}0(0x{n_ATKRamAf_$U;PfAd)=^PR;x;kcpUlC}2{T83DYK*2MLG_TXh zd4_9${Yxcbn?4g&;+7xRUocs1QCM|bcsfr#w*PORD7_7Pe5=>uZZL?LH`=z%)I+ zo34G6*Yt<_eY8u3$egYP4Q^{exF0;_JN^@|Bj^&YAd7$NQrkyl~uGHtslc z-^6!1Vx+8jzR#)ss!5@U-;?`Iifa(mw&!h0E>|qYqJ(p0NM!lm(`9~To6aX{n^>L; z0g#ygg@b?N=@`Rj^<$aKlc@zfSKkU-1-HPe-%*bt!%W?e3CYCOv$KrrRLXqZ$R8EX zr%i`%DqeV4zmJiB6W>#3L#*KW_r%24(lzrnk$D$Xqw=jB=!3VjC>~l`u!~*WzQBEQ zF*o&E%q(8|{7s9=^K;!XQZc^7psAVU6{c|HMr%;>&FRZ}FA4I~p*}w~|>hoG4>A>%;x4>H6JHOYD*YoXo9bIL=Qf=`3Gg;`qvLrSNx@RFY|fz zw}!u2J&GeOCMEl1ZFc{>%0eM}lDvNU$)7MiMc1tUhp5*~(CQd|IIwSJz|+_0_UHh4 zW628dW0w;Kjn9~Vg}Q$o7`m&Q>CG!=7B(_^lS3Tze5uK08=WlN`vZ7474WBjqhV4r z4FT$GK1)gbosBgZ`**&rXf=|j^je%;4$C~LGAXzEYSNZdap`N2FQrQ%!g=Wfa4au7 z&sfrxV`m_;)<~Pyo9`UWU8RIfb>UYuTIHx?WFeC0tFgeGlH71dU8QY&Z-tQ`Rz+b= z`+v{GFMV64msV%`7Ag3WDmAjU`zBTZ-Vi|o};ysylH@0{+yZ;{V;(eAT>#))v^zRrL_^z5sqMb@> z5yGb#T-shTYhl*wdxfJzaAfpT6MBVJDg~_G&cYP_zs|O*T^3p$u zbg$8c{=*Jn5Gs+H&U7^C((@vcmA)Y)3XGTjkAzZ`3`!8IQJHP|VJ&kCK> z{|LnQs1;LA{L=!ud}X{BL)Iz0AIk!m{MQJO$%eK2%%| zdmXpjWpO*q!1>jF`~52?OHF)MXWuY3@h&fsZc)>V7z5oZSF7Vz+gX%UQL?krUAY~v zcBru~7Nwg1QAqC(v#iDpjF?cNt;p{^7UwIw{DtDP3-;k`VF#rC_Tu8N4EL{3@@b0r zTRC2Ky|A*^x~T3?>)69AzWm>imHybe@dCr>?X5mJOXH1d3SLyh%s>Exx13kBf)WMl zzpWWCpX@tIA#o>Ddi%Pzm@IaL3sY%T_eXRH`5$0p;FDOKNsIXp>isv0^a+X+4CLGG z#sxW{BMMWOIC`g8EF(*~w)(J`#}jKeu1<9{&F8t6mG}&=_m#0~W^VOZLlS9Pd@H!h z`jRE>*^RvCcvqRdLpGXQ)8ScI6bshH>aFp_Kcp$@#jd2fm>{2dwoP3wnU6qW{@;x?fyS16`p3kvZ9P%ilbN3agA2s}Ph&YxFdS#9BaoENc4V0EN&RdB*#vgex}FF)GaLZ|xA*E-gaH0%OoU#QLW zIfSwyo7Di5)i4B5R^N>eN3MB0devj$bGtm(%IMU`^7C<&DTkn~aH4UR^PhGYdybk9{sUf$&+op)^EW8!Z^*D5)?TuEWgfp$U(QwU`|ZQj zr;Q38_qKUGLEAJL_|nhx7XIyYm)B=(c;G`!?&b2C-@96zMagpf8zDYW`j$)b+FQ81 zB(y_E4g&k9~JI_?%Y13MiW60Aq{|(>JLuUJbocZv-nL$+kHle?>x?SKN$IPRG70n?Ge2x!9 z+l$sDxl4r4uVqCnPibvgsERm3wb-{jmMpRX!Rab_&_rF~;kHdIs!%jPiEi;*n>%Pe zbOQ-pKA+}(H5UI8f>K!*An3Lwy;yA*Sf#vqbqqR4o8S>vb55*N_Lm1>B0(i^%==Dx zXa0^@AgQh;p_Vyul799gk*#N)r+-g zL2l}QjGq3%7ikLuwl44&qWot-v7_fgRQZAc51Fr`{bg&bA)OThWb5^=H7hh&6NmWu z<&NE<-vatApx+ksuM+s}V*h7j&~WkgP5;~r@K^9th2ZlWuzyFi-y#1mzVSQc|IUei zCyi@m%fCbZ|Jfn`FHpwvJIwzsTz^;ge`yb(e$4MM|2xe84)edm{F)N@-Cg|Me*M>G z*FPKhzx&g_8~DFhIM!Tv`dw=LE;W9a8ox`8H5a>omm0rIjWrYP-|O`Mca<8;zH`L& zy?Ln_Td5tScC{Nf=XH2hPnOruM(i|*zq4CB*ZR&2PD2Ox#Q2l`+YPsISiaq;F>8B> zBkRcPTj6?Jww@~I+_zC)@Ay8+{{D4zbn{LxHmj!g3LRIg^C=;Yl*~nprxn#1cxD#O zU9zdI#QVtef1fyB$vudx=pCFm-sZqew+K54haBGgL01M_xHtpfdH(Izj;FA-ua(=hpv^D4scBEw`VwA}D*@+Td3Ac%KX z783Nr|L+%)-tulkLqjVkC!yM!vwDR|k=Twdm+eRUT0hL^qP?`U+u(C@X6_UC{Exqr z?cgl$;@H!B75PkP$wugt)E_kA|J(}zWv##Y=I5l~P7!O&;uqoK=j?enaIkzEyvLw! z!v5mGFk#QkNe34e#j)C|i$?UidhJb170QFDjASP(bMu=VSUsE)?YEe5!{`lm^mD_B zQZEaAWY=(lMGfBN$54N+7LAo#8}Sdn(th|$5cvjn3hNV{QZKza9~~Q%DoMi-L`UPg z1B&sNV8_{tB`rqqd45|w6$VTR0-APHWovXd^KH@TE@EK$aiP?!J@pA0?Ieq@p&KR~n~yC( z`~4I%;lem6TE$QCBKX~xE~mw|Zm;mnk)x;C09bCUCuJ2C#XNiVtXcg`cSU7obJp#} z&3U?yv4Jw+*@C%3R~RVX7(RQ<=C{EMRPn$5zHHik#7|t`1U9rgy3vw*`fD#?Z`cj7 z9qoIXS>J-4tII56T8t}<5Q5EhaR4BTONU9EQKX;SdDPwMRLialFB`!ddriZ`x?Nqp z_AM^v=CbG1)n8TJy(=y$nU}l*$9Y(}U(LN7C5+2+tXjP7P`DYr!KG*ivd$f~-hy75 zPG1k7i9R9d6t-V~A3G^^p5}E!Pr?oQL;R-2seedWV+Ra#^O+Wl{eiK@4D86vypv?* zbJPd8DhuXV^&%ThQv|~}H}eg>&>uLatE<;sQE{z(`yUIIWjzOTqSPp4S5PlpD9A<62lOcO328g>4;&6HCZOJRTop?bsC$X_&k| z=>KlhbJCZ#%VXHfOcvb4HijuFfcIqytojp6Ei zQCl05ll372PV{g7I#CnbC2}NF+*K@i?@1M>>}+r=e0aiyzVSu1s3^inP0iWlj!D5# z@<=%6s+@#GtL*%Xpn9nl--+k!X8V5yn4N)si}pLe5HYF~02D$BHhzsm`F9|wmJ34T zci3?LBtPXR!+8^sAXUEYB&LHxa@)uSCUhPzOG+W|@Y5&_j-qCQz1LO@f2tJTUL@Y} z<0!gB^ZFzYml=>qB|oZ#O1uN;}ex6t4EO<6-~qZ7VG@Lk>F)(AY!bcx(j;`bQpIz-Wnk4!noEX z{{y&5m{;Z3S>#zjcsY?b7duDL3m9IMK7anc7L|lAe*_(CJxTNWY|&Hgy_Pxe z3ZCWVg>RKjR|Kv5scWvswEiBn{_V3p)>>W9alp6$GuDNMaC;lfe>CIEob?N<-M zId8Od-XQ%YMJMcCrzgmC3WRmc;Bb<|000GbVtpZX#hdZl6%T2OC`TUsbOP6n3*ouH z=WL4)I%q!5q>H-r2Yu)PP8_p~%s`>Eua{S0Rm!#f1d?V$ zW^>?21r|J#$8kL(u8Ln=ugO?iB++H1ZbnwibP+^*#W5e#?n9tS`4jOK66Db<;P4MV z_GCB$`M1YeIrbc7fxjr~b6_v*t_P^+NK-nGy&Jn>5BMWP>PdJhqah!#zR#IrY+1~8 zKUG@5Ny zxd)kd{1l-3C;Y?B%xd86stxu^rhmX+wtXZV!MKHBDRtgn3gdPgs6&~alm|mdD}pWc z@tYS{x9eE-K(thrm1%32Y8vaZw@mI96_tEGKG*XvjlUXELwA7kOG1Gm|hX;4>f)X;MNgsC@M})9q9{ zSz-FFMq(T_B2FzeRVc*Cvx5TrqLHSeQ!DNiLADUv3Wb5~ohouh^1TuCHiN zTQpqK>ys6ztBK*u-!&vldJW>-Vn5{Md|l{T=+GbBW$WqbneOYlZiF-${Kj*oBJOL1 z4|y-iUUKPk_$qaVP%<^nQ)!rd!oVeNPSS?d@6)^Feyl4_V3^Rj#I?F?xVmB`yK`%W z5~(DUXO2?tHA1PLZ~sH%-g_`+W0RuKp*+ryV?Cm0%X_8+e zqhikK=sf1>-bN5U#q*9ud-)v~VaGZ3v#+M7v)J!V*`(SWbUphe6u{aDf6`-NSvxze zS_CI~6t^Brjobl4hDL=XutO(SKqu7h57_U31nzxXTpVj&96VJwFknu8C7>RyYEc(G zYH*i8D#pnT9n`S$^1gB7n0KYZQr(PD$xoq>)liFPV$C`stGxAl+uNj?HYzN9oSCpt zz@?|BCk-W$U$_5En5Y&m*(2A#Nkg?{WiB7@9lN+PU!ovK-BoWZALsRTaCf-Rpp=i? zx%(R8aTXbw=~8hP%HH-X#UwkTe4lX~`Al`~NJKb~p=$bMi0iDuDpSLvXOMQQobPx7 zNxj5-wVV`J;#=OHPD;9O5dm5HpPUQ%=HycBwv%C^BV^%=kJli z+ekO}gl#0p#Oa;d9p~~~BOyep*X{DsN8aU+6$N z&vRFX=U3SDP^Amk)FJVPM8Q@C< zDNw5GD|&r50lQL`TKbC4;w{^D$HHTqeDFv_*48D@x@a{G&#vZ3DTT9&1MjMZH8hr< zIatv@7`Uny6Kk>Zky0l~r+Piwcj3j%X94-FWWVN`8rw({eM|ZWz0;w6KbB<4TYKk= zXVNStpOw6O7rlG+eWlA`1y`T2Jl`Kmyr(sMmztZK7K6CjbMo<5U2+#?#f-l@lZsZ0 zcD)~S_VLVIEY&SiFg@hwEE@lVGSn0|>LbszI-)d3F{;7yz9$tI`^H9*?pySgl&!AV z@whB6a&0Au`2Coh7b3^>kqYF_O_8%NDJYN%$#uiz2o;5;#?cs(mqh`wU_6lg_D>^* zCyJ#@F|!0#CY=lW6!|v#7i@e`gC8bv7l$Zo-=fjpIzj0BW*Bm6sbiC1_Ypy1xi7*B zTjjipug}BJA#{8?Ss9u0iuE%7r5mG#NF^YtK?%#wG2KebGq>Ug?(jS^_xCQ@n@aMT zy>p7nAXF~OgOlMgZ+~UnHN=g@G$>*0+o5QAFZ${V{DB=6Ld8O#q|a|e7-qm1MS2z< z4(Y~H>_~)R6xZm(Ft01wn}as-87v0^Ia6b};y!4j6=qXB z&E>I~ueM$!F`~0md9j3QSkjp_;f6e;&)Y00sWqoCCx^s*D$CH5{z2?)nnjY+RTTeK!)pyZmkA(n_gE^3QIrfpum>ONt#QsQdl)`L&EQ-|&&Yv~-yoPs%j* z@#%GGHwu|0=%2AMDWNngEDmBC+0blKY$9AREV|L2{+klJKrS zCWlBtS0j1j<_kxX@(YKKct*}f)O#K9Zq@T{jpyY$7^lb&ST^Z;uRfLihT9uBoo5|; ziXlxENQdiNezWGi?TTZcco;cC;Ak%0uHN1r{*>mdqYB{%4ltKgYb>+e-MH`i=M1GR zie~<0C;_wFSS!DNwT-myKKQ!XCB5C5k*~a4PD)aeD_n)<&`nJWPtlVP*=$>Vgy~Z& zDw?u(uhUWDnYD-G1mQ#*2+Tv)JI*dZ$sWvt;JOg3OyS+D7HS(CTs}icACsS? zP2^ggchCqkp!7wu1TC%{^BI%!S(g1EUBZ2B*?}}6PM48&rm@|tw%s|VM6#{DsZwEO zy11CP7GKPpD(}0v*xXkiS5v!OjHh4p8Kur2B=_`Y4bBs&zHwWQ^qB-Kt{nPOPj)+g zcILY4S0Bpc=rcomH76&hgT_pg=!D5}eE8hE`qUP)?>6w0Y3X$Eb14y|8vKRjj2(*B zAhwnBZw^~FkIU`OHIyVK&wMI#2xRJk|a4l39Z*%lMRyqP06mdh-eL5|1ykM{?+ zo6dEo3&PQBytdb8-aT$!Od1*v+CGOH;UpK!7vg(=x?qU_l<+-Yy`+|>e}qCd zvhxY#=w4DOSxM*zli1bbo4s}rmOfWMEs(39*uJ|f(`fqsF*-T0o)JpXzSpx~-C}pW zNDc^HFgolR>uw9@|DA4Zn{dA-f-A4Q={9y#l}H( z^?Q^(J@hYVPzQv$$j73%yL;O(ftKm)DafDsETA~VYnJp4$#$?y-SB7NJt8dps@tA$ zM2;L$e>T&XI?m<7$;+#eoxM>(WtaQ9^M0k<=+pXKD#`9o_c9$0XXVp8?SJSOSZVv{ z%;{UvCO(N|>^5uT{8HtU6%`qI_VMG#R_0!XRgt%C2X>U$GY(K@Puz+g2s*>Pas7jV z=(0eMcazfdsV1_827MitEoMGG8PjbI_9};4Doq^49#gvt^NM>rvJT4MIx5k+crHaQ zR3jNX9VwKqbK?gLC4-*6gv?M=niiq$W+}l7$Sc;dbT|}KZksIJo>{0=5SCl4( zicuP_%Zf}4tyh_B4IPP?mK6;RfY?!Irtm#_a(jsh6$-1Uda=QXu@ms`I_pky4YiBA zUKJ4O$%y+f%?8`s_t+KRIF48w@Y5TN5ZYL*=d$J6NPB*C6@YNZS6G$EM*M-fry*1Q z#tKeLw79nHGZK}ML5J`2#t-gNAKzAgKsGLA{7#BJA-(Tn2^^}>c8rcbbdP2(%_A@h z{1Z~pF(boj8tq$Fl2ZAc4yW(k9+G~?<2}&tnfIXh?a>m8#+SL5^{@^GQC+QQVt0$Y z;J`|?5P|VqwwTyxeE!?=&@sz1F_(wt~%@x<{rPu60QwuFENxESi*~o4KArLfe<8M4d z2nm^;X$^Bf8=U_`e)hG-xlGTV@uum)N-o!FFHz6f&T8Tbh{@OLSwUi6KRiqJ8f!lR zo8JHS3257W_)@qORhgO{GYVf9o`2m8ACaVvz`4SnT7Jv2H!q=Pqh0SK^ry3GYxsf-fvzKSME!P|he|HZ--j zuVbz)Q^uEETJPZ!On&*# zi2{bxR~;szBVyIFEPE$Ax_Z^m>~Iy5`SG>{$PLfalW3;aQ4o^N5(YeD=*g=14#ne)=!l z6_52T7JlIm#3yN_pQkza=vFrl7j9cy`0ibozNv!(^o%0S67^}q%EzP!_*I^8Ft%3~ zOl)o`h&ov~nH_g(!Z!RRmH4>TUBlEW&M0JHxy|u>i%m)X!@Lbq%$%ds)plOeAAj|U zIF9yBE=7@)am+oafgAvRBcNHX;t4v#b`QgMxjt&?)@4Nxl4qTGlq+R2KiSa!!_|3w z%Cnw;0e7>x!!t*jJMTq-bP5}+O7_NDNUl3WnXrCB^$DLm?b|>+o&NDvz}W73F^hTk zhs`J2DKh>6&l!zL)rk>_e9DA~l`^-Tj*Ul_CYU>j1(83v;!*4atsY}u+XX$Ud*Ou-Z5n@+r z?!PCrTI31Gh^?)CQhIG|MV(^0iU{pX^>u=bGUe^4v-+*=Lnm>|;Nay^$Q0W}Kjan1 z0?2Wh^f^Q|rTID5Y`2r)i1g;;r!&NUj&Wx(v{$k}EZQ$zZsdV-|EMa!>Z1LF!853z zC>UHC52#0@J+XaSz<25442Q|e5Ko-NZg#%0D{V_mqQ+mH8_1s~?AoSfoy>fSZKTp1 zY@y<*P@iVo;c7{8Jt18|Je@_;G70dYPmN#VxOyo?#T|98yYbfY`u^%TxzC%Mvo3qK z=NTx(N2m9d3wutl4vu)``GnVAOtrD81#{KAmyqSz)aKegizD}#QVdYzDfkso$@i{Q+iUPDqfktpUC;DEiht`k zA7fv7dI!}it#NLl5;Ns8=q1}pt-Q1<=341+Lg&6R!P)|3m&0yOB5-y8wCQtl&)Ast zApSLK9cXxu>5OjDa68$44b5629h;OVUw_{sox*d`^x#;FUGn1lVwa8gsXkjLoHM6- zdVAHTiTmosR3yHPdz#QDF3lA^Gda`%tW{Jm7i9M3*2|?6hRUN^ZLW4UF7*#h6zG>2 zUNo|aZW>>TS*)|fif36Uo(c70;2k3=+S~n!P$4Q;rqBm2%94+k^euV~rzqc|Kr&(K zNSD9_>6Dl8RmZ)ArTyW1{MhexWi>P$VPM}XJ&^wVVP05*&U1loxfE(yO_sE`kJanz zJ&+Jzwn&slUAM~KxaT`HbRO$Kv-9OPzu;3d($^q5onRzRuD2&mB$7w)q@HTw(-&z* zpxoYt4_xD~!Eu$ZF36UgSUDvn^ zdAlK^FW~c>G`-|vU(AtZovf@r04MQA``^)M%hrD-A^2wIg43|{HA}3ae}P&VL{on^ zvu|eYq^^-z+x)`0tzwl+JxxD}(QG!3ol8b(^DZ(q_>J@7VpxMp0cgpH>LV?2DP-(% z-06=azTaCklO$qSsDB{zv4F-aon&>MR}3SpKMYOJ*Ne?lsC6UoI|S-GK#1|;bY;vS zMAx9edW&bnhKX-Ej+~-7xP7O&UToWCbMrmW0TIs|{Ng%Vby8`1giLnwtPA^Jw7qvc z)&KiHjwq3kqD4w&Wi+JB(=bx@-jyVT%wrz3Bcme8EPE#_PDTk?k-bmZ9GvWN9DdjH zpjW-$ukY{k`QC27-#^{XARsgVy`_tIgk7{kw#VoCt&FO9mrG@8I`Ml<3N|cSwwqQeW|lRM(U}k z=5KYi<3un@MPTv6i+^?HG!_EIR)ybN@|~;On=~q%D&Agzt9ayy-=rsp+O1{AO}}i4sjW>b*ZSVR?{pu= z7I$W@v*_ey%3MNflV!z4H*M|Ky$OW8ELBUA{ynvb3Hv4$jSNRLd6wu5H{?ofzDTL(~EFc)*=dpJF<~f3Eg}c`l0=)(uUkuzr9EJ-xZR z`!SgY$-7dvNWmv`WW>L4kMH&rOYF5sCG1wyPvLDM=GIPV7)pMSrS(g0yX}o@F)fU! z3jOAI?EOf+Qq|1*XN9%n!leTA$3;3sCA)8#5WnwD_jwTEe9s_{%gNOzV_rR^w5c;6 zbUbH$7?P@}bZ_l+wnc9GuHu8DA1>H&ELjx3ZRb*%S)5oUb_$(-%~HpPm8`o>dgm-* zHa4{9I(~u${G}e14oo(tO~Off3bYqjZN4c|FutsuG<1#T{i3zKCAJU5`4brR_UhHS zFt_Urg8IeyY&Rp;i;!Mu5ABZd2NrGl@qQM2>~_-<<*n)NYi=!zOPiWL&aSZcgH)Q>2Y&lT^+{(Zra%gtQNvTKAscPbeeuE^)gm}D$X#k@3PN)pt+ z>?<>Vn`8~KU7tW7zTJ1T2`QZ<VFibFS^)>Oqoe^&l)`P(#|x&9k^Vvh*e% zJw;o#pJ{bBt7+I>Wl?4OgiU^4wX?Gmy^~egy^sZ|v;C81 znP)gEkP=~m++!Z6-b{l#uY-?p}fGFPyNvajn6n_%%o31f00@F)1tu6d_%v(^Jh&2=YKf?M4Jke}Hh z*NOWaRhe6yuvwk+^&5(v+;3~`bQ3+axG54mrk$ky$8l+?=^P6ZC#wPsPwb~o=7F)T zWkzXoVb_a%qxv#2Y?KWxF|f9U>Dwg1m2TQsMmoj5j#+1V>wrV6{RK|>`pjZ$YrENC zNFC19(zCr1)QBVA`-3M@5-v;Ioq@n_yfI+m3#%7NXMqekkD;B}3vT{w`q(rdTt}Vd>20t{e)49fpU#!Gr_tY>?9HMRjV$4^C9EUc zedk79Je8c~-s4z7R*CmCG}K&j;o=HX6`LipJwQMkwT^{Xq8y7DW?n-Ff| zcy3Xw9KO$nrMsi(!wWT{Bx9pUXQw!T1kRQfcLtQF!^?b+`_4W(?uan*W~dQk9gL)( zW-U$#>wN=fg>1nP`z1xTU%#l6PvayPeq7*f}5Q~HHL;lPi9tQ zhe98jI%1xHhp*72E0SSx8Yjb(;{l44<5u^8U<={8qM*1Q+V^X^CBw1V0!%BWE>RtS zdFl0=gvlq_MMYq{s*NPtVC@8>X~Wtr9|u`DRrDdRqJSw*8UNM#%LfW%n%O1+ZK=tU zi1*L6oi+pkUG!6RQ^^mCM``me)~CDA2JA=qyi=7J4iHvxFGY*>GK6H>D@**ck>rpn zZu$`Kk@r@IqB7UnKQFrM{Dp9~A+-1*+EZsm+83rqM<))secUSGq%rDH*An61i%)w} z4Ep2Laj7Ei!@Gtql1xHkDxXxATPy0iUEdNPJQg`^@2AqB>aX*Tv9;#tXSs1FIbIxj zFf{`9FtivubE`;vxy=?SVpTzz7F4YH$7EzS_R_AzQFnA*H#&i^;7^d(-e3Iw-1L2J ziG!rkj1`M%mz(&L+4tnqvqRf^^>2o+b3O+uNmhb-KF}j-MRDj2} z%^`@MxK5Qu89R9@a>BXAQ|gyXyu^9JJru_qC@FI&{ivsmZ5!w2xcX(&sx!S_g30{3 z)r+rRcQK4Qh-+zS?XwZ^jqwlR7`w&5mtz#> zA=wkn>S$AJhIdR8KVdriI4`%>C{fPMO)n!NGEyPyBz6|L)zS-v)UQM4z%}^Tr(25@ zS3y0P6-{?TcmUP=CetD;p~tOpw$(Vw-+1wH`IgKBZrzR}-^4xhOxP+AEnEhP6#T@D z<8GD?c}^ifyL>zx{NfR+27dL9QLornr5=MvPrfZiJOl0rW2WAjn?=D}NOZmL-2tR$ z&+s4#hQYFGJBr>HbbudLXoe4OF5ptv3IJ6iO>h8c)o$G?WUZOt4n~lvtJ9}pMcaP_ zpcx@I7ORq~r)$PSW5Hjwd76A2m}H9S@#2KJ%|eK|gko%FHuh5`K5Ek5&F7_DpxjrJ zR9-Gm?@geSuQK^=1JlJ@)Q)A()H#0q?6cl;o*CHLi}}pweYkqwv0i&S_PET)`Gp>S zLOyeW2`f*3#lCp3VRQLrbxLxLjl537)&4E`_g?C(3SiROLnmygL~xpOo##ryV|WK1!~Dq; zwyy|$67?c}C`O1;B{_o^cGjIt!v#ApBqJ~`S60w!Ku)9Q4Q&$>a-HJ(BcevX3L>km zB~>}z2%TnQhImXFVy$Dnl(Gq2tkGP*PKuoUzA#?~7my2>Vi~2E&g!AreQ54Fm+Bsi z%-$Sz-TQD)4Xa%4z+h98#_OZJK2=KKrF; zYMDofP_}eWI_tQBo}TsC@=ol@GC{>|{{`|DVb2UZ10Gp|n~S;%AO}co{E!P3N50LF zzYWX1*}MKbY~ghG(voZ6;>|e;`(ie}GRUJIRNp@tp3_+!xf$m>fr|bJG#D7izHgQo z#J0I4Ha<(kDJnfBq+1GP#5#u!FD&Tjy~1Iu>#ucm7vSpw@X*oK#UP!Njw{=1kc(ln zw6XoN)IC-+6R8j0@V+qY&!jUqZ^jGh7FRoozzMX394PnblkrNke+kyNa0IDi1OcQG z3WpW)nwP0cSpeG{IV())cZr|HUh@~7NA5ICq;`!UV1B@5rj6!eesQ!1$7A!PSBvKi zOulq#+N>L&b`cy z8yLvW$>~m1{y1xDZ*PhIZeoAZ3f-2L)VN&{@$i)7u&p*CPC8YrN{6y{(sQ>n#yVRL zf0h!LOoNORZlZ;ErYXR0joNnrz|qaVy%v3MpNRrQz9?pLNbAt82sWux>%a7v0>N&Mi^gX!O8Ip=C`)FE(PtZ0* zQo+4o&;(bYu)x3Kta*XTg+5%$Lkwy(r+p<|rf!Hij%xJesTxsIOVnj!w7sq+(t-cH z`vXd7y1O<{T2wLq$*E^fqP#A|X&XG&Xo}XOF+Z^64UOPdP8nM6yK;SfeMHul0)3T_ zm+UPp=EGLs$+=Y>XI1L$$-I@BnV-phd#7o|UAbK0o^c~p+5OB%tL`eK4VAx@J9d8z zeUbgTcWl1C8r+szWqm&?*NIK~x~jCef3K~M^bi?>;R#eV6rDEo?5og(b@BKjH81U# znaPpx?TioLW_#U5rC#n8((o}0fzsfZ%V3L>JEuEgx~46(2seb48xe2a!Gi_3QM=na zgIVbs_mFW`hoC6JI9A0$zt}e?`;6YbwC)?7t%UcMS<@!nR;D@AO3LkLk<9q+TSfe> z3!2E_{uv0(unR=6vaPCp5>(9)^Cc@8b2L3KtwCoJE%(bUO1!Nz<7nb40xuVdfQC(r zd?Ce&*@@*Wo3xZ(j#@IxIDun$%NBeoI;-ReKuS@jBVQ^jD?M=WE-InAXUDPIawhn8 z`Z8rE)|>0+Dq%%rJ;;-Vs}QI5RJJ9EQ5%R6HSK;`M5x4ToXNpk6`#uNqP2@l{$kh3NHZm6 z<#CtD_K^$SX;|%`9OLFNkT8HY9U&BL*?PnBY9>Y$eeKcPd+8%Hv#H^8DwdXQJW~53 z(r^HGV_0F=Lk&x(33CIF{vHXjL`v9VdQ_w%0$xpCTM;A-ES_4W zC1${Jcn~qgFSSW7H?zBkPWmC2#x!Yjg7vK#yOMC~^vj;*RPAamgWcPuqI7Wt-xJx( z@|3Y4ua}*8ENmR`O6Q(^AvhcZ?C4#YaqO;~%@2A7tlIA`mKEzL$1Y3*@!-y|_CqrG zoOV*5WqY~|;2(Ke1v&x%#~CE>^2;IlWACy#bcua+eGG42u)HvrFgiI3a!TQ?0I`KQ)US2LaYzXu%PVmUBzgrIwbo zmU}~OgyE^yC%K(pF+CLp`b5DCy5G~p`^v=h61ZBvfQd#B$eIKK z;ZA`DjlIf!BMf!!ngwHR=jC>~+;;_rS`N1l`Wf?Z7qR;8n!FI>h>jhr>p&l|NGNn3cLszjZh9aQ z-6RcV0`ZniZE#}MZ&T*s&X$cT<&dvW`F{ZfdnebSjhA~j0W_(b_15N}fEsq(1;S%l z$5!0EZ(kVX;%T>o$&iUogn&u?mWW=yS>D?Ri<0Xur(k*j7cpzj@JUCRppkea0tIw; zvGOR)$Fy1r>KhuK#O?Cjos8VUuvrjtW@j?@LA~L+H(YPy|7L|*yL$Z5F_qb&P^qZ1 z{geYvhz@y$Vnsns+5C`VY^!wNP(L$Wi(K-B=objB(y-gcixlk>CO6mmivmOeu+qG_ z)Rxk7V_K)gU&JTGzfSnFujcgWxy=vWUsVlfGj$U0^%Ko}yhG0IE6;f~0N47-gwV=5 zy3*@sn>t#F0Ls!qQDd(byq8nO1LC4tq7IU5MsbQ5aXcG8gr67ZBG>bh5uu#;`KS(@ zuAK7e8f$SQjU1aC-x3}~7~f1p7SetJ)peq&nt(oeveB@W@EX`YZ89xAO%zmDvh&OI~-*QjG2 zj>MBaRJs(rxzeFZC;ceYgz8QXFpo-FXJcllJfQOGcM~q*8!W$Rk{1J|b-F8xit(Vn zkOTsAiE1^ITnQ3wxH)NL%0;Bb^5KHK0dkzwq*;r(gC(8^0bM2$ zdLpPH(`IR(k^quZBYlgM-l!yJOMqY_gtq)y;joa=w%rFfd}pyFBhKA;z_sB0EKMTn zmLev}pxQ^!F2XKb4cWjTU4T;86ryJeS_fQi9vyHQ{9f^N2Xa=j-jHifm^_tvGP1nF z35^UnP^KzMYm@1c)cYYE2|w=rY4tr{k^kN-qP*-A_1(ylvVsNTQvTLVUI8f7%T|;-Ng4WmXrD z&nPZxpIpZl9%llp91Up|3fo=XSrD6LMz&Nhb$|J?@TQd@HmH%Hzm^eLkGxS-@UfSYU&bnJ`NAv>Q3Xq z+_~seOnOXhGIlp;{N+xYi@p(ldI28?TF`fYp;QL zK1&rje)Zdi#bKeLxft!JHz{$5R{*$}2^9k+fF9h19qt-y+OC*9Jze8E^G*v1=}@!| zVT8!}mXOn4dqcf5BW50&EmxF#R&(tZrS=hc=SgAiCN2QrQu9w=zFh26ypdV@ec?g0 z3iaLI$6m_R9!qC?AIWJc5ocNY`pmD}olsO(jHf*)IrD4Lf8JL#P3CGc>2aXOvarts z=CuZeTWoD7aC_FSA>0HSfI&=`g{%RZU5Z#xQw*r93$V;mQFMhN;1ynmXwTgN3pz;v zOF(rF9B4!2osco+ParKp{?o$lZ6>C*z^Ki&82G(JgUF0WGXr&6mFqq{1ka-x0xZqq zfSf=IZxTq$=<)NT%Mw`}l52f?fqei{;HW--=y^yO-PFb{2S5^-GcUhvjJb~-^m;_MlJ zXj!=2Lih#<_blLY{}#g1ICCrTRJ0WE*V0VROX%B4Lc9!F8N(>EmV`&--Z1L(0Fpva zkJokIo3_kbHglmaeSq3Zy?qqar9h3oTi#IkU|3OPQZ<8H$|!e0#5-BS

lb&npKs0#-Zb;ih6vBzwx+Ju(Xd$GP~RpM z$FBcM+CU~^?JgArp(0Kdd3h@nPf*E7V?rZ9rL&Z{l_RetC`IsTuxs9Qpn{~@Sus^4 z(TbohvbO|c`p#y&kmEUDk+{iw6Sdt|&N4})tlOuhuCDGU^YorS?LI4Ot2n3;m0!Q; z1h{}$=&_RYY%yy2rlf#$3EJ89Mt3f(jmQ5A=Mlo6@R_V(qhV)vot|2d6;C>qW;krk zj{OiF*BwNwdS6uH@hT~WKBJt+NNQkYljTl4TEDZ?+|{CLsoPsmiR>Ky(;`)(3R=ql zt#*KA$^s%@3FNledc|l-ubFQ{#i|^i?i^I1^6rAWg~&ZYSp2t(lR33%wdF0hIy*ZR zUjql9l9be>;sdlU!brTKgr`)<$tKl+q{Y?Ra^4%tr409T)E;O}y4;KtlQ%)=0MT&- zl0&+1vcRjPp2m)zb0g8S34k8;3+RtW1aeoOmib1sXrS@MJ&i}yigl`K5K3$`0ksJq#$_&CZ6Z0T|?rOv4{MG9h2^XDu6D#Wvi#) zPngb@JOmBP!op&1*8pdGb?t1=!ynT%v-4f>LMc&YK`)xSyF^06PIVe@KB>CY^ToF| zA^V;Ys^|5(QG~}e22(#gawir_n(S$lFSBdYvI`5lGryVe#ytV2NaXCZqY3e9%X#Y? zz62YN?@`l6RyM2+<(rIfJhFl4D^!|Mz=>6-_BQ9xfgR8sF0;tJhDOKel!`l(tnIFM zEg6bD^43Ch+Z3>H738#cBo@Mc#vh?JMUU2MJdFoktqY!RtF9ZOL zThoM|l)RymXFlOm=HmxBrJUxocXp0PhQE?$dBI{XMkQx*NbblU`ebtdG#kVj52=tn zHX+CJ$f$*Ud1qp$JGa-38kXJZASQPTAxcD?59?O^Qg>5-?5y}E@@)wU;L7kDh9n77 zQprKrw%DPomxM9^nfKi3g$2SQCCpG8op@=yB5Lk#b}Wuw_a5SU&jlR|xO4GV2+Quk z5^u^_KfhW4{65Rg&1?MjO{v1-i&F;Vbk71Qj~?Bup;?Bo2aIWg zpQ%|sQuC#q0iPHj_U!>Rtp>FCBvD1QnXrs5No2`!tieQOsKsj0b#C~}Lb=vv+=Y`- zs~;uK%Pi@iOcuCvmBQQwaM{6TcxEVW?NwA%iUl6Epf?8x8%TK`Na<_cB9DAV+L4F{ zDG+JTVM@dUT{V1vrahy9G#8)Fqo!0be*P42?2mGnr{S7!OWZ;VRLPKcNU^B1fLn|T zL7r#dYWUeBOWRwRunXCAeF`dJQ4oa0T-?fd0f&}a^($l@lAxlNE=C!T{070vkWX7aVn1~j8$AkhP2Pj|{j zbA#@Qd=-9*)7#nPRxU>L_#PpKY>X(U1^GluXyFgJzM$cevT!HIsVTwTdNi;^`&%W} z*@5UeD!y5K(ND?E-6m>ivwa%bjRH-(^f;9^3tM5R$aeu^n1PzBc&39?g?pGBssq z%b#$Qnh&Xjeo|^ajBwJ%B6eN`2H<3Wvv=)hB_*KdH_N76rP>5R>t3FW&xq@WoZugS z16p@KqMpxS>AkO@C@OXS`XAmOsg(m(G2TMFJALwFm*`s115tKWxBU6H6L zSdz!*d;~A{dzZgI$iQ~_pgr|6JCN*m6Ya(tzr(it%DWBMIO>19WGe5C*wAH&gf#(b1+=!*XU&MI_n|UZOfmwTJ$?vaIkiV>|Q04u-)IUG$4u#YBI^6xC z9HIAwyCMQY*w|l*7d&$Fwp8b}{a)>d|HDO~+QnHP0*q{YTkv0Raz)t!1$-LWEYuDF z$=4k8e;#_Z4IWa;ik4n~=I^hIK%#-Z#GU^_TR=8~teM}Ku{JP)NA5)z#jkTJc`H{- z;E|g)Ye}rFTi6c%&oi_)9nd-VOJkKVs18r^-aE6Y_T?+ zVttRY2Vntpj`M4tCbcXwK0v`^{%CzMENIm!ef{TuiUzn=*Wk$L8h82mblKNQy74*m zetF`-KfFA(=)T|`kRXlu^`aI-+Gr3NYpYwzb0>9nJS*~?-)`C0?rhD@Tz+rzs)WH# zVdGaq%cM8N3<<|DA)ndKq>dmNSne^&I3$Y?Yb3W1x?JLWRB{L5SNyp3pX~Vipmsk7 zzC`JWvo4k7Z)u$DGk%`yRNnCIn~s8sNk}+)xvf7^S{#kB$I*!uyGt3nHJUWG_RP$` zCM^}Vx+@nfhOO9EU|Na?@VFNL84?z)v#+k)9{*UFZpKrYuKfC`VFm`(mB#pUe zX1ypccWU_3rlerb?-#d_cMs(W3`NS%U%wm*+Q})@T!>1wctqh5R$5*E?OgZ{0i_rG zKN2I|8ZH}8H8}Js;=+ua@Rc4mI_->Yj^>W+o%~)+B>iarzBihsTk7Zi8T!tl;v6qR zqwKxOJ1{ijTG6lhN24SUI|-|}$>Zl@Kdm|$1&G&t`vy&{ldh+<_}A5k9zaBz-*<`> zP{BHP4PR9JO}0Nj00FO7l#G;V$CouqcYdhc1-)HF>6MZ4Vl2DVRYMY^4n3#iYUIh4 z6zLUxH{6!td5qBR)pF0jsH7TJEIl?BOI-QI*Z_@NY4=D+G)IB~u$Ao2X1OK;>mDkF z*K=4_aCV=@zu(`R25Y^yk5SrfDR?B<7yC12Y#}4VP2IIzpl-soF)VV5-K}UY9gFc? zJJRV^gV4bY(({|e7=j4yh4o6jXIEy_tj*$5UJjSX>I8dBdDdi?*daL2PtG5Bt`-R! zFAy)x8@r62GIk$Z35X4|UX~k45|tF`#t`wF=Y5l(^Mh<^EDE-u$j7JR1_Ir|NJ()7 ztUaH3MG|>f%YRw?6TfxvRy&XihryaXsI#PlA$Zj^pP)mmR=fLy}7_J&A1u5z0Aw&SvdP)&J;#Cb=J%R9sBrlBA!K1e|>qaRMu3=r*2Hn&gGmPyl&V>rOhp zz7Tl*;nKUPv#_43b{mU-H`@pZ?CR#ymTwn3lwP>et#+BM9x`;CjoD&;?$7+V$9Q_eu~}0)!MK3LFwz;dEU>9KZ=c5Bo==GpEQ0HQxrp*P z2L%%>_P|-%Y#`3s?3H!zpc`EYW zPcr>ZMds4$iU#vRXr*RLl&xL&F5Kby&`3?^eGNHC{Vvkqih2FWg(-js+*R|Ojy5>G zuE|r0(^No`4>m<4**s+!G6LoJR(97%Ix_eDQij!>?M+XB9H3%fw0!bP^mw5u*PU~i1TZchma}h z)SqYAqzMa5m-NhbqUU6q$5YADoDotsQ4Dx9ybeLtefdZF0A&g-JTRTQ171uj&CrWqc0viTl0NVac2SeL9?i;GMa# zb_ddItEj&1nr01so@bHW?1}51j}Iyj3tnhp!JY1)%NPHPtU1Fy4cgTfU5G_j}MqFs5Xk!T*DwTtLhQ1YS1`e$(hM-^w{_|491d5NYHa&dqG8 zjdVcK(`l77^2wfYPW>rlOm{ykiAgWwlN=pQX;c*Tuo0U~!$qd&&lNB?f3f<_PF0Au z$mQTSg36gsE4KwX5(Z2>NK{ZpLIdqKa6`>?8Ycdb&_8&W5`zKxx7!yc4?ZE*H$=RvZdON96)iw_B6TTF4=)YzMBfn^Ave$?K)j-=iwJ zcIupr@MDRc!RIz=4M?BRf3XW~KcF3h_|0P8VK&JNxwQwxwHD45_ZuICx2o*+ZtYnq zcc?z;`A&FNm3iauy|2?hjT=98Hdl2_A=&UJ@FiXA5x$MxLI@djGO?F-8yp0A&dNXf z=#Sl=qdO)f#Q1)pBe+|cVq}kU@!3M+H*rg|xKGrhJiqR;t>9-4U?xNyZx@{Y?c(oO z1LYieRmldpaS!b}|26~vCij$fXlufDlc5XZFNs}$qY6L#(j7~DM;XiZ@yC|NPQr)2 z=L+sainxeK=iac!R%0T=$P;CZT2U}E4sH{R^KjR_5tjK?-K7%A`MCKWm!lq2CQ%>y zjWRsC_caLQE!UTfMqm9+5DQq5X3T*t-9GDWCWBT=krJUw5n>|4oqXiUR6u%Rn57l_ z>^t_jHy4~W`;)q|e>)H+o`;~Pib!`}Q8ZZP6&@{Ij+4$$pb9axby-^MG=5QpPZI6d zKMtBnc!#&K1Zi|8iq-Syb2pp^?one-tAx{GM?3Z3D?_d&4UUPDHM~q2t2*yRTqkBJ zNao($4@Bm#h}v5p==9I=tP9%JSPVUsq&njPYA4 zU}zs4l4P*#zcS1-L@sLe?! zx2f(SP_*#0=f5}br}PM)LKd6E#>zUk(|Ka~X2m{(X$Iq9snzy6k>a@vLhjaLi^=MX zTC$+wcPjia7hP_g`Q$cX*)kFZJ@O-Sot|+jjv9M@pHnYvZ&!j)mBtG|F`SZ;GK%L* zL2Bd3JAbGk3cuIxg!gCUf5r1_Aj2H~n|DZ{(`Id2N2-jdER>nMcCi_Dcd_!E2)K>3|OsK^nFO7{9 zHc$+S<1tMJ7XVlYGqVVrME*mmu2b+CP|*L%hXgNJho^HLu~AWZ(SOh};NvKN;{nXn zy+y5y2vwAR2uTwc9eVLm9}MP5JA%ZOcZx@3;- znyem?Gx{a;+H-|b$=%+S7>bklY-p$e;}NOTNj@2u8KoCgvI^$Ah;k#ZEi?Ep&o;l< zIDqw>L;AT%NSh4bP+SYEMUcQ~k)nRN0SSe2@fzV9|9^wTWRbuk;uUbeHXjTzgK%otXAq8Bn)8HNizsDAeMx+I?%x>K*i);`udHbU3_eU`#Xzsk;3a>!84O;a&X%db>Dj z$5su{+er$ANdoH(=DiHZIVGB=59Z$yZ#%x0Ynlb&NH&hLv=ssMpZq;Q@828;lppe^ z1b52VWyN6mXa!h==BT38#-;1qw5}erN~p~A`_yxBBp7+1L5Ky@+Vbkbmfn#6AAL6P zyhG)Y`?1en=kr0)!0Y(hlAyJGZIipS zf1JP^qSwSZfle@q)H``=(?PG9{JIsHf}!GKX^*)q&qyh!Rk?}WGHvq$T{_zyzS1|ehpa4XoK5BI6#OQ+G@v&4|DtG1K7Ml^jB!aP@E@`}zB z4i892+Q^Vi08R34vfY|_A5zFuIPNb_XP&MG-UZXS*hcC3QL-&7y0%KzZp}6=; zcXT?n1=l~`PmZ}wF@qm_m$l21^d-hfw|BThFaryQvP3#n3ych{lnS-D%863OQf^D# zyYsBLco;x+EbG^qD{1+Idf~0I$qeNC0KM$=;Jx?y|Z;4XK z>se})NwWJnYFoix9FFOyS$%tFR_!1awG&OzWQSz~p^)g2!*Fmb=c3XucpXUxL|WGg zWs2m?)oqgay?udLvgD%cpEM+kGap28K{nhli4_TKG+7eW{_z{<9lo1t8$@DO=2~M3 zuA>Z^c(L(=D$d@IzDd5q*iN!F2;t_+FOM!++IY&jjw*_>6)}{i{~#S8HqP|hV3v>M zdGu=p`dR##)C~Rm zDvr*dax^zj5>#yc77kj_?ykoAz-<*|eC%f@KOg2c$kA3(Y$&|H_JpdW`4%Y|mR`KF zbdz5x{LHg&;b+ySwES|H2v}lIP=j;U4xDRmr(jb3x|bumy1Zc(+_QI~(U4?5q!PRq zE+~6cNq3mckmNNQjykS*Fp(IQA3jBXGeZ+IbacKUpC3#KsvB`#R|3qKMP;{dkxnfmrNlyqUzd%S-Z~JiGP>1~LMn4eaDKHt0a2E&UT8tt@8IykNYYjqNm^5w zFtNUHA>g?}p0kfolSqX9Pu%v`X~ASx#Ao+ONq9j+u7m-oP6JdmlD<-s7_oI*`elVK zXmO=}s$R;J-&zn&0RQ<=|Obk~mmb`_^?P{{`qdc8$eD9kseh(a+$2TcAK z_8)W9OJHI~S-zRB+o!)RJdY2=6DyKn%ht?$G#zDZ$j2GEM!XF+tzAGW+c=InSwM5g zWA=+hBhQSB6E1CdIw@)=i5AuDHjC1Y0TQeEwU*<(Eo?EqwA&1R)Mj{GSPBhUahR5J z+@|JF4hN6lu@p^w2`|f8QYXWVAR}`bO!8B@|Ir3QFvwD=3i@5{iVSmNG-?->ys}GG zUIKAqvGgWkc{oi-Nyv3EK>JF3j;Q7%=*8w z6CgUDPRC8#k3IGmpb`wMo|{zX`xJTy!Hrhq{15KP`L{E00f1bVfKdMIz(|X} zvgTVWIrEEdfm3sN#k?M^>NaN-o~0i_T}4ripQ{q@SqU474=b`kq4|w6*F%xvnUO|w zTiZrfZW*4doCYDksA60ok z(J*Q9!mVH+#(3dk!Scc`O*|iH$F0AE$2aY)l(B|Bb5SPq?!B8v_%kpXN0tn@Xf+h0 zC(bEARP8tXDO--hd~NO{@FAnBL$4^*XtvB!ygux8ib6wXy<+^o$|9{>hn_p7 ze&z{rU;k(40FoB%0(>&j@2MPDe_-pLe|0U^Ubcq%e(@q3 z$AYoBqlrMRQf;i9(c%D!O~Z>{`?OZ$s{4C|ewUUKzf$ca)&mTqM zt>sKCf`T)(`vf z7Z!8IV>U#}wFIr<&WxzYdiqfWbV;ULBr7VpZrq4;K;4wiosiBgCL#1X)T7{%d*I@fd%*vc?vT4G1i_nvV<4f%=~|EqJ=8E zX#%&msv0~0@Qj1Km^D6hAcZ%6yOpsAf_=V`* z$N>PGKF!n~)C0FOf4+&_pad8=h)~xy5edgIxpdKQgLVH>AM0l}>i~mwl&^S1!PHLi zZniqSJ~b~`m*A6L}&VS>o98Z;w*pom|xw{3EUV2!aQ z1l!((z_#)#G(x_DMg71C6|EckIyjjNrq}G509J@0izgjI<>0Ak4FX@zeo6VUgLWkX$T&w5%YAeU`C@vYJPRbcvYrgywJvaXXE8p3C%aG zT?IY*v@wdM@vSk?R1o&9DCgR*{tS=n1&foS@ngjYRyAAW%-pNz=Uv@~451bB#7^+x z;!)kwA8@sL6y~pD>I-T2gCRnci~niNsA~>|nxUH!mb_z>2dI?M##TK&0XQy;*@TwA zowhX%q@oVLbQd>*&^VI0%B8cgIHJgTr7QC8VEoO7JP*mCNjCK~yJsBZ#c=iiu{KpejaZXNv}Jv6rtqvtAXIY z&^>Y?YVqq4WB-VI9ILM?PCK#sm=seqR7kB>&WL+Dv>rKXYbZ4S)w&0Ig*)!xJK5%& zCP`5RLZokx2nyf1eU;r7zl(i*s12p!ca0*U0I?fg`9!jw0jSOXAZx`RfouBT;VY$j zW1}(|U556w@$Kucd@<6Lq%P82e|t^fp|J!qHN&P>sc+%fce!bX{hOR6M!+wN3}Iqf zu^F{hkNcceEmcu(m5|lgudMgE4Q&iyuhN)Zb=!5sg}r8GxVytAR{A~s@aYC;A~`be z$oL4?BE~ChpyI5ubB&3(*1`~O);LUDxNMerw+|PU8so9?hcO77#Ia%K8PVGgmK9pN z)rG+=8DIo6R^AtPewd&)UQUtSqpROrS3nbps6wlULGN895D^|JUYaJGrkIb3`Ual$ z-+^pc&!*L2w$HMe`H=WMnddnSPsl1k$d9? z!>mvsON-~Dd9l4XA6u|_KaksuibR{X0pVyMX5ts*nG+oFrB@(yHV`m|# zXO>M6V;Wu;AlH4|j?$MYk|wxN#5DmTAp+Doq-Vf@2+&N779v0;t93mQUYArO-Bydw z>6&TO@2A{q9q2`WclGhsYU-bdPwX~OnXbCd`ZGN6v1|#?W@?&b`ahrZQoZT?wdYZ%M2@;!$yZ*l*mJyAF~QM-`$XDpIm&C5cg4c2n!QV} zGgCWkm9NlVPC29LX|#v;dWb<~JP`86%(kbfLp0A377x~EdOajPFGh~ z?~(W@T67g3c>m^XUb^(AQ2$1FrD(_si+q8_Yw2{~m>Wl9J{4hm zzwd*AXJcF`C{8}AOP+UdR#CK^G6QrsBWWKWCP~RsJVBq6q8jha3u#HyAF}5g`h8f> zm^i(5KC!y~oc0t8r`H!5@88?_<2d1GUr?}#cVY4ryl7>&$AhVqT&B08ywfA7nNjbz zSf1c#3|$cqaeUrxrd(aHyYX}uT_8^CJx$=12L~fixmTY_(hg|IR2F%`iNvb~=^shV zpYP`B+~*{JlY)PqpHtV238m12$B;M_T+KOO4PD29 zANH;t2a0n7zV#NBrV6Da9}2Qk>EV>fNA>qVJB~nX+BSaFD+p_r%c*qJ<{Wk4dq2_H z`K+{qZXB-n4NhIZ-ct5~#`TH_cZSsa+R~Y{pYao0(A8p$<8lFdv2cU^Q))F3c2cN$-vpgMG(;z>2a zaer@P?d#(YC@ZV9ZTn6i%klB?p@ZJ?Ckx36;-qHfmM8M~ro#_4wppEi1C^ovrP1W3 zI-$_xit#ltbDc%#=kOe7fcE2~bd-mS560i7Je>8-Ah)bqFoaHij!zXcDEu=hvmvkI zcv5`(pmgtj_1O2gJJQ|d?-?j#b*%Y)&vRx2nZxrQQmEY$(mnbRbrL3FRM2ux%HB!> z!FWk$C$%mP_YHwl+R%>|5B@lYqIkl;>HoMD_`4sXcjhBczpr3L+3f|5SD>K5&uJbU z#Vxv0rdOkg3k}oLV4{pQ_K%b$wX~}nz zY1fY|_3@sbBAM4x?IX^SbLcxRX2p&EhT>H6nVo8^_fd&7&li3aMKK)o>X&L*CW&LYWcvVZBfe}n{@Pau8RJJtM( z;<-?Kw(J~5gYG8Ts(PtShT^;qG>02KADteLb*-sM^*wGS5_^(3W}cKgIJ6}M{aiO@ zo+Oc#b^B8a_h~7v`3#RQVx}#_N#4{>%bNpxGH`;esUIr$c3MimOwU(tDJoD}b{@c( zewp3|J>p&3S`SbgNvm-g`uh6PWY^M>;r}wtL6oD)cJN-g~AP{%qTJ zlX(wl)MxE{M}5`_??MNFA{PHfCZ0@Sq8&CArT=Au*8k@@TDW;^^LGXhcR>)i;>*{r zjjOp_o0i4*M?GD7=0NC5)0Nv>_5EEnaaKHb=#AW0FgvF$P3~T4KQG>XS9Tt2>cAZZlgIXX@7Q{n^vOfiiAw%R0p}i8 zZwcz>m;+p1K@~Ekyr2YpHeAMUGT5E0mZ2bym+_U8nOf~9T&ve zJF;JFEiF&bJfKG$71Ny@D{aWOOb|gqi&00OWMT~E+su1%mw`RgY{Q&9j?HI~O!&_9 zvlirE8*q0qBc8^GgMqs7WoxOL@Ls!uRU45ciWfycG^{F(XhwcaDV=btZ?EM$VIIb9 z>{{+xi~q#}mf+pDy~1h_=k^RYr@i|1YN9jMUQkhYebaZl7X)lj&xX8w;``hCxIBI;BY zGS`pU4qiA|e;#2HTF#tbR0>~5G&MEV3=Jdn#v$f6g>3v*UCHW;YxMIz49ud?o1Qgz zhL#ECQ!4m=^S|p>`H#R$R}tgB=$5WSlykA`Gge~<7SR;Twh|(Uj=f9f$gycuI>i)w zE%yB8KnJSJtzBk*Cfm~<#y;8{S7W>3n$Wjgtxro~)eZKZslTH%e{WvP@r1`M=gOx) zccL0{>#SXsn#yKk&c)uV7Sko|W+R~!-UTLY9x#)uSx~<1E<0gNeC$}2`d!^a)3D)e zk73?tF|)Dg?{;=|ETnCrZKshlUK(nV5>n?jvT$^5{A_$tD%-wlEACy(RN(=PoZyFF zZlA7~rRH50e8pF{>Tz#|s9>CLI;r?f$B-m9J5^Pkp=UqdOuWS_9sTjgw9?W&qIJnN zI!=M2ai#p}*5bT)J?u&yP4Q7WNs9U7tDjXr1}qvI-Vu0#Fz3#$oSj3FAh<4Q-F@pC z|IAh%3pKQ$)2`=kc?LEa-;`%(*+GbmCT?F?)5!{2Qgw59?jSRU(=38~xuM{nxUsQ`SI_v1v9o$;o zg3i?&3b$WrS3B-5uQZgGZ5zbVmErPBRdSZR;S*C^5<}L;iw?Nzc&G(Z-LCK@%ooZf?!zNcM}~0}TXz?wvCn z@qJ{6Cn=7QeIpzKr#-|PKIS!7;3^aW52tUYe#;AL;5D&4b^YfXdSXK>c%MQXP$Q<` z;DkyJ{Sf{GE~)Jl&CS}Cv(1`GQN|}sUSA2#&%3*&!iX|$M=WLJy=w95Po`TIlfS6* zh%-=_(;Sr)%O2;9-}@Xl>A=1>`~<#o*8QaaW{)0>XUZ=5(VqM;hg+TBmbm4z!A#O% zjllv=f9#vX=^lKQaKQM5)AjV`n3;n!nOt?sv&Ws=IFkxYt*u{}U&HfAe8@PL!7mgl zaFp}SWkG&C4+DeSCPvqwkdVq5OCf%p|A(^gj;H!>|3{>hjzY>jO1A7|ugb_MdsFt# zj^h{!r9wtFDYEw-hcxWHjxE`am2nP^@q3-({(OJ;=l*;j_xHaeRPWd88qe!_J+JHS z$FT@mtueG94Au?P<8#=lgmA7VBViAdgD|KBJTu{m?a4hUh@#!$< z+i1TS7~r%ZOoHJ}DnDH|XY8v#f@rTxJY-3NA}9DNOf<)TM^=sg3Uf1JV8p|DrazAp zdUyu#da0Y(Lb}dN()~x;Nf&8W?o0FoKcn2JXaH z0T(|erpyD~%r#&bJNjyf(S>}(0O#{OT%*Zd)yl7`H-V&x=JAn0R*#x0SakilN$Ct- zznf)Uk+^NX)qE;vW)L-}|A>i{kxRcU4eEnK@+)+pcToY8yP0>6PKsJxSb|cdxQhdc zjjbb|@2F-RAd*&`H*|Mi8mP(W_pPjmm?-Choif=gNuWA-hSj<0csa!$v~8b%(2bYE zqF2)0P=lHO0{=yn;Qv3GXa0zn3VhfkwebAUC?^~=N5SLqgYYQ6Na|k+{GG&I4$K?Q zWf{$No4gKsvDLiY04L9_nw!g+9rUcwhrFk%@sg(?kY_gA8lA0w>o8t(Ycw~5-dE2; zxoTu@cLr>*cTDkhcc5AND2UHw$wVM8aWh#gfkMPDZ4K_y@7oVeINY@{!W6~Dw7(O~ zLCqvOeVm<0=g5^fqjqc3)g`6ZZ}4u6@zAmnx`boCC7bx@8=SKX^D3q9H*iz_gmaa% z$ASJu4Y;|=H6GNf9NdRhPL#~Q`^@i<<$ve*lhvGs5}_0v&wK+yd9^}OMLag&F7K7{ zT7+g}sMsX64#S|a*pUiq>fo_9@K~FGqo3|^YFA&_s>9u>*WBW@4L1x zQUWj&c4v2@X$vlFv#zV9?IGzLW8m3h$I*Nzyw}rXVD}sT0j;q{+olSgrKU??_R|W^ zx)#)>mfdZDum6+_!Zl7Z#h^a80Xxza|6`mAplpkJZfNnwi2~eQlcBltKinLv|Hz2m z)mwHtHXW3dlxpDpYODd^ZuVM4E?Oroxyyg=wvhc>VJ@v2ct|D%eU5{+lN6-2V=yEf zmo}QK>+@o8PKEHoM64O_81AOLNibXbhQO$1uuJIqTa1d1rcEjws6kYbVmp~)2RqcB z=F=R5o`}7z2$#l`mAxlr@WwqdbT+Y83ka#L95g}-1rp+URk+X^O6icOm0<5pp1nH( zslQHTup_R`$r70#>{d4SD_sKN%{Y(!arhI&0a4!rY=6KUnq5L#8`4u(RvZK`2s+Qe z8Eu)!K1?9LRI=0{#}$0hGI6O|Xl`%8S%>PX=iCbm2amBsQIzH6QIAKSZ~Osr>7oAQ zgw4uPxG%%wj*NXjXXDklHzagvia0QO>w^Y96C!x=q1FWNw$TA_=Jf2h1NlcKDU15{ z0z1E2+h93I3vk|;uRi(kn^jw>YbEyj6XP(36}an?c(^TIOYBTZsP~%b-b9KJ>hxl7 z-m9?^o0t1b)w@KbY9ojhtw5h6%Td+oiS*IbI?4L6J$@8Va+QIc&j|A`P_egkMN20D zS}?p);yAAV_Z9DTkRNvPCJ-5KmWznO6Yl{NsTCE!Mh2$L zxw*L`TlwXhH*R2blfWeZvOie)JWrd`8hG%?18bLGwQB>S(>JGQd+R%HB@Ttp?mGkr zX(&DTp7jRJYyKtu!xUwPY=9uOO%JjJ>;VYabl6ovCuf{&ELWeT`RldYe&FD`&^Hdebp z_N z%zM`mBgo%_-K&}uZbFK=O888o;*j6k?_0FR54bsZR`xjmqKxd@>j{w`ulCq>x$0T| zaBWv;Y+|`YHm?rzajn%&yEAhBs4mq6!&?((q;GMwUDAG7m5u3NJ6M3#J*!lCbr|NT z>WpB3UE8$^|AxJS&%{7=TDcg2)1L&=$56Erq@WP&g_62T@W&yj&f=T@09AovKJ7z%KiMVQYQW<7WCw`xS0%^nVC3Y~>r?&Z-onz~9D5&+}m34<-y-@zBiY?iPyDUv^%X~PRzt19hYm*GAHB0w3rZu)I=b>}B`B*~aKN`L z#n#tTxr7A5EcVeJG=F`^?GnFNjI((h?qLRp93a{WBsWGHt?AJcW6zk;#Mq?9BFUMR z!C>9{#)d-1tA^EINPdC{WvtUCfr++V?aH|n9v6e)u5eo#E7FD! zNsMum_(TwKSfkH6WeMzE6LQ!R?~_mYe%U|YplchWa!COfI_*LzrK(q3QR)LhTY89} zBa2x}BGpHFS6iM#-6lYM-m4od3<=t}36{;farc@0q}yC8oPbc<=;~sm_z7TF?yUuC zYZxYH?|j1ae&1Xxof!495==~>8ebR!-8mZuDnnk~d&$CIY%<_t8=pUBd79N2$4gnx ziEYsqU`^)IHmbtncwP5{Q#^SF(mlE7X_!m9l-H0)yDJJ=g<%tt_w*VDoO9wjrvTc@ zkSPtU4jYEI&;P)%_)6N}M4RLoNw#Tm{enY~Zk_SY(UkAiyI}N&OIQQRHT6E`MNE;_ zM%?nGjrL8m(z{@$$?4)To{|~^IT}Z37#rX|xLN8_{bwY}r1-wb$$0z49j;obSAh42 z>Qvu_f;Wwa@7t#x_%GNEu-4K`R1gOwez_|`17nCY9S(S)0(p@zU{!-d3F1o0yX$p` zu=XD+pMuWT-Uea!>#vcfFSF-p88*s;qs9hp8@GyGM%dKzSW0>$5g+952MAcawnOz- z#UZ%*1F#}N!;p$rt&zq$v;l`yocL@5+!qrF)u+7fXXYDbUgH~Ul9JC2_#9)8bF;(P zCnVDG*^eFkRaQpnqg=fhMmZv@#*J&Ur-n}5Q`s!-`uIRvk-WdXv{ZY%MEhymeDx-| zQa`Nen+uCBWhRLmLy2(!wLuN~T~!udBHYkN(#6de**~aDNhIi+M!C4vJm?|e6E{yL z1zsNW%`K`_0stNRCSC47XV(I1#;96x%kRH%M0FV4jPf6V&Xd=DUUyEk0Ke-WJWwca zrwI}MWCpF|$R-?UgIXQ!7U< z-z4uk@VDEW&&j8RDd}_9DA~fw=rh*+Ju@YdWT9|c?3G~|NiP-NQ}Pbc1?^)82A0EO zh&>D@-^n2w$6hao09JgGyqHLDxgiCFUs6v_gF~988^e=bahPm_3GMkeR~(5R^ZofJ zVbN!=R1;;nI-wt==*zQBVs*+3(+A9}oC%=&$Eq!SG_;rrc6?2S4*096#kdwsV2 z_W@LLOLWoolmj(30#$^Wq7*x$j>i@we(TK9uN8$s2~SP04NRz$~=pcX!1)W9Z=q~BO4Vci*b ztZ>H_j~(d%X4GSfz^2yWd~NdVo?Av3-U_JsP54e#Sd!?W9h+p~*!D)gr%ALO+*1Ds zhPOb`9=&ZDq_Rzav_F~bk?0chwYs^PUrwhWa8csjeY_Twh^t=Z=}CdDz5Pj{+Y?xQ zuOq&B7F>^sEt7+7;sD?l%we&}ar(0Y)aA<@0)HzUo|&uTwf)a{Mhg17)S|j$K5Jdv zG&tKTM{j#e%57WI8SbXJG#}F*%dBo;M7?}xffS!m2GAcZljS?`C-q=;zX5%9Mb~$u zvg_c~0J|`t(F$SRe4ZF{OzV3)=>DtY5!wi5Qg=^NctU}5Z+%sX2+cHXY&RbXb_kJf3v_1BH9 zo~zoLx&?s+^4BAs!=uMQ^%nym#D-|8nU--Yi~J(x4*M3Ia}qB?`+L38olW5k`~mZv zck;XVt5bZFG$yvOzYWSeKYk`j7LUyn)(OlT2LR=(0jf(g#?O_^1_tw?+NGW5ADFa4 zT%C$`X63l=%jzw6uxfD%d_9;rLG}W?@S4fD#+=@S!KqPbHUp{RA>P7qXBw00SwTGK za`v0p_D&z4ZR^0G!HVV+=f$t8crB`dbds$`?q~_6@5#l*q0XxwolYx89MC*nLrX+5 z>eRBA%FXEKBnLn0I-4BHI*oc*^?!VJa46WHv8>;)kHiJMF%mkd;$XvC83Bc}(BZYW z)T@gFHJ&%h+7FfhMqeC6*iNM2qsdpMz#EZSk|rIG3ZVlUll7>64YTTFf^W%vhb zsf~CK=4awhVs!Dl!pgK0e$%_2hK#kVOx<;GnDuzI;O&#(FJ*^IvF^6II_M5wtP&oZj;fa|)B%@eR~9_4H@1~?jph>fuR#33T42f#*4B6}hoAQZg^al< zzXn&{{7f%X6zsUHDOvkP*1?|&7R#8KEesARFf@ynTYqwx*~p&R;QZ76;GjQn*0%zWY|ySkot^SZ35= z`GnUI7sx3{#%0X%yVGndK`N&%?*RuTV&bPFjhEPO5Admg4s<%2@Xv?dW%}LJ-Zc3@ez3V4C3#CSv$N*Kb^{TYgSGc1DjiemQ@Z^o*JH`kj2*?1 zYk-Fyuyo}lyrYIuo603Sx}hjT+tc&<#8xnb1x5Q4 zO~rxQ2k5tXmtqbkME~%U`IB6QBg2v`wd_8#{lDhlL_TF`tCbTJ5SWo|5ht!tKqlg&)|7Q6KAnYZyv9&cHDtwe>uC1XVudJ-xdxgMo zZHWbSC(SLpi;+?`VcXac9&KQ37`?Z9ga1=YR9wty5r~K^p8peEcCO$?6IYIdHq?V) z^wO4}`~6e$BL(8;XVL42ZrVoNf}oYc)tS}t&BYf-mt}8P>)$;OV^D!S8C7QQx~-XC ziFWply_pBPQcBwt^_n12OYbJ-p4F?;UFurHbzP3O)=413nmxV2D;Y|a{n>gm84$$j z0))9kNw>Czq{<^2J2oHl_0X$g*zUuoof`; z-)ZL{Tx4lBE9jkf<=)cAZ=rwIwtrq-blgRqw9UH1-MAn;AqC!H7m2Hv)4cO3I*UMs z{L|~WHSYg^te3t1XRmpE_50GVgS{2!C!X_T; zL;UeCp50y!#d?aA3@j`PEE}x`-W~YbaySuogx@P(t(m~ zKG8SS8@pf9SGX;`4ccG#5A0B*`s%4{hU&;wL-m4f(X5oj+wW7h=Xs;2^Vp?*avl$z zI%6~-nV5eX)-7#}gs(}yuXkN^&$RRMRFAKyTfHn*Y%D1o1S!wmdOAC+Jrla?(V8p{ zOFL>g$jeqUVq_Lq4Ir#`Jg9S)v6<}GvmZmZ(_i4hCa=(S&lcXq5$lJOOcHv`pz3De zRKwl8-r&Jefh1?RLpdg{a}#jAEr2>o|77q2FT&jc^5Qnpl@)&#Qkp~eUB~#20r~|T zB6MEe*@C^En71(G8zmX7La_ADNOm4bv!73YM44go3cZ~!Aui5=*d>Kpm+_O*qW3p? z9v3{Ach4&okhhkc?0`+=(s&0$w+}Y~BOt%ojk;IB;PMpQ1$FHlC@Ww!!d*Toty{cx zUaO#pD;Q3;Wk8E}+s-G;#dxhIq+l`;%9qwm0hx(Vv`PW!;iWCXabo?iV<0aM!c8uv z5r*Rw2Dla-(bGY{x4j6LiXXhPL(_sB4y6`?wp*oSouO>wh0b+$1(S>=omCnlrPd6J zWCZeu6*u^IB0Vi)91Leh^yiv=4^tTQxGCBF(%`U;yQ7tXVePdm5KmDytA$L!)l_*b zk|6OSk}Z7;$CdloIL#oCrz_9SN)0n`cBN;3uQ;7NrE7mnr->@$=s@Wr<`GpY}x>tpez*;4>8t==;U_#H@BxKvX+Hv z%~^>=67bYk3-=1Ewa)NC7Hrq$IeK1;ZEB#T?MMd~oO-reI@qnPbmTZD7M$9CCKwP4 z!L7_c+EH4$-UL0#=1t#8lkIefoh>C@UcM>&X>=!6rM@Ja_UcE*ucupxBI_;lzUM&5 zp}kUr74{TsMfa8+GKk!A8fN-%r|1pfes~yUz$r%a5&$* z#TTDBJq|OxmiD`A87d887rzGg1;k2oEncRyvFE)B@~}93 zsC%CE^e%ME!wEyQ3a}TIVQiUYu=pBPYFDyo0_XMQUn}DndWeX);FmPGpHVb>v?w*;?57cPt z2e0V_^i)i3&#xTioiGxNvig6H!fnqsh0ZqToT~c&}|<94zyb%i2L;X)n7bW8Sknps$eM zd)oiWnW0nipu>WAqQu^$`&)*R{}@ObE>LH}yxfv%i+}m%vTZMI-ss^ms8}`3leC4( zK{D`yPcN4ChG}71E?)qO29w4uRH;u=TXnrScR}q|h~0Q`N48mij;d-?ww!~Hs`z24 zCZ$q9&;jvQ#i8X~{(-GIOM$1=oz z8~xojXG_Tmasiy&|J~aaadpC|GN==}pkAgPuR3n(6 zWnpCjlxfKWK1wN{_$Bdyaz(U7^`7AZ$LGjAf(UW%{;J$a4Kxii93sg)P@r|k>nDR- z@#45p?xOiQ#$kAwUT+KHus%w)k41T-br3l-tlZVoloDKDOQ77iQP^n+;-{mh5drm1 zF@vMyb?ysUh!T)57}a9zh_=SfGizh#Fw{O~@A^>=%{kJXv`8?r!U-A>GPU@)9q<+0^v?-1R7Z!D}lxoh4>=wVHD9sU_(ZHd*ZXEi-<~ z_oY*QAF*wf4VFTq891ooGpCuPfO)cts4w&$lhFgG_*x&+X*kRB@#dC+DK8C>x_r82 z%GeEdZOe0Qp4%KI4F6Iq+RnuazLr*aSI$5npS~Tov~a8$mvh_ui;P+a6!MJ1om0P= zIJT(KFKPeeT4UhoXXj=IL~eq-n>|*K4#P9PzhhyLuR2cIBJ4)@!m+zBp;E5;*@CNU=z{L$b0wV$;?E@KytUBI-<&CTS z=8dm~jWWI;x&b}$u|?b?TQc{Y7HsGiGFT3lW~(?vL5 zKfaIwRSL&v;#UDKlxw31LhdcWnD}{Z{pfe?VjNgf%vd*Y1ZIPerbnFDpPDi>)9)?9 zc6hZkuUC*rp5z^l_i45*msyL0VXsT#>h;}lT?%#2MG%T-iXTRb=P8}yw>;Pc8pPQj z8=YA$E~HTVq5TNXY{aODFVG@(u$vKqf}w7H!sDZxy;+XIEDiK?5Xdy&(l`jo)+yw1 znnEeUa4~K=YA6{sPAdLK%91(^VZH=jF(%(L#A5OCSfjg(OL&u^{1;i%!K14!@!$Ff z2PN}6QNItU-@O~y-``Kz?c!Xcb6_@p!~nNSBDkW)TYB~0-*{tc0dlpTv%;F0Jt?E7 z*uI~ETANt>WSj1N#tmSEtIJo}3SDXPyiR^p;(fy$imN|x=f@eD7ZHh0xVpzX6>~P@ zb#$G0sKV$dZiqusg?h%bekvdlj0X7Wl;ee+ZsI$oYmwZ9Supk76ED3sIPS)gI~Rvd zvc*%7wZo~nmpDdg1x*C^X8H^=hN`BeJMxKPrX^h`4qVg!tm3+##*@dVK@3lj!bu%m z@B#hv@eU2)7slUWfwU(Eu!qS+_-U+b^e%q@DIPvpz#P8M+H=}}9T({*lXL>5*3(oM14*6B8?XY!f@6-TFtquL}HV2mSo&eBE=9 zchy)ux~Z*a#m#VY4Cc$our@Mm*8%uOb1e!R@N$BzOX?56nb2>Nu1x%ef z#Q?d4dNc9t9uQ#qHD#TFhL`*{#{f_`gk7>HI}%1c~~yA9ByP78r71aQkX9T ziaSXav{0Gut3#!xo?q>_^qM1%pSC`!;`u~!ANKA1MV{Z&Q7ei+BpP9a{tIXR#A^|x zO{@iw`6<;j7#w@ddooT|mHyNW$4>a30+_X8YG>@3QUhStZKR9;Xx6xSN0W;8#Tx=S zt#5?2BsO$P@UwFYB2YOwYLfM4jRJ9eo4qD15tfcdkI4J-w0SJPk4Uyg(i}Y&^vHJp zC|MCUtC`25e?{@D9JNfZDX1TDOnYul*NwmC8V}490`!TP#Md&~vS+{?SDpl4&IM1L zbzHWoEhFg#R5h3w9GR%|DR>f^?|YbiSLDt=UxMXNIMF+ixw*MD-D>9KxSbyladlZCMT0>FixhD#dmkhG>jg+ zGF<`i%Kse*NrUh%IZ_jTxE}tyCB0yf^C;f>SSJ1`{*ueFE)ky)yfDI#f|VtS@d0kO z;O<8Y=s(1Y_fNp-Ju7?Tg{L5Blk97tl~+(^BJf;e_-Ha6+W?3p$EVW_A5t0eHfFR2 zXV*&pg6?mcLI9;6ZLhLwXb&&J(G$qf5*SLfy zY@w~AzZChT2aM*VqYGj4`MsER(>q654aF6clkwT?vjZ{kP(A-FZRE$?NAd2r;1kGq z8td}D$z0xM75%oV9kmx$xJ$mgjZ^QE4GVvDPibrELuLUeO4)@17{j0bp7wMHuuXAa z|CHYuFx!RGM95ItzW4-E%bkPU|hVH z;MLU8Av$;`!uq zGhtxEACUWFw+X{tH9y&eMTP>^+HJdBP6K#_o#W7y4!%NH32_`Bwr9NuRE-&5#EtNo z@_e>HL4VufU7V`LnY%QjUwM7pN3!%n&!S_6MYkil{G#j)3jE$8QE~7|(Bh~qfhTO6 zvZ%vgq>PIa322qgev=rj6cI29o8Pu>oBIe#T4F8FHu{M3U^%ATDf@HT;}wQH)c3Z} zYgeSpfrq9s!x{bjVt#w*CYZ`}n_U+nn7RlM6Xz!VKzBD;GU+4czb2IbF!RG{d2y&bfYz;fs>e|NnPuIwBSdE^X|v8b zh}P>fmCTLQN71^FacGkpw%7pHIGgWMv(bQ>{myk11%9UbweVuQ6!JlH-KnI^JpRqP z_6%QASd}4hiFEmy23}!Y<*@J`o;d!q30GIe;U7VN@I3* z_Owr*m>EbqcMVLoBi*j`ip=t+J+|Uc`1dJY0oNdE&K`DIYK@){X|3qQ>!D>|Siulq zMfRSR8^ggSz-@H%^&sJFe3M+tRlH3%;$FzkMk;Rcfc^R7y#>&Jv1p)x`6&cQi)eLh z=Hxwi2b$By(zLrj_7#MNRYgQMx_++XrqAi{{_}1PIyPrslFzQ;?$Jl&L z;GPwr8ow?s*EF)VO5?0Ee{=9~1T(Tmoj9^smO2n!swx>=wAH6Idb_GaN6TE(Fka9r z8&$SgarA!lJENcLQ%u0@SBQjCkJxrteKS?q!bJhf14ch7yx*W0LId!bVyQHgmO!Bv zmz0~zsAhEmYqzn=>v0@T)`4{W56}d&PHF2aXZim&sR7=T7c2ElTZ)O`zQPQJU^}mM zi|g?9YxP$pRa-~g&qtpED@)72)^cwOi|W7NIFHX{*avy1R3YPXuF(+5$$1x#Eyb|Q zr*l=@-YksEZ>C>_9Z0F&oKd{|eo>>t4dq71L&nUBd|G?x+ss3qi(C)zNj11Y@k^p+ zc+1B5J)F2lmliDdkW0SSCARGvGQBEy2sOs;y@x(s0OQUCmv>qqpPCW(55t$XkBI~P zuuF(lK895m`Kt(Vv&NQ-v9Mju;-`m>l3R+O>U?pq*}Y6&;BC>T>GHMMomSo zNQ}YP^Gz+br+!0k4aA$G!K``P=J{q>ue4{@j{uXs=c8GpQ|C4F()s&~t>9l^Ca7#h zR~1M(6Rl$ju?z&AAlg>){hgr#G*E7$}d zWnZqWp=l2lmFYcUp_HnsH=3NPlBf+E(=XQ4SNPXb3PJQ;{Y29HX1?>?#|y$@Wx#5T zL=&(m(2Evw5xuU2&y+MhhINQqAr&<{Mo2#{E2{dRtu?MU%-3~_biIjUD(OnTtWz)T zP67B{3re9ppESst3)D}ll2Bsom;V;Z$8IETuzn|m*5}*Xsd%fi7LmFwZ=mL9Z}~G` zTxWZtp^GVf2ZjKZz;b|XPxq~avWPCTBdwYb^<-KPG;ryB)`I7xnDGa0cj-Mp69GQX z{P5o1{M*4nJqaFNfW}o(@_i%k4^1ePv+xZ}#LChfGQC<_eftnz4@^r~o2r*xNt!|B z5l^1;)=2$?-L0sgG$Y?NE)J%2&G4}cc|QF+3X%KzyNu2Y_`$bt)u3OuZR&*_P-AG! zXk2>ZLQN^#`p)}wO)|7wcc5F9A3>nzH9j;nG%k8VZ%P1PM6yb9Rv&alq?bmguf#oR z9r|-@&k=iXZwg$%Pa{Pk6~4cZ>_yu*6bCvlEk^sV+@J%YsEvsEO&n{Oghu4j!mD6R zIMv}Vy*ANMX~^dMBolweT8;p0`D|0m*Df*YfaoGZ!k;(ZV|WlTanqjzE<_1U`bZkR z+R_0#5$@P~m;miI#TDG&`Wqoi$>U7U)im}pfzi>M8X&$EXZ=sq#0)dp9A5A4EADqe zs_|*zjLBoAWBRr{vwRB3IfX zxTG|i3w0T>7rwCSGgN6A6l*)Q-tbEsXw*6?=|no0jv#9o5G>}@?AR}Lt#^l2BsRk7 zJ7kbd8VY(mOL6FkQbQ(w0>ze!y6j!^7(!12j2Lk zPy_=0$^b;WU&I*U$G~o@``ecoiQPrylf8aj?eTR62DnJX$1sAtYA7+S7{@!k)T?@v zkpJoC8|3WsX|ME-=j`%~Lsvfe`6)}~RBF-+J=jW_%cR}r(k);1zaxB6O0nO49x;@i z&7PH?uaYDQL)*piTdGVPI9)rTLCD;Jbv~M8oD23V!K3_NyqK>(m&@!03xv-&gTiaI z0D<|Wq%VQYzOt3@Yty?Ui{cHK6IU8HkV7WctvSkiEIpJp@|vNMa?bfNP&~XQI?CmD z%WZc+D=_z!YL!$FhCWdQETed^0mCwaKxTR6lzc;8XmSKQSqxsUy6Qmd;`REsh+OM{HJ08HaMDtwL z2FpKQ|6WocX{ovFxni6!+kNUnQFewE&s6uRx^NrEHB>7xCWsxJLDXfH?(R1Sv{(}r z(}j`oIkUzROjhbu!-`Jzt}^25`>ResAq%nJiR8?#kqd%*h6CV3Sl`F)B}%HuWl+*S zf_fRySQ%Ph=nz+nE)e2EdDoV3abBPdUv9XTQ5KKl0-GT$hzi}F8<*zTk~q?WWU&X?R zJ5GhsYk7VG=8XbXCL6xr>`^)pf>f$*c1CKkUOcrqCuH#~pH07RsL)evPj&*u--<{LY0i;%lllBOXjdrj)1U=MI5b5x%kD3wYbVcQgW z{U4@>Af&1LHrtOmyk}e!ConY56AG3(jEs`eI*IJp7Ozk@_dxH?Jsd0#P`V<(Y>W4W z&=cfZ)(-ATKUHw@^#+J$lIKcST$91?CcN}H6>EF@3}L~0RWehc#aAzEUfh3o8Vt{w zC z`jUWJY_G3X)7fSFjWy+1|IeYirnc*0{jJhTA~!<8-kG1}Vg*Eir+xJ%@;E6$zX0F3 zR_m!G-UB{dLL6|?|Ba**@D@Nd(5Nvop48?d-zT61`8WQ;kXAcVhmdS^`$g-| za`CHNB9J##5xLHRDy|o|LL@s9rP(FfMi(mFFOWy5GUy#;Xy7U2I4;_HabEFAvNh+c zINXN&001JC0Q$Vp$-CG7A|CKSyE49yhT!3)7y5XM0d%5f%R?n;t63QzOG(=){lQ3? z#Ky!DgG#m3JFZ=Xtt@%w*6Pd4fYdGU)Av)u_K|IV4HLTwS6xVP#JrUV!&6JEC@_n3 zoUB8XI*eDFP4n}dMB)clAQA^&{^UzY0uXu@PyW`)Jjq{0*Tsvm+ufvDOOhY=d}Ns~ zP)FW!AzifkVcD)=`RSk*NcHRM4igP3Z0ZE(03LF<%-lo5N-fjf9e%(Q{cy`tg)(f} z9;7`}q5^@;t-jV4Yg4O9J;dxV5~-V-Ml?N_{>Bu{j>Vt$3Kg4q<-Bq=M*mUfJ6gcW z@8-Skix8ag%1>F+tU@ocn3(53lC)21tT48Hl;<~t%p(KMt%GI_;o@NQ79wJ|da^3s zeVWRc^G%9CdLR(>&Su9V08e~8-2M$PvS`b_Ew*Bl!#t52cwrcBOZS2 z3|XcQ5&pHi=@=2$$IUSp{u;3U5`?82J;mmjwkbK<#gA@c381Q(ueOpy4 zy_qxeeb8$hNKv1?DzAF{ZKkv^67EB7QoT5;U3S|$ZJL75&&)W?Se-PpZ0o_KU2x}p zSp${e(?2c`jGGeGf=fXOM+drj=u;N%KMzs7D8(L>!4e^N%Ef+^)TNAXO*W6jy{}4J zmok*^lzjD?dQ6w~#Hloxt;n8CyW(sN* zF>co%IQw(cPslbV9T~=XPiY8es%J!Wu_%&1UCPp{`ni#XiyIH)Z@JusmJ`Hu#l{W6 zbTc*%`p{*msf^{=_Amc#TvMRgLy?RyVLmf+4dg>Au94%>o4+_Ysa!%OqDzMT&zz9D zjQI6(xC8SGS0`es@Z2MX#_ht03Y2QvZ4|*`rtcXB(62BE`hHBAA>%3eU5(|I{qG(U zueNBw{)Ex?-AV`Ja&{$8x++3*XX^xehrReo!FMBlZK}qO*1#u&t=)J@qm_>%?qFdY znC$NYC1*I%^!zNvPkyZB7yq)dGCsSZ!v0Gfzsvr{t_bVtYlVNN?Y-P^)An3B|Hh%J zDP5F;;uAOlEqjIIY%A9MU~fK`L43QP{OpkKyBFX+V;3a%&j|nAY=vfoQF8%rHI9Gz z&8olV-54VWgC3)ss`w2CpY=ztwHnpP&eVT)f``R47&EX_8QvD`(FBU-z#~0U?D@0D z<QJ!m-vU^aH${fG(;?F zxlxd^B>H?2G6xl>XHXhYn74DguHc_#!6WZ&JZAPM1p zpFxOOG&6#&DevlbcylD0Tp(pTza}mZp!J=m-MzC=>-xTTT^-hlxYQ(|4|bMemzu65 z3MY~3&%M7{H)uQPQ;VLk;OL%5Z{p^#*kgo269YR|%fQd^#(WdJre`Myy!ZvzVe)4M zmCgU8L!{oXga0FmBQ56QpV3l=wM`DxC57Z2x#*EC zX3BasfB$+j99N4fe*rDSiAHWy3we%Kuxc4{`_ARp3G*I{H|(`R!;PYB^f`#WDit0J@lPRFL3gUkyH^B& zXB-DlhHE}MjVKR3JMmfyt@qL1%-D(%%^~Rp@qd?7@_}xslLC}X8T?w$n2^Qa={@{2 zvBoyZ0PXHBzIf%lR0M{>v1x~UdoCN@dy*}$^bf5MDs%0LtA35;%>0J6bAS|2{j^DS zG$hjWoXT0R_Qf4xzcgD?bra?UI0yyjQ14o7kG%Mic^-DYAhQtns*)u|UGp<5V4u@^ z$ORq~dC)fS%4}N_#YrH^$m9ay1H*O&MMb=4WcGFdi+d>sh>FYrPxB2d?mP+VKeRzV z+n8-+7!-z1;(ypnzyD@R_$kT0NsDVv_5%q@ClI9h&J}CxwVW5-0j4H-oVQ-;>wG!K z9Rk>`Nd4R2Z4v2Sh;X2t$jj=Uju)2z&SouZJ(N8a$?SyntOJ;PO^%d5+3AZAT}FFDsn_tg22Fo*4EPti*0tP744ItR2;h7W=DS>EqD^GzDW z3tIQLg2zvRNAMklW)0f;%rLdzO2mOI^Y&zM{kiry@@;co-xECiSreQM6)`~%ZtAoH zZaVlETsZzj=I4j_`Qm`lN74{R6kcnD?P+#g{E8!`dxJxY8VdM_{5gZiL0%h%dnBvQ?PqD&c~sV z7914jBW5R)T^&2=w6ceCqb__mX!Y8Fx((?)wjMutRHTSxWHsJQiO@6!| zlstR|wv!%)>unYXBRQX}658>4ZPyUiZx%HVnE1g=L7eem=n|GR?sjukZ-{uNKrIGZ zwG(w#I7@ z#}ZIs7_q-kmrl@yOhES}3WEo>sFTXQ^WBW{@v9SWmebPIXjbvT>L2$y_IbIIP?=v? z9Bh1kWo1p!9_4D4slbu%W;fbUJ$ai>@2Wz86LVr_>Zg>@%1m8#7egHXvjIwUHPA(A z{q^{l)hdghfbXF)pUa+kR&t%N&GY@a6qhq^yCTMGs1{eJvPEKIBD;Lh^+uAC-4K5$b_E1b6REuX!bkQB(XeJ zuW5aUCPH|qYjqBsldOj3Y9l+t`>;Ozf-rKiZW0-A+n@#SLPmqH_w3pNM8($n)scAJEp zrYrRdIc=|$&ai@{w<^~5>*WazQdySfAO*Td4Dnsr3nCEg#XEQkSfuAj7zaab!`{cB z4@#`}j%tNUdp1JuT1?EtXSXbDNruy{m0186=Cd*~pUm~tVk8dFz{2mlMq}%Ft2o;m zeH{k#UU}nBS2cyTd)c3Rf~obrg?>uQD?k&I;S#OR!tQU3N zG?V$uxukw8-zCd^%pg!}^Q%dH zs5L}1_2lyp0;Q&NZZnL=Q3;PXZY`Z?0Xh-Yak+4Zy$`2zwrV58#CYvV_w?pQ%ItM( zLbYK0Di-p0yUu~IzxEZiu}(9xzBFG@AjyKvi+7B(GGX3qHdiq+6-Ff6uCsyju9Fkd z(iQb(0uCR)zRH^JbVvy?fsJ4?_li+|q-{kc;q=JNz(fn=Ccy5k- zaZ_VW>J`hWc=L@BSi<*}`Qhx_yIB!4f=JNn9^T_$(3(WWxUSV!AA*az3JhYo{P04<@{^2hDi_XOB5>kcTMb zfayCWPTx{IZt6PeUj~rr81JZoE2z9e_5vq3v|6Ppxp(r76aVn{BP#vy;*ypVi8p zT*pO>VdhM9+}VKW#_oKl(vQx#{JQ2Qs+yqgTkh(!^WgnylB6cqWNmEy=I8-7B{Ozg zb1k`EXg`jv>}#nFy?=3`b-(yafqFS!gceiZx?1@yqiMpcQX;;;zH9k$Zc;P*&6J(1 zLV(DTCQk3W{BUD1AwfNk6`M39z12Gv zdE9CN$HAHm&fuqRlQmdG_CKKtZo6yc&HwF)2+j`Wv|5$LwEs|^E2QSL85(B7adA;) zD+{+d`Z6X-1mi#$SuFEifme#hNG#9q>b`=nRJ~SzpU0|G;V@uIa?9kLNT(J)TZ)6P zW{o^9ISYKlRbhfPbtP{q0DYD_wr<&?wJ6WwW-Xr)6}($4^VP`b!uHPO9!72I8hkRg z)hupaG;nbLb*ep13c+rqg3kw#v&UCs(R%zeg~&P<45x0y($;4c;+C&&IWw>1w$_En z@<32k1EfnYnCh!^YjB+1|1}xbFA~f7S44e8Boe)ra!X29R`wBb8fFMcWz;Q&0jIv4@WfLl+ zY*JQ2gp7mAo@E_`Lxq;TLdVFV?7itImAzMpQ`z$v$99h2{W|L1XML~#|9Aba_jUC? zIOn|X`#B!Zc|Y$n9%^)uN4C0;bisgo4s5Z7!N zF5R8IXQhCXklO@)2|ufNF~@L4@;j!IOXfmheIIVM#WM-<;s*x1SaW?A&0Si6Bcsc! zVB(P@ySr1}x9uF;IwD(5?P?5vS7X0` zYV3Qi_9T~D5xz5nLW^Yuetr?(@l715*GK|iAN8#qKa_RFIdw*%edSBTqbBNoFJWsT zE2+P@B@B8a`gD~#_yRsl{E6cqLId?S*{uB@LX8uHNGsUi_y2vBPi83ZRaaX%nUa-# zS?4T01!OB~Y7${~3|oBKZk7t7rLf&!ji;%%e{aT^k-JUE6O4 z4rez@SBq*q{(-UamYSi*?^I{EIj7LyUNuT6;!Az=d;$3O=PHob4?rRU>i2#=?96=9peUomGi@j*Qx4k+~0cUil zb^kMP87S$2zh40kFM|Gk&nH!X9OnPO`u7znDKwctmMA}3nQrSUrA<=|{WrcYP?d)~ zX#t0kBBOOKCs6y{%HaX-;Sm|nrl!buO$w5=Kcr9P1-o0kd-6EtJm?ov{6XeYS7d}k zI5#%kN}pJ=+B_mJ!uB^f{;d5wbYVgV6}^27s*ZyEqR2m0M{u(-XOIQMYceu#5LUMw z3f21qrT-dYz7<_wQr$0RR@&mtWJ{TK*Cu^lV&vPm*NiXT=8A#tSuG7^wfl|9 zJRrJ<#YkPEB%=fuBetk!JhZv}$G7JEJtZOo3GNZ>*%A9wI{2qyG4LuB+dmcb`@8&q zVJZ;S>)>2F<9RQ6&`tgXbR$d-t|0`VR6=3&+sgnGv+7ysuxAx&2r5#3eKS!VPd@&w zK);3R6UMvx*+T~U`syAY9`;+m4 zz+ZrrR0B0Uz)(ClP_A}pzGqamO6nc_LhYk7ZQWU&Yk`o7JFI`Z>3y$i>-IR>{Bz_f zs(;AJDFTrCS8;R@4Acld?ezR<7k>Irm+%i@3yIyue{}S(9ui-jg;aLoaG7{OjMQlw zZl`a{zJ<|Xd=NeV`}&9%X%veo9e>vzbrx*@)b95l{S*IIcJ2eayzbQaT9oV&6!h`@ zJH7ut@gn3OE$|xry&u8iNGf>NUwu_13A_f_r}0^`!V5S9#+qtJet(?)B2d-iBNsAX z)=xv`30xrepN5-AmcWzX^HiP;kP9LG9k^)UNTKm5|N3rDa4~qqiK6yE(PF>djhwdv|_+a{EFb`f9k-BfEL8=+!tW}eOA+thi8_0wH0DVYZPKV7l*$-^#0R= zdLy7yMuZ%qb|5$9vUfZ*&lBLFA=m~1ZlHjb$K%B8M#^svM0@&`BLu-!E_&K(m=E}g zR4wEl-L)GBeA3`A++17Ivm1%SVl^sh4!WUqECMee#z1~|d5SOf5t-W2ZD;WFK!t&8ql_?G53fUzM=6wj_MHwaC(UpA|d zHh0~uTg~h3#1Y*Nh>QGb^&LnVtHEgu#$%>_6VCIUDwybi@~0}kR>0!WTm?DFLUaYb zit<}O4$Y2il5e+~Ckvz=hBe*-d_4n?l{gDdf?+U|&R73HbdxS z7;!gayYEuqp*K0+;pI#G;RczoH}AKC#} z?4DFNz_GW$Rtr==Vwq^$uBFy3x#9~ zU3p+J-0D)hE(&GmYo|XxKiCGNF0>8SBm`oxhzo;Z>Op=W!rcoY`Y)X|l6eZphwdXj zWma0hV*V#Fr>H~bi(rZ2#)HSd26z|b4SykaSswdh{0xpC00MKUEuzMAF$xlWZH3)7 zVgwdBTj<7u{7MYQYS92 z9c*s=ErfGXLJ=sEDE~3Pomm?@bmw;$G~N`EGK5>vYNS@)znam8K@6V`dw&L;bYDl| z7Du$5O$v2(lDibDSWXHN%eDN~^-H6>AJDxn^TzX=@ZCkxorGx|ezcJTn}f4SaWpyw zw(qxgk{-g!`U=JG3RvUjci=99(GPOAuT2AgcPxMakonPcxJdl9IWO$qC3spYz9M?-5-i;I`1}+?_{#=*lCj` zFYRtC>a>ATV0~#5N8I)E<3H9!E}FDo1)K=HMh%!fvuLvd{uICy@f6%oB?I>%$d}v? z*#LxabR6Ei;5mgA8>w)KU*Am7mS2zQlP^;7mtGUhzjOcN0g8aLr=(kQ^` zzP^}5^N~(8TFh?-lU#sU;XDGV0}0vP6KbEPN1;g}H|JqJf_dyNuRQ~+o`R{A2d)|Y z8=!d5NK|{k~NBR5!q5}&78nVP8E!c@`31a-QmLE}# zgYv-FR5_POQNdq%G3ZY5*ihVN!j6=kf67ok%w&Eu;buxm6!a^Z4#!5D;uPQNkdX59 zH7$!w@I8v96s)YLt+c;x!t>>54G zk--sH4lE_*OU3W<*f^1G;%Wy8XGlf8w7V&M@`BhvQai*n?v^gniMWV^0V zkdhPcn_(YmpN7e9T#VNTZs2vHt4^nBw(6h7Kwz0TD5a~H%{H2Ki{V*)C>z7GbwBNwk z%$cX6RF##}OPxk|7dI28M2&hdHuiU#=eOKD{XwsANCdn84Czqe&+XeRhHbBrWi|8J z)0uTzyY3>smHrEcmG|;P40^I-*MkyGFRfqd?kvG|*6?mGtYNWJ7@yW@B~%^a zZt6Ce0aK|ikKfy^DPJ7c7?FIWAXb3R*VtGXV$tc@SYL;fu~2tUb~Eo;r=Gm_BAF>2 z^xx9Hy6&5OMC_~1n#&k3n7_}Lj8$L)n;`GKa|(aeUwk`?(cfW6c(?4F@5DN0r1k^w z_D&ccBI`Z@*kQZKQgZ!ZkP_AOhi^7treRx~?-|9oL7)FRS!kAdK6VVWbg~WYZ-}ui z)gNlnOP|ApSrZ1dY1m>t+zF3I&A@3}nZ-N%o{8EDqb4`i(S;R)0-%itGKM-Nz!*{m zXv7m`G>rjhs=LXjr;ssMqr4^@fJ%Y03cBw2W#UDp0Ac6M-=jhZ$;J0WXv22z7I?c;V{-C@kjk0d>O$JX=X`)(Vn0DZ2MrB62dxJQVPU9Kb+>2hn`z8881v*2s_<$}g9u`HQ8s|5XDInR$t z^VaVn?=xvV1T|C892RkpZ)x^04lY+D<#C);joA=x#n-Vrv7^>BkeI85q`AH03a_Z05AEi(kd!PmY*S7#u>hVh1=}F6?;}RVI=C|S~X0w zCo@iUIXcPk``)Nq^7?Io_#083blyTkm$3o21>-HPoCsQQp&c|W z2knA+Y3F4;tOi<$_#O4|+(8UmB6@tgel2iX8Ierq6MrQ|0fi=1+)H*@k;BYj2P!J^ zI{g!6_m-85?Rt5o_8(JE!aKc?XS^pOM}6xiOv%Y8C>7s6GnH`(1(TunaxJ{kD834@O2=lnm=^hlV|N`OVP_dVaX#)UJF02;u{G8e67!#MIb{BLX8>h37w;s#8X^ABX(-OJ+Fc$ zy8ei%AFF$01=7Cjm$Yx*YzC(Mu=5X0-TQv8%rJ+c}cFFi8+L~ijc7Yl)KeLi`=Y`f;ogsbtXQrKBLUxl`_H>rtxer5 z2G>?Nf6#ChAxk5tRHXXyS)KBlNc|pc0>upyOqAqQ^a0|PUzHG-uk4C{Par{&_rFzP zn47rjQ8u?|f%Bjm170p}~m(jI`^Mzf{^DgTQ`f}X}mgh4#o1I1w z&(@E;mvzTnzqEfn&-uM5oPT4iv^A+RO>{Y>g+>5`6{+9E0#P|+pzPtqF32TRu3*@&)$ji481eL zquwaiu+WwNQZ!+uO-n{dYP6Ccx_jhDB?%rmP80A8vgTpn8B)64jJ<+DgmSY*E;&3W zS%b&jW*h?NIBftiUcAem#r}H8)2|Fvl_yogmhq=X5&ur0i@du;XLn>|va6jZcMoUA zN>Nd;_+AY-C+MzgwtpMk!`Kr;2MZETY4y$to|95iExH{wS;nMtISQLh)q0(1)trPP4%-C);@{ zQOP7{-H!pFcw^GF!OOZ}dnxNvC2?XB=L!JS=LP33L9FAX4P60Bs2D(kvK4wx5HZ%)rDowOrL}#hDRMfmd9A9f4zZyeEh!I1>KzcWyY~VLD#XWbrVvK>{sZBKD|8Yj?C`d zt1}RRDcwaC5rcK742fp_8edld2{*EqQY<-Z{MqOkX_YJ)&e^o_1XkHT-x;qdjiSDO zuc>d}Y)4kwzLY(h^Rlk+Udr@+^Ag&MMLb{}Z2h`3F>vx>NJY=Lhvv}F!~>MYM@`Aw zu~-m979}*dL0Dl8*!ea`Zl`AEyn0p%cmp!@o=m!KR@1$^S;>U1h$YY4)Qp1jb4ee5 zaN>u)*=`{ta66xRisR$*nWe>JT#RU88MoIg9!I!Wa3Q~UPOnC_TVH$f!XVD@8eH>I zV|%MHBuOStYrEF^sdSh$cS#6~i$9kF)$1u<%71MO6e>|V> zB6Pfxnv-*HbF~!KQETMFe^eqAbzIo)o{t~!G#q66ITC7P&z{&Q<~C&NCtFgf>v*#B z%}EaGGo5{@e9;4UqZ#fNw;W{01t9V9rqGimK3hA05EdMuxC+ow!=Ej(VS@SsB7RT$ zTnLc(azKFYJ_S5XJPdULoVP|T8?1|YTs7wVEOABEaC~8%pD}KZ;pceu37ZcW)tYj5 z?0D-SDI*wTpb*0JfW(4f+ZMyb%QyKlY?ife8Rx#&_ z%S-YC>pvRhBXMAi0trXEFx{B-g0hEvV1?j%bsQWZw1eOGQ$R|dnQ`(%3SCXNv)r|V z4Ei)>X=hg@4{(yFjI|Rd>e)Be3a+~85torH@Edy zTH4yJZEc#XVp_DvpD6`w%S>VK*&1*IJ{@IzAV>0?5JnIQTWFhz#cCGcL%V4cF)!ef}x`Q&ny4xi4h_X{oaIM>jbg)~u z`M!;?@uC-3)1r5*Gjx%A9EK*%@H#=$*H4P7Cj#0e@QR1t#? zBR{2g0n!y0bTmD5Cu_s>pZ7NBsg}I10aPv|z=2c^;+v=MC7J0c5)=s+vTcJbtc3}H z`XYVb-0QCXD{iCM)pWY!VrNrpXv>nm4d^R-{4xtKD!O&bepnz&|HIxOIR)pey-juT zdv;cmj6j{munTquInv!l%y%+GPimIa=FWW=mm)eesT_cd^Rctoo$l32C|d3^2WYZW zLnyzp1j^!+$5a$|Yn~jT^8^ZkY=6YoX?o|pS5WUt+s~grkTFy@T7kz1OXY9#%G^&( zB|%Lxjb_wICTEfAAis(=pt8mjs9!n+>LrYv42SU$v!4)ozw$3)#lC zl&LRFDI2!5O%_LWPG0A$S)XB)W$d%vlR51{CAgZBBYMQPOAlP=k|f@gHR^o5dN)%3 z8;-IhI{Jf!#p9x3q{~v?>|85Z;t^U%rvoj422l(i!NZD&1*@RCvgA34WfhSd4P{{Q z0m5Vif-|`Aa->QM=*S2K9YyR2y5!co)N}h)z~vAPgiGUUUZ3hbHY}muY7YpoFoB}h ze7(}nZHN?F)2|@JQOj5Wp1B#`qK*l98rR!XNwG}+PU@T zPG44!OwJ%|N{mjiw%m|Ui8qxr*%Vij+*wOKy5hOTI)S2Nk&^~OA~kw={q^FNkxA_% z1;G1_4_R5c*jzExtj;&rPW2AiQmcF3C#Q3^B~w}XqfnZs{hPfrkjddRCS5znf`=`4 z1mWP_ebuINzgtNw|UO2~;4gXs-1X5oFuQRNJk9W6Y-A zm#zoqf%Y!m=&-KX^4vBt5*TO{^Uy5o^~(H^nyLEk()`J+FdKWHS2i0p5^f66L+{jM zFxyXb+Dl;5*cq%r5Ftz(LLG|=0Q`%{Qy(xxqhZZ{vSjJ$>0_@CA0e}&w;+UV*MQ!M z7NFtX)F|Ntu-Va8zoh4W30w#vA`&k&reP;tlp8YCt;(0wer@OTXRikCIy6;$Tztmu z?8O*H>&?`uxxrnQ0RwI}wo{Q>Ifb)fH@@5xzQeCsA9Yl*W!fXCdq9TQj!<^SXQq|X zc+8-2&R(CHx{OI}y^G~o@X_-cQ+$c0P&yqL>KfL=cs34dXgi}sfSz^!8AA(b`u^zC zMXWoQE21kBqut2T?q&UY^hx5PhivaFo;g4h+~`u*^%uaseI3BV07Vk_)AQh(J$p&W z*|Im)?{&L?BU}d@;mF9lU&XdBK#yalv_%+ZHG))WQJ^e3JHERZAJ3~W~Ey(wGP;>Qu-u{PhmEz{+XxtC-Is|BQoBqB!OcTj&zD| z(K0xEnAg)Wg(H$?v)4$XXlvtzm;~Zm&3;6ldbVl!I)5iC2Q}N-W?6A~XlBAIrvd$> zyE6Z4l#9a8u{Xi{^3_o{x-6@zv<_voQCV{}G`x+TQc|_@&}U|$9t)R2&SQ+gm-$5R zTCX3wX=7#eMNCY!CV=;}0?7nq#?+z#uODF45b>FDTHvs4y$6I=GJ^_-VmL_2zthi@ zp>twd1IL32Y_GnmsQR#Wip!wpy)s3j`j)y@+CB}_E=#KnIg|3Th4dk30g-Fm@G?TP zUYUoLw;RP?e)&}mLWuz0%f<{D&|r}$A&VWEMX|?ISa;eEKt_MHY3$>v<{50135Ez- z5A*{F&isXQ+y2vY?;g8`FmaLy< z3ME@BLC`vb{pEZg6%Me6TZyxU;Eeg;x0*OutHLgG0n6lM4_e9>hV(=;5u2QBqT=_{ zMz6dMN`lOpyFGly%UaGNt36+3Y8N>a`=NJ9gHWHT_K=5*=hCRx6Siwo{htv;5~aDAhyc3;)S|ER; zn74wPAI+M-^$8O>@2ez%S16)b0p7!Bzy&`RGfJ{o-=vW5_LNJLWa)F?Rk=nh<5--= zyC=#&KT)YwNH@7=77(drFoP;npj4|M?LA5*M{9oLMyEim(A%76b1&WyxHL&_cQH7U}mrNTbz-(oeF@vuimCWEJ^$?Gi%sU|=%I)3)rgf(f z{c2ATxRmAd?a0Usi&jCR@d@}7-IyRJp*O@t%6WPl1bi9^Z^rI*V4L<3`|+yodn=8t z=6>68^&lQ1y_WX|fB#wa1HE@ES4$Gh!fdo&g၌VDb2kDlTD#vmsiJbJMd=!JC z(s;s7vCq4`q3>}`P~kViGSEOqZsypx3Cr>Cafpg@Vu*J&3$y!!)3pg8)ja^h2S0+= zbU$v{Q#(7i2g6gP6af`qIWqvljlJV@2dICZiX@}6aKY#FG!T?j`27<7`sbI`iMs!TU{ zC2?KqVX4m@_v;$S=_RK1`7~a9iQ4Ss;%c~O@PIm7lA#dAFl~J=qrZy4mZC|bKb@%~ zqwFWdG>R7(kA>|Y$SKYoXixv6TdxTD`%d99P~eGYOhu=o=WQO1n*noJpGv;dkMSE% zdNhga)h3=zjwPPW0DCQ zQ@94auTm|dxbjof)*J4~e85~Edap(uC&!!^6| z?PE|}SwgFo6~6-awr>Y@K##h+oQkR{s`u$5x5qjC+2_yqEwNd%0MzfBHQn#f7(L86 zH2AdnA|x5?z@iw#dN3L9Zt9%?Yfg=YJaD#@W8if?MW7N6ZjMhX@FrE8+A1^Ol@KLq z2FX2)^)x3KilFo8RHv!w`MyRmnZ>u3WD)MNC8JdYWNbB!qHCJmEfq+DvdF3}EGpLE z?SA1y!Ua-y^rx}FvdODX{{Twfu~{Y!pm12D3A!1|0u7)!k9OPN24Rv(zHyWu9Nj?< zy%|?}1sRfd;ow>bB)TbaGQbMw8@SY_oVvN|A;lyFp8Qw|KDokCWvaDW^=4)x+B%+n z?M4=1cWdiT!*=yKM#R`bEeJL66Y+F>lO2}dW5c&DW1v-QLPxY_2@ z-;V=B_TUy29nAZp-HO)rrj)2b#%G50!93v!uvpYRYyO!gi=jP`La3x-pol=i8(0&u z%3FcgfY(O#rdroH!}Rzln;IW0?#^%U+P?^!bhLeMRprs6RB^30-Fun?4=*vf<;E&) zff}*gg{$`L&ibv}(%xCce7{C2iOpp!zgJq@+A1_Q3zEF6(-;o^{yF*6G%u)<;2p%? zgqfQG1Q8SLcoM}d*2h856r2sCP%m_Ejru#mHHoDn z37$ylkPgV0TRz&MvV+erEM4H4M?9?U_mgC0WRk=ZA(OK!DPRMh;aVk+uwiaq+dP`i z)eyJ_ALe+yT{DGT+QVOWW4+PZy1p$VX7fHLCJ}O#$L6AQpunHpm>PKMSku-`8F>jx zoT`PVr@_?J)ZFEy5AfrmL~ z=i(=I)iowx+-|XQkDj_sUFs>jvZRQkZQZcw zuPP2QY_rNL1>H_1>lZ$%5Li+@^viw*r1Bvi#Avy{?*Cq$CCTu4BG$$0o6!BWpLF>> z&)Hx~A~RjcWZZP!L`IUY+d;PT3Q*oe20W~Cu%y6& z7cICV{o$l|@+`v0wDHuVQ_483JV@bGU$tBNi5N14E8LSUNs9Xl$zi<(g(kuxFo7$EGv!{Fx3nKub&hq6b>N3Zp7*K|h?YpDW>bbDY;=paA6Z61(`KJ6&;a|v3) zB3MEh>%=Z{Q8;1m9jK3@#QUvXTcU8le@8}bibeNYHys7a-lZM*o(x1F_GMpp6Y!38 z&KL&^QZ?!BR8r+dTEvAgE%_NsWs#)V=Vi#q#De~mBy}pJ#h|$+K1AV}D&IxHSbfpT zX2i_r;Difw!|G~Sz5LrKQ^7}wdBprrba|5XBwb<8{Ob0b_&6=Xd?!^9kBch5oX@CT zR?^;s1LC=Zm4=+3#lCegL_LMX5{N1gOFO5iJxCuPQhw{g;P(^=3I#LUrdnUW+EZVc z1)j0T^!5}8Klm0my0Gx6NfNC8_M3`AklZviHg-PQDPCjX{)nT4SXiGy2OP#f1s2xL zPCCnJ3j4LO(b)P9!kL-14@hXx)}B(%;uKahg{jny9S_?dp@u_#fg@8Y+TlzmAYqO7MRB)W;Tn`%p zXG5?!hg@(@{uJ9J zho6gp7xH zTtw^LLA7GK9oK~rTZ02vtS`D}yT-MuJi>f!51Ftl#~0jWsEZ;&Umy!dQW|C7l> zYb;qL(DRmiJV;G`htv*xKO4^WWCdHb{Sh?v<9zq2lpo8gT#57UpBgl6=jUw4EV_*q zb#+#fuvy;5=NaeT4L(VB^)@mxT3KDf+bt%hrnmd!w%Lw%tX(tymW)8`jMEUqsd=?Xag-5L1s6P zr9I=N>Fy{oV=Dovc&&*y-oK(W_oaFnd*_*M*x0Ggg}B%?=)+Awx!);ewF7gE4?x7r zAc2xgxoyE;UKrdy6!)nEBZ1Ldn=nTeu;XId)r;=rc488OjwC%nC4K~{ZkMaX;=&ye zrSpB-lo3i4Ot{8b8nBssNAmQJua>(qr_*>sZOW`lO-2mFzz9X11E*#=OTD~4_)vuS zH?<=>5+~IZmh)&~KXu_t-K2>9hY#*{*JJ|Qj3l{eB0Dc=3ou}Mc3UarFxh0t>R5Pu zyhHRt^@+r#_kDB81Y{U)n(z$${Q1LXs}dR|U7xTibGTH?^y--N%}m+qL%mT|(_61u zVoD>G`4;;7zu7L|Dj8MjnKlN*A_;_|htDGqBMAAs)0Yd8I-_>moDr~b3$!M`n9M&CWw*$94IVjao z25eKf8iK_r`s`V-!g>t==)+#T7;LWN2TRt#FyJqGaJx=D6$kY|mLeB|f8%aKhxkJ( zkgukt-Z~Uf92I*Sv&_fVDE3^H5@Yd9l@f^e=uvmJ08|Zrh9sDFYhE-UFU&YqODlM_ zOc9xS-_CBxq)an>3i*Uxrt$}V7aTXAx%QM@dfQ^jjx5mqk_py|7LgjYZ@lIPQ@Xh} z=BvH$L10VJly~nh?=JHdk8eaQE-y#C3hP^KCAMBz6oBA=r#c;7QXv=Z1~$Q|2H|L! zHl)Qr9tYgO!_su#i~a&$k#xt<90ZzH4*?mmTG*>esjna+vQm_y#YFGI6G6>8)pHGA z-IW5&&6xVf^UUwv9;7?qUmZRq(TSOrH3J$x`)F@4vHFg&S$-#G8#{|F*^{}H;M;}C zD3Z|Qg&uzg_(kgw4Y&F@BeXzSp~tA;uEu1KSEkRhzx8W_k5DpRaR;D_3|_P&I(LGb ztBgjAu;*cTnK8zT{MYfG!PVYdZ^M8!x|yNUAUmJ3av0->t-(Zsk>jnI z?dTt&S@N5&?@yGEIMd{Aq>c)k+VRCO08g+G1~RO!87F)QS}ZgR)λxA45*_>XZ zd)Xp@mp#O0=WUfCj~J8{orX&WY9xj?KVJzJT*i)Bcp6^fdp%!@5Ze^T-vKq>x@1Df z2C$f*L<{mFHN*K&aqxV^34U}Aki;szwfr!fL~IbyaV?W~C}_osIfNzQlqwYI^+OTQ zRNvPP{tCBxP&28s!%bCsOtFWPgNC-}D7n+;MOu7nczF2w?ebtGW1{CtV(gO+MT15HlAPJtMy7B9cL;p1NQxYP!8QgpM6 zHoqxkuQ|IegC#1H*5TlO4xZnkQrZlt;DGW0<}}-7$dVrB1W2{IiP=wP32@)aY#$3_ z==58<=sJ@3kyMQ^D(qgYuyAAc(0emy$Ubov@}}aEhuMOV=+mdElLzfXh70w~dc%CA z_pO1I-cJM8hAd8nY#NJ`7DEhv-xObP7neO>t(+pYH<&=e#}I>BgQs(S6bkIYtgJvG z!vd&y<9g0kh%$0%e7_1Y zm=^i-u0XuQM{UCP0hmf}W`=ZNLc@+-{@&7tiESzSAt0xs5?MPUt{yrHvV~ zkl{9sq*H9Bu*1Tf@1cNvv`?03Ql9{xfQ6gRk9SaVgL_I*mIJLSZeP53*Wjm}lue=R z);CW2a`dD2#?IKgr{Z4O((CFw%THG)zf_gI+UaikOmUEG-c#`ssl4>vv_pH_XKAEc z3QR~#`kj?v7})32xXU?63RuDB<(yJsW-8sRT%YZ3ee;${4X2)T)3|=-8i}&uv}UwP z6vyI~T0l4B$zd2qX5MT7jD#%_gT`f?} ziLW2ezua@+nQGOOZ-Fq$RDRHvkPntVI-2EKu7FIP_W-`tXB_`|;r?l?JDdv%?`5GG zZpTPqyCyJs?4!%PeS+lx00*P?jc2vopkTQtn2O)rR11a0AZ=iix)roWB4pw zg;M~a22Td>+s-5732m6`m~1sDa#c%Kd`(#D=kP0;;Ng?Xa?*8Olj4}G;{42o(`*c) zbQdlZAyI$wKzx}$slEJW8`>ly|GcF5?cHVgBXL!!xzEqXO!$N~CUunyBQx_D*sbr$ z&X^?zvja3Hi`|=^LxpHPEtL_clUCBj2xke9sP}F>P7y3?ri0z=ICFYsW*{8$g-FSv zdUQ?|sCN$dS3>6|3a>+Ut0S&lLmr}VD4;Og9Jct>3iIurqSh&vVvy{$(l@7cfeKl( zkJvDyYNHQ@focROdIdV~$ds6LsU&0DYjnulF|($L>;%P>R9#@U5**sMrLpCu-+kWk zxrITh5Qu~S9HF&=JUmI@&Vi9mst6msR-OXN&tRC0u57-#T3HhC7ZCmTxv*TTAf;)K zl!dfz>FCp0tT>4}T%iXRf!o3*5h|y*1$SgNhfngbzp1Y#K{*%Sd&{u4?K{%{MLNT- z$8OB337r$6b#(@HHOFblcyR!wq`{ht3aJ{HQ05zHZ;>L@+U-=|?*VSWgliee`REFu zZ!SDKZsZgfx8V+Rx1EW_CQw&|tb8gAN?(c))0fr2-3NKL%sOH?Ws##eyjycMO^0K% z4=U}Y1+TOjhbXRIIjy1Ue|Yi5Td~>$hw$tl(ufxxR*L<{LHUk zSjcT_Z|^6~eD}`E+WISuUSsJf%1^e}TxF~i)eR+TmX6wzM6q-A-;LB%kzWaRc&>WK z;V>{_W78iqPq^@t*cIvAsZhS<_7g8dVaM&CbHAu?f`VBBYcN5_*rPw(_H;YVJlE~* z{+N9Q?`UL`rtVe&xPL4q0rv81qsnzY;lyy)s`J*x*Z7p&AAdo-UY$qJQ>AU2niW^K z@gKx8*`UwB9xsQ7WhY7mK`K2!bmLcwAVD;xS9^rm{B}RQxo7o-VXXAjkZf{R&8J;7 zX0ll7s~F67D+Z{|8-@#=kC1Q&z)_1W_}5Q zJ|#}%H36g5dLoXg8x*gy6rfc=)1a-4K)7v;y6 z@3&V>nTI#fB{>>Z+_=&D)}+2nhA%Yq@g*oBL@5qs#?KLhN>jjXx3o9`E}{eh5!>G> z*v|(A^qXB>;D8jMUXQ4bVC?)Fob}2?3c5Go?bLRSfl^L(U!h4iD8vDT!T~FQ^N|Bf zz-siJ_n#TgeH9)mvFDWmg*O!IRv>+agU`Brje+{xXZS($OXM>K;ERKsU?R@v6v|s& zUWCG$p?ATboglaEDb;HdR4oKu+iQcco3dB0&cARx`-A114oMUuK5f`OQQw84-5^BE z`BhlhGv}iNzTyi8;?n0Dl$F=@+$lZ6KT?T_~kUH9@#&S#iEs;t&}eKaC5|xwC6qYZp*X2KG_uG))1@Am zmR38l<_*F(r*`8-cNEK3#ZaHR+s$4_;p~r^_Z@Pw%QR~52tLn$oY}aw^?IU?GfO!n zt|514K*UY|c-G?LLC5!xD9~GUqwJ;;gynD$*L}}>qyg2%E4hDYAPo5#YwfLMdou=z zZwUPAEs6wnJ)tyEhr5LC-Gk+TKr#&EuwGKm-DTeC4I*8iyof|QR0O1UVDfSq>l{3i z!?$IOp%i(5;$^a_UEup%wujo9B&lA7mD4s=6rG=_dPHjYRV?(_c*1DK@qD5%a;B21 z88vI)I*G|UU}uXItKU4F%m!=v5$HHu~64aL+LAP1+)bPA8RwT@3gZGO* zQ7k2?w|1ds^oPo~G)YdZP)ZJ7@P;d;I+5sR=&l%uI+1(Hn5Ocg4a(l#oG8$TR9c&oUz$m(*B)|MeYQc3Pv!TS3R~v)U?Kd(EuMiO zFEvVlIQ282@;hd6w8pU91!Lwyp{J);Y}3hZOgz!2NCm=XA;vgX4FK~v7r;#L^che& zg`U2LRDZYaBZjyf)9JkiCv0Ec$3rGHGm#Ex_7DCktWVY{qHoy)y8Yq_(wQLs>;jEu z+?S1?wT>q^_gx^CujzFHQwd9L+UDF$-8XgYlj6QFSJQzui|4ATi<}Xrf#sxXiHo0l zq9%%^ny1~~Ml>zD@@98rS~5xr1~LC}t%Y^Y*#zK>TERMqR&_8LDv zp>_yAyRw?2U^AU==J0eL+B_WC$?sVX8c;u8<#{07u8Vq}fEV>_q=vgPfJJuWF^qsY zI)NFJRxW_Ol2nOklk?{xXEo%l8C^+9q?rq|fTINBonIk)Iv3ac6qcEp`7}`bvB>S3 z|M8f4(0=FoPQOw*c~*-?50ZHjQYPDeOx;epi*x-p?iAvxELl4RDK)wDVa_0dC$u{; z8dlgoV7iXEskget#l>aZc24P7ZF#}?k_WrfK${dZB^N5Lp-Ttz2>sph%@5$|Z6CKV z^yPslAaXoq*2IcefMN`p-2xUgtriDGK(p^6Xi1TJ2Wd^5>HEpa*quF?jmS|WDlf|BnP399qWw#{cQ z?y65sr^y`1BFQ=Jx*`tK&0Zun{*UKhyITg}fpMTG;UveVfs=#02IyZGE>Po*jttd| zd)m%0O|-|H7Jx0b)I^0B0d8@XPAlY*02K+vvw>z0;wjib4>$YCV%y~i_WS8vr$H*G zN=oi)<5v(agMRk0z0rx!S7Lvk)xnVCAUwrXR)Ovt95}s*lD}v2AV)mR#<*UK*ib@# zkJ*-1ai_IJ;i~M_wkhs#MMmQ4RiLERq>Zg@Q_|-dciX~SRl1Q7mRv|nU(V}U z31Ou@SCSLL64MxoWU8YQKf;Ocm*U`JPcziaS56Hli=^N@2nY`^X`&Q_3|qzqY;$sQ z*3X57y=q!s*5P>B1}0EIUq1r_;T~KjAIf3S7pevjI+Cl;y*v0`YE%)LizI7vqdpe= zLeZFl>c)+A;PS{Y%^zB`RR+J%m$!y<4ZM7{XpdO& zQGy6{ zz396$L(CX$U=vw{VC;+Y%yH6j(n-x~@+s!K(eqyNd_=Rz8Nm(Kz}}BAqpKMUBrSeL^iSeQ&`ukHxp}Z7!UICgHy=YfS1mbE!}hs zh$*;RK{xyXEiCTqie^Z5_**D?qb#MjWrCUt^^`cb$xc%#_Vjnx>i0msmM|xy;aATl zR=rQPOmi!UbMIh&o9}PjiXhl?iZmpm!G9bE=^4Wu53tm8;4Xle>q}NP8tP74u(k)2 zkGCR-e8}dsB7W*50F-;5dF;sPPd)=Fr9*eM_ipyeCns!CaXul6NfG3uzOv-zGpZuI zqN5L|&$;rPi#}A*kW`J;5*5vq;i@heJ3y+MnCC0PSLLe}Eb)V1x#^a;{&hj(=kX^% zuXeonCXs$8Q6To?T(3Ha;?U%#^8m5t3|oK6LyIhi$~4|JpzA?-TXqfJoPE{xAYpVB zP@Vk5JMflla?p*ylhmG>t;v{K65ueNfV_U?;4D!nSteR$e5XRaJn^{EONlMuQIG1F zf*kCf-53EHGPXkz8X3hg8a!FK`5I-$PDa-F;`C^yZC@?dDpOWde(1DyK)`(v(}d|? z`2sQc45*&DmHrZduFU)KI-nH-s4xl77r=AVq6Xb;BJs-Ym`PWoDeN0(VMb^?ZlNI% zyRmAnX^;VY2z-7?XWJW8BH=c}x~daUdR5V4oGcF{=zOQ5l#d$btQKMLT)&P>YTcxw z{4COSkt9p!&WX{MUW9FZqHWM1Egeq!0K`fK6Oz%@Rk)5 zKfy;7!8@W|*`P}j^&+=rZg`&nAgAQR8%E#_LI#WJC6AE+Gf#ZGhCe^{I1u(z~1eUJvy#LFj;-^yf%?{B5gTX|;KtFRb zz*6;M6A#2eh#dd%&yRc{*h>R)Jj8s}0Wl#s1QrwrIzwVUNM~I8{h==Y<+<<=RX;hv zF{24v)1(0yF5=HUfHv*b#K=nZ{K8BpQaH#13v>>-C)0WIytgBw6xvV{P+Fu^6Gv%8? zB42uB|H$JF#Yd@fX8?NGE{J%`FZ=!g2glCN#M?rPi`wvQYRaCt#MF(JqMy0`r{yC4 zcYD96`a_yOH+tT$fVF6nVO!|=R@BXsr=z9 zHU3KM*X~=5mMpLm0S|qRz~5v@eFD+v`2TMqh2D2j;+duj!GE7_A|~35_XzEo_n3*; zcW(qQkM3o>cGf8V4`Gy~swI@6c@Tah!e#?->=0Mu&6`7aj;+dq!6Ffd13H9%ON&J+ zAQw$cN)lIB33!TJ??d7ciT^>Jh%?}mzp`Fxc&Hih2yKqmm}^PRW|-0Ugduxf$F^Of z!CBWvH$7dVss79HLaiTkn04CiWeUBLTyZxuYDfQ{rou_DsAd zbF^L(tWiXl=C9UUl-1if*D7@=Y%T;}GWA=RSRJS^v+$^D+Oq(l$NEYJ{(I%%EDe1? zpMt~E)87uFy77My_s@^@0pOD_*v+1Vi~l+A|5wlhFObzt<k8<9nv$yujii6=qPFhu?NBR7~^i+XBEKcvj52oOE-@E(x@S<^825s^1I# ze<}O}Uia?3J^OJx$hGt`r^J8ogpd4EvEa65^YdcerZEa@8yhQU=ltY)9ImvjtqqsV zY50%89EG_5lj+(9JtdHk=a~n=-XG%rWK8oW|*DC}FgI8U zHko(nb);Dbk@7#Hf-bF`z}I@4cwcy8X6C(R$*UWmUugdNEhZOqAee6HY(G8& z2JE1cXDf-R18A>*p_TYbEew+A0SfSsPZaCFiT#f#@F$UePgyHKRd&Na$Q(EUWH07T zH76fvGX&Uk+4$hd=Rf=Z%Wv-6M1BYVFMg|A41Q<&@XKIw^#;81wr>k-Ab8qw&M5NV zYv2YvL0vPXOHk(20o-BzzpSoB)jy}Zbu|x8cb^ySw9N39{h57su%-y+z*o5% z@_HIK9`dWE05F+j5U(HxI=~tP0x%Q5;N$$NJ{bIEn4W{>w+#LlXP!Xfz9S9(@zMJa z05F9d#xKSguY{xjUoh-M0Arniz(4v$Hh$sKR+r`ky8czKZLyrIMm(yKW+^z2(7Y}B^5$QmQl*SWJ_hMY$0T>v z`&`+p6Pte0Qm#R_Fs{Jyerp$g$}VcR0PpXwfqA?M3?@#_Eb+|?uHb|>KLus~{ETy# zqP&U|W(WjUU%=5e_Se1r_h_D5&`j#7n5gJ+u5_EznsCPd+U=YVuK;rqD};S|nx2u7 zSXWo)f2p^(S4LL0^1GMze|CUYAuu=X;$y_8YH-@vg#OoC{*M6zhQF8QqpP{Vv<<}f z{qt?iY18&@rs1ptz1BF)gBEwvHo=e-7$n78i*1Z%44wY@egF5*1_}*0_*{G)d?5+{ zb9cWN39<4t?~6!_EwuE(vnPie-h$I6hUJ3fA9JNI0uQgK-)d|81Xf+qa+8Pe-Ej*;{$zwV=ASz~Sk0MP9uWHW>e||uBNfJF_i2M}iQP>K0?2qz zi)MBYO9$J4$sPH>1U$RDKL*@K2}bl8>aA567b$P_|5zHufh$o_Q9AM+wO8M^3IFXS z#|X|pIlbB*I}wrzhW0ZhYTrMN^gqzG3qU-sR>0dAi~Se)N`g&1W$fB&as!Y}4-vCJ zf3%Bon1X%KM zNXSEfQu~j=sJ#Y%tK&tAQT%^Ln!8U-+PCmW*q3SseW$>U)U8D01M*}*r3a7=|AZZ< zgS5y=Hn)l=6uL^&##AqRg4?clclFViz=#>MXIoewXn~&`_~o&iwfYxeu?n8|qpeIS zBlw~q@=x;h{u=m^O zyTjQM1(u3?W8!^^A;#D&gJ;hlgT4)B`D!HyokRLq1Q&5=o2I&T_vuUuc{+qeAfebI zItfz0#u)O>yIX}k{R85sNtsQ@X+7mwY{i}24U`!v*RF@Q?}mQ6p|UfBG6#&Gs9d!J z^ZN+SS7tr^-U%7mkSWlxE}iCmx15*E>3Z~$b(h`Hi=}q!t~HuAtYAzb z)eLF!?s4%3u*60x$vvT{B4; zrF_RXYrcu-=>V0}c;fk~sDx8d$~sZlElb$+SwnS<7~u7tSx3Nf_!~_g|FgzrV9kZo zoFf`fKGYI?N3T_67Mu8^uyKXYJ(!sh za%We(`xhh&wlcSrd<-;iM=24;&dwsqqMN_eIW$uxL4QXzcLpg{HyzdO@AIf+2P;+3 z7^Q<4n%wx4V#++@qyTkU%)WP~GpyabM>ZSnh+0Rvs zn`NL?4Qx&<+P^d_dcUK+-ShJ~C7M~_=6a@kG{1iPM_5?cGp5-i?Xlw%Exe`4HJN01 zY_8@NAlpR-H=Tx0I-=(m+_VO*)-MllmO!5Gi3cz#C_^78KVD6CsqpUbJf$mJ%*F@8 z$(1f8-6rU$tCPgZ=_~Iq(FZ>3v(!?$Ngy6WNIU#zK< zzdsGKo>M?FWFnOz6vL~(4Io(atAgHdg(zos^C2zNUbkMM`X{o_%F0}dvsVy)HUtJ> z1aR@11)m4AU}q;vvm~YA%=t>G^ex=C!%u+QyOZ(Wc7rW59Qeeca=J8dv1poQ9Xq&b*9C}H z(>}|J6`9RX83a5zFdnK7IEFDdrVC67rbGZrI}u64({HqBjz)sMa)k6T2{JPai2tAk z;Or+}CIgx0pjG+J9rjCJW;x`ghJ31RmwUp0TJV4_Qq)J(D$(~VF|$Iy%^k*HNQV>X z-J}>XvXQ#I1vYC>cuW}`c=}~lMn*byv329XIt@{;fGOt*7hl!(U=ken>lK?Wo^6C# z@n4v`zm<9nwl*;Pq)Py3ukTx!oBT)lHzyX}%|D7=y#`JM5J}Z5+Pr8u)R=W7adFU4 z(MYm>HA6Wq&ctl$^Fd%7R=-MoOZRE?IQC}vA`s)gCYm=#4x6;j8G`G$J0?z1ROZDs zm7C#7zZwn~Gly(2@hdIA-0np|-t0IXx~>Q(k*6N*!}8mRawQQ<>dyXDY}YcAZ1B?j z2-Wh5qXB0RxFO=N9~uK|*({;A1&YdIQt5I&5(^W~9DnwwU*12w133QVclwm=Cnh11 zW?BP4EwOnz#xs9w0RR$RV)I^KIqT>fwy{<}TJBljz?9lGV78fWy{)Wx9`TAA|4E)t z^2lO9Xd}KDU6WM>E;?fJKLWZ_sa{+Us-vn4xM)q8SG&bFtWnenY!=X5)wF8WS^R;z ze#c&cz*R;f(H!+L&NxP&sYi@+^}&d|_uBybb?oXPzsDl})0oleCHL{+nD+Q+zwq^Q z@Nb}3;>%0Pr)a$nE_a{TZ_UjRr74kfhmRa-$ejvk1-u3N_M`ajF&{8cRv?fu0D(;F z8l7>uWnylknUYJe4W1G!#&05JS%mPSaq(|F{#5xuAbZ6OCTsgMP~; zjK!Qf0v4u=*8e3MSEO^%8=jEn(MO<8CmY<+vEaT& znH`!<1x$lgiYWhf<5+=E+1(HXVy;z&v~WZ&$CJRZlnmUn^Cu@U=@5>x0ARRRMZlVx#^pR# zLomPPP`kACa<(0{zEE0ZwH_PYG2eBu5*_Uh3VYr~Dh%vIu(Z7BS(UJHcOmR>wi1s~j;;J`v-1AgUo z5B9 z#IcnG?c0$wl zNbgPkGlr&DMki4+a56sZcenjJUulSemlV-Oowl_eVx?ALVG`Ub2KJOX# z(v0{dc!fbA9d(21w?y$~VTf_&(NvxGfZn01jX%Yer|-O>i@6grqt;$j(xid)|DeQ` zK6xKpg<@QlS(2kc7k5@pU-9=^JTwPG^PSI(8_Z%+*wX z=xAP4V(9{w*NhmZldt%S8q$muK7+6WfdM;M<-U5kY>PZZ{afpZGLC%S*(`v+Rkz(a7Fq3&(l)J@6h+#EkJ-(DeCvSL=x z-A+UoIrwUu$&i!@LyF^`RJPwSDL80CuN5dZ`wZ~GEAL51)*n9yZTPh{oZJB32IE}auOom;SA(7SO_DCxoF(zWdhc8 zbreujH)ejn)w_haeAT7rZc+!7k#sq2r%_Kz(#J(krue@G!O(iZ<9NeMm)8KSP__Ue zXl$ITbydj+hZh7nk>JID4adoVk`c*?9}(ocRlqW`KP$2dtAkg)xgzWA;URr2@v1j$Dpcu&vl1;S&&z{re zoYzTztR}-`84v{^>qWJrr{r##Mx=w~M+X+-UquE_>qvw0*2VPDc6rbCcH_fGoYcA%3qWNQp5CJ~jGr zf+bd0ct)^#IF&DUb*{(QSzjgYZqw8I-$i+Wn63i`-7vIP59T1qk zv}4SIUm?4%mJNvEQIW38g2!kWxSOvLr&-urT9P5P@MQL%hkwlmV^aC(ag2d+w{_&9 z_ly7bs{g4?P%~zF&3*;6)Z0o0W<(RY-|qn%cel_h7|u|Ft`#ty1ogM;w_bf-+q(;T z7;LptTRQhY;DkbxECt%f7(?cm-?FQXpb==xjp~S>XDq5uNDxpnly7Ha|CyuPrKn$p zfexY%1Yr3yoc5et^jf^dk)%Oj9<*ri4oxYR@8g19eXl}42o|T%1Js6+&t1O4XuuP86{ zH0+iNk9`3K6jH+cdM}uZ0++3*Bj96bCj_)|!Algpy|MRcFzNSfgn-__xGM2r_1~@t z`lej-rYS?r$%%CTTVJRm^&%)yTGn4kRW{@X#28k%z_gMdY4^Na6R6N5)F(W+DC|G%nJ*to#w zUQWDxgR{(3FkDa=aFXKfze?k> zw0+Ih!^xc)`KKUp7W%s&kt@@s;&0#&z!Lxntr_QgL16CxYaRbUI&D!*!HOn?JLhn+ z?zg;?eOZg9T|5O}=Cz$!hd=>9oP^vp)q@XpcOA5I8l7;BDoetE+wU0|9uFcX!U1O! z=?vV32tWe=E z%#;6LX+@fu&@w#5$(tX#gRVXX*Zs;fbbz))`PrMX-i|8ENJvsNp#e6IMo*=TPJQIY z^1M6)oTz*qEqvF?%bWw17Am-jDR53S&=+flrdBpUq@*snS#;xRRU4maxE)ZP1MW9L z?W^7R5p+}n9OyXh!Ono;1Oot9s@=A<)YGt!iTDg?L&8v@knJCE;fx2SE&UFhyozY4UUjn3KasY@q%lX zR@8Z9mj-1o4yQ+a`(}i3$Uc@_HXO3sK?uCCnm`2`3>$`?T<%*#~) zU2wR9^wMkaC|J|~v9;arLO!$A9u8jZL=21V%+eoF-V&nM40DOJYf}wz{v@fCL4-&v ziz;N2@+AyGiEg~LFGTE5)ez0m<>rctvoPSf9{y^koR`!34`fTK0@|vhFgFIU$+~2*Tmb=*Q7{O>cQPL2Zf`I1t@8nR!xRV{yIe3qOr&0mXGU`TGy5 zwfm}i|NB+9Z9`9*)ou6iVczVA-i-t+FkpHJgj~NTh#ySr&x27NoJY&RPZ&$+-2TmB zeD+>I&zTlGb6&m_jo8LVPkG|dW~+oUa3n%+e{`ry#tP>{ojY|^=b|OM;PyddQx3;E zqf5^2a0IPM%~M?i{PapLk?z5MPDb5aubgR4{S(I;)M;4V1b}o{L>BSe=?c1 zR&QclsWU*@=q%An#LbL_=<+g{!b=lnDzh;))Ra-c=TUZ3_{Zq zCP4}wYuKg`R3%jwM{p8D? z`$qzDW&ynjWX_YLbJ|&7B9-y)e71=bQ{80y_K76jjHFB*+}!M@erlMM?#7L@DviR6 zF7zWT-+?`8S+UrwZ!$VH+!x5JV?yz@*?MSBquw#VHGUuAa1@GR`a}J*4vi#(ux_D9 z7P@;&5kQRppjiwLeGy~K8UZJDq-pVW{dAIqW(T|!hA@7B`FCnP=ceEyy;el?qlNlO z^^gYsjEZS{Vw|I!OC7gvcu~e{A*4dpe7`DW+M?*%InOUp+!SaGT)Jtl%{4OlsaXqc64&Qfs%Aqot{DUh2_;K3-nL> zxz8pxZh2ga-eUKVXg*}e)aR0pzN<-$Fx%?!fE;@O z*juT>fGYLAFQ+5^^o*V?0vO5Z)ryu@f-Ro9*C~ zmRT&dNmjx6VR`S#fxz(w_Wfs9*RoY-d;tSu^!BMt63|pUBe@3PIZZoA^@| z4<8zdg1gwol<{oPA5LN|$Gv`RMUXGc#pi&AODe5FJ8x?`RB&j+s37d&&B=Ml*UiV@ zH;!$=+ef>D^7EHNqHTERD+P6a1`&p4X%Sn_X^T=Nr-Q*$23CI#!7y0@;*aT1%^#}< zcy0$j5ci);tp9_HM{Ct2&&nt!@%pL0ZVTQcU==h^wkxmgiy1SDhKHx47T*!i8u~l% z)Ce5_?RmL|w3cJF3$LG^IG*+1tg<2JOp1RvjmV^L?Fs-(jYUBeM_+`swbxj6X76a* zQzw5#`K9FTBkYoG6v&oB-Tb^S#tPSH0j8i^W$V1uPHP8mH^ju+<7bD0 zadB4Zy8~^qrI0BFC+DjG|nrh_+!d53?ipy<@?ZUY{g+TJ(lT1!5vlm!6+h!;)r?g3wX9v?mjIY^Tz@?SIJ-DCpS3Sq7RF z2rJD;{#&?Kd&Jbk6UO;hVtw#4v24NtlpT8NU6Ap|8W$fo8mvUB1cZlvJ1hse&bz5@G zyu0&S0aeMrANHdFEm?YT@sRa5QVR8{A;Kt#T6sIh+0ua)Mf7#~MP2*FW{K1o zqt~kE9UaGwiILHjX#T6m)>gRXV~l%ygH!p;tbEeyODdG0*YXXTl&ssTSeJVV zp@v15LeTqrM8I!|W%%5ia`?Uf8?|#~KKwoN;`*Cn8s7qR{V$n&16))U<8I}w_>4ap zV~99YGwvVZn>?#R*d#tW(kM8i6!-h|>;I0!S1kk&fW;2TQ4&yDo{Q$)sD{|4HM)}1 zw)ER6oDVew3nxx7gMB>WuuBR(at;G5*pew|igAzPRV6wRc6@k)d3ereD{WhP4Jg@p zlvl%tZMLU1?^$?Y6NKJN1t5_ME0rZxNOs)p#9A zN+Z;Y6N6c35tDPUa_f)H`z?InRj%@g26rPCSyY1-FM(6Fce^EQidp*abM}}CyjY(O zv|xzoqAu3>brWnoa~Pys`|yh(JdQ z@|)97#r?bT{|$2wXYgu193q6<_|~*1)ug&nuM6wr-)ETv_KFLG(H;S(I1m7g0?j3S zKsKuHM-T?B*geC%33|<42(${L5+ksa<7d9;*6@1kZRt`kPtQAAA;=cwsdr+iyv#&b zlh$@~AHw+Qz>}r3%@n`FT;!se?$^#raX^ArD)EneGo(rtClw|?+}7=X<`}<62>%O$ zB&{YLrp|6A3IrH_il7)x91QNe9XorIr`RR%xCe_MepcJpQnnQ!uj3r<=||<(?aLvl z563QJtC<7~I!FTbLJtQ!yKx&Ln_uMyVcrK@CA9!2r1>EC+27@g&u7(j3kzt+6XeXa zFaM2JXtAVF*2M$6Q3H?XQ?I>h^-J{}7){95L8&9%aO}zyH*&-7i{eSxzi8s~+wa%t za?;Sg5dll&8o9FV5u?^Nfy|A8Hw9WiBOCysziN*iAllV1E5!!gt1_gkz6E;Rv!!V%6>F)+0RdQ|5>HEp8@*8=lTX(DS^$9RTeIz1Imd zB0CQ8V#=Fr(1kTutHtZzI(4bzQxZRq5Z=87-ql%m#b>?8Dinap&T@J*j5_fC=kN{+ zjzX$X!_ZYFd{xfsMnk@n$EvNEGc@UXG?*Wk4$AwiV*{;fI$t@)?+kxkbi9q!_eHE+ zUq(|FZ62KaJJF{)Q4OrB6yt=jEOZgo` zDf9RZ7N~C2rGdbp?iL|o_jW`&`eeYLp>=EG4V~cMn=RoP3^7f=I!il7rRv){n&++8 z{p+UU>`MHgYr~E(SmVYI-KksM{x`lj|4SI9P)-IyFa4O|n+<7%W4==1j6iHZ)SW?Nfqf6Pap-nTDO%EHG*K;GRb;100ND;|Nz7e{6i1=3@2 zWJDK9()rNBy^%z=vTr+0TSS+sSR}RayT(X~>T;M1qwkw<@k%Obl660SL@5p4Kb~x} zy5zk+xJqMPIk$^_0_5?IJMn{oI!zCIlLzj4lQ-vPhI!VS10}Kx;!pd0*r+Tjb-8t8 z(fv|CVyGL@;25v{sVd8-JMX-!;G9X1KIgR>BnVV*N04OcFiaa*5~*Q|DnKo||NhmQ zFWS4s-M>4-fReq(69Twv=?*ogyFv=PDk62qaz~kR>s?izHS|I2OjgKAo5o+dfUqpB z2MzWF%ptiAm5egUkFu4lOU33WgxYC#!n(5Oj`?=W*{MVMO_o^ra6h!Q7;!3tf!8AC zO-EQh)&C9j}jz8ipDuhw7q z@`#QqpF=-ueSU$1u>R;LyxhG@SHb&WVb-Pr^p~Q>53*m<-&z1V!3tlsBCe2%>2`=Z53AZM#*i=<$2RS>)!ER1 zoX3KKn+-!BPYd>UyPS7xk{0m4U*(Q&YWd!WA}kuxCH#y*%l9$Po)N%LevCkiij0R( z&wYnr%1`C9Z_m?j<|~vy8`-ouN^ridj7eUvDWpB3Dib5b`NK{wN=~1@GM1RhRr2kd z<*{8p^muA6)|O5BOP3o%PX>41ZHB~|Ea}xOg{vu^zx-=e`vr^b6d!hwEIEzm)p5 zrM{{wwX0CT3$2AK@oNpGc}4f7{aAp<4-RUIg0GLsYlD_0aFb@fbqjdNl77knZhjsY z5uEeXMEg^tnbaT(1E%&$fk1^i@vL+d%Q|X7<0)5+aad?-zK%$q@{g*}A(7e&)`&>fgi_bGl-RVaJ=ZFto%ELq(wEMB%p#X~`<{zS zlCZ$-H7#yr)e6xW5;}G$$F0%h(Xags#$SuB1^u^uT|do0@{V|$RnHUFHvsx|e{G6f z$>-iN$x}u8@vcNB+niFclJ2UJW@Vyh^%6c`T7DS|fqwP-D7C3&D+FwAs%s=}cfUYi;wu|q96Cdp>lF6? z+BhTD&xV^Q>4qqmUiNXv*?xW)hp6h*x92WqEH-dmVCj*IiVwEy?iYWyxkx=@QpdJ$ z%`6ckwByAx^NznW(*1U5VlSV(K}~)9Jo>?+EgMTTst;n-<-l&uEbIz-@cBBd%)#f1 zq`#ma7PWHVVCPwQXdAI~oI@p+_{-|cR>9_~$!9mU%5Z~KO41rqBNa+R0}>sUL)V*MhE;W_Kf z3aYiujy3yI+FA4UF$}oXwoD;~As5E|YCTc%wL<#KA4|PbrFt-Kb5W1ZY7gaF!c!9= zCgDJl?Dc~M9|4OgXVQ^3^d)oB~Wx=1vNwpN*1bE>&5ZF;FE zE&5AnTCiO*d_lU(rb9_$K=q~0oY&Ti>D<8jK)uP4YlfYK+?fN`N%6L6cAHxB5a+Se z*~KQ56$wH7aT)g{J?n^WNyGpb1yxpMIArMHRyJ81J&f=@-6Od&D@tXToPVm_3QGCZ z!u1PMPX9Jj!D=SXn0prfqWn;>onex9c$kFACpcq>jdxf%{s-fJ!(En8Ou*Ly6iVen zU0N8`j(mJLXt|I6Vc<4WWJfXao{8Voc+PElfJbp6v=k~_n4#jarPO(x$oaGy>1;_(+K9cMq6I>?7g^#=JZf~st=(P`Q!NHK zUHJ~iY`^k5^UTJjx2a^cjPzJ8kuRZq|^(%+AoRc%C#ys(6 z^~wthW3_<;y@7@Iz||H=(gM68=x`rzhE9E=a6*S_vP6T8E{xZte$btG=^=N_7Zp9` z(}^EVd%mF_;73}aTnwl_oaBcY3pHc@`tG@8CQJFLzRt{5O2zmL7v-o1kl<)U`Xui^@GJPpEE}%PqCV znq!8_ZF4PZJoy|epwsa!-KEEr47OI6B*Y8?&VZ{6DqKe`pg%sd;RlM-_8%jz%Xs|w z%r0H+iqK`}zj#^zv0W}*qUseDxV2i|A~Tuy>Hf^!sO%LixH|mA#pRIMx(9A!=QOE^ zHBqsQj%NpWUtF5AY>7hMXzwtjJg%T*)wz#WcfM>;ob?)uvppTLI9e@MMi?wbkAc$k zk_9;pc{$xB$C20;2k2J${E`#@Of1I;~9P1GEgkj%U9!>K?A8+D4Gm zI3wQ7L$^AC=XF=8m*kXvHhS3_-m1sGDTezxl@!d+-fzL<)<7P3!doPB>zhv!p8X44 zdBX2*G;lxR%UF3F%R1$Dn)P6R!Y_!7TaU*`TFZVu#}jV{7}tvw zRHpZ0g6^xr4!|S7+q>)3Xr{tnraeqt#M(h>k$}yIyNS-=f_i*xA@a^R8Pn*+_rsfV4HICMNqZ2NLpH{ zrx<-R|7423VO>*`N#H;wA>K_v4u7H|wh=WpGU+Yg$^BLacEJDO9Xadzb#bX0d8ZSl zE~gXwrb(#5uzBSJgz4X>N6~JQSBVkTw z`hwitYYt-;cl+(%2VW*We7s`zBj}LfwNx>r<@e)mpP$DXgLWI*;&`SUukJKPxeMo# z4)6{rMU58cq$yYWN3IfZ=uaz;_Z&DMl&5e6KiL$X0)0)o8tOyYBE|O?4GOni@E!AL zXatAI#lOK&2QitgYm1|nv#(_a<3u`6%-)-k_5Nukz%WGyLX)zUIuNw2UO<%Wfgze!9li$;8nw&HH!jwYq$Qm(de54 zoU96ff9%X35v9ijm0jw;d_6b85PbWPyk!w2C|hW-(8sb7vH^W);opV5tAO20xXaMAIBMEVpAi;_5ENr=La**JqCLB z_2wFlaV#~IcdV<<3_6}GbM%^8Q<7D&g19^oUaZkUFF#(e~cSiw}wS{5Ze@u zpYNj%k+IY#=pwy>IA3{ie{u{(qrnte-itEaln?|@L=6mUtMzo-WtT|A|Kcv~uy24a z@~#K&rx%y%I77UAFy^GqJyF)*&hNPjz`Y?Udb%eGYHyf@?Wx+MN0`;oejX_0o$Wog zjd4G?>fj?@Mjo01X}mV^gS@=c#15%sIhV|LP#v+Oxo~U7P9&teK4h-_tY%xSORIKG zoagj%>?h(>_#x<2+g+m+2Nx5Q&{03VUz+ZcJ!!!g2<4M4+B;~n(&k1qf+sPAonQI> z6ud)%JGY{uz1_e-0Y=4v>9sqAY-;e%3beG3bxA2rdhnez@_-bYEt1RO;vd}j~=q(%)0& zjXmEKlt>J@JPrBlU;e~YpF`9F$4|PXUb}YETW+7a++si6IzfVraGF+jc<*c1{x$82 z9826&Iwk{dH8mna?uoV<(f`f24bq-wmd$>f*EU&hh4Y)G-&yhYo z7PvHHDqr&DlHR;Ig>rF<;C%p}BZC>GMLtaEtyUH2${gE`H(9x#re`gDHx6EUC`}A} zh&FdwaDp_ZoF_U5mx|JPD%& zmw_NhsFtsbRsA|^DZF$L{C3|a#XU-n$ybUXRBz4BcTpHH)P(WT^`m;LK3%D~)Sala zOXQ;r#?^%2UrV9EJcKP-`0vP94KythY|d)`A{`DWr1Fh^Ntk88ujUGe&Jp&C==Q4= z^1AK@?9zwR<11(AC0;_h+rZsJM%oS^pRhf9G9{ey@<;UAU}Lf*T>0|R9%^=b`+gyf z=~q?_hM-FcwG3A>NovWSI=ZGDcTKEvv2oM$zy}S1N(VC?7z}ph0AP8l@i&MN?D6~_{riH)fr#MA81QlhOLYEFYE+M-sPETBYd4z33-_Y1FII^iXpj74qp;**pmJyI{zxSpEW!Hua64;ZxB@C9UR2evEws4cUO83(1As$8_` zLQl9qNf#8Q>B;P=vnPR%0gVp%?%YE5y72# z3A+UwVEk;PMx5M}ftmJdx9B`mzoLH7AsydJ76AbKeoDEDS!Y z<`U^=Z?9kbeAW%u`PL6N(9FV8MDsjl&jMKnW64!6a;l4LJZ}pzP|1SS6Pn{SwVV)t zVbE`Qh|mrK8jVq}(2hlV)PjCc`CH%@j{q{9c2v2H$_Vsr5NI{d;2Qa&3hylnC9gwsnm${_!(WNVB;c05kVBi-gs*lszaeZB<2xOn&E~@06 z70q09adF8lDd|exe#E@b?)#M;ci{*zD~AJrf=@M0U5noqhxIi{oYvWBiHd80@(|sa8|+-kREu&buLJ zzoe|UNAZnDj=ssfh06Qns%h|cu}*Xlr|Wv3arZ4VG+#Bj9sFT5``yuw29C$LYss1L z+(PFNF9@OW1-VCPiwTG@l$Gy{GCL&Aw;uyggU!aWK0d4an)eHFdVt8AdibIqLmdC> zfay)-F}mu7Abk+u)qY%7=l4)ba7Ke9_{QYGz@ofhK3ndkDfgMV>4}EH#CV{z=siu5 zR&UjIak2R5dfj@ZLV%|;_xIk%kA1o5gKu~3vi4kQKWC=IBn&y786Jxfa)1mZj$)^L z=*FG$++1bDVx3V2$97I_E>93D{fisSMB`DIc_BZ~w|dt3n9a_NIyIW~vwd+~$Opnc z4v&oedHC%Bcd?Aa&^wJTAN(1ZTRs0$!z8^e?nW4Nc%;?1X?^QgVL16ipu#gC(V1=s zmOH7p8*i2G8DEobn20mTsVcQ?_!@5zS8mu95j=J6FtL8kq+B;Dc2PBHsj|SN#2R9l zXFD(FSnAR?pP6cq4o$7347|^`v3A1Tw021;v>IEN2E(~wv}y}L0^Rb3^N%bWaurPG zTUOR5@pKGvq4mW!Xls6$UHLaCrYBbh`q_-%Kf^gLBG~Q-;zp@;8uW{}i70QkScH!C zX90MW!D|yC<{VSoVdfk=MGSlEgTW$mcTi_*fa%otYZrbNNGFRp^gOA4>}KvFDK95H zJVnF2;X+QuXo8Zcu8T#{ud$)>NxL7Pi3um;OavAc?AkcH=M$doQugaN8T@;{JvQTL zTbW0mU3mXu&#vhxavweaFy?{Z_d{;#?J0Jv3{UjcMtttnC^t>65{_$FDw{6xd1$Wg z{q#bRQ+C%*l=T>T20A+{{HWsP7siV(Zz@lfPNs#<_wx0P*HX=HVaqf{B1 zYe37Ixnch}N>%zKkXN(s$(sEY;z1`yt#eF&SAP2B#D<6mFLe4SJ&N9` zqZ=X4xQAVG=;eN%0EWBVG1dXTm5r;|`NKoJQfLjg`j4Av#m9!o{X%r~G>b*MWYt$& zUk!iz%FOeyoyz-l4I6Q|&r-z}y39K}VA4XpGAeoOThQ`tjzLJJN8+UZ{x!y@2DsbE zrR5w&w?cK`XvaQtb*|@Cgtc@YRPiUpB&Tw_^OOnncqwhXkv_BJr^e2=@m1^WX4=Fe zivf#`^!20?*f~e+CTC3CY;31O=jc50=lO^ATcaH3l*05db$J%*ZXYj0)4f2b{+%-h zYTXn?Pvg1NOJD3=x5^cU2V;^^82kV$dOh#^sZj0Co~MdO6aat#)vFadIBkaFSj8RLUam>#4S5{u%qbwPHP zQuMHJg33JXP;An&jB8Fi-th*Q%AawrM%=~oZph82-Pfx1(bX5;Sw zU`utU*N(3eINYbcXx3l(_HnQO>9yfQkVJi6>vuEp2K<lcN}?NY(5a@G;iW3D2!?NJ@x0;nw_BJu_lf;b`dRPR)wo97v5o zpQ&iC!ZJ}$F8AP~u&&Nw_=wRA}Rb5AR&W?Rj{ zRNpWy)Pxc8(><9vfgC3neLK`Nu?~AP*SlAJ{dSs{i57{{GAe$L9>EOiR}=;W^B@S|EGW24GUGy#c)jGu|eU=sj3t>*z1BrdPB zvk92)sJccZYGlH?r}*>B@WJ&wyI6f@?G-vyodRC9grf$erb6#8x*>xsK7uR-c1pOQ zAg9g`dn!rUqH-ZYIgeb|o2^+|x&b9;9#i_Ay4pM(Df7sGD2@>oW>au$i54Q8iOp5d!5SSh2&% zde>fUzx9fyMlC9a~Sa$&jTX?dI%!>)W=^HI;W&1i_DtEWpPe|qPRo&MpBYl06LdOmd;ww3M_H-9uUleeOhah3zOHjW-);9Lf` zJjEFb?wk;Pz+g=L?w|1tyPeHJIUEo;clk=-9R0QzQL8%CRxSuC=|9xqzOyfvbTv&R z^S)Ber}!;P;$=NbUJxa_Cl;pE{Kmtc*uDahotg4#vq8A+9QLC$fX0*f8xK|$S3iUf z(36i;ml$3=?FN}jakE$C>*~xww{%}qh2C2-MRc0DOx!?-S+_TmdT&I;54;6|{V-6o zs$AN564RmIkcp9$66KpF_~(Pl&(ot&sz~Qze>`$*54||A^HN~rfPBN#_8&CBAE-fHJ5tjY-2%P!#)X%?AnEC1B`0;CdCIr6(NZlzNOt_G@LRTt%td2OmqRwnEX zY`?bhZ7dEq?`?=9FJYe$RmoF^#e-85Liz3|2C3Pti6T|+2(|_%BbUS^2uts-k}TcN zRuTkeEE7v*AYK#KL$?|*B!y_nnwczE+RBCbJZU*!sgt>N2q%ivf^XIdC8C)60|muP zox_qyyz(ll5M=3JSJ**!0s@+PUV4nRa>g_+OjT)v!uBu{{ogR3;V)&8opCKf%qd_HN^eix1UkDxbc7er{%p zEFqCkshYR3x4G=dHFv?==MkB|vi00Mbj-&{^Mf6yqR(~RR~aIINi`Fnp3X;s26cnDfeC%1-b+qr|>1*-vxk)eIas&$`ZQs{PEn#fDs{V zcw*6X5^)umM9@9S#C9TNtNz*z1FP_>&MpRlj6O&CSB~>eQBJWGn--O(h$OGA`NX>) zozaZqe14YZkj@Io(`!4HvY-Yf5nXXmy_5c|MjXd}N z$f5GZpIkM{?Z)m=n91)Jc<)#Iz-5oh$hFeKf(z_PScPf&$yPUpIMvz1nn=$M5$=%F z7WCR={0GD0NdBQtQWgmj@C)kXwtel&X*i|xafchrq+H9VmE(5nYbtEgtJmLRt6tSa zy)-tjmor7J3aJxytQpq=#%5jwC(Ju951=_&);lThU0t!N6raf?q^V0U5H0#uot9YE zO$}WuA(uO5l~nNY4gU*sj4Tcwe+wa7 zY6|ZigU|9lGLjLibSSttREJ-@apvaBxR(@E&lj<}wKci9#r~(kQ;GkFwXcqfvi;hX zaZo}fL=dT=J0v89p@)V6rAtag0YN0BMWnk!RFFoLl*U3*1O%j&?ijky{SdzI_dDM? zYkg~dYw-s$%skJ%``XvukLKG2ywY$+{I-qib*FSRnoY8|_M9V8#G^s2ifQmr=*Yxk zJf|zsOH0e!t?ovQ)wTQFytd+xE=QCUESjqQZui?$f#R=&Q=JO;+1@4-dtyKz#{LcOP;pC|iY}kyP3BA@34BJOD2&%`@c{Qy>gkSCD7uzvmm_|R{}mne2r ztRL81;Vy6gB|S-pNEKSi31MUBYp59wVmDu(Eu}8LIn+u3a`JAm)z)j|yyhq)%n$u1 z0R8$Ycoo95e`@tIg^b-o7EL89UhPVL29YDKH*l6K$22#0)y=P5qbHZC6gI!`y+^Y5 zGw3~rq?yxwa@9RsF)nL-oFB=VjJEZ*P!!sMCyq^~?7q3PnDZHZ5+y#sqw27STG#gQ zxgtE4ysi#@Y2qk^M-hmWmX}XCiAg-AM{FxZ>k#6c^BL#1VdLbC1ydv@3}l@gfV-2J zeT!~;=zNw$y1@-|^{m^H1h&Zp`wi>L1U|Epx;3w)X!_O!zxCLSYg;$D8~il3lV+RI zy0LwAFnlLDUr#M1;ziAq%qzUDt{(#Fb{s=*br$IBt#HsN^9D9uL?!-c&MyFIOfivW zd0Ao0{SDHgWRNT0%b&Zylib(0DYUQqb2HOm1GX#vw!ANgWM<+IgIB`b^?3K{kOl2z zA?m@+PXL{Zntc7FA#=S@C+cmpz-Rdjv4H2mC$5$~ULZI+9V*p?@oyGkW0|3koxaR56}%Ff1oSJ&3y>qF74AW3qsN#)q^<}83GE9)O> zt#?=CB|IoXe{88WLKo2Pif4{`RIX3oAS>wWX|rp7TmJak>kr$qSZr@+c2<2w zB%oJ1DJz{s1x;IH+0$YIFEfh__nQ|!-IwF2nYjCekQ-u}Z*zOg(Rq&4t`cIB3Iza(8w;;^OE_a2rz)_DD;dIGa?io@X*QLgTDNT6%nkevG zLt9tS$mUZ2wdY7KcocSq!5ydDBOjK`a8RkGm`^vkVCWBX@*AhQUk0Es#k?TmL=ehA znV+7Xo_Ryf71Ca;v*(&VUksmkRvbj>Ja}CfYEgaBj?Aofk6Njt@p9Jzzd)#oS4gS4 zyGa@Ke_)k7cb)O&xc*~)RS_&!SA*CzCRL7dziU@a9?$>cn(Hl?<9XY#DUB>r;&40r z^R=vizxf2i$cK-Dt3Rf0;jyLhE4;0C@S|O-2D_ZjiE4^FjArcf5=G39#)k=36`{H6 z?gXR^6v%)l;eaPuDx3C ztV|@1natb=|HZ^zSKSd6s>R?o9{H6t&88u&xF>S*bvtW1=qCNF_nga(8Y7#k$2%OE3U*`&%S}{d+TTe-sQO;zDId$ z1mV`&7Bi;C&WMx7Ywep~h}{d9Jh6&sq& z`VMfmpKh65T6~^oWS$H)Bj@|DYnXR+ez41UdJ=(u3pStnDv6vG+c2dj$6{^mrguJ4 zAJ9;%22>x6RNxIl9BLnRd{oeCbQw3EoPh2*hihA-!Q64Pv;BS)&P>W5hKb`(5E0!x z8xbr83O91?0pTJ0vFF|y?-vBG_f0K4%8R~C#z&BP=tFP`1)wgop||9sx9{cJP^GoV zAUn^n?I>5670v1NGYy5RCQ}G1NqcxSUse4S(=2g?&q~M1SwCYVIfQ>?dvz*vJ{{B1 zX(936wy*QyWP36_p!ym$_exf}lFrM1K|;EBNAY)D^4sV%J8R{Hedlj6O`6;fb#7)V z`C0=K04-C*o=U1n?BY}^v)9(x;f0la)J^>(j@;YhzG3mjTg#ruiAYwhxhze0Z)=p& zlTCIP>IsrGPi(w4qgxQ$W@jYm@&qcy6NJ zhahBXfml2D#uY9$+<(cDOf&7rQFLpT{x?K@5q0=BBWb^^R+Arwyv7|#k%yb>7L z>u)E9(;Bkp+{-M#_F$k)rL3;BjW{i4{fhKk`!-$KvbNAd#$vNy1Z}ISISo+$*@C*8w2#XY|W?UW4?4;V*ue){hmWk#XV5;W7BU zHIz~OzId&1FeOR~iKmi-B7u<;3Atj-z88e7- zUxJ~h-V>}g;-F05QdgYPc7B5qpC~fL5+BgP0q;}D6xrI?qM{s`A{2dOr&u@vD2#+J zTg)x)w{dxexFm%aW%lvT(EP%y~Wj$55r&#Ou(fH}5BS1+C_1i9Z_fex8C_~IaIU+4Y zf!;ViCGeCUoN=%Yo|VjvYL^bN-4cCw1z|rvmfl%X(U&j}O4-glwe}w>V%t4kCyj1D zTWTZX$=|i$3VledZ4`L8N8@<7SoO>t6FN?&S31lJ+KmbF;A&q5~M8JUSObHI~ zCh_^&Msi0Q5w@!XxCME&#}M+@%wc7Iu>Bd;_b?p{S08sB|1bhfdAL8)wHheZ*Qx{T zSxRsGRw|aYmQIS_l zAQe8pH9XC3(f-}=_90oBZH%3}&E1}rY2bp!!RRyk;1@ADxPcc)N`l2FuR!MijQ9b9 z@B$1}m{FVAWo2b1Bu_bDNEN_FV0XR^YY{-vb;2Y>9q8H z1m0G*ysVX@9}Y@iDhW@6$A`0mx1UuyFL43zJM8gSDG^b4Mg(bC34pnAko9EX%4$s* za3GPM=rJr7=<&;DNnXMLPN+~+3ilr8R(map;D7=knib=Rtj75Qu_p285w5RfC~)9U zk(8hblwp!z!(-*+O)@zf3F*kuMHzwnIet+%VC`g3i9T)&Afo5U{j5~Ssb5xmMXcEz zkfkBR_D1GIqOROiriPs=k3b7+cQ0p=+hbPZoC4hhi;Pds$-Tc2OEkiFFB*8Xaa5Ez z*L`CC6mXQi5k|OwWd;2^48t|}C&-jIV(HXyz62jhHHYWK+3?_ec}P=yxTv0CIN4r+ zCwIM(jvw5ZlqMVT5v&C+2B8E_Mb|f-0xpsWw1b(<`_Iv7fr|_f2TDENn?NoF7Slj& zk3zSsqz9NM?oXaPx%(S|c$TTffQQu%3Y+|#k`k??q$IJ;EK_;#e)-@*M4+#L7MuH? zPfPd+22q;dFyWw#SUS$ZJ|YM=J@QetF?>Pd`2QQ1#0ev|!2-?_JR_0eFwe((Ttbsh z3UL9FpgN<2A94@p3+O8AeM+w)bLvGUN4Dcg(CK@8FjPMLqSo=F5Bzov_ z$j-r{4wVCOe83|ek$BRvjYO~&`XheL6?tUPh>Z{V^LQlGRm@m^JM_Jk5-W#qhVRCy zLOlc4u7%Saf`xU6>!*iAQ>K-8{E@gq2Ly6*88kqRwsE4>p&wyLpa*AUaZq>=abGzo zNM`mP={bhtHS1QJ`w^SrlqJ`08}lh~Eevsc$m;<$^Z|Zn+{>CUV$VuZ2FDLP9szwd zR|bG4G(I0_2A%VSVCX#~{8piA_-HKvYo9qUJhq$!DZ4`2`%@iZ0S^+&kW#n2fU~6r z9FbS1jNi%eoPps23Fr_)SN0nCTUNdGHJclTRRJ+6L*0e;uHjF;W1PQSIOC&AYmXUk zQY9D6ecJQ0N2!WLw<|BH7%`XQ`QJw3GjI&VY4ExqcmdEG)DTd0(MXH1sLPzj^^v+9 z0P#1qrmv8Nt`J7N){wV6h044|fjn-XhkpXg)>%`W=xt728{|P*WxmoKb@Zqq6ZR(g z43zac2LM)(XP^Vtj~N{u_4EFwFVXIH2A+}M(Ln^m21SaN{Wz4`xd4p@j49x}#5kyv zkO;)_d10H=o)Zi0@7K%8%PTH2cxU3nVeogSYq3#5Hn? z6UqAMK#Y0W#G5Kb>oR#I)y3LS2kox$jg5OV-C0>#wGSVx4LEN!4?AyO>#Eyd)L-@e z)5q*}@5@48R@Q||caz%!eU%luVN~LA<&>+u4}C3Fz-`(*id+PtX$He9jN=D3jz!hI zkvfWh2tx3%N0ey}iK`UC2tMj*KT^<-gIaN;;u)|%K2ws#b^Yvlq6j?Mq=zT|jt~bt zr99)MmaiBhf*gF;>hiBSC39u)d1mI3^H5-S7>PMQKK;k6XD=?_Q;Um&mnCX--3D!= zEymnlU|{7p@O~+t>eKDPhjk^@RcfH=p_hBl(n?@H_H14Bk3zQ@X|#xgi`;vDIcRVJ z3?q=Qr#}<|l6h8ayZ!wd7VC&#AQ&4sj`~rXH*iqHx*fzY4ajM|xyMU}Bl8>yQ95rC zW3h_Zq$J9u0Vn<*!Vpb^fop6l2=aWDXsQbI)!~yopi3Q_7euFK^^>KyzzM*C5x4xc zIh=9GQ@+4@&an4+l~l=bjC&j!of2pvwQy=tMP<&~B;~*vYN!<6aIL<#*t9`+7BJ+k ztUu2EM#e38?`N?yf$RIGDpZN%09T^%N`!$|3xT;@_ga2KmgWbbe$VoZo;8!6l%jog>_QpzDJa;GwJfitaeL-7VBQ7b8_1a<^Gmb;b9=C9N<)MyG1M4SO*Ez zXI~$Q2V25Y>aaww^Pb-nB@&mGMo1sIVr-(_grj8iC_X3+r}y{w`)XV$!r=fytdH~m zaw^|OFj|x$=EKMV$3W6-_s0EaI0M-BSw)k{@m=$!C%cT@3nKc0f=a8CmKDpR_(Zw~ z)%Vf3D7K43pw$HWcb)MA&jBMdSw5bEUUaBzyevwkVtyuGyIFRBx9GPS>*M5ATzM z$Z_J)%>faN)zNaz8%c-*yse1m-iP^vZaa=J&~Sj+x+(G_( zuPJZ3E1&3fBT*3R05Csq4*e>)JzbO{5{+cL$l4pcW1O(QApiJ>_NBeBm0r+`*>qRx zPIxtzQBz|$>{1n(^cC#ZLM!8?{^2Kz;DyUMf&jw}6msutk{XuvF`76tAhzF@0uXWl zqS#&&FgmcF8n_+6LW5C@qo3mTL@j=pdS3B6og0pbV1_k4(QF$%S7EiY?|O|>%#^!k z+-@SAX1^+yht9&~o_yvCeMEJ5HV#AwuN1gU zehnr}_^)X)f&G(dvSzIFhNq#x+xrl@UaM;j0h$emC~>i^-56MuYxE^p=L}xf

Z; zGQCTHJ-qPC9PzM6dAfWyMBmt`!kDPM8JQvEehK~?rk9|K%B!`F6u1{p7ulb*?7q8J z-EhwJuu+ozaMs4M=}pb96HL}01erZ>Y2(Z)L=xB;eX)EKITI}f@H){<?>I zFM{scv-+3f@r_yEB;MaC@Ol^>Lhe|={r)Mb`Rvbu@i!MK@=M;P2}vF-WyPK2RKWnQ zy9~p&GtrLS=#JgkLoWon!2_G;kcS9$5OU?YN9#eAbmZeiiE=h&M*7|}P&7vpp-z>@ zry9inW_BZAX@niP4UB(W{i#+Ve}l&ZGu-wiP7vM}hni~OQ2Nec&8#KOa9~+~t2K7h z`w-sC9k2c3Fj0%!1EnZ`Y4hiExj)bt-@&yVfv4k}E;Sc9ISAkx@3ex%Xn&^yhb<5r z7pJJH8NGR?WFA=$6r}gNSH$)7LdC0Nq2(hnO6NSOFhEVbaqjCK7PTfNZK>>m;6~dkBu9$L)7u?`)(dOQ=@j6d;4XDlKqqa+TEJ3Pg!D9(+@tv*1Hdae5 z5oV>~fEbM&j@5u`J`x9iM&7JN*)b^35qRyTb?#TMzP@p*cblXs-a)Rz>U|>j<#n+8 zAWDJ~RB9*Lu08P8*yT6O&-rM9|nk@bFF14q&g!4|+> z&ahgd=DlZ-C{RpY^Q7_az(~2RZJgQjaD~2Kd2h7SUcA1ZbZsOC>X!q0HSn@Ja)zRc zAb5!Qgq2yl6*S%Bb7?doH}*`OQ;}#Iquk!g<`=AGA=JE-{P zVcHW+mc!bz2K(Z2o`1_$V6TD*R#WTs#K^^> zi{n&m9ID0wgp)^I*@1H3jRu~ZS43HYCzzl(h(PlD@0|2cmUl*i8KZ`?AO*hUlPs?mVZ1*^&Hk#8;DgrV zx*_5eIDP~!42^}5x5qgtQcprz1n|3o*jPUH(4^?k&Q3bM#e#Q@1Xp=b)V3}hDY1yZ zIT9UI4;H;?yL$(;zVNPZs<O^y&Jjt3D7qSjaI~6brPBt;A1sk_?tylKsiTxGB5(B+6e7oAxTP#2)8C`W> zqrovt+677n${%}Lg?Wz)YK#IBZ3QsW=4US)jq9*8)+t`ZKFJ(TGRqz`v@UwF7N4T7OJ^mg_{FQ-3=7I3MZ<%hDJJ2SWtYOzmEXSXsbaHDZo z{*vCxaqo69zI+7-NSV#oyp7& zjoWmPAhJ)bTkXQScORa@U?*<;UIqd|l6&enfcImZw=lkWSU8b(IAQ!W(gdp*iDW~} z<6L~^P4emv@0HU8krJrfdmaR#DzN!K`4qaRXdfZi9*Ux6yG^a>aWdbGZf47|U45Tr zS=N7oyE@N*iG=UCUW{Xj4fc+?NBQIvIDBW- zgnI#o`j6`-U7p4;6Y}FRf<#U|j&n)Dd4Ue?@VK`+e605jVCF|MT!Q<`{vL1ABo-m? z!ufezfPKuJ1go2t<-N89$;H^^V^4`{tH&)T^!pJ1IkGJeFm};n%aLbq*UJLECQ%qZ}YaS+fR~f0ghLbUn z>>n4`++J=QKK#Wap`%UEkX^t0{><{WBI6POKqo%q_<^*Hm-ZzlLotR;lS$%WWx?6} zw|5IKE?Hn14l0tx>?57E8Rw|8{)Ul2)iY6Bmdwo$OD2b4INXjz|e2u zGay~wk^~7nK~R<%2>@u}V#7NX+-K(p4)w|TW^*r{-}k^(1?)P670j5%K4;H;?$V6s zu&eMlJk2v3nLM|G4aQK&uc+8?z!h^XSZDqg-U7IpeSb0)77ELia|a(B2$%0x7=7*p zm}{fAchHd6-gLPC?*c((k()1c)CP~$sbYva=x*s zREe_Vn5iwX9`3F>8g?+qwJD^4G+I?4`^Nh5m?ED1@Tn$SMH84anp;rLR2BdOL=A0d zTY}=uE-STY+=*UU*KJ-w2u|U^ha{Je4^9LJBnEp0Qnhe~Hng2or`@cV*P+C2GJgk9 zx)hucYCgH!Z{2T{o!?t$%E-~Qi}y0MYqqB9AcuKg{_t>AMz?fUxbCwY*Dx!twO@6e zm)H1YgCBwr8>W`;uGiecEN{xD+lGb3%6|X-oS!#IRTQ7;Nv;AR`(HhA&mc0u6-em` zb^4zm8VE7)i%**Db$t6n=bw+)ic<5)B!lG@<1cb&c-;&V=Xi{s(Qq$x28t$qu^Z`Q z2T`)Vk(msui8`yu?RK;9tWNX9bHPE2kS7C%(dL+v0fWShWO+8yjq1)&Gp3gnZOPka z!jH!jR_M#O6MW;l+B7^1=|{04(02JAv0n(F!>MCC0IL88V|aF(B%)P$Zk9SKw#Dny z1JprW*j>WLgnI4vLz|qW-LYSu^mSAT)W3~u?*!%>90)zpY+w((M7SL9y{Mx0R*(J7 z8|@tmq0tUY?e8&Jaopis9EsHMPtwTEqA6pyTjwr);1E;Tjj;yMH9I=8q@}vzmbfgF zvKBZS+0vp`(0Dj5n!pp-QTZ2-*w1L8Ik}!weer`j1_416j=5~bTFSM?{r#$nAlNGx zrjWW~H3d&c2fq7B4(kN8N|5L+!q9Vpx5>d&?@UMD^Xi|~AE$0fe>g$R@Y3jE=s$^k zAXNW|gb<6o*Z**pw?ymH(q~K$dq-SU9v_T7-kwgEXdDoAp0J?w1t3`#Zpy&E!S+tQ z53k8@#P;nO9CE%%y=4X>6T3SK$d}9T`6pe!uJ<-AciNGO2L^;>m#f^qeY>%TQ=g8P zJ`>AlnMi$QIu0{n{ypXSr}%0$a^}Nb6Mg+;wMg8{x4@li6mYDb^ly;$U|T)0%~{wV zP?wN=uZ$Ep^*yu@3i0~R1Z#IeaVR(pDVA(KLcl7*{<3zl*%U zRuS4+s}9moJ2nJF&J-qKtV{Q{s`r(x0j=^JBS`K4&NqF;$yTm#?7h@izEs|jAnJS| z{jtFkf%7G_etXIoL1wTl*jcmd>i_$u$?xAEXs3CMp+Qui6s#1d9^KA2_;vA(7meL`2dZgCjMu_ zi<{T)P602@Fv4V&+nYk)XM>B z4T(D5ImI?zn;>(wIl=59i7Fv6^Da zIs-LdAbzlvK>ku+WaP4%XSlZT_5V-B3UXfAQ(X4Mxt@V}60$^bPlUW~-@}m&xeC-J zNaKAX+p*>%GQoS_(R0b9asy`fiktTGvHJmWVjBF#*XY%!@%TXR6Ku_JoOSIYDycoH zJ6vS=zLrFxB&ZYl5w~x*M|F%fi(#D(tlWKava3gQO|#Dm?Bnqt%m}C2_3V44^2!d$ zYxhLaDLR(<3_l(e4%Oq^d&h}D+eHBoWf1bj z3e$teR|L-K8N-u-2ECh;eoSaj|F0ziZ4Zu?2z)lvIt710Wf%lr`m(vhy2*toLb*aV zno?lnUd6(q{+odcMh6H{)i|uZz4#2!xlL+v?*9D1OYYfiPwxH|Y3=Hq2CDv``Tnwj zMw=fKXU+Ti8DDJAwkc>fs0r9gc76k6;O6ez>$m4Kdy_ccH(j^}KiY}{_CA?~@y1Lw zb@4zWZ&JomBak1VP=tY63OpJbiNU&q#yMgy5dAfZ}*l|@rwidqE0ow=>e znvR<_OEon?DKof@Zy6c-3UXpzn;&U&65Q*?+yx>UYCQT2AeTsQYlV+WS|YIfpyVHV zm8}$jvHaTCueJwLX`urt@5l;^bx$Z~bbO5XxJWQ8C zP@Wv9mF;?G#UZ`zW^C7GTiwpIKG`g=c&eqrsSpLKLQWzD>{p;Riw8Ph^ww8VJ7<&) z6b8YDQ;|W>JUju#6Qpc0FisK9O6UM_sQVtyiJ{ec5FxTI(=6y={7{>+=`2%i}_EezAe*i0$ zL}umksV^$X&Ve{vYD3%nY~VT)-xak^t@UJeC%?dJ%B62}2Y#%Xtk3?NEE3OozmXb% z2f(K67{TVB35Da4t2AiAPR!Y z^bEoJay8ibyM+&}$H+0NeD`@lL{j;t>QQ0^lm-us>1zUeqU9E_L$Nd+5AxCT0yqY$ z*9lu>Rk1z!5sd?3DK?EX0P zz5>33WEK1=`EhytwT!maudP6L>o@HRLcgf?gj&e8MHfC6#7G53Kxon{U z;E)`{$2%1G(j{xCZYywg;%7rqLhy_Lk|=8=VjNOe&filDR*F@RRs>o8?+YabuehTz zS*?o6Fa93IWw=!%V?=wI^7JHCfxi6A*&?Te>0Sm1-lM&(hkg5bBRbLGiE33~Nq~^e z)kpudW49sd2tn=_4!F+;9Mm(DrmG!LW?JC3BR~OXs6!E-2Q?(%DS?@8k&XX{*uh`_ zCutOw(ar<#?1BR2Nj3eyHW=Xy4#HKw^$cWFc#43D^YX*ln}8%Uqu+77Z0^FN+Kus3 zR7fJX)-ukZzsmS7hFJr!I6&_$s*e3ER+I6I|FmvGX4u9ERZvOGUd644N#c&{prai{ zK!|&qY9L<%UBs>wV>pkc4X%OX(m&S(3CH!EvJ3PZY#?VXliyg>T-!Hm!P>FQsDq-+&Q3I#&&3R;`BQ;+e9v~OJEyu*PXIgFX+`r2FhIb2UuV0ac|y7P z0*2+#%J}Ui(x79`y2fb!H<-eq3=?8lLMOuuD()EaLY8tlWRQbXA_e>tL?Ap&0=xCv zWBy?w9A_S*P^&WRR=dY{98?7#16X`{Y=s{oE1chnpn?xVEv7B2*z{tfvKI_)ZP^Yd7O5Sii;8n726olbLa?|(@H*sd8kBy#%T`_DmF zwrB8|yWns6_B2eppM+_$q&)=^7~wRy-=@P`CtQ#JdG-Z?L8rnx13NKnKWI2b-2_|+ z_BraPKb4P(ScR)>P(@cH|G%6;SOf??f8)dPx4e+6xXvr`UmC|g*}#tnxjjoX{?9V{ zl#Yag8gJfE+t2;u)d~bax!74R`yyGuL2m1popKO1ozJ}f&yPfMfX?yP*R8^8MP+5T zXJ=>gh44u^BA(%sO;W}mS=y1z5ZU0pef7vOzQ}vvKJ$+UJE4<^_-`zLmK^Zr7@pCo z;K-yvR)_uH7gx|~HUsvwzG~389FzUqGx!9l_nv_HxfQ4)Zm;C~`xiVQy&Ug%@@!S2uY`IT;7;udyGgxHs0n7KArT(^xsxfHaM(j zmF^5QkVakS64yK)IRZOSc)-2meNH^`^dwMz{R;Kmh7z7(-yM9PmJq7v4{n!LPMSu( z=DPhOR~Ubf${X}r|JEl*-xBJ=nhwQ(_4JVTRq$T-0@uF%Ig;h{RX~GaXqS7knS}3v zsPEi7O%TugWz9hlF$)jKTwrGjn(P5e%t~S{g%|}_w#T__W&DB`{j+=I<$pT1ziG>= zNK^555v9sU!uwxYT&Y5SpMEm(C3tWj!U^`s-_M)i-+VYy|HYxI7H^-Y9`#ZJZ~X5+DTsiTw)iV+(-8Tm)$8alRbkEVo7u!mBKLzZ({L9UAr3*%jV*F}+;p zwLy0MKLeFaRQ(ZDytQ82Gq@(!_2=_*dSPv*SP5{X+h$5f%PF7>^#xEU>s2QNK90E} z;wP*#j7rq}O8`+%%JXX&3HCv$N8ofhkMe@E;s`&VV$pLd@4nA98t*y{fk zIoUQ=ccljR+wLt*QVaUcu@8>=&Cgync^a8Lt@av!8EdyLP(0o_ye}0k5eTx>?~sfP z$t89=42w)Zi?B}*buGp|r=VVSCj7sx(HwWaf13c)Fqix$ds@jg!}Ou~RMQs>czzG7 zghAlR2$MI`9y<9fQgfJ(GWgCyD!A;gJP}Yrl_(ip2^RNVRXlEV&9D7gZ$-)GIq5J` zO5_suR7e=r85p@9SfRln87q~r7-Nuf|ISDLk?WwRsv;z|5gk%xcXALcI3fgYAgvL(G5ZJaFhPjP{_li)Ju(d!Q7dJ}MeBrq>}8k$%$`2z-M$P&;!CclW!!a5t) z3!5?iSI(tV*SIw*wNW^XUWxr~8f(Azo2I8?34;RIrOv47I$nmlR`^16!s6#N-tqK$=j1;O^IXO9b$Ao^i)wI!pZFdAP{m(nt+%Jc-BxrAS8lk>~TEp?-z01U*{~k z(|wPe$KKxa9PJ4(h(PQ7J>{>XZxW_~2Y$fnKd_=8*5v0Ojn4TsLV7&!A!}k@y zE4P5MHEE~7|0(dOaKA_*la5Iio^S@EGaxBW_lsUPj~4NNxOj@CoXX!{&DSR9`z-d`&LV*Ly&*I=d(4e+IZR1l)2u`wGK4_*dVgU%H9f>EIJ4JArjumP zQblk{L4PhsI?7|$qm7Up(zyiF-fI$73vU&8UHc!f+T<({P;J$8TP|NYTb%3BQnj1! zX~|^->-)ydB`goEOHwKi8Gu+3(Cl%#@r^Na(Ylb!rB00*HxdBh{tL3DXcKXR|z3kW)7#vRy}Hou(tW_0Dsi-FF>2 zv=pZLUF;>Y$1rf3XXifd7fYnkZ0)IMw-fAcmZD~Uu;>5L67L6Pg~Z{$l5la$6y3st z>mI=^Dfv+3Nm1XWcfz*3*JUpL*JSHKiu>h`H}ZrzX#SV>8-OEo3yH=uhroG#^+D>V zTr`N59}fV;~BrhbHwq z+h}y)RNz;3kycIFyyn?H)~^%`yYoS5id~bI)(;*ynx`&qB$Y%v-u>MmS8<6(SQ%^u z@A=b1Sx6i<&=1x;KmRCgleCv0>gY2+Tq|GF7OX-+nPpzq!8~NU-gms8M=B_|JyLL3 z+@qeKLT*Fcv(;l~zUmFMt}AES=JslN%UxS4bdwjrnl1D3cs}BD*Ff-pU5-d3xCCeJ@j3gF@=R$V z4iKNux_#1_C9-PcU%2n6Ap_ZL$GCg`UX@M|m&n(t&|`{I14QuJ#zU`{yi()9najS- zP(EHs0_qTcDALdo;(&RV4EvtY{8K-GL*$c7w5Q4p{Uk*0BHdGH|1R_hu-RdaF^hiGqVZe#;Y ze}1?u>*+7EB@;!(Cdv7+ybx&ERH@zQCg8HvVE%mUD!-n%Uqlq4)uUT9+2QBPG}Tc) z@(k13a4U>aeLH(J>^z5LtNEj~<+Sl%k(#60xd(l|go zBJC${?FmV4bqZ~K5D8Pf&nDDIlL=3kcmhx^d21Wc9=QFz@PXduYT(JJ9kf^!##ZS?}Ng;In~uXhYy=sB0-r-QW3>VdzxEz zf=cmDAQF)RME6Nou;{eHM99EjvmyGFOz=o-1*sDsEYLMV>p*}W+YafZk2V7o^J+O! zB5`Ci+?33K-(OLVY<l~vRD9yS`j-ZgZ&y>ZSrA8{;6vPmcoE_2-pWuk#(=qi6Z*D7htS%CVo?i zt8o0JB`w}bnRm~M8Am|0d?9M5;^Mg}t=9@SxcM}NIv&Qq_277QgRS6Z)u?Vdu)a5D zuiFQ(Q;Ob?#DaCfoMmof5Xp&y2}DL>+d=?YH?>>eUv+KMO}x8z-iA$7c_b!gueOaj z)qBll-@z5yu(mzz<^dMZe;Q1?t7AI2oI#K^_^55#P5r%^F<(oMc-%WKniu6dNfcgx z6i{6{X8wlkY$Y{6=iQTIaKGQrGkZI{rgfu*^_|BNxT>mh0V_vyg--~MScZzXXU2G( zyBkUq^Iy2dx3+t}=<7un?+j1pUG|jOm62LD-T$Qx2j*DsCZyuMi9gIe4?l`mPj{`n z#3h!uUqz(Ruvr}v2rpGHVhZeHo68lI@DFL;gj{6>Zyw3U9DTYCxE|^$5k%aDolZTV zpfUnVHhk`a^hxBFT9r@+<`dX^PB}ptYc{DZX1?|?%1l{T$|5p;*nRlAe_w+Fq6iWJ~^2~`%|*ylBs%#7A8S%J1lnlQKDiQ+Vor21^HW<}x|+w+ z7hP9@i)tC&4!_!7H8g5orqN{G@}jM_W*SgpI|vO2Cl_x-XfxTC6ZC!MVBVf<06VEv zBvc(DSwnNGi>pnaJ;d(~Rta>^CDuxb?p26>mDIBy7cLt72-Tfxek(*)-jY0)TMu3?<9+tLV0gi5t4?+97Rchs8x?xgoIo=;U7e@0TMrQWJw%&@a>kmad*oX z8ka$P{uGVavacS&u11!vK^&Jnq~(tB;kq%V&nk_)BGqG~ud~#q7dfebJ~4Cyobesc z=iZNkYLj`Q4U)BBQHB{Gz!ZvR+&VOcB9`V}6%Lp$FQgs3b{7=E(_5lCBR`9dKb-Q* zf%YJeGUaVOJR3!!_GA(4{9SjuOF<>C!m?L@6zHbB)j5fsak6cNaRo0LxNBi zW6jVGEd%MSJEIdlY<%yU-q_XQPb7xs<*y9dw)1{vD84aIMpFOTa8b~23YDcguEVr9Ew{IF@*Sr%0 zHxEtyc{{_JjC-{7^w?BXREE!f$(Zok?>&q=+@Bh;df~XCyC=@aS~$d`oE0yqtEpZQQfO7u^2Goyi7NngOIBo4MP~7vHz+ z8IND{X;T)-CbrrblAZa-Yww$v&Z_WOE8j$Z-Cj#$Yk1&^Gl}u0S#rifW%We$>fZ;e zd5pBPM_^H5Hi}#~pKS6l#7Eg?G1Ba+h}!z1#jzPQU#>LE4{W=hjS_y_d?&f5hVH=w z<^ru4ZHx8$@yFxI@}+!vK@oB-7QS&rT~p3ffPkIkyG=&W|D*(JVq@!-|O?N=&V0N3LKFL64z z{yVg3*Kqk>6j%Dc3iq(%!kw6)>&@Bv)jpyvuqJ&%LiGj7uCu7~OsDL3;i;5)Smz0! zi4Y0%tQnyN9{24C*t@4QKJifF5dX4Yq=6l>^Ln_uyVfG*A-(iljT}qRai0N`R$zaL zEJQBoRCH2O1_xio0dbr=lD#``KDt!1v!>k3kvFxzkqTPvP-0qepjwWz&G;u_E+3?; z81$;YUrHNA$-G0PtA4Cu^>vF>uxykSBtGnxb@;B=7Ow zT|>fqn$zGRD|jj?DDfdma*NZ?T?>WYyN*%Q~S6Nou~4DBP>;4XE+ij0ZJKWEU@ z?^4atbAf*tMO#neVp+j1eV}LWr zZi2n_sY*O_>0SrhRv17qY9urEWR<1sd}Bi#ON0@soYb0O?^dy8`dDH;6o!mI{~dW^!G7*cio5^uI#b-3OuNm=AMW)jx(crkzY zzuI5BPp)TP65fxq=ERa+<4C=@`n6eyykbL-U3>DoX+bOZPM=`{IlI3g-D>+5eeKpz zW-Q=Cy^Pe`%e0YzqgcG!X50Hdg>H;U3k%Qg%QvrNQO@;+nBqqG%H)#w$Wx3}PL5d( z5Jjd$KK2c;2TNw38@R21tySrvBQLMEBMpTNdc{>Ma=0EzF6zVXLnbzk^dDHo6 zfVbP9ncmIv1?Pl2q!~+k$`ed6Nkcc-^CVm<#a73QSCfQprbdsc-c$_}pTudOw0Uf~ z#!}ciG`{dm`|1nasx#JF8(bI6_~vii*YSF5$Qbv1uH9I4o@vcsBuRy=x$uQov*$r| zxtF3*RhR9}_;2SGyy0?cCY^9ptfK6_(n^dK-Pf_}6D`UvTYUImqvjB%pND4-DfjOd zVKX3M(3c6(%bGD|1Qs#)N`S*koTGV~=gqj`EXdhs*V1TxmWjP{(za!HOBeY>PI@fj>9{l4s3Br}FVFs}R)xn-hRa|CA}lk4n_KEHk$H{uNRbbv#@oyR56&%h# z7+@8@J{6pHrimm1Q5l|Q^2z4XhO~FauI~*>hb>9W+U0gO)AMvKLiAh}rDhQ@hM!ro z?4BA4ax3IDQ{OoqHfG!Na#J-T_7dDTN*u?Bs}?T@^wxR|Tzo)O(CV46`k6f`-16y; z6{$+zhLAAry{?AhI+I|}OGAhDiK(fVWrq5n^mW#z<99l1?TkI~n(lL!nb=8tPMO|6 z$rPiv|52^^Gr`T=4@;#>XxGUmtMO+Nd+|xdk~m1 zEIzbmHOtGJSi&&V9x|$Bk_=>vGRepNF&w(ST;zBxKNTRc6g-CCYThu0?}Wrx31H9e zyib?7tkK`Ln&`OQv#p#Z!I#|jBTblE^Jxu`F(DCNM1$2e@Fwj#=VoxRQIag7)-NjD zjjpv(NOj2W1*H1^$nGHYCckc5fK%G{mL%OToE0G-nI!% zng`42P+S6>@B3rscf6M9x^nXO+VUR9*Li+5zF4v_h8Ds`jqyg!pU0I+uK$1^@owc( z-5b?qtK86SIcZrb-u*<8A|j*Y)M7tf)`@myRxis}CbBNiw{Ykfm>?SiUpV+!{Z;Qm z-~*CJ7RZAFGJKlRQk1S}GF0hQwCdQaITS8O*hOdO< zK)#ds;7oLQ8}(C&P({Q$mnAz&!@V zv@lp)Tzp$sQxo={_*B{#o+7o5r0px0-66%;N>Dz{A~>{scDuQuWnpHF=>yL*<$z&_ zVMZH^Z`xdmf)naJ8cU*ghUf6FAQDGOvg{eBs|nbIR)RmIKhsg#?33oI*%>X%CAv7} zmo84{=;(SyV7gkV-6_YJL-JlRT;Rp_bYfPk znT@CIC7p`8{HnFVGN#0|L?gJhNM$EiVV~~X%9*^{p>Efq$q6apXEWQjaFn3f8r62+ zWoNGZ?NHW0_6qe2!CX0tZwkaT-TiP%ujB+NmvY4*Co>ornBoX?*ly(nmi8zKqI(Ws z+@qRou*p78K{=6Av}eqBXlS&V`vxPZ^f6+rA9+Q%V`307IQ!F=1}Ae6u1r{#fAD(-GrG#{_MNKQ zXWMb6+c+vwwqT^ra;gL=S2xL+yA48@)+2`0!AZ37%fJBg?z!wu8rE;Adp>2aY28E@ zw$yT2%#6~KRXuv?n8ycfMs5x?8y9sE>voc?6o|~fboJ%mcg6pNl;^>ZDmUJf(@+(J zWfTrMhYs#0?^fAPYMpg>m9Qg08B%tc?y5>`=pkp+iz2R+yC}ovOApa=!jb&1sU&sA z;|a*rQJuu1fv6k}bVt-9->WmJPihS>{c^2)yUV{wpzh7TNZ)!8iR8zTQSmk}Si0w{ z;>PDjbT>J9dfDpEXKLKnU_arU2*R?2zRum{YVvMrri7zN{2K|rYnmwq48Vu^pom}A z1dqRa-*K#chTz9J;;Y5c-uPH}5kj0O#Q&k{D}bW@zUTo#K&25>L>eTGRgjR5r5gze z6<8XiVd)N~Yh7UJM!HK%X{1X^x}=fj{}#W$|NQ?mi~}>y#^>F4@44rk``$UW!L|+e zP%o+`sYHmtGb^l%P-w}%@60W$@p0!9nvR~O{YoJ+O19jNo$Ck&*BFq~`rBccZnyAq zy)(Ou1AX`4GuM<#{#fi>j!Ldf`b=%%WMRVE+Wka`YOYEyG*?aKDyh?u5ulRMw@OsI z)egf#rOGvLDbDFKht{|c-Q%a;H? zK8mP>6Ox+cW7Y83evfktmqp5Oms=4#cd)tjCyx8SF3q_IdD1T0-10i56ZWB9e4Rg& zEv}G6iY*iGwvfgx((@D>YC@^$X|!MzlBwf!yqpe8UVlKHtJ+jQ6R5fLrMqd3m&^52 z5*8ij+kvH-i`D)Wb<97M1vN^48a4#S)-UL35JU+BG4;vK=X~(Pm8~DH!4Jt}Qp>oi zg5*}u;(7FzPw$Jp)_dzf^^{WZ{A)DUOZ@j0`%kpYS4K$s8XYzcsnd-dqIr})h1kYYU!wDRF(OrlP{AfLof%c(F?OHP zdmsGN(h0-I2soFSu8Uz18z_e8{6uYxjsO??ac~tJ20R;E3iBKhYOK8inTHQw_Piav zax)gxFTOx{M1GdZv_%S$k<22{F9;;I$HKRM;0@0Be|VTSeCt+3+6!%Zvk#Ai6gDbO zaKG8Vp`{RQqZel!^vDG3|;>J>%_sS3xVzxDnv-d9h& zy$%|05|Vp7a2{4HMmb7xLf7Nwz%4}CSZRYmJ14QOz3sC_#ta?*X0@-|N+p%sS?a#4MC>{qo#_2q65107c8epz_qlF0DWhIZ zm;Ee@HSx*uKa#Iv)w0-`W#BLvVuA$VPD?C_8D+HH zCF8spI|pwgKm|cFE)dJca4p>&LtT$yR&5e`CR0ylE388q6U9a)HR~U-LXe?A5(AQd z`6F5eR4Wy! z5vo$l*T_^VUzf>LjgqNd8S3xr>ziy(@X0QDMC=%L9oPo6*x!3+ah&MN{gPe@s2WQ~~HqRRB@t|)$5N{XvC^NH4up%tl8#_Q2 zJmC2~V@t*q{BJXBkZ7w*IdRz73Z$V;YzIbP8o8&MrPB3gEpD!VIG&M?Xzw&LzN(@R z_Qi)K>=F8v#A5-jNT=eRtQ#ny3H-vqDO?Lq=QLXCiamP8p}#A6vRmjR?)Wx^xnRV6 zFo$QDfa17hU9K9_-pWcr;+0HWA%W|84b!67Wv-@NmfcpB`|`e>N)gZShY5?y!cUT7 z87iEzH7@y&Cw-vONh*y>#S9RKkb>b{jz-0Qpt}*9d0xJI_$!pvuMZM$JP}^N~ zN{3G&`%47zoqw(CwX(A9_M*>t{2#TwygKhqo^t-xNwaz~kt2xL+#BlYT#~&^6g}n* zZ*)@_&c-6{clJZNm*^@6j?d(YjSa~7hy+#!tFexkTfTe=t92oS&|D8 zOGfxg0U*NNM)J0aU4jjrfvxwd^c#KpRB4Fy*_f9|t{5=69Fw&6zurF5R0vFPFwQ9$ zjAJp5^IK%eiZN0k_YPq186J^@ad;Z{fK2PQ18bpKt8XTY=LEYg?Jh!4xf?YR618|# zPxZ{=&qjw_5T*L_TVl^!xBar<9&}wejX97 zKw_H)7u=Y+xCSqK#9PkXnD`2j9dMi zugiv$zZLzq-SQ#ts$LPd%(gynZs zZ+RdXwx>(UAU~Kx?Xcpb%(z+@-IjssiA-;}9;Z=OO(uQ;l z)`Ow4{F`Rae&oUu>7{7w{8q|>nC)5ySlCJi^IB)v(CF^#`h5xG=_IQz z?k^eznsi7WYeGF?O5Ts`+BGU44`K9t%GpT1hP*+N;No0Vui<1VLOEjR=6TE{O3Ut8 zeZF7rv8QpVuTU9VJAhtvTYs>q%fQn_@0%cuEnUBxq4NK#=PCmH*}r;Bi+?8(Y^WPj zJ-3*HFgA5_H$4Q&JeuWdtM(Cm^d#qqn$2_Y+btC(5Ery30N*Q)9>9(!|KLAxb&Lf? zbP-MXY-0?}9LBTA?lTKx4hF%bGJTM|vW8XIRVT{m|= z2j_NAIwtakSo#G{3PL;%4v~#){EfEKt0L!n%~Gyb3p3quN9Pu5G;XKCFhfV=;aSzx zi`dHAlmB@EBkLA{uV{2G&ua@G*s zO&z*{zxt=fE;O+LPzQUC-E5yu8nB_($8{GhX6}D0&c1&99PmP#p7B;h%Q3zY5^7+J ze@E8KuQ&8PuBY)^2{r3ko9!3Rca-U|NJ+8f#*dwtD=InO0CLT^?7qI|^4qdLy~$s> zif*XuLO@%KZL*!l?`XmMFHCJf25r`Q#)ALJ2c7d$`uKiC3fH{FU+iyd^x(n2lygr7 z^^BOP<^&R&`KJ%oQnc;x|B74;_-^8lg&!>4+4P(yrcAOKmJ+8?`{huji|r|^S0iq$pBxKIc9VSMv8t^8 z%h0 zzl*-Vt9CyFK|4N&oZ?Ch4ftiO-UqxD1%VPe8Z>itU5|H0eurm z1|OOiYYYA2RC4GpdROz;4lE9bB-;KG+Ci`sDt%#~knQ!7HaT3S+8xt`5x{1~SOv zE6@&NG1~8E){IKp?pQtHVDb1iVy4d2C(X3NKIR)oI8P*a0M&fKOu9u+Hyqtx#r(O@k5SMgEcpzUE=$klaXaV&iykiI>NaRc*S;90Tc_=m_pb(yWRNmk^GS`NQ&l_8JWZAW)hNoG2WBQ?cyVIhIxH|- zSukOrdcJ;=y(zHLaG|<4ndBHMcQ&B}u;K~!IX1M|X`J{+g-EW~QQzl}sY>NMEhYE2 zm-}Dv3{CGje@=4Q=e9aIID)lgo1H*D=I6Zev~k7|RADS1+Sn0J)r>J4LgAxWE)eJT z=;S4A4;a>g840JO(xvOAPQ=)v-qPKfXh~FN0gE#%6|tp{hiSQOBS<#j?<#-d)_eq8 zCT5*pa?5fNiPU*iRND(B3ld*k>QC%mnL0=4T{JJrWeJg4ev~~CMgQMY0o26CuQ@b> zJ5T#`4U2KI&hO`p|3cZgB|9jP!gAE#zK&*S=d(3OXhbXw_=}$aLR!6oVNi){#iK6k z{!6TZ;H+%l=@u=v3Xcfa3X&=>3M5GaF&%zn)OnOukv>1IijO9~^t9I(LJxLfuT%qy ze=&>++VrwH!WJ~{`4&v&lSU`$dWz5PdS?)%+kg~x3oLwRvY!JX!>+~y<@K(Aby;r3 z8*Q(H_=zc-eLK5kT->ERre_@k4Zl2z#`Px zVl#h{-*KrRaref%h0hb>7;>*@YGaJhVqL7j5BnLJxcvXZlvWrH`%Jc_`(l}0M;g5w z8^G5}Z`g47vs1H5mV&{0`k~_yHcQGDHJ7sODTI=AX=TY0&|Z*7D_p%fBs&Z@4^b4M zOjD?}5y=A_`o~-S2ek*f(Suuu=D6nwz9@ln1cED#8;6Eu`I_sPCr)^QNxGHvvdCmR zVE^QSAtyPl%g^?3A#q-xogBR?t6_WD2@8$iyMKu8&C42E^-UHlW~!|Bvx`)OwSI(J zMMnKfhM;yyY46HP1@sHu3AdffxQjaHCF+oAX%?l#ekR10RVYG+wTSAHnS6?zGQ< zT#WnSF(|q1cM%@F#J@d;zouEQh&iZvhk;)NE0WpkaJtfRDrTnxBqL;->mlqnw-V#8 zL4=Hp6k!i#h~(nma>g6t4m_yc$;BV-Qb93(*qCZCPvd-QJ);&B@6^H2_hB5RW^C2S zWDW~~YdDhNwbWlQFmwOIy|#eC842gXvNf^V-LUs=Z2H^sz+$R=ZLPOiX>BJp=Xp^xbd$IRmU;3T# zBZ%E~&meSO>x=z&RG3GdrViTwR%4hkkNY1*zo;621Jb}MaRN42qv`V#D4a5>Kj)1s ztFUbZD@EvfV4+S+>hp(^vTbBOcw*2b!gv>N9tJ80(z^(k54+X0_Qu)RAIZcOJO)B; z`HvE<&_*T_6ROdS?spe$NK%oM;9DBdUoQ$CCFUgm_U=7DG~|4{k17NWQo=*5rT1Ev z+RjDWS;GMmHEPOQQMnn6s=;AiiCBy=ABTuM^RS`Ojg55&f4Z0AuTFw`pT_reU1z(+ zW18hGjNApO!21Ay-xS9n@n)0vWIlH{_R;5>0n$&-Qc|0k$EDmu8_AZ!FCJ<1kXxji zjN*z(q;2w(sI|K<@!i`c2~;AG&=T*|c1@2Hhv~VOYhD9`$MQ zg9J9c17kG5j%Rcw+h6WFoHkf~C`&*I0D8;IM;5(vcnOz1DI0ZqX>rKN&wRllJ@r6Z zI2dLX`30JJy+$cb$ z?p*g+n6$(b4-3iK)qIs4($0vGgM$9rxxj&}l47Fm{j<)cS|KP-BpIP6g9BemfWUN! zt1*(zshp^hd4^Lf({w$*DsHG{0`Z#!;|Q^ZdaqC0W_iod}+o8Yf%D0c|`Oz zDoLTiE?@9H>f^VaPeJdQi-cy5YrNt^@^ni#SBVsSbo`BU@#kN~rIrak_}8RWaD8fL zM9Ml9bg&A`WU1^;Z(lu*z%cza=5!*Xr*PE_%TO|<_7ZXnuj=HFe>56tJ`7RvB0nTC zJd>u?#+HM_tlqa);m;;5^?#~5fmC$d+{T>{b|P&~wSE|&+So>j#kjbpF`RDAaA>Zd zs}^SqnlrH*N#}@HDOrHQa+gHxz|!g&FAxK#Pm3$ShWQ|$GL4!bOEy||4DxKLF_RnvZoUh?O|w~9u#z+UhZ2r%Jl0tY4FiLLfi0$XX=>0ga>r4 zzir+coLqeU_WOI4T2p^lM+A>mfqWak;U=gmm9jm&qEh1`2#HJ)g0!G+a{~fQ*zgxN zwA_gvXurFi$xuwv#NBbF@8LbJ``_iL2eMTPp}94Y$-PUH+-tq8${cqywM*!kaDpE# zCMs0>*J+k>e;Zn}|3TxEX)DdPl%@GNn&f=$NLJg6T(OM_`SHzS`o(JCCEYetM&1~e z%41EQh`gh`8#l=*q_voG(l_b zn-gGDrKD3C*&?_HA}5M$b2&yr?#hb@ZH!rGW3Wa-yZ%~TmXOkZbZ4>sFAKhBNQ`+5 z1&rdwl+PA}w+We7gKwd)5SfQOqx%hJ?&6B4c-|eovN1h+;9Q!_wf;M#uREmg98wg^`p&lxQ-u=mF0Egf@+ z(2IFPoPy*5mdFKSXlMqIouPp2q#f}6b&l8}p*I>dmCiix=Wp50xj=|ugJ+8EY0S0p zCtLiiLhsz7n+QRN$Lj~ zPbeDAQ^_CQ{0y@bD4i&%y-V&wVu^=erTcp)LFYNcy?0^)Ga|%`9IXYohiM*7en$&8 zRWsDsGB4{@3iaYeA==8>^~Q#ongYy-7lcb%k;zI5HE&gV?DxoC_}h4tNrU%he5wpO zgnG+`YdN3nP}vX$2;-Y_-%6c`sYfz23hTaZKc6T!aQCebvho8WBV(-NtmZfGb=v{H zj<~)I?2SiFi{!MxuGLAzZK;99_;p+SR~l`D)2vgcNL`c5x3UlA?LauwzAQ+>{{9)m zLi6rrkC{4t_(8j{u@K3C)ET70Ia{MphHtBp8+;`r$XXcXLtaE0!RbwfUvi)2otwZT zY;dDCJ|v?7Nn93A`>@IQN9_yA@%PtOin&KfjOWu2w4LaM&JjTV1gIo_>{9`@ zCv)GIFDnq)%s~OqtoRgDdxoAbX!XqCDRvsOa9YE@AW9m}lh6;J>y`w9oFnDU!ZT%;{Xx8cTYkSX&kT;z* z*8K1SmZvz-53lPtAL8HnOc@be?r4j{qKAoJc?DE!Nklt`8 zO3`{8FjDH_rrQ?HQHp8|i#+YhCh_|AOseCx1_L?})EusFXE_ZicC|=NKp#K~6TgRlFonVwTKFWT zMhkHzKKIvC9=dq7_E^H>FafaCGIc1t4sxkfU6A*+!Q0A}G`21&U!&Nktk^AVU=_^^ zw~JxHAV098x(qHpm1C@Zt@{(Dj!RRJhRH-K?b15C#qvhqE?us_r~Gh+d~xmvBQ|fa zvncf)u>|>FiIyQ=A(?5y27WJwIJ!ep{HQAP&Fos`M6kuJT2uWMoUL75wQanf7gv>i&;>W(dScjiLw6RMhBI!oF!!I}=v=5Ri0WFO&x`>Z9do7<43!^*Fh`IG}jpyyrHY*FsTD=&A4&~`ygv?X}eL=w)2;! z{5HWxf5jOYl2ps-aet=EDTu4 z;(!n}vZMY5x$OU5c{so=l{VCvqbU_&Q+6ObEWS`B#*C!;;J!n(J#Qe);+fjm`{1%? zEwPH=@f$WQWZ~vZdj6LT4 zPFX)v8a*Q1;5WbeYBI&o$;}og(=|gkFcW<0E)1tUQoJ!G1=3(2h?rke@f&p?lMgqi zit&N!YVo_E|MRz$3q+1xU6wWZWsl4VA8Kv6|A3V3t{6#zl-adtAZGIFGLyh0?hu)F$5_;0{FaF~O4h&H?eG<6&F3Sr*PD z{QI=N3rUB0W=G}SWB3UKTzHjANZ(x(8f&XT{tPUN{{RXH7|Pt=B!wbp97>I6Lx}53Moz49G=XCp)O~T zq)OZ)1*vXdd2h>sPJ#LYly3t&xh$LP&kO#^y2~(#EHKK^dP}ni!{59yfAe*l`6Bw; zB~d7#16%2PT@$ebmipV;$)*do69tE983IE&Ck+ccd+RH{*53 zjm8RXU0T~TA;{yG%iu-ty}?Q+ryZEl+5m6#@xq>`$3{;3hpCkVavRH zWNE(o#J1OxjMt*$xU3YBZHQAbZ&H6hYnNs<6Dj`TEU@`}UHq_{z(^(?{9_?c?-yH0 zOkc*x#&ias)p}ytIw7fq^z8_~rKIkaEis4bM;(8%E_!7DGVSkOmFA9oZfG=%ACR{!tlR4TvBhUJSU*&lc@mt1sSpwKlVh(wEX)C@DO(Qux z84*?7>?-%gyFTY%_4R9SaKl2{vxgr%eH5Pin^>f?v_zW8YEK5u1WvmKZZ`ZCT#r;R z=N`#&o2fmgqliva6c2h_v~A^K82xK+T+!kayvZy#(Q%G-Ao*L^C_oDut(|=l!ut5m zClCiF%~`rGBKofk^WQ4$_H||}*X%D_(akL@U~})@$6q@_UDR)tW{ovn9?E!O^TSRc z@3sp%L~8pKu;^LHk0C3|djJKHu-4C+Vya{ZftrJw233&HhU3$}ZiF4H8oR|3e32Pm z%kZ$c6@i{R@W4Z2QoriS8B_1CWvS&(x&+h^fEYoOW-kI-WuNv#UCQVHE$I|RG$bU0 zQji6GdwdS8ICes)+%c!|6;;aql|pyH0s{Chtgo-K59Sz>mVpB`K5*gj0+7rK2kL#( z?^+Kx5XHNr<0TG<$#YEB;h%UB6eB@Myz>>I5gg*6#W+1&2=8W5yprO7-ZoP-^``Iq zU_$0$Ot0}&u}H68p(f8eeg;XCkEDdj?{)(+h0XmONHBMis!@LHY%k;a#{jBj!-7=XS8!*+nsVDEDE;|nECbk zmnMM`YR4;-Eh4cBZU!E~2QfdtgEG`C-L}Wyx)D#xP#mpF3&gL}Ayi*_E|FEb!i5D- z$G*SMR=$7uAmCJb-8J3EXWo`2Svj~+3vOrW{|E$QbNYb8dxAe?HZL!96 zRW{g3vJW0%g2Zv-g+xAUg(qxeMGknlP(%fh=s@}&ivdWd(zmmpH}1fiTI7J@`ZT=n zxWbbBQckta_mH};y}EGw+$~2$vSoA1zVFIyxzZ9CJ_}x6aOv;W3a~-p34#A2QUa|S zPV$Go-l7)rzs*FB|xVZ{rvYVt7ct7Rg`4@0@` zk1D|yX~aVTC-{7=obl6h;1tXo`V^+d6_4WyRcMaF*PWHz`cyz}Z*c9a80DPu;L%5J z@%XD)9df9V-!0$j0mTr6!cQv9aelX})}y&5hhbmaOpc|j2D?eNGG_3pBxBGCqrqLH z<$QQZQGmg&Y?QOU?oVYIPy@7V?21lStn;2bj+^50F7Bw`<_gF86!2}ozk#;OvQ}5U zalP2r5LgXMsP=Zm!Y8j9#v0*aS>9p=ce%f67s@nf$GeI#BFn_ZUqN+QwSLnzLB)b3 zGoF}t^p?|feBaO;2q|vNQ!i8+AZE$s>sHT=DZKFue6e8h+p#^c=?EA6h?(Nq%eS+< zUerz(!InjD?RsLx68(LP%vw+Fwq&xh#79w0jythOxdH!EZFMJItgm9~9kZuT)%Zf` zygTWBh6)X3_=*klbrtSs$<=?g{%e0KUcyxXTz!1+e^NXBVwGyz(9Jt8^gJo!&~JRp zZ9OAk{q&&O5|$C1l7925@Mz>zf?M>=p`2-du3G2y?bnQt{@+In+Rlo<+Xr+UL(GPQ zxh@0d4wl8ZlQX;w(}<%MEy+~P@~xy9NE>J7IFGpPt++&P3T2q-wkwsL;5FilhI$=w zExY{uvQZ~}E?(UKju!CT_axt)PZBeGGP`|pbziCX>SQePm+6DyfQrpo zM7z5~SQ%@M^Fj@`jbtf*o!|KD7RvTvO%UFZ`9{BfZch_rEwoa(NJhh_tNtgE%JtLk z%-6atdbBI7F=$M3Xy~Lu`{-*PJ^e6Use@tX^7H3+kt&fvvcYp7pV;eyGYIix*i{NODb-C%B2fmN9Oc0LmZ`;pt;+Wxx~xhMD`Sa$ zW}HGkeuF>qBPA`u?gr*(Sm})UVX>Qr13yp?rhK=d0X&#BRAi*~QPJ@qJ2fo7N7YJAR z;G8PGOS%0$S9Fa79|KrN+FS<_J>6+@Jb-7UfkYUAav9+zEp%%;N%`NfX( zD&NdvGm_ID3VI-Urn!)wCewdB(RS5*r`d|^md5(|=PpONFr)afO?n#sjp5|39WvTf z;rgrC&p9hRrR&92UD^FYW!bJRN5rF!Bc%>_lk8~cL>B4#R?FKG)9ozA=rXQA8|HET zMX{Dsf`OX8Y;`z5dO15(J;2pC2wkZ(~^mghGm^AGJ?9-+G7x3G4=|s98e5t9n zjGyx~e0;1k!4cTdDq_ir;kw&&Db(%weIcs1(xME&TjrIRzok|XlxP4`#A zBiZK&)+rParqPmERfwa`ll^}P=uR>Qp(D;j(@-TYaGJWjJ2ABH)?Pk(` zN<6r749%&zNc5#QP@SI}5P&cYD;FdDpBI4fDHmwAv>7@Nr>lF+jy^oZE6}M@7%kGb zc3Mib>x%>`@VVv-2&Y*+N5G&odN)dT;Mk2$3}^y9+{4)F`#rdTK{&yuFY-x=c;*MW#PW;uDlI(jez8xaf?gY6? zk8^D4PMZr!W6jjp!(4l8-|r?YmNrmlv@Cn3j`PJVjS80Eh)lA{fB4UUFYJu>?NT`A z791Bd7tv!P;Y{s#^N8z_q9)dmt8T+``RfjT$WW=%D?>>*W%9G_N+!_0%W~A8Jkcyu zwwgMC@a{EiZ)&Cd9;thJJA(KS-<0lmvPby3JkPms^2s5jbg2(+Xf zNw(MVvbKO`gJtHi-KQ2IR)J#tMIZAMY z+53{@UVsppBcZv(F<;W9T(;7W`stzk_$;DBMUdLq-kgv?OxgG9!-dFN-ZnhiihHkq z0^uwvek01SBzS?C9yIc}E?69ZwGHm501m?0QwvFUQTBsN>PvO3q2){Jv?;pzmkpk< zM?Xu?5u+l{vKxlrp0)8t416e5gWV`#;Kp5A2GP76aG{!P`-Ibu{j+-9i2}WyZw7y( zk<>IQx8G0OqwP}+`>6j;pKQVbUJEKXr@jH$xZWCU#}6;<^yV}*3^&8jO(n3$G7zf) z_)eL~zNKrU(KEsC-FPb^MIaFRL-6V7r%B&sL>k?>wW@j;vag4LQ1x>Tsx=^R=I8e< zlP)c_{LadHCKrfcvwM zskwlY=UB$M;)@OeO&`=EF~??S(8(Zd;UU0;lV?J zx6WS0Tc1F-WD|emv&PDFQQoG>6x_KOlznpBEng4cg@iiS_!vTIi)7iP=2Exahvnr! zt0iJCdx|~pi~|q^?ymsi^Z6&^#W(iTrno(i@y454lM@w;%keG}Sv8;k-99ho+Ym?g zh#6F_vQos`)>r@@Y1jdL@=>R9p!1YWNT4Ir1_*UX>#;fnh;j&7HB2 z><&O5=#$;_y=DxozTZ3I(zg*p;d?T3}0F#(bYSyZ z)@=&f`JI;!FN||!VPZC|OfmE0bKFCf6TA*V1Op*GQ?E-fpT}1XFIVFbP!V(lp7((^ zv%}Zd9GaO{(V5(9$e3KLH)Lw{RV$ms5^Ed5qXx6%4gT?d{ioB+_?*<)NNQ#E-CXIv zxFoD0y?l^~g2^_0XbWzWM!vR1)_bho$mnU&u9Y%0k`Tt4Cd+A_$ql+wQF&wBSi0CL zNNwf3Z?qD3fT{s#fprzoZw&lD^usmV`cbztNSo8@tIMMf5}v6C%*sqb^pxW<&Z8(K0sL(=x}J7zYqv~ zd3TSlsj4AVTEMh0zpFjulu|jgnEQG3*P&1Let*qJY8$wE7N4`NV~H#HXF+edfTHP; z>sPa!r=F+6_Hv^q3x_9{7jTME9>#-XfFkAGgz$D`vkE_8)QzKd#7aMcnF;RDnR#X% zasG}|Jf`l=aL+N$V5qmAG2nh}c>;mKV3tjJl}^P*aYAKFAKKbw+q3Sy3gtWd-CsW1 zEwd=@X*@1;(*kg8->`|51_D9zdxm~E2cY*~+PdI}Xah^%$+84-v6kx<;YmXjvOz3H z5M(EZG&$-u=oG)Z$U8!KK<@>ALpT)9gBgv`MN7Y!G(c!equ;c3{}GOl{|OgJzu!MR z#2pCzcm(?rKh8hq)M*G@otwI;E`ns{tscjaW_6TqrjQ*mB;E*ZLlA~xWnr)~KdZUC zaWy=E;(h$Og%(Yg^Y!B_2T^!`#er5+lL@UBo6g@|SCP6Y!%S+)qt8D;eoMk3fvPiC z&eLV8z?FJ{rkKRruNPN0d*&62J$Q2NC%(|F!{#mV=d6hG!vclsNHW~@h@K#lWjkZP zTNaZWj&t7?stg5@a!gD~w*8caxy_^3rU*l{sP~PWyS0BN*TnTz#|U|Itui}BD4TZF z(9z?m(Q-&uug|giSgwE3x((O>hAd1t&`9c97Xp+xX4L&c7^r?CeLne5nl!#P{N__{ z7oB{yi3k0kLH=fN)`tn=A31oHBekTf)lBXrzDz!cjTWhIIr1Ioqae0Q9;2kczDMIr zXBSSD(cK($XqnNf7&AZM+V{bS9l+iy|FTO2!X^gPfb->aE0Bn+}~zgY3H8(MDMwK>KAN!Q4=cjQ>EV9 zNP9n*0u~3YAW`qJY0~78!1i|=J*x*7G?;PF>oyxqfr0rkQfi0ErdAQohl_hM#NXF1 z_+jPrL&ip59B;m-UAd$lby9$5TM}(1d1zE&(lc0Y_wdOhjU)}z8mPV(yr;ef`GO!) z_J8s0~@~p%lbp=AE7Thd~h-cM~pWVzj8V{-2W{M)5?C z1HM@kAENE&m`*)3G-N3cbcr<+GCtpnM3Pidslso@LGc*){Zo;#XupB&0=MEnN$(yU ztC(u!3*areS20DTq@m0{Q~Va^fbNOuET7_k$e7Qwl~hU7uA*FVqm#Uw57+yMi*G0j zD`WMOdUl*yYIg}sgMOIM(zwEe(SFv{(!8g;CUB77X4{TRitE_pp-#C>T>JKVHcd~v zw*Qgs+6ZMo{OXJ6W!B+kqwMQ+ViAsm4lxbGlm&%LFY`0ws@^X7u`A(Zmpv^6nCR## zJMZmY^1AUio?5@A&13mwMK6L9b@gQ8ytOxv28v7L7{T#uuK-MpRl*Khb8hro)O-ll zeYE~<9};T(cFJLcv#JyL)0NR@(%R4kwiHBGj(_)K`6` zJN*mhsJmDM?Q6}l|GuJla|4Y3969r*iz#p3=sj( zs4s&l*~Bu04vUnY4lxiZlUZtp8>L!oIGX7%qd9Ao4YBNf3Kn8j1}wx|4(5qO2Wh{s zghdwN=li_rylbVb!s{=nw6|5g$P-73?NqLn5$go(md*Zy-ziV@e82?%0k`$v;ep>< zv$`Pa%Zcu8h*OKSD0=y=G^FX=u`AG|{3kzb0}9@lbLWzpi(u3ENOPomRKKC22gi}_H}%SB5>w@$95Y5I{G3N^lAbzKB}%5k3iBmu2)FTY z-s+)Jcg7S^9m1VEUFX%trauAdziABt%pf1+l`;F5Ob8*GSVwblQeR8AS7WiT^qil6W%6tKjFaC(r!HKzx}MT{&02hm^YsJ- zB2KkmK!T9hZKR&QcU48I=?bcK(L59jmC1(mpT%t0IeR8xF*cQqTueD1QL2@-(hKKl zn3-APVk*Jll+pH;RY6}ZBYhu|1h6gTHAwmf;hC2^Hmk_!n#{*_ErqCc$31-`l)qmg z!6(DSgfN?s+vnx98B9ZK%J)gIjF1C~i4MQrZ$u(qW~DfDqQk z3+nVPM)7GQSx|@&WFDxp&6eX_j9a%~ZY727c|j$N)fo19-;QJJX9;F}?x67C@%}Qv zX@e)QntISk^yv0=B_+j^!#GmVWh4293cjeb7=`P17Y$ZIZ-oW97QX?ZH$}8LTY)WW z>VWXqX|V%ToTZ{p21AACScs&#U^b3Ixs!J`0pz--E}}gl8=}Ak+99vcoq&Ccp4e^t zeMk4*0Zg0nC&OAR@NWJ7^yy8D#9!lby;F!7PvHbDI>jbPr9?NJbr8UUl!YN2AAv{7$n?mi>2CANed(d1! z*kf-%ZxPT7i`pa06#0KpoPgw4SPI7R41rNKB*6}xx6yfEufSDvD(jd*V{iNuixCfo zo|Yu%5udp}(W56#;(47Va+(W5Pvj>a_5N_B;M{%A zMv)oc*^kAWwNqR&P`^@pUj4H_cP;x<1ysHK9xD(D9pvN@4Wyxc?!L1nW;2jl>Q+)( zI=11!aKV8`Iy>5z%XdRMAx$2V+_SdGp57%(!E_2*5gWe_F)REdu7C<%fh<3vLK6|y zsNO?IVS>X1QylO&>Y_{sIIOV>x3~hI zMu(YUQz1@Z+Z zd3T8EZScDVesB-;)EV%}GUOWrAx70@JWPjY@X<8N?uw9Gc6-*egyU%Y`P%4`XO8Mr zoy-WSBAtlRM!6GT9?52s%L(0jy9i{+xcPEq-JzoKfImeNKGvH9x$a3n#i8BU(XdbiMeUy|XO zZHG0kGfGR#WMR{11h}2Gb;GTCL=W(Me>DLb&jos75QN_o>Xm^bxQ^k-kwxhAZvR{E zJ*Szk6XrC%w)I`Y7pHNR;Is^CMlra@?oZ#n(W*xoX}~$PKybG8WVU+wd*%CI^co3Z zGI0$L$$VVe+H_h>+$EbmREfbF09Eu5vdO>8dbf}O&NHA|o(a!>)~4R_DNDhYOBHTjI2KDYK|@ zhBrUhbtr--S=25CCa-qKhS1)69W_O_BNBH1KS%azvKPhi>IHwn)~}=OUuifF`pBHe zQheHxWBWNYK#X)tHQ1h|tBqXhFSkfW3Jr0JM&D2BFGY-IQe-vwn&3Oi-p(-CFAm<0 z1zGvNyP()VaTL(uClJ$ZNgiA+2si@=a7JM2{j$>kvL~*g^@|G4y&tDmd@JrE0;5iR zr(dU-6fUTl;v}z;ez6jk$mTSEk$?y(X?(RN<$wo>W!JBcbgq6Pe`enS>)+UZ@m_NT zX#$I|4;f7h!^7`KK&t*yt#ya)DO_41HeC=eO#U`IpP$3Ac3{GHz#-l9Jm3AXWQWo> ze^WPQ<;f27aQ*C_K#1HvovWPSeh6hJpJsTy=B8RL|014@Upq?7%db>fp(4yg*1;jP zQUPIaJX83qp?DX*Tp~piH+ykewbzl}wCOa!5snT0CSq{bKGmHJ-{eHkDp6QBy*~Mb zk7=RTI*L=`3l}+?o8tB3x_GD|vUiQ+Cd8rMDD48eh7=9zPDXKily^ zdUb|RgHe6ssf$TUzcJ8iz1q1OuEqW131YxUWi5s=YByQ1x-UmIu0ia@bhX23fpOsg zEw`x3d?iv8m2#v2;D%Uv#uMKi4hD*73ocH_a>l^x5%^3Esq)N#7oQH3sE7dKP(=Sz z_P|jAY)kpXLxL#bm~({4R??CG`g!r%tMRS)N6vGYS`I&TjLzPi^B7qtp~pEL?$uib zJ4{_5){-ykd>`t>$zQMWPx_yq8-@mah_&45&YX;gnE6WiN(^15_pa*(h6EvVG(P}d(%tJ75A0aa4 z)Nl-p4aHpL=}r9DP*?Xi?M4lRU_Qgy&swrr0^I~YO~&>a=f{g6zQ9wCi1>Oomw9^+ z_Ir>%sdj(?oFBCE(6H#HltD5^E;6s2kA#Hk+=_BTEsN_%?t32fjpLGFnk0qh!AduI z%N+jnH!&l<7NZ2oHHo0XKmK^@T_l8#XfguIh{I#f0Pb@PD)kdesu!0tJ_u2`_;Yj- zUuhLv)~C&>^XG)}Y-^zhOz!@~#lk(mxW&6Mgo?h)PPi#%%0bVDUCtTAK(7xG zReoYQw)84}UBo7zNWmPrM9z`gC|%;zP)u+J0Zw3dKc5Ps$WX!E<{@?a%i60O+c9X^7L~1RH3YjkLm+?5POoXNflwq z!uQS*0D7kPFAOrhZS>0kT*AH0q(=N-898ogyVW6tp8c<{=-JQwTA&~C(z|iRUbJA;;8S_s9%oE z2?BLAd^KWheXl9#U0ZjQfD#mQo1u2?wqp%+WHuHB3 zhBsiJ)(?~ZFpN3{Ip;@@Wy%#hX`!IiiM;3_6vi3==U|em)JC?viYCeE5 zN8$6Io{OSUZW@GUVsH+df>HQO_sWXIV_OYNwt)shORTp#-WG~ytD8!zBJ>Szn z-^FSXu18`z1-_OS3X}y`^>06(#4lwQJ~^y2i38?_pEtZOt~~rLe%V|fEAU#NS1L_X z*mJf{Vr0nu8N&h{X|hg><;Y=H12Q?evMixRbI7>DPw}I4c7g9?-wUfFE;Bm@6F95z zEg81@6NOi&;^^bV)Nh2lZSyaEfF3cbkppQwj?qUnBk+d%ZPxb>W7CexJ_RXh z#Uk!1A^yMX@g(|cd!QoAH@y%=nvZiGqVBKZZ6iff)%Z-MK3n3W-kbHVP!!^m1m8M| z%KPr7F&6sQewaQe+ork~VG}G4EPLyHhC^81F*DXMv61jRJIDos8bB zFQd0OT{UgeF1n>LRjxEkGfYVE&^CO9qQfLXd&^VNm+$JcFA&vtZUq)6tu4jt$B7P< zB03cz%lie;E4;s3_a5Z5Tm9kroMMXlbO88jwzDX^`&j zMoMxh5h*EYK~h?j4iO{-1f;u5V&K2Wc)xSb|DCnI<-1smS8$%^zIR@G?`wa+eZJ#U z+6QFNRfJ0wZ!=am__fT=Z0gbv?XTu*_vOo;{n)j5BLs}&^AT2Y!Wo;b*?j%2yVRS* zGftm>R~*00IQexcqhdu1A`wu(kY`5@-w~-#X4m>y9A!hlW)+>6gGtSwKR#PrC&yp@ zJ^r)GfXWJ!lT&?4kXYWlBkm+M>T zts7Q6W*v-sLw(gd5v9BlA=AELxBem3>N!ten~yye?~%>7ru&-BxF^tFd|olfK$et& znmx8;`D2_(K&tfCM1|$HhA6yg|E4z{{?TfGm1bXEd8_{yjd#8vnq|Xcnel(8cU88- zMW&kaJ?CXJk`~8m2TH-YrH_-HdH49UVs)f{o#aZIeVqqu8gYsqZwbfRc)H5Z^qXC6 zGm=gQu3tePRzo)N=O!HrZZ0m?u1x1r3cr5*12SuD<%g1}&KpZW;0g(UQROHZ_+Cmj zV&Sz?99_~RLwi`!@kd^I1;-kVpvfS|X{^yn@%@fr_Vru1;R zKrVrR;Dook9A~WVKm~8vi?SMB`_9VT&+2S1M#)Cml^<6SLDA{x#Q!butH1-|^;N$8 zh7_xzC6XEfFr^s}`R8rju0Q9xx>#wI0mB}LldrM*o>csTI$@A!G>?na0jiD}4jRC| zeX^Ntbs;RvakCqWeB)xb0UTgO3l}JOfrERVE^H?pg=jqHp~#`J@>wzKi|hbXXFuNH z%Ku_6=>`>>s9&spFbFEv-zxNq2w-zF-I}aH%`&fbqT|>G^c&fhPCObN~HE5#AU+;oY|kj&G?%f49%EkJ8Mueaifh@>YUI+Og6U3oDB_ zk!aFda%bC2%qbB8vd|E+14i&|e{O~{Jski|S{+T6JSxCM#lsFW(%zo82%V2OY_{!1 zek7E5{A`~$6M?QVhGb%H-d<@ShTs&CY>)MQE&E} zME>>yJh6M6bszXw;=7o~Z|AIb5+a^15?B?o>Zn}z(y^&kEuyZGYkYiPR7=8>mKYyv zD5goZ=!q)zecFOByz5|EFUtUn&(o%@)aBFN%YNS(5Fh0*)ZLexsvW>)13>0Mq$hrCyb<&7((ZLx zfEh`XVpj%)54eF?PEaS_B(D4k)6}_#&i%Yl$#1O(Qx)umH&DtDf(6}}$G6D$D?IR| z>L)<2dF-)0+$F*cX~1Aa05Os@PWmPVtbjcdrNk-g=@Hz%R4Cm6^8ou6`{MaMspFfB z$lTHr0`j$821Z$CR8_oyW$2$&cOjjU4<&-{6G1sZ75I+k%y6^dHOUju@ zp}0e+1dVs_PTQCjdA9V0W+rCpB_E2{*H~**?8PSaX@kmUH*+@O=KYT)KKX?p5lG{G zJDrVU=Gt*$D6b4Pxp@Bs$0LssY>nMnpHd#GP(LS3^W3o z4y1%3rO5G&$H>|+(S+p-LB>l3;fIG7QZAX5qsR~nq>6aZ4qEy>DTSwMnrO6j+&BN! z?QTC*`|0})6YVzCm%BQhudz6-*bqK zS)Y@?1efc1{RzMth_x>jHs>^mgi5!Di;^N6 z@=kV)x|$Lg4L-IQ)*Z7=(sUwc4}qp1iU%IQC9y`v+E8mC43U!&N3G3?tAm@Ol7GsB zV5;&>v*6sBat)8x4Y*V`vKFWRtt$i!NZMol#D?Q-q1L6G+CBNPY~v???8?Tsb9?fZJ_^#eva!R=-Q#)Vi98{Nc?k+2%rC@ln!|A0@ zU#qC~v_xFB<(_-x(jJCs;}!{S305g+NB$5Vn1;C~U3aOcS;`OpPA!81BzwQkBjO8+ z3V|x`RAVz&F0t2ypv22S#nGVzsB7=TTZJ)E%~3yG*NS>T7hMQz5-tF{{Qi*os@YwQ zm5uksmg^C`r_?bTkqvZR^qFM@jdbec&f}$*>9+L;21f>9Y8=3+H8l;DVWD?X+l^guP}xa25gsID!_u)hS*)7(Zp+EP)j@Anx(={hjfB)1Mw5H8zX?r0xGnH3CMPa^M&4!P!e29Jp z98UD`5AX4JNuZ+}>v%b-j+kV8@pEC-Pt@+CiQy6Ow!eEYBmXfaoUQBBS163Ujxa%v ze&pl&l3f*(JXKaoxc}*n`>`Y!S7l#`N1B$fgepHvY)t(9Jl2y7c7a1UUmed;uY(~S zMl<5aeeNtf+MwrDySM=-nx{u5aZ?E4pI_>F>RPw%;u}BxAW@N9cClvmjp|cHPP2hTAB}#_r~IKO~`Nq!wD{A6~w?K38lwk}3#5 zr|jd${QyyYgzFjZJw|HWv-vf3#ZL|yf^58hsgiU{WD_c|alzF<)(RX;|9xcxlN+&r zdA*BwNDSSEnw&Z}Xb7RBH5r&m@;B?J7g@+ik_S#@Ko57!Wa7bW8!cxY5@m$22Z@*mHXQWks-c7VIV z2zXO(XgP-AgVC0h*8GgWSl4GbFNPh z*=+yRUjwi_CmK!4by<1Z@^>}AB-qUQJl`6mcyrJA%ee<4zh_v+=n%~p-t?L56gWI@dm{h87ZJ)mCbI)xn6H@U?@K3G0H6u27>M5w~s4s zVVVx7da2x7Hn*U0*zvx8rwxM<7Ww0~7k53duW|Yn=DFW%*DQ+BFg0QYPp=`FZVImu zpnBe2qor4nLnM&9YKeL)^YrZGZTfgHyTf~hmVkwUb+;sskq6Sfpf{WaSM+DZcUQ#8 z(p>nxve}2P#k=A07UM)RjRSItu zT1jXi&55pQj&{Eq|LGg8otBc`@C)7T24(w(?Q`eF2h#n7oNhEQal?V?)9^IAy-{gU zf_0xBDE>Rx^mz0WKh5l)#W2Awo z)@1~=Uj3lq*oNdG=Qxeq!B|8_D=&XK{z@K-Rr|FM>lkv-e!w_PpL>yPK3Q8<)ABXE zt1-qmuP+zH>a|)_=&!)G5R)hpFrINOuL15H5?f#^iH=?YZc?Fgc1G8!)@L|Uy?I{` zHm5f?)0=KG084+Af8 zUG(w02g)=mwxwt9iK}{jA9ODjsU!Nt+QXge zb0RM~*zYCqmFZcL;Gzz*ioUz$Ge1^S_~y|qfydje7E#j|Reb&iB0q<4rE3Ij`LLpS zP*>)@QqFpZ|7eV`62c;qmlWf^o|l?>imw=a-0*}&uEuJosh2TXUe#){aV?e9sdM*J z8H!4shnQE)BN31{855O>SO&##$-yTcVcNAc?Q!acEROnEa8s$h53xwckH!l^a%Xey zNCoJN^y&gW;nVhTB`tOVbeRbF;ig*ayR>T$hWp*Q?1D+lF{R(^c5cQcUOV$|dv(JO zKS^trDoMw~YU7K@z8k^SOS8N0S9BrDOzik@UDS-pz{?Ex3x+BFhkdA@%5C2(#Ha!E zJ>?sV?WvXiVE9Ty!uQl0Tv-BT(|W*Epw0#ord@$ASrc6LBES2dzR&^J#}@QJ5ZtgS$cTti zv&rNhb%7Q&B1b?UgiODnBWS=IlgA-?Wq##AH;zy$oM^w$1hrtop=80Hn2ad{g=u9z zvi#8*4llSe`UV>BlACEV5i=6WbMo-ewR*0$)hfDCoOR7z6&76W6<@Kj+xhiuch=b> zGB2iQcPNkS-yvu@UZBmm?5gay?$pn2Pr1!rW;P+dc*L3+PB5zEP(_5pUwg-$SngO? zq&1|MHa3@NG6%=_eQR7>o596!{Qgl4)czf#ABpN@qcizKX!Qq$J?vG6{O_H1n7EIA z{VJe+xqNN$ePUb<4c;1G#{6x*J#1=HKeGihJ2SRv3tWNiqy^;2ibe=sQ=BWZ?`UJc ztnT|8>buiE<9>EKVtB5*a~M65e5@z!A*;%(xFI|;t9@oi8f3(4KLU|ZbvU2{E{7QNDF+a?-bwwl% z4Q$leR&;)y`Dz{6=={RafUa-xszX?9dW{`jj=C|EIGpk2{gjWbm}?M?>4 zEhjA-9@zp_YcBBukk9o<p?d*(6T)uN#sNuT0+v=>x zJ=M#&`W5k6b6z+aee#ET3ty$TEt?zeCxbuxEta^lv;~X!>2tvR?Tco4+QbA|j4F>B zZ$;G}cOR`HOfHUfbp<9}RNVbYUBFY>hg^lxy1V7&0cPRm- z%zZG}8y;LMpEA7VB6j}OMOx~@h#hgG5d}-;pbA}*H&a38VaBEm=~Ehs6?yt8J926a z@eihNPeI2qU|E@ezAEu9ACM6xRUmn8{--O8JsM4FZjfT}42(wM^;Z_T&mEt+t<>?X zb7ko-Ovx-WP`U&=ia3fK+K9BzI|&@+Qfi%hq`LMV`E(1+DqW2fET3|LF4XeVqLZEU z3KImIuQ@ZdkUE z%D6O97f4w2?*>+aRoDkcSi@<1d7MNXM>j%H2|{p38sbL?9auV0Eg2ncf1QP59Wr?7%d<@{x(cj|4Z8p8O5nkJ7-#dRCcOe{Qs6?bbKLbJHh9sbG{|^s6a|eXM*t5y3pAvR{i$o2AE8Zr*7F^1SO_ zfab+TYI;_B7UEUb{qW;=h!s>(!wcWNgHh@)11Hx2VOIY2R^_l0KhAv3P@dz-9+F*h#O4EY7i}NLHI*-wE_Od@_x;{d|PvlKU zpicnO|9>W<(z?Q5)TzFe%hyvm0wdYYI%+DmR3iJ89zCwAl44{h$o2qyaO4TH)X4QK zwbCt^O4dJ-W=6zi?2ZW1%-l+l5Xqu6-DA;}c-uO6UAxu3`Qp>mt;8E_w)%1l*TXR` z!#sY(T`+oXEC+Hf_t7p5n!D^D23)j_XVrJjZVw+n2uA4{%<9?U^KVTh3_Az{f3hWw z5;o<(Z8CyWHf+D}zc+Bgg(kCCR18Z3`#NsWfmapBik&@JSRJHjRe2is8zXF>`hnzk z*$9IkB!D5qL&ZS#l2KDO!_{w5-$s&X37#*{e#Bh{m><3X7qDM9;h<^YS_t`Yj`0q5 zsL?T7ri_fxt+YBv9}-mxTTK|yB6Kw3AX~h+bl}OCzE+ zkG_V=ha`yOk7haN*hUr$KW>djIIEY9E0`f;Cr&opOeqcNy$#{~>E zp<~GX|CJ0EI1Bg)ifIRffZTN7`S2+w{td1b%*!UE{rKFsGcLY-Vvc3M_()`VCHx@! zlkfWJa{ z(u#u3Zr#|**nvdMisqhQdTkDcn{`~C_de)qGJM;#AKAvLuD$;a#M4%rVb$z+$vgtg z4WaMwewT*33BEw`7*iA8w=b@nu-!|&M^%Y8;Ye@?HRvticpw$CRWH&+3;)`rDn9Lh zH-+eyQOQ6|8yz5p?s`E7Vt!XZHLPkOrP7b;6MpR*m;gbUOD?s<1=;X*hn<}6csqOW zxzDSDK>QYs2Ic(Na4SV_#HQ?-`R7`3TO#>(b)qS;@|};}J!OjE1StUt$iHODF_Ex4 zh}37MVyMGLtGu-Iqf)9$0(;M#7jR~)nd^-fB%Sw*3UJ=2{w8(R$9pYWb>m6@uTfQ6 zJZ;bc4`v0PTmRhgKMyeNKt1o3*R3n|&vIUuUb;bIpML-PrtJ<8fc1=NeqjNt%YG*^ z!Q;^1u_iyDjEb#4TOhDec+OpO-O$hB++oFd9Y44(Qvif953%7)$;IkS#6_bU!yNXa zTtgjCAIQ=ZBs}Ib?h2O!)cV=;=4Jpry#@3)W-zJn43`WR|q``@a7z0|HNl}jvLDZ`zOqr3e>lb>;L%c=t|{EQJ3hjc>Uj9fxur6%=j&Ka_K7bknJCx1c1^O z+ASyu(;XoIG(Ytf!|VO3fJZU+!;KSPtIjBn6L6Etot3764}|3HrCtja7n2lro@@k- z3z4qeIzD_Nc??wyfFW%8ebqQEo)jcs+{%Dk7c&;hE!S?sR7t63fs7o?^IZNdkU_SW zZx2$?-T+J;%5`HA^7@m0koP^_d&g5Jt(?E%(f=XN2!=sM?h0NfdjSj zeRDkrk=$pt^x;{veETPpozC$6#p4i2-ftrJk5aJdJ34RWSa|c>rGDP%J7cpgy_kFi z{NE5DCFd_OMrPyR+O=mnutFxX1T1d>JT=QPKeIBI@~GAI_bD_J0yXe(|Ksn$K0suL zg{G}l)Q1;7mnl$!VeKpd=^3B?ck3~xr7Xjic!XH4lHji1I!Zn~00%%L?D(4-6b{ou z`@T7iZ-{WzB?VU!nv^PQWQvr1sj*5F;#t3w=}PA%IY9>dTS&HlG*8a&+8_fs`B~vc zYXqW?=Lblw#{_YE@Z+cVegT}fkl)0Og0)+`rK00u_`lhremF{{1B4YH7o~(D2o;(4 zw;F8_BMH~0jbIDoXUMpI_}W?n^#ylBub zcz_1@xvQ7<ORJPlP(&1LE{i%s5fz!!zJ<)5*Y9iXjut5P}2vhpWMvG?8XQKM(7!_KDs#9o9qlV}N=ssr7nZ?99AF`SFU)Kg=-d95_y;k;m3`_S5)2+fM z5QC&wD8~;&zvxu|C_^P~IJ5|sZP2V)x(e|4E_j}vZk*+EJY#`7xb~E)hp%b&z<))s zpywnVY~jVk1xt2U>LVKhlP<_w`6RNQMg}hWQhp_X7*oM7j1@IBG*s2p2w&`8BVB1d z-ln;FQI5{Z8yt*W^87#hjZuPj@m8q~d&e6tn1xUYt)O5LCpt9_8tR`P2@D52A%#Nv zzToY`Q6LH7jt6(aT2w>l!)Jk4;BPK0K1btW(tu^C9Of}EsUG`6d|tg;V%MFK#azq? z6Ya}f7G`|bcPLy?B2YUo=xj6*l<|ro01;SuO|td&I0hu)pq~aRV6o&Ok4D@2oQo2z zi6Jm>QDlMr62TMCcK%+Jg9f>i^x$j!g61p5HC zEu#yGg~3f8<}iG3R&gL(Zx3Jkm;jL*M_6*5ETE*@R8DS|O;q#N zrorvrntW_B)E+M;e0X}*ej^nL07?X24_;uKZ99AvTws(We3WJm{p8(!lXN2sX|9En+ z#8dfo)P-)#*iuw?c_Mm)JPK-$erELbKcWKLMam94g9I8X19+G8$NTj-kc7bxskb2L zV4*w{$~}E5JQ}cw{DB;=d<-pWnZlvh!MzflcU8qX=R8g>$*mHgdb|5uWNg29s#hp^XX;KQG$dEdVE`wb#+fE<6zdlf1b z4Hn&^5Jm z<&JcHu*KLp;ZlDyQWjI@^9ed6!9~-*P6D={#(yC8ffBs@wzCw>IfWtj&%)=wx){L> zKKs47NS)hY8lQ4;NlBR!m&NBK?JDynUd*yz+n|w84~qbg5PDxb1RIqc`~r$S2KYiB z3WvhOP#^@lVAranIJgxox=9%LG;_u^ z!}!UHJa%aEPvU1cM{4VfWNU4~xK?y0?j=1X_obr5v5p#GJdFlJ#Uq>HeE=i-tt=c1VMRfk)eQ_^lcIhUx zsPK-_g-0c=y0)A1yHEXAm#jEP!{5KNxEB@a>zx^zP>6=ol_i;C$tE-O22J-l>qLF+ z#FSBzeGYE&U(C)0P_RX|6)l75`_%enVxNvh2K34Pk?em=W<8LE-q722$>6Zh|BuAs)AwanZ<*I{VmuTuNXS*eGQkrJDxbBOh zM)!mVm?Wi-FL86{;rdKW>^tmc(~T<+tg9OelSaP%40;*BcV~XsA`@FqhqR|K!F3oa za8zyl7VDRE(f@DKb?Lm*QaMf!trvXwpE4E*O9Hk>QlN3B6bH43{B1P$U(Izy8cjYv ziR(cM$ZL2nio6RV@n9GkkXTzBOkh>PyD%hKUPHA&Q|CQ1*`lAY!j|qV81}i6VYox5 z#lA0+VENfdQ06yBOx0cUl+MyMZ2WUKc`)B$;7XD*QUweXIj_k@z zrFx7R1lC0Z+Stmun!3`Xlw{DO(Ge)_iI&3-QH2h}@bDZZfKuSolZio(!7n{+MMr@% z1Z7Ze!dq>;S{+9|Mxf3g6J>K?$@vxw!58{3Mzy5^a{=; z|Lc~sk+%MoI{tk$0?ZH{m`tM|9u@czU2Lx~dla-~3l+7_dsq5)*~)iL2vH zp1yTW;=K(XDaFN9Pc=D!Y&rPGNb_TI`|t8i3|DBYD;1aNvjM$>dg39Pfg9m-bhLmyt#GE|I;SOf;$D3B9sDuMSAQRg)!XO35+MP(cM| zc~nT?35E+N+tV8qO7Xxv5euLy;G*JU(gr+H)6hsB85!~9Q1cCwfJG|5)n@$rbbnrg z9jJkqEMJ2O4OEh^uJq<#BjpaRXcG-HBoXNr4#NJ6TSzM;cTM&IpSR5g{6rSx ze}+F1$XJWtym^<021@)Hj{n!#V)BCjZ?qD=3bwSv3eoe<&b;fFCuPSbd;8A&>~8Gb z^sB{(i)VezVST-HaJ$~3jbP$vh>g@|^ohWB$faZy9UCDHw1e(zDIp{jTo6+ zy1dD+C7R`2i+uN<16>Plqyb;j$f=*fTEMsabSW*A|7%!80nWbT!A58~DQb~pSoA;f zTm{lJ35rT!1i2#PxwV{Ex>v;Yo77NAw)_@W%Z^G5)4h*$oxXl?Yc#lT-_O?7H@_-e zKCMx$s_eXTxgGwD;SM#_E7@yhVa#s!zT1ui=R*bIBdL|UR#>Q%;NL#QOe=~Mhn0{b z-fi;?F-eEHF6KCbsVE1=@;gus4F>W68c!m+j`*}{;2;h3j?zcQF2&ZZLDxxVXYlxw z#yuR9xP2KU>$a8uk*!UeenW)%U|B9swQfZEER<#roAl1bjA6Vril#uGPdY;Pj+-c5>eq-m=Ibk)a zJiFWd@27vx{K$T0I+GLI=ZN|wI2W%vQVp?@mcPA()0y?+g=Wd@S=g03cUV>oh}-j~ zMYN_wr%X4tPohN~wn|f!bwp}{pFE_$7|ytoNFBn36rtr!dKe_q#BMHi9rmN5j$;*C z9;UGF+{uZ*th;nta5BS2C0>`6u@)S2e^z*%mq(i|rq-7N z{SR*d#9le>oJWlz08ZAzBfhcg8vO z{~9lW;6hc8D{HD@Wgv}VbaYg+AeGCaw^X~znLK3dAGv`_0Cf^iVi6Awj6s^Ex@+J| zB0(TLO#LDSUy>J<5heRJ(Z?>z?Sxk=V^2&*S`FCKX1_nZAV)2NR>!Yh$o4U)o?Oz~ z2%GcJA|Mb|(_xDG&?L$!X?JdaxRd^;&v&R+;!d;Ez0Y>L%d*fzqdt7bb?>I!FyJaC zOC^r|t`o#aVP!^BOkOXU?9@aHSCk^tb4&Qny>;6}^gu=tT-JNRYyY=51`* zKWkRavKaVHTuaxWKj8o<-GcYb5DAWZfV^UQ%0E}^gbzg0F|m0T4nwElIe zFh}*qo|<(jvcz6eojIAM=>7dm)v;zDs_bDBh$P)>PF4La%Dnn|?cUzr$rCqpjmAnk zbW}$2?0G**)FRK#R8oim&`;--ZRT=&qWHICuVuQn_szI5|4Qj^Tc$z}lzC~>yNVZ$ z*5N%T(_bZT1FBE-{W~5h8s{(1c!~KeFi*ppw{-UY)McfjyI-~Z^>hx{*}bpQOve_1AxJRv2;rC{~zI@cBb8w+EalGxmFndjDm7V4@1${n7(wk_psh!K8 zI~ko7<+=M9$()|uP@7nVD67*Z$#sK?`$t!54jijK3WhXcwm179WK?jNjEN+&)wivB zA0=UR>MeizEYYfYCG~8tY_Xi7yd&Ds<^9&QUFS=`)@d!K@LO_yUl!j|$jdeM)ScCI zWt)szve}*+|4jWv;naUjBmwJ{irArnzQgz^{^Ly#Uw5;-f2zxSWPXR2P|Z0ElIq@i z(YsJ1h972bx!3x7^O|y6=_s585}PqlG4F%h{;_=xd|3|bhCapw--@rgZJ#gA_?eX} z!kj%^|1)Al3AztVZKSh%vo<*-flrT=_^8GZy8@EDYvP@Xd6yY!y8SiVi_vlM4`5%SJ? zFqn?f-JG7Q(9}B1T73CpPtF+Q&K9JB_G}}tvRc!l79_m3ReqDl zpV(bh_&S9l6(NH%$Cy!y!?2x3;`39#Dpl3}@ht>bNBKq;zgcj!kd0?AFuXdO?w?HP^LxYma`u*kfB^e@vO3ON8*?d>yW@ zV^5p2Oix=&e+O*`)t8@bvx_qwtw~FLl*z&a0|FS?5D+Xqd}j?Ejdy-S(t1m0X!T%O zSW1J2y_o!HeOi~PBuqw9GJvVA-NEPGQ%rs!w2OC=cX06GzS5sV@ODFyq2J5@J@f-X zHlxzSv3c3~R$Z3s)}%`6M4I-Uzqf#jNfZYR)!sKQE;MW^JZsdVq9R$|rsEeimz)b~ z;}}FAY4H4>F+q#?xz-ZM%OW2M-`W~7U|sp3nhSY6-GDPZIceU_MG`H*7qoP-c7t_J z>O6+u=H;HBk0|0Kz0o1LSdB?=sf`G({u#*kXoOf7siS!4Q0?W>^$EnD{X%Qaz%=gc zgw;(oQx!3pt3niY)nP@Z0FEbe^78qYF$0x|=93tICJDq_bFW@R#nTC{O3yih1)pUd zIb0!_`_{!1>r7n}jxU!W%4k)i%T~Xs_Ku_DE^(-}=GYw@UXu-%LlsqT*mL5kh$?AX z8IcJzTAz!COMD9)KIdgrs5xsV~ayTCUw}3t?~BO)u3s zN*8Jos!884#p=@Zl|GEqls)rbdaf?x3}FqnQ)fG?a4Mi8U85{>&I@Qn+yVAkWxBAb zNuQr-V`h4b@eVEoF9M2GGS7q%X1oCaNH+nLk$l0e~!lc7@Vw zpkW80At5Zl!pQz8{8r8FJQ6CcG6Rw`A`A`KE0ML^MLu57+B|-y7W68*6wp>&{`{mL z_92Hjauc@^mQXxvSZALoyWqglewSGD-V>)*3UANBkOvB=Jr_@%-6)BD>|5N!ROnpSEe_&du zP8>n6@BD0}bqLP&=)CEl_~2Y9G0>D!TJnu#-1k%{24mMo=d4kWQqi*9a-l%8d2f4& zlYLqXY)t4X0pBEB1D-<-Wz}hwn3<^;I9yR{xdC8-6g{6-`}1vfgf|7v?1nJxIwem< zMce9Ke#n5L;{JT`N_4RdxwRqdeUUPF-e|nSgSDxpE+i&_O_un%jcT`RQn9iMT-O~d z4H|^*mFM2;hIkTnQbNey-{6w@oql6Pf0`gO_jBOx6pJy5AOgT;^kQj^0Zc<1f9U@U zD=7e8;f(T}J_DI#G5*aJA|8S(FSK}hB#8_)P`Jn?P z4karqYgx|6zDLT4hh_)1Pvi7qatzfYEqV-G^wZYSS;aB>rm}ot*b*sMyK*Mc3J{ey zHI}0}=v?-Ero|fMSyf|c?2qn#U`6dAdR@IgL})`PubnF6;yzaqg+s-n!NVxnr5?>~ zkUZqi?Hv8?SUpFJO}ViUQ^uz5ZDQAqCud5&bTKO=!DnrxSB8W#AY0|`bLqoh*=tb= z;+YDm?A6TI?|Qq`8giVWv zK#;lN)z$e-v(KYMN3^^zd66)xA1`71;XD8)Onfjr;y@1Txrnu&_5hPnrZxRgN;o?k5&ZvwVHy zb~$oT%RZTBe@^dJwYLSaxM^{c7KZ4r`k=+z1Z5`IBwg^Kd$`u2EqX7j`cm|?`bbS$ zttimdy@NDDCe|{vZYYYqI!dNQvn=UuOw3zWi4y9fXjh3Hnw#LX&p%XNTRPWJ|K^P2 z92Qia5HE7fy<{NB@46JR&<*RN`O3jJjNQHoI&ckiLJoOB2_2TPU zEX{Ej=7+ae+amc(gaUivc-`V3gi4es-CiWQ?c6GdU~=>E^t@nAyKyJzEr3YrA!cS~ zss;~GWPW1{`v6c$!sR*p8;;@8gUd4Rz5P!7^|OCXm92lL%7PMs74#1yAj4)PdQ?+0 z(`>*GQ7M;dZ{M+_kEip(YqU&Y5fH$KyEeJ*!~F+~x)bbgR#nEwCM!ZYyWVeP3ogDV zJR_HHwrsZqDfV`B^6TLsw@0wbF42{3-Td{W-fFMv!4wu5)!0<)TdAGBn2<$xbS1HqT?pVTiaay&~eLCXI6fZsQy@0 zMV1m%ZtWyeyIw}OP(9B%UIZ*0<>jOW;Ns7g33c9Kd36nqM-sElTACS~Ym%XzgJfox z@*Z+iqF2VHZtF#S)N*H;ZuB9r3S?ZwOnGeipUHX_$kwr?`4sELE=RunxP%~!0*(qs zjvC3|eP(GQWUM0A!WUMxYZk7Zh(NoXoGmd?F`uBZk#R}AnRaEiNd6?8Evg(a*I}`A z!Y(v;x&$X3fd0506mOAke_`cgNx;RjFh0Sn!Qw2i>#u@rb&y1j89zNeqJHdaqOCH#;8UATkcTk-QoA^fC zl@J>w?C+yjG?q~GiU^`g{h)vtp1w^wxugUwh9qQ2ELaGd4j7L0gq=*4yrC6kI=`9J zcRjZ6?mb~~#ScUYWMsVZ`di*w%(Q0LOk`xGk2#C!t&V8@zFp~F`Tf)yO|Fa~KHMqc zJ~UoXqUlFVT`egmtyEES8A{ok-p5{;7nAHjdBhDMp!k-6VWUdlyvGfGegUc8>o~*b zuV*AJBv62;I$=uuuE{{Ae$`|?k3NoiBywJgDKkMj1c4aSfauG=Q|f=JGBvV?gg?t0 zteY+_LrL_=6N8{AK_9UcQhD|5U%A`NU6V9L6E0zE`AIRcYi%c?+0F^Y!-0(62%sXQ zclDLA;MQE4VImX#2`HPQs``ZAM56`yVkNn;Co(e_F!N2j%C|g{ctZ(tLDjVlaXh$h z$AIbIgp();_zL{jTy$u9pj@^Hk*EuY!3bn;n}U!ONAy6w-fBB3y*7rN7`{3GxE5Q9 zES)gk%I{M^fj$N?+*$5H%nc|W25L`xF7@#Kcoy8_>T+DE-J8-iMB%P`uIp5tE-LpL znK|T4UIQV~#fp=bZ!F)ucaph0AN7DK7a`&a0oPyYW4TfCE zkk;_y;vEc0+TL0tgv2t9&s8`TxfM_DOmp1Lc81J1332^3P%5)NrrB^6q0uHA^FzovBz!NlCQame!}k#HQZaSJf(ymXvXH^+@ByfJk~^ zS%$`=AMC^;3XV3yXZ5_6z)}&5BW18Vk3%P78< z>aK;A1eiohM!T{a``=6kJI@fr5Y@T!`>`~?Ag@x=-+2liJFrW8y?iAxw}5{}S>}|3 z3=fPCB;l0Xa^JT|&^SD=VIS{_mfgx?x#Nn;IHjX%SHhtItGuCXQhxC+I*VgWhkzJP zP$iF^H&2=4L;zQq)b(g(h#F&$5AfZK3qhk$^{(g_v3p3Mx>sjrME#giRx*!OmskGL zGdbC#7EL+qn-OqyX8`LYX&t6S?xSkb(Ue)MgG?Bzl^`p^c}+?j+jJch8`1)t;Xb;H zK_W2Xp)pE$m_h)x@=#k04b0mgDpYx-X0UqYKME6gXZ?Cl4CgEeM+ltV4(4ij(rS2o z)A2f(-hAQlP?6{)3ZeA5*jGfTvoV^J6D>y;HpnPkKGyQsM8m4C{7~AYv%6VF=5~Tv z-@W*f@+P4L^Dig5$bZWLa}}e_@8#UHA1!JsY^_%a<{*TAIG_4MDM{G=D@X3mEIx-! zbuBI3s8j)$H~p`8n8~6{i2q7542lk(w-z1kBdDb^272Ad2{hHLVg&gdh+13}2G~l3 zb{GBj%l!Sm9wSCCI`$7u2FYfX-+mzs!InuyV`PA_311Z;(}nyMW8p;)p1us%ca!f` zPH(aQ>C0Xhh=j;<~nLKEC&g1$HY+Hd8+DiU{RS}%S(3SW)D+~t^t*q zlw={Bss>Mcv1y)0%*R7VYRc3NF>!1)^XJ5nZ4eMT$K_7}G`wChX%C8b6A0Dza@!5m zwRy@6n%021&=45zj$g>d{AJ@J0R8J5QILr=1-|FcOpL|tb-^W&4`OYtu)f!)S8AFW zx}#!~r!mRf^y+67_2=S3v(%%I7*Np5?fP*AL+*$Fx{ZYb~TsWnesRMuWn~2 zd!Pdl6Wuckza*io{Ph79OAv5ta zE%`+PGszl70gjNGj_#D{hdpBFlTR4|KP);QfLt{1GUob1Rtaxu5`YfN@bH0-g$5?s zop@g%-eb$i)ow-cdKrCE_3~L+M*EWNcfHr_T9+4Y7AdtfR9i_^cI{c`Rq`?@7k_@?Y#xR}Um0$8!*^cHDcfXJ(K3)uw1kin(%!jgpz z8m*`m7YW)0(i^aCXc>(5Zi5=-4aA4Q$;l(<&h9HY%>9wC;pF>6L7f;xyP&WdDG3vO zcU3@Qknk8sR+K-_{Zkq!`#RiaF3st0v^`a=Ka1=mSQc?OMCWvwWBFX1V;Uk$^+gQE zezc)&R?d-Qq46oJwyQQLVt6RaL<{FKbI-;0e4)XL@6J9xE+*_XKiXmc*@QBKTQTqH z*XM+0bHS1wSiLLS042D7FK7)9P7RlM@ehQrgnW^Yk|vmFph&H1q&kEzgNIJr=$lLI zdr!|PAUNr%)YV>i| z0|)G+8+{tos=T6P0-dXA%#VC#)t|=S^`r+181OK@#{h9Z*|J9O9`l?I{_p4 zq#qyLw9w$+mq^;9+np>SUy`O_mqFq1njGM?n=Jh%UOylCZohqh32P(3_XxXKzSCX0 zV)r%vd&~CIeUMLL8S`9t>9Y8j2L}>WPv5!l1kqg<4Ry$$wC008(l+dya%29K1oct& zVS#=^2~^onby{MoLCfnsgn9c(ZJoX9kBfcGa=&v0yi4a>)*}@)F0FZ57R+sRfg`na zHyQrsb4L0Xhd)@_gUORCgL)S4#kWe;!PL;ThwQ>v*hu(=rqf9aA@P~g#XdM%<7EH3 zm-WO~PP5O8N}MnBXV!caC{y{pf&o+?5M?}YU}Ls6FD>|Bfk_lsqiXcs5nSP}C6U74Mk?%O6u z%-<(k`Y4d$tL`-d6UaL>(w?a3CEMPN{JI2Bc(&B0_&lWs4tJ(f(!~r(18U%S5dH`D z#Reu$9kcnf-!u{Y@M=dL=b?x(0b6H78oSi9KytK%fSHnHNs(`-MdXjdpGG%KigvHj z5Q>hhb|~5`f$BsEwSRC3Ob@d^(>F?}@~4FJ zbGH)#)-$xH#`EZ3Zfc>a#X?kUFAWnVaV# zC=%VFbiOmlX-E0aEg9Dk;2mG09dq-@Uww*GL{SB`L{N0Z6O@S}nWf`XKCQ2@XHw|2 zzfti6j)(Nlrk|5^%Pbe@y*55JcDF9s=(k|Xc4Bhf2j(?60c-wGzYXjIGf-=Tcr!iV zdZ8nbVbC9Mw)!ReHWebz2O;_R8>QH7cv6`hwR{Ro@_>WAEUXMNbSf&MES1S4WQHwr z>6pUOe4qTr-OfDm3Vl3Z$`o?XkLm||&|;PKOz@Q1eES9hhZ?0y-*re^ zwQ5~*#snQo5`q$Bi*}j2xEn(6M96>-uiV0G$%;hw|3Lnh7YdPO2k$U#Lv>`WnxjgL z1g>DpG79iskJf~h!36lm2-&L8wBA=_gGr_8y^nam^kh4QiKfNk0F3x-N zrqd~a%$@#6exU2s&b6-Zr%#o4G|M^>#mPH#6S`-|m><$M3>KhsQL0 z#+8o(SeoV=HbZO4;l1*LycMX`_XuI(Z9)dD_h5=LI327PAT}0g4mLegynPP+(m(u0 zjzGh@PbS>p4Ai!c!sHtRT)^x^*>iWf(2*|Q^N025as&IE4mt68 zqb$L(fC1LPLZh9||Ep~g1H*|q9P1*Wc-+7UlV@~O^z;-J#|yiRta-n$@XKwkWpa4! z2jBZu%kYy0_a1yo%6me-)I{K5G}~jIoqMW9*LDm)ml)nVhg7Z5SYX03HnP>Z@u-GqS3Y(WTDz< zi?P=3(pE&f#gE^nGlT0lfYGsMkD7WZldwNkwvzFrR1+dP%%dYbI$=o~&WH##4vA+1 z^2ae4X1EXQ2C3kk(L``_=b){B+G(6<=?WC@TAGG=%vs@-ds~fuST+F-0KgGR8dTFj zWc5@f-Q;8UJfWC&lAf~&NO@N|UH zYjwyjp}c(k6&U@TH>AgL(M!G4UjiV;EI(C>$>g;Rfh#6F=y!w=tAY<0QSR&TQ-)^W z=n2PI3%5FQ+Oko73jmm@5{Re=k#Yh(JA&K8X^PdBX}+m?A=bz~C9p-hGWfn|UG8<5 zE#d-4K_7I9qbRwk(NUYhP2Wqi8@2i2r%*IF;0hK52Om*$f+-{M{rhB-yddwoHFGv{ z_;Kwx4hy(pF-Vy9Q+t0pcQNu1Sa6uFU^!5eFBsnmz^HuRPVC2JK{x~Xcuw7tw?a>$ zHW(Fd8P~qB8A`-s2`mW_=rb7^02L1;Flh_!;$cEJ&A^Ge`VzX57T*RM#*~&W)o}|) zBqga?8t7X;99)m+L>c0!+ea7@g?SKivbLY8EBn6BaraxFi&+ z?>Bu(Bd8aD#iX37#*BQK~ue%_rQEv*o_@FIds%>+HniX9L@dd7olkMge zvAtwllBi=V*2HzuO4hKHJLZo>UdhccE;7|_Xphhs$VNh7O7!)+0h{5}eTqF6D|h%<#eH>V@I#MchYFyRkaz!mxaTpA_# z3vl-N0>b{E7SKghcVe_{3PY(+dC|pFuNE@BYSQtH3aic&y=ip?Y67bclYVJ!PM+UyV@o}(n?E-GVNtYwn>GtZv@N>>CJa?-b9}mFJ zj{^t!M;{1aOu)bp1X$?KacB1a_^whOiZlqqZ25py9FOQJ#9c?FrHeDXe`yt50GBWm zd1nQI#Q;9h*Vb!-4>0uwKro0y|FLl`W7%&Vt72owY}0HhQ?;R z-2oi&aSSW$t@k|micdUo4-P<%X=xO27yyPK$F|)oo%M#yaYJtq-iM$t+rHlQ;Eg;l zfp%B##bUxL&z({Us2hjaOMHcFum9K>eJn(`nv5|G9@6rxornbt70yq3feh#-fHeZA zsLZ!(W&*DF0)Xqob^0N)1B(P^TeVJ{oQp*ZPlbRf!v{u=SS{~_630W9dsWgM&OA21 zDn|;SONf5$OvhSJgl61+7c*{j44~4%GQcMq4dR&GgRE{hcgt;uYt?h$p6cK0`&Om( z8h6k0XoW_n+DX~?mYm@e*afr}B$yz*pSdM2=Hop90od5t*^zGt$BSa4&S9!3^@LRd zq(BMo(_VSpzy`BD7S1wxNchVUI`R?^-&IhoZIXil1_1|;Fro)Owe1N(SihPS#v6o0@YKnKiScEB?C_@D8dMquXb)q_zb9 zpacnPU}L;Qq0|92*~DbW6>d=Z_vqGJ89-qHoX;NbDa+OAYZ;9nIuo&4LWD-Wp2m~1 zyB>(@9}vs8NNeepeX8EyCvE-WDY$O#ItRcz&n3V`zm+sWf@uxn53J|-&k%S>kj|62 z4gikh7e@O#ehb9~x0`5VWEtokiHD7ez>);IcHc}?==}`7i2^oR!&>GrE^@7(qUfWB z7z6r`6QBj}8#$ak!q6r1DU5{vF*3-7>t83kht35i1~XWZDINR5St%uP7y!=bcXV3{ zBTd}8;5uLRfs2b(Tfo30lc*Hx`E9}8B{M=&Fa(FNMR#dZGDHxVkatF&oq4Q{*M1zr zK$rL^iRbgUfyHO$wmC?j!sFwKnorDr#eod48U|*w(4{-W(_ac)Q+MP-hqQxl!)?cA zSuORA_g|0pV+#QR{|}y~-M+l-XF#HNK3QPX!+D^HW5c=ry5|DM4_$3*(|_QcSfb!( z724^{UkN<}^1d-WZ)6RpWL{+DfWHTx1?jPRkzTxk8ua4PSfSs(82Gc~!}m{=-R2Hz zUA0qCb9n)cPG3L)5QjHU0F|$c~F}d`CREQVJ1^8 z*PS?qH1hCmyg%aUL&&|B=zHIsqy;@x`3+VsI3g4z1~?-i8OU1+JPli2wYKjzI**#wl}8p89m$4B^tX>u`io} zlOU*2S|70Tz-!0>Y((=>dJa+ORQQnC+RcS!ct-B!r*Tvt=WMBM{^O zxqb!sqBx;GwDg1i;yfznMRG-cXH`=U)PIsjF_TYIyns zqlnVWPY`-hFZ>4Da;KqY<9{dYR3^mG;)Q>@ab?Z`h@JEq8y?)95+b}zTXd&nrvs8p z7?dQ`n!?j-y1TJqb)XjcBNXgCbF^#l7{GTR15(&K%Q$byFMeud<_e2jG;By-@W-)k7j zVm*MUGS7}{mV>-!sC1g%3T*&|u52Uu-v!P+dkj?CT;9QniVLjcK>7vi<$xGo{2z+U zB~TYJNe%=|@T`jc4se+Pl+!{DZjkJ=_TI#O{2R2>Kr$VA8)%yQ=k9B;?sJtqzD$%K zgTvuHF0j`F^9mPMBKS85iAkXZBFa$d4?=O+v1c~_(y=aoaCHj%yC|W1U3oKX*`=lI zxr-E5Dd1-h;PGacc7uKs{E>$AS3-yjCw9;q3(VwuoCad@DinD7Fs%ZAgOgBtaj{`s zTK*!GXgE7V*#<Oi=!qHaT?bqCZYR&2j%AX@C-P@DanV0Qi^h-~O>2^#-BkGn+Q(HS~Ga6AxnPXV;< z&m<83+PDD+`pjb}z36=`)M_;N6Cc(HrA8R2Sru+ffyEREUl#KpH!mqCf3p~Iy-QSB zW)kzb)^%%puBxg^J&Skc*m&X4okILpm~5%zKt=h4 zS4^Vd+`Ig)@7^G*{0+|z=2vj?{PLa16qGB9fTQ&L3LETlg8KHdWAgz4J`>vStr9j^ zc;-vOdoa=SR=?~2hnCvbw4~5=0d~yjNiKITi_HvghrV+6S%>9FS6(hXHZ^Y3#bh-g2k5^-Xxd~ zQLYiPU4hp?lV`MkShIGSdzQ9GqtU>FYfj7&1f2O{0H&@o+2R12mZOvp-8LCFTc)n% zk<}}m>I{9!8t0?Wn+iVQOWa&CL8jy~~%BEY`St(WFsv0hNb zZ&X_WFVvIW3$(ectGBtULZD9N=dInrypeP20TH!aGGgM-AZOpw2G7fv+!TSpNV8&L zI3`iWCq#{^i7K5%M0)|p)KqXvlOtbMpuvCLu<$S6-vOwowC>y#&;N15K=P5q%$g#I zi?*KDJpdn5JWWb7l;0v*$^p?xf<&*D+`AhX$dn}FAm*pkv(1tOqv4Wu_bC@XHi7~b zK;)pG$OWY)q+eK1fNlx5OS#Eh6gLPx@E-@^(i6u39^Cen3zfRkjp@JP}$p9M*oz2IY=#)W_$i-*b z>_u6$?UOa#qG!`15ZF65Mr;`y-2I)xH%0gU3PT9}yeKsL@Aq~aBwe4ZLKfz!9Fvy1 z-AJPjIf(*|W?Tu&cV~HAw#O%0%bd%dzddzn)rTO%`~{OUG|Zg#Iu}!qt8RinxIrSc z3?{V25hw1eLn+J$i7M<}1yPxvM<6l%-bYi<5jzH8sWZo zkM0FWka%LSfBK=OILuhEC!)!1xs`xfWIoRX0N@;iHs4SdU0%v(-Q#H?aOttum9vY@ z3fdh1;+fW%1LoBkYvo6+V0z-xN`Kxk_{htdd;XvO38iBGpru7A z`^0{!J~i^TIin_;Szim;<~*Ypc+y`X-bC2QTD|nPHE5#)E;yL&(jI>Ao@uuBDu3L` zb#m|Q%5yPvLtwE#cK`8nzu=x_cJb~#n^|~G%$-@$B2u@QoYeC>1cCoxSB+sJ@3;T1 zgpG%aEVXP9!h#|!<(?AD^B)g}%!!?V2X&E86P2m_-d@`s=V! z(JQ3RdksxmJWUA!YWY`_=JhXnQG9Mk6Mz}&HP?=gDhNSh4p?L@<{r?}N~)U!QjSZ( zeW(>h$m{AMIVRv~hmeC)*3NK4Xl*HSS8k8dHz^YRv{3^#nuL-$K^C6&0g;go5i|n= zb3-&;ZXInM3?s~&!_6H#&+PK`BU=ojhDZ`I)YzV6{g#!m20Qs z2F1GgL`=X7|86Ddz6jIj8Fht*=>L?)Y(IZgz?`*2wOMa9X6389r5%*(a!NHlAqR)h zrWQJL_dMsa?0QFO%~(uFuuX2qSnn(?q0t%06u49pR_KPaGq?Py*ke$)@Vi~9Ez`+K z0y?F3AO@_a{_xZl+zaob+d7!q&V`WKX)u2QuAIbK?Kr@CxGK^orH1+E+wU}^z2<6B z`uDlR`-#Oq_NI7{-qQhu28Zk|*;@QiWs+;N6OaDshGMOd*!_u-gL>)<~K@(SXpnLmB#m_Z*bo2!Jm)T`hnXlEVTL7yiY9Y z0=?4~e}Ue4(Pqehsl7rCpmzRY3_$_`*c@RUM0zai>Cim{xuqMzC2!Du_OW8iC!ONB zHV*e0Vx!&$qiFdJN}!-1P$ZuSE5j5Q6vY}Yau*3~(_@Qw?e4burJr=K-Dk3wOCZc* zA0`k%OK2BI95C4f3CwRby~q)(76^LCZW%(l9x-|2(M5_LcAybAM>sduTY zB&mIvqmp{8?+^L`+#qptDxB!z-#SR`Hi@8GDa1SnSI4h(refS}wAMv?6JMB9`>pMY z{P=v8{(JAfU{X|M!u`ZHHxf?rmRUz#w1$AUA<=-em*zS%1|aDh2N@<(MHbUy?Yk_; zb5YjEt#YIg!ROvLrmhJPrSt6rrYob|#Y)S{<`M)B8v*0|-!m612;W&q2#s*jCCFMw zie{yahG&RBzrB;1i2dTxb!{%!l2Ja(=xu?`gl=s_&U>h~X-Nb?>fNZ#Dz=4G(mUFOQIK;O;h zx@HZ~HyvW4L?=(STa~W1{mStq2b3o^g9on2jVz-si4J@7+!q|X>ODH1CAVL67W8v( zr)F`Vxfpb^3w%8}l(v2pZ&FPe_g-@L=T}y%Z}L65;2D?;A0+yHQ_x;L({Yh%qQlxE`m}kElPT{HiF(xx zr{Z4gRrA{mr`=L;l}u$Go#+`mmZ6OgW<=x&NAC?yAt_JYUH(rY3fGs9yXTENDqU2A zs{hc4H7s^!=VZ#X9NrtOlV^6LewOlrp-Qcz_~A&M;^NP{6pOh%#fn{WnI`>!n-mrw zWscLirrnjT4Np%fNT4Ow?H=UC7VOG?{Ib$kHO?T`L?xU5!-1gncBPA6!=7gP`)Scr z<*x6~(Iu+%oV<>mDH2KEd46|gY@aWc(xWM_J*q1$L2Xtua&a3fLXc2LE;hpGq zpKXBLV3g*kXqBG?0WCd!L~j?DxLo$VS1)G-jK#JgqPe&};YBJ7HSVpnjxb#I*1SrG zb={9|8>!cma{VE061(ULBU~Ya?O2R)+l#8@K+Ov6KEVmyR-6He!21*ORDP@4rvbe+Z5=EFBEtEMF~_XD;U$WJx?STJ2(!df+PT z{xxuV=TvH+fL(Iw5FQxf!^ zcc_yt%2@e5n{WEeoh5Z=L3i=tVyP6)xZSJ>w$z{V$xXSBzXwW|SJQeec zZp_Zk@U=JreMofP;I=a@5P=Xhy>+@lwVTbXAUa;l6n`FJ!7{_AuLw4VI3YZ5NW}Mw zuX&X89x+6{g$fuD9}-nJ%hs;pxylHuQCS`xG0>O<=WBKGMRi0>q%9s5`$rVC(jCNa zpd z=hEf>fC#RmteviDpk4E?8HEoe@Mad*FH5KQq8|VfFK+$E&h_52sl*rD=`)kheP<48 z&?S;7jA3Aj1uDoVuZ7x~;)tP|^-yuD$2{M8m8OeeEmeiA@Spy{B*>^!qYMCpf>eS!v*dWO z#xhm?rfCKXnt-s`uwUNvQN_{8t7fFtG~0GL{59b_U7UedS?hXT>%foZ-EC#{TE%=z zG)=r9SH00ND_Fxyk#?^vHSLqt`nl5hy7HzuR92guKBqsEqbK0<@4$X=4RE)45H3^* z9jC8&(WcYQ8X3ygMTf8v_N#A`qWdDE0lqUuhE z^m67#JBOAz+c|JXTtMV>VYOIQ$Ix%Jf81u zGN&u`B1MvSWc!ompUrce`F9qhkeM(TH=~fK<`^0-}%jCezwhh z_`pDR_jt)X&hx2I{_U+Z>6cW_jnJH^)sdh_AgjJV|HPsssQTjllMx#&W?7yqz&BD*=+l%a}Jdy7YVE|H`7 zCF;RL(6PB48V8!+Ezsf%`qV_CH?l%Sda(N!DOu7$P7FsTfIHbLa3pxACpjjXf0B`@ zc$63Okolfdp{;@{bLMG2^+v?}&Jo~TS!y@*W0c+&*jbVzFl7%0EI1p!CMW?YdfaW< zOQ5c(LWjDjwnfm^8^Uk?BkaY^pq-wrf`>1;vTS|>F-l`D!gCTi45FKm?OU_6O&OlN z6t!5@3t_yxI#q*G%rQx4y(I_orP?%wm5p~m2A&Rb8;=k}o}6~JZ;KjH?sG$3*~$ck zD_z93rY8VWM_&b^%pzaC<`=GM4WneExK6sQQAhF{puSpuCG)hCjs{YfN^}X$lf_S! z5xhw%Lz^ip?JT0a{1~LJe+Gx*=Z{agbZ6wzkhf7K_rC1_e_30upkX_s@jcJRq26qV zLJ39E(Q0TfeMn)#O>yhxULWmF+waNps5f(b#5ZfkX?M-2$|7a) zRi*B?8~!SOgw!%{^lvqu!))3cis+Lj%-HX>4}4Adu#J^8f?e*iEkdzBzQ;4 zarAA-6TkdKw(!YS@A-t<;&|rJ>>p)3lBB?oU{_>8<^*Tgri@9C?{{(D=$hiB1LL}t zc~zz^q;`s)v0M4eN?OF?sbutI`^$)}`WMsoW-)c3{c?ArS4#U+Xtdd~JK^wA<9QRw zfS}R(COmzAHWBZ_pr@dTp%tM?tpe2gj*=<5b~0kSWHU&_7_5vqM6VX)uG-4D^K z8_}D4xg(f)R>rCadk;qM-9tsn#s{ML=wnHoe)*K0O$SKL*Trneg2+KwJb}4Y$+vU% zhOWHIbP;Cl&HQ8%F4MlT1Dg&=NbZXD*b2AYq|)3Xr1DL9e~<6u1{_UI^o2D@-o; z4PXYTp`rs9U?d4)&=It}>zOc>}StM`=5R64nEj=zME=u2Z%C#V;m6r z$$(VfeXujd2Pkn*>_=LSNfrcr_O*Q}d?^ta(Z)pW>f7JaPH=~lg2NWWs}7JSJiDWM zostO$(flcG+mhvXC%PQ`RqNNbdvbIhv_yxGyDM}AE znV65uHaD7%Upa#0vk8ihB8-lq%{Vh_9=uT2+Tdh?1qL%SGg*yiv@Z4OyeWynVYIi3 ztF$@3GLhP{5>!%;XvD5?&&(WItrykLBTMQmOo}3knGcTJDCPdRGrDO$(bEGx5-a8L zo@Vc9dc)P!Av0ffn$ez8e+a9i+_LI#tq06%E;;Jbte_N)Db6@afdZoV*VXCfS2~49 zMO@(F(xNE_uKaB?wgiBzPWRUW9Kb$>^uZn)u#3hy2>TW-|C2+HX|1LSfCvS(y!vSq z?q<(GOdi@={n{z({5D}&u4j4xSm4w(Zk%yGRHoc3zC#TNQNE(QNft2#kW}S_?8u{H z3DnsIj;ItfpG$p`KH0_Cmva{9&}hlhHV83qO-yNPDSpd~N%m1^lcsm1-YldaQDAt76q-Z6`qJ70gYI8|G+mVOgptPX2RgI%N`zISc)TQ#& z#gN#wr&sF#Xi=bI1gm; z{`q7jMMs1bEEfVJfxkBNU!z*Oq?y%;nzH@(s*9KZH7$7uA3hzT2E1&x(~^@d(j?a% z00z_Muwft%7fF8fS}68RNot#^+A=3`*5+5tN&8F1FklP*Kt zMkn|5{N~vA;Onx-aIk~U0SswpPV?`wJda#1#e`tfH8h0mc77JQdBulu6#?a^(bo#kYmy`j%SVu7zUd!4i}FpI*I1|cJ{9uu$9@Velc6f8qt8Eh zPTnfLQfjZlwe`tTPacwQMB7vQk8`WU;B1L&U|C0*^%gAg6Bw3Wy9=r!eDK?>l2t%> ztY(O4quP1gm2F{vV5`b)WGI+Z)R@;F&Qv{l18NI)y9;dhog^z`u^cp?kk=!>$^#UW z|EH&@NHebp>1_Vww!5G?r>*OT@ysNp`L=COU_YbouKkY?pe2b~NN$HEN3yZ;bG%z* zTb8gEn5p5opxkvvk?RGL1(MGJ10iFpBE)0l1<)RM?B?&x!wYuF002VnkNgv_Bk0Nbz|=d zVnf*0{vx2<6AQm&LHH7JYjk*&5ZKrFR0P9Fn~I9+x-Dm^d)yvR4Swiy3`?=f29nyY zf(l`D1b6c_Yf=PTG;~L|YZ?cUnLo^@Wn%igz^IJx=08ekE{igT?Yu@ZL1;9 z0jk&bbvo=am1g=oF?Dy{4O?|MSQ=>4v;~s{N9sB-wBzLf`bWR!PUx{fPLGO^onl7Q zs24;ZAOX2#5->CaL)Y<9_c0l$^Q+eNUVA3K_iD14a`pOFZmTL8^9c9!44(~;SlJ+Y zX3U0)K)eGosAy$uCAGlJSYRKKwW~5uG1?hvyNCl=TWxA_# ze7efQ?;(#kGgISBWfNLfc^-^b-1O{&H3LNj7E3Toi_E=CZkab^)&W zQB;T(kyF}(?-hJtAwjs=z;z#e00gn)6nusgYFB9*yR!KlY<_Wxt1BsaqH5{RWS}TV z>JVL0QX;iE>%rD!K`_Op^RQf61FY0Yp(prStix2wJTUHb;U%l9g)P%6ze>MrE8{GY zEt5WxBb?Rz+8}9Rg(@cW2eAm;hzf0Tp_&4a@MDiwZ#jTt`Z(eD0 zWHso?Yf_>dF}KXNrykKPN|KGMBZb^xR_GXRO5G*A^y>xMr12UGz3+u*L^y#|<9Uo7 z{yl8aUci>5Y`Yj=QGy$NE)P1Qo>;%YeuG7ipV($pK_KY}13*)vj|^aQ3n>}PHTWj= zI@qnny0vG|h-wBn_D(oUn@W|NC&cUCJ~j|5kNXnpqLlq0xc{89VnAwhe{7sDgayFpy}0! z${GW;XiNOOp45^1LxTA|joFz>i>6N|vqtl0ec7+JC*L3O^X80B8VTNJmCOdsgv>yu zzr@TDuAJqlLm5xxrh-7s$rgX{d5thv>TO8y4(dWd>wPR+fv0KTD3%=Gr4vT7@ZzDe z)h3U0iI%^qw&=;aU&UW$!HwKIFxs_$9TRLMr4%>bT%iK8x`8>0G9G@?t!KoSc-)R_ z_dT9NC7Bq~IP0>TbQkI^p#xqCXav~I1%1AN189J8f|2MGfEa7I1`Da(NY5Rs>WsaP zio7-{qk!e3clyk~rqrOA{VGh+I1#K748O+7w9>bbyV*OFcxr+l8Su~vEM+km@FP4& z=g(Nl{r@(@6&*|TIAoj&dp*fU=ussN*qAl`pbdegQfW#%WVLtQPN+DE`a>?7XnfBk zom@Vzv3Sg^zFS*xAR8BW>m*0@{a0h6Ow2b&0U8zY#8&pX=gZ+y`634Us`YhAJ(=bj z52u|(Znn*DR%z8NFC_pba+2X^Q1^yqMhhA=rpO9OmE#{Z&TcUYi08-guq@e=#RMQV zuC?_5Sk#r!5mJfME{6gBy$4t+pFK){*$Gu4^ zbjJheDwvo>_5o-B0!H_ei$2#{@tMh~s4oj) z0IobcS=td}m@_(dNJ+G=U(ZruYLdN659ml{feK(Z=I3t)`J(mFe^_f+E3J+(Av$}I zCv}8C^EUR$_-cp~ag4cR?hHniE2@2mT9jw|sQI#HsW>S>BS?OWmN+dxkgZnNZ~8!R zyc04fau!SA?#K^?SSd-`kyHEEYIft$SclH*so65AQNH=$3&l9Px?fb(K^n}|AzW^6 z!m2@GMV#BQll`+0YEL1JvVINnl{Z!N<+%w@7YX(hb~Q1K?O})Vw7euk4AHMH$hgWa zl61dYEzOW7DW&VIa8aew`!IO0{Mb8LWtuuTs;;?2`jz9-pC@}s$trQl308d2WYZ{h zJ}OzITJW@bV-(_?7FjOs%JuiL^6QOqZ2#PF?$|gVd z5e*}P(8NTQtQj^Xj_y;}Pv)K5AGgdq!%69`Wrjc9;?IGgWO(#~aLOa|1OY*n$uDZ` zGWeLaIH3+F#-b!AYsxs8It|iP2g6;H%A3K-;#(%8R<){2d|91c7BVF%ohoNd#)NU( zRY`Oqax+A3^~&=N2L_$8EdiZp+eIb}E<7nOB&{}z>QyP5!)m-7lCs7Jhr1a0t*K9s zv*EU?2L1MuT_2PUVx!P@YKJ1ZBX}jtktb}|UnFXE0$;{$BHDJbxAtO)HznqML&CO= zYbodk)B`mPcf5EZ92yQ%focl@cwWW5Ga!G=&js4Ivh5+ zd=-ld6f$%O$Ibx&`^NVO>E3M=LcLpgvFDpHa`&M?8yrMZqeN^Lu0nfaW_->WkHGoXZ1k5`1FzLIL? zYPb5Q&Vkh)G+Y1 z$PKcKzg{npX-`U2nG^No4{#+JSK_EiRk_V8*psA!ghfOYQ>SDr#5IWuy~vz?=N-v# zJ-Z?e<|idV_bc|5^Mt2I;2yE=4GB7bnW`DFw8IbqWlv%ocQ29L0d?hpesP{|?Rt}f z^z<`TGEpND9r2U1KH{m2RS%^}IcLw|ly}s)9s-NsnHZ_t=B$#mvc~lmHQV3O1})X^ z@FT%yH7=KIrx(*#nca-Y*{s2|SfXt{Z{W0hFBL5Cir+0I@2^Zd`WZPa`&H|j1eF$5 z!S_2;j-)vai}=i)c@S#Mg@0wikmAcftYDvu${plGsZjUq1)&kY?Y}qqMdcI~vCCs+@dQ1N(3+_oeEyB3KK4@;{!(BEx`dmg-_+8hY(fVyjT zE1~p8a{*hw$~g?_u)(7t`CZC>qaCEr_R2lnn{f6b^!Vnh=+f7qZp{u)SI#M+3pc^uS=QD@xn`+ zG)q$76h;?|s~0Z$x}3Ov;&RXqs+_ANGPqa!)LfTmFd#uiPlvp$%i7MaDyK>QZbD&J z(Ps~r*9r>ygYT*X?{3?DQT^O=H&OYT4URO2uCXWEMh`}|=7t2b0zJ*qY4=@LmU0XU z*JndBx@WGhqXaw7z;s8p`d;}E?{8ykNRPkMqayioNHmT=+b#~(4L0ep4{#|df9{(P z&@xf^G;g}8kH&T({U%yh?*X|M$d1y?D9ml zsSo3$W+laf)le?RV;mmOq%>spx(bq6yMOgA{*bJ8Fp#p#vP(YZiO-g$4wKJ#k#{3{ z#Y2*ZTk|L+fvc12Naig-LW`I$l{@x3gOx)z_KC3VH}@ zc8}I2Yo{3O?pm!&;d_I|X>IiTn8V>1$;~n~|DJmj#gn?yPuQA!c69k2pAYXvwJs(5 zO7}>`SZzo(^Zciy?`t7?*aRM4qIff+GyEdkV}YQ4#czs7Td@A!(ht07>4!<)!njyK zd$Fu{8w7qkst?~3po04bE3{Y#Y)vkIf|mC{2~hj;MS4iUUYwsNJZBl8AaAGB+4VBt z$*_Vyn z&_??T8qzaXOyB4bmC`=lh)`lw8iGuF;_kBnP#m|{g5tP~feq{~^R5k-psplqX@_#? z_8>i2ICl!Ic>I|vOINP$XS;W*h@MssBp#*;(c#j-idZ>lGD^@}Cm5JkJCIj!mzU*% ztqx@SMAHCGk)fPqMVeq&LfcEVr)JhFd5@=g~ z#FF!N;f!I-Pilxb(5g(>R{xe2@e;{&C1S}dhy~#YN|P5v{{2aG`x!(nm|Lc5>a4^|(k=>bhbvx}cxLd{`w3e5mz5#Jj{xw1 zAO@^|rby6^;LUI*1Ca(^;9JKFjSMe(+zKFCjfw`lQlYDIn0P9_ii&Iqc_99#a~ridb^C7wml?3pDnQIjq&aNbfr z-yNzp%{-u=xyvisx#%U9+ei_?4^u33BHIFZGiaYfWj!-UG65~88`{jbde1&gaP)w} z6{71U7T<7{?#%mO;g3Amy!PHe>3B!aPXN$y*ncTR#a3drzO|o<=0T|ZCj_tMLV#45Q{F< z`@J}Lq^`z5&DKN@$S7{yV{{(_3o(|Xf=d(!y>^N8Q8tQTim_CiJhVa z1rBnJ8e8Zg5(4dPfS$nZ_T%D7Fy7v{;JoM1rms*RThf-J{FMHs^5w_DLIE$MO2EiK z1_jN8p+(xSe=PS}MUP+Gc`KC+lHWLV1JXu=(V#T6ATYk|0V-=MDHG4~_2 zhD$XCmr^eR{z>`%JQvh?cl+XSFFKE?f;i)P3|pmnnp(;P>t#onqCQz=q}{sKSDYLmT03 zSbVuZKs4y}1kJj3H|}!=sX+@+USEc1lE3Z#-x~WG;(|U)CPbxeAEI!&#E2vUBCp+J ze-12k@Bz~&XW#f;w7c`67tknP1A#JO#e%gIsZ+kK6iV{7-M-k<>_3L{mSze$lb1^t zPWHz1#8Pa?`pxV{Zv~~Ww*rPSVQoJD0UZ$T6&$g3MWI#}<@k3HFRBGctOCmqY_g_w z#{CZk`g{XgcMC?Y%{3L3)-|<)+tT}JjY!ecSL@w`usWOyxAFT$pAY5CJCQ)4%zCHP zqGIgZ1}&QnMJd;F4gvsjZX_Kv0%qg0a0Bj2o+_Q=wk-gNMA`g1A8?n z$QJ2HlTi{@bP^;qM!r*kQ z`sVZQiyA~4^iv>zw^}6n_pwI^2Jp9eSa<64NZ(Rf{RH_RVg|M2BqnKX#3O%@P zN=4|gE%4cScz>nem@b6qK@27lL2D^K#R9|$2VFRc5rVTXpEz?!Xr)Oialfc>wwCVo zu!hv}VNQeV|Xb%79~EVH0~R z^zoc%gvSP0QqDLhWRs*mT*-iVce+yWx&6(*$IU44L=YG3q~1GWp##)Gy8i5I(@T{E zd=)!Psw;p7M~i-0gl!}h(|3kGJMsLP)er3+U_P@+Irdr}!%S3m96LMf>TDlpUe?rA zJGK4?XCRFK4ciH z?ZKU~uRNZxmQ1_>HZiAqLnF zl2K68r-qc+{{FuU_>GZjx;MldHef_O>c?%oBEmrwOXKvUgeLBfbqJqI`6fAz!#XM| zR7`3wjf|H{0;8d$pPzpYx{>BEMXU!U-g#FfEG;dIJO(kyiop|I%q)nsOhx_x+AkbS zs39B%4`twHcyz-j9QWmmE=`xYDA|u66l!iI_de=Zr}Agy_j#E^nl$z@I+1tIei5%h zmmA!AtAAJbHC!RP?e$(4I0FTA^pA-D<;RH#p;Xt#zO_f4I{;OKy7miz+nbfM#X z&2OyTa|59kC0VK^9;HirI7UPw*E z^>@%%(+i1f6CU3egk(ia-S<66iL~(+C5{}k{TOvmj&V2J)I9IM5O#Da9X&Wu&7v)! zeV&u3#EwciA6QDruQOWpP&qJ1Cf%qT#`ZqrCO8Yi9LPYUw~;>-CA=~@@~yWHIIFMt zdz5gUPR-%PC#abdEu{DC|09#rzcR719g#)JsyOWnzL3QM0niyQgBL=8!3_`H{3r#U z9{|ZoUC0|H9PO$PHT0bZy8B6%%cfkjpoq9G$0>Q-dP=6T0?s=_aJjU^?DHhJZfU&@ z4c_)F3*0dDJ35HnXR;j~9Ywa&O{wK=otWS@=DX8Y&<&Zv`z^##Xpgr1D6m_~y?d=^ zZXe=?&aReT;EP>S5!b(} zB%}?#O}q$xCa0CSz%mimR&$hj{`>QiwqEJvN zwJ-^S!1g)#O&=58AxiV)GX;t1bZi5|)%2l-J5L)%mZDy-)tU8iZTev=Dk&&63}>}k z!JIG5k3e5k7b=xwO%m4S0(XRK~yRjN)^A0~kvIa|T5y zGpg3L&^sBm#mssfkpQ&--Dn;f`1G#(rhG7TE@kgMt9Q5f#>(;N*moRDS zFV>BEbg@>>j~^+P*{=2%&sJ5cWsHX02d1kDw5gpmgjT6D(#P-lpyjVfM=FzK7D*?I zF?^oW4NSpk+svS@D!yA!z&U_eZj1mmm~at8qR-GxZWAB8_JoJp@kelxfaSkjCmsPP zioZKc6|?*9~?s zXTB}L&F1qi2Hl}WJ@7YT4`8JVx3%R2f3}OWN!%9m_6*5X4_lud5P!tCo?)eFQ1$X3&1ln=Do%R^s zgAb{Y@B^5HSqkg2pHMeiB4l=13z;7niGdr6{w|hw;Y$!#fCvfo3+Z9_IKVnSe*HlE z*Y09z!5d{{TLH)fegl|S*2B)eS3&~tWh|kp(cw z&A>_*6&|lsca1Ol!q13>E&rYZ;Vpy;ek&eULN?Yz1>gz%WAA{@Kcp5>z|m-!Ju<}r z--SZ)XFKs11}goHUu516r`rWB_VpggDB*bWh~YQhl0ZB%ukJCt{a-q%{W`oOlK!C~ zv%%kKV`$)Flo6)c7qUie18QE>{`4mnoE5whiA}&uaqb25NgqK_pZ|i{YXQiXW>4)~&JKS0-rK;qqRPizwZ`3-7zu5+yY05S#gQx&!Aw;Yn827vpg}QS&c=b2&;XQXz=U9A6W)Q6XSw!4rOh6 zY%@{rmxT!}uc-^}{|+%4?z6Y-d)=NOg+go37VL-rIXfFChQ5ZCP2eA^$O5j4t`s{% zP!9WxBT80Yo^08gE7>E}ot>SYUylZ84a5ciTDSzA5Q;x8=%-+PNda;_3L<0_Lp;!b zhtjifo5cE&^C&H8gihCqRm-HuTyJKZW%;NKmVYl1DlrC>Ev~oUL@?@kmXF%?Vt%|z zGU4zlcj6$h67$m|AY_fnr&`s4+&t^NN;V@^(=N^Kdv}g>*;h+8HuvAIzj(@hMG06! zEcyDJ4`=$pzO91M%FteXAV?r1H&>J;ELsvK)04zP%6Mca1z)6*x))o0r;hu{*UfU% zsE0Y3HO1giTD$`r7)vlk#zoHN?Fe*F!`Z@=5QcC6`+q|rmm9|NW5pueOht5?KDO& z?BqxS*>}T};S`+5cDq@!YnBX9>h_h5&D&wUVX4RA+qE>Jmm}fUNGdfQ zE2}hG@bFPPoLI-N8Jq3HK_6Mg0{O&Fznn31N10{5-di1F?3k9Ht8JBgEv9v!c0Gfw z?m>0gGWyQRn?_l-JlYpgp$%B zAxNiyFe2UEF(67yhajaiLpMWx=LURUpZE9v_brxdjf~FR=bXK-eeG-SbBzjD}&X3FB144e9y7=)OXz2ky(jh{wvWkNSE_@cd%P%+xXp_I_9!3+&P=t>aDARvvDo5V*-_^_#pE%A^V5f` z`y-v8b0O1bJ`I}i_{RM>cO`2$BhuYsG$_DiKKKJCbC%>@=&h}Wx80_kp4sJ81gdwB z?a{%3T=qj1r7fPC_UY}@j;Fz-CC~gyMO^QCMGr51ReS)2UHS{y68k&|Ry+oYJpfk) zv{yZc_6hv_ojZW+wQpvkpa8-m0p^P54tK)J3)EW7a8Pd!sMe92JapSx#afF5Oo11U9u2liS>PV8=TMRI?nqDGMwAVO z`Dp7^0s+C?jj3bpAmhd?fJ2g#L})VZZM1C#%nIf^Z5o0cwbabuzEYkc#cpAk&A|h> z(am-BbTf=E7|*n<2B_99y+Z{A%{vdrGaO=3A}ykc!3M)~w@js5D(pf!yg{U;KKzyMIso#r5m;xY+pSmgss8Jdk8DGuspu)fc z58~SVi4A58>*QIPrbO!+O-&Y~7S|YBX_%s(ikKFD-+qpPp5rx($=D=#R7Oiw-p>`E za#@MV8!euaAYKLKL`@aq@L;HegLJ)5T=&}XN(*kBfXhdhKk=8?w+n}=(R$5R(eS1> zgOL3+jo>m^6%~Q?z{S6=AVD1neZUftfV%8rJlK+$e=oZHnDmTvue+;t5 z>=H6M_!S0lCa&p?NTak5H%re@+%Kwfc$zh0YI-;S1IbU_-yytHk@K=J9wf*9@~G{^ zc&_CH6j+(LZV#pBT{zzZd)}YLAxtBb+X(T~xsK#naxDjt9UW%xe-mmv^Ui zl7Kw#%_#a*R6G}Zw-LIf@>U!hpEOS%uF(wWfz`4ie_)HTuEBF#I^l&4s(TntgVA?H zd&y1w_AXNI02w}ay?UfdJQt#b<{Hv9=1N59E1~3lmP{Dk4%`^6$7+JHwqh-My=40XVGd%`WO%*J8HH;ciTi!Gbj3D|8`uRbl-eNcObb3=t2TVQyT?`zi>0K9G zU88I*O+7vGq3;#f;T8rB&xZZBnul@~n8SPku@yr7DvjE?SHV-gnNht3hB}|5AMNWE zh~^vS|BUT2c*w>)6l(`(k{=TG#?asJt1WLKqz#K$6*ROrIgmM0LSYnLFvyMrz&0rT z@$@pvgB9;a#aMxeX!49y?qxg!e`Ta66a?7b4u)roMeh||OG8hs%;K$zO}#q4t9RJn z)i_WJxG+Ngiy?+BULMk{@X*QhNFT%uMYhkwl}AJN#6D}p;~!ehK9iCxcvq|>UM zRW)1VmLKKkPeo$IddBVNs8SdsO$AZi3w5`@DWzv`()}v0C%`i-N2*jab<4^4WhDv# zieKD#H6&zf8?Q%z$1RN$kS@P8+TQ;DyFgS+F<9tpob}eLHUy)iwWUFR@~bygOq38-4KuBJ6Sc z$2Y5IzZ8C>AFDhwnZ6qKdWRce}-WGPCd@vy7c;vO8vqD&DJ;Cy7 zM3_<3=!pB#`U)vOk#`hxD@X3#gkP%Vc z`bgcE;KREpRL;bzuStSetVx#?9>PQ%#@B$Y*ltXFi)+dnVDl+bPvKh?k>5w9s}0fK z{PhGS`{214AbC)n5c^yNJ_!e*V1@mW9+)SKY{!MYHa-KtXW^md#zMsc=s_mX7QyiF z=8)}nCY*sb>~{QyWz}Z3-t0QXy03@Fc4lE5K73u`>b~cT*3^>K9>EY8!lOo%91Z(% zauU0>vR~DlJ6{svn|vCy6pcq0{Ws5-2<`^v7r}Azw)+~=NJwC zsy^xNI}@6;L=VT@bK#o;(o0fmuQH1vXWqM#yx zNtK-5?i0uO16Juz-KBj=VrrviOJc=Z(-=Pm|8W!Ro}u=A6>N)5_VsA3)6*P6kx0&n zMA3)bU{6wAgmiFtqNscQp!%Tp!L<|%?C4~OwdSayqa**ALL5Jcmf)x;V}9OjwCc_C zH)Lve%ZbF8BD~Yc|9!*@MdGG2ilwgbzswlLy9_Kwy2AHRY;8G!SLy!mS}sOr4DsY` z7RIX}@rB~XPsRy?NJ25>uiO+LZuK-~a!lT@(o0DnR{Z`2FbXiBrTeB8_^4@Jbw9xw zVE?WzbpsT}f-S*1nx?iqxO_fiuvjvVWUOOBV%m+vsA&_mPMt#|`sB+B&u4B|KVLEk z1~(Kz&qU8$G=B()W`@#1WA;`&J}kKmChGG0-q!X>kMACT?IsJqQ`bTyWt8k8{RjD0 z0H=^3+rHF)m4+Udnoy(zIXEi@`lJEI+LRFk;a&$K0cfn>X@Dj2rc3_Uvo;0-Z~w%% zT)iKEfyS2}H4Hq&WOXye0h^8pFi%;*p~=zK#mn>q+xxGzE$8}&vkC5g@pXrrZ`(R= z{t!-qDIp85+V(LTDOP!|gsc|lTJxdBD+1hB1mC2(osL68j{}}k&Z?6Vn^*!eM&m=ItWfq#l8*xVV5XjhCIL8mAgR(l~1D%W0ddymnnLQwF9GLI;81-f^`+#lN2L< zDQ-22L+HhQn;X;n8Z`f2-wQ70pID80uf@aaNeMxJ61+iruc2Oaa8^wU#-gQIC1ic; z&%{rU(d=6tzWBlHW^vhIMJ=HTtVex9A`Wi<4GW1`^GWo7^w6sYU|cE=mGd|3KG=}@ zV44=AE0F0XF~`{=YF;``B2v)6{8GnzjUX`FB)!MR#vC&e0MdTL_?#uKQ+TJRB+#$IW@na!4 zo@s$wfxJITGWR8m5P}DHCjltvUP)jY(nWOx#bwCuyJXJ5QhcN+-vr-r+Yia|xk{D3 z-j$3^3{eT1@@#MzFA?xy&i!$)_)xj!LVCG02}O{M%nnBMJYi#PR zzAmU~?OHq*D~lbtp;9ar=t3w1MHLst6(BzzLYY{>`UT}sU-X?1oYxcsZ1TTmh@Ke` z9+Lnb8wy&lXkJ~MzX16Gh;1c{jlC=ws!-NKd>Lr{(UyF}BQa62sYYM-T5zKA5CHs{ zpiaLbe%CEJiB+0LSq99Y-(UtC27wG6c>Hm@W>IA}bV2~oPcnfV^EWs`ue?=($EtUv zCc?mB!yRGF>3`|izw!wXC_OG7*;vh6ohD845z1F-9*WlqAzpZpw}{HLi_7?il{n4G z05Smv6FxQw1|JCCi+2ULbpDTG`1h;-f5c_7#FqI#O~se?AxEu)z0@|g-=XF@?1!{d z98h-##tJctRAgJoY6Y1s*$1z5%!(6>0HHEF)4`)|km#{Ym@z*L^~#EodRIXmUJaJM z>jxj`4(|e$C6fY1Fi*meRP)b7z0MJf-}#W)wJ^9gu+9Qvg6Y-Prt4GiNahzpz~Q0@(OwQxFb%_?W!<5r4xAm=d+ZbZF&b{%RJ?Cgv$x^?vi zDFxq8;itf?lBAO>XpMzYZ3?x_&K7Hwn+xLKrn}AFrX&b&mwD_T&Mi~3k*Al=@B`nB zGdsYYP^|He_xF5U3CwCpu43iLG?s(Rq~x3&==(H-6+p=Tx50U-f#>IUn6*RjqNa@> z^MS(`&PR&FzA=tq@AzZH5;qp~vfAVrbF2NiXha!d{FQKzlh#$+^?B*wt-@rtz)zgW zw69K)WS|xfOan;UkgF@Bp#iXHuaP>)!nl$Q1no!OP6Cq=w}P=r!EXhSwX#-(8ERbB`rx_$fh>niLCK)|2jd=tsb2A8|@SdVLWF^o1? zqsdfIi3|Lf*I;XlgNRJ;-{cqqR;`vMwdrD7`3$FXb8R9$?T@);U+<=Ui1H8Tyh@GP zEZy|=%>AdU>GvJ_P+lzXCBUbc`~@5?3_cvoTOP9EEe;f7D40NF3I#v{1cuCJY;9fe zRfo+IDD7@)K;!t=G(Lf(q&+Y-*%bx?inYwuzeb_${Szf}IGM%Qehob?FdFk0AGK1q z3!nA7=grbJ1Ms1GEy_=iP_m0?<*l$W&IQVP*Ghf&ALS{#DomlEW6hkFAkTw@X z4v$U6zqj*w=JMl>O8~M(Jz~5L35vk|Zy1fgyQ~rZi!EL^prw8@XtxgPXybT{7byo| z{LFusno8B;3qgN1z99oA2S=4>ybwPkg{HLymug9o*V8fde5D=3$G}rSN|LN#FNo*L zmlz@DD^@~4^qW@k=3iH76goFXznF_+o8fO~wx!|BxQ+&R16j`8o&|4Ppfp9|f-WOy z^x1;4Tek_Q+%`R8sEe$@OYAE2CAtjAf9F^Z&{WlS{5OUIGANAfdBlIIyEBOl@$H~n z;sQ?JO;z#(@u*EzHIr)}Z(>sufc~$*fy?BIi1__20E_BA=n8R{?%c?UW*sMEQ2vW^ zRdxs@*5@%}k9fnSe$TS%v7P~>UX%d!;<27w=|2bnAy*z03_4PuPgH<9$=c6e&dYJj zu!UQ{zORGAIKg=v7nDot0HSzj21(bplw( z%n-hl$BhXPf1$Ybe+8>c2Hl06xYP^Y&0pvRZFvNHf5Qa^%iVBAulp(}j7iT~`xO$r z&&x0Twm+A+p$G9&!>Fq_(`jc2{LnT+UR;)MLwJ{M6nMS zgN2PF#raF{9bsJtZr;u|l~mSA-Gl(IT>$4gHu5hB(G&r?n5=;vvHusaa}PRysrfZJ zx|FsyFv0?b2k#b|zh-OUt^VlCTc9?#YS%P#Uq=l9PH5XEHlXky3%ZVmgM0~Mr5@Z2roSgJmH;qbaoQe>hYzD zA_eR>#D=mGlfMxQ2)(|IC!K3CHsfwd^kd?*fUZ$VpPq`k=YP=uK%p zW~)Dg&J%!LXFwaRhBqP{)MxwpqN8Gp7-XUFby-7X%2Nn^o(R>m7AAZdNh~n=0U0o1+ptMg?X^tADXT1(9k@yry|-7PjBh`b8Z+gpswm{G z!D=Ot$6YIlvE$(Udv2j4)Bqm193ip^<4={m5^w$SE)?XMq&@dbI;BwB*=QdN63Q6; z({{6iYuraA+Dr5vC!POrvx<$myk`8qVL0&T9nd7E$kN6uFArPmY^yG@fo-xy}pY>8r|K z{?=05|MTa!5DhbQHN)t}m^h=mVWOwngYXlVW?bJeJ|%*p_g? zt94nLUhwPT>g|7v(x8Wq+7R-gH?6*7gfd$HmGHm1O!$}fl1?s{F0XM?xYqQfCe!q& zVybPJwJ%qRxTG&tDuf43)?86ok3pVK0l*H9j<7>aRby6aSTHzez#q}ThztP!o=3Z} zZ6rj0%i#1d>x$LHW*s+Av0S`^RA4Nz8W_rhT_~S@2Bvs!{nKrNxC5+?Z7o##|5*RU zg8bBk@M}2FSw2JQtRs9~RoGO7G0NlDUWorKE$}ICPO?E4=K1q(T@Xws*hzy*-@_>O z2#`ldpUGu|SZxb&Wm(s$$#%}~{4h-7bEao}6%~&{xM3cv#ACH*tKzFl0Y#F)swHi( z{jd%=S0S1?qd9Cuqdp_~jf>3FgAEtwRs3}UeZMtYV+HSaHN{BxBO{!L1-qfOCVN98 z+NH&1INoTuEY=3MbBgoi(;WJI&?>2qmaW}gvR94}xhZJYFvoYF-VPRz-fD{g@MI4{ z%OQo2$D1cpUiYl&UxCYXt$tOswMC=`!={96Jo;O})tM`z(KuAyu(nd4ZEx^H!_s(6 zBy7HM+IWP%j$bEAhfGi0$C{+V***O{wc6eAhqDjBoZ%n4add(l=fBsfA8|O>ap`s$?&hnc;n{u7(5+4!J+_EN=JK zw9AxZS*}l&344r7IxkE2kTlQ?g7PCHW`p}aTAygs>W!Iu9Y}MYbfHiRz&gBs(W`2$ zRwz_ny%Y^eufDCH4Bj(e=)yoc(5HEXMO!R$KN|>OUX&p5(bEeMwP-44UM%+Hz9sTW zG{PKX0Ui8EXnP&{9Nty+x)}Us;rQ`s<2t?{^5hYGf#TtxHf0A;fht+RFKpjHx@=Y< zv8hZQ+kZ^H0RjAt?xo+J`?uFkU}rjS)L(!ef8a{(xOap$YJ^^&)qhcvwSM)gGY3v9 z9U85YDkZzIKJA&Q_H#I2FJ$7`nc~9FpY91tiT8~X-lWLGHe)QWFWe5m`F16|!KGba zXD7dwBIi*hVNpe6kLPBzzWw2XL0+g%5dsYDWk(@-+>o8K7HGH z{*B0M@Z*b#EhE|Uq)zm5T?0Mz=1#n!b(LEstX)sLrRUeV41!Vb@^zyWeT{2-1fy%f`8FL>{HtH{EHNiitB|3OnOC7C2Z-nu^11 zcpK5Iu_%HzO#HLKuJn`Z={<=MS}&6yZ@y@J2MMnF@q^D`uBpXQvR|7wd5Ltoe3su^ z8D;)4BTAz2fOlLv44rq-R>EqQwCS)B{1A{BHyAg(C!ThLeq@eoh2WwaNAF@hJzof% zqJyddD#Q%@bLU<`O-5YK{!@dZI(5^K&o>V1)vYq)X8=0-?%rAWrDVe zOp_j}Jvlws+8-`rc7#QljUR82hcvd5!j9@b@$Xu?I{Ikd(tzNFvg)kX_H1iCqyA6d zV73}KSKs1`=Pks$mw2@Ky5(}i1qRGm_>}UU%36{&OXS<8;@jWK_RQ~@<9ynUA22-% zo$?hpo4o;#Rr(~*9)Ej>4D|z~Js+U-3H?ut)*T@EWN$PV;&1o9GOsric}@L#LTEi! zFN!Cv+*D<-XH+x+7xHwj|9HAvlN%u;G)0Yv;RjmQ9-hm~Q!(=0co_10rTRkOL9e=G})NP-k5T+Vo~Euz0> zmW}__7=O0o+yW3cUulXmY+ECBb(?6JeM5=x^fI2NWe;@64u5y*034nF2ElF6M;91{ zhI#~;Xi1NDU;iHnE^Bhgy6@LlrCOX*KiML&U7bcK9 zu){-(iWk;h{YQ1Q=3!mOlRC7WxG?CZxKI3cCWBcjq(drsNe&umrk%0vll87qJda>| z4g+I^1|*bvKv8HUuK#~!hrqbJ#jmPs zXasi7F{wKYzJsvD7`m{JN9S-+Vrwu;sbij1(6aw!0+iJANlbt0Nn(s*fU(&Xp6JRt z{TxQ6OOmOC`k};J(4C&ES){j8dae*w+wRTLyA$>EW2~@nWE86;Pj2K5@^b&ICNC4l zN86v;HvmA)v6LeXfs&qe^<(=XTD;|-Z+?}Seqb?b&B>USgf4MkN6#NDh#BYHhzW|vn zbJ12`z}y(3fx*fupz&M|EGlYTRH2<+t6h(^>pt=E(~EZ>__y2X=(5v$^p*CCjKSfH zWl|Vfe~t>RRNG^t<1g|;4xfUBPv2T(6;^6@T3km|@*#_Pj>vN#u^|=>CY7 zMERu1>J|RYtmnAnI60txQwkP7o>k8IpBcFf8)8rpx!Ibe3)uuS&f+t?ksnE{VIq4~ zHxs23&tZoMTi^b-yz^`yRrDu($hfj68%5@Q@n!T}UDa_H3|#`J@HHn8fdO}b?hwT5{&_jmz2w_tv~(A6Nb0M z%=M0|y49O4S(v%>zZmg1`$Rx4-L2hA1m67o6YXAzRpbQws>09U2O(6E{>Z1_ ztr>P9VirpL;*}^1`zoI3@k5f|pGte8@4z<$CJSh*#y33rtjc!Bd? zK?A@VR4-cw2{|wX;+VgNe9l)dOaP-}N?^H2U{iwIC(dTnd2(}X7Dcn`#JEsON zXo-~~?4o*bowaJuAK~cphFp3Lc1nMaECzC$1YAe|>26QB>~7bQ=IxAy2nBc2MpITQ z`|+P!vk|o`Q01`)vC zKeHEg(QeR4TNDXEjXC6s)rE@xH)%e%Q`_^-6DnzF1=FcF7HRR@zsKoQ4#lX@PO*oL zSBfz_gz4iLq~QZE5Np5sf?#Q!)F^6iFEpz`_-KV8o_i^ZzRw8s!`oww2_DHkqrNM+ z*;umP_Jh|`ct4%YuYs76kwD~E8X$r=*%+dN`Iseb{Bwd zoq8sZ4a?cKh>~aDarjru25|aGm^J%{hZ#VZ%2h3imoHv~my4d=q@iQJZkYT_5{0p+ zkvI%=@PIC$Z)sVPH?+seuBTk~+9!%!y7dOHn8{$BzJ0UR`Z zsDa$RB{CWM3(laU_upQCQS3!wEnxT2BlvZs0{YF;?A9kK+}zhXGh{-~r}TcfnBK3a zhhZ6|U&Linr737VollItb;GA#LZO2 zy$~NGRBl>MYzjRJI*!!K*)jQ@+R`EgO2(-(n!K=hiD2Xk-y^J#q+0!A-0Xpg8j-MQDV4p@vd|B)vlW( ztbIIW=N}WE+*myTX8>-@U$*UM)F4LbzOuPwIRI*eZtr1*7P3sqB2MeK@r%eM+IAd}5z2o|Gc-Sn@R}!01S|*W75v6?| zU1pnt9@vITvx2!DRM$hN3;-=57Lb3XrU6_1-`M&2GxUrd3Z~FArep3*=Fo=n-OUL( zS!LI*`2iGG@L!Cx8WC8Nv1gVEWV`RYy-|Ou0r`5G0L;ns=UzGtB1nTB&MvNUNLhrv zCI6k;)$IqGK|rq68N1r^#9czd%&(*}AK1m{%c>W7&0y;Ik1;+USpW_#ml#9=bgD4$ z5L@q`TqHC9mO7PF;2;;s$ZjgA@^0F)C)E)KpX=GRZz^u%ObJhw#>yR{|Jz4KvEfH*e@vXQ>lK zO)8{N#CAEmaR4~rC+V+yhx9-{M3`%cqT zk^!*CwJpx8+sVTdrmkrVmum>l*R%$`{eOJJI!Y06^K!F(O7K~Y>Zm7c-BY~OF*mK% zI$0f`M)lO_dz7y3rLVwUKZ>f9{z|Zz3RtC^G4RhZTvKx`|7p{) z18bh{xU#eqGuecwO;~0sSXYfI#)1IA3lH<-x+UKWZc@m?sGpyHT&sCAW~VA-!qPssE)6PdpA6lzn)wZlOkCjfU?#o3-n~uHiKSYt;1gG6mK%~TzM!B$nOtaYPE|uk zXOJgGeC}V|d!81*UP8VA6;!cm&I_t-1)uNUaX<__F`Y6p!W(isDkGo$mKw?`{v-=B z8mP>wueKm)qvdq4EkA<{&Gb)jzOG7g&QvzR6a|T1DLaq#)Msd2PKUf1{(qnjl+)Xw z+b@Xq^R_`IuNz#>*|}O>>kVA4xHX+%_l6jgFQ)^TVhs%YxzuiJ&x0euuqQdp!kQL_ z-K*dWxl*dno87pg`jH_UDq+`TbQU?{a^;r~FnPhjX{@MVfk=7+bUPUs7G9G)E@iVb zQd3il-iuaE2NPt$z4Rm-dhT69lBW%w9>LG4s5v@7=aRb}-$^IgrUPG`h7O{w4+wtVn8s z32XKe)+4ocny`<_jy2BF#D14@Y|Pa=P!8=E&L?l zQ}=+AFEV)u*j10SZB{w7EAICmcCW{+t!+dV(~^VE=T5h{8J#C6jM8G)r9WRdrWt_QoHJTyWDlIp zX64o*>d7=)Pnd=&Ot*DGU=n&F2SVsqoP%TcKl}JHMxN8Z^R1oY-wJ_Ybo_Di9ATeK z-sq54*)~vcZg=yP29@arf z#f*H09}xH@w|_;&%G7W1C4SA)Eo9OJnFx+$%pBzs)#5Ff=*DAD2h-~4+6m`Z?3lF1 z55aP4k`r!^4FCszu6X!%w5qR|!u88Ozati7dR?=zeBTdG>n*l8WU+{(EuXGyXoFD zL=!SaA^yrMtSgdndBfGLRhU{l{ho-G74FmOx;lL(X6BHaD79D;vScFt_->;m&=&1_ z3hBGbUC6hRf2IWz%xz zUBEew8n2?#o%}HRmfA7knaeKV)*8uZrlRFc$kShNHU2kZC_4B#ZwtMd=IX#+& z=K8*2GZ)iLLqM6%-|xD=5<9G16jdI*fOKP-q4*lL94&mOxb-f1$LPCTX>%a zsc2cjZhBv(PUp&50ViN#JcEaS;xZ3qF{l~jN#YRc**OYkA3z^*>ci*n*9b;lDO~5a zPaE-+C|s}kF8bCz^AYWdk6~rEVxhWvG&%l9Z0jcW&0VBov)tu^5(D%rAm|u8&m5F3 z0oQmIhO+xhW4h5BT@}Cv5S^uN0eb2{XHlt!;j^98LseyzE-+6cO;!yOyP`cp_Iz&eApe-3a(VzVEW+69B^njcyoJPJ#hejf!lb5d z^o`C%H8g!!=6LrXTiuKz3-e?)y~<;T9AV_6 zsne4q393DvjAHt+@Qa!QMSm^mF>9|!H8 z-U?4)u`N$EIkMoNWGmdk_qTiX^5w^MTfQ?u)xBC%u!*5crgsxo?f^SEpfJy2R{0p= zB{4I6kZ27~0a7^KtAw{dXub1XGs2nu9>>&D90^#=yKu>o9~yA{rbC?HH)?LW^0W9bC)5_?8MvrP>pO-qZ(-)wsYu-6a<4TUPd=4YFl?SM zHDS@y|58dETiW#(=^4agB4utajR8j~rA1Qc3ix}mVjQ1(D=*%!b5!}f)cmSB<@4z& z>e3=ko_5-Y&x4xM?6E{Ec@feZ8IwI9*YBb!I1J5s$3$D6LokW}zt5kA_Wqs?)Vj>l1XTMJZ zr#H4vq(|sjzEo7OaZeq(i7YKGUF}i;`xr>|+u0|Z3|d0Wkg7fZFigASen0ST`HGomk5I`a)(EyxxdFzg$+(4;NglKJ!4fNy&2-3I=fA#ixYWN4$c zxZPy}M!T~k<({0)+)x@*0LIr(q37Zc%7^xtv<4X87Z7nC23z;r!5aB(W4rfH{4!rL zP+KAlyZ43l-7Fdere8E)|>Bg>2>;8CwK_O-pTu-$HZK6j zTFMU;M;g35NGaW3_<*}BwVr`&SAi_J|47QGx|ihuY@ zGwUkYWv=ox?O8=g4IwBxgY;seooa`8eY>meX86u_*ILt&hHAcM8Z$knXpcOJSbq;! zbMCj=kRI7lIV}J}OQEd75{*n~33!Xko8&t0B!Be^iSOXZ)Srqhtz8{ok9=AF<$7Tc zHt$l>=jOYbDna>raORbj>n3+1^K(pYw%U(9LSdvxM_ZqP2=|mZM);RX)R`w+ww0aQ zKiVF?O?CXT5e|nN)xCR~(qS70^p8X%xbOPaK@FYX5gl~Q(u*PibuJPW4G$%;l5yPR zlqt|KK|w}cd96;XWm($!o?RSA%TCX%n`VJre9<=Vcw!vTLxF z=|#D=c+>Y7XBtZYcdvkK;DRM{spr8XefG1MVki+nVbHskMQ1?JAE> z)oU7M&@F)hpxI|XTvC(jSV&FF9Df29Pc39J?qo6kBzxaL^EOPQu_ZrgRX zPq}&I-l8yEvkQO2L08C2cxP>yCmDPBP4)K~$G~4VJ=-}aly3RHt_Epk4(H1qftuQM zxY3z~*vnj|)Q;ZgE=R}x&oPoZ{!mh_t1ogVo4h1uau|w9F)CwyI|lSKd7vDbkfqb( zBnTsjbV`xMSryzoJRJoFwJd!NsusT8PROSpu63&*f6K3AY$u7#&4FI7XhcE1qKj)P zjIGps>J_K0x^eYkcef(#C-r$-Su2H|f&BVO#pm>*6FIrLA>_Y&61$y5JPUWOi#+}{ zXG_XVpFUP+V2~2Sq?(@>ox^hP>=QVANE+7gO942IBP~f?1NkBqLDF-gA5H>cOFNeP!8W%SxXh+4OaD^Op$GmPBK#0{h3{-RNT}?)A1OjL%(B$4hven6{*3Hc5EXExnXO$wOhwRT#gU-Lc2Fkn2@8-R~F*y8zy^m){)L; zB}3!y(2yvZ(+lz0nwp$SA(27Af3j~c4+e@YVK44m)M&Shc^coO2{V7u-PlI^*4-OM zKJ`0>)Xu1GwWyLeT>3t9R87KSz?s8aSNf{=s#Y4)u8q@XvxLX){R89&Y;2R8o6Crm zEk8{&x$=@o3aJp3$nR3J>^0Ci8iPy*9u>tFc|fqL;YSX>I`LEWY#PenJizz8zP!8@ zv_B-8!A93j*>p2-MgapP@%e*o}dn-L62YJ0-z~K{uIA=DXtgqq|Zb zp}NZjvvE;;TEA<$vosW#JZn4trWI^m`OQ&4>9k#U*g`eo!%S2yYz^{V!)F4vH|d@$ z+EoWv%WKI8SxFlQT2>aTs+JsfFYL?qOy4N)3~`k1DGq)MsU-b&&)6~YsKLgjQkD1h z4IhLA;+)?dvX#EM-!chw$%)Mmrdz*8oB}?wk$N`BzdE*m@G}eCSHA_7KvLxibD=*XsFamgEQzvdl41Wy9U$ST^k_G)Tq`Oz5>Kz`^i4| zWeK8ReRY1}BEA4Jp_MQrS9qS*eF5ScA1yeqLJaKe1Ysj@_oqaX@0W9-j!HCLPoSFFPqW@Fvgo{_M$z58sU+Xu%+$C?#c01wF9eEM>my#<3Kw> zQ3Y`qbEL@jmPTfkx-?bRB6EUL@Pm{x;a~-Wg~T=NQQD_~ShBVuCztTJSKi0p{pi^k zVF#^xRfs#06`Ahx+Ew3JTMH)vcjj**K$=ha^oXK3Y6a~4DY0(OxF&c1v_J$V5?^{R zhb96fxJ^w9j-T>bBQX_#=56Iu@q%qN_yCGw+YV)OX6HALC7Kai_P5U3ws`RAYJJIqL@!;+m^E zL%iD&x#^{q%=Rk=MjzSFHYX!=V!l5D#6tnqeOlbfsj{yM9qW4A!RAZ|mh~b!4=~>- zH3G&sM6HQ#Q_O1zDt9qIk4>stQah_c_Ebde@^Vhvh*MIMcai3oSE{capKb4ZR@ooUEe+)UV^2p$Dph*5UyP%U?wa6}4cidXw`<=kgla#Q{ z=fa9%VMljY*W(mt;>H`59vWz?tPzHFvbtYk_h^}xQ@mTblQ&iUQWzUJg2E8<~>hTR{Ii5V%ZE{#`BDo(*Q}f7LB)IDTC^x z=Jjq154eaagWLAU(>$hm3DSDmwZ+sh%Knp~uro;!*A|_W#N`~O+;2DTz0Sp22C-z} ziiC^+e3^;ZI<8<%%Pj5G)zwA(i3Z}q!-o&AE(qcYV|IdoRf^~(E}sA=L3Sw3fkG(2 z4dDXG5hoXW*n=rQxPI>|xO}oq#%+mvqARpfjdm$xxAl7L3gP!Z zfcvU;dd@WW4V_=NL_T{M;BsDP67W1eS@(?N>gJR1VWzfl$S!bjIBTZKM#5C66U6`;Is*c$^B&aA{A})Fly(PQ7i8>Iyiy5oYs6jiP4* z@nJh)J76ESsP*W<6xGM|x58U4W!;E~!-x^u!q($@%J}uS?wiPtDFpNAaXPF|{v*eu zeQ;pr7W}joe3e1+Yx=b2$=4N6->&wC+j!7zt1*GTgbbD`+z4?Vh z6~^CQ0PiipA>;5a_AZj8x?Y~O6ku<5c6!43axklZ(S)SF_fS!kqt9tK=uUFU;Ls2S z^0TC9-qmiWCI-@60ZF%?L^d#}Pz-vtZVi24UeKq-y~TPppHACA?YU5~z~ubumYk;% zwY;8+PQ>feVT}a)?{pL6lwpejlA9F;Yivv8pKj&m z^g5!p?XVjGl!(DTeyN27`---hn2+oIgv{e2j@PIsD)xSD1yr8h7Ed@%*|hyJ`vfOI zN&#?aWpCK&PNwaov$t1B@iR-Ec%j|dDBCfMOnOUcaOCJ?f!-sJr(Vk<( z*^%BiH{KmG4wb_QrODw;H=wLN+dpd+Tz~Ho@g-42gL}{AVU?$YLjGnvqLUqNsUN^l z<2*ezwJ(yLg`FqjdC*H^VPo?|K&svC4@KLlcV;8Q*hbU6t#PQ;vu)Sy{ipTQ-5yV~ zlhbCmdtVU|BhL#8Tia%6Jz=x^^oB;C@cRmVEGKQ^*Q*+ElOK`4k)UIe00%^XWN+tG-u<&{MdHY>n>9x1Qd)9hht_OWOs*WgH*8>% z4$=jAY0QN5;H?2f2vUek#dGJkJTmFC^_Zg7le!c6mleX zp9n@5A;)}eg9q)W)LdNCTIX8(l!)p2Bg)NhzP{YVgMBwWWZV?ydfkb-of;aKrD}#W z5$LElW(jND`gwm|F`#Eu@vLz<{C69 zAF~)N|9fzzD-Ta-gkbDsOiM)k>a+3i`Slx~uqo_ok45_U;xzCD1O$Tp7o+%Jz5Hsl zB0brd2A6LrwSgDI%-;LcAAIMmke&d73B~baySEC*Mp6zMv^Gx0ft(6-nwbZtC*hmi zUL|R(8%t4guINpfIJA%2OB&;otX8aeL3HvXFYZp#n=?AA)ny8$WIS{HuK(t7oz47M z(8FBKjumFM5T=acViiZcD`Ix6Q*5nRB8^L2^8)TAasKQMF}Q41)XP&+A!Mv@_oqgS=pQBRO7_xSUw?mOKqIRaDpP^E zU~VP7g0Yhkp_@re=)Acx%fk(8vKB^XX@kjQEYfaIYZwbffn4oU(OalW ze7_t;H3jGkJ%Mc+k&!>RiWgLrln!vQ<^^BXcdU?zA$O@NJL?mv0>$gx$M9_bsI=Y@lhwYg^?Idks-Bu7`V^DqfwY{_Vw4vc+ zHBVd4_UxTHMNc>P^W1?a;5{90q;NKbdcUFW@2)CT;jQhiv*SiA@!W|eIDB`WTn7&0 znj#G6m^q%^wO&I|C<&ETdoadK^~A5E(YEhS^l~L2a9xdz7ID;(Ubo#pe()H%qo2gT zfbVx7G(zA=dL)y&!sXaQGNSZ3QhX7+!pe6M$R=&tCFRq-)&~#q|Di(0Shc_a7MdJm>GOg__%4v zrwmb@w`G^b6^_|!scfKdJZ(M8B-j#>N} zyb4|&LHoYwf@HCc?5{JCImD*Ahdba5<)%K1^=s_H7G~Q!aiRuuGVi2ou9MFq2XFGo*Y0s1_KW3(qZ%}UJlo90jYY|&{z?Iuqha1nnNbiE$v`+OhA`y@7rHNPyjyiRRQq>j){9;(>1H*I-` zqQ|1FT|5S4ry!i6-YH?DHCYDkU*fo5OWxAwJ z6(Hv>?k+B$jprpqjUSNh3lx#ES*s|f&q`A;*jM1^(C3NyE?}Cg#%zHa*SzCv(6sNf zMc0cvzqLE(-F6lj^HrdP?`~fB=6V!2cGzyzV=Mk8y*g&FHbA&7YFAVgmAvn^en2wC%@N zizmAtN9q@WrKQLL@YYj~B8dg4!^<^HJi}Tv2{TmFe)#WDV(OBe+g-8K74KVh1*w8q zp?wC)E?%Ir_V&e_eDRfH1(DQ6Myh7{3$&TGfluL6M^c+fHC8UdQCKv$*d1J1A?!qZ z8A?uxW}iYX9Z|t%{=>Q|Q?qgWZ_14FbrR|Brze-`A45h87#AhKy!qd7yv>z<7(<)MxjmI4R~kb%0Np|AQ5EjFIlGNm#VM4p~u5-f_gTT&)Ze&()$%1zxX z4TQ#wiU(Y?C7OHtR?q&2A2g2Nc02*{xyQ3DNknum^BrJzZZ&KjIKu0_ZQkwn8=U9} zQ7*w99w;zlc)V2Vk>gvn@V(6>UpYrqR?yAE_~vJ*{PR}m z*4DmK^J^fXCRf&fV&`PzuL=AxOZ&X z-zdqnBuv@GxEXr<5D-ZA+S&N$qi6T()t*{bHh@iO+!JIG98wDgouHxqb>=c%#Ddhk zBGjrvHpWx8$E`bFpDTXZR326@_D`!S`MHywQ=U`S41SY5xfa`Y+;${d^^XW-B=qZg z+og0k4!CviJ*k|W4vDYlPdr|S`u28qV85zgICw0caP z3fL8a_rl%Xzly5ieu8JY>#DjY@mX@-JI3;tPhk^H_5nm{W5<$YvpQ6)YWS9s#z@0Q zY)K3IGZi$H$A?5QDrCN)h!yO{)oDir5M22W zq~l4V2i8_CM|CI|Ja_-vN<4E@A@b{|#2dy~0TruZRF#zt>&r>2`tfBtm%)P{T`2f< ziO>HxWS`Ak9!W z4(hTLpUeJ=v3*k1iXZ!Lk1~Layk>beyv_AETp-rTBcqJ~^z};THZR`z$}r&JL>ucF4(5Q0gx;w;&;anS(AEd=C>YxHv*Q_qeNjd!nmXcUW;A z;LDziVdua_)EwgyR1 zyn;41G=Lo)fh|8f-L75!-t*{IUB64r1{eAHD<63mavxRCtSLN!2?0O4%C>JzbE*YO z&$F1V(++$sW;dRp`9?+8RUyS)fKeEDHrSKEOCtdpP2Iw!NryYzG_K>|s@AWsu*ygD zMmNyjT+kBNwR0c?x~s!R$>!84iT;a}-#~|pRPfL(*O}7cYGZU@_TH&XP&V0vreTfe zg&+I03Q)stgO;a!(d~-OK2i&dw)}jTc(r$F0|Sa#6_x!1k)XPt;tebprXRb0umj6ev+6FQ@;0lhH63FoDz~ z$2{N;x)C=6sLsby>S-=FfsDRy-+qPW%`sPkXTB}V{t97sq zCq%_ll|WLFQ9Crp7WmI(L|uNec%148I#ICeCBpU(%(=i^VKFh$A|MiuJT&!IYrY8? zvHCr0v*aFNXzT60S~i!GlA>YuZm=3s2v_KwqvjV7{7oGY8sQx)G*O-*J0qX8j1(60 z27-onU`f?0JyCzayiBNUdr_VBCAK^uMopcy1oSxO4RLo#3&g8OXwIgqZAtWtMeVQX zl%^N}bD9mhMWwC!pT<;MzF0MjO-=Mv##Dp^6N&OZ?>W)pd*Auc+IPycLA~d3eLvr6 zz60w&-z+qah8Q4(UtxD!kwWe^u&zfrAxdBvU(f-t_Up%_ipxjrvG9Zxdr(=~o{aBp zie6FuG&`VJCm@PYWQG);XK}Gs8Ku00GUiBJ6xXcuwz3L*Yp@w17TSpB%NJFplz0>P zFzva1)l$^P_JocvY4$*o!8LaEddZwO5z%l>s)y_{aNC80i_Ag88wwp#xlIy2 zF=aSWQ%^q@0fxfUbFqgYD;mX88~>U{v)WLd~5)u875`$$Em6yg>Jjw+}n=74?~1Dt*Sy+ea**^oCo<|^Q4qRB8TXd9l`AfhA-mBXO!_rGtrBxXX69;?3XwEJ z0kPeNT7H$<{REM|6WwgrT1tw{(lKvYbtw;iSew0&AAL)g(7uCW8zTlkk?Wr_w%{JV z#io%;BNB3n#|K*pM(@B4p9(a``pvT&dJjD=}@tm%a_?W3EGXpxFn2jQE6ivuE`f2 zmdzXD547}!{P+6@*5fMyXIpzaBRciAVP>f+!6^ln_qYbJO{_Ql_9nhy7h$C)`^C-> zwVCS5Qc^c*BLFSgbxmF5yu8G!^M1}L?EBM9XEB+;uF|r^RaRcf+Xw#PKL_RR>=gR} zL+$09$6C1+e%a>rZ0ZNatl`xT{KZm6cAns`lls0D_%nfIM-ZzG@LRD-Ss0fu<~B@Z?MDV$yhC`Puw_9$)c=C>UfNJ*Sy^ zk*HXLg6$4-;V{UFdaHMt^&v92GV1n`xpe4PaxWuu^BYPdav35cB>TzfS#HoGNs;hI zq41Hlpmvdfx6jak6`^|Dm4K4tUx&#%#v~*>?)w2_hxI#I=OWkn8SAU`9=E-L&C_t* z^EH_zA%j2AlRMzug2HX%_n%ZLNsS%0&XrrRKDT@vVZt}&EfgkgH7SGB?~K?@dWUi^ zBFI~~0G-81^`bk3QN;Ft5I8;L+t*EItJ$x#uKkKO{&x0}(%3aGO?ennoM^@(3etrT znIkD0!^L`kwxSc=R%X z<^lA=8c2jUBvuOVeUs@&eg`ZaRra-e_S4REd4!Nd-Ap}x@#`Ake3ZI%hDiuAcPr&b z`lC#4?TKbI5g11Y1+fa)8DSJx{hwGM<@*KBvSnxGV~UU}V{G=c0#&urL26VD77E5V zhPZf;onnxjB0Vz#SXNMt4T9(&Tud5mUB5VFF=tL6Hz-}o zo~HD1zu3IdVBpQRa#F{DAv6{b!VoAalSikY)SBdPLEUfB#4@^Dx0tvPYi~|qUL+^- zn$Evx_Sum>B!FyxV}=(UY)cOjtio?`Rta*06MLAUo9tCZp_F{~@%I|>XdrP4KRKo$XiX*Z43Bt;6ZPZ6o-R-yJ0AaMoDb=t33ozVV+)Wp;esN zr))p}Sgw`Z0jEF1>y*x=UFXLN4TnW39@8-!rQ=~cQ9j`zj&f+tPNuJz&}pzbzRJb@ z#FUiEI)fP2d08*M?@4rh6&Y8PCG44B)BT%K@41n_Ozq$W!Is&lBv zZg04TiYL2t!XJR=;@Co`Nf~(>v!fj}0Vnhi;MmNLaDRhw%=)C}(Xd+ocjgXAHg9X1R;Psc=dz7=zzxL z$m4Kv5PSn@;du(x*y2DabR{MtgZevYk$&cwnpr(WU{9PngHiaUVSa_weCwkj;SER; zL$%zx?^{7BQa=2|1iy4#ad0Xz``pM56 z!z^~A=gTa7-Iy{}O)A}*E5*Cb^LtbX9Kdq+NI#-i-m|-_m)YYR0 zhlYV3gjS=}Oh+o9eSmFUadab1`vyJI&jj1T_6x`7?=DZEHI%p0Gc-N)C8NX#W(k}j zk$bAowXY{WPzNZFt9DD!Le598MeKXNzOYj>pg z_KGsdgJgfA39GaDeG+NA@-RQk1!M$E65jP<`lGLkHHQEQjq(>!sE1o~b%Xu1{ClfF zKp?fZnWQ#$;KmCdZyI!-!@s?7>hW%_z`i#r<~&#UMHTd0!jEOagf@!zf}2Xm#N!6$v-$?A6N0{tWy2GdeILaFj4+biuGm!}V zxDDZV1;tRk;`qJ2W-mxX%ZL4G_|Inq2i_G%d{UGhp0~IsI?mq4P3~ht;kq@ydjwnf zT8*9k*2`>nH?>shm1ZVcG&)NOoC)p(xz0DP5qgHo?ovm>@~W*4YOZPZW=>gJ9v#n| zmg&-YTbF097Czgc(7uEukEG1#>y;sZ2eZ6FNq-&gotGVG43b6mgEX}aej-Wox(Tp_ zKbqvLwR^Bjb-1^`pnSTxMd$?~J*2;=9y#I~{%Zt21A2}x;lM6|SJ+~w$LmaE@}0st z3VI#~X8!B%;{a38?Ac{f?{7~}-xnmBLHOwwWLxhr&R08_crStm=NljnZyq|sW=L{| zD7|p5+}fQKQ;2{|rV@pibwYKOPW2IkT}Jt*#21wk`81et9FS3hTewR`tqOKKdb)d=zYzsmq7pCk^Rr}%YMUTufxEk z(b;S`&oxqqqqn6$eKgVO_@0Hn3A(DC|7BhObwFs)ujVQs6CKX?k@kPbl*}@33GT~7 z4f7;c+ylvcfy?dc+h&;4y5j>LoOcO2;ln(sG4MB(r#nG0GTgnxe3^<>c<>23wt@!!=4qn_r~Z<(K#Of zb#3?twpQ8k8!tQyAg@#5)d3pLFHcCvJAIGxK z^eCAMSzr!sb4Z4=HLn$c`i)Nsd{Fn@G^4irozQzL2;WrDjwpkvJH=GOf^p6~;N=*; z|7vxWvIMb8Pfq_D18Js9r5H46BF7`%kfVQ=h_I4V=$>P11ngmpiMvp;<2xw<{>z+HJTy-qM?LP$VMxR}4;0PcuR!q+tH z+WQCl$7}Bg1&e<{1M~+tv!uCZRsDS_zD;B6Xrr99Ahwr~d?^9W$ zC>OL4m3@m1g-GS0`aUcmeH1MDiuy}qRuP59;q1|bMkY8HSyPki={YnyrJE1S35G4l1r3=xhf<1$NOs%lvxl~ zmXD^lpXC#uwVpZEyp3;r!;>osgep${o)^03kvL1bO<7D)NELL!V^=>0;THx4&Q@3= z;|sn1|I8=BcA#T$H$ocQ;gpR1HAsPbcdmdRw^ARD;i9S)SF;ub1~!z<`iFX?x&hLX`EIwc0ifzhnYeh|G#MJb-H+hQNua@8D+4M&Fz z%A8sYXCmHTwnCnK-s#op|MIx~rqOU)KaK?9fA~pTD=>`|j|O4`7F1J7zh3cBih33uDn`)nuBLBKJSeUHG@!%s?NMxcsBYY) z-kyouCLQCiFHg141MaH$VpAG2)&X;<1xn$s-|}T};NIM(ZpR6sk)Im$pJldc}K)DTiKu`%YWhT-S}Om>~KSIxAK zSPS^7eY9^S!Qe9<(kG{S1i`w**{KTz6B4A9=mlkYuDChrgxx>JT+M>JS*VhL3oZ** zrc+B$aiqSaI#zu7s%S`4OD6>nv(9;=xbo-TeZke*NeE7% z?(D1qmeLh-hHyF`apUvRVpATp(JATM^DTV5_4DKjn!M+*qJ%-IkZ|$z33E=+o6b)r zjCJI#3`JV<)#p5rWJlTh3T`7*`|E0xhM@3(b@HR7Rz5h_QST`c8czKi4zY(_j{UhE zx;DR=6aDiD?g$A%mQM&qeAF4z7{r{#^z9G9RDmafeHMJLZdX5&#CW^n^l=aHu0@?- zaJ7B)I$Dr5&GA~rm8qbSBg~30|ZKs^ZE%9>Eq_mO8jn_4yOhuS9MTWLdRT=g> z7p!zf83#;>8xyK!k%SkI+mc{TIGUrIK#_p!N9;U}xq?`wQk`O4HK%2&g|;I;4QQ9s zqDFhu5npyV)(fbMuJ61#>op~Lbcb^UsbUidpN8onF#AY!s8w0Vbd1D2{yX;Ap?g{N zcWE5)Xoqqmn+DV91+G^hg=K`{-VCeLF>?1JrKUNyA|PU*Ro7GyoMKzazf=BN(`739 zPH`(ac~WZqmyniZcaWOLxziZ=jLwDhQQSrMvkN`v^UbTfysHwX(Uv2wwtS-?;m2=D zXR}DvsepOlv&8QLrsOSPWYv*0fotl0z~u~SdVw@fXj=>bKQZkUR^cp#gJucnVf`*1BW0t~wFJU5n_8na_ z`$}g20S?+614E~gMin#5m^VlqJ~XsYO4RGc!qn5QR-M4KhBT#73c-@}L(zo;C4N3U zqE68ZGh=bJ+yr^>dUCSrGn)_&0@yb@z<<}3z)O&j*n<{ZmHrk1?(CkCXqP5K<(}st z!NF{l`vAYyxlxs6p`;J&9}~9Wla--_3i~ z;~&JkOEp*goe*il=<&+0?H}ygi?+T+1KWa1gH+JtPM{E4Pgm_cw7(edXHI(%_F`Wapn5JpMhK9U6HnX*qixiE>cdxoAl8yf{=Ai6 z{MM)GDw#+*jx1$Ze`hvxu*A3iW2HbI$FR{s#}rAf zZS)-ux#Io9-E02)+VI`$`)4j<*vuG6($B4zZhxYSlGA_xeIG6VE7}#`4)N2(X^XfD zDP6^@hk*>ldsi2`0PJfLw|A9o2F$yz&^6Rve2v|;?V3O+V61^ z96pIPr>xvcs8kEX5#gp80<l43Utptn)z7diC)HrLm^?Ccaof{M(7yA0jCe(O*K(SsYHOF4TQ1G9Jr zm79iIBsaIFnigiVj7@MLp<~emoMj<*bY17*>_J7u3!Wb?RQjF}#M`bPPCa^Dt{i`p zw(AaGhU3P zh2FY~-qp3;=d9y=TYSa=zKzdpEH*Q$?s=si_hYv}O7R!DtJ4B1IH4=$$p2Yj?TMvRdBjr`~Ze9f9&=so@PxaeAfh|To*4$VI(Z$d22 zP#WA>xWZ)xamg5owkJiNS38P4*r(>!PLWs%e<0OEk{ZtM?Qo9h_JSE3ozU1m!(jQ3 zqt>5zOmoZ}nbYbM7J`V9c^xWR)wP)(xpz7+X|{a4dU5+wSXjh2Y2UvnPA{)0M^qvG zEQ4wmO)o^@<@To$XX{7Ii@o%hb3Y;}fFWP}r@uuSsWCNP=e6M2&QuLK)F)2o3CXRd zIs}1^!Rel9p)oTE7TEs1E-}ocHl6!xLKLOn^Y)dK_M5Dm7t1bd8bf?}WxH8a9godj62|UY3Gw8qyjQVfIA$?k z^Gez`>;{CEA1rV^Xg$w$+Uh<7_p$>f0;$Y4^Cfc(?ryHf(Y>A{?dhHm9zXJ!iSbxPxSsw}a%*uw{u z^xG5uI`;4p>mB(xO8JVpO2(2Ag_;^eE((H+W|fU?JBFq5n2=>~gm)*`1Ht3X^f-Uu zQsc~Z-wOPSMsSWfkL0r2B+ZoC>#DdtB#`}dw?q*;H?>r%iQngzM)H?T1R%7OwN(mB zVSGy==gr^DJz8fcluL20N=@n7AheeR&A~Y{ohVg4LR27)5ybA>^s9&Sl+3`Vv)fj! z`)qEMLS=F9+NV|TIF`!dw4q|vn}^0$XZW<`(_9KgVsy!m1+gB{_KW1^Y9b4#%h-2M zC|K*Dh>Pt=pSgMun(VC#(`Yzq&Cd?ruV#JsE&D(mtyX>3(s`hI%#J!2OeUf2AuCz& zMi+-S9TK)bv{+@iWuR-Te7irpUd9)<2BZe43JA{PDImokckfuFzkaSIZ7HllfQ;pL zbJ@dRmwI-z`g6u~c-a|K`gOwLE{k3MZV?Tq@F5L&Dnf>*VN8d;aBeR~D=ao+2USm~ z6@)hC!*ax_e=7M@Fv2nPx8II0spQ-|D>b$Z$-ZBc&OtzY;ot*|b5n>?qea*F+JHiH zXKnv2eztHKmB0>FGlX@8Xl~M+#7Txl`OVaoJ6NDDF|FZdjrph!MJwd0+4bAM`}YhCx2u2sn1}rScNbdRYF(1x9kG3`u#5V|s6})9_qUidNFOcE zWZy8-zotdo?s+x5H8J)z0tSQMmF^HpM1K&ZB(7tIWNQnSFW5x2zOCQJVa5WhyK2wH%@l}^wVM^aEwr>dGzL3s~zj^UYA`zWj zmj({|{s$6oEqu6&y;&9)J7<|^Cg4Jh$ds5%XF@Kjvs>6F4|?lN&8OIcM+=AP`_M|V zJG}E{D!p5R5(N*=Cm}vkqjgVBROQ^vM`I=lRX+iQPvN&(H{JDW=?hm(V_etr<4Gkc z>g8l4qlzRG+SrirMF1Rw=l4w%OV5r@AGpVArwqZgmt*JEq>QWd$vGl>TmY-VRV&KI z!@sk#xOaHWJdmRIs-F}%1Me4khnCH)xm{|tU-*yVO4C;`uc;CaD?)2$uLxCI` z8Gu-DThjN2*iNauD#=pmzH`^|Zjb_j1^?I(!n8qqaf4nly@KE6Or$iAuou)aik}`b{{Qpo4E(%e^dcIo%^w$?*2ChZ9ln_ImZ$ zV4m$$2H+N$Z`vA!D@T%vAjvAyG8Y-W-%I!VsZNM&9`AH|4AEn+{key_p~_%&l66H%zISk<03ZxYNkD3$;2Y;LF?gpT|5Lc zD~f>>3D@RI`rPVRoCQHXx4ZM3ddnhdFW+!6oeosTl}!g)Lm?U|hdH}6HIrNk^H$9Q zvzz%U2$GIwpIR&GQO`uR_P~~nnw>sp)JWqTlR92agy2NY7WGN%dioG~3JztZmpn2w*@OLq5Ka;c*{$!WsT{F)zIXKSgx(_8D+=RrD z=S_Zrk9RHmPLeS{;WJNEM_GZgG z=xrqKDs4cdqZYgpmIZn`c`&1>FFYpV#Na!JN(6I4P+7DVLIT2L>{B(qbHr~ecxh|8 z4g+e;6mPl@SJ!m3QGt<+N3lyFdjr)EaI-WCuU)}@MQ}7Z?e8o7H&HZz0!__?~;swf3u-vSA@DjpiHcXQ`u>Tqoa}u9S#ZS z_Wopk`x-8JVyuPrdK#mqV+FBT^A(~$11){$t{gf1KAG#XPr|I-I|s^PP@%SP0hLaO zxY&51JwcI6oh;-6<{G|x7n8D0S5+c zSZu?(p$hahBtZt+S`%H$q{KRkWkK1?y52xx;qFc zT`oj%7x*Lv5B`wljdwI#QD$XA3t~X~UU)#ofTW6i3X*HheV*@oM2S@#7hRb#k$(_( zJ=i4;Z8)6Qv+6-zK>#MGgRs^GK3bjl!J@|Txm5gI*=Fg6WT86pbOBJf4ovdj+}r_r zc{H=9V2QV{a`1YJ=598VR155{$D%9ba9%QgtWpOfMH7#G5XUA?N-1AWSs6x6hl@^} zhz}Gk$@1=S9pg1Daga!=vYHp)A7h4uihWr=&LvRYRpqYs5prg6<t#t@9F$$w9r4DK zU6CKLz_xUu>!(6+u2*m(m0_Z%MG(qP(L4Mr@0Jr*11SmhASKc9UvHI-Km7%lgB3L( z8zy&rqo-T}Rf5d_*pq=`!2Mjd%fHjyfTx7iy<1)evqXL8m6Y&xd|%2LR> zCdeEsGYm1#e2&W`Rl$?J&k@)P^G<-zfRh-jC1IPHtBS%D;V@)I5zZBa*L%ub81ng7 z3)lpOt;|{WbF!2~xp7mAz`(UPHU^Rup0u69@1ifG9!yes$Y(c6Ii7e>sL)znvi(t} z#0w-+NyR*J2$>{+;i@ypt$)TTgU4EX-mkL24i`J_X;OraxC!=f;&5yTsK5UsLQ!m} zR)CP$^22M_3BuIQ`2-#%PBi;fP9rRF){=?Yn2y)=VW{v(+c(E2(EeBOAS$k+Hr28GU!Zt} zYHCNS&FWWasEX(Fc~Rf1oxhlKO3@#}KW8ZaU_w&{>B# ze4irnY^jy*qQ)zJH+e+?l-9pkG~9c*9>rws!Lm5rF5Fa3po>v-PbU5K-k7!$$JV`T zh;Zw(c>;H;y4KH6TGn6^$upN_WeM?DaeWfcf0mdslE;}*HKIZIW5*xAD_@sZ&u`^J zV>`Z3gbQ>&iWz|veuz-4PYsGZR*jV(;H*go}JA60XIRVq{H=k_Jf$mL@>%k<{(vUN=g($fF@3`iO4aNVfPw> zaNRkz-EpP-ZeCiw9kj{G?j#p3AJy_uP2yP@fxA;mQW0b4Ic0$j+w~undDAP;@3CtR zM?;&}4LA^%C?Awv;-G^hS6TRX88j<42sFMD3z8Dg0UD|jLAMV}3Mwb=Ry!V8Qc9b2 z$wxTJm2eE1SttxyOwZxt(5}}|$K54uFbGXmiknjv8pcJ|U(m+J_%WP^o6Q>0HcNz= zA3m%+M3XL8wvA>BKU?$Od2@z-POGk~0$b5fP~ppX6AU=T@Q;LdRI^Q>M9ycU^~?8u zH3W#RUxr2%C7>=`W*J}3lT5iE%y4iR5$0#cyU`W4fZ>;!_)-m5oy~hcE4X4?alFK~ zef*weM_Uz`_V$G@Va=1?UJ|`xV5W)_rn(=!a$k@!3{)1rQwk#5lj3rG#D9QH1}o;9 zhkT|e-gV+&yO_bHePLkfEEhO+<>8PaCn53DG*=6fEMpeE@M{&NW_vZXrRD_zMFtDL zr-$P8>vT-=?Mj;}kU2dAL=NRJR=p(W`=;{u(vhZd@tN)XE=;XKgitmKbWdz_6KKU zAv`8cW8vMB*Gm?f^{kd}Vp3}nZ+B^v(6QED&0Y>#G~+JCT5B#RSPMavO}nocMV4aJ zP2m+j|EP8$_@>!egLn4%joMp`TLjukZ$OtoXgnpYdFSP-{lh^(OK@tpu+W7s$v0Ri zX^?~_WFsw|J@Gr!*s2>_H1M%xyO?$;D5-D=Wz?>(*qvMYRZ?$w%F3FmO_O_77WR*_ z1t&~klg{Iz2>4z4^5UCw%xE;UakIsg=I8ef2yeMck~n_*Ox1RYFBGqPgm#o8B#ay} zMUhEp(`{;Wt;#>Ixmh>bsj9e%x|*NKE_$>XuPZ%l7n(J@#g8J_;WoXqICEZHkZIqZ z$x)fNXkJpN=`Ij@$p{aL-q*26kvW#HTQb2J&%fpFV0o%Q37vwASLZUh&at)vYY$d9 zuH*sW6X#v314kY9Y}IZDyZmIyVizH{W?I|zCcLEPeGWREmP!1@w`AdxVGn>CX9HFo zUkX0_Zr^Qu1!M9AWUST3+e##oKW#F?Dl2e7n$esDaFTbf8%*h!yVCHIeWNqHD_Ij8n3K>sJ%=1N z$zSre8qLnk6Thn#cT-K;5ji}1VAlli6;$d}W^Ruc8u>4~Q0$)w+;=^)W+#~)gs%~tSs3OA@1rW{LZ_+NlnH)4qt;AS67k2tU6=d0kR2EGOc zI&jTa86{2b)Zl^=(U&gvec?#jaT!;feV)8Y_Iq5j30gw@{z1 zy9sYjyVs#BHQo8Ho#FmwL9!pj|HL;$md9p=(kHqHZ9wCuopR1=;ftyzl_bWK@7Boj z{j*tfI4xaDq0nL5((-A%eRY?^%5n$)?9Y$6$_q$}3s2Z2FNeiQxJ=LFa%|g{M}x9i zyE4&aH?rnAt`f_xyo-78T1`GQ9M);E6C zp*~?cFC_&n76TF!uiZ@7m}w%YZTH%KZuZp`LM*170;l6&aq4SeK1AJOyRd2f&}*|2 zBYv0LDbKMH%OaT_zse33<5hmU*7Gb zB>hrhGXW`#83=65Udx-j03c(9)a!N@7e}$_NHWQv+O-dxZ zZ60}zj&qSd%y|Xcr`X{?5(h*m7?Ms^R}`^8XzJrj27G%%^deC{J05?^nwq|h?fQxX z1pyq*uupjR^USm6!r8oTy2m&ruso00uxZ_<&7q&C?PfeccK*g^hoF`m%_)B+J)Qhi z2`1tQEm(xLqz0~BVhFtxfMZ(rI6rDQ4}YGD*{Ck+BFUoGOFcZ$p%0(5(EtrQH&0Fu zdJ){#A;s5GbF!WNzZSp%3xkBK5RYE2pD4@q{3ujnE7<{REZZ%59ZfprA|K5-r;&zW z8a74$;s!N<9T5`hb(|@jpxc!%)Hw4$#>#ZZSjHqym@7{PsI-Fhu8OM6-sNdaqOzr` zO5QZ1e*p=T@mJ>9N#H%u-MgMCluR~!uq_tL8omn*e6HVZ&k{LX@h(ePxn-TDnF~dF3)?Z3&U!LQd9%dca@#!4U@eptK*md;&|DAdXZW3wd;Sd0}IMOTV@aU{AFdA z4MfgYY7$%~pQZ0BC15x>%X5)VkhqXe**SdS(Mdlch32(;>q_Xf{FN3ZYtQ%ueEBLt|vY z>8;xW0Uv46%aJ7|7Igs4#137X@L(au53eO)r0DvgGInDrVj_%v2`?q}(F(E4KNSrd z+`?+)DGwB&eNGDg`i|uT(DsAI@3RhxDDx9IEyATh8IsuSGIDLTmHeW8i*x1XOac!Q zdiK(mu|93d*}Go+oY;0+P#AmN`ABOl_Rcexd(#PnY&= zi(Jz0!zuio$)bS5eLQ$X3IUws|Ab`XcI305Wv12&N#n&AkNE|NENC^e>#Z@Uhnsg_G+oQh9E!a+XHw1Nyl?~==d>3Qw2crr+gODk3n?bQjfOu9}n6@I)pviRKJMe0AGsmR;1INI_`H z@iVUTN$&1uBPc5My?O_*_3mP+9s?C+Zl(|(EX~!7Y8Gsc5bzj11{(LNxIz!+UR25S>lv$yLwAhQrY_z4ckLB>Ukl>s ze?6m6cdAkQH<)%*!-Mw)-O*?)f=D7mStzEiyoHktCvilrdY{HHdwyFMc+WllBYD3f z0*w3st|YuaA}ZSOUNjV-b37Q@x|8l1ffTGiUC2X#|vxzMKzLuq)l`9 zTS<18(GCA&-P^{z7RpaCxn`GwtruZCc;DcW(FyO*l?xP-NVqL>Sg(V5bM{wgme;pF z)Megb>IgG2BE+kUCy0n~v;C`%^@FF>fhxMwM|JqLn*TTZb>hF$`rSWXmTR%``_?}@ zWMTZz!ip|`25|n(O^M`eor}Jq_aLFhtFw^ixza_^(|trp7;s=}pSAQU9w?nFCBel{ zT(qFCm?a$1h0Tr)mL7vmgTO7Ew0|kuKs2_scPcR=Ps)s|tb4i)%PH0d2C7b4ZT$Zo zFhk42Hp*zE2%ZRm2JQ@w0uh9Y150A0PP@M*m# zm5lVp{unrbOasTE$V?eH6gTmX9Xz0IeP0o*}jf%xhd<&Q7Q z;z3i{ZNkE^NR7XIYH7g1?~6F*w*wc-^3PJ67N9a*HfHC7))PEzG1zlE6#bg^l`gQD zqE(?F!kbHPT`*o5t)GelUAm8C9puk2bAGdAa1ky+p^AWF_GU4EHEF=1s9U)+war^Z zqXk5_qSA4RZFn`MDBN@~*|0k7n_unz;kqa%=r-)f_r~466_xd@chj`%S)9LR`{yKS zSXSr5seH9(jC6EVUN8HIUEbwnt2lS*!W9BUC%(J8aMvPhzABL3TnmBxFCL>=XI4++ zJ0AmQ?(RNYdkub{X?3jfef0`OTxui?F@6)ATH}|XTWLrhB6i|elT9)~NwA*RWQ{v+ zzeC9_S*#4l&{>aV94{A3Nw*Il%Dbu-f<$=vJN>oYDgLO!ORa1AW;`j$$JGcC~P;-D{QmwrvXeB9AH5f&T z4=pew#nE>4m7ad2Z1TsKsr$;g>Cq2^tr*m@w>?eG+&>_OGLw_;=OW6e>|!zMcXo8lARyh!)DL!3&rQy@CEq(bzaJ3C zQV_>Txn0kUJ-ljBqMva)>tjM=>@6dCDWIXbym#sJ<$L<-%YPaw^;n=nbnM8D-!KuX zcBLZt7nRcg<&USf!T*6%fLQ@7prZbc=#N7pa}eu?qZaG;te_#3pjEHrM0uW!p(6*; zjjzMt^~?|tZ0RS-uMclV4-0=(Wf92cfM{v|b|*2p>YX7;5u!TeCUUUf6YY3d*Z(Uw zM-+M#q!h{tk|s%YUNZ+|q}lcPM#%@)p~QnIdWm{j))2xGCcfc)hsHEJrk5J-@Fou@ zmo<9`Q(~|E!JP_L6Fk8xK`Rt%To}C+J3$IH70xKf>NRtm5vfV2FWc_((i1)o^!7E_gv3&{^4T7 zhCQD(v(~z2&Asly#R!}K6hp3xI$L%m8I9jWlx>u$N7kVwr2SL@QQpzRS(xChu24@T zS!+xzMTU}G>PzJ4=DQ2cx!Z1>DbR3aa3gr2ZY#1=5n}Q6n#7n;U(%qOo<%I@4F}tn zxU3wxvwB(BOGtEvryt8AuS@+8EfH)fZ{OzYF&XW>qw!T#DtWd$5jKj8mtM{Y)Z!hx zI;4_rDMh?Q0eSTv^FLx-y*eUZ`mzuPApf@f{Aqn>(#}Nl!@SCv#0hjErCy1*Xi$LSn}9kexJ_(d_Bcb%!I9lYM-V%5xNs zh6gTF&OKUcqJ?VtD-9+wpg891(4NSCbuA$GX}dmUt4r>OiV{c-C=m8Z%48`J9g;m2 zU{B-juf@O+h`p_jY!u#6c-HwK1 zvXDnMawERS2zkTM(V=8f?wIjQJqz%*iw~nJTa}kN48BY6e4T z(z$XA4-=!bMSXxARs{^21Z{!`#V9ExxGhlf=A4-aWJB9slzx7W=={sS>9kUz3` zRHG3-hZLhR2DSiFRo5!ov>Zj)3DN2s!!(_0{m%?Z#a}r>|M-{!llBZv;i94=GlmRp z-epx|nr_W%gzrzZ(>aFb6Y|Kuqd-fq_ic<6$qa8Y0ukQ>@TTq^~UWY|hi2 zmoPG#X{$dELG?5v%h9jKz>$-Q+cJTZJRwXVt-*|l*$1NVIx8mt2RmaG(@y9Pa;uR} z6Fv6BeZ;mvuAE83tB%WvX;+-G3y1Nu6Am{S%K8!1^$CeD4A+-S|-RZV*v z;XYlCrYa{S_QB-vsDYkcBeQ$eQVfb(SZHeUJ><>CxfN>7hpa;7Iu==o)k6H$&O1G0 zlJ0)wb}zRPQNFcrEiz44Oz(5SnR&W8#%IPo0ek$afdYT_4?}wQArt>1Az$W)I8P)m z|5G`t6?lcEK}`aQ-L?KhT`rL0*^OJ1yw49gY7`r&ugL_ZEK;Yu)%#A@nZ^sEB(tsj zli4Bk^gW{Fd{2IPwpDuy9(|nODfchWUXK(Hrc6G9$iy#)ucm>WAJvZurOU}5rNMn} zW824l3*qA)980UZ2pmfY-j`P{Nf{8`rw{eEyHu&*6Vw2u2f93g0kVwG9xN}s`d4n(qtDG?r54?BM!k0Ui zzu!%RN+~-gcKhbVtSLkj96#m2{J+x}))3#gDRkAxO(P_7j4aB}FCz#bdgd)JU%d5m z`u(tYP={hhmk~^qZhTR<&;u-hOV>};0VaLR3ywSs9xbFe76%2AIj|hUcntW`nl=cN zMD*ujw8<)# z!v@{n9XI{~rQ?5mG*S}tPyI&w&7b_M(N&qGsc`W%+XR`@k{t^QzStG27d?i`Wvd7AZ7@sdW08Y1UD^{0 zGXyFG+KC*B81dbHdIT0s&MvjJ6GF7rninHMy=dNpvSgf-H5l4pxBp~7C@Gk%G>+la zT}6oDyX_n*&tzWV<Y+De!* zuuc_t3^u#$N*i1ZD$$lM6)PP`*N8~1;wGyUpL>)%i|sCGV(N!~RP+78r|;Sl7M6>v z@Rm>^K9Fj&g*>xM=>)bq$c zu=5u<6HOOL1a4xX0xB@1%8hj1rNfe)=bd=14#uv)TrUTz5XYKqCDS$2@@P&k&`a z+en^$KZ_yCq-9@FgC|v4`=B0l2^6J|O(DpyU9~*Am1ZWl4{d3!680wSV_h+Z28KhT zd6KOa*4G^kcp8GV-3Gl+8k|cS9M^_iUKeK}D&a%%soI}MH&dR=c^~ELtU=wWadG{l z7wpT+l>2e2|)L2JnKe3$GlPZXBHp3>~P0wLw` z%-S-3@u_D49Vw7{5zy~X4(HDMr^caUcfNRTmdmQ~k6XJ(W~`;IeNL*I>ei^Zv>rXk z@i}Zx;B3xn5XuVPz&%o}MpA)ruTo!d`->^Wv&19er$wy<@u5E9_pP_%#UlNt)|!VJr1_YR zm5|5&1bKH^aES;tVEFg5O1wik;A16Mnmk4HsWK@guyuP1!2ds1({K~;)k7$Icbrk! zjeQueB8W9qHoj!ikC^7U2v`IY~BJLp7 z!`^#-c-?1t6kGVp43sxZ&fD$gzV1IfPHlX($bV`sHP5rN*KhhJ?^$fGErVeRwH6}; zf3ROXUf&}Oi8Cp>qS`sC0QIA~v9V&H5=GuKEfyGzCmCt#RL0tkKpw;aRnMB=bk7}^F%0UGB0~Y44F0pEiWQYKXRn*1e4hlUPOy{NW>Og zZ$3V95W@Eoo6EKdaV)bGQ~ogXGsBfbs)KzHYq=(BRh@Z@eAT%LIW3PdYK9I_r>n3d z9|#pChEFM52$QgZ$oTLDxg)}^N5xkC@rcLrq>arI5iXvw$xQ>eHObTeUwG5&=vIGG zEvL=8WPNb8P569C)xu*2yRNkODTMxHCu}=$LY&tk;;>TzEmB>MyrffVdH=gYWw}>K z`{D)CL*a8sy$I}AOcx#oI&K_!Wl46-@p#pRKHFfJ0M55L+vWOC0qv%W_oyG9 zc7R5oh^FsG&*YSJ+gP`7cj~{=?fmjPihgK3MiOu`ux&EVFcTT|3C9j6O6uvd-+9}( zcBW?2V}6$C5RyT3DB^-vgS%Ur;_X;N=-o5FS{!}q)+Psa8K-GygxOW^e10fXr;4%06sF@hXPP;_0wE0xS9O!!V z3Uh7RyckXsj$z`A+{4cww3;&?hfqv}LEe-t z|L%Y{$spod&o@(uAA7cze!$p)pIi_Gps)0YRzLZ1%q2;1ilm6}BM3Qk%fx1h@UFsL zZ*o6mi%d!c9D6e9-o_sml5Nd4*!hS6I(#aGX<(}b*B9n{qK_PsaXx8}{r#Qv$@B8+0~;TP9m2G?MNVna+@cr*86fr8Ep#th*`{xd znlD}2C$zJ)5$KCSRAKA>76XDV;hk@;H?hw-VW z>%Dy+-De-+VOesF{Hj96J|v;+9=0j~)?&BOMY)4kAB6YH90$#L<8;!-*rR_h zC@+B`nF15HA<1jMgU`+?ZV8akXsOjIq2c$>_%{>R)}FljnJ;J@)v{RI91gu52uc-y zx0%#SJ4iP$;}}uMMTmn^Y!ozvG??;Ox_QB=?Hu)*@V=!aE|81IB8dWbVXn)6WTGmq z040W_?)r+z;ALkfcpFXd69=FuGLr0SL2yoKs@qBAyK1@_-H-%=%=R=tY#b|&Dl|~~ z^2k->2c+5eL`P-eZ8Q$<#BY{!rYYQ?UqJ^9|5tS!t^tA5Dos`h`eug7DJvkdvv=Qi zzuP_I&Rxr5bbAeYbz{q;q%(Oz+4YLR@;p4h&S!I?j@6BFF2=yb!^ObMC5GjC-QViU z?ub)n(N-@)>&V4ctTVPgfj=;dRx>(_;a6+-%T3J5*_5&|JQn&X-kE zx=@sx+&vx^M?)PjZEdlbAcU8T@iM(Ld>M^=Ms!Y6m#;;#IL0-d_Ba^8v2e5G8g3y-y0R@+S5!_E0GlggGK0>0ep9xNm?r5#i5BF|!Ekp1{?*D!@ zN2m3w>b)NYPqOZqi??z(h)^Y>E$I6GnGwm3%}=A{U2G`;M4j{c7__n1Z^uR|WZba#L;{ z3fbM8^}CvnB}Mi4NYpPK_|h|@xEX)vYUwxol(Np@Z$O4j%(;^-q3f7Cq!sIw%+4PYYhNKX(=BuV^IdjG=k({_drbA70e!+bZtfPzg_8V~miTdJB<$ zwCMP;J))3)C&$qr-!$I$QvK9;k{39Y+A?XvGrP%d4pE3iv*!s}JqYYn3b){wJDFnK2Oas&iftHqGio}q(tyC)qs^NgIB zb4$02qvq4ux@Zq<7QaNH`T97lGg>d#W4T$*%UVJ>G#QFl;jAS6d@{|N8q4DgoyxCs zIFjKB!be59YkUd?f$evd6<&!FvzPCZB2*s^pL+a@3xG;NpOyx?LOvKU1pHE$FrwH@ zLok<@!}+wNJvVUvVM}w^-6QBDo@LT>DA)xj_HU3r8p!8^Y`O}@1Jjd zEjPnGLs#0b4s*dBbrH6qT8{WqicCwZkMwgqnf4yKEDCwU+D^IE`?qm=))pI<{ge{1 zC{z=TMWR7xE%VyT8YF9DgyG;`1pCiI6rz-hS~(#mcrZtZ-PS7?tJ#`F%kj^*uCw`x z>Y$%h8Ifb@0=>?UHiXI4xY<^C(fQeW_FOl%cu+;VwNSd(rIm2L*IH zJ8ju3c1g_DB;pu6=e-ynK=>RsT%+~^2YGL?&-wHM5FF^;kg95KyUzhJVSc9t<}uDK zN6EBcvr-brj9~0-GqTsUXuj|xOyO}3jzC4>SiLXnFGaz;ZtQOhPWbsP1xw0o#`>}l zn=`w~c7=Kz3Gd8$zjlUuErtsgxQmV>J$2ul8GZe2_zgYlNmQ1(KT|9_kVHy7>-bK) z@Foj~ydbXtQdDmr(X2-C`l-|-%tvvqAR#MgesB1JO=)?i>#qE#09@G0n;=sG!9$q9 z>#E4$C`w#5Y>3rum&LouvY2D3G_vyVNCFbCJ3DvOUHzH3-b6pgGqlmL$k8sjYSC^! zzS_(U=)FI5+LM?^xF_FWDERSB`0>lqSIrZ*GsY%mi<^Gch$BGsE5l;#p;Znt)8lN5pFk!L_x5gdDT6M7nl+2VgmbSVBvsDDSNIkb-pgb$Nve8dWBz z%($l5A2olmDk4A!ThM^slR7O%JVAi=e(c@#lwrxUhG1tD$)fdiKX9GNb>Y6n-5bo9b3?Q@TW#sXHi;d~70gW0N#c5h4Xg*pnMBjS=>SI>M^ zhNu>0BIW{+QyLf)zb|*@>qaExAB)v0Ct~6Jk!1;t1X)OjF~2)YU}O>GLlo;WnxlzYQ8dV zn*mLYjY}|mQv@r zpM9wUno;(B;UcT;&uixg%xfpFU>$1ZKptV@<9iR{2;p^4rp0je;?@tON7#afN_~<& zq9Kc%hKZ)FE)2raa|EbjRP^7ETO#r`vuhJAwDGiOA{*n?&CdyFmaCM!s=m zc_kV#EyY>L_M$Dnxr42yc>8ceBTy41ntC8H4wik%@$BVp6{aR735K4LI!gGdSHXOO z*wVmIQWXw+-;L2@5(Ghb7e*Nr1=JfV5E+w0{+xnA*r+quh}XS{mX?p!O#j(J^yh<^ z0%t<@V#L=2-bcdOwG%9k4T6g{j%OH1i-m3&=~&QHf*_}Bqg9w|%JHCysI!eZ)IAql z-aR!r&7E!9>rD=hZbaC^D3H@+ZQXEwDzhw9jkv?A?gYWNZww}T!n1;gOr1ra$J&tU z_&_u@Z|N}X?P39@hQJ)$RM+z+_yeOJ+PX2lEYuMlJ5WL|(RM>#hS?Lp;U7 z$HXy+3HQOCd=Bp=W5wUL#tHq0o@ORqk;)b zWj|#=5cH~2R5sjuGn=N`J8ZIWM=gR$xz5}~*i9P@c)kT-YF8j8pi;}wYMa2qQk}># zx<^uVOLbm32lj$?a_seU9ZckJp4AizEmcJrI zc!GItXLAt(zQ0foKAkUZ@=b-4Dls6*(^a&+m-uZdJ`LEMWe?RRUZ8qd^PG#eFRLDJ zNh$&`!?+M-F3Gq2$RwLYg#{Fe42WL&tOlc9y#4^%%B4GRu{q_4mDX=Z*C)?BbR)LZ zlbkOW0MXOYB+=LtZTsfQy8$(siR z7I)P%8l@E}O3X~US@DFQZ!5lMT9k~bRd;lws9y$cCqvwZ?ELi1pfRCmQt%B4h)es6{i(OEmZBgekD`NT#** z^r#{{)_yilYt{?C$^=R@Cuj2xeIM>(1K;jc3#*xmq8L`75f+(3c53ZXtv0^XXm5Yb ziU5@?N>q~78~<`ItyVqdpLmyAgT7pE%uEQ4exkIR3RwAw?jOI8?c2@{zSl^fJaFJ@ zfkGYDsBc2$QvWGjiT(Y#hjC?nhl@yB&}(bBB&UY5m`4sZM!ZPC$-%I+tolB<_57-3Ztlx=@~>Y^?h=EiB`cSIJeye(o~W18eue&d`5YyM z6;zP*Q2zQk13bRtne5j(uV4DOL z%3GS4PdX}YA~N@nJ=-KOf@5cpLUN(dtIr|bG~OPIsMXCY!H&PFnnUl_L$r$-fl#5_ zR^xI)5g5)U{~prSd3FMuaX1;_3LVf=3k8k}=Eh9iGjj~gPuT7eDq^*#o{^D^w;J8T z8?D_VP6`ABN(AUc;WVtGaMUPLu@GZaD-T1PCm@I=u>0%61@DeY>GJzP`CaW#c8ihT z=`INHjXvO+l@ys*$6pz1OA$L+Tq97u>!NwVi-pEXhDwho=qs_x9I9<(*Q#~`>PGJm zVI?J{YywL}N6sr1W?4|6tm-3BR#-`yLP=I_m+4!y{x;WpckhNJ18#pB4**Uth2wNL^F|Z6@D{-7lFiQLK)G>-D(AVp zD9LQeV1`>rTRSdF_rTuKANrSk3o@u?-gTW^UTok_u)e`L|ZtVOfy^yeJ zoTJbPL3!cefdD=E+fHgPU%wY7^)GI&T*2*by(zm(>x2eJzsP3!jI_;w+m!6Aj8-8k zfQnmjD{((BW@b*QtnpV<^$8U8BW&eJZpE^iG~DN{gMB?-gxP-thVMbRK33Ki%5p8qyVgrc|Df|o9JG9t&RSgmlP{?GS+ z8oJ;ml4RfqoLap?l7Bsc1-z~xwD@o}{7=Osx){s>kIwt!TEzTvuoUyf{THtGi6DP? zxCd3SI@^lz_9O6;m$rgROhKoR?iU8N=KM(V#X(ze-a2(9fUEI( zUeEFX37tyX#Q#ReaJ~dn)6U43eMy+2*#Ahn_3g8mMe`w%CH?Y`#9wpr$y7G{PQLH= z?ZiG#{N#Gxcw-I+=e#HzYh}<3qN3xeun>0Lh!5W!o~DBKzBQwy{rcZGMvn;TOBvd3 zjPk#z{)epre2VT^ORyHWC#UU1W;ZbIZZ5{p}> zs8syR5|)BzknwOApT89;j%=`|KdPtC|HX;Sb-7K2697zP|)7A!BN9+P$c8s4iOz`KRmQLp@4OgLl`v$X5)+diYQbgCp#$WQ=YO? zkQnwpEesdbKjxX2W^HGMk%^17yuD*hsYt>REZfIe4E=U(}pm=RF%u$6nqVaj} zT>9LOajx;`h7mHM#ML3Y8VYna)z3VzL!KBac7_oZ5uyxYK`)(uiEDS5DcN3Pr+H%P zxQVoC7R2@G2V&Iq91ml}Cx`r9SmRxw6lb%FzOwbFy8avmBL`UOP5V3O zf%g&f^!oV+TqArCi3jkxGx&xH`xR3&%JgQ0fN}9{AAM3cq4FFc!)${0wejFZRE? zRpH_Cc1KU2xq5TLx@sr`_Y@g%@daVs07Rf3{bcUee*MeoRN>u2pl~%{-Y|*Ycb;{D znaQm^a0muOuUZDpX9g=Yf0!?-){w|DY&&_ggdys$HfWY79Cft1bc_Ezujo$)rg1K0 zn1P>-!poYi@UrGo(`rUFVv{bIG()z*(}Y+=I9Il_>WA}(XQB=dG31yZboKs>Pm-&8 z7KDipg7SBGOdWV1)cpb+Xljin&zjPfPLtkDUgKOIdLM)+3o`&!9xDMa_Dwj}-4yO- zOXSK8M7k7sxG*BPtW3bDOdU0o=)A(Sq|Z)VV~|P((-e3r`w%X5DM#*vyz0R326N&m z_q-V4A+<8Yd54fUnBh<9d&Gtt7BsLobLIhacq&IHixcX0`MsiVGEU6~$RMQmS~EnT z_=rV%^h(w)bQ`iga{IMd&8otfGjP6?u*ZmDusllgaU6%M9oMGz^qp^!N@hU^2}zyS z!UMKwITV9kFLTO}9-Tk@MRyi-u%UH_~4ZudhPJ5zwQWrX9JXp$ia#!No(4Q z5zv7oexA9KV#((wh?%x#EEUSMQcTX|twQ9;2R0Dde6sooeI`Ubi8dy5wRMH=1CkY7 zN>zYt^O6n-C@cArHyVLR5CGo+!bExXzU#(;=cdg6mBz) zt=68FUpmb@{VT7*S*9;M@l3V#u_VyT=tm_Wl?B+|tFsGl;~7}*FW6tOsUHEcP`b~uKxq>CfKf3EokJIM)`=QR53vM{g?+z<6gm6(%n8xz$M5-F;_|MTlazEW?aX=|b5Rz%(; zuf=Fys(o{@f`hD)cyPhO`SX~Q*p?VfFj0f=!VOW`Kwly*jPD1a-G%TwL&+1L? zMYA5|_5;R4I2@#eTX3e#$KkFSF*9T94tX{>cE-6bPx;79dt$*>f4X(}`C6Ur&yLu| z4($+0~f^j_O+k8kZVBu1~jZNVNPVJhCm0uyc@6vEWhmnfRda1iZ<_%|j4R1J_Z zCkd-F#ov6^3vUg7nMrf_j|z7gloPC=L`_aUg5&Ya_ zvlhA4nCl(Z8mMvjTAj{2pg+)4IbZak`amh({2JMEH3lNG`>mYJ_jFoM#k*#0Q?SS< zO3R=zWZ`uMypLce+}XDb>W%UH+IZDFk1=k59ZI*lZ(-Hk@e167Z3ie8pCR~_fWqua z$b7b)+?z{y9`rX##KYt5U~QkT#XrRzc|Eu^25axXG#eH5!Rvf6p9K$HYDqXap)zXv z_EG2aI+mvVwmb3)T2{=+OWM+ez}%1NH1^^q7~0?(Dm3H)Dzs>nhUfETT|52S(h@++ zY}2O6r43j%Iufj`{9H||4TR_5jeIkxsX9WE)JwiM(|5U)i_H^H< z6Azxpod4<}P@@;3-kB^a?HiB3qGbQIxx$Iad;Wu99Rz>}7|3W72a5#Q--|6mOa2yU@42$~R{#7=IcL#_2 z1~>YbL093B_-Bes#DyL;?)^B)2Q(33{nkjXrkt#lG#jBt!P+^KMC12$@J!cc>uxQyp$mi*7}&imhfWBrw-L;I1ss~GJK{}Lio00v-n z$3igBV*?Q4rEV+v(`JY1c?Xou$A%4j{I^bj<@5Vgo?f;2rz+9t8 zpy1jz-2e4Ul~MpC)+n~@Ed&vb-Z%iFxn_Z@{@Bg!(`)sb#Y=F6Ay?CUI$k!|KcwK` zX8c*jf_0D%bUfffsX4(%u6335GP+=zhh3K5M&N3?uA>u4Xu@_xB`?&vX}aQ1>ol>;S!>)|Og$rA`yW0mT%5i4f7uIe zNHprB`|y#E;!y|W&9b9YTM6I}KK}d;7-WI}BxTq7q4)Zj=}ZL$7AaE@;cP7~>+3vS zFZ)%h%`;VJdniw z3jbMNaEqRv-p~iZC1z0%f%^`{&cYY+-PyfW^oa7@J!V$OwqVJa@-hzzD}wRleo^54 zFXN^+1gpJE4vGNY7hnhG&NjEsAvCTjZ*m5SHl!ZcM%yvD-89)40Eb0^(w-b8<1rXi#kiVPB*3iX>8`-6 zOe%54X`H~UI|KzCmdIPg!!`D5iuf%3f>(S~Z)Mmv$4h8S3zdSm9_K@G@mc} zGG23aF*QibKXmA)d$hB_2=8uh%zDF#Vly%*Cq-OMUhhog|N4;|g)cGTtL{`01zPbs zXR!mB!Frbo>?^OpmscRRN=SL>ItK1Uprq=h`#(Kc2nwbFDOes8nD?_kCN6a1r-tK8 zh}F!f=D@823}UDHc;e@3F1;S6_RLg2k>ml>F06efY2H5Z?zjtIucfiti%5!8oK>#g z)=$#%iuHA!Vwpe!f8(eN9wJ!n4dBOzdwpLCthsp0 zfv`3YIEUgDO|soA>iCF<1nNvw#RSpUCaHJzIv`j&Qr)BQK{-ryY+yGb76~XolxGdy z_&6D@{X_m;lr|d6cQ!gYt&eTZu_nBnx2I1a1HIjaZtGyc4+VKJRHMOwkvnTS{>y-| z@xdUoUQA#hqBVl39UB}KparmqnNR*a{~a$>Y;irNtB*3S#lB6ZTLC^w-0#mcXgnlO zs%)=+b_WE@MUiscom~$dLbHaV5r-1n+0PfRQlm}AhKJ*KCJTpte9fgOeX6U=7$tFO z*{>C)y{@mnnD2SevZkQsQHJz^p92ahwxX+mLP>zjH!HspafL6#tDqvB{}MfKKvdb0 zq+^u^OSg-NI)Jc3o4nG+8lr$#@$?r{vgdN+y_9ZLoR13U*RuCcb_nFG>ZLv}BC>kc z6>rcb$0jmO^YbG-Sgg3HE;DY3dGxM@3a|9aNL0z}#Eve%wUs?jzQKMG?u_xbH&>#v zD(S8#3oc^=`?IxV2Ucr0ID=_Sm=u$OANWjw`yU@P5(IjJ#Tnipbd}5sJalS6L-4=Y zeuO~Cw^bLrO@E{9A{I|LHuhVM7v{yH41~p6iWr_B>fm`{gn5)`Z3Fb({JM4W7_1>0 zk{Ud3+#*$Qn3U}v05dX*$9!`#c`uvJ#3Mt%F7J0bR_t_nhzn^!ToH)|6d+Q1!MnNt9qpE?0(x}*L&rrmvbjYPvnuKQ%tuJK??f(03Opn zDEf|Hyp283)!FSHQN52KmV8&qV`1NJei;vt!|q0IzP~ZU!{o%bfl&28{?g^xtHZWu z*1PE`I(ILLq)BZe~lNBgO{g+6dM(~|q;ODWidl0yK@hFJqKdE`IW1vKfD14^9 zR6pS?zSitmw#nhTGQTyaD+5Iz1pzq+ztLgs=GTuCF+?H_mge;>N3|!Kq{+$2&UZKG zL*v;`+V0BMOB)-%lZhC#dAr4Jnwgry7m!(hm5G2C2T3EiQ#|&{T(xft0~gBU;jT^T z6WD}OTc#2lCvCpH-Rn2`;Lx$kGN0T)$MnE);3xS`G%ruEXbsAQ|34^2=0d$oj}hHVOG_ix56>KAAOn6{x^M^n-*Kg5{pa>x%+3Cf zBM2(81X8pC8~fY6#f!ebW{n5@y^9;#9^ypB*i^JH6qiqsAmX6|7Yt=PBWd9d*r;r_ zu_&IJn(CoUp#UhRrly%7Z$-FD=K07jP|eA0(ljGmvFjfB@#N?k9N4?bjy#Lt%DRg*n+V%dvf`ZLXc)Jz;|9J9Y z{3_lcX~;m?tk!{kU~GAd+0^gXFv-fNVoxql4l%G0(L&J>=tn5uPE2Tet#l6P>abEj z+AfwFDaj-M$BLvF>?W5r>?UW<-qR8&^L1`pHEdw7$i@4pc6M=RLXam2WrR$lok@-=tmZ|Jqp4*S(5T_A;C@X$4l4 z1YUeO5)^&{k4RQ@2|3zO*gXaN35cZ+V?zX|iLlej%=^*& zd&_PwOHYEJJ%}!tqwlQZVx=*Djolc8d2(aft5f|Ft1Ia**@#^go}jVFyO*GW_;0rg zc!sP>RhhtrX+EUTL9p?x&W@=NI}?+Zp)h~oE=lAG=kTyACZ;d#{|jLPXb}HIk%v6{ z=JCST3$p}o%*DiHE1Kr?{l^6n^326>7uGOq_-zV4WJP+!d*@t@yyzn6PyQhUkqP<(d*HHBBig!7r??};KxW@3Z}v<0s*H2s!a zj?zURn6LVl#K=IceX^u)FjsUe`vfaL!!;RSYwUkM2R>(?%@9q9#b`eHoO!nB2L5&z zT%cyL;%K{TaX&9~e!j!DyDaAhv1U{+;4D!W#oxzOsZM%-iMs#Nl_q%(9yFq!ppZfA;uFmYuavc8i z=D*)Pf>Nj8;v>)_-CG#ON2lf|h@8iL^b`(*y6FHGR%b^4V<}w(BH6i5wNqq@bDvy7 z;u8~Ro7)M9hX!{!QzaL+f^-e1yAK*7;~SPGJW9g*;y7XLZAMd8@^E*-K>!&Wd%5<057@k&FSMjPQ)ydO$4-~11=LlFD} zw}L@(5p#2GxjHiTb4(PUwa}#WWNu-U4t?CPKn>9SrJa6f0QWic4Sx8?>t`*88z$`njLsDBpdZPIkU=Yim1guv+9jUw<1}Pxk>omWW@{+;_Onvy^x})jQX=@L`iHx>1Ls?Oj>n7JrK>XyIA_MW zm!3`+1{aol_ArK_%uLFuQWl){3{MaT!ZIJf#XzIB5+YQT`d6szBK*DB-BjbaN`+XV z%S0`HbrO-&1z%Rww}_iWK4uf$9z&>9_V>SE_szH_a!@(V8D2AWMHYY^J&&2@)IZh_ z@c9$L;x;Yr#uf<7%$&D=JDu~X6*>_$no6rqT~W+lx9|J-#dxHbF%0Dxb1k4?w|Sj) zqkxb3*Es(5UEmhlRJsVmvacI7Pj%yZh%;e>cw6vkf%D?ry$(T-7qSJ0+;+`*c__g` zQzHQQ@uK^}k{f+6BY zdman!Lq<7>IFh#-I2Q_3s*T|R@k)51Pya-yxJA*{4q_AEb%x^NV(h26Dr8Z%7mlf> zrUKWK7PL$6$#5NuKtK$^{#gHyK_4k+xsTZkH`|p?+0L^TmXzEt&2UyBs-k=3`g{jp zxYaMcZhu0iB1wSPp0MRh#*HHaaLWI8QmB^@?Yz%ohX

    KOtEVG2F&l0?H7j(bcs zFlas@k;9RF0isa)wW^pV6aIxN^cT*r>*D}pu%wqJb|z{I>B{GSdU)+jyDjm)kenO?T#|KN<^>d@0NSWC}PGg>W3Q<7GnM&CuEU3^k?=R&Vw!|&aSk2AO zE%4df{r%RR$7@Xnf~+gLdr0^OX+r%&TWLF;{kK|v%=uSyYQYl6G$F|`RL5kH-8sU7`~8>J`1WIk=GNsH0SstmPQ{FQAGK_**6sSR*+R*@6#cL zMz+YZl5cr;XpHlVt=YTvr4)0LFcK1_qfRy!~?>P*st(9PU*~ z$K6;C72)5xeB4HcY&ONu#py4X;dw~}+w4?5IlA*Msv3O~#ZlFzemVmWTQ?!9XBCyI z$jyGwVfVhUvb&B<2l@xty}G$T)Lr}o*lHc8nlH{p>4DI)idT#SO({J?hTg zHlVcf9+A4{vXXA=Bf1~gAG}srrohHBZ?;?gPNg1{z*vm0D$I2>vHuOdpjwEmL3~cv z4jTsz%htP^?I5o?5vmtPbKD&(s!D|C3O`F`c+~LxqmH9FgLeK_P}9p+}ORW%u+ zO?A7Pp*~ZzXMZ`Z4O5Wab)~r@2z)JGN|2k8IrCGR$*t1g_wJLgKBSuKI#)Hq`}E!T z|Hs~2heg?ad*d*)C?NvULy4e(N=XkOEg&t8O1DTz4+aQIh=kG#NQ2TLp`diPq=a;L zzx!s;M}7Rh*LkmV&hM|^CBh8M-1pvVuU?@g z3_mUxc^IQVLHFPUnW6pn>=Vuok|3+gPl9N-ft*I%RCdw@pv^eTe9Qr4#i9IpQrv~eQ=_HN@pB`Cjwhjd-9x?@1AWE5^GmveXP** znyM%5FvZ;8KLwL%yDl;87DY9O*R5O;b-SlO_=CvGz)Xk1%i>qyzrs@zew z`AFb>U12=uxT@#~i49uaVJPEaeCJMpjpAw3d70zBLM*N#~YX!jWm{9Q86kc9q1tU&>O9(AEIS2rbJ^S=CacJzSzo3)+` zXO^&n)iz_kC5h{hQ8;1_b zPw2R9S`{I8nUR&9Jq(97+DbhvtV{2^ka~(%*fBdJZeuo6XNT24ZmD-Rn0wy2Ae^*c zHaWyaqDox)Gcs$2(pVWT?I47PQr{)`DI|t$u?C$Z82D)mWY+zD0*}w#G?UmY2bo9a z5qx-lndduQUK1;sP|mkXVJ}E0+TWUT)U^3c)+%s<&PQg>- zo1V(|)F;Qq;lIlV_EEG%paV22Yq-5>Lu(IIcuFN)TzkkJYxtU+G;O;h$N9tVmi`d3 zb)HWyZb*;n{?=x)B7^Jrdr6HOa*tp78?fV27_P(|kZ|O9E!B(eku2;xVFvFt(YjiB z0~3$u?6&Cm+5*@v;sj!HAn112dGeEaGW|AlQ!KRumvqoXm!S_F?59>f7idcXB5l9cdv~$S!__g2lwJ{M1uIzK zAU?*SBt!4TojF(aw?YT~xn576d!fl{JzkC6uBEYW>Wa{b*xASi)2sLMXM8iV=z9Cz z{JhXuzf$7DA9IQ2X3>L~@WU@Yx=ae<-O}@HxVQ`+&8etkS|T4}sf0*sC%rv8NRQ5x zP$Rv3#BT&-AieK#`JVPe0?xT;dd=%`^dQ`(84%7amaN>&HGAqbpD6z=m%e{PFFE$` zf@#ZLe8Rt`7214))S@thPf|ENGWMZD)B_{(VQofWab|{z-k|ogS1E{U)|Glw4YIt1 zx#-Fz7RvX1qy$~>tnXg&`Y3ndiJIv7v5Hu2na6%Zo|y=d{R~~HfEL?4Is>$+tbS~r zk8kW;_Az7@$y*3r>Q(N{KBhvbUWP6>0j8IV$j_zc-H=0WBR$jRu2$shn z!W;+7@LRBe$3gCr#Nu+j_j}JCs~u`6tNgJli3jCGojz}umZKsP$DjGJPMUnZAzX0` zX(V7><_42#`O*(c>C;5%PxP3D&3& zQ_wikJZ zbJ@`TwIbOG=MNUvr2aD<>FF8wJ*A2YwR#?tAXJ2n2wz>4t`O#cf`;t!OC2S{6%}!W zbIeci??vfhu)rm+C-0wv$U*!|F6b`#6DIfZ3xPx`12xq?xJ6PUYLc(=rlO}KJrm5= z9MqGdcizfzd`Pq8ZmG{vC3lNXID6= z>7=l~WiqnJu9m5D!GK+7rfjmF<>kM#lWQ%jnIx;PupRiW$9MB;<8ayg7cV2zfS}EJv43&H0aXn$!aJfc znOpF?c#si7CSXO9h^%-MiQBIwh~2|wx;9d|rn_Ckrydd-eopkIP#9c=Q8i`XF4uka z%av3?PhD+?`cMK@PnJ4!@0#l@Mo*y7r3o0u&WpS@$7u&|BA7I91?O(8!I`D5r#r^- z$;%%PgH++fcK(&*9m(}|{`R)G(>ZO5g7nB`x;|Eo8ezGKIiBnItv1sU!DXZGKq^Fpv@Qvl^)yQ=gqno@+~BnIAZ}aEzLR zk3cCmFa1#DkmB{2qgz6D|`f=Plg+;MRaHC)W5%zzp`U6RQQ1VP$)h^ zNOW<)T$qiT%VNN4k(TWlm&<6=$||~WnRlE&Mwm*TYc8p08eOP48!~YQ1ShJ-o8%Ki z`8^-BiHMk1Dh##{h;s;c6N4JnNh!u zR>iIK4Bh?mi)@Xja79zs-O-tb2h^TSe>WglI#4p+7zO6sGDXvaYgo8`p}=jWY(F)^ z_LTZl*VJA0eYUN2&t>;%cdcDyO|>QlI@9F-4hvA9*8%$DNABlI6e4A*ZJUN`{6Tt2 z?fcDco%}pnhcfrgZQ5&E{Y< zw))o8QphBR4fCT;YS{%b1Ra#MZ+sw!(w~>~5FTc?lopKm@9h!mra$5 z2H0URFWq?fswR%*GPp_UdKa$Hyv38#T78; z@e-kMfy_b4^qimYM}2;Y-MX=$XQx~vTQicw?_(&w)@R$wyRUCki>AC!66ij=<@@Xb zmUU-OjgrgTwlYCr*FWrQ8{U`A%ywM@@?)t8-t`)I5 zr&;}=8b(lQ;I?Ex7LrPC?=PREBgPfO?of5hH%0QblqnHV&}U|s>JyMQ(?2r$`Yx3m zUBbGlsS0H%fiu+@@>FPci#H!OU*?JR~zMzvLnuW&IzA*K2bgiwO$>x4&rQJd9Q zgH)@5Ir*(zgi8zWE(X3+*JlCTYSnm)LG7b-(E|UjS9dz$LSE@NsYO31BlsgT@bi%# zrc(e7%?-J{Z;bh8(V6t#Qbh@L+YYd;l4sy6|9g(DNKBqkQW#^#oeq&$-@<-@Ax2vFZ5;Af1=C*wO#7a~?M zAH;Z8yVqaZca04#J|dI~L|Fj@agd;!Z9Ng-m!GxX=Nht1>x8prCE5i#Z`fPXr} zOG?fQRTa2eZ@b{5C*GXN;XPVOAnY5kmGcVFN~W*gvJkJd@_t*s_;L{V_tK|DR_3qa zvI^W=kb={KZH&Hdq&?zl+#-2Hr3EH_b#-GNkLjt)1s--E^+Ft;{-IBfo+4j?cQ!^- zsYhJ^KI1^&s~h|vvVN!h|uqheB#yTJqtql!erP z#+Il}9upYJ4mR*tN_xmE%unC6{kVgAy%{~x9LC)1Pts{4X&`i6rs>c4s$*)vLO$rJ z`A}c{aj?MkS8U|EUFuJVSYhaxjYK1ui9YvEkcwCcIz@MeTZO-fTYPV0A$luhbFE}Y zOck)N(4pG2;t_J-%TbMU^_kDrV`QRh5ek`T?x1ug2t{dGnEGsYy=FtWHgn;0pZpb; z$;eJv{@-_n2nD-(gE{Vm$#4en2mRRyX(k|93|U*c*U?x-n&*zPox%6Lcj)G9FS;kD z39tDkkB=_|Wq#^#)xR+RWsolKxgkB%`nRock6e=Bs{pctybP`hlkU2?CF9{G%$*Gf z`M?Kp>;lHMuRrlW3V!A?!+OU19*nh)AyY;5**SWo(c0E|TIkR#(T|Il&HMCN5rMRf zfP<%y&1UF-6S*Jwpff%6hCv`r<3_E;Tm*ieiaQDN{REFuQSxnu5xUcIvqP#Jw1SEu z9ojwvv$dA;6TM&7yLe-KfYQ705G%N| z?%r#~w_{99lLud4bD2W0$}iN@;NXBx_P%yvj#5y-aQ8T(VaZ|IkorX{{IhKDs}PEa z>uNuwP%cBHmKgZXp@5cJY(L-&9h+eqa>vm1&PPTAgRbf4}Z?k}PWmsPYiF7^4*QcaU>=jPVc!tOT;IPkf;f|=fqFoSTA5En@#CX9xR`D;T zMu^N{hkiH^QSAaDf|<&q$yU7vpd-M*NHSQ{DY&_^kR*5v!G=E((xsyB!enF_R9F~j z-d-nlgoP|h;?~s~V4s=44fGWUtw<&+NH%|j^sm$Cg3mFh4XLU>G8TIfC)Onzy# z;7*NfLwVn+gZ+A_RHA8GuJ$Wsm>x)_#U00*RwMwpAHsmQg0z|Vu&vV8ysZ0 z=7GBT6u_m)E^gnTL5cq#D_RCF^BGR_4G7e58-PAoH@*-rf*-f9*c@46J3nx-h?9}L z#H})w^XQ4T`uZ1#!>#aM%A_nkFEKkzIAll;qRr2;72L5g@3RG;mFEMNBLKp&Hh$Fv z7NiQ1T_L!Fb4%O;>R_H|j7$||s%*>&S-#>Gc9Z=+K42Cx;rZ`i2I8qpPM9t5W%=DT)x;dQII@AJvVTB@PnSA5UU`SGPL%CJ zru6ra-M6)mwGPj$5vS3K%PA^^HLTz>^JgYS(eb-VBQAiqYX>~Mee78(ju^Jd@#m#Y z4S3O^jAOv;!@m`>!P?*g;Jt0zccHXU{0p$(n78x{>%&I7#)QG6`HL3+X^kN#@K_(J z?lvT?$A1j@nxL5wRP{00w9ff(`!K`uSZDH#3qQ|V{BHVui1GX)l)r;piewJN-`cUBSmCJy$sYP#70+bRt;KO#SUJ%7n;i9N# z#tgV5{qBvEuq+}wH}G-I^mG@a*@r!rUpeZiq9@AqOuAwcJt;dvJ@KXj)YJak%LCqo zA~i%2NvMN06Gv{|q9sPH!>s``_PqoNknf+naHr0L zCDo+fAVD#%34|xRXQLKyr1Agwh8zSpm?JY5h7RK{czoKF&SCNxS=8MF08OSf=$`hI04+_aA&EkW{AeFQVBr=i02joFgavu#6}cv3 zd;)Zk8d|cl3|QMP0^t-+3hqn%R%{$71X-mM`jD?Ri?P^WiTMnuQls(S^H3rj7XUP6 zt8gCv2PFbBI8*?lQ~RNa5`FMEnKz|xlCZw~(e!6N&j)`=#A*wC%rUs$zrF7$pp*JA z>#jh05*rYf`>go>>0dI3t>QpL$?nACV(J1GV$Hcrg79yAnoiuR2Y$am@rLw$#&5-a4~sog(zGz$W-Ut0x$pn zZ}R^=xtvn6vcd`F3UoH_3(?libThu;DLPJ!CIj)bpD~)oNB01kKuKOIn9OsRyF8#e z0p#67yz|K7BauUZh1OyTyMAI4eziwLoIs5(hnUboQoI3PdE1t{jo=?WDwT&#jxxI= zYT%WMF&91P<3U$XVy%h9QHzfhLi)&M%t3)or*E3{@+^Oxe3x28gyOq?tow8YF`-z0 zbrQfBakP-aI3KSL{_?i%nnLJD- zj8TX6@*D^#^8A$l-jnejT;U9rrUrC{)AT@EiDwk%v{c=+tI|*)G&JvOz!P$NCjp5u ziOhhBKiz@@`;Onp{?eG|c_@=gaGa@ws{2yk)UvYBk$?6L_JNQ}1W_*yz2Q$4Re28V zpI0mg0(HlGgN2^*xlw`ni#wbLAPkA0&z^_;S6mQ7`}ty^o8l!hJxyg)X+~$V+a}cC^%H{Bx*$fvSLf57{I{&egFcsA=7sIYP**_RVX zhx-gf8b)(d=c&;#b)le+`QclI9>_ayN-+msJh-U$ONce`E%)n{MKD~SciqwfzMwS|$CKVVxU0*M74vYx**r5lheILjOJJ^My zj4<*XSR^+Erty)I{_|U~A-Dq`Em}WBOjaJ%NfBUF#`FBz2|-Jn z0&=lKG3xw&5o9*{V^~Z65r;32p#zSS>alnG%pkQ5<4Bq^UH@#(8dSsb^ zI47bFC%arFvI_?kFl~BagF1jGz&B02|1_5lVzZwI{o|1g1!?c+bUPo zvd;ppuB&~o3^4RXGNAHM?5=*PYUU~{kAeU>hr@HUOa{a9BZq?oyGT`uTh-CbQ2DlSV`wnuW?#uDxyiEgphxf*QXlIS@2imXU|i9GnBguI8L>>+75=E!>&2p~^=B96b&suC0uX0@*ALv_ zDFN}-_jjVyINZBI=@}5d*;6V+RYxFE;wz%E4{Re^*q}t>NPod&!PC*yCjGzCdkO@q zvcr8hC?FwB1?(yFdDPAWWLvJzeIi4><0XiS>pr|vz6_3AX{6ClyHAu|b4D@tU8L>3 z4m(-8@G+5-0#cQTBU6}$%ATB@8`Z7*-_faND)9%KmWpB)As+>U;l1|l>iTVgl9Yg z{LQ2d6v-PFA|tQ=bW5F)^u1|&3opsab?ZfHPWIh3e792^*FPY-K5g6YY~8h`rvVd0 z%HeEJq5yjoDd!TSv{xW7IpekO+t2(}nGpk``o9FwO|W2VKy34?l>d1FYv0)TgGKcq zo?4KVQ+V;zmPP`X)#`J#{!P`rx1AX-9c&DDK8942RWIBHHtCF}4KIqfE1}qnnoouD zuOVo>7WixA=PwF?kCEZvF*h~--|Qzc9bm9SL2Ia~lBaSue)h~v+QGcp21>Cdd{mj1 zxR=-z{LRVgBjx4OH(CxgvA8faKxoBir4Yf4Oq@16f1I$E0P2?F%K7@gHs-*;fEC<6 z*|-8$gP;Z^`AjSroe8F?Pk^qj?j%&7H}0fH7OLUnpZFO;<^7eAPkud;Pcmji^{^|) zwm=tr*m6j8w@Q?&BWL0<$2`2ETDl%u;2iK`T@{sZXvcw4_6!&P=Thi{nTWz)v0DsX+9hCUIOHt+_0Vqz!v z^4=V_x#M699)siv#jJdg^?m8U59I^9mhtnw%PPfFT8kiZzs(-EwLS?tcKu}(Prr(Z zi`aVeCIv|-A`Mz%Q46@nTDfZ%iX1q?@Bc8Ts6T;~L#M)T6r2b7NvIwmjKAu1=@Xz% zXv2TiHjvNZ8-;+l_C{TSxD;Eo+WyX!Z9wG0&F+U;5l?Ms#$XJ;y+9q`OddZep~+UcYqXJzL$B;~nB|0q*Qdx6vM` zu!_UA@|nC*^@bBpaQW_SrN(_7hndD{ZQ}DF5J!6|f2@$=m4Sj1g8w*!-wka%IFMy( z$mkvS@5{@}Pk7_7=%^bTJ4Wy;k^U`#MTC3i>dIu2{ao4lke9jh*(!zZd}f{Ts2+Vh zSIUqm>NoRU3#LoGKF;H{vkhg&P)^X6h%+2alJL8e!2D#&leAZ^Y)3i-Qsh(7#BF6Z z#V^r9bj~Tnp%z&Uns`;_0{Z@*2qytceszi20+Q@c5Y{%K(}fEG_^<{mM8{V-;^dweN|PsN+ERbgT_XSyUa`= z9XYDN@<}J`1x)6kaV9Y{kG3pPJhyu>)XPV0o00&jPYTYNs2)c?V&D@N#6O-|O$DlonXG7auE8`v!c zpy&C3@QDCL>R`+mbCR{7rV!sZ-dCvJsey+44}pXtmi%P?NBS8gcNeQTD7dZv8ZERw z0>1K5La!1EM!2Jy|E{aC_LM}S(oN#V)PzOQO&%{GhzH9GN$#5Z-v>w6_|#W|s>0rX zZJu%z99Jw%IzW~AbBFRdVm2Ouqc_i_YCvoM_^Gd;bGVmbSTod@#=^3XPp94Y8T`rG za6C{r2lsQ(kBuD2BaKfjw^RSs$lU@)?nIP7KE(9%M@Ej}uSQN2FFNh~iw>5+hRxmj zq>Uo2$ms_^{D8abfH3@<|H%*w>7(ul~lM{^|+1fkf~? zLeX=>Jb}fUIq9uzsbGz)g)__tcy-&4a*hGO%}{Xj;Z^&wasET_nn2{zNzJnWnU*zR zw-dkV{ynG0I6_yl#N**vbFz4UB)zE_Z@efb?dCWFId9r)jU}jr0dsQ&?($HDy$`oW5pO-#c@`fh&lyr>4TScLJ zvtUYz4vW$-ifp}%z~QBp)<7BJKh|6605scMrQjWK=%B;~7}cHf@3dsKD@jvP^n5QJ zSXQc7K5VeSbK^B8sp80EWF73I;?9ZWIN)k9L|lJG)|~W3_nuXi0RPHP4)%J%R49Ff zqV4@Fp_cb-ckHQcio?7rX6jQ;A(t42PvzF&R? ztjvBfLh$;lnSD2Vi%3Q#bXD&R3H@yVK?X|0`{V*_Kk}}{3E~?9CPbr0vMU!IhufhUJu5xE16g|tO0dViB?iJ(6v7H2Z%ss3D zGK5N59KZ4foH$l9B5@0^MJCz#p@ud>8K@_G2lRwA#@{i$nlMdKK0kYr4`DEhTnO8 z9#Ht$c_%B4k~{VesEknF!55n<%nV)7@>)@JH+Zf${{nhxE;C4(FuybWh<{}!ur#vd zSvF%Z5s(MGTc)a@P+*t#rm^4lY4a=kT@gKX)?HJP@qO2Hk^S*e!J5+gCnkd|Bi(2r zk$GN=<#mN~9ZFvXwt9rY@TPsr?eQ|QGU$ewul@Sc0_!6rC4MBim%872PIl1rn z&1LqoM9IlLm?07$eccA&G$C0pUUOhZ(?jw|4S?v7rPIA~D@4~$y$3rW)e_`b;W(j@KmyxEbdjE_>#9$QUhFBVnnQ9 z8cLVtA)D6VL0uo3gXz!2;92JQPm;bTCTll$8_FCFKwVjOzD*vut>5;Chds)r%h~%b zCKGD99P%a8QSj*dOTyW__dD)Qtnrz8&8KTx6vnoDjuv-3P0-dIi&8y8kDKY<>PP}v z__qN?#}I;>cuOMG7qq&e6!E{JoJwe7l1bB33rL@Tfi&lq`&Axf-e_2a(P*_BtQu!+ zdh8UP-cWI_&x?2_%mKzrhmS?&hCaBc+*?cOa@eFZt$Q1CXh#T(p#%xdx+j;FnP5`* zDMTs?%TbWG8n7X@RXcTARR~6m53+PkZac3I8&0vr6C?b$l)qsguV@MA!RoO;W0zyU z8O&2U-2$oo5o)kl%I5Y|^uz_x67l-_mIL#ZnJ~audo_ZVk}-LoKqPvaGhSS#*u@3? z@x_}<51y5bWmS_)W8v;Wg>hOeP*#nALK7&W0e0JlU)Al%eTp9DSJifC4A$_-1(vk& ze%L1wkl^;K1dh_mAbFftS03fsz2dEILGDxzUSkm42V6);`Ees;U{N-&xc?2hhnC+8 z0Y+kpz&4NFx+E~l$5JCZq%FaQpmk_zvd749xSUcE8A4>2T^X-q6Q;n-OorG{@?=Zz zv9;&Kjj;szC)v{HH;7O^SQc@L*9jHC&Fz;fTHE$X5cX^L%~5qe@0U>4f&VhqU$UzJ z{{cCej9;!j2bvsl6Nv1m+y0A^iYMU;URz?@g^ydbf}Mx#bN5nP`{+9T7geilaC;fw z6}}PA+Nx8+AiL17$sD`p;Kpl6iunfkP-mFog(&w1iBN-0kknugcunWI10Vup8joW+ z7*6od2(dL#WK2AKmU3Jc`kf2C??Y)o-z#p=s+Xa12HoTnr@LTMQ|~&4(dw|sJL(g3 zUb^RoSEHWjLiMjvM!Pm1(eTIw>C7fzFE4-rG}Wf}oW$bsz(EKx96JWVGT!xAs}t4D z^BQ>MgIBb*4fw7;Xa4M93#N5V7~e&$ff`^i>XsSbiv7N%V>$T* zQZTIvnl0mbRPKw~ABKSpr*aTW(-FI!Jy*H=?if7s$MV3_+aRyaD)sInbZsUC*gve5 zsviuNII6zs0_Z{AV+p<#W0(x~si!I^<7k(H*sgw$9%Fp*=4guI>>)Y>DEUck*5Jei z{Fwc9mIK`L8%Co0Z`Am4cdOGWhka8GT%0dBoyq$11R9RkS*;51>)p?Ih(vJ1WMyU13%yheWZOG*q;r`zk5oprR=mTcX*-vjT4fftv~z~Y zGXi*6cz8Vqp<+Q~9heSrFP81uYQn8Ur}ab32`L){w9+XR>T2fD@7msNEFfYW*JFbmth0-a-Q*Z1j-04l?k zjM79MIGfkca~$XGu>omo(88Giy5dV}BU$eg#wRrO^V8@=9D>iVyM1Z3IS)W=M0UxQ zjuJHuUz&6>zq1HmQvQDUjeqPi6r+eAN2j>JQ29wqm;#arDEWRx9j~1)vI7M3s@G97 z=GF}V>vfX=80Y>p*7o|~7)%3v0w?LUQK*ImkFTiQn+%s-=&T%>a$79BfPtf*?6s_* zCId=m#2_1KcRutH0>a}8Q3?RX-R>gLiSiPWf>17n$MnC>XnC**Q>XBp6_4&K)6<#8 zySo?0R~j4M=tb@xQSbKo8zrk;#Wub7Q?o=&=C8zi8T1uR+g={v@stO7t2oDAL1I)S zF9Xa~;-erWj1nl~>aicdIzNyH2%W;aRv=2}9tc4}jx^9aRS*vd@@V}PuZ+Hm#cMMF ze~7(*F`k%I+;z{H#HhWxzsWZE2$+O(La5ZRc)UsP43UV6wZH`EFM^WObh`idGlC@W zdm~O<)3LjWdsQB+;hz%YADZ!<#E1c196k7aTfUnXhEBG{s~fl4_iHp|gKRYW-;-J% zX#{RnM9lrXv9XqkiHUUMvG19uG+t?LgS$pS>%Q07F|1zap6Zj75a zUV72JIWw2(iKJ!Di#~#<5&rZx_ae?AJ9YUNB*6F%koB-_4*Z5+G>uX5E!_?{{>NIa zQb9BbRb}^M$)0xukfVNCEtJ9aeh12a1jmV*f2m5l7(~j8U|k%sDsU;J7|hG{?(>25 z*=1~XYW-9o=Eo^|jpXALefGV8YzAcFvfcSp-r$oSgYPprklA(Mcsj;nRp7D+xW%GgIeuMT zBfYOqW`TK!6Efenzis19ZsV;hL{9pGf0NN@qH#;Beidjf7~r48)M545B#x)&$fR%Y z!Jge`(ufAa>>^$4lV=S-*%B%?&O@XIqx6jD|g<=|YT#FY6j>>3Tt5&@GXfJZ*3faZBVqJ(&alr5GD~Zg} z(W9`UX%T6M!;8S4^5TDS<_9RCbrU{J2hy$C3foX$+DHaL=P+$%S~k%Rv*HLNCLH&e zS&ZVPZ`;dW4C2CsH3uas7Q=#@I0AXp+~V4YtN$+>-rC>0%74f9x zNdI0xeFH~e=_pV7q@uFY8)K}({DCtQGIRtTvp@Fty{V6Yidmm1BVW7_82zaDv;>7R zo@@+JBsv}^ysQP;{jZT~v8BPi8*r15{oFPJCJ(jeH5`_3IhQI%}7y=QoE3Q=)NyXQcSk)P^+ zrx9d*lSNwJz(hUfsK9!ELAc|c(|K}q%#%R<8r8<1C-J@tQiA@B1*dd~5jp73nTdeq z>vU$|33D73thL_Cd^m2)9F<{H;JK7!Sa&ZG76hDWNf2w8ZtI|O6Q^8sX%$1%iw%81sKo?o$(eHN`f}}sOqErJFp)d!k(q>8L!^WH#$=Cv$NP#L!m7Tc=K{cs{%|5qz@xWImDXB52!)AJ#}v z`$aG60)U5X)qAKP{udK{A{ykNcI{cPL%T0m+)(*i(D7au0)^oKx7oh+k~tQ76ebX? z6;nBs2k}Dpb~rr>-35ORJG{!Q;;}SPCw-9L*{Sxl=0$AnN-q_5tA7N#GvM^*;6iH<)8Qa?> zaO+Z6lYVsaLnih*6wrOcJZ?QaHj0Coozv30$)Ze-EP*;n`Nif%iG&SpnQAj2k&FYl z*nZ@OIiB}Ua&7c7Uqo`2}IA-{aT zJweOg4H%MN0C4{h%kW)gOoW_X#A9ZTqM1zANKY| zkbN$D47t%Y#+Y;0*l}O7#D5Y4IXWiCnwHqKZl}4c=6!sEMvPsy<)A4k`+fOP+Ax^e zy_PMZ4{ATx%VR}rv@6C)TDU3fuP8nDCqC)`$o;MaWx4J=zEX*l zEbn|>g|!LfFX{~#XdDny30k3 za=&pSghFk%%Yl#u0Mte#4)QcXJ{&Mp#m@SVpZM)942soN>OapXj?Ibd$t4m1Y|@R9 z;}RtXo9)F%YU`u-?DrO#yVuEe`y>U%74g5i?1M&!l=b-8i8nemXU|L40bC9=Pg|=6 z8*=?5^Kq{+Ub339W7S^`&#xv4BCmfmd_D00wv}(NGS3x>)Lu;!$K@fn#kxxGJ@3mt zL18Puh@4$CvJE})xMIL_2G#3uR+I@Q^Cc)4TNebcb&^%xAG)gKVKsLB z>CH@#pHB?wb)t(OBS;W(OK82cK+&(Y(4LPfw90XEU(G)ijXWdmRUi(33CRQgZ}I@9 z5m(vLFnL|z-`wS);{Cw#5Stibzo{w)CZPXh$te~?2Q>4sF9BD~TH*5f$AgkQ>xnh4 zp|Ofz;wZ!n^xVz7{_;7?ZuyS(x&0l2;%%SVfS%N?op^@>F&t}vcTSu)(^bE@KEuGj zvyq03RbP_9f@XT7!P-uw*d?6c@AksNr|&vb!OHVlk|BhN!Eux#k`XH2*_;B+W?ji~ zx{aHeW;LJpam`pp*3jAk3P0QhRa}@iA(9mXFx$3KZOlfpBB51Z6^-&9i`%m}26A5x zTZT$2`<26-4_m-TAu7JvIJFk!4IeQOe?ekpUBq;INBqG9c04gLObOWIn||=bY&8y9 z+z84`i2Zx4ls0V7A3d@tVnRyEGUbsfiK8DrPC=N-rrC4sdU}YqL<-+CX?gWZI5(Rs z*K2l_pRzflA-CLeeswP2WWSrsIIUC=YgVjs0uzp~-$wi7PK1X3+m~5LrB481JNVC< ze^IOCWw`!_qB?D5YD@&$Z(rK-Xt2cY6)Rb5B6vRL9VLyN1#jQ!EZ=g@9T7`I0qdew zUQPS^Bg^c0)6?^7U!y9N=N6Tirtf}+*Al^tUAGK{aC;dpIlL2p9G%HfdlCk3HGzS}Qd~;I z`F+o5=n=#(x%;GFKWTX8Qw&Fg{rVNlf`%S~;d>UwghSwuab4T7M|f>^c+V$mzqDbh z=uTjJzS(ZGp2Kv0`>BECSa*g&uMRn!n>K@wG>3boBlJo`Ht4RJ-F)&`MzrWoCOyV% zX8t)AOqtAYy@A?NUMnY!4_A9aJ;r*}WRt_r5frab`n#|7I&t^42{gPqQ&zEz?7Eb}T$|FH&EEX>exZ8qIv-8AnCkal~h8Y~Djx>SOL%>TvzmSD^McT+}39e{1 z7n?_L#L7T$L864Tr$u>6ax7XZbAV&&X)>k3t$DYpI9u)13mn=N=J?Rl$BsQ76IZlz z%)}xt(pZwg60-=sU!xZvy6d@o!TJ88)Y<>gDHqM}v26G9tUXLYtuUQ_UDMG;e zKFA&{nZluqlJTU&HSX8=vb82=_`!zKeF?uj1nsp%@V-xv8{w_Q;E^LH7bW_iXWl~u z&YVXAOX9aPk59%0gslALgI4GLa1`x@s~A#^kV{&e^6IM9bL-_#xl=w94Kt+XyZ3N| zzXldDpGsl0FH%QV;~eH@gTtn-4M$e%c7HL04Uy@M2ZSXa>@GU{Xq_=`jr~rZBI8ljN8%$u~Wx`DJ zGv^^e1fkWy?NUzr&)T~zYwK-${48zeEO9B9#(6#Wzl*reHx0Ma5Q;GPUm0Wda!t7% z7m0Vc(u`BV5^L_2Jx|nAw*AV~d~<|$q79D=$L!m5GR2k9Vy>Dhf|%=172{ay?e&~R zmSj#zd!7qY*L%r(rB?hRZm<8#0NfnPM`~bum7ggpWWBQ*0?jkFf8#|52OB%m)oqrm0mzi^z-+sE@Bz zr1=kiiU{ffQU?;b?&l8&f7Oui5o>TIemnP=i-=KN?g;%-e<#9)F6N#%nN|-=jX;uE zyW*}x2TtMgd95n@9zD-j3}$U60-NcY9rAAauO|!y;=*HdgR{!lom`zJIlpv$oMF?kv3K5y^8Y=3x76m_rT5Z~_kP@Egk(56i4%ms};@Xqo}?gd7uIyT|Ki zJ-34E92O(P-Vo+AnRJA{(sgNjEJvmlpRiw6LO{k^_uyPv38#P?x}MjzaY<7ezYQ=_ zm$RfUc6;u#Rb93oPpEnDo{Muk=e+BnCPxDEQ!;K91^I?vKcw3IN$dJKkK&|xx6GUa zu4W7Q2myKorOS%nTWij}Z(b=|yNAT<<3n5d<{sr{&(3)`(JO4FZz~i&1t-gc(<`oUj)Z+2f)f@O$6xVH8bvyFDzA-> zz=aRia@Q6>^5j2d*4z*FZ%tyYY*SBf9sKH0T#Png<# zy1+7vcf2+_Ncg)MJ?5Be^TC=q!=h(kM&3OC;{;k9y?11u+l4(@r{@)()R<4+33@5j zpL0V`KIbMWJjrP>QB-KHh26ZPA&lCyrOJ9*9~fd=PAU>uK@v%qoYwv7Hwg!-lZ-RU zf?^3az$9Mb5J`6k?UCN5^I-{{M1pME7@NhT9IDhgBM5crS$oZcnKdz9ba< zP5Yu@x;_GeM?h_sd?L8H^!DS^+cKOxC!ZPt8KNNPm!bJ=daQyiQtt7SovnjK#o69} z`Ig#MMuAofSzTeYTxWe~w_fDLvHe9Z1nZj4VuhRT876CP`Z5t+#|CXlp#*X)bCi#g z{@B`;*r$jyPAlj7lVGxD(J-3|6G;Wp&Xeoa8d4{n4=p%joRidg5@x#tpI%C`slJX( zEHG#h$x5chV3!sqm=S7{61#n_Wx@417@BhN+8v>Tpdptio6vL{_vp+M!3UGEf<*hZ zpU}jRMkj&YTYlxG_@Y@?#+z{84=b|v$SlM#-cWjkxvV_-1;*};?0{@f9`WTu92k>x z0C@%$D5q?>mq(7`40G_Q($oVL1o~SrW!RBVK-uLt7Zg$X8M8Y}M{0c>EBcn8cd%F^53Mp{9sJ^Xx}?uC?)RN<`h&KFV{x57@TJ~u*-`-EK7*4w-|TZ{ zVbIezi6$v5)mNhYGf%_oHlienMlVP07vo=JI#*_b3D=%H&r;POqfM&gDqEQPwk&_C*)I5J>?d ztZ|?-5}&S?E)!&Fy+>qtq`GM#|L4E#??JYooe}GI?u$QieYm-~ua!DgWIP+G_9HRh zdG;!cPMPIIaHsO!H3q=@5udN_eb8s3=6|5<#USPuJD{tYr2TBApQhg-DxHN^1>f}9 z%!q#g18%CMifh|*qlC^;s{EU2%09Zb7cgZkc4z$_bY|o|;4GzXs@684(-1~)$Kx_> zeWTQpkUJUo{bftBI{Im<=)P6?`Z8fMj{eIrcER*7L`vVWcb~K!6t7aUA$Fv*pWr#O zJ@ua5)^w^S->Z<)9KKd@PIc)a-DycJyYV1q%2aIQBV}uena2P;#1h^!W@j;}CD33@ zH76f4OTZC1kMLe0DoKDGuT)=pLqf|47^4RJ%;3M=q$u!`9K4f#f5>Txcx4wXkw(y| zb)fhjsa@`5w75>3XZZ!VPG1TMmW-+HhiCe3t9%Y~%F^ws!(~18&YFXcgu+GJc+b6B zx@%G^vRmHG?I`F>eX*5y6fLn~kFnc3$(&7!e%h4xG~qH==_d+nOs0H~^&N)(*9$e3 z6!?-%=SxraZD`;IH*Q(rNDjvGMuw^fuDs-7L)?zKskX7;MHk<*F`qEcIrk=F;{(HW zWAt|G9$jk}+5)Zw8gv!GdBos%Okk=$oke-OGMHB$RwTB?*(6ls-8+AGl-dw%e(Ekv zpBz5ux@uwkXkX+_RCk8``V$G{_{_J5%TQGoQ>MU< z|1$4w>)Jo(t&M&T%qRDO;t(_(Wy59o6J%8F3I zx6M$b%1298_F3;Wv%Y*gzQEwZQhWF7z|lm~+m0&8#C&-gWqHepuFK|*BDlfY3VFio zV#rR~tQ69Jw89MWQVBqlcxJ1ytp{J-;m4E-q@_KJ17l}CiUW1!)2|r={h$D1pHx}o zd4fOV16syQ<~c#)#6gZH{$vX~;iKh00-g>REJXW$mH5RA(qPfjcC4UsuOX zbg28EQ@|3PZdiN7fKV-#)dKS$xBUXdiQ;wqF2K+?->f`MWX1=%W(sH5A0j<~4(K!W zIsJ#egO3=4Ke)I!wDJ8G*xPZUYvKoEQ@Z! z9k9W)7F_;6c`7~v*o98{gn;tjd{CqtE)=P~^i^fvY%MTL1!AXxh4{*mHJta@W^bcx zv}e7VA$_65mv&(9QEjQ|>{#b;PqLb6zDGieueLpsbh zwNKziy924|$(SLSBFjI4uJRP|hLXq*9UPSIC7Da_C!w9(lL#dEE#?G*fn)_Qxg_?- z)}v*B#=;5o2V1kCZ~K~UAPIbbQ@??9+qneDIjt?VJ4kid<+#>A@|i0SyLkz#F%#*+PVFc(jKa6|fMiS?~0zjsEv$u#@Kt}B9@XTN$4kzO#{ z9MO*?3hEe2F6bP$Oh%iI*(7KUl_0!4kw%tgU3O>#g*8?I66suwFqL$_f@C0czsCd! zVMuSJ?$Rxhtbd06A#dQwl6^)ui4 zBgEI#&nI!3g>?45a!5G#ea3HjLlw@+6{X0YyrsyC}m+ z*96Wx)wREG?fvVVbFIrCOK{9F-|^&q-_J8N2?LO@4*$zt5XaFL27cRlu~-P4!g~Sn z+6?_4-c4Y{EET{hd1LAlTA@*f_i8}9xD|~-OZ2mo z^GHr@$ao_#1nlsoImF}eaoXm^JvBc-9R`#Ujd91xZDyT^yYI1L>rQHiH*#G_LJw>& zQSa9eqsFM<#~^xx8OIH~hvvN3eobhK+uiaM&t8rcZH$=R?boz@UReffR0xd`nyX*; zP8Gadt%L;8N^M`@ZIZ`c4o#KhddJVSBxrND()jajr601wm)etx;0_S6rT*x)(oEW4 zCy$|dK6}^9Gsj==&NPuO=Vg+4x)OoY+DD5Pf@5CvTMy;(4Z>fgif<5oo86H`1+DFJ z@x58#!cAQ`lR;8^;8b6}rRoim@1T-&p1TJl6q9i5;V?dhP&7bw0bKowE`eCU*Mr&Xy z6Bk1@>-n4o3q=j@>C%YgyfeMujSucaEu8Q_0ns(6EnT>1c;LoUR4wa4E#ZJy;T=ry zdelTjPv88M6O$iQKKWLgIUJaEAs}(@E`=NauX;wRjT3F&f5W5hX0D#drRa`o!E*_b z3C{I)^?xM}q(CYVz~fOdW_nNW_dFM$d7lKEjbnzZi=WFHH`?Tc_U9!VemQUDd6k*jU*Vs)Efjhdb(0<#k9is#Aztk8B;jtf}^VIW{(%Rw`pcJ5}$$ zV#P>KAejb2(F+&#To?5$hwi|`YMO{Vxt60cacmhb(Va77owq;MDR-!~&U$v|{0Fc* z`T1@g|Fm8jiFYB1eqO|B9h@g-g#c^jKKQ1rX^;<99mm*_XjP9N%I@vN%i(MRptk0T zSreahz_|U_s0oPU+`u~=|EDA8pu~Q&2Jgm1si6CC7@RdiIwYR<_HBr<9^>|(x^NRI z5RoU1PuA8bZF=sjA!=@?H~+yLC966$yot21Gs_nq2IEpCK}yX9I___D_xypo{IddLJ9~VMc^xKT5h0-iHHoAyYln9%dlma&^AZ5f`c5- z*M+Pf5HO1gU-ZEpN#=gMEl~T6`}Y;3lG@JA-~?BYQ-4x14n(C7a)@OhQqQB;$U0Kl z0|>k4d#a0zatHjZ1lfk=cPIAWl>-$Wc*Y^O2u=Q74dW6zenfgu*Gs8AixF zdPK%&BKgFgkfwo3NCebUh$~`wvS6c#E4P)N*_cBl2hs&96j*a+5!W9UM}f{lv*u@@ z&`M-H+6g%ZUa0_!s01Bn0DvUM1yIrq`-9Z_G3$Xb5^@u`&l;86=l{o0AA=tpOnT-D z{`W1@Eb5_DlF;lsv8I}}*HIREq}8)zci*a5;f zs04XtIJVHAreN0MS%OEUmMT<@&iVIg19RK>_iT6Pc1dYd^W697&q=DB(Kb5Ed*NuX6 zSTsbrFCd`^{^U>BTWpydd+(cno55-SSMmc`KmxeqyA&RXN|}WR3+*a}(){P6C~0`kw#!^crYdvAt6BymPJQlJ2caU}x*TH^mU zN+itBk!|@s64-K2toZx?7mx5u43Xf3IuQ7u=2HU9d%zm7bWlM|WQ6@=A5F8t4%ugvm71nkMDE=UKVv3so!e#zy|{i& z+>}&yFj^*jA3A)#ks)5$CGRX9cgzU87kM^`jC7T-L8Tz-3YJDHfXI9Vr=nJV!)j{6xf& zXLbzX_(#8TUH-Q}Jxv35_$e;uzZy(Lg(-!BNbJbn?rGpVay?kZ2;2kTw#4V>2ruU6 zur_OrEEVpbzByg1Ra_?0JNlKbI)YEx|9pFV@v~`pW9JFI6~o(%5sP^U2a!}w zKailMBDxs9BTW*t8AN71ag?k0-}khr4pR#y*#7 zy8>tAd~+`sV;+HMoOTh3qx`FO0S;${IF;MtiT}%cZAig6&&YiW^PXdpVoU1N%b?%8CzOgB$VIT}hOFK2B$uuVCVevH zVL~Cmzv~2!&C54nL>zAu1^KLrOG+W(Z%kuGf=oHB+I)!mpF$;(r0`Fn@-d1t_a!)~ zdc4I`k52P2I{cs3jh6jSnSY->=#GDRoTRvwp`j_x(K(D(rlBc*_YFMgjNeqFBQFeC zfJxVB;dfiPBvxe_D0S>+86a!_xfeV~ROB;J!N}3~QHapn3|>*c+^tYzYf5_$`xXny zUEx?3Rg}OJgzfv;YVlzT1tYQV@TEkWtMd@$+Ax^+i2>$lN&5FUa}-17#*Q^!f*swu z3&1js4RS=`{5PUWRbj`TS^ZSCxAh~%e8WQislN5qThZR205V`YJdz;(7|S7DSL!in z0frJdU_b&^+k&u{>(j@1z${-TQuGD}cwqgyY|{0%mEC7^RT1l-ff>=7UP;Znn>OZ zBm01B*L=fic+_TxHoagu`l7V>_UYvnP`22)0r&WwiovZ>XtcdT$=5#@Z3zVQ_5N5x zm(+jUc{l=vVF_0ALfg(derTh}bA0}yl5CF!2gM3p{3{$havOADjry9hlz*&I9a!VW z5q^+g6aqfda%RCmnP;FGv@Yz*GT2giDmj)$A8m>Ny`&rn0-!D%pPP&W>YjhQn=0y2 zDeZrfu44n-qsfIqSKmV>K$nH2#E?y=gpRWM?wKFaZ8Vn6(}WW#>1y3v*Jyg`o@IJ2 z`YRu44qh(k4h9+}5;7rhl4srl0Zo`h5$J3rDjizzX_s?Uf9zX9M3`t@mrdqzHQ&qa zF$lxna6&LZ$Fq)Aj&S!tZ?tqSsqJ^rv6Xc<B zDKP620{{X46LuzFr(N)5ZsiL^0*bJKu?2)p?Yp8_JlO#|s}_X*9_& zixIWS7Ki}Z3P{&?4^e(2KG~$0Z41X=S@L4^F>R;MXzZrXV&mQZqJsM2bO;_B2a^c< z`G(%~52P#utXrLA15O)!5pv4bM?+<*o9+IEIB)MWA-Wu?A#jd6Q8|F&4_x`7gsjb! zRU}=VPe3?MvbEMh=b)OOZSMQTO zfmYMNL5w5U{?dz&nIt0Wyh%HZH>H|qa>>!~Otc}|hM!F!sGJEiwme>10^wzj7|4K< zY21Y!$q((IFjT};>2e=Cep97#U_?{2rxc#PgK~m(o3dP!)*6~lsJwb$wA`0yfjr`w z=1UY2Spn-^Bd~OSzSE@1qe^VLkd*(>oPmOvREH)|svcza@m+kWOPWOv5T^#oaSc&_ zLyZFw=%jM$*d;CjKL`W;c{Q^=3^&F{g;@cf+1G9s6_kS zo0@p&VM0vs$0TegOpW75&xC96Z@7|_!{9lZp>%PWPH1hmdWjv&^dz)KYIrn8jKRK$ zn_Ik#07feV+N11CwD^)bp=3H{i7Gm-pen`7l$ftr;qnEC zY+I!)VVw#b($fDV)*=wJU3wK=2k-tDrdn6{2UD#W9C@XqQh`Uauj1a?S*yv04-Jh8 zmU%c?=CoPW0>Y^Nw0yzvkV}27f zOwq!ekAVu~AEYFmMT+3MQWwuBwYl|qj^@A&XEJTH6+=bOnO7ppMDeb3tF44g!8SP$ z5w?zyp~-@!ws{5V?b1n)Een7*X@n-lN@dR?ZvF%_+YfK^HaPo1M|t)?Ryv2UQj=?& zIZ^N%2ZM&W%)cY)ffq4;#hga9&8|O_!h`rzHX~P}h-k^Guhz!l6NhYZ2#?1vvf!12 zFZe#IOqZAcHdyw-5DV)s)-3V}WFo@Exv95h=!%g(LX0i0C)zJ{13JuU^_yxpZ8*H~ z6G}88Vudv3dVV$J)hu|ww4_}O9}lv~iS}YYi!+iK%<5e$){VEnw;S=6u|R?V^9TXp zc|VOEk^6?fIFJltVW)ZZy1>1q*B2Kr;U*#(BsaUbXRK5aJ126KyWJerr0UkAx|QKB zEBl`<5n*_x5dSv9Y61%}T|rkP4=KW!*g?sE>|k2qn#R5Q*=wM*a9zJ1N9J!p7#2n> zBHYLEB%#=kp=b$Z{WWi8*6i+=+@m+;0k4rTuW9YefAc}Hd0epjo1LTI!ERnU?qy0w z8pK8^3)GqK$!Hc$KoFUye0+5K_qjZc1l5KWDrQ|K_;H1GIQb(md=I~l9xDB?sYuxp zy0e?5*_72YLL;=#*bw%4-@lLo@kOQ*l#7yZHZBRKhiu(Bw5WbxtL;7FS_WOaSWHGdssj4KI%4_VvC@|$a4_09SChevcS1kcNlNs zlF~j+dChUs896m&f2Iq1skd4|Q8dHkD02ifNo=DR(PpEyXA1LG<2!m{#!an#q-8sz z(Y{3B-IV?7yt)K&@%{Ao*KoYph(M?jvhnkqO2wDrq~Spv<9lM7MM}gelc^k`{nyE0 zVAdfX!mf2BiBvWfvOk0wO#%Idv$p>zsY>>q(5ANY*-)b1c%SR@-_8~^~ZHwoMFAo!)Hy5ARk=x5wB zKV3G1(zw`{^fYS`a=q(})tW6fay_=uy=&n@x7+BL@;V^C+RD0&l8kiPLYfSCzeIpR zH%oloG^@w_)pf_AD^aB@?ylEiFg=NrEZw;knHyTZbhT7NxvNx#X&qg3A_JxWp%&dU ziqcd$We)x2Ra`U{s{0W z)aHdI!YajP(3{xja^&hCX5aBu0U4U*852iO`+}Zk|ANabcGBZGm$;zzMXBy*Di>se zh(yD3hDncWq)xiYNsqz%YKE)Cz)ZX>8Af7E5rLzOKE@wGrbs#9gZiLiU*4ff1Jv<* zF~}U7YQVI)5|U2%?s)L|N#K7k&KIf5jSfiUCSrLkUZu@#jIW>^4~`uxb~;nncxLD} z8cJg5bIDpkYZmX{Dya%u@CoG6Eyg&Arp;@Yb}c!}E7gk1BmDz-HCi=NiGW^5a<66^ zcP`m^nZrR;hfJq)%BWtoCIb<}^@vJ&xyDBRC%Pr@W-mGg$4vZv^kfbEaqLWr(0fdQ zQhk{H;u|hUwR25L7xpcdoF?w)Nt|WW#DP*(1T4;Vtxw#aoEjYwVxxE3P1QhGc*ZEm zl-Q}vCErzq2Y+4S*^e=0BaTSyQML=Fe})~>FIU%NpB)Pa9fT=2rr%Y7DdXRV zG4Baj+)52(kZj4h4RKHg@HX?2!W=LMh+j^5aPTZ&^*_ya_p@f8(vAq0ezf9tU%{Yv zz2K8{4o~oFHxS74@#J2hYYcpl1Cp>rVAfIbGQxO{w?N}z9f?~0yC=f4;Rb>INw38k5{?7dn}v>Tc=k6pGSq^aj9f@Spq3fQf>HX z<=`ofENtDm3yACYHSeN83(m`CfrdOqXp;n<9CSTR&`Yh$C(+mF&E5r90>NbpGQ`af zylWhhexaiqhdN@VMhlh&AE~R>mPg))8Z$6gx3aIXW)0yyiZk_UdDOSctIMS#n9`N& zbZ~ieYGfk0xEa6>3i;LWXF+k|${>L%U{K8=$d5oEkbq1*M~&VSiEo>Q!J|*0m+#DH z>jFz$`xS~rM)mKZ13hgU3}#5tv7y&-uJZbjEl=r%WWrDx+Uu?jJt1nC(Sa9Xv}?Y4 z&-k<#akPHE#39|FHOWDuPKLqAyQ$~>__3?qs>gKf4Q`QtF%$5fphZ}!btrj10{jXf z`&w^$NUD6@=F)h*1O5%QIPG;tPyW^@ce(;VZ|dwyDB@V|{n8TCW|z`dc(fdDl;07@ zaE9ZgH2ozmGQWd+_q9JXG_k|%U6gifw_v1lZiyp{)QD#HrvU(6*weLrr6-{&8T$ua z`3(lSMg~kSl-$1?EJK7fb-`Blwg*29?K!jMpbLOHbrpFVQA5A7 z0AT&Nc#c%S9XFz|IQ|sBHsZkHC;8++CT00IqcJ1NJ!x|Zx?)qNHZ3aW^XkKt?bf&L zwBauY^^GlQ6&TFpw8aFjo5Ly4!**8ul+T;PT_s#N_WkX6@bI)^Rfnueuo3z3MVEcq zmCWxJ4D3DzA>L)mv*EijW^ALFcd&Gzv+x#UeY*bv-2GyEtuOJ0i#VcgrTPy4;pY{O z$Lw;-@!OV5csDjWf#@>9xt|}78<%Q|k>i^)wCGmqvKZO%EuMyTgUTAnByhM}-(*e; zxZ`| zQ~-O@WN~ZiyGZEF5-50R*wEdZsFhD2QdPc8iE}gz4XFzo>a}n=87%8qS>Mo$pAfiT z-C&ya>(Ar}P`xg~eu%o;*Zlw*s+=6!nU9y2!I%HY&B!qBhNSdZ5c#~OjlRY!T*{%n zh)AK%GjEVScoy(HS}fjO^4XOXqk^yE26pbAp^$7&_7{~C=M?Q_l}d|S3q<61(L3+t z_*3J*+I49_aq3`T)dLCOf$mI=Cik zQ0JrEfULaB^H90E^C#+9dIOfr`=4{5Dh2sGGwBx`vPywSCH3pgS`I^3+Y?8lnFcoJ z$%94g+>M%ovwLQ{1<5Br3;v3_F&t$rfPDAEiLKuc_f@)w79X>fxM$)=0=fk%uP*fN z)l;Q>^hU|2mOQ4QyxE6pVs`H|nI3p}x8ni}kRbX`{|r`gB4AZj^^aL(n5@z{+9ihc zcd>8%lxti&_}*OC8u2EV=yo(zQ^{xn>%Ikke2jF?_Ql{v9=3?!>T-ulz2~@v zU-FU+3UOQ&w>>wu72orKn*T}bLLo@-43{pHcYm7cx)>z5$>nse)qj-+JWOpHdfHpn zSE}<0k@~^X%gtfR6dgn60$fiv4iE{SEJB{u!r3&~PBY73X;bb?+q75G69EO^c^!2> ztd%QO67e3lV~sX{rgh0b2znS_3@CuHG8h&CLKsacYP0vF0lVtEi+WClhU9{~KO8;U zEcEc1WlNe+1$i4r&biDGD|v-q$@L~Z?&!kR+=BK^hf+A^i!aSOKs#JjphU<#X-8)k zn|t3$xr2`Asb!qHa#;K1Xy1jRkRtwT(#sUzKy105$){}^PL!Pzk}4Z#eIInGHV>7e^JvKHN$)aK&#)5nc?c|)=s;hbQ6}(*m6_QU$ z;AT-^D5mb)9$~5s+CnZR^BG6k8zwvduKIb$4hD5w)Y;r*MAKgC|A0ISKV(cm1&`+V zETm-SijT`wzN+Q?{-4ZoM%JilLEBdzxK+0_IEz#njTDCbpf>r|=_n9Uz6v|r=++n@ z{jrM~yIO6hj}(jyyNi4x)0{J8`qf4}|D&o#z!A6PygKAM3`%755m?Z^hc`U{;{Tw5 z1IeC002=7Q)_hP3>m5Y-rWswLwn3aM8Qp+L2 zhkiSENyO3cS^PId#Ziqc=g}xF^?fKmJO}c@{2n}mSh@Qz4#o+Bk>P^dtG%4L6T;~a z`;TTm?9&$HFshJ_jYc?O*RC%8! z;+TZI<43%bbVbw(wn6m~-*H*%&6+TP4~uhN?n{{_X-OQkC^i(YB8ZugtQutzO+-+W z0Q8jv=?;{%Lp3LLfKEZLG)FBm%r!4zL6fNoN{=uO^L{!m%bP+JBXl4wmU%CEH-Wi7 zf6D*rRq8M|R`lmNB|j`<$fey;384o6N3paG(v$e7#gdgbmWg@f1+`vr%tS{z{fm7S zKZZj;-)UpcWl-Ze3wq2#!kUo$Mq}w|K<-j3`d6To-lJ&?%A4@`@yNq>qiN;#^>tp_ zX5>)jT6A=qG9C}jFel{OdT}1Z{-e*b=FMi~e4@$sX9FG_^!OHwJ&qFU#*dG@o6&RJ zcm2H|G5SE;cutb8$Y3r~1+eXJ(OCUw{V3#WMYK{i!j%1VG~dg6wcU*Zjt&9M2~t+c z<8?%B$~@E(i%@tp-V`Re9n#C3VkrjY)p&kydqpQq&Y>XjfxJ^sBcN{W+w3M`)_cp( ztvl#|9BBlk-V48GZ)o;q?OZGRUyyTI6OG!rpftn`Cke#G8I#Hop+`aMq=lAQ}X1OVpd=Q}Lu@6A{)EUN4QY z?r3vRbFtlS(n_=aB4dSDj3De(@$o)rNQnfP5G3Csky{nr3Tn2>oB6Wvof_w__R5AD zJl`-of}N%lHQw=`hA0V4mn;_7(v(lsFu%i`QT|o=(v|2cU(nsV8$oSH z4ish@nVw=-&ureGIr0^ajNs#^1fnbQ#iYqA_}}riV`@8?-&G4@ae0sOT0seN{+9U{D6SQ(#42PKM$p5F*z7&)uWv8 z6Q*6VEl*6{Tt@#s2&u%yWu}KS<8q>G3taj=cI>KHXaK)JJU~~Bq5|st$4DCYXHQ4n z%o-x!aTqVIvQCeE+MiLtj8?W}>x-bFqzki|{^ic==3apmCMYLxvq?rP#cRB990VOp zvjB;0ClA45Aaq*+?*e|gaR`;xW~GFFvpK@`pG9F{ZHxo?1e;vp-P%UVledPU9bt0P z93-X5tGrerT`uS4pBch+IJ6;t)&=VJ+!+#r`Giq`R#Gl+%E+@O4Tj=y33ZxlVWaQBtV{duVa4)(=O8ZK&@)8RgEfaI ziFir6=(e8CZ{$gUo7s!vH~!sf%`+iE1pVPlupc)m2*Hh`V=R}Me=slhSP||p=K@?` zCog`$sx!j7r6Z&ljtupDHi%CuG$H!L$|6!^REA$fqPghFUQ)|z*Yg6E%?ghNVyHQ~ zkF#sOJ@qBCI^Fg1z;ek_&Z*ns>9Jb2rrz1nHT8{Vy?S&<$C9{k%3AVlWpb7M>uw|z zE*i&ysj7k-7tUPD&rge==DrOWgHxszokU$r1qkW7wse@A&hzs|=qL6>+7TIrJ?UPm zV%K|6VdURn_bHBEp!$wH7D~Q1{rCGrvtW)?iD|3gOABKBH|K={}tmvifZT1#(X*(pJ1fdH#lbi=sX~({Y!rhdMW3gBWo~U(h}z?|jI; zZD^+n$qCI!Jx)^-KCWVnOiRXGm@sWANPDk{WkIUO{ z=Q5NC=Ob{e{Pr!Ya~-@~b0r8~sB0zgIHPu*t$YhH&nwbSR^gf{MSq!MwM^T7Y*HfY z+m@mut`#SOO#nSzQAyusBoXY{(2LAv0{0`~IoP9^pKs5cqRgJWmIfa2dd&}k6TY1& zlFZ-O*#=-n$oN8_J8)lHw+Nsd`g&a#7C^2b-FTMEJq*1$NM2Ui8NP&+&(7!L`fB=Cue!@CymdQt zAqrLGXUmL=ysgu-m>smZc!)pM69#pRxs6LU)80|a!h*m(m2?qjL`gpme) zY}rYi zI*WTXSmQQ&5K*~28-FP&NW>0lQs9f$1dWOK1U<2 z8>lhEE3=uD$;msPmvgzdu`{=$Bydu%uP0|ijn;rE~^kCgPfR=%=?tc?gS{J3Q9 zOzc|)ibsH;`bW#}f(pKYPK0K`ahlnI8~l1ua2;lGqXTn*YTbai#>4lN9#PYKMgRsu ztED^GaKY=pB844J6CX!LDr-%6`MtC- z6G+SRJ#JPOGreiXrtYpZV3r(OE8EZudW1b8T(u1Uj*NrQ4YrAboEy3bYv#)u@E%P- zVVF-}L~VyNCYW)!#M4s{g`@Xc7HrB_dp55#wvby0Bp2jTQ0Z(~oqLZJ8FFX|u(ST8 zk5ioE4uU&hkv5rT`@oD9B+tKV1b_a<;kO9JjbaYZDGEwMYCoRKf-A={6DL{JGk%?5-0T$_gb8}+qOETu@FRQ7RzA_0j6v^r0mOH zolHO-TxDYnK!-%R*;xBRYNG2b-n&tL{Kmwi)g=@u8wFp)>y{^HgV#8$mN(^KR1}1C z6`-38*HoGl|GthG_B^g>Q?d^^Y^mEIW)xL0L5u}V=eBFbv^sWT%L%yw0X$u4e4{-H zUC-B2D=?SOF9HiErjsUOCb{D@iXHM(R%+&`-~CI{x24_)uDS^i3wuiB6OT^q6zmB% zs;M7lElC_hq>`%zp?G12`SPy$pJ)<>baJ|-3&OQ0`=vp~$7OT)QKJXQ_DWduChX|F zmq|eWo=cm?I4K{ANWJ}JeHCEhXTFgkK2bZJqS{*9FGN zQ$9XKcD+nlQ~ml0DQtFNz2K_fvH04X)u3Ce+AxauUNp8-jFGZsKIsWEM2bRUOvU%R zcBLlhN#I0$cx++E8FB90 zv*978a`&WQVkErxs8ZJEK_0cSxLddON(eG7UfAW?b zQ$C{Md^E42W}Sc4Z^IzH^x?cyhT@TgnRu1C+y@4n*dPEdKED9pkGk)&CjG32Wga@A z(wvo|9FqY^A$p>>AG#k?V;|iDww(YzCI4A4^l@B_*u;{rrz&>|{k_gyEo`&_v+P$pn)U6w&X-nUyBnLT>369-V?Ie9 zKPM;i&NB5}vR(;yA__HBy{PGxE|C6WSWXnh+-=r!-t>eE>lyydqgBBCB+jm^@?`kh zAT8(3U&@T^h)XR0qTl>gqA==&`fZFXOT28#Df{Lw)8a!2>D`bpNTha{Lv3X^$M%|@ zg8l&poG40`_!;L^dyvU>Gz~M~BOQ7jz}v_ddsX%S9Xg5=N*xAcIXmvd9H}vl0qGjo zbIZZ6>13n+(E?bHB&l#|(w&z#?ofF<9uh7;zayL#JbDafBUnR}QK*f?=r=|P zaZ6^OEsYSXaOHzZZk9qvv~1{Am$F=_1%t)t4o9;1^DSORx&(>=rNO?TsS1ef7RF60 z8FGAoPNrdZG}2b-$eRsWetEMqQ^rOsNEePA2*v0JbnFxiNP5d-#{R5?ZJ+Z90x1$JDssGx zq~a|rz_}x)H|LpX;FOn*=31CBs}Tg_E$SWLPWaB<)t6w>w9Cn3-b#Ke*0f|f&Lh|U zRwC>lpED7}&#t$0sj36=3W4#Zl*8C8d#ws)#dp?{=pSQd{XC7ZQN4b-8lhdFijL1% z!jQ{q^l^f8v-&=9%%Exl;!YIIL7gBp)AHd{) zVLRv4b=K(MJAoQD1iN?V!~utNfKV&)EQ1JIbuoAqGuiL6e?4gC2T4{R=}b;n@XdJ* z-&5&JSU|Tb3u!vLnhOfnwy+VNSh*3jz9Q~i#)`|}hRvQ$j}<~3hKwCcLmLOfcD;Sa z_}-p-`FUKE-pKd#`SH{+C#hrI3|+0Ax*uGM4sxIH;fMfq@!k>)^;)8snPwdo{P+48 zdb0~}Zd>@W@B&J4Z=hZ|uEXEMXr@#i1vv-_E1^vdZKfrQEJm(VSYA}AnYnx~?G`FB ze%u$1{qug^M(-S7A=S9hbASvyG}g*jHw74tQ~B(lv_4%=s2L zl!HW4rFn>mFj97zgkQck5_Qga7{r*sf}3u}8ExJ)Es>W`A}~=a*^P1UHk#3K@3Qc3 zyh~~ah^Le|=2l@qIyVRRa8av4+8S|twe@DjdeglCA2~j5)B0qCOEZ^(GabSPCLzL| z?0NI~+$>mBOOPS@=>upkvoY zPTl+M3_i`B^y*k927d1(^ipK6G)TeVL*$@~K(c}Fnfp~Id zKCS8*4{r7^Xj*(0JzUq#ZXTYi4r)~2(`}+@^@Woth;y*luCrzEieZOp4k#Nud?CNOYshBu+Fy0CkJ(AHz zV-z-8lH5Y2CKC8C=dgQ2twZ`X>l?1`FBv*qoN&;?TmrL}JbWKR!IW7PujIrUA18ir z*|as*#U-}%aTU_cCFicse0(Vi_iJ0g@TLB6>Q#SvihLe*!>|m~&^o)5cgXh3;{uYk z1j`>9{|rO5RC=S^dr`vcPBnKVibeH%N?@MCp#HS<@nuSmrY5GxPbOhy{H!7V8NsBT za(s>y*@px>Q1duX++@Pp&J!}mj~_}huY8~R_b>WcwbXzGEQl^+SOjz#1S){f8DOoN zSbd310NJ`+QEX+Z!8fx_*(~!*$(G3OH)Ar2C0v~yhx_AO)jnaHpdjd5uj$}s5jxh} ziKk;p>aeqRenaEMYlx4xrAA@2*)c>A_Qo^i*}K)vJHC!wrY(MI+>qATB471zZ=#5A zodXM2VZzR?jCRmf-u{SAqvM~4Ga;x~zqR5;c*7C4Xb^=}CWLrKM?!6`_O+Y{V8Cdj z^0f;`oPUBz)jO(1h>88#2X04n;!Af)jZfR!U8iIgNY{?vXzA~2MZEEEU3$jY8&9?m zP*f&5K1ui+CpB!aHsiQQIC;=sY)h#CxcU~G@r+l5w@#& z*lNQ+|I)slH6a}JtECIQ`z?^hjsk|P-o>3!ygiUM;*mmFt3_4l_$qtQl}+n zNFv(oX-E)25Ej1nNT^)#7%;P{y~{z_xWYYRMk28 z+~ttR@REXRK}%KZ3lSor;jJswH2_CJ(0FdYwZsw;Kh!2i#s2<9RQInC82&8>)zY<$+zfq|U?yHtkQTK+yet9`jth_ysIj_pZ&q&4n<*_nH1%&7#M<28%@zy)nYYY4Z>`0 znjBc=)$hMo{^sjiS{wD&Et7dM@k*5)g|6jGEVuo#HQ~iJ%({2ve#hn4H~aopTveJ| z-K5zqYoWs{0-uH`O#+#IK}0I57*gqCC`0V%cz!XjY-b`ux-W)QkZ5Da{KfO zJ7pU_i?H%TSRE9E&WmYJ&J9r*W%j_3meq@Y!l+FX+~=(t0-@)wg@h zOyGL5b!apHGr^T!zbCisa`K7Bp;gw;k0rC!*O;`L)v_FH@p+^#?q}hZ&DoOfw5h6F z;9v-+N-MI)vpXN5ZLL*4iN?(>a6#V}^*5rafE{r`idmpjpGrO|aifev#TqiM;X{nb z1d|@mJAf2$`dbO(9Wb;{5Lt`{gzW&n-=-aHh3hYTB3mzBEq%;Ak#^-QY${q}cJVr6bancw5S#}%VqgrT_6qkUc2lI0)uo%ncpy1EBx zOi({Ndva0x+*-u10p~K6;e-1e+cLGViO24px{IzPwaM^hnx|O-gr|HBr21&GGxJt1 z!o3DV*79`E!ru&>@>zZ%$QoKbiHsKsYVoB_)Eiw6>evnrZ%*ENd!<_D_lIV8Bm|s( z4PNWlg@zZCBH#24Kx3iuv{iGM7^NC1OcDt$7kP8faa8mtqXO<|xChXddFFg!C65Zf z$Z1wCSpy~K>LLT8Se{VUy| z1EU#;_fvVFsz9A?k`t2wKfV|NNz0`HEFaq3w;Kx2c|gE502b8oi>2nbXxKT8CAz+; zIG2t)n25kVdLu<~?_J(!5|z%-@1OQV5n-|e;s-?tD@}Tx0+|foNjY*y;mejc{%${LT&x|5#oQmTCG^m!!-Ki zC!gJd!A!5$pc549HD9x)LOAx;_+L$i5yjlU6ulNBiC`>s?2tSdDO$fyTFSDXAA(OM zo31Vf&^6LBv!8gnU2g_{T@Zx(=?vlYWAtMb^<$ig(w#N!`B6cBwS;W;C36;l;nISe|Lw$I0bo-}4AkqQRo>MCz?3~@GW8a=J5aK9d6jN1`dXAXdu%x5sV}GbY43)2 z=BM?ncDy_>f7FDp99}z3mV@`eT!O|a;M}uA#GBtkOE6vy2a%vk;PX;%6v)w4EAL^U0W`8yS6d60r{$20}#iY#+37iASG;Zq{pWo>4O{c$=?- zsNN{(Z^a2DdHqgXZ}jMU7VH*6R1Fh7z(`#F9yC(*sM9<5j|~>omv1^tmBzw&XnR zFarT+b6U-SAb%3nsJR2ih_^w=lAZJ)>*FjB(D8-C5(JFnIMkOL%_p2Wl$YCIrGL#S zb(jc?&aT_qK%M9k%-+I^J!|{*?Pi(KaNz|L7^8BRq@5eUMb`E>&vSFY=RuHml+;4+ zraWqFLLt;KQ+w7{HNwlU?Y}M@GbV~_UOO12cVK)pdefnFH7ZJB^(F4wpQ#=oUD6U7 zJ*&kmuYl~qi?5;1Sxc9nfK-|-FHxDflhz3;DOIg#cQbCOO(uaxBS+64k+>Rcv>bzn zPH5?E38Tq4f1&io^=wIDcb$i(?f~! zC@@$$Fa0wNL~{kE;Ng^T6CW%6E91__eZHd6U&FE8{dD<|h4cISH;VHNfbA9hHo@8A zeBsaJGCnKw@vFgHz3dOT@A_u3t3<+r2^0V2$L&v%TroA~xCeh^5UxN5k~wC@Fzv4``f6vEPlq=2kEpme;Qld4%1GvQ{eS znkFZpJzGVhy!COcoTayPxBIu6pJw zN2}iF??SwL>s3=R`e2RBprImt3DK2rd;7{8g_^h;wO3GoVonji%(TL13UGpA%E1oLNRdVQ#%IlBLSTRVYx-BaK6MPB~La=V7EQ#vUPgbzPXkui@sO&yS7!$Hu zX54rmnzDok7Yg{4buo9<0F>4}68>gLcQ@<+a@ZJ~$61_5zkkgN%fp5g;d4x!HR-0_ zm4rK1Q&D=~#4N{A0$RxW5S`UI-pC@)xj{B1-2#G`C$Zpone$GDH-o5E6iDaBWs6rY zgx8p{Pm{w+Q#mD6yINQGe5S@al!T<#FZS(fP{C9Q7GCG9GSX#=e>;n!CuC}}B%~LC zN@wQ*gBcRI=>Z~y>llUM!D zDGOKU9j!%k9UEG;rH3iBj)hP{=3-MD|6Id9O+`OCe|DZqhDU7Sxw zJah*x_R=X@xkJz(Ib1G>%AlxKHVE#Y^;s*y;nylr*c(=YclNoH*s&51PjxTm^U^x_ zT5P0ic$n{DYu8ET(SN@iR=vjqR&Xo=H*NwozGf zd=JEN;~@j*u@9*~2`Kj`(p&MHM^HBX&GK#J$R`erCmsa^Np5fxF^P59G_xbN!dD

    jNn~LljPlrZ_3VZaU;P!TIXN;irvm)$HdW7^av135LduQrh);+`0Jm zmxDeSn+Y4HEN^}P=&ItcE0VX5^MJ67Uh<*Y`+cyO?6#c<*mn4vcm$M7(RHgtm4->* z72ysTlfy2k5mGc+a>OX@Kw+@-OHR70pa3Rt&|Wv%t>Vb__U_c#yg@17(_|Dj)&kgu zq1&C#ou+cb=gRo5mep~_ABgT?u-Xn68dFS=-nm~{g3#w}q-ztZk?x4nfgJ&L5}6y> zX@p=zj7ofZo)u+a&-&#qkp6eX*$QvXwX#CH*Nc^^U#_|@!JH{5OV}za!V#yNJAxUk zN6J$&YZ*6s12>2c_sTWQ|8muDo61Jqv+y?PV#k1lx*9W#P(gB;MNz)QkRx=%bYbX7 zp^qrisOll`uijA{(u+Yev7L%@2&kwvwa39u$6h~7HS!i%%sGM3_KYW<>Xa(&>Tkz^k>gAc2s{! zMuO4D#uZ2}2AL)0VkA!9cy?*U)5Xo`y*KV?r_1ME=rDO_n%EcC*Ny)!k`8NM#s9jd zCuFDNOXD^&y_=CJYPUc)#$dw}ntR!b?{PQIgZ+%6UA64;Mif=te?BrrXm*eSC-I-g zy~>G(>QJkz7ifR2dN0?xvXPE^UFo(mzLyHnG{I*cV4lL`Rpb zRFpm?ZM#H$miTf(H)fMDjy^vsO?e4vp7|N5+y>L;SLomoB986Kv~r*+F2(M5s#v`VhV~W8 zjnk=9G=%N|fp`+HWi44L%aj`Em1-e;FE$iDUva)=j*^pn33+An#SRQ$InRcAO_~0* zj*jA-^q9}s*d#+FOa#E%u%+WTumFjAcQxlM0uL?u`qO+=!*VN{n;{0V5N8GTN8p~< z3rWD?mdxXPe68t+ERl?J(Dbc)`n=z@@x<4PKJMMm>DT{2sX$O}(A1iM+S<18(LOq- za35`kO=_9j-j|RWYP-&DyPe-UmjD?Gck~aVx_wK@H-|mB6K1G37zX*q{Z05-Bi5db z8my+SGIdJoCdIt9HT}G3pd-Ccror_$Zs zIe;MDARs9X(h|~*G)PK!-e<)3`|kb!wOAt{GYn6hv-gR8KKdKCVa-+=pr(NztqY%co2S_lUDdtZS7E3cpy+ z!HX{##fYBxI3xJR^GdsZmp*YA$ki~MjMFD|T$U;)OgJS6KiV=C5ymm|z}r`PiJoE4 z#OyENu^5x`3vKiK99Km&2AOH=@bV3jxD zg7_QGSOd~&4${m7b{s(0h`XOjL`&bgMDyI})I!99(3(ewGlS$#cJ;P+smUrD4k`aK zJEfR_IYkKaVO=*v9}Q#HNuK;_99EL+yE2Z z&h|Y?bx{3*0&``u;GCFv2!k+4rLEwAVq^jIkgFAY>YsdjnOM;tFj6kH9Stqc2B2aS zpHxTpV?xK;zD>&D@sI8YbHF0(qk}51Z9fz*6OdbeEt`>xK%tr()~-*+g1$h&;}|W4 z`=lqYNM(&+BsQC{N1{X&+E*S8sslqA++P$-G0?@EvRS$JeY;$IooMixC8=s8(iZs_ zagIAL^^U68vk{`_lls3K-OD1aV$e`cXRBD*-1iwj%WM;QV=2UtDI88Vx7H}}9NtSc zRRw$7+f}XZaU`eBESTMRUnca!8Z?JKwJ0r=QYEg;7F2TfEZ7M1GBF1^zh)u6Efsk% z+hq=n7P%&#ij|0W4(^)eKS4How<$BAm1#4x6X(BS+?yFh^Sb0!wO8WC*|1LB(-zWKc1sB<6SoNOt+02$dSTOBPn84GC~ z(D??Hr$fygm^v&~W$Os&xMZT&s(XflM2iU}BeZ!Q7Rjbs`W+$F+@C*^r2|7J7nM5v zDRtDxxK7E!VrI5_1Pa@vTGN=!z$@Dc-l zC#k}1|N0-LsnUdC8s-yGjcnSViSzyBELAUGUK9t||7fe!nxlS(1|-Do4fsa;ss0er z?V03_fZgt;l=UH5v*j$u>vMx1Mx@M0rB_+mXyYQcc$EoXq^vGnEsq$9^Q-jLQe8PR$mOGZhE9!%InV(H zO2AT+CNEU`5VMwPAmXXi3m3RRtBl+3P1V$wtE7=!-;p)m86IV>m9Fgud(k`hyx>sm zPR~5Tv>*2kab$mAqqHu-^}SFA*fQc~V;gj0TqWY~s-L@SR4Q3gbgaLOnzB&#?#q1M zn&r93aG+x>{K<)c+464PJw@lUAw1?fN??xAhq^7&s&{O3WaNcPbShR#I99&tUq0H0 zbgqC7;?Bg0sx3sm*+GR!UwB%@xL?GP;xQ%P@FO}q`c8~7y@na%Lor{Zr@wR0F`AUV z0pbQQJfNG-A99n5K9zjM6{(zm^O^CAD81#B3fJbn{fT4zJInqB8_#TrmxLGoY9?xEmu?bgV}O2s89uQjc@Z2FW<3c6J=#HeG*vC8$% zO6t;n|Ky#<-H-t~(>OKrrO@FopR?(#=1)_8j9S5Cp1CacS8h&aN9N==yL~NbuBT-E zx(ynMFoKoOB*s4Noy7$|X8QoTUPD=COhv8*NmOvCDYNv-CqUh1I0MnhNMSI$%1e?a z{-3|YmhV<#hH9%POsi!aa>xhzvS=e4J7_V&PZ)a*@z9?Z9MF#H1{w0qN49t4Q`*hncNzz#=3`7Y z?m=_*+a9-qEz^Uxg~AUz(i9+n6<7GGHP+rI7KCqx*NyIOuI_r3SZFVnO({g-z3jYN zu)V~_Z~Ea*S-U;FKxKgvvCZT+ko~m?8YmAm;EVI1oL}H{^lf|jx9%9@Z{0Dj45gEb z5>+o6EITY<(GS{Qp0E7wRGb>5z);{-;sJ@xn&0kJcJYIy73aw7tWD>e0_nhk{SpQm z47UR^N7a*7)r|bh?FQEYkvi6WMYfE>oGkJ*G)+^lB|++q;P>wwmAt5Ms?I&mOCg5m z72O;a2`|B zYoR4{k+y29(@P4WyT3Iz5kQ6G0M+~y-}qKmc+^0zD5Ry&xROjGcH#_qLG^^yJiT*S ztxiV$ytf&=$p}`73VZ~Lvq&v^#u^}8ayr4Y|8nkR=&YN)B(iqY41NSw=)RhQSan(I zwmIjAWMOIWYGCH;*O%cFKP+D2{#Og297P%yt-9b4MP?m)WY;w5wbr-HH`rhsaDMjz zHQKOsY&pPW_(@;l_;xaVnq3t!M(4MVn!pH=GL;w1s1AZVN4{M_ajbQX!EZZ;gD+Rg zlW(JfQK{Us+!+Wk2VI+VJb!X$+D&eX^=yaZtX3T=PT)GX(jq zJkc&#-ebWBMogk9sHnM2f>2L;UTux$aCob^$dP28Pd;x+Z}Tur_oZ+Q`YG!ykGhV> zk&)zDV7xI*WFRDts=7gaY>y0-6xNxR5t!3~BZd}!IY$f*l>KrW1MEPR0c{nq*uS9% zmZ#Umr2L0Fi$!lA!YeX|S9lVarx)(+t@4anJ6|Ua&8ISM{7pdDCc^U+cY*bT^|xn3jXCoNkht(^yLE%W8A-q zZ|^?jvKIOT3NI51j%6kkQu$~!-S=ls47~annE9qv6M)#`)2ZF0?(kLA6_@$NX)vIA ztyf6ovf%wYM#LNjunghxzp*6JUp&{F9vtrs=nU;|6ri6HfZ9E_wILx5v&h`~V}DtK z+|ABbfn*Ln1OW1f%l5@B_#3vW#}riZU<~ql5)^$9P1bz*piZClYLlmOR<`Ca4Acru zClqZr%+OTFYAHg$z26}$gkEOWd%A@6brw^+|JcZlgzc10TCTuxqz%@U5TG! zxOC;{=7J0wezJrm;*q5kCeeL`+IeF=;>@o)8uTV&8RsZtG|2dtERKo2h;^vjqzRTeCWz_9+XE@y}Gj!9mJ%(%1HdNc|O z>)(K-%-{IL3|qCP4kM8MA{LY)1OOkCO3tv&GjDaT|CF`!sv!p|*Mzqfg5oU;i+=G( z0Tm3tmhG6rd91lH9Fp=ML1?z+aKSBANtC>wy7U-dNNl}-TdhN4SI50}2!?gF*smw< zRr}`BAo>Z6V3DjHa?-8l`h8>o;?G&;WQ0Hy<7p7i==j>|gM|v35c2oJ~(b&zD3TNq z%b?$u#td)B;)B+~_Bpc>jR@~miiQ$l4cpkJk}?rol1-8K`WVyx!oH5Yr{eU`H9an%;ewYHXcrCi zqHu!7%{H3vfubn2fI&?ztl2`p?wa(vWwHa>Cqf_U-`+7Cxe8T@k7Krs?UW`+N}Ghb zh9HXf#4zCW#d<}GS)9qgAVh+)Qh~nf|1=r=*QG)5$pG*5(hVUm7R=fIRf7OZ1*vl- z+xS2L!eBEe`F~VDc1;YFtXXg-I~RBQz4vFl_#4kWFs_yJe3Y96R}nzrUPzr|@H z>y*CBo1cHh;BkAxeFQhEa?*U__T|gQ)@6F)+uO2;cujw13<~d~{!64G3(i5qMPLUnSq9FB*zhuszTLgIG2#N4&lqaI}f22 zyfVZeVQs$jT&AVuOgaxqQ{lyHRZ>!-Dc?PG7(US-=HVM=V5gjx^6p67Ec)_GS$(Ze zdVrFXQr)mnK7oHabn1gklF2nKEs;;$G2%m$25}h__k$F)Rt^KiMBT*BlQID#2 z*+5^sDJ}lcbXNWSiV)QkQzS3@(!=|#p^9xUUw->Qc1G@8f}uyEXSLM!3dJM0Dd>uk z>2${lCFHg+yL@4j^kYOlNDz@fHFkUDIm0mHw1FJ)+!O6*3fr)y6Wg>?GMw39tI+Yr zF())BLA0Ci*HWjrtJV2;&#KkB!sPQLtq?>p$~B^xFL$0|g8OkBp?F>sCa7e+$kqz# zea=wzAVo|po@sjzf4Rb*5dHKuMG9W8u8_f`LW%kGS?89O9sl@ob}~22L}(5~@(|;< zQcQpfh-yUhM{HE7R1#h^I}p# zK%nU*rPB`@^SMWDP-_>B>bb1k$`G#{7K2OaKpQu`ILSng;i=a-vUIxLP%9&C2cKoo4 zbpAinykXI`P-sGEpUDmy{d7x{s>JxXit^7a^7w9-FGj`k#`Me5Vku_6cwDziyGoXk%hX2z}Acd~pn+mR~Y!K5Q z-WRwjRH@9Pa>Jc!RDy{2O@7G1%9kCT_X|RN%pNnPgo-rXu)e$GtqYMgx3qu$t6!%7`S0GB^R%}U&keOY zjkG>U?5; zds|V4qTAB$iFdjxAk>Gok418jIKjZZ+&9%43Ycli0rjg_jnsyhMFhEFiwtiM4TB2s zUX=6U#K$WuEV+a_DJWpvoP@RXeK0mQ9`U+0`R1M!+2X!_d%pawLiuR4xNn@5N8)7p z+$_t?jKW0f5QD^K;g_7s6O;EvnaR%wvFhq_DDs2Dc4+Pm=>+BX#A@fSqZJE1(+AJX zny+}PBhIMjEpl#TXP=C_tH_3|cC8x#PHD)hDl5dWCRFOXd~R)lIrq zunO(EPeqd^t-5IuD2VcvKWtCLrxDyl6Mx}tc_uMgig#rsI`U+E@gYN-&3K5FUy`aZ z>aw(ON(@cP@@I#T0FlN?s;=(1hQQ9M6MgdCmaL*sx-EZRVfS;)b(4gv#WsC1iGrNs z4*l#Abho@8zdpxG+kT@UhR58?6};}?ofuP$5WLChdVoOjrxNy1Tr7_T1Q>zj5mh`6!* zn{KKBR-RucDJF-6ifcFhfxpz)skh4hilq*a!(3MCk#Uoq=nggi2&=tB>JJ@RJB>c; zu(7B;CC>OFhug!HyU{{y20|WJ=x)oM=K*w(uI97n_1!ge{%TZA&|X_?3?;VCI8Y0P=Zlx7a~4h1D8#+^82bg)EH6PYHF>3#lBml0t^dvQ73-2p$e z?LdC3`^}d$maw!clZ!_QQ)wFCsCJx|gxk(vgQ~AV0$g&T@3aUxKZYv2m=(X}K)@YE z?0Jf9o}^=er`y2*RJSX$I6^z`iS z^J5@o3wkYfs~t55880aK>1Y{G`s7#p1TPGK@+W?ilI-&@3uacOPv7max(G?tRE&+I zrKMH4*3vM&Jxm-lUG#>psBdygZ?3YN?9O^hKQARE)wvKC7uT7%-`n<8qR9C)j5+=) zVXKIaj-DB>MOH@U#FdWbuC%IH*^qBJY&8YG7bl(n+RzI7-l9yAk(8X7tDoe=(6O%A zP;pnc;We3WopKpD^5(u&tuq<-vRTer%`qIel$qMwc{K@erdY~iE}u^c@t@42L?)## zzkkv=4Z)Fw+8|Kk;CSMPD)TLj8$FU#;m#0H|C*Ckv1>=Bkhi?A;|`jB9_3O{$t$M9 z6%>peC4~jZUAtB9_%|-#Orpdq#$wX;Pzx#9xQJ*?WNDD5Er(4tix?i1ScmE4=Tn6A zx6FGUom+IQ^WqXwb~GS(f-+!zV2RWKd z0K)@ZTvJN#BWZW9e23BuBG^Tqx_97?X)UfE3sp9RIM^w1pVyw^i?+HqJbEV5kfbay9L`Ai} zu(9nZ#|-!BL8m2?dwE#P&hLD4IwhSVkIkPaO=-JHF|r2^H~SV~6do#f?~eG$-fc_E zb-?9GgA!4VA{f`T${_aZbz1sjYVz;S{HzM!U-4;uCiwXHMLWHHeVrV><0wdPXJGUS z{2gK5vw!kJfvXP?>b)tguMg#*Pg5&0!5OBDnxcQNP=DsB)?!je!^o}z7wftHIbbl6 z=#*=FkQ%hr5R*G2BarYbtE&Lllpe#P6t{iW033r!<5u$9n5dwH`qg-6KYz2$F+)ObmHU1vb*pr=8$FC_oGDUy?7OD5GG; zA=e&~rFsWrEruS5d5AnKkZEXl$&-12R0G>5IEX#Qch7uOJUTz`O~`W!eoVQ{Qa3tk zRR3V}fMv~+eWO26sNpj9r)H69v`Gm?LwY*;uOz%2%VZvAOeoQ}7g1hAU(;PRc1xua z2{iUc`VL$Adiy$Zt(lt1$ea1qv{k8?#%&zQtMeld-_gNd zR7e)Hrbh|m_=0hUUjS8AeuM$7C{eBnWnD~A!@P=5TV`~P-BO+$Q><%H3@(5sCAGDJ zxPqfO*ew&IfMmcvNFVK!&OrKkK_XX<_wI4+LVSEk&|8}W9P|)N&)l>lY0n|0jHCX; zP}@Pz#2BI>0QZN}%x-Uhuo1W2I|w{A3^T=LN0r0cNWJZlbzAIYi(e^vfqmgLPzC9* zaE_qolmo0n7;W44;G>)$?n;Y67{kCC)29p~o}v<*o}<22iYeFjRQX)WI_8pv)JOB6 zHN81AVKew@?XhsK(OR3ZsB7}MzLsQlmwsMv@^`$kUn(DpXwJBMSt;4$G2NNE4>)od za9e$~rG#(Qc2n47UaG=&!kP+xp1P=r=Ss^V`i80WLyKLV;GUCtQsk!(4k`d$xt#*C zHt4CQp-Sg3+ZdIXQ($7ZcsaFK%*O}Zl4tlLDKbZr_qjWD#nc2ks>_&E$<${|EX`%R zc&V6b)e=vSZ&}WgaGesNb4_*Tu(AH8|E4W z6DEc>sl$_K`7aO8U)4MQNj~rk6hXH}_L=v%)G*HB7_5?}S-cl+=f?(6GmK2r-s%Oc z$Cyg_i7hrl&cmfj4}%nw8D9Tqf31nP_d{UK*!Tr;(91~Re;$d{CT z3K9`Oai^7`991YgNG#%>i(}5On^UoUn3IzuEGZ$2tDZQ8^lhcIu%-X^EZ>!;wo?)p zC1(I8=1nED`b#S7dkN0(!UzlyTQxP5CAqR2*^`7z2?`kuHUhop7q z6;D4YG~0YS^xdzH=_qde=-^mPEokf~@lHUWkiNy?3VY{ZGw0f#=0XM}O=W%HB)HDr z;gQfvSTC1_DMa+xWe(2n%jSQLN6aPR|{ZMqn-RE5|YwYUg!U#PI>ol`x=FjdWv0lai z(tx$pUPTFOX6-2x{zVIXQ{>no;WGE=A*rw5{WZWK@xEa~Kp#d#{v=IO02flp0}biZ z^+6OnIIt0fc2M##MrgXPP=#l?*Hfb=y&K=VPIO&QSU90s(XBBsu zaxkvCjv}8zNndc$QbMmboa+7;yb=VVh|ti*D=kg;6m&`(37IEeEx^P-s#>rSrJPsF zwUl76mprM<2)r54jzdGBaO*^!_vJ6fEGR9^tdMk?Tj#nX_?ptxd2}&wq|U%YnWTBJ zwC1Y`d;Ie{(F4d9ttBVMJy5Ur`_uKLlxcDG4o_+#X$2W~QM!0whb1M$2V6k(ASUUft7{M62_lXfcCSpmeNSeW zy1HvUvc{Q)9%<9#ke*`fa@SI|AN@Po1($zy_Eum;J-X58-#5jOBx5(1VG577=oipVQIO zo>UAZW?b}qPPrg$mZjA>MCwNIsGp`0XGN?)Ihk~*km zLE`58vek`X&nZPF`c7MOXL`^5k6ugV0s$Dv;rYt1n*==!QelvfQ7@G}e=VJ2h_J+%%fH zaT>~OPw$m++`atddbar`Kg`>qh1)Zn$jgvz?dGO0_&7)wTu?G0k}oy}u3nS8TO%h4 zKkN%O5Z9A1#6q9mdtwH0xT{n|KZHaEm>x@?KUq;Ezc;o@W5OoEdQt)kBEfwWH<%1; zhJt@sslay#f&ilUj+~h~e%-PbG1mFtvP3(_K_{REf9{{Adv^f;w~2#@bAlzH=XDgvPhj4va{8!RBX};Be2ZxBUV$e zsD)cM+Pqj4;6uX(7RG~!>n(Yxc!NDmhU-Rww|{6vxLQ_So%2Ac)bFGnKa!4qE=$(f_#sZb zvfD`~odlB)(`djSg6H=o4jgIxE+U*N{p@;rA3>TGLI-W#YpAPrPxz z8(-4*stz}4A#>btk22Z#rr#UnCm23jscWy1K)9XP%^RuzuB0+TSAvZsN+X3f<5vJJ~h6vdO>;Xc#SX!D_@n}$pcqI0w^4P3X zO+`y4*Wo57pCXQa{;Zxo1W6vMQV%SqF%Bnj4OPSgL1D?m1-Kkf;$vyFsbC-NQpzT@Rl zrk}Ro=4=yJiHi$Lwb^*@xd8B^-rnet$lDX8<8B_eM$=zo_3UeF>911I2UwqoJ^kk+ zsnCHTN3j3Imh%0-=$oTc;}OcqmQBNw=XYIuVBfK|njQD%HrJCJqlmBXm6C@^u)BBj;eE3z{sxuR4Pk?j3>I5tiH+tp|V{J|2vXJZUm8BBVlZ@5ELd!8Og^e-h8tv<74s_gC=zV6iV9H z2?A?gms+8U$qThs z(IUi)Ooi9^j^{ER$+oH9%~_bK#N}d5Jc*wV9JD7SWjjA>HYO{#(=`XQ7&U7@QE~I; z!Z!s@Vy`DH*$C0!`$#vKkNoP~`W`kfW@NBGk`crQ493Dwro*q2R5=IP64;9G&B-ST zu$oE7dM_YV7{Q<53UjKUMPDpd`-Pi-UsM-u(b+b9Dagwc^-N;^NNrX-l=nL2=IU&e z(CHUx%UFY)&P?65ffczJPst`vXvTh;z%A(t{nR>d+e|PEPe;dxaDU zaAHlm2(ur;b79dG7G!K~X)K$F6c|wzl@{g}C7au-sZ4q_A8{*_QNHqt#cb~jwxHGq zW*|zn_$LP5#(%W{s9>6PI(hpdh0f7$R~?pd*}o8^pG3FIB@cf^xe_qcgP7ndxWbQT z?`ih-bb24u6&guhE@x{&$7{_nu zfd%w60FM<3W4J{0`SEu>v$A4A#3#Tc34e20ZosxH>Ehc7eB* zT3pGTX8MM2MVJ$)%1*L*s{Uj*wRP7dGFDW5jd2{bk#W!HPiYxej zes4@vQ>sTIB6DaAh8bJJ20)2cdIh(qcN@#cI(_k43s>;fpYYzR zw}|=eh){hv&g1b9t7P41?QsZ4R3UDKzSWHFWX;@FK|TR_MD82W=^@(1i=`f z#&vef7Swhbw(@5{cZm!u`ih!QCVkCAqGruL^JgEPSe_Els^ix8 z^!lo`nMidV{N? zC;9ssqa+UjV-NEo(MPUFR`y)yj1mj60*9#6(Y+l%HExi#eJSAez4tl=$=8828^6f` zvLO<9Og{L}l>{K;q$DE~LC4vg*M(Pxg{29N{`72dM#Fh1ijYfGPCA+F&F%K>&nUw7 zAsY$HqUEo!X)*%V;iOlmMaj=&)(QaIFgG7#o$69xtkYzYwC@ZHl05H?m34PW4vh2) z4@BD$lfq8a|7nQ-WER+m?8~n>eFZz`>$Yp6PIE%)PH2<=_kz+OgW$7Sye^;%?i6IS z{H|uZ(#b8>Y23j@hXS4Mp4IC;uvp}- z^nn}xCvJ-i4%{@Ev^9$)Fs-4O@4_6ww8l;Z{a7w=o>92_@2$r(&=-=LS*!OL@?4rf z5iUeTl;O3lT3unihL&{)(E(PO7H}KOs8cpua=jnQ)*T2n=?f#3)i&;1%HSXn`1w6B znfXdfCi*6lS@*~@@e}&Z&C|^CH}KS`(L|q?UFgnc`*7JgfcJ1yX@Aptwm{yz`SVThBaMa|9#@auBeo1D z<}ZukwZvd0g+@DvZ&OdVG9^`f=IV3o?74kq2HbQ8+I*wRd(zAR@sSC;zGBexCo`3L-o@l8^%^hoTX5Cj4ur`B8o7zye!igH(|vnxM1czGuGC;{pRM1X&ys95A?N zV8~5GcVKs6ZV|4t9tf}!-Yj!eo6ch|XV*{b`y=PKVzEjR3R~=aIN%R4FDhLdbTS{9 zj6HW>w^i_^Pt!a&s281D$0Ofti~oY*GPuoRX>EiYE%@N;`9hq%(I!G&*0vj82JVqQ z7potA8V&jQ6l=^-4F3~YNDDAG4N^MbbBF`PA4TsHO(i}hcuRq{9Otxn(AhCtmdX5EnUM#k= zo$tC8oXA*_O_mF}EuZY{>HHpEh#EI>ccgjrAc#=R=D-FU4{s->BEj6kyztxWeIffN z_RsRj#4}75(v`L+-ss&Z>VeVM5bETB24A?Iw$_j`3y#6?m-6=ixVHpA(+v2yz_saN zAcKJrjEv;6R(TBWq!G;8)Ks;>%=(|zN%&dfcrE{`)9s+s(9>h7N4cLhIM441=aJi; zHB6EI>5t(6Jx%(_NNLB0{NVQ_*|lV5M5_-z{r4mv%6SNM;sjJIF=^Qr*r=+$gK=8i*K7#zO31=W-iAR({T2ZM z$0LpE#|=1NliG+y0U2&Yxi(*3Iqt7g*SjO`DCBlwbi1j}Jr0t2v7ss8qx9<2>)rl- zQD8tfjM9mzexMCIsHq0?Dd?wrqbo%ch(z6*+wP#Lt{}rgW4*t}!TWN7bEu6vk|r#Z zzn^*k{Ts~iytcO{fLtT{WB*N0WK}3?ic5)0qbEf!FE2CrVZul_LL@As;9Y{UkB1!} zA5XUZV9yDK$>NXGw?Nc&6a)(7=>6BoQT$cH97`?+mS0RGnntula8y)QL&Ng@A#!=y z;vUWKn;eju_GctDj(3!xT&CXT-yeSi_s4p%c{-}rag;hedzILcg+s9`8Q|e}^JNba zT9MxYI`n85e0@&yxZ&>UTl-qx?VbTGVx>J^k9J!KLQYv(CI<2+L6T$yP>*in>2m#y z$NA1$bK;HB?^YUQ2$eNBT)=)fZjDsGcxgtmzjVd5QI4`wus_58q*{UUIV3htiH4mq zqUhX>+ljJ{p1x3(r`nCJKU*#gW08Hw>Fd|Ld5;e@pG>+M9st+b|FaA*ef+VX-253a=i{K+u1TN*La3z~ksGpaORN`w)~<|LcOD|3-Q zAb<8jt5E6cl+Cr{r3z6>iOH2%+6BHTujH+5>a1DvbV08ukCjj zlV9f!*R>beiQT!ZZ=+Vet29FLNJylfGIBDdo~kyV?;o_PuA^0X&hjD4G`+^|1$*z0 zAuhxB<3Vt=&b989#FS2rNy)da$tC&(kzfF9fY1{wu;~4;fy4)^Kod5Nlfl;;=kGlW zI6ypH>RZtnn)ACtY${8SOO;WNc&_g9Z2t+wQYxdYAo%d`=)q&$BDFh?j^j0JwRJQha_x=en3V{PumPssnPgK7HDV>7xNJ89b)6IHbhX z(Lhk5CS&;&H~T)qd!OY2k7bMi5n8@@%W*HD1FI3hLy*NHnd7@KyJXnw%UE)8+a%+A z)jf)e+viKU(8UmJI$VtZhsXJoFG@5Ft9Wu6(c#2H3ua7gelD-4ow~`;esDFIXK;00 z!qk&0>34QVi0C2AyVRh$YL0*s2%^YRl>VY(B~PW2P8Y!|)jT=MR&ag$$K67N9|QC1 zygw`~Kwh}y|(}eP5r3Ie9v^E2RZmc zdtfoKf06O&CdL>%9zTKfQ(NzJpu9~;ms@Ajly@** z@)coQ3Zee#G)98<+DcOl5%HuA`j=YJ7cV_rdeh(kf7gup-Z?1+azcdeQStviW`J4G z6w;{wXf8M$!7T9kDADhRzE4N-a&@T(a0^o4I@3t)F8XLA;OCwfh{RE`8TE^sPgmjN z#B1iyAw3-h1z4-UeSpAXG5*`zR|;}lewDSF%uJc&-I2re+qWdzFW9e%0He!FlW%^XV=5OcblDVOWMU$zxPfO$T#J+Xd;1`(;3t`1& zeBUJS>C*b-5%ZD~4b?Bv-uiYglW*h?&+^RV5OIZv#UnT2bxIo@J=_=Zh+x&3w0Q3DCg1;uuijj( z&N*2p=`ahAA;J40b0~O^)m$+`&z$yiYyjPFtJc zTl-_XB+r$ui1yo&iDptT$^%%XzaOg9i)`?**l3Ihk4pQJY>1pk$P@II27a%F-Bj$A z$uIl#$f4Rula!06>x5(D`yiZ@yKX;qU1FCr#?^t38Bz1}7_O94*v){~R+3T|lQDdC z6^|oW*4mbaruy|5i-uN27HR$f`2&;^o7u>imS|=B$oOh{!(&0b-g!I@M?&T?If^R6 zMFDn>Fw5sCmM$)}n|b;^jH_PXM2k*J_$0*VQL*Y745BOfKeIf7B9W<@lWKB4A#SCw zk5zAXLg&hx(2oj?c4LJlTUqP^Dn$d5(RwEzpbS6}=&GG1E#H-`)8Fc+=T1H_@=}bH zD*8e-fNFBB`VshfFMso1$&HaleC3-zcP>p+`n!Frm$zueL?glHCSZmxD6U&ezcvwn zn3RM+#@V3s!?gfr$GWciCq?vu7~uu%*p zh~iggd-+_uwL(wd1L$P7UK&OCO!P7Sf9T|UFCd&F7=UuS0{oUgA}1N3WoZ!draN@BFV+ zPM0i%zSN3imOwD8{g1SjLR$_UKOA9!yA{8&!<{=p1adyx3xbj~@*)pU!U^fH0wnUJu>3khytp^=8gbi5s2WZ-<$H`(S+r zY{fh8e^hlIzt{6>ht1Wv{$^nHc-bFLg87x(#6qJHqt$$ctfOcDlmeTh=0@dvw@6%V z7>oJ8KBnSY;sHvlr;1-{sK;xc8JyPsug zJ?ouxy)>?WjV9MW>JsKutqxa_e9$W`){Hl>bCB`6j~ zo7W_rM4_Y`W%~ga?;D{-`&hBB3&(V)8G(-nA&0+>`x|9vTcVy&*T)l(g&7wMW+lF{ z=(H_5-y+u5Gg!hvjn;2E-#w_Vj_I+~qo|p4I~&f%q7Gm~fmak41~=;)nY?p#!Y`a} zw$rBrg5HK#==yLZ=(zCIDACOzzj%3k9fC)#IfG^~lYxOO9RIx>i+W>1z^}=Yfa6*X zApiHue`Yd|0Ku|XlG5@_> zB&x8jYNhY_OO{wyHI744+VL%7IPrfM=3k`H99)|U4dF$!|2B)iAE+qMmcdET<{(p& zp-_E~@yBNP&_l>gvPdvfZ-Hht+mz^zD4f~Brx^4ELx8rEZWxmUv2TO?Vt_0eKWY-y z%3Uv&+avjRW4+?_+jkQ$nm#JiR$|r|*o;D7p()&H^Y|JIJ;KFXvuI^8DR0tkD?S)C z9&Q#AyY3Jf+2d55CY@Jp}P{VcIG1m+wRYWn_u(Q+1)Wt=y4(9Qi(01eL6Qyi-#wLm-)J4 z*w}Buh?6n`aPGpaV$_=_VLL`(7k+teqhkO&YKfJ`Y|IdYi zAB_RMx{A7_VHAtH^Q|n!``a3z4PgZrfJf-7AP;%04sms7gb)?Mkc=ur{zp{&fM<}c zrN7O_i7CL7fX6gx(5QEPP%?L$mWami{4=*J1qg|a4c6#NLw)n)Jss&$tHfoM@OcfJJHQe`^D4~ zPyI5aJn38*8@q?V+*xeZZN(hhQf2)bF}nJ>yd?Z&MX&&m!^#$W zYmtD{21g8dk~=atrv)yfmqqpUtd5Lx_-{tf&Vv;9$?|`3%(L$YSP$SBB)+SOU(A~1 z53`P%ydRk~Cwol& z!iWkRNZgS#zHbn?o>PO6oe=g01wM*)U3jB--i1)G6Zwa|$>cS&PH|ycnq&%!`ANg9 z*{qn`hO1eV_0k13)zF4pkc)#6FuO@fl9+fBzLdH0GMLd%h^L|w#(0M72AM@i0_ef? zxD_l@>C&1#*G~oG?1J7M2E;7JzQIC(*KX#Tq~i*{OzZ2Lu*_kfbwr+oX3^{DP^wBY zJt1r$oZ(XUh5?3Qr@aq36)Wji0inRqb$8H4Vi3Rb4meu>L9o0W{Clu^K?I5R!H5;# z4N6m`Srg&B?>;Q@jsT1O!U?Bp@q6cBJuC9#4@b9-h1=YP_t^ZtBxJj)%B(J!z54=# zJf`C%Kgl=bVh<V8n>|FBYISlSni>ieH>04DO+xj>W5d_xF0gf&=55HZjEAKB0xM2+!6 zk;OTQ*gpejutMpsIDxVzL^JzKt?aS5#!S#M_U0t#)liC>>JLGR1=ucH-fv3or^epN+!tQRdPt&{k&tfjpB z7br^P%KTld@xllx_9gw;)f<9`svlN_K`U3(BfcpCO0lN}fmBUyA1`m)quy~^DxCeq zk%Iz>)B05L7tUKr>!!1(e3$@>QH9;|f1d$s%)-kHU>c}k7x?}NLO{v?92N{M!XxoG z+~u-&iy~4*e&7xkVxM#|r&>a8-vd$nUH9>jkQVyO1n;lWwi}BLrYi}2_}l(H6VdQJ zt8mW@i4Z9K_}d@<=L7EDwl^r%BH3&U==~pSMUnxV?dyXmPTl-gMyQ*QD&ry6`u=t+ zgbk1yermaNoTMOLXS;x1k^sy*al$~%gU9%9Vh-g}G1McOG@AwSm~}8@I9Eo#^ER!9 zi<>_UJS2r$?XrG?lfVpRla$MEMEkaFj9v1OVbw)WR!J#xH~e)O|2KdM_M>BtO+$ldF_~+K{qxPe{&ug#7fBX z(ugkg?l+4J{t-Ii1zA)n9$ufFW4}kch-i+kNH2neHcU z{S_f~t7ijBTVMKl$1z|LzDr_54oCVGX3x7_zQr;ep1Ye*<0%L1Pk9fr=P_K9SLi(c)s3G+u17Dv?yVT zN_k2U^n9JRx<^A^-MF4)s2mvwuxy^W(hxtV<5vuggC=>{b3G4uD989tc-Cp}2}pxI%U?^aWKd|Dw=lxEXJYCsLMDGx#epuR-7BV|K0tc($nAZ_<$@SorIU)=1t zOUtXCUC3T~a5Wseo+p|jS#Uwp^wBpWH8mC@Xf8+_;zo0zlhG=H&}O~v-jRtvm8lJ{ zIDIe}(#fsLY;@O;$20+re~3dMu_C_f%xd(6P_~C}$M7T*e>r^)Q9PTNNTNieI)lfJ zFTusgX^X;NBDE63K!^$>+}qm+b|KImQw#OZ76gKxdeOa@2if#;%dfj(bHPq*p?5&@lVis_-@qy#Qh)E-a4x4?fU{H6+uD} zP#QrxM3C-~k`4(e>6XqzDBUR_C2;8O1|_AX8>Abg8{R&~z4!Zj@6R`egP{i;hx6Gh z=9+7+wUJA(7f;Wg4SSx;1|g_eH)v3R8|arFE4fkS&hVs2m2!}+*$s3 zoA149kxIgV3~~2triXXf*nTVI ze!a?vltd@^HcbxHY{+zY$b7eLqMP&e)Wm#_Ng8bCQiY#h;1<)ii(Kuw-95>k`HD!x z%9q?!YwKI(UJ-Xf!plJM8VsHVcQddvl(={diC&n>zSdj2TA@FRYgnpq7$@oO{q~t- zwf_iJkEYaLS=}5ji#FrskmYRMg-rA|#ts5%hVVJoJ%FMLTyzhHpO|B@60p9l!I~<; z3S9a*S-SN8w5wQl5M+S$gjM)0TX1F2J{Dj%173;g+xiE(eDPf>GwQ+6DDR6l{r37> zdap&haO&=?ZNr((eDBKgMtOSU-ps-Y?|h99wx6WCOA#3OdZ7>=voD_4dcC?cgZC^8 z9X=;%px01vFU7-=Qt0utURJN_4g`lrBr*m*aIGbV1?)v}^AsZJWRQ>a(I0&K9m`|T zDqq~|2(*6#SjpOj7ed4z^Ia-}7EkH3q=4|RAev>TxP(<%-crn-45xmNAH;lM%vbkr z;fYp_bL476i5CcxXsSu%rmG=qPx{e`LH9OZgp7%)zvt)ZdjdNmn+R}k3DHA~`?yFC zk&^7W@bFJrj)lJPBPCU^Zv>+L<5)l194v-A9F*ndnO}7gKM&HVe*dg}CgG9GR`ADu zs+-8}kPTMTezfu@)+l_xJH-DuotGdY>(S@V{BRtgi<=4e@Dy+BO zLj$|T1HBnks6#D$F=qjg*Fni=^PY1J+74HZdxW&*TO&=3kh8IFzeNShJs!tO1~J|f z@vOe|frF;|>xkiyRU8}@F-40dZ%Mhk$!Iey1Cwrd(Ld8>kj7j_UIkNq$ngr5&j$Vo z+b>+bH5U+Js9zr_Ety$iC6E=9SWf$54KGMehGPsqQN27LE|4X)`CX(&rNx4HLrzY> z9ZX_b{O0qQ4UW3E+M}-QXt*-~ACZB)5|Vm{byYSEhx*W<6;1n%FwmHw^*neJhK@!^ z;O#~G>+sB@X_fB36e44Qx$hC&zs{Rf6ZV&myYJD-lPX_KQk_VMq882xu+@{WhcLVN z_rL(B=ndm(d_k4EZgL$BCFHOoI*F@N&<$q#EsTXVk_55dmB9se4zz?7);sp7o2d!1 z8MQrwl^Nsr^;p`6MhQi$C?ttmTQVMexD2jiNOD=I#C;;&(<_OC*w3`PR>(LlCZK3U zgf}wdC>@5y0;1)Q@sGaJNyLFYfYw7i;R6&MAaYGrNGEoO(c)0*d0*aC8|V~@^!veb zTG?lc7|gF0($~@S_YSk}z#BKn)or%^1vo^iV6=QC~l>^*Y*@r%0q$b`+u0+qv&RGQAW zbo~$84FyxRpKQrj1lewX6;(FVUpwb#cXIw-YWIpH{&_3_MT2a{T!h#I>P!~V$lhk@ zB_{hNip{_TwyQ7Q;bVy|^JD!!Z%S|WCAaOVo4|yKE7z5uJ~-l$A?w!~PGfw!NHk6F z4lx}VnixE8k-{- zVkwkyb=D&u7D00m`tk-r_Dv_qqfY&g4T%vbGbCCu{^w-9fCg(q!>hvc)}wAIjAwcO zbayxosHAJidB5ZR%P2gN{jz}o4M40k@n=}(blrBA7rJ8{s$ONpXF1;}7qSQjtuODD zGH~4F`?2ZFLGV3S0t3h%XZ&?0d@kApP33 zYP`))c%b2fCH&BUY`p(QCRom2OHvr)uvV7_BDUzh(8oMM9btrqF)kDT$v?kK3K@Vh zhJ5bD!c6%0A2N6Fmc9f1OpM_kHc2GTwse&yLp~Ty_2fCz3|HSCyB6o*Fl2I!C!hIN!9&uP@gm`VKEn$PzbRJ(U4WP9}( z5BZ2N98Cb3OPp>H65izIZfR-oT=UUVqa%Hz$5=R%$07Q!@XeD5l#>B4N$|&~ziFnA z4$M@7{Yh{pB?4|2mCYX8>JuFb4-pi4_V6 ziTA&P(VhQ&7TouM+t_bUu(-Y*dg$`)55F@O2sip~NF=L+)y+&g^hZSxO}5`0yKT;t zHs)z25wCgEDv9|e$?i-K`CAuOyuTEJ=nC~k>UXvrl#5qefv{8b%b5BfRXU9CmdtQO z#l^tqg5Id5MitGEB*U*W%-OFp)1$DtG>87|I-r?&gNf!9N&rw|s&RxJ?5(u&x(KFgoK{UqVjL=W~{OO>Ob#i(X%(Q8R^(Wt}X5Ek<awBfb}AINQQpEHG-fKj%p_vLPT{F^q@a=8i_FX68}{?zS3;{s>H z?3+omU8?#%<-j;LVdaX45lq9q3Y2AQZl-IOM>`85@Ps8;xjSd#Y=IHo&9$Q0+L}zK zqweeS7ajw1Q^_`hGO*~~4!Qfv%5Dr^`{;#?)UK(x-%u%lP3oaD^tqzg7 z+W%?Ue~0x2cEGNHzmE?NsRd{Y6xAjy#skU;`JP+iK|Y=u%Fp`)1tDu{JdF<#4Vf+# ze81IEi#h%Kes#Q_P!g7qj<82zT=^(~S2gN?vo+<|j$;U|E7jx=gwh!RzWT5NOo3q$$t^2rIVz0#_7*rL0c%h2T5^ax(tqnQKe-4Y^Cq00e3 zbdH9<5qU3sxXbfO={|M+u;-B1jZ?SCbor@%x4=2({$6j${fl>N7-pko$nU{QYb*+w zvG7NnXE%Lb3=*Dv7stdk!bwlhApVYPjFb>Oy6h+{vc`RGm7B}?5vwaCsoK(79g3%& zj!PtI6^=Xa9j^xB@S^HVBCK5BQ6^8<2#qwlY+BRo`wyfTc{D+5$h}`t zAL;&9Jc&jQ8p0~}7r?_t)v|u)+yYcnC&9;m6W=c8TaITg_-^X7oDyHZqC41j{(U5>iZa zNc?9#?-Fxg<-V~5-Mw6}HX!X&Zc|uK>5UplzU;ATKFc>n@+$6ad_T9S*Lv}iuR90! zpjp=!ceiV<5~b}oD_0(-CcTHibOr@+hwG#S_IU%V@*295?52BJZR^D>0Z z+)aLu-;5_lmcV|7_he`8l0~cLU1XTY3I6`pwrsNKODJ#(GrHH0z_>;L25^A%))#8U zU|o6=SFnE3+M3cN$2`mdVg`%&GOI1=%Z_}o$x79->e53LKXFjMmdbnQ=jn89z#r>j z7x^YP7yKNt;p~OJOjzCRwKX00#=h@uf782{X}z+OpQh+{@Kd=MpWn%xbEW{%IsfP0 zUNc1X2uBKOxyPq7Zm_A5Nx+@>W-S=43sNXwV##we=(TjddMz;MB#wcm$r0)<5e9$x zeIc;x6IYw7*ss`gt(MyNNQdDUmZG-$FMkE2ue=A#wcp*C*S*_MvLNHhMt8W+lU8ury|B5z+%mX@{EgPo4DAbi@IiQLrsH629NMB7TT}G0IiH=|IQjrfRUJ7dhz5 zsOfX|t-8cH3&dB0$f!@_agI%L^CbLG?$2!n40Rh{{Sl2GExu$RKs6BHf9h*RWe>0#U8hEKfUYHp6vn80|AKN(e6nwsLT@q6z$lzi#*nq~1 zn9nySf9k(wzd!Zg@ce@ZTsyW5!QuT4=YG9Gth*)h0cgSRx|wxO79YLxqV#zl$NM-2 zo6uOn$gv9R=fjoM&!2)&2jxIyR_u9jn%l8D<9A&4E^ylTdLo)7u$&CnW54!{Cpcg# z)Q#;k%Em~3k1vEN3Ny)D(m+M=>}&B|Z15vD9O_WY^T>#gQn>RT;#UOWzSlM93G+y} zO1bEY(kD+dCvt`lSq%vlPbl$RAiCvVTm^3unAE z?ZY1Q!(IN^uY12Y6gh%hc()z^#!j&?IBnXjXb6Z=T zLjyI%t)=d!#-n06{uB59*AHk*Y1Vr4VujiT<{NIQxs;+cNSh`jN020>_AyTfuXgJj zMPD))a36MTy=C~?rfg=XFnKD283yg$?$1*?Y5C~n`=gsfI*N@SFTlb1tu19%{mvx} z*@IA5l^XxZYw34NwD>0_ehK^y8@MuHu;HQU|E+pVP+;imWN%*1X~xE%_XdpJ5A|!M zFvBVQn+(3tfjC$uMfu~`4I`p{zh_5frTZnPq*IP80t@M83gHTADf%jSH3BoaOE2(#c*vX zVOz0#yU0vu7gx$HJ(SJ-B{7xEJ(Q|Uz>RXDJ77;V!K@L0tttm^%pN%-ekuz|k0Bo3AGzu?RJC&&vA`FSdSWl4Hi zN(@q9>7<_LYpP;@p1|1q7`*_b6wW}wg;5RL*PNzptKql^(4TMLZH}%edu&8uO_z{S z$q&+jrY_0iD=FhScaiv?xMY&=NjwPdb`sTQ*%$%C&hba9%w&~ zckjfN0P6r)kZ}xbJQYcs^zXGruu@;!4F)e7R`yCO@??HnU(Ib!0~r11=5x=JSCPn@ zZhWpmuasn&e?Jr1)SXAo$-jXn{(ntoA}e$&=kt%G`~1IF%;kmFdV3w`;CwT~77y%F z;6OzDzY+g)FffX`cWM98^sK`ebq^yGco?Iu3RM4k}l)XqHJkx41;)N$RKm>2{RhoHA)NAwKkBbGn-(D<^+4a{%W(Eus zu-R-)0c25E>TL{u?knu2tNUOzgMgjk%XHnxE;VZ=Xc7HIuF0TAe~PzeFpxxS{|sa% z2|%tqbck6b<%1w$lhJnO#+uMLHeCrDI(MhC27_co+i1ztZ8APL5ISVd)cv*)BU~;k zhA^?m;r)4WyIi4ul_KBof`Pt|_Y48NGYk&1;{X3uco9>htZxS6E-`x;R zF4b_RpWA)Vk*J#d`2)181n&co*md?p*bACki;FA);~b{JeQ6H_e)5VMm_|M z!iTh6#rN^B(?2vzp6=PnkQl7? zty;>pKfCFY8#Z>)8M5TkGC4eGuOgAINY<${;%hA1#cb)Fyspo%?610BT5NJ%FQ^Dh zZg%&r@~n8x$?Y!hX*ODrYq>_q(*%YD?e`B<%mxeY0Xk84t5kH_rDW*kBUpHxG77pj z2i5ZJqqj)^9yLHMU{eA3@a;KoU;Xp)N0?wcvjyH5785>y1>ZM&))K${<##)`t5i=O z{Aek#ZOSEmRNw%sst?KbX*}3G^}c_l5o6DL9HFR4b9t^VbcsSKsA%-M?hjZ$`k+!l zbGwS2>~iQ3IY^VK&T4k~En`_q5`ALLmvK*V0Y6pg?Gi)x#eO*An0}ffK)x~Yv0@wo zhZY~)E8O&fB~%WzLr@})n(Zh{ThFEAa&w?Q+)gHPFhZrD_?k5tc`Ny{*7-arXDD4= zsZ|We5r-`KY?Ez=$KHpaTg$pF84&w=A7FSH;y?}^=|?N5C~bk22vrGhZ#96pS-=lW z3jUW2ZhHa}v09_R54MfZTU%6A65GF$X^t?hOv=cZ%smx<*+|Pu~r?`N=kY z$N$?1#(OQ>sJBV8X~gL3v%ys1uHTh)^mu>MZA>3Eg@Gb&uzf<1?BrKIdG-Hpy{c`1 zz8~WEK?&?dpYGm_hu1mxeEv9DrtYMF%zNt0cLB54f8p+c$URTWNQ`lni}Wi<_L}h> z+zXh~LGH&+(mXWEJPSY;Bc_tW*u*)F6EOaWMSpI@eo(1$c0Rw}GRC+#PE3;eik5>N zb49kdX&0aYRgFYYuH1a2qhrbv&mQpW;Q;;gX{)X@mQ##ivpd2t zqD9P>NwXG6fD4F-&+h!&%mhZ+{}foRoql-aKZM;QQDCMY1POKhQl+@561&}}b zQi2JJ%?kw9h(ti4?C#^E>zAxEOgO*ScNmE2+7= z^~-HK+B!AB^Y*}EBIe)oje@sd23zc5EDi~RG&gwnr>zLtz;6(+U@;??W&4I_{riHP zVJnXl}r_s4T?kh#9Y1tSt4n0ZKeag#iv=mY^B zSA_uk=RAhPG8CjyTeMjx$e_{^x9k*1|6l}B$vmUs z=Sl!r1WS<;YJerB+(snZpTKzE51Zci1svfoC`w-_14tXU=$~BQ0JVnuIJ~ArH^;C% zYjF=8PF8gGKv47H`S@MyB9uv59nss#YfJ^eJZUjAK5g>x0BWgE?b77vCONub`0Cm2 zwBt@@@N2%sdxbMEbocWYVTVro=g?&o`aW6mm`rY>A84#@JuVhY{Kz=uK9|U9P&0$? zdAGz&`)65-Y=x&eVYjiclK{i1E!Zx~|+ z(Kzi>ov!2;=GfNjx2)JYyJoe^5Bppm%IvP!r-0lUj}B85wlu)15*!#*4hMYE0?X~m zPocUByKIx0KP9;bap}a$b{rUBtfR9WcY!L3_$(@IBsW6c?-*>|kdU}(clT%S8Mob0uKa*YL1{d4G zOTrQvnOc-xlR{q5kTt?Xqx0lSzDiw%5mN0Xa7F`HunyVBrn*6m&J^vE_%m{qH);%Q zZ{K-t&N2EQG;@obXUhe@2n<#gqD+D5S2&nZoTDG`{c zCHAa7iiN^YAsd6ZtyzHQBQ=sEUC~8n(uA z0Z8#{IonKVkZ1YCT4M5EaEORv;77PXUmh6g*;|sc+EITiZNRm-cyrpk(cO7zupeoC zLea&M=}eR9VrS;E2eDlx-@P5WyQkDO$$Fp$N+jWT#~oWwSYrrqZnQQMSX)w8MRrfR zEuarcZ5mfa_NY!M3aX${=^4ZA;U$4BuPJgl@r)|28q z?CiG0V@#I0qKJJwY&?{NAJW%mWXKN&!Txb)*k8U70EaAlN%RBxZzBE$%xE3ogAwgo z1j-fsI5V9`hsYRV4?J0VR|XQtFZmidCM-h0VZ*wSJpe@Muc0q~js%Cm2vOpDNiU z)e~JjBoh+le$ULmdd0Gzaz16?@+w45E*f`VAdbkaC~Nay$P}sx2qpa~E!Oogqo+ z9}G5xxnY9*m;Y0@c$2~E7CHt7sED{god0-@%~DBoMLWa4{Y`HZnA}^#AarjWOrX7E z-Vvqe-X7>0&Q?h+_{JWq(|;IE*SU-_ye&o9G&5?;C8$nKpwnbV@{G<;2Do;~V7t8t zz19EN=!XYXBVBQ*PK(Z ziDkewILu|lJJZs>IwPJOUz_DDlAij=t5@G}E=SZ{SNmhQ-b+yY^dbY0yl6H9wdh;z zYKZqDCg6e-ZuWi%(Opf?e}fc$@1Z(9{xAn=a9=2p05viN)VdoL=uQD8((bIj9m`jK zV>*+@&2!xSoIGkVL;;42b2eGd>wqkeu#w%yZ`m-PBOrLO`}xJ39-HCOhZxo)?S1nX z@KKbCkxUuo&a1xTszYFDc;m&}2VGls#&>!#+G5q6TM|B1@62s+`0o)Q?YfRXi@bgkH7OE5dhmkq%DLKN%EyS8#rte#J=Vb!>twY54U}R2>+>`pg7W7| z&W@dq6&8IjBV#qW@VT~*fe0vYO`r2=4Cca7t_3VjVbW)evz&yECsXJU;jbmbW>iV4 zSFwtT)A}Dh{hF?5@U3~;>D%!9@#0%p-txoTEZ%4wFrS9!jXN24)98w% zg&6DpVRGDVH_sp{AySCXYJ)D`hu&a>ZhhMBX9Sk45g4S~$8>UT_wtQhn0lS~!K{E& zp9oFv=|b2iT`5|}`&7?XDu24p-sDgP#GDXHGHemvf;5b_d+l~Qa)@`37wo`bVGmCT zaJ@^G>h=S$(E24Z|Nqh8onZs!&h+-?+P;(6=^ISm$GkIh%#seKUj7oyf2yxRGGlO~ z(w%zf4|A{P?aS4mwT)U1c1t>e#YV*+p^S%$_ zVp<>u0)=E2J#*5YOFBo~8E3kSU8T;q$|v#k)768cr&T>gFKUfBUl8xs)&zpQjivc3 zYNh}@?#C5uMb}hI$t`U@&h$K){WCt^@jJi)Sq%)^CsBNOV*8kJ%Y=Lh z5%L8>KyY^z+E87vIoA{WGfgXg3JwdLvr6J`^cJV zu3qIaM40DSo55Zu@*dDl(XJ82 zliy!C2ZD58PN{C}K96m8bKli+T+=Ta%n_K6ITW6~PDzlATR&I4&T}azxXK!73G$V| zFLIs`*Ys*ze3JPyp#8Pjxb*TwIVQs=H86@(*U6MDBqfY8OYCC;Qzc{aP>i#r3=A~X zOwikXTBaXZ8l><;Qy`7g-HJcD(fFm3-|AA$VMrSzE2;+tI{dhh)xcuYj1c1c^!o;> zg!^t!Z%1+?gr$t0>s>IM^KJ8BFnZ*0Ab#U(rtT$pZI*#fr2}aVeE%pdMSlp zkMXPRRxpVD9SjUDr-^EMQ9eF6Zs)M_?)QIRXs%JiVW4FXXk7njg{?;e<7xWT-Fp!2 z9_TAphGnJxCHbvKXiS=H=2S=l4IgJ6rC*!HqzI+1taO>*O%*3~7T1D%5;!}7QEeu*LX5Tztz=Wna#2+8F$)c; z6-l^<1Qdq-ZeV@Pfd`qP9IElPa-v%|Ur6U11bV<(gxb1Q()V5yIFCSS$Giq%7bwV zfpR{WAWF_c(^dGH=SG(tg+)xH&;m+?d+H1=-}&qFHm#eMs8pg}_82i*L9@|@1a~2^ zyo|LTL&mp^m5CGoTdan$ZJuyPp?e zcE>5w$Ol@fagR1X2wDKbeMOCeX_p?E$9?tZ!X|DWCYRp+K*liNq|^sPjD_)YyKC5_ z?!`TaE=?D!MYj%$K5c&LND{^L-W>8qCM}CKPrxSRdOd!*OTZ)arU^3^ogzX|kPGhx zi^^xGL`AiSuiVMBa!&hf(;B{$TDWV>?Q~0Deq}1BPk0I9)S8Jkn6>#dEA`l|@$u)F z8NzNxEKSfY(O8XB?hJ4Uu`G?3<>pa!vpx?Z6Dic6$)36t?iR!`VN@=PH8Uq~_CIf6 zi$9}QCbiJ4)y^x_mlC4B+`nj;^q8E2mcqoeVBspgv)bFG{g0@#luCIqU=*K2b#s8!4?avD z6?wYO75^3-jeofBU?~OSP2wF4tA7|982Kg`f&=p8#zd+9Q5%%jHUPF#7MZz`XouMg%3?Dqj+mLL)XVOV~Iv{iW{Gla$)IpE* zJab`Yc3aq^86A(hjOC<*D5+-;A{2kL>3nWtv1WH@&dle1xw{r=%IXUzz0;#DdDSX6 zY2a}y;O2y=CFZaPxk@F&&yjfFh~E~4CBOCD6nn%i_5B@h4IOzC7O&BKHKdU#%?FzE zQ+ZgTf(nsMxuY8LW(t&V(G{l`5vjFf^$zKs;lGRJPT-PYs%H(BmQ^GswztDYFYMfH( z-d&cLKpfHHDMRM<@P_Ni3<)WsAqq;2g!NO?3odGY@d;AI6-Tx>(t9$j%tZa`8k_m6 zIls~7^PRb#XSbJgX2vAfjh{pp$gllQ4C-Chw+7r1o!oU?2{-xt#yt*2hn|`rd-%>Z zX%V^xboI$Tbe&$kJ4Q*TIA2ITc**B#^0}NN4fC>J)UrDFEJDehZ?#s&39id(w{Ia_ zw30IMUG{ErZHL&#)N)5x_*VS+Ma=rt;t`+vp@y6zA6v8Br*&f;Okwrwko;43`l6(q z#`y#je6s5`0rk%>NyA4SdU-z@karpI6i)UPbEz2|&pDQv$8t%S;Mu2p-fC*06&Cfa)>cm;OBR+-HoFVEoqa-e&5hH*9G9yRwyM9G*md4;wjrkQ ziV1VTZDYSjhHL{{c8`Q52>0iUFtMQG@VM(wJEI4GUq-2bgNF*rh37ABBS@@bvdzw^ zMef2y-N0wh_C44qbL7}r?@Y8tkUD|S+a2C&!^u4HHJ;|vQoGevsf^p+XqBbv*^m(p zoGQC`H-4pOXPi=2aPib@*?{T;W-h)bSV-P9vqW2+p_j6V#EyH06TZqv zl}e72xKlEx`+J%3)7AClvkh*3T$Deu(DP5}leV|hIx_q~(`T(-vNyH{XV}>E>kPZYx70=lNaDms=_|?# z97pyyxj$O;C3-sUcoapAZi6=glAE_>_52%_oal<5oK(v$Tq>svWp2wm>3sC7`+Fgr z#Mz`gWRvPsJ9OC{mfHDsAEt86E|#~J(+~ZrW^%vx_jH}shCL9pEt4%Zz8)f0Yke z;OazmBz4?E-;hhMbhXXqRwXsQ3dJ|#$>uO@!xV-@@NDypo62caxn`K$V}H*L!x;^E#BckX(D@+R_Dz2Gw5CKZDH1niDd2kK1^Kcqm-ueOVu=%{K{38 z^$}Bv!7FXIy!x;0I!e)v#>J~8`_|Ky3(YDz%%Rg&Jvw8Jnb*yzmV7L3Sr3zj#{0t4 zGxUEQUWI(w4>AM^^=E!G8%h=@bDm-W@$_%= zlrQk4F*{wTAHCv&yf-OhFR)rKz5K3fJscRvt?DFC+7d?UP#RLdoL<3){yg@@0M{6J zZ#D8jBowkzCM-d+SKyylsf8Vud)?Ij`Y$^(GgqSGQd1SlJ!Xg!74?#Q;1GzB((UaE zNXMz_Pi2#Cre`)nFSU^(lw|@D;AE{sUawK-Qw$d2I12bV)}Q1YS~p}%-Mp#RHm93p z8g~`AKs4RHP?-;(KW;7ZxXyCP-mA>v-(F=`f~TxM?U||L*{|ZJk{YMtP@Qxpmabau z6QFlY`(%)Y&tZ86A!Z%7sBVoWu~5X^B@oz*!jR57{z9G3Pby!0b~jpGtFw1p(=MQE zcR^EiPvMeS&{nDXu`Hk0)~2TQxZ{KE-hPEQodw0ZsgRhhRqW1Ea$Jxsh5|Dz{uB7h z-gv<&ew2IGD)KuNdiSDX33gu**;gT4f6zE4hq}Ji>3U3@O0#v;+H6ocF)O;%j*fl7 zB;krq7XbCwi30aLQehslY$CR6^SMzywZ59ZoWc<6LXeQP@{t^IHojI#R3+Wrl%uZ9 z9|(-wl=iJ^avFHG6Bw~+c>2}Oc>LL#8$AeangbT5p>T>-y_8FWWCXtsWsSawfOc!F z_yu>7dZh-PAG%yFmVi~5OztyTVK{FL1ckb;`Ec{-n`hzu;8d8P>2e0&5hYgr7jlw6 zgFOWM9p`6aE{X#NbG}gP#Y8M~n_oMr#q2p458t_CxVGEDGg%7ru6^KLtMn^Du$`SzrX?9`EAo&PMptuKMru>1&{Q(>PA`b`^xx6l=682INc$X( z2~kO|9!O?()6lqo??u)|GG;)JY!)cRBOn zqGqqw(Yf{&J2$Mof)A_1t7CHq&D}+Fy zDflj#cXWd_Jxuk&)5u+2skBxes-vZEI#I}KwOIAInq0Kq))Z;SLM!TQnQ!mCuE&fgz<>Bho945UsPTG9N+w)lJ8=FnO3Hv zsrq7_^rPbmZBL%(^#_Mza_8CLfd{+?7k5YfPr$)=5w$R(komqFqIv9%g0{T8TtNtt zJ{&|bKV!8Ad5m)y4M7MeV5qOuW2VM>F1d5_&b#{4=i9<`7u-@O{6bg#BC0~fc73aa zM|~W#)4s9=bH(ony2~yn-x2aF?w|uWIN4a-J{Y=!`&xE{#4p2f>E>GhaHo6YC*&GZ zYMmmWoZ_l$G<3MLI@Wb*zRD$C!5RYG&DG|j)wlpbBe$l$L;9U?Op%OL!&OZf*LA}2 zFPu?-PmyXGnc76C`|LE(v-zi95ApoSRLw#*Yo%`F6m8dLn(8LJ8vYxHmQsrpd*=$b~mGexIN?Neg-U8N2keo0g`lq_hO z$9wnwD`b^)kBwrty#(^|UAY^lRI+g2sedsd$QEwE$PM>qol!Ud;-=yO-{k7p*xFUmRE72AO?Zj= zt#}o}Re{#U*3?b6M%R-HwrAsJ_#_UYj|Yh(UWV7nZ<>u0ykTzs>6av6FG}a0grf@( z8+;0p)SqBxGSB0lk3H4Z# zxNU+!Pkh(j;I4~-o$>byyHLw6ehlrlS~bc*{68zVyv{?fhKDNiD_13A=_!@b_3A4Fem*19R{y*MxZ8R?oy?*)BN_j=y7p6IMopdxWFHJ-10 zR(4>s<7#h~Rf(Y5QclC4%PW0Ko(D0(dR-MV1D3?TBXJS?CBHVyf2yZ`bCIf5@8+NtI#k&e znm{_vayTGdMi%5cKTMRh)$7m;+;{bqCmPwYcRW{A5BKnxda^aG*OIa>EMeZ0#HmZ{ z;AgnC6RV|$ypk{UXsa;#XXNIWH*uqvjjk-opxS7}>C4^hBzphKv2T2hO&zP-9a35! zrlo9?a^-pw;xo7ExF`byWy-qSO`1({M;F#7w%pWKRQy5XEC>=-suIrOPrV+_)n?fK zJ#Y}F(pv;JQ5RTt&%H036~P3nx4sP5SgfVDokqeFFb=5_92Q{z!rG!eK{L z?yU-MNhep;HO^VCw%tb5!@jLNEe8rGP=$0AHIkau|6H$MSDSUVoSPJ%?Jb<5S$7Ju z$eD-kaHeZtI7iKf6t&2{maEq~si-jN1j__ZyH|E(5-SJ~mq_FFJ($Ib^XKf$0Qh@3 z8|dM8beiY~F-|wyF?x-5z3?J;m*j1e%e=*C$qqz+oI+&)Rzk$40JHM&6em5D`lc#H3YP%(2||U- zC4*k1pmuF|%O01P+Ap*cw@|H~hN%$J@j^SG+M82H(}v81kU`(X?*}Gf-VUw!5yOH@G++H#DHmx%-M@NKWWp44FjtvMg%V;z4N zqRoNWjp*X&;C@oG0`wQutB1)Kb?53ToiF#fU&3|m(A`_xmyTnM%Iv`-Y?n)UE}8Wz z`CP7JeI|EfHWV$rYiBHKd39{Y#=1>~)?%cDxhnx3&bT^BnG1T5ST=~-R#X8TMTz#Bb-imjF zQI4e8)NWBv;i3Jv3`TXx2oeb?-an2=Z;cyV6&>SskRn=m-A2N^c)Zo56N$LCcZtGw zlfWnGKeu5=(8K)&P-bh2$KKbSpJ%1^vmtP-9<$`y7R<1_q|clUCmft)F(%j9%x@n`T({BB30+ z`E~_U3iGckbKZSDlMbJcx5g0Qx_+$0QI=@U?^K@eJ3O0NXcmYmv(dRZ)+oy>S=6+H zvC|W1Rrgv)Zp;srneSj}IujJkjMLBdXfmdh;L#wR&@? zx%R(@?{9Hud1#Lg%#WQqQ8LP1TjZv?9ZS6iS;aLEKU*hQs-KfluW?o0I3021d=s(( z7?>#SPHe4K<(f=zDcBS;#ZoK7g-GR0Z+RVi&P}&CsoRgB3p@ReibUcUY0UyjW)aW% zYY66Zap(_Ua4HOL!t0wAB_YxF31u4C#nCut@ypdf`df*MAp4d%=dK6d3B-=`c0QJN z@(+vk2bk3v`ub%2de3^OCG~_N@slbZ=z3rAbAerMUT9FZ|HMlya7zR{R6>^L!^=OE zngA?34@p)eh;r{>ozya zmYA=N>Jv?(_=5x8l@KDKr^?&jHm>et_elm-V}TzJc2NbNuVz`Z?@Hm5UD?v(1No$jB#4q3NM z8aWi5PRt%{KAz7j=IB=9@T5jfrPPeeqpCC)+ww-7B40fm4LzWHxC9M5b5zz8xr-^w zDGPB2)kaL*By(i(=pzRwjd~ed+sLe$%u{QR0a+hwEtF_(2 z89jEf)pIHrPYf$%XR9*9!eNOwJawF<(h86FFp1%IDRs)1QyEKk=ViuN#?6;J`#TXa z;dv_@J$XDn8_LcGGdaNq-9FtIY?)w-< zC|2eofUWfTyIB2Fs?vGh{EZ}|#(hf7WsMzMOE2y|ntyX);qPz3rEa}R@N5ge*^J|5 zWyaJoPSN**V=UB zk@!wg%zEZnpS0&yIq%_H2Y;-h#L6Y0rDu85DawCI!=b|8j}{5+4@(9w^T|I2GZ_}S zxBT6R&8@AKkEX++8(>*|TrDOaLE46X&Y!v(400hz=*2%Zez6IU3rb!y$=NS9x6CBt zDHVjbQYQ;q?q6sI@5U#)27M)8G?OEwZkbAh)_D96+o z06Z8w#mEOZ1XO+*}Q5)(gSHN$SscNoE?>9={mVsnaRZ`pDB zc-*wlsVb$vh`74{Z|%yL0;i~E999Pgh5mA_->hKIY-FT@Yk%cRWpu{XW4^!OQW*#7tsMrmo^8=dWUC8MRe41ci9KxT)E4l z?M8^=Q_Fa7hJr&}DldPG@7{}%?d-5gfU&?_m!FS$t@{}=ph-lLT;=)iRnzlbPgI!W zKH`pca6vj$rRKYBil%<(B4MHU!vSh`Ef6m1cJ(Mg5DP}+S!7wwV7BloiBtdz&~VS) zj<{#8*3e)F85=pG#pq#Xv;NaWLPtT=={%W2Z3KzjS{uCo=2Ylw0Q+1aro?|A7#1L_ z%TZ?TM3fmXhqOt<^LarXTc@t3dbHVPH2FcLe{hf6VAj41cVjX``dglr4ih|7UXHY# z;)fKd@aUQ3FsQti@9FCjavgou-WbZLx8IrNDS7d-vBcA2o=Jy*?VFPyVEcPQFPoT| zxqBmkncx3<`JHi`g*0O02^I)V2rTHyOz+_ov+s!vg73<|!V&wg6ToaOGST)<$R7~k z0jnc5c_G(~XFKHw>ta<}C)vVWtY!33_hqs7HOO4N7aJvO&|(>TEQ`e1;I zBa_8%2WlyxL*sw48(Y4cNUl|PilTY0uCupm$Sk!z8~WU2GqlM+0)-5{@8Y-}E`wSS zj@E6BcipqITifO3YBe~&=Jw%XQh4Z5&7DFO1PP(U2b)6wSSzsGr-E>S4*&Q(R2>}8 zCvS*F(}MM0&pKgK@o|o@gvgyZ>6J`;Nx<|#3f>pB(3QL!#Gb4?6@e8emE^t6m*UTX zt+jF|NsPa>2n<|Xe;pW=8nhR zL66zJzVKx-$aEB#x;ZLdY4iSI@B^J=Dr~w35ryo5;DIgQpz1SVp2+$nLkeQwZ^L54 zScZ+*z=JpJY`_uh4s)p>M24s2A#V!*%MQW-AGibZPd)j|IK3rcJX6uejnXW)i_v5c z3>Qs(@o3g=tzv0u0k%&b;3IEvVCZ84AjWAv z5+Z01J8^+iSpIXGT3Ykxk3|d(J=o{j`}UQ1x^lRQ-8f}Nvc``%cQInTIc5yF z_;DxBbyxELAJX1BEUNAc8wR8V=}t#M5F{i7>7lzzkW%SVT6z!xrArWy9y%pO(m-0I z1q6$bRw?Osp8*sffA9Bw*Y*AJaE;HLIcM*+SKRAfYwt5ws`k*t)~cw9%CGX(-LFwY zb}ECxh4uPQ57+4V+xyfEER|W~$Rt9xxBOP_$)Ex2j$7~K`Aqt3et5k4Ua#;He-h8q z%pHmY*=mS`NCFiJ$EvFW-M~};uxztqMZ`0#QcHx+7>ihJrm=kOYMqvKexn~&;Zh!s zcWl;>N0b7@G+ULn{O99nK?x*tBZ+MuBJ!R9;X2QdS7+nGWF6knaASO?Z13o>5kw@+ zWPfJVz%67&kT)o5)o(8vZxxaV^Xh-&N4RD@l}t_$Wo1p(|jTj zix=4WHJy33QQyBe=%+Y7Zx8awQhQdG)>B!oaT0>ddCwHC5(Jz9l+f5zaN@#aI?hOf z%9dacLAz=H98*)at2wtXhLE-{ykAwORCZbYVb+PldFU_VHMq>evSzvSN!OMAC%k*l zB)K*3H8WNA1pGkd6-R&IkEgJ}xS8HM$iD;^^ zbIrZe^Rdo7_;tFbes9raBkT3#gmu+4GXhCiGd|j;h&C6fe*in%SZ%UUZ)eiC; z=m$UXZ?|tr%bUBO|2E+Qv?MLAANE(+ko-+C(I3FLxVgEfi5v#ZlmU%?=(`Xsz>ft% zpUo426JJbLFO!o->$s#i2E3gF@C93$ADW`L#KvUyetw|)LM#UfW!ZJ&?em!T=qbLB3NKu{KA|ooOX&M6^F3Cy zj)uaLitCpaN#5_lba7lf^=MH-(K*|UcfYFk=XgimUn-c$uGzbpxdGn9(lh|`K_cF{RHv?z{%VgE!o9Jfl_@J_B9C=CF~cSwlb|?lVPBPa$4G+h>Vknd z#Z$QEF>r{~1$!0NaWKL?>nCDhJ8xirsK~dkj;=4}BE( z5-H(X#&`Ki*s;I63;h0(bTIFo19nQiN_?MvrYFfFJ|>0fXUpPi!LvA-daPHD*%Rp_ z_TG=05_&;@}+E>DC=V*>*nKAOMipAPBhkYz^)7TZEaE`r`~G3-dk3E%PZZxbSmA&u|6l;{o`o;Pu)59PB!QH$G0Ei zF;wqPbN~GD#v|NoP))coR9Ob=)|22E0H2;L=4OXj>O4>g@+P!*7_?QY*-;yC>9uXk zhs6}mDsWiR!D;!|<%G)jd>uAfiY6WuV~EZ$MCe>osuT9SqSy28o_gU&zLkdhS8Qsh z6y{|b#n*TER-P3}qjiP>V1K==qk~Id?aX52Lr~xV^^IEEU*Z@qNFG4Rk?;tQpdo!2 zpwzPNGI;xS+_Z6YU;WAuW#pMA0{*KSo!$&!|9` z#$-4js2c>p0Mx?!x%lfk9Xhjr;~W7xW&=R5Q>i%v2^ALrPK+L1@8@OuMke7`T^Wuy zmam_c>is0xr}*BbgZE9wH4?rY8T_Bd9ww|`!*c#&=V6%QH|b5(Gk_t*jEgvO!~-UJN}KQ1M=|(1^E>~B5Xsqk0|qz4aLLjANG4p$ATcA8C|V$ z2Z@naKP?9CFMts{B~UcQ#?Cbodq_%gt#e7->89rn3^g;CE+d;Z`$P_UakKNC zc1Yk_d*7;`_$)Wu)z9R^dU39l*@~Pl7+A^R3hyJ1{OR?fbD$RL{IzMMC}e|JAsyum$?GEP`o3JlB|@Q3un045*kbztI8C;2YQO_ettB~@};M@iKU20}L`1R@AZ zsuJY|Frxv&vlDuDt3Ei2ce@zhHpc`st$A;#oTn?D#h@3_yw_^R@)+siZ_41)J zuapITc(&5W#(6%P3*GU9I!Biqhw-IIcarZWhy`l!y}s_{>&yu9ZqH2i6BeDt1tdFv z3GsvJFfPN#0LaNneN0G<8xAk7))jD=C>+Wb$J8y;;}msWoLdm_SfQj44eeB3%Fr1+ zxONfjSs>uzirH;2$s}YWfrobDxeLBMY#nm)&}xYU>Rf(0lOD7GB6v{5GsR>=Z?q12 z4y7uQ2+#L=nPhqA&K3!DXC|6UUet}6h!p;zJzXW>mS!TIl@VSy@yjf#=~gse9{|r2 zZ!*0~j8}h7ch^VyPH;8EecO2&vCO_hWK>qHO+|%Jbt&i``?uXqgm)3;6peMg_^P5J;%8+uW za%hBcd}pt)@?fO0q%R2^Z@$~Sh6~Uk>v_j&plx6P8|X;=ZqNAEjnHB8ZQH;1TirK7V#9yLPy% zL%KIkyUjjz#nj_K*^)tr&^ z@sgCatLuZvg|czBy)&<5aP|(pfKwHV);4;C2ph0v%P(V7t@M4^7@Fiz{OH+r{^Jm- z(UIMy@B=Sa#8mnfu_O8fG; z%?kgVBCLmwe9_v<>+_A6^RbSJPXe&u(^584Xl%p-#LvzH?Cq#@5f>ek144(*qNUGl zk;T`Whrd0u9I`E-77)#m;&pNy9Z)+sembXs)K~L3WO3DG$K6(*@6D(iSQTiU5M z&J=C&k7Z=KxHfkbpPu>l(R6TG@m0u~A4PWAXF&jp3(1nkP{@4D5(P3Ltrjzorz&_&n*dgTedMF! zKP*p7cZ;mVa!q(&!;TK02zu_3_+`gZcptrwVes@!dE6vog1Va_$wOOsuJ|U+(w_=PJ=^?{)Wx-Wpy@r{DNl>i0!*( zyhhLNTyyD>d7u~+D|d$BJP_-UyAnJ5MTJZFAVmBOu5kjQO%=wIhs9zD2?;H|ndtZ7 zk0taY6uyq3&>7AUE%RhKD~^MIbuw^uN&%$pk@9C7R}Ua#RBrCep6;5}rS9tIYPhxT zOEIJAOE2bgZ7ua70CL-SO#siuYA%xH(B<~>v zEmfnxs)0RuCobE`edY+5q#Ce22}3l4@~1#u%(U=~T-Ys=_G0*9`QpQ}*w&9k_1Kpf z_t=t*A5LzH`%6B`H{D*C^tNm--bs~Uez_BB{k!ECMT33=j`^W(z;A$Z$SBz`0&O}9 zwGXzx^TdhJPOo>yRaEfW#L=^H(s2L=%rFRa558IX8Ynw@^&xxw;I`~LWunV-k~w3~ z$EpYoK*|LxXm}rQDyL!1Jm0ie18~5wM*OW<#t^N(1q7Sk`OD&?YjfIPuB>(sAWxygg%FkmY_; zliJ4cU_0de{1-o*1I*hJ@$OUnmG{1;XQP_%wN_nvJ6|w}kQ%-Z@{f&B-&C#POE8X@ zabGu185|)EN@^g4O{e=tuBG3xV3)X8`WC{=NzVl8lFtwVb(Drd&58qWkAno1kwA?W z-9&iaTS0$2J|lf!883ZP*m6App0{)6QJdopZi?kXPtt5J^44R z;RgD*v{1v}(k4&~|5Bu~9y8aoNekSb1Y;FGHOvZW zK)fETzR!_Q$u~7=fSjbzlemlYHyIOkzrFqtIix0{=kV?G*`e8t0>Mr|*D1AoO?{1; zZrmcW{irRO4nj&->_=6!xpz3;7USHMwo3+B6|q5dUa|-?9y)b*&-?3JoS3r=dLR^<}3&~FRZ$0WzWAB&Fh$7g<#_t za^=i)YcEol4kUx_-L0>3-1me7zb45u#whBmDG2f`F(Kf17-1$xEOGv8OkcZi%%*ae z?HBAVkFfF6ZEJriyLG$a*S&CRVg9wRH_>p-vs*$(OcOf`1ZCzy`IwDP-?@zkV3RkZ zd`c(Q=Ox$4rU&}`ya#~oWY`eO^gFt!9m<*w1TxRGd7xKU_KgwT?#g|GU2xla(EX~? zW>I;$1|EJG^2Fah`NZJOduJCJ#>d7K^G}zL`W<-IXs)yQUTO>SQXvOC=R<%vwwifgGV+CI z{q6;T^Wr!byGL&AzRRm{8-6N5M2020mg+yaiG$*dmY-=`*nbtN*vYX_9KS;g?+;`t zJXe$v+e8cC=%CPVzGtCtyb7=9vB#sMFz8qxWJ5Libf?Kyh3IraMqIKW$ca21bvO%i z&AG4iu86H=P^M|spH1a)cn7z3@rpnCBJ>!)D);chr z*TJ3s>(Pt53-KKlW;5Nc<^3Q9ch$<6@59onHVf$M^!Hua`PuSHV!^!VvqUP);@506 ziHtl9)uGaN?c;H!KQL9)B330yn2k2jD*l)`f*chwEm)Y$Bz8PmR8k7gvBU$J=R&up z2MgxSl9CFW1wBslskXOruh@ehHpOXzM2wt}x}~%*>G9nVUt39JPYSZGPH}$Hh{?_E zwy-GQ)(u+^B#Crz`}a)g>2WI4DQgd4MyZy)$=F{G!f)HHT3UsvpaTvkUUcvLr)GF@ zj%pk1TwzUsbIp~i7J5I9l*Kh(DWg@s zph>sn(M7`?M8@j=zNC%*rczC|+OC%?9lT#G7OSA!@}ccJJ00{izLn?ouc#MainxCq zX|qZbT)%ROX!R-k{lVL7-w53bgzOC~u0_7SX-GBym^)oR=6l$<=v;OHYSCJ9)&5V_Dl=l z5VNX{xO#tfk4ds+9NW8+tAPb09kDsRbgDhimIRW1yc%z?P6pjsEw=(L<*Gc=-(Sm9 zFh=zidYc;gM>PAyl2ejYezPkom>C5?I}UyqlVU6)DGk!mDXLmeQr*b38}!H6M+LW`>vU=-;YweGQJ^nV@H z&Qac6{@Jmyl!bAHT_qbZ?iuDcVmBF|u*w@o1?huOtKdZ_th%$uI{q;+P^89AWS#z@ zF9(jtqQd)NApoeg>vz`Jxnj}+u{&l%2s`r5`eY1c`S#xQNMlgndp;rcYw>GTxJE=_ z*Y3Q>7BwIon%zwYPARhziK(kGtd`sl>tqM)_tg-I_m}D>yj`uYU$LaRT(Q$+@-8My zA_1_O)`M0QaIGs#zt-cNzr7-g+VZ_IUb5M-G^234+g6ZcmHPS2Y*({SU-0(xd3mD3 zIR^u*q(PqNIfVuC&DswB54_X89Yc(B)~*x}%l;0=Y zx@AIM?SV`8oq-YEWyzTI7O@(%$YR7mHrtLI2og<^#IX7`!I$@cFa+aED)lBX5S;r| z*OIaN_V%X1%N4(aSR--E;@LgSx);6TodnLFgEOEE%^U@@!9E%HL@7?NAZVKE9O*mn zvePj7sE?_VcLreXQz@o)?MWVdoKTo}Fuh(>o}qzHscUpu>V03k-Nc=$kq=ec_{;^V zv7n`~9YVoeq6X9t(X={|1|H`@n@@@s_IP`!Q=ez#!+n7g?bWU~y6P68rJu+#a=9 z%UAO&XOg|LeQY##G-4SoH9scU4jD+lb1F2IG0;lerrf!wWAczEUm z!wi%Bj6d}Ty_GC3{|3LNXt1Fl~pRAXHYc0Tk zRTB~2f`WHn=olL2ilyq}(-Y8@Vnt?coXe()9G#T2<6yaDXVNF7c*)dP)^PKz|I4zJ zr$uzLrF0S|Z}$x!hO6AEyxCJYG$uSn!5IMYEvMdgkRv7p-(Jbwx$AP4*;a#v}OC#Fg76y4NXbpp6Q2%V)ySIm)LmRQ#3`$8Q$+VJ>maW zMVQH8Y_in3D&bb`{OeWYSmN`U%>;ij|Hn_sqCuN1L$uE2{PBXlDfq4IDgjVQlTNo4 z!An|D6d1R~Vah9u>)qSDk51z&m^a{m$L^(Zc>KWUDDhZF*7OfI1R3j_*0*DY#(Y11 zXiBa;C@C6fnyR9_BEM5B(9XJYd&FqnQ>%tOd??48sQ#P+@2x8=>!Cz$23PX0kGXHR zqCfmlv*(jpv)YfaV;u|5d$MuM!)Opgq1oRt=7Z1bpgRYjW4)^7?sowZcBrN)j;kZ~ z99~GtVLdYNHA;e0U;W9H^vdUNU+$d?v{*Gguq7{YcNuRgRU7p8Jcj_x1?kck_obnc zcJ4I8CtF{9N&&*_{X{b#iKZ3&X^!%wF>(MH@f3axN(1x34_7TGlH#eph$@_{r6TRVG$)gCF+b{Fd1sCQiM>7`ud@6KpvF}3F zdeO_%upqDPi;U>{p!Q|tIOZZpOYJ&J#p)_&!b+NcpyOD!yl>?MVu(;-NPREa?wV@? zIuk)YVtY69XE2?J&vJR)jg*b(iu^`Uy1ez~H29N7bK%$*0ov;Zw6UayCi~bDa8rRL zyi=vY5+T0V&JZ|*m}K7ZkgzmK%{JEF1;j1iM9hXPLr zi+XNK3UaWw;No6tm|Flm8dC?Mg~j%@%;ID7RmH{&8%@+*B<&sH$Vq z2ndb1vPY@D9IpKKcbU`mzCF6Ijo!3rO>Ii1at23U>CU`-Cg?O^~w-?HUlOp{jLt&}$s}RDN7p)g!(?{?nFo_kiTizPiL+RSj&ISMZ z@`adr_ce>eu2?El#amb}9Fx(s%4ge(=7GsvP!u_dTtDRi`Eg&d!9TW91=!_JmrN6e zpRuaUhZN*BM>bB~+}dbmk9Otp!^Myvys?-yqoFuf3W!O*uLrn3>%7R9N6G_URP?dg zrb6_f-pZkill|hmmkJ{hXSoBFmF%Nt(J&iGHR|ac#-8ie^K>puMEvxcwyCEDXuxRa zw#}$nS!XT-rVrV>R|@sX?(H+2DLKVeD&y4)KY2b54`1QNj0Z9uMh6d7gt4;b8P{?4 z$iLF@yJ!ctA5T?5I+$aq?Tr0+@Rp6H4ftAGtkTA9T2Hh6mkJ-tSAzA8U+pivSyAe-yEO6S%~M<$ zumHQ#CON;Y!7u!qqzyCa|J`eYc{^8FzwfWTHtC~vazb>+8h~P9zFz|_ye)%23watjv$abWJZ_Mj!E0y`{cbWS{?9yV4+h%2; zhXbc%(=X9q8KR!0AHCzm*uc;vg3U?wsESNOxkPpK?arCt=CM~!m=KiI(EE9XMqm-6 zZZ6@h(q}YB9d)dlQum16smQsCtumu#tsh_M!6H+;Pi{GnH0o;c*x%Pz4K7UW%jSxI zK2|N)bGT=PKbbN!`D$V-Dct$=o8vYVRAC~T1nSef`Sfw|czGY9tE)?SOL{!~+W^~< z(#godgQ6J^hI*%dm~zAq@0FCcl{m8OY+OieujOu17*tbiAil#+++42-v4v%x>wsq; z5dS`KO8!Ipr$-!1Fa-QPu%RG_H~CZn*1!e*^xv;vvnE^b3ez7vrC>tckB#96T40no zN-z{>SbyzYIlC4JkH^Hr$05G@^X|)9`_e==yMwY~AGe@3&H;K1TxK$jO*EjwT3kEB z6fw=Y5dYA78Xalf&6oX~0Rnrh7BMmjVI>BYa&nRMHrepVeeX6wYoI$N?^*U4uU7B4uZmRf=uJaox()ru29>r zc1K09Oh&R+z`ooB+O%JG2K-4GV8W0D)E)VKgAD9rHU+i|bsI^9sL7SPj-EGoZvWKj z+Z0=zdFP;_n%Xq=@nwf4WI=GRCd~noV}J3^1RfNVb}(_%6O^JyD=b_(MYlXXd2hB8 zvvIRgR%`cvKPRo_7)BPzoq+*W4IVsySfGecy+m2&aiyPPa9J+LA}}f_qO)_n8)F~;wc*%Q`;CL02 zcId?!$aNh*$NlEA3={RJ0MteP>HCr^smaj9?t`?pkr+Lz$6W;){AxZNIJ}&XT zAhC8FW9vOwHZa(l4^<0_StBVhj=@eO??UYFRcmN%8CX5UWB-8{1^1N>54>AC0$Fv8t*(6Ie~k^T-Q)E2yh-TFWn zml%Fi^EYD-Eiy3{IPIaZtSs?*uBxDsvT~QjvUC)7UsCchC`50=U=P>Puh-LS+72x? z+QMm@9c8$tqXCoM+vx&?i1{6-hf}lbH*1+i_|G;qyT8JApvs177Gt#n){;yV;!2xg z6A}HKDm%dKU;%MxKDK!C0PYSrnVcMgf=kBbEmp)97?Qtp6#zAJ1lTXmQ?%6{jdQ2AhgXSyR&Ap^aJW z$?H-@wYAu@+4p4Ub3(!fSJqR*ckyWEhMv}I&d$VE0)q-L$5@@|=50DWgF-j|>2W75 z_4eUzBPlVKPgQdW&gcBUEJbVM{HEa@6n!B4BM<`8#6#tex;E12RnR)fPtU&0FNJk{sd9g92U4E-IaXU$r2=uiRlC2`p%f|U1A zumA3BX2azdToR>J(5k=SP@1^+G%Ba2R3Iqf>IDW2KyoU+Km?4H%FQc&3p$0ZU^Dv3 z)m1hHHDMd@hWd;cUHQkB z3#m1%3eq4EH}Ye*hwEyRafLDt@ZJv@HO@a7^wf62x5dfcZ4kBT>WB%w!Blmdf|Bsv zJ-PWu^JKk^C9%V(+ciK%prG>(Fa*LBRbzW@Ya>3dKN7PPf~IAjmlhxo9oX=i3hkBr zA=QhHDN1@5#EB)zz->E}e@2*v$~OeqSnl6r*KA%JY256NHrVVvqvA_`?~hS}PZxn_ zD*n^W6oABFj~*NwWB1ZfWVp6DL)#G}U}NhTJ2Nw*FkiqBHmLluT4B#*$tXTCog0)E z7fr}JbYEnUz+v-X>RC+~)DjqRXBlt^g}(Yn`5e}j+8BM>E_nl;vgkG9B|Q;AD-3@3R@(E= zgrta7dKT~s99@80foKYhHObzRCgyCYsx!goOfs!@t8OAxBqC__D0FA-?CMg#yPCeW zk~otZ=EM2qEovF0pq4`N0ht(F*!RPJ7pX)>hb%wOAKNboK0dn8e305V7(VV8MQVwIACHT0-mC*+VxlqeUrmg^mY_I?w{R^#9@4@qu5jvh1Oy5pqWI-~dZf z2K3O4bh7#1%X^;FY|^2F02BQj)T0K;=u=QjgHUC-8eXf(Q*!zrQ!ktN?R?ALCkkZ**g)^N4&n6SaTizOtcg$S zhA1Hlg&+QDfdVh>!i_ob=26$to43l3Iu2!TeLoqbK*KZ>xR764HuI!wL3tHoW)uLq zl!pQ!yjSEv!$)kMDdsNKgX*e|m{w>Jp9ffO1T&&-~BEg z?w3wKH%5Seyc8&-fgK${qXRbH{AZ}=NPq{5eVC=`S+{RHy{R^3-Tj_b?sIrsHfu0szJ(V2&S99tHF zEEZOU(~w|Kw(yR7x0P(B`EE|8HCT9ie2@fES#~eS&_} zruAnX5hEDJ&az$`-@dq2Hr2?Zb?7?^OT)5(&~|4x0fv&hJQGWzRbN^_Kz7hEb#{WwZqlpBc-G>AX`JB4(#>WV2&TumX}c0|ng&884A`>&A7a$8 zxZ$6`^8ekfLtEQ;DV;u&h9NN_LD|I5FAvWqqP2>+ALNkk)>hktY`i}08f@I+%AgHW zc!-D<^xah>@EEQvjWU%r>_6u)+=zW`czrxIdZZ|Eu&G)T4~Jax1m-%@k~2O`V1JN= zEct^!0)jIG22|F0jE4W8wY|#*G+tR16v6^rAzrlhPTNM=QCrb3IpCN;o_aSAe-x0R za;Dn^HWSUw0}g%Q)_1uxUdmtbH{#5(u=4(RmGgI$#+A_n3&o!cHk$FzBx(Nn|1e?! zAjumkJb%FKZ)I-*SQ76;yaq7_?lqwIKm9$CU=O!k*3x~~Io#3Rkw}qqTM}B(su*V! zPuczK%A-^Stcb?(4Zvc3*8%iF-je{E;-k}9^7Bxqv#^U4U|Zh1h?G{vgeJ6Nuct{E4!X^X(?m*wA;6agG`P3|l5p_~rbbJY)NQ$t2 zKet8uDE?Np|IfH1*A< zG|O?`uepAd_n)J~nm(cb2zU)JvCa(|OP~MV5*AH~ik5b@B_*7kd%uP*f?aUPIUH~{ z7b;o*+&w~)gU7gU@JMv+`;Ux5JlNiRhgFtL%eDW^S&JM>y`hQ}qpjx7jN$0UNR+ny z!-1E~(H~jm!CTa|TL(jbs}pdpPzw@ykb!0RKSTQTr=iCO0(oRr1u1T~%VI9E_`Le1 zWa8IkzrP@$c`zMx@^;`Z2<-rW#_mViqB^hH^z);b5ajc7*+9s@m$O0-LT}sKUUe2@ zuD6_(4%2bVC3Bw3C$2iY9ybzX0iU|D9)bIc0R32QXb|51u!_@JYOHGL5)^a_30fc7 z=ryKvSF8RL+KJHvm`dvUH#{-1JBfqPNW=cp;CmNA=SwVxgIIfWVc@g!zt4sQ&knGs zdJK-`;CX$xZ)nzhyC3x;=@KICKdKF@dCicA+z5(!zYeep$z=FNX)-^=9R-|0P&1*#_4PKG#$_mWwb)OGy3RT2Ja@G-?Yg3` z+u4~cL2b&67@Q2DJJ07W&Ak6B#s9nkQ8CW&selM`|LiYdSOE*CMsP}r+Au$u5OsyB z$IZ;J1m0c_jA~c?Sg$KC(f(E*{g-|N_XYlyF}C_mdec)!+ZKK1Qq&FyalCAKP%Kb# z0-Wnf)&g3eXc+_=sWM1bfCq?+8z}t%HY?s>mngFBSz?a`Wug3oLVbgdx&9AT1Jq3D z$c=%e4Q%nqS#0Z7yqhRlejrX?vnMePOk@(LMP2i9}utC1|qO%%thnS@Kr@2OpgMGd17Kw)EhV1OY ztekhdzu{Xf?(E~u?sH9IhT?@6wCcZB&ySwX|IawVnwK@^Y8hU?J`LIwr#de+ zT?)cl^dW2TZ_KL&yw%)#j@z-yL+;@_Y9rdwxlqhMfBr;P*_R9QR{fnuqhx#)9C0~C z9Qq0n0Y_W1$$xi{{#gWSFr8~B&-4Rtl7(pe;bH?OCizJICzpZ=5|ed{UVT!F_*X8@ zsD=Ti_8t;XGk4v5`KmInx~k%Qfj93C+XblRc{D{qQUXx(>&|0cI5D4l7kLl8=J?9&j2rIUSs*?)d?UkkYoManNk>WWJ~L|`D;$_WB`hDPA6FY$5R0j)&L$noIj@eXW;ihBM94rN-5QoV?q6bm_6X=bXwh>AD~)< zx4s5cp%e(X?#mZ&5W;_#v&92=vZlC!b0=NIRfu{*dno}T5-sTBI}0Yhg;1gTM@|R?e!%gz@9-}_ z&kKDmlI1e|r196oVSC`Y{kK(!{`qY+{+YiJ`=J1F z6I=c+a?+GaYH*$$ws|8ELUH;{?Ehe2u%vUqh~ph^jDZlz4QA|Fpm_bC(IDWn;48Pu zXv&i-21XC+XRSAR3~Fy=XT~QcvI0hYpiJ)H*&q;L{|RXqIH~s~Gf*s*O4*Wsp7E!A z@dwwze?fTPu+lOjoL;E^aZu=uS2-XAfA7Tu^v_8j4~7-cg5RCO*OkvBC_fc2svA%? zTG(;~otzZXpFZlZbvDXxllcKc6DatSUIL-%IcTg#(=Wm^m6>JCdL`ljr zr>e?mMtseBqlc6UX!GB92sn`)|A{*PTy+Mj#}JE7oHSHK4r(`{IoM5T75b0Y1%y2o z=q(K0&M^@ZI`3kszhx4b$Z*_@&2qGTx?%BN4e-z~U2?2!*`Wo% z@a~HvTt5x;PJioz^=pxxGy+s~l=;7wX#Q-hMO}#c^9y9e9UjiYe`p!-G#rdfA{c|H ztIqzGXG&nEGwweRfkiV0I2yYkcbcbi^PJKRK{o^${{#%YBo`bB`505_*!9S1A?q56M@W9DHm{MoqV^3Z3!l`+*l2hDizwS+}Bz8^TYFaYirTNr3uAe?c0KYJI>(x7{HtF8o!Rv9 z9A*FvOqHNC*-of@ymWBE9;$?775ZKF2y51DHKOcLkzf%5}hlJMG za|rS`x{5uxNEYDtt_Z&U$H)M44tlNF4(rZ;hG~LlE{moZAi6p(%WLs-eM}X8KdQm# zeAR!GUU(H2R66!Km|^Ru4YMKkIG7S&(qE?foc(>p*VWz+wA^{8@@-oK8x0-y;WPZ( z!ojNGHi5|Ssh5P}FY{Fx2AdYln<}50s7X`~I-9IQfNbZf`hLd`7IdF$L^Ee`hfgg8 ztlu$0NN|rGICUK76dn7dJ;$F%@A_(>Bkf8VM;xuAB(~`4egt>Krf|+NVJI}HKY$~F z#>?71r{VZ#b?(E>jtT(T{T(YdGz90tmDs6Vg`>d3jt}odEXaVUgy^jioJ*)G90zA zCty5H5D;>vX8uQ5Aj)LmO>K6>luq0;WN5y90#NFN9VsCEHeLbwoc1j9X0E)7dIR8itMr;pE{YBc@!V~BfR)}E$_BXJu zt!AG;*lkb+fvM^sZ7qjv3@nlVwU6*OKYl%?I&&nhmi5X#Q-2LdK(7{*C&`F{t(2t+S z5MofGipfpm{XXQ}fBpOrIRu=+>o(Q6FrT>pd^3%PcVRz?ruaeWUl;*(4B|D4e`k({ z0h$M75pY+?by$_s9JiqTnZ|vv<4)`9%tQ7QH+=9M(v>o_H-8;rb$EA>tlgvl$M=DW zktEv(|3y)vuE>OV%21oE-`mf@EY5>>q_F)|Mnn5|dO(5fz|j?uyY08<)BEQF+`!)u zv)~9~24%Y1p2QacxX`cPne*xV^J_RPspaUkEx+6kF)h6r0JK4e9_uPT&YR;mUz~tx z0UQ9CLOX*jI?VPZyf-np6m=QB4wgy_o0i0=)QP!a;j|h!6jwV|F9wi=3!OJrQR8Rf z;bFgVndis73{pv4L0P6wg+a_);TekbeC&QP^vn-gt|Df(R4~-gTu9?-u(JbS8f`4^ zOnXlj8aEbsdu*02mzI`x`HLm7^41p)8#Y(x4y%-SE##c&5dp)dM#qRI<%@UyA1v2n z(0hRFSh`9U@a!}(g1nX_8?=!B1rxyUL0u=hU*O9c*tu{FoC5CRg3F_vvi@az5G^WL zxbS2)ZYp`076vAWJy)&&hZ885L!dFUV4YNP8BH*li;;K^|08-!QNqyJ%0cbt3p_CWVk`EtVcTltsJ)_iy0XQ=9IE~X7skA{($DjD4XO+) z)T1pJKeqzUnPc?#O6Llw!huzfbK2Ax##Y`{%lYg=aJ(%7=VuKRU|Px;9_G^lt9T*4h+84 z&JPHr>_@1M-Ie}3bw+bF9TgmGls{u^nIaDHshJ@(AS|J?kP^WNp#f3njMm100nurn z!q6m9utSUWqp8=r?we*qoj~p^+8f^4fMh_a;|tvKNvyB!x~S)6akh9-E&a%Ji#xsO z{p;N5LHp!#$946|^Q=#vSyY5Ur#muBA@=HJ&M>8-G~=2)Vx7xR`#u`-SA+qx;RnZB z3(q zJH`=m(#bsuRby|vu!CbL2KZa~&&>Is%N(^=s?%b4pF$9ZZxqXT~32~Iwxw^^|bL5sqXCamg>Gq+j>)y zV?d>jeF9Z604QU@cu?;C2FmqxOCi=BDGTSGvjD51z3r9a7;qH_HZ-qm!=EyNmQ~G^ z6m!#W-P5vuB37xSU#y$grGKs}XVf|E;fxCw4?&t$?v6TUBXr{|R-h@3JejhGN(9yu z>vku_S57#ZRzq9VFw;&Gzom7fXTuk0KRS_6Lq{E07xP%9TU0&ZDERM?NtTNM#+mPGSn6>iOZAODOGo z3?w+g;Wpsn0K6#Q@EUo*Gi=zSA`6kb?VLb!DrolZ#IA~JA2d_?7T0OG^3u(%b~_Jw zrrFRYOu2a4RA1LJKCQ%CnLD*A+B>C=a+n}6Ut7uIqpU)CbTfCLGnJLooj)kv*DbJ& z)-B99;=CRAE47z6`0&g9K?3MBDEDt;MvX=x5Qv&quC6u;6L!J1+bzy`Ku5&%RFBfr z=#{xS=enC#eEnIH@~u*mXI^zUPbE0#obSnXQ4vok>v2a~*w~Ej-|nh--o_|iVSaFX zVsOOl0^XquzX4W1C{yLoURNp28dElLk$HZ{a}LB2?$4L)k{qemE4pFD(m$I-dvS_0 zxmBT(MyMCIK8wYpS5Z^nrn^woE6RC^RcOF7&7hP+`EBPzn^N&!BX;f*bIs28FO!y>uu zFen)EbMs%zAGIHol%HIb9czuCqpTMjDwd~p$d?U_q*TfYz?-y}&yF03OYE|+seV!x z#l;o>MCJJ5mN4+VQ#xEI{7WnZ7Q3B5k)2isO^ZD^N$tEDzRU$QMTux6y8L!mj|p;V zpeMXlYh|@tsRyj8zY3eguIG%i#V8>Mt=kh5`LbEdkdcKSqHz-RC^fAl1+qI1MGs=Z za>dI9)-cP0Foq0g-cyJLHm69dx{ZoQK!evK>z+b^Y~i%*uh|m!@zYmr`Rh}rUnSSy zaB`XsOFO^rC0%%BmMkylrliUM{aBZC-T59pyNPbOYC~h&_2<5+QsUfoALd77m{b%7 zPy)RPCP)=1ne`AgAZ=hbs;cc+7{nkDqRDUtz*R73C=i45RXI86@~X3*x!8CFmvUat zuXf92pIgrb0VN|TD*SUVUr}{v)uM`GHXq#(otq7R>THq_-^aKmF_u>fe5mk!o*g`> zFCg=1OMl z&_%H`8+@Z(s(_l$AQ{zzM|=S+69kq4D>C86TrT{{P>b6At%~LV%8Y;_S!pC!(+fG2%bG}6D~1bRseWt1`A^q#8yY~<-;mIaVc2- zXOoIW5opNA5&4!Pg*ruch9!~`2^M3c;T4xewqD%};m?i%PARr`nmN6fK<6EIOu{8K zW3O;VkwSX$iow07f}hwdU-KUV0Sw?ONT@NBQ4sI43z%^dnf8mvV2%;cc9hrGz#N%< z-(0g>6Ym{FV*;U=HaE4W-8W5$<8C`0hk24vhe2Svwyk@P?Lx*QyaZ&Oqmr6? z=;}O-hRLXI*~_fp&O$G=QRFy0A3QYES(^rA6{K-!0b96WCC_>s=Bc>)AT}GVAcy48 zwC>C4WS_orHGg3FJxw_Oz2NX(g7 ze&3uk8rm@Oj$qR(o2%MzVt*@R6=x`qCi+9S%7TSnD6xntYBg~Uz*8Xf2!XoCVPI*E z-WRxn&adcP%qdsQo6N*I0DSBjrsK@>DfE~jU&sWeS#jwp?Xr+-`n%jJ3msX~=6{mC9zDc%jqF19M4mp9a|GV!xwiGtCsVb|jQi>Hx z_Th3dxhP8I{so}y3HtQrNsuW9o9$6(Ynt8NO4u9`f4hRe+n9sRdNosA z#{(N`cS>e4%WDOKv|1&@i)YmbmLCdtDWWVV23StmE^!|mg>le8Ai;o7k-Ct<&}Rm>(i%AF_BV4w0%xALllu|3A7v1y5}kxaKL zWezfZ+WjDDDEvoux|KjOL4hC;+d!4b8afhHB?{00&yc#2d#LY1z?t-TqUAFNkU$th zj2Kr9K($XCb)+Z&a$_}yOZzIxF+q~2zBld#Yv5bkm z(~F-n_xs-=GLxGwelnN(sjZZg#SqS)UnRm|VAx+4AG+5%;55FN=NE}u{|9-%jzrrc z^dJMo4F|A5FpWxHULE+Zbzvv)^5nP1m$dVB!R?MSD`c z;b~bp8f_?DS+nDvRuCl?7WCy^xbn7WWrfx(vjbzE|6vm^F*i{9yAf0+s3hVs@XU_9 zfQbb6-eL4p*pR5G_1-FBy&XfQ_JOx({uviW0R*G3NUO?2DS!?TOpezt=@n6Qh0M4z zz4x}&<^V_U2V2g`uEyB&yfyy0;BoI zV=TFO3A8Il!@sfklo*orJf2Eu4eEcVq9C7;P(L#E?1oe1a~%|W@;@O;ojd5Fg#}$u z6~pZ|g{Px?bp`m^@zD_2T0l_CVcj#==Gp{2Q?F>0j2IH#R=L2z(D;2rsd^9%7^0e#8)^C_OY47W>pYcztGwRF8i}d|)^WkjLYI-Vo z9V@WLW?+rGZ3s|l0ZY?u0;IO_W)RWx3}a;JnbKO~b*|<`yt?h=OKpv3+5nP)a-rN) zC}lZ=o$PWzob{eVb{nvCv@CXYaQcPT9%1sj1>}rv|CmbCn1T z6Alvv@L{6p=ecY++RBYY5D?#B*1803VbsZ!U zC#W7rpGtg(6y~ce`=JJ?<=6_2B zf|?h|UD*2kwW$@@s${nFQ#ZOJj<@X-O%((I2y~at(oM_)lzTT^6!`$rX%jOcY+wbU zH$pWe^KA2@Q{@(o7RUXkTF^UWV`V|%@Dq{A0!o^xJRQjIoT_e%NmiDQ_wJdf*57=( zd%roj^P33i z{g~t$pwyxG4g27j=cJO{}anzvoUG%gtx@K9L+C)Q9TADG$uQp zar6R(Akg6`)cc9)aKV<8bDy=Rth3gy$(4l1B=f0RpRtMM33tC6ft;CRj5+1OAyyHC zQ9f)wR{GlvqOd@F9X&6#0FyqHkY)mlzRu1Vy*8m zhvoX#Qpj1szPy^4O{&dzHnrj|IUf@xpMK!hcMWfAqEC7V?A}24!2wCU55&@TqI9W_ z6{M+00Dgw?>Bz(Q0~`k=yB;nH9NH9TdaQ8rY+clv6zY<5ba}8Wvf#`4%m5{QxRO%P z;{m=U@hl_e;{G>%wm_ajcQ(>3c58A6bc%||wUrfAWXN&{=4oV!nO8!XKSkD0{S2&m zYWMQSn@GWDQ6eAh$>&GEn(mWShzuCD%2#_>dx{fbstOt=vqj0K;G=F)!U0=ThTC)I z9MoWbBzJ>!QVN;EWh0KUpb*hABv#Lf7(}I4eY$wp^JgDx$YC2{*K)LC6;MMTmIQar zG+RD>W0~nsTD#M^5%?sSx?Hrp*XxS>*nwbsCLx z9U?r904wYt30Jk3HsL;IyF`ElK{r5opzbq=s&+Jf+ik_1U0?9kCvV%!HcA(p@N1pZ z&nQH#W}bZ?dIrZv85wkx@GFlUi+wY_)G?P8Zd_=v(5WGXT$dIQgRmt^^$hJdDp~!8 z?Y4u)h2;nfQ}z5}{)h7iJC50i&EK`Wrm~-fk^yfRtVxCUZDUV_A0g%|GGWqrNM*)^ zDbJZ#5p^`Ut-EiycLU2A#*k{e1(^uow&m8LS4AAgSSUC&h7X@1jdU^a=Dwl0Qh!B! z7d>RF&EG+fJO3)G%guKb#)8|KxN2qSF8N8>;esDN@J-uumM`T=GAD^YI(N%x*2;0}x6FXVNx)u#o%&O6Gphg1|t*EOUg5sp1 zeNGV>>A+k)00g#J*2UbHdTIqVi;iD|WVX8v{+%`*c;X$Xe`y9ryY4ku#Q&VlKO1?I z=xw_{hgNE2XcXRzF$Zb#oS=!7)wFody+e%D=ex{rSyKatd^sj|B2tdqH*Job9Z(fl z;eM<|Wgn~cC9NHs^FrAcFQEnE7#6dbyA(w0Zoa(n^RUFKkwTD}NQhE(#fWoTfO!Lp zB3Qi$!3I1aXL0_ry*0{J{=~VD2jC~)6r}WU)VrQM7z-X7zaAFIL-GO@>{eR!GuJY& zAoyjUAyFo?NUFWpYx^xg(H<5SZ}svveT&dSx>Xg20JCYy|W?UDj9` z$Vz+yC+mHc$#U9WrN`&O*{r^2Rp0FNjM}v;#!AOJRf1n8Zyv0HUUtli5L$Wi%f+75034>gePK|IGtR%*tPvT&|OO)#qsCe;W$l zRpp79qu{inIMa0<05&!JUIL@dr{fw?EQJ<&xomHR?H$YM!(O*Fm}@rog%l`w)r*R$ z4CgGeXx9g*&sU-WYXb2-FNv2g1*~EWn|u$5({1go3gLUAQve3 znS$w^T4bB2eD$Op!th^XC@#moiM98S8pzYTFyhaNkz)?#e#{~G3Tf7y5A~QT<@*=8 zv(0@yP%x(T=FXz6qD`nml!V@b8Y;SedI1f#6YeUSU#ol?r`TkB)Sx$!G_?HRp35+fhx9zYzYsh^I1us891O6 zbmp+D>iw4B*0hy%{DB@5b{ti{S(2N7l=;a%7^?&AjU4?-RDhKu%KhiXFmqM?uH00n zPX>GBwA4u^_8BpF+m?wyOF7{4%^g0O3zxku5nAujZ`e;#-(OXJ9{XIOMdOB|YY)pz zMy3&SN~ICDZ}&-v9~3Z^1Ne-Eztc-}KCx{BN)Dj#Ko|2q)$z+^H4{Sm*trOFO&xYJ z-D7TO^$XgE4HTznBjteT=d@kAZ>IWS>$C5QfhPaS0T7e>b?K+`{ak5LY0!&TXaTU= zxfdUF+b!WgJ#)=<`r>=Ap%k2R<>-{wwa(H*2`2ym6yQ%zVea)Kg#U%t7N%bCoXx{G zR!R3E32s2z^yG{FsMIe0CMYuB;O984pL9jJ{A{H`H$K(OyXL;r{x(d^69+Z%-IkuZXtE9cf}rt{dBOY)Q<6L zxJPi4NssZd_prVs{gGO?kJ`J?L3hR6$BFFKp2n>WyGl6r*2kqf{60?mNU+%}PY|SC zk7d_hL9PSBB0$J6!lY>DM}CaZQEiH$maKiEugJ!YY{XN@<~e~w9i_iC7dCyBG{Pf<&8paOy=Q`$=*3^7^L1sMvbj2AG zxXYwovI7m0)U;0Jj2nYr*y~?o-R-NSP$gWLcKumj&mrmJeruJ=r3LhywDA^(!gm;% z%!Jfz=AR$6ogvpDS^`j7$YT!xvaOslyv~~h$Sdz`*w^#=oDA}|JBS`y7P2BBo%H}Z zP+xp}%vRM%y_6E*QmI~97vZ!@#y1A|Wp^J73j_svjx1ck2?|#agZ$a33dzSnwF$15 z0{{~IVFfiOU!2ubD`q#67v{$RI;_q-0@tu|mnf^4`m%Ud1x+odkI{87&jIf5qm}Hw zDI*6~YSyD{?4V@vTc|ccD?$t=UUeydE7vwb&KsfDl96wxM$48{3U6@uH=iQCX^erB z@D8^RBgNZzL2|9-nr5-ve*QZJtkAu{J4H0BRcsNg*N5bDhZ4?AZB>zitGf~)w;Nct zgtm%10}i1xYU>FOF^|8SGgBbGV|YcaWvIyjV8{8YgEXT}C>Rf?&DR4rfXIfLe*oDV zaxVia9shP*n7Zsv%UwFN0ACteE-d@du-(K!O!cS3XYHRl#aI~xl4s(jZsmK*3ST)9 zUIaxD&H$=)9tRHeHR@I8RR+}AVTdJ|{g6N&fBzr7&ds8#q> z=2TxFJ2i<krT91&1f$;T)lR8b1E1Vt!sQ>*u<#X1%Y$x!48j=zG(jqPTlgG{0>< zZp3>cnHX%R^Q89+fI_m6sY=_h_j8U+f{cR@hz)^pNU->FF|~P&##rzPVZ*qVQO{%3 zd5S6C=<=w%I7`5b!{jw%mFLY${q{KZ*2zgVuu{$)aRPm>2FiPl}gd$mks5+Shy<4lE76f~dy0nmd_M`C5*o-Wr#+$CY zAih}3`fw4FPpD#=;y8nLpG&~3J2n2#%zt+HoV8|{TqIKXWZ0JQEn(Lm=OBsVgQBOm z9Wb^h`tXtrF z(eC9ZH;l6k^Oeenjyt$<6>842tux-oDAnHd$0f$@g;#A>>lY3^iTG?gW!EHn!}^9a zib8(JPIAA8PK~G8D}J$P1s7Q09@E83>^92;%Gl<`*y^dAymy#i_F;eJb_;c zzaVzrm$cjnb8HcDy%GD4Cd6|p(pG9k&uMS?0)18m3?+klRo}`hAJD@sNt3wv6jbu{{ z+puQ^SNK4I1;?*baei*XL|<&1lV1m5!ZZ9S?w60t{nFa8iOLDw>rieL0x%mk%$pAg z2F1EV&eU>!wlZA`*O)<_GD%&sE~1+M0>9GbZ*#;RSozn9(M~?xhEa z6$%}zO-@|kNJ(a5+WKrvg;|Q5Gd~ale}AGjJl*I<`+PqK>2eXUb)dGKT36v+((GQ` zpb*!zmbye!m=zvvVT>R%q|7#gEo>~w%J^lm?hX*nn%8k_XK_5943Ez`2F3k8Ymr4;w=u0w5M*0vQwqez#-xaQ5Hz+5f2sprxUnfEN_(fAY1;V> z)}49z+m~JGcM64Ws;?`b;k?E91P3BB&N@82eNlD@&o+PeIuBPD6oi{|0h8KXHr8bs z=%`1|1FA7rq;L&A9oWKbVtsneQshGj1MzP@3bcv!_82nd-PA-h*})kG|LiNW%tG+j z9zl-b`~t)8t}0ySqvA`9+C}!K&pYZn#c?x>O$m&_P6Gg{^4lgwT{S(F+@uIrP2H9sFa{ERuqK1x6nZrpJOpN~n`QPgez8!&_`W5Elbp|Bfn% z!)H$FK-~zLKwW@pREjHm=Q3JK?Y{DvU38eQ>FI4+SMluQLwN3nIv*sze_kkJW9cOl zVRUT7FWFCC`Dhnwl*k8(8C~%ubq9v5AExP0hP=|o?vQT48=s?J2vQAnc*TiaB2& zdhoe4Greh2?umye`{Aa%A-{?r7^{+qIH+hD-v>K`b4tQ@vZ!Szk0wQZGN>PV7EJvq zLqg<7-fe!RhF<>ri?m-%G_phlvT=yqMcfI?hlt_F80jZXrCiiZ;AEfy@-SJ zsR%yel{65?W)atQF?$A%rWFS_33Fkq+o^Tl%$^G@eu`-&i}JnpVujaCjs(0abf3O< zXj)cwog*K&wdL&thN)`;~`@kii9`loG(w;Wj=vvduDV&3P z;&;>3anBj9k10^gNvDe3pkx<+HrCzqjXR+`xljo+U)ybIex^levGW@9scgM@cwH(3 zPMUhR_W`478cwn>Yp4W~yHPWLjVC5n%Qz!i)S$OShUPU&&*F%d!{r>i>X)LH?F|f| zif4YD*w+K~jm#o^wzr6is$cXYu>#`7xInBp7swf6Cw6=0wvm`IfZ$% zT|M!s{U9kYa)O~E>u>I>Q-A3#juAIzKurLDN7yvK@@dvNb0~adB22*riIU4zskCO# z912T$BCjFWSJG#kR|TSirPE{7UYFGJ`>oxE8<>K3gR*z$dmpH`lr(*qq2S&a6z}e- z7-r+u`J72;YWjLJxBGHijnuDb3aEay+gp7 zNk$`>9rDnvvYcqCwdjuQl4xn}^L9CGlxpW`pKL5GM=V`BYIQ>k=J>&H|3Y@EGN%paG=( znGKW__kx^w67Alb99uydFbB11bqnTOw0$Qc53ZZNf{@XNu=z2a!wyw9QCy_A4#DO= z22vcFGY=FU|1^9G_ddsbsn?&)K^(Mgq@&Ucm70n&=fo;4DY&TV`zF`7@5zf+sCQH# z0MflcDAo&wXZ2-BG#FF#r+aha85mUVo0dyo(eip;Wn9iW7e59!h#6)d{qgFpYuvNV znO(TN=u&@wW$$>FU!7GEvIa;G3F#zAE5umxPW?1mGaBfq6SB1}${2mPYw# zjJ&C@D1Co;OI3-~(GZg~Z8k)Rl080zZ79-`vxwUxj zO(*%SNr^si?((Bno00-a2b9Z?&lzdgNO%yssg?Q{l=Xjx#a5oxDG@&J8ETLxii*M; zZXJE&n%8~xC6~g-p(f{`>=YX@th>U;imijAcuNW%>ZJP(R3=v)4@18NhoA@*_kl1`kDGw5?tvgo}EpHzWY5saZY4_wM-vfGN zZX*S3u2+VDFbM{Wd-9{ddjVEA2}b$@3v#<= zm**%*tsaICOfxuSL~j{x%}%Sd+Iu5X$JXkZ5=Y!D2Xfx4NbX`MrA48_srxt8{c>P2 zs5yNr1vLqbPwq5eShEn6ok-$cVB?D8E2GBoxb$rgRJKByW4CZpNT0nYcH%CdJoQH40__F> z7@z0Lae_!ia-eH7OAuz$9Bz(dGQ8y011`)cCXo4!P1;nxPIa}sF_V$A8l5`&Nr<0! zDrFiCu3YU>xV27$Ey>hF97WMme_6r0sx}l+a9Hu449$FZ z-M;szQf=>eSNQFG?L}Kv97252A%^8u_Ttx)w>@hQ95yLL>!Q8=z0;~SL{(9*NVudC zx|j1D4?UVB0*f=8z$Tk=n?ku*7BLs&hd5DJGQ+&Vr8)k@crTE3yN_|uWr4TYvyjzw zvm<)&o)^7M09Ud5L933jt)^m@N4Gnsa|(y}(pGN|?H0K|y{Z30CXaOkQwMsEoR-JO z7Z^%U?~%i0Y^aw$9FlRlgQU6Ohds&Z-LbOmah7t8BXJFMC7HkBMXdMAez~nM+E9b` zY&j^sbx7DoX#j7TQI43kkon>IM#O@!6hWBUO*^XM;E8GlQNnfMvDjza*aYhaYYPLX za6(TH_D+pfoLin<^@Eo`7SJ_|?WV$R3fy*x(w^j0q_pPz-Vb(Ti}XC-zWMqccXTOF zLi?~kjo-HPrsM8H`Ka~bE#%mA3({xYuk3vD-aWj>!33J`orfTbTp=tgukPkt9A%qL z{!C8}ni<#C0=Z<7Q@GCc&Ieu)LSF#61nl|wE;_%{>;VCQe^XiSu5`zLTsKH+38gq4 ztJ3<^Ya~YbrC`gALhX3{276T3GjzmL^u%M^^v-RQOznX1G1MzB5lBBTx)O)P6e z&PwgHuRd2r3&d)C>^>t4Fe(a|0GnHxfhF(Bl-c4J%Lk9KNu1jv%u1In7BDKiZQR^Q z3br+Zd;4L=qlMjCj2NJuOPD;!PU*$`I@`G-Ppq80x;hQg-jvO zS}U>celtNv7M?@xFWe`D!=s#rcF_9Aq_pF9KjTN=2bTL;F5j&DNzN~gv(lu;oyst% z-+BZ3Wy4}|l3Rw=(q5@un%nGUv-m^Y@f|}=BNNh?uV*Eh2$$`@*>0@_5JvbO?=9o% z9;8+1IeDJF1$Lg}bl)O2i&*_;$hKnc9GIb5YvXJ7Nf`ApX*hAOLA4u=robiJHEdL) z=;z?&eP5c4s0kYD%Lv-lm=B=W`bEM^)~``;X^7vZNHb!`<#PZ(O!}V0KjONTaTb8> z;qbMdj7%E5I*PsnzCdV75;(^hO4Xnj{PE?gIotWEwawh?<$}xky8Iz#5-bHzBKU_0 zMBAD|O;&ALa2qr^)SBB^i>H(G(yzK*KIGt7ev%2a$nuLBTc9#G;&gV6&r2N8=;$z$ zT#GTc=;K#!c{au+axI2oYs8WDSMdK|%H&xyTvSqx*!eQT33m3ct!Hh$qon5Ijg(

    J=kWGfZpny34+ zWVf_Aha2eq2aD5u(`K^>n_k9sS3*$^)?NLqhl*-rWOex^&*5^bwG#{T7Ug=pu>diB z&4)ufE!=c#MtuxR{vf1YLmuc^AI4~MTEI@p7!V1Y!=PNRimd&{U^Q6Y$Nywk^UnEDa8tYz9rJUN1t{7Q--O+CX zwp;H)d``dh@M5ppHBj{$v+OZmJTR0*VP?4sG7dGx?i~TimS}9N7`1-iIJc;rN>gSo z@0BO9URfOd)?N6OjEp>GTe1BbFVuXh5qxc+Zm8>G5pg-^6`~C|vKD!nOEaU^y;#a) zQKawm@^M>Gc~O_$K$WfJ`ROyH;cPEb)UPK=(Rslg0xF8!D>)UX-p$K$^|S9)OOuBy6Ki!qm6YN?rmJ(9#xHrx(n?$T&Env)9co zIv*+85^ba=OkX1}JC%1uN&jdNi6n)knQ+=^78=Ft7s`~`c~J}0GJU5w z@2T0-)EW20hqqf=1KafWMznoZb|VQ_LlxWJm%gruD%?r2YPCtaksc>*5Xg`O+ys0N zKQ_;XfL4_Rn1Xn>bd1|O7**$a1b*80_G$cwu(G{W%yZk1phC?F`6VZbPiZOP!PJ~XgWP5O2Td26Eg7WGYE1f>-HB?0CF}rVcxasWJEhOGjdAo~ zD)w{QXZW^KP-JlIma(@hGGfeij{UhUKjc?noli_V4^hAGweZAY-EouHa5TbQ)cfic z;fak!^iC>`J;O6u{8#pk0&d0z+f!_OfK7?3LZPozcl?uX~y#(M9e8qfIt8RXJ5Y>zwRM zMw;b<4fA~+H3%>Az3xL>^x>KxagUlD#7H}5ExRoM9_m*%ChzRmJ&CWG9p3s>mA+#& zRAE&Et}3$yi>3v!&epkQ!2E2P`sdUM8xBRGQ`4KCjNPLd-4gIPs zLrHKY<)GM#x3jeyb2C8{3>%EtP{&}7IJ2jvtE7R$RiLWtiTp??f$=z0ZVH@lw0exWV4yk`~HKnj1ye{<8y>T6L;0&esqn zlv8?~{PwFsuF3YzqGCrIlitsDBi*qQY`wIvZQpP!)Y%!Uh7MP?o|ToaGFnTsFC3dq zG^i@@ldd*;)4p|DjJ2-xxlNF8Zb)UR!0z#(4y5N(RtXoS%XZ(mFxZy)X;Htm4aY@> zMX<0jA8$gUE;9T<_18BU9!1epRqkiE>&C?n=ti5Njw|lQbYx`1f9dBWAZi%6zC*h zZ}w=rJVFJlghx-~wqu)JR55DxP{7p9;E;zg%pYL(6cx)dS0$U5NzIapOt1-5jIJXPH>!BL zl)y5Bs)(($`6^AsjJZJHc>WdR?JvdiFF;S?i^p1}vu`qXz3mCEsB6M2aL~{ zy(iA_elYOg=+?vNVJ4&_tfYbCDoG=@rFl6&O6L?(A!c6eek)YuXD+Ir5xi0E)Yok; zPv(Ebb+!hbG@Izwb3?D1-M=oo-;&4=54ZtLwx zXN?|zGqiB{!q2qib|qKt8_E$E4H{f+Xm6d3DJ|!kPNwc1&%qGYVw2j^E~Ftyhi zw96vrFg5eIi*2l&VXDXo@m7 zf`!gS_Sur3;k{AxV$k%Qe)*W^rGr+v*;jH5^Fz$3SIs5*ChG&P?)Z%1OVZ=|8Q)R2 z&jw@Au^pgV3W7*KTw)y}5BrSydcQHKAK!l6En&o8G}`2hEe^O<3p$8~nnI_p=RnKY ziM?yxUPv#|>Fdvy=aTXZWetW~9h$C=IlendXD_0j494lQt=6NxFJRpl(A@5BC}F^7 zlRJCg~ZbFAcdX304d2fu?>6{++eZnuh?0=hmjIm3h@AYjekMg4@ zMIKsmciiBMiLHu?bsH!VW~h-5@)+s7YV`IdXuAqgB70mv6g7foq1T;iC^8rSX$qIr zaxmwXRw(DJEX&0?I}A6>+H7)nz0c0kee=0ocKu#y+V%&L2TL+-t_aYlB8K2Edt8;! z5Ru=KpP&yU-<1@MR9W+3-KnS=u?bEMzD8uT*lw6)UVv)s5~mk;~J##1ggT(gW-johdsBxPNT7@Yg5Je zRg@bNfuC+VH)!PDAZ4~DtaUCb`~qggHD+PN)w#NS)=pNZDji`jA)zVwjkI#K&I!5Ih^S57%^d3^w zACcn_Em4X<*&O2os0el7YTEqiH&?W#0oZG8kS`__8oimL|3;;5w3Mvi9FaeBpzN+X z`xa#b*#kRZ53E=mcSD;O4l+l`oL5DA%KeOxvDT^DOT}6k^NPSq!u`$9y*5oHNnKD` zzG-kNlQ3w!UR|JJ&>1^fFS})QcF-=1ks;&A{b~mcR=1#lq$hk#=( zB&gYYEJICkQ#K`8ofmX7yae@1@2q#~mAUWKTJPVp#BC^Cdws{^Y{TvI&uUgTG`GkX z6$yhfhwiEwpF6dD>}vGP0be4Nbqb8?v{n2qImBKfF4qQB!2_B`jC9|Hcy9{NHKySy zt80M~>vCYSkRTj&QNAFoB?$T6fllO%zufVIE)6zSprAewf~)KNj8DB+(1Z{@hl=`E znWUb7I~7-_)jrh3r}Nn@BW8(;;YiCCh8Z;AYHwnW$Zukz#Te(H6)-}o01(jo>9i9Yv${+ zdg(bVYn9$833|CN*OX9N{J;-FVJWjgE^>i7OO2ffHbpm%QB%oyy=NK@HkWa-I0(i% z2|GeZ&L})3$zGn$rwf>(vV`4$Km&xrB6^6O3jPt@#PXt0X$#|$>M9J=69?T}^{KAP z!zKyA^i)|;s`pOyvCkDXKmysDbI|TBA!yp~g?~!gxdUB(*?N)Z+8M!!($0esbW64@ zQVku__W+!mB;X2AxK1ka2|=MtB+5Fj6^- z85`;+tzHhR8Hy&3`m8cYYYvavR$VWii-YoHNda>%Er`B_+R~9aSh~jw5o-H*&qk3pO9LXl-FzW~OQ3aP z3UsO2%C~G{;91d2bhY4r_vC-wI&cMqqx1Mw+{(LtT%8YxU|c&swK!lF1{o1Ea3)@s zhqd#@83J0eVncO$c)stS&Owvnk8dXUvh)G0dNcY9^#5SAWy{YmpJ$z2`_L*QLxX!IY@Z- zWcfiek^|Vv$E64Q^;Bxy2^Z?h5EiGPiY#q*YZ-t0>{)b^b8)#Fp=+qA+yyC=91B4K zMNKkYFK!o7k2Rsj&jG$m8O*Ujd4vi9P|l55Q<8;)Oh=oJz|%)$^c*^#Ru$ z@iw;Ra{=%L^hGjSrI_YVBwUZr>72iBN_|oN8hqRGhMxX~VZG>)qw+DNwTWRVU58O) z89n@pW0^y7aXOdpJbN7In^MgxUdCm=#3icxtWz;2oN^WltL8v+rYcewbqnG_7i7p! z`|d&!e6}L2ef}=(J4Ff5c7KF`Y~eG)-LIKnlB_+6{*^8VqiMlXgn$o}Aou`L=VR*O zCcb!`EsRP|JF286F^nErA*&_kK`7F449q;T13d!2U?C{)ywft3QR#Ha<^-bF3>C}F z6*?Y20BfAIIuG?&tR0qHJsj)b61^uSe4ynTBanDdp8$GEMH$@z1Dji2$s+eEO0^1A zK=Geu^zFS~BrjP6fc^U{!yCi>I0}s%zRo&)Ejfo14}+^{g_^7X9<9v=gFKN*hQ!rC zVDU%CHj zuWO6mQ#~)Qjmk*Pf4yaf%Izv$IuT~fSAzNe#FW74T_^k1Yn4I#`v-v=941sB7Cm`d z_LrM9a{=SYRFw@iGpBh|zDno7!witj(ER!?UpsBo6t$cQC zS8!l1q(WcdqWXhTFEJ>b0j3THj8fp2DRAQ+5Z0R~fl>PTsprD4Urm4(F*cv<;IY}# zT{M_%(3|EhKLI)$+^P(j4uBR|egV+nl53{SfpH)d_1i0V{{#jI&T^j_=n zw@cZN^Cz4y5EnprR>F@Vg=fvK?XT@mvaQ& zh5{*N`aG5h{sNcsKTK}R58T(c=2yl|_pQ?S*hG$zN;+QZaAnYs5g8Gt;&D@N} zT;7`_!_q51Roy>1Yf6~yQ8_APAN;4Y5FYu_NYBgMSDnyy{>L9N3PjZyy*UB_5d+xd4^SmB_DLS=4?pg zK0BUaProWh9)5%!4+w`_2N*sJ-KK|;w4bGa z0@iNAK>v|dT-JaD)?E((o<}c!ZKC>+8cLL)N1z{*u}HL1UNr-+ds&}4Q$(2VtB;F9 z!JgWMF8K#swAEneLuO7sk;HTmiF-_X=*fL3pPvNnLki5cUo0sJvwLSAC z1Z;|RX+Q}E{lwq({(w&{1&%jy0i9#z*P{ctO6Vgzdjt^gE9Vpl%%HOs@!Y)k zD%i4dHI~>HKzUsqsGl|GRd3o;nkJ(p*$P>_AhXk7ta6b^4bVY?z#0 zvK-7F5Z96A;y2>7x6o7YP+Zo;<`WpLC}7%#B9QEjut1qWlI#rm8o&d%D?x)Sd|;?a zSRFN1_ka*CPTY<{s|e|0IJ4ccYrshey_hvVbU>*5~mWDXhima671ZZI7Jlgx4@zl z$_A3d3LWUz=|I#b@Ogt)E<-E-t6s$J7N!g!1Qrta3E_IgxQ7-2o@zUSLpmW05Ut!X z?Ax8ga?ienbQVd2`g5v)&OR@~fik*$42**b6+#tWup0oZ05~;NkPv5OA@SMt5^A7h zU-c=ktAHodYPs@&sK!w9nJB03h5b=^UWi^#g8nDHn5pvz=QG)i*-^uubsCt$viguKpxk z#%iq4xW>MH1$Ko4j9buO66+^M0ss8;-`}qRMdS7H1=*8mo&v-mqIZKn!Scb|a=V&( zW95Hs>;Z50>xDqHNFygM->>*%dD1^Ggx(_IDxeDQGLKX&N%rCIV*g{ksNALx_O!Um zj~hA@dTnXvlP}vT@Z9@no_P4puX*C=5~Du0Kc6Y24hVrG`;~A1VHX$iY|l9&+fBW~ zWxy_*k3*~y;LckmqO1S+yx=SUBma2R`y;D0#qD!>sRuRQlNSv;#{LkjHuM`h>9irD z!&4CNLKOPnB$L-q}eJ2TLTCCX*fi$8zar6OeQ0 zS!%&bc78n9-Jy4Rae?R#e+jW1wA|`h|HEEl^ZZ}JxBrh!B%qF+wd5&V(ny}0;2P6E zg&Rs$$zeGUjEH-Ea^wHy66FVVAi8hEPAx*+M}NKM*SEP)kLr>7!YV$i=aZ35 zQR5d4-dmpsF}2R`8Jxk?!E?@stIp|X0Svb`z0Qa$NLgDY2qa&2I6b@mfxV-FcjcPe zAhuWz=*+{e}>Y(G& z$B}PE2;{CT!Z~!Wi%{cgZu1w1&A&c_0Ot0L?DYV6BuTd1shYnw0F#3o zHME*D43GV1*`p0915f@vqPu`Ja&ysF$p~vx9O+F+W!!6FzhiwJ41D446XG4PhdjsD zH186|$=OPwikU;>Zmg-PYEMF6tm$E~t=!^OQi4Rw?vN%z#cykCEVAHTo+u@koE+vJ zd)-RR>RQvX>fs+_Vr{%4Z*=?^$1m$&+r4xTi~1Aeu&2!W=%S*U1PzImW&; z*U-r2jq#<)%YMjK(=(a9i!v>3eG;cGCE_lT-8V1RLum^~qxc zVeCU3nA)k-Ynqa4j|E={Xi92Vv0^J61-roB?(u8mFL5-*BP~npUhC<5?dk2sO`C@7 zI;tNzn8$s8^t`mUNON7tFSA0;n7V`;xg6Y--ius7m^y4V?U$)9rZU}0WcWk)VYGJv z5x`!LilhI92$Y7fnD%wUwdO{*ao30Mi~{t?VGWyue#(t|)zzD~T~-GQSM%p(ulLZM zpbZc|3~tG3Sv6p6xVyAHBkE1CmD%!iY7-^cX%rr*)~9~Hx0_m5Bl@^Q!;bPfeUAwj zAcr>s6S^lT@gRAxO9xNaAQRApTaBL*X~Mr2xxnmFG5MKq+nYg{A|Uc$qYHM?seeOq zV5Frl&T4gfw*LCkh_9j!;de?IsTlOrT;Sno_?zp9n(5V!qR2PJHBJNlPQHH`K@)y( z=8ykzSwGQS!p4B=J(l%TDCOMcd*w8{ga9{8tvO4_W6^@X`6H|CvQ)2+6oH8I7UkZ6 zJY#!ToebLfG2ABy(bF)u(X@Jg=gY>R=F4^vveUk3&hSPRR^x(FO2Zhx-hI$D;kK%b zd=Zq!&zXGL{srHK`Ws_z_|1&qzD{ZIyQBMW&LuaSr zyJaNz!f%3H_vp#DvyIp(MB_qAS~{i?YXF(4Xr)T1`Z}Aw9wY5LnS;B2OeFPLO|z`; zB+F`f)2@wl!}g{xfxC^ZiF-ymxp5~5AMW~YbIr6ZR!0NjSOi=}z~TCB9ltw}DAx_D zUW=@D+l&qsjF$FZTHM6$XP_!GXg;!J{)_cJ-FHRX!H2tn-`J+J8Fpv(W64=KKuZ2~ zl+ln~j8=?q{BMD*FX?aF4t{a7_|XvCPcXNU zPKp!!WCS`YJ6hv6{3_A2IcE@Ge_M`Gx3r!2jGtwfcIQ7;CsUk>R0zO(qp)cA)cO?x z#@wjPJR|};E!b}3hr2V1<_tCXnMY1Schcj>_b1a#vd&(b^UyDOGFZ~=GgC}JyCOp;53s+Tp_%$7TVFNoQWpHdk>1K# zpZr4os!GlG7{u7(Qce>h-_U2L-)auu*COW9aeH z(2-7$>ixc>3J@IitE*q#6Lp~(*%Z#}h$k$CZ)iUMPS>>DCv3Jg>h5abhdZhBX}BAG zx+Sg0lssDSA<7sNy*7s5ygicqk*N!hJ%wmvR*ZN+N2z0?4l@fZS34HNO~N}OE?WEz zbkxU7ZT#V}{F}i&vXKew|DNNoujHPAAm|99BLJN4N4^+!FEnTR&P`Zui(#f) z<|~=uCZIIq1<`3eN0mmB_cW-&ubC{7TGfh%1>vYh+CF{<)1))!n|;+lFKvy+yPoqP zX@Faw|8x0lz(sIYX0p(0e}sB}1?O)e4&x~sC;A2B>XwYCCEjjS6@O+L?oN5dQO_z# z$8wfKh%o@(z}nGZb>Y@?6}ki^nEF<(qyO88u&mZn%1#}B>?wdAqy~KN*C5gI82Iv6 zPKpQU@2W#<)L#tvSHi$x6T}M2!*WN^Da>>sfpSSk;nS_SbDd(}z+@am@x7;FijM5W zMGgtYBwH6rBWEPZTHd4rTUU@z?Be~cQ+wbjQ|M=$|Lv$>iASXc`A?&`$Qjy$#8ZW0 zZzeca95{f0=(=;lF?>SiayUcV86o2U_*mQT}OdV9$ZoN`UWT!UoK*3rMry1BFmJ%BwJS!M4>2KU(2n z8yIc4ToMR)LY#h6K#m#nKu~s;8@OA^+U07WB^#$*D~m37*emIW0rPH#+`AnD@y2eC z4Sl;}0|r|T{zZ%cDnJW`SBxWG~&yXpt(Iwp2=Rf{Y%p(+SLJ*Pv%m45n zXh78NgQ1;cq?J$3?5E2&US|?l)BVH8=4#+k8uN);eg0JT-+ut6U!lZ;;}-;elU51U zQCk0&{V|gw3+?jR$XxFjS-RPCOAme-x^W_dElfNh2B(RL{OOnd5x~@fAv>-=&E7BO zLwZi@4SxTg{pv5&DH9f%8~?!5QTB*%_+{9*Ca@(uh%?%etQlXkFW}Bjck{9} zFlJIV)_)qa0|X#UJJl0^u2}y=qvVL_*kjp8zv`mbul&*^dUIAgP^pI~iNBtb_5WMT z^~*>%W*Mv$R{P!o3H{7<>hIHEpMDiP98g0P<|TFK#QV3OHD4#X39=8qw$N;}Nh9e< z8_;~r(WAv~fD`4pJb%Pnw96nuiT*V>`6oSpTdn)x_rS|5{er&f$@|uw_yv~~?|tm0 zAm2L-?7@`g2k~dvuaKu}_Br@Ag3cd~W;$iG2smN!_YE}T#{MK+1Z~zo(2{@M0r={J z5u~C|K)U>YR?#Xb?R|OeN$oOHV)HiW7G3i>6!?vd@t@}7d3$Nk(3FQ0an1AJ1OKu? z{|EDQDN#)`lxD%Et>Dm#4OFvY5!QbW_H0{uiN`T+1tnZGBnx;*-sj@LtoY&|3iNrf zd+&$~MoGG(dmlGVI~&KKG$m~s$*wF0P#IQFp6@lbuh}c(oBC-}bhQ5Tg}*P``afE+ z&U0?p5eIL2h+n3ZFA9(oBR-M$Eivb3`m~S}0ZzL9A5n*e?R(DI_pgEH{EhMdX7K;J zp@IZ}3~V?h);|7U1ppi$6n`6?FBE(Oo?SJ`>rbfyOVt4O)jMhUrwIOGreNm8w9@4N zs>R@FrK%Sil{Ob@s;Lmaz{^YMfQ-%Ne+pjSqrVGY%U*9`f0-&FDEvnKaJ=x>Pyclu zw88;k46E-|YeDZt2f+V}>~cvF&5wTZ|7;3ZN^mnPk^fKP|Kb1Jh^JrJJG$LzUrE#N z=LJu+PJ%09{}lB`C^S-U4fsnS|NR5GJz{8d^#9Ayh)0BEuAi~~8h&R?dNnX6{j8+J zw#jzMfr0HV1h#5#_c5lTczcL#u$@ZN*Uw)^b0#MmjxT3^3Bp5`9LB8}xTw3&Pt>ZS zC?Q8$zG+Ld8fpES=Vz^^?7VHAtb;GkX6ilK>u%;Q!)*&lVcPgrwn`nw*6ix$+N5@r zS1ZoSTBX=uU;6q4v3$*Q>X2;3@W%UF+t}Fj11f#n*f|VW+e1jOqCHQvIRk)o@>Q?- zA3_ujnW@#bgwubf0{;?}gcQPX-lrdh1WU#~?A}4x990gXepRcqYUN;_6k;^qu|aU@ z*w4XwqnR-0$TnZXP6n%v&VfrI9Y_l3Zz}Vd-A-+-p3~qCUU#pL5=K=ACz# zaX{GJ?|t1@e69xg)UWf^j#W(LUNCz)P9pFb+oN40qHL>qrQ6ou4@ZuIX$AfVPmn{_ zTMaq$8Jm~1X+H|0%XBl+I@KCfoo%|=Efnyu9Ha% z7K{X5;G^REckeY2QcQ}7n;Kh)en~F)KM#uUpiK|h=5q8${x)HpnZ<=txlD9}U(SK| zv0grTU4etj#R`Mz=KWzx2k&*WvFfM02wqny&aDijS12ok#kKR^VWH!Ti`!n#`vn!` zg1MFHx!OrH6JJ=$KGGK!3C+6lqbN5e4&GD*(~7T$du%e1T9nSoTSmZ^V;G3wAkKCVd-6+^_(#Ok|?yWII#7)5cXrYX2j2i!N8*2 z-c4?3&BzF>aBjP4r1^EyrJ@4~`j8A}QI(k07Lz8+y(V_4{W6(@{mu4G;l5;oH(+AG z9O^mU99_yX64InNUwrGl>=@CGx;ipw7N7QaL2wz_!ky0k@zqRY3?cP>4U zZrTx>iDK$?rm9g458xI%jCTKJ&mS30Y7C}#9;7h= z`FF1Hiq$>(oJbpVm|~rBb3JA+{y|0C#%@uUVlbKZihm3VLj?6q^kPNZ z38kJm@1fWo+pp9&9j)A!VoOf^-&kR=W2MP+y_xZ(!Ix8onU6-C&WPRLM;@x&$3Ucu6+I#8mSF=^| zZRkOh#NawVw4gL0<{w zRn{hvk%NkG{kPPTm1nMz5?0kh-cLsyu-}D5w6f8(3gODfr+p-R|JD%y^m>E5MSP$8 zQ`!66H){t7ZdL6`KYKL#@`T=sglwpss!n}FlRFJ3w|mPKO5@3c?H)CGJH>m*q_nh> zV)1qO^6hF>v9Qh6wXWz%blYlRw!ZULV3)b>Y;V44^Q(3T;wne&$nkXUh5zs}wEh>nz>Nhd!!Iu-x zxJ#o{Y&;8jW{)0C zNpaHH(W7(dFtU(FKC7U%y`krN#qV6J0>YZp zXaySIZEvftBF@=l-zn2C9duoMh#m@s`s;_8B@bDMuC5xr@;h*L-Al`v>tHtAj44aa z;k(JVBch@uIESih_dM7SJf)SRLm5yknY`a@m)S1TGg6_e8@`HANi^9U8;{VGfG&2{ z7F0B9H-C1o#19ZNUAy#1glEdbLak!7&XU%xY^!b2L725s5yy0`u#UF;fN#ZFJ)5P_ z@3DeQ&vFGyH>E*7cT&i|Vq<2(P1j*Drp64~ z%OC-nI>7?;do8#b?l{U+?02Vl;+F<%`jK9&Z%;GUef;HSWRNMNcI*b~RvY zef*|oV_6mS95&@Pug-lqxM>knR>^X4VA7FaeJj|1T{u0zaXx?41WIFtzB*<{EA_T6 z6CqaOK&gE^-7HvJMpE$(Y=6RkU1Xr&u-px%9XU75o-d{|Ubcz$Y40Au`sYork)=ghY%KI8 zlw|7NRazoKlvz$wg@c-PQBUi(QsJV2AWc&2NR z&#ds;z2;8&^uiG*-KO!67DK*lYo8ca;adwO<3V>Xy2$5b37}ENou?jXnRaJH4m0cU z_dXCw7n~B_w{YP>H5F71Yj|x)Khy1nXiYlr1{l2(JK9lNaQRBiL=_{{-;d3mVsfy% z>xxB;m~M-Ff4D)+VtykD%$R^CWBzmNgz-)ko(#97UdT|)cd#)89lIN_Fk3C#WLqO1O~M!P!UPzES}UHp`4pkI_Y$|o+LzpYI9uy zS6-00FqYXAYMq<4FZ#jm4yya>L*uyn2Vih4m(!f=#gF=e0U0(*$fGXWgUgn3!bcg( zN+;LWVfAF8UoxK++uzx(yBaS1tvKDcZ?@uFQ+DU@aE!1gi)Ns*d}xMqj*51_zbSeB ztI&zfm0W)#v&ePc`&7&jq4j(9Zp+byDc6@RP-+DNCdawMIUkoyB=N^z`(hMezaNz( zfsEU~7p5~BHvDnK93Q0yDOVsnB}rVL{Y8$!U3Ko_%^hqNA_@4Gn+*GZ{!uCbc^@GH z3;0~K#?DH1%7-RY*ttCBEwONoTJy7#l9X}VX!4TJ`6^A8^i}5V+<#k0c$TiWH!X zbwb}8>!vKT?OKov5FPKgL_lq9R#sLSp(_Co$I9DQ*_mciCFz)L4KwA1>DNN|c_|(h z3aHnQm_lmdh4$F_*&of199HZDeWjC z(UOB4X`PIr5cuVpMsGReL1Q`72X&NEzJWtd8x10hTwp$j{KYqzQ5t>7w zyys3NEwe9OVsb3a@D=y4_%Ni!JN% z`fc_6?PXOIlR^X}m`RbPv$vB2+v3T501GS={YCe_mrDL&2|aESpZhYvd0GDD9K1@H zt`Mhlb5WbatoNX4$+~|lo#72ap|tjk+ALsa>p20=$J7bsLc?FaexS9~-R`?#YG63z zKCkMXPd904Re1K{CNBDXvZ6=2-8G8@>bd675ia@} z!Y(x~vjJQ3CHG*(+dF7YH&-sZ;*f$xaQW8#RAzK;DWoa0@YYQWn{#dm26_Z;V$0{S zGnK_vJz`M z;K_u)<4VaNSe$ez)O(zXQUNGN;{M%qgI-DBq1}#6`=Ml5V4GNI1E|VVeC!{@bf)30D@>)U~py71=^6W`7}<-rRC&l$bs0Ex)g6p@O=b z1?7b%fx5{ZFIVx-C-w#HYWC1Baq*48nb$)2(BFnmY*W`QsW%tQS+LLtZB<65} z1e8Xa7b2}R`%Mf}~gDoWS{Rvcjy?J~sq2#K@+wO+6(pqD$C-pn2CfJeHqb z)-0Gl%o$2|H^oc<7CmoV_w+E>h2Z^Y81Yz!pw^H zH>ah|g~r>8^wkIlO-c>5$Peo@H=MoAEx3r9?8>OnLS6MpH~O-7)}`cFSyZ|qe*b-2hVPRJhS#UV&UvDj?m~X?}7X z51-1+&xJi_r4S`D#^#+$zz%hMDfbmI9c1s<(uJIP8#J}Bo_@C>me9o zv+!he1g}J%MGnGCI{eIO3v1L^gYtv~Mrb!P3`!?zr&MePETXF4K2We}qG7W>e)7>k zOVoS z6DTQ4V_w)>%^UD@j~tLyEK0wNafGklD)KLKsUyB9I!e)&A0r$xA9A~^&HR;}g7I-R z;|M49`oX8Sj0ro;0z2VwqXKc`OoMQ!gzCY|bi@!NqSt~)Tz6*T>}Q7v8&`1yn09?U zAp}OGJR7w=GJ+M`y91fN+CAXEG&i7*maZDBa#sErV`S56w656 zRswzr(Fr!~{|aJ6VRkl42#(T7S|9Hi=XEz6I)^d}uIOUYL`3eE9SPiRIVUKBs#7bJ zpVSdVo9#q3wQkVvXGFU09f6g4iIzsSeLefoIuhSiU>!~>ji$CtaS8^cZt~l)KyV>! zcpGNST@|i$!1PIpdc;=~x$o_Ud@~v=k1d)tHi1TJUSXK-H?(Un5Kvz%vFa5?_tXjc zWfrejq3ZU4xp3`OCAFPSb+kNyc~3X0R5-7_OTQ)}F%|=}#F!2Rv`+o*k>fp}W^!hLIKlRyhjCv}R;{e|s?LNUKc*kkXZ)?RYXyq0@p9F4_| zgY8UiDOMeR)s_~+<^vnJ%&~?}ctHb9z_*HhVAa?Pc~r6KhBM5r zhAAPyB4upL)x%h{vd+b+3fKbuqe`Xa0YW`fc65Ai=6bm$j0`pFTaal;p=h*F`$Eyc zf_9Mb>(2Sa8xpZj(c+OB(Yyl&F0q6FnH326o{U_#NeWvR%snukR#41@FTeIStd!j= zJ&})cNjEqWsKqS%a!?W{^(H}(1y)!`=^-HFZF9+)zbrie9_k4Mh33j@s~0nI84Kkj z(Ik)=mT+3;1JiMI&clwR1iqNfot_&wzYx;>L;DXT*W7i1IYt{Yd$bz z1Ydm7Tn{1~@cR75cGer=2osxOvYnl(mCp9_O{Pt={e0V+NLr|!2=}LSmg)XPk5pap zoWTc9dr{W+@8B|~JTW((%J6tu>)kNa*XY)tSb>UQnJ%#69l>SH zb>7RE(bUlxyZWSLKS#RUc746BomX%*lJ?e6^^HCO8S(8w4NO#{b;U9rUd?}Hy!&&@ zQb#+z+JzeLs7YalJ&P3wVuPalj295*Uswd&b@T>j3y8|q@MLxl0>TO z{!8o6I6+9eO9Qgf04?i?e<^t|i20!U9ij0U^)q(scRuvRxmHV{_)zHQsxoV+gOMZRSNRzQJr*~E)XuWMXWE{ z5Gi|6uN0J*4{aOy7meU3bH--xruXUK%V_F24=+=+UYQIOp2)G^A^L!ZWz3d=;6?#- zh~pU9`8#)AG7pM6(;iq>*beHBtfC`8bgv{1$Dbmm|YCe))EOG9$r`srqBld_Z`2BuLL(ziN3FR^$@2hKT7R~kwC?#MLErV!V+#aoS;j#c4 zKKF%WP zr2XT>*sr1512x3TaJ1GVXxf0rTPWS?3XF5$Dw>FFhuX4C@FnWYnJ|}I0^Q{?@ftfV zW;`pquXa1F++H8&&=J;M3xKtTi#<5Y(c((;Fw)F22&=4%u6`FhqLfi#;$(n-cD6%Mp=th3`L?cbX265+2QS@k$ zNlJcgad_9lb0So--H_qV=tVL+;i_p6NNmh`qnO;3Kk&kozX(KVtdSK8*>nm+k4%T6 z(gbH%SsZo;f+T5o*I`_=^7*l5IHAh#+;N?G?cLe` zkWMf~$Z%-wgAqsh>W0FYW!V&PC}~{mO9&xFL`QdE6lTG87X4)>%F^YpJ%xrxcG?+i zK42bx7^RnpYI5Vy90XG*>tl4;kGl?9Sx0D7eTEAMdF}Bj_2fzi@a?wbH(G#QE6`3m zS#2IJ=;FZQ!t~m9hrliXOA|^iWs6I%WAkUSd(3W$tr43+BLwtI??gY*)_9O%ma$)H z(N$)r;!0%}>_0?yfP_ZZjTF~KgM7k7Sb%NIQPjQnORozUK%lcM-5fZNCT@1-Y~#`B z;YS4>!fw_?0fxsRMLqAJ*#yG#!v-0s0F}5Vn#x|D`Cb*?tp)oFQC(C7r1VCa27SOE z-D^18Z}r&3xIw~qxzGj)$HePgOJHdo6oo<|)Ev152N*XtR(Cyzz_eA^;^QnTT6_}@ z8*WB*%*QZh0wXnOe9Lq<3P;)0_hK0q4vy;vv8*@Sgk2I?XbcNLOICvRGS=;xlZ7}z zb~a^?XS|enRdZ1Z6%8OrPEFfur?M`xxaR=$P4Tu2g0k_~D8U0Pv?^wExK})F+Cs@T z=3W=>FFLKvdihDI_mW2RlI~C1>`6eu5TnwS&Q};HExY%n-<9rFOJMav!2ZK-z(|VR zjpp8V&(epsBr5jrjh!l7F7&$9jT*gsF$rlUraYG1?xz2VaBPjOkUX$OV78H2SJdSB zvc$-CZiFSzzEgp?F7NzC9w&{yucBa=F}!WREkxnkQ?2{Hm_R8AbRmmO%VUNn?|mi@ z^hHxv_ZtqCBFwtpgy}pim0HWe1hDGUUV4$(UbfeWjB!Js88hj{F%HaUU0px5-k*q@ z+sv#WTSl*j#CNgeK{9ERt%=o`I$K8ZE4t<{w}ylj37ER8A_=r3+qQPP3|qH#VLFj( zE=F4;v6H6ICK8_M8~tJC$e+ReMkAryPGFyGFnu0NZ*~i^b){q~>1l_s;CZJ7&pf3( zO-Dw}lT4a8vfAkq?Ii0)fXA-0^&%59^&MOm&yLwdz)ZicZqRcm?D#$`^jBBB*12hv z_EJ5xyc{LID`QdiZ99*lCoYBhNw> z$@!Hrdr66LrYT;41AE%yCs@$%3+8L~*xDRuk3>EkTm{Iu%?zi}ytz77l(AwUr`3LG zZ9HMecrbK5j~*p;&~m+{OSXkP5W@{AcQ96^RKnl@W&YUT`; z9igIZ>JNeL+Fkw(d0j3jbTp9X*`1xQgNep5!9^}?8fDQ!3t6i~4jS||8(xQSD5FWjsp~ISXUn1sl-zQme!_IM{TVIqT z4&g#suzojWyd!1tZ%hM5Ze8`jcN$1aVznCLeB?bwCCO7|dvP<;ZIErTbTh_zGSSs~ zzvv=Jv`|t-HrSDG9;2UsJqEWeBz2o+GAk3ev98D)@v)#I$tBq<;R3ZkM+nU{*IdP1 zWE72Q#1qH5i3;UVohrB11p6Y}QLIw<3bQ7=mbqm$Ubf;!EV-@lmA1%({9EEe&T9Us zj+!Uj3ir43I96TF4hH*8_C>pz<*567%X-ZmVj~d1=ceB^3Rql5uGl8b+i+Dx+d7vqdAQWWS?1fF=;Js zqwkEESe7_grRcq-0@myXd4 z7QWYJD!#M44Cg|$K%SZab z%rCKqbnF|a;$8HKviv_|Q)TxrN!tjit=LO1t9((O+J`ilJhq)xc^l zP7D&kWpt+~6XLJK#4t+h0>R0^bQ4-FcfXygd7nMm-t33;4E>a$;ACn+o|nCGl=b%# zKy|CKYG?QNF&Dp%<_ZXj?il?(rR6UHTF4|O20~w%{lHCKzydCIZ}}C<^>TugF4cPm zbAQrLbvL*%sb0i21E>$uv7`Shp7|h|#e+G;9RD*rgdOEsN7U-8gE-8Ejsy~;=;>;Or(=cKO${eWwcdcbkTD`sYn9^_DC)0{dlXJN}`W^K}|$8KZsl zc59?%lZ5N4W~zO_c_C%j?0q#VZhN|>PCbEzu+B%FVdi+CLZ;2Z^7XeH3a{wYFwv-I zHNun9ubjp}KOJjZfv;=%?o7q$rdy}s%B1_V>64pyl^=MO5QzzROO=Rc@F{?I6*%F` zkbhjQ^CGe~^Sp@0hE*=MIndlQjrS_7jdIUG$Hut*&C%mio;qYl4_{mn5WJ>a0Ve-N zE6_v(1hbOuasRD@=T_N50wlMZBq1~&09+bDeT*?pH2Q#m(3P>;@Aq?xC0+6~5~LSA zqiybjCMs4o(^>5C3?5qGawg*IbN}o3poa!TO!WZcVFr8nT!Vl_OW#LblJ6}R@|cqg zzV!?Nj5b${CQ>Jvxuy*d9$XBLq&%LmE^Ge7@-KCP=;Ym&RNn~TgrZ?^p(c>p2SMJpQ zm?!lo)iI^nfZ$&Lrv%pn8ejDbHI+6-H~ijv0aRUM$r*>kEcWkbW`EZWCx7UMQ!E7E zxoFIl`+I|&pa-zj!#9N&{^4tPOol|?M9(&#T+*j4BI1yL2S%zeF5VyfU#%C)YoN`8 z6%Hm%#Zesrs3O#Fr}oO&BYeDJ@RIo&+ggHGu`n8cZXWBe@I+gvlq4af0xz#^5VS{iAB2 zPUnn0fz$&L@QUBs{z@^IWl#xbD+O0k&#G~i51_wR@Hj+WU$xlz}4BYiwcL?Y*I#M{XfwGp(~P{?6)1_M>Z+m7d;OX@yWRUG^AGGLf$c4 z6OR_VIT4le%dbLr-zPQ#$h@|cEG&(~O8wiE7@$HDKL$YN|DI&U1>@F$FnXNn%+MzY z3_Wm69TXbdP;KVi9i^ObL%0eghAM-x2hUlr*moMD+aKu2o)KDYQ@5_4L+jta%V%Wd zbx}Ri$N14BLcHiB<-2~CO{j`J=mh{}YGPqcjMr{Uz=1l*!;{_iHEL@6Xi#iEP;lvJ znck$bkj<7*N41LP@K0Bf{`gVFD86e<@-cy%@YKV0?EfB)UCk=g)`)mG{PHco1W0xj zWJDUR9gS5a|B`K*Ktgi`V$8-z>Ct)i>XW57lJi2h*Cbj|GqDoPVQ%xNOMS~N`Kb;F zog9NPf{ugn>o^{xr?NC79}Dj3vd- z$EF8L0e8mY&Ti?EBk;}3!A&1dPZMfmh464Zf=&aNu3RH)g00xGj^{D=J)y?{$e*`< zv281TF3N*+kg>Cke47&$DMN=cCb8QXWh2+P3P5DalqMmAUiAm&WDp z`Ntd!7l_BKTXj3Xx5R?TXL=xF3)`8r0`^JT_HTh=YgYqq(Sxj1ioM15@?x&pqa-0<5DCc z<==Uz7ntP6@=<- z9?in=ZtpyqVWGyfPf z<7p}UppkOe1v8Zk(yGbI1y0__x7a!L|{ zzaRhObsYRaNt&T$-rTm;fV;1HnpIN?7K+V4$|Fn?VpL7cy%s$a}z#rXy-|C!b zzm_r{{xOn35tBCgxtX9}pyu%%r1)xtuV}1YSt3)>3y{1^epagQfi*uVqb;y=en@E^LR0a58uHs9G9G9t_( zrtOY%niuczUqU!MCIgyl*nF&g)PA^?Y@-C7eAtJONd%&E^MpwBe^lotey?X32jMcz zRCla%xl5*ElDj>xjraE*J?vl)app9=SKwl1b+uG@WY31<| z6CL>Q!0#g-MqaR_+ju~PQ{ z|FZ4o9IGx=YKpGbrWp|6YWJi+9zO_Jno9<2M1MqJtw3c2Md7IgV37||O5AL`3eb6fwSNA`yG3#e)VZC zzd-v~S5ZIRWs;v7wtwqu$lJ=%q}5voZ;?@bHbzy|uuHvi#fP+j6(+E;c`9-57v=sT z(hs`2lYt8r&$?cmAD^tfMn-vB%5hjFn=wm~Cz^|048|U`^A`n3A|L06tabP;pmBcgm@njgTVf-8{4hfX> zpmw+u1|29!gqQ#HSBOGb<~b1>Wy<$6Tt#~tUj{QZBsP-YF6ml3G!gP4Ghp2RHDC2B z-m#4WIXRW3DOe|%?810pEP4nv5kmC1KJQuy@e!Sz9xA-UmUjB{-Qq*%&IVouMcTLc zZ0twIVsA+>WHZou_4o=Os5E>mdgva(yJH@ZdiDQ}OrDk`CaeeO0qZkyfD4F`L1m17 z`fb)N6Zk-ji&9Si6@fhmUOC}TxpWs#2KSreZzCJyFoiINcZmOAV4vGKtdMg742PZ; zxF;B%k-he((+ozd{?941*1%eZ;OkUM*X!&V^?jUK@9qNlJ|APlD?5LGd`Kg>iy-dp z{~y$>f4#*VD9TvKaY^s?=+B`4MU{vDfVlO?KP%AwAE<}#C1@p^z4>Uz2+e1}7!k+U z@&3j_Y)@*9bYGL^BT}yPen(yh7oOa{qqp3Nj(2M@Ynv;|(fd|<8g*$1JO1kdW}->@ z>=1Uk{#}1aR(b;|qgP`X($E=V(k}K?rK&y5xrB_jF?!xB@=r7Me~g^?-PgXAZzg40 zq%3QuaaD}EH2^$KuKXvKYQn?c*?j%%*onmAt^Q&O&@Y5O2uTu{1l|So<{?Cx1iZmi zzWO&hzj5wA4_|b`mTJ!EC%ks`8V<*CgHDl;vg) z9r&`uM5PD*$`sgvZtoITwSbp)9}#jdjb3r+s#WKaa6Z`6oo9oGb<#~1Aj8+qKm3#0A(X3|?giKYT9OrmrE2dcDgMSQzkYPbcsSK}{3;|?9l_x8xVtcfPT6>^ zT}5ban|R6h2a%=e846P zAfb=49sn*YpPxyDx|MIhsriNX!XBIx)8d4--s&@_d4mCwu%}6f#(pDx2>h0;Hdj9ac@ZJ46Ht0ST3kO@ zJPUjs(WRI2FICcU+;Ed~GF#qtQ_~32{U_tg_G?apD|?bHHgicYz=^VO>YJZ+o?-9u1q@Ptb1>GTK8s*1h3V4TYK8ltpu+h_FuwBT3}LN6&Iab8Xj` zPPW9bA3xi+s)O@es_qo9JU%$#Y=?Bhzm)WN2tczc$3^`Se?QB8ruApM-q9BUK({wv ziXH9;)!B4kU5_z1zm@1st#ZcD(XrwQ{DzT>-p{Y;lX_wtb8xl61I0JJx4;Z`R$`;s zL+VEh@Yw`g!|Gqj1h%mFf*Y)Tl=X=14?;3xRKW$Wa0M*Yj+xgdlTCkJ_#c7#jrxuN zw~r6`i2NqX;P@UaM}F05eRw>YNW^2h@|@*E@`=OunhWl=U3Ksf6JkC3C*7}=!}g}8 zK5i1zf6**TAAT14q;7yMTUgI1i6KL+-T0$J$`I?>PfCKr;_#jSi(b5dsqbJAxQG3I z?k0c^F{vL9i8R(7V`eBOVz!lF-^KT92Z;f2~@-5qX3>M)-fD)t5&#j7__juexc zWow)~Ox`cbb+fEoF^%YcT44&%);2sf!ov&V_74j59=yYsffIN7-*S)}%M_-*x_*BK zHaxXw-@fdMC-d&E@=2^a{`pZSeg2tJV?;{JE#nm>&JvQtdgNb~1->K17*VyLkl#E# zaGaMwosGM_=fAUD*ssx7;TjEYHkK`JN+OEcrA~`x`Z?7>MwZ*dowZb{$ed_Z1AiEm zUB9Q!PkxzKbza?fiBI9Zw=r|kO>q1Az-+72n)hB^f2?Y^!`#&odBGY*4F{w+bFsMF z?ljlS{l?ObbePUY`Dd%gYE)rES5xn z1V;}?^QttJ*ag4aksCXWL|7+u6kP9y z&U*{HXzfU)Wadv3$dX^KUoQS<=SY_`9 z#OCkO*zPRNFpDV$sASS{_23_)*`&0=kyRuca7C%U&fyVq`6l*`3W&d`{NI%Ai<^KG z8Cz5jO}!Ta5WeYDU_G`@_EeWPO8Sdn>mJb~20 z=(MCYx$uS!!XuS#lOtq%e&CVK+MKMheW5iX4Ga^ZJ z%P-`hH+n;8lQQ02^SXRs+QgUBBz8a1=G$1mU?S=zv#?iUi&I{M>saQRnsq(Y37KhC zS^+x{H}JJnSkDTloP74iWt-xbJG&1~5N#%W9vHcE5|8uV#}Y*IeNA>t@qcF0X3t)| zJY_i~s=k((b!CzrH?gv&^4TA>ahGJo*l7LaLvxsyfVnbwcZxQ_%DD8@d1PT)OUc*j0NEk@Y1kz{vbMI&3Jh;{lS z$lI`iA)9}q5irm@Fy1Qpki)P9__Q&W(&5jO5)m1C1#G*zM6DIADPJ5+NNJkWtWY?y z_l=ClutUt0c^ou&=Z(db_^J4PI2OhBELmRhs7#(1ocMK&fK8Jfk~SfZfoA@hReYzH z#@Hf3``b4(4D;UWYsq^kBsDYTjM02r=sSkEit?P{t-e6d%|tO10ujDlRhwB!p`2Bh z8Qw{r87rxJQ#lQ97@JDqOtwwmUY+S3_-F`0RQ$*EcOR;Fr*7U&GNUt&)UsT2eCD4$ z@KUC?XfFFpH}o?6dZ$-^JdqW8y$k6B8x5m=n!6uuYST-u=oHW0{JAS&dxY=OnUj3F z!9Y_Omb!WmAGoKi@L2LR)Bm)@_;2*ZZ{n~ri=Y1s?pwhCgT+s(|HvMO*th@~{O)Og zhnOs+xHdN`HtJ>!Pm@j=dYOS9bDrh`0*1l7jJ+qsa*y0icQ#X~Z$&%L+ex{ztoHA9 zO-RLkB2jt^x`OPVpK`y_e1D`cWzcZ9T}#2+innPO{ZkKi6Fw~Rv`vS$PCSqIiLfxO z%4`uZjhm}UiOrh7ZRYZfZ{b7mN(4$ z3ZYb)Kpi^ta2@&&j-$8_w7&jcDjl{;#1!&z^BywJm|sJ(@hs(wItWP&2`mOB<3S9b zXv~oa627Sl*xrjXU&7-S{uLk-tnki`r3rRrl3tu3U3NjaTy2_k>iwnmK?}0^scY5e zTou1GnwxLFyK!mSRL-?p+;#h`5l&+|`!r;mXab#a5gZ5RgCq*`m5o*krIySoFIUiY zRAk#7&(D}7G6v<2K34R9fm^!3k65M?(ZF(KC0Rj$k!_+Q$?^~R`pTk58oVXe!5oXr z$eS0(5Qj0E^#S`qC>A&oq)fyqnXh-A!c4L{Q?CT9L}P0?`65guTuX$0$qS>MOCE^M z&4fU5eax9DV5U+)?W%mE;M&w&Wc37>F|U=Wl36yqe1tVx4h|0+k;AuxYux3;WnAUE zOu9*`ZmT#OlH3A@Mz*F)PgMt&EH3wGNmTI^J{7|KyPnX`mXn?;n%zw;AtS9-ezzQ z;iA1PI6b2}DN05o*{Rg;v^floN(`ko%sUQI*BznZ><(@#)iI{si}IgT6~JXg%nlyv zeExAGH!ms`qFnm+s@hbIYNSYl7*fc5lC6J>sfhv0m)GX#nvbfuFrw^c{lYTBh)*(1gINZ{HUgP7<0(qt)x zNTK>=Lt4cAx>IbdWWY$ys#Qho#Ldb(hWC)&BCVsU5Ps?&1t zP3j5{zVtOb8L79AHtv}ARp_oUChZH#%bFpyabzt&f9@=?F{arpj+OztpQdtlCO*?2 z1fD&yMiOFWpsudoKQfnLc3gv^9oq~3<%&Ls{5FD-@RAB}S+T6cNldr?5(n#{jU3-) z^&z7YX!kcd{a9woG9Mua{HMBr4~ldwz`EW(&VVL@7!h!nlBj>dWt|E*qPMEC0kb)Jl9_V|^W4@+?KairxU#-yl zMDH^Zh)TUb_qQXUf!`8B{{YN^{8y!TJOLWT6D14Q*B#Qqpo)duE`}&(guw(ZmqX;m>$y9&M}s@Q12Z*&{X= zd8@^Tyki)|I1tM_xdJGE>cKEN7h(|8V}vhqaOww$$m~43%~X=ULgY zm|z4Q7eD?n`)zd9$N@bX#g6i;Kq8F}A}tndi^I`@u*X5Yf`%PsLhR&{P2quff9O#h z48{$r1Rs80-&&WPvMIH9pTONbM);+Wlz8t&5_abmRY`>$~Hre*gXvl2R(k zh-74iGDA3$%{el&WsguIdo?udy+<65V`e49p`xtp9jB6Uj7~X54u{|MK0f37z3<=s zkB7rK=Y3tT>ouOQd7Xj{zWW^T-DOI=_iJP>MFR5=ChoOUFyb#Xf7Gi-Ie#S9_Z(uB z4>!?1Jp-$bh~3QyP>T(XWkjBOpMJ5>GCVWz7ZxFkKllpEPGo2Z!Z52E?_KaEg2DZ7+Lyg z-~c};BxQ8v_#f{54=~GB&_p@szqB8u_rk~NSN3pT+fxx{1td6%%jNO~iL7{$FnflH z@#F>&`IxBL^e=!nQ6$g`X2IPt2e{SO~k=jW`$!ThMU%srWzOOR!-Gw7AQX=T8plB#Ia!!F|4H!0QeTzNDv zk~M=p09!r0+*a1avil`S+*;xRKc~ZWAT4+{ImFS+UXk@mDPrYyye}r$vhn$sBRft5N_*anr`K`6|z^|Cw z@c8-Pl^gj`?Lf3d-G9*dK;~d4>76MYcFGN+Qq2 zoMhX(Iyd>-MfnkF**XOr`e3oJ#P~39Dc;IE75$cAkaYfaqq65$l2oxhtq!Q*4Lso) zs$QGZCgD{0B+ZL~%T&c=1a&;gpn_ejcW09|{6&v;Z}k($cR6ZcsMD>Rz(xlwez?uY7TCHnA?9n{JwjNp(3luW3mgwvw< zjJh4m>6CQ=K-pozFz>|^m{dz8%N*v51F#sW;za+1NYm%64?dNxgP0k7N7#+76OBpd zU-)qf*7e|etR)Ftrg?`-YLbzCUR)~E{0ssEu zTlRqcL+Jc+lqv&o;mW&woqFRsEPd2|iAY>?%;9E2Mo;8)X7M|Q_Jn0_m*reXwkf!$ z*15A44=PP`wEm>#RU-X54f7HjOUoTXX_~7%8t+9Ytyax|9)thC>K}k2^d0GD|4EeA zTo8s1wff@D(?2;Wq#pZP%gD$i=-?d!rj#>ytq0CHvYQ^J%GHVYZS2I-@xJ{XaT3{I z3c?<0dd1jr;po%xWvMyF)o$%xv)-qhJr@V7Z9&f9CL-3pEZ+RLA>kvh+P zwCM5;7>#vEqvUlfcKAAMUbx_&i*U;~_~i&pesp3}7L{wji*evi7G{4hxR4cOkxuyF z+24C7krk$YTA{G#tre`Hij!v5qf-Fh!3`S`ZWP?^4$WY`ICy#V=a~~z zicS&kD{$o(@5Ftm2Ez>Il9oG~j9sX^rH6~)u5W=@66?);O%!&c1m5gQJHg^BxH59zH3S8tAiYP9+KJ* zBaHrrJSbu~4)%UUigA&#|7&zk(EZ7tiEMbdQ0>jv1~t2{brHXaZ#EflGC#GZ)TOe9 z>JaB=k)|^@Awp>8lyR%iear2N8ThCyrTkh%T5LYgd zMTeAy1*C9|Xx->gtl2WV{jahKms!SIh0%os0G+W>!ARWrR{EJU$@ar=Cnu3DEy|N33SwenshrbIe4ig4Cr1vFDgY%5Q06u74i@HL@9Mg=FC@mT9?O{=!4v&@`dWqh7JTdDz$58{)fYjl zm#tW*hmp8U9Y`_@dX>vso#FrM~-dM^Mj>*TS1fW$K;X&YzO)Wi@q}#Pi?fU?B&gRdo)KJ z^k^A=i7xU1SKpe3g<=SQw=P~`{z?+hg*P-EGIa1$q47yrPiHnCm|0gk#R~a zZu{ud%~7Ane&6p(Ny%t+4-}v&ZZ1ftYU++u;U%Uf&E_{bP)B$x2HPG);i3o~ zX&67C#I-BQJ-Q&3T!L7-Tce+xtAxZiBo$%V1;uQ%5L^|M$cPSF zo@@424i0nrRz;Yr=-C||5~6wCt@o!C2R}P*E1e-4D2fN@;-r!aqru5hq3GO*gvKP_ z_!6O(zDv!f@K{I^{($d2i>)e{BLnOiAQuGCx7Z1JCRtTQsLqpLvG@du(RIk$Yg7Yn z>SPb5(%_SuCk`K{So^ycAa*4{$LFKxA(7OKrIOklph>R4B?h%s)TKs7^luK_>x%c6 zIwuR*^uZScqYb_bqycr5@&Fb z^a@nN=S_h2xW^$onQch9M-oNzATU4x!tEjDQucdu(i{xXEVR9w8}THDQ^ECdOd0B| z%TOTa7|+Ndg$<3Nhya(x;=p>3*UdFK50X|DgWt5gR5ml057}{!{W&cM*Z+Q?W4TU% z?afgpbCad(1sv==pZydYgrSm>U{$~eKg;X#RbxL5byl!;&rU7Y1BZ3ubXoeNIP}Ls z%jh#O2ItW-4^=0A4tdpHBY69X#Ns8iU0)2b0+G)vh7|Zeed_+F06!yMF=Yz;Q+-an z_E!!d|JdRss&UNY*yOuy|DTbbbLH&obTj_GAnyzl|;P96_Nd}fie3ng|JMQ^cE*9a) zL|270+{t59x@5-<6z<@C{rQWipvO{19u&6!-mpQMxCLg`x_(`B{DJzYKCmM|14k;2)U9=OhhRvF7Oj`b<~B z-4Vg>PtGm?05RBXt~(=AOa{w&-aycVr>hE`$h7zftwx%x_?fi0x+wI`12ZM1e1mdv z=dKDRzF6}b2>sfh7$=T^xynv*_vT(K`T`erb-oj~`9Op}kU~>Mi_Lhj^zGa0t|fri z^$76+5Toid0vpey7OzPD;x}Ob3l~51f!k^UNPm-)6cJ_u*!puR)7F7rH3f`!HP-6e z6r2M6%kFC_O~B+pd`Abo&v_A5T^)HZ~FpKBV-Q*l&dl8lWwr0R|85tSTxl;BPu=V_SP_2KmY6x7@ z4Nl@21UFc+mdF&K=LT%0JdP&&Ppt(ofEP)tW5nRRki%$x7MHY7KX-bFJV13DC%Jts zzAn_Yis>8PS*Hfb%nN6ny%+4LV)xNEn>g)Izr;2>%uN`?xVgBe%+EXW=-F?*<>-ws z;sL9>HyEVrBO@p#ye07IARMR(3BP86t~JCQKOlSZr3$t14vj zh^XdrT{DTvo5sc$Ero=lKnEfxsiojsPgwRG899gNp9S2x`1Bbqxm@Hi zBx`ifn8+G9&NtiwU@0(LH&$9ZFmM4c(d3OzD(SMpI%V*ma)F^6g#f!O3u@3KFYquN zb75cK=|P-$vOip*)0njU{PzT=?+zp zHejzb_h$vE^P#HxWXX-BMj#6&OFTOJWS%qxz;IqU2}+|4e=aVqt20_+k3sIeuGUA$ zFa3GyLlTh~y&JbzRy4Wed3I!K)$sRE zUsQT0BPIPjC(YhUq0#sLwIetnP3Yf{(k{m$Y zYpXg42@FpLNQ#_8DwQrRfzuCescM+r=_%=Q#SOM^@ASk-r(Hmb58&a#T}>MPrs=hs zpiox^EC87r5w{n*`IE*hrUbsek6)kl&Q-rYzqcRy0Taf`ddv5_)Mn1s78c|iIK05kI7j5XqUd%gOQn?8ee?yVBo%MtYbE&FRRO~e<&*6 zBPa>Dsz2SXLGh|*`4j|8Ci7@@_4xMqHZKSNfC^n7^Y>rH*0F~f3e3BOD@=SV<4xR9 zzCJIg)?jv5uSODHH;U)fK0`qMx%0pX>E87O7S^j}S*&>w39lX!q;fM31jkk_Oq=?H2|_*b5#j zZBcKTKAYyZt?EBFlIqGTV%l=+xRUL?Es2YVsfl`F8}g%g;wy7~Ua)wsyWlhPO|9}q z8Yg%uIuF%uQ&zGms|TEl^D&E?nd}r)+Sx>~H7skY%-Y%MYTL$Y4G%qMj*>x)`Q3l1 zmo#{%HO_#wPLz$hFf}I*$4wLy@7_?PVEcOi^o5 zFT3jgtuemk(eA=j_h*g{0#a0pG+$c!*0z6m*tT7#|8rYn9fqwL9uTyX~K42*>NAZWZ?%Y}j)e9>g9#c)! z686&MisFs}xI1tAx^mati+fG9Fc&Vv((Y~vpPNqmKknF!^v6X79!sOIpGssWFO{z~ zNCqQz8T)%1!icnpuyyAd@~^xGEAm0YqNJg?CKbca~q@UGXBiB=)jgqyLH1|A#V4oXX= zPxjjwW*_yQzT?;-8z!B}8gI{yYhvv2ZRvbN2_zMkcN9nf;pX<1wgDTLyowm_~!GsI80@(Xr27 z{Pv}^voGe&uS`FDnq`I!sg5_-i&rS$6E9arc%;(G;KT;X;lhL9_5+)t?ru$ZqCtb% zW#iIy;~AobLKXUVbcu(IhVwhbL!Gn2DZi`%uj zFWM9dr1cB+dE9}MH@F3PSV39`wZ`@Z$!~Hs*BFU9kE_}6lH==~VQ#oy@mjXoh}iJ9 z{p!SXyhPwcK_@$)fU~IHC1#{Wc~8!rv9t~gjP^x+{S1YX&my-*JJYvgILQ^X{o3H^ zPQgGxyK^*WZbLVLrbuqDo8@KWCCr<@O#jCEj+89YIyH~yPAnVqhe8Vu0mS<3{&}xw zF_e=IHt4WA0poCo2kOP_`Etj{dCTz;jr;AnZ@S$R{-hSFs3^VE#P1xJcwE#fq-{hp zi*Rov6sp$g#9sZNfTMmoUezcOBO2Mn(ejULz*eyb#Idn@| zA>dvA__IL0K6{VBaYIGPq_}}Q1iX17e#HCAXG)~nXDY=k`Lk7x*=SZxKwOE2rR}AD0GwC)%JA1#`l5b6eO|&7 z2D3hT8CzvtSSjxu_9W7QB0*HqM963s$-|H|&@DHhe#lGmvU+7z)j(q&Pg7uvUYys& zHNnnqoN}~#(e}ccRQww$V9nr*Kr28 z+)W;>pO8{e;4zR5HZN`oH@tz3lhd)e8FpTU^9isK-s(URdILoW-~SHn8-_Vhgxc1~ zbI?l}nuj^~B7&WGAnfsi?ebRflLR27cjb6oQeoM)4Fbt8fMTo&vzqR<;*ARRXAPmTI59R>q?&@&5hb(yf7|^;(_Y z9OEA;qaoG$;^@dt)7kQS0l51wRHp@OXkBn4LV2ACl4c~JDHr2g3c~a&9wCC|FH$r= z1fQD^Sl0;41GSfO>4baq`ObAl>6q+>=B5j$VKZNYf#O+3sL}+n3`%X>@T+5}?1fiL zJ+(~q^De3$F6D_LlI0<>=r&!pC}gT4>P}XIp%mO;&204YRLWbju*_3?x0J$73eCz{ z?CQ^4b(LZ3jMCt~U4bh99pKf73i=-AgM(p~-Uu<>Ywd zP{4|bqV;9CyzASDhzQp?;w7jg z+0k3B_e$c6>JO{CG}H#cQx#rymha-c%>chtfB89gpfcOQqQJ6!+cpGqpis+t0R@4B zV5zjJ-37g*7U?hD7Tr?QPmtIHu&qazEnGoGYMu6G)yLU~A9-l!kgGYqUdkg(NA<8!mTi{ay2OIw z=*39B7{1mnnyjVmxLJp9xDI!8EHeMBvN^=0-cxlV_0183}0AQiS4&{E!EFlbb z7oa~Hr_RKOrx!;c);w0h;D3s2F`Fm|*)AXUbhcVfjWNX^hq*Fq7Hgk6${aOThaw%y zYf~g1b&*RJr;1uPyuk3T>M-TpQJUGSmz&FG<)w5R^pjl+ZL^NqW8P9TN-f(M=Oyml zY5SAr3!8Z`Dce0ys;%r~gyEV#T(ARhrJBrU(aF9T3yOr3bcPiyH+_iVo)4sG3FC+pY;Y8xn|%Any>G96 z4%LH8=V)dJh~1?IHZcmGVw5+c9YmNS^@Wn7F`w-t_#{?>B-4wks8YP!tO)lTSNIi8 zDGjZf&_v_X0OC;GO<&S`NS$&qwtxfnV76&Fidl$<22C(lklFT)Ld+LPH_gzk&j!`w zl^pG_e&d7<3m9KITd02p)2D-a>Tq?0xYN_X@I^&jdhq)bt=Fe7cc?ANpqlD3JmB>$ zeZ$f7Al`Jb0}Ap3(ei6{|K;`K2B<&4C3QJ`gmF#hm7#VQ**~wlGWXbJa}K{r z_YVkAyND`^+EG)h?wGo=FDJ-kxwTF$%a-SPogn!;Mh2XhiLCx)dN40;ZNm@6ISOfZIxb;UHom8b z&AW=d=yFtxj-G>$8G7h%C2DiWC&e@S%epIW{rk18iu}%kJ+M6FNopje$|%~~xIb<$ zc=n5dz?nibsWChoLb!VVsUbICdQ?M*jBjI(hL$x;noch*$B2Aw*H9!2A^R*Jd5ymz ze`P8?6x&{aG3ed{Bt&B-?fSKc@K~pk&>Dr8R+CRi>3_MDA5Bzzarbxco{aAa6ga=n$G!Q(4JOk^l*S6L7XXMI@f3r3O zSqUYVj3d+scO|Wx%{Z_ws+7_>Wk2c$YFnOMAJCXtvgZlhUJcjpUBgKVbh%AvlFdSY z4sieTr9aBhi(~7?P;WqDiE&Y#OwN#%)Fh~Ojn0g#gfKl5TB=nj$!o`B*P_{VMxlIW zSFvWKQ*CHyWD<9JGkUgHe8oHQ4PNW+A+jkDqs+!fCB~G@6Q`X=Eil4lW$NaV?%hF0 z=lTHWazjo`QXcnN(OOcmK(Bg~G(TUa-@rTfBHhWZH9-O20OOmNw~p4eg-*8vDDLNz z=dj}pf3_^6lvoTE48xzNS|^!U1*20I66PZ3e(^9rHKP@04+<|1$#uzjD{sFtyB=7j zG*roggBW^|3`hBh&2KOu0s;VaFG^oj`!9?0(+<<|v4c0BMSP|h9j+MQApSmqmD8i7 zM7`k5i}~-qHA9cfTF)s+x$wHR2>=QDudbyS%&U1y72BTA7b|st6 ze6Sz(^cyd>n{_x*0eq)*1}}JO&N@=b6Ll%1OP3rCqx1yZDLPHauHU;3HL!7&knPgV z)utrK$N9x`S%5H|iQV028ws()dq(4}wseMb74eR?O2OQakGZ}JtW@tj@sa{b1EK8I z2)yv$_gnhB6Upuv>?**YM#2tg!&3cnlr2ntb$A-LRy^vAQMw26I z!8E)$AmgT^A9%wdE8z@`$at}wW%Bhap1ac!41Su&A9#0Iqo{cU1TXFq*&E2>1tHM6 zbbP?bq0Y16+fUr&vIMsXTS8MyaOYXKOD|Qmw7@0PyH` z@AZVQeh)UO%Aw(M>5rv7R@b=Fc6Emz6=n0%p8JAb4L^_6OsUeCcK`!wC_h|fvzzln zfwIQPQf*bM2c|kb>ez`)Pv`q`ni_-w)l{tp+uh2&daKPqQf;k5s}^3nFrlNyyUj8* zrF$+~rVBUX!HdaVi#w=ExJo(SB&$8&^p$u90eSF5UC5!Yz!G`qpa#gs3zszr)8<&B z@gsrUZq1i*TO48bK~AQhI{Q~5*6q1n?oi3J^kt<>=g1laP8$aDBfJY3{4bK3y$N>n zQHW6a7=EPe*#63C>9x5fSBiv?E0z35ake$I z!{cH*E~><>0NYqvnnHa+iLZ_UlYitDR@KEw@(h~;Zr>jMo-B}jrh&~WQmQ$88fROt z_I3*T3|wUF5`uxq%F(HXwm7N4ioW998DK1)ei?4%x=>ht}r)D33&s0YAtrVTp3!sKWh@E(W2%Zito_EioZZO63|KeJo{$ z5}zk@Cf!?$qou{6eF7S@t%$yC>jf?!)!mnUGt>+3?CKC|=AXWN`8Z0NT4(|x?B$y7 z=$}E*G1f|cp~LRr9Fm_~!MYSQ{JA}7EQXC7DkoilmdkOl+;9(i#;{@C>9*hp7A<|X zW^up;n@UAP>Aij}puYq6Z7?suK3|J5I#VR5Wu$JD+n!+<9;i=D_hcUG=9EP4yDQOL zNMvPJSpC$vGVp9PnC(-N7*iB)Kx6#2Lp*ET4{xb?v_nSMoNK3>GNLnyDN?myW%G0d z+an|!VW1XmKj?Ql^Gwok8Y;IaJtrsLLXn?{iB3KFB&}B~KZFL`H~W`|76&}EUH|D5 zBESn0A$vj15Xb@c63;`nm)FAIeD*!uyxrRo;Z?prgKe2H1~u*pdpi|+T_eFK%0erx zk1lT&y?-B+%WCC$5emPyn!-K?GF&Zz*mblM7Oku@KdvaUo!^j>0zc7x*VTjcSx)3# zNN&Xu=I(i7c8&*98P*uGP@6S!31R6Z+%s=_c0EZ$(-dK!QSpqHJvusyb$k2~GGi=2 zL;pF_(5WXeDPsM3sk_*~SJV(iXH}617f8E%tW+V)9YnK>$MTPn4f6 zl4gK*?d#vQ0N;th7k4ChrriavfiL-L4uqT})FRIy^H`|=9tcESa=sfUdU5+UXH*`G%%o2^xTP}Vw_m7ETiCoUMAhyRenn|+- z4Lgr@5*p4zGSP|w7`owAU(f;ue;1mBz{p3owgMX_UC_XZTH)3p{x$r*enO`vL4Np- zwvedK64IKMNz4vOzVgzb1?l#g<$VgOYlt?m^p%@E3u9y*&?V6)~sU-$Tu#^nW1 zQ)M5$BzISX?f9kD-Iv!cBv1iog=onuFZu+^Dg^=6e+9d}BF>9{8HT6ejlMX>@Ci^o zn>chUL_&{W5SPSSJnKD2pqU-M(9DK74siA0pB-;G5uS;t-~0Tib#JaXn>L*BDlOF2johf0~%_+e$CT5Geu_RvJ8lmxpcnDBFY}TNOxfsZyq1oq-8~K4 z06;(#5EUdUY99js&(cEi$gqScaJAbD3m3)`@e-MRzN0yM03&YIDEFR(*}Niq478={ zA4{bxKosa#f%5faFV*$;KfQ5irA|YANxhHDQz9D;Y>$y+i=Uo3Uyk{j!HuxIvKLD|Q;eLuby?Ba#C`|IFpASMrdn1E%o!EbEvEt(p`(VfzK>)VLlVgd zC@D`cyguKwxOrO(>fZ4Kj)GkaVGu#ijNGsUYB0EEs>w-h^QU`Aqw4Tf?i!R}YF@n@_+apIimR@&}|brA9{mkPG}Cj(_+x5UFR4K%%zs((_HlTlQVC zus|p~wZPoo1WHsuxZT|Z#CfWG4IF{+*#O5OW;>?u{Z~3k2G}6id4}WSSV`%j>ep<3 zl5%o`DH_w7^bGvR&P4JFf{^Wfsb?CGo-Z9rsgCh<5+*y$%RlSrZ~20@saK0G>Z1;r zKKi@W4-?jTQSo<1t8zAnvOr$u$va)LU-l9Nd3tHq#j#IDS3l8#mC^wM#cYeN!_^}o z-a7$KaJSc&EljeZ@7nbB%7L-E=JD&H zELnScXhRSJ|5WQv`k56QiX8pYNc&)R>owbcIX`2x{X!+yA9sS7h0gNVpII6yMM^eV zq4tBvULP(rlpe!(sP%5EJc^~Kk+8uE>7&^uppIase)+4^<&PVr{mv%;8fv?cNa-MICmsOa(Unh@fz z!SgJLdFn-__47>>q?duD@NM|xZI8Plp@bcLQ+kPpQK+R?h?2MG{x_|kFM%y(67pS4 z{~cU}!AsPIkLR)m$ndE^puvu;4`bv{o>h+$CX1_MMYj6nX~KO7ZnOSaf8bG23WO#SY{-0&DtuDs}E9R@RE)`gg zd*f{hnd18xD>*9wj*Q0eLXN+ZtkUnO4WetuyYIK=W3S{?hN~77Y%KdenIMWO{@G`q zPHVaQjWR)}(0)s)d0eu;t80IMJNqW9^{duDeVwG8AJ~woF9TZmk*DkTSG$8G`C5qArj!!zkmHK9i4F*~u=PYN?DTAcsLo95UQgSuj-D_z zBa051WNj-G!(FG+e5n%7d@Z0*v_66rE-z+5%oO z;!U>TTR_r!P9IOlH;#Su6Nk?41lj2#^`8)g{MCMUu(o|dHZ9j& z(Yh)eq*30x+&_KQ#E;7JZIy{}ZeYpE`0x9&HUr%Lxa3gJbXoR{iT*oyuJxyWn}>y4 z)yFCUY_8MkPWKV_-oAjP^z73S-{#N=V z1&Hg`vael+1ag^7D@(k+eZNGwKQJ|q3&PbK{RWn*bD1R*dbQ-HXV}Hr3(l24In5v^ zWYXo+re>C7i{@>&MLSHC*$7%jK?YmK`PI$O-*oR8%a!l{8804&YrBF}n-hB*!pL$r zO?rmP$uipyv{J-DCCZEv;6Q5H(($&VztN&pbRWoyTl7J84n4>&T>|mX-9BGTaJf+7 z_b*%N>FZyPFjuYIxL^|SeeUdB*?mN^Q;vUQ3c*L_i@9V}_PHuv*{aM3rkh zk7&Ow`uayN`$m(#VX*?EaKtCaYPCFldD2I@C-3NC;$!VXYs=5aqBi6iqz9$mR9zt? z6@utQp=~GKt?JQ*b^I!&Hu!izW%Dj>{najUPC-O{4y*LNygC%iUcKnMmlz*vN0BgZ zIkM5@qTt?n>{)dMkfk|nNT%C%kfQbP-5^;Xdg39{%k1AnjZgQ6toCoWNCYwxECWeS zdLIdhVB6?(8`zk{A%r={8pf)#Vhq@{jYgWrR3)qvaUb&erk7){K{e2eDdZ4rJ(0O@ zwYl{z1+S87b^hrM2B(4=ex~gPtd^Ss>UYwGz~M&PL%4jc8oP;%>`|n$7F!uI)u%E! zS5(p2=H{%!+bg2!NkC(EB3Z*6l0o0erSzV=tIQ~_i#sm5tGE7=qkUk}<`$&r36}dC z23k<2!dr-(#dJCUJp`}{7d?av$su5*_)ZLp94Ye;pfR~h)kj)N_3B6m*PDINt~+)j zc-eMhV}%ey<9BDLz(+~`LHj+CquM|pay;H+#O&E*bH?iw-muOX^9b!<*d<~2uSeKx z=%bOpSvW!^DbTkrBL>(xvduBw3k0=*<-wi3uf+v!C@EZk71~gI@79)Li(Q~da|O0} z7PC9%Kbr|+qHOs&CF>t%9f@8I+~T}mJy(RI5>v{wAH}XcX#hU+oaQkG);^nEaBXwY zGwh%KrL%O<2~BTb$1kgb^Ra8GA+B)s;m1Ok&9=W=Ti6(Guv=p^;nu2*`d9b8@VcVJ zcL^p|**g_Boonr0e@;dQ8nAe+vQxJ7*^??f<#=ewVFm#*z1n}a7c_D8-=r8c6qUh0 z&hHSPM9{lhfLqz%b8BkgKwMNeVK47o4K8R8M`{1io%< zIM)=#?D8v#_NJ5-@ju*Dnhu=%5v40lA*njALeAd6d}oC~69&qh5fliPn2nyVT?OI6M7-Lv+}AMGwV|C* z!Ka+>FdUBzepd3kPrkjm9_wRS?a^!zFcGmQ#9Mg8G7K+aiaFXm6c4rK^dBN`pVM2u zdIgqOK!MapUwV?i9Z^FwYo|sLZ5M%($w7fyVQ04RyN#A{Litk}l_pJ>#St6Vg4r`Gq`5kntpN zM4m}9wI7^@M%Z=*kD>>;?(W$$a5NKoqg!u`YfgqS=^utaJ0_OJteLj$!Z^4w7OWz= zmur0-O<@lTknWMA{aKcnq1FpKM9Vs}lL~yFT0+JyJyS1Vf(x^iT4O+j6Z~7ZuYiY# zwtT}kfIFUKn6I|g5O`W#P9GCua_LOrwWbgm#hD;dg`NP*a$JCH;d5G%%S!gnTD>O*HoYrg(N+tnH&4q zkjs&ZgNADY!77e8oN;9)Z(?PzJ^GF`5ALF;P^o;mKxX{637JCN{k13gi)YmXOvA4m z+r$?QImuZP9iEO}WtQ>CEf#!c=s}kYS&FM+H^bxR5|+ zU%y_Xy+Ez-Twv~1h>3jpr(3fU2)RLR>vfZKOH9qfu(DQeg_D}OIPbZm{ycPAl^qn^ zmKfn5y}%Ykd6{8^Q|QGd6nHih(`l=(pve~%gh9F6Y@Xv(e`^M=!{_dX{a$-P_`LHr zX_N29)lYt0nYk3m@H?Y1!tGn>9H-YP0HrCy=|DMxYF@kmH9?-{b;`^)xF4;$#NNV{H=yro|5a(U>J#^{xO{f0|V z9{OJ&T3mdFj8aW$&|V&RGd{C&TcS|_QBR)k_ADEO`ByTsfc2!HssgF8NQW8&7wEVj zfaBiKg;VHzDpI|VoQEqd=^puX2Gl!Q!WNpeMl3L;uWw=4Gb}-|&GZ;PBikDpGoq!s z24b2X-oqA{?z#JFd5INA9cw->`_5^ZzNlIDoj3_f2Ai$*p<9)?oWU98srp-CjxK6y z2qX4RLnx}CIJxt&PAai^IH{5&ynMVIZP zE`4e>0n9t8W+Wo zJvUGd6l{afVHM28yML$GFj1*PgGPXjw+Qlnim}BygPF)T0Aw zjq=Qx(D1sDBkOsi3LYC~-e2R8vYN8Sy`Q(zt6V*VppuIZAXz918r<3sYvh4inidWb zYk}7muQYfgqVjl~JJi(q((mBUH*5~Qkc`tTU}F~x4$R1TXrG;eiHhrB@f z&481ci8PQv?|`t3m=ba#jT+RrolWBHBlF0j6F9064~{PK#5Tqxpps-Xsp8(s(xeN; zXQvE1$02^00;zKXshtWpkT7WZ7MQOy}VI2_KyAjUWh z;EyGsk>b_*kkn2`-fUZv=x0)TBgBMoz<28m)EP&0QdNa^rlvBVxYSb9!Cn{#)|s2N z^xZLufkIrA)cL$%iG`#2WprPYCf?A5jTbyTT+Z7x#+lG2Lu2@!ozA{ zI3KFG@BK<(!z#i@u^{o6j^=|@S|BL5wZ+mSztYhXQvP)qT4H45@lbbCbt9SO>a5zo3`m^1r2*d-i1c6 zFrEiz`=?e)imWOx0g^6^qJBd5GzhpxU#2oBczp}<46g0$iaC`0QCqD7oiwS3m__L2 z*rD=~E2lo4?E!hCqxa2##Lti7)zqhyB6{T}H2e)ZwCOV|v2z4RJwxe4slka#bY}wy zGv0cU7;8Ix8x74}>n{S@C-d$X>0bcGG6bDwKf_BQ$hlfp_hYB$e=X|K#jk8(KMLO+ z)M>_he|vaOtX@(#vF|L>!i97$Sj7x&DE-s=m5l{kpX8u`X7l%QGFAJ3Qbwcx{ZyQ+j^2oEICkVclK zp>Ka^=m-KOAE&G_!c6q~z$)AifL9qNt*sEJ$FDB?x=XXx9~20K?7$c;Tm21iUoP^K z%u5uAY5aIDDE<69DABpYmhl40cw7Osk&*s~^N8^$>3AtLA4N#|X;coxmm&2xJf#RF zFC))3Oe{o2$7qSmDm^Q<5Y=wcOM3b8g6U<&`HcbO!N340BGMFd#2=^vaA4Hnz_|N0 zn8;%|^X$Uk^!ZYA`Fwd2q!dQMi?#m(9!g2K*qBLs0w)EWv+CZBrjm+7W__$KEg>f8 zq{7R<@=JFREh&0G@{lF$8ZfE;kGUscpj1*)uNQZV6|m;OXDc_TK(kpco;-S=YZA!E zfQORS`Sam0s<r{e6S{Ra}575A}Yqk}lodaW}WO z{a$Ex(xlu%W9P=MUZ4f=`R1*ywGnu+n*Bfw$j%+lq^ zb!G>P-`xxCoO7F-Sa>ih-m+F|(mm*)9%0SsbpuSmIPzmnHa*7@5nR-ffS_~sl z`Bv_%^K1KUTiIPYs$SwNbbvAPYH03+gts^uQ9_|cQExBt8?E)w*4k|~Ulb zBr*im1I}fLTJ-#%y}M!HT0FKtn~vYyu_uu~@L8 z4P660M(T4kR7YVY#--xP-A(%Z5fQw}TD^OkjYVJ_3;8Tbz*3AM%9TyYT-^vv+n9)H z4!LHrA3=352Tup1$MByDV``C=6WJ!CfnD3jIIx zKc2y|XuMm73}Q+|mwk^w;ENArRI-|m`apfh+#pAEcA}`@t2^D7kSXT$G(Yr%g_7jb zxqcPq`ro`BI54R8~p)!`UeTQnTbSlUmeU3tiQMX2#mh+ zgc%4Pk%`L451_$%Z(d`F%rr3dvcPqNSB0EK?I~MARTxlN5qnY!l2e&3$qGUJn5Aon>=-nli%44)ctZWyB-zw!&7$|U|n--`RbVyVhmRYpS<>24vrwt$5iN&AaToPES_JZ6GMl{#LqBt&I+lILZ#A^n?N#;`#Dx z7igw!GhKuFVnE$HWCO6TU)t0CfmXJ*0~;iBD;b~uk00xmV2|JGdw(wECUkzEG3)pn zrC_*zka63Ak%4%I(I1Eaa7lB(Kq7FL=qa+y{`GaRPXLBIIR<|~KC=J)111IppWb>u zX$UJK&ly^q@;ms%u3?PjJ&Qjy7u2CuLx*xSTY*RI4vX`X-nf1Mc^ z9s9f02WI>~KK0j|0>C&-{Xayo7*F_tQ0Xq!#=My$IUk(ysM?N03U;g`EufGgr#)rRD29Ly72$%UCGEn%B#d8D7 z>EB?FvHib}V-+0haEzJLmW9{0ng3=(cHL8p?U7OkTs4n;puqo}eEH$}*B^;9JhASf zwZ>zC_}%n#_t%?jZ*E;R@`7Tv$tDX_y(yQhFFif4oM)Ovr0{`|>R+S;VMS`Lf6UUq z8~x`u#<yDwBFNdLbcq<^VJ zcdQIB-Jz*kFopQ<_58as=(jX*E1~|M=7$u>U%j1VRQAvRS(v~>Xbbtw#~x82{c0uZ z$f2JD{@_}IHq}fcNH(wkY~^1T{;Cp20q#_xu7M2d(Zy)?+3lFPzDpk)keSmQq_m+-wpp4Rk$IB=FN=0{(th!k(Fr)o;jd?y-W`B zv>gZi(*F{i0EF*_dgU$16PEa0Ec~A|1C@M`F3EnKkNo!o|MeT}BgFeUi#wFf^soRB zTW;y@VgPj@);74oIj`1y)jiLsZycU0)53k%Y9u10P+fG7#ahP?FaZVCLA#K*C*cAG zx_zC7PdSD5XUEU1eUtc{U}y4HxZ@O|bZ>k!w|mswGk!IeBw}}3;ZS%S4NK-j8q9tR zCO*`UnA)MTfBXMY_vPVGzU|+3T4^Q3l+Y%r?91?Np@d42XjGK!*%@ZWM5$zHRS6Z5 zY}t1s%93Svw50L5y@(W{IqS8P9_^0KVsK z<*Jqfp}%O~6`TThf!EU>pGT#{S*9svYsn98Su=g&iJSR+K_{F#hdsCpBs?H;7k)5Z z<48CdQ0wq^1G)V?yycs&Y*kp%7bD~}k^14HUDYEn&94B}SLyt{HTfZ}tv4I(b`b0O z9d2igYtSx+0D7-8CwjSiV+t$hu{L2@Wb6p(ey9j z3LFRabE9qW;?iy}D`vGqusvWur3>@+sH&mympenCSE}cWc#-?dla&sI-c?H<+2_SL zJ%01J9psG~7rKkfHF`ALA?LhN5rgxq`muiu8Vo};+!w4L>us_*`be0ceP_4>7Op6r z>18*Tay3;eIe#ogs9FvFWIb3*Fev2SXPo>p^Fa6E0JD-4@;xHJ;%Q%i= z2QCvwQW&sNg-ZIoHuO1^Rr?1;mBq|da4U6+RqWXiQN(|LrH zJ(IYhQiz@FEw=_OzOHJUIabsbEd1;Fxds;G!{Q+W}Tx-U)d%vXtn6Ft?kfjqk-FxR6CrmGIP$^SGw zft)TjxN%g0VqaE=yy4c^C2_!)d{tXqMkQHBqxTvnL<#`MQ^0ew&)9Nc%&Kq5%A-2k zK%;#+*dSugno_OT3is@|wkMaCC38e>@tW8y6Cs0OCr&ICoLdrwK$A(zPdYASlGEhc^HK0-I~J94sF^IO+uNUFE~8yUBPjvq;paxuzin}@ zQWg(WoFyf^*sZUH2g9LXrtZC(`O|sxrE*#R3b2%*wd*Go%QR-{{kn}Gi*C{^KHv3RexQP&ws?m$kf$~K-I!tV?z?b#EHPN+3>{++ z6&h4AROb31H8IAI)Dc6FD=!D>Av*?t@FO(ot+csj2jy$s|9B05<+TKsGViq>%p-ZH z3lK6LAii)1*GLrTP1+{K7OxxexlRmZeNpm!-?%CQ2Mg#gh)%peeH z!Mnq^X|!e$<1@jIuM$V5Kb7E+;hrs;yuUc68WQiV%a|L_(Pc6o=e{{$#y7vKGHjC^ z>^j6NZpB{UgfNY< z9NPrkc2|WIYp%6hT)5Sqd%Z~*W1cyX{>n>Q`lJa58hm3{(TtMnOi- zd<2u%G>L7~LoQHo()di!C*uSn8P!ijNoC_EX&o-}I}6ws5F#deA6^DvOPDl{<3soeWwD7{9MjIderX8o%~%Fqlt+ z6%&Guf#l9gPoK5FUEtKzJ>y0(|H&wJFByfN=dYlqE&}dM;Do-gFcxh!(rOdMZWmG9 zRlfzTWFLIfZuqpIgdtbQ<_C>!y}S>ceuQkd-ZYrRMic_Wpd@s69~jnKuHG!e0uTC4 zb~t&41=Fk)10e&}xkvE-os&-cX}(n`PjkYE!$ogu7KdxTEt7KGeQST29P~=e$#>Zt zk1YTbXYt+-10%29CeN62K-ybP^Xcd{&MmKST*(XkZM}e-7m^nG71j=ypvvPP+QZp zL#JZ-G9;G(CO~|}oKjzJidG7?kdB%b^8z`B{QPo43kU5H%PIa=En9UUrI!26uRTHT zYNi*z_kyrC#dQyeT(x~V|I?X$Cv>mfzWph=*jp+#`VyM&rGoMii3RBgfi{dpNn&LC5~49dCd0&Xve?S zN0xmjQ(S)Fe6w6+4qDMB`Z(`tcGDuNkvwJf0*^*|L4aK2R{Tw%;~I66@>>@ikRxu- zuI|m-RY8GrDL7_R2cM+#kQJyb-&`7yn+U%%H)T+1>EtLw1p5M3vj=X^ydxFUd&O3C7W3;A_7xBpD*=Du=zYyIcZ=^ubvZkGoU%CFoaWm81o-Eg z)D{2CWXQk!XW&azmlLS31n+^m-dE9tW@dYexusyp^Dr;BL-lsQ=J9d8;oexo$;w&p zBmf9hdWg&=1uV^4$uA&RmJ%p#{871`No!Ah$*6p?<0n+h%r1DtKToV60>MnHKB921 zF|L9e4Ov3pxPH?>g9YYGo4)>g(V3-5LhF~DoHnpl zMQLrW1uZ7(+j9u~>cK}KU3w6Gi)V_Pj&57#6Q)aJdh@z_FMv0{e)`$c=oCRgZ2@yd5%rfZz91z7IKw10tIqx4)qaf3BaP0pzk?}7wU%15lxas}> z^5ecvXJe+}AF_iq$F6Q_%Oh=X&$HJhwOgf_$k^7J;9eAXE^V~1+;vEOGO+69(Ylqp ztF$1Xg)a2I6=U}J+E)p)Uabw9aEf9|Wdg>l=JxC8V6*Mr+m8g&dmHl~GZ58Uj!%zo zgd6sbG?LbK^&Jw8t4XD0IH5wzViC`trBNy7J0k2W#|!ng2Z!W)6bQu?SDO;cf4SM= zIHw;5xu*j)MY)+&{74sLzEmZ3ikF#TjV95bm{rmjFF(Bf11_r#*JJ}x9=Gq{)FwY+ z-waAW=B1+@rRe?SNEEc!@FU|XV=NzUwRPA9H5=zlH?f*EXQw8f@XS@z0l1xQETd@& zhe!I@g!%L?20pHJFYniLZc_aqJ~~9Nw1#rS@T0%ke5O&!z15GuIswxz;ix>QY2wzi z2Ko7?hqe6P=}2C2xhl8ehvi#(Ne%yadX1DYLrJfZeLoGrjGJj}sPKvxy^X3PWwDL=YA`%<{yKea%cjy9>1R7bT7&wucjgw1 zXbFQLcF{E&DO|44SiDCLjHs&+KWM(+G^Mc0Kz`wneOr|7&VakZj(2@tw%%WZt+vfA zh*{Sd2!g0RvIu8^1~3T3m!z|krPD%cz@}F=7<^3EX!j3A#6*tM5 zfwI#(mAc+|9lz@Ur^p1F!riF(5&6CENg=K6Zq(l7rEJ7`G;~ers%?>|jTwB9N4t!ow$t zQOu}jG2^XJ)W>k-osa@@M`TA3FJ=g!r-9m=cu73;9+4O+_Q|5X20QjbTMT=CF7cj) zg>%P4>57hYjed$rVR&q{Gx;qVAAczj+yYCspZoc)2 z35&G6Q}W27x65ziR)p>%j$dte%9xyDb+X0`XBypQKw0BrBg&O@G7}QZ9Ubkw_{gK` zD{o_PhO2rev{o(WOAumKmITsVcim zygNmR!H`E?f@*qe^waBu2BYR1cqrd%tsd`YkXkSf!pCCjl)Y-aGW8<$+1c}~XBr$~ zR%#=35y#sBU#HY@6XdtKA^v&Flr%?}c|P$43X!yZ`Xt*^ex0zGB{se*k)OEbH+=O+C#CIriziVy7c!DkaS-(Cvw%ztg!olv5aTZrFrH6{$4Tt37uuT|gYnrtLST}Z z6xch#a&Dchkn}+lEu!jpK34u+Wg$_jmkZ3{1Mj|k=WZQ7Ox^7Ob5vj6!%vs zwKy2u!K6&XOVl^*TdFnmhKH_YCEHLY*;=Mw*vFy(76#IMm&=TC=hidB3P#$)Ly{LPCACXi=^MS1%!rU08I9ri4WGlfO2tG`up*w3&h zqY?wiHK4uB-u-x8wwsGYUa($XYLOgU+5Wfcj&V*_eaZvP`eH)?sqx{kNRgW3CHZ1L zmywJwrJ@GhKcmd2UnB%cTPIE45Xvxlc%@Xk=7-Wbf?sO3p5IaKx-oYI9kA12 zymLNir0=CS^7dX=;<0QH9Ay)-PD)9^KF=gxI$&J=KH#83dYqFfI1gU)(HY(I`Yht= zkpv`M&=KCj8pW(ow6Eq5W{`v7Tp2lM31DLc!!2g?`l0QUIipypq%LyYp3JL#=%|8M zR(hqmqT?8nZcGl0^QbK0wjkpz@a@yz#HB?yl>GFbNelaaAhthc&UVK!2BXN%#NHx1 z%2ycYN&&uZhZHY{qDLl#$xdAgDI;f9VT(33lQz&{I}2jhRaM&}VGSmTcd4|3&IId@ zp=?+b?$sd>Eqco-v_k{59_!7fzHha7_5#x~*&BkY8oc6Rhwl|SXgQ-U{mh;Er8U)h zCSLqbr$62SpEPDG8s0IpHq9;~SZ_AXa}9ksW|Zj%m1b^;y6hJRx!aXmnZXW?`zjG9 za(haxgr(08pVpWoxFT9Pb8t4>Uw}nSGj~*S8%aBokYeRdqstQq%AA*nI;0=}$BA%z zm-ZrCl666j#pUI>0ASiXb2dgxjX@u4D8I_^5Qn9p{Sq^1SG-$463(19u;tsf{{G7w z)D}vNlRrlkSiC@Wx4n8x`)YU-e(Oq6Y9F=CoUt&#Uy~4->DAJ43`LU3uGGVh$ties zq%6BYQ*tM*Ucz}uF*(X31s^k)qCR4gEOIJ>4E^m|pEH~mUZ2`=#r|niP+}NrNJD2u} zs6`3%WZG%HDzkR6du8#gao}8eEZ~w|-g6{JWd#%3p$Dl${rTfBY#s74huSjSXLdNZ zeCE#yUbA45-~ZBdz)g8)IPd~+6C~NXtnLaoB)(Fx2!4rlY&XUt(GcZ0vx7Dtrchzv zu6Dh1*fmW@AS(Knh2hZ@VhAG6%x#ierZCZ~(i-1@$#$eWDu^Eo*3(C5`t=me3JSu- z!i7hMr1!R@xG-x~3fF@zt{kGNkM!Hm3CHaj&b;Ph1}H6}Ygr(+GeMuT_3BQDaj zU=v#sS}A1Ivqb4hS)TqY#7l5SqiZmtCrq`kxfK>}yj}Kw5}81&!#oXni+VDm6uZXt zU_SM$_M11J`?S-K!H}(YL@cm1*ddLX3sV8vP>qzr&Vp(G3}aNfHr}}&9V{(pO~@Pg zs{Po?6!8tVedho&)g^54EBR(-v5`#OajbY4EnV7rc%MCp{+?%Xj%KcJ8E?^-_HpkK zUaEEJ!_T^o;r>k2e#Wy&vQ~f4mznW+;H}uFERhBacFc{m)M?0KAsH0EL@Vgt*zKtL{0%ohWBifaCCFow{v4mKa`|vk#f5zJQkY;I+a6pZrP590v5KtF&KHsa;!+?Q*CP)*!V+HJ zRz@y@J(WGujKVF9edgDD^<}D)sR6X)P84xNQ53FrLj>Gw19>uS-(}9n8LgslV`eYvPMHe8AQ&vrUQ$dGoM5u?BHV*!QNa!r2S$c z&w$SWd8R1eEun6Z{31@6_}AiX19rq;QNzB_Q)<)C)lua)_|2rGGo#y%5`mamE`7nM!)Pt2v4UieE=WNE55;qhX4?Kl5~Hs!(zw-qc6Ls9 zu@7*OX1VnD)ZS*v$Tge@Q4iut@fmou z!HpQsvwf)2(de*YdJuKi_aLGyRu*<~ZbvGwG-YS_6??Jl5Yr_6pmjYimg=Q7oq-}f0yN!!so3y(Wc*+0Ffk?Lr$HQt(m$N=PxMTQaD zHAw6I!sBQ{su{@6E4N@eB|V_?calUG4gwqV){F~|zY^?6a1B+PoNmm_8ytTGEt-V4 z8upFaJ22EgJFOr6K7LsrcBS8naoPaoJh%1;Tcud<=dliVUUAp1f-KubS`*3TgQ%37>u$t=0%6->g)uBPxjEk2!Qxr-`1I6bUtNTBg1*{ym-hK9^`RUN5?hK-)jDD8yb1MbHD&(nZx0Mg&koQw6 zgQXwew^lH~qQ>oJ*V7hL)Tn0BqDYjW&9)zcvr4fu{3Qn}*!IXgtG%}p>WhoXk_hwG zjaduKboosG7J;(z-RlZazZ@V2zf39bA3#R|WkrQ%FGAhNnsT zZ4)le8{wQNN-`}u<=5$_M-pdtByj|eN~d}%OI9Tjc$5{uXpkMSZl>d~IPMb1RO%IO{&wnV3r0Fh z|4PK#>kOZf%FTqr8^@Epy%47o*kX*Nz1zS^o;!lRUeu@I=%cqp=s1q(yNjYj>31g@ z2Dot&OQHhxQxaeR9(0sM0}+_7}q?$)&XPbN40|nQA^REbQ!P(?wZldSP`10xyXjAeE%bwNr@RHs7ZgOl&W0^!EK|S`Rp@ za_!TJBMM$S$5kk2z2rTxuH${m4(J^Yv}%|^b>XFJ_tXn(zaD6PYsJ;!HZm zC}b4(f<8_X!1@cv3oR84<=S)Fx)Azrv95i(we>>S>z3p9W@R7ekQ)`xsl_x#+h z)V#mG$pAC?MuAVZraH;-ws3Z8{fJV(d}x{pTf<~|kzuU{>A9Em0k0Wn-EuJ4^3+s( z$MdKyzQ^{;oMvuvy^Lm|gUni2f_nUanyFO3=(t!mLuN))jlFRZo&||2 zlUzhASjP|egVwJ14pF|n*_{rwmW22)R}kW0Nx&o3{gYrQlySAQBL^(A9(kT)B)!*! zRvLd02DMF6c;aDeM|SQwCuGr>XApJ35xRKTuOT598%zDd8OkIY@^Ga3l>UH=F!WFX zd4DeTy|!AN1gz}+Eo+oLA&;maCZ)9f>Xpl*I1%n-1Pb&yx^$|Vd70=TwXdA8?_$ED;U zUXI3H%}~iKNA7-swe})06H%V-o8rW2hAOvq9=9=N$MFIleQU2#t@v4|b^|J0PXx># zNq87LGOKjU(IUI<1Gb=Fk}Y0@_=@JT<=a35a(`%RmUDXJK%0RRby82UVi#|rHJ(V) zaA3clRCP=Fm~LZ@Qz8Kzv@kL*`KsAATyY877o=&@XXd~ZhovQ~E3TUFn>L8-me)T1 zO|96(}R~wWz)M_%3umL9ui6o8pz90w%LZy z3wR*hpj5yqF>~TsQY)<4*)HwwiLgaHwzr&+Uz8MbHX+rzvfDACPuxFj`{B_op#{_O zGGXcy)ky{!+&hV>l_AF2<#KVp-)`M>4G-a!WE&!ycR)Z9#9!+jf>7-SLhJ-uWb45~ z+MZM+>3;5QI;Yv5x+Iv;JV8P}x;Uv0d(-OjMu`^O=YIjN-rXMP|C1RF<=Ja5ZTa@Q zG$UlmB$QmY_@xs*R_ie;a@K^V7zq5MlTk+YhIT zOp~k&?G&=%{!DUt$UWN?>kfa)BR*tG@qjW8m>gEw?_uVBsah}NO*x1^Wtl05glcwi zpnH`iMw(9+5ot=ebJWr2R05o;E(vzxRmp6kGKU(lB6B$XkEz*VrXt>J)=q8~9lCiu zQ$BcT00E{GwmcegGjvaIZZim*$0nh0)BgM^w98O!K-khL1zF`n;cxZVr$XyA#@tmD z@?IPT)LU9DG|L$wd z(P&-Y<9Fg46ztrTfgEohbfaFa8yTpyRqa4oO*yNCZ_{Z%9h%*m#tjUqyhMY2sA;ay zI2p{@gC*F;q-T$V@XcXI)9}bwIqJ^p%KL14FojZYhQM|axAM>1uoTC5zyzomEywG_ zOFIy=kj=6h+Lo%lsweibopwvqi+6~-l4CFI2Gro?g)<`zZxHZzpK%{MP2$^rX5s*j9mi=I84<(R|~CYY9fMg>!M5YrXGlfo(8AQlk7r z%%`{SM8P{>B9q>kphpnzsgdO!%GwF zuXaDJZmmkMBPHJS=`&7Y&JwjtfZ*_=3HQ<*>)kud90^~N=Z>kqudts=#JAe`#q_^- zjBQn%&hD_ZCUmw5`Q9LmdM~DqKOZ+MDvHlXG}@%=eVq-;%QjE1oN4+{c2-z%9yRbN z2n-{>^cv%wt(l7}9QVu}4ir?uC%9>N;CLvLL9J9|qt7{*gNVKe)gYf;CnzpUZPw59 z>d2^DW70sG^^?P(UI_-6M&mxX9Z_%-_O;|+wBF{~IP7vwoO;Fnd3Ka@Q)2Bbn=jR& z06(`+=ZfG?*~Pb2x5mk{l#=Q;S4y@+sH1rie#WI^K6PzXwM<3Bcw4XQ2W?`TC6re6 zIb40dV&tKx)??j;$IQU=M~P5*0gqQcZDX@F+f#&{^|k`+en{QOnjjeGzCxZ2^0tQ5 zaVH0@&$Cn#F_(wOi-J2nLLBf=I)H%`Oq;;;`P{-)3+RU|#fc(*P`9_HN*Z{wME}P` zq(<~?hLdQyzQ&L`xouY4cKsZla#Xp0&?8f~0)g0nc#<4Qu1au0-_m_4`?53yQlDrL z9(aRgnu>ra;JF*v>5HPvq0}EPo2zH<;bX@x8k5wwG&))GW!CWMBBLh)lur%Mh-YNFL0K{b6bbp%vq0`yVo;nnJLw&!_VLNP{|vn6rzUHlXEEwCf+UO z!&SGrxrT<=Ne)L^t!&kq!(OM9T;$dOU&c zf|*Fr$lm1Yc{6LDP5G1U)6w2_>jLn!+Qyd`iB_aH2&g#z%lHe^77w4uZ`7t>BleiG z`fEO3^en}z9v$iXkp1-e33Zt$J*an$Tl3C0X%lwvpXk<>D0CrQeyo2}RCLhwPaWw) zsVI8baC#BW<)hP>>1)M+S64{Fo2gDXtiyP9QIyri$Vfe$lH{bQ?RwXYb@XHRtvl4A z3mYKv_Y1Jxj&DjEYjzRz-EZ5`)iU70@^3~nh&K}`cGHgt>&ga5`d?p_QI8>0+85Pz ziC#_zC%tRykdwM`2*=J!6E`DQeJ779rrxKH#8Fq(ozk}-0f&bHJDXt_nxDKON}&cVee z0MYc-Y~kVxO;fN~&|ST^@{kR!0iG@tk4!csa)RL z72WjG&CBdDp5tYLhD1zxSQ&9LX?#)l!TU&0ySWl;AFbImT1pb;oS5Ib0S)3%^GFdL zN{?UQ@GPKL64!NOL!fwtip_csUUj^~a}TAUc^-~48ie^cl+6zRhxX>ku3;OI2r&{& z!C~XB<}pEaVHtOFTfBIL=XVX{h$ILokq#q?4!E;cx&>TP{;?1|Ir=G#` zt!+ZF-j(Z+X?flzW<=EDL+=u4=u|sAJ7AmUqYcNBzt1n`-eM#SYY~Lf9(Ip*%=wT? zE(8&xzmIFri$$~(?F$j?FZ<`!S(3(5yeXw)&H1l6EwjHd%lj+@4J zU{er;TwL;bY*KF7PBeVRX8n=xMP*mz7^VCg>ONF+dgcP+k{NW8+HvJJB986CsN?#E zqJ^phpTEvtQDR60FcQU{E~bc6y0*{tdl685+nP0+TAX|~wk8wCygF+J(kY#48sz|6 zCUWh$Mr`|3T|UPB0wfcBVbAe_un901tfUrEDatmKDUws^K;&CD$3{1Um%*o zvUBrm>dd^a&P0{2huF7<&1VW}s!te+m|CE&1ZA5t zj7DZWA;;s$e`l-b*2K>XA_jB`I8g#46F=iRLou-@=@+9?98wdHU>A${n% z4KaNv5wOlg25PP_Qm^!~E28tLL0zoFtEWD}-{J1n%WpvL%N^4Utak% z5fDkAzC%obAvzA<@KqEuE}Ed&mmbda{x;Q9t4S(C1tKVgQ2Mipb}Hr~zkAl&OO96U zqMCL^8uu^t%pJteNZ_IORfzh>GjZNMgzsJC5c&k=I^%_G`;q%)lWrV^8FSAmmRvESl-?Cg6I&@2CEy-?n)G&_H4Z%Xg&;(bCkEu1?0AWio~*8+)q z14@N9beX0sdcl6BJ%#2LAdX2$6$B zr_!LS6+>yhbEEHaVfR!mR;0GGFh4HXQ#ngoLGb1|=GUL~w?9v!G+^=E1v!NLs~vs> zb?EY3fRt%bUF!NF!aFXtavn+L4k{lcs0A9|!X5hbY0E|yF88OEaeV zt|{o*1{D*VZx9^bK=39Elzb0U%l^MN<$v|_TQ$$%%>vs6=cy?gqD6~ce$8kzN=ejO zE5taay7L`CC5cNoO0TtO;nM4G zRgLNK;cLjEBd4l!O|}bYR*^>|hV$7PJbw`NqXjfOPFJ+9VTN75)FgDzm#txMQ|9Sa z3t7b|dea&JQGQw9=LNaj*TiT+3}h+ZPS6bp^u_sHr9$JZ46SyDTX_Wo>2@Amr#36g z5H6xcpprIrmm}vF&u)|qsY}ft&pRrr4v*0=H-=@&zD2Y|o~@12_c#RcZOVOdl%*v) z`al!PwlbSm+QxEmt#=79UpF`wN~w{90){tWNISCwrfMu~<2o+vF*Sny8&58eVgr-01zppY6DYUjY;?f27$AQmynYFyNuL;mwFndy~G(hkPYB7 z$}NFZ>&s3Hn(vOH)zrgPwzR8NvkHVmC;F<(-;7rqd`Dcp_3y@Ov80PvfGYa0>$DdpM=tj2-ry2v~g!qI&z%GJA04Qm?Uz> z)ZxtP%h}K{@OJnMm!1EvUvTHh-aDg{LFZtV*UFD}-eiwiZxEThW$^ftUY#%@_j)dH ztXbDP)ZVlqY2dQrm0HQeSJ;24wYc{Rf!6f!+#CI5^*_MmBO5EISr={reCnx^@ZMXDz1=)}=aRyw$yy!4YY})Frx-NyKk6 zU$PQ0V7Pdf16F_6<EXrTqet=w`^ultvJQP`S(Lfj2um1~ zW%n&EC}M>56Gamgeeho1rLXfn=9)Iimd}3c-_0ShsiLtuHBwK)RetstA+mG47Gz3sZHnjjkO1DI>3i_f zIs9Diok__$QoIB#nDnsbB-g3kp&+eS4?mUm?bw-j(o93iWBc9vSX!+LQ(POg_nBEx_y@tkasjKmVmkFiFsZ;AW)+ zpVrw)ud1HCmtNO#wBup@LOpoTef>n%9>4_zG>6%1Bv}b)g_0}4_|gUPQIAs7lr1ZQ1k1vfzso1q(F!HC+E-Hj;P zy>11S5Bc5-L42@uE;tZSdc24jd(3Y@d##6DrI!rclf|QksTJNDvy?kO?}Y`s@6OP; zgN_bS7cQD18EU1w%#E$7^NfAhUi=;x%A%VhX4Rm_MUJ&YD|k$kJS-=d`mr=b{Ayhb z+mSYG$>^yEC?2PRF+tR>&Y7Q#3z+77hXOe>mu9q9XNNkUfY6uXIE|^-6HgP6smNAc z=dGFx6{PF{gsk_F?@3%R*cOeib8@ouIzxo;Di9=1u)EKHE0#R1ZlrewSdHzW_!dW1 zPnQQMoAM2<1R;mqnOJag5^JOa4qXExe^7b)uOb)X2Yk^x;1tRpLqLoKb$j6H*cq+G z65W*bsKsOtN$6gg&5$qA#Wmw9LJk_9Uy%4fb61o3#t;T2xb^0wKcR3&2K;DJbtW0p zT};0-AtkGO%jwXJ*CKav;ZQ8t0Ra`GQE%==k_{jr2fWKrU8;}2DEscwc}A_AAF0MeVFO!1^J|cG=AQ@_+^KFIXEZ?lL9lY| zVDlCA@Eb};?(f+Nb_39U*PK4otzERER-;^Zv_|X9uJP$aQ}tc-2PYgO5iq6MZ(C+C zbXjrQiJZ~z(1-({n#c9~Z3?`0j4r1+{2ss49P$w#A`3%6BmgV!c)b_La&MJ0iy>UW z>P4HX4*2Cf^LE{TxPtA1-PHFwltT_fm%k~T;L>8!Ej~`I{zCn6RF(8}3gACYbG`U? zL7Hv3vQ@-3v4t{*uyaCmAq_9SfWFh4T0R`{Kz1BOw+K)RKcy;jMf#lJ1KAPAw`%hz z`BsK;y6-N9k|nYUVzY&#A0zN3UR->8Aa|znH{K8D7fZ}`?Kh0v9K1N?vE{b%RdDLg zM_Y~i&EYtM2)>U)*Ks%mOOObu4+jWxbV%O48k(*4OH*oz4YRa(LyJ5ag!|Rz^C9{Oc(3aRQ9~6Iv)?hM2h&sTXa27q0p4 zV=-N^NRDaJx!wwH>7s9rxA>a;_w_*rRts00&8(%$v#F-ms@8BHI%>AXcbHxD~yKY>!~#o?zF zDEnG=qdyk|!rMNJnQSd3ASu7@ij-&eD}@|Xf~UIbv+`yril{kmAQL^i*( z(VF>Ijgob_dwmvCFrj6KLJ|#<{=pYvgxg(jzbf927Seb`u#z7;yu$oTM=UFPI)7EZQigIwvA$yaq@c ztj6#-B2GcpyQeMHh7VwzPJKxQ6)ep7N8|g9X+!9?S_*x9t^{RXbx7(u>dTUmJs_No9`NG&yD*2aPz1PvaLD<%j+-M6z4f5;^YN+T|>vWL(pD*ss)bZ7fE~eDM-8~5bV|)Vu#r@YSsFp5J>2V8q8q zfC;a%$4#1?RrNw^$M@~_Pj2Tu`QGoC|-Idw7*6%vya@6BNt>h7XLPQhlUuASp> zvU*}p>1z#WEg0*#(?5pu%d?jn-(udy@S77`Kkm&K_g1*+%;N@s%cT|$53f-)pSwRn z=PyXf4!nBOxA+_vo}Ps*u{B4L&W%O%hvR=217{AwyxNQI%$k(MSVyL7P3y5)3p^TQ z8LlXP2js;gGmK<0@l_9}le*;LT>gxUj<;`4_?8>N z+_1$D)g>px%!FanDD&%9Ph#D=+Mg5o@4pYZ2)Q3juM3o2BMYD0)_qczKGYeo+dJZM zPBb=Uu|Xq$*UZ{M6%NkL5AwxmWiyHcv(!XDc?A>SshD%E1p^jQ{DN0KhkRXeQZgLc z;@-!6MzC>rZ-Je?wnR>z)k@F!5rSN8IpoVeh_9t2%Z*jFK(slwY}sOZiaP&uMl=X{ z?6}Mi2mkr<6{&>kp|89mr%Esm_>j7JOH(b%)071Yr9wBMU1~L?-xdz=mZZTi*k8eew50lGmR9Xg!3%y z9=a3AVf4p~NS1r6PytY!40i(-v%RzzJN0G^6_s>AmCHc7nw^Nn4{fEIiXuKnM8 zBV=VXEN6KAH%7gLdw-UTZ6GYn#=mRN^`7&$QZuYS zXkc4>dG+^}TEGDQCGA)>0HM(0Z{NPXR8>d1A~ZeR!`(UVwFe0>{>3IWvw>Q0k8Vya zVmIPNd!jm=N@o=Bn>rNk3-|kJyV_^2G9zHqc5yAme`s|XMR)R;l%Md=Qq`Q*X0Dlm zB(&_NRJM2k+XapO_Ps34zY3Z^a`Y49@Amr-u-SEq>=FBY!Lb#YC~y%|H`IP1TB2EOIjh|08kQUZJn z$g*l`F8%RmOTyDI`aTw?@8966QD*a;<*}tUkkX&yvMfCrGzC&}EJ=0*9g%;%j^zm` z*9HNdXuQ1tZ*m80xX<=zEQaMPtFRe74&KOLASSy_b9htj%|HI;4%1dOw%_|7zLBm%cm||KvK&NZ`@Pa@7AR*F7>`^z`h~HUDIe7)L~{6Er7xijDUl`9m!g>C9f- zGH2T-K4r%u>NF{k?q=?2H2>qTm&tv47Z5QoEF^3#WQK3x&o?SDB`O{W(Eq>nM!w|= z)zzNg6{_R6egM&w`P|(QvdYh=z^x-HPI-FEcF1BSpqL9x4}N}R{EY}6uIJe+!r$yj z&hqFKt5s#(r8^AU{L#;fd&PaSKCt;7n)#=YndPB@Tp{mB9_m50eD|7((L%(V?WlgglyksFKqlD7U^m_6 zE}$?spjv+{^}kpZ5bL0&JIQdbNhAO_+IHx|>ObDt;Z(0Lv5&Y!#g_+v}` zu7?)opD{mn)4k^32b8Fb)53oQX$5!yN8Hg~v4cg5k3rqodfMe3FU!4EbJaNupeHvB zKhB^%;Vm^Nn9$BTwm2a{SZ@_YR5X%?iu}=6O?U47>z-XlNaoX8r)F2j14`(e)Dm>U z>@HT3YJLXzxgft&2+}k?M^h!5Q<&LbDHS?@ojvB8 zm6jiCvn!H|%iw%{($dE-*Nq8H7r3B*RXA$vJpQO4KLvO?^N)!L>u;A0Q9$P*7_Pf5 z)l)A$&ZpSvYVX+?PS~7Gf1&s|cFUDN@+kC=wHLP^zib})!;axrpbB6!R`8a^Ry2}STEv1BZP6sM9nD<^Vx)KZ`YmmNB&geg=>YI|LOC`g&g@m?R|MXlWD z_tW}0=d*p6*YB_I>Ge7<^E`7u&vjq-wY;z8zUPpayob@Bq1!(A$*=nfpGy7?(0@14 zO}_%NZ=VsW%MPw^8&;5W{$Rjhu^r_kY#R}^`%{tp{P^PDz--@=jb8G$ZQ)wr4@vaGTwNJ z(mXf*xu^J76pD1uDR7CGzAbi;JwL+|&(*7L-{5qj>hY|d$=2433=8>Gu90g6dj_Kf`y(w%8+s2oIcUVp=SKV3F_Ookr0IMbXk3BNh zNx~N)f2eSKh@|n{C)aGxnY&!aPQGigFVw%9&XQAJ$%F>v5TY| zxAEJ*nB`px8ygj;-eWB-r82Y`u{Sb)ePVQSScUiD#S0l`yC9yrnNisKeHoyj2r5L8 z6tw->{eS17g(xUNlVm9SCAQtRt;&|(53~jqhgVm=*csL;E9C#sJ*&hwpmD~`-_hg< z#vMFyH-h3e#w?Eue($}#bbzae&5`!T!h3OJM%*qa{oXuSyy_>b_6`SBFz-tF zq1xXC9?~8NMG+20LVZ7o@V*5C7qvv%Q6dVWm?(@%-z*edNZv0o9qmQ;e^3c(o-eD(Yp^ z%;T^uThc!7p9J&E*SVPk{scPhSdZDP`7WX>)?AAKnLb#bocoWt0*xHI)C2$g?r@ZA zajOsR{%O#2BOKGyXFqZD2gSJSz*`roS3y4)LeX?i0Rk0fZ@wS#gCG}F(3<;|>Bs0F zFAtD|Tv@+eLFla?*D$U;`P=u79FqIjRCV5Sci*3!5abq?s~(CztNeH7>l;tYb05nn zd0k@u_WAqsp&0C;=>E&iP?sr#y zss0VbaZDMs{bbdiBmxfFZrc_0la2W<|4%Zp*f3Yo&rQCNbJbroQ}trG!)m7u<60v{ zZ)SNa#R|J`24%hfI{_R5DdPOQE8k!4f&?k?skH77LBdpspB={BiP}y6A^G7xpsSsG zmofd&vG0#r^Y%C>pwip^2cO&%{rL!WN9iRW{CN2nOyI;z%Rb^;QL+f;13)mRi0Y?eC3K!)7`d&Zx?jxB|I;xnnm$%5?eD`eDPhNPo7k`H^vZQ zfZN(s();L*G!(7|yjhbmn22w)AuM0&Fl0Y|Ac=4J3WE6SvYw>#e{;h^`c=Q&?Oq=! z@p}yP8&=4bhKlaZ4y3yjA1{QL zj9s~Vx6*W6%3A)=30@wa0L_*2Q&i2QSj>MLa^lBeA2W2&MrZ3Xq<639;*1>f6}hEm z=~2^NMv2}<#+XvNvV~f@whKc=`}yy7pSHN+_B*_enZv-G<;~LEU_Db4%4S|*yi(j) z_Hb!E^H~9MVMy6lAG`jRMoE@Sgs3~8ChFW8Zkr*NLUAlN7C{I{BP0@ss9$%*49>8QDJpvUY-&W z5nt60yH>)AVHg!il1}=kcg{I#v2ySH6ti z(zW4D!(C0Etk$d_8-@PHpyI8Hjl6A#WW<4iZ?$o>+OMkoN$_rWzf;4+!CuM5`lC^I z8vCAv-eC%e_FwX|6+>c=+rFZOdbS~ti6A}b@R}wbS&IaXI6O@E;aV4wU?3V~mB6;U zx3qMHcb zTn}f1d@Ff9nQb{yc}wk3RnO&u0_RVIcOoJZ@IpZqZ{I66ni!`CCf(}C&q)l{97|mq zL$mr&(^ZUE6rKT3{u&`!NCCIbbmm8I4=9plK+htDbS96jqHLjO#P4R@)+YAl7dL(E z8C%u&P8=KRxQrp*H6fYWy_SEy+Wy2TLnE^d{VqukW7gAW{2ZmLx+*`jOE zW02ir?cuTeW=Qol`*&C>{aU)ew@j8K(F8r~IZW1YBlcHlITX_>4h?=-tEDh=l#!Pr z*I)OM61H%{^Ya7ocLUN_;;DHhyw7f_$>ZF-SMRR)FiDBG&C1Tclyoa;)-I`6_vK52 zjkoga&~*$Tze z`6N#1uHWWxR+psIez|Pj_y1IVp@{URsiNLpC zEVkVy{8-a|_F1f`SjmAO3!)5n0}r^(1AInjbUmM{XFbuFKI80_yKQ?KA3J-5rOPji zz4z|8W(_49DGO`M*ntL}$uoGeNEeEXcW@*DwAZ z)QsN;wCwYMMi{czcYT&IvV0`A(tU8W@!8fgikM=zBUMkHY7IuZz z5DP+gH!feCGmW$Dc?l!KcfgV>x|;G(n3|#*yyn9`IL8dt_pCa+)y6it@S2;YGo>U( zTh8j;84sJ?9P47Jy!7kUNlAFh**Hmg`7OPUD&q&8trBL(k_Z&P(TK-dx=W<_uvv+W zuN*lJX7jiZ5mVP=Vqm?zS3QLNzX9W%(;tJ!)+{aB`BXk=bSRs6f!cw>e0#Nq*!Grv zI{9YH-mW&zz{Or8s!P0Pkec!|x znxRw7as~6s>GKqlg1LgZw#C67tu**U!TkLUx)!DoMn1oNsn6AqU+#0}PDa1WXT!n3 z;O&0@;FG?<3Ca5J5*7F@0HRA}L zzA5QJSJ_6A53#HCta|=pzSBhgQcc(O0@=N0k8+B*TU(_s%)QH+DXu5qg!6S^T`kst z0<+<}u$mqLY*dn?n%iaM3B=f3<+bL^coGPX@^#m6(ed6%7{d>aK8$7-zcdo!{fGJ^B-)fl$X61p50mByzpDteQq;g~iv+S4}P zqTMEQB|T{uXeQ#pbi_2xc*yYu(d0(L^*2G#U_7X!2I`Hi1tNAx&IYVGH@TKp25ODuJW z$yH>zs!q@5T6bxmqD+j0VHXOM0;IrU+eNSU#gKs|e5kzPj+Ip2sqhDd_1Y-!QRlIw zZBC>9!kADpo;Q5MlTDL&^f`?<6mV}~(f4DR-`+=4%w;?w)dY9Jdjovo&u+OGh&e6F zNsj8d%~72kcyLB0W~@9s&MBWqm{rR@2MsH;?y0tR$z!ug(OQsu5I;On010< z+RkJXWaai{4IiP&E>Jhdlj5hG7aq;|H4e%*Hp(y>W*aAFceG3>F7ChG+cFS;XXXli za*jMVBXOI!mK8Jf>(XP8r@%b}_kHt(YE20=O69J-O63z1+Z*3HJbf{aKr!g5KlABr z_67&8?IE(;s>z|f67mPOB&VbpiB$C*Eh&2VN>5HOj@mW;cATBDS7ue*{$m-`e=TUs z*2>MkOGNu1d_I3zJFZ1d9G_<}LO;@arhMPnY5BC@ZsxfV=Qa7{_n`&-+FL`=lIX5C zov`%rLQA95qSpYabfQsgu6cCu&d6*J{D0w^j7JI2j=}iSkZIE5OrBrHj`+yQ^JZT( zZSwqfx(X7WH@@4yDB&M|rK={bd7$oUQ4Z$5%l`;;_yTGq2!joAG1}iOyzD;5iIW=a zHA_*Yxux1&e+UI#i4H4Dd;v}_#(Zy8xpqQHA3l9&99Lsl*#W6hw-(P>1A{f=!OG*8 zsapN24<`+po&<(>-k3?QD}ca);L;fVxZcZuhUOS%;8Orj3>RxU8NCExs}^-$*>M07=X>)k+B^ECjifu3RJW;QszwLy>du zNm>`(H=)jr;sT7YxP8ON^;tpgn>j}X5~HUbkiG$)}1bvR8m(=G5n6q@)ddD6HYrdOh%y( zC0zKi5*b&F!c$rRY`SCLW8LF!d+&C;I}O0E*RElE5{Iz%p<0))mEp$@`ar~F2~NKI zB0+f6*2btYl`$msDtK8FjT+wS~U~HW2Cp>zi#ZgYZc8|bl z6i3+0#+Q+Lk9(U4sk+jYNRSc?&eDtt**0IIn71ohWEtYc^yFWGh1UQx>Wm#LoP7w( z5%52|CH?X?Y{2-~Iq#o+Qdz}W@NT=~k?|%!j~TB|(^u?=ol_D_Cnm&vnxfRMQQ674 z=?t%WuDvL*j0i~Z@V#bzMIhC)itk3R=J5``{6)vMFm6JKWs;{xH6OX;Wj^I1QcxHc5PlY6)^mB<=o%OsYTPtQ~2R0nZypBb^Ze_Jv z=P&ar=Hh9mJKcpw6RS1rFW5U6?|54{dwM!Vl&+*rk|2lUsGyip`#y z7~$i1Ht^hh)l&<4)zO!9OAC<%M%Gp&z-Kn&$284150S&x0)aIk=Q?9GPA7SgE^C4Y zfJtP5@4Y>OZhXpV2t}dn5ZQZB2e~&nus1`d+gpp#_EBu;xDIp%NTsW**Cv>}V(pjO z;XD|xWVxS%r8?(PUU6-s=9>hMtI5u`iDaeULI-PdbXes_+^Wv$upV+KG}i815K?&A z`{JplrXVNhuw}+bLv}vj7!T=ei0?VfhLnJaf1XI_P9kOwtp^fkAx3DS`n84;5&Yh= zjG;-ZeP%@00+J_pB4#gS>_D5=`1c^9BX8xW#lxiEsUTWs4HdJ%N#pdeb=X79kMvu| ze+j>F=xBKjUUPox>C>lpkuA!(2CSE!J}q$;JH=jBO7zPj&T+ehY>f2$z<~=HbmJKU z*?MYvx!`@zr%!4IVv*^-Q86*=s~tRCLa4ZEt~}!Ij!>`D?)fFwI-L}Xu86w>Y;^@1 zxUb{HqSCtady4Tio`!Z;ltkGD{$yr}bUaJO0b&vhT|~1?T7HV@*L#%z%B@H@YB4K+ zb>!7OxAKQHRV0Mi!`iPaNQARrJTx8^AK$V0uzhT7Qhz^Q+-dY?%hS;#_3S(yV4>0$ z8FiCxQPJ%Rsq7>ZBhxNc7Wd5tX)D7+s+FQ*9B9^r4;Oe<v2*a>MA*d35IgyGNZP{G1t%i=fAQzID0wSCA5YOu z`yycJ4a&1a#e}rMWwl$ ztnQ|a8yFd7je4}4hh$4jrK);dqYt zjEqqJF{C@{eI+e|Xp%-~8mnl@ohTb*2eEexRVqu}7ftFeX5XF@pDq0JVKpOucQ)|n zt8LZ{WNSER-FDl{{``?cLB3Y}DY~M1%*UMd6SynvSeGF?#l0`&v5scMw^@i}d(g{E}}dBx@ExspeDoAvZ->1AyF0qEAgGiDZoz`W+yO&Jf8*4Y}T(G-$`19A^>ZgEKhUu@%=m)(aH(@>UNs7eAJ zAglmjuvu~W{O9=jx#HqiVuO3dvIAYbTniHhxVL3A0|%p5g#SPK|T?S2NC5fjscEzXiq048ecSm z!ZL0%=6tFSH?KlBA#pTB{#TdwM?l2FwA!sxVERabyVed9^8zMg*N zvuE~<3ylpL!GsucP_N@rXZ44kQe*S%OQi+I+7t4ek++qS@;JAVR6<6oJ^Eho8ra9q z8{sKlwn>^?s=Q<1zZjDv9@6$7E?DE!PRcF>-8(!&g}q*2&1GPv^33{TR??~6tvA!* z+W)S>=yXaiS7lapmWfv@hU8^_jvhsqG-W3Ta|?d@^h!=?saeeQrnB!{B??uhgrwoxO=RNXcpynkq=D1Rb z)m|4jaT~%hP<~bxXQ+u1*IehTln#jMM#ZvD2$GK3mrDD|$8m*mbBFwqP?IQ8#jdPM zBo5Qh>GVlR%JU^BtI^Bs3nTDnQ}PNpMf%kk@#8&xFFYM&nwzSX;i`$+_xbzG!iL-qT4;^=+??sWZ_WWFtwHE8j zEmy1S%j;*o8DqVUisP3ra8)Xddp_Gr9V!eFxL4GtHE(1cB@{&Jy?!)QrO)xV5d)+A zt;&kvw=MHc);inXA9zj}F}qZ8%6%}&?7G@%F;Q_b2Oql+DvvI7Zh3}fhu?!$KOE1( z$ojIao!}*}Lt_ffH!LSrc2e$G+~} zZrTk2p2ALe1h&Z<*&b?x?NQor=d(LB7FH>L4$^WFeUSvhl_O-vYy&RWCo~~pdMuR6 zuc4)Nsi;GHjfwS9=*k2lQJ^lC<;n(y<@VCukTKnXV7%xKw`Bniga@xr6+@N#3So zHHI{FrU$wLC7+b;P)CN{9ayBNFufsDm-G|r|GitZHIsa}5X(gF4I!3yrvseatN46+ z8{{Q&nmw`LBiXC)UQzAyirxW7ob&4Y+#Df;)e{|j5^Hotx1^h!nVC@*B}a`oNV@iP zn?6mI@!`-bWivCy_^8;<%{M{@s}>)dUJtossyrKNpLN#lWNfK+9xY%y3ixY**l7hacKP;P9m1Q zBaJWgNKy@vXpYeHXIgQ6&G0r79A~t1v_#4^Z%2Kf-6IeCG!C0&nK{^W3i9^>>_H0r z1axETtb%=Y9WnqI#pnFqBG13GC)`Bso=p~_{rT7xr0-4} zt6+D9EbC4zAo35o(osD1BKtF6iCL&`{5>)Z85#jlDXk}JoD^qz`<`}FDNGDlq8BFgJ zlt(EfFM~m^t=6?n1lF0Vb9h#!>d4;38iC)-fd z6>GmRvbOp+;4go86ey3#^Yrdt3qDA}u@M1P7zP~wE#(N1b`6VSkL7%jLU!W6e(R2Z z`E8g8Z#6mCh);qERwN1PIgL8hl_vsCCyYs3hmw0jsT=Aww1SJ2wUdOWrDSE<)dVY# zX>C{3rukBI*FBJIHjWN4q^KqF>h(FEAQI`>^>rHhB-b-?_S7v~b_-Sl=3P+F?W&IG zi;wAIJt!tBsz>q*p-NVpMz`~g3v4Bw2~inocjS$QaO!_<41j0<%(^799duZI->6)0 z)paXeia|(1qLusVdo_u4$0Ib>yW6AgfjXxL&CT}3ytgCu(|pfv@gAzK!S>_t zlLH6s8m!E4^d#1ik-u7n`K!vqEvsP#SuxxmM62nHNB#-}ky07Ts4t{E2Mb+Z66+TJ z#jp!*%Trc{TdfgX6}DSYuyEvLpW~4Sl9CC!BA=x~25wrja#j~uze_a{c3R6*y$Ylf zaMw_Q1HFPEp`ptna=?tO^4L}NOpacaWh117~tryw<Kc>aZ&HFi=+wzf9eD^5ab<{>wy|w4@-TxuOvw!mQc#Cl zfFFjEdJeqxh8e(~ErgPCG#MzXw1A`h){Ym#eZ|y_zzDZ@Jb{ zq=WC=no7^jDbiifd0KEacnnZKh< zuOnY5D8=e>+Dxjhh^4b#@od|K>jQyR_qT<$Pxtja=DQ2-C@MBKrxcisv#3M%%{#k0 z^i_p4FOZ3fI78%(C=>kMV%4B_JMcCW!YGvo?FSwB<)SdLj zEHC|g16K~YkDQ8%kLmc-|8nf*lBXP`gvAEk?CcIziM=k6wT(WcYwpEt zu~*b-EdFXqz3^4 zORuIHw3E<@BnX-=D&=RkT1DA;cqEVb>eu$?JbF~D+}zoDR!C2okaGRt_4ce29qx`h z19OTCBfikQ3uyk@)jR{x zZVoh;xjMZ?G)Dl{ zmXT8CrWRe{I&d0Ds1>R_z?FAMH*=eTf1?B?e64QF!Wz$F_~JiX<8KMvzmtY=-v9?@ zh&qUtc3!7D%3nar<-kpju$kgwywKUQW^LKLnywJ5aN*=_F=0h>+MO#Y1;&KV7eb?F zTRf|Vt;!(T$TnsuYh#mKsY|)JuvPL)va<*ZR_6%V`DpzSB=BVS%F{j^*&^E zF4ZG`V3-9p{D=lo2ohCnP0(5K@NBk;xIVK^y$b8`>T|lR(r}0b(&Pu_(cZS&*q<9f z_;@w)1VH%wariM&Y$gbwMFty6BOC+1VL(xQu`~fFdJAD^?ZgKjS!Ajp|J`q!6?zlt z7X-1Nc0KyAEy7XtX0Rmp6kUup&N3}M?3LvL@~9uyT53wAj@Hz%5%5lE-I zFlHqiUT%q=k_r)i>RDwP@m1QwQR~=&uB-6$6-@xOlicB#-q43tpx5gO%{0VMen5t% zMz+VwFuVHihp=i{@Gra1y)EQTf@fWQL!g!OaMn?>M;U<HLh52ud z+GUEYV!%IZ)>|8rEI@96qy9t)_4I5&+Pw2d!VGLJW?|j+Xb@Q5lqw zThu=m?NpGvsiM8zNQjd~wxDkXk`exWs(@F*qep*3Xv_|bM$=)*S?t3_@a?N8@6FJ+ z!u4Q9GbIzT4(b13NYKuGIZRDo%5P3oN$q)~)0vx}^{W|}n`ejF*mxh-IB_dT@-PW? zAwcgT%w`qqkO%fAHF(pl=6gCAaTY+_w{r(5Ko8-N*kO#ge?J7;X_#5~zc!B8b2Yv^ zV7_46*)QZ6%vq_zP~j z`+2s%cU$(|VYF=gEU<>(T4?Wr{me2y(uP$!IYBi=>ux9vm|9pYyPuz*pOaJcFmdp0 z)lnhNtAa}$9&$Zv(W7zvC*Tp$Hp=GcgM|UeY|g1JEFsbaraZ|`dJ^0276c`T_3Ev@ z*jxYo5F8PCN2opwiL2atW&}KUdR1|B4KgrrdMiOk$1rD2n}d(BLU(r`yNHNSPaV$89a4^ z$8QWgDHu?9!c+rXY1Y@%AaDi1R&=&2X7QZhTVKyrg5xYxNw1>!(tdp|nq$pGG06%9 z6@(t6%9@&E>VIH`GEOLzt+}KW@MTxvlUN|OSh(2|TWo-N!(Y#h3qJ%@x#}cWVpM@K zZDWpB$Eb3J_YEX#1Y=Y|yqL9GDiWym#|;2gbtmp@U_Y}5UT){?EL@dYzb%aWV91d~ z#O~@}%+Oo-!@c0q6FO(G?GtdWhIiYt8#>sdC(xr)o-?5jcbVbQU$>Uw2EbR>X(cSf z-m@CMwOUmZgS2(vmhQwWG4tcOUP|~sgvJBtl1SKhXlqE4m3vC!l`Crl(JOHl89-ca zwB|)@lr-SpBrGBIt$`BTlrcp9vK!-HUX0=jSlaP=t!~}=2ZFya!GFhp(>$7`WbwEE E2RE77i2wiq literal 0 HcmV?d00001 diff --git a/docs/zh/26-tdinternal/slow_log2.png b/docs/zh/26-tdinternal/slow_log2.png new file mode 100644 index 0000000000000000000000000000000000000000..1f8481764866cd2c9620c4250e2aff41bd0a0254 GIT binary patch literal 47342 zcmeFZcUV)~w>FB33O0%$(h;PIfQSf)6qP1b1VOqYy(IL|5m1pX(xf-(B#4v%As`)v z(2JA=q=Xs*QX(brt%SYLch7Tv?YaNm=bk^dd(EtwHOCy|9b>#>tQo?dXsKL0$8wH> zg5u(1RmGY)rz#IAU_ES06ck((j};&2d6}&u9V!`z zlku`I!D|Aa28q#xdZ=S%9F0!x7<%?u5LLc$$cvEn%5!!PBAHYfp2R5((1|gr-aE&~ zu6pOeo!}JOAS#x)=$z^K$Uz+udyM#ij>6`=JB0hjC~~ZR!91dN$E2!6{~7d|1Jno+ z;CA6`Qc9A|VMgspJh5!S+lO5~;Gib^0mVt$*A$do6ep+@{?%nlq=G0Vo^SIi_Tj&p z_FvC=twPb18uIU+`qy;L#}rKnLG-)+|406eP3*+6cDPC?Yw%x}nOB2J>I%9;?L3wDIIfob)&FsC zZmHqXy4L30oHMK?H0Dz-4u|PEdy(yhk&%(BYLM_Fk`5KL`M>#(4$5{PlAoafJYMQ7 zSpy%=o}7euuiy54Ltd&L4WYuM?S@k?qCT8DOT*YfSQ_lOusBsVvwM~5*lwNNC)N-* zeNLwyi{k`*-|n)N9C_FO-6&B7n zC^O@YzjyCmchjTLQ{U6?cs+q1a=kl#lI2_AN&5GL*vacXQ3r&s`rI{nygDN=GTfl_ zB%@M67|?cGZp1WujBX6hV{!fEAO=HMCm)6`MnM;yBm!C91o-c#^vtHWNuRl}2;Qr# z-g$23c!7-VlsI4+x4nxc#{x40nzF24yhv6Y>hbG+m^AU-@bRLNz|7lbRG9Bx+l3fT zHYwZq@o#s-#xuV7AJi}{`5$;0G`q6aLK}ULLQY{mH2Cj*IwbBVN5AUmZn|7m&+Dna zmH(GReSP?PUqmS5+aPxe$b8&*6*p?fVuKv{TnVIlpJO z9u#sGRsQO&?M73`nJco-GwEKF--};C;B9Lng0$SK>Is3fb;J} zI#dS_MEjnE_^Ld%|DAPOo8|t`t>tH4u3NSUX}`Jmg#DmXd#NM0CEy9gCW2kL-0&DS zY5Qn4*=5vNqq+Pb9Bi-!K~dZiJ6+ zBcv@3!~d|hrPW@^QOMO;)1&rTj7*`R&%tzdvk~Yb{#LFNJ6INSdZ#tj@-Jun*L(YK zyMBd^oJK!C4jXx-AcQSRtdq+^NT@lT>SBN}nIw9JOfZ#D^*WnX# zQqJGpJvj)6YW+Xg*g9fZ-71NNIw`JISsM>l1e=XxG4?vKv&_xWg2x_g1Nf}RnI2d2 zsE-yHU2N*f`RDgCY;`E{iZ%9whG74~!z#I3hxtZ7i^EJGSI~{px1GHtpeeSd?Ck&w z8{flw*%atFoq!9bdpoE{FwH~dUZOM;%%ac3C#ndU?)>HyH#}*r9tDD%6VMzy#JhSc@JbL%}368 znnL^Fl6`tdOe&3>)U<4ID^_Y{@a+G)FcY;FHH}uF5g@_`UQdh^#N^bT->!TnKf##0 z?Wl0{Cl&)D;!Y6veYcSzDLFr^LMUA&l-z{w87~d^64tD)TX0JIwW?QvjBDNJ??%<_ zJ?%gQg+}C&uxR-nK>WM3pufx_4k$+AQCI7+Dy{(|X4fYicLswf;m$l|w2k5T1fIS_nCHHp0UFN#>U-w4^ozJM8^Rmgn^J1Gm-Z z*WT7XO$@~?^!?&dru$`IEG%LiJPrhV4HcCKYz8bIdAbGUkf`8BKix3f;LsBnK@PSb z=}1F$PYZG+&XNWKyvvlt!uK|NQjGi#h&4|Da%l91p`f2av^O`LJKgC{OD0{OZHl?_oMe=*wDz+q@ZE> znu5SiNSA#_g}KJ58rGEY4ein`fP*||z2p4aj=IRsbghD;puL*CEPj^0VfZ)$ohjgl zjJkEa3Hu#-iSsTF$Bwy0y+)7es*-ZMA-=cI={K!8h%gWpm&#Yj48kjEkQmJeQVv9; zd;F8SVos(4Y0^guh@5K(j)2xN4pL-`Aqm3Hd77jnZ?eox?mcw%{(6_#?>Uakoa6q| zDEO6>e+{Pg)?*(sR6Knx#OO7ERR*bq{REkZt@i#Yh7nl24P|CqxX zjXR9#mc9m^A!ms(?r;7~Ycy*J$Mk2VB!xY>NS4CBhvpRtUU$xuht|?1KS|*&iv5Q%?Du%nmmVrLWQlT zUr)bx&J!P%8@i5lXnX*z*sPIuUVl;US-b37A}4u~90I;-0{Lv36Hr{HZfOWfal*%n z>!qC~EAG^>;-m_Gl6YH|5m|8w8QCUO7_)w#0zUZRD!BRz+}PMS2iA(t!YbR+u=Pzv zP-qWKdN3G5-&eSzG#xk+l!?!63Z$L6S!9aKSfyd^xKn!sLeBW`o!(d;mHMPpjzaud z{A2R|$9ahq7k}|8*Wd@_!QJF_b%5AdgJ<#BdCmmLtsqPGz2VLJ@+&0gP(Jp|mDO#5 zCS!1bJBK_*SOW6gEHF_&iG*2iOM=tHyyi8Y=G-T=qA6kEM|Xq&b~930X=Pv?ej>4@zVu7~W6%NlD6; zBNfB=M7^d~o)#`q`^?;^QQmp|L(<43BRpgv zp+wuaL(EOXu1`ir9a~lz4QCvY7<(LYmT~2Ms5G(!V>#iMsnT%dpT)YL4cY9;??n{@0O>Nrf zN{KxlE3Sl<(NO`SSFQDFSL$9P(b8Hv+~bvq2z>dg^n3{341;m<+4SMm7$K_z0*I83 ziO>higQM$T0s#yaTqecQl_i@rA}P+&k`N{%M~-5@703d|F4PNB;z>EKe`n{(FGG1+ z@Dh5WWDt7gI>*GK?Ga6{j0}*^F0rk*U%c8kCGguAmCs6$QSpH7SMG=vTRHLM4y#l( zJlm9cth={OBPUSG&aZY~tJniIuN=+KSReOO+nmg6Rg-|U6U0w2ty>5mBgf4svQLim zT;C!2aN%h({EgCtu%yD~gT%4rrQyH1FXEo3vATMML5cBFwG+cv#-eS)DS!kl z1um}B0epXrm@dyU5&nxF!Rv#NcO1n(n?h7oRT(=u%L>7*+4vE@Kse5d&Egor1RkAo zo4lHfr1_2OWxvd~)=?fRN6vnIQox$0cP1fGG9jfGNIu9x_Jw7ap691XzL0yIhxRUf zC2`~+A6k-jOFE<@DwOjn!AcrJANQYkh~+LW0F2ki*Lg6SW^t-_i#qIc^>&R!JLg!b z-Z>u_Hgk45`w>t=K7YKdH~ej+&VGI%p+i%g1A75SSpUU?nRV5{z=*J0oOi%g+cE_&q&Sq$+aMH;8a+OCv`?WpWgBuo(br zKROLo5`t91Q+J|YVL#cs?w7LOxBXzCZX{93d$S~=!~;4+T^s?u!XWJQ_Jz`0nnm;X zb0eX8uk@HqP~j76G?QIaB#KdQaUCCDHJmMKJ7B*&;~EJ5)mH>xwsTwI%VJ z1t&=?>+3*avCcv;70)npBo%bDnPg)h3_jE*$%#@YgU?%AG?7&8zSkzpd1&`tx%VA4 zhCZ41NsSM4&TO)_Nu>xUAZC% z_0SWo&qcQEKMT}%>)>-K6IyggTMls$p&s$^T%W!sI%A4554r#S-qu^P_wE6Xv0~BV zZ*a8P5l&Lo^0EJ#*+hAezdJK2-BHxYib?6Y8A-P4jJhp&$|g_hI*O10O3R6E5q3E430zf8j42$1F4 z2vh`9KmD!$a*_?%W`CwkMp;q>wFG?1)Gr2@Wi#BFQFIOy?he8)-ePB`eEDs%Ej{}M{+q%Dv-VB%%qJFfsMFK4dW}z=}B;DC!^S5a;c>E$FuVodN{8#q6kUM z7veUI{Z9WLZ9N#jpIeUJ8>=>}N`>90ih8p!A_v9fwuGMdsAo)ozTe1_K&ygeAya)% zez@mp5em!_h~YDU$lE#nxB!Ni0i7^u=hytf27}ir-16ys7KyERMUm+4fD5RvrvM? zwRu%->XICiEQXtETl%b0FJ3c!cbo_c39a`T18u>drU<2&KuwmNg(AD62cvRV3S&FBNXsgxQdUeYYqNUIkh3}h7Zu;)8TrYeT{*Oa%fwR zqZG!bm$blg0R1C{&e8LhK&2xOx(R%skoEEyke+{@UnK#O{g~x!)RZ%j1blZ4$e+Ku z{xtIj!;NJoz;H^N;g+?*@fqSc<3)7W1r9wHrF6?)kY3f(Am60{|A+gpPY{t?i}nwS zdT>4iSebj{C>^C-PNC1jN$+8{67Vrw$KfS?Bxs^?tqD-G{P^A!z-_mr&1Cz`oI2_@@gJL zhtYqsv+PV*b28djode+#^CzQnM3T(zp9_)Eer^sG?Kj4KYYM^sEZ6GgPFW%upGCRvL3h5T2B}h0KN+ritWUq-hbX0RaaZxDS z*rT*dLXyR(s3GAu;v+UtNaP78 zCTJO=$moOm1J&fHHLC3Wx*X8$OH2%&20N`Rln~hYQV{)%3BO;kkxgkCK?;aTtw4v^ zL%EzJo&pOQegje}-6Yw#R>;Gz0m|hx|Lrd7j>@xOxIEIUuN|#wt0Z@}3TW|qKdc6* z4r4yRaKKX!h1Xy-_7VM4XO(h=W{j3PhbPND{Zorg0;{*{h}D9_yMVsxCe>_29$=h2 zkM?|(YT(tCDHx=$MBGt1BUx1zfIAAsU;fn);qrX(BuQ6c$3$miqs^V9t8((NP41>U zDeLsLlBG@g^GDy(=UFhJKNqJgQ8TQkJ&{juQ0HdMG@-f8ex1d^@$sFV=?D3cGLe2M zfxH@`OyE#ero!k?j5W|D ze|67?A*h%VEE!%$OZ1zJpg3<&-?C4@FAQUM;2&*Q4~ZI6a&J{ZRgZuk`_7Ihp1_}Z z=jZ6VmbccJYXnUS3;iB7AeH-EAOTyXR2u`T&P?Z&@&+#u!31UCA63F1j= z2Xlc*1FXvqDWCo0JI1loP`z1Fdlq0Ks#2F6=wp$3Q&4Dr9weL3^ZEJkIel|UXbrT@ zNmop9n8w55H4U58w~}7mU0jQDUQz|FM(my9yx()up63AfH$C+0i6%$aSYW_`?8zVx z{^1=ZuIg#wUxwW9oDB4xc@gEX4*G1MQoJ84^Z0ytd*UYy2BQ zdggUI%~iLn>){6rwBdCVYDF*>jQs<}DH{_c zYZ?etT@{2enSXi*k*V-`n*Hopc1tMIXu=+Vu~g58Z*rlR*msuSB&9B_$MRqo`yUqn zj?-ds^x3r@Fc!8CKX=?&qo5oBxOftuCieKvR4EYlx{7VSoF{w!KfWjAbh3yFLtn(J zUvd%XFjQ7o>s#8WsDw&;S!8_inZD~(I}n!%ysS}LxkW{O4A&EC;H8CHq1bWgO(C(+ z3hxWPBoiuR(F3U};mNBDWLw{11!h`rUft1G<6>+wj+m$$S9>Iv71*fidmRK}ld<00 z#u+~~-naEdEDequRbzHGX4~0hJiFB^{)Xug>6MMFT?MHh0Wfu-wPTvwSs_9m{o{Ke zS?bSxUL~;VWrt8DvlYXR@=K!I-V0%sv*Wf$0YIE+@L#UA86j)v1eKl!(5SI+zd(w0 z6hVAIziU*nr&*5_2x$v}y56*H@vk>)ngFSRg{8eMV!W)Xxi05AS19eeRcP~GQvxOc z%2WUF)8FLf*a@h4V}n`zUtJ)sz6$`vQkvAsO&*Gq!+VH ztF@RHsG@`)v`d}}8lnW}T?LvMPv$aw3Cr`te?$&c)PAeJa6k4|4Iq1U#mn3}*1tVT zEaex;SIAGf4OBKFx@+IA9&dmS&_7F%)wl*-p&AemKxJ1T9pxW^n&6Tz^D84GW6jKw zD}&<&G5wi#WCsC?Ig$o6FwNw&Nn02XXdrKnOoV|#j048DuO45_cv z*FW<$p7HcmK_unhFRiMs{FDeh*x)2Y^!l2$EZa|VOf4ftlV$H6^1z%lm~l-#HBA!lir^W>aUDM8c|W+}f%Ml#ex(~sNJAiys3 z?Zb`?fJPNyK%$x-CpvPF2LJF5MuqF?x|~lXnoZhgsB(+M!wRoJgzSDuE0fpln+2Y? zEtT~38rddPq^iCA{~34Z0>Ptyp9T1DF}P0&c=DNJ|b0O068dVjK!^FR^ z<}1~YZ;kZ7 zQ2)nR_+P01W7PjI)c=W#|8G?P#{uX6(^1{lz6k*wwlcH~|IdQ&ZS85q6cM&NxgeM1 zEz`yPei}Wr*5dUaCA0)Og>_h@v3;)N8$#gPiiVRGU$8>O)7O=j$hr$N>H(y|tyKnN^CN!5J>Eg&%0gD?odM%d10KNrj3M zNT1jj3Ryltjvf}bJOdPz?7-!cv*i__4!TGBbVJ?gPB6q?Ai3}?pr^G5J(fbWs-(mi zek=LL-z0`B2_PJ9OP+VrNi2xr|h^o(kY{m}IzUbNgJndfi9g6=mN8Qqj2b83!qZ1JBld^Ih-1cu*~g`l-V! zmKXrJ3aH|WT-AXk;4qt6+D5Er>ap8~BPK` z;;_!Q*QhYH&f=+W6fA&=VXs%a=`3n|%MXCbz8}QL&QU%kiNmdtAO1I$>f{B4FsVz|EFf_)rB50O#=sJ_9V z3RseQdU&r=2APo9&ie_vdhr>m?oI~aD;l+@UU7Y32IeNaD2ZHnu-@}Fb?iDNJ)keG z$jB440@r}!r9rP1wt=Nf$mfR??g5vBzdCOMDD)tGI-4|ZONk$Pa$AQ6*fBaJ?YlnL zhriAEHycwBA=z^P=Xj4q=LDNAizi94Kv%}3C~pIYV`bb#nz*RuNO(}5jGj)XQ+c7B zPm=ow`_)rH-GI${B86OEQUycE=h!}c0w!8mUU;oQeS#8viITFIA2_Ke`^Xw74+a2{ z(=Vx514lKA0qB^rHx4QDcon?+80JF@?3iOtqxxjf-)8(TLPy3{63Avy5RglYj!$dUFx7# zKr@$lUQtl00h@5+;(8(rEWONp8(@wfE&-SBQn$|oO>K74r+@P9vNXv^#A^(N0Fwz^ zHeVU3KL2G##=kis8C$jPGO1dB`wkpf!fnOp?xmn^0LMLP>OgOT3{yHdkC#m=-KSVV zlH9Im+ZDLPfw|(d;U-sur_;|%Ry?M<&Ic&zD)T9>Cl7&>$)`>PEd%y3=>5P9GyvQI zDGFL*4S~yL;L@N&w5p#9Q z=z&eJQBmn>155k7>jp}-&m?zZkM0Hl>+i!d^_M9@9?x_|l34`E=5$-0cFM_g_s^(@d|2qweSrWHqWV@hkwa6s<-$sjS> z&I`cY?iy~UcL0i;QI7N{!vKBpE4Ncn#RJ&HD*SH<{NE7x|GN;F^HAK(%dW(GpT-yU z%%D3`qGfFq@`-?%GGCS1nLKY1<=|f655|jW6Yb+_U3Tj0J_S#kGIc)_{=MfQ(+xJC zvO8S6tc&_^?NLceMAI18m%t+g)}ZmHrJT=*fZNDeZpF*^-g7{bs>|zOc{`1+hnem~ z@mlXiCR;zBSjVCjFWsE_>K!KroIm}Zq~%bI=Jo;(85ZM9Z}TWe-I^7-n&fwjhyLnmk3ugMg~~kVu{yc^`&uyMN3?jm z<7EBVY9L-#+-+}bi{GZ5LutAeHxaSV?E3q=^;T!B1bS-3X`(yBf4{qwS=_!(D=&`6 z%Ro&8Q*$&`D!+pYN;w z3+&>Xx|zkp^ljdB^$yWe7$AaeNLf{l4hf0gp-1wY-Wfjl6UzD!<@b9j3X7evVWn0B z#Xb468dvt|Wq}7#LErxe^?Ti;mpR_Q<`?Siu73jWeXo#c1P$ELHI$y|a*S)Xt+MkW*MM+ljXnzE$%@}i#!oBB)$ zFY)NzB?*L?Z`Xd6Axkjiw)SH`_T5+=Ojgwj=iRshEv>sJkeBIx9<2oo?kv0wG=?4g zB6M#hk6i}Q*0-3}&GlGWfB$G@-TC1fRvW*jIk%tH<0&VLm=1QqjzLpPGQ@i*FY{}N z^~xI1zPTc5W_x9-_Eygqj{b%CY$xlZL!!i#WH4l}Ln#NHf%HYyOS^Udw(1%%f6ZLN zy}pLPr(|C|JZj_ZN=)`OfZqPv=Ja(1l`i98F_lu~fLg6}L)MymA9{<|L!mdKtFA^) zo1QOzD-EeXtlKSl99KVew0gLYs*Di5WTNqq0EM8fu1D`rNEn#Bq(dV7!KjCD(2<(b50rzbj`FhQ3^dl&^);D;Xowtwg zTfd$uue0W#aqCs@%O`Pxa|I6FC^+hppXYdy6Ly$s>UTyh5q3Cqa}$j)I2&CA zpBCnkgRE3x`?qO)9v%oyOqF^of-9&o-uo9+7ZazUJ2?FP7jhoMrTKfS7LK=qGUe*p zCj+{wFpq#EM-3Gtzrt1FpCO2bj zcwgZBu#@$l^_LQ2Ifj&Y>)oRxUiBoFuw;Ai+IYfaopQdjMbv!ga)+ea^#OI+Si{(% z-uoMhns4R>7}l6K2<#FTZ@h=Y_M%gSzPsV}w_bppzdZbt zV8Ei(d82tB#bkcLR24W9y^-bFJJV*3@faQM>AcV#Ih}rjpYl@{lUUDHnUSuHr!h>T z06}oQ#B$9`y~b~@fX0UsRyV`K<0A1mu5W*FFSI+Zaq>BRd8xp>x=4yO=##(t<5t(S z+JxKt4Z~AnqV@H|0ywx+lF5CPiR;GDPr;Vi(ggiXxAu&emXKNj2kPvFNSWNurMb9J zFS-Gq9yTY>#`2bW0_M=7bu07Yim8Rsr}r4DCgc5~fZ=7j`E-va;{f8=E1Hz;7?_wE z4p$u^nrtnSatCGNVpV^K@L)kF_&0LO?5W?@cOe^VHNEP7bV0_5wb>t3My1@-?zj^t z@BT0rL}mJ6hc7(Wvp*ROaWc&b)F|_xd$74)cNW!Snm6T?s~mk-mrq+F6KQ9*%6V%Q zjp(@VXCX(#+(o9#1x!+Ij-`UcI9W51OhTpUsGDBu8YY#_L9tTM>fNFr6?awFP2oEa zDy%l!XZ%uhmzw#tldZ3a7^N|v@I$vabWf(96J4|IoqZ~04%mMmlk7Wt6YnTt91lr6 z4W{l=vx1s4F0J_Mhth054bwjAC|SvPYT~K7uHR)}>4MYpnzi=KU1kcHkFfs5J{Ash zik}vK;Js7j77@)XVgJUk!NP8qZQgiy+|UgbLIc~(K}?NT2qna0sW97+GPCDW{(q=l z?6K~f-%=7J#JEd>2_Y*Z$(qL1Uck{czHHa!x$o8(P06-HxT5B_(;CHrr_#mjwXf`E0I{e>uaJzQ#jeIp*q1&s7XgNqFQo+H1tT zvk-9A6MqmT2C0xK-`>s6GEU!8*h02hgk2_m{jmO&Kg3pT(Km35$8F^!Ec2Q+a*^8& z6}Hvii`@Ds3EDEW8SYwrR~j;nrd=(+V*1suw$@?SwG59p_sY%}+@Y&~Rr`4-eVNxw zT`|Cs_y`-j?VzT(e0?TQvB3dUGpYWIeV8Gg%|P{N`{(K@@f z|L5ZzM5h08LZ7Uq;rbOf)_Leu;M;#%q72aGD?U~XRB>IWW1q6aE= z4dR~@L-5eMNm_zOW1}W1gPCh6`AlrTUzCPd|w_peL>4lXIYifXIcLUqP>xxGf4b>B+QmuxJ=>~QPrx%to2{K%{t zH~i>`y?G8eQ)gqcVW+()G8&KZS%Npdd^c2d*LiXgL|wVRgTy7vS2{AmO*3PC^8ZB7aEl#A4xYDAGwMgDoNtl|_jAfXOiuYZ z7yvKUEBL&G&Ovr;uB$%m< z=OukyHi+(s`aKfC!#V=(iZ5BY=7sm(ciKES`on4V14P{flBoup=X-s&(%|h!Go|^$ zys`Z9kGtMH-|Uq1Jz76ohA)f>sKnZpghW1EkDf1n=U!|R52Uyia7b!EfOtkIUd)lwF5F z@pGNI@L>J(#SpZBJpatpX*Z;W)dw5pz?j*-31toq9ogH}J ze0!-2wJ8_meo%A1yOS;#h4J_8ZUFv+q4OXIuN*jO7?I#KnTZ20rzB?y$LFbbtam5< z9DyKpumiOoU1QJqW|f*#A6UMdyd`NcABno|Wt=VP*!7qsu&zk*V%{0Jru?Li?CQ&w zcfK|j`_&6-iRwZ*bx+|=-WIvZF56a^^{N=hTXCYA3vX{y;?-p7!D+1XNJ6@N8|%=t znc-#N2anpj2zAsNJ0ij|-`uT=c|>0|SBTDUe^Znpv*=y5z1?VCtxmg6Ke}tp$U4z~ zJ2m5R>d|&sol4GViV_cY)cfj@B-1xm@k!!I)|_Wi?pfn|ywV#Y1b7N&K+JfS^dE zBnWU08?7I9mA{Ho9R4zL`7{{r38`HKiPdax zZ1$A67l(IT^D>BpiJU@<|2F6UYB6N-k`iz91p1-}K&MXspbI6KI{mV!bb)}UdJS=W zW**I*z#Bg0H&LbZOQsl8v8SEpt`%$aM6YI~0lU`~?v{8tSm>L1|A*_%WkOmV<);0} z{V89#mDk>^s^9#VEcP3bR$4L} z4^>WS-Vg!0Zw*ZG_6L(2uj)CuRQnDl({`~HZgmC1Nw0}-mClDh^e~=q-kdN+Af+B(K3LL#|VtQQ^Yf8roG(3i+&C$4W+1C98og!@u)%`l@~a^DR-Rs zy~&Skpu*_z-#Xlk<>av2!TC@41}T_S3@i(Ksr%_|RR(w%12`Hh5WpAd@~2*89nHty zAT{^+G8VtuiFfB~8rPG~zZamta(fAR?9PkVnepjtE&a|@M+|?%)H$WrI=)-WZeDQq zGEOxRLiD4mjy3>dT`8!`;zYj^O8a3QUD!;%{p& z?1cyh^}YHT#r5}D05jj5>2yUd_uV@Woju#I%B5xELDT3(aM14+gsAh-*ZliH!?5Q8 z6ma7TaHDRD&u*9c6^7GuQo-;iW9}n5Ina^RTXq^^HxMS*zoc^Sw~uXhkK%96Z4xAp z(yFuy*gft{CQ;=$y0k6Tl;)0rKTqS7vy;?&IjTYn7XUapq3y>+%J=)Dj2{gqthBvN zGXI$(g@(6+j4zu=d>fV5rwsu#akO|=sN2-saC+?XUQa-|{ExEx;7cdhBA5)NeG}OL za@F%;*~46V_V(t_@r%k;buHtKr6w;Mz7&24Qxq2XVccB9U9S>-bsok4@_s3?Wvi)Ij9iR(i+8BH9t=01M(@m&m{~%e$Gs>r8dGz^6!L8# zaYMTG^MhJZQPZZrW!1#CC-}3|_f&?5Qp=EeZyCSxdBKUh>lVm0SBs6hv!ltinY*R% zP!~>Chr360l@0de2g(7K@jCH@xyHuUQlg>v3Gk1O0*2f^y86M*H~RxY11TjLzE_Np zzz-`tg_@0>u-!{$>5f&UR2b0K?($jv6vK-A0xaJ#UTje*9&ITbG2rJ8`I@O|UeT_n z08OnZ=_^g$2=?9nkzloQReh_Wk1oD~>z28UO|%Z$1v$N2z>#uAdN?F$@cUOTuk*xD zF0=K$M*3z~%R911fPy%RdtncLSLiUE!&cC!TaM;j4&EAkS&AWS#8te5k$O2Z^Q$sDt0{ z?~l6UdRJ9tmpyjnr|k3(+_Dpqt)RTe@W7t%#X^#w#P!st45HayIT^VrL4k{ zctfMqxAZ+m+D#LTaVk+Pzw#3Gg~22fkM^V(GyEbgC=0Y%?xzWQLIzU`*lR__E4aP7 zu7qKX;oulXcBj)aQK^zP(S{9t5=Av=G<43N$O5cT*m(MDq9`1FzvQZuX@njIqzGjk8kMi+ z(+!45EOj$(98QcPmX9)g4^QiK+}x@SgvDe?+_*C!*TcQ$=d!BiCi+zrw>v4jVuCy} zyJlsQhOB*6kut*8B-2v_bIOe1*S<4R&15ebN{xAZ^AEm7OECYg9y@q=vq!BfcN*I1 zOpUP*cOToX8=#dGag|v!{Nyrx2dJ6&da-H)$wCkP;JO6?lYd~oH&%iP4M;NQK_ZhP zTH!aH0*Pa28k{(U{Q~E)>!NYQyc<) zlCPl>d7EUuH%#7ZE#4%qhShaG#;TTpEAqt+?7_aGmFb>F@U4x6-#R~5k>ExgTH>*8 zDB|w-GDD|cC;Xn4x`ho(FubdfzTILuUjnRWzs)tp?N#7A1E@m#(R#q&tvJw3uv@QY zwy7)u&(>cXg7lI*1*Gtk0_|IWP>nn#fkiVO` zlW&RncJCIX!lwiew2$zRqw-SQyv5(C;8fi9#o}$#JXQ-D2#zX3ElJ=|8^ZXsp<$;z z+5+b`jFiI-!G>Tqf$n;bSs*AP&}^u8IkbTVB`A9dG4AKCaNblNo{+Xm%Bf1wH}e?H zO<{H7Oc;>0hkaQ9IH`)?T)YKt&1mS3=V0E1El6@k<~$un-rqf{mYDc(%VctVPVIZc z-dUk=yK&P%bHDF*+}aWB7&R^*(Ikk}>u^Ab)@GTxm}+%`VjSz~Y!mz9VP8NQ9k&#b96 zn}+*V9sFf!c5)uivX#G@Y4p`=%U6|qSwo?tXsM|R@t%cAl>Iu;wtA8#pIBl4le!7< z&iR#Adi6A}hXBnZeCkCBLoc%rmZrD^?t@%Bzlltlw6`lbQx+DdrBUpTr8niB1_c!F z%hFX>w>siRM}*+S>%V)GWv055WqMaJ$QF%cy&9FF=7E-obVgJgXsWx+0eH!qo0kDW@_&7@m%QF2JbiPNDh%r#op9X9~(2w7O5DeGa&PXaX zAS7J|>ks4i!uvAhUM1?hdo2Y`6kM7u&8*bcrmN3gw&$Op*?~9v)E2ray;lFEulu>q ziQbxbJRxl*uzp-4(=Y`)rd%C!)!S3d)o^A#XQEN>_EdA3U;oYeTs_AM2iX1wk8=Ow zNHh%p`kM8g#9&nWrN>;Yb{xaP=7hd`qUnf;Ne3I#Kw?j#Ek&ylpx&nCa<;%P271!p z63xWjd@T1Md5nRJisyV;f0;+yNwc;|Xspvzw9Oy=D4F0UFfDZdl%v8twmnTS{oi%=uI&C(YP)eIb?Oc) zd=W8&p7{yO|)*M;75lC8JR;(h zcqxAAfXv2}JKt5=*gUYw*aY(fZs)9=$;`lAtIaiEytluT?$ByqoVLi}F7U%-Dh3JO zcp>3>N;PT}(9M`4bqW6{Fm6ABb>_he1}ERpZsJLxJ`bUC~#o_cmPLn1P;DY<2;C=z&&o z`Q68oY}FV)z2d_^nTZ1#RG5U1>fR<9r}2RY-B#El^DFic<*=WyPL_nj`&LW|8VmUj zG?$l%!GWJ-58d$~r*+q&={TSlGi`%$_Lj_^0FS<}P)?1Ck-IGTt~MtSI?x5M6wkNv z!)pP{W9&s7VsMdanZOZB{AapG`VQWYDiUEsM}GKI*%sy8SwI`gFsl!jX`F*MXsD~7 zUKzc&=m-1ZJQq69F>k=g z=d^q?h=r%37blOVm1E%crTc{W@#4Cb;zo|);;_Y-6hz7baA>8kB%t)L>F}u1zfekt zg>FpE4mO0fx9c__iXf~zbEjpk1}H21jjsipSH(Zmm0r;d20wUxCqsdqk8Pij&9XrM zx=SML+j%D8=w~!&D<$LTzxl{2I#BlJeE4oCI6c72={)_Szhusm9# zq~sKHlp4bv*8J#Qn&Zx)OUEa4JB%an&o6GgfZ5KZ za(YLh_kbU8IE7F1)y(@~*f(R4M__9eCAJLqT!;H=A)CdaT6n8h5 zBce+9Wn2cNl~1}#%Y41=)ipxu7)IVUs9Pny#puyzS&R!!88w@-<1lP&uZM8sT-f?2 zG*@7S>t$18&m?)FF`7WLz~Do5vxU)qe4!Vq@p4*?I+p(WFi>ooyZ*YWEil47PQ>Ru zEYA#eKx90&m4mEOx~=VA?q)5+-4boA89w-3)Z8U9Z=bWjTiCc9YJSHqC6=pnr;A@Q zMN;dt`l$G@0YT-Kag(uQ;1}yh)NV-2tXhn>Z%i%z=zV))BQ{aWf7|M^dd?T%FD>2o zt(4v@CBMn^(pkXQqqk4Zsz}~vz46rGcBU_%Ne^sVxM3kGeKDQ6OYd%m&$u(#T(t2y zZJA?4s@RLx&`_VLi!MsPE9IExu@BZ^XXn=?K*v4BUrLEDTgJPO;E28bpHON50~vur znk};K&c9Ayt3u)riUbFe5!(oZfTN(!PJ5iD1+=}^>34il+oD}qi&`wlP&0Qu(CT#v z3@@ynsp4+orNvLcJN9K#IrKf%-81i7r6jw4R}|ed)okAT@{&H0z+mq9#rI}MkF0h$ zX2jomVF(g5Qa-bnh+ zm|wHWcRog+C2UvUkwa|RQB=)u3gd5$+pB7G2Yx8-f-u-lKOmUSY$j%c#SoAy1KGZr zsps3hAG*|5FUyp4=S-Z_aeJ`&4F?akpA<;l!2v4bQo|o-PO4;39@WS%aDa@`Fqy&H%_EN#Pa1CaWa$|{3bRj6+ zLrHaF@l{yq-E-qr!fNZDZICEk7BKZhu5!#R8OZ96*Nx)IYM`y@CD9*}BBaqJVxXvB ziHw@N^mTjVqQfJeP6uAEF1fr$I33QS7-^W_>x2gBmp*@1KXoA3W7Bgk`Ywd$52MS` zw-jfw+e^~+`b`s2#Toa!zm9!@t!VICJ#jjciv1cuhcH3Uacw+5S^MUD7WQ26gY9n7 zR5^6Say4qdR6D~YQUp^0&SWyh9Ue{1I*h&p{{C6;msT(8D$tu+%?+scld_}*giBKIT z98F&vn(0#v7X}z!IB~WC;ZqfBS`GdwB|Cah0Y*S6lit{-zVlyLDPtYOR~M#ih}ck% z!2ZB0KChm%$Sovf6$VI}HugL-68weZ01p1`ixk`EX9##9VkdNh3nvV?haVASLG2j^ zEShs8_~>xTvLDQ1XMh4BP9()X;GpGVEYHWGbuF1@WBJH5ukX@rogk#&dLNOsE-YwS za{EUZwo3AxMB?laADrz6vX7R!KqC}*GXS!zFc;8E!9jLB^5_#f{~TomKCq0hNBcav z;kJL%0q6v&-Rt(hr_Ccjz-Y(7AOS6>2l{_jiOK#ge!1~;hgYs&3Z<@6qX`%_z(OvU z;?hPuiH3B8^bx>I_Gvk%=LX3q8ZUz4>)v>8(}dS}_P)pJj_+rAwv={-$Ij3%5Kki5 zXGbDwE~lC{O_-`jwhl28k?(ixE{BU&1FB75`2k#7>|&1m>r-Of?O%}&kS5!5BNGnW zMA1eJ2nnf+E_-1pR#bwxi0HVhbD#}3!X7ORrUU+p87S-%0<%~*#lY)02#!FsqQ1u! zu!Lnsfk2d%+#$Kk&k@@>RtUw}su<8}H51OepJ@Sdxq(&C%XsAO={nXR(IXVnMPCeB zr>O(dMV1{LrhJrDYfbEQ3{*AJ`n1!i@GCv&+M`HpnxEYgJ8ZiZB`f-v2UC$@$_1#6 zJE7xf`%liN8~w&~t%rJ(Y9~ZCbh($S11GHGSrAuHjDfSt42T)XfY^8eyJNq|5Cgd}IV-}ArY-ZS2D#yjrk{lPJk7FWVybT`>}z97Q$8LYGM@TEqd%t{^0HkRd&24z$h`a%zN)UO1)#cerQ2%5Ah!~ zGkn&o7-n$_Rhl~}a5FU+#eP=s^xM3+MnKM3_>DiHHqk5==xGCxB|*W!pDK*6{5g)` zmmKOJ|FR=%dGJt&Tlm;(F=JMju^l$j-q3i@8|SJQ_MoWAG&U|AKjp_%$(9|$&JH;+ zM*Bm=tx0KoB!WparmJs9zF?C=R&v=vhb)LtUPtW6m>q@S8xu(6DyTo^{f;?WRbq8a zsKYBSOr|ELL8(}KL*rK-97s6f;~P@w{5jTw7&G56aO4WYRWamHe%YtfvO2qQKNSRr zgz>Y4we-M-*(B}5Enm*HG#EJt*}}mA)UQCmR+T#}m+>U+OWn+2RMh%Rfc5+`Y-ejk z6fDH|Xl~ceAdM{hvHPH#0AVsn=*^r&QmV>B?-|Vu?%aX+ffOXIQ7wv+3iPAVaUQZ3?a8P*nJ}+J?--n+sLngJ59faVVQ= zRSo|2Wf)d955T14h{$@3zuF`S3#DBI{#~~68@qqItyoH2uNL}j=lz%6 zQwyh76!sns_U=#POxBtJ8F{Q-iQbp_S_w)3Ma3bJ80~nC{jFe^9Sgf6vDy&OiuD_w zoXf(tkDF>eIq`F4XDr1@30V89X`{Ncsk?uMNXOonDdizv(I)=_PK8hG3r&TL?9Fhw z3t!6xafoQ4EN$+}{P)dSS99j4n=-uI|4J393|&6iQN-jitug;xyEI^ja#lMbTgvjM zz|L*?XE81^oUB&L%&Hs|S;4*e9Q{(OK`Pjy85CO~6WZASG_`AzD(xG~d}--TFG!eu z^(o+M730y$y9|2D^Ivze4+|E|5dzQ6bQ=fwp3xT4Q}!8C3o|WJWb#u3Y@4unCqB}Q zn&GI(`I!zPmMn$r&F$#+GPl@p^fhUokW!Jf*VaAs(wuc(5I1X51pjg;e*u#~Qr6v0YU1&1EV1nGA!FMr-Lm${tUB z;UjDG8Ipg40Q+r+GhJi6q)|oqB9*WS(fwa7hd3@lSCnN)Tjgl|W#fI^@T%DvzBc$3 zsNW?Q9M6(NUWDRhIhxwfWunP{L5!n*U%6h+Llc)oA?V zc`lGr9o%n|Il*xh!np)#DoR4*t90GJjIpsBhEvJkO2%zBVwypMqjNG?kHIaC65&Cl zDaDa3?1AJ9Kr110ys4Z3@3P6@fxYWTCU;_8)z^Tm@xEHajV9;it&bX~O5plUF9mTc z*T+t2_ts%&N=o`Klbss$Hl*D;-POu;$3!*N&b8Exc_*!@Hl} z{(MLyxhPgBbVW|^S8CnfgvJdkG*77Ub;QdiXP;?0bD?*ZZ!^FtqR87}Gva>8N`AT3 zsg$}Slc9TWH?3uAu_igM)^zfxowhm)rVx!0=zOmI+YH_T(4Cg@d%;SF9aR*?b&h%> zry7}tj8-A|pa2m6%aA8#V^c2v=@g1xh?z&VmCl-!u>6B~{qz%2qK#8ka|>1xHpU3w zHkN|4{)Iww*2ulJ^ilu8=!5aaKma$Asrc$FkTTak>biucdRVqDDYH*~R zpa);3T#jUl$lQNsH2JyZ>wHj$!d$(~7q-e0E{H)Bo#>ALiPS+V&pcQTC%gzje!lQ_0O(GaRs zYAdyBcRDQ)=?MhHH8;&W8g7*P$eN-*$HIo981)kI5_XV!p=MpAB=T;>&`ST_FZri8 z2T$U*?)}B{Xkz9wscBCm|3dM>;}O=)YGE&j>llzeL)GM4&YhWSrdux?{ia26v#%J` zyqV(Flt_a;wg65^E@PuEPd1EYOsG_8>X>{&+`HvwWzaY<8O2(4vkm+4mrktzuN!vI z$LE_tw=3(L-ki5rXy=H>prHvD0C zKUZ={cl;&K7i*PIsT>oc`}13PyHc+;xK-}jDb+T_!rq*cbWhYhpL`8@WzuO{=%YV9 zu{Wroh`R`(t6Z@7h86fk1uIX@A*>{<;Xy z;R5D@sOm*BD74bc+iC0X@<}LWI{XCA<>@`a`dSl|%yZG0rM`DhMun2oYR8L}Y<>LW zi;^Cj0dqv$*}7}ObzIOp*tr^WCKi)~Q}f#Vu3=Y*UE9B{(^9SUrsP6cBy$FQL?!R7 z0tn!Er?`zBGY2)UmXkMS>VXK)u^t*50Q&h^DK#{QreU{UC$*Vc}-^bo0FVDsk>BA$B_A2^0|Qar4ukI?@bGt zu!+Zk2no!YCpqkf8xi*vL%6QQOVmpV|X+ zM2}bWxA<)`ZcIP>4;BD*V@*{ci1#7;O0vL*b}wbWW7>zWB&!5mhTi20D>;Z1HV9u> z?UOp$Nj0##`Sta=@4LLDx}~bw!+okN3a(~R5-@fhdv)CG`HoJMW+m&M>7?l4%YD%g zrI+>Upr%`QTI#a^aK!KrCCQb;2}vh$>;msAAfF3itd++v$KDoCMblDZ+yh){T0?Hj zwLE#lw|aOaAt8nyCwk)?l6b9t0eB$_uuHMrlzp^aCf#i{h1We zmlRX7=|83*SZkuHXt{Ej6x6n7VjPAY7!RtiXC<0T!U)=G$-wuO%ghdjHN zIkA7F0yyqrg)>`T#@K#rbdb9(nurCR0F^etxRIfyWam%X$$={goe5L$X>FHKzwYS% z*V}79+&<&8yNFxiZ^9(CN_;Vfno)AG%+K+Ky%Lq8G8zCrJ^ImQ=3YN!!iPbfn2iTY zm+VkLiTkpk1cKJ2-tOA0$-XeTJ9M~PS6tgC$Jf;KAYk6IOR?X4E`vID;-+*4ex~JQ ze7Q8+FZIY4KMscJJ$9ZbEjbytr|_qiUw+bU?=z-rmWnuREVGMd_@EdbVXyrp$QaJO zHj=IY(AEkqBep{-i|J&SW#h1N17?>LV$;V-jkT<bZNIyMHjEy#+*t_j=GkxVh;zzGrAFf7#Jgg(v0wY<61LfTmPDG4*OxJ%xZJK|0miAY5bJUCUh@QDfiQaZ z?ZDvz(bp1QClLtofsB3y+$}*_W$Zp5TO-J5_+6eIfLeg#zR#KfTGJ@s%4lxW_r z`D_ZoZbNGzrzpi5Y8ibbnhi5*Ov9EM`S-YF2_%P>>cavUTNYKmQ2<`SSc}H~Ce;(7 z*67Z_(UTn@W3>@LFy5Gn=f=qB{yd2z)>9pEb$&^d8K%2Y8MO|uB2sNTb$}t&UtZ&& zCl$6XUeZ0<$KZ|sV!NIeRR-*^pX{Gq(g#xbh0o$HSQ;@hXwDDbH#n)>dY>kMJh_Bi zlXQ5Dd|Ivt+5}Is!@1|kAH_-32J@qpP7~I5awy#gtLnnmI7 zM-lAp!f@i^1O4p1^-Ct!KY#+EcfIV^lFN$x#hf+$Z&*8{W16<#dAr);3V_;a4i~Lt z1YY)L+O9#zl(u$+9iZH4_z$NQDUi)3Lg}u<)%^Cc^cPc<;G}babsZu1b~i1C)17p# z1uJ4Z+k4^d?u+^nkvPb(bmY>GgWft&$ThcyZU9ZTHV;876R5SU&g?0QsOGm;s}XVt zB*%k*P#JoduK~+*+(ihRCErwxMm&&^`{HxSVKT|Q+CM2#iE&;y3#isO=a(|kKb}2w zEW>Qn;*TJ)ZOLVmA^CNy`i9xk(^-o0o%hT`999Xnwe0={i&j5<2id9uyuVFWh;hV- z`6klH89*^3Y-d1uVLu};Jz(|`O>iw);I(ue0}jN%e$_Q;k*c0;5Yiw<&{E!b-w?aE z>5F8zKH>9C+m`+Lm*Ul-8*5F0GbhhY-&3jJpz_)Tua%EF6?{Ve5Epi&h(1d*2hI4| zCZfO&f9D#)Ei04-fWVpSFG=@EIYAOfd~OQ>o2;ZiS}}RnafeIYJxk^1wT>C40~i#S zps`44j|nZcVc0nkFoBb%jtPk&{_t+|WM`XdHV((~#7uzoI495fSI@uwyV1K_1Qp6+ z%yjR+enb$*Z`(Eet9J6I2#&a8+a1lBtglY4k36$Q^BdMz8xg=|WMk9S)MOG)9Fpu7 z6kD{lY6hXR;hb`C=I;<#ZPQ-OK>jx;AkTG(LP(Z`;H=+Ddh=C7X~+k90Ch(Fo0Y!i zhI^eFACIxBgN$<|m)w9`spv@h0zF5aJ?so!VKT8`U?H+YSZSNT{BbcKB=%ubJ>y0AQ#Fgay_w>(Y8{2*&DeW= z4`X5Cbnj$y861#7Gu&-H95Uh{5_Y~L{&nROh*`UInEa=-7%+Qk{)El>EEjnn9k|~f zPvPRKETu~{uhO|C!CRhsT0gj@Z<3Yl~*xc#2V2#}x{ZRTo;^U4%}r(w$8Z z^JMmo946y$z6=X3*zz4m2nC@@v6wJr*0{EXOr6ThhyEquv8aOS5&g|uf+^rN=(&pF zQeb41{+AiT6NORlVa*eF;t-2t2MNy;xyO-?UbZNwJD~*`mG_-mb$y0j-Iob|Z#V-m zT(0+LYnQZ&a8C#1`s*t>c#TU+0Q+qlsje2w{t4uyOO%>MEY=7O$ZLlFnB7wD%7 zlcGumuJ@`=)EV4$3C_h{c0%!yIEz!VG7rRaTIO#lhOq|0Shq5XUP`g9)bF5AeI$kn zk#zX`NoH$2)M1nH&Z9d)aFUuQ+hp-!v^o+q7I{O*6fi)1zD`NX+r_L$x-H%7@2E8% zOP)%a^6xj7+scx=NJhIEoAF?Btu9o_tsT?f`Q>)UN54F7O0whF82!iYR$IS`{=F=0 zY>*uDZaNPQY{>YK8AuARuAMFpK28U~PRj2~pDc&&ZLiOk_6+@HmgTaR{za}Dw6E4_ zGGf4{68GcQHT%SK54f{lsv1`8U;rj1*ez~DBiz=+xmrM%DTwQ#5uVM8{I z45cfhIMpruqCo|2!~MdwlvBf;re7_WN>78}Um=RA>6xE}nkOtv9y5lSh>JpN-^2tLptzPpY1ju8?D(TmaSJh5?j;uh8&jdPrwk6UUX|3fW z5?6v^j|KULVOfKlK{i$A!X!>)JSF=F9Qf;ts0_fLKD~RwML0=s<6NQMCtDc`60Y_+ z;i#`G_jrnOu}+Hlv)svomRA-0By$@BrbfO>fR*=@yj8B>c`n;sgBlyumhs14EEL!` z9M&9E0viR=Ki|Q7Hdwod-7jyIHzQb$Yg70jnA*^Lph=Fm3Ar784iFWII7CW&#fk9# zTWv4MX$gojJyd3+Vr?sk0Vs#P8f64c zT$*Y$!yhC3=NBaUp0icswwuQEWJux`Vs`xP3jyDxAYPiaSKd-Q*DR}`Wh;L|F@A`! zZQgF@WYc&iR$CJ{n=i{l+pajwP0c*p3#~6@yJ4Q^AgmgFUgV0XAUR`IB+`&LecpG# z@H7#K>V@=;Y9!3#047N_!R%GBAJY@SrV-XGTNJHgF%om)+T%J3lJ($^w_7I2m`onX z2;(tX*xpKSsl<{=E{&a{ zUtREhMR=N`j+zu9RbTwkuF#CNztHQuE)#pNvYX_zqqih&4gyT0*wpbEMt{u|#mYO+ z{yByrC2Z$%XwaO8W-`21%cA3LKjQN*KFnc`7I%-(lJ2A^wlg8IV+9B8WeixZ2C`f6f2`{uEWZ(8(&rbYjE{Xr1O8`wzcI?QW zH@CZ}9>x?C=>>N(p_-3?k`A97t}@0!5EYZl9cIIMvMm$B66i7>&mvr{493^Y`IF!Du`ap~QYb*f7ol+69J7Gmd4`l*5S$M)pL?7r_2k`st{dH(LvUxt&hFkb9S>3%wDv z7~2mUh9>KR(QH4bwK{9+93ccObJviY$P-}OSn#*q!N#og%SGXGZoyYrvYTXpN9ihh zXb%e)Wxul;iczY*wqSo z3}UKTgM(gKEPH(cc=Lg+p28{r!Dsy8r>~DDWf6|gFgTwCnr-^+OQ$65E5{*`lZ&1s z$vRs-v&F1rz`V>YdOz=9lIOj4&F!Pe*1KzfeR;(h+gO@|*LzFtv0jYsIlOb_V}13nWa<&K6qAVqhjOl$mQctSoLC4Vc0Ut91UI@@hliLMVc>U9U#}Iq zet*SQT^0G}bVqg22w-WuXdrWgwtZZ+`LDfOThVlJb~d52%Y{E@#_+ozrHNb%5o z7SobLdHcwETkpl93637}Rx)Amy)LE3@BW(c#NWvAX)byFr-cdI-$J9P9&`@m#@x{T z25*;i2&=Sz6?(lc#CS|}gr3GXl7)R2gnW?S%IP+wXS#{As&>bRgH6_wBi&w0x9A#_ zw`4dS79=4@5m#mjUxC86_pX-Nf5oCNuUft;1HmLRe@|PtkgQndIYXC6zs{khZ@$QK zsKtxjSE|2vCjF)mSI$w%0`S|W1tB zMA0(4rxSsLc^NAXdw#d`{PKWho%vFn#H;9SQqUocBP###ZDRbj%=YZp<7l+74cFAz zztRy)=*iO^$<{JE8Gc$Q1C3_Aaa!Bm3SfZ|cuUBhD+Jv))4b7fo z47>L5P};HAtCl{d=o=L)hei%2#E6#rfaF@xgbMeZY`L@XY_5OnE4MOM)#8fJ6>RD1 zU_u@&`sQ;OT3)kgz4z3#OVX%{U%+z=XOBgMzm{uPEUGW47#0_tMuc|cz#GC@xs9$@ zLyPQEe3FE4a#cxUkrCFPlO*1rK}+Ge0Xh2pQS83Cpcit>shZ8YW|ELv|vnJh95s83!m;eGNtmkw>^bVe5P5e(8(O-WG~szLE~B`jAzgb8j26=9_rl zwnuFV;6jtrGN*AZE1RKd01dH|Zz!G%Bb6T)9p4qWv+ za_S;-{EpwB(-uM?AzMlYg2&I>q)@Eaj}tvy*yPpMxR4XMJ2}D313OrMyh5}xWJm6j zMCgY7gW)_IiZd8iztqK%=r7jB=PYcuc%!#Q^yeBXaV~V@=q+EnWumB`T!RI`fPKG( zTX$IyU$^&_f5R-FqR@_doY&_bSM<+?s4b~(L!-+_O^yB96~Z9fxvchUrg4rIsyKW&)CMXDN6-CHwjFcGrDC7!f(EX(6+9d)sVWcsEAt#Tw@*3G5m=h02 znCLG6f%<++l`p7v(yi97(L>NF-@)DFUw%La<&nY32DXXiK=Uw{h9*u=& zDrrG1bJZ*VUOup}L^eC{brq(r3YmW*LX)98EP0)2@@|^Hut4P>tr4!>X)W(%=%`KY`gfdO6v5tg5O)r$97K|Vn0ENE3+ zCIfqy(U+8{O-1@PGui9g$<9@V;MM8-hTZ4dcO9q2QypZFOKRb5x4F6(GrXG|Vlk)b z8trLw?C&G+H|4*g2n8p(O!ySB7F4p{pLI?PP-My$a~h~y05P4Xr5YH_C0 zC3-WcO@mR?TFLouBd+;Znsv4FyksNF&02Ty2Y*^@%C}K994art8yDign2UL;)F#Jb zKK$wxFsOp(JHqQ^HZUVeT;sMy6ZU<-g_k=aGL z^Mn*)-Gef_+Il}17$r?hmxPWvfkUY;r8=%V1_{0lgnG{j^s4&{@lNEVaaw}OOIUm!#i`rk%Kl>ykPoPHhF4sGWb^;N03F0r$5Gcab9YOMu~Fu;q=^EX1Y>9*Nml>p_Y7Z)H`&l zaKB#ly77)B?~We*&D9M}7OS_Y;HF|G}vU;U?tarqGj|skimux>1b#K+43|)bZ(VbXi zu6vMYc**H^nX}?tC|4WpCa2t|yQuePLGX>n`t(Q&+@qF}%G528o(nWK?i zsZ1D_6H%<&H2l6^sd*v|NytHT;aK~8?O$Mc2D1$~>pe%NEqNa95IgJiVlj*!c|(?C z_ph&MO1+Z#JGT{6aP$Sr0=#`=yOa~oOLGT%gmc-);iKw(ECx**614&nb<42p)AKg8 z5{V?y-m3%YvhE+gXIRU=m0}*_WsTgK#9lURa)1!#8l-VyrndV`#&HMuW}ykpADfQ( z9G+nkWWj25jr!T9;JmhuvP%qImwAlTXYN^iM>(TJn z7(hX<$+w{WpjZ1AJ}J}ik$dauoJ4wMr3>~Ak!X3pzq?-exOUU9YOmO)p zJw^~G7vtBksh6EgaT2Q@6uskN<=E0CV(FV@6ViPUT~?Dof4-L&3Cptb!*)V@Q$1Yc zrgAZx^9y0b&wNc0A)sO`*xQV7pkDH+RW(U#wE0tj$*p~so3S_+5 zX+1jMWa$4sJZ+Rccd)7Pmh+wT8C9Ext6I$yU=TKR3F)(CCqiGK;tcKnB9NkJ#tT{S zuO!+mRj-n{N34h@V%eZ`!pc0kNCW(`gC_UCBTdwwOI6ug%2*- z0XUF@_dZyS4@Mwy^SkK`DlR&Er;=7bS#ujiFA~kguFdMnbtFzsQ}~x1)g+#C_AscI zQeSr}}@b#-u64RBeW^0eQ4B*yI7JbHX&bm z|L3cY`R-n%n&FQm|H1Xs)qEtUzu?S}^ji^3QR*cOX-I%DRCdzPr4(|gR%iy5g+CVG zEMBJFB3*y~(K{pv<{MO~g>g!B|48=|>P!`P3lbt*#;;Uf-BUvE{U>$hEl z0Y?cl_hdhs^Rj7u!KCQtU>48bgu#~MM2Vyl_*TE-Y@Tq*-F`t_N~$emU83LM{0sDV zA84c8jAHl1p~2mqtVXDu`5l`{4S{qg8OL-oZpMlCgxqudFH2T=YahwQX+f6=F zK;Zj?R z9dUG<=VX9ua|ji#{xQK7JmKRFdp&{rR^7kS9w>hT#bU_2MW5^Y!JJAkrq8ok9^@*m z3o#_e2*+aZrmM`MOu%<++AK1LRds{gtk}ebd3hjEF8dDSi`)x86QsB4Rh*{JL#z5s z-tL2CH3I1$8qHqB51a0=mYxLGpFKh!R})wJfP~YtlZWf=tJSyOlC`R>jG2{~Z}*=NF}lyhrAdX;h(Potqu0I&i{@zO!8!ZX;z{JaK|}Q9 z&#alX(p+7e%rbaMOK8dLyP{ci2DOpRFow01=g&}F72gQ3pS_d{2!};E0vjB@KO5n? znHo3c^8UUtyZj_Bx*Gk8)ft&N*cuRD;$!)|uO|QeG1k~makc5qXAWRUpH5>CbEdm; zRiod4`FJ9J59Xl=%vOCIWIA%R@qz7df6Flv4;Lr>hH_J&6o6Ne*@lq@HVuZF3q)q_z z%*L3d;;q^WOqO~hIf!#UWLGSSz56rK5PhT4*TE&EacpBLSF-RGAnT*UM$-FgSG?0G zt6nYGxRtrTF!}h-oSKcx*-zPB-7IB!hYc6j;=0t{WxG=&Ic`Y(<*eD5u2RKtPT39z zCROc$ak1#mq1vvc-*M>D_m5$sq_~S_=bqlbBQ~%2D4*~y@;lko$x1NN&MSoq0TTJ(c2`1vXH@P{W&k5gI-2;m-1C8HQ6Dwir_Jby*?Io5z*;vckLUMAyqx zn^cutX3D!BVtj?%`lG&i^ShyTeI{QdW%qQYm1OQe$QaW!|2v?9jp@+<*nAIa(xwo{ zVxTw_HX?MEcIfJU=R@@2@{v*2H<#LQ!^5OwuWEjwO%Swd{S@Uu)QaJfENTI3Rom#` z!rqIl1E7AReb`jj*XNMOQ60No%?i6>lORaQ{RF z&y~)n8+xlkoPl>91l}9Y*X;M8;Vag~qPH~ZqhNi{_fHYUtD7C1<@MQob|!(WVulr$ zod)^dwK>vCj$d@jpe|V+9_~_b&~vH9q6)(rn#>GQUG0^ueh4~qxqKw&VIIeDTHPT8 z^5T6mAF21QGc+jg-H6#kr*zvGXUC!ObCK}fSe0HC06-)|v*|EkIm-Q=DmE>y_H-Ac z_tpvKWRU}9$Z5$N`z6tNA>b#LyUYAUaoEmWMkY&YbQl^b*Y?CCSy$RIre0{iK?9`5 zqYyb+9X8Y1=9ZPWQ$*hK8nfuWmBDT32i;lk5~^ZHb}r0kD`lna9P##3283W6=&E%O zq;C#WJMGFvT4{PKJvNLhDY*sFd-r~lBNnAm-j5ztx}(94^)*Yhx65SY5j4CKg;J@G z>hQ;)bIc32$5sfMy@eYK<@Q<(GH3rmk>>%??eC=#exYdKAZJ&2AC$Q>+OOme*d{kC zZ>*)W-7|w((G3B)Z*L)rmErPFbm1GT*D3r;s+jl?0tYb@ZWjx?*P}8Mna8S399{7Q z8)EBne<uvlNk((?sSioQ;C#In{mc;E+Vub4&zMHG^V76fJ z#GI&^cu%rY$VbwL(91g^{|F^4xbp)#RIy7*wwxhJ4c% zK%_VAq*Z>nLklzQrns7{G0Km8ov~$vyqI!HfADer_ZsVtht+J_kNt?L4NND}$uXaH zW)^6)%u&i-BwGyc##W+n^a;uy9C=N?p5S8Ey)@R`tzPZ9dz zp@vw>$iz48I#3FgAL{ucq5Y3e;^ry|@ZP!Xo)q|#JyEjv3ZL~pz<>d*=q+elX;k8H zWY4-1k9*}QM$1+)GcRtNESIwxTc&w_j%P??CqjC&hd19fAB9(i4#14Wnn5AEV{AtQ z9D9$ymjcuOQTvP4*YEH0MdXc#Rpb%?qzZZ4SjG!!ocwsdI4!$>qrU4OvYcHpQ**?{ z``GF4oSe@6X$B5ZnA4%A_|zBXSK_B!j)3iAF}Pa^B=5Wnkq#}0=WDK7_?$5NdjFjZ z(~g-W$4C;J+*aBG$S1i+?oS;yom4qCMUB%|(@` zqmW!alD8V4YzDyJmUjwbkCX9)bS1+o{M+ewq2zjXOj=OxuseLD5!>^EU&1Y{#w>>v zwLTvgJ~5VxpyZQD8%+@WD%*nwugukc@nVX2npZx-{DIvCTE+Q?PW$|^C=+tCP^8@V zh9euw%^nANr-GbGPTPzUt_d= ztEidgjKAAgYqXe;5x9L>_0C=oYW{1GS-a3(m>)R+b+%vk{3tVVDwiR1O*{j414@c9 z_y)*_0n5)TrGBGfm1aZ9hZt`DC^)kAwLdj~C*SleZBxVX2~e=a+Ks|u-S9~N!_jDs zP@x)$%ROlMdb6!pdAc!|=w95wZi?uS^b?6JFTVXK{#V1dy)8(%Me$l5BiKk>mz*HiGD*Gx z&oyA$PebrS8!*G88ITn?o<&n!gE{3%v8Evg*v=*mWx|Q97o6TBvH%8b5DIwsyI5 zcfkgaDd!)(amMU&25Kcl)xq>v{o;VTe@2tc=B%sd^&!<45O7iO03&YS*!K5chp~rX zb{D1xOR6q?_oh`b4BMu7bEip-=KP%jHOv?dAJ)|Voc>Yuuz5(-?EUubwhY$+24}gT zR+mEK$9K6|V$~+b($6Y!)1Fei9I8l=5>hVpV)0%;+4>Htky;Rs?T1Mlt|7P zSJR3t#He?N=7$|YNU7uGCe6cEKl^i6`)+@7V#u9W3K`+H)jSVK+lOZ~NT^%b!Dc`M zmtw92$=_S3{`786h>_ZY>GA>IJm4L;1EuKJ*`=@Npm&X71aF#=bC2(amvcHrm1K97Y&cXGUD5RwcgU@4 zBHEWCcI#)jw0H6ZX;v!$lSf~8c7G1-e}281RYgRlm0P06+J%A!sVT=gJH~>U!3&KY z6X|z&8s5kyZ7XImyH^e*_Xc}@K${jll4UxxNv)&?WOZD6H%h4~O)EvN`mx5i z6PrDE^K*ITCk#Tflb)XF5^dT~$a#qB+_JK-hb9eJ?BBRVF@AxD=y#02eN5_02M6f) zOVaFCpIy#wJc*MNG2H$;U?tti{7`LUK#V84b@1|^SC$HmY~x8nU)Xd}ruNtdsu_uTH@JMjCosI1+2-Sm|DDCzKLGRT|u@CYg^m{n?e4NXDv-`@JmFi_&#`b;CM-mrvhYS_PWQ-1b+BgLtF zyG#~KDAvUhcYhj9{u6)D=$tLtE#GU{v-NG$_UHN}jVm2rCHbQ^*u^yr0>L8#jhof= zwqZ8=<~pi5K4GD3)239a2M)O#1A4IODw#7b*4e~mwd>nk3ptN85>eE3OZ6^PK>QaW zDVOjNgoPWTrcv%9w&L+tD83WbNjexz_C$!g1t8gn2EYu?*|p@r5Y;e#@$&Nxs}AQ zor0P^WAK3=76%l?;4|^^m?%MMe-f906?JUh8?v#_%->UJ^b2{l)vg-G@_oPdWnSRVCqpY-yJ1MQlEpv21ss-o-m16y)rND7uw>G$+6TqCdmB3xayc{I8RceUC!5o z%100qk9sU9UOm%{Rj3YTGYz;lps^b=Pn@)1S60?js0W*|*(c?0l}%z=$9^^T5;qHn zov8s-V3lcf38{|DE-yzN&P1qY`zRhH_p97lV~p5%LbhQ*UP@Hz9OgbRt@y*L?gNXb z6YPGir!j(tMw_FRzJjNVLOiEnA}nL{g<%4!n(7Z$qgl=vH~1Rh-pX)DW! zg_T4WEeBDLE#VydnhcSzF*QkBR*UDSokG4U(uti>LU{JQU%8`!kK8R(5MLwCV4bqN z@_ciJdgOkTq-kWRblhNOki!?&b$<@d6jwlrp~ws#&erg14_mKkHySuL6L%#+T+8k( zSfr$or5tc5EDm@7X@?lY)2|{vhuhH^ElAe~Rb+=1f^ePzA$Z>D7GQ)vytV)G-)V~h z4#C*fU^#=b=i81Y`FR$rJ-T3PtTD@LXPR1Q?-^b`uU;az$?Z;5^UU;%pzi5Y-rItrd^hDrXj}`ozIGrdz)AjY{nr80!O?&znh|gIrCP{a zuC8oO#rKR)!0M?;(Va|ub$h`mXQzL3m2+P_mN|2)PWLlCT{A^4{n=DUEdrD2KOqjg zP~zbO09fHPm{a`|hdx|t#;J`!U`5om!rQ?4xe?E=ABKwv%*DK0p zcYCg9f+$AjKf_0wF14_=GpWexau@CVe3aHZ=U>;4+J_BI7Zlrp{$oi7v%i;DDoqkm(KieE$mfiZvk#TWcq zK%Rv`;ef7vKI0%4R<698>4ob{H+2x(aRCRs4@ha04qHh1(ZK!MhpS>5XL~d)z@z24 z*CNu--#!1WY4exZ0Rf!o{(JLPSl`I*bdaA}uoH|==b_UW(QwE6982Ab9=Gv%su9@i z=ZhsIq|@X2psYa!pCcG}@eP(10` zv%=@**!HXGi+3bW@5^v~f_%G(h$Q5*vC#u6N1AmT-W>UQlaIs}btP)YtaJo?O{VQo zPeg;w_0sul<-%5w^a%r0ud0p5qFr*6lz1$RqoB`NeB?=QqoMYH-{|W{_*ofZS?PwH z4+D+AE@=WKJs&%N^+q+OIysUISvz!#`xWXxD&kp~j*Kj8^k8E&0teKZ#>Ei+8 z?uAFwG!0U|dhUx^VoR-#4O*LRDrU-rIox!Bss8rx2s$l(r`o%6;J>5UL|+6JgQA{4 zuHoB=coHIYF3hwB0Hy+y?_fX_MNR6WRQ-;kZtxEj4x33-0m`6A`h8bDCf$uqtitNp zt&S0I07VJgjyBsKXvP|9H)odQdmcMI3IU2xlD=;60I+G6CKhxV*oY{lNsx)hc{|f6 z26btH8w9Ct-lV?ztD9=D^!Y?Dox2%T#QX@GISVMjprcvG?tijPQl?o#?Y$o5Od$rs zQPu3%QAw$HsZ=Yb2A-V{n|E2U<|93f5142o`stp3(qXT9b=7>fit#;9f4&R||9e6$ zR%6`y^PXa}^<=#>h|$w8j?PbDVJ|OBwu;2U1YVb#kp2@SM7`@hM}T@=m_)Cn ziYx3VbRc4PECLpOg?0e;wVN8lZ4=FA(dr6+#sTj#K^q^$L))uxTMcJ`T(#F6Nik5^ zFbsx0buO%uB|pb7HnBqYBB&wK^=Dvh;0%q|;N;gLu3m-C~k zW+DBJuq?G0PE5#8v*CMKJU!V+b}4JjrOfST^#Crn zrYJbV_$v%3g~EiU4EY0sGn}ODP$^uC-)LPYY3`1a)yKolATbh*Htx-J4X3<>DA3$R zRnCTqzC^s9MMGSCRehypgfZV=L3scNA255huvDV39KZu4JhV>7v<|I6>IgZZgk^QL^royurHaV?as|LTTPKeGeZ&pT{rgYU90645d^8q@e)#ro`4Jir zQvN(1e_*nK`i8{pQkoni&goceiZE5QwUBw`Ks|g1Gt6K9JhXjPi%t@N3Ih6&CV$*D zY2Z^_;6VLvA)$AFh#6e`XE})-i-j4Q9w$FK3rI=cJh|2^e|CWNJFR`(;6tTtRSl+G zfRK#nZy}kpVW@*=1%5yF!N(7PK=oBXqKA0t>VX@7?E4eA=iqb4_LH$Nv541_ zwm=)t#2pE|PlJJWyVoB*xddEk3mcHw7y4VSA>$URuZ-`_w2QFgDj^KiWRiZL5`pIwfxoqm+wXN&;3#nU@eON!QvsZ>pr^>8r!7G9{J&L< zvVQkY`t04`GF9O(?gJ`IEr9IJv2TJ(z*v9FBh78|0xDIKN90}{093S?fAFRjT|0Xg zXz#C!m(qS~7@YugE${x;$3h>`O+5Bn-RSJ?+rz&#mi%w_w@3^8Rzh055qneM%WoyC zTP;AT&VM`i@0$d+I zp8vm1b_x&8qdT1!6)(-Vsrx5e$UPu{r0nR{^ZolRyxKu7AP#(u$J5jEb*8pwnEU9F zqeppb62$mEpZJiQt4XKRp_NCEAAk0bO0nEU{pi-ki%tzsN`ts+2gL#T-&0SQmb`WU zB0oDhYXafZ*|)1(#)hAJv>I*%csFN%DD5*%&Rr_pBi1=cKXB!{_V=km^0b-FZi{Hs zgBTh!rm`|qIdg=(x%+k9;+?_K(*b~XvH;+D{P$|r|NE*wpaV;s6{yqBj$A!3_&@mz zfkoYYg@uLDVBxQo8aP%!$JE9D`y>9-2OdCN_V=dzf9=nIZoc+08}OyDX4&=sv)j*} z0=%FH3YU2PdolO_^C}V0G$q~{@8SJF|KtC8{{QFQH3RP4!JEL34miaB{T=`BtBdD> zz6AXARQ;db&SwHRcc;vcT>qc6{ns^Zz_*2^YK8m{Z@&n*kj-}qBR>WTO?y{YtxZyIn}bNh68AdJWh z`(7Tm%08!|L1^EI57N-nbM89xOd61w1>Vagp($MACB^Ok`UO563M9pyPtgCTu~%>Z zSa5l5-7NplH@-U`0~RD>VT+Qs=OQCCe*O40)RJIoYTCO!S|cv_p!A=f-95N32NwED z-S_!F6Sz;8@m~{p=qb>YWI>PkKTrLb3^b(8YC5+sBqU`1$0sulTGAPDadoA1t0(R$ z*Z=EBd3p-yOWP^osMz04JpBO-`NoO1GyhE5)?0tkRolNzo%rY5^YeE{=*OCMH!YWK zA~iHL7-y)3At6xSZ{Kcuy1Tc&K5{%8C~k2X;RA&dZd*!Px-1=1N>^JZm)r`J_~!*4 zc=N{s@s9&(a{v4|f6D-4@H_n9yiZ^ZP5OX>Z_41`jI66sqdh%6wa)pi0mNR!sz2IY z?74?UMppe!J+H*<>?6-7?(Pr5t1tf3&Y~62CRm z8J$ouQw(E|V+WbcFsK~H7JGZSzdYCLnsvYTb3gZb*7Cly zPU?Zb{1A003NeIotd&v9Ii?xdpJNfIg=uZ&+6osIhBmR>M`&?k*3G1e%z~#+{V*6J z(e&CETcv6i+N;tcXdnEkClAVj4=T0TU=CN}qthTd|LKF_I)A?b{s@2cZ-%3D)(IlY zad&CqVDq9^V)=ItN5R0rIDnw7pg-iz%2_r$cEXoh(eZOgMn)P(eRNM@;n}S{iL{Z@(p|pXdL&^p&|K`?x)1eL!GgHvwlxhYi*Tn#}>xE7dp4aE9A< z07$$?&HLA*01$OR2QTlVn3ie=yGjCVhQ(AWzd2c>q!J7dJ?p+n4AdlY1~gD(bi1$%=|2Sdg zyHDdDE-b9UFF@vi4IiuKq~(n!a{a5a(UM^Bm{^8`J!)h|IECexS2NJ9vDa&-R20^o zqe9~i&UpzbB?WO_aS#M4N2!0n!T-L}#XSnPcnR^JC$24>@|s=lt!Zhq*Fp&d9kkXY zwW~7R)bHhxpfR^4ZEcw?!(#5+Q%TLey)`TM%1UDuMWV+2#1mfU3f+)_F+DNNDW}3p zSErFEi|$}>d_0&dt89USS#O9t;piGrH$|hH_|yM=ZJXWQe)lcTsJyw(Btd=WW_3;& zf+lDXJfo)uBoFQxYEf%k(GKDeo|EH8Mn_Twa#OiXV#anEu6EGem5${~&yG^N>*~tD3-`cYOvZjA_uYVZG2CK%uS54;+G{`w%ZA&q{5O^pvm^T>5^ZUQ&^w zbp><40_)T@2B<3>bizqocSSzlmg1&$Yg{tlOo_tZW zgYy5+S>~$1DS9%hk7mAVZWQ#4JujmJ0OP9|kJL`;bm=m&Z*_$`FYU5xXV?))!TU<_^fAQ#7li=m@~C@4{o?E>SSl)na5D(uO?mFc0=`v6GTc;`%o z;I#P|r+vzqPo0LH3aPByoo)Iq&4Tnm-#=ens59xltW;ekORk|6%>)a^_*3V7I^*`kmW+3P!hVa$k3z-#}IF_~&8dUaY{2`v65b^2>8}~8QJg}uXCPP?0P2U4 zF+V}SLz6f_++zm5yvmW7^9#n$Pf_J+*%2x(D(YijP;6Ayy$>a_{hFIDmM;*wE-JLI z&VyAJ;*t@;Cv0}wWcIn%tQVA-@Ak_zRg_!0k+}b-TgHI2V~gQ|A;mFpkD3p zDVU7rRZndwZ*hkpGP-o5_nJff!~OLTyLK>2(c9*}@tID<^-b8n{`&a5w^BeB3*SaO zp^#o(Fgr7Epn1k!*}eiis2F?q?nefzpijuwIQa}8D9 ziDRHYzQs8TS+$h(OEAdg?W69f=xxPN>;)9ez#%@zH6Wn6I}aca)fD>&c_JX~*uu)1 z5infAw;KeVf9dvwse7!euf2g z5DQY1HV>?))?pelzLPF+u6!lCh9H(^K`Us?u8!+2nujL4N7`^`FL6Xl2Ss10jnt@I zeTWh0VArsL$yKu8kgNR1&@5;0CgoVXlCP82P>H*Um2(+?#{3tf%R3LS8rk2+FQ*r~ zEzNg7Xz_Q>$^)yOrI3VxN$FXY@giN=;1YqsbuHfA&90tW+K~d$cI|gKlg`l@I0#f& z7Azqni6YzxkL`s)IYtLPk`Ag}&@(cxe|{{oxN23?ThI{XdqL~7YYUIZp}5iWk%fPc z%HB<(nfNmDTGX%C*mcqtjqo7d)VsW$5V zkLfKnC}&WsJ5-Yr>@f)Sjc`W^dP?Du38?LZnq467j-|3!n}p>?<^*|f`{SO!?=f0K z3<~VAJYDh{)CaM0mIfnS=^=BV=kiZ~xqY$MEo;SII34k>DmCx5*TOr~LGq2b<-RF3 zzYjBfq~x}sjYKl@%CFR73AW#PdVFk~2HE;(7fVV&61_B16R-EfM1XWL9iYqF08(}7 zHRvFM`*w7u`=Ova`2^;d;J#@NbW?DH4cex@(O8Wr^Z?GU(=WULua0>0EUK4cOJ8G! z7-8Wy7bTz(9`H_Y5#PjHy|(`WoQxLDDJ!t#Gq@;?Oq;0A7;;X2`^DL~;AzuqVeIWY zkfi~U{FdrkJ)#VxB_0^Fg@Yi2ZfVu}(h4w!N9)EgYsy}P^mhlM>p;FVL~J;0R$Z*v zKEw^lJynB|kIl9=11|Nul$G|CNy3T?k{LS2zbvpsbUK*8+720GV`7>Qi_k52MTZ}( zByJh$srz`ollo6PtRiE_*+LaNewU0Ara4c;#>C-8IA_Lykpf}?DM%l@M9*Ab{;gBxLx)9%Z-kvpfSoEppZ=^1P#6f^E(Y0NvyTU}!_Gg@;h?c=iu zNpOIzKGmh*r&%}T6>s1o&1|JW&AgY!uTigDt3&uWR%+#sw>zWfr^;h{by!@tFLmvBCvKh(WbhYF<| zmg5JSM%L==xhW^w%tUi4EdTHJg2YI$l&Xxh9^V&6}59 zy8Xc%($xxB=23Q|fQj|i&Tj{)J!lRSw_JUT1mRrFhG|_BTr`1r%*=D+wI^ zPE>QO45*tQXNHYa1bl{meU`V{#p3)N!DoWwq>;xl!Wdcqo9yQmLmGyN^rUoZo^=(C z)lG4TE=xfxE-lLFpKIzKy$Y^UcBB1#VVpBOe7^;J=V0Hj7k!i#*Voq)P$dU0vvFN3 zD~wbKQPsNLp`+7qP?6034I|Q(D2Kuwha3zaJ6Vw_I;R;mj^HvMl8i)7Z{~9I)5!T+ zsxB(!0w!&Pu-EOgd&eTN!53BUMz;v?JvL6%y;y|bETNX7f1MRFB0^95kmQvOto;1a zz-_6}sRc`wc$Yrwv)18wr=YIkqzSF@y!FX-dRKo9OL-f+dbwn5E5xB8D31lzcew2Q zy{7*rUnkz;xNwI0MC0Fl^E0-Ly2m-yPAGtk7mAuEN{D&*qb8M^3niP>g`H=d{sUW}mF z;`9}{_2yij^JKP`!ztAPoKIMT*)K8w1ZPy1mErYBhDG^J4B$*!_GIk_5q3&!S7D@|vTx9fIN%h(^MoNe6i}w;auauhkL%{ZK2rCO_!Du#%HM=S)mcqN$Xotjwb-N zA>GX2zVmgw2e65@!o1C4M0$DksRK`a}d} Date: Thu, 20 Feb 2025 11:19:57 +0800 Subject: [PATCH 03/40] docs:add taos_slow_sql_detail & log principle --- docs/en/14-reference/01-components/01-taosd.md | 6 +++--- docs/zh/14-reference/01-components/01-taosd.md | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/14-reference/01-components/01-taosd.md b/docs/en/14-reference/01-components/01-taosd.md index 246469f6e9..c7519a955d 100644 --- a/docs/en/14-reference/01-components/01-taosd.md +++ b/docs/en/14-reference/01-components/01-taosd.md @@ -548,7 +548,7 @@ The `taosd_vnodes_role` table records virtual node role information. | sql | VARCHAR | | sql sting | | process\_name | VARCHAR | | process name | | process\_id | VARCHAR | | process id | -| db | VARCHAR | TAG | sql db | -| user | VARCHAR | TAG | sql user | -| ip | VARCHAR | TAG | sql ip | +| db | VARCHAR | TAG | which db the sql belong to | +| user | VARCHAR | TAG | the user that exec this sql | +| ip | VARCHAR | TAG | the client ip that exec this sql | | cluster\_id | VARCHAR | TAG | cluster id | \ No newline at end of file diff --git a/docs/zh/14-reference/01-components/01-taosd.md b/docs/zh/14-reference/01-components/01-taosd.md index d58e0570e5..7377833e10 100644 --- a/docs/zh/14-reference/01-components/01-taosd.md +++ b/docs/zh/14-reference/01-components/01-taosd.md @@ -1702,6 +1702,6 @@ taosd 会将监控指标上报给 taosKeeper,这些监控指标会被 taosKeep | process\_id | VARCHAR | | 进程 id | | db | VARCHAR | TAG | 执行 sql 所属数据库 | | user | VARCHAR | TAG | 执行 sql 语句的用户 | -| ip | VARCHAR | TAG | 记录执行 sql 语句的 ip 地址 | +| ip | VARCHAR | TAG | 记录执行 sql 语句的 client 的 ip 地址 | | cluster\_id | VARCHAR | TAG | cluster id | From 99d6086c5a60eca931ea1b0c18e9e76a1e4a7762 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 20 Feb 2025 15:07:24 +0800 Subject: [PATCH 04/40] enh(stream): kill too long checkpoint trans. --- source/dnode/mnode/impl/inc/mndStream.h | 3 +- source/dnode/mnode/impl/src/mndStream.c | 16 +++----- source/dnode/mnode/impl/src/mndStreamTrans.c | 42 +++++++++++++++++--- 3 files changed, 45 insertions(+), 16 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index def817377d..d694dc67eb 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -116,7 +116,7 @@ void mndReleaseStream(SMnode *pMnode, SStreamObj *pStream); int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb); int32_t mndPersistStream(STrans *pTrans, SStreamObj *pStream); int32_t mndStreamRegisterTrans(STrans *pTrans, const char *pTransName, int64_t streamId); -int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, SArray* pSlowChkptTrans); +int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, SArray*pLongChkptTrans); int32_t mndStreamTransConflictCheck(SMnode *pMnode, int64_t streamId, const char *pTransName, bool lock); int32_t mndStreamGetRelTrans(SMnode *pMnode, int64_t streamId); @@ -159,6 +159,7 @@ void removeTasksInBuf(SArray *pTaskIds, SStreamExecInfo *pExecInfo); int32_t mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList, SVgroupChangeInfo *pInfo); void killAllCheckpointTrans(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo); +void killChkptAndResetStreamTask(SMnode *pMnode, SArray *pLongChkpts); bool isNodeUpdateTransActive(); int32_t createStreamTaskIter(SStreamObj *pStream, SStreamTaskIter **pIter); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index ffd271fd00..4cd4721bc4 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1260,6 +1260,7 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { int32_t numOfCheckpointTrans = 0; SArray *pLongChkpts = NULL; SArray *pList = NULL; + int64_t now = taosGetTimestampMs(); if ((code = mndCheckTaskAndNodeStatus(pMnode)) != 0) { return TSDB_CODE_STREAM_TASK_IVLD_STATUS; @@ -1271,7 +1272,7 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { return terrno; } - pLongChkpts = taosArrayInit(4, sizeof(int64_t)); + pLongChkpts = taosArrayInit(4, sizeof(SStreamTransInfo)); if (pLongChkpts == NULL) { mError("failed to init long checkpoint list, not handle stream checkpoint, code:%s", tstrerror(terrno)); taosArrayDestroy(pList); @@ -1282,26 +1283,21 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { code = mndStreamClearFinishedTrans(pMnode, &numOfCheckpointTrans, pLongChkpts); if (code) { mError("failed to clear finish trans, code:%s", tstrerror(code)); + + taosArrayDestroy(pList); + taosArrayDestroy(pLongChkpts); return code; } // kill long exec checkpoint and set task status if (taosArrayGetSize(pLongChkpts) > 0) { - //todo: - - for(int32_t i = 0; i < taosArrayGetSize(pLongChkpts); ++i) { - - mndKillTransImpl(pMnode, xx, ""); - mndCreateStreamResetStatusTrans(pMnode, pStream, chkptId); - } + killChkptAndResetStreamTask(pMnode, pLongChkpts); taosArrayDestroy(pList); taosArrayDestroy(pLongChkpts); return TSDB_CODE_SUCCESS; } - int64_t now = taosGetTimestampMs(); - while ((pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream)) != NULL) { int64_t duration = now - pStream->checkpointFreq; if (duration < tsStreamCheckpointInterval * 1000) { diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index ce82522029..4c3ba0c077 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -16,6 +16,8 @@ #include "mndStream.h" #include "mndTrans.h" +#define MAX_CHKPT_EXEC_ELAPSED (60*1000) // 60s + typedef struct SKeyInfo { void *pKey; int32_t keyLen; @@ -31,7 +33,7 @@ int32_t mndStreamRegisterTrans(STrans *pTrans, const char *pTransName, int64_t s return taosHashPut(execInfo.transMgmt.pDBTrans, &streamId, sizeof(streamId), &info, sizeof(SStreamTransInfo)); } -int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, SArray* pSlowChkptTrans) { +int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, SArray*pLongChkptTrans) { size_t keyLen = 0; void *pIter = NULL; SArray *pList = taosArrayInit(4, sizeof(SKeyInfo)); @@ -67,11 +69,11 @@ int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, // last for 10min, kill it int64_t dur = now - pTrans->createdTime; - if ((dur >= 600 * 1000) && pSlowChkptTrans != NULL) { + if ((dur >= MAX_CHKPT_EXEC_ELAPSED) && (pLongChkptTrans != NULL)) { mInfo("long chkpt transId:%d, start:%" PRId64 - " exec duration:%.2fs, beyond threshold 10min, kill it and reset task status", - pTrans->id, pTrans->createdTime, dur / 1000.0); - taosArrayPush(pSlowChkptTrans, &pEntry->transId); + " exec duration:%.2fs, beyond threshold %.2f min, kill it and reset task status", + pTrans->id, pTrans->createdTime, dur / 1000.0, MAX_CHKPT_EXEC_ELAPSED/(1000*60.0)); + taosArrayPush(pLongChkptTrans, pEntry); } } mndReleaseTrans(pMnode, pTrans); @@ -371,3 +373,33 @@ void killAllCheckpointTrans(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo) { mDebug("complete clear checkpoints in all Dbs"); } + +void killChkptAndResetStreamTask(SMnode *pMnode, SArray* pLongChkpts) { + int32_t code = 0; + int64_t now = taosGetTimestampMs(); + int32_t num = taosArrayGetSize(pLongChkpts); + + mInfo("start to kill %d long checkpoint trans", num); + + for(int32_t i = 0; i < num; ++i) { + SStreamTransInfo* pTrans = (SStreamTransInfo*) taosArrayGet(pLongChkpts, i); + if (pTrans == NULL) { + continue; + } + + double el = (now - pTrans->startTime) / 1000.0; + mInfo("stream:%s id:%" PRIx64 " ongoing checkpoint trans, id:%d, elapsed time:%.2fs killed", pTrans->name, + pTrans->streamId, pTrans->transId, el); + + SStreamObj *p = NULL; + code = mndGetStreamObj(pMnode, pTrans->streamId, &p); + if (code == 0 && p != NULL) { + mndKillTransImpl(pMnode, pTrans->transId, p->sourceDb); + + mDebug("create reset task trans for stream:%s 0x%" PRIx64, pTrans->name, pTrans->streamId); + mndCreateStreamResetStatusTrans(pMnode, p, p->checkpointId); + + sdbRelease(pMnode->pSdb, p); + } + } +} \ No newline at end of file From 15384dd0220a487a57233ac74a0f98d758e4d978 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 20 Feb 2025 16:18:56 +0800 Subject: [PATCH 05/40] docs:add desc for charset --- docs/en/14-reference/01-components/01-taosd.md | 10 +++++----- docs/zh/14-reference/01-components/01-taosd.md | 1 + 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/en/14-reference/01-components/01-taosd.md b/docs/en/14-reference/01-components/01-taosd.md index c7519a955d..449e529cf2 100644 --- a/docs/en/14-reference/01-components/01-taosd.md +++ b/docs/en/14-reference/01-components/01-taosd.md @@ -84,12 +84,12 @@ After modifying configuration file parameters, you need to restart the *taosd* s |Parameter Name |Supported Version |Dynamic Modification|Description| |-----------------------|-------------------------|--------------------|------------| -|timezone | |Not supported |Time zone; defaults to dynamically obtaining the current time zone setting from the system| -|locale | |Not supported |System locale information and encoding format, defaults to obtaining from the system| -|charset | |Not supported |Character set encoding, defaults to obtaining from the system| +|timezone | | since 3.1.0.0 |Time zone; defaults to dynamically obtaining the current time zone setting from the system| +|locale | | since 3.1.0.0 |System locale information and encoding format, defaults to obtaining from the system| +|charset | | since 3.1.0.0 |Character set encoding, defaults to obtaining from the system| :::info - +#### Explanation of Regional Related Parameters 1. To address the issue of data writing and querying across multiple time zones, TDengine uses Unix Timestamps to record and store timestamps. The nature of Unix Timestamps ensures that the timestamps generated are consistent at any given moment across any time zone. It is important to note that the conversion to Unix Timestamps is done on the client side. To ensure that other forms of time on the client are correctly converted to Unix Timestamps, it is necessary to set the correct time zone. On Linux/macOS, the client automatically reads the time zone information set by the system. Users can also set the time zone in the configuration file in various ways. For example: @@ -551,4 +551,4 @@ The `taosd_vnodes_role` table records virtual node role information. | db | VARCHAR | TAG | which db the sql belong to | | user | VARCHAR | TAG | the user that exec this sql | | ip | VARCHAR | TAG | the client ip that exec this sql | -| cluster\_id | VARCHAR | TAG | cluster id | \ No newline at end of file +| cluster\_id | VARCHAR | TAG | cluster id | diff --git a/docs/zh/14-reference/01-components/01-taosd.md b/docs/zh/14-reference/01-components/01-taosd.md index 7377833e10..58e654f982 100644 --- a/docs/zh/14-reference/01-components/01-taosd.md +++ b/docs/zh/14-reference/01-components/01-taosd.md @@ -459,6 +459,7 @@ taosd 命令行参数如下 - 支持版本:从 v3.1.0.0 版本开始引入 :::info +#### 区域相关参数说明 1. 为应对多时区的数据写入和查询问题,TDengine 采用 Unix 时间戳(Unix Timestamp)来记录和存储时间戳。Unix 时间戳的特点决定了任一时刻不论在任何时区,产生的时间戳均一致。需要注意的是,Unix 时间戳是在客户端完成转换和记录。为了确保客户端其他形式的时间转换为正确的 Unix 时间戳,需要设置正确的时区。 在 Linux/macOS 中,客户端会自动读取系统设置的时区信息。用户也可以采用多种方式在配置文件设置时区。例如: From 0a672f1b9628a5d24e9f16bb5c1a3b11db6f4767 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 20 Feb 2025 16:55:10 +0800 Subject: [PATCH 06/40] refactor(stream): update logs. --- source/dnode/mnode/impl/src/mndStreamTrans.c | 7 ++++--- source/libs/stream/src/streamExec.c | 9 ++++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index 4c3ba0c077..9c11d99c35 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -388,7 +388,7 @@ void killChkptAndResetStreamTask(SMnode *pMnode, SArray* pLongChkpts) { } double el = (now - pTrans->startTime) / 1000.0; - mInfo("stream:%s id:%" PRIx64 " ongoing checkpoint trans, id:%d, elapsed time:%.2fs killed", pTrans->name, + mInfo("stream:0x%" PRIx64 " start to kill ongoing long checkpoint transId:%d, elapsed time:%.2fs. killed", pTrans->streamId, pTrans->transId, el); SStreamObj *p = NULL; @@ -396,9 +396,10 @@ void killChkptAndResetStreamTask(SMnode *pMnode, SArray* pLongChkpts) { if (code == 0 && p != NULL) { mndKillTransImpl(pMnode, pTrans->transId, p->sourceDb); - mDebug("create reset task trans for stream:%s 0x%" PRIx64, pTrans->name, pTrans->streamId); - mndCreateStreamResetStatusTrans(pMnode, p, p->checkpointId); + mDebug("stream:%s 0x%" PRIx64 " transId:%d checkpointId:%" PRId64 " create reset task trans", p->name, + pTrans->streamId, pTrans->transId, p->checkpointId); + mndCreateStreamResetStatusTrans(pMnode, p, p->checkpointId); sdbRelease(pMnode->pSdb, p); } } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index ee34648a47..8ee06a82db 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -777,7 +777,8 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { int32_t code = 0; // merge multiple input data if possible in the input queue. - stDebug("s-task:%s start to extract data block from inputQ", id); + int64_t st = taosGetTimestampMs(); + stDebug("s-task:%s start to extract data block from inputQ, ts:%" PRId64, id, st); while (1) { int32_t blockSize = 0; @@ -807,8 +808,6 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { return 0; } - int64_t st = taosGetTimestampMs(); - EExtractDataCode ret = streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); if (ret == EXEC_AFTER_IDLE) { streamTaskSetIdleInfo(pTask, MIN_INVOKE_INTERVAL); @@ -825,6 +824,10 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { // dispatch checkpoint msg to all downstream tasks int32_t type = pInput->type; if (type == STREAM_INPUT__CHECKPOINT_TRIGGER) { + + // Injection error: for automatic kill long trans test + taosMsleep(50*1000); + code = streamProcessCheckpointTriggerBlock(pTask, (SStreamDataBlock*)pInput); if (code != 0) { stError("s-task:%s failed to process checkpoint-trigger block, code:%s", pTask->id.idStr, tstrerror(code)); From 6d5984421f25b8cf5e506edee68c8e7ef8fcff2f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 20 Feb 2025 17:27:41 +0800 Subject: [PATCH 07/40] refactor(stream): update the long trans threshold. --- source/dnode/mnode/impl/src/mndStreamTrans.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index 9c11d99c35..fb990c6fe9 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -16,7 +16,7 @@ #include "mndStream.h" #include "mndTrans.h" -#define MAX_CHKPT_EXEC_ELAPSED (60*1000) // 60s +#define MAX_CHKPT_EXEC_ELAPSED (600*1000) // 600s typedef struct SKeyInfo { void *pKey; From 4edc21e446229adeb63a2962f54d87d130f86970 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 20 Feb 2025 17:42:50 +0800 Subject: [PATCH 08/40] refactor(stream): disable error injection. --- source/libs/stream/src/streamExec.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 8ee06a82db..267dc88807 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -824,10 +824,10 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { // dispatch checkpoint msg to all downstream tasks int32_t type = pInput->type; if (type == STREAM_INPUT__CHECKPOINT_TRIGGER) { - +#if 0 // Injection error: for automatic kill long trans test taosMsleep(50*1000); - +#endif code = streamProcessCheckpointTriggerBlock(pTask, (SStreamDataBlock*)pInput); if (code != 0) { stError("s-task:%s failed to process checkpoint-trigger block, code:%s", pTask->id.idStr, tstrerror(code)); From ea549364d490f82fe7c6bbcf0ae9bc04c5c8e7b2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 21 Feb 2025 16:38:26 +0800 Subject: [PATCH 09/40] fix(stream): free array and check the return values. --- source/dnode/mnode/impl/src/mndStream.c | 6 ++++++ source/dnode/mnode/impl/src/mndStreamTrans.c | 10 ++++++++-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 4cd4721bc4..18e404564d 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1328,6 +1328,8 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { int32_t size = taosArrayGetSize(pList); if (size == 0) { taosArrayDestroy(pList); + taosArrayDestroy(pLongChkpts); + return code; } @@ -1340,6 +1342,8 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { "checkpoint trans are not allowed, wait for 30s", numOfQual, tsStreamCheckpointInterval, numOfCheckpointTrans, tsMaxConcurrentCheckpoint); taosArrayDestroy(pList); + taosArrayDestroy(pLongChkpts); + return code; } @@ -1379,6 +1383,8 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { } taosArrayDestroy(pList); + taosArrayDestroy(pLongChkpts); + return code; } diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index fb990c6fe9..f4f7c65a00 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -73,7 +73,10 @@ int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt, mInfo("long chkpt transId:%d, start:%" PRId64 " exec duration:%.2fs, beyond threshold %.2f min, kill it and reset task status", pTrans->id, pTrans->createdTime, dur / 1000.0, MAX_CHKPT_EXEC_ELAPSED/(1000*60.0)); - taosArrayPush(pLongChkptTrans, pEntry); + void* p = taosArrayPush(pLongChkptTrans, pEntry); + if (p == NULL) { + mError("failed to add long checkpoint trans, transId:%d, code:%s", pEntry->transId, tstrerror(terrno)); + } } } mndReleaseTrans(pMnode, pTrans); @@ -399,7 +402,10 @@ void killChkptAndResetStreamTask(SMnode *pMnode, SArray* pLongChkpts) { mDebug("stream:%s 0x%" PRIx64 " transId:%d checkpointId:%" PRId64 " create reset task trans", p->name, pTrans->streamId, pTrans->transId, p->checkpointId); - mndCreateStreamResetStatusTrans(pMnode, p, p->checkpointId); + code = mndCreateStreamResetStatusTrans(pMnode, p, p->checkpointId); + if (code) { + mError("stream:%s 0x%"PRIx64" failed to create reset stream task, code:%s", p->name, p->uid, tstrerror(code)); + } sdbRelease(pMnode->pSdb, p); } } From 4d21d5e055294846f3685a715cd36e3a947478e6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 24 Feb 2025 13:48:21 +0800 Subject: [PATCH 10/40] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndStream.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 18e404564d..6af54aeb2e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1298,6 +1298,8 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { return TSDB_CODE_SUCCESS; } + taosArrayDestroy(pLongChkpts); + while ((pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream)) != NULL) { int64_t duration = now - pStream->checkpointFreq; if (duration < tsStreamCheckpointInterval * 1000) { From 91271f3fa46eb25ea208dfe8157662737bfbfa11 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 24 Feb 2025 13:53:28 +0800 Subject: [PATCH 11/40] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 6 ------ 1 file changed, 6 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 6af54aeb2e..337e07756b 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1330,8 +1330,6 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { int32_t size = taosArrayGetSize(pList); if (size == 0) { taosArrayDestroy(pList); - taosArrayDestroy(pLongChkpts); - return code; } @@ -1344,8 +1342,6 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { "checkpoint trans are not allowed, wait for 30s", numOfQual, tsStreamCheckpointInterval, numOfCheckpointTrans, tsMaxConcurrentCheckpoint); taosArrayDestroy(pList); - taosArrayDestroy(pLongChkpts); - return code; } @@ -1385,8 +1381,6 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { } taosArrayDestroy(pList); - taosArrayDestroy(pLongChkpts); - return code; } From dd5bee0120dbaea19b261715972114e96aed4dfe Mon Sep 17 00:00:00 2001 From: "pengrongkun94@qq.com" Date: Mon, 24 Feb 2025 15:20:40 +0800 Subject: [PATCH 12/40] test performance --- source/client/src/clientStmt.c | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/source/client/src/clientStmt.c b/source/client/src/clientStmt.c index 4f912ec077..3a7ffa40d2 100644 --- a/source/client/src/clientStmt.c +++ b/source/client/src/clientStmt.c @@ -39,12 +39,19 @@ static FORCE_INLINE int32_t stmtAllocQNodeFromBuf(STableBufInfo* pTblBuf, void** } bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { - (void)taosThreadMutexLock(&pStmt->queue.mutex); + int i = 0; while (0 == atomic_load_64((int64_t*)&pStmt->queue.qRemainNum)) { - (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); - if (atomic_load_8((int8_t*)&pStmt->queue.stopQueue)) { + if (i < 5000) { + taosUsleep(1); + i++; + } else { + (void)taosThreadMutexLock(&pStmt->queue.mutex); + (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); - return false; + if (atomic_load_8((int8_t*)&pStmt->queue.stopQueue)) { + (void)taosThreadMutexUnlock(&pStmt->queue.mutex); + return false; + } } } SStmtQNode* orig = pStmt->queue.head; @@ -53,7 +60,6 @@ bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { *param = node; (void)atomic_sub_fetch_64((int64_t*)&pStmt->queue.qRemainNum, 1); - (void)taosThreadMutexUnlock(&pStmt->queue.mutex); *param = node; @@ -62,15 +68,15 @@ bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { } void stmtEnqueue(STscStmt* pStmt, SStmtQNode* param) { - (void)taosThreadMutexLock(&pStmt->queue.mutex); pStmt->queue.tail->next = param; pStmt->queue.tail = param; pStmt->stat.bindDataNum++; (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); + + (void)taosThreadMutexLock(&pStmt->queue.mutex); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); - (void)taosThreadMutexUnlock(&pStmt->queue.mutex); } @@ -423,11 +429,9 @@ void stmtResetQueueTableBuf(STableBufInfo* pTblBuf, SStmtQueue* pQueue) { pTblBuf->buffIdx = 1; pTblBuf->buffOffset = sizeof(*pQueue->head); - (void)taosThreadMutexLock(&pQueue->mutex); pQueue->head = pQueue->tail = pTblBuf->pCurBuff; pQueue->qRemainNum = 0; pQueue->head->next = NULL; - (void)taosThreadMutexUnlock(&pQueue->mutex); } int32_t stmtCleanExecInfo(STscStmt* pStmt, bool keepTable, bool deepClean) { From 1bf82888e9e35e4b8d69298fa2e7f903780dbfa7 Mon Sep 17 00:00:00 2001 From: "pengrongkun94@qq.com" Date: Mon, 24 Feb 2025 15:55:35 +0800 Subject: [PATCH 13/40] fix thread block --- source/client/src/clientStmt.c | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/source/client/src/clientStmt.c b/source/client/src/clientStmt.c index 3a7ffa40d2..932b5e8980 100644 --- a/source/client/src/clientStmt.c +++ b/source/client/src/clientStmt.c @@ -48,12 +48,12 @@ bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { (void)taosThreadMutexLock(&pStmt->queue.mutex); (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); - if (atomic_load_8((int8_t*)&pStmt->queue.stopQueue)) { - (void)taosThreadMutexUnlock(&pStmt->queue.mutex); - return false; - } } } + if (pStmt->queue.stopQueue) { + (void)taosThreadMutexUnlock(&pStmt->queue.mutex); + return false; + } SStmtQNode* orig = pStmt->queue.head; SStmtQNode* node = pStmt->queue.head->next; pStmt->queue.head = pStmt->queue.head->next; @@ -61,20 +61,16 @@ bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { (void)atomic_sub_fetch_64((int64_t*)&pStmt->queue.qRemainNum, 1); - - *param = node; - return true; } void stmtEnqueue(STscStmt* pStmt, SStmtQNode* param) { - pStmt->queue.tail->next = param; pStmt->queue.tail = param; pStmt->stat.bindDataNum++; (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); - + (void)taosThreadMutexLock(&pStmt->queue.mutex); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); @@ -782,7 +778,7 @@ void* stmtBindThreadFunc(void* param) { STscStmt* pStmt = (STscStmt*)param; while (true) { - if (atomic_load_8((int8_t*)&pStmt->queue.stopQueue)) { + if (pStmt->queue.stopQueue) { break; } @@ -1634,6 +1630,7 @@ int stmtClose(TAOS_STMT* stmt) { STMT_DLOG_E("start to free stmt"); pStmt->queue.stopQueue = true; + (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadMutexLock(&pStmt->queue.mutex); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); From 417d1efa5d3131ca3d2faeeff248fb34b01202cb Mon Sep 17 00:00:00 2001 From: "pengrongkun94@qq.com" Date: Mon, 24 Feb 2025 16:33:01 +0800 Subject: [PATCH 14/40] fix stmt2 --- source/client/src/clientStmt2.c | 31 ++++++++++++++++++------------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/source/client/src/clientStmt2.c b/source/client/src/clientStmt2.c index 8e517eb5f2..bf98ba6358 100644 --- a/source/client/src/clientStmt2.c +++ b/source/client/src/clientStmt2.c @@ -39,34 +39,40 @@ static FORCE_INLINE int32_t stmtAllocQNodeFromBuf(STableBufInfo* pTblBuf, void** } static bool stmtDequeue(STscStmt2* pStmt, SStmtQNode** param) { - (void)taosThreadMutexLock(&pStmt->queue.mutex); + int i = 0; while (0 == atomic_load_64((int64_t*)&pStmt->queue.qRemainNum)) { - (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); - if (atomic_load_8((int8_t*)&pStmt->queue.stopQueue)) { + if (i < 5000) { + taosUsleep(1); + i++; + } else { + (void)taosThreadMutexLock(&pStmt->queue.mutex); + (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); - return false; } } + if (pStmt->queue.stopQueue) { + (void)taosThreadMutexUnlock(&pStmt->queue.mutex); + return false; + } SStmtQNode* orig = pStmt->queue.head; SStmtQNode* node = pStmt->queue.head->next; pStmt->queue.head = pStmt->queue.head->next; *param = node; (void)atomic_sub_fetch_64((int64_t*)&pStmt->queue.qRemainNum, 1); - (void)taosThreadMutexUnlock(&pStmt->queue.mutex); return true; } static void stmtEnqueue(STscStmt2* pStmt, SStmtQNode* param) { - (void)taosThreadMutexLock(&pStmt->queue.mutex); - pStmt->queue.tail->next = param; pStmt->queue.tail = param; - pStmt->stat.bindDataNum++; - (void)atomic_add_fetch_64((int64_t*)&pStmt->queue.qRemainNum, 1); - (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); + pStmt->stat.bindDataNum++; + (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); + + (void)taosThreadMutexLock(&pStmt->queue.mutex); + (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); } @@ -343,11 +349,9 @@ static void stmtResetQueueTableBuf(STableBufInfo* pTblBuf, SStmtQueue* pQueue) { pTblBuf->buffIdx = 1; pTblBuf->buffOffset = sizeof(*pQueue->head); - (void)taosThreadMutexLock(&pQueue->mutex); pQueue->head = pQueue->tail = pTblBuf->pCurBuff; pQueue->qRemainNum = 0; pQueue->head->next = NULL; - (void)taosThreadMutexUnlock(&pQueue->mutex); } static int32_t stmtCleanExecInfo(STscStmt2* pStmt, bool keepTable, bool deepClean) { @@ -701,7 +705,7 @@ static void* stmtBindThreadFunc(void* param) { STscStmt2* pStmt = (STscStmt2*)param; while (true) { - if (atomic_load_8((int8_t*)&pStmt->queue.stopQueue)) { + if (pStmt->queue.stopQueue) { break; } @@ -1762,6 +1766,7 @@ int stmtClose2(TAOS_STMT2* stmt) { STMT_DLOG_E("start to free stmt"); pStmt->queue.stopQueue = true; + (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadMutexLock(&pStmt->queue.mutex); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); From 469515bdc3de581de51360e862c5e0498eb56b0f Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 24 Feb 2025 16:36:00 +0800 Subject: [PATCH 15/40] docs:add taos_slow_sql_detail & log principle --- docs/zh/26-tdinternal/12-log.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/26-tdinternal/12-log.md b/docs/zh/26-tdinternal/12-log.md index 6a401e8446..831486e464 100644 --- a/docs/zh/26-tdinternal/12-log.md +++ b/docs/zh/26-tdinternal/12-log.md @@ -54,17 +54,17 @@ TDengine 通过日志文件记录系统运行状态,帮助用户监控系统 }MonitorSlowLogData ``` - 说明: - - 因为客户端进程里可能存在很多个链接 connection,所以需要将慢查询日志根据 clusterId 来分组。分组方式通过临时文件名来实现,命名方式为 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand},processId 为进程ID,主要为了区分多个客户端的上报。 - - 如上图 connection 1 连接的是 cluster 1。connection 2,connection 3 连接的是 cluster 2,所以connection 1 的慢 sql 数据写入文件 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand},connection 2 和 connection 3的慢 sql 数据写入文件 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand} + - 因为客户端进程里可能存在很多个链接 connection,所以需要将慢查询日志根据 clusterId 来分组。分组方式通过临时文件名来实现,命名方式为 ```{tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand}```,processId 为进程ID,主要为了区分多个客户端的上报。 + - 如上图 connection 1 连接的是 cluster 1。connection 2,connection 3 连接的是 cluster 2,所以connection 1 的慢 sql 数据写入文件 ```{tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand}```,connection 2 和 connection 3的慢 sql 数据写入文件 ```{tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand}``` #### 上报逻辑 -- 读取 {tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand} 临时文件内容,每行数据作为 json 数组的一个元素,组装成 json 数组上报(文件里数据每接近 1M大小上报一次,上报成功后记录读取文件进度,上报采用异步上报方式。在 callback 里继续根据上次的进度,继续读取文件的内容上报,直至整个文件读取上报完毕,上报完毕后,会清空临时文件,callback 里成功或失败都会继续读取文件,失败时会记录上报失败的数据日志)。每接近 1M 上报一次主要为了防止文件太大,放在一次上报失败)。 +- 读取 ```{tmp dir}/tdengine_slow_log/tdengeine-{clusterId1}-{processId}-{rand}``` 临时文件内容,每行数据作为 json 数组的一个元素,组装成 json 数组上报(文件里数据每接近 1M大小上报一次,上报成功后记录读取文件进度,上报采用异步上报方式。在 callback 里继续根据上次的进度,继续读取文件的内容上报,直至整个文件读取上报完毕,上报完毕后,会清空临时文件,callback 里成功或失败都会继续读取文件,失败时会记录上报失败的数据日志)。每接近 1M 上报一次主要为了防止文件太大,放在一次上报失败)。 #### 上报时机 - 客户端运行过程中定时上报 - 每个 monitorInterval 时间间隔上报数据。 - 客户端正常退出 - 上报所有慢 sql 日志文件, 上报成功后,删除文件。 - 客户端异常退出 - - 异常退出后再次与某个集群(clusterId)建立新的链接后遍历 {tmp dir}/tdengine_slow_log/ 目录下tdengine-{clusterId}开头的所有文件进行重新上报(这些文件可能是另一个客户端进程或本进程正在操作的。所以每个文件打开时都需要添加文件锁),然后删除这个临时文件。 + - 异常退出后再次与某个集群(clusterId)建立新的链接后遍历 ```{tmp dir}/tdengine_slow_log/``` 目录下 ```tdengine-{clusterId}``` 开头的所有文件进行重新上报(这些文件可能是另一个客户端进程或本进程正在操作的。所以每个文件打开时都需要添加文件锁),然后删除这个临时文件。 #### 一些异常行为说明 - 因为上报数据和删除文件里的上报内容没法作为一个原子操作,所以如果上报后还没删除数据就 crash,可能导致下次重复上报,重复上报的数据会覆盖,并没丢失,影响很小。 - 另外为了保证性能, slow log thread 线程把慢 sql 日志写入临时文件缓存,只保证刷新到操作系统的磁盘缓冲区,并不真正每次都 fsync 到磁盘,所以如果机器断电,仍可能丢失数据。该异常出现概率很小,可以容忍此种情况下的数据丢失。 From 1c352aa1bc0ea332c248ab5e679e5ce83c27939d Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 24 Feb 2025 16:40:41 +0800 Subject: [PATCH 16/40] docs: minor changes --- docs/zh/10-third-party/05-bi/03-powerbi.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/10-third-party/05-bi/03-powerbi.md b/docs/zh/10-third-party/05-bi/03-powerbi.md index e55f6b6d88..ee8d656808 100644 --- a/docs/zh/10-third-party/05-bi/03-powerbi.md +++ b/docs/zh/10-third-party/05-bi/03-powerbi.md @@ -12,7 +12,7 @@ Power BI 是由 Microsoft 提供的一种商业分析工具。通过配置使用 - TDengine 3.3.4.0 以上版本集群已部署并正常运行(企业及社区版均可)。 - taosAdapter 能够正常运行,详细参考 [taosAdapter 参考手册](../../../reference/components/taosadapter)。 - 从 TDengine 官网下载最新的 Windows 操作系统 X64 客户端驱动程序并进行安装,详细参考 [安装 ODBC 驱动](../../../reference/connector/odbc/#安装)。 -- 安装完成 Power BI Desktop 软件并运行(如未安装,请从其官方地址下载最新的Windows操作系统 32/64 位版本)。 +- 安装完成 Power BI Desktop 软件并运行(如未安装,请从其官方地址下载最新的 Windows 操作系统 32/64 位版本)。 ## 配置数据源 @@ -29,8 +29,8 @@ Power BI 是由 Microsoft 提供的一种商业分析工具。通过配置使用 ### 使用说明 为了充分发挥 Power BI 在分析 TDengine中 数据方面的优势,用户需要先理解维度、度量、窗口切分查询、数据切分查询、时序和相关性等核心概念,之后通过自定义的 SQL 导入数据。 -- 维度:通常是分类(文本)数据,描述设备、测点、型号等类别信息。在 TDengine 的超级表中,使用标签列存储数据的维度信息,可以通过形如 “select distinct tbname, tag1, tag2 from supertable” 的SQL语法快速获得维度信息。 -- 度量:可以用于进行计算的定量(数值)字段,常见计算有求和、取平均值和最小值等。如果测点的采集周期为1s,那么一年就有 3000 多万条记录,把这些数据全部导入 Power BI 会严重影响其执行效率。在 TDengine 中,用户可以使用数据切分查询、窗口切分查询等语法,结合与窗口相关的伪列,把降采样后的数据导入Power BI 中,具体语法请参阅 TDengine 官方文档的特色查询功能部分。 +- 维度:通常是分类(文本)数据,描述设备、测点、型号等类别信息。在 TDengine 的超级表中,使用标签列存储数据的维度信息,可以通过形如 `select distinct tbname, tag1, tag2 from supertable` 的 SQL 语法快速获得维度信息。 +- 度量:可以用于进行计算的定量(数值)字段,常见计算有求和、取平均值和最小值等。如果测点的采集周期为1s,那么一年就有 3000 多万条记录,把这些数据全部导入 Power BI 会严重影响其执行效率。在 TDengine 中,用户可以使用数据切分查询、窗口切分查询等语法,结合与窗口相关的伪列,把降采样后的数据导入 Power BI 中,具体语法请参阅 TDengine 官方文档的特色查询功能部分。 - 窗口切分查询:比如温度传感器每秒采集一次数据,但须查询每隔 10min 的温度平均值,在这种场景下可以使用窗口子句来获得需要的降采样查询结果,对应的 SQL 形如 `select tbname, _wstart date,avg(temperature) temp from table interval(10m)`,其中,`_wstart` 是伪列,表示时间窗口起始时间,10m 表示时间窗口的持续时间,`avg(temperature)` 表示时间窗口内的聚合值。 - 数据切分查询:如果需要同时获取很多温度传感器的聚合数值,可对数据进行切分,然后在切分出的数据空间内进行一系列的计算,对应的 SQL 形如 `partition by part_list`。数据切分子句最常见的用法是在超级表查询中按标签将子表数据进行切分,将每个子表的数据独立出来,形成一条条独立的时间序列,方便针对各种时序场景的统计分析。 - 时序:在绘制曲线或者按照时间聚合数据时,通常需要引入日期表。日期表可以从 Excel 表格中导入,也可以在 TDengine 中执行 SQL 获取,例如 `select _wstart date, count(*) cnt from test.meters where ts between A and B interval(1d) fill(0)`,其中 fill 字句表示数据缺失情况下的填充模式,伪列 _wstart 则为要获取的日期列。 @@ -46,7 +46,7 @@ TDengine 采用了一种独特的数据模型,以优化时序数据的存储 根据如下步骤,便可以体验通过 Power BI 生成时序数据报表的功能。 -**第 1 步**,使用 TDengine 的 taosBenchMark 快速生成1000块智能电表3天的数据,采集频率为 1s。 +**第 1 步**,使用 TDengine 的 taosBenchMark 快速生成 1000 块智能电表 3 天的数据,采集频率为 1s。 ```shell taosBenchmark -t 1000 -n 259200 -S 1000 -y From cbd9504e8fd208308725eb6f147c5eb442b2ef92 Mon Sep 17 00:00:00 2001 From: "pengrongkun94@qq.com" Date: Mon, 24 Feb 2025 17:16:08 +0800 Subject: [PATCH 17/40] fix deadlock problem --- source/client/src/clientStmt.c | 10 ++++++---- source/client/src/clientStmt2.c | 8 +++++--- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/source/client/src/clientStmt.c b/source/client/src/clientStmt.c index 932b5e8980..279078b759 100644 --- a/source/client/src/clientStmt.c +++ b/source/client/src/clientStmt.c @@ -46,7 +46,9 @@ bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { i++; } else { (void)taosThreadMutexLock(&pStmt->queue.mutex); - (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); + if (0 == atomic_load_64((int64_t*)&pStmt->queue.qRemainNum)) { + (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); + } (void)taosThreadMutexUnlock(&pStmt->queue.mutex); } } @@ -69,9 +71,9 @@ void stmtEnqueue(STscStmt* pStmt, SStmtQNode* param) { pStmt->queue.tail = param; pStmt->stat.bindDataNum++; - (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadMutexLock(&pStmt->queue.mutex); + (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); } @@ -1630,9 +1632,9 @@ int stmtClose(TAOS_STMT* stmt) { STMT_DLOG_E("start to free stmt"); pStmt->queue.stopQueue = true; - (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); - + (void)taosThreadMutexLock(&pStmt->queue.mutex); + (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); diff --git a/source/client/src/clientStmt2.c b/source/client/src/clientStmt2.c index bf98ba6358..2d5029ca4d 100644 --- a/source/client/src/clientStmt2.c +++ b/source/client/src/clientStmt2.c @@ -46,7 +46,9 @@ static bool stmtDequeue(STscStmt2* pStmt, SStmtQNode** param) { i++; } else { (void)taosThreadMutexLock(&pStmt->queue.mutex); - (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); + if (0 == atomic_load_64((int64_t*)&pStmt->queue.qRemainNum)) { + (void)taosThreadCondWait(&pStmt->queue.waitCond, &pStmt->queue.mutex); + } (void)taosThreadMutexUnlock(&pStmt->queue.mutex); } } @@ -69,9 +71,9 @@ static void stmtEnqueue(STscStmt2* pStmt, SStmtQNode* param) { pStmt->queue.tail = param; pStmt->stat.bindDataNum++; - (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadMutexLock(&pStmt->queue.mutex); + (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); } @@ -1766,9 +1768,9 @@ int stmtClose2(TAOS_STMT2* stmt) { STMT_DLOG_E("start to free stmt"); pStmt->queue.stopQueue = true; - (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadMutexLock(&pStmt->queue.mutex); + (void)atomic_add_fetch_64(&pStmt->queue.qRemainNum, 1); (void)taosThreadCondSignal(&(pStmt->queue.waitCond)); (void)taosThreadMutexUnlock(&pStmt->queue.mutex); From b83456de81903459dd197f8873e30ef3a2e1c1a5 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Mon, 24 Feb 2025 20:47:02 +0800 Subject: [PATCH 18/40] refactor: implement dynamic loading of test case classes --- tests/pytest/util/cases.py | 25 +++++++++++++++++++++---- tests/system-test/test.py | 30 +++++++++++++++++++++--------- tests/test_new/test.py | 30 +++++++++++++++++++++--------- 3 files changed, 63 insertions(+), 22 deletions(-) diff --git a/tests/pytest/util/cases.py b/tests/pytest/util/cases.py index eee8809ad0..4899bedc69 100644 --- a/tests/pytest/util/cases.py +++ b/tests/pytest/util/cases.py @@ -51,6 +51,13 @@ class TDCases: def addCluster(self, name, case): self.clusterCases.append(TDCase(name, case)) + def get_local_classes(self, module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + def runAllLinux(self, conn): # TODO: load all Linux cases here runNum = 0 @@ -71,7 +78,10 @@ class TDCases: runNum = 0 for tmp in self.linuxCases: if tmp.name.find(fileName) != -1: - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[0]) + + case = case_class() case.init(conn, self._logSql, replicaVar) try: case.run() @@ -103,7 +113,9 @@ class TDCases: runNum = 0 for tmp in self.windowsCases: if tmp.name.find(fileName) != -1: - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[0]) + case = case_class() case.init(conn, self._logSql,replicaVar) try: case.run() @@ -117,12 +129,15 @@ class TDCases: def runAllCluster(self): # TODO: load all cluster case module here + testModule = self.__dynamicLoadModule(fileName) runNum = 0 for tmp in self.clusterCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[0]) + case = case_class() case.init() case.run() case.stop() @@ -138,7 +153,9 @@ class TDCases: for tmp in self.clusterCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[0]) + case = case_class() case.init() case.run() case.stop() diff --git a/tests/system-test/test.py b/tests/system-test/test.py index ab1bdc21d3..a05eed5a3c 100644 --- a/tests/system-test/test.py +++ b/tests/system-test/test.py @@ -24,6 +24,7 @@ import platform import socket import threading import importlib +import inspect print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -58,6 +59,16 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') # # run case on previous cluster # @@ -68,9 +79,10 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - moduleName = fileName.replace(".py", "").replace(sep, ".") - uModule = importlib.import_module(moduleName) - case = uModule.TDTestCase() + uModule = dynamicLoadModule(fileName) + + case_class = getattr(uModule, get_local_classes(uModule)[0]) + case = case_class() # create conn conn = taos.connect(host, config) @@ -350,10 +362,10 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -522,10 +534,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): diff --git a/tests/test_new/test.py b/tests/test_new/test.py index ab1bdc21d3..a05eed5a3c 100644 --- a/tests/test_new/test.py +++ b/tests/test_new/test.py @@ -24,6 +24,7 @@ import platform import socket import threading import importlib +import inspect print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -58,6 +59,16 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') # # run case on previous cluster # @@ -68,9 +79,10 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - moduleName = fileName.replace(".py", "").replace(sep, ".") - uModule = importlib.import_module(moduleName) - case = uModule.TDTestCase() + uModule = dynamicLoadModule(fileName) + + case_class = getattr(uModule, get_local_classes(uModule)[0]) + case = case_class() # create conn conn = taos.connect(host, config) @@ -350,10 +362,10 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -522,10 +534,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): From 566c961e627ea080a86290356829f678922b19c1 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Mon, 24 Feb 2025 20:54:16 +0800 Subject: [PATCH 19/40] refactor: implement dynamic loading of test case classes --- tests/army/test.py | 30 +++++++++++++++++++++--------- tests/develop-test/test.py | 26 ++++++++++++++++++++------ 2 files changed, 41 insertions(+), 15 deletions(-) diff --git a/tests/army/test.py b/tests/army/test.py index d37d08b406..bf14f19e3f 100644 --- a/tests/army/test.py +++ b/tests/army/test.py @@ -24,6 +24,7 @@ import platform import socket import threading import importlib +import inspect import toml @@ -56,6 +57,17 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') + # # run case on previous cluster # @@ -66,9 +78,9 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - moduleName = fileName.replace(".py", "").replace(sep, ".") - uModule = importlib.import_module(moduleName) - case = uModule.TDTestCase() + uModule = dynamicLoadModule(fileName) + case_class = getattr(uModule, get_local_classes(uModule)[0]) + case = case_class() # create conn conn = taos.connect(host, config) @@ -358,10 +370,10 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -530,10 +542,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): diff --git a/tests/develop-test/test.py b/tests/develop-test/test.py index 6b1c63a1c0..9106e38d9a 100644 --- a/tests/develop-test/test.py +++ b/tests/develop-test/test.py @@ -22,6 +22,9 @@ import json import platform import socket import threading +import inspect +import importlib +import os import toml @@ -57,6 +60,17 @@ def checkRunTimeError(): os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') + if __name__ == "__main__": fileName = "all" @@ -295,10 +309,10 @@ if __name__ == "__main__": updateCfgDictStr = "" # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if (json.dumps(updateCfgDict) == "{}") and hasattr( ucase, "updatecfgDict" ): @@ -434,10 +448,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[0]) + ucase = case_class() if json.dumps(updateCfgDict) == "{}": updateCfgDict = ucase.updatecfgDict if json.dumps(adapter_cfg_dict) == "{}": From 1e569acc3356116c24301a0bcefbd83a26928273 Mon Sep 17 00:00:00 2001 From: "pengrongkun94@qq.com" Date: Tue, 25 Feb 2025 10:45:16 +0800 Subject: [PATCH 20/40] change sleep loop times --- source/client/src/clientStmt.c | 2 +- source/client/src/clientStmt2.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/client/src/clientStmt.c b/source/client/src/clientStmt.c index 279078b759..b16d1e8aac 100644 --- a/source/client/src/clientStmt.c +++ b/source/client/src/clientStmt.c @@ -41,7 +41,7 @@ static FORCE_INLINE int32_t stmtAllocQNodeFromBuf(STableBufInfo* pTblBuf, void** bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { int i = 0; while (0 == atomic_load_64((int64_t*)&pStmt->queue.qRemainNum)) { - if (i < 5000) { + if (i < 10) { taosUsleep(1); i++; } else { diff --git a/source/client/src/clientStmt2.c b/source/client/src/clientStmt2.c index 2d5029ca4d..bc8b03ffd4 100644 --- a/source/client/src/clientStmt2.c +++ b/source/client/src/clientStmt2.c @@ -41,7 +41,7 @@ static FORCE_INLINE int32_t stmtAllocQNodeFromBuf(STableBufInfo* pTblBuf, void** static bool stmtDequeue(STscStmt2* pStmt, SStmtQNode** param) { int i = 0; while (0 == atomic_load_64((int64_t*)&pStmt->queue.qRemainNum)) { - if (i < 5000) { + if (i < 10) { taosUsleep(1); i++; } else { From 09a8a8131634c4ddaab45401d2b6d0af00274858 Mon Sep 17 00:00:00 2001 From: "pengrongkun94@qq.com" Date: Tue, 25 Feb 2025 15:06:59 +0800 Subject: [PATCH 21/40] fix review --- source/client/src/clientStmt.c | 1 - source/client/src/clientStmt2.c | 1 - 2 files changed, 2 deletions(-) diff --git a/source/client/src/clientStmt.c b/source/client/src/clientStmt.c index b16d1e8aac..165c7b8bfb 100644 --- a/source/client/src/clientStmt.c +++ b/source/client/src/clientStmt.c @@ -53,7 +53,6 @@ bool stmtDequeue(STscStmt* pStmt, SStmtQNode** param) { } } if (pStmt->queue.stopQueue) { - (void)taosThreadMutexUnlock(&pStmt->queue.mutex); return false; } SStmtQNode* orig = pStmt->queue.head; diff --git a/source/client/src/clientStmt2.c b/source/client/src/clientStmt2.c index bc8b03ffd4..75d763ec71 100644 --- a/source/client/src/clientStmt2.c +++ b/source/client/src/clientStmt2.c @@ -53,7 +53,6 @@ static bool stmtDequeue(STscStmt2* pStmt, SStmtQNode** param) { } } if (pStmt->queue.stopQueue) { - (void)taosThreadMutexUnlock(&pStmt->queue.mutex); return false; } SStmtQNode* orig = pStmt->queue.head; From aa33cf7456b764fa02c8d9f9b0fe9fafe0e6c03d Mon Sep 17 00:00:00 2001 From: kevin men Date: Tue, 25 Feb 2025 15:18:34 +0800 Subject: [PATCH 22/40] docs: update document structure of seeq/tableau/superset (#29920) * docs: adjust the document directory structure * docs: change chapter name * docs: modify the structure of the superset document * docs: modify the title of superset. * docs: adjusting the superset directory and correcting title errors * docs: Correct spelling errors * docs: delete empty rows from the ordered list * docs: modify the format of ordered sequences --- docs/en/10-third-party/05-bi/09-seeq.md | 16 ++-- docs/en/10-third-party/05-bi/11-superset.md | 71 +++++++++--------- docs/zh/10-third-party/05-bi/05-yhbi.md | 2 +- docs/zh/10-third-party/05-bi/09-seeq.md | 16 ++-- docs/zh/10-third-party/05-bi/11-superset.md | 82 +++++++++++---------- docs/zh/10-third-party/05-bi/12-tableau.md | 2 +- 6 files changed, 94 insertions(+), 95 deletions(-) diff --git a/docs/en/10-third-party/05-bi/09-seeq.md b/docs/en/10-third-party/05-bi/09-seeq.md index ba33d801a9..2010ffcbe2 100644 --- a/docs/en/10-third-party/05-bi/09-seeq.md +++ b/docs/en/10-third-party/05-bi/09-seeq.md @@ -22,8 +22,6 @@ Through the `TDengine Java connector`, Seeq can easily support querying time-ser ## Configure Data Source -### Configuration of JDBC Connector - **Step 1**, Check the data storage location ```shell @@ -42,9 +40,13 @@ sudo seeq restart Use a browser to visit ip:34216 and follow the instructions to enter the license. -## Load TDengine Time-Series Data +## Data Analysis -This chapter demonstrates how to use the Seeq software to load TDengine time-series data. +### Scenario Introduction + +The example scenario is a power system where users collect electricity usage data from power station instruments daily and store it in the TDengine cluster. Now, users want to predict how power consumption will develop and purchase more equipment to support it. User power consumption varies with monthly orders, and considering seasonal changes, power consumption will differ. This city is located in the northern hemisphere, so more electricity is used in summer. We simulate data to reflect these assumptions. + +### Data preparation **Step 1**, Create tables in TDengine. @@ -246,12 +248,6 @@ The source code is hosted on [GitHub Repository](https://github.com/sangshuduo/t } ``` -## Data Analysis - -### Scenario Introduction - -The example scenario is a power system where users collect electricity usage data from power station instruments daily and store it in the TDengine cluster. Now, users want to predict how power consumption will develop and purchase more equipment to support it. User power consumption varies with monthly orders, and considering seasonal changes, power consumption will differ. This city is located in the northern hemisphere, so more electricity is used in summer. We simulate data to reflect these assumptions. - ### Using Seeq Workbench Log in to the Seeq service page and create a new Seeq Workbench. By selecting data sources from search results and choosing different tools as needed, you can display data or make predictions. For detailed usage methods, refer to the [official knowledge base](https://support.seeq.com/space/KB/146440193/Seeq+Workbench). diff --git a/docs/en/10-third-party/05-bi/11-superset.md b/docs/en/10-third-party/05-bi/11-superset.md index 2ac8690c41..d8dc200fd6 100644 --- a/docs/en/10-third-party/05-bi/11-superset.md +++ b/docs/en/10-third-party/05-bi/11-superset.md @@ -12,31 +12,34 @@ Through the Python connector of TDengine, Superset can support TDengine data sou ## Prerequisites Prepare the following environment: -- TDengine is installed and running normally (both Enterprise and Community versions are available) -- taosAdapter is running normally, refer to [taosAdapter](../../../tdengine-reference/components/taosadapter/) -- Apache Superset version 2.1.0 or above is already installed, refre to [Apache Superset](https://superset.apache.org/) -## Install TDengine Python Connector +- TDengine 3.2.3.0 and above version is installed and running normally (both Enterprise and Community versions are available). +- taosAdapter is running normally, refer to [taosAdapter](../../../tdengine-reference/components/taosadapter/). +- Apache Superset version 2.1.0 or above is already installed, refre to [Apache Superset](https://superset.apache.org/). +- Install Python connector driver, refer to [Python Client Library](../../../tdengine-reference/client-libraries/python). +:::tip The Python connector of TDengine comes with a connection driver that supports Superset in versions 2.1.18 and later, which will be automatically installed in the Superset directory and provide data source services. -The connection uses the WebSocket protocol, so it is necessary to install the `taos-ws-py` component of TDengine separately. The complete installation script is as follows: -```bash -pip3 install taospy -pip3 install taos-ws-py -``` +::: -## Configure TDengine Connection In Superset +## Configure Data Source + +**Step 1**, enter the new database connection page, [Superset] -> [Setting] -> [Database Connections] -> [+DATABASE]. + +**Step 2**, select TDengine database connection, select the `TDengine` option from the drop-down list of [SUPPORTED DATABASES]. -**Step 1**, enter the new database connection page, "Superset" → "Setting" → "Database Connections" → "+DATABASE" -**Step 2**, select TDengine database connection, select the "TDengine" option from the drop-down list of "SUPPORTED DATABASES". :::tip If there is no TDengine option in the drop-down list, please confirm that the steps of installing, `Superset` is first and `Python Connector` is second. ::: -**Step 3**, write a name of connection in "DISPLAY NAME" -**Step 4**, The "SQLALCHEMY URL" field is a key connection information string, and it must be filled in correctly + +**Step 3**, write a name of connection in [DISPLAY NAME]. + +**Step 4**, The [SQLALCHEMY URL] field is a key connection information string, and it must be filled in correctly. + ```bash taosws://user:password@host:port ``` + | Parameter |

    Parameter Description
    | |:---------- |:--------------------------------------------------------- | |user | Username for logging into TDengine database | @@ -44,32 +47,34 @@ taosws://user:password@host:port |host | Name of the host where the TDengine database is located | |port | The port that provides WebSocket services, default is 6041 | -Example: -The TDengine database installed on this machine provides WebSocket service port 6041, using the default username and password, "SQLALCHEMY URL" is: +Example: + +The TDengine database installed on this machine provides WebSocket service port 6041, using the default username and password, `SQLALCHEMY URL` is: + ```bash taosws://root:taosdata@localhost:6041 ``` -**Step 5**, configure the connection string, click "TEST CONNECTION" to test if the connection can be successful. After passing the test, click the "CONNECT" button to complete the connection + +**Step 5**, configure the connection string, click "TEST CONNECTION" to test if the connection can be successful. After passing the test, click the "CONNECT" button to complete the connection. +## Data Analysis -## Start +### Data preparation -There is no difference in the use of TDengine data source compared to other data sources. Here is a brief introduction to basic data queries: -1. Click the "+" button in the upper right corner of the Superset interface, select "SQL query", and enter the query interface -2. Select the "TDengine" data source that has been created earlier from the dropdown list of "DATABASES" in the upper left corner -3. Select the name of the database to be operated on from the drop-down list of "SCHEMA" (system libraries are not displayed) -4. "SEE TABLE SCHEMA" select the name of the super table or regular table to be operated on (sub tables are not displayed) -5. Subsequently, the schema information of the selected table will be displayed in the following area -6. In the SQL editor area, any SQL statement that conforms to TDengine syntax can be entered for execution +There is no difference in the use of TDengine data source compared to other data sources. Here is a brief introduction to basic data queries: -## Example +1. Click the [+] button in the upper right corner of the Superset interface, select [SQL query], and enter the query interface. +2. Select the `TDengine` data source that has been created earlier from the dropdown list of [DATABASES] in the upper left corner. +3. Select the name of the database to be operated on from the drop-down list of [SCHEMA] (system libraries are not displayed). +4. [SEE TABLE SCHEMA] select the name of the super table or regular table to be operated on (sub tables are not displayed). +5. Subsequently, the schema information of the selected table will be displayed in the following area. +6. In the `SQL` editor area, any `SQL` statement that conforms to `TDengine` syntax can be entered for execution. -We chose two popular templates from the Superset Chart template to showcase their effects, using smart meter data as an example: +### Smart Meter Example -1. "Aggregate" Type, which displays the maximum voltage value collected per minute during the specified time period in Group 4 +We chose two popular templates from the [Superset Chart] template to showcase their effects, using smart meter data as an example: - ![superset-demo1](./superset-demo1.jpeg) - -2. "RAW RECORDS" Type, which displays the collected values of current and voltage during the specified time period in Group 4 - - ![superset-demo2](./superset-demo2.jpeg) \ No newline at end of file +1. `Aggregate` Type, which displays the maximum voltage value collected per minute during the specified time period in Group 4. +![superset-demo1](./superset-demo1.jpeg) +2. `RAW RECORDS` Type, which displays the collected values of current and voltage during the specified time period in Group 4. +![superset-demo2](./superset-demo2.jpeg) \ No newline at end of file diff --git a/docs/zh/10-third-party/05-bi/05-yhbi.md b/docs/zh/10-third-party/05-bi/05-yhbi.md index b0a25eac83..caba25e891 100644 --- a/docs/zh/10-third-party/05-bi/05-yhbi.md +++ b/docs/zh/10-third-party/05-bi/05-yhbi.md @@ -16,7 +16,7 @@ toc_max_heading_level: 4 - TDengine 3.3.2.0 以上版本集群已部署并正常运行(企业及社区版均可)。 - taosAdapter 能够正常运行,详细参考 [taosAdapter 参考手册](../../../reference/components/taosadapter)。 - 确保永洪 BI 已经安装并运行(如果未安装,请到永洪科技官方下载页面下载)。 -- 安装JDBC驱动。从 maven.org 下载 TDengine JDBC 连接器文件 `taos-jdbcdriver-3.4.0-dist.jar` 及以上版本。 +- 安装 JDBC 驱动。从 maven.org 下载 TDengine JDBC 连接器文件 `taos-jdbcdriver-3.4.0-dist.jar` 及以上版本。 ## 配置数据源 diff --git a/docs/zh/10-third-party/05-bi/09-seeq.md b/docs/zh/10-third-party/05-bi/09-seeq.md index 24d3457f87..677dd39635 100644 --- a/docs/zh/10-third-party/05-bi/09-seeq.md +++ b/docs/zh/10-third-party/05-bi/09-seeq.md @@ -18,8 +18,6 @@ Seeq 是制造业和工业互联网(IIOT)高级分析软件。Seeq 支持在 ## 配置数据源 -### 配置 JDBC 连接器 - **第 1 步**,查看 data 存储位置 ``` @@ -38,9 +36,13 @@ sudo seeq restart 使用浏览器访问 ip:34216 并按照说明输入 license。 -### 加载 TDengine 时序数据 +## 数据分析 -本章节演示如何使用 Seeq 软件加载 TDengine 时序数据。 +### 场景介绍 + +示例场景为一个电力系统,用户每天从电站仪表收集用电量数据,并将其存储在 TDengine 集群中。现在用户想要预测电力消耗将会如何发展,并购买更多设备来支持它。用户电力消耗随着每月订单变化而不同,另外考虑到季节变化,电力消耗量会有所不同。这个城市位于北半球,所以在夏天会使用更多的电力。我们模拟数据来反映这些假定。 + +### 数据准备 **第 1 步**,在 TDengine 中创建表。 @@ -240,12 +242,6 @@ taos -s "insert into power.goods select _wstart, _wstart + 10d, avg(goods) from } ``` -## 数据分析 - -### 场景介绍 - -示例场景为一个电力系统,用户每天从电站仪表收集用电量数据,并将其存储在 TDengine 集群中。现在用户想要预测电力消耗将会如何发展,并购买更多设备来支持它。用户电力消耗随着每月订单变化而不同,另外考虑到季节变化,电力消耗量会有所不同。这个城市位于北半球,所以在夏天会使用更多的电力。我们模拟数据来反映这些假定。 - ### 使用 Seeq Workbench 登录 Seeq 服务页面并新建 Seeq Workbench,通过选择数据源搜索结果和根据需要选择不同的工具,可以进行数据展现或预测,详细使用方法参见 [官方知识库](https://support.seeq.com/space/KB/146440193/Seeq+Workbench)。 diff --git a/docs/zh/10-third-party/05-bi/11-superset.md b/docs/zh/10-third-party/05-bi/11-superset.md index 337fc53825..a2f3847d29 100644 --- a/docs/zh/10-third-party/05-bi/11-superset.md +++ b/docs/zh/10-third-party/05-bi/11-superset.md @@ -4,38 +4,39 @@ title: 与 Superset 集成 --- ‌Apache Superset‌ 是一个现代的企业级商业智能(BI)Web 应用程序,主要用于数据探索和可视化。它由 Apache 软件基金会支持,是一个开源项目,它拥有活跃的社区和丰富的生态系统。Apache Superset 提供了直观的用户界面,使得创建、分享和可视化数据变得简单,同时支持多种数据源和丰富的可视化选项‌。 -通过 TDengine 的 Python 连接器, ‌Apache ‌Superset‌ 可支持 TDengine 数据源并提供数据展现、分析等功能 +通过 TDengine 的 Python 连接器, ‌Apache ‌Superset‌ 可支持 TDengine 数据源并提供数据展现、分析等功能。 ## 前置条件 准备以下环境: -- TDengine 集群已部署并正常运行(企业及社区版均可) -- taosAdapter 能够正常运行。详细参考 [taosAdapter 使用手册](../../../reference/components/taosadapter) -- Apache Superset v2.1.0 或以上版本已安装。安装 Apache Superset 请参考 [官方文档](https://superset.apache.org/) +- TDengine 3.2.3.0 及以上版本集群已部署并正常运行(企业及社区版均可)。 +- taosAdapter 能够正常运行,详细参考 [taosAdapter 使用手册](../../../reference/components/taosadapter)。 +- Apache Superset v2.1.0 或以上版本已安装,安装 Apache Superset 请参考 [官方文档](https://superset.apache.org/)。 +- 安装 Python 连接器驱动,详细参考 [TDengine Python Connector](../../../reference/connector/python)。 - -## 安装 TDengine Python 连接器 - -TDengine Python 连接器从 `v2.1.18` 起带 Superset 连接驱动,会安装至 Superset 相应目录下并向 Superset 提供数据源服务 -Superset 与 TDengine 之间使用 WebSocket 协议连接,需安装支持此协议的 `taos-ws-py` 组件, 全部安装脚本如下: -```bash -pip3 install taospy -pip3 install taos-ws-py -``` - -## 配置 TDengine 数据源 - -**第 1 步**,进入新建数据库连接页面 "Superset" → "Setting" → "Database Connections" → "+DATABASE" -**第 2 步**,选择 TDengine 数据库连接。"SUPPORTED DATABASES" 下拉列表中选择 "TDengine" 项。 :::tip -注意:若下拉列表中无 "TDengine" 项,请检查安装顺序,确保 `TDengine Python 连接器` 在 `Superset` 安装之后再安装。 +TDengine Python 连接器从 `v2.1.18` 起带 Superset 连接驱动,会安装至 Superset 相应目录下并向 Superset 提供数据源服务。 +::: + +## 配置数据源 + +**第 1 步**,进入新建数据库连接页面【Superset】 -> 【Setting】->【Database Connections ->【+DATABASE】。 + +**第 2 步**,选择 TDengine 数据库连接。【SUPPORTED DATABASES】下拉列表中选择 `TDengine` 项。 + +:::tip +注意:若下拉列表中无 `TDengine` 项,请检查安装顺序,确保 `TDengine Python 连接器` 在 `Superset` 安装之后再安装。 ::: -**第 3 步**,"DISPLAY NAME" 中填写连接名称,任意填写即可。 -**第 4 步**,"SQLALCHEMY URL" 项为关键连接信息串,务必填写正确。 + +**第 3 步**,【DISPLAY NAME】中填写连接名称,任意填写即可。 + +**第 4 步**,【SQLALCHEMY URL】项为关键连接信息串,务必填写正确。 + ```bash taosws://用户名:密码@主机名:端口号 ``` + | 参数名称 |
    参数说明
    | |:------- |:-------------------------------- | | 用户名 | 登录 TDengine 数据库用户名 | @@ -43,32 +44,33 @@ taosws://用户名:密码@主机名:端口号 | 主机名 | TDengine 数据库所在主机名称 | | 端口号 | 提供 WebSocket 服务的端口,默认:6041 | -示例: -本机安装 TDengine 数据库,WebSocket 服务端口 6041,使用默认用户名密码,"SQLALCHEMY URL" 应为: +示例: + +本机安装 TDengine 数据库,WebSocket 服务端口 6041,使用默认用户名密码,`SQLALCHEMY URL` 应为: + ```bash taosws://root:taosdata@localhost:6041 ``` -**第 5 步**,配置好连接串,点击 “TEST CONNECTION” 测试连接是否成功,测试通过后点击 “CONNECT” 按钮,完成连接。 +**第 5 步**,配置好连接串,点击【TEST CONNECTION】测试连接是否成功,测试通过后点击【CONNECT】按钮,完成连接。 +## 数据分析 -## 开始使用 +### 数据准备 -TDengine 数据源与其它数据源使用上无差别,这里简单介绍下数据查询: -1. Superset 界面点击右上角 “+” 号按钮,选择 “SQL query”, 进入查询界面 -2. 左上角 “DATABASE” 下拉列表中选择前面已创建好的 “TDengine” 数据源 -3. “SCHEMA” 下拉列表,选择要操作的数据库名(系统库不显示) -4. “SEE TABLE SCHEMA” 选择要操作的超级表名或普通表名(子表不显示) -5. 随后会在下方显示选定表的 SCHEMA 信息 -6. 在 SQL 编辑器区域可输入符合 TDengine 语法的任意 SQL 语句执行 +TDengine 数据源与其它数据源使用上无差别,这里简单介绍下数据查询: -## 示例效果 +1. `Superset` 界面点击右上角【+】号按钮,选择 `SQL query`, 进入查询界面。 +2. 左上角【DATABASE】下拉列表中选择前面已创建好的 `TDengine` 数据源。 +3. 【SCHEMA】下拉列表,选择要操作的数据库名(系统库不显示)。 +4. 【SEE TABLE SCHEMA】选择要操作的超级表名或普通表名(子表不显示)。 +5. 随后会在下方显示选定表的 `SCHEMA` 信息。 +6. 在 `SQL` 编辑器区域可输入符合 `TDengine` 语法的任意 `SQL` 语句执行。 -我们选择 Superset Chart 模板中较流行的两个模板做了效果展示,以智能电表数据为例: +### 智能电表样例 -1. "Aggregate" 类型,展示在第 4 组中指定时间段内每分钟采集电压值(voltage)最大值 +我们选择【Superset Chart】模板中较流行的两个模板做了效果展示,以智能电表数据为例: - ![superset-demo1](./superset-demo1.jpeg) - -2. "RAW RECORDS" 类型,展示在第 4 组中指定时间段内 current, voltage 的采集值 - - ![superset-demo2](./superset-demo2.jpeg) +1. `Aggregate` 类型,展示在第 4 组中指定时间段内每分钟采集电压值(voltage)最大值。 +![superset-demo1](./superset-demo1.jpeg) +2. `RAW RECORDS` 类型,展示在第 4 组中指定时间段内 current, voltage 的采集值。 +![superset-demo2](./superset-demo2.jpeg) diff --git a/docs/zh/10-third-party/05-bi/12-tableau.md b/docs/zh/10-third-party/05-bi/12-tableau.md index 0478b3f7d5..9fbdc53d45 100644 --- a/docs/zh/10-third-party/05-bi/12-tableau.md +++ b/docs/zh/10-third-party/05-bi/12-tableau.md @@ -11,7 +11,7 @@ Tableau 是一款知名的商业智能工具,它支持多种数据源,可方 - TDengine 3.3.5.4 以上版本集群已部署并正常运行(企业及社区版均可) - taosAdapter 能够正常运行。详细参考 [taosAdapter 参考手册](../../../reference/components/taosadapter) - Tableau 桌面版安装并运行(如未安装,请下载并安装 Windows 操作系统 64 位 [Tableau 桌面版](https://www.tableau.com/products/desktop/download) )。安装 Tableau 桌面版请参考 [官方文档](https://www.tableau.com)。 -- 从TDengine官网下载最新的Windows操作系统X64客户端驱动程序,并进行安装。详细参考 [安装 ODBC 驱动](../../../reference/connector/odbc/#安装)。 +- 从 TDengine 官网下载最新的 Windows 操作系统 X64 客户端驱动程序,并进行安装。详细参考 [安装 ODBC 驱动](../../../reference/connector/odbc/#安装)。 ## 配置数据源 From 0e9319ef691cd7363dc90c79af2b03a6bb7f4c3f Mon Sep 17 00:00:00 2001 From: haoranchen Date: Tue, 25 Feb 2025 15:40:08 +0800 Subject: [PATCH 23/40] Revert "refactor: implement dynamic loading of test case classes" --- tests/army/test.py | 30 +++++++++--------------------- tests/develop-test/test.py | 26 ++++++-------------------- tests/pytest/util/cases.py | 25 ++++--------------------- tests/system-test/test.py | 30 +++++++++--------------------- tests/test_new/test.py | 30 +++++++++--------------------- 5 files changed, 37 insertions(+), 104 deletions(-) diff --git a/tests/army/test.py b/tests/army/test.py index bf14f19e3f..d37d08b406 100644 --- a/tests/army/test.py +++ b/tests/army/test.py @@ -24,7 +24,6 @@ import platform import socket import threading import importlib -import inspect import toml @@ -57,17 +56,6 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) - return classes - -def dynamicLoadModule(fileName): - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - return importlib.import_module(moduleName, package='..') - # # run case on previous cluster # @@ -78,9 +66,9 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - uModule = dynamicLoadModule(fileName) - case_class = getattr(uModule, get_local_classes(uModule)[0]) - case = case_class() + moduleName = fileName.replace(".py", "").replace(sep, ".") + uModule = importlib.import_module(moduleName) + case = uModule.TDTestCase() # create conn conn = taos.connect(host, config) @@ -370,10 +358,10 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -542,10 +530,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace("/", ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): diff --git a/tests/develop-test/test.py b/tests/develop-test/test.py index 9106e38d9a..6b1c63a1c0 100644 --- a/tests/develop-test/test.py +++ b/tests/develop-test/test.py @@ -22,9 +22,6 @@ import json import platform import socket import threading -import inspect -import importlib -import os import toml @@ -60,17 +57,6 @@ def checkRunTimeError(): os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) - return classes - -def dynamicLoadModule(fileName): - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - return importlib.import_module(moduleName, package='..') - if __name__ == "__main__": fileName = "all" @@ -309,10 +295,10 @@ if __name__ == "__main__": updateCfgDictStr = "" # adapter_cfg_dict_str = '' if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if (json.dumps(updateCfgDict) == "{}") and hasattr( ucase, "updatecfgDict" ): @@ -448,10 +434,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace("/", ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if json.dumps(updateCfgDict) == "{}": updateCfgDict = ucase.updatecfgDict if json.dumps(adapter_cfg_dict) == "{}": diff --git a/tests/pytest/util/cases.py b/tests/pytest/util/cases.py index 4899bedc69..eee8809ad0 100644 --- a/tests/pytest/util/cases.py +++ b/tests/pytest/util/cases.py @@ -51,13 +51,6 @@ class TDCases: def addCluster(self, name, case): self.clusterCases.append(TDCase(name, case)) - def get_local_classes(self, module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) - return classes - def runAllLinux(self, conn): # TODO: load all Linux cases here runNum = 0 @@ -78,10 +71,7 @@ class TDCases: runNum = 0 for tmp in self.linuxCases: if tmp.name.find(fileName) != -1: - # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[0]) - - case = case_class() + case = testModule.TDTestCase() case.init(conn, self._logSql, replicaVar) try: case.run() @@ -113,9 +103,7 @@ class TDCases: runNum = 0 for tmp in self.windowsCases: if tmp.name.find(fileName) != -1: - # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[0]) - case = case_class() + case = testModule.TDTestCase() case.init(conn, self._logSql,replicaVar) try: case.run() @@ -129,15 +117,12 @@ class TDCases: def runAllCluster(self): # TODO: load all cluster case module here - testModule = self.__dynamicLoadModule(fileName) runNum = 0 for tmp in self.clusterCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) - # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[0]) - case = case_class() + case = testModule.TDTestCase() case.init() case.run() case.stop() @@ -153,9 +138,7 @@ class TDCases: for tmp in self.clusterCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) - # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[0]) - case = case_class() + case = testModule.TDTestCase() case.init() case.run() case.stop() diff --git a/tests/system-test/test.py b/tests/system-test/test.py index a05eed5a3c..ab1bdc21d3 100644 --- a/tests/system-test/test.py +++ b/tests/system-test/test.py @@ -24,7 +24,6 @@ import platform import socket import threading import importlib -import inspect print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -59,16 +58,6 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) - return classes - -def dynamicLoadModule(fileName): - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - return importlib.import_module(moduleName, package='..') # # run case on previous cluster # @@ -79,10 +68,9 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - uModule = dynamicLoadModule(fileName) - - case_class = getattr(uModule, get_local_classes(uModule)[0]) - case = case_class() + moduleName = fileName.replace(".py", "").replace(sep, ".") + uModule = importlib.import_module(moduleName) + case = uModule.TDTestCase() # create conn conn = taos.connect(host, config) @@ -362,10 +350,10 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -534,10 +522,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace("/", ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): diff --git a/tests/test_new/test.py b/tests/test_new/test.py index a05eed5a3c..ab1bdc21d3 100644 --- a/tests/test_new/test.py +++ b/tests/test_new/test.py @@ -24,7 +24,6 @@ import platform import socket import threading import importlib -import inspect print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -59,16 +58,6 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) - return classes - -def dynamicLoadModule(fileName): - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - return importlib.import_module(moduleName, package='..') # # run case on previous cluster # @@ -79,10 +68,9 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - uModule = dynamicLoadModule(fileName) - - case_class = getattr(uModule, get_local_classes(uModule)[0]) - case = case_class() + moduleName = fileName.replace(".py", "").replace(sep, ".") + uModule = importlib.import_module(moduleName) + case = uModule.TDTestCase() # create conn conn = taos.connect(host, config) @@ -362,10 +350,10 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -534,10 +522,10 @@ if __name__ == "__main__": except: pass if is_test_framework: - uModule = dynamicLoadModule(fileName) + moduleName = fileName.replace(".py", "").replace("/", ".") + uModule = importlib.import_module(moduleName) try: - case_class = getattr(uModule, get_local_classes(uModule)[0]) - ucase = case_class() + ucase = uModule.TDTestCase() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): From a987d925a17972fdd6c4845a8be109bc8bcdabe9 Mon Sep 17 00:00:00 2001 From: Jinqing Kuang Date: Tue, 25 Feb 2025 11:14:17 +0800 Subject: [PATCH 24/40] fix(query)[TD-33833]: fix function projectApplyFunctions for null pointer handling --- source/libs/executor/src/projectoperator.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/src/projectoperator.c b/source/libs/executor/src/projectoperator.c index cb91bae691..eb448a13f5 100644 --- a/source/libs/executor/src/projectoperator.c +++ b/source/libs/executor/src/projectoperator.c @@ -876,7 +876,7 @@ int32_t projectApplyFunctions(SExprInfo* pExpr, SSDataBlock* pResult, SSDataBloc } pResult->info.rows = 1; - TSDB_CHECK_CODE(code, lino, _exit); + goto _exit; } if (pResult != pSrcBlock) { From 7bbc42aa30d9eb9ac80d0871eec611ff0e1e3948 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Tue, 25 Feb 2025 16:13:57 +0800 Subject: [PATCH 25/40] refactor: update Jenkinsfile agents and comment out unused stages --- tests/army/test.py | 25 +++++++++++++++++++------ tests/develop-test/test.py | 6 ++++-- tests/pytest/test.py | 11 +++++++++++ tests/pytest/util/cases.py | 25 +++++++++++++++++++++---- tests/system-test/test.py | 13 ++++++++----- tests/test_new/test.py | 13 ++++++++----- 6 files changed, 71 insertions(+), 22 deletions(-) diff --git a/tests/army/test.py b/tests/army/test.py index d37d08b406..5827657106 100644 --- a/tests/army/test.py +++ b/tests/army/test.py @@ -24,7 +24,7 @@ import platform import socket import threading import importlib - +import inspect import toml from frame.log import * @@ -56,6 +56,17 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') + # # run case on previous cluster # @@ -66,9 +77,9 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - moduleName = fileName.replace(".py", "").replace(sep, ".") - uModule = importlib.import_module(moduleName) - case = uModule.TDTestCase() + uModule = dynamicLoadModule(fileName) + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + case = case_class() # create conn conn = taos.connect(host, config) @@ -361,7 +372,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace(os.sep, ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -533,7 +545,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace("/", ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): diff --git a/tests/develop-test/test.py b/tests/develop-test/test.py index 6b1c63a1c0..c0a090aaa5 100644 --- a/tests/develop-test/test.py +++ b/tests/develop-test/test.py @@ -298,7 +298,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace(os.sep, ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if (json.dumps(updateCfgDict) == "{}") and hasattr( ucase, "updatecfgDict" ): @@ -437,7 +438,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace("/", ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if json.dumps(updateCfgDict) == "{}": updateCfgDict = ucase.updatecfgDict if json.dumps(adapter_cfg_dict) == "{}": diff --git a/tests/pytest/test.py b/tests/pytest/test.py index 1b185ef189..eac9d9ea77 100644 --- a/tests/pytest/test.py +++ b/tests/pytest/test.py @@ -26,6 +26,17 @@ from util.cases import * import taos +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') + if __name__ == "__main__": fileName = "all" diff --git a/tests/pytest/util/cases.py b/tests/pytest/util/cases.py index eee8809ad0..9a76e14790 100644 --- a/tests/pytest/util/cases.py +++ b/tests/pytest/util/cases.py @@ -51,6 +51,13 @@ class TDCases: def addCluster(self, name, case): self.clusterCases.append(TDCase(name, case)) + def get_local_classes(self, module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + def runAllLinux(self, conn): # TODO: load all Linux cases here runNum = 0 @@ -71,7 +78,10 @@ class TDCases: runNum = 0 for tmp in self.linuxCases: if tmp.name.find(fileName) != -1: - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + print(case_class) + case = case_class() case.init(conn, self._logSql, replicaVar) try: case.run() @@ -103,7 +113,9 @@ class TDCases: runNum = 0 for tmp in self.windowsCases: if tmp.name.find(fileName) != -1: - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + case = case_class() case.init(conn, self._logSql,replicaVar) try: case.run() @@ -117,12 +129,15 @@ class TDCases: def runAllCluster(self): # TODO: load all cluster case module here + testModule = self.__dynamicLoadModule(fileName) runNum = 0 for tmp in self.clusterCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + case = case_class() case.init() case.run() case.stop() @@ -138,7 +153,9 @@ class TDCases: for tmp in self.clusterCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + case = case_class() case.init() case.run() case.stop() diff --git a/tests/system-test/test.py b/tests/system-test/test.py index ab1bdc21d3..87febd1bf2 100644 --- a/tests/system-test/test.py +++ b/tests/system-test/test.py @@ -68,9 +68,10 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - moduleName = fileName.replace(".py", "").replace(sep, ".") - uModule = importlib.import_module(moduleName) - case = uModule.TDTestCase() + uModule = dynamicLoadModule(fileName) + + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + case = case_class() # create conn conn = taos.connect(host, config) @@ -353,7 +354,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace(os.sep, ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -525,7 +527,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace("/", ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): diff --git a/tests/test_new/test.py b/tests/test_new/test.py index ab1bdc21d3..87febd1bf2 100644 --- a/tests/test_new/test.py +++ b/tests/test_new/test.py @@ -68,9 +68,10 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep - moduleName = fileName.replace(".py", "").replace(sep, ".") - uModule = importlib.import_module(moduleName) - case = uModule.TDTestCase() + uModule = dynamicLoadModule(fileName) + + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + case = case_class() # create conn conn = taos.connect(host, config) @@ -353,7 +354,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace(os.sep, ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict updateCfgDictStr = "-d %s"%base64.b64encode(json.dumps(updateCfgDict).encode()).decode() @@ -525,7 +527,8 @@ if __name__ == "__main__": moduleName = fileName.replace(".py", "").replace("/", ".") uModule = importlib.import_module(moduleName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict if (json.dumps(adapter_cfg_dict) == '{}'): From 67c06cc02bdb8472bb68aac47d72cd73b31339d8 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Tue, 25 Feb 2025 21:14:07 +0800 Subject: [PATCH 26/40] refactor: simplify module loading by introducing dynamicLoadModule function --- tests/army/test.py | 6 ++---- tests/develop-test/test.py | 20 ++++++++++++++++---- tests/pytest/test.py | 6 ++---- tests/pytest/util/cases.py | 8 ++++++-- tests/system-test/test.py | 19 ++++++++++++++----- tests/test_new/test.py | 19 ++++++++++++++----- 6 files changed, 54 insertions(+), 24 deletions(-) diff --git a/tests/army/test.py b/tests/army/test.py index 5827657106..56b868c7ae 100644 --- a/tests/army/test.py +++ b/tests/army/test.py @@ -369,8 +369,7 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() @@ -542,8 +541,7 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() diff --git a/tests/develop-test/test.py b/tests/develop-test/test.py index c0a090aaa5..76d5732e80 100644 --- a/tests/develop-test/test.py +++ b/tests/develop-test/test.py @@ -22,6 +22,9 @@ import json import platform import socket import threading +import inspect +import importlib +import os import toml @@ -56,6 +59,17 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') + if __name__ == "__main__": @@ -295,8 +309,7 @@ if __name__ == "__main__": updateCfgDictStr = "" # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() @@ -435,8 +448,7 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() diff --git a/tests/pytest/test.py b/tests/pytest/test.py index eac9d9ea77..9d0e8651b4 100644 --- a/tests/pytest/test.py +++ b/tests/pytest/test.py @@ -147,8 +147,7 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: ucase = uModule.TDTestCase() tdDnodes.deploy(1,ucase.updatecfgDict) @@ -181,8 +180,7 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: ucase = uModule.TDTestCase() tdDnodes.deploy(1,ucase.updatecfgDict) diff --git a/tests/pytest/util/cases.py b/tests/pytest/util/cases.py index 9a76e14790..48f73c50f1 100644 --- a/tests/pytest/util/cases.py +++ b/tests/pytest/util/cases.py @@ -63,7 +63,9 @@ class TDCases: runNum = 0 for tmp in self.linuxCases: if tmp.name.find(fileName) != -1: - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[0]) + case = case_class() case.init(conn) case.run() case.stop() @@ -98,7 +100,9 @@ class TDCases: runNum = 0 for tmp in self.windowsCases: if tmp.name.find(fileName) != -1: - case = testModule.TDTestCase() + # get the last class name as the test case class name + case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + case = case_class() case.init(conn) case.run() case.stop() diff --git a/tests/system-test/test.py b/tests/system-test/test.py index 87febd1bf2..bf66b6a765 100644 --- a/tests/system-test/test.py +++ b/tests/system-test/test.py @@ -24,6 +24,7 @@ import platform import socket import threading import importlib +import inspect print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -58,6 +59,17 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') + # # run case on previous cluster # @@ -69,7 +81,6 @@ def runOnPreviousCluster(host, config, fileName): if platform.system().lower() == 'windows': sep = os.sep uModule = dynamicLoadModule(fileName) - case_class = getattr(uModule, get_local_classes(uModule)[-1]) case = case_class() @@ -351,8 +362,7 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() @@ -524,8 +534,7 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() diff --git a/tests/test_new/test.py b/tests/test_new/test.py index 87febd1bf2..bf66b6a765 100644 --- a/tests/test_new/test.py +++ b/tests/test_new/test.py @@ -24,6 +24,7 @@ import platform import socket import threading import importlib +import inspect print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -58,6 +59,17 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") +def get_local_classes(module): + classes = [] + for name, obj in inspect.getmembers(module, inspect.isclass): + if inspect.getmodule(obj) == module: + classes.append(name) + return classes + +def dynamicLoadModule(fileName): + moduleName = fileName.replace(".py", "").replace(os.sep, ".") + return importlib.import_module(moduleName, package='..') + # # run case on previous cluster # @@ -69,7 +81,6 @@ def runOnPreviousCluster(host, config, fileName): if platform.system().lower() == 'windows': sep = os.sep uModule = dynamicLoadModule(fileName) - case_class = getattr(uModule, get_local_classes(uModule)[-1]) case = case_class() @@ -351,8 +362,7 @@ if __name__ == "__main__": updateCfgDictStr = '' # adapter_cfg_dict_str = '' if is_test_framework: - moduleName = fileName.replace(".py", "").replace(os.sep, ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() @@ -524,8 +534,7 @@ if __name__ == "__main__": except: pass if is_test_framework: - moduleName = fileName.replace(".py", "").replace("/", ".") - uModule = importlib.import_module(moduleName) + uModule = dynamicLoadModule(fileName) try: case_class = getattr(uModule, get_local_classes(uModule)[-1]) ucase = case_class() From 6ff179aca21e6c3c5a39c040dd2fd3e224f98a9e Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Tue, 25 Feb 2025 21:23:34 +0800 Subject: [PATCH 27/40] refactor: simplify module loading by introducing dynamicLoadModule function --- tests/pytest/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/pytest/test.py b/tests/pytest/test.py index 9d0e8651b4..5d1a15961a 100644 --- a/tests/pytest/test.py +++ b/tests/pytest/test.py @@ -149,7 +149,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() tdDnodes.deploy(1,ucase.updatecfgDict) except : tdDnodes.deploy(1,{}) @@ -182,7 +183,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - ucase = uModule.TDTestCase() + case_class = getattr(uModule, get_local_classes(uModule)[-1]) + ucase = case_class() tdDnodes.deploy(1,ucase.updatecfgDict) except : tdDnodes.deploy(1,{}) From 192c83355fad082d9b356c7ab770ebc179155780 Mon Sep 17 00:00:00 2001 From: haoranchen Date: Wed, 26 Feb 2025 10:17:01 +0800 Subject: [PATCH 28/40] Update taosd-ci-build.yml --- .github/workflows/taosd-ci-build.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/taosd-ci-build.yml b/.github/workflows/taosd-ci-build.yml index 4972aebebe..0f65684027 100644 --- a/.github/workflows/taosd-ci-build.yml +++ b/.github/workflows/taosd-ci-build.yml @@ -6,6 +6,8 @@ on: - 'main' - '3.0' - '3.1' + - 'enh/cmake-TD-33848' + paths-ignore: - 'docs/**' - 'packaging/**' From 41fd22c8a3b41aca3bcb84342f46963896efea41 Mon Sep 17 00:00:00 2001 From: WhiteFirry <89558636+WhiteFirry@users.noreply.github.com> Date: Wed, 26 Feb 2025 11:20:41 +0800 Subject: [PATCH 29/40] Update 26-grant.md --- docs/zh/14-reference/03-taos-sql/26-grant.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/zh/14-reference/03-taos-sql/26-grant.md b/docs/zh/14-reference/03-taos-sql/26-grant.md index c3fd1790d0..779655e265 100644 --- a/docs/zh/14-reference/03-taos-sql/26-grant.md +++ b/docs/zh/14-reference/03-taos-sql/26-grant.md @@ -4,6 +4,7 @@ title: 权限管理 --- TDengine 中的权限管理分为[用户管理](../user)、数据库授权管理以及消息订阅授权管理,本节重点说明数据库授权和订阅授权。 +授权管理仅在 TDengine 企业版中可用,请联系 TDengine 销售团队。授权语法在社区版可用,但不起作用。 ## 数据库访问授权 From e814d300b1a80fdc92e7a62562a272d1207dae7e Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Wed, 26 Feb 2025 12:42:26 +0800 Subject: [PATCH 30/40] add taosd-doc-build workflow by charles --- .github/workflows/taosd-doc-build.yml | 102 ++++++++++++++++++++++++++ 1 file changed, 102 insertions(+) create mode 100644 .github/workflows/taosd-doc-build.yml diff --git a/.github/workflows/taosd-doc-build.yml b/.github/workflows/taosd-doc-build.yml new file mode 100644 index 0000000000..43767e9282 --- /dev/null +++ b/.github/workflows/taosd-doc-build.yml @@ -0,0 +1,102 @@ + +name: Reusable workflow of TDengine document buile pipelines + +on: + workflow_call: + inputs: + target_branch: + description: "Target branch name of for building the document" + required: true + type: string + + target_pr_number: + description: "PR number of target branch to merge for building the document" + required: true + type: string + +env: + DOC_WKC: "/root/doc_ci_work" + ZH_DOC_REPO: "docs.taosdata.com" + EN_DOC_REPO: "docs.tdengine.com" + TD_REPO: "TDengine" + TOOLS_REPO: "taos-tools" + +jobs: + check: + runs-on: + group: CI + labels: [self-hosted, doc-build] + outputs: + changed_files_zh: ${{ steps.set_output.outputs.changed_files_zh }} + changed_files_en: ${{ steps.set_output.outputs.changed_files_en }} + changed_files_non_doc: ${{ steps.set_output.outputs.changed_files_non_doc }} + changed_files_non_tdgpt: ${{ steps.set_output.outputs.changed_files_non_tdgpt }} + steps: + - name: Get the latest document contents from the repository + run: | + set -e + # ./.github/scripts/update_repo.sh ${{ env.DOC_WKC }}/${{ env.TD_REPO }} ${{ inputs.target_branch }} ${{ inputs.target_pr_number }} + cd ${{ env.DOC_WKC }}/${{ env.TD_REPO }} + git reset --hard + git clean -f + git remote prune origin + git fetch + git checkout ${{ inputs.target_branch }} + git pull >/dev/null + git fetch origin +refs/pull/${{ inputs.target_pr_number }}/merge + git checkout -qf FETCH_HEAD + - name: Check whether the document is changed and set output variables + id: set_output + run: | + set -e + cd ${{ env.DOC_WKC }}/${{ env.TD_REPO }} + changed_files_zh=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`| grep "^docs/zh/" | tr '\n' ' ' || :) + changed_files_en=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`| grep "^docs/en/" | tr '\n' ' ' || :) + changed_files_non_doc=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`|grep -v "^docs/en/"|grep -v "^docs/zh/"|grep -v ".md$" | tr '\n' ' ' || :) + changed_files_non_tdgpt=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`|grep -v "^docs/en/"|grep -v "^docs/zh/"|grep -v ".md$" | grep -Ev "forecastoperator.c|anomalywindowoperator.c|tanalytics.h|tanalytics.c|tdgpt_cases.task|analytics" | tr '\n' ' ' ||:) + echo "changed_files_zh=${changed_files_zh}" >> $GITHUB_OUTPUT + echo "changed_files_en=${changed_files_en}" >> $GITHUB_OUTPUT + echo "changed_files_non_doc=${changed_files_non_doc}" >> $GITHUB_OUTPUT + echo "changed_files_non_tdgpt=${changed_files_non_tdgpt}" >> $GITHUB_OUTPUT + + build: + needs: check + runs-on: + group: CI + labels: [self-hosted, doc-build] + if: ${{ needs.check.outputs.changed_files_zh == '' || needs.check.outputs.changed_files_en == '' }} + + steps: + - name: Get the latest document contents + run: | + set -e + #./.github/scripts/update_repo.sh ${{ env.DOC_WKC }}/${{ env.TD_REPO }} ${{ inputs.target_branch }} ${{ inputs.target_pr_number }} + cd ${{ env.DOC_WKC }}/${{ env.TD_REPO }} + git reset --hard + git clean -f + git remote prune origin + git fetch + git checkout ${{ inputs.target_branch }} + git pull >/dev/null + git fetch origin +refs/pull/${{ inputs.target_pr_number }}/merge + git checkout -qf FETCH_HEAD + + - name: Build the chinese document + if: ${{ needs.check.outputs.changed_files_zh == '' }} + run: | + cd ${{ env.DOC_WKC }}/${{ env.ZH_DOC_REPO }} + yarn ass local + yarn build + + - name: Build the english document + if: ${{ needs.check.outputs.changed_files_en == '' }} + run: | + cd ${{ env.DOC_WKC }}/${{ env.EN_DOC_REPO }} + yarn ass local + yarn build + + outputs: + changed_files_zh: ${{ needs.check.outputs.changed_files_zh }} + changed_files_en: ${{ needs.check.outputs.changed_files_en }} + changed_files_non_doc: ${{ needs.check.outputs.changed_files_non_doc }} + changed_files_non_tdgpt: ${{ needs.check.outputs.changed_files_non_tdgpt }} From 9576108c13dc7e16ad4ce09a4ec2e4e19b5d2803 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Wed, 26 Feb 2025 14:10:56 +0800 Subject: [PATCH 31/40] Update taosd-doc-build.yml --- .github/workflows/taosd-doc-build.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/taosd-doc-build.yml b/.github/workflows/taosd-doc-build.yml index 43767e9282..03d8b05068 100644 --- a/.github/workflows/taosd-doc-build.yml +++ b/.github/workflows/taosd-doc-build.yml @@ -64,7 +64,7 @@ jobs: runs-on: group: CI labels: [self-hosted, doc-build] - if: ${{ needs.check.outputs.changed_files_zh == '' || needs.check.outputs.changed_files_en == '' }} + if: ${{ needs.check.outputs.changed_files_zh != '' || needs.check.outputs.changed_files_en != '' }} steps: - name: Get the latest document contents @@ -82,14 +82,14 @@ jobs: git checkout -qf FETCH_HEAD - name: Build the chinese document - if: ${{ needs.check.outputs.changed_files_zh == '' }} + if: ${{ needs.check.outputs.changed_files_zh != '' }} run: | cd ${{ env.DOC_WKC }}/${{ env.ZH_DOC_REPO }} yarn ass local yarn build - name: Build the english document - if: ${{ needs.check.outputs.changed_files_en == '' }} + if: ${{ needs.check.outputs.changed_files_en != '' }} run: | cd ${{ env.DOC_WKC }}/${{ env.EN_DOC_REPO }} yarn ass local From 4af1d27b8b2a043a3196f8b23a1621a0056eedbd Mon Sep 17 00:00:00 2001 From: WANG Xu Date: Wed, 26 Feb 2025 14:14:01 +0800 Subject: [PATCH 32/40] Update .github/workflows/taosd-doc-build.yml --- .github/workflows/taosd-doc-build.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/taosd-doc-build.yml b/.github/workflows/taosd-doc-build.yml index 03d8b05068..52a35fc3d5 100644 --- a/.github/workflows/taosd-doc-build.yml +++ b/.github/workflows/taosd-doc-build.yml @@ -1,5 +1,4 @@ - -name: Reusable workflow of TDengine document buile pipelines +name: TDengine Doc Build on: workflow_call: From 3da6f5d237e059560d59680c7b09b0d031b4bcd1 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Wed, 26 Feb 2025 14:15:52 +0800 Subject: [PATCH 33/40] fix: submit doc with taos-ws-py 0.3.9 publish --- docs/en/14-reference/05-connector/30-python.md | 5 ++++- docs/zh/14-reference/05-connector/30-python.mdx | 4 ++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/14-reference/05-connector/30-python.md b/docs/en/14-reference/05-connector/30-python.md index 8956e04d56..8e8551d128 100644 --- a/docs/en/14-reference/05-connector/30-python.md +++ b/docs/en/14-reference/05-connector/30-python.md @@ -71,7 +71,10 @@ WebSocket Connector Historical Versions: |WebSocket Connector Version | Major Changes | TDengine Version| | ----------------------- | -------------------------------------------------------------------------------------------------- | ----------------- | -|0.3.5 | Added support for VARBINARY and GEOMETRY types, fixed known issues. | 3.3.0.0 and higher| +|0.3.9 | Fixed the issue when fetching much data with 'fetchmany' API. | - | +|0.3.8 | Supported connecting SuperSet to the TDengine cloud service instance. | - | +|0.3.5 | Fixed the issues in the crypto provider. | - | +|0.3.4 | Supported varbinary and geometry data type. | 3.3.0.0 and higher | |0.3.2 | Optimize WebSocket SQL query and insertion performance, modify readme and documentation, fix known issues. | 3.2.3.0 and higher| |0.2.9 | Known issue fixes. | - | |0.2.5 | 1. Data subscription supports obtaining and resetting consumption progress.
    2 Support schemaless.
    3 Support STMT. | - | diff --git a/docs/zh/14-reference/05-connector/30-python.mdx b/docs/zh/14-reference/05-connector/30-python.mdx index 42ed67e927..1fd06e5c12 100644 --- a/docs/zh/14-reference/05-connector/30-python.mdx +++ b/docs/zh/14-reference/05-connector/30-python.mdx @@ -68,6 +68,10 @@ WebSocket Connector 历史版本: | WebSocket Connector 版本 | 主要变化 | TDengine 版本 | | ----------------------- | ------------------------------------------------------------------------------------ | ----------------- | +| 0.3.9 | 修复 fetchmany 获取大数据量的问题 | - | +| 0.3.8 | 支持 SuperSet 连接到 TDengine 云服务实例 | - | +| 0.3.5 | 修复 crypto provider 中的问题 | - | +| 0.3.4 | 支持 VARBINARY 和 GEOMETRY 数据类型 | 3.3.0.0 及更高版本 | | 0.3.2 | 优化 WebSocket sql 查询和插入性能,修改 readme 和 文档,修复已知问题 | 3.2.3.0 及更高版本 | | 0.2.9 | 已知问题修复 | - | | 0.2.5 | 1. 数据订阅支持获取消费进度和重置消费进度
    2. 支持 schemaless
    3. 支持 STMT | - | From 0c207e96db237be618444f4572c03ad4b5eae7f8 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Wed, 26 Feb 2025 15:58:14 +0800 Subject: [PATCH 34/40] docs: modify 0.3.9 version log describe --- docs/en/14-reference/05-connector/30-python.md | 2 +- docs/zh/14-reference/05-connector/30-python.mdx | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/14-reference/05-connector/30-python.md b/docs/en/14-reference/05-connector/30-python.md index 8e8551d128..9496cce0e5 100644 --- a/docs/en/14-reference/05-connector/30-python.md +++ b/docs/en/14-reference/05-connector/30-python.md @@ -71,7 +71,7 @@ WebSocket Connector Historical Versions: |WebSocket Connector Version | Major Changes | TDengine Version| | ----------------------- | -------------------------------------------------------------------------------------------------- | ----------------- | -|0.3.9 | Fixed the issue when fetching much data with 'fetchmany' API. | - | +|0.3.9 | Fix the problem of incomplete data retrieval when customizing the number of rows with the "fetchmany" method. | - | |0.3.8 | Supported connecting SuperSet to the TDengine cloud service instance. | - | |0.3.5 | Fixed the issues in the crypto provider. | - | |0.3.4 | Supported varbinary and geometry data type. | 3.3.0.0 and higher | diff --git a/docs/zh/14-reference/05-connector/30-python.mdx b/docs/zh/14-reference/05-connector/30-python.mdx index 1fd06e5c12..03f601cf9c 100644 --- a/docs/zh/14-reference/05-connector/30-python.mdx +++ b/docs/zh/14-reference/05-connector/30-python.mdx @@ -68,7 +68,7 @@ WebSocket Connector 历史版本: | WebSocket Connector 版本 | 主要变化 | TDengine 版本 | | ----------------------- | ------------------------------------------------------------------------------------ | ----------------- | -| 0.3.9 | 修复 fetchmany 获取大数据量的问题 | - | +| 0.3.9 | 修复 fetchmany 自定义行数时获取不完全的问题 | - | | 0.3.8 | 支持 SuperSet 连接到 TDengine 云服务实例 | - | | 0.3.5 | 修复 crypto provider 中的问题 | - | | 0.3.4 | 支持 VARBINARY 和 GEOMETRY 数据类型 | 3.3.0.0 及更高版本 | From e95ce441aeec199c9bdeea8089272d4746e4baaa Mon Sep 17 00:00:00 2001 From: danielclow <106956386+danielclow@users.noreply.github.com> Date: Wed, 26 Feb 2025 17:25:04 +0800 Subject: [PATCH 35/40] docs: add roadmap --- docs/en/29-roadmap/index.md | 14 ++++++++++++++ docs/zh/29-roadmap/index.md | 14 ++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 docs/en/29-roadmap/index.md create mode 100644 docs/zh/29-roadmap/index.md diff --git a/docs/en/29-roadmap/index.md b/docs/en/29-roadmap/index.md new file mode 100644 index 0000000000..e79565ff40 --- /dev/null +++ b/docs/en/29-roadmap/index.md @@ -0,0 +1,14 @@ +--- +title: Product Roadmap +--- + +The 2025 roadmap for TDengine OSS is described in the following table. + +| Quarter | Feature | +| :----- | :----- | +| 2025Q1 |
    1. Virtual tables
    2. Query engine: conditional expressions in REGEXP, GREATEST, LEAST, and CAST functions; improvements in single-row selection functions; time range interpolation with INTERP
    3. Storage engine: support for writing query results into supertables; KEEP parameter for supertables; performance improvements for the parameter binding interface
    4. Stream processing: support for virtual tables; decreased compute resource usage; new mechanism for event notification; faster stream creation
    5. Data types: support for the decimal data type
    6. High availability: faster recovery from downtime; improved client failover
    7. Stability: LTS release TDengine 3.3.6.x
    8. JDBC driver: more efficient data ingestion
    9. Ecosystem: integration with Microsoft Excel
    | +| 2025Q2 |
    1. Query engine: relaxed restrictions on JOIN queries; support for all mathematical functions in MySQL; integral, integral average, and continuous variance functions; optimization of the CSUM function; support for COUNT(DISTINCT) syntax; enhancements to event windows; faster filtering by tag; faster INTERP queries
    2. Storage engine: decreased compute resource usage for TSMAs; improved write jitter
    3. Stream processing: high availability of snodes
    4. Data types: support for the blob data type
    5. Data subscription: support for the MQTT protocol
    6. High availability: faster replica configuration changes; faster recovery from downtime for clusters; improved data recovery after power outage
    7. Observability: diagnostic tool for data ingestion
    | +| 2025Q3 |
    1. Query engine: more subqueries; support for all operators in MySQL; support for all time functions in MySQL; improved window calculation; reduced jitter in query performance; support for specifying columns in count windows
    2. Storage engine: faster ingestion in SQL mode
    3. Observability: diagnostic tool for queries; improved EXPLAIN output; monitoring of long-running tasks
    | +| 2025Q4 |
    1. Query engine: window functions (i.e. the OVER clause); support for all string, aggregation, and conditional functions in MySQL; sorting within groups for partition queries; controls for query resource usage; faster aggregate queries on subtables; time range interpolation in INTERVAL windows
    2. Data types: support for variable-length strings
    3. Caching: faster row-oriented caching
    4. Observability: more insight into operations and maintenance
    | + +For more information, see [TDengine Public Roadmap](https://github.com/orgs/taosdata/projects/4). diff --git a/docs/zh/29-roadmap/index.md b/docs/zh/29-roadmap/index.md new file mode 100644 index 0000000000..facb9c7253 --- /dev/null +++ b/docs/zh/29-roadmap/index.md @@ -0,0 +1,14 @@ +--- +title: 产品路线图 +--- + +TDengine OSS 之 2025 年年度路线图如下表所示。 + +| 季度 | 功能 | +| :----- | :----- | +| 2025Q1 |
    1. 虚拟表
    2. 查询能力:REGEXPGREATESTLEASTCAST 函数支持判断表达式、单行选择函数的其他列值、INTERP 支持插值时间范围
    3. 存储能力:支持将查询结果写入超级表、超级表支持 KEEP 参数、STMT 写入性能提升
    4. 流计算:支持虚拟表、计算资源优化、事件通知机制、创建时间优化
    5. 数据类型:Decimal
    6. 高可用:加快宕机恢复时间、优化客户端 Failover 机制
    7. 稳定性:开始维护新的稳定版本 3.3.6.x
    8. JDBC:高效写入
    9. 生态工具:对接 Tableau
    10. 生态工具:对接 Excel
    | +| 2025Q2 |
    1. 查询能力:大幅放宽关联查询限制、支持 MySQL 所有数学函数、支持积分/积分平均/连续方差函数、CSUM 函数优化、COUNT(DISTINCT) 语法、事件窗口功能增强、提升标签过滤性能、提升 INTERP 查询性能
    2. 存储能力:TSMA 计算资源优化、写入抖动优化
    3. 流计算:节点高可用
    4. 数据类型:BLOB
    5. 数据订阅:支持 MQTT 协议
    6. 高可用:提高副本变更速度、提高集群宕机恢复速度、优化断电数据恢复机制
    7. 可观测性:写入诊断工具
    8. 生态工具:对接帆软 FineBI
    | +| 2025Q3 |
    1. 查询能力:支持更多子查询类型、支持 MySQL 运算符、支持 MySQL 所有时间函数、窗口计算逻辑优化、查询性能抖动、计数窗口允许指定列
    2. 存储能力:提高 SQL 模式写入速度
    3. 可观测性:查询诊断工具、优化 EXPLAIN 输出、长任务观测
    | +| 2025Q4 |
    1. 查询能力:窗口函数(OVER 子句)、支持 MySQL 所有字符串/聚合/条件函数、Partition 支持组内排序、控制查询资源占用、提高子表聚合查询性能、INTERVAL 窗口支持插值时间范围
    2. 数据类型:支持不定长度字符串数据类型
    3. 数据缓存:提升按行缓存性能
    4. 可观测性:增强运维可观测性
    | + +欲了解更多信息,请参见 [TDengine Public Roadmap](https://github.com/orgs/taosdata/projects/4). From 22a6577e520ab3225e006fad872e9b0e09e0bd5d Mon Sep 17 00:00:00 2001 From: danielclow <106956386+danielclow@users.noreply.github.com> Date: Wed, 26 Feb 2025 17:30:55 +0800 Subject: [PATCH 36/40] docs: correct period --- docs/zh/29-roadmap/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/29-roadmap/index.md b/docs/zh/29-roadmap/index.md index facb9c7253..a68e97a410 100644 --- a/docs/zh/29-roadmap/index.md +++ b/docs/zh/29-roadmap/index.md @@ -11,4 +11,4 @@ TDengine OSS 之 2025 年年度路线图如下表所示。 | 2025Q3 |
    1. 查询能力:支持更多子查询类型、支持 MySQL 运算符、支持 MySQL 所有时间函数、窗口计算逻辑优化、查询性能抖动、计数窗口允许指定列
    2. 存储能力:提高 SQL 模式写入速度
    3. 可观测性:查询诊断工具、优化 EXPLAIN 输出、长任务观测
    | | 2025Q4 |
    1. 查询能力:窗口函数(OVER 子句)、支持 MySQL 所有字符串/聚合/条件函数、Partition 支持组内排序、控制查询资源占用、提高子表聚合查询性能、INTERVAL 窗口支持插值时间范围
    2. 数据类型:支持不定长度字符串数据类型
    3. 数据缓存:提升按行缓存性能
    4. 可观测性:增强运维可观测性
    | -欲了解更多信息,请参见 [TDengine Public Roadmap](https://github.com/orgs/taosdata/projects/4). +欲了解更多信息,请参见 [TDengine Public Roadmap](https://github.com/orgs/taosdata/projects/4) 。 From a7cbf6f8b2b7589f2f654bdfb54bbff88f3a76ec Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Wed, 26 Feb 2025 17:53:39 +0800 Subject: [PATCH 37/40] refactor: replace inspect-based class retrieval with AST parsing for improved class order handling --- tests/army/test.py | 22 +++++++++++++--------- tests/develop-test/test.py | 18 ++++++++++-------- tests/pytest/test.py | 17 ++++++++++------- tests/pytest/util/cases.py | 31 ++++++++++++++++++------------- tests/system-test/test.py | 23 ++++++++++++++--------- tests/test_new/test.py | 23 ++++++++++++++--------- 6 files changed, 79 insertions(+), 55 deletions(-) diff --git a/tests/army/test.py b/tests/army/test.py index 56b868c7ae..6ac0948b7b 100644 --- a/tests/army/test.py +++ b/tests/army/test.py @@ -24,7 +24,7 @@ import platform import socket import threading import importlib -import inspect +import ast import toml from frame.log import * @@ -56,11 +56,11 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) +def get_local_classes_in_order(file_path): + with open(file_path, "r", encoding="utf-8") as file: + tree = ast.parse(file.read(), filename=file_path) + + classes = [node.name for node in ast.walk(tree) if isinstance(node, ast.ClassDef)] return classes def dynamicLoadModule(fileName): @@ -77,8 +77,10 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep + uModule = dynamicLoadModule(fileName) - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) case = case_class() # create conn @@ -371,7 +373,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict @@ -543,7 +546,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict diff --git a/tests/develop-test/test.py b/tests/develop-test/test.py index 76d5732e80..3525fd6332 100644 --- a/tests/develop-test/test.py +++ b/tests/develop-test/test.py @@ -22,7 +22,7 @@ import json import platform import socket import threading -import inspect +import ast import importlib import os @@ -59,11 +59,11 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) +def get_local_classes_in_order(file_path): + with open(file_path, "r", encoding="utf-8") as file: + tree = ast.parse(file.read(), filename=file_path) + + classes = [node.name for node in ast.walk(tree) if isinstance(node, ast.ClassDef)] return classes def dynamicLoadModule(fileName): @@ -311,7 +311,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if (json.dumps(updateCfgDict) == "{}") and hasattr( ucase, "updatecfgDict" @@ -450,7 +451,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if json.dumps(updateCfgDict) == "{}": updateCfgDict = ucase.updatecfgDict diff --git a/tests/pytest/test.py b/tests/pytest/test.py index 5d1a15961a..fa91d20c00 100644 --- a/tests/pytest/test.py +++ b/tests/pytest/test.py @@ -19,6 +19,7 @@ import subprocess import time from distutils.log import warn as printf import platform +import ast from util.log import * from util.dnodes import * @@ -26,11 +27,11 @@ from util.cases import * import taos -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) +def get_local_classes_in_order(file_path): + with open(file_path, "r", encoding="utf-8") as file: + tree = ast.parse(file.read(), filename=file_path) + + classes = [node.name for node in ast.walk(tree) if isinstance(node, ast.ClassDef)] return classes def dynamicLoadModule(fileName): @@ -149,7 +150,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() tdDnodes.deploy(1,ucase.updatecfgDict) except : @@ -183,7 +185,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() tdDnodes.deploy(1,ucase.updatecfgDict) except : diff --git a/tests/pytest/util/cases.py b/tests/pytest/util/cases.py index 48f73c50f1..ac9b97a874 100644 --- a/tests/pytest/util/cases.py +++ b/tests/pytest/util/cases.py @@ -20,7 +20,7 @@ import importlib import traceback from util.log import * import platform - +import ast class TDCase: def __init__(self, name, case): @@ -51,11 +51,11 @@ class TDCases: def addCluster(self, name, case): self.clusterCases.append(TDCase(name, case)) - def get_local_classes(self, module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) + def get_local_classes_in_order(self, file_path): + with open(file_path, "r", encoding="utf-8") as file: + tree = ast.parse(file.read(), filename=file_path) + + classes = [node.name for node in ast.walk(tree) if isinstance(node, ast.ClassDef)] return classes def runAllLinux(self, conn): @@ -64,7 +64,8 @@ class TDCases: for tmp in self.linuxCases: if tmp.name.find(fileName) != -1: # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[0]) + class_names = self.get_local_classes_in_order(fileName) + case_class = getattr(testModule, class_names[-1]) case = case_class() case.init(conn) case.run() @@ -81,8 +82,8 @@ class TDCases: for tmp in self.linuxCases: if tmp.name.find(fileName) != -1: # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) - print(case_class) + class_names = self.get_local_classes_in_order(fileName) + case_class = getattr(testModule, class_names[-1]) case = case_class() case.init(conn, self._logSql, replicaVar) try: @@ -101,7 +102,8 @@ class TDCases: for tmp in self.windowsCases: if tmp.name.find(fileName) != -1: # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + class_names = self.get_local_classes_in_order(fileName) + case_class = getattr(testModule, class_names[-1]) case = case_class() case.init(conn) case.run() @@ -118,7 +120,8 @@ class TDCases: for tmp in self.windowsCases: if tmp.name.find(fileName) != -1: # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + class_names = self.get_local_classes_in_order(fileName) + case_class = getattr(testModule, class_names[-1]) case = case_class() case.init(conn, self._logSql,replicaVar) try: @@ -140,7 +143,8 @@ class TDCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + class_names = self.get_local_classes_in_order(fileName) + case_class = getattr(testModule, class_names[-1]) case = case_class() case.init() case.run() @@ -158,7 +162,8 @@ class TDCases: if tmp.name.find(fileName) != -1: tdLog.notice("run cases like %s" % (fileName)) # get the last class name as the test case class name - case_class = getattr(testModule, self.get_local_classes(testModule)[-1]) + class_names = self.get_local_classes_in_order(fileName) + case_class = getattr(testModule, class_names[-1]) case = case_class() case.init() case.run() diff --git a/tests/system-test/test.py b/tests/system-test/test.py index bf66b6a765..cd0e60160c 100644 --- a/tests/system-test/test.py +++ b/tests/system-test/test.py @@ -24,7 +24,7 @@ import platform import socket import threading import importlib -import inspect +import ast print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -59,13 +59,14 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) +def get_local_classes_in_order(file_path): + with open(file_path, "r", encoding="utf-8") as file: + tree = ast.parse(file.read(), filename=file_path) + + classes = [node.name for node in ast.walk(tree) if isinstance(node, ast.ClassDef)] return classes + def dynamicLoadModule(fileName): moduleName = fileName.replace(".py", "").replace(os.sep, ".") return importlib.import_module(moduleName, package='..') @@ -80,8 +81,10 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep + uModule = dynamicLoadModule(fileName) - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) case = case_class() # create conn @@ -364,7 +367,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict @@ -536,7 +540,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict diff --git a/tests/test_new/test.py b/tests/test_new/test.py index bf66b6a765..cd0e60160c 100644 --- a/tests/test_new/test.py +++ b/tests/test_new/test.py @@ -24,7 +24,7 @@ import platform import socket import threading import importlib -import inspect +import ast print(f"Python version: {sys.version}") print(f"Version info: {sys.version_info}") @@ -59,13 +59,14 @@ def checkRunTimeError(): if hwnd: os.system("TASKKILL /F /IM taosd.exe") -def get_local_classes(module): - classes = [] - for name, obj in inspect.getmembers(module, inspect.isclass): - if inspect.getmodule(obj) == module: - classes.append(name) +def get_local_classes_in_order(file_path): + with open(file_path, "r", encoding="utf-8") as file: + tree = ast.parse(file.read(), filename=file_path) + + classes = [node.name for node in ast.walk(tree) if isinstance(node, ast.ClassDef)] return classes + def dynamicLoadModule(fileName): moduleName = fileName.replace(".py", "").replace(os.sep, ".") return importlib.import_module(moduleName, package='..') @@ -80,8 +81,10 @@ def runOnPreviousCluster(host, config, fileName): sep = "/" if platform.system().lower() == 'windows': sep = os.sep + uModule = dynamicLoadModule(fileName) - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) case = case_class() # create conn @@ -364,7 +367,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if ((json.dumps(updateCfgDict) == '{}') and hasattr(ucase, 'updatecfgDict')): updateCfgDict = ucase.updatecfgDict @@ -536,7 +540,8 @@ if __name__ == "__main__": if is_test_framework: uModule = dynamicLoadModule(fileName) try: - case_class = getattr(uModule, get_local_classes(uModule)[-1]) + class_names = get_local_classes_in_order(fileName) + case_class = getattr(uModule, class_names[-1]) ucase = case_class() if (json.dumps(updateCfgDict) == '{}'): updateCfgDict = ucase.updatecfgDict From 078280ef94f64ebd2bb2ea32ec268ac87de3a6d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=A8=8B=E5=BA=8F=E5=91=98=E5=B0=8F=E5=A2=A8?= <16949192+coder-xiaomo@users.noreply.github.com> Date: Thu, 27 Feb 2025 05:52:51 +0800 Subject: [PATCH 38/40] Update 02-insert.md fix an typo in sql code --- docs/zh/05-basic/02-insert.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/05-basic/02-insert.md b/docs/zh/05-basic/02-insert.md index b129fdbff1..c5d3096aab 100644 --- a/docs/zh/05-basic/02-insert.md +++ b/docs/zh/05-basic/02-insert.md @@ -111,7 +111,7 @@ TDengine 还支持直接向超级表写入数据。需要注意的是,超级 ```sql insert into meters (tbname, ts, current, voltage, phase, location, group_id) -values( "d1001, "2018-10-03 14:38:05", 10.2, 220, 0.23, "California.SanFrancisco", 2) +values("d1001", "2018-10-03 14:38:05", 10.2, 220, 0.23, "California.SanFrancisco", 2) ``` ### 零代码写入 From 9d3a00920e5a2c2c78fcf31657865119f6a0a4e5 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Thu, 27 Feb 2025 11:23:40 +0800 Subject: [PATCH 39/40] fix(stream): fix mem leak && rebuild event window when the window end flag is modified --- source/libs/executor/src/streameventwindowoperator.c | 5 +++++ source/libs/executor/src/streamtimewindowoperator.c | 12 +++++------- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/source/libs/executor/src/streameventwindowoperator.c b/source/libs/executor/src/streameventwindowoperator.c index fa6008eba7..a2304a2e6c 100644 --- a/source/libs/executor/src/streameventwindowoperator.c +++ b/source/libs/executor/src/streameventwindowoperator.c @@ -234,6 +234,11 @@ int32_t updateEventWindowInfo(SStreamAggSupporter* pAggSup, SEventWindowInfo* pW pWin->ekey = pTsData[i]; pWinInfo->pWinFlag->endFlag = ends[i]; } else if (pWin->ekey == pTsData[i]) { + if (pWinInfo->pWinFlag->endFlag == true && ends[i] == false) { + (*pWinRow) = i + 1 - start; + *pRebuild = true; + goto _end; + } pWinInfo->pWinFlag->endFlag |= ends[i]; } else if (ends[i] && !pWinInfo->pWinFlag->endFlag) { *pRebuild = true; diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 5cab26b9d3..3b6c1963dc 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -479,14 +479,12 @@ void destroyFlusedppPos(void* ppRes) { } void clearGroupResInfo(SGroupResInfo* pGroupResInfo) { - if (pGroupResInfo->freeItem) { - int32_t size = taosArrayGetSize(pGroupResInfo->pRows); - for (int32_t i = pGroupResInfo->index; i < size; i++) { - void* pPos = taosArrayGetP(pGroupResInfo->pRows, i); - destroyFlusedPos(pPos); - } - pGroupResInfo->freeItem = false; + int32_t size = taosArrayGetSize(pGroupResInfo->pRows); + for (int32_t i = pGroupResInfo->index; i < size; i++) { + void* pPos = taosArrayGetP(pGroupResInfo->pRows, i); + destroyFlusedPos(pPos); } + pGroupResInfo->freeItem = false; taosArrayDestroy(pGroupResInfo->pRows); pGroupResInfo->pRows = NULL; pGroupResInfo->index = 0; From 9c6e424cd987060f64672b3d7bc9a1dc3a1b4cb5 Mon Sep 17 00:00:00 2001 From: Simon Guan Date: Fri, 28 Feb 2025 09:14:35 +0800 Subject: [PATCH 40/40] docs: update keywords --- docs/zh/08-operation/18-ha/02-replica2.md | 4 ++-- docs/zh/14-reference/03-taos-sql/20-keywords.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/08-operation/18-ha/02-replica2.md b/docs/zh/08-operation/18-ha/02-replica2.md index e238e3f4f4..aff310ecbe 100644 --- a/docs/zh/08-operation/18-ha/02-replica2.md +++ b/docs/zh/08-operation/18-ha/02-replica2.md @@ -67,9 +67,9 @@ alter database replica 2|1 | 异常场景 | 集群状态 | | ------- | ------ | -| 没有 Vnode 发生故障: Arbitrator 故障(Mnode 宕机节点超过一个,导致 Mnode 无法选主)| **持续提供服务** | +| 没有 Vnode 发生故障:Arbitrator 故障(Mnode 宕机节点超过一个,导致 Mnode 无法选主)| **持续提供服务** | | 仅一个 Vnode 故障:VGroup 已经达成同步后,某一个 Vnode 才发生故障的 | **持续提供服务** | -| 仅一个 Vnode 故障:2个 Vnode 同时故障,故障前 VGroup 达成同步,但是只有一个 Vnode 从故障中恢复服务,另一个 Vnode 服务故障 | **通过下面的命令,强制指定leader, 继续提供服务** | +| 仅一个 Vnode 故障:2 个 Vnode 同时故障,故障前 VGroup 达成同步,但是只有一个 Vnode 从故障中恢复服务,另一个 Vnode 服务故障 | **通过下面的命令,强制指定 leader, 继续提供服务** | | 仅一个 Vnode 故障:离线 Vnode 启动后,VGroup 未达成同步前,另一个 Vnode 服务故障的 | **无法提供服务** | | 两个 Vnode 都发生故障 | **无法提供服务** | diff --git a/docs/zh/14-reference/03-taos-sql/20-keywords.md b/docs/zh/14-reference/03-taos-sql/20-keywords.md index 8d5a538dd5..33eea11618 100644 --- a/docs/zh/14-reference/03-taos-sql/20-keywords.md +++ b/docs/zh/14-reference/03-taos-sql/20-keywords.md @@ -78,7 +78,7 @@ description: TDengine 保留关键字的详细列表 | CLIENT_VERSION | | | CLUSTER | | | COLON | | -| COLS | | +| COLS | 3.3.6.0 及后续版本 | | COLUMN | | | COMMA | | | COMMENT | |