From 02598a7b456b67b0ec486c0e35b69acf3aca0911 Mon Sep 17 00:00:00 2001 From: Love EvenWong <39328362+LoveEvenWong@users.noreply.github.com> Date: Thu, 19 Sep 2019 12:28:13 +0800 Subject: [PATCH] =?UTF-8?q?flink=20task=20support(flink=20=E4=BB=BB?= =?UTF-8?q?=E5=8A=A1=E6=94=AF=E6=8C=81)=20(#711)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * flink任务支持 * flink任务支持 * Update zh_CN.js * Update FlinkArgsUtils.java * Update .escheduler_env.sh --- .../java/cn/escheduler/common/Constants.java | 14 + .../cn/escheduler/common/enums/TaskType.java | 3 +- .../common/task/flink/FlinkParameters.java | 219 ++++++++++ .../common/utils/TaskParametersUtils.java | 3 + .../server/utils/FlinkArgsUtils.java | 110 +++++ .../server/worker/task/AbstractTask.java | 3 + .../server/worker/task/TaskManager.java | 3 + .../server/worker/task/flink/FlinkTask.java | 118 ++++++ .../js/conf/home/pages/dag/_source/config.js | 4 + .../js/conf/home/pages/dag/_source/dag.scss | 3 + .../pages/dag/_source/formModel/formModel.vue | 128 +++--- .../dag/_source/formModel/tasks/flink.vue | 388 ++++++++++++++++++ .../conf/home/pages/dag/img/toobar_flink.svg | 211 ++++++++++ .../src/js/module/i18n/locale/zh_CN.js | 6 +- .../docs/zh_CN/_book/images/flink_edit.png | Bin 0 -> 123946 bytes script/env/.escheduler_env.sh | 4 +- 16 files changed, 1153 insertions(+), 64 deletions(-) create mode 100644 escheduler-common/src/main/java/cn/escheduler/common/task/flink/FlinkParameters.java create mode 100644 escheduler-server/src/main/java/cn/escheduler/server/utils/FlinkArgsUtils.java create mode 100644 escheduler-server/src/main/java/cn/escheduler/server/worker/task/flink/FlinkTask.java create mode 100644 escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue create mode 100644 escheduler-ui/src/js/conf/home/pages/dag/img/toobar_flink.svg create mode 100644 escheduler-ui/src/view/docs/zh_CN/_book/images/flink_edit.png diff --git a/escheduler-common/src/main/java/cn/escheduler/common/Constants.java b/escheduler-common/src/main/java/cn/escheduler/common/Constants.java index 10a4460678..7eaa6b7926 100644 --- a/escheduler-common/src/main/java/cn/escheduler/common/Constants.java +++ b/escheduler-common/src/main/java/cn/escheduler/common/Constants.java @@ -906,4 +906,18 @@ public final class Constants { * hive conf */ public static final String HIVE_CONF = "hiveconf:"; + + //flink 任务 + public static final String FLINK_YARN_CLUSTER = "yarn-cluster"; + public static final String FLINK_RUN_MODE = "-m"; + public static final String FLINK_YARN_SLOT = "-ys"; + public static final String FLINK_APP_NAME = "-ynm"; + public static final String FLINK_TASK_MANAGE = "-yn"; + + public static final String FLINK_JOB_MANAGE_MEM = "-yjm"; + public static final String FLINK_TASK_MANAGE_MEM = "-ytm"; + public static final String FLINK_detach = "-d"; + public static final String FLINK_MAIN_CLASS = "-c"; + + } diff --git a/escheduler-common/src/main/java/cn/escheduler/common/enums/TaskType.java b/escheduler-common/src/main/java/cn/escheduler/common/enums/TaskType.java index 1d589167e3..7e4fde6a34 100644 --- a/escheduler-common/src/main/java/cn/escheduler/common/enums/TaskType.java +++ b/escheduler-common/src/main/java/cn/escheduler/common/enums/TaskType.java @@ -29,8 +29,9 @@ public enum TaskType { * 5 SPARK * 6 PYTHON * 7 DEPENDENT + * 8 FLINK */ - SHELL,SQL, SUB_PROCESS,PROCEDURE,MR,SPARK,PYTHON,DEPENDENT; + SHELL,SQL, SUB_PROCESS,PROCEDURE,MR,SPARK,PYTHON,DEPENDENT,FLINK; public static boolean typeIsNormalTask(String typeName) { TaskType taskType = TaskType.valueOf(typeName); diff --git a/escheduler-common/src/main/java/cn/escheduler/common/task/flink/FlinkParameters.java b/escheduler-common/src/main/java/cn/escheduler/common/task/flink/FlinkParameters.java new file mode 100644 index 0000000000..54dfcb7103 --- /dev/null +++ b/escheduler-common/src/main/java/cn/escheduler/common/task/flink/FlinkParameters.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package cn.escheduler.common.task.flink; + +import cn.escheduler.common.enums.ProgramType; +import cn.escheduler.common.process.ResourceInfo; +import cn.escheduler.common.task.AbstractParameters; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * spark parameters + */ +public class FlinkParameters extends AbstractParameters { + + /** + * major jar + */ + private ResourceInfo mainJar; + + /** + * major class + */ + private String mainClass; + + /** + * deploy mode yarn-cluster yarn-client yarn-local + */ + private String deployMode; + + /** + * arguments + */ + private String mainArgs; + + /** + * slot个数 + */ + private int slot; + + /** + *Yarn application的名字 + */ + + private String appName; + + /** + * taskManager 数量 + */ + private int taskManager; + + /** + * jobManagerMemory 内存大小 + */ + private String jobManagerMemory ; + + /** + * taskManagerMemory内存大小 + */ + private String taskManagerMemory; + + /** + * resource list + */ + private List resourceList; + + /** + * The YARN queue to submit to + */ + private String queue; + + /** + * other arguments + */ + private String others; + + /** + * program type + * 0 JAVA,1 SCALA,2 PYTHON + */ + private ProgramType programType; + + public ResourceInfo getMainJar() { + return mainJar; + } + + public void setMainJar(ResourceInfo mainJar) { + this.mainJar = mainJar; + } + + public String getMainClass() { + return mainClass; + } + + public void setMainClass(String mainClass) { + this.mainClass = mainClass; + } + + public String getDeployMode() { + return deployMode; + } + + public void setDeployMode(String deployMode) { + this.deployMode = deployMode; + } + + public String getMainArgs() { + return mainArgs; + } + + public void setMainArgs(String mainArgs) { + this.mainArgs = mainArgs; + } + + public int getSlot() { + return slot; + } + + public void setSlot(int slot) { + this.slot = slot; + } + + public String getAppName() { + return appName; + } + + public void setAppName(String appName) { + this.appName = appName; + } + + public int getTaskManager() { + return taskManager; + } + + public void setTaskManager(int taskManager) { + this.taskManager = taskManager; + } + + public String getJobManagerMemory() { + return jobManagerMemory; + } + + public void setJobManagerMemory(String jobManagerMemory) { + this.jobManagerMemory = jobManagerMemory; + } + + public String getTaskManagerMemory() { + return taskManagerMemory; + } + + public void setTaskManagerMemory(String taskManagerMemory) { + this.taskManagerMemory = taskManagerMemory; + } + + public String getQueue() { + return queue; + } + + public void setQueue(String queue) { + this.queue = queue; + } + + public List getResourceList() { + return resourceList; + } + + public void setResourceList(List resourceList) { + this.resourceList = resourceList; + } + + public String getOthers() { + return others; + } + + public void setOthers(String others) { + this.others = others; + } + + public ProgramType getProgramType() { + return programType; + } + + public void setProgramType(ProgramType programType) { + this.programType = programType; + } + + @Override + public boolean checkParameters() { + return mainJar != null && programType != null; + } + + + @Override + public List getResourceFilesList() { + if(resourceList !=null ) { + this.resourceList.add(mainJar); + return resourceList.stream() + .map(p -> p.getRes()).collect(Collectors.toList()); + } + return null; + } + + +} diff --git a/escheduler-common/src/main/java/cn/escheduler/common/utils/TaskParametersUtils.java b/escheduler-common/src/main/java/cn/escheduler/common/utils/TaskParametersUtils.java index feff4141da..c8ceeb44a2 100644 --- a/escheduler-common/src/main/java/cn/escheduler/common/utils/TaskParametersUtils.java +++ b/escheduler-common/src/main/java/cn/escheduler/common/utils/TaskParametersUtils.java @@ -19,6 +19,7 @@ package cn.escheduler.common.utils; import cn.escheduler.common.enums.TaskType; import cn.escheduler.common.task.AbstractParameters; import cn.escheduler.common.task.dependent.DependentParameters; +import cn.escheduler.common.task.flink.FlinkParameters; import cn.escheduler.common.task.mr.MapreduceParameters; import cn.escheduler.common.task.procedure.ProcedureParameters; import cn.escheduler.common.task.python.PythonParameters; @@ -63,6 +64,8 @@ public class TaskParametersUtils { return JSONUtils.parseObject(parameter, PythonParameters.class); case DEPENDENT: return JSONUtils.parseObject(parameter, DependentParameters.class); + case FLINK: + return JSONUtils.parseObject(parameter, FlinkParameters.class); default: return null; } diff --git a/escheduler-server/src/main/java/cn/escheduler/server/utils/FlinkArgsUtils.java b/escheduler-server/src/main/java/cn/escheduler/server/utils/FlinkArgsUtils.java new file mode 100644 index 0000000000..308103073d --- /dev/null +++ b/escheduler-server/src/main/java/cn/escheduler/server/utils/FlinkArgsUtils.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package cn.escheduler.server.utils; + + +import cn.escheduler.common.Constants; +import cn.escheduler.common.enums.ProgramType; +import cn.escheduler.common.task.flink.FlinkParameters; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.List; + + +/** + * spark args utils + */ +public class FlinkArgsUtils { + + /** + * build args + * @param param + * @return + */ + public static List buildArgs(FlinkParameters param) { + List args = new ArrayList<>(); + + args.add(Constants.FLINK_RUN_MODE); //-m + + args.add(Constants.FLINK_YARN_CLUSTER); //yarn-cluster + + if (param.getSlot() != 0) { + args.add(Constants.FLINK_YARN_SLOT); + args.add(String.format("%d", param.getSlot())); //-ys + } + + if (StringUtils.isNotEmpty(param.getAppName())) { //-ynm + args.add(Constants.FLINK_APP_NAME); + args.add(param.getAppName()); + } + + if (param.getTaskManager() != 0) { //-yn + args.add(Constants.FLINK_TASK_MANAGE); + args.add(String.format("%d", param.getTaskManager())); + } + + if (StringUtils.isNotEmpty(param.getJobManagerMemory())) { + args.add(Constants.FLINK_JOB_MANAGE_MEM); + args.add(param.getJobManagerMemory()); //-yjm + } + + if (StringUtils.isNotEmpty(param.getTaskManagerMemory())) { // -ytm + args.add(Constants.FLINK_TASK_MANAGE_MEM); + args.add(param.getTaskManagerMemory()); + } + args.add(Constants.FLINK_detach); //-d + + + if(param.getProgramType() !=null ){ + if(param.getProgramType()!=ProgramType.PYTHON){ + if (StringUtils.isNotEmpty(param.getMainClass())) { + args.add(Constants.FLINK_MAIN_CLASS); //-c + args.add(param.getMainClass()); //main class + } + } + } + + if (param.getMainJar() != null) { + args.add(param.getMainJar().getRes()); + } + + + // --files --conf --libjar ... + if (StringUtils.isNotEmpty(param.getOthers())) { + String others = param.getOthers(); + if(!others.contains("--queue")){ + if (StringUtils.isNotEmpty(param.getQueue())) { + args.add(Constants.SPARK_QUEUE); + args.add(param.getQueue()); + } + } + args.add(param.getOthers()); + }else if (StringUtils.isNotEmpty(param.getQueue())) { + args.add(Constants.SPARK_QUEUE); + args.add(param.getQueue()); + + } + + if (StringUtils.isNotEmpty(param.getMainArgs())) { + args.add(param.getMainArgs()); + } + + return args; + } + +} diff --git a/escheduler-server/src/main/java/cn/escheduler/server/worker/task/AbstractTask.java b/escheduler-server/src/main/java/cn/escheduler/server/worker/task/AbstractTask.java index 213f4fd3f9..6472873d8b 100644 --- a/escheduler-server/src/main/java/cn/escheduler/server/worker/task/AbstractTask.java +++ b/escheduler-server/src/main/java/cn/escheduler/server/worker/task/AbstractTask.java @@ -22,6 +22,7 @@ import cn.escheduler.common.enums.TaskRecordStatus; import cn.escheduler.common.enums.TaskType; import cn.escheduler.common.process.Property; import cn.escheduler.common.task.AbstractParameters; +import cn.escheduler.common.task.flink.FlinkParameters; import cn.escheduler.common.task.mr.MapreduceParameters; import cn.escheduler.common.task.procedure.ProcedureParameters; import cn.escheduler.common.task.python.PythonParameters; @@ -178,6 +179,8 @@ public abstract class AbstractTask { case SPARK: paramsClass = SparkParameters.class; break; + case FLINK: + paramsClass = FlinkParameters.class; case PYTHON: paramsClass = PythonParameters.class; break; diff --git a/escheduler-server/src/main/java/cn/escheduler/server/worker/task/TaskManager.java b/escheduler-server/src/main/java/cn/escheduler/server/worker/task/TaskManager.java index e23a29ae08..986a6179c9 100644 --- a/escheduler-server/src/main/java/cn/escheduler/server/worker/task/TaskManager.java +++ b/escheduler-server/src/main/java/cn/escheduler/server/worker/task/TaskManager.java @@ -19,6 +19,7 @@ package cn.escheduler.server.worker.task; import cn.escheduler.common.enums.TaskType; import cn.escheduler.server.worker.task.dependent.DependentTask; +import cn.escheduler.server.worker.task.flink.FlinkTask; import cn.escheduler.server.worker.task.mr.MapReduceTask; import cn.escheduler.server.worker.task.processdure.ProcedureTask; import cn.escheduler.server.worker.task.python.PythonTask; @@ -55,6 +56,8 @@ public class TaskManager { return new MapReduceTask(props, logger); case SPARK: return new SparkTask(props, logger); + case FLINK: + return new FlinkTask(props, logger); case PYTHON: return new PythonTask(props, logger); case DEPENDENT: diff --git a/escheduler-server/src/main/java/cn/escheduler/server/worker/task/flink/FlinkTask.java b/escheduler-server/src/main/java/cn/escheduler/server/worker/task/flink/FlinkTask.java new file mode 100644 index 0000000000..bf6f0cc9fb --- /dev/null +++ b/escheduler-server/src/main/java/cn/escheduler/server/worker/task/flink/FlinkTask.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package cn.escheduler.server.worker.task.flink; + +import cn.escheduler.common.process.Property; +import cn.escheduler.common.task.AbstractParameters; +import cn.escheduler.common.task.flink.FlinkParameters; +import cn.escheduler.common.utils.JSONUtils; +import cn.escheduler.common.utils.ParameterUtils; +import cn.escheduler.dao.model.ProcessInstance; +import cn.escheduler.server.utils.FlinkArgsUtils; +import cn.escheduler.server.utils.ParamUtils; +import cn.escheduler.server.worker.task.AbstractYarnTask; +import cn.escheduler.server.worker.task.TaskProps; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * flink task + */ +public class FlinkTask extends AbstractYarnTask { + + /** + * flink command + */ + private static final String FLINK_COMMAND = "flink"; + private static final String FLINK_RUN = "run"; + + /** + * flink parameters + */ + private FlinkParameters flinkParameters; + + public FlinkTask(TaskProps props, Logger logger) { + super(props, logger); + } + + @Override + public void init() { + + logger.info("flink task params {}", taskProps.getTaskParams()); + + flinkParameters = JSONUtils.parseObject(taskProps.getTaskParams(), FlinkParameters.class); + + if (!flinkParameters.checkParameters()) { + throw new RuntimeException("flink task params is not valid"); + } + flinkParameters.setQueue(taskProps.getQueue()); + + if (StringUtils.isNotEmpty(flinkParameters.getMainArgs())) { + String args = flinkParameters.getMainArgs(); + // get process instance by task instance id + ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId()); + + /** + * combining local and global parameters + */ + Map paramsMap = ParamUtils.convert(taskProps.getUserDefParamsMap(), + taskProps.getDefinedParams(), + flinkParameters.getLocalParametersMap(), + processInstance.getCmdTypeIfComplement(), + processInstance.getScheduleTime()); + + logger.info("param Map : {}", paramsMap); + if (paramsMap != null ){ + + args = ParameterUtils.convertParameterPlaceholders(args, ParamUtils.convert(paramsMap)); + logger.info("param args : {}", args); + } + flinkParameters.setMainArgs(args); + } + } + + /** + * create command + * @return + */ + @Override + protected String buildCommand() { + List args = new ArrayList<>(); + + args.add(FLINK_COMMAND); + args.add(FLINK_RUN); + logger.info("flink task args : {}", args); + // other parameters + args.addAll(FlinkArgsUtils.buildArgs(flinkParameters)); + + String command = ParameterUtils + .convertParameterPlaceholders(String.join(" ", args), taskProps.getDefinedParams()); + + logger.info("flink task command : {}", command); + + return command; + } + + @Override + public AbstractParameters getParameters() { + return flinkParameters; + } +} diff --git a/escheduler-ui/src/js/conf/home/pages/dag/_source/config.js b/escheduler-ui/src/js/conf/home/pages/dag/_source/config.js index 9c1065870f..5305bf7476 100644 --- a/escheduler-ui/src/js/conf/home/pages/dag/_source/config.js +++ b/escheduler-ui/src/js/conf/home/pages/dag/_source/config.js @@ -260,6 +260,10 @@ let tasksType = { desc: 'SPARK', color: '#E46F13' }, + 'FLINK': { + desc: 'FLINK', + color: '#E46F13' + }, 'MR': { desc: 'MapReduce', color: '#A0A5CC' diff --git a/escheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss b/escheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss index 95592e0754..37d3acaa19 100644 --- a/escheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss +++ b/escheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss @@ -70,6 +70,9 @@ .icos-SPARK { background: url("../img/toolbar_SPARK.png") no-repeat 50% 50%; } + .icos-FLINK { + background: url("../img/toobar_flink.svg") no-repeat 50% 50%; + } .icos-MR { background: url("../img/toolbar_MR.png") no-repeat 50% 50%; } diff --git a/escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue index 863a44abf5..a46b894d11 100644 --- a/escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -19,13 +19,13 @@
@@ -52,13 +52,13 @@
@@ -96,68 +96,74 @@ + ref="timeout" + :backfill-item="backfillItem" + @on-timeout="_onTimeout"> + v-if="taskType === 'SHELL'" + @on-params="_onParams" + ref="SHELL" + :backfill-item="backfillItem"> + v-if="taskType === 'SUB_PROCESS'" + @on-params="_onParams" + @on-set-process-name="_onSetProcessName" + ref="SUB_PROCESS" + :backfill-item="backfillItem"> + v-if="taskType === 'PROCEDURE'" + @on-params="_onParams" + ref="PROCEDURE" + :backfill-item="backfillItem"> + v-if="taskType === 'SQL'" + @on-params="_onParams" + ref="SQL" + :create-node-id="id" + :backfill-item="backfillItem"> + v-if="taskType === 'SPARK'" + @on-params="_onParams" + ref="SPARK" + :backfill-item="backfillItem"> + + + v-if="taskType === 'MR'" + @on-params="_onParams" + ref="MR" + :backfill-item="backfillItem"> + v-if="taskType === 'PYTHON'" + @on-params="_onParams" + ref="PYTHON" + :backfill-item="backfillItem"> + v-if="taskType === 'DEPENDENT'" + @on-dependent="_onDependent" + ref="DEPENDENT" + :backfill-item="backfillItem"> @@ -178,6 +184,7 @@ import i18n from '@/module/i18n' import mShell from './tasks/shell' import mSpark from './tasks/spark' + import mFlink from './tasks/flink' import mPython from './tasks/python' import JSP from './../plugIn/jsPlumbHandle' import mProcedure from './tasks/procedure' @@ -284,12 +291,12 @@ } this.store.dispatch('dag/getSubProcessId', { taskId: stateId }).then(res => { this.$emit('onSubProcess', { - subProcessId: res.data.subProcessInstanceId, - fromThis: this - }) - }).catch(e => { - this.$message.error(e.msg || '') + subProcessId: res.data.subProcessInstanceId, + fromThis: this }) + }).catch(e => { + this.$message.error(e.msg || '') + }) } else { this.$emit('onSubProcess', { subProcessId: this.backfillItem.params.processDefinitionId, @@ -413,10 +420,10 @@ if (taskList.length) { taskList.forEach(v => { if (v.id === this.id) { - o = v - this.backfillItem = v - } - }) + o = v + this.backfillItem = v + } + }) // Non-null objects represent backfill if (!_.isEmpty(o)) { this.name = o.name @@ -455,6 +462,7 @@ mSql, mLog, mSpark, + mFlink, mPython, mDependent, mSelectInput, diff --git a/escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue b/escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue new file mode 100644 index 0000000000..1c3b18a64c --- /dev/null +++ b/escheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue @@ -0,0 +1,388 @@ + + + + diff --git a/escheduler-ui/src/js/conf/home/pages/dag/img/toobar_flink.svg b/escheduler-ui/src/js/conf/home/pages/dag/img/toobar_flink.svg new file mode 100644 index 0000000000..33ba8b7b3d --- /dev/null +++ b/escheduler-ui/src/js/conf/home/pages/dag/img/toobar_flink.svg @@ -0,0 +1,211 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/escheduler-ui/src/js/module/i18n/locale/zh_CN.js b/escheduler-ui/src/js/module/i18n/locale/zh_CN.js index 7ac3dda87e..cfa0224185 100644 --- a/escheduler-ui/src/js/module/i18n/locale/zh_CN.js +++ b/escheduler-ui/src/js/module/i18n/locale/zh_CN.js @@ -476,5 +476,9 @@ export default { 'warning of timeout': '超时告警', 'Next five execution times': '接下来五次执行时间', 'Execute time': '执行时间', - 'Complement range': '补数范围' + 'Complement range': '补数范围', + 'slot':'slot数量', + 'taskManager':'taskManage数量', + 'jobManagerMemory':'jobManager内存数', + 'taskManagerMemory':'taskManager内存数' } diff --git a/escheduler-ui/src/view/docs/zh_CN/_book/images/flink_edit.png b/escheduler-ui/src/view/docs/zh_CN/_book/images/flink_edit.png new file mode 100644 index 0000000000000000000000000000000000000000..b7c232115753ed31d72cf869dff8859ca717ddf9 GIT binary patch literal 123946 zcmeFZWmJ`W^fgM0Gy>8fARyf#EeJ|?w{&-xo0e9jq`SMjLApzFgLHSpefD|ZJMIzB zr#tSKcijI67-RbkpZ&|Z)|zuJzR1aZKt&=zf`WoV7615N0SXGv68LFCL;(I}ZkWgc z3JOKsOhiOZTtwuRoV|^)nWYgF)W*>KE6^t(}^P0q=wJ$vmy$(H> z7j8EP-#*eLcpRcY@o@0vv*S*|zmipW6&i`CFD))@@U;ONhB*zD9y*~nPIAu9ZVamb z=>G8JhM+3YeonOSeB}C()*U822_5RiGrK^c;Ss+pT&SZLElMg-P-D+_icN4o#<&u6 zA+W~3>QbN_R#$os{(~V+3AeIowIeELvoyU+X?6<{fKOeGj5E<+d=aTr~)?%L(h8zT!MG zEU1&XRSZdbT@&~i`0*$4hL1A6gp~Ydm^^ZK9}9+s$l&+Q*YESKm=ix?QX9{TM%f+Pg_yv)^{KeS-TE4>& z-Zq_y;l}P&5)%)@=gYiOhH|4>=sgr?7~UY_o*z)iyn!4BRbveIaGw>5l)40HJYjp z;%6ks6)Yo^K@f-S=kq#PrWJJGcX;uLpWcywglE%>6Sd_ZXtQKSr$@+6BPv256vCNB zs_|P~`A|m64(eQiyF%51B3_}bgIASBcSX8sw_px_h4?(30;LPDT*wmBq>K3Y9n4qe z*92caD13E%jr{c$k|9Y2HqD1=a^;_#S!mbzZ~c>mhRKqDO8+Jq4{Z`!%EXfpL6u{h zL?aHmNwZ`}l763?sctHu5f+<9HzrZ_axs)MgL)KsOSuZ!D+EmsYN8`Po^r$YT91hS zrDP{73w{k0T_Bvlq(im&= zg(?OY)+J^n7VnqEF6b9u45jR;;G;f?^C>{*f3Nyd73Ub{_+?VWU2>XSGitF9{F}YP zzsXyO@MBC+A3Ph}RIe zN=wOE%4wM#AFmicnfg&gIjxbUoPQoYtzPV*acFwOPE8QR8J?3?oVA}poV%FSoHeoS ze5U74(EPdOo6ZM8@m5Kn!l6-;!l9{`rc9G1)7FzMS@%*w83u)&(+TN{MT+STLYo;f zsf?mCGI5y$6WVE-#U39o8xB5`h>S=xsW8df1T2v>JGe)li{Cik=-)0NXl_ot)~X35zH{HNmS|D!D?8pUSA|JO{3D9oo|$t-C>_-)VQ7&)2#e$ z>P4ZKic3-cw7_)2^lq_c@vRlTCDq(uWq$2-XY8k41G8q78*8k^ z*t+VvW?L8g%|_Km-QzS5e-HXI(+gLDCATKqyVzqcKKztuzSeWg^SiC}eachwqn-1b zW0}j6lmHM_1o_IIur4Rs(Di zj4Sj!OaqKMXdATr%o$!C;n>&Q@8Fr{Gh|;E1i??d6_s-yb0fl9B8FtoM3!vtxKH2q zijprk;B)XjW(uYQMZ|{}sGRtSB z`^4N07Jnh^N0_%jQ0v!6B@B8srY#%<#2F+~BLu^eAf4~KXXmleLTs=vkzkPvk(Y;MQoPbWmVQ*2*yO{zyi!Xd|v7?k4?%*O%RI)si|4((1nyKB-jXW!)wQ z+w4eIOB!X*w4OgY@3ML(xT@dUN3JKv`p{y1PiAsqD1Gx-=G(e@=VHCA?9Elh>R@zB zTdJ(7f~6{{qN|BjWqhg8T-|NaI3ukwQYRZ4czUL={|FWeb(7xg+^GV|c|M!7GqZ_n1B z7uaYyNo|u2Nlm#+dN{4Vm%r6Szalcd>b{8k;TPs-6#j87spo-8UQyM-++j2QOOS@< zohI4?&&7;(5Cgg@DGOi4&YN@7zOi(J?p}ll$;iv(pFCrWJeB9ueKgdiNk2Rb?j0w$ zD_a8|olj712g@s=P0&+Md0n=eMy_*^b#Mvh`R36c4z6$8ol7rzLYHt8c|CbGc&Hso znz0?XyVgesTD8GC^|l-v=9|@77F;E=jZ01@hZqc}X^iezhxhlVCaM>H=(dogPBIPxH zs&wO6R(dAeENGGPmhX1GbfxvzP43wxUmQQFd--bi@ql(|Ppdg`Fk#&-&*Aw>VnuC_ zjZ5pPo6Ck*|Lq6AO2H@rC2z;O*%O+biT(7ZbTS&*9z`p7X715JDJY#in9nIlmuSRL za=8T~ZnWi_@^e{N;b*~1BO##>p*WF3 zcj3TAO>srE2@2Ht0WD+>y^?8ZYHE8rI@(LjJeLSXG@_cy6eH|!;&(_Klh+)M)Dr6h zE9&hnI*qcQ4b&FTZ9l35#l=fX$Pcu*!s{a_C?P2E_wSTkq4yS_HIXPMKORSq3L_?C zga{!vUw%F#;K3BlDtaZV?#2*O6moE&yQn$X3>hhS+~e?7v;QdBnT`nJed@^9 zQ*M`oLB#__*2@hCozeltoK_QSox#h8;ROA|5!3dZ&oD@D(*FD~lMpzy6C5G%R81pKtl+rD>mF5IAvo$qpj^9)h0U z8po4SK|U%;wYBAZqV8Ihe=;oKBZ_4c!r$Kw3VDt{a3Xa1b|KVK=w z0u7rOl8oW<;qO5+m-T7E9=6H z29>e`*M@4JzEC1@dCh3M1RygtF@*R)og3dSmb#j*ZNcH3x2#loX{Yn_2qJ_F$-)xS zYDI9b2HWly*9KzAaFfwjPbx5oXxnw;lp`6RPpP7}-Yy>g(0DlNL`R>|a**aes9WoZ z;61?Gh&PB37y#yF1@=W|M5HUJ`!s8k>UASUcO#T`_RFI5Jn?K=(+0jlS)Nj95|Q5Y za=}3s%x6(tt)k|Ay&n3~>S_@doeyJ}?{qNmw76lX;J&8(_czHN=0FGgqxNS(sgHM? z`eUNF`?V!;I_@-s95Y(?!-O$!{cUZUj>TGyDs*iho)`8+l9JZ^GB5n~+LxWSpyjHY zDR9m>vG{_Y>=$_`{fmo5m*rnBO4!m!*8KO~R+^5cmUFheU8^P>|D%HkX#y z`4pH=_gdm=-^PbQX-+xG#V!>Mv+U0f zHH(fD(qP@kbKOeaDKA;(i{nV2Ce9IHYWT2;k=|#SUL`@mLE%K;oO7E)hapUSiU=a= zX}KWOsMNdP)UaGva3~8=SpMwu-WQdr-Hf6Ov!(SLKki>PUM_hc`$g@AIo8u!p!1)} zt>or;?I@-ecuCi$SM-ht5#wnXrhCuc8~A1-RkwsR5#i))t}mg$vHQLD;j zdRELPRdn3e$JK4KI|2^G)UYD5#hg&XL*kqXW5niIT)tg@p-!32?FbB^Y=|=2Y`(vk zpI>YmpnC#Wa&DlMXPjZ^r{Ol96rOtqq{s_tSgb`A{j{j*z-f%I-O0}l40x&WhRAtV zKj75oepfdV_9d>2<>`QYrr6Z99Ci)!S4O_73OjqTFg#4)$V>Hae4)c@^XUmE#C^t2 z(w5=%QXk5o;49+iif~#y0%dMkD ztm$-+O*VD?{>fO=+T>nrx#_`@q$6uPcpv$Z`rjV5w&u5Oe3NASB5RXDH`#PLtm%p- z7D1$bbuZNNI!#Y25kMr?qSl8S>0;jl;d2sHnXS&W9u$AA_1MzM^zrqX)(5A(5EYvh zusbZHLlc~`(c0~6^puS5E1~cW$ zNj%wr687_7CIY7kWu0ie--l0D{LllVWq4@lYi2)5KjZ8ra**1-i7z|uA?!_UV8fBh zA`h_h2_d)@Hy}#!q?B;>E>PzDWtzFCdd#mc)iNY-y9ctfGDUeKh09b^xm(g0SZ>hU zb6EKZCSrZej?*D{sq1+xWZSdSHAhQcP(XkxG02k&pd|O3v#%dW5*FWY-#$JZKB6fx z&OMDdcoH!DsFqZ+DMZ6>HgO(){l6V*s?~}dv7F|d~%IS1wr{9 zt5wLleG2;yz#o%yu6d0_P}=8mSDvw!vuvh-A+F^nM`gtvsprdg8(fmPZ0TcZoqRC7 zq_Y?i*a0f7ck4`@?z_dfq`1SchWvH?Ep+d9RFN-1A!I2nPD^f^-MzkS69CN3QkS(i z%IkI33Gs=CNM!9@(=2Q~)6Hp*+8tVBe~K#zi4a`*KE*PGsS7w9UMWU-9Dz;b_7DWo_MqD z=Bw3E@CU&gc`&)Z4&Nu<>Ru8LdcN~X#chKXyJaPKJE9;GchlD1^cmx%V+2KnD5|TB zb<*SqpT|p|<6HN~bm7eb-3NO`yHdI<}!t&?XK}~<` zNRYtX^}|`NV0YASNZj?m~JRVlPXuX}K1z zA)Jx*K8Y5PHT={J;M60|5)ca2t~)H6dxY9{iz;P2*}42AA1 z31-j6_oq~KQE&GuWtq6) zUjl)pqR*>^n!8^G_B>(h+0#Q17~d9!YKg}im@B$IpCumM5rC@`|LC6MDWe@$aQB@e z<>7rT1H$KxJ`GYJAz7oCJJs`l%XbZg$!R1>re;TPd$0*i^%;9X2mYPHlk5bfu9UV3 zLX|b;dAM4eUo}qCX%L;V@P7qd0h?BJJjXg((9qfT#M;9YVLZ176kkC8nBL&rwOeX~ z_Ij($Ih!)4MQO7$5xYB(NulxK)6Pc-+-$WS6FH8)n=9RId4wchSM2!>Y+lS?OXT3h zIG;|=7#?(!{{a`yRqK={H);Ih2O&JCl}*|+x{qtyN7y;;;X)=0b#~{*`MlxO#tZ<$ zIBl94uY#VD;H3_BW2Ls-@7n~imobx#!{cgUG)f*TmQDy;ei8~$ zL?Mv??irtGezKTqu2>9(64-O!LU*?ozvolFa3(J{>|q1}2klW+j=NQivWgkqZ}Pp5 zV2S4a-?`;Pk#A%TrZIXTc_DaGQ8wIQgvk2}48EARLH|JFW|)?d|C6_McyAMuU(H3f zs6M$QLLhWFcaqsb0F3mzaa1jL;Rb=s3Kcf7cFCPifTW=q)yWq8jMS4^%jf=_g*&}} zD!Spyz zkbB-dT(*+ZSrQa;@!p?J$l92Vf>kaA8;PH;;v42!$VASNf)6-F4W^g_haT@n9?#6V z%;&*09|RwcBP~+hM2w`z+J4R@Y;m3g@rQ%HRGI6`yVC@=07552iO@7735 zmG5m)G_9(YoojNICSqve<-%XKjr?`9s&;gaCizVGFNR3SYfd=fFe;L}PzoC-cxs%S^o$i|>*w;VL=mXr~@u^YCBDpb-j@6Dz^u6 z!x#-0iikw0Kne1W7$6O*uuU`QC}J__={h!50S*YW)6QgFc;$!yFR*9Cc7%@kHw&8r zxV?R3XBoaGl|!7Ni&-U2C;eAjRxP&>JQm3T@>jY*N2U8LF<^#b%fy1Y9U%~y^Mh&tVu4gWUT-FO#p|L z?`4wH{q=}Xgm%t}GN(U}?sdN^%sbSz$59!c?X06v-8=FY6A#C)zpNYgZ=8rTB{m=t z9>vM~@pk!{Z=#L&Y2sLdc_NTR3f=*ke%P+~BRJ^}s7xeD6WxvQ8_bPnx@Y-Sme{*D zrJj`KejoK_!}PK?m1O1?D& z2Vpn6pNfM-P{x%diW4ZeZ97r9y7qtjrK?Q$8npIC5EGInB-5#^`zS4zPXo)=Vl}OwYg(3zrXCIm=zYXNS-Zv1;g}}@pw={DUtwym zVmS9n!ulD8Ezll63=~c+^XHyG5E!o?rZecI-OcjJ^nj0Tdf3cm_+chPUMhcKH5r4R z9$S`d>9fDT%ul|p&6qib<5K;vZwsvY3>u0OXeLQ|S~ek}hyh$iyX*9V$JKRfovjrl+QD7NjP z3*4T+&mieIkaqK5xBbm%0r05s({Sk@EB_B$ z>!=P;rvq&zI)A^=R2g`r%(D4c?g$OvK+o(EH*>i27Y+!4)OGZJRhfYyG$2#!oqfE3 zm-`tW%cu8hw(anpTVt7>KfhAk;^EJpfsg$DW2A4P0j&cF#cc*{r~BbtOTpkFMOl6v zNNK|Cyf^zRvmMp)M8o?pj09Hfm5>^n+sW^vox)O1t$85M5_TrWsOr28G)A)>`Po=XNtfn-mUr)Fk zh5~CliL3V+4C0LCD8tW78TbncKf_SMFg2ek^nLHqtpr|TK)v}FR}2lW@#16TfwfsW zy0Ni3Y$E7qeb|32lPG}SjH?aie~(q~6@cHc@(%caKS~P)9%*+uO+7&*Nc0hc1E4)) zgZ1x6Lbw3xHw5n%{Ksa45CPPGKV0N;6f3qSfsjr;R z=%Op;%Ai7&1fZgzX8$fI0 zcK)!o?sR%`3{kwGk)U#<6JPouT8TxHEKc}wmTlA267J>*Az1{X628Qe`zH57{x8D0 z7ztx`)WMI0=x)><|CcM*{0_dug9K8RwVXc(GGm<{-dp=DeQC;(@4VT695@s-;QQ)o zf0IOeNDC7l@Ow<5Sh`d0J2%iqjmz{;LiC3c>w%ajqa5UcSun6K6_>nBY36o^P1JeQ z7+!)bmm`)*_+9$RsRhX(WqAxa9?tbS#T5^<)N>~S_6bMbbvLp<^~vxPJlzYm9n^m9m()H9pbtz+5o~u z#JtC5vdfL<;a`jwb{-S6$9U%T95>rSWJ za?(!+N;7D=-av*hb#$SeNtgX}r6_krmzpM-Ogiq{*?7K_i8dwm8%UuHHEPZ3<`V}1TW<>v!0j2KKfP>y zZ00Q98af$ZB7Lc0nl2DM&NEwU*n!d+h>9G_)Z&`D_WEZ6rD@0(HvF9iJyv*xJJQ&f6w{ z!2<)4Jdalw>9HyvxQ$0DKP4QAefmDyZ`UEZ_iD_hD!bgj)EA zH)_-x>jFr5#v{d2h@1hgOue^kR$pcBNC4K0!PY^l5g@5nEt((q0Jh>DCeaZ~b{+lD z1}NssrZp@AAbJw$=*>KYgcpRc@Zc1WgFcy^(-ebvkAwPKPt^c)CG`q?oVjb#1clr-=KSrKVSE7vzuwdc_%r*1XR=dfMkXS0gRjeKl2T} z*cX79oMi1A3{hE6hXj`t5?BGvGr+25E@zp&r4CRPVW!{~es9bJ>yr3bkUyf7Q&hJa z;eeH zE&$2`86pCsI6zd~er?h@gX2H5)lmlE2!zoux?lCx0wp=$l9YARanH5*@hKpfV)d5M zB z%JmCVcccW7dUKB&6`6;B{$g#dXDgAPRP1<|GymNd{FHF5wn zsDuvncD(1G0N4P*KK~sFx?*hO)aIm+{-g|RV(FrWwMvXwIpU6RmKY}9gF1sNK*lGs z@y!2~?ja#h;hMIK0*!o%%VAxiSXDxE+e4u0oygN{`!+?j;eO09=TY zI%dBus_j7~DsBV+*doI`&Th@Tb?Cbhp4}p>XZgd7(O!}F0H1eXZ2EmwC%!JmHjl_}YL+&&$G&t32y(}TdyYLQ;M^i@faC*#R8GU8tVBC$)QX?c#o0x*!}HcB;U!$rjl7PHduj5z^$bhxCZ_~u197|7S`utl93T8R@PbF?Ickb zeJBu<%tTD*M_SqH?2Pjr%_amI-kChHDX&BZD{|K=c zTw3jtKBtBV|Ksp;XlrTTIjnzO(^#MwYGKb>0)#s#;FKHoxYa_u`k&A;J~E&}(<*4QQazfuau_=effa>Q!A?a3BzEg$bo z{Z5CKYW5X-dc_x7ELs`vXZe_;7=f4@S(j!6o4KZhW;~^Q5y_z9=fX8fWA~x`WCUcW zAk=gXbpTo3T|~A83Q>~ImRV}3da|tvI#y?lWrr~;hZEdZo(}W&uGttri0wKw{JA>u z|Ex|Z;YbDJJGeIs!7s$*wB;u2?2OwMJypMDxYd6ZJo!lxN)_YnGOca`-;v_ZKscRs zy|r2i5X;@=s%pIedraKOC7=Cn3RPdCMN?9Pv$+Hir9q?wepnM*!rjtN=Ki9N3HdXK zL5NkZ#h)n7GR%YR%<-V&SUw|W{&<_a?TTn&%14Rf(ZM2&(co+MfM`^a{&I>sU*U^h zV48mjK`g~j@D~YxI<3@e2|{7A4vdShUX-8H6>@bb-Zuj7fz^h*uzrF4kn>S~#sEe$ z{ypSnJ1@I?EKAHRj9J6pMl!us*YQ_9X4gtADy1E7h7xr`h5OitonjfJGtw{(1wXrg~z?I$v2=z1$UQmk8yx1S%dVTv-=McLrM~1I|q7TWkbbuIL zcvMgbI)NXdqEUymK=}RIMyn3)OPFWEUp$X6ROLXAa@Qj9Vr9Z{IqW|f7bEI8V>Qh4 zxto_tUc@uII_&g_xW1EnclkDDL6j0rUlr+f9uGN&x$?(L-tA^F#F3;z^FL1 zb3N>|HhB%79>l_XW3YaXhKH6>J$|P{@ZGplXX^cW@wG%YX~wbHwgf*YnD$k}w{n#5 zvcTQ2T{>dH<=i%^R{or)a`_EVJD#1J-ePeTR5_mhx`eeq;<`joVTTy9zdu_}m?X)BMm1oWjW`Z~A1tU(nfb&KX7v6s8fZ0iG7KS%8z? zamL@R9F@8fg6c+Zf9ZX9lxOB$mNkGS%SP+bk7H0}*csx8sFW?I{$7>5qj^e2lR<{O zEu&CL{qu*bcX}f5sx9*C!t08eWr0G$@A40UEQ4DHRau!Ptgj^c+;-1j#GfUf@a1Mi zEGnJb$n^#_d~#FndZQxd*;oO=WOwqZ;2*`ckSK7E=uHJ2pNk~p1iw@X|48N*U z63vRW?tczjVK8qCt2@1Qqi|96I^~UMd=w1f^zqk7&(|s`5ODzIA8}zV69Xxm@P5xZ zZz&CV3X&YBGAgO0f78yPJ!HwU4&0@d(hLzLC5lV zQ1N<@a_$Q>GVDF)1dxc;sfm6|jVpKoLxo^8OhU?a#MhI!gK9ndoNbFyX?l?sp3@&O zEL|a6Y}TJlI|h-WpFkhGSE3rTHxb=ZVWuu)eWJ?psC}hp_C?vTBQ>omBL7twh5mCg zRe3sx;bBQN5mox0*7_A+Xg#rw_;C7b$V&XJMRM*L#v4pHus3U47kh^_ zgBPi?nKT+c_^0$bC;`)HlI^$OQ_hOWz;$&e2mMYIN*Pm>YtigBOr)b}A$Xkf&Al91 z^iu8z(g_AiuU?@cdK!zO#GdOcOnX~txnl+AMO&}xiHhBI8das^GHKt$ ziWTMnMmSDKw~PYX#*`{Snl60-?>f#27c&x_=R1g?x(Gxof@&Z?rF32+-_qAJ`jdlT zrUQ(tRc*YjbK^1=cPW79O&+R-IDESKaQDG|| zu6hL{B0raSy-1a-iv0sh5S_Yd@EcbxgOymj;5HqORzk-Tcv_3a}Zo7rC0T@&B)P>U}ga@OKXCqH?^yiR%F z!yyw~4M?gRs%w9$QF=d-XDJJY6s5RDx?#8$!*1K98BzxbU|SRdSOMQ!Sr36l*Bstp zta+lPGJ-O#qd~ePqPD4m@&HiR;YaZ7Je-cdqq5!s9C{WZ(OmkYmVER-TvgyIs# zP$?q5L16)WJS4eAd=Pm%R06}Cl zbFf@+L87GrNr5FqHoS5wsOuY!Uc$0xc|b3I(NF>L@Y_EVyqrS;o1f&i55jdE9 zO{~R`Jk~U%077Si7!-)Y&VBtm+TeT$$N~vlZO%v=;tXNa-A^Zqjb2$@F1nxto>`sL za&;VjF? z-D4K)Z2`)5_!9Ag!Mwpf6^e3Q6e(#Ol>z~Y`D5ijc%^t{AW*uy=G4<{P{kC4DQQP@ z%5HVmkt`5l!Hv4E2gV#w(3F8atc}(GagNe4tElP2x0KA|ab@M*ijN5!8T`N=N_=F5 z$IB-~oj1J0DTq|Sh>1<_&ngHX{TxCr%to7ZlqnH-AgI*EzH`!Thp9Ch9(JIt6fwFO za{iLTGLK4Mht}ds>hm|N#}lTwD8n1+a?GfIJ8wV-rCwPn0w1eA#{mK;vf|)e8d{n zQLvalMG;eFz zQ|Y*XsqOx4xZMXc6zUGr>-y#Om{=XRR1&4mTX%*=l+$d8-sNsyZw7}Y3Vv^0(?A;9 zLGUsy^kYjR_VNre0K6RZvN1BvD*TO|Ko>3!QewZs z(ObYDA8VhzFFZ?2dZOcuIsox6CAzUaogOH?2vOEMPiLRV8}6@}n?B6jvHh`~|C`j5 z25u+&^Y#A__=QR##~9#zpS%tD2%-WcmLLB1e|gfd<Vx+=<6w z^UTzKo&BHk4oF@0e-pZZB6B7@XXU7^2at|-gaCDZcvG|RgmP^wqKrD=m}oX(idg!G z$<5q_$rK9@nTN1^uDAQOjV=6KPZoyR0?{_IubGNr(c@zEC#t1;`W|NkBl8H~X_s!& z<+6`p6X2)P2ed!3m?9^yHR~q?u-?z-wz7TAl$Y@f7B8l459bOwJxV!$J7aIvv;nf< z8r4SuU4SUKn%K8$@P0D6bS5FS=!3T&PF4SFE%-54a7};z{93LRE)b|Pcjmts{->5C zBJ&wK5Wg~KCWhnrcpH`Y6^|j@8~I(MgkMphQ=umtKufWL#~Je$=OFlxD)gV}AE86K zRN(#`4+Ne5&+Jk;Cio8;ot8>ZPq7?p_2ADn{O5EdaO%MReawZrful_8EWAMc_rkQb z^;kUFddSDTctB0)mI-eB4(Kvf%F`3}|8+l?^Z~&?zy=6D4$IDm zYd{kV2|%DVN7$@DY=(gLqvHzLws~3(#OV0e!q{L0|3+fG__jS*O-oZoQMpry(`}HY zmk11y{3rm8Ws?f)1>{d10P{Sy*;s+@FvOt>X|U?2>OFj}Ji7!4K}3L9)OtmHdvwrv zIIYbn4N<5%fqt9>feSYy!BT+0X9iMCX7{tLUsbis-n>nKLB4wAe5vdJuoIUp(FaT% z0GZp>eb*N(7!{U)0tE(~Bj|vk-jaJO&99>anMo^gZO$YW719}C zk-FPVR&)&*4^?@P+xq~g^NT=qQW7euc)7$wp#Q{z2V27^Mj7HT*VU{^;F9G(SKWEj#C6<{`z(_KJAo3I2>k9E0#a1?rxbo!Oz!v*pttvh#i)nSO6u71rWR~ds&eBuXn8{U7{FiGt|rV;d-+ZkWmPL28a}e6i9{!)W@@l~**~Dt=f#8u$GBYAy#TpQr8u0q}Cv$Ir4*dRY zxL~G3JZtVu#EGbDXNe#c6i|stvh8@zQuTJ_Gi(RYzJebIh+Y95z#&-FLw$_r1NymU zk`Dn9E}ScX_pr%9&9c1P{8YiJ6V;imSRP(>jHmAFJmA_4r8>WjSR1N=h>QyWTQXOQ zwCu1Z-Uvt>kY2e3JwH?grS53$4`UwI4fWRyg(~EA`ULuDf`gJA@+}1gri(+~AM`qg ztf&dNj<1CR&tC{jr|`WEl!bGh6BEU|jVk@-z822a(R{f?=m!D}z54`456Ahx*lS3m z6&_mla`x}>EvK3}V`ODyB*Oj5iGOK}@%TWB0l!06R3>~inJa(;*ff{BXyZH41^Jfr zY%?nX@eGH4>`rxI;U8HQC~*FfIA?+0ty}sgK&CpsUf4!oxUwnlehI6=z*P=xpxML# zCmR#!T_)jL4%bzC7@Q;=4tql_W{SLWTll9M%voE9;c!+QStaG8x0=c^$*ohDN^C4t z3e*JpQu2Bt0WaGLxNeeCJ%6fHSryQm(gf50$cO?zZ&Eg->z zlvt8B9D8vVC8%$PLS>lRS{IaDrP~!_%wlvG}`R{t+8L=)_Tc62z3P6uFk;@CxVVUUh5skp(2IpQ0#4XZgZF^mg|fc-$G7tUQfcpAbPX*+ zl#i{8@76mdEjK&1K+{jx=T}~rBiWiBy}Hd4x-CAJjmzC_ff&K%RHpvi3tP40c}*X= z@{8%Y*fyl6FyNGm{W;8Z)dP*{(~Y%<;26#363%%5#uM$-n4xlz#K&bAVXcJ2}5ONdKl!E}{pbP@VIRvST^lZuTukjE8^zK4*tO4|9JF zc8`2a^-e+I#cgu#4&CD+)M+d{C!EFId~d5R9Gf5bl6)8J@vn3Xp%6%@HU{nddg^fI zdUS`b!KLG7mZ9v)Ec90`OLb&YvR2Dpm&MABN5qTzjCCu%SMTG1gUx}lgn?2$UboGA zwc>Y6-{E~SW2l}d8{rX zFY1$9fh<7)XEBj(rrYzx@m9e-^i5Xnd96)<*}~!)4J!_pWiJ94c%0cCV}xX#Ty8YM^3CChP(*_U+QyKA>M8Q-u9Q2HIQZ z+`hYbu4v}n^HZEF+9cUYr73@PtkW^B-{n{j-1-`}1*Wlr^JIrXC$aJYw!hoLKgw5* zYR;%o>FWMd3t)4ge(-*=a+ZNPoFze=q9Ll+*~*qb9^s|uQyG3lc}v{7=9o-tp_IbH z?Wz&DOeakDfNG%qM>4b(0i3%YC&2$w7(lil_pr|7(Y3&zC#qafRy#eAV z;`_@y5p;g`r8L~dEUcr^I(1?WbEQ-#9IscFG#-|=V|Sv4~{$UgF8SS;LtR9G4N5L$5+QxsQ@-Cb`xnvj>hN+SFI5m45^orm!02(8qfIU!V}hA$4} zKpku2`F!9@3PD-#Mx#{x;J5rBT0!()iFMgh0uW{I$Z@vVIpz$;EeQ84UCf0@cc+gXCd!~eJ1p|7S0KU__wZhOhtnabj{JZ6&jFVyN)VC#$NkbJ zGY$9kjbv)BY-rsHp=>kv#?l2TOadz)!#l=YfRn)0c9i#g|27?U5w|YiaC8#z*wpE| zAXvM*5pl)vLV6Ry2N3rWD3!K)5|s`fZ^3Z}**YWFSI7dX0P%Y`sZJb9Y$4JWqIRWmvs0jc+UCSXVox0r&MIQt0n=w&Q# zB&3$;17{l9vI2RG!Iwo~OM~564E{M7u2&YLV&1yZ$fUSA_eK81nvU(>+_XeB{3+0U zS(S3fGjX}-njI=WCM~d>&B(MkdoZYB!e#laVHkVTme`j6l((%~YCv_~)+_h0&UmZi zqguNQ{RiA^XTrt8egr6M=fyp0zpzt}!9$+b)O=dTTN{}AMf)YiI6R=lv?)I2+ zq|YXIicHXi<@~=i3!q07u*bH|8LjA859Y}MI-oz|yJ4U=nR|=L$7+PL#~aMF322WV zO<#F{#JnPU-W%d%>n-7ew0ah0h|~SfvFh-DerzUx*PZz8{T!Dy^J%y_bb}*{7^)Ax zELY{+b$pNRcK(JQ35lc(7mfAmRSGSs^&Uho_2x2*)3O&|*@Sfb&vRwZdG)t$_B75- z_csldR8C!Q{ZO6^5l+gu9(M`L9M6kU7Gc-0&XYBfB7g}5aW=#o^+w=tx&P9Ru=rxF zoMh57PWEb}PsMR61NC)TI~6}tk_D@ih=q%;_8)%pet`b!w?fKg=B9xSg>=COtbLU; z5_x3@8=FI<6sE#>lG)V#&XmkaOFsyZ?(8;b)e zjdsm8JP`b?vkN7>la=V#{qQ>8!U#8H8rqcMx8_XX$Xwr>oz;hIhPGuVQ(GH_6B7I6 z<~R%~E{c4N78Wd+&&x?4cmbko9r0(#C!}g^oKrc}D)W9QCjUvkLOHtLQ1NdoxX>jq zTMg!ojY~-1rZR=n408&sPhU=<&<*k##OrQjx)s=ls^A>8jLJEi7EoNxNyl}&5DYQ0 zIpG3=4zIyXqWfB`oR*T5o6f=>hU?ox@QEM1GTcy@e&^09&a>@6&T#4R-;9Hkb0H}&V;eGU8AEy+6B2ln z1ar3Y1tr%w)1NqVKL^h2f%Gd=GqUZJuDy5*1|9i(fn z+>q8wlu}nio@KTR^tB~OOQBavg>udEkf&^0*DUvfl!AQ6rZHf88LDv<`SB>gE33SO zm9|a?Kw&mD>hTZj!HKH9-9bFf*FG&3w-YLLF;!L@R|%$^VPJw+ySQTmMD@iABkxOS&5*rCU-uL>ejS5CQ3K zmF{iDDGXb20k)gJ;R|eE0Z|#!X)ow2QRP`ly-X+QZ_~0}=L` z_=mD=Q;a-VQkzcL8j*P3$i8w4i9TlRAooDUGzKEOcAym27kIG!6p5THPZiJ%j7TQJ zFtYMpTpwW$c*-aw`Ti74Qmn6^0qg~mrWD+zZ)i?&yl+mZ9A&2@+M)@*s{6JyP+A6} znyLEr;U!!8wY~M$QQ+7y*0W|JAwO@`imfGIn%v00D_DQwa+_*7{r%BbyD2&Eqh?Sy zJzUNbGEru)kzD|sooeuQJunxFr&4)8=c<#|wTl{C<#LhHvCAig^YNLsspXB|y2W1^ zDcP(#Nfyo$e^!-a$YBGI&}P;>PkRr53kZpm6(ro9v<#>c6XneKs4h6YE?qsW#CBNL zXzMXmd)m{L#+)H>E;cI;qbard{5C%dq5Sds*B8Fu;fomO)kE^+W4clkMB}|@y+_YD z*+4(&sUc(RMmu$ryOEMwRas?qte`t)o(4syz!aCG!aE*@%tiSms=1O3omItHOHE51 z1HE+J6+Kq^QoNq4LM6EI!N^Ib*io8`&u5{mSGvb}57wexj8C5AZh3#%VM=o6W&7E) zbGP6axj1vz9wuXHdge=Az36$m=s8@_OI|s#@yCN^9uz}lv%mcY`F05Z%mSt7LZFND zqRVnuDDM{`y12KlvyUe_tJIK5L$GHMir@iyjvfvBi{VvT+K;%YEPCs|o}qa6FK-*lXWsS6pmk^RcQ7tpHe^F}pOs7CDx3>DMwV!wM?NX%CJ=$WrM z1>`w{!FGql=T2-&tA`EyaS@kqS;1^v=~o58{ga}D#I~sV0l8P&j|!{9R#256s}9xh zog486%GPa?ee$tfdztq5@urW)iw?DNni{sr^Yd`or|Tg_NW*Yv9Lz4Zr{3$u6HX>t?z#g^pLB`&Ij zvntkWmh(TlpJ&bWJw2j%9GDj$w4;4`FxTfT%&%6PowgQjv3Q)iAK*!gH^E?pM4p8s zeOFo~x!@^jm4&stYH!^APVc%2Y2mBPCttkY6qOTIbzjtP`Mh5<__S*>1cbffwN8dt zmb1;8>b>u6vu00b`>2AifquEoE!Ct~30HoX_pK_++=obctyCs((lbQ5!t*w@hhhmz zRzO#xY#U85Fh=CXFEZl{^&es7M2(NR=THTooOY?wNbq$D+6uxxz4Ev6XN<}HK{HD& z_?0RyNAe|p5`TlMqsVM&e9ycBFO07hapVVw*m%Y!qu4lk%4mPAT@;*CwU=wYcYyc6 z2e)+pCod}D>IFV$@nEQFC_I>Z`332-`eJ2`p@3XGZI@Y>x`F+=<_ig|{)z(fgTB54 zxzfAbGR{2+{ZsO9W!uI^qfni!$bdbaAF@OR8)z!pW)|EN99Oo;&>k%8+I254wD@>O z0IAsf*F5k59<$03P+dt(CK=%xEkSleYHT}a#+Gpx6QlUB@ z$dB%mZ3G9GXflLS-^T$wZ5^8s?$}#APJC(}msy?zzi?&o=Dx{@6>IQ}s8~>hsD`9k z#C(lQWv$4 zmT5sEt#*j$aCBxEc>OhQsbJ#vIoabj?}*@jG+GML4}RVPMfTb^$Lb8E0kLC4EW5^c z*b{$~f^z>vVMa&=dZr7|mlX4MSHt<*Qd~0QSdoFmQb8(UudgDhD6J@aouLCr6ZNc; zrZNNA-p+e6o0c}qv-U5o`%c+do;OP2jWl1d^hS{W8`cN!Zx^s27&GPpPYpjfQ`MX``fBa3xZzt)6< zW-MWre^EG?CXC*2-pII3`jJW-X~R}M?mlnny$q3;*S;DHBhdX-8qgiqi~tISx(E56 z1>L`NToiDKs52yMAf4|TlioJ3dDeXNY@{_yXrxeGzsz0DHx z9CHR`a5&^=6u3~F<^kXfWa(URI!s@_cmU_5ZzQ^Wbr*f`Al+>eD)TB+gRM<*aN2zI(+YrX6tQ7+B%6%ZcUaznkxtkeQn#{ z>;!=TkgO_1vLZ8tJmRq|FLgj^bdlS4)1h-N2N-&6)@M^ah9sodHZ52EaRZh^h!Uz^6YO z0SFsj`Q|IDt_beg7O0!eMoTqpUAs-n3TYz@fIUyUeO>?f{$Z2!wP9>p3R#yN@@C#? zhMbbj0N%V$&dRyY`FK?0uH`BWIQj}R`F;fa>M$TAmH5JIKUrN3emF#q*oZIF?`&gn z(&8C5Wa4U?Z8Z%9$oTE3<$9C=`>Gz*ROWB8QCp*XI0dldb{#z;}XqqfV7%e4yEIH8Dx3bYxU%@tQ?IlQ2PK?U>3n6>(AGm`x|IHg$=SmK7&ZJ z);1f5z_k!P1R;bS(zQCUlQtJL5zMcUj|?Ac2Yj18*-~4C|9X4+6N!MNu3mci&su*^Vl+-5WabA8%faTU?&D5*qr=nVP%)QSj zfKL-~(dR&CYcNH}EPqu;KLFCx@P2`5_AvfFKA;Ef8Vt3BIZjF(l}yR6%Zy=$?tz9#J?o zAH9AR3W>gafOFK=f$tC+YG2W#;JsDINHE&U_Jksd&VASREsq~otYrR3>|kUrgHAJK zdKSo9XhmiQc|vMsBvHo(2A7q40VmgXGP{bQ5<9#D{2utYI<{PH?Tl;hShAWEY;Dha zUSQ61xw`LmZ=oW#3HoP99H7tt6B0Fq$zl^fazSM+Ev-HURBX!Cbh|2|6c3MUBQe(- zX(oIKA{3wWG!zmra`(xxsw=$eYx!Wo3y~p9Qj7ghWqIw~vXwCG8y3UNijs7r&X0hW zS1b(4{V5GHs8sY$>dV>LYXy}b?t_WYu2&0Dny{NO!(Njl5uOXL z|27Ro)Dy$0eC%o{C>GNX$v@MnF(^5NQ+SXNBoW0BC-`Ry(qLMAEtL06z;-^NbjpT} z+1p5Sx1We13ybMDI?aFQN z8Ble6;~FOA`tGz}0CjTDACD%eKX`>zHOM57&54wv52= z8=KdX3e`op<+_LL$+*&6x;DHjL{{JRWPY&R7$}-}TprGiR7Ts;$EwP9-rS@mpvh#j zi}~Ch(EaMek-_`sNvEUmuE+JAL=S#?{UGW7gwQ)V(%@0bn9oMCyyxs|`ATEe>?>qC z_cWcjx0*68O^F#*dd{XGHg)U|xB`0Gy5MQomK4{CboH|$LY{&nOx0y?30X;N1=~e6 z(=)d1r>!y^kaX@$$&Ymc-^is}+dYdnYJ@$2l|E74DBIFJs08?a3sz3;z_NDq_TT8q4a{ch@n1C>gDZsih*LMXg z%3Fd-xKNG1AyrI{Oc7?cjHiscv>?aUkJw8w2s4<#GE_B{<(5m8F1<_K2s-IpkOHFW z8L1O^Q%J@ax_=D|CO*br*o!#4YlT&=%Ao17_mH(oNWR<&W(8}V6fAd#MD(5%t`}2L z^%u=&*G;!lr$7b8dyeJqb*}9?Xc<;cnk_E7+?A8DH2T|iG-F{)vR(9z2==#()dHoE zC{!}3K9gTRQclFLsLJiuiRxBjm`HdUIAUuaH!K$Sgk@}MJH(rKSos&*-ShsAXY>SU zuxQZF@1dYc^z%vAYIC$-gkXnYA)`We0uF@QnG3Pxh&Pg$k%ixv;MzEI-p@w#EtAbl zFB3+#0et4rpY6N+wWx`-mzxug znn`tCt)dQziCD9fZf@ZVj@DCMETFkmq(fqrH@c3-br$~4H~RXWjzm^J3zpx~EQ0A! z%5CR`Q$`84YtM0iQFz1i1zT{rB-{JZNgHwV;M#`5)M@#zY>Q8*cMgwIhT2K*GV`~Q z`@tU8T3m1!V@ZVIh4Anpg%Q{-W{fX6r4|AcCq$cNKuOTAILuVOpA|GS!{0yJ}e!ln=Oo%jaSN)uqiDiOC2; ze+bW1E_;8+ZTs+m7+>?t>lJw})c@8IWxFxM#LCpLzZ$+n#pl>p#>d?Bj&h+GgLs+o z1k&KoK3*?QaEj5Z1^hf@N(nz;W5<8EzIP#9{xgk}f2uX2?p&T(+EPnn8W<7SS&Va2 z%?1cbh*q+Y-&R+ZNm45zqrsi6fvv&}ZQQSDwU?fjYpok=zbeYDvYE-Rzm0~6uwp(4~`E1$( z4{7-dS6()eISgEDO6N+B0cuCNhdgS7#Ec|wF^mv4(}9;=xI=TQRbjBHgu*-ho z?uoPxmV{_g%)STAm!v6-s#&d-Y65TUj?i%b-|RuLbOT$xN<{TvH@%PqwNt9qbZc%MH_IEyl}UufHW z-KRwd_&JsJq^&Rdsas0vm5r8GESBAcOeD@&FMy3Jj{E&b#V&Py%UPuCZ>OU0QKgFU z%z+=Hy%l~^YPhK>01PU(dmCJq9|dzyG+!)Wx5@$v;HLmKi$?f(VTtOHWrR8%xs|$N zd{^~n-MMdXb#@OxgTSJ+)!S28>%JZ%Fw$p5rcN>O^GfA5llJ+pmhGc&=8Y7kvpjefKiFgu~(5D$x#_=$$+fH_5L6sbnTxaN$i_mbkQ#sXZ2(P7Dr`~ zOm4Ia@0burWxX;lezUq%N6UL{%R>2pgvPVNU9t~&x_ihBtokq#*lno%pHz*I3{CEe z%$vr!p*VtyH(8DKY_j-k2x2_>V>xsG^)P<04Q#ljcGjbJ#65oV8B)n6!LtQuDL#vV z%xyK)<*H?LutprgGIsK?RJvK26QI7UI4@ptII59GYeg5`bc^?4pEUTPAsRm!ibuTi4*t=T1>zk(KiMC0Z zJpM4eVT89KXnfMd(xJ8TgU_8H#%`PO)9tYmyvPJTLwR+}H) zYN^uvSYur5cZ-PqDx!DM>qzRD#D+6`+XBjS!0t!L1E-)o?6I4V==aP+b<2cC!y56p zK<;he3KX%3Gr^=AVv?!6N-E*!BmD4B1PAQBl>psD`$v8KN3BySlRCR^VHZGqUFzEj zh^aTHc2i4AH4a+?x{y&tv7-W?c*zPHJzSShpc)C?CBgFI*5xfMAszYND=DWy@ghJ6A^ysNK>>n+!guRE)7*MaHLD%pql zjj|xTaavuqC6>=fAqokp7?{I<{Qeg0boa-s843It7)EOuu7RyOoXxHjVbNA?=GFo1 zJ7nXVVd)q#B_KGz5e^RqVE{)jTy)lZpmpr$bPB6a>n|{4!rk+1i$8#U`2I<$Kpw7R z{YWW&9uk9yvC4N;Sse@x5lSo?e2c-s(hkMNdl&hiublhb!)c%A(XIe0eB|teCL!ia z8dI7{8muM#J2^2_2~~p0T#w46BciAp$6S20*PJQ7y?R?Onuq_ZPx<+$H|wmOu9il_ zr5&fA?uz5R^Z?sOHl1IEhiBLLufR$S6sSGcC7r$&&vhH)1{)CPfawy(gk%z8@UL+@ zZ3Q^NL-Hg7ZNoi2*2c6!m<0)SG6n&vnZo7iuzTPy1lV1 zTgjs1+r-^Z!%=Dm*_&akB!20(eFNf&+jpmgVRgd=L%pl}*BeBjs%W>Gkv;wxUA2_m zsD_OKQ-C?xxSK693eK;`Nf${KPzOo|U_=1*?aKjHm_tJ(m%)=9(x(?RC?wjt?oGG~ zi;@BHy9VvKO(6y>pO{~gecqv?!D*HIZY*bsW8^PH6hv08L?SAv^Ge;e?cE}8ulBa# z!)N_}BKyD{P#t%n>|?#VglZnru>}@s{7&yM+kp|-CBNshpR!E7)5(V_fW47v#tr*< zyP6b>a;M;0H#Aq$8pWGp5^HyBr|Ejt^sioepa;1El^7*SZNBT<#>JV}EO=Kxe-Vroei(^&kIm zcf-Lm5ytxd{U3m`|30oi?$`g7#}(Mh?1Oz>No+%YCi;F1r z`3_e#byIJ=$JH8#W32Dl&q6n=FKO9cn_jCbYT@6@-!`6WU!I*e#|I$OQzN1)NFw1q z4?q@y9GB|9s0| zpIGNZ@~W|-?=3OqjsGe3_Rk;s_kW>~%2lW!W!WSPV4igEur#Cj>!aXY{`!-I1u2Vh zyhWtDy8w?yEt%hsSWV+hds+3F+w!+FaGNtQ(MSF2Z(MzXcJDgEnO z|M%y?PjTlXe~;w1S38f9l_Ip44{BUX{GSu}>s(AbVDvPMs#EFjIsP9lWsz#%#&DXmIFf`NETf& zk!z{^cDmC8E9ig!rCdYgFxqkT5x#L1i-E<`4?+FT>PNR6n79Fpz%y=Rq$C`qZq;qC z9Uwyk;m=R2Qz8L+zB~ICMA`nBu%c2fRk!=OT{r#bBGX;}d(8fEs=^79#ucP- znykO?2#-B7+@TA2yA!G>4ib<&n#5R${Nq3W{xg>iwontehHBmqHXLCe?qLYD$Ials z$v_YsR+v%Qu1Yyo{@ZAg+=X+mJMY%N=1;nQo1n*;OOH%ouQaSqr1kexqo)p>qscWL z5pi%({fAeD866QNBc3m@^B=}t?h_dET4`&k(SFdJKl)10+4`1pX9uTDZ-%I@OQ~b|4{r^`3b(K@`4o5LE86!8C8o9wD z$Za=X!Tt1TE9{xuU37PmK(LPq2Z6;M0oAp; z_OGi^QUjOqQ`H9L@6$9myeeclj%_m?*7&CiQk7oin&kX2Bpn%cxheeUx38A6+>6wa;&r+rrPlI59iI9LSvrY=L?{66AmF19EFZ8Bz>yvjY!N zRf@GQvG~7vL;wNCgUnmQZh8YIL%zP2owY1s)rz+*kSUW_W45Wh`S6bM-4`H6@b9z( zsf*bd;Lhn!K1@)BB=e#RCyV+nfG83M3g7xXEf;@hobdLvY7@Gb z^vZ`JhK#?QaUN2z-(o0GEPk3xc!(EB)mIE02?roL>Zv0P?K{T$C_qv+vpmnoY!= zxxf~fF}tZYFESKVba(C0*-4=URQ)5!?e+p(ngeqR#giGH7;?d?JYNx)k~>a)SWbpd z4O0)s$<3NMaL@}@ce$co*Y?5Su^0kd54c<|O;LDLLtqD>bNg{meI^Qi1NV0T5R|bP z6Y&~j&E(1KV{%sp87$=;i_j$3qQ4DRTw3)&6aSjVah(dA{ z-6~}dH>CQGE&v%OL!QVM2MhVGz{PXSpGAq^J&+}5H~8)1duiUU&4xRTQWInW9}N@1?X?0$D31Or z(05%z(Ms(GG1|va=4IYfcwqEqijdSE@AY{YZ=C?Ch|u1-v&9>EGci`+E3@d?dcn|A z0gT;Iwt<4CaN%ZepWYZ<<6@C*sHAl&?tSGG;1|q=04FCH?u9{yP+Bu z4>nQjf!{2O;ehkyOQuO`P57iB1+S*)9SKV*Szq9VvGhUE(2{27<}u4!cR7~X8-uo} zZ9sEI-}Rno%6Bd|)rw8ryjFn*zMxh&Xbq#zL#FH$4ox$vro96c#sxxwk}t$TkI=h#O`h36E}Q8a5~(Yg<<5sD4KI1|~Tdo!0QnkEi`kRyZ%&E$&G5#j|J? zV?~q&zedVOj9hu>n{d+#{_I4%i=dN?-7edYNz&oXO^y0HghRyx!^NTbs!@W$pCD#f z7!d6 zQbv^POY_rCt_~=x+wQoqrw*$apZZ>KTw9hBGbCA1$U`uenoRH10kv@_kz~Wdwzv2S zO7O)|{=9xOJUIA(`{XIaF44G4L1?`RxvZX1wB^FtZY?cW??Sw(=W>{J2pVs+fBB^y zUA7cOCgyOa%>$~M$&9XQMeA6UE3SQmO?8Y-o1_Z}Un^rh>A)wr@B+iR9ZHFt{VV`z z2MuVP){tm4QzPPp**5B%&y>q{YgArLP``+I{|jGp)!JHG<6)6D?HoSB6fj_f`eO32 zlLaa>Y++K1kYpN$rO!>s@Rgjn1@OY^HUv0&oJuz}b2nc&_e;MICCqtL;}5@un*g_9 zx|b<5`u*m2&mET;Vccml?D)ROhG(e`(Q78WQlHRt6mYJ$A>?DATU1Q!_v15DLA2J( zl7&w1Z(tUVagj{LBNb#dWREa%TZ&|oC#EaLmuG(mvZC`J5LI^n>{da~BxZ>+I0C>V z4)QP65o+FQ*_O|Y>`xUSQeVH|I`VG+m#@LpZ85qLQjkCr%4@Eh>at{a_iXwIeB1P1 zlaVJaYZA#-^ zOktJuCkdDB53U++_xYWci657W51}42n}4lWs88dw)m%t#WyK_}?-1*KZWkfIr11EI ztatOto6|$K1$Nm%Xrg-SNJYr3YK{G`TA}KIpm|f zyZYnQ<1w@6Ydnss6)N*NLztYnT{~{>sa`+Kb|VD1M2DHa-Wq>9Kg5CSP@OLs@eUaI z=fBo7en?G(sY_fhxS!Bq{D|RbMT?7J;*Iw?p~IR1Ly=vcO^!#^@|kS4)TbUyff-)W zyUslJhhoF_S)N}^U(2g7XS2LEH`&jWtZ2QBR>OFohyUrl{xM3YT2Tuf?r|>Ti8WEpM_?@fb&`wBHD^B_j7R&KwGe>H0~PM(&AIQ5ySf> zLYEDqn6*RUgR@oDsb$tufdL_*QW_Msr#M!m6I`fG5qwv>z!bH|{Zq%n=WH(QVN!11 zPbxg#&eJr$z)j#~=C7vtOGg=9#iScyUW>2mnnfimNI!gXy+kID;(HX>>6YJL;Jc%7 zH^hZXBF|}gohBvSqnj0@7v2#;fAoSUADM;gTSpR#KYZd-fpJfE!Cdp{cD)EozhmrUH}b9=Dyvp)1O@i>j$s5ewaq*l!L^P>^Qm}5dw1CpTt7-40;{ZVkW0RV}y>XG)2K^$7lR2P9ZG)Ue zi1g6Sk!gf+kArb|B%O)d3E4%st<@E2QQ>umg=vLwbY~vABI;ND<6M-YD*mf|o-FS# zJF@oeKi8tx6dRCHfBo4{$$#G2-M-$*s3TIic!->wHM&N22;$uD_P%&X+V)sUggd~A zk*HU_BUK(l^_@a&A8N8JepQORCZ7F>%UIl^H)PG7pPMujD?~Je*~(`WJr+mx_o1Zj+^7WAb2U73gNP*A=!l$f%|AUAi<&>Ea z-12vrd7TcQfKD*;WT8(~a~j|V^+dlRuC&ml?9t$2&?^FTSlfVTonQL0WLEh!z5sTM zp-OfsYo!ecWN?UAP0R{;jWAB72()^5oYd8(Uw7}_E9m0DTb=&#eDBB&C+zgi$wLw` zv#G{T)%^a=3Celv$GkNR=BmOhNXQdRDkHC7RM?T#q$-|bbsuALy@#ufL>%6adXiA# z`aKCpPZYVNLP6gh3yHWUsk&6L#=rWk%^+^GQ9+xd$69DyBL@^(;;6ff0z?B0>}T-D zk&2yKHV(%jE;bEn5hgZ2U#W$-#hxL?NO%e+UE+FEX+t z#GAIWi$qB1vn~X+<&w0bEHZrPXA0|uT%W}+H`KaW-lIGASc7RjZVB5QhX}`qx8F6r z2z@{3?S)FH(Lf_&O++o91&`E3|x&q#s-WZ07xeV1fQ=-De$bu%^#Ubr%_-Li5lPx+K#7v4y0^lEB3fe>43+O6Lu z=Sg{s+`C0NxC?SRgj5?;(FnPy9r&;;Z#|kia=w3Un`9FQy?W)LY`833liaCgR%Y!B z;T_^~EaVNT`gl+VW&i!O;oRHzQ3^#}w#}T=DRNhhvLnQX6ucpUq~ob+*buiA_u{C$UHFxAftv*hDPk(4F zZ85!FPHQ5tkljiH!L7F&Bg}vwZ`Gp+6|`H5A5L%$KAyS7eGaPYk(kJq4r`Sr@MR zvk#xC?U+Us#Lf)diax}xk~1v$9DDozsO7;$C&T3TnOq>)J`tqv^wQHKljFa$af+X$oF z!N#y7pbCqVg>9U#!I_5s`7kv;>XR9lyHMauTetX84RbC{6z;3`QBlMYQDmj!6sA?o z8hvVpctS;USUUK6}yl&CAAimK+)*viL z29`HC ztY&}9N2g^FzDYCTsZ`HTBoX1!Ah17czs2x_mdc!xrUL$!;T?+AKydT|D!Kh&{d2`z zj6EHFc6&>PpzU_8xs!EIvXyZ{x?zI=JBaj_1w)&u6X+OJe%}c>ghdK|3s&sA9h#xc zFU6d}v;<2=@8B*=b`CfH?hry~D3h+(ealUyJ`wUdTQ*bXc&#l2f+t>D1u`JlNW4G9@m~1$Qk*| zWlne@sp4t+cTR1a(PN@9dx7F3|8a-uBth2n<5^_{jdGi?+MUW8{)B6_6a8QgcZS#1JV3sRYiycgUe1Z~)y zceZ}B>s<#0IxeR`L&jo@5Q&%yNIw?os7@6|r193aL-@S+lg^{0*c0z8X06;~PO|)a zlQhtgW{^Lq|8y;fYPnF`E*0`ix*@>xflEuHBt!?pQ6AZy;e;#yhKosrlayhsecO*P z(id>wh~A(koNn59YEx+UVCqmz{J^L(J&c=uw{-rO_N8k;x`5xS)^b0n z?KayOMAPn-^3%t{Jq{0X4n7=GWXd9d5_uZ9q)ARwSYorQ?MaUJ@!K-r&cWO`jKsp! z)4o5vf2}_Nfn{zm_gK^5COjgletN5{&&5y@GzNAAyl*(0V{}zwM?hxQzR`;d@KP5cIVXA@Z67D#VV2W zwhvxAHJFKY@D2HB()z(3e{S^f1>8@yjB{TLsT=2@P4d#*D0rHHb^_e5;a4%YKAk~! zQ;kYyr`iHFj5@}>>9TGAG2(3a12ueg>D>&nY?}^b*rggIwM&VdpHy2)*0*YgP``T( z2z%u;M_5#b(atQ7w0&Bl!o8_GoJGx*>DE;GJH4X5gqrfEZZv&PaV^yDWo6ni8p438~n4w$M^Kj1oC~V1DVSCMvr71&x&WRjk{*T6`cf4d1 zO2&|w;+WFkkxuC454_0;K+D!xi=jV0^Vgs2st~?7wKHz;=Sls$d;b6E2aWVLhYIH3 zC0Y+;&pv)!Cyi%=1_X_TPfIA)xYe#r_@;Z{no$S7(1}E}r1Dsp+gkOfk4Y8PGUKfq zB5fEe9^YF_9*zIAq|X%u0M1XVrYA;~-Os3~Hu(TfTtl(Llbwh zbXT2>uNN|d;DcCMd3W3q0s)xR8Mu~t$E^)g(}5+WO*7Lv>~gdj>s%M1!XV^ z*Cz#BhP+wgcSBE12Y+b_noJNGw-(!Fyg6n~7B++I@|^~~65=!+pftm4k32q?RTlu1nI^>;YI^oOm%Cl zfDsKNdSVa)66XMFHm)ce+`-c$6->rEOr{cf=hKs9N~WdATTECbIfNXocpf{YF?%9iBs%%u|HYyMV=0n>I?#;f`OcyHE>5qa zSR^vNx&hM0{TJC3MU@dKT>qy)#e=%7R$nsj1geP}xpWdP0#E7BuN-@KKLRKe@*u<> zDOR;p{x#~f+LvbbeY9*5(%NS30zpfKR~6)3wg{k{ig#)rbZfsohk?T#u`wXjIQx|4hB`o2s1$n-Ith@#U8EsntNX(P@A4T8_V817PQ|Z5l#V zrMeXohXs{+w~(~$GKRm((Irm;pfdjF8|ZsMXj7O?T2#%hkx71W~Ydlqs_(P*y7BsGW?97*t!lTQ3S&8&Hh9d<~;NT`UNjWe0+0 z2%Da=U;Y7kLqKQOu>EiM_1ys9C|z*N#R7d6Jwf>&27ZYV%7Aq-`by=3Tw?=}yv?`b z4r}L704eyS$&%OnLH6-OLH>Fl0Q`2{n**Q|*)fC$SloXDA_Py)LF*S*y_hHE4WO_# zpiw7T1-m-^qlRG#r~*(@S%5U(P`>>n(3G>jUhqfG?}4&>*M~ZHrBD1jFhf8&F9D7T zBw3K(p9K2-cBt#>Jmb=h^2Oxn;gzZ9Ioi}FipDyW)ask6=Jz8Z=@-(~D=OcgLt9lk zDJcL8=LVE=ziB9svn|E_Wks#ht$NX)gEE&#ua1|EcD?l`O@pz&Z?Ael^$&&I0#F5D zLNWu>3hhdX=N#Nzfso5;h?CDU`69sQ{Vf5A{Ii6)#Ru5?VnAu5mcBzsYg0guYH<~C z@!P6s)=*Cq$$`)wPP9Hsgiaj1*hzjznA9*4kRT=k*rDiNNhVr0WU>7Yd^2FpY1WED ze5>03HQ^P|G$SR`f$fzVfdD9-n*HV>e0QcxRc;QDAh4IaVmIH4f=Rk6TE`>%xAhH`Oay73MLMC2XqRjd3AwdhYvv3B#7y`un<*v*`Li$`ZJwD&)sLCn^ zI7Kh^+V|Ih(Lk2b?A~q5D=@r?;jh<2Eb(>uV9F38X30+>ISgtkw#}Y5E&MSkgbsJb z;5jN_T64n7rIWW_U#7b9^f_X}V9NG1C(wdhuDcT!L8s$#2z8|D->nM&8rU;jJDtYP z;yFjVkrrsy4u{hjB$)-0b?Humv&wT5lRC!SgVoLWzz zQLpX$FrgSD(mk`xvjuG;3=w6!x5dg--9Z)iBGr+OKA%SB+L-2YV zN#fOcjKijw3F!7b3!T6EbsTu`q&+&TanIW({7tN=S3=O#HZY`^{Ly*_g-dw=5&>d= zN5);wi^OUxv+AtZJBrhfd=J9h!QPM3dAZQS4>XKr5lc$3Y;z3eLD$1bALymqsKDde zduejX>a&EO6LGa}N71gT6doR86J!&THiHHT8A^9*XuSbdqyPmwNJHZ^-V}6>dPjwg z#vCB8hI&eA_8ts@#BB(ntC%o!>+jhIR_XSJ#(?j@y4h3Pa;R*b{6$~Mof;A${r0o- zbuPBJNCxDba0Ch(b~?Zn8SqP5x)b++orn)>CQkInC%@^_7s%w<1;L7$2SAL?>1*Bh zOX_gkN$-lGmOTZ(nN}*ToOJzD`?uhtuyc{UFN7BA~NA3+e~X47et%kELE$1OXnf0@6m&7f7`9 zyru;T4ecb${_OQKHI&#`sJU?MN_>$){AV7{7&~DXI@n<=FBJI+zMm|+KSen`DbN~* zD27hVz&U4cj?niY<>G8h94qF4*wI6z*Zmfp84~ih`Nv0)(X062-jBj_l!EcmS1^<0 zPAEr6H^{pjj4A_0kxQOwlFA%6P4nUSgLVVENXIdQ#NAULiub z^krh~P2{aQ#`E`x(jn9oz_Ve$X)XtH>HpT540W_`R0IhLavFOb3Ze^?w%6N=o6$hx z?r(a}PMmtMW#$OlNTUlO8LR;pL!IN-aZuxciGCFK92f)Kk~_BV_i8p9?z=)0*3*Bw z0|E$=jGPHOy%XG95Yezab#*=Z*fW8%<0a~#Xf#iB7e;;`^mgp>{ct|Q5u)1}{2AWh zS2MYkapI}py!VkL?}b`rh=@)B`pXAr$Qvs+m*fj{1*-FKM#>_^*)Mr2nn!x;?{U0b z0mV}o$ZmX0BJ%L+N>4<@7em^YJ5FRNf)3+JJN^x`OArC9h~?Fq`d0TV$on%NP&E;NqkoS29Vc)P8t~dPFfU$zVqh1JnP2QX5_c~QL zlK32qhu>3Hg3-phGs4ABtY<$p!+6GK0^ttoYI;yi!a4P0b1knQ^AQ~HD|;B{x-6Fu z_MS_QG2HtCxud+Rc>kkkA8uF_@wPn}k?}R&IcPVu;k1Fo&mdw!X{rANLNpqV7A7kWb(?w}e}ReV9A-+W zxTLDD(Ps4HeE!PMJ>@Lt>NMvBONEo-;^V><>VR~`&qoaAFZQL8l>2;^wn>AqEz6E2 zZu{FaUA*z%+tnr6`GshXt!r#3WB9T`-giIRq~iV0(OxH0ulh#^@+DfX#sSA6 zcaMSGx@f-j^|KM zu!hv@HHzg9C*REni*7mRh9@9JfYRvI%h1_{5YYq@6LYM9MI;^_)fJ{e*VZu82IS%u zq#3ZtW1@Y-Iy+rU?z4*KCc1Ayu&w7n(S|II+w2dSHJ^XzFD!^IKTR9(Db5)~+j%f% zj`MIJ!7#shFO=({9?qjXku2^r@kJ^c@(tuepIcE_GolPL;7t*92SCs(+Ki~Q)?ywa zdkguy0mqO)-@~BXcwgo{xWW>A$4V4y0n=_~G=5(A^!DI_>MdFlscO%J6ug?Jlk!** z_f*^;k?Ien^6WN+Pw0-jIzqh-^IhengVTpQ#ijI*}-E_3GP z@Y!vjX$hOEq~WU_-6NbI3=7^>*8J^<7X2U@u#ci!j!fw!K$8xa;Jw^+I6BGO8i;n#{E=Up zOgx8+Ouve9v?g(Nn(mAAgW8IONQ8uC;j{&l=8Oy=HTLUu%#+#$+Es@fg!UvI_*)Tg zZ+57adDOxCi1CM8&-Mj_QZuq{-axmD`xzEKUvc+y1tW$_f!hl`Wk9pBwrVL)E`91J z8TiptoF-HKrpnBtw0A~;7g&?f3`ksSb<7vBrfo9Rk59bFWtna$7gp%Ek7h^UP2D53 zI)A^?Ge8?E)PdHEZ@ydDlJ>3FeHFQ*hf)UqP}7fyLpYqdD=R&|S1VcTO@%(nVtPwU zLb&70phwmNi?zs&NeG;H?++L6M3KuEt6YT5;Fq{slN8_wv#K`*7@GC}!p#9sIPma;qNnLHY|DSCyX61$m!(GK<&=C*@MtU-e`;jHd0cY^AT-*DA^t-9A(61J zF)saaE^V)3LFgCSGg&2zZi66fi$1`OV;?M$theoW9MuRX#@UoXc96}Wx?I1j)2+F2 zk!byP)`J|KAiJ&;*?@J@^l<=*v~@g-!b=9>?}rzKFDa%!osHJ0oi8aRZu6;k83fx4 zujm(Wb9V>P=X|2V4PvQ0U{O7e9q=%&22>|M-j)ecw+8t+!x|h~C0>?Dei(Z_Ho9_Z zVkuhjhts#6sR#SdJKE@T;N{iowCQ(naFFcHPlKO_SQ5^_H`n_L(e#W<+rAfTp^;X5 z?{^Xx2huz_E-865E|q2M*vk}H z3vY?fvL48}x>!4rsAb-gYv$B<9)4xIl@vRlHvcN?qt{~pfjfv7;u6KKv4Ic7CPPZb9*cOPtv7hI_cdE}Fzi-*l(=3;b(Nj^UK-s*0x!W{^lGe3CG$Ae39kM3 z(<5w@d5B!6>f?nOH0AlibR@!D+4-)D@j+R#GC{H!%NMC%4HZ3+Q-&B(ne(sK1%p&; zNm#!c+fguf!7$%{RxM{cFNxC_-;NB2QE(jfM?WfUOI@b1*~?XsM!~^FrO+>|789$6 z!D2sCu2|)EQe(b|IC{kpHd?OS^mZtZab+`G!U?avG#VOfaR>t@>^h$KUEyB{2VUfd z=*uw0oDjpl#ELdW{~-O|NI)Pic1+g43++-G?%!@Q5jaxy&bH3s&%8l{%9d#`@ada) zM$f3teR)@jb2OSp+$RQC0d_%8xz2Ah88KbFqN*@$k0zUQ<@HpIl+xqxt*KW#+vzy* zD6{tiJ>hMJ00x!;P-iyK8jcVjM|sAdrYub1anCIj-Q-eAubJsz9g4E7*zxipyuXOo zeKW=8%>iCG&m*$%|Ha;0M^zPdf4|Zpap>-nE@`B@Lur(d1_2QerMtV45S7pa(gGqK zQqm35Qi8NJ@7g}`Q=eztamTph9pjFB-~SvQ&ffd%z4lsj&H4R)rhwbdd*7pXLp>)p zlIMort{UkH4f1%F3SZjzXGf`^kw3Tx7h3b)Ptid7>R~2F*xDaf%X)%lft`pSi0mvl zGBZ%sxK0s3?@ZO|8;6-3&^|T%D>s=;Zx~5NGooAa) zcs}ZSXeCn<8nV-)5`PaG@Rr~$3!DerQS8kk#{w@RZr7?4av3#)b5WZ85}e8jA>E`wk;)$nm@hB3(^ zy_}b&iUzCK5|`18DttwsaXCVtQOAL#N`2CeOT;S@NQbH|%v@tUbdx|1WA+W+t;yVF z1^Pvlmh`+#=~gJzoy#T5qQ7rRsO8XSP=cej-x}QoGXaKyfAu_};zYiZ05p74BbRm@pvGx!lt+Wf>qrjKFl`9k5g z-L(ch$|_N+kYM37;705@BUL0Eh1P0qDkfa5FV3F#P3XJWlhLGM{?e~rUa<#$p&x>Q z;-SOeMg{*pP(1Ib=zV_dbV8f?z4!sA9P`q__=8;*-wUVl!q>3jB2!o5Dqb4 zzE`$Tn>AR~WRuMhH9yAvT2~}z&uZd1mgTTw%JtR2&64ia z*pl%^kzB<@lSRdPn%dym7Msa-iHad9G`_v4WS+zqB4@TVu%!g_L~N9~{Tm&@+6q) z7MA!ASfwjoNve*P$CV9ijTu95?0BG5v)n-^qhdzXedr2Zmojzid zghuj_%OF{yY=AOF!7e0zAjGVJ^HM03h=YfMT5OHiT0Yy?FxS!=WabyH;8l5Uzm`Yf z1Bik+lP_lL!^g8Vx1TX2TQoLv8@8?tWc%tWCL2+^jJ4yTDq`K*^=ZdV{6b)TKIYBc zP2q5>oog7XGtc$8xXDy0N0{}Th9R{QBXP^o^@O^N7h}B1IdgFuN}Z~=>U}|Ys6g{# z%F-0%5K!?K$XY6iddcw-75M2pn<}cs4Q3=CN@B@3 z8@eZ=64#~pA_N4RSac1NC{K*1# zr)CBzLX1gmIA(kPVhkrQN;*QZ9zE&Zl8jPx{}5Pk)5GdfM<4qLq!$b*DKcfs5fqg2 zLfk+_Lr(p0tepc@9xE2+0JA@d$#VzDPn^%(*|-i&5>J+RdS$UJzQrXAbLV4~+MLQ2 zMt>or>ecY|Z%(YSR3N{*PxzkY0ZmO3d4@fYAG`@9?JW@dGMg(?a0d!BRJ!j)t(lZ0 zo5;IHyd5`RMnN2K&KW`%sO|O!&8y<{P`z3{uA3BNWpntFqI0+xD*#dbe)CQG19?w;jb#DoyCG>ho z-twirRfn+u8lzs=m=2Jl2y%5(r=6h}&nM1(dn|&1m1P!WCezG#k%s4>)Zhd8CaA@V z(NT^r_BKBAjo=~Ko9MW_@4g>xu*tZU=ICZ-q4@%NY4SF{7z+PEek}*9rW|u{ywUk< zABt1!x1djwwN1olny|rkk-cq=2qB$xish~J;Eac+K^Z>PVJJP^rRq-#ijaaBkl6xn zG1T7W?^0c}l9Dlgh{8Vm9^#>X{%}hI#yrSK_#il*(DK&ml=6Ioom@PfnT0k5L0E+7 zuE$5bma?QxYeUAt4dnIRCl*U&(I_`c$)_@$E38&ed))|M7Gy@T?%DW)t$Mo++V#N6 z^&#kGz#6$~;6A=t=E?=Xekw?p={1Ax1mNdT^$+ZmydWfFvc=s z@t5vB`9Czn%alOLimqiYw?5gB;&ZY255~r;+Hz>c@?^TRqsI{C*7SL3)jv}-Kp#)Xv z@Ka1MOX{MV-TTa+ek~$T)%noW&Fq0=4Lxrf3HAekoI@NqVu%&a-}i{WCTKrCnHumV z&J~iEakXf!BRkY{^He`iS)93QK^4d<9=i;(XIXw$4=BLzoE|;9bBP~J3b(GU@zM_c zcsoLHja(z*c>syVp}xVuu>vaN##8XZAz(bgMCf#FKxX-7BT?j{O}0S*QuXuUK7j+i z$CV-F+9R&$i_uw8ogQ|%hyA_Y;ylCYA|*dRHXu*4O)kiK0@GB-hX8{TSh%1w{g)X+ zwN}uqb=2i>NU@7GAMgJDBMHOyi=eJdBzFAZP$-)`gdB5|%$y-hi&0Z#HTgs^>_Y~S z+h|{+rQ;~p0fkBhQ?+HAB4GTHXtQ~t_A18t6H zpi3vs2Om)5iRI>bi_t2sxe-2A+1~waE~57sH}r#_);2j*9y_ZOB-u{mcp6nhR3iKO zrE?&W{ny8t*Y9W1SbbL6A4CL2!YQ8V4Jx0XLSCM{5ey#GB`aY|f%|zjpQQG!6yc{! zBu%7uC?FECZFBrQ%cV6)Bz_I=z(U3R{t=iP6arXQ7d`7Rs^rUFTK*EH83vRoHOq(E z_3T@y%P4jcGVOybcWg@lFM@Tm$*8&fsFc3$gBY{Nolhpzs(Y|hyv&gj@|B8|tX3&L(24S#;ym}rHR=r~4QFOtIvI2=M7o;4 zzMLOI3^__pdb~?aWbXf=x)#$6A;_$K#-LzTfl-;&aDw*f>ZPdbn?UYyA(y;6`w?!< zanfisC-mW~1Qz)Ga!gH6I3BNi6QnHy5O$1{;tSg20RB=3@n+cC3GFfMl+j$1Eta+j73C%7}AH{x**{qvt@6#>*>3rY-W|rC2diGysMBaw$<%p#S3UJ3Ol0V=irj!VACQm&!^?N%Na#BIMnhcDe@jt z;Aae$7cd(Ri@_JDT@CIn6{rZPFPThh+uIjg#W#yywf^w^`O)8ytdi7xsn;taCCSA5 z=~mDTjtmZB8{}P`zgt10+dhQxK5gA)XTmu=bqm}MHEw5gho(Ma>ah?W*P>QwUW3xOem=mlz-8!j7`LVz& zpN#Gxw`DiwFszzejYUo2PpYW69-%)}bBYpxAtiGlZAMpPI>oHFExa=F+Q$k#5DNHI z=DlR3ur7n#d(*tlka~a#0V7&GV3h2Yt{h8L`$5_o)4&Q~TQ}(_<+uT|xQQ?@N?Cdf z$hnxje0%2y%skQdfaZFSX7Mva-1oJS*cR zj-udAHp+gQAxjhLi(EXd+NhW7S#&y7_{+ZTFZHav?P*HdK@wl(Ja10k`({!SgwQ=w zYQjS*C84<|y54*JZt>Sg(Irp6@rAZ&-H7^Zgh3b1OHUkSIIn^B zLRVONv;2hZ($dP3Jgtjd$O)-}VAEHLj}k?zFh&q&L{DB=mT*cDV4T;RtdfF7H$dhM&NhHFGFK$f&iD zLz#KFw=+Dc+OGGZ3VKx|9n8IuWx97NPxMm?M=o+`nAB!8-N}Jp@VvO6-{C77d0sKG zF~NfFS}k+e@C~<5>DHtE%XocW0RaL`*FR1qBU9#q_mXT*fA~6YvDnq&OQ4zMb)IDJ z;wH_Zq{Bhe35^o1`u(#(l4d^PPsk1uA2t;iEV9p#4SrhKIYR0bEkVSM4jgX7tF5N1 zcRJqpPnNklZvJ==bO=b|0;FrypqP|$IfBR?-(C0>OZ#cw;Df<>=7g=BAKE^F9Zd}5 zIOX?mh1ngyptB~zWijAPWmX=|b8G9kzm+RIgJ%!Zb?Gjy=I&v#;(E35NYD0d7x@tC zP-LtgKSpYqrgPnidRU3}{i+8N1?tBaRV>ask?@F$bAbvqD&>BS!mmnScK|Qu2}=Ng z*K_G%Nx#Vmej1pcE;pa+uRLI}MtApNpspNeG2WA+b zy9>_-xgH=O+Hur(U;!dm#hkYhE^ZhsyXXX3#}25b;g&EAiV!JzoP;AN#-=K;)0-H;sJ%%R%d7!M*rD zKk@%O#lMF4|FeN=wo{?sysw(4jp*ET>5+ctxyLGi0liSbb?Ws6Do{*7dMWjW8amXx z=u-Q8IOlmoZ`Oo$YOOM&^QR-dg&>0wWsDYdya-&3J!_yYi_O(wE42E12?$tZ8T3di z@&o3DMX>bSIrNi8=Yks9A}9zrZFuDXdyNZl^yrPr0L_1$W=ayA5U5?0{l28ME>uiP`J37Xu2BvVG_;=`EF`~((t^5CYI*K zIp{ox>)f!X5JKV(dGXtVa3x61x{tcUtj5coZkdGYaQCPhvbq&UsLB?nb^T5kf?jK> z1LSFY^S0)L!x2qfz#e#T(xcyq?}AXA0*P97!T;(@Ku$&jpqO-^!vF9~ISHJ>7&Ir% zXq!7TeK3ruSd(%74KEQi12mPVm*LZ1d;W>a9x^WjE zN+gXQj`kqqtpZ3n&JVAF=dDn~jwvGTz-KL6Ig^g`qUmplkqd}RD#b#K@S(&)U<=an$xOweARZ#{jIPw3Ow9B2Bv98NWuvg z8nt(4K9<|9f)eN^-^D=B_=1zeUKe!9#4&8g^4h-`P2hg~z_Z=5g;Eu&5T%b0l@xAS z9vI#hc4X%TutAH!%TZ$BFo=b9!J?L~rC;`|Q4e?oG{&*g$A&i9_dXEm8(O98e*M|_ zVaIpOV?mt6#6`NgKjjFb@JP0V2AuGAQR+_Rm>QS`ptJ%tU9zdb%IdrKjiTSm!9&Fr z;pS`Jv;^xLFZXn8a0oWN;+gtK<}H|9JXBh6Z2}#D1a`X%-}`JkAZV1Rd6kDc+}tb4 zkjt*M2B|kyuBpD4oHIfg8j*sWvi1u9UM~JIg{#QsIRQWKF~y@=IwLd3sv(6%prFDA zQQRXsQVUNryMXu#kPiv7N;4J26fhtiRS8FfM4ayPy- zt>;JS)zIByqzO-SK&mv40Hs(tEk9YnfMiszEaF~@rZu3rrOCT#GhdROvf40fX z{Ra&NfUjSLz^N>{m6l5z{K`t|%W zu-8-Eq7hu_eGay1ozK2?prKB&rm8ZviQXJtgL?gJeLwp0jj>fL2hw4sSbknk*44nk zmAr6?HsuF0+n$;#|FN4UuzU7(3`4%BPk{PVj;_cEi>f^XQgmw@Sa<>XWQ=qo-s7^y zU-%LKuoRtl1WQ_(t`QBcrVjV$-&-$TR&6${$YvKAF&(@l_1m^Y^XK#vAOx!W^hpg{N{@nYNtV0Sd);uNO zc>kItdQ|hkmVNI&)Br`Eew#FzfP<6{rI%`{GQR9>3twn0Z=bImS-4-&J!FesFsW0Q zI4phPJZn1~Q?raAQu*3}S&pjDXq{%yOxHM?wSn17wpk5uCAhFVE@}BYwKj{7?>(d& zQS;hA$M+}EFMx#Si~3Dx_o{4LAG4{)#h4{6buCbL`ZB_wrn_Qz-ZFV!%LPlTu+Y6S z>A&j4MA^zu)_IY0_>pLSMMNhM+x=DSuHb(>x1R)1(J|gsjsSIeTO}MOuXpQ3RyvzO zgBeD`Fwkt?!}A!fx1UGnL>^I(5awlE>S5fAa7OJ97rZRLZmcOP#dJeaSVN(Omw?Tg0M5^MyopY@KGQ*9glXn6E@5DrBVd@ZO6sO6}5 zNLz8Y$2>m1ZTEudKuUgL3VdFBC4U%EYr^;QUI&q-ehQ>9&YVtx~9I#stC1NewR&8p_W z&=c-{qxBa6nRnU#wg=`YIwM;hbM=M3$JQKJ)Q}vr_d7No8a~R%G^s1qifn1 zpTKe)ubdlO}Y=KpjEco+Ltpkm&4-5mzo(%J+qJod3 z&wt;+{0curFsjtbT8KiWYSf(7y;I<3Zr7JT^{byMaoq;YGJQH}4kI0ZBV!agDztej>A)_Q!Ykm!p)u>w?Rgsy#Fk2hT*fuAI$^l@X zxzP_tZ)y&|#B#ApZ2kmOK_;0z0O8B&q?G+9M zPu}BRwQ02Qvp9ex*Xy3QxGeh8L$b)F<1&;t0a$HI$VUXN8F!)>hz8#NZ9n2{u7vQ6E>gCi}{Feg{bY+QY5VLyuS$`xjNC8DBN(nG9mkMZVlc660h8 zkmez1BRY^Ry@K@-Lqv@Z2Xo?ApvT;8@3WEI*w!9kC!sJ+P2a%SLoG*BC2zzdO$Ek- z9y>gvGRj5q?tZjII(G6m+2x6J%&H~9q%DLCr@^ag)pnmEwknom#S}iggtBbkf$QY7 z?cJ3GWR8F2(1E(?4Qwpq{g{AXHGO+YroB4RVs5m*wiAbevtaa=QLo^U43#lzZ}eYV z#^e`Ly~q^8ul?)1fOP)#^g{MVj`DZiWSFW>D_@W>TXjKi=WtMRZZ$?!nzE0s$$>Jf z!&Krpo^yzQJsOR-%i`@7JIVH&7UK)g?j^%~U|f;F{eD@wg`OX+AEG9erA`Cq1sY`}69ElAvF;H^H0r}$KlTNU7g>7DCr>`2s8mi! zv7ct9{k^Zo;tnv!b2Kt!vY5QP3eO;Hn`0J@3yh`JDpb+)}A54W_Bxj?!cRTOfT<9(iDV$WM29}Mx8T9gnw zxUqqHEoSUf>mOc4AuZuXH0hWEjeRC(=ofl4-U; zJvHz%la`TsXj14B(ZgafOM3>v3QyJdeguQBkag(WLqnZbiVpTn5iXJCVc3HaYm87bHk2wt&w$?ZJh32#5F{RPS% ze1{yWUe;2@jSlQxN*~P0m1Sppjk3dD4)RDNCygk$C5re_fEGQef z78HUZES_a*qVSror0!d$wwmzO$#NV2fH*<8LmS}?=vC^iq(=;V{}|q=NuL?EEJA3L zIs7(P_1nwFdBKI-mGmRkXvf;``BaE8@PJtQnK|%d^_G@?jJ6u8Tg#czdhwz@W7R`y zG%rSXcQJy1CA2WtSM$IIFCoazD^4 z=xGJmzI4JR&Zh=CZ*)vN@biJpfJ>tLmc&dZDdA3SXbHD`CNc0Laa$3V`T$3ZIL?X& z?zwNFT}|85hZ7=jGswLs>IgRwtxJght;;ajI0G7JhGfjQ{wHhlGR7rt>bVl1#=p@y zzxu9R)uHG&(Q`6ktf?J`;hLr)Ca)3<^N z0%p0gpcB%8ZGmEeDIJZ6o~YF3DOgh8NuH96E(m1exFG03#$-_qlp#?@*!>)6SK@LQzyaqF)DuC5nv>AGDtJcG@ zO_b3&A+Tul^9^bZgO!DvQMFdgh+|_gWxS(CeFgQ#t#>a0@YQ=i*{63e4hFg$g8Vy@ znAO>)-!N_29xzVF(^dKj5R9O@;tCZdRKca;LquVdKBVArR!j+%4j*j-8#DQP7(?+OI4R&WkY}U)ou`6+a=z6NCl`n z79RJ_(Ocumjr0Ie?nLYFF9LAlC(>0J!mbt+Uo2Ap{1!W?uT%x*dp$u8U>pbJX*MI6 znXhTRCnPPH+}+oNNhZfNUl`u&@rO-rn`ScyExKpr}y(5?1N3;27hVaWqv(5ejv-ofMhAeA2g=VJ>6HaKe9(JhmSL9t^^+fm z8rvnszrW@^(%^le+i=z|`{vm>y_X(i zYfu|?Z!o^*OLgz=UxH)}F|@~r`p)x!NJW2K=pzppNj$)-8vsNb0RTdNKC;N*H5BTiE`xnL;bZQ_imjWR;(mA&wa=V~ARRc(mE&|E` z4)>=+9nw3`p8U!|gIqi1H$Y1*h*U9D0)LO;Heh-=zUmDX9%jRQ!V-1vukcb%Txs=e zmj6B`!AxHtA$6qvKxnL@yS@RKVO?GBfR|wx#LZz4%v>C>StO=3eUHB>1j*Y4@K&zx06=aY#aTmI z_K84Gs}ocr_#s1aNQWVCZ~U?&horSqo$0l1fWSX?Vi;h6&bC8ckg|R@tliP}-}q3z z&{a7gLGfNp(3Pt3htCi8$*t^rzjD<8+mC2P$uz6oOzX3~K2M6-$=Bh70|i_wyTTVIL=5(RM2H1~(@3A!mY! zPm?@{WhrPm@9(0;P8SIW89yWssM&-`TysS?bC%qmO|wP@(dZg5TG>FrhI{3qbD-9B zybE%YR|SSK!%zg zGha7hwHw0kn`u%T;L|mltKSAKI3S<{YRxQFD_^a;-Heh14!{Smrny}2>}bdbVpHHD3X=qOgMi&w zfJQDDR8?r}WkBM``rc2=HpuqyVIFWLK!|@LTP57868r=d>=djaBJg# z6S;j*D0#_>X*Fi@+C}#{z*{c~M1wBso2 zUP&&KApTc-AE^Nl%5>irKysB}M|U>p zW4uYmVej&W-U)1=%mE@Ngop(f{xDo1s5uBdsr1tYg26KFQ*-r@D5`UdZJiSD(`B@C zF~cLsF`a<6M7I;*059(}t30fgK&&;gFs&X^pcspmzl(g6E@5pLf-W3Qvx$Cj|6{~U zokH>fsVwu3W+`|^uv9!rA}qrfkW(i5Q3C=Ff#tAjrxALyE#v3v#XP!k3DUacqVvS!Zh(}S7aX9wg*Ii1Sv`4?!G$epJ5cxd8emkot@px6J z6}eI@L5HgRxY;VK-2=bU2 zsnMA1yk$z)No7hk$X~Kn5NCp$n&r+8y`_?zJX-1h1XYMeHR(lHUemgJNO zOJq)9i@{q;z~ydyT=#XgW5WB3cvvZ*;oTZ{;qdz9Yxwsf9IyjI-z2*S7G)Q!+` zJLw4O2b#cJ{0m(=jO=}c)`K7Lc>~2OFn(f5OaY1m$vxlk%>JSZ-7ro#n3~G_i%H5Y zT`*dSit$v@;c2Aa3>UXE?G3wu(V-_EAGj?&q)F{xZx5Hp_%ck!bFZmHI1sjk^07`b zvIU*r2QF0$p4~%lI*+^69&&IxTOsMvh=}&`KVhJG=U_wda5|kVfRr*JTwUfnjDtc4 z4x2Sm0wQu<2uHL0xI{7r$DZW~#eG$_bu6S-WC}qtRT3G;I&u}%QUrNEJ${8@j(U?C z9*{nS9tGQ{Z{|$pg^H=mr|5yIYAJWS@-Kc7G1LsPZf896hP3j@M4s4zl*E27hg5(edi)ZS(A)>dQ*tn?^EK0G_qgM4;vbywdcV@WCm`-?B zWu(aZM)#_F0(ks)+eN~Il6Bq+S<#bv;ssT`pFhHI$#(q&%ga1}xEv!y?E=%&5c>Mm zk`O0joAV9}>i2aZbBA_A|J>*OQsq3%OqRYyGYn!kG876EivqQb#3}ZweyV_$nT^6+ zYMYfsb_53D5cHlJIuusT{Yh=idk&s7VHKw?N#V(Dg9M&%fXrBol;Igy zAlMP`Ql&t(ZCCDQ2~L50+n6lifvHoro0|rObbuLoBl3;rRqY|g7Hy9rR>tl1uOLtR z>YfI^?r`Ge_W;N4nGqU9$F*o7j=C5vMM`y`cNF!Eu55nc^;W34o7!q>@Z#T zxgLL#ZkH}wJku#tF5#_mMnreZb(??uo*^~S#q#3%dp1TD(>?d5DB`N^M+wF9^5uBa zT7+`as3F#RQg~Vs!USq^ukhVD_3>YwZbkNn*H@^G3L~Q-v%tuZA0{fJN=wCS%cDxe z@Inflq_X_{6o2-59lhNgr<(Wq;q$!Ye8Ornq!iP9$PQN5jCNbR31 zG0!rXo9{j`6xL8vmXxa&{mb|@cISY8V?EMgPrHd#>okn4pBwBF4FF$%jnT5r=D zzgn!wxAdyI?X%le0?yolxS?*MNzp}gFC6Zx-L!a%IV6ZKaEO;rs(lO3ZbWk?J+Uil z&jqZ!Z}deZdl(RHx(U(^fK*Qnwbz9?DIa$??$Uv@e&xJ2qd`N2Zax zLEW8Ap-qV{v64U`qm@Lbz_aQAN_x&JfBq;U?-m)IkR~7{1oAySbZh*Esv8+j z*s`>7*K1V0?#0F#O%6H_k2Ahw5lz&-gAt)jQ-#%YJk z6tFDgXP)bb6Qai!7^Y*RJKSIYzjuE+0;lJ-###LF#9 z_aZ2BNg8MIg{GKl=D@R^JbFD`(`QrnR6ixxi(aqs z;mL|}&qh@roDH)UAME!&z)v6mDF8*eC+#wIt5xpT zZN-$^x*AcFQm-*JYwY<7Ms8!!tDoa+i3MYJ-q37WDmIkMlv`1~&pZ;6juS$6KN!y` zu&v)y?jy-2AuS`QDXXI0{c$S3I|{r-}UD#3;8hROOQgjRMN=U%ZQeRJb)$m!w67gO^DI_- z=HvH9ZH&kS1#<2VaOVJnpmaGxEij;khs3Skelch#r3ggYyUfU{q7#2d zM0mLdaUg;Fa<|sXRQF`qlQJ_E;*p};e=OhN)g+|^+0v#8xp1!z_WRmo*a^$uTxY|X z>Oe2C2P&GQ<_HTKc+P}dLZxzMH6ahMDJ%%bqkK%Pp-_7Y z+)}v(dU(Yn)}M%a82D^Kx!jrKtl{8`s)o3QR?PFOU|G5uSrq&h;C6}FHCaTU(cLo z@>qGBI`^cVrNFzZDjl>?^RW{{#Ka@da;)f1eiF_mw*d2MixSX)`H9LmtdQ~cM-*!U z4Uki2V!Nl>r^WIW_c(NG%n6WFI9mdZp46gN?t|WHDBQ@H^Z_+sZymuxRL*>Jh(9}8 zQBC(JWV6anBo3FIac=VIdcyp#(DHwx%swVa!F?!rv#-W5USNQa#KhodaM$h1`x;XQ z&aYK=H>^v@iTyr-n&(Jz7Sx|kea$>3xqwAc{*EmL_c()evh-mcNHYxw*6-Qckn22(1 z=1$bB671xT48Lard880Mzk1p1fAX?_bh5Cg!3*^-dpErAGfDT5oWaq4w_Nx;RR2A2 za?aWa7A+5@gc`hj<-6Ltzw*F$Q|mq1@Kpx=_~Ep|&%bIfAjcTZ9^f&6O+7Jf4BQL< zb3gy}PoA3~tnF*ns8;@Ct^P-7-{%N!I{(LO_^02Kum8OC%6KsMb9}#>1sGrdI@&*T zn(@tjP_?7-d|>*&5W)ZNo$~SlMqk#?w3iK!S5=^L7bkjqIewQ$l&Kp1ALomPE(6`% zcyt%NFIQ61s?nk&a*@d53x2vh@{&P>r46w_f5Gq9XQl^o8yQ4%whY9aM#;1s#x?M# zCyQYy;IOx03#5w%ElZPjN;~?0+>hql(3eoxiAA0w-X!5ti(LcRD4ki5IqcY9?Tt#` z=RFzj(?;Y^y3vK)h1!MQg=vA^70dkRmC^)^b-h;^S3>-RXVCD}bvog50K}{R^sKl0 z6C=+#2JLt|FaNo*!_D7IPdo=8Fz#!RsO-c>c9A0CFo-XBUZxPHEr?xqVgCisEWmsm z&Rb#q#r+fZZ+esT0I+OXn-vVNV@%7Sj`}gtx z&ry>O(ZIi%70x{TFGuan0+0FW&$#OU9N|i4IKab`5tRKX>GvD=&!dt$gE9R7jUQ>? zg98*~1CJ4*Y-YLRR0SC7L>)aT>^5EwPC0+SqF6!A4PFP4rw3~!QN)~V#9SuA(e&g8 zcgyVBm(%!TnnCE8-JY4D=e+eF3b18){~P|Ry=jFc`Ta{BQHn;=C|L*lXX*T* zZ-o31i5zF2j1H`@!pgp3ZcjRr_a8A%d^0UZ@O!exHQIyHUe%U@*ytG>!homb$qwUv zMUN^g{&?N}k*$#fb_LpgQp$r~oC?xYeE(&<#!Hca0bB2H1-RJ%Wn#qytClJMFB66wpI<+e>?0T^dXm7#$O5Bf6Uj>f#4;%b;~&P@3HXf z*?A34iwfh)p3 zDF^g?>%avl+36pbGKg)oyFiVN#bA8!0tFdT^%DJxVE~WOo4CsQiTc8a6(HB=gvt%T zB7Y*^;?*=WA3ZeYVXu663iKE}M^TZS7meItBDXxJpMy023BA`&MvdNHJ>1t$S53}} z*BcN%OceDBgr`Ep8XtE+K-#_y)>JOwx}GP>KDO%{i%t}rUGn;vFr-qxFis!fj!B! zJPcL^lBC2?amk8YNX0B3*@adUWE2q#y#e-*eHc_n;Ki4VHcay?JsV^-+=HoB0gRn#ug9eU{YxtLLs${{E`D z2U(|mHWHPV)xH36Kn(SGjskv>CP+JOKavZwz7aQ@N+4;!!wA%TP<07+jnOx%a0Zkc z>X$&7Lv=Yz|QV_&St`2wKMXAUX!nABj!d`~x9ga~C?6}m^TD&LU0|*V`zQ_3 zsmom!Tc>^MTp+FNO=Kc2{I*)vNU0|vL{v91cudzVZy+}#H?8sGyJFQBPdxO?v|mXHaL zujT0-IgZkLYN@^!VIg;lxnvK@#atlm<}np=@hJylUwwt#-7L+AoK4q|7fD5^l?~_h zt<@G?d&$GUAJUQ8GE6(S;Wm0(S^CY~4gJ2f%m#oU$ zp6vRiY9uW-FI*KcAACcMES3j#B86=UZ|R5q#`9<6D`nsC#DjUS=jJa>GxG__+B+CM zdjZd$vYjV_a^l zgL=gf5Pf4ghPMc4lP>YXH|piM^dbaSzBuFQ+G&bzye4$zHAbeGR!GKfGQUvJ!w}q? ziID0$G(&#D*1{Z|4VoVIkfX`EEdBIp2}EJteEler!skmhx^8_Pv6*B@oa?&u*1tld zI^=qM$J!HNKvyxUJ_bIjjr5&jkPc#O0Uck1;~DPmzU9w1uXT&?mkaKhzuKNJa5+>O z=(x?@p5e_lqA0PSz_=q<|MdEcXQS>erf~(^*$r(LF3~#LRT=D8v3=%>wiYEhv+2*E za-@di*@mW{(JrQ}w{)4gD50k*uT0!lw4D{Ep!<{1rtC%qHOpDIt^~1nha$S)GG*o> z>_HA`r?$+7hB6VhwV#6zN$6{|(#jk*D! zM{j9NT!QT+-7yNnn}ww)N$wpL;>f`kIkbf)b%c&z;_mytGU>4qv(hKiHBKj1_FH(Z zM|LSs#ri6eK-`cZh?UTly89qvBOyBA!dqT4WTfk^;HDJW3!!(x8yY4D4dzG(q^i88 zx8LiJ*-YbE!oIjHo2buh!qYrp83~%|{Rg1}eI(rX#a(f@;^a4#nuh#%WB0FuY{F5{ zw@57oVg-7>Fz_7gpHiMDe@MHw`L3azJlt%GtCCpzc z=Md>shqy~#qEne>jWkwhx~sf>ve4&3A=R%dKl*9m^{ps?v{_5XXki{+{ZaOYGs~&Q z&}cdAx{fuGLP$X7r01Mjlc4L4&(~uUqX~_tVJ)2d-^5$JLPb})lwPl9qq9aA6^oWL zGZ`GAUAZtJ-u&WKm%x1X!UOw2oT5uRS|k=@v&&#nh}N0gN4kn^8DsfpoJ8l6cQ#l8 z{uMtkFb$@d1kKl zC*l%+aF99in$^HN>{XOm>r$xrR0IppCy+qMUXgPBK!AoVd)RWikRKf7Hcd=?v?N3F z91x0xT}e_N8`M7iIp?YQ#p-)fQwndp14MsJ9hg z5PiXa7g1PKPH~9DWYDdp2SuWjYWa<7nns;vG(i{rxbPGyNt?;V@EV8CDX>ns^i)-W zoT#7(r?3#D<87J7HF;P`Gr#EX1ah&;D-6z!43guL77feDc%k#-MQB#Ec`JcvUXRsw zm-tDJyfvSY)qsjT3FuCHz)WdW} z4onVR&KU@AT2xXr%2{cpe^b{ceJV^DUhqQo7Regh8Iof9$9cNU1rBY$7d<2m3n=&g z7klp+)zr3zZ3}_~X$c?+QWHRmR1pY8YA7mIM1q1e5s)GQsnTnJ5JW&aC?HLgA|fCn zHIX7n5s@ODNS7*|Z)Kn3KKtym|9)e<;~nom#?T~d%{A9rb3XU|T=y3P5_7i(IW>0n zmvgwFi@K6+huyawyaz+TO=ZDr27AeVQ>qcYj0HC)Qf>WtNPJYFZjV~Np-89lsc2Q? z)7~DOOZNyUSZ^7H&K28K!!c$065!5@dwD5dl3$AS%hhn!oGca0q;Ig;ft=~cx<2s* zZAC{OTQ{J5cvT6L>Uh<)^b&I8IvLST;f}MyQfznQz?FWe*I2vct zO_6zD63@^rWzwrSz%mE3%{;VHSDcoUta)^BJZ3BO^jgSX+A*uPBy+S>&2hu89x4@*F9hYa~xa0$GoJRoY zA}})6%6y;;-G7#Jv<#je#Z={rBN_A(B$k>Z7zpa`v-%3cWrEyJ`^?CC2Y^*OpjXV5 z%5O6g#sF!3>&1MtXJM9CNqdFRl%n_*EGrojf2~FVI%a--%DIjD0`xJ2i$x#7?okmZ zBR14QZ5v3@)20veoiZo8dW=_)TIp_mQ^)Iy&}&Fu5XhXt1q;LxsJLK7O?n#`JM@A70?iicyer$Ui^1jYE+Sxt*#o+a?v4t4YVoE7hd z%T&-vSQENsw?%$3myHHw6im%t%S*6o-&ES3T|ovkp>38XcqfvM0q&9RaSHLm{_F1) z-gfN$lBc7xjwaPJgc!tM&D-OqaR)F1l-$g;n81&9rsmT`YT`1$Ta|=s^>TAwA%OJJ zZw;=#cyY?yjTN~Idx0b+m136jA2|m*KYRA0LFuSzLcm~h9qLTdCS%@XArzL zw975ITmuc@Y~8}ZJxMmI%nKu8z6O>ou-2blII6mq8GsR;3c zEx>=xRB^OVguC;K_j%zT5@;k)?|u*q4tMblB+ic7#EOqFn*+DfUdFv0z@p%*a1J5S zD$D88{SIr8q@Bfn9@M@mK}vVHwM0~;ud!}IlYZfQ4!j1~iHdk-Qf_6@<+P+vwQX;^)?s$};UH-|}mp^cgXR0enV>m^qvr=|gAcBIJ-^18BM*@kK9m2Q9#{MKL*(6%S-5j#4D8nZ0s2# z_e1Bp6M|DwAgWya(X`5u?UcB`#rKM=*vg)O6^}MxSJpPOGyMp1IhZA2z)1UfL|Wcq z1rv0X+zuTBbajba{4j!Ms)(;B11}c?6hzYxdF|9;SYq7kXpInC?Ih;I#D$uA-X+=?=W4=4Xcl-1utRC5qzH3qA776)ZHA-Hn+>% z>wef0E|3yN)f)-I5f-(pd86j_I4?==AJbrJ$ZH20h$+s1v#U}DGwXwT8cJitQ(M#M zBOp>)5~M+Ir7Owi);g~^{K=lv5)aArCR{_SEMt}G0t!E%J4IV<7idSY>akK9L60L% zafv%QzhoIjrg6#ugcUj&xp5(s)`{DNi5g0q%zYYDpB z>V24fHDj1P!7fu{-SNH~sb*>qPBmR*0FvE}OWApq@oOI0{}w$ZVU)J#hdwKb3xQo~`ZcH+Le*T&agv@`Iy?Y}*kUoF z@J-+8YLm9 z!7K0g=abi(m;%cuXswX+x)x!-knHprWV(7I2Iuy~Ew}}oE1CJy{X<38TBu|Qi`Ibd zxAl3+H4APTmMI&5qN2^YL1DysfqOeGTmb}5Z}f|BTJcfa!)OP44X>9ULTyca6DSx_ zfh6T>KJH$XpYh68U3^a~m@;4Bupok?t{_~`t!#mXN(*P3pD1Ql$uAI%P1ET?Pbrcc~x3OhibGf)Q|w%tEz8?6&Z-rvr9 z=dbwFzpYa~mm5*h;fp-JGz9FqjD2}iPYL{H0l*`Z7P&~_H5HG749t)dSvaW(w8ndg z?(~qvs&{GL`Co#%5pB0(NTUd`T@3rEg&SAg?Pv0+hn=+ zJ3c*@uWnNwaLMi1UzHnO+d_HKCv(||C=lOaHQz%2^24!I;rJBS4VNoS9(%DH3Li8_ z=~ZW*et^)Jp|dN1O;g9Jelo^c`iA`5R+u1{Ep9EQA`2^o_z5QLX2nS1B+aXlk)0zO z$Ba*%Oj2tIa%a0)|HKZ>D)SM7&&H_F!>-hBi&`2JK~pq``?wy5I@Q$(DN4h_BiT;x{llBQWSg5Hqmte|wIKqJSGLwo&)>{j1)J*aP9RgCSSW zf-pt}hdv8CadpVK8VA0rY&xA>RWQ$!(z@7gO^k8?e7$EJh(GBur~}PT$%%?T{z#)1 zzTVNF*F4!Nai^GN@@G{T-MmN`#Cms^I?zO6o%h127j>yE@J37+)8R)8&uYw?_IDI# z^6Kzp>16ptez+0imR9Se`o+v^4T7gb9kMa6Yer!DMj3l;sjF26{yg&=^&1V{Z{w#t zkQa> zC7g?r>B-pR^>;V-fbtaLyfaOyOd9Mth}Y4&IUPNMQnKlLp@-8oabDWldgivyM0ug@ z_Hn*(|65C6i}iPvfy>|XBBN&(1KIEEsaL0uUvBZy22W^)gQjB~^-$goOLh_&ZjXiE zvKZ_|J3x0xj`GI{D$m6M(=C_pfhgUAx2O0}0R$CdK$cYXu7&zfe)sdaF6utmL)bp< zmyW3%SIKe>Eq8))$K?B0SyK26LeNsq>$svyaLamS-7Y8b-oX2N<7zqfi+Gw2*S%LI zu77ruS^0q;hz+^HH-I8luB}V$RVdxt8v^9=ZPuV+(cY{2I68rXX}2bAR{Df78N~sJs><&BLa1 zW0K_z!hANDVLr95we39gqj}d{Yywcbfd@{Gez9lR(YX)istaqYc=s=NdI-skZF37)sUTy9fV%~GGs&>9V4ANk5GCEJWj zGnkVLd-$H{#1~=tx?s;=qC|luh5E=@;<-=;QZd;6W7q~oq(`T3)O4IJ_1YsykbNAX2Ppzr~J48+nj^<_eOi#Y!UX?w$eOXus`pJrA-*?Yg_WZ@?d=L`=#o_ zo+9)_#$HH$uEaW)i7oSP2bgpRSWUc*0f849<&#&@g+(3d1g#|EefX3jk!c9&lbUs~Ef#^~dd)^Go65`TG^TNVeN8527%Rq@Bo=}Dl2?w&x zwTYN4)r*oJFrzz7LO}zhZ~9xC4}ULh2|21?Lm3!P8QUz-+ebKxcTtyR2zEEXVFR-) zBWb_l0QBt;xulmQ7|P&HOgr0pQ`B;|!?aIRAVI_v>NNAKH$L#M#%Oh1>5ka zWq>lg+!`SRwbzZEYG(n-nZrm^R~jt+wQ%i)T;*)KX7vc3!LexS@U$ynN-gsu5?(cY zAVo7l==F%V`W)mMc0F1#K5Lyf^b{#E)UK|8JLJ)w%+W5bYUhbcsp#<0LbVCEOsbnY4l@98d+0(# zWF(E&Nl}L>?ZNFYbo6jaZ1Pd+Vj-Th{i`6^*^dJvgTCCo22D9$d>=o@>%J>t25M>V z&rT<8efgmnXjv>lJc)ofv6`4+w?LAkNmiUFlv#GRlDxrV9^0IH`IMwQi%4H_27HrD znWg{Sff3$tHnkwWSW%tL*iVri;`jlMD!eFAT zS|U#CUoDGldC+%R@OiA}%4o&*RU~JEAoL+BiT4M%IJ9Le{cHWXhU|4bHry~wIy_i4 zNvE_{)dnbNeipF=c|emLjg;oGbepOD%Flzv<=h~h=`+B8FL_PUOL_67i^V^UuEd;!Je0pLx#HeJJADwM z5!oL(+Y9y`uIs^_Bq*1DjNQ=X-X;2meN-jfK#eB13E-gwPREIx$|NIMMu}Z%{l&c( zC3whIBTQ5B<%Ndxw~~6RcwFwD)1f$uAbYZ7jrHXPF5S&M zDvgn~rEd;hi^0ZyS+i0knu_kll^a)EXJ3_n+fOL)P$Zfh9L;&!X32IZpd>Ij(=1EG zB^!|s(NzM+VyO(IZw`5@zhzMzUX?WTz4wdVwaer%(L{6@@&OP#pzm(JPA4J`Bj3^m z4h-VizTpqDDhPB_6#6G7+d)hRJ+XjemqI+!qba1m25m=S)A7f=cUQ zii~33e&it6e3?kMa-R!Vk}+#n*pU=u+*3|R+dQChvYZ>Xcr#w~w%04uJClbygR}eL zK-e-?ugbo?1cZ~!EgvG};ab#USA9t8Ia~fU4rT7w*Q*W#@7`o|VxP3Cl0s4MaaAx! zJ$If7lhz7qJ1dbtSc$xgvK&!gPY-b}d$mm=Y*Q zfn5((VM>qliwgwm1YZ-P4ldef#;dkW{*+t&Bo3CSJ!QnkDBzTX3YazrZ#@k_?XTgj zhGviiZirqUd*)|%5qCoZ6v*(H+y?jtabLir15We?lHxNucJyTF;bk+_L)z^ z6k!*04a#er&O(3IT4}ti!A^aH|beiU+oN(N#GldAhQFRpR}I=IWyg(`1?TtF}jyZ$v}Yux$V zOgCL$BX5Kw@>O#*v9^^WTIx#RXvN(t9E+@ztrPU{2fXM>=l%~SjDf}$d%KJjC!1#H zOpFkl9E@y6Vw(aZ^K?GEk8iMVF=^L|*F_3(ak`1_DfQ5~CPhzo?LOy(hk3;M5T(rJ z-aPh0WywqQGOkZ`UvkmlRoi(nqEcqCD?J@Q%}NnEow;$E#E!p(l3y>4!Kq)rtAdh1 zK!kG+!Rg=PD+sJa@EM`EU%hzwqrmDa*R%-;k{AFXzakXT@{~S=;(JxRSFb%h$xt7g ziqdFX3r9xmbnUVU@=)OXk%Z#&hgY+uJQ-vm@0qu>KzuHl5<+sV62qTbzm{$##%bW3cBJHGU_D;n=la_4St?5)8 z>sCSHKDCYN{*yKt$F(QQ+~h59Sv zPw2}m62S%CdzZ}75fw}Zh-WV5qO$DJPFS~&?qc+&dCdi`7=J)sM#cfV3Gddlbxp-y zzABEWEWeHBXM%g7N7#4eMvUGG(^Im!QB&{c0}EGBjF4zll1|{7{yWTVj8!8RQJT0Q6q10-sDQg_tT%)m@l*uMAl=Xq+`5i*rU0h_{yxDgxrw1Eayu#OrB1xZDJH74_0)3= zzI;F7HrE4stO#R`jl-c7Ow(CGq;KukFyjt_o+Y4JJ_Z`;!AX}!tlrej2?EDMO2N6X zvLu#J1{d|(OZJvbHD6r0eqzv3D`?x*@mii1ZS zf+ST@su$IQiS67~MnNb99)seb+CO+3<)JKrDgfUvS!jgi-ETkNno;t=nPae)wJFwv zisUfNLh>5c7=sKt*e3TvH))OOAQrnXXU-zzhTD82y>5IXPaC}2L_q*22d=$t!^A{! z?6(1`aRO9Y-64G32}H$quta4O@4uG4`qpV^VlxPEV@LqqMQ??TIL)!{_BgupD}mF< zFYMkQ(EW3OtM>kE#0Bq5n7Nvtaf(loTtHv)4lA6JxY+q(@L^N$G-N`Q87H|$E=yVD z&v~j26~7ze8&mwghQA^x-V@e?6e;c8h?J|he*CoAVuW0bz5D}RxzK$cx`l+ApWx%i zP_#F9?%FnERHJcfUCrN`+(gOI$PXZq*kF&i*TE-)H-cf*_MDoj;e3QW9&&m|I(JhQ z^Ma~EI8;11T4~ZI`=}+93xA4|h7mLRDgXASU_g7FT&?Y6ua$KJWcDb8J0Bvi+6r1K zR>ae~tlTkQ>f=Fza4%)|A9TFVSS|K8Nh2o!FteDklq@{#uo5-&)@Q(#fp^k7w4ctP zr<%Ee@Yo|~kr$u52eS4g&y8$zG$V|&z~%~?#b4H3jNiKUe?B$LjK{p)|N30}VjH|A z;V!Ov`8ABE-dJ#_y25otjsFzl1d-deM3nihum{)?b~S6r&=ggl`WTf@i%=wLd;>>< zzhZ(RAEdTPzsmG14SBjRCux2#>UOTS&blfeDc{q?y(SjV=+v3G)?RTr$V2q{mbb-i z4Ckkr4QGpyQSpCSrcysqA%2_~XPX|zJGCA7DSYrkc{teaD(92VQe^naq{$i_wzM%o z&t>6aop8Sjwr(JJ)E{-?`I;&4##!*wGVJ?uW=Kk8nt47XJClS@QymYBX5kslYmcK~F=$kGam`j~qPGCVt-fWST4)Ax+2s^>e`Yd!l2BYygL zlIQ3dl5AnmznU!wT^7y1(yAe-;S5)&35ix*t`~}b7X(tgYC6_t2brma@9?UEXImOO zX?XNGyv4uC(ZjKxp`aIYjP59_pO>K3Zd0y7zHWr0>7U(0sH%xCaZ-U3@)kFg3yNrH zF6mR&XQcMaRj9g*>`8y$kQ(>;VEsHV09&Wm>{{zgrB&*xa6P*r=Kj$kuZoBBn=_hq z!ch1tXvpi7HMRG3E;OXOfjd5Wg)&^{jU*O2?wgH0c&cOitNX}hKXeawGNrDX-fzFW z`#ERJp_cbHU+%U$XWWRd*XID2z~9S6U4hRx%hmkVK07#JDe4v2=?0%vci=vUY+M;q45i4WGk%Va5B|Y<@tg9HcuA8<}e8 z9rdXAv`PFOO;LC~M+A5>W(%AfmlUDec3w{0&G$6DcSQ7pwQhLcHB)6H)v5@^U5s2} zta%^bkwYGzDl4GBfj({lTZ1w)j@o+!h@J122;g%QOsWA|zY(td=t^BKr@T3X!_Ov;3ph4C zxta{cW`N;Y)yu+L87BsVsGEuW|1 zPvBN!V~0G0!r>ivrPK=G6;Iv7K@h#>i+Qq)?27}E45H&I>+#;Xx~JgkhK#ciyB4k^ z*{xWGog*k^%gvU%vOhJ$afR0_l&^lVsd@b@>wR^IgW_N|Vh<#hQXG{|6{#8QpfUK< z#-$uR>VvjREBozD0BAf`v^YP8j0$GwQ?Y%cbIgDVOnbZm1^$Oey8DF^sV&&9Q?nml{$d6ulXzexW^;kud~msjI{~N(lL!dXP0q+ zX(?JNZ^6axPg~;ePaOQq$0~HM)Jpnv#}laslz_kYirsG`d(`c0>c1TSAAdHE^8RU* zee&BV+t?gI*B`P0crRIjhTQuV{=4f4^})#7HUInZQodk==zqC$(PVe-t$^#)W_SQw zX03bs0Ov$NA$0n*Xd9O?AV#i(n zAu-fn=M`{OOZgfpTT0EO7e1z+D7IqZ_s@}`0vU1Gr4E-;*lYat!jaukR?eWB$WKEL zBF_Asc>c#jI|yjAoYwhmT>meduB{f}dF@hYUHJQG{V~wg9K?SQ_bX+g@%x){(gky&o`z*49nSHGgc`4!xRIY_5`VaJH7tYhi zaqPhTA+-qpFO1HgpT7YOp}NZv0&viuqN|zBUO|1*ACs#f3YecQ-w zjheA+XMi>fPPOR7Sisuf)pC3n8A1k{M9WCmv)}kF3jO&r1!*TJPIRV77)YQp+G3&c zRa&QI`Py8pTlm2?pYFYQ`GnOl$&l6Pe>%kYKZ{XNaoly43=4zGUg_1kUp>MteTZhp zIvQeS0K<@1g;f(}TYANk@OdEN=&z$@br;Ng;>0*J4~r#B@2PQ*>jnRs-q$Igc`+AM z6V5u(zY&%Gy?OmTav`-EC&-_m8~0#+XEs1Z@4p`GpFb~724GNh=Ks%c z0Vik03e1}StE}#)vNnOZ0*M9yw+%giqZIaFkgszUAVQ0iSHM#MeSp&}jqQIQ>#57Y zKonF&-3M~Yr)X4m)Gz>FThCT(@BAJR9eCcgevM!QP&`&8_F4cLr=$mrN!AkF3X9k@lsM!hrp;`9Ey!0EqGrlez5JgZ?SNQokb7DKm)9 zd$F0P3-ms9*nC&o>av!$1l;T*Hh@p`(wVj+GJ1I71=*&ZsuC1vuv<0e_^r~ejpyiR z)D&>9>AVvKjL~S1Pcn4hUo1U~;ui6d8synXC0-Xut6Gi^Iv(U&fV(KbvUYDXSzv0tV-lW_NR$-*Noq&9w)hP5K2J1HFi~05xAM z7Qot+lBxPJKs)Q(7fXLQqX9y@2)W&V%H{sUTX|t$O$cHvIqn_8!UFhy8s(oMPaYmx zR+8zjZFxM0KaNKv&*gwmbOUj(Gj|$h1SE`V*spyclSTE6=V2enOefRlyk|3kh>Ocg z?cU}<+hh!25M(s3ncO*OHwe5o3M{>SF)TT9K+w|^*ww( zE;Hbzq_N~T=neNPXh(lIWl!#KxVoz6h(Trl3>Vbp-v#X1*(;T=*%%q^j>b^A@BrS* zIdTI8o5XE%a+AVhNm9v zV@|pBCPWBF1}bRQN^WQG4HOm>v4-O<4;M`5-vKopJ zhlWo-a(GQ=IHFsur zb`_^cptJ?rh6ShX|MR>L2?Q31n^()NgE%R2PYlDl%>`SeK53?rZ7lP6ImYV^?~RoY z1LMJ-KnHGt;u_hcpU(V6FI)5)E>G@NC_^tLmQvT#sC2%-KBIqX!V<_XX|K=}LRa3o z%zT*sD}8U78fyM1{PD}`VG&JF)M*4Km<q&*W?T_uRW$bBZvN)D@?x3?=F z_#$Q1(nRIE@w+(pmP)qoz-D?T> zHI{X`TJGQe4T)6D8JQ6s<%qKIf4 zWw&pU-FtmS^{iLz+<(B$bEhdNPhs=F14!Mvz##^?+qJb<@{zAK?xqV5(N_>tWmSs; z65G;YNbRh?CmT(0yJf1(4LGY3Zr#V??;x=OQ z>VQWj>uA%_g6W6!m3-+pVNE@I3|TmA=)sCg?-}$L)(CyyxT%N3Z4bCs=2(&q?T~YB zb|R-JS#8eti%*!!=JNWi1z5D|uY?niP^=T-&GFdN2w}8k)Rz3mC~R<@OmGp4@oefs zXiKb4^Cn}##^+a(dug&iElkEP6E`Kp&+O49JTU4Tq6_MD zvRJy(HT5hx;6k~^rY2=qzm*y?6bz?MebUB&H=;@)H$l|^9odPhBUi)OjbFfMW{A6? z)2Y)*YyE&u%2CX;=un|dZyQ-ZQk}E*u{J_^*1SKS*JSQf)ir{&)65Jh!M$!L=S?_F zkW=MAGNs2e!PRiJDs7tC%VhD<+9eW8jhkx-Lp`$R$=Bg%56w-nQ1Q+2|Nr)Mgqd#Fb;?ud(U(lI0=o z$An+r4I5dBY=(C4X&CD;KnP89>dlaoHvwgn)672^nuDovpwmA?NMfO?sCW6vcgYC0 zsgVrvRa1szA9?F)TJsD#H+`F)X;su>rRaaqp^xb1>c6Q@-`YMat!{=r2Z5+s8Zw5m z^wOC`zP^0u#+DaoGOysm=U!|Xm6#|t$nRR2O&7{AT{F%iZ4SQ8*OBKqOn=z@q8mtG zrvoPPgD3Id>JHbJIa_cH>&;fD>B3F$lRv)t=eR0IrJhx%MDP({s*g%owscyOiq$x5 zv#OqyP6L-=Q*L>7 zQVh@~s7#B=yx%hPVSHL}ecj^ET~Zf6rFoe9*aM)?txy2;GsHdUMTwaC*L7a@1Db7@ zI{eGwAoFVBA>361`f_gbnINDoK!>pzSk9zo-dNe0=9dhZ0_@dSojy+lC-YIb;Zv_P zW;qtHObE5Hn7M_bM?!T%+%h4q3k6&gGl{0(z_bcnRnH9-x=c;1FV9V|h`k?T@KgIB>%k!d8+xj&}K945_ApDmRl=$k^8QKOHJ zx}e!u4P=X1zb=_i%6?*p|B#Y9q}%hwA)-g)1(*ZMh`WSSEvmE}0|>J-PNJ_#E1 z*k{Jy&Kz@B&O1`c6$MlIjCw0`(K0H*qqeZv>Zabat(=$9O({gNsL2!Vp;M5LNKdTT zs}ZiYFZsHe7%5L3 zf^%8zG48$5|fnZP5YO632NUG#o$<+Q!PjScx@I+mvla_0e zho0tj_ir_fEj)8t=f7GnNhj(x-TnAJs^t^4Z&n-Eo$u2tUM4rGLaOmxMlzt&p8fY> zRUlcD*@|EXC3ryzci0diDeHD-hg#B@0XMQV^F(ULHIpFyv%0aWpJrEfyA@YQ4KOVG zH}n>*MJI$GjDQoBgD}=kBK7FNs|w(C+TPHrjiI_^Zxt1D#r2HP}rS@mhRYw)^vp4M3m<6 zrU6gT4`Lm)Ip$Tx4{d#O~X9b_R>ttW%+JW2V)L(kFaf77)%?am$aXf{(D1FTrs%Ie#0yH2&y; zgJp70%K$9}EhJ|R=9z!LlZ}+H?sfc^s)JBp9O_lEPb1^kbkUO`764p!9?pxBey!B% zEJ>XK|Apu_9J6`F8rMSfV?zd;;L+D5GM9Ef^ecKd^q9#oP|&>M(3U~SD9rMVPxBAT z8zVjJTG9wc(Qhm}#^X1R;lUT>Loy}Z2HfT*zn2;IUi<=>6Oq%%U_0X57^i%aqP_!G z8kYjFd-lS@qWO3rbosL=?B~ye*e;>-<1W9%3iG%(PfmjdP1qRSZrc}x$Y|CjZ@?8w z=6(1t5&2!F+aB<%OGH;cGf$Oa_Y_1WdRh;~JK*ZJA2Hd0RIZd}u^PK{?6R1JBV!+3 z^Yn+bYp`}b#a-}OJ9#-D>{MN@%nb2^pVKmP!H1PxTnjJsFV)J{TjdM_MlZ=LbsVDv zzw3E^!s}rJ&s~nLTYtW)fST{!OA{RZ5Q4*VR*p;`>s>T&$obGkW&A&UcWoo(XG_cDfy&Amuyr^{L}D~(e`y=58WPK z_wXX7Ok02kby)7+3UajmJb0?&i9ymCAX!S(fG_$Yyc9^of}l(gT1&=(Oh%0M8O31- zyaUO^2AnsB;3D!CNSo6iuN6u+41>z*lmrU4$Si%yR8p;=2@MQZ~Na?Q8YcCcgI~uy5F8r z%)WcWE>GF}`j;k`KTAqhqqMx;6Av+C5fw_*W#8_cmVI3;cRc)L)t{s|-w#wHZaSAY zoie`p(e?$69STu?TVHS=Xz5nKq@;n8&Ou%n-|pYKL`l<&!cu!OO6i{$jPvPW^kmRZ zSvJuMgs)!@8292poV4rx0Ad|e2mncgXh1Th#O6Iw)Rm+{`$uwR^O9`$HLt7lm$DeD zt9C*CmbY5tPDGsu>$fny?S?IOA49E!vqBrysPwWvhV3+N$`Gz67KJ@k3#ZGarBo}O zW!YMs&06b5ss=}If0=DTK3j4fwoEZ~E@Z%3Z&DapcsfaI`h72!riAY6e?@9qsXu~W zi;Q?KPfB)qg?s_*U5Y^* z9B-ubY=FK;338m@RL=2k4v_B!^9=A4mKbD*hd%3`Qp^=>M%?5paBSW75k%YrN2-Pu z1@DA`4DC<^a(}!S8D|(Ra}mK6>pnSy;BIB*9Mqb>u@ij|xu(xm%#?dA`9vfO&H;=M z{NRZVarxR_nl?xie%4Kbp zJLaDsR^0Db>{WDAFOU9FT(=2wR$9YVg#18fr<=STfBIHCoC#r`jMm_RhyS&RdC0=s zcULw><`EE$9OREPVMLa|7*96qxgw>>TnHCfV7AxioV079RxMoa+%8?$6M?QI#F#wK z1GAQn_%~PFPC%s5cdDIrnQQ|foK*+mkVVQ0E#1;G=6%G}qw%NoBsM~Iz^D8 zlxAsWLB=se+lpG6yTt-}rs<+UfrhoGx}!xcjtSuc<0w^0?= z8!folOU%?(WT#|dYu3x=u9yp@6Hj#y?C#(BFGvp9?aF{C2aRq<<}qX{rP?FGD0*rTIll~GJ!CbT&8M&#vphSa5ad)Ny%mYv;0!egmYxaCq+G- zMxQZ;ghn9kA5UVJ0YbxkyE2M^Qgk@k3q!`3=wG0uJtT|;<(f-o%C??S-&yzXO9?oz zUb|N_H9E&=jp{PIowWhH!3T0GkQTi%7qc9gTSH3PiYKc93JM#lzx=jd6@((0KhvcWZ-0NC%oVd#&U)lze5 zb=o7g!M8J}mVec$?x~kcxmwZ4Xx^R9#IG z*MrDoFV#ziD^p3ZQIlvx{BrhtoXAKJNFJ6qm|{zg?s01Ahu>39*$C(kR;f1#E=*)g?dCsujJ2BPJU^GY(6G z=tyFNqdx=<*eH~NLg%$Z0Ek*PTw0USa~;MAFL*Q2<$0k7K7;wi6Jq};;F^hS{}fh1 z?Wx+*BCBU(6(`$Et$ZK^>15>X4HNhTdoNE0Sjsej0pDO-^C0$FZ3PpNryi_cc-}^= zEHUTL?GljlBiMM-upIp=nB${4pZiPb*O)c^I<#y^kKU724TV>HQhd(Lkxj=yYUL zJ}TJ2d~4#WDB^gA++N&zf53Nkx3%uN$#inS&f47bzoz0my#*A3$C#bcxV3G7u6C!i zy8SyS?{LFwzp(Qw9?PywR2e|}r`?q6!;$wGXRUb$;ArN-=WWiOPPpvRZiq31p33N+ zH}r_Eb;JN>83b@vn?_oek3B;MNvDydTPw5U)wV#XVZg3g*S zrHot#^gfLgy^$l#$y(P`<&AKPj7(}>)24h@k8G>1v?(ok?4x>@(k1$5i5|RkbWVRR z8UTd~#t(e6S|6v;Z6wXr8)Ce=cNqF#p|Qps^b}~sjq*m}MVl@GTQS(+iDCbZPo!(F zU0zVtoeMV9OS01S9(I2HN#PaK7f<-gU&KpE5^c?opP4+AuTM7eLbT#lr9M^4mw;?m)qDr>=X?FHzd_DW$g_3%Cw`vJ(=)(1-AL8_ zUy%v8+q8g()7=N9VvblUA9}fJKFAy8 zJcrsp2vUU4w4&xUl3d7mrF9+5>ib`8fV%4KH}JXm*55uI^6hG{qv6gI@)uxbfN!Y$ zqHXeMR>x|H3Xm2uV^nwRkB|2IordhvKm2{{y8VCl6mr8+=+;t@)RkMlkq|QDpI_vu?6~^AZG85h8<~ zhH!eOXvjZZyq{3u;^C?O>7V`&646H-Bm~CbweEx2pt2ohdkc_ux(14 zF_vyk)NSMxmS}gOje-o<^qq00f0)A8ShwFPGYt=mS%h|N1~*X}L65Oi7G&H?n;Y9O^-tZwWjfMMo{0WeZ!Xh7mUy$&lN zFC5!|aK@WJ7>{x>tTltILm+1!X~gKse!G*ZnepEY>p$)!EdWj4UU#3Yq66^Q1Q+3$ zLZw+@;0?DH%U`ZECl#NzIR;>CGaEDp(&X;Dm-mtJpFh=Eh3RC;Kq?GC5jzJFTIWxv zk(vdZn_g?cz$hnskRP1n%$7f(T4<-2Z~CgqPN^@?O`B|iSaESM4v5MwwE;p*#Q}Pl zj|T7hxym`Cc)#lvz*?V`j6I9JAFz0<4banW1J7eO<{s;VlVt#z5z3^zV^smXG6n>b zUJu`U@mx9VIFy?1F;Wh<{>ej$7Aa16ZF$P%{Jys#|EIq=ZLAb%>3eSFbl=!QZv)A| z*Zx4rKjZ=8qK&JM7e5>=wpNoB_QckB1AyR1P5C(yP`9Hm&N2ajdr~;k0}#j0HQfdL zyrS8FvRjx>4wz(iCTn+7WT9<%8-UiA$STlS&j3hY=O^4LP2H>D4p!A zZIc~wPDEVo7$+X5Sa_L{$$guvK&VvO_h{8!fZLa)JPydn^M~n9uY{*I$T0S*mkUgk zT>u@xe9L53wm@jssq4A`;xvhzzOol1^t4FZWO%)r%*%6R+3GQaUvU8@A~oU*z{~7^ zQGZ|M*tlnZ<(-6G$WXZ9CK*|G#gcbO2}mcR-tq@f1Adbq;j#>N%L?rb}GRMfP znyHwXs@DX1JZvNoxZ?^<^X)kLeLA*V1(hpgZ6b?JPzRqITFYHEI zr~~>A_d@l}9Jb#q02d(HUe`v?;=69Yde>yc>xhpm>OBvzvZ!6dpgpfTDvp9UxJ|?( zG|{!dB`0n=v^X7Sitn9=mF>bN{LMV|JNoH2HjtYV^ypPw(UtBs={B%)k(}p^@-*T1 zcQ5^3pQoM&PVvg6I2}L{CKW2;MK5M5cB7hji@=_JcTnwaYaGTI9gg0BnIP0O8MuWp zj&W&k-!HqoVFV}ej<$<&^cUDHT;F}y<3A@Y9Fv~`%5~=kf+*%uReHPcN{%Hy9IW$S zTn2cCZ}2CK^ujI_XoOFM?yv>?YCOr5FXL=e2Xk5kuO*f}3v&ny4s%fFUh2&4_h<)i zf@vUhX73!U(PK3Z{S6o{D99=EB({7q066o^fN*ns)+f2~dWOD`!nC+UMK_(hzZrkz zwH+TYX|F#RiM2eKngUSSertfegjIH{@@4mv`jsYW4TOB&ycs`S@xFXQ zHOYE%#+t*1PU{XsC|x&yx2Q``+_7&3Sc*<*+Sz8Qr2!;6BlDGt1L0&PJ5s77Ugt87 z#M5<)89W^B@7r;`DfnWf+AD*55*Nj zSpkB0d$`qW!0^Xvp^-#u9KV3`bSx(+|El%8+~*a}g;43xLSG{MvTGs%uM;t{F2|o< zKd*N`-N*g9e0<3LgPzP_4sN{Cw>U*H2BCe8J(CC=Ae=8e^ymD zeoApY(}9-8AtiXd`1>Y5sUeumHv7 z_v0Ch_e%)k59BG3O(&WS@H5%SS>a!&==v?(R!UI0LWkS%6R@a!c;86_)1$ zzkQ+LV-BVXX7#Y)6@pdGi}bit+lSalDGTsyAWzHf3XS}RoAvtS%qAn!%Sj}oR& zRF7qAe9W0n{~@@CU2EEz9lG!Fy}0DV$X`}hS_WhSe;^Q4c7_}>SN&XPQF?~MC)8M} zExM2^IgWUTOc{ojqy;HS=}u{oMnp==ySDW{@B92e z-7)UC_rv|dfU^&4@4ePsbIth8-~1fLm)c%fAuw$c^Maq}GZSOjdp9FNw|9nZ%Q><> zMpeOlnmYS&)fo`-N!eqWctOq8ZFz?$B3pcC@UR{bYjWYdR$;Eyekn>f`1l)XdtIh= zyI`ur8TZyr=dHYqso?x(+7F9kP}`vgn%!M2UIXufkHR^7u>IgaNLTFla@c7v_UAa+ z&R&1IdfXERjdJj2_PYH`M=1(3iu@2xk`LIR-Cr7}2Pw-T^*VCc%8jh}m86yE-B>;} z1RkPA>i;+depvwV`*FX5L9zB2;O5Ln*hRmF+!cHgG7T_hjIh8TJ7y9r1LEg94>q^2 z3go*w&Oy6xE(_wDl6$wN7FCS}$CRQa1V=@ zu1Llos6tWP$0&>nmsPlFxL^1LjnF?k8P|%LzGa#_dTYS4%`^iDUAzA0zySF`%-#3{|oyA6YHkYeHJNCmW62tswR7&% zr5si_nu{RXqEu5kG@>Gv`=Kh{CALl#xU#uXW}fo5^c%RPyTF5u6{L;VyjvUlis+f= z3b_^WNMRVt2HO5LtX^F?Dm)d0E*JR&QIgva=*R1osZgx9#ab$1tlH^$xg+zoDHcLSs9o zgx%`C93k+eGc(+QE7e*yF(mMX!^ZWq`V?=|RBnEEIyLrA+4R+px?Cl}qj&f3CIeWf z;p!8mF&q7PIWG3IUB>pr19uE>Sqs`JEaUp2y2C-vEh#7LE+lE4If7&9v3up~iFdE< zJ}n-4CObfb({Whpkb*adZ!O3HUU~EMKfw?Z2@r(H6<->lU-}_9vvm6TLEW3r{sQgc z7Y!FY!~=ovcy@13RM1+&t8MHG-8#rnzk7S$TQq;fVhR{RnQ=t-TMlgFemE`-#-aqq zTtrwsTsGI3>+l{I2M8W#)TK~n!8Sr`JFNqel|z_bYhGc7g?8M^OhKb3b@UxUIcN8M zn72TqQj^VSlz=kLC>!2(NYxAC2>NAtfhS7y49AhjaocUKB+6#fOg_TEGC)-et{2~p zb+se^To=Oav70ZJ`ZV)m9vU?tCZA$h~$h{9;ccfepcWM5ksQbxcjY^oY<2wJN$8Nv;D z6zR8+TAAtO?40n5goP3XYjXP80*eaozY>Mbt%pdXG1juran|w>z1&m|@j(lal{5U* zDA&iVqPL})U`K=hLs+*;Vx+b1qi(oUn;S>uOYfdU^p)2c-u_B3iH>7^!rkDt%wx*@ zs}f5M3e2Bhe?Vs6Ur34mxUvB|0p^DMFXG;L|vjoy7_yImtCyHX)CUB zsI6pq>n4+mUO}N2Zhp-whMFyB_-u%dDCChmn^yAG-&e6+_rd@3R7J_R)6> zU^R(?1&N&=bqlt)2-ZAlwjHFUn3a9@A?O?Xy54}}YL|Vt>nGWve;oNg1H=f&qoDh5 z>}%nWz<0}`EF*Whqj}$Tptv{f0w+OT#YtD zkVu2#dFUim;;PAxNm(h5t1>q8Gy$eQ93lfXNxOjyB}xmooT|eMDA4%1a(&Z`!VUdY zW!>?mkTEHe^+kIyF>#znWe%zyW_%>aL@2(4jYHsLGZvHcoyW&P$cbF96nLJeTsy>r!l<>Wr~PahYNxdE4=4N~uUiTU-AW>#tBjO^dkE*+-BTNOWiy0=W9d z(XkliyNj}40Ht@@tJ_@5iN+C|i4n*lw4$#Qj%*X-$2#2zc_(hs?2Gvm;*+8AhM>HM zntCxB1%AON)gcrD7?Bp~f%0j9 zuX5C)ACz@}mB1ITN(;G3k#py5*cUv-gZ$3+tMAVRZSm)#eY(tM^sjL2!>*)Q`{>`_ zA53$h`q<}^skoa-Iyg;WuMmkKd__g&#iyBVrYj!W|b-KoW#41tvyYLQSkz$R5iiF1Ees&8f zc{;;4A~>#bZ>ezHyXF?}w}}^b1?&@+)oCb_rN*YY%oGUjR*jAjmH>orZ5`-`e1n?m zNR&zDR<@5_gq@CX3Aq|O@lt>n*TL2}p1B(xmTN=O>uZCatc!xQ6Na+a_((@dyK!56 zuY#RPzPHoQ8k2uPyAu-pzQ-SXf#5~pnpgXe)*{wxUn%KutzTe(#PaKRv}21->XY76 zYwkE(->fN!ovrW9sT>MO%gj}^)qCJLoYx%!y=d6x{{>VG=!J^P%Jz2Ba z-fwsNa!he!JrU3L;4Dgl&VBrCTK>1nzwUihiyQD*3N~>mPVB6|-4YydhV!I}j5%-i z%<6U@$xxrEx_^uPAyQ)2a>AJAq#kM_RSeio=DC*w6L{#!5U*3LBOt&#qIAERLTrCl zRI#JqBnD{wvD9b<3BEpdnJ+rNT>+O%s5IoELjrt!Jv3?0r-C>9-F$WMx2HA7nHTOg z!*$xc8xM`ssf)WK?-QSnC?n&G(|f3@P*m~n+z)Q+BIw%CK3*yn^~bslUQ7_Ni0yC< zVG=C}tBFF*58q%qqg*e&Fn1e?qCQu<`9v+Y%jGP6Q*-sB%gD-`m~_vNF(J&UWA+u( z!t8}4k@XoHj;A-?wbm`Vt(=ms+QY^NJZHhidt7$f_tt-ktc%{C@dFA-zq9lN8ma!q4Jq|C&jU&eH){dnF+gOT-~@V7r#iIG<<-cx;}ukZsIQA+gkhUThFutQ zS&v-~zCqzudcNGXOg1zkECdR;+TDAL<%^PIB|)zyD<=mJ@h_!CW=SFllo@tVZT8n} zg_rT9iOMP*C-M|$q{U@>@a9JHnx}&8Wv0WlQ-yoJOn#8crNpO+&|IwkrmeOSzxmcMpAf*C>Z0U9L%lFztQ(4t?PEb>Rrz~t#3C;qBldoFU1 zw@ui`&&0@^|51WTLg5FTJf4-B(?z%BQ3`rrylfJlF5$+zj+De~H2sUWj=x~-i5C5) z4fvVGlQMj-bv^A(JoP`0%D=z1d;&i1Tdh)1iw0I%tM%d>~1pK=GG$|o;p>?l$985n8$_;#_SjV;ZzVk9& zN==9f1STfuNvK#0c}8&QJ0*8O|?_~NSV(^yvBBXp4JE3}qh`|JX@Sh*NZ@?u~F)l3y1QbG5$$$Px)^71t z`j-AYQvaWy{`?Y|Cb$%kPEwTnUxPuz_Olqa680T%NlGPbT>Zc914i`w zOvGt8`hI<$Ui)8{P@zkijGdOu|NZ1lnUdcyNXjkd9lu1g{OeKxbZLmdxcYxyicmGv z9R+KO_shGJZleEwUvNnZy0r1)?W2Fp(66}~dXI=|oqY1?`RczfWr9m&hwVCS|2u<( z=8qAO`1_yli8=iHk}9}_?eU~I{-4Y7^Jf45a{Xgrz;dDTe#v^i+D}&w=xWMAY;S00 zhD@vM@f>ULuKk@%$ySuJqiteTTq+7UE;SV#n_O!@`)L2qy8C@50P^Fx{=Oi>ZST`N zE}c=$>$|Ps-!#zhd>@12<_mgr9?P*ed4jgM>VE{?{3f(+4tXj2f*R#{xw*F=`+b(@ z)y`o9i!X_fj5ZY|bRU)0hcJNuu3#lqz@HV|kM?bT?8Qi^PU9Rvr^Dm0+Rf>-p&Gy! zBsxkJ3wDO&wd6s5@s-nJdh<53jJtpCrkY6tBq-FLN-UJy}e77^(>4y)dl3=o=3 zK}65auex#%GQ2v3gd=7D{LdfB!t`2oBFG%V?s*Ni8zGFwYa(~B={0p(SN@;v#~T24 zxT#>Tl3SPsKT}t+#fO+Lv@An`h?|w{UdMd2e!<0mzDlSJw)K*7!S$R!pZrx6?!=si zF(*9cm;c^0cHrSLT;;`R{w3qx4dB-AGo-f5{xx0@SQU*rjV*+K@9KH530E(%9{K!h zJf>7I-iI`G-1~o~;jf?^umj!FI=BcDk(8zNTZQ{$8Y<2Wc>p?v4K+b`JDxTZ`P2!T z{7&(>aek;yoFT4KRCwDiOcBU&=yr-z29&L>;wc$#zvaJumT<3_WfI-1O`2DX! zVtOU1Qnhf${gcm+%O#K;#psP^E2^sdyl&k*uDBKoi3SWj`dD1^x3odf(-5E?SXgz+ zGrz?Z6%PZEHSJ5pp9j&qJ64{DhnZ3VA7WI*NkPZ;8*WNwK`p)C*E0+(c`raBHthU6 zXq>UEFufk>eV83ZRi<0XpqRqXCVH{omS!`v_D2Ir+M;;8zSQ#G*GFprNvpO|6$wf> z7O8qu-RG{F1&fZv0Dc@7R)XRN5vH91XkS}dCF;I5F|`J z+dq?_3)(;Y+n+b3>RNul9d3@cLD^Z4bPON$Y8^?Uy{F3PRY?nag;+>}!v09}|13ZJG z1X`u~qz&JBpR{}ina&rNQ6j>zV?(WF{lyUl^%l5s7Oso?(&c^- zhiBHsX1zq7!XzMHCwmN*y?(Oew{WRGLzsQmIO{F4kr^-(j!*}{NKm`C_Ls>fQ%<=i zl~yC$$yt~25#xG5#%Kl+cYXX$mWJ_2poIOMz37%!P;5$_4hkRyk9&MBx_sKh!b4fK z-48&6vT#2?sQ+~j*^yVWtr17#p}J_PAJTQLV#RuCnMF9qwtSE;;$?DWkYB(TP!4^n z`RHEMt#E1?yq(n;752tJ!da-LiS^r#n@ga)Yo|(&%&!`gy)N_cY$pwBP<75Z)W5Zn z9f`wi*cL}s->fEB3X=T-`}*0=adepIWgz)2cBbB?ZQ#W>G3)c|oG{mdUU$-aMa&r_ z#jBrN!Mu*fzY5LR>OE_8Zdj`pAI93PeZ;1(0u)bofFulwpAsS3{ae|Ov$7>92Tb8~ z0!rS{^szVONd;{67t4k*rHT%?4As+oj*1LAXj(y4_MD3@t3?`sJaEjh&2e|iWBB3? z(bBXzI<04!@EEu$5;L9UWq2J==Ne*9KR8TaP*ru|G8&4bA!N~pwb5|oRiN`tYXa)PwGPy02@xuDaf!5sx)*gqlRdLnxYP#2GdBltO3 zpG%rzozr!XjB{C&{0Nr7&~WByTm_O($^+|hA&5+PtBXcq1Na?sI}&+r_D?n)S4a7? zThO*rSC}$|5d=;Gg#t@%^evLH=5WrY(`kE{M#w2Y+nVA`#-<~4iDs#k%*(t#PfJn_ zdar5PJo|l?Wrj-9S1;7PkrjDmyWn&;pYqq!~YQhNML?RGy5g9yN)zPG* zc3UDhA<%(MQyqZjMs{!*$NvEMh43Vy;x1e}?gyvR3_~9hwA`~Wu}Gx}p~@TOgOCxMl~UkGZ3M4`aA`_j+DDk&lwjskm)2Hy{wo13mA zVNH-8%6rer-peBCKHDT|d$!U+pMRX0^r!#Nk=62gRn>8WB z$IDigD0R{OZF6SoARbva+h=WqrhzYalP*sdBnBQmAT~>9G?zKwJ?Q0#tYoxTg-8LOF$tZKlki?vL>L^bdXYdpy1@G2PV2xjN* z>uYx;hB>MA-7|VV-5M}j$F*?UDPV~+lH=i6j4V>eEuHa{TeG`d2n68j39D!u;HVgW ztYo*4pOWY#vdANwnAhRmPvW!E>RZj)Cd%X0+Bj&p=V8azRdlkaTY*bYawAIiHO2L_ z%O5dXeFk76OA6xPBeb!oo?dGOpoXu6lo3U5{4cmF$p!7$Xc)^*42l4aOfnX(L^D6;2qDYk&DC&8?FftAOj z!DGEw`&y7r4whs`t9AogQz{5QaWOeVz!q5NN=`2cY>?y&1FS!Z`>FhUvN+0<;Jqg% zSh+1X5oqg!xiamzb4yc2w)_wf*!(4&%(9D!Te-;Yj&r}3C;Arg+k;qD&P?AGm?0zV z#4gtC?#X*DN*q@Ks~)Wf@)YfL0k9^*rcoNyGa6|`h+I(Lc5Aa?1&Dm_rOdbQ-q$swt>ZADNuvgO|+C4nk`v_zc>s$y4RiZWlSmYQ|RH$x6 zhjN8?ZkLc#7heXmQlLKHwxmszg>C%kcN4G<-cvo4Vz?&5nIHdw|Gt+}0PZ)k3Vk$W z{Hl*P5t*R=r#ojI-s-9$;v@RGboBY6CHyE>Q#eQUYw>zduDxtgs@VYs4%aNaNX7*7 zEyouB^W&X5GGoGj?i9;rdq6p`QGltLTSGpW%Vk7XaCxsGyz~)rw7Qlf+ZT+PDt%=7 zZw-&i8Em@#thc$RpT5ad&ReUEbdAS%Q0#jqnSB%&m(r_|whq?fEAA&=zF;9+t3=%o zQl@3AuKbW-RfE@E5E~Z*%fAqp!a`(3jzTNVS-f7G@#A70X;hZGn?sk`LUr}PW6K2Qj-6!7ote7Fy9_c#dz2q)zg!4EX|%DT%X_ooE`$hSz)&gI=)Gsz z%w>wb%k9q|=A)xk9W9Ep6swgC_5Q$3Zq3BuMZnpD z4x#=IyOrD0UVcE;ucUFw(jtd&@eNYw$(PaFWjc2T0fR zb~*R3D<#wtE65eNpLWa22OaAB8?LPbyAUTdfa#lP<^8{_>4tb@**xr$%~^n8lQJOh zPM)8>TMHnU+%;lJ_LZVh?Xu)yMN?|uL8Bg3doJR^x+wGW61qqg$XUQ4aiQ9*6jK49(W6##~iK9bHUX56VEJR?&URPN6hgavOEmD1Hs9^AeH-HXV( zz6QJ@LW11Ca}Hqb5LrCXz$C#8@&R{fF5sM6miEBz~vp*Lz_PVkwF9N9<2i z|Lj#2Xs;sjnFz)hwRPkixZGEut z*+jj|8N&Vq>t=UcdIW~H$9*xNH7gZb$q>aAWt5R#gryyPAj%$F%p{M8z3vI}{1pR` z^hVj5^IloT-=*k8&cW8UcrU~K0-9wwV0M8+MOQ>!4~o~Y4xP?Rvb|o>_8a6%H(AFVkt>Tue^-E^&UFq$@DU5uTz?~@R^#Ofr-5lfM8Q?Gk*kxlyu8ougti# zZV={P{0iAaJdPwj8vwDSAbNe;#>3!u7~z0%=>D!i=XN&d=WY&RS8|X2-`u zqcAFe>BJj5fNsBOq=>n9_s%lGZx3#W%}=yyA#v_ipT*2={jgQ`VY=fz)fJ%xLjn&eWU z?xN%&Mk*fmw2g!ZYGr?+&})u|H}s&ACU^aOE51lKcJFYIf%9JGTI+S{Fa)N0pJ<{s zr)5GiVUu<{=Pf|o7N&7G-j&o?jN7hT^9had*<*sktd~< z4a&uXUI_wH^raHsyQiX9$*CByhQueYiku;Q^G`taFmH*$pdRX$$hSV8ZC#X;lApG> zRy*3B##^X8XrQLOPiFnbsA0f|Evmkt6 zmjpD`&<3J2`V_CQyxfs712ebd*2C^lTDMmiQjGq57NNU`o^rLGNjx$)5g0o+jP{>2 z?H3WDN;BN=Xj0E&Awfv$=bvSby4`kFso}x1!37&2SqR?cc`7uQvtb)&dguPEFo3tN z=m;QZZ=^r{eas=Cc&Pin)VLTgI+Q(U({n}S4-c_gB-+epNmp1-!^gD+~7 z1nH*LIm94}ytnBt&f0#&7Q{-4C_J61@_joIL)5wJj={uu7wBv)@A;tQXnX4z25oY4 zUWC$Ajrv#i)8<`TZfu;zV}Wp2jsJB7lezNdm*Fy z9zkX;TC@E-rYP_2JiV=+efsLy9V;D)APW%Yd%Z6F4g@h{8Z=CUh*S+JLFliTK9l2b z0;8U3Xci%5>(a1ZM@{?hr+>oI;Gs51L6BK&Wkc&9pZ$(!`cXkaYvWkke+N4MpZPbE zA!9uU84TVi$!71%4_o6!E7T`HgWb_6r|+}J&fIJN5qpcF1I*^iV2V(QqXqFJe5K-c6q{EK1=jGl9Jc@UniW1T_27OO<7n z+^&0{NQyVD#S$QiXuK^MmE=O^gKTpzoNQdL!Jz>%XiK4R@Y$j6cS}Ui@1tmo7zZtW zNGk@_&R&lp!@sg6l!!$hfkgaV_zttq=*Y+opIi8g1$->|*}L_M&@_i$1O8Ee)#zz9 zN!4a(<~E4^SE5a6=jaYN#ZWSCR$tiZR+=h7l#E#OIqXPE_m-BxSENuzh%BL4{R&T6 zb}32$IbmyZG`$>Z1upgk;bH%h5%bU@qUZ5GX}cMI#sg%V)3A!4DyoWzvn{VAI1=Ml zRO#3dn$CMS`wNz#C3hz1T^{%vDxLMQi0Qg8!a7W0+_*pIxDn-)b(!ad3t5I5B}&>h zO#1|imygMSNfaBKGs{5VM}o5m@*+~`R}N0iMI&My`blbE-`(jE6!22@a?Ju?mPM#H zy;1%kL+#f?Tx5Vi`H#x|$5JSDk5-6M{L}SuL4wcn=KVS8UDSs>Ni5(EsnK!r9vLjG}^ z750=<(g5+_M8uz)`68xahEYq1Lq1+w*=e&?IeS?aN%;$`D<~54KVLsAmQ+wsNEEbx zG6hwf&h3NnJsDK~(qhG=ygkR9#PUD?0R1^fKy!`9+hzg_A?cmtI{f>$_27w#36ex3 z24+-7(>}6#DYc1WU4m!WQ?jUkY+44mE)cCI_XHta{8~W{C|NBZ{!O~DD+VTb%Tjrq60}gQkKPaRRN{li` zEl<8INMMn2=PY|*-PSKd>^Pfz!kzE1oOje`B^0g|Z&`Az3qKOGsGU_{5yedtgDRX!hJx9o3=mZO@s zZH!~lh6Bc68W8;=w@>Z`wLe93!lGY6ITsv&yhTxsDpJBF=mfx zt~mkYsl7!hg-F=V4#dsiCks9js8LX~24}(v$osFw84>`qbk;fD`0Gzq62DlO%wkTq zb?kvJ#x+G_ArK+rd^Z0cM=VbzgX%FL&8fS5LBQC#hy|$ISKR^kYa57><3pkIk;8=P za4%5LvlOTO7#(6PJl-rkSpYaMlH!7Tf6PCN=qHQ@-W;jE0W#Wu#<`gdD?vU0&Q>$^ zI!}gHk_XJhKH#Q>BOsx+PlDK|9+80pDsC$TJEfs`zVp^-C{rr;C=nw@5-&*rSK`7> z5Y!SW^j5fnsOGz5B>rNnzsDw-Qj1hD`EHmF^)Cq_)Av;*?itFGu`1mJ{ux04Uy60b z13?h&6~;Ogy*wPXZvfoN&r=MQW=Q>BdtaWRAd19c=t_jSYuLCS^MULXCZOMZ6r3bI zebo3j19un)R72A%7l~i9N+?V7bV`$4wQj`d#+xKyarGtAI_70dmj`i!Y(;1hezDO4 zs98p1c8i7C2UEI);F2QcR~~>03S?AC<+4p{ub-;>=U`s9fvA?{%tiiMF%*0nLr5P( znc2vY%#ToNuKRTXp`=l+TK2u5aDn+c4FuF%rbCNL0&~1BX~2O4-c;w%!#m@&4OC@w z=Gozhuu`ZR88-G=oJYPtT};H*>_Z^BP5gh7a&KyVji<)WvFd(j3mq6FTi{p;w-w@% zQLmfoCtDl2ynzC6SmgYy=T1r|qzj*c!?j^I2STXKrX5k$8YNL&GIJyuaMIj6M5qj= zzRM62O@T-L;Y|F`(D=_+6Js-kE2+G!7yGp|80_FKtO?8-dF_fRP&E$)P$pcQ(s)7E zDWG2KHY=>Ys_S;*(@=!q&70MLFjs5t>h$Z1@iBHQ<<_GDygphDbHFTIR5i8!SJdjE zO~%qJ^2=19F+c>R+qh1*>Ciq@P0LX_Yfb?)<%f7ZThf{Z?{hnVb8lbQ9bi=!)7@BZ zcba!fnK6zOTeemkuThZ5xZIpstwp@fq(Z>8=?znd!Q=t zD@`mJje-`)8Rzoj>XHQqL$R|51kx$z!PY)as8=a@`4JcdZD9DDB(jN@qjRB#YGy&e zrdAgFaq-Pw)o3eTKDp=la{G%-=072INk8}va1LfE2gm)A@65L-B*yUKSW6y3wdWH? zQARz0#(u@d^@mX^MTI5W1ZM*wkJ9*!Vj~}{`=Z`9lT0;0i#&@hY>{j?+2-W5FEq2r zg7RrU6|5gBJ}{4H7(ZD4Rt2~Gco+BU!;EEWb3x95?>!*KH!3Qr!GJoLkO(=LBqWYF z%`s=giSy4bbQAbDdVVe{_#sJ+K6Se=GIK8c*Ua3-qX{_xdCd+kE0Ws0N3=yj(prkH z7_dUfD-k3gd;ENtsD9S@1uI$JcT@=VKgL`9yBi0~xRGVp`ue z-)@n;I%5i4Gg!^G)$c@wq~APYxq{}it$Lf^a>!5)rm2gSh}*_=Fnw-sBk^viA!aSm z7LF8yd2{ka{B-1EO~ne>#)ypZXkBio(Qe_^c+{ zr*qww01tc{h>?P1J4lF4ZB~66OJeSVDpkv^rz^^Vg6u+gA@Zpl^Y1GOV*DYm=C&YEhOl4nV$O-cZ3!i+QH$cn|7!IEj>XKj?-IdOet-7fUyF%Q5mdU})g3QZ8k zY(vQJFez3%+rzfk_5t|2yBQo+7a1~UI12D1a1M$vKC<@4R3St`^`=vw(WUhT4%%Fxm5_RoBTicNYDET>e zW3e)LkVgFFsUpZ3n#KOin`V}@_=2M9KS-FnL?nEcnxed=<{XdT3tkZ&>4>pIX@_gA z$%oUikz{%b(2c%;wK+_ye*O#K2A~%goMggI<~#6~DSXrEvCCK;1e9cDUj#U5WoKV< zp2!hZIUY8d$?mY1}M-K}LVay9;Do%SpDdte?{_hNO8 z{xu#`Js9tGhEXJxiU03=IWQWwFu3OTVf<2G|J*afz=tHf#Cq=ezpu*_B?9)$JCUba zU}gSiyimyU-Y}b5(fIFI04a#{2)kQjtV8km(tr)HD4rXs$!M zp&r^biHMhgO*!B?E>2r8E-u&~R($&1}N>?*fY7*q{n<<_JB z{?BiKs#jKIsx2a6y%dza@OzD!GgyB(t$nz1^mn`Xnc@K}1Iz3kdL&mr>RKY7lG}lp z2}-i+FOVI>t^KIKhu8-o5X_&Lk9_|}hQMC?0`|s%H)=`qzxUezn{YeZmXotTTmXO8 zPxAjJ^p2C}%XPy`k(-cKCK0qV)VDa^nO;gVOGk&=5VVyWwLu{zq~1wjoQ>Su0|rB# zHK4j;5u2P&gWPJ+`^B~9(fAs0;Lwp!d9U>B`;+;MhF{Hpc;{fzqncOTpg8ig2vtS)ZDPK81w0u+Tc0v&1IvIzhzn(eb%j+x-O(o~M0 zE*uj&?^W5V`oqT2hklSn^9gkOSQ1(IuDqyl2~vS(iCo6_1VL)+i7NaY#D~pJ2Y*}) za+-8UNlHGMsj@H{d!v@cf{?N~njZ|&L|9jRWXSUQamA+{Lo*1dnx0$&(bfI7z4eyP zIXC(c&|=h1i?wo+Jt1IJ#$sb*`-kU}tl|4oq|&q>1E9vfa9v}sngiK`w5Yc zH+YSU@3^3NWkE6{`5?0kiQj~#SpQu`RlvYr)Mj-5{swToCfUnJl|_OeFT}8itBYdv zbC5zjE4oI$GWPS`YHh5Eay!s@1cEe%A*ArLOr9duY`JjqgTNPpd|`Ds?{+7la&KYHm>mf_s=Q8z|?JTA$f%DWN8p{JUHZ>Jd)BhVi{X}Q=j!%ABriUUe$5OVNc}1UQyNnoFJSr!;>4};Y3iaZ-mwU+_H^yhUZ)K z6CO2Xr%lZO9Vsl@Y_gKF+yfN9JURJR8u22g zzx2WEcJHy#O#ZKtV8k0B&nPip3)p!|C*K&BEn*%A;EHd2%ce^^2CSEF>K34kb^kXx z3VT_>vyE&Sj48X@4B=hU0Rg@!gIqOAazQIQJ&UcdP2tI5@8^!XmUhFLg<2aEa=HnA z2}L%u>%VkmH^x-c+5POtyDZ1^02BKdwA7gT{D45jWDx&E1CG6m2HfKz+rqJJWEqO}%$FXsGj(67K=gK162(*C zcZ5ihlumKl_QT0-bqaCjJeUr{^F+oHnlIKyxJ|?BK?LUm&`!F1#-XeQlrB@WqQH^O zogDWFo_uJvGgcoyuOI2O%}cIXeaQ;w)YzYhK|TPac6ftZXoD$k^mL&q%3T4R%boXt z+Gq*t4b#B7UNr!UI}`R?+!X9KUNEMkg^5+Nx01PxBnEJsox7D!M_fl1H3RIJFj^_upG2y z;Ao$Ysq1Ma?DxdyUQwwj_v+-LEiVT#222^Ba_F(+IFn#Greldhl`QY9{fJr>LYP)4oda5 zEBe#j*uv?r3W)~DvAN~VNblR<{v8_4#6?MN^y;`92cDF@rYzCU=x}4$2&kcA!3JZt zT^2;DG@=Sq5TPuY%6l~Ff<62CoXF9Ig$o78tyVr~d%K}@1TCaC)X;oUfo<=wL+ zz5)a^-u0L@g>r*tZ!WL6f|RZZO7g_7OkT|2P_8DMulQ`s9<`W4Cm2Y0>s@dI7;9o-j$p1c*fL}9hFSk z*{sFLpH~oq+1bYU!Kc}cyX~HaWa3`FW-e|s!1xr^nbP0+f~Dj#*v27D4CW2EF^&%z zp4&VxFzOmxbt0wjHvO%;qLGu|d36L{gAg1$$;u*;_UCR9(s(O}qbkoFr3zVgiy{WN zc7@qx?OVfrG0*pGX=Rdkx!h$fQ|zN7ic=n8bfscXNSxj3!E&GgOj5>Y6_jvLC0=$tOo+JBG#xy&9FMm#wQLq7fVpHC@Nj zc}gia4U`3LQ4wqaZetk?^6#A{> z+Q4tn8JSY1?yPp->-LUvrwbEU%&xSE#nxB>m(@9O^Xc7Fhyqo;#$MQ!vbtLNldEqgzj4m{jZBJlbsWnIa7#QiO^8xA^0pscj8hg zs)5TK^QJTN(>Us*I8>u15LQ`vNJ6pkIu=8Un2Nx6xtD5iXNtR6PMc}syi=eF|JkV3 zyW7`f7B@jq1q;IUyQsZa;lwg%s?&{>#CFb{)y0^?Yp0uqUsG|Xz29zdz-llb*OkLi zW|dG_y7LS+*|7==0A7Aks+YoN&6YC%Etj<@)&^cW`_@><|*(IFPJ-~17jc;owBfz|6G)G zy%}IWKp0bBwztNoDEv}Rz-6Y-31LP0hE2=Zk(f;i=8MK@amDkm5b94g(+t^z%}}OB z-t6`{O7g^Qjn8Wd1`);;pMWCf+78M@G;m@@do{795dR^OEe>Nge;eOGu^A+461otgzvNiwEb2)Fz-)+&j(!~~q# zIiQGpU!9R6B}SLxN?;~#I1vUm8sY`6FK3dK>XRpkY;KpQXRtPB8yuL4NupjTFC~&jvrv zCUd75-EGW*B>MpDk$l#EC;!psX9m@{B1L6TMjNC63BW<$4|U2()Snl` z9}^*vPYj}33tJjItYjcKD=aDvwb7_g%*V9<#GQm{!##7uge_)Gy~J)f4AaA;duaYm zZg|R^mKl%MvGX~np1_x%(Ge$DrZy5ktZYE8Y*Aj0%nWYJ11w>K&& zbTgL_`8}t(7CDVeQ$d?TE}~fKK5+P`q~IT*k43?x;n;p52bnT_2u-(`4{b`dNLb)+ zwKqP?xZXyj*QecN>Tbh5D~)#|x&=coJU*)uEitfZ-1(Ms28b6k!oq{0+7#2FA)WhD zRNUbtRY|dP4kGpUxkQKpCWn7C#7nC(IUDtsX6SFwKNg28^)!7fT`)g!&kzM@pW%?N zafMHL`5rZv{7#|i`k7iAV&S+0hI%e?SA5i(Wr>QYj{{ehZxX z5=&SS4{mRQM#AaLO)3C9(MSzD$3$eRjZzsOqY>4l(zoxU-W`C&L#0nqr}skGa$ zVZ*R^OQcJOBY*s><~o-dS&VZ!$#!B11iB{0${>#WHIw7gC-kLGsy}#{9KQ2!^j-in zrHf-h^W?x;*rV_55RK4pD56TA=-o^P5uS%LIL;5S{Q)MhsEu8X_gZ&%Ru0QU3z1JF zX(1mfvW! zi}jqB4|Uxt!rn2-=v7?!tGhG~?C~fMOuaQWt`XQBi@<1&U5S)vnzccxfAEl8G0}x` zcNmo837L*KY2wBoxty0*UN-!!d(Pzd8wgG-vk?)ixH9MR zgakF)bo&Ig2Hc@L0%WB%a|yx-H@Q>a5STj26=|MSaZHGS0>PrITt9b!0az^D=6vIT zv6HjQ-(w>$_nBgUeq*DIKL4H&)i#6}VH)y8?4M|&lS#(((J!wD7Q`%!bGhLXIP?+E z0w2|nyB|2HiqB<1dXpfiHgW*1Batlw zaIuZ#EkBHh7=mz`6gXFb{T!OTsi9rLOd^`__94Wy%0Fi?B{7?W!+QpEqa2* zQAJoks8iWx!XHO_uPq3<=XZ*aHxwM!kpr@3yDXjKD|DE!3XAK*pR<(ilLuqE=x?lu zq}M3(#LwGO^Jf=yLBa1sICQo^>9^!9g#_yQ)Es*&66`}Lc}BTvzi@WBO^4~!sC1pBn3~R zeaZZ`w=3zv_Lzf$3)(a_^6PS%%3zva2$NZ>RF2MZrpeRK&)iT@G`9LNX5}H%nf|Ci z!+QhJuk~_MonJGze(~<&3wDdOb=n5+@4h|LPv`%n*dmdEZMZt5dmd_SlPB2>=zgDw z1imybz^lvJtp;(c-2AhP_+yTzuo>5Bc~?e_cLKA))AjB@a~_%Jh-_8?x2gLZK3&@I z_OWcf>>DSn=ZSu&p^#zHR9tuPfeq=#Xi$N=!MalvvKUD({2H1rx0p~PPrmY#(iq!|bA1?4R`x>~$@cy_J{L3A45jr^M6nn=)KO_N znJ?|V(B5+n;2~5EU?jl+p{awq>s>u}ZjAKmG%7q15(u-iaK5id%tvbdxq z;bF8*t4YfuBDn&ic3*%|__dG#X^s2#Z2;kn=;SiRTtOeM%X%Aa+VdK}Rq;_8QN#=3 zp`+r;kr;%Of#bD5yajKRXuj;S_}@ad9B~V*rgR=Gy-QpSN_v~C)Mn(6wJkE7^YO}! z6hBxSRYV-{|6lFBWmHt_8#fHYfPkccf*`4cbTLa=7e=qjI&&=xN)A9_ykhn2!_m<)ErPBX~@+!+`$|Cru>%+8QX^i!wn(3)@;& z^e5+8bSkd)^<46ME0H$lb+XlULt`r?-$xXU@U7 z|MebX8p5_VB6W=Zx*lUaxL#A1DZc7oSkxDK4r)h|mv68KY5U5I-j;jst{$}cLRBeM zb5yanNyt@`ph2 zwFD(9z@jVh9TlBa7~aNMR2?t4uk1!RX1(gc`dUeMhA=;doseEHq&i19ViA}ZWVBLA zqyYxc?r`a3DY%0eI|xD(^S%8h6~*X;2UZg(0?1|uS`G$T(3AMWQV!f?)louw-$cv* z#Du8i;GVWUou91h*dT1JmKLyS5ZoI)CyW>lwX`b$PwHcV?UC3da!;AJ$$7Ji>;Fb> zgZPWP5bkwW{Z&Ma(_===o~+tTVT9^J;|b?Px@bP zflqZtweOMZUcZ$}ML<-gu)j{?(v60lC6oVgq@lcD9v*1;Q`|4F|CMZZhh z01twwy4rxf2sy_$ff6t2Dd-aId0g+c(*)6q)Lg8}DexE@#Q*XM!ylo4RM z;Z{RtzYuzScqUL>Zyn4(3cLVM|(D@4-y4H&u zHw1t{LO2*w+XGTc(bV0npZf$D*i;EHoZRUa57kbG6G?e7Sq#7$-xiaax8|CgTkaE? zITgHk!3z;2=RoX;3knai0-#3U3`l+x>s|o#nuGW&Dx$lZDAF97~P?2x|gv*Z*dv^Tr+yGQ*z6X9?I^JTx1E zYAhr{yd^69MF3V1u$+b{TF6a}->@hHPcqYj9Jx|LN#M-i`7^DS}WZVTG6ETPT zqHNwpkRb0U7mI*4^0>t zy_P6*IbaEq+UE!p2ccIoTxP?*jhce4bq525Cu2*@(pD9pnKcx z6h(q2M79&`#fxNYk5S5)U(5DuDAW{YUULR+0X0GIXCr+ZT5KWY`&&|2@LrldAl(-Kv_x^bECDB{`-#bS2Mk&<-~iFo9!oc|!RQtuZ#I@t2C=@*om7M% zBLMz4zOL~*L&Ab@R~FQa{yNaY&%UqTfm|=Da>$>D9ZqfU}zg)F00wov3%tHrGw!qo5!D( z3pgWf|7Pl}NFV{54qhvz29cG3)ynwXFc4fd#8`%#zJu7NV)QFgiSMDE;2aYrFi=?M zdzanv;36wF@XJs(HtHsLXZPn;5tV?NVDplDbNlCZU5*Dke~B$J2%{1U#+UI3khWjy zg)qJ4pK=|53_gM(N+VDfv!3LWkbMqC!6Op^?7s{&Vg}OR+|KM`xbCpwYuy1&*>zx~ z1g(GICIm@tL{>lK$?{Krs0Ja(SRPnHumS)0TV`s&-;0nsFS=srwjOVQ4j0ZyI`z$oWQnf* z?JyQX3{K?sfmMmWDMgu(jO5$8j>li;594ES{zzut#+U!=6#T*hPQhNQ-rBz@MKlnl z=u4Hi>aV-~=PZn|0tc>_n9uM}Lf5}9ip>O$pP0g-puZi^L;&BWV`r>#^RMfv-UQbR z5qSnIq|Uy#paT|SW^bn3&!j6`2I+S!_Px{SwYOVTnbT<`w~yX$rT#r~^;op_9Qf(f z;$X^uaVae+{&}&fix510&%w^luDW^-Vr3#TFa9L%6;^-V1IrzG=FSEgA-PY@QrtC# zCLq1Bz4z!TG}^Bj8XzG|0uYq=__%a|J4Kv8(ZJ_)onMNFS{6tnbP!|>kvmR~zL|iw zEi`V@D_>ggK}!Kl%zxxGcK^s}Nc;A39{$?JvWsAOGEos>tt1F8Y7D zd%u^}{|DSXw7Xhn%q56R=!t9wmlJ_;7NEN}11?nEf$ga#@UliWK%q^xW*beQ(_*yT zBuYol?jKjhJ_oS&rwVpLyaU?BB2%Du`=aQh3S1;nP_4x#4kG6(qqA|deE-C52&5@U37A^FzA|>g z`-d!Rv2Z8^%I%Bsq21&o<`!9-)s<_$dnC&%A{Tz|zQ0`<*}ujPCvV7S?yvIHpRqz9|Nq0?!w?Sw@s2y>ecYs>B098@$_A2p$mM1ZP_j0h$RSAx z2%H@~0h}(Z&tB&QqT>9pN~_-JLlgf55YL4HiBWk~y0FKTo0cbFB8`hIqx~N8KMUU) zvgmv!+bo(25~*b1>H;NG)oa85@~FeB=Ctn1-U5bOp#au}8xH8uY(Xu2!k{U4>jyya zTaST<-nnhg!=n!m{fPVLLBtYlGf)r$nibYmD*ddT|M^BR4+<`>@B?vH-v=YF&>I?d zLvyq7;*AM!7$HItwcQf%n_F}B5Z&%>-cdCJ#mTFr`9q?ss7c zIg5a~Zd>g~nK`R{iA|sMRLlTyxnL-msEGR$^ql4-&`D6$5CBmSuiV)IZDY5b=ZxClC{?d;&a@c&K^Qt%v(i`8$PgjHIX;YJ=9V@tczdWpVsKdhUv=YFL>=3$;ub$>zrKe&k^Md}gJZxK{o0o- ztwE^@iRwo|Bj+YSzE@zELnV(6nU(eW!%6#6xOS54aw_G13f3qRDMVNRPH%pIB;QWRr4Q&4 zjX}MjdZ1n&8X3nwbB`NCL!SI(MmSPIiaNAulnruCz-j6miePgE_EmE~O36%BgU;DM z*ifr4q$6zYDcpC8=F_ZQkuCQULCN0x$qvgN_l%f9y^*dn+=>4$}hkvSY>N=`!={8*mF|<3B380woQ%D*(ew zYu&rqjuEmW465r_3d=?l4+SNUH37qRSzuGiKEZH4kmV5o9=Pj@T(ca?e?ZOOklo=7 znVz)ogKrQZo3`^m6<}R2&$6F8fchVdTC4!2)Om8pdgn>Ji$~VIiJDX}7T0(h?WUiG zZz6kuwmI)`uL8JrFmNw{hJO)l2CUlg2I4+hE)7&6#kGQB_U*1r?7bp8ik?T?ZD`0m z(DJULD!Ei4QhMh_neQ`VSGn_8CyyNsxqRFsZ3_+L))8Dp#cd5W z$&kSrR(0eGV63-#LUNLo?ZomwCN4B=;#H6dWE<7gsT&YC+~WL{B&H_;@hI4hH z>AUps&OwEY-tFe}!>6~OMTqx{@4xCrZ)Ia*j~uxRM6@lXfI(6C0BR5Nf|OiaL^2oc97qa6#tl_-EZ^>7XZE*>EmP^4+379zRdha24g zNM!dzZkMLZ9FXPaApelI*hFqs$5cTkQLRsj$9KEo!I;yo!%D^de4fz=qL6GQFB2T9 z_xJ`rq#pDnfy{7w(#jBvA#5}}qQ`S0Toa@r$WZ4Ty^ASQ@Mn}>3$ZmUJvS3!{H{Bl zmh&x-F0W9*+{o?ACF&3wFC)4sq)q-1AOCDzqBN>54WZen;+78~coHSdFm1&B7p~7hVBK@q>#78TZ2jFjJ(7Cow(q z=NG;)m)6iylB0dP%K{%gpsNY;+Pi1D*0#MPHan>Td;;GmoZ7v0p4z<;VFJ<_i@f<3 z!DG`2cq=J>sN`{bjqRYMafOIL$}?uv46PC69VMGVPEXUtqcb}TwH6EwY#%eXo^coi zRd6$G=VhjZV^(dM9YIcM>ljV=9;aL|_o@<2jWA%mHem7#dPZzE;TmWlch+ z12P`t!$MNjQsy^l)8IsOX)c&|#~>v!6g=d;uS}&e!c9iqy!5sJeHeaAPO-)XVyc&K`jtWK2YLu&05MG*G(T3ad09C z6XpauY-y>bwjvSO$V-8D)5T#o;Az0|%?Bv+Fk_tS%_mhcT?tWywp)mrQO{AtJTa)f zmfUt2)PT1}SkxGE$K|)a(sOk z?5W6P9vK3OG!SOJD9FY|MXzwhEFyr@%}QS+ZoI@5NZPv$U{m&u#97=ReD2$XCr}e+ zeLjoMgk(bL0>$eP4@|_!KB!61?1@w0ZF6F^B4#89aCaM%gZq0L5hUj+T$Xq?EfV&e z_VpJ2a*#cHMS_4#0Y~+ph4d@v?$dywoy`0=^f#CWrqdN1{Z$_#zcm@X+zLuOrDAdE zRIE9PM_EX|#MvJJgz(lEa%K6#N~`MenQKvcmgjN_G-*lIieok;se`}oB;k&hvk2q5 z=8tH3Z$i#@L9|5Ww5l!cE0<*wu~A96;-B6pj#!RGWwq*k2BL;SEF@#m$zK^^$ z)XhLSr8Fcti!d>^RLz^mI*E1mI%5x)6Rx^J8c;{SowSV-P$#V>Wu)~9v!`jr&%Fb; zV|;}1Jo^0QuT+5#AalKLwmfp;sj1F$nRoi0=$$b|N2X)qtjO=Dx+)$CvcW-_F z$*?M!mHh^t?S{I$iByfmz|b>_qf`!?((QIqf|@#I0&RJmrCh|hTg~F;tNNf-?1$)C zd2C|mF5_OsogV79=J_#z^wgJs|G5`wS*T)?X1gnm9RqrvXXYW~lb0SFQEHn7TK;|X z>j=%MBR>SOM*M{NZa2^s3}16d3cz3FfBQ`-ZPzoJsY8SMfnKe-D2Vo+lV91_=!v+Ga}$3?sloDA z-V>Kkf(r1e0qd#%$iF5kfZEv3*<;dk%yLKsfOf&x@Td<#YS7A*J&??xO;|)z7mP#h z*Pislx@@U~x;g><{6$T@YtbZ9)tM0Huqh8GZHEEol$6X(ErrzGLJ^mOd%`wex^M4! zu|DKHg2-VHgKU~A=iiwdsqkT=5?45pB9f85%p`%=BFUtjazu=SMgSw8^c)wxMsTST z4t++gz}&&a%n4r~?;0c&7$J!`<}BJ`0DKKn{aO$~nqlBJUVW)VqY+W6>H6uq_`Z3* z>OQC$X^FnCiN*L9W{j(&R3FwKX%wjk>XZMC1hcE*CuL=dQ9jgkK@wrWhbtP#z~X>V zp%oX6pb*-u>$c{z?uY@e1&Z#28%boXbvchC30nnJy)DLN7!tOVXa3wS|`5TV&3_;B2p_ zX^O_@^8@Cb$_XU7;@1=SYELF?N6XAIh`+%8?~7tdzucI ztuowhr&0B;XE~^!;}XNX5wk7f4(Hb8M6hdwCS9t~6}V`qV2OpKUKIrH8a3C&`>`~) z^Nzu<(V4_UJK<7w?5xlBbG_1ON3+#}qms@LsF`k5@2N;|O1d9#OM3XBdHXJ9^80i} za*PYgsN66a_)O>d6zz2XX$^P?+v~6W2bnaLMfGc%-|W`5llLj6THCX8A85&j?4xtv z{cY6vXTED|AyK=%2h?)9@Wr(UYIt2^i+L9LdI#LXBOicd{t=8c(nV5r0k}@wpGxk3 zgIU0Fz^Wslc$~pn;tz7r);qj#9ZKMG(r=34z1Zf*#~mEvx^03(Vp*DdsIdx@Ucw#F zB62DwE~mNa)Gx>Ij?JJ~{KviSBjbX{#N{S}mE4?B+$lmf-u8QPC_!WukPBQbI=O&& z!X{j}J)5TQ3$XTWZA5n1(=vyCYGHy;w1?A3ojS>(DHQ5F{fPbJ5T8uIyo2E8y#9r+ zIccX}at@d$PH)Jnmw<;wR1i~PsX8RY0@;`gMSR$EpZ@HDv5k=(zg6ru&bX-|HMt;7 zkE-auY>0}avlrhb8R=_uMU?(6&j*MHKg{5AbAjWN1;byp=|6)CiXBTAt#u}*I&jjK z{O~=$o|%E*!^Xg9foBx!xzJy40eTAg4BFo>BZ43pr<};t?eDJxMXxvK{(c$QYythM zF+*kFcTK9if zeE#(^7-i6xWj$E?4+8tobsGYrKL5V-zkU;tL2xFS2)Pn*GM#9DMC#P z6~1=+jEP=$8GMC}+rP2t#gygk76ZI1T;>iQDKx$>eD*bDk@*VZ+T=CgsD~jbxyeE} zF1f9fDH@;KJ(n80|5j3w-?2E7x9Yvle^gZIX5_hdz)>eQ^x1XLqSy72ZiTr``nLyU z1$^VyHz?9;*d8%W6r-kb5m5bsF@LTHVxs^y&YflU@N>1(-jw& z4%Pu5=;kj(2El)HuId$5(``_j>4a1>U5Nxd(0AM^7g?~7=ByMrDx zXFqhk0i_vCP+5M)AYNbjtOps}nX|7OeS-JpMIWH1d9T(vo*5l6BIDINmO-WgCT%63 zxc4SE!uK&g6HYVNTdm##_K6D9$7hD(Yf1yupQS&)>*Gl>XlR3wg?$oj@(JBn7L5Sk zP*kUK_8UmFYpmraXZmQQ(rfEf7I$7%BGE8PGMH>{mfZ9H{B<`LAbNJr`+qngKgXf3 z5T-b%lz0PjIxIJ#I>!7wfL&9mkG*9$C)(^2wy$b*360N}b@Sxx4A!J}3%M+`kY|&2 z=(@bgG6HTdep<@civ!q_sZaR+H@d>5FSl0P2zoMow~KYo|76*}2{aq;GnQ3*Eo3gc zHPYYDj$@M@0H9F2Ch-|R1{Q7AI)9-~eDM-rMUsMo%$@f>Cfgq$)&}Q0Xx16Lx_j$| zwCMLEfEZ46GzT>i@{;i-rP89Gn1uabF1MV|Gl#6 zwvA0x-8Z}S8+F3mhkfc3pAtXo%xLYer+8RSf7=*&;HAH&Q(@c~*p~ykKuG$Xe6#Xe zEiP1)3m4r21dfh};wOw?Yt-B}E2>%nrSDi#pS^ZK5vLg&8*}&6zc;R)c-3J@WctI@ z6swgfCZ*g!ft=~b20KqFjmmUe>%^ma{SKjr_uX9K`Kv3O9q&8iKIW1&XJR=Qo|p$M zDD}p=Zuk*13keA=Zhg?r(1=b+x-vnw9v9kbdFa-X^^V)>rlh1~%l7f{^4D)^SDCpD zU^*#;Z7p5OB^hkT4$l({l&}17GtWv3jJnpSa)BE4KY!tLuygp72~%|*X+J6++6OLG}K^5E^1k*X@Sg@r})`;*nDW$v$?*mSFIf<%-EY^X0pKtg&KaEw#!TwPt;OAYGG z*M^^gxl`Rzf5*TeqaM_CD?nsjLuALXy}3!r{li9X{ztBX{}U$5Rz*2yr1`mu$66E8 zBP0CDUN5R%%B2;$eZ9J?lpz1dD?VVVF7RlSl=3%ild>DNAsplafz)ceYR=8Z4?+B;tArlbsE5BWq( zOM$%FT3yhFNus#8I2MTh5x|6ij}ATsl!EHFZdu$&J32yMa*UJ;joJ|cWT&@B-`C2% zfQ~)#a&q@~`ib7)=FDkIkJgt1v!2p5C-`v%hL`YP=L#rx?UfpJgs z+M*RCBr+H55__Cb@6QGs?k7a3FfnzgtKEJ)g4eri?I6r5`_wM~W~KFA7g)0a-@1T& z*@^#*M7)W{ZCzsb$;|*pGFE!B=Zq1$#2b&*HXUv6@cu9avkuonF6xUqofvES_i&X( z#D69YG>4F$)B$ltyK5s|z$<_OgsheNAl)gokx-QIx+O0kVJH43Au+Kt&G8KrL4Ydl z3-gW(OziAQfS#v$2jGTYz>!R_i2^!y{NmWXzwHX3SiO%b)L59YO7ve7nJAIzE9Q^>}t9L!|=N;~|J z_=*||4X)?gg{+Wq*Grf^)eL{{zAb{-}uLRk}mdiJ!r9-ygNq7;0;K{^?mFd2@V)Q- z6F#kzOq`OT?VieWu2zMwM7V6zk?M4wjaTiKI=dySe<*Hj$HV+nbr#=oIRyI&Z{^D= zXBN<^xjJ69sWhGD7YJdQ0Q7TaZf=(FYO;Ej~jWX9O&2_FfR3g?mn*j2=T{)l8z0z1I`{+x<%shogcq~w>DmM)0w5G1hR3&Xn-x7d{=?2(qV zzA;hV1;{p$WlENo1^U3+vXqnYVFU?4KB`M}U&xCph8lL7MeQFZZ+IpUZoc9*i8_kCDJs8G`Asi$sU zO7#gA?fddDL4zZ(A-E*ldb{n%G08^En7IFDNel{l>~cwBu^`&a#QU0KXKJ`0oiMW$ z_oLA6lI^z;^h(kL65m>!&Tu5ZP4#~SX5$n6S z+w=tOP8k?KF8eCZi=#a&efqSG^b2iXyB=Hc1$OfD%e^ntTH8I#B5tGlF7VKv@SOKkpC>2(!n+uV^iBe8AZXt54((1gf zvW_b@)EK;)qELlGH2E9=IR4mE-%2GxDGPwjX;zT!eMu`D0r$`ZM2tMsW8h$!u5r!-u#*gEsfU}wY1XcXzTfa%dmF&-VfW>p zw;p=oOr;WhxDm0Pe7-Z2AWDgKsdpMpjKw9C$+5zW&R01luIQSh6#cuN@fVEhtx@~> z=2kQ@5=1!eX{GAZoO{At<{Yx-?)zCXRlO5OmiLO!vP4Hh7B9SKHgI(7_MdRN5*FFBE=EN2!&UsA+sq*+c6@xxCkoNS)1+>aJ;ZA9ZTpk zbW4jY$TUHEWBRq;_{78_a1e08J3kR>gj=iVC4I#;T=MCbwuwTiX;ey)S^OXlIvL?} zd1UL|JG((+yfu~;-owet3bj)u*KSN(+t|ck>{YH>aSwf}-|l4`JW+5}VMKY=ST8Rz z@wSBU)|~11U@@1!atPtpY=(wb;j`v6J;mYy7S>0SRz=_Aqil~R=pR*fN`jvre?gN} zsPszNN&f|=ds(sKqOOx(MhR*(>Agi7yvP~WX_y`6-0~#jXv@U)*?VUnTZ0tJ#|PD2 zM4_HaVb4DUAL%#%5)X&-343hvIJ_x)wRD*xf^0ex3r9w<2b|zUJ0f~_iCp^NSJF#D ztG7N%2&z6_ChxMLQNWnWp&m?*Ofg|IELl`~Ia#DU;miLtpwh|DVx{CQ-N8^WUbIv5 z_r=ErkBbVGbG>`3$aYLT2XHR6zl2o>g+-IHQzj@py4Z`8f-U9rY4g)iSMV3p?rUf| zDdUzRw`RM7DmWsUnOU*x3`UGGqATf!Vi+&*-D~m)K!<_DINNyslFfA3*-c4C&iIL? zNQvR=ako9d5-DJxouRBl@WHRe%UQ?d^eTt9`?ZOA z7HFfQ&BziFC=w8a_Pxf+S0e}wC9>jl*RKc<(*Vrl-G;wxj5`@8L6)J6qu8(wsgEJ{S z5ZCZ)ti=wnF-hBIbi2$(y`Ft<6(GcQ+4t;cALjCj^%qV((0opc*DNlCx4Gp<_Aqd)B#kH~^`g*5W#{(8*=G(LcXdVGE4T)(i92QoV`mD< zMh|y(2Oe?EnY!d*IHbfaliu#gM8Ahs$2Q>M(_Ohu6%0EY@F+Wx81U z_|9+)rJMKWjVmHD=fJr^c+)WS>?5L&Kxk6EX*5a+_w3(vzF=~PK_~E#*tVw2~W8bKPsLpeFo1!ZQ5WaBRy_@UTP}zF$ zs!03!55%pNLBn0fz@+Y*XgZvH+}O)J)`t!^l&QP!eWXGn3L31=6|{z2ufOr2px}1= zF|4swCO(40*lTkVVG}d;P{nhsDr=KxOkiunW^jkxtsb<{qj<1Et($1Kwv(W_U)C?q zII5UQ>>m-ui=@0iR=MvbdzcZ}W8JYWayAf7riShA?xvzTkGp=9Sn2^yN-ZW{I*i8_ zsa9V1UIflfoZM0!gTv4C&@-}gr9pPIKKfh=uq$aFJ$h7X!olJP$o>Y0CrU;I^mv$* zRv6N8p2@;~R5vOtlNz!w5v~grN~*lwB)^W&MH?+BbR=F|=d)n#_N@C>p0Rn!tgz5+ ziIy(4c_Z?z2NHssL5cE++ z;}USf<8l)FapaQ1Z50g!T%(5cU+-TUw3d8$CE*coLZiw4y8(s@Now1#uKV()ss3dR z?E(^?@L~BD?L|nGfnv+L;*Jgz$xbhBau%v5Lb_&L7M(*2mrkyS?F!dw(!|sA^Aou1 zcy?)*ktK)-R_b$zhqJVI%Te(>ktyiK+A!sn4#6%ApMULy;Pz6s*1PK(Ew8p?;S`n? zrB#lM2~xb=-AMlO*6R_50@>G|Lc(>D&4-7Fk`sRR#v9b>7#)6-BX5UR+Xl8M&+ceW zNCNFz*W!icsAZY2Mr6ts?3nE{VjQp5D38g0WbwNAuTG?IX>Ii2A zJj&z}z3TS~e4@aAq4aJ%u=r~a$1?n!x+5_vI-2?I-urDAmKHQjK7tD|SX_jt&oLI2 z8OZ_}2^9{?cPgh__Ud+s#@Jq3@K=x|*zKD6aXd38U;f!f z;o|6XAq^ReN*QC#NW0}_-1|of8=F#JS@8>`aR1*}YKtf$mWdrgG`UwmWVrS3R*Vl{a zfRefOkrR1)=hzFoBu;@)?V!DnTNDzB$|4xLx%2q>H`_%d)*QrPW6XvF1_^=^G2saY zfr1h&Hfm@_3A3(i`mwk~OfkqnHkFm{@NINw^`2n0I3As{hxX*mO*cf0-eu78UJ>(N zugSjmM7>UrCKaPA$5`1Ot;y)gd$gr@j8=gwTyp-7@uH9Y1cx0lt|`lPUL+Gj*A*h^ zd-cL5G1fsRdhON-1^);mD;YuZY$eekZ*X_!%a?c(^thv*!rDlSCzd#42(Jqjen*%k zWTdPUZVPRq8={P(+82(l{K(W4;+W|s z<1Q~Ov_s}-dw~@0sBL)WTlV8s9hlUkQJ37S&>k-LGuE$5l_T3ZrUP+Dn^yRT22imk zi7wESOVBDRs(m&&z@PVgZJFH=K)Pr>K$pNXmSw7Jk+SB|>@s)srNUGL&!SEGu)DJ} z#+Aex_Wgdt(fcA}SS(|Qnov?&e7*~1dks>1XQZ(?M}gL?1DkI&WjUEW)A{B54>CYz zkN$!J$CRLjqD9B|eQhmLS*M+#iE~0`GBd0f`K6SGodQ$O4~y?*Ne>gL;4)w4MEqCT z`;TmRGxwKX$o;q400KH0xmiFx03kVv|Q zserlFaLn=0`h-IOgAm96efY~6!KJJ_&HM~S1v3X8t!wXgRLVr-c9=#ZgD