From d3928000cf882badb305aa45d210adf477e28a53 Mon Sep 17 00:00:00 2001 From: Anirudh Date: Fri, 2 Sep 2016 14:47:30 -0700 Subject: [PATCH 01/14] Adding kubernetes cluster manager support. --- assembly/pom.xml | 10 + .../org/apache/spark/deploy/SparkSubmit.scala | 27 ++- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/make-distribution.sh | 4 +- kubernetes/pom.xml | 54 +++++ .../src/main/resources/META-INF/MANIFEST.MF | 3 + ...che.spark.scheduler.ExternalClusterManager | 1 + .../spark/deploy/kubernetes/Client.scala | 71 ++++++ .../deploy/kubernetes/ClientArguments.scala | 85 +++++++ .../KubernetesExternalShuffleService.scala | 146 ++++++++++++ .../kubernetes/KubernetesClusterManager.scala | 45 ++++ .../KubernetesClusterScheduler.scala | 200 +++++++++++++++++ .../KubernetesClusterSchedulerBackend.scala | 209 ++++++++++++++++++ .../KubernetesDriverDescription.scala | 39 ++++ pom.xml | 7 + project/SparkBuild.scala | 4 +- 16 files changed, 901 insertions(+), 6 deletions(-) create mode 100644 kubernetes/pom.xml create mode 100644 kubernetes/src/main/resources/META-INF/MANIFEST.MF create mode 100644 kubernetes/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager create mode 100644 kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala create mode 100644 kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/ClientArguments.scala create mode 100644 kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala create mode 100644 kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala create mode 100644 kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala create mode 100644 kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala create mode 100644 kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala diff --git a/assembly/pom.xml b/assembly/pom.xml index ec243eaebaea7..db8ac768a877f 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -148,6 +148,16 @@ + + kubernetes + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + + + hive diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 85f80b6971e80..3291a1e69fea0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -69,7 +69,8 @@ object SparkSubmit extends CommandLineUtils { private val STANDALONE = 2 private val MESOS = 4 private val LOCAL = 8 - private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + private val KUBERNETES = 16 + private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | KUBERNETES | LOCAL // Deploy modes private val CLIENT = 1 @@ -230,8 +231,9 @@ object SparkSubmit extends CommandLineUtils { case m if m.startsWith("spark") => STANDALONE case m if m.startsWith("mesos") => MESOS case m if m.startsWith("local") => LOCAL + case m if m.startsWith("k8s") => KUBERNETES case _ => - printErrorAndExit("Master must either be yarn or start with spark, mesos, local") + printErrorAndExit("Master must either be yarn or start with spark, mesos, k8s, local") -1 } @@ -274,6 +276,7 @@ object SparkSubmit extends CommandLineUtils { } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER + val isKubernetesCluster = clusterManager == KUBERNETES && deployMode == CLUSTER // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files // too for packages that include Python code @@ -596,6 +599,26 @@ object SparkSubmit extends CommandLineUtils { } } + if (isKubernetesCluster) { + childMainClass = "org.apache.spark.deploy.kubernetes.Client" + if (args.isPython) { + childArgs += ("--primary-py-file", args.primaryResource) + childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") + } else if (args.isR) { + val mainFile = new Path(args.primaryResource).getName + childArgs += ("--primary-r-file", mainFile) + childArgs += ("--class", "org.apache.spark.deploy.RRunner") + } else { + if (args.primaryResource != SparkLauncher.NO_RESOURCE) { + childArgs += ("--jar", args.primaryResource) + } + childArgs += ("--class", args.mainClass) + } + if (args.childArgs != null) { + args.childArgs.foreach { arg => childArgs += ("--arg", arg) } + } + } + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 71e7fb6dd243d..2f95e359d9daf 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -98,7 +98,7 @@ jersey-client-2.22.2.jar jersey-common-2.22.2.jar jersey-container-servlet-2.22.2.jar jersey-container-servlet-core-2.22.2.jar -jersey-guava-2.22.2.jar +jersey-guava-2.22.2.jarshaded-proto jersey-media-jaxb-2.22.2.jar jersey-server-2.22.2.jar jets3t-0.9.3.jar diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 49b46fbc3fb27..1a92bd43a1a65 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -154,7 +154,9 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCac # Store the command as an array because $MVN variable might have spaces in it. # Normal quoting tricks don't work. # See: http://mywiki.wooledge.org/BashFAQ/050 -BUILD_COMMAND=("$MVN" -T 1C clean package -DskipTests $@) +# BUILD_COMMAND=("$MVN" -T 1C clean package -DskipTests $@) + +BUILD_COMMAND=("$MVN" -T 2C package -DskipTests $@) # Actually build the jar echo -e "\nBuilding with..." diff --git a/kubernetes/pom.xml b/kubernetes/pom.xml new file mode 100644 index 0000000000000..7183afc625562 --- /dev/null +++ b/kubernetes/pom.xml @@ -0,0 +1,54 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-SNAPSHOT + ../pom.xml + + + spark-kubernetes_2.11 + jar + Spark Project Kubernetes + + kubernetes + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + io.fabric8 + kubernetes-client + 1.4.8 + + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + \ No newline at end of file diff --git a/kubernetes/src/main/resources/META-INF/MANIFEST.MF b/kubernetes/src/main/resources/META-INF/MANIFEST.MF new file mode 100644 index 0000000000000..1692acd078605 --- /dev/null +++ b/kubernetes/src/main/resources/META-INF/MANIFEST.MF @@ -0,0 +1,3 @@ +Manifest-Version: 1.0 +Main-Class: org.apache.spark.deploy.kubernetes.Client + diff --git a/kubernetes/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/kubernetes/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager new file mode 100644 index 0000000000000..f477a24d191c3 --- /dev/null +++ b/kubernetes/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -0,0 +1 @@ +org.apache.spark.scheduler.cluster.kubernetes.KubernetesClusterManager \ No newline at end of file diff --git a/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala new file mode 100644 index 0000000000000..09b492e1f98d2 --- /dev/null +++ b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -0,0 +1,71 @@ +/* + * 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 org.apache.spark.deploy.kubernetes + +import java.util.concurrent.CountDownLatch + +import org.apache.hadoop.conf.Configuration +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.cluster.kubernetes.KubernetesClusterScheduler +import org.apache.spark.util.ShutdownHookManager + +private[spark] class Client(val args: ClientArguments, + val hadoopConf: Configuration, + val sparkConf: SparkConf) + extends Logging { + private val scheduler = new KubernetesClusterScheduler(sparkConf) + private val shutdownLatch = new CountDownLatch(1) + + def this(clientArgs: ClientArguments, spConf: SparkConf) = + this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) + + def start(): Unit = { + scheduler.start(args) + } + + def stop(): Unit = { + scheduler.stop() + shutdownLatch.countDown() + + System.clearProperty("SPARK_KUBERNETES_MODE") + } + + def awaitShutdown(): Unit = { + shutdownLatch.await() + } +} + +private object Client extends Logging { + def main(argStrings: Array[String]) { + val sparkConf = new SparkConf + System.setProperty("SPARK_KUBERNETES_MODE", "true") + val args = new ClientArguments(argStrings) + val client = new Client(args, sparkConf) + client.start() + + logDebug("Adding shutdown hook") // force eager creation of logger + ShutdownHookManager.addShutdownHook { () => + logInfo("Shutdown hook is shutting down dispatcher") + client.stop() + client.awaitShutdown() + } + client.awaitShutdown() + } +} diff --git a/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/ClientArguments.scala b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/ClientArguments.scala new file mode 100644 index 0000000000000..d4bcd6bf289be --- /dev/null +++ b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/ClientArguments.scala @@ -0,0 +1,85 @@ +/* + * 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 org.apache.spark.deploy.kubernetes + +import scala.collection.mutable.ArrayBuffer + +private[spark] class ClientArguments(args: Array[String]) { + + var userJar: String = null + var userClass: String = null + var primaryPyFile: String = null + var primaryRFile: String = null + var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + var args = inputArgs + + while (!args.isEmpty) { + args match { + case ("--jar") :: value :: tail => + userJar = value + args = tail + + case ("--class") :: value :: tail => + userClass = value + args = tail + + case ("--primary-py-file") :: value :: tail => + primaryPyFile = value + args = tail + + case ("--primary-r-file") :: value :: tail => + primaryRFile = value + args = tail + + case ("--arg") :: value :: tail => + userArgs += value + args = tail + + case Nil => + + case _ => + throw new IllegalArgumentException(getUsageMessage(args)) + } + } + + if (primaryPyFile != null && primaryRFile != null) { + throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" + + " at the same time") + } + } + + private def getUsageMessage(unknownParam: List[String] = null): String = { + val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" + message + + s""" + |Usage: org.apache.spark.deploy.kubernetes.Client [options] + |Options: + | --jar JAR_PATH Path to your application's JAR file (required in kubernetes-cluster + | mode) + | --class CLASS_NAME Name of your application's main class (required) + | --primary-py-file A main Python file + | --primary-r-file A main R file + | --arg ARG Argument to be passed to your application's main class. + | Multiple invocations are possible, each will be passed in order. + """.stripMargin + } +} diff --git a/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala new file mode 100644 index 0000000000000..ab9c4f39706f3 --- /dev/null +++ b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala @@ -0,0 +1,146 @@ +/* + * 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 org.apache.spark.deploy.kubernetes + +import java.nio.ByteBuffer +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.deploy.ExternalShuffleService +import org.apache.spark.internal.Logging +import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage +import org.apache.spark.network.shuffle.protocol.mesos.{ + RegisterDriver, + ShuffleServiceHeartbeat +} +import org.apache.spark.network.util.TransportConf +import org.apache.spark.util.ThreadUtils + +/** + * An RPC endpoint that receives registration requests from Spark drivers running on Mesos. + * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]]. + */ +private[kubernetes] class KubernetesExternalShuffleBlockHandler( + transportConf: TransportConf, + cleanerIntervalS: Long) + extends ExternalShuffleBlockHandler(transportConf, null) + with Logging { + + ThreadUtils + .newDaemonSingleThreadScheduledExecutor("shuffle-cleaner-watcher") + .scheduleAtFixedRate(new CleanerThread(), + 0, + cleanerIntervalS, + TimeUnit.SECONDS) + + // Stores a map of app id to app state (timeout value and last heartbeat) + private val connectedApps = new ConcurrentHashMap[String, AppState]() + + protected override def handleMessage(message: BlockTransferMessage, + client: TransportClient, + callback: RpcResponseCallback): Unit = { + message match { + case RegisterDriverParam(appId, appState) => + val address = client.getSocketAddress + val timeout = appState.heartbeatTimeout + logInfo( + s"Received registration request from app $appId (remote address $address, " + + s"heartbeat timeout $timeout ms).") + if (connectedApps.containsKey(appId)) { + logWarning( + s"Received a registration request from app $appId, but it was already " + + s"registered") + } + connectedApps.put(appId, appState) + callback.onSuccess(ByteBuffer.allocate(0)) + case Heartbeat(appId) => + val address = client.getSocketAddress + Option(connectedApps.get(appId)) match { + case Some(existingAppState) => + logTrace( + s"Received ShuffleServiceHeartbeat from app '$appId' (remote " + + s"address $address).") + existingAppState.lastHeartbeat = System.nanoTime() + case None => + logWarning( + s"Received ShuffleServiceHeartbeat from an unknown app (remote " + + s"address $address, appId '$appId').") + } + case _ => super.handleMessage(message, client, callback) + } + } + + /** An extractor object for matching [[RegisterDriver]] message. */ + private object RegisterDriverParam { + def unapply(r: RegisterDriver): Option[(String, AppState)] = + Some( + (r.getAppId, new AppState(r.getHeartbeatTimeoutMs, System.nanoTime()))) + } + + private object Heartbeat { + def unapply(h: ShuffleServiceHeartbeat): Option[String] = Some(h.getAppId) + } + + private class AppState(val heartbeatTimeout: Long, + @volatile var lastHeartbeat: Long) + + private class CleanerThread extends Runnable { + override def run(): Unit = { + val now = System.nanoTime() + connectedApps.asScala.foreach { + case (appId, appState) => + if (now - appState.lastHeartbeat > appState.heartbeatTimeout * 1000 * 1000) { + logInfo(s"Application $appId timed out. Removing shuffle files.") + connectedApps.remove(appId) + applicationRemoved(appId, true) + } + } + } + } +} + +/** + * A wrapper of [[ExternalShuffleService]] that provides an additional endpoint for drivers + * to associate with. This allows the shuffle service to detect when a driver is terminated + * and can clean up the associated shuffle files. + */ +private[kubernetes] class KubernetesExternalShuffleService( + conf: SparkConf, + securityManager: SecurityManager) + extends ExternalShuffleService(conf, securityManager) { + + protected override def newShuffleBlockHandler( + conf: TransportConf): ExternalShuffleBlockHandler = { + val cleanerIntervalS = + this.conf.getTimeAsSeconds("spark.shuffle.cleaner.interval", "30s") + new KubernetesExternalShuffleBlockHandler(conf, cleanerIntervalS) + } +} + +private[spark] object KubernetesExternalShuffleService extends Logging { + def main(args: Array[String]): Unit = { + ExternalShuffleService.main( + args, + (conf: SparkConf, + sm: SecurityManager) => new KubernetesExternalShuffleService(conf, sm)) + } +} diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala new file mode 100644 index 0000000000000..a163b5ef8d749 --- /dev/null +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -0,0 +1,45 @@ +/* + * 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 org.apache.spark.scheduler.cluster.kubernetes + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} + +/** + * Cluster Manager for creation of Kubernetes scheduler and backend + */ +private[spark] class KubernetesClusterManager extends ExternalClusterManager { + override def canCreate(masterURL: String): Boolean = { + masterURL.startsWith("k8s") + } + + override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + new TaskSchedulerImpl(sc) + } + + override def createSchedulerBackend(sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + new KubernetesClusterSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc) + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} + diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala new file mode 100644 index 0000000000000..b2edd9e9edd2e --- /dev/null +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala @@ -0,0 +1,200 @@ +/* + * 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 org.apache.spark.scheduler.cluster.kubernetes + +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} +import io.fabric8.kubernetes.api.model.{PodBuilder, ServiceBuilder} +import io.fabric8.kubernetes.client.dsl.LogWatch +import org.apache.spark.deploy.kubernetes.ClientArguments +import org.apache.spark.{io, _} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ + +import collection.JavaConverters._ +import org.apache.spark.util.Utils + +/** + * This is a simple extension to ClusterScheduler + * */ +private[spark] class KubernetesClusterScheduler(conf: SparkConf) + extends Logging { + logWarning("Created KubernetesClusterScheduler") + + val kubernetesHost = new java.net.URI(conf.get("spark.master")) + var config = + new ConfigBuilder().withMasterUrl(kubernetesHost.getHost()).build + var client = new DefaultKubernetesClient(config) + var myNs = client.namespaces().list() + + def start(args: ClientArguments): Unit = { + startDriver(client, args) + logWarning(myNs.toString()) + } + + def stop(): Unit = {} + + def startDriver(client: DefaultKubernetesClient, + args: ClientArguments): Unit = { + println("###DRIVERSTART->" + args.userJar) + var annotationMap = Map("pod.beta.kubernetes.io/init-containers" -> raw"""[ + { + "name": "client-fetch", + "image": "busybox", + "command": ["wget", "-O", "/work-dir/client.jar", "http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar"], + "volumeMounts": [ + { + "name": "workdir", + "mountPath": "/work-dir" + } + ] + }, + { + "name": "distro-fetch", + "image": "busybox", + "command": ["wget", "-O", "/work-dir/spark.tgz", "http://storage.googleapis.com/foxish-spark-distro/spark.tgz"], + "volumeMounts": [ + { + "name": "workdir", + "mountPath": "/work-dir" + } + ] + }, + { + "name": "setup", + "image": "foxish/k8s-spark-driver:latest", + "command": ["./install.sh"], + "volumeMounts": [ + { + "name": "workdir", + "mountPath": "/work-dir" + }, + { + "name": "opt", + "mountPath": "/opt" + } + ] + } + ]""") + + val labelMap = Map("name" -> "spark-driver") + val pod = new PodBuilder() + .withNewMetadata() + .withLabels(labelMap.asJava) + .withName("spark-driver") + .withAnnotations(annotationMap.asJava) + .endMetadata() + .withNewSpec() + .withRestartPolicy("OnFailure") + .addNewContainer() + .withName("spark-driver") + .withImage("foxish/k8s-spark-driver:latest") + .withImagePullPolicy("Always") + .withCommand("/opt/spark/bin/spark-submit") + .withArgs("--class=org.apache.spark.examples.SparkPi", + s"--master=$kubernetesHost", + "--executor-memory=2G", + "--num-executors=8", + "/work-dir/client.jar", + "10000") + .addNewPort() + .withContainerPort(80) + .endPort() + .withVolumeMounts() + .addNewVolumeMount() + .withName("workdir") + .withMountPath("/work-dir") + .endVolumeMount() + .addNewVolumeMount() + .withName("opt") + .withMountPath("/opt") + .endVolumeMount() + .endContainer() + .withVolumes() + .addNewVolume() + .withName("workdir") + .withNewEmptyDir() + .endEmptyDir() + .endVolume() + .addNewVolume() + .withName("opt") + .withNewEmptyDir() + .endEmptyDir() + .endVolume() + .endSpec() + .build() + + client.pods().inNamespace("default").withName("spark-driver").create(pod) + + var svc = new ServiceBuilder() + .withNewMetadata() + .withLabels(labelMap.asJava) + .withName("spark-driver-svc") + .endMetadata() + .withNewSpec() + .addNewPort() + .withPort(4040) + .withNewTargetPort() + .withIntVal(4040) + .endTargetPort() + .endPort() + .withSelector(labelMap.asJava) + .withType("LoadBalancer") + .endSpec() + .build() + + client + .services() + .inNamespace("default") + .withName("spark-driver-svc") + .create(svc) + +// try { +// while (true) { +// client +// .pods() +// .inNamespace("default") +// .withName("spark-driver") +// .tailingLines(10) +// .watchLog(System.out) +// Thread.sleep(5 * 1000) +// } +// } catch { +// case e: Exception => logError(e.getMessage) +// } + } + + def getNamespace(): String = { + return "default" + } + + def generateJobName(): String = { + // let the job name be a sha random alphanumeric string + // + the timestamp. + return "" + } + + def generateDriverName(): String = { + // TODO: Fill + return "" + } + + def generateSvcName(): String = { + // TODO: Fill + return "" + } +} diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala new file mode 100644 index 0000000000000..1e6da63ad4339 --- /dev/null +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -0,0 +1,209 @@ +/* + * 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 org.apache.spark.scheduler.cluster.kubernetes + +import collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.PodBuilder +import io.fabric8.kubernetes.api.model.extensions.JobBuilder +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.cluster._ +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.rpc.RpcEndpointAddress +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.util.Utils + +private[spark] class KubernetesClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { + val config = new ConfigBuilder().withMasterUrl("https://kubernetes").build + val client = new DefaultKubernetesClient(config) + val DEFAULT_NUMBER_EXECUTORS = 2 + var NO_EXECUTORS = false + + override def start() { + logWarning("Starting scheduler backendaa2") + super.start() + var i = 0 + + + logWarning("###->initialexecutors=" + getInitialTargetExecutorNumber(sc.conf).toString()) + for(i <- 1 to getInitialTargetExecutorNumber(sc.conf)){ + createExecutorPod(i) + } + None + } + + override def stop(): Unit = { + super.stop() + client.pods().inNamespace("default").withName("spark-driver").delete() + client + .services() + .inNamespace("default") + .withName("spark-driver-svc") + .delete() + scheduler.stop() + } + + // Dynamic allocation interfaces + override def doRequestTotalExecutors(requestedTotal: Int): scala.concurrent.Future[Boolean] = { + logWarning("###->doRequestTotalExecutors" + requestedTotal.toString()) + + // time to create it. + if (!NO_EXECUTORS) { + NO_EXECUTORS = true + createExecutorPod(requestedTotal) + } + return super.doRequestTotalExecutors(requestedTotal) + } + + override def doKillExecutors(executorIds: Seq[String]): scala.concurrent.Future[Boolean] = { + logWarning("###->doKillExecutors" + executorIds.toString()) + return super.doKillExecutors(executorIds) + } + + def getInitialTargetExecutorNumber(conf: SparkConf, + numExecutors: Int = + DEFAULT_NUMBER_EXECUTORS): Int = { + if (Utils.isDynamicAllocationEnabled(conf)) { + val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS) + val initialNumExecutors = + Utils.getDynamicAllocationInitialExecutors(conf) + val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS) + require( + initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, + s"initial executor number $initialNumExecutors must between min executor number " + + s"$minNumExecutors and max executor number $maxNumExecutors") + + initialNumExecutors + } else { + val targetNumExecutors = + sys.env + .get("SPARK_EXECUTOR_INSTANCES") + .map(_.toInt) + .getOrElse(numExecutors) + // System property can override environment variable. + conf.get(EXECUTOR_INSTANCES).getOrElse(targetNumExecutors) + } + } + + def createExecutorPod(executorNum: Int): Unit = { + // create a single k8s executor pod. + var annotationMap = Map( + "pod.beta.kubernetes.io/init-containers" -> raw"""[ + { + "name": "client-fetch", + "image": "busybox", + "command": ["wget", "-O", "/work-dir/client.jar", "http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar"], + "volumeMounts": [ + { + "name": "workdir", + "mountPath": "/work-dir" + } + ] + }, + { + "name": "distro-fetch", + "image": "busybox", + "command": ["wget", "-O", "/work-dir/spark.tgz", "http://storage.googleapis.com/foxish-spark-distro/spark.tgz"], + "volumeMounts": [ + { + "name": "workdir", + "mountPath": "/work-dir" + } + ] + }, + { + "name": "setup", + "image": "foxish/k8s-spark-driver:latest", + "command": ["./install.sh"], + "volumeMounts": [ + { + "name": "workdir", + "mountPath": "/work-dir" + }, + { + "name": "opt", + "mountPath": "/opt" + } + ] + } + ]""") + + + val labelMap = Map("name" -> "spark-executor") + var pod = new PodBuilder() + .withNewMetadata() + .withLabels(labelMap.asJava) + .withName(s"spark-executor-$executorNum") + .withAnnotations(annotationMap.asJava) + .endMetadata() + .withNewSpec() + .withRestartPolicy("OnFailure") + .addNewContainer().withName("spark-executor").withImage("foxish/k8s-spark-executor:latest") + .withImagePullPolicy("Always") + .withCommand("/opt/spark/bin/spark-class") + .withArgs("org.apache.spark.executor.CoarseGrainedExecutorBackend", + "--driver-url", s"$driverURL", + "--executor-id", s"$executorNum", "--hostname", "localhost", "--cores", "1", + "--app-id", "1") + .addNewPort().withContainerPort(80).endPort() + .withVolumeMounts() + .addNewVolumeMount() + .withName("workdir") + .withMountPath("/work-dir") + .endVolumeMount() + .addNewVolumeMount() + .withName("opt") + .withMountPath("/opt") + .endVolumeMount() + .endContainer() + .withVolumes() + .addNewVolume() + .withName("workdir") + .withNewEmptyDir() + .endEmptyDir() + .endVolume() + .addNewVolume() + .withName("opt") + .withNewEmptyDir() + .endEmptyDir() + .endVolume() + .endSpec().build() + client.pods().inNamespace("default").withName(s"spark-executor-$executorNum").create(pod) + logWarning("## BUILDING POD FROM WITHIN ##") + } + + + protected def driverURL: String = { + if (conf.contains("spark.testing")) { + "driverURL" + } else { + RpcEndpointAddress( + conf.get("spark.driver.host"), + conf.get("spark.driver.port").toInt, + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + } + } + + override def getDriverLogUrls: Option[Map[String, String]] = { + var driverLogs: Option[Map[String, String]] = None + driverLogs + } +} diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala new file mode 100644 index 0000000000000..1ae790bd213dd --- /dev/null +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala @@ -0,0 +1,39 @@ +/* + * 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 org.apache.spark.scheduler.cluster.kubernetes + +import java.util.Date + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.Command + +private[spark] class KubernetesDriverDescription( + val name: String, + val jarUrl: String, + val mem: Int, + val cores: Double, + val supervise: Boolean, + val command: Command, + schedulerProperties: Map[String, String], + val submissionId: String, + val submissionDate: Date) + extends Serializable { + val conf = new SparkConf(false) + + +} diff --git a/pom.xml b/pom.xml index c391102d37502..908a56e0d8765 100644 --- a/pom.xml +++ b/pom.xml @@ -2592,6 +2592,13 @@ + + kubernetes + + kubernetes + + + hive-thriftserver diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e3fbe0379fb7b..c287695bbed1d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -56,9 +56,9 @@ object BuildCommons { "tags", "sketch" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects - val optionallyEnabledProjects@Seq(mesos, yarn, java8Tests, sparkGangliaLgpl, + val optionallyEnabledProjects@Seq(mesos, yarn, kubernetes, java8Tests, sparkGangliaLgpl, streamingKinesisAsl, dockerIntegrationTests) = - Seq("mesos", "yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl", + Seq("mesos", "yarn", "kubernetes", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = From 028f36cc4cb257324e3ab34955e93f622ad28caf Mon Sep 17 00:00:00 2001 From: Anirudh Date: Fri, 7 Oct 2016 15:10:34 -0700 Subject: [PATCH 02/14] Various fixes to driver code. --- .../spark/deploy/kubernetes/Client.scala | 5 +- .../KubernetesExternalShuffleService.scala | 146 ---------------- .../KubernetesClusterScheduler.scala | 159 +++++++++++++----- .../KubernetesClusterSchedulerBackend.scala | 63 ++++--- .../KubernetesDriverDescription.scala | 2 - 5 files changed, 148 insertions(+), 227 deletions(-) delete mode 100644 kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala diff --git a/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 09b492e1f98d2..a83a6c98bfa36 100644 --- a/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -43,7 +43,6 @@ private[spark] class Client(val args: ClientArguments, def stop(): Unit = { scheduler.stop() shutdownLatch.countDown() - System.clearProperty("SPARK_KUBERNETES_MODE") } @@ -60,9 +59,9 @@ private object Client extends Logging { val client = new Client(args, sparkConf) client.start() - logDebug("Adding shutdown hook") // force eager creation of logger + logDebug("Adding shutdown hook") ShutdownHookManager.addShutdownHook { () => - logInfo("Shutdown hook is shutting down dispatcher") + logInfo("Shutdown hook is shutting down client") client.stop() client.awaitShutdown() } diff --git a/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala b/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala deleted file mode 100644 index ab9c4f39706f3..0000000000000 --- a/kubernetes/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala +++ /dev/null @@ -1,146 +0,0 @@ -/* - * 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 org.apache.spark.deploy.kubernetes - -import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} - -import scala.collection.JavaConverters._ - -import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.deploy.ExternalShuffleService -import org.apache.spark.internal.Logging -import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} -import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage -import org.apache.spark.network.shuffle.protocol.mesos.{ - RegisterDriver, - ShuffleServiceHeartbeat -} -import org.apache.spark.network.util.TransportConf -import org.apache.spark.util.ThreadUtils - -/** - * An RPC endpoint that receives registration requests from Spark drivers running on Mesos. - * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]]. - */ -private[kubernetes] class KubernetesExternalShuffleBlockHandler( - transportConf: TransportConf, - cleanerIntervalS: Long) - extends ExternalShuffleBlockHandler(transportConf, null) - with Logging { - - ThreadUtils - .newDaemonSingleThreadScheduledExecutor("shuffle-cleaner-watcher") - .scheduleAtFixedRate(new CleanerThread(), - 0, - cleanerIntervalS, - TimeUnit.SECONDS) - - // Stores a map of app id to app state (timeout value and last heartbeat) - private val connectedApps = new ConcurrentHashMap[String, AppState]() - - protected override def handleMessage(message: BlockTransferMessage, - client: TransportClient, - callback: RpcResponseCallback): Unit = { - message match { - case RegisterDriverParam(appId, appState) => - val address = client.getSocketAddress - val timeout = appState.heartbeatTimeout - logInfo( - s"Received registration request from app $appId (remote address $address, " + - s"heartbeat timeout $timeout ms).") - if (connectedApps.containsKey(appId)) { - logWarning( - s"Received a registration request from app $appId, but it was already " + - s"registered") - } - connectedApps.put(appId, appState) - callback.onSuccess(ByteBuffer.allocate(0)) - case Heartbeat(appId) => - val address = client.getSocketAddress - Option(connectedApps.get(appId)) match { - case Some(existingAppState) => - logTrace( - s"Received ShuffleServiceHeartbeat from app '$appId' (remote " + - s"address $address).") - existingAppState.lastHeartbeat = System.nanoTime() - case None => - logWarning( - s"Received ShuffleServiceHeartbeat from an unknown app (remote " + - s"address $address, appId '$appId').") - } - case _ => super.handleMessage(message, client, callback) - } - } - - /** An extractor object for matching [[RegisterDriver]] message. */ - private object RegisterDriverParam { - def unapply(r: RegisterDriver): Option[(String, AppState)] = - Some( - (r.getAppId, new AppState(r.getHeartbeatTimeoutMs, System.nanoTime()))) - } - - private object Heartbeat { - def unapply(h: ShuffleServiceHeartbeat): Option[String] = Some(h.getAppId) - } - - private class AppState(val heartbeatTimeout: Long, - @volatile var lastHeartbeat: Long) - - private class CleanerThread extends Runnable { - override def run(): Unit = { - val now = System.nanoTime() - connectedApps.asScala.foreach { - case (appId, appState) => - if (now - appState.lastHeartbeat > appState.heartbeatTimeout * 1000 * 1000) { - logInfo(s"Application $appId timed out. Removing shuffle files.") - connectedApps.remove(appId) - applicationRemoved(appId, true) - } - } - } - } -} - -/** - * A wrapper of [[ExternalShuffleService]] that provides an additional endpoint for drivers - * to associate with. This allows the shuffle service to detect when a driver is terminated - * and can clean up the associated shuffle files. - */ -private[kubernetes] class KubernetesExternalShuffleService( - conf: SparkConf, - securityManager: SecurityManager) - extends ExternalShuffleService(conf, securityManager) { - - protected override def newShuffleBlockHandler( - conf: TransportConf): ExternalShuffleBlockHandler = { - val cleanerIntervalS = - this.conf.getTimeAsSeconds("spark.shuffle.cleaner.interval", "30s") - new KubernetesExternalShuffleBlockHandler(conf, cleanerIntervalS) - } -} - -private[spark] object KubernetesExternalShuffleService extends Logging { - def main(args: Array[String]): Unit = { - ExternalShuffleService.main( - args, - (conf: SparkConf, - sm: SecurityManager) => new KubernetesExternalShuffleService(conf, sm)) - } -} diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala index b2edd9e9edd2e..6986822b6505e 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala @@ -17,9 +17,14 @@ package org.apache.spark.scheduler.cluster.kubernetes -import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} +import java.io.File +import java.util.Date +import java.util.concurrent.atomic.AtomicLong + +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient} import io.fabric8.kubernetes.api.model.{PodBuilder, ServiceBuilder} import io.fabric8.kubernetes.client.dsl.LogWatch +import org.apache.spark.deploy.Command import org.apache.spark.deploy.kubernetes.ClientArguments import org.apache.spark.{io, _} import org.apache.spark.internal.Logging @@ -28,34 +33,71 @@ import org.apache.spark.internal.config._ import collection.JavaConverters._ import org.apache.spark.util.Utils +import scala.util.Random + /** * This is a simple extension to ClusterScheduler * */ private[spark] class KubernetesClusterScheduler(conf: SparkConf) extends Logging { - logWarning("Created KubernetesClusterScheduler") + private val DEFAULT_SUPERVISE = false + private val DEFAULT_MEMORY = Utils.DEFAULT_DRIVER_MEM_MB // mb + private val DEFAULT_CORES = 1.0 - val kubernetesHost = new java.net.URI(conf.get("spark.master")) - var config = - new ConfigBuilder().withMasterUrl(kubernetesHost.getHost()).build - var client = new DefaultKubernetesClient(config) - var myNs = client.namespaces().list() + logInfo("Created KubernetesClusterScheduler instance") + var client = setupKubernetesClient() + val driverName = s"spark-driver-${Random.alphanumeric take 5 mkString("")}".toLowerCase() + val svcName = s"spark-svc-${Random.alphanumeric take 5 mkString("")}".toLowerCase() def start(args: ClientArguments): Unit = { startDriver(client, args) - logWarning(myNs.toString()) } - def stop(): Unit = {} + def stop(): Unit = { + client.pods().inNamespace(getNamespace()).withName(driverName).delete() + client + .services() + .inNamespace(getNamespace()) + .withName(svcName) + .delete() + } - def startDriver(client: DefaultKubernetesClient, + def startDriver(client: KubernetesClient, args: ClientArguments): Unit = { - println("###DRIVERSTART->" + args.userJar) + logInfo("Starting spark driver on kubernetes cluster") + val driverDescription = buildDriverDescription(args) + + // This is the URL of the spark distro. + val sparkDistUri = Option(System.getenv("SPARK_DISTRO_URI")).getOrElse { + throw new SparkException("Spark distribution not set, please set the SPARK_DISTRO_URI environment variable to " + + "a runnable spark archive.") + } + + // This is the URL of the driver pod's image. + // Any image may be supplied as long as it contains a + // ./install.sh file which is executable and sets up the + // spark environment in /opt/spark. + val sparkDriverImage = Option(System.getenv("SPARK_DRIVER_IMG")).getOrElse { + throw new SparkException("Spark driver image not set, please set the SPARK_DRIVER_IMG environment variable to " + + "a spark driver image.") + } + + // This is the URL of the client jar. + val clientJarUri = args.userJar + conf.setExecutorEnv("spark.executor.jar", clientJarUri) + conf.setExecutorEnv("spark.kubernetes.namespace", getNamespace()) + conf.setExecutorEnv("spark.kubernetes.driver.image", sparkDriverImage) + conf.setExecutorEnv("spark.kubernetes.distribution.uri", sparkDistUri) + + // This is the kubernetes master we're launching on. + val kubernetesHost = "k8s://" + client.getMasterUrl().getHost() + logInfo("Using as kubernetes-master: " + kubernetesHost.toString()) + var annotationMap = Map("pod.beta.kubernetes.io/init-containers" -> raw"""[ { "name": "client-fetch", "image": "busybox", - "command": ["wget", "-O", "/work-dir/client.jar", "http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar"], + "command": ["wget", "-O", "/work-dir/client.jar", "$clientJarUri"], "volumeMounts": [ { "name": "workdir", @@ -66,7 +108,7 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) { "name": "distro-fetch", "image": "busybox", - "command": ["wget", "-O", "/work-dir/spark.tgz", "http://storage.googleapis.com/foxish-spark-distro/spark.tgz"], + "command": ["wget", "-O", "/work-dir/spark.tgz", "$sparkDistUri"], "volumeMounts": [ { "name": "workdir", @@ -76,7 +118,7 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) }, { "name": "setup", - "image": "foxish/k8s-spark-driver:latest", + "image": "$sparkDriverImage", "command": ["./install.sh"], "volumeMounts": [ { @@ -91,29 +133,27 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) } ]""") - val labelMap = Map("name" -> "spark-driver") + + val labelMap = Map("type" -> "spark-driver") val pod = new PodBuilder() .withNewMetadata() .withLabels(labelMap.asJava) - .withName("spark-driver") + .withName(driverName) .withAnnotations(annotationMap.asJava) .endMetadata() .withNewSpec() .withRestartPolicy("OnFailure") .addNewContainer() .withName("spark-driver") - .withImage("foxish/k8s-spark-driver:latest") + .withImage(sparkDriverImage) .withImagePullPolicy("Always") .withCommand("/opt/spark/bin/spark-submit") - .withArgs("--class=org.apache.spark.examples.SparkPi", + .withArgs(s"--class=${args.userClass}", s"--master=$kubernetesHost", - "--executor-memory=2G", - "--num-executors=8", + s"--executor-memory=${driverDescription.mem}", + s"--num-executors=${driverDescription.cores}", "/work-dir/client.jar", - "10000") - .addNewPort() - .withContainerPort(80) - .endPort() + args.userArgs.mkString(" ")) .withVolumeMounts() .addNewVolumeMount() .withName("workdir") @@ -137,13 +177,12 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) .endVolume() .endSpec() .build() - - client.pods().inNamespace("default").withName("spark-driver").create(pod) + client.pods().inNamespace(getNamespace()).withName(driverName).create(pod) var svc = new ServiceBuilder() .withNewMetadata() .withLabels(labelMap.asJava) - .withName("spark-driver-svc") + .withName(svcName) .endMetadata() .withNewSpec() .addNewPort() @@ -159,8 +198,8 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) client .services() - .inNamespace("default") - .withName("spark-driver-svc") + .inNamespace(getNamespace()) + .withName(svcName) .create(svc) // try { @@ -178,23 +217,59 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) // } } - def getNamespace(): String = { - return "default" + def setupKubernetesClient(): KubernetesClient = { + val sparkMaster = new java.net.URI(conf.get("spark.master")) + if (sparkMaster.getHost() == "default") { + return new DefaultKubernetesClient() + } else { + var config = new ConfigBuilder().withMasterUrl(sparkMaster.getHost()).build + var client = new DefaultKubernetesClient(config) + return client + } } - def generateJobName(): String = { - // let the job name be a sha random alphanumeric string - // + the timestamp. - return "" + def getNamespace(): String = { + var kubernetesNamespace = System.getenv("K8S_NAMESPACE") + if (kubernetesNamespace == null) { + kubernetesNamespace = "default" + } + return kubernetesNamespace } - def generateDriverName(): String = { - // TODO: Fill - return "" - } + private def buildDriverDescription(args: ClientArguments): KubernetesDriverDescription = { + // Required fields, including the main class because python is not yet supported + val appResource = Option(args.userJar).getOrElse { + throw new SparkException("Application jar is missing.") + } + val mainClass = Option(args.userClass).getOrElse { + throw new SparkException("Main class is missing.") + } + + // Optional fields + val driverExtraJavaOptions = conf.getOption("spark.driver.extraJavaOptions") + val driverExtraClassPath = conf.getOption("spark.driver.extraClassPath") + val driverExtraLibraryPath = conf.getOption("spark.driver.extraLibraryPath") + val superviseDriver = conf.getOption("spark.driver.supervise") + val driverMemory = conf.getOption("spark.driver.memory") + val driverCores = conf.getOption("spark.driver.cores") + val name = conf.getOption("spark.app.name").getOrElse("default") + val appArgs = args.userArgs + + // Construct driver description + val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) + val sparkJavaOpts = Utils.sparkJavaOpts(conf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts + val command = new Command( + mainClass, appArgs, null, extraClassPath, extraLibraryPath, javaOpts) + val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) + val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) + val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) + val submitDate = new Date() - def generateSvcName(): String = { - // TODO: Fill - return "" + new KubernetesDriverDescription( + name, appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, + command, submitDate) } } diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 1e6da63ad4339..870bc39c058b3 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -23,11 +23,14 @@ import io.fabric8.kubernetes.api.model.extensions.JobBuilder import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} import org.apache.spark.internal.config._ import org.apache.spark.scheduler.cluster._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils +import scala.collection.mutable +import scala.util.Random + private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) @@ -35,46 +38,37 @@ private[spark] class KubernetesClusterSchedulerBackend( val config = new ConfigBuilder().withMasterUrl("https://kubernetes").build val client = new DefaultKubernetesClient(config) val DEFAULT_NUMBER_EXECUTORS = 2 - var NO_EXECUTORS = false + val sparkExecutorName = s"spark-executor-${Random.alphanumeric take 5 mkString("")}".toLowerCase() + var executorPods = mutable.ArrayBuffer[String]() + + // This is the URL of the spark distro. + val sparkDistUri = sc.conf.getenv("spark.kubernetes.distribution.uri") + val sparkDriverImage = sc.conf.getenv("spark.kubernetes.driver.image") + val clientJarUri = sc.conf.getenv("spark.executor.jar") + val ns = sc.conf.getenv("spark.kubernetes.namespace") override def start() { - logWarning("Starting scheduler backendaa2") super.start() var i = 0 - - - logWarning("###->initialexecutors=" + getInitialTargetExecutorNumber(sc.conf).toString()) for(i <- 1 to getInitialTargetExecutorNumber(sc.conf)){ - createExecutorPod(i) + executorPods += createExecutorPod(i) } None } override def stop(): Unit = { + for (i <- 0 to executorPods.length) { + client.pods().inNamespace(ns).withName(executorPods(i)).delete() + } super.stop() - client.pods().inNamespace("default").withName("spark-driver").delete() - client - .services() - .inNamespace("default") - .withName("spark-driver-svc") - .delete() - scheduler.stop() } // Dynamic allocation interfaces override def doRequestTotalExecutors(requestedTotal: Int): scala.concurrent.Future[Boolean] = { - logWarning("###->doRequestTotalExecutors" + requestedTotal.toString()) - - // time to create it. - if (!NO_EXECUTORS) { - NO_EXECUTORS = true - createExecutorPod(requestedTotal) - } return super.doRequestTotalExecutors(requestedTotal) } override def doKillExecutors(executorIds: Seq[String]): scala.concurrent.Future[Boolean] = { - logWarning("###->doKillExecutors" + executorIds.toString()) return super.doKillExecutors(executorIds) } @@ -98,19 +92,18 @@ private[spark] class KubernetesClusterSchedulerBackend( .get("SPARK_EXECUTOR_INSTANCES") .map(_.toInt) .getOrElse(numExecutors) - // System property can override environment variable. conf.get(EXECUTOR_INSTANCES).getOrElse(targetNumExecutors) } } - def createExecutorPod(executorNum: Int): Unit = { + def createExecutorPod(executorNum: Int): String = { // create a single k8s executor pod. var annotationMap = Map( "pod.beta.kubernetes.io/init-containers" -> raw"""[ { "name": "client-fetch", "image": "busybox", - "command": ["wget", "-O", "/work-dir/client.jar", "http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar"], + "command": ["wget", "-O", "/work-dir/client.jar", "$clientJarUri"], "volumeMounts": [ { "name": "workdir", @@ -121,7 +114,7 @@ private[spark] class KubernetesClusterSchedulerBackend( { "name": "distro-fetch", "image": "busybox", - "command": ["wget", "-O", "/work-dir/spark.tgz", "http://storage.googleapis.com/foxish-spark-distro/spark.tgz"], + "command": ["wget", "-O", "/work-dir/spark.tgz", "$sparkDistUri"], "volumeMounts": [ { "name": "workdir", @@ -131,7 +124,7 @@ private[spark] class KubernetesClusterSchedulerBackend( }, { "name": "setup", - "image": "foxish/k8s-spark-driver:latest", + "image": "$sparkDriverImage", "command": ["./install.sh"], "volumeMounts": [ { @@ -147,23 +140,25 @@ private[spark] class KubernetesClusterSchedulerBackend( ]""") - val labelMap = Map("name" -> "spark-executor") + val labelMap = Map("type" -> "spark-executor") + val podName = s"$sparkExecutorName-$executorNum" var pod = new PodBuilder() .withNewMetadata() .withLabels(labelMap.asJava) - .withName(s"spark-executor-$executorNum") + .withName(podName) .withAnnotations(annotationMap.asJava) .endMetadata() .withNewSpec() .withRestartPolicy("OnFailure") - .addNewContainer().withName("spark-executor").withImage("foxish/k8s-spark-executor:latest") + .addNewContainer().withName("spark-executor").withImage(sparkDriverImage) .withImagePullPolicy("Always") .withCommand("/opt/spark/bin/spark-class") .withArgs("org.apache.spark.executor.CoarseGrainedExecutorBackend", "--driver-url", s"$driverURL", - "--executor-id", s"$executorNum", "--hostname", "localhost", "--cores", "1", + "--executor-id", s"$executorNum", + "--hostname", "localhost", + "--cores", "1", "--app-id", "1") - .addNewPort().withContainerPort(80).endPort() .withVolumeMounts() .addNewVolumeMount() .withName("workdir") @@ -186,8 +181,8 @@ private[spark] class KubernetesClusterSchedulerBackend( .endEmptyDir() .endVolume() .endSpec().build() - client.pods().inNamespace("default").withName(s"spark-executor-$executorNum").create(pod) - logWarning("## BUILDING POD FROM WITHIN ##") + client.pods().inNamespace(ns).withName(podName).create(pod) + return podName } diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala index 1ae790bd213dd..00ac7ea4da08a 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesDriverDescription.scala @@ -29,8 +29,6 @@ private[spark] class KubernetesDriverDescription( val cores: Double, val supervise: Boolean, val command: Command, - schedulerProperties: Map[String, String], - val submissionId: String, val submissionDate: Date) extends Serializable { val conf = new SparkConf(false) From 043945417997c98219d13961ef7b16b0ee349fbe Mon Sep 17 00:00:00 2001 From: Anirudh Date: Fri, 7 Oct 2016 16:58:45 -0700 Subject: [PATCH 03/14] Fixes to cluster scheduler backend for scaling --- .../kubernetes/KubernetesClusterScheduler.scala | 9 ++++++++- .../KubernetesClusterSchedulerBackend.scala | 11 +++++------ 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala index 6986822b6505e..626f776b32664 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala @@ -48,6 +48,9 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) var client = setupKubernetesClient() val driverName = s"spark-driver-${Random.alphanumeric take 5 mkString("")}".toLowerCase() val svcName = s"spark-svc-${Random.alphanumeric take 5 mkString("")}".toLowerCase() + val instances = conf.get(EXECUTOR_INSTANCES).getOrElse(2) //TODO: default 2??? + + logWarning("instances: " + instances) def start(args: ClientArguments): Unit = { startDriver(client, args) @@ -151,7 +154,11 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) .withArgs(s"--class=${args.userClass}", s"--master=$kubernetesHost", s"--executor-memory=${driverDescription.mem}", - s"--num-executors=${driverDescription.cores}", + s"--conf=spark.executor.jar=$clientJarUri", + s"--conf=spark.executor.instances=$instances", + s"--conf=spark.kubernetes.namespace=${getNamespace()}", + s"--conf=spark.kubernetes.driver.image=$sparkDriverImage", + s"--conf=spark.kubernetes.distribution.uri=$sparkDistUri", "/work-dir/client.jar", args.userArgs.mkString(" ")) .withVolumeMounts() diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 870bc39c058b3..3765eeb063cb7 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -41,11 +41,10 @@ private[spark] class KubernetesClusterSchedulerBackend( val sparkExecutorName = s"spark-executor-${Random.alphanumeric take 5 mkString("")}".toLowerCase() var executorPods = mutable.ArrayBuffer[String]() - // This is the URL of the spark distro. - val sparkDistUri = sc.conf.getenv("spark.kubernetes.distribution.uri") - val sparkDriverImage = sc.conf.getenv("spark.kubernetes.driver.image") - val clientJarUri = sc.conf.getenv("spark.executor.jar") - val ns = sc.conf.getenv("spark.kubernetes.namespace") + val sparkDistUri = sc.getConf.get("spark.kubernetes.distribution.uri") + val sparkDriverImage = sc.getConf.get("spark.kubernetes.driver.image") + val clientJarUri = sc.getConf.get("spark.executor.jar") + val ns = sc.getConf.get("spark.kubernetes.namespace") override def start() { super.start() @@ -158,7 +157,7 @@ private[spark] class KubernetesClusterSchedulerBackend( "--executor-id", s"$executorNum", "--hostname", "localhost", "--cores", "1", - "--app-id", "1") + "--app-id", "1") //TODO: change app-id per application and pass from driver. .withVolumeMounts() .addNewVolumeMount() .withName("workdir") From 098f55197fb6f7185a1ab7ec31c5460c5e6217db Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Fri, 7 Oct 2016 19:51:26 -0700 Subject: [PATCH 04/14] Create README.md --- kubernetes/README.md | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 kubernetes/README.md diff --git a/kubernetes/README.md b/kubernetes/README.md new file mode 100644 index 0000000000000..ec4adc526d51e --- /dev/null +++ b/kubernetes/README.md @@ -0,0 +1,26 @@ +# Pre-requisites +* maven, JDK and all other pre-requisites for building Spark. +* An accessible kubernetes cluster which is running 1.4 or a 1.3.x cluster with alpha features enabled. + * We use init-containers, which require the above. + +# Steps to compile + +* Clone the fork of spark: https://github.com/foxish/spark/ and switch to the k8s-support branch. +* Build the project + * ./build/mvn -Pkubernetes -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests package +* Ensure that you are pointing to a k8s cluster (kubectl config current-context), which you want to use with spark. +* Set the appropriate environment variables: + * `export SPARK_DISTRO_URI=http://storage.googleapis.com/foxish-spark-distro/spark.tgz` + * This should be a runnable distribution of spark. + * The tgz specified above was build as follows: + * ./dev/make-distribution.sh --name custom-spark --tgz -Pkubernetes -Phadoop-2.4 +* `export SPARK_DRIVER_IMG=foxish/k8s-spark-driver` + * This is the runnable docker image we will use. The docker image sources can be found here. +* [OPTIONAL] `export K8S_NAMESPACE=` + * This namespace needs to exist before hand. This env var can be left unset and it will use the default namespace. +* Launch a spark-submit job: + * `./bin/spark-submit --deploy-mode cluster --class org.apache.spark.examples.SparkPi --master k8s://default --conf spark.executor.instances=5 http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar 10000` + * The implementation is such that it is interactive, and will clean the drivers and executors up upon termination. + * `--master k8s://default` ensures that it picks up the correct APIServer the default from the current context. +* Check for pods being created. Watch the master logs using kubectl log -f . +* If on a service that allows external load balancers to be provisioned, an external IP will be allocated to the service associated with the driver. The spark-master UI can be accessed from that IP address on port 4040. From 196aad5b3b9800875267ffbb03516e181f7ce9e7 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Fri, 7 Oct 2016 19:59:04 -0700 Subject: [PATCH 05/14] Add files via upload --- .../Screenshot from 2016-10-07 19:57:34.png | Bin 0 -> 110846 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 kubernetes/Screenshot from 2016-10-07 19:57:34.png diff --git a/kubernetes/Screenshot from 2016-10-07 19:57:34.png b/kubernetes/Screenshot from 2016-10-07 19:57:34.png new file mode 100644 index 0000000000000000000000000000000000000000..0776549f4c7d6c5213af6209c974273e24bcc3e4 GIT binary patch literal 110846 zcma&Oc{mo_*Z!?ZB}s*ZkQ9kXl8}%jO-PeOM5&0RQW;VqNfIhaLIX|4N>M7QG$9%^ z7?Nl*L`9+ZbKTE79M5sQfBdfJc)oShHSN9ETIV{?bL}vz1+xc>$cP9D2@RfOVP+#F z)ccQ+P_M25z4?jelM{kJgq`QkHWT_M_`g#(&qec-fnFBNHwy_38ZP+%UP8xDNb|$~ zK6B>J>R%%=ptru<+T2$OIGzDSWf%#-7|+*SQ;^7m=3&uaFXjuz&9g<87r&KxTEi_qOs|NSvw+kby3zUw9UbC*~D|Nis; z_m_JO{`;MH(|>;$=l1#U&swqn{`mj><^O*1j$h&hTZ2PFLc0I{&aQMZ z|3@c=mom<6dz@=OG5)Epme#25d*f+Q49E_n)5(-tJmfE_y zx*8haYwqlsKYxCt{PaJyp$d9(&oWm(7&rg8&W?6Bz1cz?J-1(v61p{V%9JUQ3Nt== zUk@}&JAeNCj3c(QXV3QZ)C_S_S#V-sWTe6m4@XDG6)R-dcXWD}B+k%ORqarjn|1Eo zxhq#B3p0*tI)(%XJ4xTKYjs||dcuSWwOV~@^33)s3Dq=@kq}wI!_Kb5&nUvY{P}aG zVPWc0;g9RZ#Kg94+cwwIa{c=C$yX0uzC2MX{6%&3LgitSl9I78F&%yZuC8Y;UUc8Q zIXfq(y{1Oda*3jN%+aIK7Rr1hrPhBZik?1ws-dB=XU`rV{gsOsFE%wzzIs)vP>Y|5 z7V`Y%%i|6n6tg^?mX>z;@<9vb^A|2We)w>e^b&h}^)X`#*JNL}znq_+pOd2#UL+$e zedy@XrAwBK6CW`5hidhN=U!D3>M9z9|$hYi@vXOIe?b*}vFn>Q`K z`TO_pKXF1%EVQoGciy<{yu5fbCDvS9TC}zO@uNqlPoJ)M@nVwb1PzU`(Z$D)ANMu9 zb?eqiW5s#LwC>!!n|$rs(PEchKYvO}NIWbqPWmxkQ?vQ&*WusQ^+I|H4u6XQU-FvA zTk|BV0ewyV{coN-*YGHI!WbnbXJ_XpPoKW1sNi#j43?FWmVW)_&5gi+Jx7j6$wVx1 zbX=$ub=m&9(&*8(-@jivbEc@S^?rnGsL~$3?y+OXMj32%f0I-E_U($UBTrnkr+QY+ zQyw-)d02FDdu!{aZQF*3hzt`6p7F11a)?txLPFc8r+NN=bwixgr6#DV?lV@@(bctE zzTD&ejm5KOae#~N-ZfpZBJa|rOXtr&x_@6dv~UZ@*2ib>-o2u-y6%F*J#L=W(xsZ3 zn&+%kJ*)CtTUx|}4YoB``FC}+7KRQxWr`2S+qDH61*7FfriIx{j9GWWQg( zehK&O`?0=0?ZSmY!FSJ|9oOFZ^;LS|-Mh-l%37iu)~|n)vo1P3{Bc`D3E$#dQ_})P zaTbkJw`ccm+r^7D9dC6tmW`P*!;{^^=*&-XWeek_i@c=6)1XV2bV^US+`J*i7YS-Jh!uO*8Yd%V5+ z^zq|C2c3ijsjBPXHP31(Ny(c#ep;HEe&}p#=;-o1;PymrBH zopoMbhl-q8`~!DZRlIs-pszn}?AZAU`jUlOa$?*5w67aEGA=h)FUjiIi4%SwA4c8W z`TJRd!M2CuaU(PxY?myl%G(xco>Fx0Ue494KYa5YAIk&%q&^k(u*L`1}g+GmfCYHB*S-1Jiq~!X@uRD|`|sbs)^bu(QUpTp-n|!`G#p5x7o`4x#LJgAE?s(*!1z!Uer|sJ zC(e&iqTZ$t8BT>-qS=|5WzV1Q4-Yr+s@b(?&&&-QHtg83W8=n&kq`Lm%t0l`Ce0)f zzW>QVy6Rr}fd|ijP&YwMZR5M#JGEX~qVWdXE=NQ>y}U9{*Zp;dSFP#NrAze_7AQpp zXm}EQtlz2f&-{gxdj0x!jv*&}$&w{3t#rt`4IAR)<4K_unB~itw|?dTDG%t|JIqWe zy13w2ZbHKQD;r+j$jf8X4l2wraxwO?v3dSFbM+`WIW5u6Ten`#$+@ScP7cb-dj8_v-)7iLIKSYn^I^-^+j??Nr%Jx&`o%uOlN(b16>HQ%=Fclo)6oqj#1=R^pNlc?|z|MvVBt1_4HAj2Mg_bIZCK!#8_qJn3x#3={s^RT=-fPKEmyrScuceks}S| zheTwIpD~0=^YP)F>2M4e1`mV{S2>Z8aP#I(Ss9tF-`{Lu7e-EAv(Hp&%$O9x3vby{_Sr`_e&w=d zdQNxy`uqD23v2J`-l@~~9LH&}OVgJxf4WpCET4V;@V(3KHdo!OYwMi(;n=|8oAy865Y3#zVqyaf-)%-8NK7MfD#Mv5q`YXz!E|4_bu z;J`dDFE3wTUk8UOzCdhjY<+#b%EFUfwS}Slx8rqh*T+XR4j(@3<}@yPd(*pVSZj^QUq3?Cg`Os$fgy3vnRyJpQNc7Oi7dG}=Z8&01-P43z8 zDH|xOe*XUCWi8S2va;n@-IrTgmBvn3y0QA!O}8>7Sy|adi^?6lwvYofG%nTD)U>y^ zgO?~#K3ld79{4I`h=Rk?rRTsW`)}U3as1Az12R)A{YI+AzNxATsJlP-*!E#UZttlW zCR7zCC#P4JSMF@73ChgeWNt3}B{$B<|Hjp;zd&{B8X8o27Z;Z$7nhY)R$gCQB2T5Q ze|+@omoHbZTp2oGuTxuwbgLE-=SOzq{(@PBOj1jDVbL*G6=|qkMrBWjK_Q zl2g1tKD>R~nAh0e_GvLVH=t|ToH?HX6MXz5F)=P$(v;{!(b1c?ZCko%QTV=n(vzI~ zhB#5fl2vZ7i)D3{eS4dL{gy0|@=r}o-RE?SI&pS>{K*-CM&M6t)iIiy>FMbpK27^v zFXEEz0TnqqRx1Q{SsNa)S+wXc)r0lq>+!s^Nl8hmsTPXjR1!&2yQJh;@c~EFmuCfZ z?@)_DXD?qi@_3^| zGLsV{Xumc#re3%ZZg-5}EHz78uwX&)a`|oNN zZrWhLlP+|3cVBa_?47L^sL<1YZRzZ6wd(xsZPkx%_jLaqBqGxE?s}h41tFoX$)_fN zO?+8ZB`+>cu!a^Wj}Hv|2Si9;*-8qyu;kKRt&5~a%3a1H`Vg9u9!o=c%^5Qk1~2J| z&b8mjDORx9Z>E%Fl0dPb&h1Y4F=UXa=-#vRKOHJ}UMFZiudL+I-CmP@=JaWfZcJ<} zN0TJZUpRgP`|f7lfI&b`@9V_Eym8SF>c*?7QAI(+;DRGZkJfqR{`$3*HX!D#m8whT zvf$w0moHx?5AD;~cJ}Pq4U4;`X#uxhl`;>E76u7otj&M2Tu^>|Li(=`-` z-@kv8896)DbgCP$736sTzWMm^X;EWlzkKoH#mkpAsuEODnTR!RZhC85(IoOh(g_nI>sXn6Tl_?tT-NTvSz4v%9(DN<_rqxOom4t9n^xU%9e{#Pt06 zbJFF9f)}Nw95<&VD^;?ty}kRYRpP^j9p)sH4Ri(>*|pvd>eDzf5lC5l@1B{e#1dQE zj?T_$1_m<%dkodo9#vI2%$imI^Xn^pef?p>hS3b9I~H8Mc5TRj0lu_i02&$viUH-# z($X^Iqo6h2<<$`QEPR9}UEgGnH#vM~LLw-rmr|64@-P7k`Sj_2trx#-`%pA!`t+Nm z30`OCjvXJoaw#w>qFrK1hy73AdP#Km_x7?i$$Pc7c4v-&N~CM}uHlqn-c%`Ge* z0v*ft`{?_>%iTnoA?mrtMG zTUgsvb(6ZeLPB!;&Ycs+kKevy$AV*8P9N@t%{@A?`+NR&U*A*xW9ce@=Jtyg?QuF5 z7B+Cl-eo2xCUfQ#-@Yx>{r=S}mE%wQg%;Mf`pSv@UL5^6=-(-h-^-UvIFMy0^aF(w zOEkxiw_dtb;m7(`aK^ak;##lL$8Q6_UmBVS-YPFGJroo3(fj(maf@hv+s9cGUZHnh zG=$3;`T0^64dt5!wf^O$$B!S+tlgbz-8ZywVBg+JR=(h10D8V(=i#So1lZtY+idG7 zRg=}L>&Tp8%OfYzPz|F zL{2Ybo>8Q`*$3s8({S7k9z?Bw zd1=Mb_;`a&AIvE3OO~uzwd!_V>+|yRhyw@ogLq3H?a z^}H0vk6-sFW;{8`OsVy0{M5?I${F45B*BPIXOA|_U!|`Aa`ExiemWX=vcI2dVZi4Pr`bncvrhDLYckQq8t zr>6Xp9P!^{-MY^}TNCXr(IMY$_3Ax^wHvGQ$7^Ve8ak9>0VIkr zg217}n-w%5Zr;`G?3pvSo;YzrN_|<&$Kv6w)!LIL4IDIRjQ9W=bLC-SR69Ys(bg6b z6Z_QayJ^!VH*bFuj&_{8r)MDD?!$)<69aoHtE;OE3;PsmamM*(F~^S_dAWDs@X~5E zH8qF?UJf3mapvaDn-vri`M7rn?dg%6HfFMh`fN%5i491QvdinBKW%gRi0k}NZ z=ks$=!0Oej+03T*d1Y>w4jd4jJUJVhKx5?%8n5;j|HjHF-LLm1pZUR*05;jN$~$+Q zo}ZrM;NY14YjD zA3vV5ZUg3kyaQQ|*(_3Aa#NLhS9w93L?ZEeHg$t{z9V<5G5JaPkDLA9UPF2eoY*iP@))F25(=W;X>B;FU{AvY_LC;CU z?IgVDvTABD59yHkC+MB;g-M5$x#f)R>>qduY<>Q`nrLCF?fL!tEoaT@_ea1?H0FA5 z*|K=)(nOtg&pA%--@gZL%#M)x3HG^j$3%Ef=7v`jH*aohYT7t!)+~`=o8`;bxw;nB zd$Xm32M_l5^Sg25hVt040QzDV?e+Hgn;QoP#EJca%?TMpPK8aVzQis-d{PByme_xVY2JPP1AaHH zq|uRy2swKVknQd5?dCH`K63V`Q$U60>H^2Sts0?)Wv^Z}fBW{at&=Ym6cj|>Gu-^? z!2bP@-dx!bI#^a$`Su^O9QjpC6i7@v3~;kmTkPNB0@EC)J44FZ`BZrjQ_HL}4P8Z7>0m(;&7Pk5t>c)Gx zy3$uDTPEIF_3hpD?Ck7CUDryzbVLhHO-<=z#N-Vdo0=|Pya?S0wFKj-B^n(Qlarar zF4eU95)t;~YRttVJg%pW0B;ykz$k&hy~T?qZfc zo#0-Oq_kp_dj35xD>JE9Q&oL(Y*OK!JH6%yuUfY*E;g2Aymay6SSrw=Ls|q!M@I)+ znH-l)qkQJfnMtCYX=>^QPtWwUG;3R1da2$)ih%wtZ(-4Ey+|x!H`)N`iH6&$CH;1e z6ly8?k3~pv=cudIcg9o3{ zbP(f5G#vwo(Uo~y;WcC;uDTQtlQ`%%ji7^#nLYbKON$q67l(!21F9Ix7kJk{niVbu zjeYjaneUa^{R_2}qNL^ISfoR-u~&BfZUP;rB`2HCof~d>hH86+uEW3}#pUf=sL$$K z8{g~u`*%Zh^RgWszB)6<(G0!!nz=rp;KGd?Z>p;UX~O?M`D2fRhwv<->fe}ClLrZU zfd3Pb#_!S$z2;{e+oW%Tr?mKo80g4&RHRxrdhD)gE~U)Bey!*JTB~=E(3#e$QL(YX zrkw46qa3F{RqV(ctD*vTmRg<(7Xh9uqf>Xld|3pms-%8W}iI^lv8G^_EC{6$S z`9l9&Y5L#4+@*<>((#HzRj|FyyuF&93e@@i%@j0{_XcK4o5;sxDXS&_^Xk>g^z0u$ zx&#?3_BdHu?r-#5w{9KOm8vSguZawLb}=v!N+meB$WvVlP6vSe->-!3CU_pBXZq7e zMcCg|DeC^gQG;gnWp;Az~H<8vUj%l za@PRh9a`pA|}< zOXJ>Hzy~7Ct<(iS-)m4kgcsTV!=)?`j?}D~{PsUT9%x`-AQHU0u$K1T{D5??eggfI zcen}3Z=xCF$;ENAM?YfV}^whZMJG#=#8vo_~C8G%{#l_NJ4SY(H|-fyyIxW zy3-n>LP4fj)-{W?wffGPGe;FkvO^*0@0;7f#V*4A`T@60DuX;nBrKqb3iTEo$)S$# zZxlAP{r&R?6jr1VCtM>-JIbR^J?|vm)FrlP(V{V9#;6*#Xg~|@5vZI!=WbiM{?DiD zb35>Gqlc{EKlU0Fq+hr&K~YiBz~FE7Q}b1;YDl^TF2yuM6oP_)?mrivBl_H=bSg-% zG1AlPfVn69X%ki2LJ$GjRR1zo1+;-%QXJTe5g_=4jaGW}H6l0vb&;@|Vfl zpO^O+XR-zBlrD#k`fugLe!Y}9J}{CS{=2=MQ&);ED7&`|R4 z!K3o0Pdz?AKl3kb<&(PBs~g|{0eA#+l>^ z2~Ztf{10D4)`(3X7FO~4^|7Nz&z(JsL^s3xqlJEggPq+Y7wz#9oOMeoNL*YTs+sCg zvCu_Jmb`rZ8t4P_!zx1uefj!zx%$hRnpG=S#1pyvQ=q6==<4iBXiW;Hub~4iDeqvR z%-(}Ve*Ew@N=!e0e!a7^(t0fmOUp0K%}|$9wY3i)IwUR5HLe7+ROdI zQ6ng3bQT=%>*qZFA1^=_;Rwc}?^abkWs}A;?I(^n1KZc>cgyW99S58mp$C{nCVlAQ zO}(JYF)Ky|Jbmum*RNk?rKM}U^vW7gdBN$1@(X!%ozhfOJD;BZpsp3P4T6^rIdkq@ zxf_JaMBR7|O-;{@8;x8Jeu7`&WTMf+HLY1=ey*GrA0pK3)s_5w!-+>Xcz9@>?B(+o zPIZa3by-5F$@&)Ol81?37&zK7R6Ko}HacVFs)gNDw(U$qKd+ z_+s^Jj-CDTf9Su8B`QI&dq~!wNR_Chx z?T%Sq3l==1J18tHq+>S=HRG%}JAW8)w&nfTrlwnP12blrxb57z)5Ya%K) z*}P!uCE5^R_`kNDPVaAQp$`o-O0Z5b2l@f-?%uu2fe%GYnR|gIoVKCohdbCb&KSvW zZ~FN&ppKLa6FN$Q4Z_00c>h<=pC68idGhGdkCv8YSJp2!H@|;=@rAPs6Bj5C;|NZk zJh|lL^b6LlbxYpLzf`^mM=n6Z3)X$wcvyk16fA&5J((w)$RdXg20 zc~aBT;9TS*1!+P{tHtBQI?fi#;MsU`V%l*3#Vb}IxOLDEBiM@I6CALsgAu!xp?REj3n(V#C-Sc>d%wXM)9!0WXin z#tsd;x2f*_^&2-R5(zT`9TdgUyu9^JkWF|6aIWBoPz5ookd)oszrp1|Wzmb^X@K~M zl$-Ya%^R8F!^enR*=|@xKZ=q`^{M;*J)*Zj06ly9v{jldo7?{U8nBo*<#iUNRrDJq zb;?wF*mlQ)fSYI$OO|Y{y&F8n`~azu^0cF+#$>~W3nx!j{iuITHMI8IN>Cp^F0*07 zS2#ztShe+E!Vezg1(9j(vs{WBJRQeRoOp2m{`KqDP14r3%W?_J&-Wvc(eSr^dkx0} z=dkJ1lQ=dES`Q-H%-mcTx>j0RedP^X($$o32c+Hne0$Ou*!%YFn&)Tcf(a|LT%|+E zPw0(4kd{N_4aud@UXaCmv$HpE+SCetV`Y;(|KXD-`4Czt;&cm#$IfM~Lu?1o6=LOq zenwWBK7IN+9Xo`G;6WqSl$`8t|GX&4N^kS0C*v2Lr4O@?_OPetduELQby4eq#t_kC!J2%W) ziHIsNIfajy;oGu`2Yn0g6-EMxjPzW;UVle>BPqsA@-PL>U|Z7xtW8kyG}nVhOcd}! zJzdjcEm!7xCDDhSKacexW10H$<(1Pme|r7)?eGD6`ACq9^iCX|rzfUurgv+!-SP9w z;`Qs*#6nkOthxj7((3C+dEB;bYW&lx+)XKPKL8UP6MOa)!Z@HvTm1nRG&MG6WMs6o zw|8B6bT}(ZTYn~HO-odE@)}XKXo`=Is{D+A{N&`j2o7wBTLWDZNjJRMg~H6@CYM{; z6Xfg!1{4}uJR9l(m7W_mfHvWaAJkF8@t5VKr+eahp{7AwT)y1I83yl@+OP;gA8q#X z+7}Vw|MQDvm>ICKt*vd3+ox^an-iA1>k@OXpFB~AbAoqdweo#F58ENXOi|o-#|}Ol zYWR8>L5#2sMHhgk|{&Nb>XZOOvy;4y4i@`Ve3>AYY* z(3B1ryX2gXj4a_C#2-I?Z^CBMBs*S-(c$Fuo!`EFD~(f>5CLiFZhY7H^{WB=)67t@ z!J?uc8yX@<8Q{0$;nn(~DEW^GX0gOXN0M-K^l*{aBu5Bm3~Qci){LtX2+WHXFWv&S zBReAA4)R~Na-}~+?T8Ua>C{V0OUWq-dYgK??kQw`H0rm*8TeR`!ISnsej>& zZV72=Uu1vc1fqrF0urh!2vk^HkMTrmqP^bfbt+0qA8314T-zvFAk*{Vl0P*NtpZ;Y zTEQF|9Ve$+_FNYoh{C|z0?r}7Q-TrEVcVy=zox2Hl$O%vLxJAF6mW}%n{%f`w1*EZpv#n^a#ue%*xv3VT1Y+=88iq#3_Ih9 zcIWBoXlM7Ou5OW|qvoxRh}uDY`g0s1*k~=Fh~B<@iIMN!`}frafp~1vQ&V%%k}HaT z5N2a$A4Il>e<7zYTE2Wcb+K{x#P!qR35?h>3L^|1pTDK6b2DoOTHsXE<`7Fc%0b2% zu;`@BO`A-no@f2A3W4!l)~?OV%iDb2zM{k%oeGPAQDSpb)5#-8#+~d0A1uDGq`RxF z1P8?0%w^eYO9+Dn^XLEk@dI}y5EMtju_+st;8u)(>Og%zBgk=!RVW!FpNMyGBikoE%@2mYO?JsQ9WE`fzP&Wp6_xqFJFFk`}V1v93RpT ztquQf(xgdnv+lZi=gzGqK>-@c`xNt^?d=y)vFFd1bUk(E%vd$G^$CkJe?JSHYGa!CBsi4!Jl-m;~r)(aL7jg2Kx(3Wn3z9KDw#=-bHIzHqGAU243 z1jMsf?Om`N^66BMH!*Q>l;b`Yq(30_;GD5n*AK1%ZJ}myQT= zn$gu#2zn!XQxxEr6OSI9dfLF9GyS{iz3e1sGe7`rebLSrK(s`3&F$6#NJJT(I+gmK z{0Hz}3l|wpBR)i7qr3aXvuAZdIIW)_V|)^r!MZxz=)*u(_}pgAo}HJQ>+_$cl?ueo z%JK%^$%(Bcw}Rebp~2MBA((Uo1R7RS8yjyD7tBNINiR?yJS;m2@=TfgjzmG-fV>5c zT%|BU2@{pys|)_2PD+QcV-xLicoXu$H_8k-t!wq7Pbcv~BqfEe*|3vQ@8N=R&CICmeKo4Z?u8 zTxMgVG-k|9Z_unU<>8sn)K_}D<3@)iMXASljGObG3kb_fEe^ZBRg%AhWM1K`>wh= zF)^|E%a{GV`>Dn1Q?V)K1k1$bS;~YDS(n;MmKfTsU8qHuz7}Bzs{mUmv3O!&8N@jD zOVkA#j1wp9{rvKG@9rJqq$ek~G9M{BB_+k$)rTcX-5va(PSB#c+RmRJf!Vh58)6ea z2YRI(YwXd*#l%>P98~RFW9!EHJIkoihjH z3SQs-{rk7d4ht{ZK`r9@Jphz8w1t^{AQi89dfXKf`Peam->NKg|FbDr_SUXF2PRi~ z)!68!9rvxVQE&DA1AJ-S_)!W93Zq6v{24*OP%Dm~IC0>hgJ>aW2;GpPz^Zrn^?*ia z1a#qW!S~&`Z{GmXLSP4RO72WeNonJF;ejStz`H(=o4i z*eNSjehy!QWi-r6Rej<_vLKt#*4ipCOf6pgqG9JDEG`?r4jp6ewtoF(G7D}! zV@3v>W@cuNj1yHCMAxzf)5k%r?h1ktwxeIyevkC=l8;fqin-VSjnj` z0LG&3Qz8Jz$8dz=H_hEtCpvgAIsNs!cWl7-`uZynPDY6{fci8R6)#_&PffMI?p;sE z&l=fJTxxE9_U6q|gV(%f-SIzIsped{LYF-1dBhcn05G>`A#4CiIt$T}2%`N#keXv* z@g%{3cpN)wsZ++Z@H~^dMgv({ z-@pJLdYIX;ZYf+!h3~f;I=~^OCa>PGVM5g0 z57^YKwf`5hnMW+&cJK}xb zj-Th<`l!r1*3x}*!pQ~aQkD#71$j7pFR!RT3#&|9bk3@NCywl7amRb4EbKs6a^2Rg z<5Y4WwiO>9i@ncTJI;P0_F!*Pq^dvotW{fKcd-k+O3LaXFJ`i*Ex}_{1zxX%6W9l( zC@iph0;U@Y9`aLKp%SnXq@F*YyqKn)r(0*wh_b-X&@pnb!_=4{=nixsb; z!Ym$hYq)MGUEnfxC?6tvzgRQb%R09l*?HMAi*w~A2?i?qBdp%ve)kTink-pD?;&28!3K-1J0WdT zfgR}7DWR3EMER_szn;G}=UP~tx{JDp#5C$j>aV*EZRh62-K}d~a6+%_&6`KHUU=TA z1pxT6T{Ck29MM>zC7Kubk4c%Cn{>rOqhn*cyHq6?zlNQG5T#?po?v#aysS?KTS!-E zvEuUx?I}}cZIf%KHQfWsLPs<-l#5*t$Z~d0_RR(5M}}N*mWI&qY5O$4VwCzxck%{{ znwdFCM)J0ECfL8yUH47joE4bNgH1a_PCoB&6x~MUaMQ7~x0jP?S!`#AJ1>nvm>mw{ z>^Q6$JTZ)_QDUUskD}p-y#m5KX=W%z13EhwbD325*e@TQvnzo%)I@cu$7N-cpvqEH z8!6}re`GTvg%Dul7A=~LcMqVQT2MnZDd^~!8aEHa;9@JQ*BMTIqAc7uZoC!HeS?tb zP>ct-JWHHW&2#|GD*g^P_x)C?^A;{l)OHmKaiY7T;Q>LW_J8#DZM~Cs?zMef>WzwWg4bKu7w8_blxM^$)kqtaWDF7izhfuGXxJb(W zK;TmB6^Dg!4EXTLLuc?lP;7h)3J!cUrv=jT1oBjr1@bHNEMh~3yecj2$j@634#s4H zu`)GfaKAaUIHpO^eW0}cU(!JKWO`CovCGyiTO#DA-$+TBIe)&#gM*_Qe=r(0rb?qj z7CRvP6>Wm`6qd0WEP;FXuUo~?$nRGB4iwf}6FbEH~?voKK%A= zHO5@PH$~X;X&7FuFERDshvBMZJid3#8Vid7`XlaLmiErpXpeiaAz%PI(U3F zswFs`lE+$)G4t`s>1d|Pg1Exg{%sT67E^IZVbmn&y@wA!2Rv|;pmzXnDapwb9d8}b z-g7W$v@GJ(K1=z~!og_={5u`QR~k!;3g0;S1S<-e+AlF#r%a$vC(rI5a4i*Y;dh7v z7x530`PI+l*T+b9Pd}Z|!C9kw2Rnk7aDp^$n%w$rV6drFc<>kZnTP*$ZJY6rVnPRy zJRgaIir(`*pwp3bd~d_6#v$mYYO(u8wz8BTrVeV&o+IJBf5@@nn&Lox?^FPp`J7}G zYctoMdBLJR^_d3r2jHCPUNyI&1)%UJ(NBPAV5oLHEc;wnw;3A@ustL+RNxf({suOi z!bw0CKYlz!RFwV`Jwy(q27XMwFcU7FVVC5T6i${P&(58Pm7KGdos|Xg-O|(j7fw+> zq3PQ^~b6kHC)Rd)FB z&3}J+h!%>C(udpG{h*F2=7SgN!S)1my~Eu znk1NU0~P$u5Bo3zy8u&mEXQD;eBj_gha5L+l$||yR%uDYY0?eT%U1s$kH1Wpm)1`( zqI02D;86Ql3HLx|+Rot+qWF}Sm(QI4HB@ZWab1t&WGUcgL)&IQdAlT`8)U}pb zOG-aZ8{`)DQJ9S-|G}3~r)4z%GF7LcNU+V+wOajwYKD9oo^FEpK_^Y~L&K0W4?1lB zUwE++7i~B)h(LN|e{6Nm&d*Cr745X~GttGpduM21ARFe?C2a5q zlWk*?tSpkO(C|XK+<0Y19^BpC(a~$`>$fnsP*FJX(RoCkiz!$Qhl<38FGJoP zMkQd48LZNxtTUtcpBX}rSp{N2MtIAr81&@?B z>5f=G>`7jmqx*Z4Ug;qr=Fp({k0WRF>=XWVAvIOcz#v>k7=sRIm)46BM3^uud;0@w z(^KokLknk_nY}9rMBwuSd0|uA{`Z%LT^R@a(W5WWg~_vp2@;eQIa%4iwT1oZGy%gp z+B;BTT(moxt|0)K7_iIr8h!Nf70i)BZVxiT(#xs6R0Od<9pOACvRzkdZ|4A4y|t1yf= z{r_&Ib!N&DkE+@4Vb=1nv9M^JoON=rgv%+}nLa-4fMMRu&)*;E$!fxHrg=zyj4w}= z5W&cazVGbQ+1B1}yJ*qedGp%a+OXP=3^KM_vV`_hCd@1=BLhNp9iC&LAxyyTJ$qA~ab`LefYqwEWepbLF4ia_=(+LUV#KnG>K$)d(?`H}r)l$JRs}a_ZD6supHs zQ3WGN28Ev+<*3;pd|+>KnHv=~9spHSb1E%u{Q7Xyo4UzH^ulKsB%Buv`PLB-3L{5$ z;zXAdy9R~)R~2;V8ue!Cu?bCuwab& zkj4d~22+i9Rl$Al7l1bn)X6l=e>(LT%XCS?q_1{HLO^?nV!Pb=0JlWgl z<2P`8ED-HOLheJbGu6|&q3k?bE*nH9eOT|!8AIc~S2#*VHqs9Y-=uo)wOz0ey`bLkRNo-;&+dAP!Lq1fyEK5r_dpMTp^bn71m3 zSJ2c*S=k1LCR%;T2@r{()kK4B2)NtXBU1SkEv?slJdg>P8{(Bx0e1vVX1h53GS>_I z&%FSP7njM+z$FHLhs8NDdF??wx}@Hv7nhADqQ4AIvPr}6!v;lTGZ^c-e*N*p#K@3& z&@yCwXbNm8xv8mce7_HS<{`3B0L3LF_R70*YL{L2K6CSCcSF>JhUrPXe}BX-hRQT2 zYN6p0LhXA^)~;3AQDd9GWj1OW%p{pP>+02w>(|rm0V;vQAHw_`Dtz=uNmiu)bT^+h zixJe0KY#W|_UiDUq~D~wK^!4?SRmpWAP)UR^T;k&{+9aSD^Q6bEX1f4@1prci%X#KbDH*tI zSK8YfC3^fPqx^2hIm)#37t~4A@6SFrI#O~!%s8KtvKj3PNR9|&qSKal>&EqfBPxN zpFe*_#s2nq>F00Xy2ym=1X}5xJ^hOkUv@%l3m)${jgFl^G4M*n?>Eed!hG#x3_=%^a(JYX+mDm7G8AO-<^f;X}~&lnRNBmS<01trDBsHw2QY#!e_TCX7u=4~msL3U!49(8Lm{l}80i7?Vd3l7 zyVv?PO`ku{v9x4<7dsI(mzXxoc8CBxIXVVf^mpC1Z5tjLt{5OkL0nT0!OJw&(b0=Y z+yE@N@DPbB(19EP-UB5D&7f@8Z~UObdnT=~P=&r6F+yeR*eNVvS<5fd7B59e(KTmO zegoH00)_XCesn%2#`eE9ad?C|9tOz&#Wpr<9CtRLUSCX0i^3M1kpb$$v;rE7o3vW^ zhC%+0pN|+|T)40qs*kZKdP2CbIQ=bZGAewd7q4DTUGqd*PHd@zg8;jUg))vPFj(0Z z`t@{O&(H5_$W}geCMx_*&e+gsjy~S`>uv(2AtlZs&GrlN8PHMNzhaSC?X&7pO*}s-BpMNV zbjj_RxowVDX((7k?4P+OdI{Z5AKmjx+r`uKU;j@ReP9_jZ~m+K(Q(0orUvFFUS$jD zt@M?%=)cE~?YAJz^C0SjOb_nOn)rc?=Q_^vyXB6luwa10P{FM+|Bp-cmTPI;&*quw zzo>t5($d^^LPJmD)swrNmnVC3^K3 z8q>l9joKVnI6L0zx|XLY)G=iD%Y?v@j8qL7Qhn89g_+qsFs z51liq#7as^oYq@v^@xbTp6>e+i)-S4fuUMjS_-`WCI9X@b#yj5$ER20Hg$TMYBrdb zfB!djRwgx-XH>7uN=c~$BRZb+X1s$c(yUta&-uQ-dZC4#@%x+TpA#20|3!FI6U=7Fj^rP5 z)yBvZ=WAy4zA>#wTTZNjhLRr7CA*S1!2Iy4yqqAXl|%`aNi4aRojqvqV9CTjm%v5( z`gkE;H1>0l4k>>4(7h^u+Vtrw^M~e9eV-nm!mH?LYySy+ZBUYN_Q?0zW}W92y^}&G z-W+c+d}!ZH1zC|_A##~cefmk5%o0DnG*fQpb+unrvu5p=@0+POe@CctjIrsPzfL_R z0Y8L)#_VjGUjM9l&yc2XV74dUDmT2e%YI3j0Y}10nTV(nN6g(Z5d{?s56iq^|NfyY zb^yAPwziF;_(i)b^s9@q#;1=C{4n5iVvtDT+H;2=DUR-Av0{RR zNU;k|rS`*dkL$fnyp{A5#B4(F5FM16A`)5%Ns5DImsOFXtQ*d2)n#AE$O{(fj-Y!702*LSz=(6!~CJPP)79PB*uGrb*mepdB~wj14;9 z9fjKm9@n)R=if;`Qp$u=dHEQr@c%AcVV;N50|A(s8CB$5Z0CunhxnCP@?ez~MC0Dy z^LDCp(T9)y9Ozz zU;qB-?$hZIrcLWiJUXMx(-I;0N!(-tgyEZfw2~-`n5d{;+Mc_#TEM)B&6PKIa-G7I zSj+Ft&5l>rzaW)UeHoKAY2xmkTv`VXH>V#t-iTKXRS2-4U1D%h{3G!T0fUsLs;LR!Ms0PTK0v9_bH1ja&83| zV*Jt=7Dfh!Q=5D!`Xg@}uj}!nwYBo9M?BC6QG1d!cQZVre&d9D!#FM4thKdCl9icK z83fG#$X+MtZ=OHjt;p2^x9LWXOkBAG*vmFhFnGy5jn@Y3=8_~5A?(c|V?{ia>QeE1 z)C(76B_$UxU!Inpt|>l1;8*eY$E-piQ!c4%lw*!9moVA?rg(mTpLg_eJw)Whqj^a4 zkV#~DYP1>GT4CNyEsf3gaif2u8Ee6)~^ds>K%8zN3vs{Hv(YF*H13 zrqpYsj*`+{?(eyYtZA_ z8lW{wUcRZOW|zod001UXEWO4t^aby-Tn7ai>8@D?EQPKP9CYUvTP^Ymh z10BE*FKuI;J>3X}-b~3_c^In3EOYaNT-89|_dl&--`-dl2bC6y|40;;$?lPcVPie9 zZuGKA+*p})2E#oihVl-QLpfo_S4YQ(JCw4P!lXu7^j%U_5W=Zphtij2-UY&b6jD=H zmk2Y%g$Tg|6h#q*k>cqz%jPLTIuE@n+h3U|igN)K2ogq)Py$9_4JA)aZjI?mXUcRBLnTr*XA9%ySj{T?BWB$Ti`;$yN~B4i5{PO zf*4X_1Q=Wk+ijEPliPO4R*gt{`tTuy$bH@u<{AMyJ~s9zgSc|~K8%b3$yVO*S;yAV zK+%V%j$5baoY_HtPh9LLVtF!FQTSPk1Q#EWTER2GS!@DBKAyzoOLyMP5U=iZ!6Z6^ zW|*L{j)VM{sRL@pb%Z@#v5 zQ#(n37;8CDM~*DPyAff&lP(4`7lA|5v`*&@8Pn>!yN{0#tttl?pp0FHTt+v_RXbco z6Lb9dm)crhZ}iS|mrNsN466pYoB3CJc4c*DYA=g06i+B!Oh-X8}%yxYgh))vdk zZS7$BZ%iXFJk%EZh;3xVd01sMiJ-q!ck1#W3+3IK3g2Q1#wp%$fi5R6F)>Qf+^vXU zO8-o2M^V5ww*}vuxC^{DC)>aK&lboPQ`6mCGPYZh8|MZN7~r;H!{36RB2syLf2L2E z1F<~w9D6)#fda#}@Wh~xX;>q8Wlk=pG+Io;3`&F8&#FwS3lcWtZ4uKyei-gjr=-J zaYYJan90f9bA{iv<5CEs*ABeWOek}aNyf5oh%DGJe{mfX6cL;}aE)|dg1^$1YdJts zB&FrihcJ{8Fc^lQ;M&_tX5dxoT{*x+oGf-|2NMMQpRQ@^?81Qr#dK)OI>c(2jw2T$ zxjv79QCLH09+Lku3+x9OIW!#*l#>71uGJ<^-00yk%VRpRimlEqkl`VvD7TFp18Qyu zk*_b?T~TAK;;N>2Uwj}WAq(Zh7Rp{n@iXsE(ssVL4-N=i$W>nSoQK|SA#}N<3ovLr zWd&=*9ge4Gugq9w0&Biok$l6t8=r9f@0OGr7a+hPa}UrzqP1&wC4?tFj+0zGuv?Kt zHF?bw%v{_db>Z^mU%!7N!wD&ULLCa&bQ=ob^rcH4fNkSpKY#yT2)p3vIa?gvzT?8W z^NQlCOvI zP`E-4y-=|mYJ)kY>@%&*J!rP0!!yph$hJihPlmLvFmX?o}DkR|(g778- zk)nESw<%k3YI&$hH_X4WqVWo8iv`|inA3Np%_8;$CmSO$maC(frFsB*7|Ql*SZOY6c~$eHd9b^h>#(qx#Ka-)+u8Rk zSy}tB^?&^u{bKB9Jvl4&KN-f$`qpzR*Sr&YL#J1kmk;-iOzNI+!JfNVuhg>X+MQ1s6|WWwMusLmH>|EIs_MI z#Ef5r7R|LXyT;~_<8yPDO_XffFFAqRBcH#2KMDo#&MJILTwg&}HmUjsTq0xQ@`Xtopn!GnJU+U`UaGebJ$~;nj6#Q^u?Q8d6I7%T* z@p*DuQB%3?PB8w$kOgYRG~=tiaZRwmlol@^9}+!EPRGk~EFV(g7!Ux+8Oz)`_FmNq zq+*&Ca>T@=JF%~_va}fT-B)sR1#>MRO!+D6C4~1Z=a#~y*AAw{3e5KMRMIDmCCq?( z0vZ#n{d2~per!#+g=zhEtyg2J8`foY^}de`jad}{G`X210(7QY{`>M`0#+*sO94On z*(b_E;20-w1#Ix6$3?i4+qVe&2_Q>%^5jD2OpsJRwKMc~wkh1QO0rG0RXh6ldt>9F z2_k#$z|F$n^)@+#z)86mEwLLCBpSAIOrZ0f_ebx1RQax0KG9NkB;CtCw6H+3Dlt)-Kb;7U}KMJtRP|jy7b~Y zmri!Dl zctI2~$_xD@x3zLN9amGTL>D8i3$D~b{6va{53(E=O{GO`I-iz?oa|eBw;ztze*JLAeID@5iK+lh`TPnir<`3b*Z_}0xq3C9RrA3 zwhXUb=uvL?EFk0KEctJ~QE>g6vvU@=4pjCNn9RKH1uz{;%SZ2nU6kRt-uGV<9`O3N zV!fWCwBzp~Me)6aO{*}uvhTRrB}R^AxRNFKq!0b=yCJC zw{AUn-~b)WO=CsMv;&&SJBKnta9p(DGM=@G#kP+D6wl_Q*<>MOmY!oki*dRMMSGxNxD&Z3u9ht#bp3dwMIb@+sUS6e% zE9!PRH$+I89Ab3kqGQ3ydE?%`ehr7ur4l$J*3q+q%kho;UzDA9T+aRf_8VtIX-F!G zP9z~@M^R^2NJ3_rg(NN`GBVndWRxT!6%s3S%phIfJ!K8NQi7BY+@hxFJp(xYNpT|*`yGBh;FPcOJIiM@*LB`NDf{|{~Hrz-4 zE8~Zw=p7pOxtDbR@}d(54&}#@Ii4YOxDZQIr^;8~$(pGk67A8q=y7dNRedAzf>Ds}Ho>O62=lq4cK zIcx?5H>F>{u3x)mJ$`)Ng-MJ!TK)4625eYSQV|4Q^dVXx8P*D!GHFt<(8xTbBc{71 zBBbu@Pa5LJ-^%bkl|~Ecfy~9J`dLDv z4jeQ{RaqHM^(gbvc6JY`MY5)3om{bUrR%a~73m&;f8e{=g_#l-scK=A&YV#kp2P%b zge?z;LZ^b#k3j<8N=I5!MkZ&^9^d^!%>s>{q3xo>l^-9^LtIG+f<5CDyx1O^(OZzL zf$7tMFP_x4XvET1v?>niaoAkwMOvdl^qjrVxJhaB~t7InM*Pl%4@%ChtAAM^JkwycT$_Q*+wvX@*cR-7PT*I6|PLfvmR7-|HsKmLXo z!mUmA`tK>deq7(P zOwx9nXG&R?wTUAh+TO|AW2j*MN!I4w>$ItU9~Do+I(xQ*aK?oSM*A?Vc~n%_Rpahg zhCz`?{+_nB1mD~LbJ+f0NAtc$9mrf6r>mVlXJ;)J!AI zp;3F#cY2x!B3Ct{fkDMetl*mcQLAVs3J^3idVAg_mf>$)O#5@8?~$R)@==Vor><@T zdV^ewvv2%&Dxhq|wRKBTaOuxiNO8nW$GxR-UHTg?IjF;DymUL4B@kRtNBP{(r1s5! z_z;e`r`sy;|9=!j7>K{o#m)#DWRq zo2czFe&D5XpO8z2TAxc1iZSeQo~6Y)bLe{#Qj@-Uhc{Ha-(!Mo43H+0ICfc|Aoib~ zb$!KSLy?4VYI$h*ef>xLrCF8~p?5BNa6#(Bc^-+eNm%gvq4G^I{eI=L>_$A_<#+O#>6uw)i*iTt%ayX;@ z=+0c)Ais@ua6JYZd1@9dQr=`%^^hE9@n}Fb6-9K`b>F|wVB%O&M8UPSrIBlX3?vYj zMrfW@QQUtjvi^ey;{}VQt1d&i2)oy(6P&5ZI?6AlXBQV6#}sCOynB0(3$wx7s|TJ8 zOo`Qdf~I*gQQKobM)OxOc|#|zC=z}Xj4L3^WXKTdQG)j~PxblvxhKz`F9$_Ncn|PF z|8RfhR%EO#%|HL`6(&gf?4NKJATUXfRLQUt#Yz0d!RY5PkAOvvu#INW;?}LP{$k2U zD(&;<&QTM=(6@m}@d@TET7-ivFBoA}JkYr%s;itf#kN{`_rV4ZC+MfpmTNFdA)4uP}Tf#-})9jjSM&(B?3v zE=Y}%7D4xK=cw?d07^hOE+^O{cHGXtM>3C{&f+^exEc?l?a|Xv0;>k$X*xW15rjOH zg}_(x^O8~kB0%cdd%(Qnu(hwuAss~$<~4Aw09K#~1QGv3>z(4r*}qOXDUy&KsbtNN zKXSzAguq|?w+Z49Up^TNKaQhKCR+lz(oP-hZDAqpOHRiHN)v#FgjsWE6^?swGXAhhScGQ z;B}*Dr&%h98kUP)Ag()PRGtq9`6K!sde*W1FP5&;!3PE2cezk9|aNhoTj?o=Gp&XWOve9fTZGn}5#6f)`A zJU@xa0?=Y;;QF+w*Jz0$;TyNNWz0p~%*GUZN5{sa7q3$&fAc(((Fy34>h>A!(3^Tf zA8@4ex~>`;Rtb+M?$PN)e;ppqz`aWnS~9>o#&dY3}5&sbABhogW=tOpdZiT$i>DW+1tBt!TVH^HEB=^mQ zc~@)=G?=ENMSKTXz;i{X1R@N*fJ~1tKr~131c*R01t`@yB!?#yk(fzG#zh3=LTG=9 z-y+Z)#>bOj;^;`S2U!8?LWaQ28>X{|(93jhNX~1-LkyLh_eeT_9{J}(XA?YTx!n9L zhr`3))Yd{;)BdA_>$@!ptVaOKn!e{eecDA5%75RFw&!?af8F$Bxw3KyDkAFXSq~Sc zERkbQV&+WQ6aB+%kRP{l5$)ruVYx^R43vWQgS}7{fa!^4JUMj#*D#e&CvAY13wJjdVrT*YMs|ifI&6f;h=;nO zeuU)(Pc2<_)l$s7wH4#e%p=hL0#|^&xFqS0UVjQy96YK~1D{p>F&pkfQ-Izw@<;(< zr0MG7!k6Yv%`iF$;+!AhE4Jan;ibY=VYLQ;2CaHg%(Vqy8RzG-`|A9rn&wk-`RVfD z8Zpf0Iz#MWrw#vq$G(p z%fY0;N5Uqli`WeVez+UZDzjbv*8svc65ibF2hY03(uv#_Y^;bE%&lDn9$UQ{JBkj` zQ-A;X!4{FP1Z2=JMl$iSjIyI4W=x0$7IBv@QH#ZF{_?WV{3_aP9OM}*J8|liu-3^i z{4D44ubne?!JHXE98d1RnHtOr>zs%~hX~k{<4d5J_#l|t(EBI_zW(?TS1~>)KMopp zrpz#B?UUBbC>$NPf@>1zlbWs${U>IbPlN3Rz@EhGqamTq;(y zGSFCFaHI=%xoa!O0f6y=|1>qNGd)Ioi;i8|BLROYr1Pef1$BGPG0Aj48pbpbKQiAJ zPup3<@Ve@M_^0Rs^v7=1TtH7C5S~D3g5N-TItQ*(ur+$o>ZGTXopX)nK<|a;Vc^h? zxI>tpr}P4)*!T%|h_%QgVq$#AvvkiV95B$~%P|lS*E(wW@FKU%7;rEca)Nb4Wo3UT zU%)2_rP!i~?LYc1x_Is!Vf;+nJ90At2^qeXx-}EwNE`9&txTIgV&uq+97K{C#E5ME zWJ(?Sq~*(Vm!xnXp=Mz4pjV04X%tDtqT9;>Urvsc2^wg$&V*IVdg_ zsa;~AfmnqR9|XAn6n%DH$vHkYe8miSTz@gJDmShQcrTUumcb5@#CH`UAVG>o*TE%V zrwdILBfB@ZHx2FE_dPH*FLR>03s@)7|MI0vBr5VWBr@)@7Y{H_Hy^DG?}2bA7z|K& z@%PBmP)X8fvM3FaY-;Q2ZRgDLWubvL660{##l>m2na@LQj2sN%F%l9cIyh`begIxb zTBp<^0MIz_zg7DYGXj=fzi|VSgF5NKMdeN+xZ}4RWfl>E_eVxX5(^?92%cH|M^Pwm zd+xk>OHy92tpd$Ebtx{TLJXlI0>q8sKznLwHtf=fKhf zic+kO!rW3KspeR7fi;3fnq|m&!55LN#)eIJY5^IAvy)b^HBT4b14UG?urmT*nrtst z_ua-g&e>U@J)0Y9r(;J);r;QWZJhnxCG}J)#6;vq%h#;=T3szUS&W`;1+CY~CqHjg{YD1NW9{nR;ttb8S!$%eW*DBw?Mi3uq7mjaSh8?B%{`Rw+B z6L#J86X1SNpbyFZ##%HQdLcQy5d=yV%$nc2wGcXz!^{yQ#ljzOu6NRrqd=zE!2beA z0&K33VekC3w|$>czkYC5vf(500X&Y+LiV#PgSz2l z?VOr`Cwy;3tN0nzwCbYKK5aKwHIeSf8zc~F3M$<57ca^y1=CWJGao&g>g))G&gbIp zXE-(?V%| zqtXfyWV=i%f%Nr{XhyyP%Dj+3OMBWU*0zA1XLJtiLSYG!&>&oV$w})w<;VsK>UZy4 z|IT;@stV&-bEkALEk#XC!8drY+Cjt3Le{35!FoQH;k|$?B!c-Dg9EL4dTP~G;-{7%P^@B}yX3B-`=p5z6_Q*1+6ir)YRBL9 z&?b$OWa*@c%Yfn(j#8aXw%#EfP_-baV9LAb6S924qh@LXX{YtJLzgR3^oE*RTDX(2 zz%9(q5QtlUM4sPfB~;r#NKceFvEK)Ip6n!=&cc_+kHZJ}zH%-4g5b~GmNk^b>-U%7g< zin8*y`qy&7*4MZGT*+*8W~Q%oFG?^~llA(<6+j+b570N!5%9t8Ih!zk28?}hra zBOh8$D1fCyqxc&Y`35)Qv|ImSynV$@$#lwZx9PMTnF*{$dvJC_+uB!QqZK~}o-^It(+)edHi-7c#qfJ+bpB|17Ga%vS=60&M*s`_;`9t zbZ@+%`}d8EjUVD;b9HNm?1rV%p3@|wYdvrq0Ff79kBj|Wa%Hclj<2*Gy?EZFNoP}2 zzZb>*!yq){AuQGQ0woI{P5huiCYG#1MjMD><;z9w?mQgKOU~`B$ zb$-6Bd_XazELNspz6?xUd3*g^C6j!yBA!$c4EDvI{J=fDf5vf!#GAfPHbnT zg5a8l0D%CP#UsV+*3U%lb7S8<53seIE`Ioj2OE;5~cpI-5{{P*-5L&x3R{zDhhEH0bM} zfSc%!1`jR--Zj#s+H(O4AiHV@TZ7QgyS;t1zZlUYkCYu#9AaPrKIDcm{D|AH)!%vg z^d~GvoLiO@_MD2MuDI!A2g)EAcH+GAcpQt+mz(f zs(iQ(*HT^vs~4oaG1AUlqV1xO>R3RV!AUJ%1jcR_?Xj;oX2& zqn1C?WAQK*7JW_Z$cdLnF$5QE-F(iz?O%W>zN@7irpC-#$jHjlg%x$K-Ef_4nSg47 zzgKoO@~7{u_Z7LHqzF{6z`NKT=*kDI!|!iN3dKlf%T}b;blcpEZ<HS?_;LUwQ%d+kIVMngN~`tO>?pxK;q^IlttuNGmM|^u!4d(<6;w zjJ_HO0tiu3OFtvm67PW)!S+n2b}BGJhr5GWmIvZ_*$2{GF!IsT?$Skwuu=Q6tEIZc zpT9PY;-06QH1A42;geq!pCLKW2b6>++5y2w3*R#k&G<}xTk9V_H@^_RID4P6H{vkW z9uQVDS`BXf&%<|Nj16=2MQoccb)Y{kX>;Xchmm+bn#c zRQ63^2b;q6BH9F~bxsyTf>f|Yn!?wgu>ac+2D^b1==oT2!~97{`5%;8zzM}}!wGw% zA8h4fLAKj_WP>w%57EtKLIWBGC*m(3KdwZkmDbdG)h0Q~2;cNmjFag1`zlt{=+Wc& z;lqwIXJ#{ZiWAJM|Z+qCHyB{6k*1`Dk=e3-%D4Mzo~00aUM5w2hKp|NS* z#w7wQgR`H%e0dfPCF%sg1x9nQn?#8rr+R=J6FbfC^kV8b4kx$b;N$buXk4D|=AOxf z;gB4hRaWDab4~ZnAAFVRQP-d{hLxniuHdG>~30)8)4heB%hpn_hbg^<6vK2Zan`9RSpKJa4;x+NJB1x zaY?wq{>vu2k`kGRyC|V5D=k35&Yx%1F&UL+S1oPp{h=l%p==Q*7=s$nF2Y^RUAU0# zw=H}@0AG~#%ra8tT*57i5a7?6O!Bq#(nQkWwQHUTzu8_v0^&MB-SWS5H9%0n z;%wQVaS^~hvH&L*G@zxhPNngO*XGc6pWZ3wS86usHGVv)5EN22<1CXf0!jvl_vFcj zySq9$%ieL19~8Pua91@kse1QLUeqb}y=F+xnP&V%NuJUUQnLo^nqz6Kg>L7B>-~xb zR#*P#3lNQCS$+YzV-Nw8w@KR%Ga@)SC67{jew!3x06zlNgw1@;66sGPbV^fu{G<)w zjsVbMIxiJ2GcTT~w|v<$>CUSV1vBBy(i7`ml0p!PH}0(I8pCkh>`?zrQq6G;tToK# z4A$IL zCg$yTI4mGvpgaI4)ao@ZyHFc4ng-g0lQw2&;$$SHuZoKiHw4?p6S&|{N$^!x(zEJ$ zYnCgWI0Q5h1TkjOnCZB3Wkiqtn>{>G@W^$34N#4jSM=1r8o?-YDSKc%V1}4oXB^zd z0H8oNGK^LmRG6K>JRO{7xKzwoe)%$%-!wnC3y!0CJkH|5d-skQHjLnnDz~ZWeGDA} zBqAYO65yO7_~pa#)omb35ophB?vxKPh$|;dYafj~W=9C9^m_b9Rsi2%xr+-Wp#P9d z%$73nzK3n%8MR$m&NzuI8?|bw39i$T@EJ6LGJYl~AOYbp z)h^BFPO{KWTT_$aI0*Q)3^*9Gw-zRnFy%)VATZ)j6XDM}0$i*~V9COm-V;&^>oW)?H0Jy~ ztnGAk)X^D+{@2-wi39KwpiSaA<*CiIX+G>C;TX zt$s>{0ZV%_W5(GhI82{hG#4ZTtt;lGPo6Z8A1&HjJe-_j_!^+R$Cieor=$%K*uF3u zDk1827dJPXUSUv9McP@IMPsVe&-Dfl$ zu=W`%x-C$0CQGT5`OAW3B8V?2D(8*1priaHWC2C*sHzwU$rv}ai{|Fr5F}VZQ&P&b zeI&&ZH-uTSvAGNfS;e77d{|VZ?)LnxQuX?D0@}2xQx{#|(67h-4EJi8<^pCQ!4wgS z>*3=KX-K`#^d223^dbj?)62if%A-Wwj+_7H&1u)KZ>5!CMDm55=`kV=L$p=To&%P> zzS|-F%0}#Gao3u&c@p_t9cMi7BoIj97Adnoq0cVF0FdzgB{4{=+p1_V&aTz~_5ep6lQftl+ zt3{wrc7XB2QgMWMXq*!;btu=vD_Klel!eLyF+pCzznl?L6nP}B*o`iZda&i$m;kFd zEY)$;l9?>%QgUW@=4Qbo~F4O*RlcIM0ZOpea7E%Bk}Rl&*wJRJy&Y%b$`T-W0M=P2`@Su zZYhqfpK4=J!V9o;?opca)?w{R7Z-!X6B9S~cbiVL;k`-G__Y)(I$)lvp@Czt+~G96rpHXyHem(F-%-wj>dBIhe(#7qC?=lR&hUV3vANuwpr8lF> zsBOtJye9&^^R2+Cn|M=X)6CUPWCL!xbM*-Fd7sILd@WbPg(Bj2PE_5sdxBv&T`i3s zbtqO1d9IgW%Fj@N!?Mw969uSw)eEeQwc6{l)reSZ_+~%?_uODy;L0Bjo*U52MFx(f z{9$2TKXpZ3`aNf?5wJKbwrr)@f*PwfdYwj!tfY3 z&+`5?uRm993;-yzXWw=-o+FVg=FZddkZu_UM-7r}V<40N0{QW}bo@7-!VAj$`3;Gv zqosvqJvr%XWrmt&qm$Pmga@dFp^q5@#WEaRBsu$8em-Mh<&Pf^BavLas@3p>QE(<9 z5PX4LhwR>6^Q3|l&vKTR$g#OlTxBpD!D;Hob<)E=#-YE?1$yD<)LoTV#L~TdAjTBrlzDwmjN4d9;Dq*le0;G^n_&Lzs5NjIpdk_+d2ns zifn-803APW7X%-%qq3qVLVrkLwO-K>Do{TZilpSFOCMv+^6J%^zaC$@ggd1`^0>l+ zcFQUyNI7~`b*qzqR{=Pqks@?(%~;Ga;Ur@mww%DMEWhk(Km@+E2tE0V71G^=-8Z3% z;LCJe{0aF94AgI~7xg>A6AU(UQf`3PO(HIOw~;T#4x#7|v*7?+8-_EFtj9ox zBSDtQ@mRP80u(rwFqD^0p%R^vQoq(K1=f#W#hE(l_bYW7yySg&RrVHr&kztSJw3hH z#y?#O-mb0KCK&c|XaVBZo}KM8YDtPG%08Y9RrTKuJne{wG5pn7*+7uV`1{hxYevj4uI5l79J8c)Cd!PmSB9^j8^OQ(2P~ zVSR@R0>V4vXMYs0PM!b!%m_F?V36p0P{4$de=5(vuY6nIvt2p*XYT1iZ3+ffZ6UHV zcUXyiy4m7{Da!Ah=P9r5Q1IFqsQD$fRmgTTFh*3*fQ z+_$!+F${mhb*Hi}dwyR8ytKpLj@1RR1DF9Fluv1^%Qi6-ibaD{$CMvO*#}Eia3>=s z(qU7}j=;cHGr8S4uL&^tVYIv!ju;vMiTWva(?Q0hZt2qY$5%4}4XRUx#sDB{szcqQ z1@G3UBkZfR?cE10UuA|x%ZMhLQ~G9jL#Lab!VUqas129!D5EA27w3ZNqo(YbU?!;J z+i!e+dF9ie*N=GaK?+5GBEb0tD2a&3c|$7E+0ZUDqyyZA#58qn1X%5V1`5a+*s|Q1 zJI-u>F0re5UveL<5Sojcl1D8QCrz2MFx7=sBcsFf1nVxOfGkz@E>QUCzygk63_Dqv zOxj3&1xoSzWOLgIc~G&=6!#gP?C=d$+=HFoB51j$N$;MjGpFZ_^KQN$W zjWZ()H5s|&3U;JJejzY}xsR}ou#Lyn23-w>o9Qn?Pw(j8XcsMRk$~2Iat-gxT;s zk@U#o>*)DzBW6aOz}F!G)1zQYHpoUIg&V9mrvb!+Zv>XgS{X-U*P&6&m<41R&%fF+ zC>w}@77D&RA~KS(#QUyti0`=Slzw)GaPX8*>QX{BFxEoF#|PF^eO_8hmb838t4Eks z3M^r~Zpm5XC+sU+mtIaog`slp8_8=sr9cf`T~rJFbs9hpBQF9ZCm@S46uD{mMi-q% zO)6J9i5i2qXf|dHI9Q++I2gART!Ym+^QCLmMd_&g5bV&!K{Tn%h!~4N2t1tnkw5~g z&nE%Ofe#GIhE@si)rRq+aZE^15RKLEad%FyFxF}jrpceBdmNyP_&IpeF2_L(VQ?97 z5uB`S48S^tS;68LEC)g5VrjnFyocILdoIY=xs}C|fSY(1ovFs&hCt0RSGOlovyv;SY}Kz~6<2bB(p!^K0BqdGDZO_%UDw zKS=hVy%Eb-^wCrsp$fLI?F4Z!I~JDxQ7njIU+{hC8biuNf(ooN}}m zNj}6jsK!O^Flg8IX!Up-w5;NNX~nm%9}JK z1P@+x%pX+EWQr(uP?rX!`&+JFxx$J|nGdz3L+}vsAdE`NOG;9K?I6$W=P5U6`RN+z zNP{$*nN?1j>`fti#i==}Zm^#rPq)e?PsPS0F4x|4>6%&OiG*sk}FFoK4$ zE7!6zs->gHdnbRNWl}%auAkZdCRp3L*ASq7m~Bki`Fm6#M1Ll|L44`oYz)@By52+N z`RLIgzI@M~5~_@4%j9%$Dj5R01BrnN2?}|*J$QDO8|99O_>4xx<_}gg+y;tb96-xgD)5p%UE?1OHrr2yaArKt&JMFnh_&WOC-3Ulukp7#pO}cnO!}4|r`dI|cj7jj z3k?n5QDL(hg@lhkFd|JR;{`tL#}AC9tb27Iy&l5JfDCM5;RN1fXNV#br{f_82B#R{ z;KNewG7dxCOyw(hM~KyVJf7w_`*aQM6oaH6>sYub%xe&x6X{p2`p<4Cg<%6$Au!2C zm}5;FZ3(;=daGzUZxDL$If4hp30F-j@HA2pcmxM+FQvfs0B{s{JYwDMf`v3lf{?XQ z(IM6#d17dQ)&pdMPYtuhGjDDFl!RX%Q{Hxl%r&7Xb&aUXMDYqik3va4ASxzCVR$|& zPm&Y+%0<6Q`2^fRek>-`p*vHMjsW2xo)Z=g~|ri46-PmD|1hDE)>MjlPm}nZ9g@9yl5A|=Y z+s^w0w@tZpsj#4c7b-A1{cQ-52m=IinC-o35K{aG`x)&6dUWlI;p+x!+HZny8k-i5 zMMcF;D&^OA&HMZ|e z|KU_!&5H3UQvhP4;4sNNJgPt=Vb|?$3Tsmd9|)c0qpYmz z+@mxSIM&5!%e9OnrZEEq8~UcceqGO=ag=bVNxsZC-`%cAR+krvfVb`0Scc0m8Ufm3 z%EjY=g!}g|U%Z$F6=v&Ri^S_!uU(tLmJNS@JT7CXCU~n65wqbzPM>~9)du&4yDQ$G zxrK!myakRC7sF>#L^p!xtlpGuseD)35B?oCOGH4JeB543j+$Y}#sD~z26WlX5XFat zJlK4gh0sG8SQ(trA~Q=B;MW^_)+412kEdA>ZO|u`#54+9B{?Fui!c2rNRsev=TNeB0hFqg_ysvg$56;Tv)YKDw zqsS+Wi(n5;7YL&Z4$t4jb{o=`tI1!cB_j4YpJ5{YzyX1=M19HpHf;jYjP)IePN#|0 zZ?F7n#|pAhqoBBp%nY1OUeHbIkKHOt9ZLZQ>_`ty<+2RJb2m3eEIy%l|MUrI7^+O# z9ik0_SGR3XA3of}S)^!!ui_hVu>8_71g4HS8XB5eSm-o=KDsYqJ@K+-Ke50dOR}y* zBFQ4dbG51@?B!reK~Gngs1EQ#N6Ae{==)L5bqdsYkNt#}R|EmP)JKlI%@G971GEN& zLN!D(<4p9>)orFq2UR{%T+2x;G4$XJ5|uc!Bu5H6L}PxxW&C4#(}*%qvk7_xk&Lvz znCB=S!3Zz|JA6h8H^DdyLr-I^`566m>ox#K@1$~Y>kU-*tT3cQyqlYg!416$$OuCS z_?{tCM>4e(*S2WxWuU z5@twXSdP<>;PPnOUI~XvafJL`X7+g!54)m?)LeJZty}5RvFH=LLcVpFMxhcWf#B9f65?d3npW40fC?S+(j|c=$;HI6Sc)h@4Kt{cBTQwXE~vwslaC z?H7$-@&f3CnDhJa@gvuQ;Vk`K;(!y6$9abrM<_~N*0G&MG&o*?8> zUv`kF>+8pWV~_?Im)G6fqgyvzNBpy^u~gVQfdr10Y`Trj8I+%+Mr{VS;I2M>D=Z4n zeM9?CRde^Q2#d9d2o4Ii>8#xI@bqjXzTW;!<&lVw_l+CdP%iT-H|uT0On`>`=FOY1 zwa#;-q@Iq1j9~KAfCVj3)u>h4~nmGY%z4>lQ9lk%V&7 zIkernm0|H>nWS*}8p1B;yaVl(<# z7ri`Ew7k6eE@*G=Rr85oo zXP>O?aP3z^15?Prlaz4M&jlNk`==d<9i2PN-n^YTjrKJ#X9>ZVB8r`%_)(#>(8)UA zE~2mFh@sA+sN}WtR1)u8Mq0r);cjJaN9-V%dj#l@FHDf~o&AAqMbXO{;O~mRj~O-U zH+vq~6Ls&NA?$HM!IWNMzB3jg>xWjNKm=RGP@0J%5C`1m*$uwdnNF3f5MkQ|>2_!+ zVGGqw)X}4YOjpqrA15Xn?>L(f;%H+;G7`gZjt!*^{xSI0;S28&z7Ufb zm@*y2U=7^E5n22p&fUC;i6kqGADfr)Nx@l2CH!m@HE8lJ#ROVan=xz$#CU^wbEQC; zV-KmtIAYEM1OwSQ*U{0%#l_bMM9`Va6g&ciiy>`TWrPMbHRa4z-|s<;#^!{~4YJ7{ zV>$%QILUz;PI}Xc25pk5cvKjqL9dgccuEmCrL+ zhXotIb*pYj4uH^Og?H_bIJ>dm`ukZjcS!%;Fm7Vb)Bn{1_=qk)Q5O|&OIR@@so~HW zJVn{10#Z-xr0R^^9(P&G(zQc`?cQCx7(1rg8BNhsUVe<(TsA!y^x}0LJ${@W=rX!v zc)}2`Y01frhYWSS{$xbOU~5Zv_`&N9F^j|0u=}c5PI}d>%l9O|Ys)CTtInHSSPV8Y zvPoLm@boJyUw8@68W3z$V(ihf5I>q{`k^;`C!C}LfHV;_X4J?=M##Nytb8w9RbAal zON&{5*(kfyPSO>+#Y)95IgQxlMkR6Y4nnCO+|CxeeCL&Q~XM=B`3o|(!#8Gxr-O|%2_7sJe?s> zJ!mUHAW0Qz00Hae4(pS}nP2hbv$eBxadp+=&h`+yUt*fbTb=s~>y69~-2R%aw{ne% zUyNT-^|?p5=F3#NK!>7H0W@w^sNHsDrG~#N%^MGVijVpRwa{XpfA1oj5qx%%+|-!f zDtW!LvZs=K>C(#=w1&$yTRzS^eanfzj?4+<>GZArPAhAMJdDO zM&4qsRnh-Wb~c}pc?^oD(0ihnvq3Xr`=X11e%})uOgTgx24w{2pxY!iad~NJDSY@w zpk?ANYqoBEnVl`LDSRhRre*tL(jHCF<3fJBS#qqMY4FAA3uz!=W+?jksc0l_AVp&U7#m)0?U51xGRsS4 zOVI@)jmrhiFhs>0`~gOfX8#Xk`L-VKXwD~&A77kwlotccLrSXBnMXaXqr+-IHfnd_ z4q!DwIV2cDTQcB?0}f-wSe>)RLy(RNMg#)~vdx2d+pphQ@F7ZVs$R4u7idjMIzYGt zc-SM$V`{sU_wC)gkU2D_zHDsp?R2CBr>@hlPo=S{VACjl7y&&3$~HdrJOlqV{PriB9nue+x2;MoLg4UV3$TKVZl0WJUDR<`7#9* zh{XfA@G}Yz{LAZY2-eI{c5-!-9{3eSBmp5PJK^+cic;P_0hw#<8@rT38^u2J@%+c; zH9q82hC&oYuPNPGpH5ASY$qd6b$`y1^lR6ECaHKce;opDVpeBsdK&MHMupxt+I$OR zCr|I)J56Lye5WeohrI>Gz!o$GR0>(IesI3Id33g3lhd}-_?hS!4;`ve`_wnP+pSXR z1<0MJh{qn9GYhjVP*o+3GhhY?0ZML3K%YRFtf46({_GL8SCD(fJJLnb{5aiFoyimi@zr6;&%)-xIsP zulI_Rj{~5KX!gZ*`}dPInBf62LAK@j>vI$XyZ}XzxR!T? z2_PCiw1Y&yf`Tt}RCo;lp;27?`8XcL2{4OOLsCQ>^r80!cBxDKv=^`bclMSoFchf3 zIo_CQa2jyS6-JiiZ(G#6GNpF<^tJSK1P%h^c`4K?j-td$T{N_d+&(7Hq)Dkn!a)o5NwmNxTV%CXb6Zc5Tbt{U#jA`9llO9x% zE3sxojz4$Vx_{JXu{`k0%MsW#(IEO+esdTC;>Qv}vA77$2$B+B6JQpoh5(P)+q)Ax zXuU^_=wfitWZ1BPJsAv#D*9vi5XTUufd*5)ApAKDN>ErZBEY63osNf)VsRw9cSo4Y zO@zNh$F20}HcX7Vs1IL0J|6Ra-&v_bLwWWIPi}BuQIs7mp#0e*yn)0=1DsfKBXTR^ zLX?$t+jEbTh51#nTf=|z#95n2J5>{b2HoV?+3*>Y42kUgqKo`T9g~_3V7-3_ zeJCU$uvRZwa_8Q?t1Q^1gAnf#;_2MEKpNDLg1>2(E~wjn>^JH~i#K}nSNg&5=xBx> z^MpBIeTy|KSL)&)X({G?b6t4xjPO!k-hL^=7%j&QB^d#fLq*lWhZwx@`f$G9(YN`O zZ#8n@yIR+Vz=YJQuplxISdedzqM4Av^F5uAkTm99n;6*z<7(|x^@7jF+b;3}RMj|= z>6>jr!W8Oyo*M(#NN6PcGuOl6R#DlTbObO-KF|tT15c9MCs+%!VDa*0z=V^?y6Wm+BgDn)*U=JSih1 zgKq%XM(ARoj`11OWviLU#-JPPwY?KKuAEU$oEqL__fG7$p6BTJxh$=a6-8jnWOfcX zu^w1OJ^&g|tiFo|n@RAi_LjQ1LEc@C+b8^P>To6Y9eqGGpjFmjVk55Bw zCogXuFMVe7LB(_K-X+dbvJ3&9&|jG2Y{Cbj4}|Mb@`M)gfQ4e z-*5CI+-MfG-C$qEo;@5(aGR2&Zi(i(6dWj35}uobEa5Bv066h=bu|JeO>J#@)n6Ab z@{iZKyVEM0nwg=&#n_=fX8oR)qwl`5W-pLOO+(*SCv;5;3gGy1AdG7Vc9!C_(vs6x zaf4YzI$-c%BoyzNNfxWe8qA(E=PI64zZO%GxetWYIdy8sgp+VLTRc4he`xe+v?-8S zWS@KQo&W%rl|gKXre0SRoj`lx>dKs|EAMgDDmyDHK?=Tg>pYhAJa_WhxOuaX!+5Z} zQj3?mx}FBxGc=rFW%U{bO=Zd}H+HAK2fV7OVHXKC5yS?r66}amh`$;;RFKJ4WfN1W zs56&}%+q-AAP^W+Y}t~YoE%D$a&+|MwE(vO$#Qvst*j$EDO%&i#DrkAJGCOaIml#) z@zq7(`a5>+8Vd@R+@8vp70|IzWFkma8Se>xKQOAot3>^@)H&`USiKO5HH964`2#!LU z4%{N%S)(Ru=#U}NINuPRGTNFyk$zbX25t)Rs-vzh?%jLl!iB_%0uW-K5}vwTu?Qn0 z&M&f6#?QgzAa@LnjBo!l9%D@Pyyv zY{>;=LEKUn143o?#+|oes-e9RQ`xkKESw?Y07?OnqZuSVu-rnZ35ivqzc>}sU0vTn z+WVbLq?Z%6GExnZOsT)Q>=REGlf^kcn2ot_-1z0g2Vs2JeQvQcLmkG(FjS+>%m#30 z$`bsJs)&BluKBe7gb#P*Sg|gKdHE&0O8FmxRpiJSm8^hUMi|#jB`7VQgtk8*W8}hf_W?ip#i@;GcA3 z^I%7lESUjfu{*bnav#$XP~la+vo{wK|x6^9{|Kd{X}a*n)E-CuRnM| zQ$p3?1B&eN*aQrO5you24d~<;m^g+DJ9H?e`^$8XzGTuxYXEl?{kZ}B2*{qvHucQb z&U3_`!k%9X7MS<$Eqm8g3R7{aU2Izz1(8~2gR?KSuuupuD5WDslaKU~mrFQFm_FFu z!$W`U)`Nk8%Xrk(nxHZe!gLbx<9}zBJj}^q+7(3Nad|mror0erPoz0h599=kVC;MV z{)q5!KZ%KWNk0b6M6>Ca7;IyJSfBXe$gDrk*`r2jpVB%M5y5xB0m1^8A=`NWj#72+ zy6NaBBH?Cu_1QE3=uG_5t$UHFBJY_)*vfw;dlq?H|4RsPE}R;MN0O~Ojimf z^PqMnZNv>6;9&DHNW$c`d$(>#Tb)s?hv#t_=sPy&Nro9%b#<n=O zSUS6|20V1r-jq#;?-XZQrj5M6OaE z?_poS9EHv)u$yn+rHdC=bBsBx7O&^fzZef3h{%?}$YUT1P$pxZ>{@Qrzdyq%Bo9nk zv`@Jzcjy2{2?)+iGy_%0Vo3LtixjSWe)7I7H!q%=3;}M@&E_g8k>0Rb{aFI2;*Vnb zU{q9xV?a$PeE{9TtD4gnXtz_ zA45;WgYB_jVM0evk)A5_XU4`)j5bGNlTWohr-+gAwbwG1145gCDW}x1_X`{z`E4Oiv$ec>rApw|e%>nQ&D@Ei7UpBE}nrlW~m(PY=ba$3fG`6inBjM%}bj`3`1rB=|E~!`MaQzW(?F2j(8^K7WeL6EOm1`934GHtgdraRkmZxVBeUS z_R2_$KS5G~R>c!X>ooRP7l4&d%A6*m5ie}vc#}FW-&Aj^A8BbnevdwyJ-*=6Y$xp> zaqhwe3_HM|Vekb{bk4X&FeX#Ga^DCd=BOy7p(D-B@l2hksT9HHu2kK1rh4>j$p47Q zwEN7UF%C(@&vRLiQ6sOF4q7B3+>(93N~Df%K_$=KbczBHb_E4 z0!9l=&p#3#u&pgRW`1Qod7_Eu1QRg3cT@2I_@AZa?#r87hQ$t4I5vtyis+V{%hpq=!dBoT zU{yK?@(j`gJ`-&e%Xk4O#gB5bv)O1o!@(f|)VfCx)dPR%uK6}l!N0$hDFkHE?NTfW z_1K3G9fGo%Ji3dxXmSu4kuuKKoNnNl3A{Z)hWD;E3uWvJ<-n{_nd_uBt;HahY54~75N(IZF*Fr68RwVu}e zy>cn2Dw9GPD;A>)!S8mI#f117emA8=66i;Z?-BYp$Jp4TCW}jQZkOQT=hLqbW*qIk zC$07QJQERGb80too4N>W8KG~R;iQuOaG#b|tv7C8NlvcIotoS(4<_TQ*fwpYzR?cZ z=J?Xmxm#E4Yws+zalU2W#-olX;Ba%DJ6BqxzOwlRKfHre9b;*h6Uf;uZXoS5!zpZ$i%ET{FPK zbED+}!Jf48=uAb4J?gp0yKW?PRG>a@m!jFZLA!I#f{Bl=?i1K#OBES8$E7)d&xM`X zAFa7{X>5Me)7Nhw74-!UV*jpPjXxTs!QnvFq$JBWZMsUa)BdFuj28{3?3_!LckzY- zUg0XBfobb*m`VKenr%>4P%yAxKfI>-gLl>0K^89rr_9Xe`gaB0@H!d1y1_?3R9%*r zmibK@4@r;-K-czDzktksb|3O3(WY^9&{!#F`OI`G8imq@|&;C786OQ z{m;F&&;qprVyv9u)Rd43=WZXfy$e?}bK*56V$AUAm-Hv1q&~BU%pocn(%u{xQvLVH zsKNzsZHdhduR>^{X|$*~DJD=j_}Ll490lPTd%gmh2=y)R-!>zu=Q)oN;-Qe7A62zw0%J^1Pq?cL^0QaRB~YvE8l(O)}hL}S3T+|J}cw^*TP9Phiil7 zu)VQUpIgE%I|~boms#4O)-0D(IaWcEV8^zUs5ddJ@YrbCo2K~ZLkjrLIQ9&(IdJb@ z6_JElHu4GRAl0*#Rj;vQyZgt2I*|xabFN>vuC%P|abmt~zYcn;*hMd1wW{L;VYwpB zZfR*(NI_I?LpCY;Q~WT=iR6JsIMVDZl?^U=9PM=hkz~X z)6cO=jCzWw56445tosEw0fWtJK|x-8a|Ey~PzynWnNCPC(FI^A08+u*4?##tXFVVv zSeRe*nGKOAc|I-Qm9hNA%FTY}PH`=_i9Le&*g8|eGjEH)B z3sc+h6axl4xO;bp>-l45n=jJ*gT2xoL#&X^PhAmed*&;FgaMBUW{Uodh*aL%78w=> zW$z*ESXmomGokg68^om2}? zAy~H4FFEG}uM0#Yfd+H={DJ~G-9&CCG#zK6=|JxQ+jxpy)?hR>H}fAvL=?gPqYl~_ z(53w5@l#)1?*0AF9eWrG-a|?96xU2QCx`ms(ax6Cs}8ruF|{ zEkMZpE76ac6RZGEq;ACg#8l7wZGM-`JSL%e4nWiJ4Hy{o(9lTrY4--$p~XIOTwWxh ztU}4k&jfYUSzDXY_(@nnWEt)h zWSp3BkY^be0ocRxyjPe`j~;!gS) zoP+7|)y5=JS;7x+w+Ah_X8HK3bltCCI8kq@tCHUJ9@bjMee2@UQR&^c-xFU{N8e zY|iO%bj_5f*dU&k7D>1Xh}|hB;21jME_X|=!l$!||6Ct)q9<-y(Z`OV(enAWS^DRr zv^#0wI^P#9v;|IF|wxFvwuQ7 z(dV*RD=js3D|G>TLI7C+K#Gpw^CmYYtg2{|$7MbuUxrn_vRUKNyucx#c<9TGy07j;> zh6b0F#-H;jkw}ML#mg{w-@bA9$xa!cN-sy5$^whpi3sv&SHMtd3rQ$cLo2 zqQa|7ht6z-@bH-F;K05r+C?gkwOBAuzKH~YQ7@ct0ZDnF3|PHr$K}zXF>f zVBj3RXp!KY-L>oVnKLKb*#RZ^=je1gVlycn^?8sUb+V()|z(twInCaO+o2aja42 zNIXhm5S3w>b!j{(XdhB28>f%h7yeGcHy8Snufq?9`x9ED=3T&7edPn_L1jH)a<700 zGS$LU2CYQI0Ml{h3Q}jRm^J)knQ5)7a|a7}DVF_{3SoS< zvg-9~upj_`PO2k#`v>X_Dp!a+{BCE?{K3;j79q8!iVjUS&=oY1o6tYPcJ0z354+2= zPPARJc5M=6z3F7;CS>28W9);X^~m>@Lj&hSXYvbz{y=y*lriy<#0@^oj$~$!Sl`s6 z>QHQyDmk>lRuJ+r$Y!(UA9b8v5G_IG@`DNfX4mikg50D0ZQPI&ISj(dR@E5{Y+0w z(`I+mwHuZ@*AvFPF1ZX%hngf`(}p{*ZiVs3E!cjp!VOJq+~e#s=GKY=vdGQU1$eSa z;%C5;gKDk#8a}wo1tcxbCWjdqY-}{u*RO18G1uKI|4vmkNOng&wjQcMyg`%%MB`W& zCh7npGAg?gq@9N#%q1f+Tq0Jdg)NLWQ@FVy)4l1?<{(O57Qum!1!WImgI~3K{|LOG zJ8NmlpPF}dO+|Bjn(t1!b9zSc4~3c?er6DoUET8OnEA2Q4}ALk`6kxRbxmlU>Cl)m zDSp_U_gh=bkilDIB?thCYi!#lm~ym0>wGWQ4-WYZ)`Ij0@I){wHZrQMsS!0g`*em{ z6=xti24xfod^3G{+i{ss*trLQ!)KuQAaq%-Lx`}FfD3luXrV4*2?Ed(+tJvNuBfP} zXSS)M4vr|VEhQkkV8_^U(H!Go%A~8Z{dENIB{iPh!xfvPXxmt*q7-;S?LdiDJgaI( z&Zaz=6J2A|K1g z__At7AAk`2;J+2}GAF1L!WJ z#Ze|o=LHguO0?F14}libTPXVXU_%=Fl$(DualuO!wmYLqTCl(zxgaAY>`i949~K{! z4*nkL$?jKoM(y z&@2*-bUF%FGE5?H-pHKJF|qsAi(AGv?39bJMeWNOhq25wV*ZUzsNwD@t||j<2#_-$ z!>fea)bv*yjoHfb<#eyy>b{MG(U-n`yKvE>(-w2v?)M9~Gh|8kSd5v*j$McW`pJ{n zKpp&5Is3Pqq=?)??h)i6`s45`C?RP|B+e54>Bqya2RW*p07&qQ&0*`(>BQ2V3j*=hxOi-v$j4AovfdSo?q`Mma^P<{ZFh1(@g z^BZX0??dTcvWD9Q`?ZrNKXQDC?xv;)mA*>opw6tF_4zV@mda-M zu5iX{f&2KO)cdtiiGcVF=LMZS&gavQXUE87%@|bCbM5UdQ)lMx8Hpc*EjOMRL`a4K zKlOD+=})|+l2`UUPw~zhHuG!T^9h=%$vpAaGVbX2j>me zR|m}j!pIH$#dNJpmp(yOPTLZPdBJ3ns@7qun*2qOT;MWWG~^&MCII-SyIvgEltmfR zN+NRyhduN}h|3}SSy~f1uoVPMtrdV?7Qg~{!S%5wrsfxlZo((OBuZH3RBeqsVBE=5hg@9M@STuBc5Lk$l0^!(cmDu_|F(=qU5ES z#BX5!ZSUU4z)JI_pJNgFGbGKR3-4myfG*oDHE9#k3k{+=0Drn|=ZkZi0?r?25`=yv zZkEcM3qp3s0Hh%~SAIT3jNu?~-aCXFxCTx{R>EAgtF{CFXZ5zeb2$apmfT~mU(eZHS1B0NqPoz9cQ}Fy;ky%UhgAIVelV}-+qF7G!H^1`=yQwBNrg|rpB4> ztqJ+}@BFF;P-j98%|7v!uZC6?RvC%?S1ibXjVl0kGt*!8{VxYh5XDI0i#IO_{oa(b1q+hWf@)5kg!V&!i)^=hh3p)R26B9#U; zT-|l>ps@E5+TlijH@|hNO)~&&lxyG(sV8)zryxvbz7L;!J^;4lV-M1nV&+t?sMAxQ z`Q1C2Qh%au<5n{V+$tf*I21Wkhb3XOj530PdW4G5tm?n6b~x>5oK*m=!Yp!xr8|U! zzrRV0^>f;FObZcONW>JY+vm>1BRwVMHCk(a570vr6|vQl#CMTs31x+33YMJix1`ry zC0b*VOb>c3Ufh&=R#rx0Sl`fqfc?Joh@f5uD_?dVzp-;mr@ZTr*ku9kVCC~LulINd@@6b==_fw@-3Iiw1(I3x!I11f%IVu8St zfH&9y6Y!hd%QmfDnEAoTgxBk7meO*55it>656);-y$=Uel7-Hf{<85p(m@9&Yc zW2wp*2KqviT3Z8QGzspxUSsio`MuX|=s4~c(URLK`7|u`=ut$O{)5JU55tTnC1REY z>ZxHoVCLW^Nv>>+H9kc!eECW%HlS~UnlL*PO7iaBx9^|R67{jZ9((i#kE70o&|)NC z%EE5Lh8*ao`wO87p`(9Cty~YE%ua%1>ru5(jDE=Nl?OZ!@bK}u|9UK32uf%fGZbt{ ztMSkJGC#(uaX#-}Am4>}&h0>QDwDg@`Y_m;0ao@REJ07iGKnK&CZNm&6$U@;Eo%Xk zfUtoGK$pl2?(d!9lxB`!*$QIkTZrkC z_WfJEY0Al%zLuzR1bM>TcZge(A)M&g`q|I__U&8EFw8@eR1(JR+qFa zb9dJqG$`Vw^wrhxA5goHZlFSG7}Jo?wTi#1w~p5zV%C+Km12^Y38P+VZ!~r+djI%6 zd;VI0brp%yZ?EY}x01UKKf(e$=y&~imzgt#l?s&3jKbzwF{jR`{dr&;|AlT6MYzKg zaD5|g0}QWXD31v5%a=`J3@91q`$G4g3YlOJ;HrFB{_GhNLx8I%ZIzXS2Hx+FXb5W$ z`f-$#u76svq~u`n%n(C;d3mMNnem!2h>cI4KFzgWu`fY|DEj$x08Sp90OBlqROuI* z(jk>>jHNOLT~Wr>)GU+_MOjFO707=EPuB1hfk-s+e=^=z7I*tS`HMX{T`cYs8<6lb zi0UY)D7~D{*F3+xzKkP86r`RyX&paV>j8O%Rw|+e)_)Irie*7$h+{Bq#^_$I3r|sa z@$L|ZJ%=JDicyXdrjZ{N6>a7r1peWP|4aRZxdq9Xes%o5eP4h6#Ls^#bY6TslRfew z=6djmGe-iQkQ#86V6jqq?!}p;w5&`3^bJdx>SF~t!ax*3%!<y*qC1O!s)MVmlBuJsD>Ny)6IQHT`KqEK`@~86!9SqMK)JLG0iUD#WfJF@KYsWA zJ$+7e{HCPS&?MHwJTStsfK)0h0UtAlu`@wRfw#CIKN@rr`W^xjIXNOhBy3<&K0L#s zNO$=zU^#dJB6-?_X{cIZlBrGvw{e;!`F$uKz}1WpK@ABaF?Ms|>&Jm+&f~s%4JZ_! zRcg9i`z}o0vcm$QMQTJaScQYZ=jNZdjRT9NB*Pd7DK4fY8a4STNe=BV8YyN9nF}S( zkpmxi_5U@Mz)M7Cji5r@6m(ov`_x`@=f1$5e)uP!_zuhrig!NpHO%o3hQeh?tWBgx`0_{CIrCb1de!LwRtTun9VOtNs zkbH@FBjss5;huA}u2}iHgQ%-T*GyG&drPZOXFVy+8>8K4ceNwrN?NpJeq z>e+Z6N$k1C*6dr++(y&J?6QA9{Js|TeKpG2mPczG8XXt}jgDsHwK|J>sy8zfMty>m zkY6)q&NN!{;bG&Mcq&vrEV1sz-=RR_ZV!cUdLp1S6gcN&V`p)D5FBnzL3Tis0ZZVQ zm)C5USRO+h$G$LcpNO!Jl`*4X0Z|7jy3mNtVI z7W}{z+K6Ckiy7akxgg9TF!_&^%Ww}WQPmh~?*pKhmin~W<2DkJsL$53?THE*|6!pm zjf|Jdkjq399SsJ=4n0Zsi@+32nR0vQfT>8rgq6&A7B$t+r)vZP!2b&L#wHWYe;46V zhJO=E9s1do+1Pn>FNyLZ>F zXXlsolnVjySb z{`uJ_-ew*qj5Ov_tRhi_V0W~(?&UP?UF2=3KkD&boj3V;a_&hMQ&VS>E?0PZGTkbt zy{n>a{Pih4X_6MPhx(N>%(g4CAbl&k_xIvQ4n*ZUK3NTV(?!K3#9V|9{PpWlt9cV< zoWtRr<#NX~C?J`T5Fa700>rT;7z}Mc`9o>x44C}-^8vI9_~1ToC$K&nRfBX0Tbv9; zdl=dje&Ngwk%pyx^qxI)q4BcXeKPA%KQc^oGj|ILY7hlNK~|>n@@Z#tH?V-|Y!OX2 zQRHu^r6qf+Yesz~Y%d@d<_?TNsOPe9_$zlU1H^J!$oXJybpPB1S!vq83xyleLzlv& zXKoI)Jma)@f3TIA<`zLCFLo%sIUI#(Ve)nX^ek9DSMX9q9$$w?B%4HuC2T~16GP*_i&-+{0ns}+C`c!{qgNZJP?{q| z~*EBN>>-aVJ65v~5W~Tp%7Q%=3v8Q;wE3`?n$l;1h?B@3%FD zb~=(rc26pAO`**Qr7`;03N9&4RpkiIAA{z5qoaexcv%-PzRBqzfpgMmdo!NG=UlR2 z!MCH!=_w1F8hKx0GVegl)ER_?jnxobgx*3 zUe+pR6Sg=V+__Uav=6fbUN0|Kpy+V%Pj}KaO2GO_qeqXoU%uF;&MfL)iQPY351(nT zuJcZmtaulxNAdMz_w8e00Xme&(3@njy?fi68mG%<1S-c1EnP(6xp zGsn9rZtfqrk8dAUgwD4k4_Q(1Ag|l4+T<*w)M4$=4s+KBbLX&!q~t7t4JkZ&7niPY+A#T5|$&vpq$MyAki)G69M%Gntz&sFAk$JrZ?^hWfAfTf;>CmBr zT`^?a%8ora9o+JI$T!_Vg!cRQ+r7CLm9H6(YK4f-JGgMcz->R9F|Tdx$e$=01d$!` zcHr>g$h8>0BY-}A_H5QECn?Wg1S|Tj9mB_KHsl^T0q_QS@!3LuDmivaI#2)DO;H>S zDo!lHuK-^R7+**^7zA2sq@-t{nBaQEGkD&bYb+Qqwc!l%D3AfOHOOz#F2V1#&bUac zfM0_#K&$fR>r_@SfC}t{ypmg6+w{0-svs-Pvfkah z53_Ha5q+%mwAj3YvR6LTZSUMl=RUKv{Y+j$hi)4`syBVUk@0un=ulkWPsaL`O*W68 zv`#PXYob0~OF4JlaU6yF|mpU4OD|EG$l9$_h6ZHScZ@jNqI*;D1|lCWkcrnXUg zIms%+@ZUQZCYFya#tfQCkCNI|*)D9G)!vnFu5`nWYfA(n%-8s4WbZAhmAEY+uvNoh z14uHS;aULdU&jG@U62Z{rxS$86j}vLg5Dji1PuZ@bQw!*c&xbeTZleU2BA?H%Q_(7 z9}03Js-vN-G&=+%jpL{mvRsD6SZ~|0xm+ldl4QFDDqUHtM@! zT125c8-YZA#|eDARfwH|f!X+wftr~V1argrUf@rXjO@~#W<+h@9ulEeTu-ou*&eHQ zk2^unJEWirPWSrJOOP>`89gg5KAD?qrm6W_qH!pqdeRcj7;*|r?>1Jfga?8ja&mAO zuwr|+#QqZbwzYcHBwjdv?m)B_Ima;2r013~vq3|wWV^c3=5lCY(68pYNd-$CQDqa1 zV3c3~JKp~qg-YSP9jribiU-%~nyJ#@A`l~yWE2zRf*)T|HC#AuVKxU0yXM#-<}z60f#X+|XR~F~WQU1es za`kVsESU6xI}yY8r~V1rM(+h32}h-Fuo94#wK%|T9OU2%0qew*A|hw?r!>0tcuVgh z+W6)8aob0F-eJiY?T`r2Hcy!^g8x3zBf=70=;-Ye*`h*q=Vr@j5{VJI{Ki{#6)U>= zc;$kL(l;t^ z3=%(OPczdYRHh7w#nQTcr zouaYM^%z4!v`-(RcNDhHLOEb^32x@2a;A`w?>%@>v9}X3jA!H5g#`vyco&{oVuAFK zP7lpOBBR{o)FVf@cRicBQ1CeG?xlFubNy4EGJL=Q^cTX)19AxOlvP}k4cQa4cUfu9 zz=-NdW)b$gzO)sSY!QS4?|~S@4}PN~f`jW104K za3$1jgs-06yE7%sOC}5bZf*`p9D!)tO%G8OId5|T+wD7U7%2g>^7JTDXu-!p?I57! z>yT0C@lsiM-aGKVx;pGu1u{>X4WXhFbQ$O;u{eUg6A-|-)~FA3VWU0GUhhb#5!9Gv z@4rJ_@I=(VXYpCVxWvKjAEP8aYZUeRQMl#}`j#kLzW;b?>gaoC6`D^{_+$qD`Gpml zQGzMMM)R+2q}>#ltMx~n+>bpUucEbd^Grr?*YN#lRO7YHL@5h{t_-YoUhJ-2l;KyY7twE9Yq33=Z7CQ-7_WYJAFVZ%W=Kyj#$R8R z9ARq#18xjVGn}WksPQR)639Yo0EENwpeTon#SAczZ7|5?P&mDOd6hD@Zp<99ygS}S zpFeY_Ifsa!-@JXRJ!~2%mc0_lGUv`sAWTwIGrhZP*^oE`jPPmY18K+u3q3rjdx=aC ze>s?e1Eb)6h&jLr;7UZ!NoK5>j}7|$`64y4bae$y;aeVOY0UCQ%e^kJVGX<|_>=3OtG(eT`vaF%%Mp>ro@~=wH4Rc0tm!aUsOcAK@_;^h) z4AYO6Q5_NU&m2CC{kGTq`57L!cGmT|z+-?O;9)PKw0yGz;74jP6vP&t6P zpM!h_6*05kHb<{*x+40P2d~ub1rrMxBS}4Ll9kRUdEXeizroL+88nbujI|dV8?$|h zGujEZi5NDsfdFNXw;`}H?tI-M=?-R}ni6ktASu{3K+Nc@3pB;yoO7`ME~A}poerCe zfrk*RG6uq`CH1JIsMbxdXI6aTXA&1#YeA_CU8iS`aXsBe4y1}zNQ}9T%1j*FK43V7b zSPnV(CBz(g0=vy9aGdVZ1;L*Kk~vtC!398?#mKX{n7SG3U8an1vgW+CtwrJY2Y>FR zrgqUs%4-U{c9=T2awVa-1`B&+ralG+ikr*mdwXW`4c@Q*m*k&d8@qES+g%Jsj9>%> z>d`GXhC{_!rwZGeLN3gUY{+rQt8QSvJ>EuJaWm!PcN@4jTh44IY!Nc#_vmtc)Q&Tez^ChXe-mNymWVCG|0RI7^1gt#q z5E~kpC<^uc2;{4)!;~F<2OBmOh4y}vQDq@mDr|fpJ*yaL?v5Szz{*=wlFl^y2Vyu# z3Id@&*0#_<3iUcb%4&w4aV3hKWKJG#37CrU3GFWOh{(2Xzb4H%cl`JgFRwc{Z^|F; zucBbF$YO@dLl8clTxRun%&ZNr{?dNlEY=}ct}k3O5kSl@x;k4^^7?ezy5p{=OhFt^ znODH*i9$H|B`&p{)eM$1nV7I0Me*mY?J_PqoJTq9(r-R#E`|dsd-zaope8FfQy_=2 zZ|jwsx9Ae$zU#lWld`jWS(QRb8|**L*347cm6O?<5IjQZ(;>+}MMk@gMLO%m5+6)- zzBRXyM1qQs47iyai-wYxCPiLL=E~j7x)Weyc}89!FmeDP4g_Oi^9uiJ$JP+h&APAC*Tja^-DurxqWxU8 zU5I(MOo(}F0>*wM8m9R(olb!p_!VW`Z|Km^4A+}A1%G{Yv>nx6pvYsCnOP7~wfgl% zn#UyfEi`6Rdd%FBMO z``Zs65K4E_)DCR^YNew>Hl|V2UtanR z>t#kZ86~35YqIuXZh)6r#MTs|2Ju*Dt8JW~g~>44Q1SR=tVn)B z@cNs+qGm-j@O;As{v$;Fi4$83-I3i=oXj9$!Eyl!k*HwKSX|RXMFsgO$BJB`AR7ur zekZ~BBB_QyRZw;R{c0ve>a)%0SWzXZD4>XBDc6>iV@;n)VWxpC2-#8AEkJ@zWqxM6 zz>}7iHhiiWSwD2PoeH5Ec8ZS9F@&kP4xPG>6irKGNH)7x^7Z ziq+TMqKXg$Vuph2U255TVe$%5X;cv;BQx0hsHW|7T%e3mgF!?>rZQ(lkCX4#6YVu! zZWy!pc+?vh88onmhTfb^M_=TKbkGnEfYSikt~x4LY0Qlo^Q5SVv0`*nfh>??QzRTC z!S(qU1ZEU>TnWkuY6$)m6fTWU8P|t*Z{J3N3wiNc^AMv9BxMc*Ody>YKsSX1wE)a^ z{rB&OOBeICAXuro>0r{;r}BhwqIM^J@~8N}6gyNW%tWroz)jHpEh(cx%GPBp9-gvC zhPw!mM6bcn1D^stL08M>^{p-0bcmX2qX-%Z2v4)^VbLP0nsvZ>n%cB;!Sjv|4pgCx z%Cpy^8B=bZtyFWE;ornku~Vp&$^&>#k`iEos)9DQKsP>nHi!8p6%~8hsBdn@&#;k#hhg%-~|Vv5Hl4OiJ-!$B(}Tu0lkxJ>?y6 z%fgAl{j-cIQ6~~j*b<2*qY`BePyDaNH7l7qcIoot>(_r9UWen6Bn)DRc`ItFs_bY8 z;%ouFP(x8kv4qIr-kQSNZ&WNW3b)-egdQ*t4_Nu>NqIS~p=ZER{dhs@LpxU{Q(^6R z{Ca7*Kt?i$aS%aSw{Gm941EBMl563l{LK7D$L zHDMH+=KCpchq21o!C?)3Kl{O0urZL7oRs8-MntTGT#oh$oUnS&C(>jWvPg?PN8kne zJU|Y>l|D&ZA0eQnDWpgc3Y)0^p&`{QbMne%3R_oTSgZzgv=d!|A==R}Y;UWz0$FJ4i-Xzp( zfovKlcJS!ZPM@|CF+G9Xi=vWJWN`w=i#D97N)S`coqGk(%yoyBSGV?o?E%iJDDcAN zeMb*_yo9z8OpE+7hI3D*rXnBXx$!e+sPNP+MV2#W_&hfdvnRfwDlb<)l&Y8t1t1RS ziO3P#wyj>ZYX67@ zH<Qv2|0dn|FWf*B_~U|MU;QOQP}&9u%t8>5)sL>n$$C&>m|Uz za)&nBu~YbX%TLq1(53CzLUi?^qT)QWP6@3$h!VG%zQ5UqwDoAm=UG5m>~^35obo%peK$j3~WLa*$u&XN^YD^f$LSXm^! zr5@yJ%34rR5@=w-P=aFl)=_pmgWI9)J7IuDK`8j16Jlj8VAjx1!{i24U29L5f_F|V zrFdi__CHPv5{LxN%ofbl29!6fP)qlsrl2D)Gp_XEL#?Y(7`ziyu!R||tII|(fDeqw zwvpC&aEGi*f|-P7Ls-lfNIw(x*y5UR&GMnp{K3I@^3KVO5~d}Pd5e1WiYTa3Hw&ck zFr@X%9z=;yo3X`wTwDx0tV||}$4oqg)A|kcRcvYa_xqOOn!uIjzw)1XkhoSG1)$k+ zlljUMqaEDTk*p)$XqaHzGs04^)?j0lrbsqs_ih@Nm zM?SrK$HZEZuRCa9@!~98Gxg(f0y&tFK+gI4=cM;v#P_Dw($zJW@|IGTzXw7GcVu`X z#KG(kpx~!7xRAZgs3L>Zd-SSCY+8KwCefkB45^#}>qh#PzQn_Nr6M+AMjWP?% z8-M(OVPJ&L<+5K)p}Uk-6I26oM5rNpT3j{|V^MXs%NH8>G{sPNIi7L~5pSW-fYNUN zU92HhVvT`7u+t2CGx-smAT8Y-EA{HmuPB+T@jo(o!;FIP@+h(4cfP!RyD8-uum*jr zxmah|u!*8K$kvJbbddo~ylY+(WHcC;E<8yX1u>DF&kW^f^zZj$glmk05^Q1A{~#`T z@%*`PmoYBFl(xznJcNx4v2E0v5B8>s05_;1O-=%?Ys;2x;6?BwNe8HjP1Nxr!xeNP zGF&e&L?m-DL?DF?`saujqp#rfmO<#D7??pLNjn4XWemGnX~zHp-Q8!py0UY55jqgA zouFzdha|E$7J4Y$X2L9xA0>&{g)Yc8}pSHY-d()x?-I1B}Q3p}-C# zBxJj+U>d1y&gC%v#aU+9nDX}NyhET!txQ0r&ysiFYZJ^LIc-D`msD=3~-Pb3;Vk$wIJtU zrqF9`8V#I5(cvk|oY0w%M~`Caxm5;DrMhkQw6Foooj|MavKR@cN%UBKbFo zUAAp$)#$Koiz4-s=qBZq!@~bVCE50`Lh;y^W@`S{o&VQz^Z)rbQ7{X~J;Xfe`G0>r z#J9uWYPGFCZ2Q;$m*4b%E@_3I)Y87+HO;1rOe+%wj6apo*bH+XV)8#)02M3DBp@9P zDKBAX;eOqmaUSx+r6l0yFeZ}aSLBlQ;ER(X@V zPUnygZ=6xG`5FJczAS%A2@ zUHEHR0F*se436tUeu}SP2wOcnl)|5=WCEc z*1G==-T(bOwXq^?AL9Sd!4^1;|GwXU5BmS(V4E51Y`r!RdD2t!QVsUP;9nhYE?P#)c438cZgY*3h90JGcD!IEx_%IyylQPIEwrM2N&% zkM?BX5|JQAx{}gv(-KYpK!{7qJgAx%-QD+Notf?W_03c;B%juP;gs5R8@_71^dl%D z_!HDo1L4)ySDalaxoylu zmq22D+jJO~N;xK<~%bqto%*B56iYbjj0n`;7jOnQ}5y0S{GRDXVp<7qP zOGnGlH^Bcd;=*FhO12)i;pditOvUX)aqQ@bGTw9P(o0Jp4MwEF17Q4*sec%LogqWg zT(TeBzb`tqSy-#fkH9`}!GcVWTgt=|SqmOuBdQ&SVd1pdF*5TE8Yy%z`tk6Yyk&s1 zMKqooY$=bv>xq133A@YDtX1#5AZ*6_TEotnhqS2y!hB(P4I@Z5;yHU|(Ld615JgzZ zT0q(Vq!LAzh0Y937Pf?dPyx#yG->`j*XwkaYyn&1n&pkHMt_FSJ8*oc%{d-EhR=5o z`iJQzD7XA;-vM9pp?(`SK#+&+qCg;FmwWw$d0_rDKa-{rSkhW7LZ=5E)zhSypUN5H z0Pz#jJCk$gqr;?!TS+}^dTMt&7$mSa2#Uvu6)rBX@7SkC{}-!os&zz)RNZ-y;3M%! zr&{IfBbbCO$XC9Fmlq9TWThfY9TiBGiER@o1dzFGi=oDuG-(m8s!Xm!R1P?h^e`9PNl;lZ5qDViSozUq+A{+7{x%!VEzP6 z!+AgDl0Eaejy~F16!=a~%=PtBQIXxr#YJUA5dn3_X-Y18U*631hB4qTpdjXx;ttLN zYQj199P=>rvqeRV_>1L33HoUJK7xglu3x{a#O37e@Qx_;z+|%9ff_IP`G}8J8(l$$ z3pxk8B~3zRGIZYMI40=*E>m~H^Ri@j1|#en7Uff{Rd}sMm$LsZgD=8X4&q15B#BGHQXhu9&0ugE0SlA;q!DMiw^2X6&VM$`s2M2 zVW}W5U;pC=y&NvY{LBjZP=-m^hA@;Djp{S!Fc_BBC@d^Y8rOT0=r9c_YCN`Lprhsm zkZB=X83G_4AvULdiq59m#*uFhU4{7})5%m90IoJ#nIp>ZAlz$KPR>+oYXqjVot^t4 zb|sF$&dG=3>&%pUEsPB$3|A7y0&=QL$Bvjy3>3*z0I&~I!ncI$1%@+vnEq@W9~w<4 zf+*qjIG&2rn}3aiNKFA7vIdJHgZVnTrHyp|`E%^`a=j|_ zIpoQ9S?+~fQI+i3yLVrFJToH6SFgUjd2=xbljmikUI+7!pj9&l&$Y28CTdZVAd`OF zFf^H5QI0lRSvHMO5gP}#WN|V5{0w7@p_q_Aagji(0B;5$cSc9c<2co9DZ(pDk2L}q zb92-wn{|79BxX^B?$?_y9V+J2o<4bU4ko0W;a;6@5RQc@lGCRXVqzK~A{bk#t?i(# z*l^(zN@cx2As1Fd*`au$NlZn}Yzx3SSd**qF9e~T&H>Yaf01JVU97bR>*_)d_f9Nn ztgp{g|Aw%ff)E5w4MCZHXy);tCE4GG`+APUlZv}G@lbY74rp1!#4oy}mi7VDRKlnf zVW0Ls$IRGR&^H0e`}GsL*(U9IoXB?2v*E7K8KKkKeoHcGg!TaNjiU!Wi~bZ?8e*Of zD=5hF<3`{%FD&rr+>L??H_S->4YPGZKwQ-|3W>5I-Ew-iWS3-z2yONRpG(- zvCxlt=57d9puatCZ0iLbj5wB3lg=t-G=>#%BNj|#p)i0Ue;T(n7vVe!FepgEU&GKj zbd=)eRm2-=Z`!vQ+qzdj#VDWAdGp93Lq-y~!VHUZEWy-_$%Kh-B79_pA>BEYTIadQ z_~m;0vdNn)kLV7y3uJSbgGPe~PhqPK?fZWHsyOxZU^6pUaS?5_Vc*wN)gqIw{K}Vh zKX`~r2CN2kq&pDq3?3N%fnE%UhjQ18($tfQ^Unk;0Nm?;W~2V%>VY1KV(2+=%T7+O_)H^e?^%5hK zLU#l*a_(IG>&e>d!PdnVTZuN_^<)!`Kw#Dif0#HJKWfR&g9qpFF_`J!nnD~EJbq`e zM|uCmLlKEW@L@<4q7)IWoUCkmg)rrU?GNn~#OE+7{P(L2|64>;rIWb{N(4Eg_7IdS zQ+aqE6zU`{?m2ECq!YE?z1jNo82LGcCDKc(TLv$Ip~2D&J|J73G-DLt2|+ER7ZWB_ z!^mSl;0=m3mAFBOhd0Vp?rY1$PKaO*6E<;Wb&{k0E zW#pKQ*l}8sc7E zeWSH(^4!MXjQ*du0f_k3)C5f6+E8tSV3}heyM}MxZ1(SW`OXV5sUUWc1IVw{A3|Zz z500ZBJVH+|ntx~0BBDJvw@UH}IUCouIQl_Q$8?r=h;IsrlQr8QgX-#4HFR1T~hFLtxUn;pt;v-TSCB zsgjJ3=k%!)CoUmYVNMQf7LxLk1_z&s)=jS`j)k$1 zEpv%gG~Hn1nFYpyPsT9?sWh(_9}wp`b6A|5Fm40xH>dDlro4bFA%?=YwZC_rQWb7* zTAW^fQBf_N4cuS1#7;?TYL$dm14J!yd8$(yAfOk1@Vbx|E=lxhP-*nPNA%Y}JU$rw~ z0|6K6EnMQlI$;J+h4LC7a3-UC_8xBQkjzC+4l~Y=qx9oVFapdkzK5*E;!#fpDLRs) z2*eInGrn_{xJUa8eFzb+5sA`HW3u)9`5(A4yuFveQcNqR^%<;NK|C;N-I^fnn8z_wv&A(8$)zR{#OQxoHlnl+rI2{#r zLvJx$8ZeKagS#2Ofq41)oM(y$(hL7Y=|uIw0N-4U0}Bem?s}Fj9;LWBEag~8 z2rj)lq2kFp7cY(p%eS+$%L3CaUHai#c&Dzih(#+ZDu}CKH%? z&d2RfSZfjQ_UkX)N(D{bKqs9phJYgopJ1J2o!iw7xTNL==}<9)e6WXsMQbSyLzIxl zj!BDXj?rAIO@=O0QpU4<6egYPOd_hbk>(0RH1G|u%(N6%-ZGrdf9NOg4n5AuEp;LN zOCi=FzZrR&=f$uOA1G{|xn_(bR-1HiAbw#^SzuG$5G)KU(8Q6sahlD^ZqsC$E8uR_ zwRy>d=R$t9_l%=RZf7jz6hak+pBdTx31$v$8VnX49Y1=sgS8QAxnw&-1B12i?kAIh zI5=`8QLHypEL?T-5!y5Jt43H z*&IZu0>QCRj{Pj`ieqtOHO(>J1`8mWb`i;TR8W|mvvz2x7$iG~zjVWi7lm_?X+ZKm1DLcU64EphMEQ7iz?&1X1*fVD!az z@E+Veu<7B~Pv95%iUutW7{QOZb1_JiYm|T-5&4$L2s}WgNq?(@GBkB58+gIGcz__8 z)KQUMC$-x+NMDQ`N2+88NP5qd)5nj)3()pz>d&8}bLu3`5uxnb_AG-=wH{B6TSDdj zwYh~RShAh`n;m2*ntrMYtRDb3651#kg;kT}8vYli!eB;Pt{(abbM+>T7NyIF2Kz{D zE+8JAt@y;xB5CtaFx!F)zX%LI*|^0rFnX1ugyz`&A$dM`RE?KBIE*e9C~%+8%gtKW%kA`E69 zQY=t{pus>#){hbKHaEP!LRrwQK}%p}9mFhv5J9Z11#?cJ zd)3XpAcOm=^p(?7>54E|jMiI;hUAMr0NVCHyKf`&gCa;Eg5wrGmgAH88=-)a8-xVP zKEBO4ar#A2#Z0xN7VzO~cz_cV5*|n|hSWmKCn(6wQ7{jtMHIBJ85xJ-=5G|V{7bYw zU-#w-f;8ipjj_<}p||NNV_-%1)rqla7IV@6z}QrA^T)b6q-aSGMx<$o#*r3vdkh&p zdQV9$BSxV!1bqn3```-3bBb#yqS2nf3L*sv6&%x~?%s9h0Q{nZWU&N_bH}$aQnXNV z6U}Z&Js;;buPf6$$=;7nH~=k7X-4y2(0_5kM<4@ z79kLq#CHu{!4HOm6%vialbux5IItWjQq3=ph`&S4;Nx4%u}AmymriF9WuNslandtB z<{pvNBvruhG75o=j9uBRbRdkJXBh}Efr#9p+kA);Lk(T)R*hIGbu)4(w~FYy{g#LH z)b5_@&|-3uve9zhALByYmYf#+oJPBvk?aQlacILe?h z{7n#n;`m#C(w$O>xhsCR(8B&C$Z_$%c>M68;1B|#a@>A!`8K4iN}XqzPeH5C*F4Yp zWiaqr0ejo{#Ux@w^$mnbLl{$Y!0DW%{yW5>HR|aJ3WqpFK-=ii%Q2Q1G)NF&SbEN% zpI_rki36GtM%j+2gCUWG%32^+C;tfQOIQoiI{vxJ5wfR^gUW8SKP40P(^4CeIlZrT ze|oG|y%V9hZP;rnNh)rRE2xO^16~;eo!t%$H|-GIU2xs!FQ}@mWuM;RloVecoq%xr zpL)WzOFEEeC$ESDI}xA^jGJ$nM_4Z`!rumEXf?*CFJHC--MKfEX6R(Pe!7$Q05-`| za|A|}#Bpj$QYVBF<1G2=-TG*=gpC;Y#d zsn80=!Y5oF zI)C|en!ucE3>G+~te?Q3-aIGll>6t);E^j==4vV9Jj34r^$bqrr#yJS8WGU(;}IQ4 z^Mc(hy5`3CMdi(YtKqDc;n3lx%-Y`N9$}+>8+PX}6^aL_l)oS>kZKD|HDb6@jI~PL zYE}yLX(#zEw2JL1qO6Fp!h`IA{<(g=ur<;5ym+ z3KOb&po5b-?SWF^wALNJs1-LZRXXh>YZ=@m3Oj{0RvPf8YUisXk#yD z3GN)o+diFX3S5Sw9lm59L4G-4r4-xS1La{Ypt~V`JmWXBVcq%!{l>?gsaI z;)D@GLBZMc=1u&Qgf{e@JE)Q6&sD)t=+&dT;5c?xbg)}~`hT_* zZz@pG$jTYqW5Ghyi(?UUw?^v}Rb~=a4-t+&tK}iYB z8+G>Ty=Ui6)~#$o+M1TuVTf1y`5*!v2#~rQ5Kb}E3d2^9oLd<0A=K9uogdq0njO_ z@c$u$-+pV3TT{D%U_e!Q&cE4L?_Ee^t;E_9l#I>}{O+6WGAwq;ux;%~`TRD>T>ABF zSpYI6DcGr83HWUgRU~6OgcP8?s~N_YM%MZb8?bt6H$i1S;^t+`mvdmSrjj83 zV-SmBB9bUbn7d>#9E)6TF)els^GcA2{rjnWgmlP# zc7my@W_eeQM7My^Uvo;z^g)$YvO1)Ftuu7>~SPSYY7pIraV(ilpe zMSBc}8Ae!QhC!i)`|!@71AGH0e@x4v9GO-aIAq94oZ&ItF0R3%5|-P$oZ)1CF=0>i zAOPn*qPK~e73+-Iw2F=nr%mi*NM92*V@RbCK)hbTfDC30-V{`TsxZwt?DyKSjDYeu zFl0jN1B=yTn2X_xMUSbC_yuKvO76J*2PLzXq?DAiFp#8S0SOI9!F2XkK~_yBO-K$F znf$eP7Qq8dgarq-pe)}G8J|pb!e5rI>;X(zu|l0jA(I$S>$AdfRdNz-<#%xZaVrW8 z5?s>DDH>TEMLs0K@Ls=jN*R2iA%ayO0E)NDXrOe6s7H^rye<9sZ4(bsNkJ3U$OK>T zs=#f*V8M-T(2#iX3vCA|wKZ#KM+jooDXD>hOZW+V!hzb_D=N;mk19eK9#ksPs{0U! zQyG*oZs2h?M_LXZFR94#S-^~3+f3(7s99oP%gh*pAT9>bk2t5~{qnQ1x%1@HTB&&# zfl|P}poU=bBfmI2{M8J*Ag+s=<3cHjECWs!iieC-?Vpj@O-)T$s>oz%<$wp|WQY~Y zbtEQMnq6^aPd3P1bMe0+dOSI;wcPdN4b=$#b&=Cqe!H6M0T@SlQQbhJ?Mun$WV%Qg zi2%AG3MU?-o1p0L1s9mNsi18`6+x6YX7=ZM87~WA2AkHcwWLS6AF7DA-)-T)RGhV%K0OVLhGrICf_n(kvaz+5oBnJnFWZ+a6&TgEguX5!sRZ z4$$a}dxflvO|JYvY5)WPjQUsRNkQc#+ff*I*=vvNANHui>jITuPR<6Hu6lK{I?AlR zmu*qAlAE}W*WE{yXMcYC*3r%mjU5ZSuDkmzTsWZo41L;9ENFN+fC?Q7Mn?s2E`EU@ zFEavwNT>1{Gz`x$WTG=Gv?$V_(>1V$ItZboYCcQ~>RX-?M-am>uLAUY* z_5+N4*At&4vM7yo4k5#+{51&b`KNl+EIT{BfM2WmIz|ce937FCQ7{cqQ2whEwOmI?1eHekLsqQ)^fdUJ0+7FO9&MM1my?X!p^hJ?7#qg|Cra;gk$KCq#07 zpzPQwio}Ff)GpV9c0)q5VA=9Q?C0rnCNaG0uU~ggyT;x@WYV3+n}d~U-jM;UwfO5+ zu2{k9cJZcm_6Ir-i=6}M`0N=Dc(i8u=HZ3zc+)};afIne5POI%G|DsjBarz-+C4=|CKl7;Rc^tKgm%g@#97HfrcRH1j(06lz zl*JMEjI2wSsC>fKeyOZvB2t*$0PCX%Bh|rp!s&6&6vAi1msfe4pF4}kU2HKgDuKJA zdKEcb2kCz}IG{H=oIN5(j~IbqwNK9jOal!xO;q|>&q{2VDsT^v<|X@74dxa*&PWCqKh>!I6Y4wEUh z--PpWW-ELr6FE2CS;TP;(F5|D_g}u8NluOdneAMzDcV9kL17}=Fbf~lls{ZD;3HW) z_@6F_Z6LGmLt0%3X4MOvg6`~dIv9A4PiA2Uw?VW5s~DMsNPqE&_yJeorqK;2h+p52 zBTJxcC2X(()nNHPG<()s?48=qyOBReC_`5d{d?YiFif}94+PBE&fMtLgGM_C_^g~@ zh3TI0@@fEzb?&nPBQ#%74WzO_Y*NEhnjLEYMCbME|#Ww4Bi9PE~%;=qJtv4HKTWz>#J<`#T> zHGE3v8~Bh$o?IXZz?)YhT`_^L56aeRoGvFT2`Kx0L-Mi3;j*#lW}@8pwxMaWF30|= z;939lKjYN*pEnyfOq$hLIU(TBxR#;CKNHJ_yZ!i+`s<0tC7n|_l63Rdm+hOB-oKDc zUB$yh2=KXamA@43t?b1dY5V<4u3};dv`tHp?;Zk4lAZ!i1$_r@9~5OSX5ELJ=*1ti zfHT}RT6KRmwBN}^XX=*1d*428zQQG8`zg}`zc$;B=C)%(=W}UiZkUs9AWh8>nGT@q z9vT|#K58k{+WJZaqNSN{=6@r8kF6bk*15!>%dy1Hlh!5{^N_l=bZ z0z)eL9lz^gA?Vq(Tr9a&-2h4r)+y?lJo7lO?APr~q+`<44$_9C#0|I`{o9xfiS};C z$6~aeM1{VwL0?O1>4%xeH8;-Ba0*&1OMUgnL{gQz9Zi;w4!#_(hzmfYpulUi@S&Jn z2L=qKHMB)AQJQ6XzWb8s_ItLT)A{Rn{&2(lWVEbFlqD(IQ4Z`o^AR zuLu*7tdIkJB-_!cfM!pfIViSxVP=K-;R%{C022(427&X(`iPCMx#=N!MU+Z_qM}uf z09H}4xyd1Ux$B`%8#1p8^&JCK?UJz_HI3el5@U|5YY@vTh#`k3pm?E9#5tlv?{sPy z@%aHwkFbh>fcWR1k$QSfuMNy1EFYjxY*Vyef>P!-6Uou2@jyZMG1gcZg9I%4$%l`R zHgq9qS!;!!-b0l*sy8A>m=lUvKmrY6gefU0d}Iv03f)JjN8y@EGKHFmv6Yo3&ps|I zBMoe9!-WudaW*)jEXu4v1Fxmr@tBTGL@$bElD3{bv&|R+jOxwEx{R+H!H?cYEkv4I zr;tSwTe(?vLLZv-Fs>eX--aoF1fJ)!A0Y%tz`7ktAyX*@ZS zyi#W#2O|*Vtuxx|?`Adv8ie~|HE9x}1R0l)^xzp`Ztsjq)TH0PelfDpBN7J=Qi&vX z=8mBDSVi7{CpB9Z_hf?>{&p6xxVjFqkG+WK18 z-EmR6z~9PN#$Vwuw5pvq6}mH*2-`1?u@-m1kkfnvzjx;G)ewmnFRt_O5Spv!&JpD~ za?$rT7FE;A;JArfcv*sVM^X~mkB_}QAO1>4&wF5dG9`b!u!&G zutVF(X-FE}+NmL+_E7DO=2)NFu3R}>N48>tuc2DfT4p~b2Q7p8XA0Cf4 zA4@BM@`zmN@g265$?k&DOWVpK`dZk*Nmp#A8$inKAr^ zBg};OKIU5#k=bcjo)|cBQGv@LBM7 zVgd|ngk|W5b_eAEbtuAo;*Sha?#N-waAMn(K|k5-%gpy4`%c2;@QSyNg=xiYF0lQZ zt<=Vhp*Qw{(h*S6?LrOwIV zDe4w(T4KqTyl#AGZx9bd zTeFQet9YmYK{4qSUfd|UX?T~@Ogwq=*^3uLMe<~eylC|vJ#fU4J-ip(MK6}`4bsjR zi$6Lt!;J?8F8RFmZ7r31SenCdGlrBQ!wx&?9+`QZ$N;Wp^y081nlM;R{)zTEdbH=_ zFaGf~FkpmtSZdGF$n|NSwTzBgAcFCp&00rYLR zJ@lyx8$VTq6;%t16wwU^?> z+PmV}B7&1BS}Wcj+DEbmx=B=WR4BNO;E&FRAEN{ocd+M$LxOHsZuby0$Mu+G@un0v zlL6Snhl+{`Y6hQSl*yv2=rf)@YkqqH91ULyJ&q^U{+l<*`a%YcNB6{L68Js#1L#a0 zlXT|HRfHr&n+oMLOU>-{-?aBk!P!x zqvPJl$f=06C>QQi5N(H=dPdM?Oqbyuij)Y;6a-FuENC&@?CiE8wQ@SYbdPSv58f@5 ztE|Tr{*HA}nqPj@554B0j|#5S;p=1t3P{8=b5TFC4INy7NGKPq&3N~h*VWCFm2QB|qY=WjmJ5s7wX6E`=ezjC^&YDbj^~vN!W%Lv zRTXkXurM}yP({PKxiZu3de)=k&91sC3cze==y#Qs^^h6-D6SY2rUv`FT$)KSjXy_N zKH`L!?WjwZ!O!yvNtaVn*|msCQSm?*{)Y04D>Yci>fnHd3VsgzVb`E@0Vd|Ny;T&t zW9ZEWbXpi~LtMscilm-eXqDZ&1+0Md5RMs)BNs-F_CZ%<*93J-zM&ePc<3{_Iqn$) z<}?jaG4do-6c$op$Vw;Ydvm|>Y9n-s*h~GRqr#vvl46FcU{+z@1sPva zHRS`&ql$fD@Uq4L<)CffToe zIarO*jW)%Iy3@eQ0-?8>B!3WK4fQ3agBlEXiK|UU_jM zx9nArB!Esm6*&sZ`1oFOVf;Svu3PkoL0Cx%!6Vs!2L>6A>j#g2N-BF`pe&)IiOsYc z+q`f3;l&5b+)Gszvxu;Kw{|aJr5yNCbSD%yW9)m`Sa6m_nnjYF-sO$+XcB9FMY>>y8&14JK#-Y4IYQ3`VF zVQX|=yV_=F24p&;z8R(*f%;idb^ zJ><$RZv3uC;9-$r%{J3DZD^PWY66rTvBu~>`u(Xqt_>mExA>0>g( zh`}THok=Y?A0ePN#sSA(1vDn0Tn`N$KYxDmC)UDG#DtqZKTT|f@Ux7dpu_FGy@#^0 zJ|zzY5r7mL9}WrO0Ti_H*s;`HOqAo}!welW&k(bL%$aa(A!dOyW-awH_a`&0OMo)a z*QY^p=iWWiG3gN<0Cr5PF(+nCS!CgCg!ItKoC(J3xzoHjOazE96k&{$U{Q^xleEE- zNoQ;#ustDrB6e;!@_XcY9J{jUMjGtz%j^;wJLtOV2J-Gwx+c{P7|)TgARWlOE1zfi z5bf2Y@Qi@#o8#;ZG^RixQ@Z@G-rhVe$A4}AElGnYAxSDpl8_V`8YMJILQ)!#nT#3I zs6t3Ghsv0-5=zqetz@dqbEU|T3TdEWzpt$IJo~rz+Ry9xYxl?cTGZWrU)Se4hvPhs z<8*VIrWqwL*sH1vD!xOnfj?hITg5G)r{NR8p=ImpW}oTt+la~Xh3S_p`aL>|CqS@JID3}i5OgC6adA9a z^%Qpr{$Gbj2HTxub9vV5*MErEq!DZ9e2T%kUCi@5;nj&1IWa%;FA1k|h<{gjl_I3q9yK*R~w%rN+3 z0K|lWu87!4gNKFesS*?-xE=_F5R?aI5zr~9KyVU+SJnJ8+~Oq2KK{QPP{JydUM9zM z!6k?bc=kJ5tASctxX*Td5zb=(mI}=PS{o6OkX6?I7!xBlQ=TeedbUT^1Ya~V3N@32 zbUTkX%$;vvzI3|b%xwnj#2-sw-=?KeklpfY-3)ijcW?l2miNVjA^O%6+2P9*yRZG5 z3&67~9rC`qx&=YGUI!5mECLTD`v?N`0==8ebKrYGBc)%V{ltM8+2tqdB}#7J+rfp3 zPF=v?;r*dd#Y&g=Lbeaa5~JmNr@*Z2>~icy&Es?j50=>eGvJ`tiWOT4*BmgeB6cx| z%!smVm-B~IsH0%rwv!O}E@}Q`D&7!tGAbspFb$}eOtTJ{$1!~XJL5TJ5YJr!pRv-A zuk#@MPMQ>g2$Du6I9o(}3CA2qT6$TThd&O{5gUZ<=SFUaijHL_5;2{ zG^H`_>CEP;S+l$rEkdgt&I7^OWvGFQrg=OQqqD49f5X`g#c)hwoyQueW8UQ{Hz<-%6kR(V7|bx zg^0)JmROHn(=Eb9S>@xB5}>Z`yDi}#AVZmAWz}hozPpu3Wfj#Vh9`UWY=GZ1MFZv- zg*xu2XG=(yOK5{=Bd(qk2~EH>4b`3fRk66P{4%Kd{0xH>{+Tlj%@JT30oc#_`ul~2 zk$%A~o07)U?{I$rRs9~Zf-;FETvV|9WP8ThSoT{$)BwQe({Rkgckbk6_&Rp{I8PLv z9J>KFP8}#GNpg_tsmX2_u>;p3cQLP~jKTX-)&{rTf zP_F{2GPLg0)4uFo3@ai6O(AS|3KeMW;5-4V?lX4gi2`bY(xWDaiU}Jt-UxOoJk8h& zQ%G|s5cJ_(1?sr5g*(e62R;%BjFO$hfIc6SYm`=);LtrzV)9X;j&Zuo1Q;evbbzp6F%lhG7*3wUtGHuW2snsEy>Old^vCkZ338? zkXwn+MogOd6LlInnBd0|qGpBV$?1Wc#iPK^&6aKo2k>q=J1xdeLtZdRA+2<*9ze;+ zGvdg@2S*!vW^u9Z%8)ifuBn{#ZcAbeq$V;p6)WNTI-55wEh$}|y?dv!!{V=YSgijU zjusCq4;Q1bC66?y&MBe>Ss!G3&E2p1g}^5w>+Vd~a^&DakGzXi5HtmQW#|R-JZm;pzu01I%JP&_>&1x4fh8I)>nwIcwWCE

yXN8Sa2fwgl-x*wPMyB&$A1HEL?^XFp*3=rK9 z?8S8ri?8T;k=xJC%1o^j80%Dy=gN>KGGQ z{5M{Xq{{+ZZy0RI$Tzk6KJqlJUSE*3Q;ho3@lpFEIaMRN!hRYww@&|#pPgnD4}B_$ zvZ(*I*sEFo%orJwf@IWF*CVfBY$Q1H{RrQRjUYX?-ik$hl-N}|{YD0sc@jSea0d?H z5RN8z9@H2t%3#=fCS~c0#|`GU_j8?cP-3@bur$mNwS#a)S;*1f#Lwl%V+#yk*NOIB zK6?DHVcnc^-2=L0i*gq^;qI#~)FX+ngj)C>#E3#A`P6NQcNgO~vAw7&B1-MRstG?- z?|Ny*4mZ^eTb`;cflTVSLPUY}nMZ)<&7X`2p>+ecU)g)RBs-NUB`b#fK_zIA=&3M+ zAv)dN{F_;)%!_|Xe)Z)RYwTZ%)drruQJJ*u( zM1=F^%S`n=bmI}T-lRV7*JProK-b95O@DR?>sbb#*;nP=OAGFzh^Qi2K2yYBo#uHCb-6_>FXzUQK*H|anRS8kI| zB&uDnb3QYLo|Vl_T}NMu2ok#7%M^kP3fG%k|J>!M1QsgMx=Pf!>1x=eY~ymf4dgE#$VE9>neARDArM z;^MuK5ya5bVqgFsK}ly;CYT$WIqtPkCoD1$g4n%E>S$Q*S=U+h6k#5RT)UpeOQ2C(oyiAYVF!Yyrr_NJyh6+aPiH84?*nM@ z4P0hvs;-`Z%FHqy-7>tYE@#oH-}5Ip)L(_PyM<0s4MheQT>;y z{yFXNnHmvvgCc~I7>tZ)WUmFSqpxH#9VL{0LK;Okff$5__X%iHR24kyH-I!cV7|uS zryOH!1eXgZ8NDdN(Z$3>Ig5+FEODZkv_7aqs^S4IYi)&i1tQ)&<%Pf0i) zX724ufk!T+=?=)jXv~Eu z3k+9)rE+Pr)(a~aPkk+C9cp%V#U%yr>5PKp)_2_yAc!tW1r^&y1$JwkE4ysv$}92l zf|aG*ju%{LXJ={s%7s_h7W)lf8M=5m)t`UHC!!LgnBk@}k$?>Z%K)!_njR`|cB!&u zYH^p$Uq2}cgzCnU)t{jSoBA3TynE+| z;98J-R!l=LiZ*zaQ7^{u{xfnpWnct&CwKbF!b@SN5LWS!7ztVAyT8 zQ_9%4ng0H;YUv63s;Dp>6|O4kU@k#|dDkoi#sdc;;=Uwkfc6ae|y@S}1n za(Jd8f!V^egL{VkH=y4P=+mP{vGV6GZ8~gVwgkL5yaMxMOt$bDIdy6VJQCnxit<9Q zV6sbk9x@K%83 zd2Ts`Ha-~c2WF5OU%mQ`+DWr% z=~Ieu1<>6>R3!n7`unn=-$kdvjys^ZVNYV}0;o zX8_BhWt7}NTflykEG@}a!WtPsJU}pgr*_!R_>-;xc7@*6a{m^b0jM_|HM?okRQvVo zDkIZE3+t%8wXvpuOi>Ynl}neNJbsM00M!T0GW#uvtxS<1Lb_)=V-3b2MlJs#tvgT!TBP9OUEZwfKLJlk^~hAV0`b`0-t;rnMl)iR4=Zz^bt6KE`7UTM>QaSCn?-5H{(ref1!TNa)f7mJM9T6<604aD4$bv*)4lj?h z3fmEXeR+y4x!|9a`U z<%Mk{F#ba^$odAY$SKh9tmEuihLSbQ8!Qr7ie8^G32zs)ln`p@H7KTO?=7_t05`)w z5S9H#9LhHCni$P6k$p0N4!42Xi9{xCO#>CezkT|| z!a%k{G5`WsL9p|?ug7#ha7UF+i2^2(hSaA3O^l-NBK-fmS3XHqxLDg$m674x*e; z$MA(OENwfoiHXIzxdX?IbN1WXInWd}JZpj(Jdv7-?)}HDn?f(Q z{FxK8;unJiX)4ncaAt@}l%~Q;kgP2H(zBWCF~#gqEjCj3Ql>y?gF*N3AsU7e;MLW- ziv9bupd5l0Kg)n+?S>5}Fbt8j$1)-W;xxljWa1Hmv3uRmpZyli)ID-vFcl$cFnRYdWT_7@wu#*KCBg@n4xtWu zOrFoa&&7WN`bX-9J~eqV74{z18yd$27F41AoQY9oLpjoT0 z%|y&Y%+yiXLOn7;CFaBl9Hv3FV4QIiXFEBeHBjl>cdBL-U?u!;ngLi6WNs{660z!m zV@xy#Q9`>3lal>HYe1F==U30#sYjA}AQ6LC;=OU-D;XRPx`m*Mnx)k2%(_AxKRQcY zW1>0H+&pXiEk4S_hbzgZbScP8fe@&`q$DL#9jc_%=7T(PLKx;;zux;F9XmEgtXsz@ zv#_+(&DK^R_N1jzs`gtR%;{!2_l89cAtB<4?o2ble@;Y3mOOu+!W{&N(pPe&%%VtJ zy=RR3O|OLusb9ze^?zzJ6E0nHxv{+O3rRBU^5TU|YM6-9SBzt}%Vh;->J(V^`7^Ec zkj#+}IJ4|Fsi7Cp+(_*8oWphHP-r7W&7owV+bN%#2eZRc2jDF*_@ z(&w;e`_%Sk8bc2?<|0EOXkVy*pLNTPwfNS9x1q|u?bkXE0m0$P?OEEnS+lU4g&|!k z50DD7b=fU*9##3*uM|J6DGHZkJ50OVH7PPk$cf5oSZ`c*9%TKjadtlbF4Q@R8FKbL z0R4CsN&Wxw>i)Oa_m4gwi)L6L)SwObUtE;vsQvBpXUa#6LBl1yZ@P2|{(ivhBYg*s zIt!L|DSQG39I*ZO^HTXc8{oDy&7EqOsS<0BG%QeSfpf*;LNK(-_Fc`GZw7Y-lAQd3bYh&+Sw<* z0m_~GuW|uFnv~+&OZ)7$u-o`k%_nT zjRc!8;mpe(A4R<974!<13H_=}vA{ZEYl-x76Ype(M;IPm zS)#Zgju|~##ixx-EuiCHzR0;71tkZs%03(-_kNF2`dq@?N;hzXfHS98%K1Sbp9 zB=}H1-rls3^=z)~JEe!D1tSiI&0ra5;&{H(b>*7+&(7~tBI zYO2P)W=wySi3`^cautFaGG%rVlMVp<9TxAIzyPH!4{BEi900i_>3{{p+3Hf63hi34 zM~{(&1e!LU?0au^^FV?~*b_$#48+Y3kk_a&)Uq3y_bfQaJ6Pl6BiprWRO4Ohbfi_d z2D5I=R!1Q?x%~S^!X+hGr;&lAHMWbQ*n{AY%ii5m?K|cD>p%K7uf z){dfNU0l=Bm1`%6-DxGjn&%hn!T%n_RoT$n9T0*m%-HA^9UVHR;Oy-*zmZdjOw9Et zBi+^tuKM&V3?~IhfQp1QO(JQIrYFDRTm)%SZP;(5XK?N@82!LEvf!C%l7@UU%^`0T zi`Z)bU!|qOw$pXvj?-tnxrrH@z@aytah5fN>DPdEu(+)*FBfKm6I4`t^qB7AB9uUX z?Q?z;)w15Bjs}Hd!~?Rz>kf)4WQL5}EnpQ#nlUKvIZ>)@Z7M|$0Z|YE;5&zMw&%`F zCbXnqEd9CQY>_lO;{ZE`T~=+7cBsrX)zv+THUzcK5@e$v05PzHMEv&^R@P?xc?j9z zT#Vnoq8i=^7W5anf^_>#>X?D>T>arga~**BNFWPz6=_!HrL1&az4|+n9n)xuOl()l zJE$l*N{rDMIZ;NFa0MNojm>^rG$xXj{oWVNy^pD{q?8md1}*^lwarQA;k|pw*RRti zUgst<2jgV1iEEmgoT2hVauiJxUliGY41`VEBr(8mvO0hlA5HMCAlYy-X+s0=-zAxY zYOG&>9g2yQj#md=9Y*SuLgjJm?mT(2-|igqk)WWsx#{*&)15&C0q z2BRcGoa%1ft1j|dN(#drk>D;e=S{_?QDW{&rvqVn?KW_du;UBw3?2ns@)lL+%I^)1vb`m2dnH>`(psks+W}S_XCw#%S&icj(s8h^d?s{gXi6JOWbcex$ z+WRo`I1)Nb1i5J}J7*Y5Xi$j=GgTvj2l^Sx@A$`ESANIoq$FIZ#nW}+p#UR!^lYh0 zq|j0b7Z4qcJmhyEp&q~qrelQ<`06sh38OyfP@H~lF~LiigeWTqobPO>p1F%I?zF=Y zrQL0<#(D_uIZA>ZPk0_79ri4tF(H9MP}r}_Jb|eotClg2RX9AJnFX};FY ztP*>6nW{1Bpx+a}D9>eGTQw#!MC@K0J35o-m!W>cmrruK|6vXB7^W9xY`9s+u5^4e zz>A@AjNSl`$vDPj<(MeO16ZEor^H8a)W(KQub?eecLDAEk<;IhJoDFd-SFE^J1p6- z;l}mr22NXuj6@wq3}toMMQY{n22F=BKcjLI9Uljd62+#he?15@w(L0lZltFLoH&(oN>PLR(O((0)%pm|~fE zKV}+mV9n!BpE*M-%Cq5~^a4nMiWTIJ=>@b4_)q|)lxjTqGiUyS3hf?pC*ltdN#=Mw z$3t}BnFRS8F;9@ea3^>GQd66F%e)E~UZzVfb3?emHk!nXfILs`8a-!{6KKz4QVfwO z#_#tID3b}h@NcEvV@wA~1cXbjC6s2(ro4nYp~@zQ_fK!9r&ALn-<#{~Os51x!<~ce z4YUcMz`p>(-%L%tbHB#Tztw<%0SrO?sVKvcY7vD1TYu2-L7YGs21F1(RT;r8-r!7I z`53~MD^)Amj1|k4&DT2u_5fvtZ%=-f#(q5zR7hhD&Qo(*9q#=GPG4|oTlJ~JE~Fp$DX7Zi?p zE+8lk41F1;FjXjhnmwB~cb@=&qRB(4HBCK|R*MGv!?X{$BvwS22(h0hmd>4T608tC zHypz*p{xZ)A87`f(@Y14SAavjUvMvwNM;uJ+BoSPpnIG@!6Ae-m{+d`+1AD6a-lf*EZT;8 zijo1)55$K_Ty*qnc%dW-s$O~%G_=fYev_hxxn)2@EQOtiGLdwUVn<1Si(-*?%(vf5 zi6;Yhk5^?`S}i9UN*MJuEfRYW7?ki$yrJ>#q3L{PhP!JQc$g& z%h?fxu+8Ds=ijBS=DC9F)#Z|~S*1BsY^m4ZT`bsjWKbG#7#5ss(F(^->S%G9779wcWQD+)4i{H7^3URs4_&X`#pcOiuh0jwJu;1!T zVy`v_TnycCx8hFHhW&fzOi^IczZ9PT|9pi1`oRC8)A}DS&wu!C;a>cIUFrYO#U1J* z7GCBLL2h7F;Z+l;!e<;l({LkeZ(kO5(jM^e$)iU>3&lHbjJ{9CR1LOqM2}hMj}o1= zf(+-#O~wdrwns#SiT~HP&1CunKOJpYOBfZ#)pi;P_-k3{ErsH*R4l)yWsvvk)n;+_ zyxSoXO|YpL31q2v{5$g6($Sz%%C|@07Tm}^*;Z;R=)FF zOuoHc=;HXf-K1@lZNc70Z>LPPO1L7rQ+h_v{ami8BZ zt?3)#E%9dgOTJc1bo}Ri_m2Pk)A21_s*e9B=hN|@|HZec)pUI4zh3*lUnQWnM@&(e zd6R0_I=6YD{v|!y{**SQY`G@F?<-BD(`IDX+ItJTo`sozTUJj}EvZT3+(+3L>6_L< z0S)y4za)8N>FOZ!g$DzCm9S+x99-BRt?w4m6(!&~mYYaROw9{nskL@;j#o0XGM3PX zc)S1tf?#paE7||K3O1CGWMQX%&l^Spp^o_l{%E4Sj(BOf7cI&?-z-K0 za$y*~mez<71-h3g5=a${KNGf;&`ojNO8;B}$z$Krd_`1^h#F$n2xg5mX0!rYsmNaf z3o2T#7{LI*Gvv#%tF!so^yZZQZFNV|gyHs|#!VCE@wc?FVc%F%@&OzMys324nO_YJ z8Mw#5{!pC|$`pesoBv}~Cw1RNqIHcIQM)TEYeW??!>9akbBnP$)BQD8p8_pC+a1;^ z%>#86#CM=Chx1QhTk@X3_Q3(k-qgnBs%L3)DWE`K(W`M(X){<_!wXE&p!((cLa2f( zWv_i?SfF{V)IkLg-Mf6@0$ft$#~fK8{ucw*Z`gndDbi80B`*o9a@|nqBoq+)_wMCs zlJDI+Pj^?ICxZ*F1f>d$J&#&;X2UyRDlKe??Qpt-9*)5WdJ>q20=j<|SD2{6OF|u1 z%}3=)fK9=SB)1%BA4rldlyKjn>f%_y8%D5(B2+VK3G~4M1Hj%nC!}6T!$6!PWgv6& zUp_4FoT7<}wx|mKBn}2XK+s$u5b}|;&0qzhxCZ6fxnl>WKbT1FQZYjFe+fvAkqs7e zc)##os5iOS01cEb=#L62pkJfQq03sg$9{U(T5Lx{Stqe$2NSN?i*;7zlVq1bfdyA(c8bswX(P&Eeh{4owJ^8J|9i%~4)b9(4k9eo zJ$+p#(Cu(B(cR$^y<`b>Ck+Kwg{>oTZQ!$+%$%K8@%FiMgSo=&w_-YX`7*8|keILHc%r5jU>?UezF_s_*RQ8WJO&$N z_X;{gdwZ0lDt-Di!a+sAiol6Jo}v~-X4mfL;lC4pd48?a#BnS<9F$1G(3k#`pO0>3 zJfH#;sU(5{qg+BMnd2507t|6y8q25-?%yAK3pSOtqvL)sl$kRRgofrb*CI4+o7T!7 zrY|8T>{c>Z%Jh$G1JP>imIlEmDEfghl(n?9-#}~Vu3n-kKYbdF0B_nGgk)XCL&=mEwZE z0;hrL{8gd`A*+-`@)7;V8wkgJIKU7*JcS`PAvWM^{(EGa3Cq22nSjiP9T zipVIG3`hqEDn>&;dd+O+MC4ccJ}eyYE60Tj*pD4*fT8Rw=73HtX~CbA#+)ocHV|fo zjG6zsq0mS2XfTrnuV79G^&F4{A15vv|I2x&-6jDq(RT(mLG@FWYzGM{D=XmGF_#r9 z0Prh#b3_2U7~|t7PEcR&d3532xkXM+FF?Yn*J#KIHNqSU>dew7TIH@QUS2)JnDj|J z9@{f%$y}2>^B{CX^E}zXrv%$nWk~2#_$$Vd7pa>O21;^!kK7Q7;DSq7E4qvJKBE$l~gEuh=ERs7~~}0K6Bj+ z5>HtItQi~4pY-$`43Gde4qWE9;0hubw$~qng~OsHX0H^KC;(uluo~UYu6w8lm;t|y zzoyrRJV#v3^r=13VE`uV0f+qs*rn3$fB6OVKZgciAxxjCy!ho-wzeSjXq6#6P@2%G ztI4<5&OOPtA4nFdsn~F_BS^RyNQw;%u=2eHNCaPE1)hc`e(C&Hm~a zV~%v#fdGI&e7ZtCpnmXsuHLwjuggkqVhHv$OfDcjjU@m8QxGILZ{$K?Vmf6OM!mbS zq8|=E-en?beLNK*)`IXb5g}OPk(B`|$ml)kWi=2V2sn!{1{fJ_MYc=tj3mj>uVo`x zEc~CMA`|d$_!z+G+}se^X<$&6mS;)S1Oz%R3-u(1{!Dhbp$LpHR$p-g=ziejJvPWE zkW3^cMViN}x=R4D*5^P*r6m_g?1W5>tR>5T~o_K07#WvOMDeT@tA_b}EO^b^??#qhvJk z5x9^63~`k4otdc53qLX>Vd^HMGjvUek&CV^LdGFbF`z;*Ii}VjUm0=rcs`kW@PwP2 z=MX{}ZyY;v1W@`GH7Uzaq0oO~vcl8Xg0Pm11LTWBpy@~_Fl7XMHXZ8^sP`b52(=_@ zP+%w_*MG-2g_?Q^V9eqQ=y4o)VeuQ~9Q??>&t9x!azkap^TFg=sE-9i`K--NZVx;k z`%3m)f8q*rwsmECLM}2`h7GuO$76*gJp4tGpJsT{M zoN^;*JPAz@QTQ_jK>SJC9VmPhY`UIkyM$Y?2E(=z>)PYw~!eDI{66LK4E+mb5J$f)KBb8;Af1#wrV! zC~8M3?m|V$Zz*1U&0~`uW+G8Ay7DKPQkn3BJOl*P=;(f5T&FuY zuo8%^DDhmN@^Y4is2NTmZ(zhuDH0ecibM(X3jV+hoxIMnBv^kKZVF*NHda_!&&S|e z5VCn52nz$`cThnn7*F6zFgfGe;3tRy9VG21Rg7`(Qw0|=0k1p{|J_)X-~ul z{7QkO_L39~;o){McbVB6EBe&5wZGKV;8F+7V8TVbmJ}CPPIbfLoab-eD9g#kUcNjm z{UY7i$ocuycN9cy-x&g@W8y@?j)5^Lq#kHKtW!_{ifFw>pCepo3kz)SIHN%2iDDQ8 z0UI5RjO46v?)`p!p+sc3Nkd#qU;6DElbzu@IxTCuNknp6HW58x8FKldl#}pJxx$U5 zex~6<^M2||BXN~ygpbKqcZ30{K?+NS7QO4Q7}LVafw-0v-F~T+g+AW z=8SQzzjrv>owrq8nyLEqF70K{o;|Z4d>&PRe2t-9=l0$`d*Z2+^VXR(2ugsTAcl=8 zY*NDy4oig5(XP(UG~+9;?LKzw7-Jh$8Tf0JnaSkBjhp=Lg7~0-%~G*@t_0|Kr>SqF?=|@L$V1pKoZKjajw?ZU&r{Hz5o7cD+WXOKUBN2G=pbR z_)UF%Hf*r$txXS14B?si>zK)5c|lt39`=}a0Q}6+QF%^4&}2k@+A&~CWFsy(7J6h* zgd1&3&41q)56vy_{9*KuCKG?7Qx~V#J;Pgx8i6AFcLG?~g$rNq3>3_QiMM12Cdu|C zQu7>B=U?Y{#nwz_n1%tx#&kXsvR2BC`X&aoFPD!F&y#m4q;I8fNzCp;Tb^CjwPT?= zt-WUw*rm1tzNB&X_R3=^cP%aRdHgYPwpEX$(juvJUgeMFwYTE((SUc+Hk#}Jsv=SY z>T?D7Uvk)|2R!8EE5~I1_ArwO^*cs&4|s%59)Ja!QtS%LXUC*wcbCdAW%MgJ7BD-( z*NAbbmaeYcAYoV{DZP{c{b|m?Ob-D7YtxVS8w5y&rDu_U-&5Yg);o18Ya-bWYKyUdMzF&oxKubyW}hYah^dIduXyC~iSe0Nr&- z51cMKfY=VB@ioNjUIy?v&eG-$8G@@d#r5EU1A{#PGnBYgd-p0fJfb}3Qz?eR!Lgq@ zb&1or+w3*~b>k(OQgE+eQ-M4JAIzF{yV}~ zUwl^a@ijFm3X>>@!0_fP61~?z>!3WxGakih{U!P;_}6h?e=hs@meoX%K3Y*>VX1c9 zZp6@`5I9vvdD+mZY~6aBZLOH((0P!0A-aP?!?+tw^8)Y=#)5hP+fT_23s^Kh{ z^H{2HbD-p`m|Yg?{f&J$!U4pGf=oq;KEvMLd)+$aXL?sRet!=f}|guAQgS;{C8?vS0J#I&@Kp_R(me0UADh z<%>_l;ML;$M|(isM*&g>bA>3{r+4q;Oi+QM2{GNnm;{#I`6$Kr5)JYErj_mzoFveq z)_w$zf-wfQfSb6^-JON!gV_*js)h&Wtl6_^`QZB=E?j_DuT-rSlyqC$Z)}ePv!?6C zvjX~%m#=TS_XB0^Tw)j3+i$f-1FjmJ22R3Fl&CZ^CorlWIkHDY0ei9yJ4{weva>~> z=2P!e#0yU~lYTxNP!vPn@(22yKP9MDD0@CajZu1Nl6@ZvD5ndCTlA1&@*8wOhg90Fj#n3IIkduv|7yS z^>3`iW}Fg%MwfDdB91Eel;v*uo;?KyA+ZI8h4I=TI$a7{Zi-+N3rdHlI|3f+ekuUW zSuVTW6xQbPL2%N7qKEXgBJw%{O6k@$3Nu27sF`(8GdHX;M7IrD3jVj^r#LP7pL=ykcm# z?%m-IP>q_Yo?Rs_`s-HPW~dk7Zag?{$`2*7GH1Xufj^>T_hZZ!R&4WA?DSGv!LmDz z`i)Qw5srGt%)*j}1HcKTHiz5Cg90<@l;5SEnl^i*q9$hErmbG}Hcgm%@CdSb5%kNB zw}KTD$fS`GvRc}2c6kHA(u6>Ai2b7A!KDOU;WIG#!xJA7hoI8^*$izR8b!`r+I4;z zkR>&WlezhO)K_2#6zELQX$PskF-zc^nTfGG07AGT0$t;>})RIZ^3(|!`j32NL_Ahn2oW4d4O!%A?73WEN`%}w=bH%`tQ4jqNLKnT%Wc$b3l%$bES zQg}ND4?d%B6lniz)==$wJv@dN>{8a1tpCDP_z|xRa16%Evjov{`YFq)Q>H+_iuEYM z8XhXquYkTAgeRKoX?x7D$JX0eHUPRKBLin3??dN|?ONNuQl)bzl?WgZWDTZ1E<`!j zk5HaGc~S|7oYJkb(h-36;6Y||y_J+G;k14GLY`#^d}^mOidc{sBoMqTEF%!L7|Ti1YMPE+=QS`zh-9wDib!c zkt4`F5p)dBIPZfi4P6|I48b>`yup)M0nH`9bHbK}`)}WFC3^64oNsul{y!>r&}~7Q zq&%Y0i7Mo<#OQk98VC?TtmU4^iy7jLU9>=ta1P#i*v1PT@{S!+Y?!}x4YF(o;0QOE zHFv0NLnJa7m?55`niup=l%Sh75&B@)qJ8pE+(4${#!{GppUlP{8ozq;xHmKzv~w`C zNcaF4f*J%c1xN$?8cON2SZ8CBLJp@FfCA}wzM)%o>$}K$fBhtCHRc&_{ z>K#D?dK9~QD?grON)393-?vxzmr7tyK+wft7Du5K{m1HiiHc+#AEBd8sv5umV#p9W zZVFLUp$a=cBxv$RDk?^;(Y)U-sdt182@pY^V@Wzi;49Lat&6unv z%#yQoW&S)}ZRsW!QGh6A>*_T<>Z%CNJy+)cmVH+yxBwtPPL5jD{XncI*$?mHGGXxGMV0El#8Nqu6vzhqJB|i#qxElT>S5b=k5`0Ty~;+) zLe$EHKykNul*bH9S%BA5Wt$8Wa^GoC0p@Umd9HZS>*URK6z&mwdD!T^skMlZdAbaT zb(jh;CM2Qp+mf5<=NfRcS-)Nnf^BF2mw)o*mx%)@lIjEF-$ zP*agNKtj}xHo$m^>-}RdEM^xr?INE7fQTqaep$`5zvgA~WF9LB7`eJEQ$*ZOVJsUh z){AT1G%s6S-YvYnJuH9E$w6JfyZnEDK8HVu-VxurGP7(8wcHay9*XQOt*OCBoEq&N(3s@&lmv;{xQcnFI;S9naGOp z5$S~}@BWRIZ?)5xiZ=Bz9~I1!hVEhHD7EyD&7~1`=V<(-r(JD1KoEoa#uh4#ji)m+ zRj+rVpXg|}>wj&*!W2yfDXnht>rwvl7$=8kkMbw6W&{}^zF7)z)V#MYwy{m0b6ZAK^plW~!0O=6fd&T5 zP#Vh4-DGM*H9<4utSH_x9?kR~msvkmZCU`#+FxB=1I;2Ssqam@yxFhdY%Y2g+1`!c zMgy6-rpvn#ziD&%#!N43rQHn%zvlVN0%C!DN)=@`-bNSDl-paIO|@69*fp*ECD9-U zwijm)TNn)riK2l{8;rYXM4H6)@ugBEQAm{{1A^vA(8uziqwU6+JQ=6R@^J!{qx{@` ze4amOP|BHo*jqN+diCm&!P%x?e@gkE2a07=-I0wNGV8rpu3DwqFdZ15BDLo~YPb-R zA_x=k;3P+E>elAqGMu*72WwJjCRAC}OsZ5QOO~*0`4xFCBZD=l+uSRNK*YW}IKSY+ z*}A(Pc)>C3aoMH6oLqoH0!u}V5fc#Lsj!Y3ETgR5=C%?k59Tg_TgUa6?1Q_8W}kYn z3?nvocU_?QYok2~vuwD6zC{--#QX|M9b`cy3mUQetVt(g6SvgV)If;Yq67_x2aLuB zq=^!o7-xPx6oj0b0AeZ{Z$ZPNI((4+GH*)yn4)PN)(X7HKf2YNQhfDjw2P`xbGER0 z%^pI-&z}?5xA7d&ud=QK1<$y(A9ul4KwgZDAGBlB1eMKqw@LLxBXQQ|2;KPTYO(jV zwG4uBtX47h{Zv_bn)QT>7WtqhkNkcTkOv7j>6r!t>M0N*^GTdP>Eqz}p-S3F&$@j1 zB`PLLd7QB6g*Q!jM+wfbws_8;rthbMgJCMsn=-9x<)Tq8@K5X|6ZRL@YZKl<5Q?8Y z1NV?|e$hgwLlon(FtZCNb0@!WUXJbE8_>4p^P@peVrr>Qc&M235Nxh7Qvy&5ds=}% zd8{<+&;(OG^871M0ayY6Ua1Qpb1;;_&YbN_8(M#_uV0_C{G}e83t(%|0OJ@wIXy`F z?b|d4V1i74^)5;8>%fWd*L_!(*eGO99^OYaIO`16dm7B1c2kq$S|s@jh)jI`h(< zz~N7ywy1js_&meenIic2?_bO6PZBw~xgcW5zkZ6=03!RO>j0B zc3^hvMv-ER3I4h?E+qgt&a#x3Fb-a`B}}^qh%yPkp)rsDKar%x-@NV6=qmRz-5jd6a1Hf zkYL|4`~l#f3XuSjIPU?o9qd_)idIn)1o;?U{UF>5`T+ngI9q!u&J`7#-gg=TuM#E+ zfW$Wtbf9UT#7R%+1WjO^azpp^@|s+>&i&xz^0&atT`B@p+SG&`S~nhw@?rI zEw{B37Qeqv4OYIsCx8Nvcp0jHdj9n9T&K_LxD(@!6=Y8StEey7)psr86WANz%eY-g z9sk78j3dODQ))EWP6Ho+lDS{cp26xmX=xY~w~!1tj_!)W$di)}2Ydv3RFTNA+U&(p zHm38h`a5NC@^TTvM(%?}EtB0F?935df%A=D;tVoyJU`L&nNDAU&1@l3^@& zE7AUzBUXi9gAXvM3IT-^%`O!ZDmSivO@xZkQC@>q8NNlE_Yh1H?iCioBLKW3kuyL9 zH~iV$B24r7Rzy18x(TM!kYHfXF+AmebEXyDxdSn{lub4~up}e+>F`Y076I8Bx#{}m z9}CAF_rnD1NyLf;`;P!;A>xEEk6ra6@^ATQ!_yfVfR9X6HlcYY^H~ntg`qhY#MoF6 z-|$<6F)sWNJ{LEgnMK>&>z>?dCiFaXoR^r9^DV(8)U|5u3kT zc4tBeU?=?ZUKM;Ms!)7vm3s6@;Am3+4}#T;e|=a?QA_=V%P z%i{$@DS8|K<`0J$Md`R@9J4&d)7RY8#Kg!ILw7(!njzq}vvqBh%oVk@nWQ_y3>Yhy zURQA!RU4hx@8)K~B?asg{9obP)00KQFq?2)yjj!N*u=nK=EmQ{ zdHa@b_zf#zfzB9|z9sVm3xY~en+P{5)4!c6CgX)E&%guJ&J)nF<>z|=Pv!K5?}>2D z#leAx@fSY@pg(N*aC(4-@89RpSut$`HXXg<)dK8eS+xtfg7*r%sc6&)*MtwW5jy3* zeUJEE5-?NUbh0XFuPBm)F&T%PoWzKC|LM#nLILz?-ETfX$f(XoY+(CMqc^{M7XkYv znti6C(5bmdbn(So;dOohj?VeRPI(>x7nY*(n3Y_rO+VAp+jJlz{Hd`O!neV=z?z7mtv79cdRtC zfFIQw@DsH-5K`yps%dNU~Qi`FtalHjADO^A}+gG>%V9PirAt8at8_J@KmhXZ? zH3N5q%fOEm+A0x~;o#`Mg5QBp9FG@(D|%XTiStqcssnCjnNCSbkD0o$<1}P)2?IP} zZ6*&#Ut~hQWXTbiseJO%?}C-d-!c;c?7x{ zklv{p4iuIwpbIf)pw5F6O_!n)5kowpumZ}Od}WxG!1>Q1MDk>CPJyS2EoSA=ZKpsL z8!*6Q#fswx4*Wus%cdmBV|0dHTwg5AWW^tCEBS(8NJ!JVhMm z6ZgDU&7@wrdn4X)a4YDPP^C^#@&9nQH~z8y{{CQCOroiHxC9{M4o%5rwYC1}F`lf& zLlub6v94R_g#ldgp~HvoxiEoOrkd29pD$BCulVkD-g?ejG3Ycrv&2Z`8Nj|>_Xxdy z{E}5P&sy0~MVT+qq{ZI2q3~e;EOuSKe=mYBfe0IzyQ3l^yvj+Vw}>^q7m;|7Z~!)Sx^P8|I+Kw zz1|wK`lhKJ5?ElwUmy~w(FkLFahc$}0NQaVNu9JRjGgHxnf3CpaP1-KPzS)cA(2If zpKEArWNQ%QCCB8ZuzukKYJ9B$2v*&jX2{cm{ofex646uxP{8?-4PxXW4!n z4V2mXQJu&LZb_aYNlD+S10$5jLYe@YWT0__?HAM}v9+b-Rf++w73ywb!^Wmf0#nM( z|2Rw9xT-PCPPDbLROFg75&YfMw5w$eC3EZDzB%WSZ8*5qLep|`Va*y#>c} zY&$7+bO@w_n{P_fPM5Tf7&q?d1243IG_1o64TWWeZO zxi?ib=fKyveRfnl9gO$zX5U6NaIgAjM9b;@mI*G=%PzXl>U!;hFERr)8zL_}6}Y#g zdwAQvJasf3G*TOXR7!n$xP7?xHQft!i#z~Yc<;mLsI0BAq(O{1@p7PIC}T>PL$UMO zhAk{XNHb?($U=^(*_Ssv{j6aK&K(1g1^jYw{VedRq|`c^ z$%Y|*D|^$J-cT38=Af?@AZW^hhY!2=hAOE)ZW$Y}jvFY1ha4O@X;4IHe`t+FJ}57z z`m|pDRVm*2kirog)?((Y4ztl@9>t6%BrFUbf1HAoqt2)CammS)>9Z5-h6QKK$!zbT z9Kb+-=~U(BN0c^19#{mdgRTZSm>wIJ`qKyi<sAv$GoM~p zi6{V`YI&w80p*As9&7pDjTk?C@gs#NkHP;XH$aWW5NGVwC1P94LfUEEh34?%{l13x z{Keon(KRPK={y+<66gqR)_a=LO{2~Bx*>se!9jCZnVM#iPFEXZZ|{#Yy>#b+K6*b& z6+>BS#vWsu-&VHrQV8&5(AbszY>rRJr)PsMjBxoXlU>IHLESAO%uJrVh!7E8u-v{1_qNGC!XXn&7luz$G=zC-Y0Klx|S?UoPK&AAixKV&*u@X1mAM zDvXYP3x`c_sm%Clk0IRzCu_qeoA-pOc3pIy^)Mi{5X>ALf*$6q>fO}G+OjTfV8TSO zW#E1gs*@MfI%uJi>b-tno^r+knJp4OVWo-ClL-qnepA?#_)%VP$U?u(adCkX*+h#= zT@5IebIO+a9jzoX4@M`T-u`V!{nvI!4}e!^BfxG)5utJ$^72J_LQ+Zz1)j(q5q%r0DNR{Q5y1o4$qQYV8)abn~H zx?y8>j$Al~x~O+Ae7J`Ec7YB^!BB3H2nGdzn+&lSBnP6~(!{n)Q{~#sihU#kGIu~g z2Y&-Ar(oWPzMpm;S#{Ah+wD7dVo>6Hc1bD5TO&p!Kq3jtHNrTWW0ARRV~-1R=5Jq9 z*AvGAZ>*V9lu&9L&JHf5}{{HE- zk6`*VOf<_@8#XjB6Q%SYLkKBSrb@X0;KtCA@JmL8(?YXAwUy~LR$ws*&i)vtFg4*3 zrqXWueIb&?9l&C6g^<4NQZle|a9FdT_Y*!J$Oc}6Ygq$;=r8i}eeMHhIM0|I65~d= zrdh<okwTyz|SKaSZHf z^tKJn;isxt`bu3f?)VjCQc8EgdhM6MBwchn)9e&b)0fAFAPF@uDlJjqjqJ`RV!v>yU@>fpos>#p9_b3^B z$qtr~=`x=lm2o{73vDO?kx)c%H}9#{Ql7A%_8Bq}9^y*n87b33gGNTa1sRNZQW$9j~8jN z&EL~M8vbn0o@-%f!9-vyGV=nN=y}UuG(cKGJ1!8E6ILeDMgebLa?3oC=ShizQB<2x zUVk{Sgmz_0#vwz5Wp|u{ zAlty39CG~MDEs|e$Gu}XN*v%BfW||51vKm9xJ>F?3Y<0gi5M?I=%a>`{j;8h@>X4f zdk~7trf$@(CCz`Doy|y>hV*=gEs5F~~Kj zhcDG8NDkqS$D2jFiB!ZIDx_G=sGAT6q@;F0zkJeMAq&h1wjd4HO;)!3X*Y~WFarbE zId=Fk9wBTnxXDuH@}um7ehxat@EvaPoH;_LEbK~ka!Pzz4WmES+LP@Lv>~})Wa0h7 zuK=mQ(r)RGo=A{P7(i{@*hK$--f<}b8*l}BC&?c~iHih&VgKMv-Fp5ypOq&BWtKfL zn{xAO7I#gd<={gx^-&@~1DSyP+b4YEQH4GXRspx?lxP5_nkKlH^dtZb3yTF{CYYv- z8sfrJ2Y&&bjrl^)6)XDp=z)A2BL(KqnJZq8M$-KGb8}dx1Gj=x-$z9y!@JyECKs89 ztg<1z26lT8t&3B)QFWiRc1AJ9S>gsxF6z%h8g9|rwaL48KNykOqTX^Ve2PZswr$E| z*DRne!&L(ZB_gfc@^yb2b6ekKxBmBRIretXfK}F<3uot7s}@R%w5fp1P$qEyh_v9w zz}}d#VA|eR+Q^arzzKmVr&U?R>g#ZYVM>%ewMW1H7@t}vxIz#s+FvQv+TTK!+%BRL z(MT}@Py=rP#Et%v65c%NuUt7pb@Aof^ck4VlP@VS&3xXrM*>GfJKbN`2irhO?$R#? z?ZS=@o|yOwlMY3)|1fBOzfxB=!>{u;grzKK0SF|Hw@kwYPKm%AK?lyDghl&bDcyP*7d5)>JAWVVA z0bd+1TmYU)fu}KeaM4i%4hk)mWWs(8nrg&W>}&o20RV7(zZb{w96eA%g5Zcn!0pb& zjn7Vf^r;l$5@kw1!b4G!thDqa+->osW^Wg|h(q@t+`ISl*Dt^oA`=MCYj99VcQa;0 zVT|c~Yyu%5BSXb*s7P6~NXMH8vux&-nX*ySBfjzH0VYKNd@*kjra_R;!ySO4JQVZ2 zlBXRX2V;NP{P{gqRjClL0UEhH4Dr$8&la2sk{?qt*iq~G%|z3GbfF_OqH2Rx?RG5v zN^C4EfBnco+&~HmrfADYZ$XQ|vrWT@K|^8T67Dw9ioS(jV{%s=v?Nrty?Mie;JvKf ztZT1Rg*HGiG_@mTSiU^P+WIwET~?MUPEs*NRo;e*p+LT;6b2D4q(fMoIk~1dFHgL6 zKED?-{llUnWG7@jIAw~V#A!Y(`)~Y5)(G3}IRqdmK%*2vKN=dml3#LLStqdAz-rv_ zcU%R^3T_Zk)3tpFxj*_OE>-*c&Wg6>mfP<)L&!(_^tW(GS^88PeHQ~aK&SfI72!mskxTXeoxm7Gk^0u1h@uRX(Z8bP;QvkM@rI$WV!yK6MdA zsq2&}lK7`gpKf+5M|U~FpNoi0JI_;U!!L|1yNCq%OA9KQRRlRKFxh8`hZl221UwH} zAOq=3PEMD0iHQBAx24?V3@sCrn0p^s9fsPFHh4U%%7_P3rw*HQbpdGuA7-YxgIH?e zw-X)>f-koJ>MAnrGLsDr3E|G&d-KK@2O6Av&ts{UtFF~A!fMU%q5#q#9$A>?nN3jaGPTjHt87~qv6D{;WsIU$^f!+AHTbxHo$NA z!B~O|LIK7%Z@%JanmUR>Xs%Od&NLV?Vif@$*@N)FU?0NB7 zH^v+~6qdi#L-F9=y~VFz-@;~}%Vk)?l^^h?w$)6Sdc#0rX&gG*|Mrh~g`;MB+aVRu zYA(d0Mduhz(6Ak_B4QLhc))`Itu!S$`QH8e*OHULeF%$nzkW?Bva+*7SKqrvi-^*M zsU52vu`lO=gNkK#5wrN|n>L}GVv$G6jT@GMmLlD5UbuYu0HS|bdY3PAun+CsE6jO@ z4P$d2LmbkhRsk(I+Yv`8%1}H)TBClH$)%&ht#jJB#~e6N;Eo}L6=4yR4+llKb6y Sue~gMsHv0eEUuWl2mN0!2Sf<~ literal 0 HcmV?d00001 From 24d48968de4c97b0cf6c808e27b8f837c9592a9e Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Fri, 7 Oct 2016 19:59:18 -0700 Subject: [PATCH 06/14] Delete Screenshot from 2016-10-07 19:57:34.png --- .../Screenshot from 2016-10-07 19:57:34.png | Bin 110846 -> 0 bytes 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 kubernetes/Screenshot from 2016-10-07 19:57:34.png diff --git a/kubernetes/Screenshot from 2016-10-07 19:57:34.png b/kubernetes/Screenshot from 2016-10-07 19:57:34.png deleted file mode 100644 index 0776549f4c7d6c5213af6209c974273e24bcc3e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 110846 zcma&Oc{mo_*Z!?ZB}s*ZkQ9kXl8}%jO-PeOM5&0RQW;VqNfIhaLIX|4N>M7QG$9%^ z7?Nl*L`9+ZbKTE79M5sQfBdfJc)oShHSN9ETIV{?bL}vz1+xc>$cP9D2@RfOVP+#F z)ccQ+P_M25z4?jelM{kJgq`QkHWT_M_`g#(&qec-fnFBNHwy_38ZP+%UP8xDNb|$~ zK6B>J>R%%=ptru<+T2$OIGzDSWf%#-7|+*SQ;^7m=3&uaFXjuz&9g<87r&KxTEi_qOs|NSvw+kby3zUw9UbC*~D|Nis; z_m_JO{`;MH(|>;$=l1#U&swqn{`mj><^O*1j$h&hTZ2PFLc0I{&aQMZ z|3@c=mom<6dz@=OG5)Epme#25d*f+Q49E_n)5(-tJmfE_y zx*8haYwqlsKYxCt{PaJyp$d9(&oWm(7&rg8&W?6Bz1cz?J-1(v61p{V%9JUQ3Nt== zUk@}&JAeNCj3c(QXV3QZ)C_S_S#V-sWTe6m4@XDG6)R-dcXWD}B+k%ORqarjn|1Eo zxhq#B3p0*tI)(%XJ4xTKYjs||dcuSWwOV~@^33)s3Dq=@kq}wI!_Kb5&nUvY{P}aG zVPWc0;g9RZ#Kg94+cwwIa{c=C$yX0uzC2MX{6%&3LgitSl9I78F&%yZuC8Y;UUc8Q zIXfq(y{1Oda*3jN%+aIK7Rr1hrPhBZik?1ws-dB=XU`rV{gsOsFE%wzzIs)vP>Y|5 z7V`Y%%i|6n6tg^?mX>z;@<9vb^A|2We)w>e^b&h}^)X`#*JNL}znq_+pOd2#UL+$e zedy@XrAwBK6CW`5hidhN=U!D3>M9z9|$hYi@vXOIe?b*}vFn>Q`K z`TO_pKXF1%EVQoGciy<{yu5fbCDvS9TC}zO@uNqlPoJ)M@nVwb1PzU`(Z$D)ANMu9 zb?eqiW5s#LwC>!!n|$rs(PEchKYvO}NIWbqPWmxkQ?vQ&*WusQ^+I|H4u6XQU-FvA zTk|BV0ewyV{coN-*YGHI!WbnbXJ_XpPoKW1sNi#j43?FWmVW)_&5gi+Jx7j6$wVx1 zbX=$ub=m&9(&*8(-@jivbEc@S^?rnGsL~$3?y+OXMj32%f0I-E_U($UBTrnkr+QY+ zQyw-)d02FDdu!{aZQF*3hzt`6p7F11a)?txLPFc8r+NN=bwixgr6#DV?lV@@(bctE zzTD&ejm5KOae#~N-ZfpZBJa|rOXtr&x_@6dv~UZ@*2ib>-o2u-y6%F*J#L=W(xsZ3 zn&+%kJ*)CtTUx|}4YoB``FC}+7KRQxWr`2S+qDH61*7FfriIx{j9GWWQg( zehK&O`?0=0?ZSmY!FSJ|9oOFZ^;LS|-Mh-l%37iu)~|n)vo1P3{Bc`D3E$#dQ_})P zaTbkJw`ccm+r^7D9dC6tmW`P*!;{^^=*&-XWeek_i@c=6)1XV2bV^US+`J*i7YS-Jh!uO*8Yd%V5+ z^zq|C2c3ijsjBPXHP31(Ny(c#ep;HEe&}p#=;-o1;PymrBH zopoMbhl-q8`~!DZRlIs-pszn}?AZAU`jUlOa$?*5w67aEGA=h)FUjiIi4%SwA4c8W z`TJRd!M2CuaU(PxY?myl%G(xco>Fx0Ue494KYa5YAIk&%q&^k(u*L`1}g+GmfCYHB*S-1Jiq~!X@uRD|`|sbs)^bu(QUpTp-n|!`G#p5x7o`4x#LJgAE?s(*!1z!Uer|sJ zC(e&iqTZ$t8BT>-qS=|5WzV1Q4-Yr+s@b(?&&&-QHtg83W8=n&kq`Lm%t0l`Ce0)f zzW>QVy6Rr}fd|ijP&YwMZR5M#JGEX~qVWdXE=NQ>y}U9{*Zp;dSFP#NrAze_7AQpp zXm}EQtlz2f&-{gxdj0x!jv*&}$&w{3t#rt`4IAR)<4K_unB~itw|?dTDG%t|JIqWe zy13w2ZbHKQD;r+j$jf8X4l2wraxwO?v3dSFbM+`WIW5u6Ten`#$+@ScP7cb-dj8_v-)7iLIKSYn^I^-^+j??Nr%Jx&`o%uOlN(b16>HQ%=Fclo)6oqj#1=R^pNlc?|z|MvVBt1_4HAj2Mg_bIZCK!#8_qJn3x#3={s^RT=-fPKEmyrScuceks}S| zheTwIpD~0=^YP)F>2M4e1`mV{S2>Z8aP#I(Ss9tF-`{Lu7e-EAv(Hp&%$O9x3vby{_Sr`_e&w=d zdQNxy`uqD23v2J`-l@~~9LH&}OVgJxf4WpCET4V;@V(3KHdo!OYwMi(;n=|8oAy865Y3#zVqyaf-)%-8NK7MfD#Mv5q`YXz!E|4_bu z;J`dDFE3wTUk8UOzCdhjY<+#b%EFUfwS}Slx8rqh*T+XR4j(@3<}@yPd(*pVSZj^QUq3?Cg`Os$fgy3vnRyJpQNc7Oi7dG}=Z8&01-P43z8 zDH|xOe*XUCWi8S2va;n@-IrTgmBvn3y0QA!O}8>7Sy|adi^?6lwvYofG%nTD)U>y^ zgO?~#K3ld79{4I`h=Rk?rRTsW`)}U3as1Az12R)A{YI+AzNxATsJlP-*!E#UZttlW zCR7zCC#P4JSMF@73ChgeWNt3}B{$B<|Hjp;zd&{B8X8o27Z;Z$7nhY)R$gCQB2T5Q ze|+@omoHbZTp2oGuTxuwbgLE-=SOzq{(@PBOj1jDVbL*G6=|qkMrBWjK_Q zl2g1tKD>R~nAh0e_GvLVH=t|ToH?HX6MXz5F)=P$(v;{!(b1c?ZCko%QTV=n(vzI~ zhB#5fl2vZ7i)D3{eS4dL{gy0|@=r}o-RE?SI&pS>{K*-CM&M6t)iIiy>FMbpK27^v zFXEEz0TnqqRx1Q{SsNa)S+wXc)r0lq>+!s^Nl8hmsTPXjR1!&2yQJh;@c~EFmuCfZ z?@)_DXD?qi@_3^| zGLsV{Xumc#re3%ZZg-5}EHz78uwX&)a`|oNN zZrWhLlP+|3cVBa_?47L^sL<1YZRzZ6wd(xsZPkx%_jLaqBqGxE?s}h41tFoX$)_fN zO?+8ZB`+>cu!a^Wj}Hv|2Si9;*-8qyu;kKRt&5~a%3a1H`Vg9u9!o=c%^5Qk1~2J| z&b8mjDORx9Z>E%Fl0dPb&h1Y4F=UXa=-#vRKOHJ}UMFZiudL+I-CmP@=JaWfZcJ<} zN0TJZUpRgP`|f7lfI&b`@9V_Eym8SF>c*?7QAI(+;DRGZkJfqR{`$3*HX!D#m8whT zvf$w0moHx?5AD;~cJ}Pq4U4;`X#uxhl`;>E76u7otj&M2Tu^>|Li(=`-` z-@kv8896)DbgCP$736sTzWMm^X;EWlzkKoH#mkpAsuEODnTR!RZhC85(IoOh(g_nI>sXn6Tl_?tT-NTvSz4v%9(DN<_rqxOom4t9n^xU%9e{#Pt06 zbJFF9f)}Nw95<&VD^;?ty}kRYRpP^j9p)sH4Ri(>*|pvd>eDzf5lC5l@1B{e#1dQE zj?T_$1_m<%dkodo9#vI2%$imI^Xn^pef?p>hS3b9I~H8Mc5TRj0lu_i02&$viUH-# z($X^Iqo6h2<<$`QEPR9}UEgGnH#vM~LLw-rmr|64@-P7k`Sj_2trx#-`%pA!`t+Nm z30`OCjvXJoaw#w>qFrK1hy73AdP#Km_x7?i$$Pc7c4v-&N~CM}uHlqn-c%`Ge* z0v*ft`{?_>%iTnoA?mrtMG zTUgsvb(6ZeLPB!;&Ycs+kKevy$AV*8P9N@t%{@A?`+NR&U*A*xW9ce@=Jtyg?QuF5 z7B+Cl-eo2xCUfQ#-@Yx>{r=S}mE%wQg%;Mf`pSv@UL5^6=-(-h-^-UvIFMy0^aF(w zOEkxiw_dtb;m7(`aK^ak;##lL$8Q6_UmBVS-YPFGJroo3(fj(maf@hv+s9cGUZHnh zG=$3;`T0^64dt5!wf^O$$B!S+tlgbz-8ZywVBg+JR=(h10D8V(=i#So1lZtY+idG7 zRg=}L>&Tp8%OfYzPz|F zL{2Ybo>8Q`*$3s8({S7k9z?Bw zd1=Mb_;`a&AIvE3OO~uzwd!_V>+|yRhyw@ogLq3H?a z^}H0vk6-sFW;{8`OsVy0{M5?I${F45B*BPIXOA|_U!|`Aa`ExiemWX=vcI2dVZi4Pr`bncvrhDLYckQq8t zr>6Xp9P!^{-MY^}TNCXr(IMY$_3Ax^wHvGQ$7^Ve8ak9>0VIkr zg217}n-w%5Zr;`G?3pvSo;YzrN_|<&$Kv6w)!LIL4IDIRjQ9W=bLC-SR69Ys(bg6b z6Z_QayJ^!VH*bFuj&_{8r)MDD?!$)<69aoHtE;OE3;PsmamM*(F~^S_dAWDs@X~5E zH8qF?UJf3mapvaDn-vri`M7rn?dg%6HfFMh`fN%5i491QvdinBKW%gRi0k}NZ z=ks$=!0Oej+03T*d1Y>w4jd4jJUJVhKx5?%8n5;j|HjHF-LLm1pZUR*05;jN$~$+Q zo}ZrM;NY14YjD zA3vV5ZUg3kyaQQ|*(_3Aa#NLhS9w93L?ZEeHg$t{z9V<5G5JaPkDLA9UPF2eoY*iP@))F25(=W;X>B;FU{AvY_LC;CU z?IgVDvTABD59yHkC+MB;g-M5$x#f)R>>qduY<>Q`nrLCF?fL!tEoaT@_ea1?H0FA5 z*|K=)(nOtg&pA%--@gZL%#M)x3HG^j$3%Ef=7v`jH*aohYT7t!)+~`=o8`;bxw;nB zd$Xm32M_l5^Sg25hVt040QzDV?e+Hgn;QoP#EJca%?TMpPK8aVzQis-d{PByme_xVY2JPP1AaHH zq|uRy2swKVknQd5?dCH`K63V`Q$U60>H^2Sts0?)Wv^Z}fBW{at&=Ym6cj|>Gu-^? z!2bP@-dx!bI#^a$`Su^O9QjpC6i7@v3~;kmTkPNB0@EC)J44FZ`BZrjQ_HL}4P8Z7>0m(;&7Pk5t>c)Gx zy3$uDTPEIF_3hpD?Ck7CUDryzbVLhHO-<=z#N-Vdo0=|Pya?S0wFKj-B^n(Qlarar zF4eU95)t;~YRttVJg%pW0B;ykz$k&hy~T?qZfc zo#0-Oq_kp_dj35xD>JE9Q&oL(Y*OK!JH6%yuUfY*E;g2Aymay6SSrw=Ls|q!M@I)+ znH-l)qkQJfnMtCYX=>^QPtWwUG;3R1da2$)ih%wtZ(-4Ey+|x!H`)N`iH6&$CH;1e z6ly8?k3~pv=cudIcg9o3{ zbP(f5G#vwo(Uo~y;WcC;uDTQtlQ`%%ji7^#nLYbKON$q67l(!21F9Ix7kJk{niVbu zjeYjaneUa^{R_2}qNL^ISfoR-u~&BfZUP;rB`2HCof~d>hH86+uEW3}#pUf=sL$$K z8{g~u`*%Zh^RgWszB)6<(G0!!nz=rp;KGd?Z>p;UX~O?M`D2fRhwv<->fe}ClLrZU zfd3Pb#_!S$z2;{e+oW%Tr?mKo80g4&RHRxrdhD)gE~U)Bey!*JTB~=E(3#e$QL(YX zrkw46qa3F{RqV(ctD*vTmRg<(7Xh9uqf>Xld|3pms-%8W}iI^lv8G^_EC{6$S z`9l9&Y5L#4+@*<>((#HzRj|FyyuF&93e@@i%@j0{_XcK4o5;sxDXS&_^Xk>g^z0u$ zx&#?3_BdHu?r-#5w{9KOm8vSguZawLb}=v!N+meB$WvVlP6vSe->-!3CU_pBXZq7e zMcCg|DeC^gQG;gnWp;Az~H<8vUj%l za@PRh9a`pA|}< zOXJ>Hzy~7Ct<(iS-)m4kgcsTV!=)?`j?}D~{PsUT9%x`-AQHU0u$K1T{D5??eggfI zcen}3Z=xCF$;ENAM?YfV}^whZMJG#=#8vo_~C8G%{#l_NJ4SY(H|-fyyIxW zy3-n>LP4fj)-{W?wffGPGe;FkvO^*0@0;7f#V*4A`T@60DuX;nBrKqb3iTEo$)S$# zZxlAP{r&R?6jr1VCtM>-JIbR^J?|vm)FrlP(V{V9#;6*#Xg~|@5vZI!=WbiM{?DiD zb35>Gqlc{EKlU0Fq+hr&K~YiBz~FE7Q}b1;YDl^TF2yuM6oP_)?mrivBl_H=bSg-% zG1AlPfVn69X%ki2LJ$GjRR1zo1+;-%QXJTe5g_=4jaGW}H6l0vb&;@|Vfl zpO^O+XR-zBlrD#k`fugLe!Y}9J}{CS{=2=MQ&);ED7&`|R4 z!K3o0Pdz?AKl3kb<&(PBs~g|{0eA#+l>^ z2~Ztf{10D4)`(3X7FO~4^|7Nz&z(JsL^s3xqlJEggPq+Y7wz#9oOMeoNL*YTs+sCg zvCu_Jmb`rZ8t4P_!zx1uefj!zx%$hRnpG=S#1pyvQ=q6==<4iBXiW;Hub~4iDeqvR z%-(}Ve*Ew@N=!e0e!a7^(t0fmOUp0K%}|$9wY3i)IwUR5HLe7+ROdI zQ6ng3bQT=%>*qZFA1^=_;Rwc}?^abkWs}A;?I(^n1KZc>cgyW99S58mp$C{nCVlAQ zO}(JYF)Ky|Jbmum*RNk?rKM}U^vW7gdBN$1@(X!%ozhfOJD;BZpsp3P4T6^rIdkq@ zxf_JaMBR7|O-;{@8;x8Jeu7`&WTMf+HLY1=ey*GrA0pK3)s_5w!-+>Xcz9@>?B(+o zPIZa3by-5F$@&)Ol81?37&zK7R6Ko}HacVFs)gNDw(U$qKd+ z_+s^Jj-CDTf9Su8B`QI&dq~!wNR_Chx z?T%Sq3l==1J18tHq+>S=HRG%}JAW8)w&nfTrlwnP12blrxb57z)5Ya%K) z*}P!uCE5^R_`kNDPVaAQp$`o-O0Z5b2l@f-?%uu2fe%GYnR|gIoVKCohdbCb&KSvW zZ~FN&ppKLa6FN$Q4Z_00c>h<=pC68idGhGdkCv8YSJp2!H@|;=@rAPs6Bj5C;|NZk zJh|lL^b6LlbxYpLzf`^mM=n6Z3)X$wcvyk16fA&5J((w)$RdXg20 zc~aBT;9TS*1!+P{tHtBQI?fi#;MsU`V%l*3#Vb}IxOLDEBiM@I6CALsgAu!xp?REj3n(V#C-Sc>d%wXM)9!0WXin z#tsd;x2f*_^&2-R5(zT`9TdgUyu9^JkWF|6aIWBoPz5ookd)oszrp1|Wzmb^X@K~M zl$-Ya%^R8F!^enR*=|@xKZ=q`^{M;*J)*Zj06ly9v{jldo7?{U8nBo*<#iUNRrDJq zb;?wF*mlQ)fSYI$OO|Y{y&F8n`~azu^0cF+#$>~W3nx!j{iuITHMI8IN>Cp^F0*07 zS2#ztShe+E!Vezg1(9j(vs{WBJRQeRoOp2m{`KqDP14r3%W?_J&-Wvc(eSr^dkx0} z=dkJ1lQ=dES`Q-H%-mcTx>j0RedP^X($$o32c+Hne0$Ou*!%YFn&)Tcf(a|LT%|+E zPw0(4kd{N_4aud@UXaCmv$HpE+SCetV`Y;(|KXD-`4Czt;&cm#$IfM~Lu?1o6=LOq zenwWBK7IN+9Xo`G;6WqSl$`8t|GX&4N^kS0C*v2Lr4O@?_OPetduELQby4eq#t_kC!J2%W) ziHIsNIfajy;oGu`2Yn0g6-EMxjPzW;UVle>BPqsA@-PL>U|Z7xtW8kyG}nVhOcd}! zJzdjcEm!7xCDDhSKacexW10H$<(1Pme|r7)?eGD6`ACq9^iCX|rzfUurgv+!-SP9w z;`Qs*#6nkOthxj7((3C+dEB;bYW&lx+)XKPKL8UP6MOa)!Z@HvTm1nRG&MG6WMs6o zw|8B6bT}(ZTYn~HO-odE@)}XKXo`=Is{D+A{N&`j2o7wBTLWDZNjJRMg~H6@CYM{; z6Xfg!1{4}uJR9l(m7W_mfHvWaAJkF8@t5VKr+eahp{7AwT)y1I83yl@+OP;gA8q#X z+7}Vw|MQDvm>ICKt*vd3+ox^an-iA1>k@OXpFB~AbAoqdweo#F58ENXOi|o-#|}Ol zYWR8>L5#2sMHhgk|{&Nb>XZOOvy;4y4i@`Ve3>AYY* z(3B1ryX2gXj4a_C#2-I?Z^CBMBs*S-(c$Fuo!`EFD~(f>5CLiFZhY7H^{WB=)67t@ z!J?uc8yX@<8Q{0$;nn(~DEW^GX0gOXN0M-K^l*{aBu5Bm3~Qci){LtX2+WHXFWv&S zBReAA4)R~Na-}~+?T8Ua>C{V0OUWq-dYgK??kQw`H0rm*8TeR`!ISnsej>& zZV72=Uu1vc1fqrF0urh!2vk^HkMTrmqP^bfbt+0qA8314T-zvFAk*{Vl0P*NtpZ;Y zTEQF|9Ve$+_FNYoh{C|z0?r}7Q-TrEVcVy=zox2Hl$O%vLxJAF6mW}%n{%f`w1*EZpv#n^a#ue%*xv3VT1Y+=88iq#3_Ih9 zcIWBoXlM7Ou5OW|qvoxRh}uDY`g0s1*k~=Fh~B<@iIMN!`}frafp~1vQ&V%%k}HaT z5N2a$A4Il>e<7zYTE2Wcb+K{x#P!qR35?h>3L^|1pTDK6b2DoOTHsXE<`7Fc%0b2% zu;`@BO`A-no@f2A3W4!l)~?OV%iDb2zM{k%oeGPAQDSpb)5#-8#+~d0A1uDGq`RxF z1P8?0%w^eYO9+Dn^XLEk@dI}y5EMtju_+st;8u)(>Og%zBgk=!RVW!FpNMyGBikoE%@2mYO?JsQ9WE`fzP&Wp6_xqFJFFk`}V1v93RpT ztquQf(xgdnv+lZi=gzGqK>-@c`xNt^?d=y)vFFd1bUk(E%vd$G^$CkJe?JSHYGa!CBsi4!Jl-m;~r)(aL7jg2Kx(3Wn3z9KDw#=-bHIzHqGAU243 z1jMsf?Om`N^66BMH!*Q>l;b`Yq(30_;GD5n*AK1%ZJ}myQT= zn$gu#2zn!XQxxEr6OSI9dfLF9GyS{iz3e1sGe7`rebLSrK(s`3&F$6#NJJT(I+gmK z{0Hz}3l|wpBR)i7qr3aXvuAZdIIW)_V|)^r!MZxz=)*u(_}pgAo}HJQ>+_$cl?ueo z%JK%^$%(Bcw}Rebp~2MBA((Uo1R7RS8yjyD7tBNINiR?yJS;m2@=TfgjzmG-fV>5c zT%|BU2@{pys|)_2PD+QcV-xLicoXu$H_8k-t!wq7Pbcv~BqfEe*|3vQ@8N=R&CICmeKo4Z?u8 zTxMgVG-k|9Z_unU<>8sn)K_}D<3@)iMXASljGObG3kb_fEe^ZBRg%AhWM1K`>wh= zF)^|E%a{GV`>Dn1Q?V)K1k1$bS;~YDS(n;MmKfTsU8qHuz7}Bzs{mUmv3O!&8N@jD zOVkA#j1wp9{rvKG@9rJqq$ek~G9M{BB_+k$)rTcX-5va(PSB#c+RmRJf!Vh58)6ea z2YRI(YwXd*#l%>P98~RFW9!EHJIkoihjH z3SQs-{rk7d4ht{ZK`r9@Jphz8w1t^{AQi89dfXKf`Peam->NKg|FbDr_SUXF2PRi~ z)!68!9rvxVQE&DA1AJ-S_)!W93Zq6v{24*OP%Dm~IC0>hgJ>aW2;GpPz^Zrn^?*ia z1a#qW!S~&`Z{GmXLSP4RO72WeNonJF;ejStz`H(=o4i z*eNSjehy!QWi-r6Rej<_vLKt#*4ipCOf6pgqG9JDEG`?r4jp6ewtoF(G7D}! zV@3v>W@cuNj1yHCMAxzf)5k%r?h1ktwxeIyevkC=l8;fqin-VSjnj` z0LG&3Qz8Jz$8dz=H_hEtCpvgAIsNs!cWl7-`uZynPDY6{fci8R6)#_&PffMI?p;sE z&l=fJTxxE9_U6q|gV(%f-SIzIsped{LYF-1dBhcn05G>`A#4CiIt$T}2%`N#keXv* z@g%{3cpN)wsZ++Z@H~^dMgv({ z-@pJLdYIX;ZYf+!h3~f;I=~^OCa>PGVM5g0 z57^YKwf`5hnMW+&cJK}xb zj-Th<`l!r1*3x}*!pQ~aQkD#71$j7pFR!RT3#&|9bk3@NCywl7amRb4EbKs6a^2Rg z<5Y4WwiO>9i@ncTJI;P0_F!*Pq^dvotW{fKcd-k+O3LaXFJ`i*Ex}_{1zxX%6W9l( zC@iph0;U@Y9`aLKp%SnXq@F*YyqKn)r(0*wh_b-X&@pnb!_=4{=nixsb; z!Ym$hYq)MGUEnfxC?6tvzgRQb%R09l*?HMAi*w~A2?i?qBdp%ve)kTink-pD?;&28!3K-1J0WdT zfgR}7DWR3EMER_szn;G}=UP~tx{JDp#5C$j>aV*EZRh62-K}d~a6+%_&6`KHUU=TA z1pxT6T{Ck29MM>zC7Kubk4c%Cn{>rOqhn*cyHq6?zlNQG5T#?po?v#aysS?KTS!-E zvEuUx?I}}cZIf%KHQfWsLPs<-l#5*t$Z~d0_RR(5M}}N*mWI&qY5O$4VwCzxck%{{ znwdFCM)J0ECfL8yUH47joE4bNgH1a_PCoB&6x~MUaMQ7~x0jP?S!`#AJ1>nvm>mw{ z>^Q6$JTZ)_QDUUskD}p-y#m5KX=W%z13EhwbD325*e@TQvnzo%)I@cu$7N-cpvqEH z8!6}re`GTvg%Dul7A=~LcMqVQT2MnZDd^~!8aEHa;9@JQ*BMTIqAc7uZoC!HeS?tb zP>ct-JWHHW&2#|GD*g^P_x)C?^A;{l)OHmKaiY7T;Q>LW_J8#DZM~Cs?zMef>WzwWg4bKu7w8_blxM^$)kqtaWDF7izhfuGXxJb(W zK;TmB6^Dg!4EXTLLuc?lP;7h)3J!cUrv=jT1oBjr1@bHNEMh~3yecj2$j@634#s4H zu`)GfaKAaUIHpO^eW0}cU(!JKWO`CovCGyiTO#DA-$+TBIe)&#gM*_Qe=r(0rb?qj z7CRvP6>Wm`6qd0WEP;FXuUo~?$nRGB4iwf}6FbEH~?voKK%A= zHO5@PH$~X;X&7FuFERDshvBMZJid3#8Vid7`XlaLmiErpXpeiaAz%PI(U3F zswFs`lE+$)G4t`s>1d|Pg1Exg{%sT67E^IZVbmn&y@wA!2Rv|;pmzXnDapwb9d8}b z-g7W$v@GJ(K1=z~!og_={5u`QR~k!;3g0;S1S<-e+AlF#r%a$vC(rI5a4i*Y;dh7v z7x530`PI+l*T+b9Pd}Z|!C9kw2Rnk7aDp^$n%w$rV6drFc<>kZnTP*$ZJY6rVnPRy zJRgaIir(`*pwp3bd~d_6#v$mYYO(u8wz8BTrVeV&o+IJBf5@@nn&Lox?^FPp`J7}G zYctoMdBLJR^_d3r2jHCPUNyI&1)%UJ(NBPAV5oLHEc;wnw;3A@ustL+RNxf({suOi z!bw0CKYlz!RFwV`Jwy(q27XMwFcU7FVVC5T6i${P&(58Pm7KGdos|Xg-O|(j7fw+> zq3PQ^~b6kHC)Rd)FB z&3}J+h!%>C(udpG{h*F2=7SgN!S)1my~Eu znk1NU0~P$u5Bo3zy8u&mEXQD;eBj_gha5L+l$||yR%uDYY0?eT%U1s$kH1Wpm)1`( zqI02D;86Ql3HLx|+Rot+qWF}Sm(QI4HB@ZWab1t&WGUcgL)&IQdAlT`8)U}pb zOG-aZ8{`)DQJ9S-|G}3~r)4z%GF7LcNU+V+wOajwYKD9oo^FEpK_^Y~L&K0W4?1lB zUwE++7i~B)h(LN|e{6Nm&d*Cr745X~GttGpduM21ARFe?C2a5q zlWk*?tSpkO(C|XK+<0Y19^BpC(a~$`>$fnsP*FJX(RoCkiz!$Qhl<38FGJoP zMkQd48LZNxtTUtcpBX}rSp{N2MtIAr81&@?B z>5f=G>`7jmqx*Z4Ug;qr=Fp({k0WRF>=XWVAvIOcz#v>k7=sRIm)46BM3^uud;0@w z(^KokLknk_nY}9rMBwuSd0|uA{`Z%LT^R@a(W5WWg~_vp2@;eQIa%4iwT1oZGy%gp z+B;BTT(moxt|0)K7_iIr8h!Nf70i)BZVxiT(#xs6R0Od<9pOACvRzkdZ|4A4y|t1yf= z{r_&Ib!N&DkE+@4Vb=1nv9M^JoON=rgv%+}nLa-4fMMRu&)*;E$!fxHrg=zyj4w}= z5W&cazVGbQ+1B1}yJ*qedGp%a+OXP=3^KM_vV`_hCd@1=BLhNp9iC&LAxyyTJ$qA~ab`LefYqwEWepbLF4ia_=(+LUV#KnG>K$)d(?`H}r)l$JRs}a_ZD6supHs zQ3WGN28Ev+<*3;pd|+>KnHv=~9spHSb1E%u{Q7Xyo4UzH^ulKsB%Buv`PLB-3L{5$ z;zXAdy9R~)R~2;V8ue!Cu?bCuwab& zkj4d~22+i9Rl$Al7l1bn)X6l=e>(LT%XCS?q_1{HLO^?nV!Pb=0JlWgl z<2P`8ED-HOLheJbGu6|&q3k?bE*nH9eOT|!8AIc~S2#*VHqs9Y-=uo)wOz0ey`bLkRNo-;&+dAP!Lq1fyEK5r_dpMTp^bn71m3 zSJ2c*S=k1LCR%;T2@r{()kK4B2)NtXBU1SkEv?slJdg>P8{(Bx0e1vVX1h53GS>_I z&%FSP7njM+z$FHLhs8NDdF??wx}@Hv7nhADqQ4AIvPr}6!v;lTGZ^c-e*N*p#K@3& z&@yCwXbNm8xv8mce7_HS<{`3B0L3LF_R70*YL{L2K6CSCcSF>JhUrPXe}BX-hRQT2 zYN6p0LhXA^)~;3AQDd9GWj1OW%p{pP>+02w>(|rm0V;vQAHw_`Dtz=uNmiu)bT^+h zixJe0KY#W|_UiDUq~D~wK^!4?SRmpWAP)UR^T;k&{+9aSD^Q6bEX1f4@1prci%X#KbDH*tI zSK8YfC3^fPqx^2hIm)#37t~4A@6SFrI#O~!%s8KtvKj3PNR9|&qSKal>&EqfBPxN zpFe*_#s2nq>F00Xy2ym=1X}5xJ^hOkUv@%l3m)${jgFl^G4M*n?>Eed!hG#x3_=%^a(JYX+mDm7G8AO-<^f;X}~&lnRNBmS<01trDBsHw2QY#!e_TCX7u=4~msL3U!49(8Lm{l}80i7?Vd3l7 zyVv?PO`ku{v9x4<7dsI(mzXxoc8CBxIXVVf^mpC1Z5tjLt{5OkL0nT0!OJw&(b0=Y z+yE@N@DPbB(19EP-UB5D&7f@8Z~UObdnT=~P=&r6F+yeR*eNVvS<5fd7B59e(KTmO zegoH00)_XCesn%2#`eE9ad?C|9tOz&#Wpr<9CtRLUSCX0i^3M1kpb$$v;rE7o3vW^ zhC%+0pN|+|T)40qs*kZKdP2CbIQ=bZGAewd7q4DTUGqd*PHd@zg8;jUg))vPFj(0Z z`t@{O&(H5_$W}geCMx_*&e+gsjy~S`>uv(2AtlZs&GrlN8PHMNzhaSC?X&7pO*}s-BpMNV zbjj_RxowVDX((7k?4P+OdI{Z5AKmjx+r`uKU;j@ReP9_jZ~m+K(Q(0orUvFFUS$jD zt@M?%=)cE~?YAJz^C0SjOb_nOn)rc?=Q_^vyXB6luwa10P{FM+|Bp-cmTPI;&*quw zzo>t5($d^^LPJmD)swrNmnVC3^K3 z8q>l9joKVnI6L0zx|XLY)G=iD%Y?v@j8qL7Qhn89g_+qsFs z51liq#7as^oYq@v^@xbTp6>e+i)-S4fuUMjS_-`WCI9X@b#yj5$ER20Hg$TMYBrdb zfB!djRwgx-XH>7uN=c~$BRZb+X1s$c(yUta&-uQ-dZC4#@%x+TpA#20|3!FI6U=7Fj^rP5 z)yBvZ=WAy4zA>#wTTZNjhLRr7CA*S1!2Iy4yqqAXl|%`aNi4aRojqvqV9CTjm%v5( z`gkE;H1>0l4k>>4(7h^u+Vtrw^M~e9eV-nm!mH?LYySy+ZBUYN_Q?0zW}W92y^}&G z-W+c+d}!ZH1zC|_A##~cefmk5%o0DnG*fQpb+unrvu5p=@0+POe@CctjIrsPzfL_R z0Y8L)#_VjGUjM9l&yc2XV74dUDmT2e%YI3j0Y}10nTV(nN6g(Z5d{?s56iq^|NfyY zb^yAPwziF;_(i)b^s9@q#;1=C{4n5iVvtDT+H;2=DUR-Av0{RR zNU;k|rS`*dkL$fnyp{A5#B4(F5FM16A`)5%Ns5DImsOFXtQ*d2)n#AE$O{(fj-Y!702*LSz=(6!~CJPP)79PB*uGrb*mepdB~wj14;9 z9fjKm9@n)R=if;`Qp$u=dHEQr@c%AcVV;N50|A(s8CB$5Z0CunhxnCP@?ez~MC0Dy z^LDCp(T9)y9Ozz zU;qB-?$hZIrcLWiJUXMx(-I;0N!(-tgyEZfw2~-`n5d{;+Mc_#TEM)B&6PKIa-G7I zSj+Ft&5l>rzaW)UeHoKAY2xmkTv`VXH>V#t-iTKXRS2-4U1D%h{3G!T0fUsLs;LR!Ms0PTK0v9_bH1ja&83| zV*Jt=7Dfh!Q=5D!`Xg@}uj}!nwYBo9M?BC6QG1d!cQZVre&d9D!#FM4thKdCl9icK z83fG#$X+MtZ=OHjt;p2^x9LWXOkBAG*vmFhFnGy5jn@Y3=8_~5A?(c|V?{ia>QeE1 z)C(76B_$UxU!Inpt|>l1;8*eY$E-piQ!c4%lw*!9moVA?rg(mTpLg_eJw)Whqj^a4 zkV#~DYP1>GT4CNyEsf3gaif2u8Ee6)~^ds>K%8zN3vs{Hv(YF*H13 zrqpYsj*`+{?(eyYtZA_ z8lW{wUcRZOW|zod001UXEWO4t^aby-Tn7ai>8@D?EQPKP9CYUvTP^Ymh z10BE*FKuI;J>3X}-b~3_c^In3EOYaNT-89|_dl&--`-dl2bC6y|40;;$?lPcVPie9 zZuGKA+*p})2E#oihVl-QLpfo_S4YQ(JCw4P!lXu7^j%U_5W=Zphtij2-UY&b6jD=H zmk2Y%g$Tg|6h#q*k>cqz%jPLTIuE@n+h3U|igN)K2ogq)Py$9_4JA)aZjI?mXUcRBLnTr*XA9%ySj{T?BWB$Ti`;$yN~B4i5{PO zf*4X_1Q=Wk+ijEPliPO4R*gt{`tTuy$bH@u<{AMyJ~s9zgSc|~K8%b3$yVO*S;yAV zK+%V%j$5baoY_HtPh9LLVtF!FQTSPk1Q#EWTER2GS!@DBKAyzoOLyMP5U=iZ!6Z6^ zW|*L{j)VM{sRL@pb%Z@#v5 zQ#(n37;8CDM~*DPyAff&lP(4`7lA|5v`*&@8Pn>!yN{0#tttl?pp0FHTt+v_RXbco z6Lb9dm)crhZ}iS|mrNsN466pYoB3CJc4c*DYA=g06i+B!Oh-X8}%yxYgh))vdk zZS7$BZ%iXFJk%EZh;3xVd01sMiJ-q!ck1#W3+3IK3g2Q1#wp%$fi5R6F)>Qf+^vXU zO8-o2M^V5ww*}vuxC^{DC)>aK&lboPQ`6mCGPYZh8|MZN7~r;H!{36RB2syLf2L2E z1F<~w9D6)#fda#}@Wh~xX;>q8Wlk=pG+Io;3`&F8&#FwS3lcWtZ4uKyei-gjr=-J zaYYJan90f9bA{iv<5CEs*ABeWOek}aNyf5oh%DGJe{mfX6cL;}aE)|dg1^$1YdJts zB&FrihcJ{8Fc^lQ;M&_tX5dxoT{*x+oGf-|2NMMQpRQ@^?81Qr#dK)OI>c(2jw2T$ zxjv79QCLH09+Lku3+x9OIW!#*l#>71uGJ<^-00yk%VRpRimlEqkl`VvD7TFp18Qyu zk*_b?T~TAK;;N>2Uwj}WAq(Zh7Rp{n@iXsE(ssVL4-N=i$W>nSoQK|SA#}N<3ovLr zWd&=*9ge4Gugq9w0&Biok$l6t8=r9f@0OGr7a+hPa}UrzqP1&wC4?tFj+0zGuv?Kt zHF?bw%v{_db>Z^mU%!7N!wD&ULLCa&bQ=ob^rcH4fNkSpKY#yT2)p3vIa?gvzT?8W z^NQlCOvI zP`E-4y-=|mYJ)kY>@%&*J!rP0!!yph$hJihPlmLvFmX?o}DkR|(g778- zk)nESw<%k3YI&$hH_X4WqVWo8iv`|inA3Np%_8;$CmSO$maC(frFsB*7|Ql*SZOY6c~$eHd9b^h>#(qx#Ka-)+u8Rk zSy}tB^?&^u{bKB9Jvl4&KN-f$`qpzR*Sr&YL#J1kmk;-iOzNI+!JfNVuhg>X+MQ1s6|WWwMusLmH>|EIs_MI z#Ef5r7R|LXyT;~_<8yPDO_XffFFAqRBcH#2KMDo#&MJILTwg&}HmUjsTq0xQ@`Xtopn!GnJU+U`UaGebJ$~;nj6#Q^u?Q8d6I7%T* z@p*DuQB%3?PB8w$kOgYRG~=tiaZRwmlol@^9}+!EPRGk~EFV(g7!Ux+8Oz)`_FmNq zq+*&Ca>T@=JF%~_va}fT-B)sR1#>MRO!+D6C4~1Z=a#~y*AAw{3e5KMRMIDmCCq?( z0vZ#n{d2~per!#+g=zhEtyg2J8`foY^}de`jad}{G`X210(7QY{`>M`0#+*sO94On z*(b_E;20-w1#Ix6$3?i4+qVe&2_Q>%^5jD2OpsJRwKMc~wkh1QO0rG0RXh6ldt>9F z2_k#$z|F$n^)@+#z)86mEwLLCBpSAIOrZ0f_ebx1RQax0KG9NkB;CtCw6H+3Dlt)-Kb;7U}KMJtRP|jy7b~Y zmri!Dl zctI2~$_xD@x3zLN9amGTL>D8i3$D~b{6va{53(E=O{GO`I-iz?oa|eBw;ztze*JLAeID@5iK+lh`TPnir<`3b*Z_}0xq3C9RrA3 zwhXUb=uvL?EFk0KEctJ~QE>g6vvU@=4pjCNn9RKH1uz{;%SZ2nU6kRt-uGV<9`O3N zV!fWCwBzp~Me)6aO{*}uvhTRrB}R^AxRNFKq!0b=yCJC zw{AUn-~b)WO=CsMv;&&SJBKnta9p(DGM=@G#kP+D6wl_Q*<>MOmY!oki*dRMMSGxNxD&Z3u9ht#bp3dwMIb@+sUS6e% zE9!PRH$+I89Ab3kqGQ3ydE?%`ehr7ur4l$J*3q+q%kho;UzDA9T+aRf_8VtIX-F!G zP9z~@M^R^2NJ3_rg(NN`GBVndWRxT!6%s3S%phIfJ!K8NQi7BY+@hxFJp(xYNpT|*`yGBh;FPcOJIiM@*LB`NDf{|{~Hrz-4 zE8~Zw=p7pOxtDbR@}d(54&}#@Ii4YOxDZQIr^;8~$(pGk67A8q=y7dNRedAzf>Ds}Ho>O62=lq4cK zIcx?5H>F>{u3x)mJ$`)Ng-MJ!TK)4625eYSQV|4Q^dVXx8P*D!GHFt<(8xTbBc{71 zBBbu@Pa5LJ-^%bkl|~Ecfy~9J`dLDv z4jeQ{RaqHM^(gbvc6JY`MY5)3om{bUrR%a~73m&;f8e{=g_#l-scK=A&YV#kp2P%b zge?z;LZ^b#k3j<8N=I5!MkZ&^9^d^!%>s>{q3xo>l^-9^LtIG+f<5CDyx1O^(OZzL zf$7tMFP_x4XvET1v?>niaoAkwMOvdl^qjrVxJhaB~t7InM*Pl%4@%ChtAAM^JkwycT$_Q*+wvX@*cR-7PT*I6|PLfvmR7-|HsKmLXo z!mUmA`tK>deq7(P zOwx9nXG&R?wTUAh+TO|AW2j*MN!I4w>$ItU9~Do+I(xQ*aK?oSM*A?Vc~n%_Rpahg zhCz`?{+_nB1mD~LbJ+f0NAtc$9mrf6r>mVlXJ;)J!AI zp;3F#cY2x!B3Ct{fkDMetl*mcQLAVs3J^3idVAg_mf>$)O#5@8?~$R)@==Vor><@T zdV^ewvv2%&Dxhq|wRKBTaOuxiNO8nW$GxR-UHTg?IjF;DymUL4B@kRtNBP{(r1s5! z_z;e`r`sy;|9=!j7>K{o#m)#DWRq zo2czFe&D5XpO8z2TAxc1iZSeQo~6Y)bLe{#Qj@-Uhc{Ha-(!Mo43H+0ICfc|Aoib~ zb$!KSLy?4VYI$h*ef>xLrCF8~p?5BNa6#(Bc^-+eNm%gvq4G^I{eI=L>_$A_<#+O#>6uw)i*iTt%ayX;@ z=+0c)Ais@ua6JYZd1@9dQr=`%^^hE9@n}Fb6-9K`b>F|wVB%O&M8UPSrIBlX3?vYj zMrfW@QQUtjvi^ey;{}VQt1d&i2)oy(6P&5ZI?6AlXBQV6#}sCOynB0(3$wx7s|TJ8 zOo`Qdf~I*gQQKobM)OxOc|#|zC=z}Xj4L3^WXKTdQG)j~PxblvxhKz`F9$_Ncn|PF z|8RfhR%EO#%|HL`6(&gf?4NKJATUXfRLQUt#Yz0d!RY5PkAOvvu#INW;?}LP{$k2U zD(&;<&QTM=(6@m}@d@TET7-ivFBoA}JkYr%s;itf#kN{`_rV4ZC+MfpmTNFdA)4uP}Tf#-})9jjSM&(B?3v zE=Y}%7D4xK=cw?d07^hOE+^O{cHGXtM>3C{&f+^exEc?l?a|Xv0;>k$X*xW15rjOH zg}_(x^O8~kB0%cdd%(Qnu(hwuAss~$<~4Aw09K#~1QGv3>z(4r*}qOXDUy&KsbtNN zKXSzAguq|?w+Z49Up^TNKaQhKCR+lz(oP-hZDAqpOHRiHN)v#FgjsWE6^?swGXAhhScGQ z;B}*Dr&%h98kUP)Ag()PRGtq9`6K!sde*W1FP5&;!3PE2cezk9|aNhoTj?o=Gp&XWOve9fTZGn}5#6f)`A zJU@xa0?=Y;;QF+w*Jz0$;TyNNWz0p~%*GUZN5{sa7q3$&fAc(((Fy34>h>A!(3^Tf zA8@4ex~>`;Rtb+M?$PN)e;ppqz`aWnS~9>o#&dY3}5&sbABhogW=tOpdZiT$i>DW+1tBt!TVH^HEB=^mQ zc~@)=G?=ENMSKTXz;i{X1R@N*fJ~1tKr~131c*R01t`@yB!?#yk(fzG#zh3=LTG=9 z-y+Z)#>bOj;^;`S2U!8?LWaQ28>X{|(93jhNX~1-LkyLh_eeT_9{J}(XA?YTx!n9L zhr`3))Yd{;)BdA_>$@!ptVaOKn!e{eecDA5%75RFw&!?af8F$Bxw3KyDkAFXSq~Sc zERkbQV&+WQ6aB+%kRP{l5$)ruVYx^R43vWQgS}7{fa!^4JUMj#*D#e&CvAY13wJjdVrT*YMs|ifI&6f;h=;nO zeuU)(Pc2<_)l$s7wH4#e%p=hL0#|^&xFqS0UVjQy96YK~1D{p>F&pkfQ-Izw@<;(< zr0MG7!k6Yv%`iF$;+!AhE4Jan;ibY=VYLQ;2CaHg%(Vqy8RzG-`|A9rn&wk-`RVfD z8Zpf0Iz#MWrw#vq$G(p z%fY0;N5Uqli`WeVez+UZDzjbv*8svc65ibF2hY03(uv#_Y^;bE%&lDn9$UQ{JBkj` zQ-A;X!4{FP1Z2=JMl$iSjIyI4W=x0$7IBv@QH#ZF{_?WV{3_aP9OM}*J8|liu-3^i z{4D44ubne?!JHXE98d1RnHtOr>zs%~hX~k{<4d5J_#l|t(EBI_zW(?TS1~>)KMopp zrpz#B?UUBbC>$NPf@>1zlbWs${U>IbPlN3Rz@EhGqamTq;(y zGSFCFaHI=%xoa!O0f6y=|1>qNGd)Ioi;i8|BLROYr1Pef1$BGPG0Aj48pbpbKQiAJ zPup3<@Ve@M_^0Rs^v7=1TtH7C5S~D3g5N-TItQ*(ur+$o>ZGTXopX)nK<|a;Vc^h? zxI>tpr}P4)*!T%|h_%QgVq$#AvvkiV95B$~%P|lS*E(wW@FKU%7;rEca)Nb4Wo3UT zU%)2_rP!i~?LYc1x_Is!Vf;+nJ90At2^qeXx-}EwNE`9&txTIgV&uq+97K{C#E5ME zWJ(?Sq~*(Vm!xnXp=Mz4pjV04X%tDtqT9;>Urvsc2^wg$&V*IVdg_ zsa;~AfmnqR9|XAn6n%DH$vHkYe8miSTz@gJDmShQcrTUumcb5@#CH`UAVG>o*TE%V zrwdILBfB@ZHx2FE_dPH*FLR>03s@)7|MI0vBr5VWBr@)@7Y{H_Hy^DG?}2bA7z|K& z@%PBmP)X8fvM3FaY-;Q2ZRgDLWubvL660{##l>m2na@LQj2sN%F%l9cIyh`begIxb zTBp<^0MIz_zg7DYGXj=fzi|VSgF5NKMdeN+xZ}4RWfl>E_eVxX5(^?92%cH|M^Pwm zd+xk>OHy92tpd$Ebtx{TLJXlI0>q8sKznLwHtf=fKhf zic+kO!rW3KspeR7fi;3fnq|m&!55LN#)eIJY5^IAvy)b^HBT4b14UG?urmT*nrtst z_ua-g&e>U@J)0Y9r(;J);r;QWZJhnxCG}J)#6;vq%h#;=T3szUS&W`;1+CY~CqHjg{YD1NW9{nR;ttb8S!$%eW*DBw?Mi3uq7mjaSh8?B%{`Rw+B z6L#J86X1SNpbyFZ##%HQdLcQy5d=yV%$nc2wGcXz!^{yQ#ljzOu6NRrqd=zE!2beA z0&K33VekC3w|$>czkYC5vf(500X&Y+LiV#PgSz2l z?VOr`Cwy;3tN0nzwCbYKK5aKwHIeSf8zc~F3M$<57ca^y1=CWJGao&g>g))G&gbIp zXE-(?V%| zqtXfyWV=i%f%Nr{XhyyP%Dj+3OMBWU*0zA1XLJtiLSYG!&>&oV$w})w<;VsK>UZy4 z|IT;@stV&-bEkALEk#XC!8drY+Cjt3Le{35!FoQH;k|$?B!c-Dg9EL4dTP~G;-{7%P^@B}yX3B-`=p5z6_Q*1+6ir)YRBL9 z&?b$OWa*@c%Yfn(j#8aXw%#EfP_-baV9LAb6S924qh@LXX{YtJLzgR3^oE*RTDX(2 zz%9(q5QtlUM4sPfB~;r#NKceFvEK)Ip6n!=&cc_+kHZJ}zH%-4g5b~GmNk^b>-U%7g< zin8*y`qy&7*4MZGT*+*8W~Q%oFG?^~llA(<6+j+b570N!5%9t8Ih!zk28?}hra zBOh8$D1fCyqxc&Y`35)Qv|ImSynV$@$#lwZx9PMTnF*{$dvJC_+uB!QqZK~}o-^It(+)edHi-7c#qfJ+bpB|17Ga%vS=60&M*s`_;`9t zbZ@+%`}d8EjUVD;b9HNm?1rV%p3@|wYdvrq0Ff79kBj|Wa%Hclj<2*Gy?EZFNoP}2 zzZb>*!yq){AuQGQ0woI{P5huiCYG#1MjMD><;z9w?mQgKOU~`B$ zb$-6Bd_XazELNspz6?xUd3*g^C6j!yBA!$c4EDvI{J=fDf5vf!#GAfPHbnT zg5a8l0D%CP#UsV+*3U%lb7S8<53seIE`Ioj2OE;5~cpI-5{{P*-5L&x3R{zDhhEH0bM} zfSc%!1`jR--Zj#s+H(O4AiHV@TZ7QgyS;t1zZlUYkCYu#9AaPrKIDcm{D|AH)!%vg z^d~GvoLiO@_MD2MuDI!A2g)EAcH+GAcpQt+mz(f zs(iQ(*HT^vs~4oaG1AUlqV1xO>R3RV!AUJ%1jcR_?Xj;oX2& zqn1C?WAQK*7JW_Z$cdLnF$5QE-F(iz?O%W>zN@7irpC-#$jHjlg%x$K-Ef_4nSg47 zzgKoO@~7{u_Z7LHqzF{6z`NKT=*kDI!|!iN3dKlf%T}b;blcpEZ<HS?_;LUwQ%d+kIVMngN~`tO>?pxK;q^IlttuNGmM|^u!4d(<6;w zjJ_HO0tiu3OFtvm67PW)!S+n2b}BGJhr5GWmIvZ_*$2{GF!IsT?$Skwuu=Q6tEIZc zpT9PY;-06QH1A42;geq!pCLKW2b6>++5y2w3*R#k&G<}xTk9V_H@^_RID4P6H{vkW z9uQVDS`BXf&%<|Nj16=2MQoccb)Y{kX>;Xchmm+bn#c zRQ63^2b;q6BH9F~bxsyTf>f|Yn!?wgu>ac+2D^b1==oT2!~97{`5%;8zzM}}!wGw% zA8h4fLAKj_WP>w%57EtKLIWBGC*m(3KdwZkmDbdG)h0Q~2;cNmjFag1`zlt{=+Wc& z;lqwIXJ#{ZiWAJM|Z+qCHyB{6k*1`Dk=e3-%D4Mzo~00aUM5w2hKp|NS* z#w7wQgR`H%e0dfPCF%sg1x9nQn?#8rr+R=J6FbfC^kV8b4kx$b;N$buXk4D|=AOxf z;gB4hRaWDab4~ZnAAFVRQP-d{hLxniuHdG>~30)8)4heB%hpn_hbg^<6vK2Zan`9RSpKJa4;x+NJB1x zaY?wq{>vu2k`kGRyC|V5D=k35&Yx%1F&UL+S1oPp{h=l%p==Q*7=s$nF2Y^RUAU0# zw=H}@0AG~#%ra8tT*57i5a7?6O!Bq#(nQkWwQHUTzu8_v0^&MB-SWS5H9%0n z;%wQVaS^~hvH&L*G@zxhPNngO*XGc6pWZ3wS86usHGVv)5EN22<1CXf0!jvl_vFcj zySq9$%ieL19~8Pua91@kse1QLUeqb}y=F+xnP&V%NuJUUQnLo^nqz6Kg>L7B>-~xb zR#*P#3lNQCS$+YzV-Nw8w@KR%Ga@)SC67{jew!3x06zlNgw1@;66sGPbV^fu{G<)w zjsVbMIxiJ2GcTT~w|v<$>CUSV1vBBy(i7`ml0p!PH}0(I8pCkh>`?zrQq6G;tToK# z4A$IL zCg$yTI4mGvpgaI4)ao@ZyHFc4ng-g0lQw2&;$$SHuZoKiHw4?p6S&|{N$^!x(zEJ$ zYnCgWI0Q5h1TkjOnCZB3Wkiqtn>{>G@W^$34N#4jSM=1r8o?-YDSKc%V1}4oXB^zd z0H8oNGK^LmRG6K>JRO{7xKzwoe)%$%-!wnC3y!0CJkH|5d-skQHjLnnDz~ZWeGDA} zBqAYO65yO7_~pa#)omb35ophB?vxKPh$|;dYafj~W=9C9^m_b9Rsi2%xr+-Wp#P9d z%$73nzK3n%8MR$m&NzuI8?|bw39i$T@EJ6LGJYl~AOYbp z)h^BFPO{KWTT_$aI0*Q)3^*9Gw-zRnFy%)VATZ)j6XDM}0$i*~V9COm-V;&^>oW)?H0Jy~ ztnGAk)X^D+{@2-wi39KwpiSaA<*CiIX+G>C;TX zt$s>{0ZV%_W5(GhI82{hG#4ZTtt;lGPo6Z8A1&HjJe-_j_!^+R$Cieor=$%K*uF3u zDk1827dJPXUSUv9McP@IMPsVe&-Dfl$ zu=W`%x-C$0CQGT5`OAW3B8V?2D(8*1priaHWC2C*sHzwU$rv}ai{|Fr5F}VZQ&P&b zeI&&ZH-uTSvAGNfS;e77d{|VZ?)LnxQuX?D0@}2xQx{#|(67h-4EJi8<^pCQ!4wgS z>*3=KX-K`#^d223^dbj?)62if%A-Wwj+_7H&1u)KZ>5!CMDm55=`kV=L$p=To&%P> zzS|-F%0}#Gao3u&c@p_t9cMi7BoIj97Adnoq0cVF0FdzgB{4{=+p1_V&aTz~_5ep6lQftl+ zt3{wrc7XB2QgMWMXq*!;btu=vD_Klel!eLyF+pCzznl?L6nP}B*o`iZda&i$m;kFd zEY)$;l9?>%QgUW@=4Qbo~F4O*RlcIM0ZOpea7E%Bk}Rl&*wJRJy&Y%b$`T-W0M=P2`@Su zZYhqfpK4=J!V9o;?opca)?w{R7Z-!X6B9S~cbiVL;k`-G__Y)(I$)lvp@Czt+~G96rpHXyHem(F-%-wj>dBIhe(#7qC?=lR&hUV3vANuwpr8lF> zsBOtJye9&^^R2+Cn|M=X)6CUPWCL!xbM*-Fd7sILd@WbPg(Bj2PE_5sdxBv&T`i3s zbtqO1d9IgW%Fj@N!?Mw969uSw)eEeQwc6{l)reSZ_+~%?_uODy;L0Bjo*U52MFx(f z{9$2TKXpZ3`aNf?5wJKbwrr)@f*PwfdYwj!tfY3 z&+`5?uRm993;-yzXWw=-o+FVg=FZddkZu_UM-7r}V<40N0{QW}bo@7-!VAj$`3;Gv zqosvqJvr%XWrmt&qm$Pmga@dFp^q5@#WEaRBsu$8em-Mh<&Pf^BavLas@3p>QE(<9 z5PX4LhwR>6^Q3|l&vKTR$g#OlTxBpD!D;Hob<)E=#-YE?1$yD<)LoTV#L~TdAjTBrlzDwmjN4d9;Dq*le0;G^n_&Lzs5NjIpdk_+d2ns zifn-803APW7X%-%qq3qVLVrkLwO-K>Do{TZilpSFOCMv+^6J%^zaC$@ggd1`^0>l+ zcFQUyNI7~`b*qzqR{=Pqks@?(%~;Ga;Ur@mww%DMEWhk(Km@+E2tE0V71G^=-8Z3% z;LCJe{0aF94AgI~7xg>A6AU(UQf`3PO(HIOw~;T#4x#7|v*7?+8-_EFtj9ox zBSDtQ@mRP80u(rwFqD^0p%R^vQoq(K1=f#W#hE(l_bYW7yySg&RrVHr&kztSJw3hH z#y?#O-mb0KCK&c|XaVBZo}KM8YDtPG%08Y9RrTKuJne{wG5pn7*+7uV`1{hxYevj4uI5l79J8c)Cd!PmSB9^j8^OQ(2P~ zVSR@R0>V4vXMYs0PM!b!%m_F?V36p0P{4$de=5(vuY6nIvt2p*XYT1iZ3+ffZ6UHV zcUXyiy4m7{Da!Ah=P9r5Q1IFqsQD$fRmgTTFh*3*fQ z+_$!+F${mhb*Hi}dwyR8ytKpLj@1RR1DF9Fluv1^%Qi6-ibaD{$CMvO*#}Eia3>=s z(qU7}j=;cHGr8S4uL&^tVYIv!ju;vMiTWva(?Q0hZt2qY$5%4}4XRUx#sDB{szcqQ z1@G3UBkZfR?cE10UuA|x%ZMhLQ~G9jL#Lab!VUqas129!D5EA27w3ZNqo(YbU?!;J z+i!e+dF9ie*N=GaK?+5GBEb0tD2a&3c|$7E+0ZUDqyyZA#58qn1X%5V1`5a+*s|Q1 zJI-u>F0re5UveL<5Sojcl1D8QCrz2MFx7=sBcsFf1nVxOfGkz@E>QUCzygk63_Dqv zOxj3&1xoSzWOLgIc~G&=6!#gP?C=d$+=HFoB51j$N$;MjGpFZ_^KQN$W zjWZ()H5s|&3U;JJejzY}xsR}ou#Lyn23-w>o9Qn?Pw(j8XcsMRk$~2Iat-gxT;s zk@U#o>*)DzBW6aOz}F!G)1zQYHpoUIg&V9mrvb!+Zv>XgS{X-U*P&6&m<41R&%fF+ zC>w}@77D&RA~KS(#QUyti0`=Slzw)GaPX8*>QX{BFxEoF#|PF^eO_8hmb838t4Eks z3M^r~Zpm5XC+sU+mtIaog`slp8_8=sr9cf`T~rJFbs9hpBQF9ZCm@S46uD{mMi-q% zO)6J9i5i2qXf|dHI9Q++I2gART!Ym+^QCLmMd_&g5bV&!K{Tn%h!~4N2t1tnkw5~g z&nE%Ofe#GIhE@si)rRq+aZE^15RKLEad%FyFxF}jrpceBdmNyP_&IpeF2_L(VQ?97 z5uB`S48S^tS;68LEC)g5VrjnFyocILdoIY=xs}C|fSY(1ovFs&hCt0RSGOlovyv;SY}Kz~6<2bB(p!^K0BqdGDZO_%UDw zKS=hVy%Eb-^wCrsp$fLI?F4Z!I~JDxQ7njIU+{hC8biuNf(ooN}}m zNj}6jsK!O^Flg8IX!Up-w5;NNX~nm%9}JK z1P@+x%pX+EWQr(uP?rX!`&+JFxx$J|nGdz3L+}vsAdE`NOG;9K?I6$W=P5U6`RN+z zNP{$*nN?1j>`fti#i==}Zm^#rPq)e?PsPS0F4x|4>6%&OiG*sk}FFoK4$ zE7!6zs->gHdnbRNWl}%auAkZdCRp3L*ASq7m~Bki`Fm6#M1Ll|L44`oYz)@By52+N z`RLIgzI@M~5~_@4%j9%$Dj5R01BrnN2?}|*J$QDO8|99O_>4xx<_}gg+y;tb96-xgD)5p%UE?1OHrr2yaArKt&JMFnh_&WOC-3Ulukp7#pO}cnO!}4|r`dI|cj7jj z3k?n5QDL(hg@lhkFd|JR;{`tL#}AC9tb27Iy&l5JfDCM5;RN1fXNV#br{f_82B#R{ z;KNewG7dxCOyw(hM~KyVJf7w_`*aQM6oaH6>sYub%xe&x6X{p2`p<4Cg<%6$Au!2C zm}5;FZ3(;=daGzUZxDL$If4hp30F-j@HA2pcmxM+FQvfs0B{s{JYwDMf`v3lf{?XQ z(IM6#d17dQ)&pdMPYtuhGjDDFl!RX%Q{Hxl%r&7Xb&aUXMDYqik3va4ASxzCVR$|& zPm&Y+%0<6Q`2^fRek>-`p*vHMjsW2xo)Z=g~|ri46-PmD|1hDE)>MjlPm}nZ9g@9yl5A|=Y z+s^w0w@tZpsj#4c7b-A1{cQ-52m=IinC-o35K{aG`x)&6dUWlI;p+x!+HZny8k-i5 zMMcF;D&^OA&HMZ|e z|KU_!&5H3UQvhP4;4sNNJgPt=Vb|?$3Tsmd9|)c0qpYmz z+@mxSIM&5!%e9OnrZEEq8~UcceqGO=ag=bVNxsZC-`%cAR+krvfVb`0Scc0m8Ufm3 z%EjY=g!}g|U%Z$F6=v&Ri^S_!uU(tLmJNS@JT7CXCU~n65wqbzPM>~9)du&4yDQ$G zxrK!myakRC7sF>#L^p!xtlpGuseD)35B?oCOGH4JeB543j+$Y}#sD~z26WlX5XFat zJlK4gh0sG8SQ(trA~Q=B;MW^_)+412kEdA>ZO|u`#54+9B{?Fui!c2rNRsev=TNeB0hFqg_ysvg$56;Tv)YKDw zqsS+Wi(n5;7YL&Z4$t4jb{o=`tI1!cB_j4YpJ5{YzyX1=M19HpHf;jYjP)IePN#|0 zZ?F7n#|pAhqoBBp%nY1OUeHbIkKHOt9ZLZQ>_`ty<+2RJb2m3eEIy%l|MUrI7^+O# z9ik0_SGR3XA3of}S)^!!ui_hVu>8_71g4HS8XB5eSm-o=KDsYqJ@K+-Ke50dOR}y* zBFQ4dbG51@?B!reK~Gngs1EQ#N6Ae{==)L5bqdsYkNt#}R|EmP)JKlI%@G971GEN& zLN!D(<4p9>)orFq2UR{%T+2x;G4$XJ5|uc!Bu5H6L}PxxW&C4#(}*%qvk7_xk&Lvz znCB=S!3Zz|JA6h8H^DdyLr-I^`566m>ox#K@1$~Y>kU-*tT3cQyqlYg!416$$OuCS z_?{tCM>4e(*S2WxWuU z5@twXSdP<>;PPnOUI~XvafJL`X7+g!54)m?)LeJZty}5RvFH=LLcVpFMxhcWf#B9f65?d3npW40fC?S+(j|c=$;HI6Sc)h@4Kt{cBTQwXE~vwslaC z?H7$-@&f3CnDhJa@gvuQ;Vk`K;(!y6$9abrM<_~N*0G&MG&o*?8> zUv`kF>+8pWV~_?Im)G6fqgyvzNBpy^u~gVQfdr10Y`Trj8I+%+Mr{VS;I2M>D=Z4n zeM9?CRde^Q2#d9d2o4Ii>8#xI@bqjXzTW;!<&lVw_l+CdP%iT-H|uT0On`>`=FOY1 zwa#;-q@Iq1j9~KAfCVj3)u>h4~nmGY%z4>lQ9lk%V&7 zIkernm0|H>nWS*}8p1B;yaVl(<# z7ri`Ew7k6eE@*G=Rr85oo zXP>O?aP3z^15?Prlaz4M&jlNk`==d<9i2PN-n^YTjrKJ#X9>ZVB8r`%_)(#>(8)UA zE~2mFh@sA+sN}WtR1)u8Mq0r);cjJaN9-V%dj#l@FHDf~o&AAqMbXO{;O~mRj~O-U zH+vq~6Ls&NA?$HM!IWNMzB3jg>xWjNKm=RGP@0J%5C`1m*$uwdnNF3f5MkQ|>2_!+ zVGGqw)X}4YOjpqrA15Xn?>L(f;%H+;G7`gZjt!*^{xSI0;S28&z7Ufb zm@*y2U=7^E5n22p&fUC;i6kqGADfr)Nx@l2CH!m@HE8lJ#ROVan=xz$#CU^wbEQC; zV-KmtIAYEM1OwSQ*U{0%#l_bMM9`Va6g&ciiy>`TWrPMbHRa4z-|s<;#^!{~4YJ7{ zV>$%QILUz;PI}Xc25pk5cvKjqL9dgccuEmCrL+ zhXotIb*pYj4uH^Og?H_bIJ>dm`ukZjcS!%;Fm7Vb)Bn{1_=qk)Q5O|&OIR@@so~HW zJVn{10#Z-xr0R^^9(P&G(zQc`?cQCx7(1rg8BNhsUVe<(TsA!y^x}0LJ${@W=rX!v zc)}2`Y01frhYWSS{$xbOU~5Zv_`&N9F^j|0u=}c5PI}d>%l9O|Ys)CTtInHSSPV8Y zvPoLm@boJyUw8@68W3z$V(ihf5I>q{`k^;`C!C}LfHV;_X4J?=M##Nytb8w9RbAal zON&{5*(kfyPSO>+#Y)95IgQxlMkR6Y4nnCO+|CxeeCL&Q~XM=B`3o|(!#8Gxr-O|%2_7sJe?s> zJ!mUHAW0Qz00Hae4(pS}nP2hbv$eBxadp+=&h`+yUt*fbTb=s~>y69~-2R%aw{ne% zUyNT-^|?p5=F3#NK!>7H0W@w^sNHsDrG~#N%^MGVijVpRwa{XpfA1oj5qx%%+|-!f zDtW!LvZs=K>C(#=w1&$yTRzS^eanfzj?4+<>GZArPAhAMJdDO zM&4qsRnh-Wb~c}pc?^oD(0ihnvq3Xr`=X11e%})uOgTgx24w{2pxY!iad~NJDSY@w zpk?ANYqoBEnVl`LDSRhRre*tL(jHCF<3fJBS#qqMY4FAA3uz!=W+?jksc0l_AVp&U7#m)0?U51xGRsS4 zOVI@)jmrhiFhs>0`~gOfX8#Xk`L-VKXwD~&A77kwlotccLrSXBnMXaXqr+-IHfnd_ z4q!DwIV2cDTQcB?0}f-wSe>)RLy(RNMg#)~vdx2d+pphQ@F7ZVs$R4u7idjMIzYGt zc-SM$V`{sU_wC)gkU2D_zHDsp?R2CBr>@hlPo=S{VACjl7y&&3$~HdrJOlqV{PriB9nue+x2;MoLg4UV3$TKVZl0WJUDR<`7#9* zh{XfA@G}Yz{LAZY2-eI{c5-!-9{3eSBmp5PJK^+cic;P_0hw#<8@rT38^u2J@%+c; zH9q82hC&oYuPNPGpH5ASY$qd6b$`y1^lR6ECaHKce;opDVpeBsdK&MHMupxt+I$OR zCr|I)J56Lye5WeohrI>Gz!o$GR0>(IesI3Id33g3lhd}-_?hS!4;`ve`_wnP+pSXR z1<0MJh{qn9GYhjVP*o+3GhhY?0ZML3K%YRFtf46({_GL8SCD(fJJLnb{5aiFoyimi@zr6;&%)-xIsP zulI_Rj{~5KX!gZ*`}dPInBf62LAK@j>vI$XyZ}XzxR!T? z2_PCiw1Y&yf`Tt}RCo;lp;27?`8XcL2{4OOLsCQ>^r80!cBxDKv=^`bclMSoFchf3 zIo_CQa2jyS6-JiiZ(G#6GNpF<^tJSK1P%h^c`4K?j-td$T{N_d+&(7Hq)Dkn!a)o5NwmNxTV%CXb6Zc5Tbt{U#jA`9llO9x% zE3sxojz4$Vx_{JXu{`k0%MsW#(IEO+esdTC;>Qv}vA77$2$B+B6JQpoh5(P)+q)Ax zXuU^_=wfitWZ1BPJsAv#D*9vi5XTUufd*5)ApAKDN>ErZBEY63osNf)VsRw9cSo4Y zO@zNh$F20}HcX7Vs1IL0J|6Ra-&v_bLwWWIPi}BuQIs7mp#0e*yn)0=1DsfKBXTR^ zLX?$t+jEbTh51#nTf=|z#95n2J5>{b2HoV?+3*>Y42kUgqKo`T9g~_3V7-3_ zeJCU$uvRZwa_8Q?t1Q^1gAnf#;_2MEKpNDLg1>2(E~wjn>^JH~i#K}nSNg&5=xBx> z^MpBIeTy|KSL)&)X({G?b6t4xjPO!k-hL^=7%j&QB^d#fLq*lWhZwx@`f$G9(YN`O zZ#8n@yIR+Vz=YJQuplxISdedzqM4Av^F5uAkTm99n;6*z<7(|x^@7jF+b;3}RMj|= z>6>jr!W8Oyo*M(#NN6PcGuOl6R#DlTbObO-KF|tT15c9MCs+%!VDa*0z=V^?y6Wm+BgDn)*U=JSih1 zgKq%XM(ARoj`11OWviLU#-JPPwY?KKuAEU$oEqL__fG7$p6BTJxh$=a6-8jnWOfcX zu^w1OJ^&g|tiFo|n@RAi_LjQ1LEc@C+b8^P>To6Y9eqGGpjFmjVk55Bw zCogXuFMVe7LB(_K-X+dbvJ3&9&|jG2Y{Cbj4}|Mb@`M)gfQ4e z-*5CI+-MfG-C$qEo;@5(aGR2&Zi(i(6dWj35}uobEa5Bv066h=bu|JeO>J#@)n6Ab z@{iZKyVEM0nwg=&#n_=fX8oR)qwl`5W-pLOO+(*SCv;5;3gGy1AdG7Vc9!C_(vs6x zaf4YzI$-c%BoyzNNfxWe8qA(E=PI64zZO%GxetWYIdy8sgp+VLTRc4he`xe+v?-8S zWS@KQo&W%rl|gKXre0SRoj`lx>dKs|EAMgDDmyDHK?=Tg>pYhAJa_WhxOuaX!+5Z} zQj3?mx}FBxGc=rFW%U{bO=Zd}H+HAK2fV7OVHXKC5yS?r66}amh`$;;RFKJ4WfN1W zs56&}%+q-AAP^W+Y}t~YoE%D$a&+|MwE(vO$#Qvst*j$EDO%&i#DrkAJGCOaIml#) z@zq7(`a5>+8Vd@R+@8vp70|IzWFkma8Se>xKQOAot3>^@)H&`USiKO5HH964`2#!LU z4%{N%S)(Ru=#U}NINuPRGTNFyk$zbX25t)Rs-vzh?%jLl!iB_%0uW-K5}vwTu?Qn0 z&M&f6#?QgzAa@LnjBo!l9%D@Pyyv zY{>;=LEKUn143o?#+|oes-e9RQ`xkKESw?Y07?OnqZuSVu-rnZ35ivqzc>}sU0vTn z+WVbLq?Z%6GExnZOsT)Q>=REGlf^kcn2ot_-1z0g2Vs2JeQvQcLmkG(FjS+>%m#30 z$`bsJs)&BluKBe7gb#P*Sg|gKdHE&0O8FmxRpiJSm8^hUMi|#jB`7VQgtk8*W8}hf_W?ip#i@;GcA3 z^I%7lESUjfu{*bnav#$XP~la+vo{wK|x6^9{|Kd{X}a*n)E-CuRnM| zQ$p3?1B&eN*aQrO5you24d~<;m^g+DJ9H?e`^$8XzGTuxYXEl?{kZ}B2*{qvHucQb z&U3_`!k%9X7MS<$Eqm8g3R7{aU2Izz1(8~2gR?KSuuupuD5WDslaKU~mrFQFm_FFu z!$W`U)`Nk8%Xrk(nxHZe!gLbx<9}zBJj}^q+7(3Nad|mror0erPoz0h599=kVC;MV z{)q5!KZ%KWNk0b6M6>Ca7;IyJSfBXe$gDrk*`r2jpVB%M5y5xB0m1^8A=`NWj#72+ zy6NaBBH?Cu_1QE3=uG_5t$UHFBJY_)*vfw;dlq?H|4RsPE}R;MN0O~Ojimf z^PqMnZNv>6;9&DHNW$c`d$(>#Tb)s?hv#t_=sPy&Nro9%b#<n=O zSUS6|20V1r-jq#;?-XZQrj5M6OaE z?_poS9EHv)u$yn+rHdC=bBsBx7O&^fzZef3h{%?}$YUT1P$pxZ>{@Qrzdyq%Bo9nk zv`@Jzcjy2{2?)+iGy_%0Vo3LtixjSWe)7I7H!q%=3;}M@&E_g8k>0Rb{aFI2;*Vnb zU{q9xV?a$PeE{9TtD4gnXtz_ zA45;WgYB_jVM0evk)A5_XU4`)j5bGNlTWohr-+gAwbwG1145gCDW}x1_X`{z`E4Oiv$ec>rApw|e%>nQ&D@Ei7UpBE}nrlW~m(PY=ba$3fG`6inBjM%}bj`3`1rB=|E~!`MaQzW(?F2j(8^K7WeL6EOm1`934GHtgdraRkmZxVBeUS z_R2_$KS5G~R>c!X>ooRP7l4&d%A6*m5ie}vc#}FW-&Aj^A8BbnevdwyJ-*=6Y$xp> zaqhwe3_HM|Vekb{bk4X&FeX#Ga^DCd=BOy7p(D-B@l2hksT9HHu2kK1rh4>j$p47Q zwEN7UF%C(@&vRLiQ6sOF4q7B3+>(93N~Df%K_$=KbczBHb_E4 z0!9l=&p#3#u&pgRW`1Qod7_Eu1QRg3cT@2I_@AZa?#r87hQ$t4I5vtyis+V{%hpq=!dBoT zU{yK?@(j`gJ`-&e%Xk4O#gB5bv)O1o!@(f|)VfCx)dPR%uK6}l!N0$hDFkHE?NTfW z_1K3G9fGo%Ji3dxXmSu4kuuKKoNnNl3A{Z)hWD;E3uWvJ<-n{_nd_uBt;HahY54~75N(IZF*Fr68RwVu}e zy>cn2Dw9GPD;A>)!S8mI#f117emA8=66i;Z?-BYp$Jp4TCW}jQZkOQT=hLqbW*qIk zC$07QJQERGb80too4N>W8KG~R;iQuOaG#b|tv7C8NlvcIotoS(4<_TQ*fwpYzR?cZ z=J?Xmxm#E4Yws+zalU2W#-olX;Ba%DJ6BqxzOwlRKfHre9b;*h6Uf;uZXoS5!zpZ$i%ET{FPK zbED+}!Jf48=uAb4J?gp0yKW?PRG>a@m!jFZLA!I#f{Bl=?i1K#OBES8$E7)d&xM`X zAFa7{X>5Me)7Nhw74-!UV*jpPjXxTs!QnvFq$JBWZMsUa)BdFuj28{3?3_!LckzY- zUg0XBfobb*m`VKenr%>4P%yAxKfI>-gLl>0K^89rr_9Xe`gaB0@H!d1y1_?3R9%*r zmibK@4@r;-K-czDzktksb|3O3(WY^9&{!#F`OI`G8imq@|&;C786OQ z{m;F&&;qprVyv9u)Rd43=WZXfy$e?}bK*56V$AUAm-Hv1q&~BU%pocn(%u{xQvLVH zsKNzsZHdhduR>^{X|$*~DJD=j_}Ll490lPTd%gmh2=y)R-!>zu=Q)oN;-Qe7A62zw0%J^1Pq?cL^0QaRB~YvE8l(O)}hL}S3T+|J}cw^*TP9Phiil7 zu)VQUpIgE%I|~boms#4O)-0D(IaWcEV8^zUs5ddJ@YrbCo2K~ZLkjrLIQ9&(IdJb@ z6_JElHu4GRAl0*#Rj;vQyZgt2I*|xabFN>vuC%P|abmt~zYcn;*hMd1wW{L;VYwpB zZfR*(NI_I?LpCY;Q~WT=iR6JsIMVDZl?^U=9PM=hkz~X z)6cO=jCzWw56445tosEw0fWtJK|x-8a|Ey~PzynWnNCPC(FI^A08+u*4?##tXFVVv zSeRe*nGKOAc|I-Qm9hNA%FTY}PH`=_i9Le&*g8|eGjEH)B z3sc+h6axl4xO;bp>-l45n=jJ*gT2xoL#&X^PhAmed*&;FgaMBUW{Uodh*aL%78w=> zW$z*ESXmomGokg68^om2}? zAy~H4FFEG}uM0#Yfd+H={DJ~G-9&CCG#zK6=|JxQ+jxpy)?hR>H}fAvL=?gPqYl~_ z(53w5@l#)1?*0AF9eWrG-a|?96xU2QCx`ms(ax6Cs}8ruF|{ zEkMZpE76ac6RZGEq;ACg#8l7wZGM-`JSL%e4nWiJ4Hy{o(9lTrY4--$p~XIOTwWxh ztU}4k&jfYUSzDXY_(@nnWEt)h zWSp3BkY^be0ocRxyjPe`j~;!gS) zoP+7|)y5=JS;7x+w+Ah_X8HK3bltCCI8kq@tCHUJ9@bjMee2@UQR&^c-xFU{N8e zY|iO%bj_5f*dU&k7D>1Xh}|hB;21jME_X|=!l$!||6Ct)q9<-y(Z`OV(enAWS^DRr zv^#0wI^P#9v;|IF|wxFvwuQ7 z(dV*RD=js3D|G>TLI7C+K#Gpw^CmYYtg2{|$7MbuUxrn_vRUKNyucx#c<9TGy07j;> zh6b0F#-H;jkw}ML#mg{w-@bA9$xa!cN-sy5$^whpi3sv&SHMtd3rQ$cLo2 zqQa|7ht6z-@bH-F;K05r+C?gkwOBAuzKH~YQ7@ct0ZDnF3|PHr$K}zXF>f zVBj3RXp!KY-L>oVnKLKb*#RZ^=je1gVlycn^?8sUb+V()|z(twInCaO+o2aja42 zNIXhm5S3w>b!j{(XdhB28>f%h7yeGcHy8Snufq?9`x9ED=3T&7edPn_L1jH)a<700 zGS$LU2CYQI0Ml{h3Q}jRm^J)knQ5)7a|a7}DVF_{3SoS< zvg-9~upj_`PO2k#`v>X_Dp!a+{BCE?{K3;j79q8!iVjUS&=oY1o6tYPcJ0z354+2= zPPARJc5M=6z3F7;CS>28W9);X^~m>@Lj&hSXYvbz{y=y*lriy<#0@^oj$~$!Sl`s6 z>QHQyDmk>lRuJ+r$Y!(UA9b8v5G_IG@`DNfX4mikg50D0ZQPI&ISj(dR@E5{Y+0w z(`I+mwHuZ@*AvFPF1ZX%hngf`(}p{*ZiVs3E!cjp!VOJq+~e#s=GKY=vdGQU1$eSa z;%C5;gKDk#8a}wo1tcxbCWjdqY-}{u*RO18G1uKI|4vmkNOng&wjQcMyg`%%MB`W& zCh7npGAg?gq@9N#%q1f+Tq0Jdg)NLWQ@FVy)4l1?<{(O57Qum!1!WImgI~3K{|LOG zJ8NmlpPF}dO+|Bjn(t1!b9zSc4~3c?er6DoUET8OnEA2Q4}ALk`6kxRbxmlU>Cl)m zDSp_U_gh=bkilDIB?thCYi!#lm~ym0>wGWQ4-WYZ)`Ij0@I){wHZrQMsS!0g`*em{ z6=xti24xfod^3G{+i{ss*trLQ!)KuQAaq%-Lx`}FfD3luXrV4*2?Ed(+tJvNuBfP} zXSS)M4vr|VEhQkkV8_^U(H!Go%A~8Z{dENIB{iPh!xfvPXxmt*q7-;S?LdiDJgaI( z&Zaz=6J2A|K1g z__At7AAk`2;J+2}GAF1L!WJ z#Ze|o=LHguO0?F14}libTPXVXU_%=Fl$(DualuO!wmYLqTCl(zxgaAY>`i949~K{! z4*nkL$?jKoM(y z&@2*-bUF%FGE5?H-pHKJF|qsAi(AGv?39bJMeWNOhq25wV*ZUzsNwD@t||j<2#_-$ z!>fea)bv*yjoHfb<#eyy>b{MG(U-n`yKvE>(-w2v?)M9~Gh|8kSd5v*j$McW`pJ{n zKpp&5Is3Pqq=?)??h)i6`s45`C?RP|B+e54>Bqya2RW*p07&qQ&0*`(>BQ2V3j*=hxOi-v$j4AovfdSo?q`Mma^P<{ZFh1(@g z^BZX0??dTcvWD9Q`?ZrNKXQDC?xv;)mA*>opw6tF_4zV@mda-M zu5iX{f&2KO)cdtiiGcVF=LMZS&gavQXUE87%@|bCbM5UdQ)lMx8Hpc*EjOMRL`a4K zKlOD+=})|+l2`UUPw~zhHuG!T^9h=%$vpAaGVbX2j>me zR|m}j!pIH$#dNJpmp(yOPTLZPdBJ3ns@7qun*2qOT;MWWG~^&MCII-SyIvgEltmfR zN+NRyhduN}h|3}SSy~f1uoVPMtrdV?7Qg~{!S%5wrsfxlZo((OBuZH3RBeqsVBE=5hg@9M@STuBc5Lk$l0^!(cmDu_|F(=qU5ES z#BX5!ZSUU4z)JI_pJNgFGbGKR3-4myfG*oDHE9#k3k{+=0Drn|=ZkZi0?r?25`=yv zZkEcM3qp3s0Hh%~SAIT3jNu?~-aCXFxCTx{R>EAgtF{CFXZ5zeb2$apmfT~mU(eZHS1B0NqPoz9cQ}Fy;ky%UhgAIVelV}-+qF7G!H^1`=yQwBNrg|rpB4> ztqJ+}@BFF;P-j98%|7v!uZC6?RvC%?S1ibXjVl0kGt*!8{VxYh5XDI0i#IO_{oa(b1q+hWf@)5kg!V&!i)^=hh3p)R26B9#U; zT-|l>ps@E5+TlijH@|hNO)~&&lxyG(sV8)zryxvbz7L;!J^;4lV-M1nV&+t?sMAxQ z`Q1C2Qh%au<5n{V+$tf*I21Wkhb3XOj530PdW4G5tm?n6b~x>5oK*m=!Yp!xr8|U! zzrRV0^>f;FObZcONW>JY+vm>1BRwVMHCk(a570vr6|vQl#CMTs31x+33YMJix1`ry zC0b*VOb>c3Ufh&=R#rx0Sl`fqfc?Joh@f5uD_?dVzp-;mr@ZTr*ku9kVCC~LulINd@@6b==_fw@-3Iiw1(I3x!I11f%IVu8St zfH&9y6Y!hd%QmfDnEAoTgxBk7meO*55it>656);-y$=Uel7-Hf{<85p(m@9&Yc zW2wp*2KqviT3Z8QGzspxUSsio`MuX|=s4~c(URLK`7|u`=ut$O{)5JU55tTnC1REY z>ZxHoVCLW^Nv>>+H9kc!eECW%HlS~UnlL*PO7iaBx9^|R67{jZ9((i#kE70o&|)NC z%EE5Lh8*ao`wO87p`(9Cty~YE%ua%1>ru5(jDE=Nl?OZ!@bK}u|9UK32uf%fGZbt{ ztMSkJGC#(uaX#-}Am4>}&h0>QDwDg@`Y_m;0ao@REJ07iGKnK&CZNm&6$U@;Eo%Xk zfUtoGK$pl2?(d!9lxB`!*$QIkTZrkC z_WfJEY0Al%zLuzR1bM>TcZge(A)M&g`q|I__U&8EFw8@eR1(JR+qFa zb9dJqG$`Vw^wrhxA5goHZlFSG7}Jo?wTi#1w~p5zV%C+Km12^Y38P+VZ!~r+djI%6 zd;VI0brp%yZ?EY}x01UKKf(e$=y&~imzgt#l?s&3jKbzwF{jR`{dr&;|AlT6MYzKg zaD5|g0}QWXD31v5%a=`J3@91q`$G4g3YlOJ;HrFB{_GhNLx8I%ZIzXS2Hx+FXb5W$ z`f-$#u76svq~u`n%n(C;d3mMNnem!2h>cI4KFzgWu`fY|DEj$x08Sp90OBlqROuI* z(jk>>jHNOLT~Wr>)GU+_MOjFO707=EPuB1hfk-s+e=^=z7I*tS`HMX{T`cYs8<6lb zi0UY)D7~D{*F3+xzKkP86r`RyX&paV>j8O%Rw|+e)_)Irie*7$h+{Bq#^_$I3r|sa z@$L|ZJ%=JDicyXdrjZ{N6>a7r1peWP|4aRZxdq9Xes%o5eP4h6#Ls^#bY6TslRfew z=6djmGe-iQkQ#86V6jqq?!}p;w5&`3^bJdx>SF~t!ax*3%!<y*qC1O!s)MVmlBuJsD>Ny)6IQHT`KqEK`@~86!9SqMK)JLG0iUD#WfJF@KYsWA zJ$+7e{HCPS&?MHwJTStsfK)0h0UtAlu`@wRfw#CIKN@rr`W^xjIXNOhBy3<&K0L#s zNO$=zU^#dJB6-?_X{cIZlBrGvw{e;!`F$uKz}1WpK@ABaF?Ms|>&Jm+&f~s%4JZ_! zRcg9i`z}o0vcm$QMQTJaScQYZ=jNZdjRT9NB*Pd7DK4fY8a4STNe=BV8YyN9nF}S( zkpmxi_5U@Mz)M7Cji5r@6m(ov`_x`@=f1$5e)uP!_zuhrig!NpHO%o3hQeh?tWBgx`0_{CIrCb1de!LwRtTun9VOtNs zkbH@FBjss5;huA}u2}iHgQ%-T*GyG&drPZOXFVy+8>8K4ceNwrN?NpJeq z>e+Z6N$k1C*6dr++(y&J?6QA9{Js|TeKpG2mPczG8XXt}jgDsHwK|J>sy8zfMty>m zkY6)q&NN!{;bG&Mcq&vrEV1sz-=RR_ZV!cUdLp1S6gcN&V`p)D5FBnzL3Tis0ZZVQ zm)C5USRO+h$G$LcpNO!Jl`*4X0Z|7jy3mNtVI z7W}{z+K6Ckiy7akxgg9TF!_&^%Ww}WQPmh~?*pKhmin~W<2DkJsL$53?THE*|6!pm zjf|Jdkjq399SsJ=4n0Zsi@+32nR0vQfT>8rgq6&A7B$t+r)vZP!2b&L#wHWYe;46V zhJO=E9s1do+1Pn>FNyLZ>F zXXlsolnVjySb z{`uJ_-ew*qj5Ov_tRhi_V0W~(?&UP?UF2=3KkD&boj3V;a_&hMQ&VS>E?0PZGTkbt zy{n>a{Pih4X_6MPhx(N>%(g4CAbl&k_xIvQ4n*ZUK3NTV(?!K3#9V|9{PpWlt9cV< zoWtRr<#NX~C?J`T5Fa700>rT;7z}Mc`9o>x44C}-^8vI9_~1ToC$K&nRfBX0Tbv9; zdl=dje&Ngwk%pyx^qxI)q4BcXeKPA%KQc^oGj|ILY7hlNK~|>n@@Z#tH?V-|Y!OX2 zQRHu^r6qf+Yesz~Y%d@d<_?TNsOPe9_$zlU1H^J!$oXJybpPB1S!vq83xyleLzlv& zXKoI)Jma)@f3TIA<`zLCFLo%sIUI#(Ve)nX^ek9DSMX9q9$$w?B%4HuC2T~16GP*_i&-+{0ns}+C`c!{qgNZJP?{q| z~*EBN>>-aVJ65v~5W~Tp%7Q%=3v8Q;wE3`?n$l;1h?B@3%FD zb~=(rc26pAO`**Qr7`;03N9&4RpkiIAA{z5qoaexcv%-PzRBqzfpgMmdo!NG=UlR2 z!MCH!=_w1F8hKx0GVegl)ER_?jnxobgx*3 zUe+pR6Sg=V+__Uav=6fbUN0|Kpy+V%Pj}KaO2GO_qeqXoU%uF;&MfL)iQPY351(nT zuJcZmtaulxNAdMz_w8e00Xme&(3@njy?fi68mG%<1S-c1EnP(6xp zGsn9rZtfqrk8dAUgwD4k4_Q(1Ag|l4+T<*w)M4$=4s+KBbLX&!q~t7t4JkZ&7niPY+A#T5|$&vpq$MyAki)G69M%Gntz&sFAk$JrZ?^hWfAfTf;>CmBr zT`^?a%8ora9o+JI$T!_Vg!cRQ+r7CLm9H6(YK4f-JGgMcz->R9F|Tdx$e$=01d$!` zcHr>g$h8>0BY-}A_H5QECn?Wg1S|Tj9mB_KHsl^T0q_QS@!3LuDmivaI#2)DO;H>S zDo!lHuK-^R7+**^7zA2sq@-t{nBaQEGkD&bYb+Qqwc!l%D3AfOHOOz#F2V1#&bUac zfM0_#K&$fR>r_@SfC}t{ypmg6+w{0-svs-Pvfkah z53_Ha5q+%mwAj3YvR6LTZSUMl=RUKv{Y+j$hi)4`syBVUk@0un=ulkWPsaL`O*W68 zv`#PXYob0~OF4JlaU6yF|mpU4OD|EG$l9$_h6ZHScZ@jNqI*;D1|lCWkcrnXUg zIms%+@ZUQZCYFya#tfQCkCNI|*)D9G)!vnFu5`nWYfA(n%-8s4WbZAhmAEY+uvNoh z14uHS;aULdU&jG@U62Z{rxS$86j}vLg5Dji1PuZ@bQw!*c&xbeTZleU2BA?H%Q_(7 z9}03Js-vN-G&=+%jpL{mvRsD6SZ~|0xm+ldl4QFDDqUHtM@! zT125c8-YZA#|eDARfwH|f!X+wftr~V1argrUf@rXjO@~#W<+h@9ulEeTu-ou*&eHQ zk2^unJEWirPWSrJOOP>`89gg5KAD?qrm6W_qH!pqdeRcj7;*|r?>1Jfga?8ja&mAO zuwr|+#QqZbwzYcHBwjdv?m)B_Ima;2r013~vq3|wWV^c3=5lCY(68pYNd-$CQDqa1 zV3c3~JKp~qg-YSP9jribiU-%~nyJ#@A`l~yWE2zRf*)T|HC#AuVKxU0yXM#-<}z60f#X+|XR~F~WQU1es za`kVsESU6xI}yY8r~V1rM(+h32}h-Fuo94#wK%|T9OU2%0qew*A|hw?r!>0tcuVgh z+W6)8aob0F-eJiY?T`r2Hcy!^g8x3zBf=70=;-Ye*`h*q=Vr@j5{VJI{Ki{#6)U>= zc;$kL(l;t^ z3=%(OPczdYRHh7w#nQTcr zouaYM^%z4!v`-(RcNDhHLOEb^32x@2a;A`w?>%@>v9}X3jA!H5g#`vyco&{oVuAFK zP7lpOBBR{o)FVf@cRicBQ1CeG?xlFubNy4EGJL=Q^cTX)19AxOlvP}k4cQa4cUfu9 zz=-NdW)b$gzO)sSY!QS4?|~S@4}PN~f`jW104K za3$1jgs-06yE7%sOC}5bZf*`p9D!)tO%G8OId5|T+wD7U7%2g>^7JTDXu-!p?I57! z>yT0C@lsiM-aGKVx;pGu1u{>X4WXhFbQ$O;u{eUg6A-|-)~FA3VWU0GUhhb#5!9Gv z@4rJ_@I=(VXYpCVxWvKjAEP8aYZUeRQMl#}`j#kLzW;b?>gaoC6`D^{_+$qD`Gpml zQGzMMM)R+2q}>#ltMx~n+>bpUucEbd^Grr?*YN#lRO7YHL@5h{t_-YoUhJ-2l;KyY7twE9Yq33=Z7CQ-7_WYJAFVZ%W=Kyj#$R8R z9ARq#18xjVGn}WksPQR)639Yo0EENwpeTon#SAczZ7|5?P&mDOd6hD@Zp<99ygS}S zpFeY_Ifsa!-@JXRJ!~2%mc0_lGUv`sAWTwIGrhZP*^oE`jPPmY18K+u3q3rjdx=aC ze>s?e1Eb)6h&jLr;7UZ!NoK5>j}7|$`64y4bae$y;aeVOY0UCQ%e^kJVGX<|_>=3OtG(eT`vaF%%Mp>ro@~=wH4Rc0tm!aUsOcAK@_;^h) z4AYO6Q5_NU&m2CC{kGTq`57L!cGmT|z+-?O;9)PKw0yGz;74jP6vP&t6P zpM!h_6*05kHb<{*x+40P2d~ub1rrMxBS}4Ll9kRUdEXeizroL+88nbujI|dV8?$|h zGujEZi5NDsfdFNXw;`}H?tI-M=?-R}ni6ktASu{3K+Nc@3pB;yoO7`ME~A}poerCe zfrk*RG6uq`CH1JIsMbxdXI6aTXA&1#YeA_CU8iS`aXsBe4y1}zNQ}9T%1j*FK43V7b zSPnV(CBz(g0=vy9aGdVZ1;L*Kk~vtC!398?#mKX{n7SG3U8an1vgW+CtwrJY2Y>FR zrgqUs%4-U{c9=T2awVa-1`B&+ralG+ikr*mdwXW`4c@Q*m*k&d8@qES+g%Jsj9>%> z>d`GXhC{_!rwZGeLN3gUY{+rQt8QSvJ>EuJaWm!PcN@4jTh44IY!Nc#_vmtc)Q&Tez^ChXe-mNymWVCG|0RI7^1gt#q z5E~kpC<^uc2;{4)!;~F<2OBmOh4y}vQDq@mDr|fpJ*yaL?v5Szz{*=wlFl^y2Vyu# z3Id@&*0#_<3iUcb%4&w4aV3hKWKJG#37CrU3GFWOh{(2Xzb4H%cl`JgFRwc{Z^|F; zucBbF$YO@dLl8clTxRun%&ZNr{?dNlEY=}ct}k3O5kSl@x;k4^^7?ezy5p{=OhFt^ znODH*i9$H|B`&p{)eM$1nV7I0Me*mY?J_PqoJTq9(r-R#E`|dsd-zaope8FfQy_=2 zZ|jwsx9Ae$zU#lWld`jWS(QRb8|**L*347cm6O?<5IjQZ(;>+}MMk@gMLO%m5+6)- zzBRXyM1qQs47iyai-wYxCPiLL=E~j7x)Weyc}89!FmeDP4g_Oi^9uiJ$JP+h&APAC*Tja^-DurxqWxU8 zU5I(MOo(}F0>*wM8m9R(olb!p_!VW`Z|Km^4A+}A1%G{Yv>nx6pvYsCnOP7~wfgl% zn#UyfEi`6Rdd%FBMO z``Zs65K4E_)DCR^YNew>Hl|V2UtanR z>t#kZ86~35YqIuXZh)6r#MTs|2Ju*Dt8JW~g~>44Q1SR=tVn)B z@cNs+qGm-j@O;As{v$;Fi4$83-I3i=oXj9$!Eyl!k*HwKSX|RXMFsgO$BJB`AR7ur zekZ~BBB_QyRZw;R{c0ve>a)%0SWzXZD4>XBDc6>iV@;n)VWxpC2-#8AEkJ@zWqxM6 zz>}7iHhiiWSwD2PoeH5Ec8ZS9F@&kP4xPG>6irKGNH)7x^7Z ziq+TMqKXg$Vuph2U255TVe$%5X;cv;BQx0hsHW|7T%e3mgF!?>rZQ(lkCX4#6YVu! zZWy!pc+?vh88onmhTfb^M_=TKbkGnEfYSikt~x4LY0Qlo^Q5SVv0`*nfh>??QzRTC z!S(qU1ZEU>TnWkuY6$)m6fTWU8P|t*Z{J3N3wiNc^AMv9BxMc*Ody>YKsSX1wE)a^ z{rB&OOBeICAXuro>0r{;r}BhwqIM^J@~8N}6gyNW%tWroz)jHpEh(cx%GPBp9-gvC zhPw!mM6bcn1D^stL08M>^{p-0bcmX2qX-%Z2v4)^VbLP0nsvZ>n%cB;!Sjv|4pgCx z%Cpy^8B=bZtyFWE;ornku~Vp&$^&>#k`iEos)9DQKsP>nHi!8p6%~8hsBdn@&#;k#hhg%-~|Vv5Hl4OiJ-!$B(}Tu0lkxJ>?y6 z%fgAl{j-cIQ6~~j*b<2*qY`BePyDaNH7l7qcIoot>(_r9UWen6Bn)DRc`ItFs_bY8 z;%ouFP(x8kv4qIr-kQSNZ&WNW3b)-egdQ*t4_Nu>NqIS~p=ZER{dhs@LpxU{Q(^6R z{Ca7*Kt?i$aS%aSw{Gm941EBMl563l{LK7D$L zHDMH+=KCpchq21o!C?)3Kl{O0urZL7oRs8-MntTGT#oh$oUnS&C(>jWvPg?PN8kne zJU|Y>l|D&ZA0eQnDWpgc3Y)0^p&`{QbMne%3R_oTSgZzgv=d!|A==R}Y;UWz0$FJ4i-Xzp( zfovKlcJS!ZPM@|CF+G9Xi=vWJWN`w=i#D97N)S`coqGk(%yoyBSGV?o?E%iJDDcAN zeMb*_yo9z8OpE+7hI3D*rXnBXx$!e+sPNP+MV2#W_&hfdvnRfwDlb<)l&Y8t1t1RS ziO3P#wyj>ZYX67@ zH<Qv2|0dn|FWf*B_~U|MU;QOQP}&9u%t8>5)sL>n$$C&>m|Uz za)&nBu~YbX%TLq1(53CzLUi?^qT)QWP6@3$h!VG%zQ5UqwDoAm=UG5m>~^35obo%peK$j3~WLa*$u&XN^YD^f$LSXm^! zr5@yJ%34rR5@=w-P=aFl)=_pmgWI9)J7IuDK`8j16Jlj8VAjx1!{i24U29L5f_F|V zrFdi__CHPv5{LxN%ofbl29!6fP)qlsrl2D)Gp_XEL#?Y(7`ziyu!R||tII|(fDeqw zwvpC&aEGi*f|-P7Ls-lfNIw(x*y5UR&GMnp{K3I@^3KVO5~d}Pd5e1WiYTa3Hw&ck zFr@X%9z=;yo3X`wTwDx0tV||}$4oqg)A|kcRcvYa_xqOOn!uIjzw)1XkhoSG1)$k+ zlljUMqaEDTk*p)$XqaHzGs04^)?j0lrbsqs_ih@Nm zM?SrK$HZEZuRCa9@!~98Gxg(f0y&tFK+gI4=cM;v#P_Dw($zJW@|IGTzXw7GcVu`X z#KG(kpx~!7xRAZgs3L>Zd-SSCY+8KwCefkB45^#}>qh#PzQn_Nr6M+AMjWP?% z8-M(OVPJ&L<+5K)p}Uk-6I26oM5rNpT3j{|V^MXs%NH8>G{sPNIi7L~5pSW-fYNUN zU92HhVvT`7u+t2CGx-smAT8Y-EA{HmuPB+T@jo(o!;FIP@+h(4cfP!RyD8-uum*jr zxmah|u!*8K$kvJbbddo~ylY+(WHcC;E<8yX1u>DF&kW^f^zZj$glmk05^Q1A{~#`T z@%*`PmoYBFl(xznJcNx4v2E0v5B8>s05_;1O-=%?Ys;2x;6?BwNe8HjP1Nxr!xeNP zGF&e&L?m-DL?DF?`saujqp#rfmO<#D7??pLNjn4XWemGnX~zHp-Q8!py0UY55jqgA zouFzdha|E$7J4Y$X2L9xA0>&{g)Yc8}pSHY-d()x?-I1B}Q3p}-C# zBxJj+U>d1y&gC%v#aU+9nDX}NyhET!txQ0r&ysiFYZJ^LIc-D`msD=3~-Pb3;Vk$wIJtU zrqF9`8V#I5(cvk|oY0w%M~`Caxm5;DrMhkQw6Foooj|MavKR@cN%UBKbFo zUAAp$)#$Koiz4-s=qBZq!@~bVCE50`Lh;y^W@`S{o&VQz^Z)rbQ7{X~J;Xfe`G0>r z#J9uWYPGFCZ2Q;$m*4b%E@_3I)Y87+HO;1rOe+%wj6apo*bH+XV)8#)02M3DBp@9P zDKBAX;eOqmaUSx+r6l0yFeZ}aSLBlQ;ER(X@V zPUnygZ=6xG`5FJczAS%A2@ zUHEHR0F*se436tUeu}SP2wOcnl)|5=WCEc z*1G==-T(bOwXq^?AL9Sd!4^1;|GwXU5BmS(V4E51Y`r!RdD2t!QVsUP;9nhYE?P#)c438cZgY*3h90JGcD!IEx_%IyylQPIEwrM2N&% zkM?BX5|JQAx{}gv(-KYpK!{7qJgAx%-QD+Notf?W_03c;B%juP;gs5R8@_71^dl%D z_!HDo1L4)ySDalaxoylu zmq22D+jJO~N;xK<~%bqto%*B56iYbjj0n`;7jOnQ}5y0S{GRDXVp<7qP zOGnGlH^Bcd;=*FhO12)i;pditOvUX)aqQ@bGTw9P(o0Jp4MwEF17Q4*sec%LogqWg zT(TeBzb`tqSy-#fkH9`}!GcVWTgt=|SqmOuBdQ&SVd1pdF*5TE8Yy%z`tk6Yyk&s1 zMKqooY$=bv>xq133A@YDtX1#5AZ*6_TEotnhqS2y!hB(P4I@Z5;yHU|(Ld615JgzZ zT0q(Vq!LAzh0Y937Pf?dPyx#yG->`j*XwkaYyn&1n&pkHMt_FSJ8*oc%{d-EhR=5o z`iJQzD7XA;-vM9pp?(`SK#+&+qCg;FmwWw$d0_rDKa-{rSkhW7LZ=5E)zhSypUN5H z0Pz#jJCk$gqr;?!TS+}^dTMt&7$mSa2#Uvu6)rBX@7SkC{}-!os&zz)RNZ-y;3M%! zr&{IfBbbCO$XC9Fmlq9TWThfY9TiBGiER@o1dzFGi=oDuG-(m8s!Xm!R1P?h^e`9PNl;lZ5qDViSozUq+A{+7{x%!VEzP6 z!+AgDl0Eaejy~F16!=a~%=PtBQIXxr#YJUA5dn3_X-Y18U*631hB4qTpdjXx;ttLN zYQj199P=>rvqeRV_>1L33HoUJK7xglu3x{a#O37e@Qx_;z+|%9ff_IP`G}8J8(l$$ z3pxk8B~3zRGIZYMI40=*E>m~H^Ri@j1|#en7Uff{Rd}sMm$LsZgD=8X4&q15B#BGHQXhu9&0ugE0SlA;q!DMiw^2X6&VM$`s2M2 zVW}W5U;pC=y&NvY{LBjZP=-m^hA@;Djp{S!Fc_BBC@d^Y8rOT0=r9c_YCN`Lprhsm zkZB=X83G_4AvULdiq59m#*uFhU4{7})5%m90IoJ#nIp>ZAlz$KPR>+oYXqjVot^t4 zb|sF$&dG=3>&%pUEsPB$3|A7y0&=QL$Bvjy3>3*z0I&~I!ncI$1%@+vnEq@W9~w<4 zf+*qjIG&2rn}3aiNKFA7vIdJHgZVnTrHyp|`E%^`a=j|_ zIpoQ9S?+~fQI+i3yLVrFJToH6SFgUjd2=xbljmikUI+7!pj9&l&$Y28CTdZVAd`OF zFf^H5QI0lRSvHMO5gP}#WN|V5{0w7@p_q_Aagji(0B;5$cSc9c<2co9DZ(pDk2L}q zb92-wn{|79BxX^B?$?_y9V+J2o<4bU4ko0W;a;6@5RQc@lGCRXVqzK~A{bk#t?i(# z*l^(zN@cx2As1Fd*`au$NlZn}Yzx3SSd**qF9e~T&H>Yaf01JVU97bR>*_)d_f9Nn ztgp{g|Aw%ff)E5w4MCZHXy);tCE4GG`+APUlZv}G@lbY74rp1!#4oy}mi7VDRKlnf zVW0Ls$IRGR&^H0e`}GsL*(U9IoXB?2v*E7K8KKkKeoHcGg!TaNjiU!Wi~bZ?8e*Of zD=5hF<3`{%FD&rr+>L??H_S->4YPGZKwQ-|3W>5I-Ew-iWS3-z2yONRpG(- zvCxlt=57d9puatCZ0iLbj5wB3lg=t-G=>#%BNj|#p)i0Ue;T(n7vVe!FepgEU&GKj zbd=)eRm2-=Z`!vQ+qzdj#VDWAdGp93Lq-y~!VHUZEWy-_$%Kh-B79_pA>BEYTIadQ z_~m;0vdNn)kLV7y3uJSbgGPe~PhqPK?fZWHsyOxZU^6pUaS?5_Vc*wN)gqIw{K}Vh zKX`~r2CN2kq&pDq3?3N%fnE%UhjQ18($tfQ^Unk;0Nm?;W~2V%>VY1KV(2+=%T7+O_)H^e?^%5hK zLU#l*a_(IG>&e>d!PdnVTZuN_^<)!`Kw#Dif0#HJKWfR&g9qpFF_`J!nnD~EJbq`e zM|uCmLlKEW@L@<4q7)IWoUCkmg)rrU?GNn~#OE+7{P(L2|64>;rIWb{N(4Eg_7IdS zQ+aqE6zU`{?m2ECq!YE?z1jNo82LGcCDKc(TLv$Ip~2D&J|J73G-DLt2|+ER7ZWB_ z!^mSl;0=m3mAFBOhd0Vp?rY1$PKaO*6E<;Wb&{k0E zW#pKQ*l}8sc7E zeWSH(^4!MXjQ*du0f_k3)C5f6+E8tSV3}heyM}MxZ1(SW`OXV5sUUWc1IVw{A3|Zz z500ZBJVH+|ntx~0BBDJvw@UH}IUCouIQl_Q$8?r=h;IsrlQr8QgX-#4HFR1T~hFLtxUn;pt;v-TSCB zsgjJ3=k%!)CoUmYVNMQf7LxLk1_z&s)=jS`j)k$1 zEpv%gG~Hn1nFYpyPsT9?sWh(_9}wp`b6A|5Fm40xH>dDlro4bFA%?=YwZC_rQWb7* zTAW^fQBf_N4cuS1#7;?TYL$dm14J!yd8$(yAfOk1@Vbx|E=lxhP-*nPNA%Y}JU$rw~ z0|6K6EnMQlI$;J+h4LC7a3-UC_8xBQkjzC+4l~Y=qx9oVFapdkzK5*E;!#fpDLRs) z2*eInGrn_{xJUa8eFzb+5sA`HW3u)9`5(A4yuFveQcNqR^%<;NK|C;N-I^fnn8z_wv&A(8$)zR{#OQxoHlnl+rI2{#r zLvJx$8ZeKagS#2Ofq41)oM(y$(hL7Y=|uIw0N-4U0}Bem?s}Fj9;LWBEag~8 z2rj)lq2kFp7cY(p%eS+$%L3CaUHai#c&Dzih(#+ZDu}CKH%? z&d2RfSZfjQ_UkX)N(D{bKqs9phJYgopJ1J2o!iw7xTNL==}<9)e6WXsMQbSyLzIxl zj!BDXj?rAIO@=O0QpU4<6egYPOd_hbk>(0RH1G|u%(N6%-ZGrdf9NOg4n5AuEp;LN zOCi=FzZrR&=f$uOA1G{|xn_(bR-1HiAbw#^SzuG$5G)KU(8Q6sahlD^ZqsC$E8uR_ zwRy>d=R$t9_l%=RZf7jz6hak+pBdTx31$v$8VnX49Y1=sgS8QAxnw&-1B12i?kAIh zI5=`8QLHypEL?T-5!y5Jt43H z*&IZu0>QCRj{Pj`ieqtOHO(>J1`8mWb`i;TR8W|mvvz2x7$iG~zjVWi7lm_?X+ZKm1DLcU64EphMEQ7iz?&1X1*fVD!az z@E+Veu<7B~Pv95%iUutW7{QOZb1_JiYm|T-5&4$L2s}WgNq?(@GBkB58+gIGcz__8 z)KQUMC$-x+NMDQ`N2+88NP5qd)5nj)3()pz>d&8}bLu3`5uxnb_AG-=wH{B6TSDdj zwYh~RShAh`n;m2*ntrMYtRDb3651#kg;kT}8vYli!eB;Pt{(abbM+>T7NyIF2Kz{D zE+8JAt@y;xB5CtaFx!F)zX%LI*|^0rFnX1ugyz`&A$dM`RE?KBIE*e9C~%+8%gtKW%kA`E69 zQY=t{pus>#){hbKHaEP!LRrwQK}%p}9mFhv5J9Z11#?cJ zd)3XpAcOm=^p(?7>54E|jMiI;hUAMr0NVCHyKf`&gCa;Eg5wrGmgAH88=-)a8-xVP zKEBO4ar#A2#Z0xN7VzO~cz_cV5*|n|hSWmKCn(6wQ7{jtMHIBJ85xJ-=5G|V{7bYw zU-#w-f;8ipjj_<}p||NNV_-%1)rqla7IV@6z}QrA^T)b6q-aSGMx<$o#*r3vdkh&p zdQV9$BSxV!1bqn3```-3bBb#yqS2nf3L*sv6&%x~?%s9h0Q{nZWU&N_bH}$aQnXNV z6U}Z&Js;;buPf6$$=;7nH~=k7X-4y2(0_5kM<4@ z79kLq#CHu{!4HOm6%vialbux5IItWjQq3=ph`&S4;Nx4%u}AmymriF9WuNslandtB z<{pvNBvruhG75o=j9uBRbRdkJXBh}Efr#9p+kA);Lk(T)R*hIGbu)4(w~FYy{g#LH z)b5_@&|-3uve9zhALByYmYf#+oJPBvk?aQlacILe?h z{7n#n;`m#C(w$O>xhsCR(8B&C$Z_$%c>M68;1B|#a@>A!`8K4iN}XqzPeH5C*F4Yp zWiaqr0ejo{#Ux@w^$mnbLl{$Y!0DW%{yW5>HR|aJ3WqpFK-=ii%Q2Q1G)NF&SbEN% zpI_rki36GtM%j+2gCUWG%32^+C;tfQOIQoiI{vxJ5wfR^gUW8SKP40P(^4CeIlZrT ze|oG|y%V9hZP;rnNh)rRE2xO^16~;eo!t%$H|-GIU2xs!FQ}@mWuM;RloVecoq%xr zpL)WzOFEEeC$ESDI}xA^jGJ$nM_4Z`!rumEXf?*CFJHC--MKfEX6R(Pe!7$Q05-`| za|A|}#Bpj$QYVBF<1G2=-TG*=gpC;Y#d zsn80=!Y5oF zI)C|en!ucE3>G+~te?Q3-aIGll>6t);E^j==4vV9Jj34r^$bqrr#yJS8WGU(;}IQ4 z^Mc(hy5`3CMdi(YtKqDc;n3lx%-Y`N9$}+>8+PX}6^aL_l)oS>kZKD|HDb6@jI~PL zYE}yLX(#zEw2JL1qO6Fp!h`IA{<(g=ur<;5ym+ z3KOb&po5b-?SWF^wALNJs1-LZRXXh>YZ=@m3Oj{0RvPf8YUisXk#yD z3GN)o+diFX3S5Sw9lm59L4G-4r4-xS1La{Ypt~V`JmWXBVcq%!{l>?gsaI z;)D@GLBZMc=1u&Qgf{e@JE)Q6&sD)t=+&dT;5c?xbg)}~`hT_* zZz@pG$jTYqW5Ghyi(?UUw?^v}Rb~=a4-t+&tK}iYB z8+G>Ty=Ui6)~#$o+M1TuVTf1y`5*!v2#~rQ5Kb}E3d2^9oLd<0A=K9uogdq0njO_ z@c$u$-+pV3TT{D%U_e!Q&cE4L?_Ee^t;E_9l#I>}{O+6WGAwq;ux;%~`TRD>T>ABF zSpYI6DcGr83HWUgRU~6OgcP8?s~N_YM%MZb8?bt6H$i1S;^t+`mvdmSrjj83 zV-SmBB9bUbn7d>#9E)6TF)els^GcA2{rjnWgmlP# zc7my@W_eeQM7My^Uvo;z^g)$YvO1)Ftuu7>~SPSYY7pIraV(ilpe zMSBc}8Ae!QhC!i)`|!@71AGH0e@x4v9GO-aIAq94oZ&ItF0R3%5|-P$oZ)1CF=0>i zAOPn*qPK~e73+-Iw2F=nr%mi*NM92*V@RbCK)hbTfDC30-V{`TsxZwt?DyKSjDYeu zFl0jN1B=yTn2X_xMUSbC_yuKvO76J*2PLzXq?DAiFp#8S0SOI9!F2XkK~_yBO-K$F znf$eP7Qq8dgarq-pe)}G8J|pb!e5rI>;X(zu|l0jA(I$S>$AdfRdNz-<#%xZaVrW8 z5?s>DDH>TEMLs0K@Ls=jN*R2iA%ayO0E)NDXrOe6s7H^rye<9sZ4(bsNkJ3U$OK>T zs=#f*V8M-T(2#iX3vCA|wKZ#KM+jooDXD>hOZW+V!hzb_D=N;mk19eK9#ksPs{0U! zQyG*oZs2h?M_LXZFR94#S-^~3+f3(7s99oP%gh*pAT9>bk2t5~{qnQ1x%1@HTB&&# zfl|P}poU=bBfmI2{M8J*Ag+s=<3cHjECWs!iieC-?Vpj@O-)T$s>oz%<$wp|WQY~Y zbtEQMnq6^aPd3P1bMe0+dOSI;wcPdN4b=$#b&=Cqe!H6M0T@SlQQbhJ?Mun$WV%Qg zi2%AG3MU?-o1p0L1s9mNsi18`6+x6YX7=ZM87~WA2AkHcwWLS6AF7DA-)-T)RGhV%K0OVLhGrICf_n(kvaz+5oBnJnFWZ+a6&TgEguX5!sRZ z4$$a}dxflvO|JYvY5)WPjQUsRNkQc#+ff*I*=vvNANHui>jITuPR<6Hu6lK{I?AlR zmu*qAlAE}W*WE{yXMcYC*3r%mjU5ZSuDkmzTsWZo41L;9ENFN+fC?Q7Mn?s2E`EU@ zFEavwNT>1{Gz`x$WTG=Gv?$V_(>1V$ItZboYCcQ~>RX-?M-am>uLAUY* z_5+N4*At&4vM7yo4k5#+{51&b`KNl+EIT{BfM2WmIz|ce937FCQ7{cqQ2whEwOmI?1eHekLsqQ)^fdUJ0+7FO9&MM1my?X!p^hJ?7#qg|Cra;gk$KCq#07 zpzPQwio}Ff)GpV9c0)q5VA=9Q?C0rnCNaG0uU~ggyT;x@WYV3+n}d~U-jM;UwfO5+ zu2{k9cJZcm_6Ir-i=6}M`0N=Dc(i8u=HZ3zc+)};afIne5POI%G|DsjBarz-+C4=|CKl7;Rc^tKgm%g@#97HfrcRH1j(06lz zl*JMEjI2wSsC>fKeyOZvB2t*$0PCX%Bh|rp!s&6&6vAi1msfe4pF4}kU2HKgDuKJA zdKEcb2kCz}IG{H=oIN5(j~IbqwNK9jOal!xO;q|>&q{2VDsT^v<|X@74dxa*&PWCqKh>!I6Y4wEUh z--PpWW-ELr6FE2CS;TP;(F5|D_g}u8NluOdneAMzDcV9kL17}=Fbf~lls{ZD;3HW) z_@6F_Z6LGmLt0%3X4MOvg6`~dIv9A4PiA2Uw?VW5s~DMsNPqE&_yJeorqK;2h+p52 zBTJxcC2X(()nNHPG<()s?48=qyOBReC_`5d{d?YiFif}94+PBE&fMtLgGM_C_^g~@ zh3TI0@@fEzb?&nPBQ#%74WzO_Y*NEhnjLEYMCbME|#Ww4Bi9PE~%;=qJtv4HKTWz>#J<`#T> zHGE3v8~Bh$o?IXZz?)YhT`_^L56aeRoGvFT2`Kx0L-Mi3;j*#lW}@8pwxMaWF30|= z;939lKjYN*pEnyfOq$hLIU(TBxR#;CKNHJ_yZ!i+`s<0tC7n|_l63Rdm+hOB-oKDc zUB$yh2=KXamA@43t?b1dY5V<4u3};dv`tHp?;Zk4lAZ!i1$_r@9~5OSX5ELJ=*1ti zfHT}RT6KRmwBN}^XX=*1d*428zQQG8`zg}`zc$;B=C)%(=W}UiZkUs9AWh8>nGT@q z9vT|#K58k{+WJZaqNSN{=6@r8kF6bk*15!>%dy1Hlh!5{^N_l=bZ z0z)eL9lz^gA?Vq(Tr9a&-2h4r)+y?lJo7lO?APr~q+`<44$_9C#0|I`{o9xfiS};C z$6~aeM1{VwL0?O1>4%xeH8;-Ba0*&1OMUgnL{gQz9Zi;w4!#_(hzmfYpulUi@S&Jn z2L=qKHMB)AQJQ6XzWb8s_ItLT)A{Rn{&2(lWVEbFlqD(IQ4Z`o^AR zuLu*7tdIkJB-_!cfM!pfIViSxVP=K-;R%{C022(427&X(`iPCMx#=N!MU+Z_qM}uf z09H}4xyd1Ux$B`%8#1p8^&JCK?UJz_HI3el5@U|5YY@vTh#`k3pm?E9#5tlv?{sPy z@%aHwkFbh>fcWR1k$QSfuMNy1EFYjxY*Vyef>P!-6Uou2@jyZMG1gcZg9I%4$%l`R zHgq9qS!;!!-b0l*sy8A>m=lUvKmrY6gefU0d}Iv03f)JjN8y@EGKHFmv6Yo3&ps|I zBMoe9!-WudaW*)jEXu4v1Fxmr@tBTGL@$bElD3{bv&|R+jOxwEx{R+H!H?cYEkv4I zr;tSwTe(?vLLZv-Fs>eX--aoF1fJ)!A0Y%tz`7ktAyX*@ZS zyi#W#2O|*Vtuxx|?`Adv8ie~|HE9x}1R0l)^xzp`Ztsjq)TH0PelfDpBN7J=Qi&vX z=8mBDSVi7{CpB9Z_hf?>{&p6xxVjFqkG+WK18 z-EmR6z~9PN#$Vwuw5pvq6}mH*2-`1?u@-m1kkfnvzjx;G)ewmnFRt_O5Spv!&JpD~ za?$rT7FE;A;JArfcv*sVM^X~mkB_}QAO1>4&wF5dG9`b!u!&G zutVF(X-FE}+NmL+_E7DO=2)NFu3R}>N48>tuc2DfT4p~b2Q7p8XA0Cf4 zA4@BM@`zmN@g265$?k&DOWVpK`dZk*Nmp#A8$inKAr^ zBg};OKIU5#k=bcjo)|cBQGv@LBM7 zVgd|ngk|W5b_eAEbtuAo;*Sha?#N-waAMn(K|k5-%gpy4`%c2;@QSyNg=xiYF0lQZ zt<=Vhp*Qw{(h*S6?LrOwIV zDe4w(T4KqTyl#AGZx9bd zTeFQet9YmYK{4qSUfd|UX?T~@Ogwq=*^3uLMe<~eylC|vJ#fU4J-ip(MK6}`4bsjR zi$6Lt!;J?8F8RFmZ7r31SenCdGlrBQ!wx&?9+`QZ$N;Wp^y081nlM;R{)zTEdbH=_ zFaGf~FkpmtSZdGF$n|NSwTzBgAcFCp&00rYLR zJ@lyx8$VTq6;%t16wwU^?> z+PmV}B7&1BS}Wcj+DEbmx=B=WR4BNO;E&FRAEN{ocd+M$LxOHsZuby0$Mu+G@un0v zlL6Snhl+{`Y6hQSl*yv2=rf)@YkqqH91ULyJ&q^U{+l<*`a%YcNB6{L68Js#1L#a0 zlXT|HRfHr&n+oMLOU>-{-?aBk!P!x zqvPJl$f=06C>QQi5N(H=dPdM?Oqbyuij)Y;6a-FuENC&@?CiE8wQ@SYbdPSv58f@5 ztE|Tr{*HA}nqPj@554B0j|#5S;p=1t3P{8=b5TFC4INy7NGKPq&3N~h*VWCFm2QB|qY=WjmJ5s7wX6E`=ezjC^&YDbj^~vN!W%Lv zRTXkXurM}yP({PKxiZu3de)=k&91sC3cze==y#Qs^^h6-D6SY2rUv`FT$)KSjXy_N zKH`L!?WjwZ!O!yvNtaVn*|msCQSm?*{)Y04D>Yci>fnHd3VsgzVb`E@0Vd|Ny;T&t zW9ZEWbXpi~LtMscilm-eXqDZ&1+0Md5RMs)BNs-F_CZ%<*93J-zM&ePc<3{_Iqn$) z<}?jaG4do-6c$op$Vw;Ydvm|>Y9n-s*h~GRqr#vvl46FcU{+z@1sPva zHRS`&ql$fD@Uq4L<)CffToe zIarO*jW)%Iy3@eQ0-?8>B!3WK4fQ3agBlEXiK|UU_jM zx9nArB!Esm6*&sZ`1oFOVf;Svu3PkoL0Cx%!6Vs!2L>6A>j#g2N-BF`pe&)IiOsYc z+q`f3;l&5b+)Gszvxu;Kw{|aJr5yNCbSD%yW9)m`Sa6m_nnjYF-sO$+XcB9FMY>>y8&14JK#-Y4IYQ3`VF zVQX|=yV_=F24p&;z8R(*f%;idb^ zJ><$RZv3uC;9-$r%{J3DZD^PWY66rTvBu~>`u(Xqt_>mExA>0>g( zh`}THok=Y?A0ePN#sSA(1vDn0Tn`N$KYxDmC)UDG#DtqZKTT|f@Ux7dpu_FGy@#^0 zJ|zzY5r7mL9}WrO0Ti_H*s;`HOqAo}!welW&k(bL%$aa(A!dOyW-awH_a`&0OMo)a z*QY^p=iWWiG3gN<0Cr5PF(+nCS!CgCg!ItKoC(J3xzoHjOazE96k&{$U{Q^xleEE- zNoQ;#ustDrB6e;!@_XcY9J{jUMjGtz%j^;wJLtOV2J-Gwx+c{P7|)TgARWlOE1zfi z5bf2Y@Qi@#o8#;ZG^RixQ@Z@G-rhVe$A4}AElGnYAxSDpl8_V`8YMJILQ)!#nT#3I zs6t3Ghsv0-5=zqetz@dqbEU|T3TdEWzpt$IJo~rz+Ry9xYxl?cTGZWrU)Se4hvPhs z<8*VIrWqwL*sH1vD!xOnfj?hITg5G)r{NR8p=ImpW}oTt+la~Xh3S_p`aL>|CqS@JID3}i5OgC6adA9a z^%Qpr{$Gbj2HTxub9vV5*MErEq!DZ9e2T%kUCi@5;nj&1IWa%;FA1k|h<{gjl_I3q9yK*R~w%rN+3 z0K|lWu87!4gNKFesS*?-xE=_F5R?aI5zr~9KyVU+SJnJ8+~Oq2KK{QPP{JydUM9zM z!6k?bc=kJ5tASctxX*Td5zb=(mI}=PS{o6OkX6?I7!xBlQ=TeedbUT^1Ya~V3N@32 zbUTkX%$;vvzI3|b%xwnj#2-sw-=?KeklpfY-3)ijcW?l2miNVjA^O%6+2P9*yRZG5 z3&67~9rC`qx&=YGUI!5mECLTD`v?N`0==8ebKrYGBc)%V{ltM8+2tqdB}#7J+rfp3 zPF=v?;r*dd#Y&g=Lbeaa5~JmNr@*Z2>~icy&Es?j50=>eGvJ`tiWOT4*BmgeB6cx| z%!smVm-B~IsH0%rwv!O}E@}Q`D&7!tGAbspFb$}eOtTJ{$1!~XJL5TJ5YJr!pRv-A zuk#@MPMQ>g2$Du6I9o(}3CA2qT6$TThd&O{5gUZ<=SFUaijHL_5;2{ zG^H`_>CEP;S+l$rEkdgt&I7^OWvGFQrg=OQqqD49f5X`g#c)hwoyQueW8UQ{Hz<-%6kR(V7|bx zg^0)JmROHn(=Eb9S>@xB5}>Z`yDi}#AVZmAWz}hozPpu3Wfj#Vh9`UWY=GZ1MFZv- zg*xu2XG=(yOK5{=Bd(qk2~EH>4b`3fRk66P{4%Kd{0xH>{+Tlj%@JT30oc#_`ul~2 zk$%A~o07)U?{I$rRs9~Zf-;FETvV|9WP8ThSoT{$)BwQe({Rkgckbk6_&Rp{I8PLv z9J>KFP8}#GNpg_tsmX2_u>;p3cQLP~jKTX-)&{rTf zP_F{2GPLg0)4uFo3@ai6O(AS|3KeMW;5-4V?lX4gi2`bY(xWDaiU}Jt-UxOoJk8h& zQ%G|s5cJ_(1?sr5g*(e62R;%BjFO$hfIc6SYm`=);LtrzV)9X;j&Zuo1Q;evbbzp6F%lhG7*3wUtGHuW2snsEy>Old^vCkZ338? zkXwn+MogOd6LlInnBd0|qGpBV$?1Wc#iPK^&6aKo2k>q=J1xdeLtZdRA+2<*9ze;+ zGvdg@2S*!vW^u9Z%8)ifuBn{#ZcAbeq$V;p6)WNTI-55wEh$}|y?dv!!{V=YSgijU zjusCq4;Q1bC66?y&MBe>Ss!G3&E2p1g}^5w>+Vd~a^&DakGzXi5HtmQW#|R-JZm;pzu01I%JP&_>&1x4fh8I)>nwIcwWCE

yXN8Sa2fwgl-x*wPMyB&$A1HEL?^XFp*3=rK9 z?8S8ri?8T;k=xJC%1o^j80%Dy=gN>KGGQ z{5M{Xq{{+ZZy0RI$Tzk6KJqlJUSE*3Q;ho3@lpFEIaMRN!hRYww@&|#pPgnD4}B_$ zvZ(*I*sEFo%orJwf@IWF*CVfBY$Q1H{RrQRjUYX?-ik$hl-N}|{YD0sc@jSea0d?H z5RN8z9@H2t%3#=fCS~c0#|`GU_j8?cP-3@bur$mNwS#a)S;*1f#Lwl%V+#yk*NOIB zK6?DHVcnc^-2=L0i*gq^;qI#~)FX+ngj)C>#E3#A`P6NQcNgO~vAw7&B1-MRstG?- z?|Ny*4mZ^eTb`;cflTVSLPUY}nMZ)<&7X`2p>+ecU)g)RBs-NUB`b#fK_zIA=&3M+ zAv)dN{F_;)%!_|Xe)Z)RYwTZ%)drruQJJ*u( zM1=F^%S`n=bmI}T-lRV7*JProK-b95O@DR?>sbb#*;nP=OAGFzh^Qi2K2yYBo#uHCb-6_>FXzUQK*H|anRS8kI| zB&uDnb3QYLo|Vl_T}NMu2ok#7%M^kP3fG%k|J>!M1QsgMx=Pf!>1x=eY~ymf4dgE#$VE9>neARDArM z;^MuK5ya5bVqgFsK}ly;CYT$WIqtPkCoD1$g4n%E>S$Q*S=U+h6k#5RT)UpeOQ2C(oyiAYVF!Yyrr_NJyh6+aPiH84?*nM@ z4P0hvs;-`Z%FHqy-7>tYE@#oH-}5Ip)L(_PyM<0s4MheQT>;y z{yFXNnHmvvgCc~I7>tZ)WUmFSqpxH#9VL{0LK;Okff$5__X%iHR24kyH-I!cV7|uS zryOH!1eXgZ8NDdN(Z$3>Ig5+FEODZkv_7aqs^S4IYi)&i1tQ)&<%Pf0i) zX724ufk!T+=?=)jXv~Eu z3k+9)rE+Pr)(a~aPkk+C9cp%V#U%yr>5PKp)_2_yAc!tW1r^&y1$JwkE4ysv$}92l zf|aG*ju%{LXJ={s%7s_h7W)lf8M=5m)t`UHC!!LgnBk@}k$?>Z%K)!_njR`|cB!&u zYH^p$Uq2}cgzCnU)t{jSoBA3TynE+| z;98J-R!l=LiZ*zaQ7^{u{xfnpWnct&CwKbF!b@SN5LWS!7ztVAyT8 zQ_9%4ng0H;YUv63s;Dp>6|O4kU@k#|dDkoi#sdc;;=Uwkfc6ae|y@S}1n za(Jd8f!V^egL{VkH=y4P=+mP{vGV6GZ8~gVwgkL5yaMxMOt$bDIdy6VJQCnxit<9Q zV6sbk9x@K%83 zd2Ts`Ha-~c2WF5OU%mQ`+DWr% z=~Ieu1<>6>R3!n7`unn=-$kdvjys^ZVNYV}0;o zX8_BhWt7}NTflykEG@}a!WtPsJU}pgr*_!R_>-;xc7@*6a{m^b0jM_|HM?okRQvVo zDkIZE3+t%8wXvpuOi>Ynl}neNJbsM00M!T0GW#uvtxS<1Lb_)=V-3b2MlJs#tvgT!TBP9OUEZwfKLJlk^~hAV0`b`0-t;rnMl)iR4=Zz^bt6KE`7UTM>QaSCn?-5H{(ref1!TNa)f7mJM9T6<604aD4$bv*)4lj?h z3fmEXeR+y4x!|9a`U z<%Mk{F#ba^$odAY$SKh9tmEuihLSbQ8!Qr7ie8^G32zs)ln`p@H7KTO?=7_t05`)w z5S9H#9LhHCni$P6k$p0N4!42Xi9{xCO#>CezkT|| z!a%k{G5`WsL9p|?ug7#ha7UF+i2^2(hSaA3O^l-NBK-fmS3XHqxLDg$m674x*e; z$MA(OENwfoiHXIzxdX?IbN1WXInWd}JZpj(Jdv7-?)}HDn?f(Q z{FxK8;unJiX)4ncaAt@}l%~Q;kgP2H(zBWCF~#gqEjCj3Ql>y?gF*N3AsU7e;MLW- ziv9bupd5l0Kg)n+?S>5}Fbt8j$1)-W;xxljWa1Hmv3uRmpZyli)ID-vFcl$cFnRYdWT_7@wu#*KCBg@n4xtWu zOrFoa&&7WN`bX-9J~eqV74{z18yd$27F41AoQY9oLpjoT0 z%|y&Y%+yiXLOn7;CFaBl9Hv3FV4QIiXFEBeHBjl>cdBL-U?u!;ngLi6WNs{660z!m zV@xy#Q9`>3lal>HYe1F==U30#sYjA}AQ6LC;=OU-D;XRPx`m*Mnx)k2%(_AxKRQcY zW1>0H+&pXiEk4S_hbzgZbScP8fe@&`q$DL#9jc_%=7T(PLKx;;zux;F9XmEgtXsz@ zv#_+(&DK^R_N1jzs`gtR%;{!2_l89cAtB<4?o2ble@;Y3mOOu+!W{&N(pPe&%%VtJ zy=RR3O|OLusb9ze^?zzJ6E0nHxv{+O3rRBU^5TU|YM6-9SBzt}%Vh;->J(V^`7^Ec zkj#+}IJ4|Fsi7Cp+(_*8oWphHP-r7W&7owV+bN%#2eZRc2jDF*_@ z(&w;e`_%Sk8bc2?<|0EOXkVy*pLNTPwfNS9x1q|u?bkXE0m0$P?OEEnS+lU4g&|!k z50DD7b=fU*9##3*uM|J6DGHZkJ50OVH7PPk$cf5oSZ`c*9%TKjadtlbF4Q@R8FKbL z0R4CsN&Wxw>i)Oa_m4gwi)L6L)SwObUtE;vsQvBpXUa#6LBl1yZ@P2|{(ivhBYg*s zIt!L|DSQG39I*ZO^HTXc8{oDy&7EqOsS<0BG%QeSfpf*;LNK(-_Fc`GZw7Y-lAQd3bYh&+Sw<* z0m_~GuW|uFnv~+&OZ)7$u-o`k%_nT zjRc!8;mpe(A4R<974!<13H_=}vA{ZEYl-x76Ype(M;IPm zS)#Zgju|~##ixx-EuiCHzR0;71tkZs%03(-_kNF2`dq@?N;hzXfHS98%K1Sbp9 zB=}H1-rls3^=z)~JEe!D1tSiI&0ra5;&{H(b>*7+&(7~tBI zYO2P)W=wySi3`^cautFaGG%rVlMVp<9TxAIzyPH!4{BEi900i_>3{{p+3Hf63hi34 zM~{(&1e!LU?0au^^FV?~*b_$#48+Y3kk_a&)Uq3y_bfQaJ6Pl6BiprWRO4Ohbfi_d z2D5I=R!1Q?x%~S^!X+hGr;&lAHMWbQ*n{AY%ii5m?K|cD>p%K7uf z){dfNU0l=Bm1`%6-DxGjn&%hn!T%n_RoT$n9T0*m%-HA^9UVHR;Oy-*zmZdjOw9Et zBi+^tuKM&V3?~IhfQp1QO(JQIrYFDRTm)%SZP;(5XK?N@82!LEvf!C%l7@UU%^`0T zi`Z)bU!|qOw$pXvj?-tnxrrH@z@aytah5fN>DPdEu(+)*FBfKm6I4`t^qB7AB9uUX z?Q?z;)w15Bjs}Hd!~?Rz>kf)4WQL5}EnpQ#nlUKvIZ>)@Z7M|$0Z|YE;5&zMw&%`F zCbXnqEd9CQY>_lO;{ZE`T~=+7cBsrX)zv+THUzcK5@e$v05PzHMEv&^R@P?xc?j9z zT#Vnoq8i=^7W5anf^_>#>X?D>T>arga~**BNFWPz6=_!HrL1&az4|+n9n)xuOl()l zJE$l*N{rDMIZ;NFa0MNojm>^rG$xXj{oWVNy^pD{q?8md1}*^lwarQA;k|pw*RRti zUgst<2jgV1iEEmgoT2hVauiJxUliGY41`VEBr(8mvO0hlA5HMCAlYy-X+s0=-zAxY zYOG&>9g2yQj#md=9Y*SuLgjJm?mT(2-|igqk)WWsx#{*&)15&C0q z2BRcGoa%1ft1j|dN(#drk>D;e=S{_?QDW{&rvqVn?KW_du;UBw3?2ns@)lL+%I^)1vb`m2dnH>`(psks+W}S_XCw#%S&icj(s8h^d?s{gXi6JOWbcex$ z+WRo`I1)Nb1i5J}J7*Y5Xi$j=GgTvj2l^Sx@A$`ESANIoq$FIZ#nW}+p#UR!^lYh0 zq|j0b7Z4qcJmhyEp&q~qrelQ<`06sh38OyfP@H~lF~LiigeWTqobPO>p1F%I?zF=Y zrQL0<#(D_uIZA>ZPk0_79ri4tF(H9MP}r}_Jb|eotClg2RX9AJnFX};FY ztP*>6nW{1Bpx+a}D9>eGTQw#!MC@K0J35o-m!W>cmrruK|6vXB7^W9xY`9s+u5^4e zz>A@AjNSl`$vDPj<(MeO16ZEor^H8a)W(KQub?eecLDAEk<;IhJoDFd-SFE^J1p6- z;l}mr22NXuj6@wq3}toMMQY{n22F=BKcjLI9Uljd62+#he?15@w(L0lZltFLoH&(oN>PLR(O((0)%pm|~fE zKV}+mV9n!BpE*M-%Cq5~^a4nMiWTIJ=>@b4_)q|)lxjTqGiUyS3hf?pC*ltdN#=Mw z$3t}BnFRS8F;9@ea3^>GQd66F%e)E~UZzVfb3?emHk!nXfILs`8a-!{6KKz4QVfwO z#_#tID3b}h@NcEvV@wA~1cXbjC6s2(ro4nYp~@zQ_fK!9r&ALn-<#{~Os51x!<~ce z4YUcMz`p>(-%L%tbHB#Tztw<%0SrO?sVKvcY7vD1TYu2-L7YGs21F1(RT;r8-r!7I z`53~MD^)Amj1|k4&DT2u_5fvtZ%=-f#(q5zR7hhD&Qo(*9q#=GPG4|oTlJ~JE~Fp$DX7Zi?p zE+8lk41F1;FjXjhnmwB~cb@=&qRB(4HBCK|R*MGv!?X{$BvwS22(h0hmd>4T608tC zHypz*p{xZ)A87`f(@Y14SAavjUvMvwNM;uJ+BoSPpnIG@!6Ae-m{+d`+1AD6a-lf*EZT;8 zijo1)55$K_Ty*qnc%dW-s$O~%G_=fYev_hxxn)2@EQOtiGLdwUVn<1Si(-*?%(vf5 zi6;Yhk5^?`S}i9UN*MJuEfRYW7?ki$yrJ>#q3L{PhP!JQc$g& z%h?fxu+8Ds=ijBS=DC9F)#Z|~S*1BsY^m4ZT`bsjWKbG#7#5ss(F(^->S%G9779wcWQD+)4i{H7^3URs4_&X`#pcOiuh0jwJu;1!T zVy`v_TnycCx8hFHhW&fzOi^IczZ9PT|9pi1`oRC8)A}DS&wu!C;a>cIUFrYO#U1J* z7GCBLL2h7F;Z+l;!e<;l({LkeZ(kO5(jM^e$)iU>3&lHbjJ{9CR1LOqM2}hMj}o1= zf(+-#O~wdrwns#SiT~HP&1CunKOJpYOBfZ#)pi;P_-k3{ErsH*R4l)yWsvvk)n;+_ zyxSoXO|YpL31q2v{5$g6($Sz%%C|@07Tm}^*;Z;R=)FF zOuoHc=;HXf-K1@lZNc70Z>LPPO1L7rQ+h_v{ami8BZ zt?3)#E%9dgOTJc1bo}Ri_m2Pk)A21_s*e9B=hN|@|HZec)pUI4zh3*lUnQWnM@&(e zd6R0_I=6YD{v|!y{**SQY`G@F?<-BD(`IDX+ItJTo`sozTUJj}EvZT3+(+3L>6_L< z0S)y4za)8N>FOZ!g$DzCm9S+x99-BRt?w4m6(!&~mYYaROw9{nskL@;j#o0XGM3PX zc)S1tf?#paE7||K3O1CGWMQX%&l^Spp^o_l{%E4Sj(BOf7cI&?-z-K0 za$y*~mez<71-h3g5=a${KNGf;&`ojNO8;B}$z$Krd_`1^h#F$n2xg5mX0!rYsmNaf z3o2T#7{LI*Gvv#%tF!so^yZZQZFNV|gyHs|#!VCE@wc?FVc%F%@&OzMys324nO_YJ z8Mw#5{!pC|$`pesoBv}~Cw1RNqIHcIQM)TEYeW??!>9akbBnP$)BQD8p8_pC+a1;^ z%>#86#CM=Chx1QhTk@X3_Q3(k-qgnBs%L3)DWE`K(W`M(X){<_!wXE&p!((cLa2f( zWv_i?SfF{V)IkLg-Mf6@0$ft$#~fK8{ucw*Z`gndDbi80B`*o9a@|nqBoq+)_wMCs zlJDI+Pj^?ICxZ*F1f>d$J&#&;X2UyRDlKe??Qpt-9*)5WdJ>q20=j<|SD2{6OF|u1 z%}3=)fK9=SB)1%BA4rldlyKjn>f%_y8%D5(B2+VK3G~4M1Hj%nC!}6T!$6!PWgv6& zUp_4FoT7<}wx|mKBn}2XK+s$u5b}|;&0qzhxCZ6fxnl>WKbT1FQZYjFe+fvAkqs7e zc)##os5iOS01cEb=#L62pkJfQq03sg$9{U(T5Lx{Stqe$2NSN?i*;7zlVq1bfdyA(c8bswX(P&Eeh{4owJ^8J|9i%~4)b9(4k9eo zJ$+p#(Cu(B(cR$^y<`b>Ck+Kwg{>oTZQ!$+%$%K8@%FiMgSo=&w_-YX`7*8|keILHc%r5jU>?UezF_s_*RQ8WJO&$N z_X;{gdwZ0lDt-Di!a+sAiol6Jo}v~-X4mfL;lC4pd48?a#BnS<9F$1G(3k#`pO0>3 zJfH#;sU(5{qg+BMnd2507t|6y8q25-?%yAK3pSOtqvL)sl$kRRgofrb*CI4+o7T!7 zrY|8T>{c>Z%Jh$G1JP>imIlEmDEfghl(n?9-#}~Vu3n-kKYbdF0B_nGgk)XCL&=mEwZE z0;hrL{8gd`A*+-`@)7;V8wkgJIKU7*JcS`PAvWM^{(EGa3Cq22nSjiP9T zipVIG3`hqEDn>&;dd+O+MC4ccJ}eyYE60Tj*pD4*fT8Rw=73HtX~CbA#+)ocHV|fo zjG6zsq0mS2XfTrnuV79G^&F4{A15vv|I2x&-6jDq(RT(mLG@FWYzGM{D=XmGF_#r9 z0Prh#b3_2U7~|t7PEcR&d3532xkXM+FF?Yn*J#KIHNqSU>dew7TIH@QUS2)JnDj|J z9@{f%$y}2>^B{CX^E}zXrv%$nWk~2#_$$Vd7pa>O21;^!kK7Q7;DSq7E4qvJKBE$l~gEuh=ERs7~~}0K6Bj+ z5>HtItQi~4pY-$`43Gde4qWE9;0hubw$~qng~OsHX0H^KC;(uluo~UYu6w8lm;t|y zzoyrRJV#v3^r=13VE`uV0f+qs*rn3$fB6OVKZgciAxxjCy!ho-wzeSjXq6#6P@2%G ztI4<5&OOPtA4nFdsn~F_BS^RyNQw;%u=2eHNCaPE1)hc`e(C&Hm~a zV~%v#fdGI&e7ZtCpnmXsuHLwjuggkqVhHv$OfDcjjU@m8QxGILZ{$K?Vmf6OM!mbS zq8|=E-en?beLNK*)`IXb5g}OPk(B`|$ml)kWi=2V2sn!{1{fJ_MYc=tj3mj>uVo`x zEc~CMA`|d$_!z+G+}se^X<$&6mS;)S1Oz%R3-u(1{!Dhbp$LpHR$p-g=ziejJvPWE zkW3^cMViN}x=R4D*5^P*r6m_g?1W5>tR>5T~o_K07#WvOMDeT@tA_b}EO^b^??#qhvJk z5x9^63~`k4otdc53qLX>Vd^HMGjvUek&CV^LdGFbF`z;*Ii}VjUm0=rcs`kW@PwP2 z=MX{}ZyY;v1W@`GH7Uzaq0oO~vcl8Xg0Pm11LTWBpy@~_Fl7XMHXZ8^sP`b52(=_@ zP+%w_*MG-2g_?Q^V9eqQ=y4o)VeuQ~9Q??>&t9x!azkap^TFg=sE-9i`K--NZVx;k z`%3m)f8q*rwsmECLM}2`h7GuO$76*gJp4tGpJsT{M zoN^;*JPAz@QTQ_jK>SJC9VmPhY`UIkyM$Y?2E(=z>)PYw~!eDI{66LK4E+mb5J$f)KBb8;Af1#wrV! zC~8M3?m|V$Zz*1U&0~`uW+G8Ay7DKPQkn3BJOl*P=;(f5T&FuY zuo8%^DDhmN@^Y4is2NTmZ(zhuDH0ecibM(X3jV+hoxIMnBv^kKZVF*NHda_!&&S|e z5VCn52nz$`cThnn7*F6zFgfGe;3tRy9VG21Rg7`(Qw0|=0k1p{|J_)X-~ul z{7QkO_L39~;o){McbVB6EBe&5wZGKV;8F+7V8TVbmJ}CPPIbfLoab-eD9g#kUcNjm z{UY7i$ocuycN9cy-x&g@W8y@?j)5^Lq#kHKtW!_{ifFw>pCepo3kz)SIHN%2iDDQ8 z0UI5RjO46v?)`p!p+sc3Nkd#qU;6DElbzu@IxTCuNknp6HW58x8FKldl#}pJxx$U5 zex~6<^M2||BXN~ygpbKqcZ30{K?+NS7QO4Q7}LVafw-0v-F~T+g+AW z=8SQzzjrv>owrq8nyLEqF70K{o;|Z4d>&PRe2t-9=l0$`d*Z2+^VXR(2ugsTAcl=8 zY*NDy4oig5(XP(UG~+9;?LKzw7-Jh$8Tf0JnaSkBjhp=Lg7~0-%~G*@t_0|Kr>SqF?=|@L$V1pKoZKjajw?ZU&r{Hz5o7cD+WXOKUBN2G=pbR z_)UF%Hf*r$txXS14B?si>zK)5c|lt39`=}a0Q}6+QF%^4&}2k@+A&~CWFsy(7J6h* zgd1&3&41q)56vy_{9*KuCKG?7Qx~V#J;Pgx8i6AFcLG?~g$rNq3>3_QiMM12Cdu|C zQu7>B=U?Y{#nwz_n1%tx#&kXsvR2BC`X&aoFPD!F&y#m4q;I8fNzCp;Tb^CjwPT?= zt-WUw*rm1tzNB&X_R3=^cP%aRdHgYPwpEX$(juvJUgeMFwYTE((SUc+Hk#}Jsv=SY z>T?D7Uvk)|2R!8EE5~I1_ArwO^*cs&4|s%59)Ja!QtS%LXUC*wcbCdAW%MgJ7BD-( z*NAbbmaeYcAYoV{DZP{c{b|m?Ob-D7YtxVS8w5y&rDu_U-&5Yg);o18Ya-bWYKyUdMzF&oxKubyW}hYah^dIduXyC~iSe0Nr&- z51cMKfY=VB@ioNjUIy?v&eG-$8G@@d#r5EU1A{#PGnBYgd-p0fJfb}3Qz?eR!Lgq@ zb&1or+w3*~b>k(OQgE+eQ-M4JAIzF{yV}~ zUwl^a@ijFm3X>>@!0_fP61~?z>!3WxGakih{U!P;_}6h?e=hs@meoX%K3Y*>VX1c9 zZp6@`5I9vvdD+mZY~6aBZLOH((0P!0A-aP?!?+tw^8)Y=#)5hP+fT_23s^Kh{ z^H{2HbD-p`m|Yg?{f&J$!U4pGf=oq;KEvMLd)+$aXL?sRet!=f}|guAQgS;{C8?vS0J#I&@Kp_R(me0UADh z<%>_l;ML;$M|(isM*&g>bA>3{r+4q;Oi+QM2{GNnm;{#I`6$Kr5)JYErj_mzoFveq z)_w$zf-wfQfSb6^-JON!gV_*js)h&Wtl6_^`QZB=E?j_DuT-rSlyqC$Z)}ePv!?6C zvjX~%m#=TS_XB0^Tw)j3+i$f-1FjmJ22R3Fl&CZ^CorlWIkHDY0ei9yJ4{weva>~> z=2P!e#0yU~lYTxNP!vPn@(22yKP9MDD0@CajZu1Nl6@ZvD5ndCTlA1&@*8wOhg90Fj#n3IIkduv|7yS z^>3`iW}Fg%MwfDdB91Eel;v*uo;?KyA+ZI8h4I=TI$a7{Zi-+N3rdHlI|3f+ekuUW zSuVTW6xQbPL2%N7qKEXgBJw%{O6k@$3Nu27sF`(8GdHX;M7IrD3jVj^r#LP7pL=ykcm# z?%m-IP>q_Yo?Rs_`s-HPW~dk7Zag?{$`2*7GH1Xufj^>T_hZZ!R&4WA?DSGv!LmDz z`i)Qw5srGt%)*j}1HcKTHiz5Cg90<@l;5SEnl^i*q9$hErmbG}Hcgm%@CdSb5%kNB zw}KTD$fS`GvRc}2c6kHA(u6>Ai2b7A!KDOU;WIG#!xJA7hoI8^*$izR8b!`r+I4;z zkR>&WlezhO)K_2#6zELQX$PskF-zc^nTfGG07AGT0$t;>})RIZ^3(|!`j32NL_Ahn2oW4d4O!%A?73WEN`%}w=bH%`tQ4jqNLKnT%Wc$b3l%$bES zQg}ND4?d%B6lniz)==$wJv@dN>{8a1tpCDP_z|xRa16%Evjov{`YFq)Q>H+_iuEYM z8XhXquYkTAgeRKoX?x7D$JX0eHUPRKBLin3??dN|?ONNuQl)bzl?WgZWDTZ1E<`!j zk5HaGc~S|7oYJkb(h-36;6Y||y_J+G;k14GLY`#^d}^mOidc{sBoMqTEF%!L7|Ti1YMPE+=QS`zh-9wDib!c zkt4`F5p)dBIPZfi4P6|I48b>`yup)M0nH`9bHbK}`)}WFC3^64oNsul{y!>r&}~7Q zq&%Y0i7Mo<#OQk98VC?TtmU4^iy7jLU9>=ta1P#i*v1PT@{S!+Y?!}x4YF(o;0QOE zHFv0NLnJa7m?55`niup=l%Sh75&B@)qJ8pE+(4${#!{GppUlP{8ozq;xHmKzv~w`C zNcaF4f*J%c1xN$?8cON2SZ8CBLJp@FfCA}wzM)%o>$}K$fBhtCHRc&_{ z>K#D?dK9~QD?grON)393-?vxzmr7tyK+wft7Du5K{m1HiiHc+#AEBd8sv5umV#p9W zZVFLUp$a=cBxv$RDk?^;(Y)U-sdt182@pY^V@Wzi;49Lat&6unv z%#yQoW&S)}ZRsW!QGh6A>*_T<>Z%CNJy+)cmVH+yxBwtPPL5jD{XncI*$?mHGGXxGMV0El#8Nqu6vzhqJB|i#qxElT>S5b=k5`0Ty~;+) zLe$EHKykNul*bH9S%BA5Wt$8Wa^GoC0p@Umd9HZS>*URK6z&mwdD!T^skMlZdAbaT zb(jh;CM2Qp+mf5<=NfRcS-)Nnf^BF2mw)o*mx%)@lIjEF-$ zP*agNKtj}xHo$m^>-}RdEM^xr?INE7fQTqaep$`5zvgA~WF9LB7`eJEQ$*ZOVJsUh z){AT1G%s6S-YvYnJuH9E$w6JfyZnEDK8HVu-VxurGP7(8wcHay9*XQOt*OCBoEq&N(3s@&lmv;{xQcnFI;S9naGOp z5$S~}@BWRIZ?)5xiZ=Bz9~I1!hVEhHD7EyD&7~1`=V<(-r(JD1KoEoa#uh4#ji)m+ zRj+rVpXg|}>wj&*!W2yfDXnht>rwvl7$=8kkMbw6W&{}^zF7)z)V#MYwy{m0b6ZAK^plW~!0O=6fd&T5 zP#Vh4-DGM*H9<4utSH_x9?kR~msvkmZCU`#+FxB=1I;2Ssqam@yxFhdY%Y2g+1`!c zMgy6-rpvn#ziD&%#!N43rQHn%zvlVN0%C!DN)=@`-bNSDl-paIO|@69*fp*ECD9-U zwijm)TNn)riK2l{8;rYXM4H6)@ugBEQAm{{1A^vA(8uziqwU6+JQ=6R@^J!{qx{@` ze4amOP|BHo*jqN+diCm&!P%x?e@gkE2a07=-I0wNGV8rpu3DwqFdZ15BDLo~YPb-R zA_x=k;3P+E>elAqGMu*72WwJjCRAC}OsZ5QOO~*0`4xFCBZD=l+uSRNK*YW}IKSY+ z*}A(Pc)>C3aoMH6oLqoH0!u}V5fc#Lsj!Y3ETgR5=C%?k59Tg_TgUa6?1Q_8W}kYn z3?nvocU_?QYok2~vuwD6zC{--#QX|M9b`cy3mUQetVt(g6SvgV)If;Yq67_x2aLuB zq=^!o7-xPx6oj0b0AeZ{Z$ZPNI((4+GH*)yn4)PN)(X7HKf2YNQhfDjw2P`xbGER0 z%^pI-&z}?5xA7d&ud=QK1<$y(A9ul4KwgZDAGBlB1eMKqw@LLxBXQQ|2;KPTYO(jV zwG4uBtX47h{Zv_bn)QT>7WtqhkNkcTkOv7j>6r!t>M0N*^GTdP>Eqz}p-S3F&$@j1 zB`PLLd7QB6g*Q!jM+wfbws_8;rthbMgJCMsn=-9x<)Tq8@K5X|6ZRL@YZKl<5Q?8Y z1NV?|e$hgwLlon(FtZCNb0@!WUXJbE8_>4p^P@peVrr>Qc&M235Nxh7Qvy&5ds=}% zd8{<+&;(OG^871M0ayY6Ua1Qpb1;;_&YbN_8(M#_uV0_C{G}e83t(%|0OJ@wIXy`F z?b|d4V1i74^)5;8>%fWd*L_!(*eGO99^OYaIO`16dm7B1c2kq$S|s@jh)jI`h(< zz~N7ywy1js_&meenIic2?_bO6PZBw~xgcW5zkZ6=03!RO>j0B zc3^hvMv-ER3I4h?E+qgt&a#x3Fb-a`B}}^qh%yPkp)rsDKar%x-@NV6=qmRz-5jd6a1Hf zkYL|4`~l#f3XuSjIPU?o9qd_)idIn)1o;?U{UF>5`T+ngI9q!u&J`7#-gg=TuM#E+ zfW$Wtbf9UT#7R%+1WjO^azpp^@|s+>&i&xz^0&atT`B@p+SG&`S~nhw@?rI zEw{B37Qeqv4OYIsCx8Nvcp0jHdj9n9T&K_LxD(@!6=Y8StEey7)psr86WANz%eY-g z9sk78j3dODQ))EWP6Ho+lDS{cp26xmX=xY~w~!1tj_!)W$di)}2Ydv3RFTNA+U&(p zHm38h`a5NC@^TTvM(%?}EtB0F?935df%A=D;tVoyJU`L&nNDAU&1@l3^@& zE7AUzBUXi9gAXvM3IT-^%`O!ZDmSivO@xZkQC@>q8NNlE_Yh1H?iCioBLKW3kuyL9 zH~iV$B24r7Rzy18x(TM!kYHfXF+AmebEXyDxdSn{lub4~up}e+>F`Y076I8Bx#{}m z9}CAF_rnD1NyLf;`;P!;A>xEEk6ra6@^ATQ!_yfVfR9X6HlcYY^H~ntg`qhY#MoF6 z-|$<6F)sWNJ{LEgnMK>&>z>?dCiFaXoR^r9^DV(8)U|5u3kT zc4tBeU?=?ZUKM;Ms!)7vm3s6@;Am3+4}#T;e|=a?QA_=V%P z%i{$@DS8|K<`0J$Md`R@9J4&d)7RY8#Kg!ILw7(!njzq}vvqBh%oVk@nWQ_y3>Yhy zURQA!RU4hx@8)K~B?asg{9obP)00KQFq?2)yjj!N*u=nK=EmQ{ zdHa@b_zf#zfzB9|z9sVm3xY~en+P{5)4!c6CgX)E&%guJ&J)nF<>z|=Pv!K5?}>2D z#leAx@fSY@pg(N*aC(4-@89RpSut$`HXXg<)dK8eS+xtfg7*r%sc6&)*MtwW5jy3* zeUJEE5-?NUbh0XFuPBm)F&T%PoWzKC|LM#nLILz?-ETfX$f(XoY+(CMqc^{M7XkYv znti6C(5bmdbn(So;dOohj?VeRPI(>x7nY*(n3Y_rO+VAp+jJlz{Hd`O!neV=z?z7mtv79cdRtC zfFIQw@DsH-5K`yps%dNU~Qi`FtalHjADO^A}+gG>%V9PirAt8at8_J@KmhXZ? zH3N5q%fOEm+A0x~;o#`Mg5QBp9FG@(D|%XTiStqcssnCjnNCSbkD0o$<1}P)2?IP} zZ6*&#Ut~hQWXTbiseJO%?}C-d-!c;c?7x{ zklv{p4iuIwpbIf)pw5F6O_!n)5kowpumZ}Od}WxG!1>Q1MDk>CPJyS2EoSA=ZKpsL z8!*6Q#fswx4*Wus%cdmBV|0dHTwg5AWW^tCEBS(8NJ!JVhMm z6ZgDU&7@wrdn4X)a4YDPP^C^#@&9nQH~z8y{{CQCOroiHxC9{M4o%5rwYC1}F`lf& zLlub6v94R_g#ldgp~HvoxiEoOrkd29pD$BCulVkD-g?ejG3Ycrv&2Z`8Nj|>_Xxdy z{E}5P&sy0~MVT+qq{ZI2q3~e;EOuSKe=mYBfe0IzyQ3l^yvj+Vw}>^q7m;|7Z~!)Sx^P8|I+Kw zz1|wK`lhKJ5?ElwUmy~w(FkLFahc$}0NQaVNu9JRjGgHxnf3CpaP1-KPzS)cA(2If zpKEArWNQ%QCCB8ZuzukKYJ9B$2v*&jX2{cm{ofex646uxP{8?-4PxXW4!n z4V2mXQJu&LZb_aYNlD+S10$5jLYe@YWT0__?HAM}v9+b-Rf++w73ywb!^Wmf0#nM( z|2Rw9xT-PCPPDbLROFg75&YfMw5w$eC3EZDzB%WSZ8*5qLep|`Va*y#>c} zY&$7+bO@w_n{P_fPM5Tf7&q?d1243IG_1o64TWWeZO zxi?ib=fKyveRfnl9gO$zX5U6NaIgAjM9b;@mI*G=%PzXl>U!;hFERr)8zL_}6}Y#g zdwAQvJasf3G*TOXR7!n$xP7?xHQft!i#z~Yc<;mLsI0BAq(O{1@p7PIC}T>PL$UMO zhAk{XNHb?($U=^(*_Ssv{j6aK&K(1g1^jYw{VedRq|`c^ z$%Y|*D|^$J-cT38=Af?@AZW^hhY!2=hAOE)ZW$Y}jvFY1ha4O@X;4IHe`t+FJ}57z z`m|pDRVm*2kirog)?((Y4ztl@9>t6%BrFUbf1HAoqt2)CammS)>9Z5-h6QKK$!zbT z9Kb+-=~U(BN0c^19#{mdgRTZSm>wIJ`qKyi<sAv$GoM~p zi6{V`YI&w80p*As9&7pDjTk?C@gs#NkHP;XH$aWW5NGVwC1P94LfUEEh34?%{l13x z{Keon(KRPK={y+<66gqR)_a=LO{2~Bx*>se!9jCZnVM#iPFEXZZ|{#Yy>#b+K6*b& z6+>BS#vWsu-&VHrQV8&5(AbszY>rRJr)PsMjBxoXlU>IHLESAO%uJrVh!7E8u-v{1_qNGC!XXn&7luz$G=zC-Y0Klx|S?UoPK&AAixKV&*u@X1mAM zDvXYP3x`c_sm%Clk0IRzCu_qeoA-pOc3pIy^)Mi{5X>ALf*$6q>fO}G+OjTfV8TSO zW#E1gs*@MfI%uJi>b-tno^r+knJp4OVWo-ClL-qnepA?#_)%VP$U?u(adCkX*+h#= zT@5IebIO+a9jzoX4@M`T-u`V!{nvI!4}e!^BfxG)5utJ$^72J_LQ+Zz1)j(q5q%r0DNR{Q5y1o4$qQYV8)abn~H zx?y8>j$Al~x~O+Ae7J`Ec7YB^!BB3H2nGdzn+&lSBnP6~(!{n)Q{~#sihU#kGIu~g z2Y&-Ar(oWPzMpm;S#{Ah+wD7dVo>6Hc1bD5TO&p!Kq3jtHNrTWW0ARRV~-1R=5Jq9 z*AvGAZ>*V9lu&9L&JHf5}{{HE- zk6`*VOf<_@8#XjB6Q%SYLkKBSrb@X0;KtCA@JmL8(?YXAwUy~LR$ws*&i)vtFg4*3 zrqXWueIb&?9l&C6g^<4NQZle|a9FdT_Y*!J$Oc}6Ygq$;=r8i}eeMHhIM0|I65~d= zrdh<okwTyz|SKaSZHf z^tKJn;isxt`bu3f?)VjCQc8EgdhM6MBwchn)9e&b)0fAFAPF@uDlJjqjqJ`RV!v>yU@>fpos>#p9_b3^B z$qtr~=`x=lm2o{73vDO?kx)c%H}9#{Ql7A%_8Bq}9^y*n87b33gGNTa1sRNZQW$9j~8jN z&EL~M8vbn0o@-%f!9-vyGV=nN=y}UuG(cKGJ1!8E6ILeDMgebLa?3oC=ShizQB<2x zUVk{Sgmz_0#vwz5Wp|u{ zAlty39CG~MDEs|e$Gu}XN*v%BfW||51vKm9xJ>F?3Y<0gi5M?I=%a>`{j;8h@>X4f zdk~7trf$@(CCz`Doy|y>hV*=gEs5F~~Kj zhcDG8NDkqS$D2jFiB!ZIDx_G=sGAT6q@;F0zkJeMAq&h1wjd4HO;)!3X*Y~WFarbE zId=Fk9wBTnxXDuH@}um7ehxat@EvaPoH;_LEbK~ka!Pzz4WmES+LP@Lv>~})Wa0h7 zuK=mQ(r)RGo=A{P7(i{@*hK$--f<}b8*l}BC&?c~iHih&VgKMv-Fp5ypOq&BWtKfL zn{xAO7I#gd<={gx^-&@~1DSyP+b4YEQH4GXRspx?lxP5_nkKlH^dtZb3yTF{CYYv- z8sfrJ2Y&&bjrl^)6)XDp=z)A2BL(KqnJZq8M$-KGb8}dx1Gj=x-$z9y!@JyECKs89 ztg<1z26lT8t&3B)QFWiRc1AJ9S>gsxF6z%h8g9|rwaL48KNykOqTX^Ve2PZswr$E| z*DRne!&L(ZB_gfc@^yb2b6ekKxBmBRIretXfK}F<3uot7s}@R%w5fp1P$qEyh_v9w zz}}d#VA|eR+Q^arzzKmVr&U?R>g#ZYVM>%ewMW1H7@t}vxIz#s+FvQv+TTK!+%BRL z(MT}@Py=rP#Et%v65c%NuUt7pb@Aof^ck4VlP@VS&3xXrM*>GfJKbN`2irhO?$R#? z?ZS=@o|yOwlMY3)|1fBOzfxB=!>{u;grzKK0SF|Hw@kwYPKm%AK?lyDghl&bDcyP*7d5)>JAWVVA z0bd+1TmYU)fu}KeaM4i%4hk)mWWs(8nrg&W>}&o20RV7(zZb{w96eA%g5Zcn!0pb& zjn7Vf^r;l$5@kw1!b4G!thDqa+->osW^Wg|h(q@t+`ISl*Dt^oA`=MCYj99VcQa;0 zVT|c~Yyu%5BSXb*s7P6~NXMH8vux&-nX*ySBfjzH0VYKNd@*kjra_R;!ySO4JQVZ2 zlBXRX2V;NP{P{gqRjClL0UEhH4Dr$8&la2sk{?qt*iq~G%|z3GbfF_OqH2Rx?RG5v zN^C4EfBnco+&~HmrfADYZ$XQ|vrWT@K|^8T67Dw9ioS(jV{%s=v?Nrty?Mie;JvKf ztZT1Rg*HGiG_@mTSiU^P+WIwET~?MUPEs*NRo;e*p+LT;6b2D4q(fMoIk~1dFHgL6 zKED?-{llUnWG7@jIAw~V#A!Y(`)~Y5)(G3}IRqdmK%*2vKN=dml3#LLStqdAz-rv_ zcU%R^3T_Zk)3tpFxj*_OE>-*c&Wg6>mfP<)L&!(_^tW(GS^88PeHQ~aK&SfI72!mskxTXeoxm7Gk^0u1h@uRX(Z8bP;QvkM@rI$WV!yK6MdA zsq2&}lK7`gpKf+5M|U~FpNoi0JI_;U!!L|1yNCq%OA9KQRRlRKFxh8`hZl221UwH} zAOq=3PEMD0iHQBAx24?V3@sCrn0p^s9fsPFHh4U%%7_P3rw*HQbpdGuA7-YxgIH?e zw-X)>f-koJ>MAnrGLsDr3E|G&d-KK@2O6Av&ts{UtFF~A!fMU%q5#q#9$A>?nN3jaGPTjHt87~qv6D{;WsIU$^f!+AHTbxHo$NA z!B~O|LIK7%Z@%JanmUR>Xs%Od&NLV?Vif@$*@N)FU?0NB7 zH^v+~6qdi#L-F9=y~VFz-@;~}%Vk)?l^^h?w$)6Sdc#0rX&gG*|Mrh~g`;MB+aVRu zYA(d0Mduhz(6Ak_B4QLhc))`Itu!S$`QH8e*OHULeF%$nzkW?Bva+*7SKqrvi-^*M zsU52vu`lO=gNkK#5wrN|n>L}GVv$G6jT@GMmLlD5UbuYu0HS|bdY3PAun+CsE6jO@ z4P$d2LmbkhRsk(I+Yv`8%1}H)TBClH$)%&ht#jJB#~e6N;Eo}L6=4yR4+llKb6y Sue~gMsHv0eEUuWl2mN0!2Sf<~ From b99662af1b7082942db3ab30ba95edaf6cb1f658 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Fri, 7 Oct 2016 20:00:12 -0700 Subject: [PATCH 07/14] Add files via upload --- kubernetes/spark-submit.png | Bin 0 -> 110846 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 kubernetes/spark-submit.png diff --git a/kubernetes/spark-submit.png b/kubernetes/spark-submit.png new file mode 100644 index 0000000000000000000000000000000000000000..0776549f4c7d6c5213af6209c974273e24bcc3e4 GIT binary patch literal 110846 zcma&Oc{mo_*Z!?ZB}s*ZkQ9kXl8}%jO-PeOM5&0RQW;VqNfIhaLIX|4N>M7QG$9%^ z7?Nl*L`9+ZbKTE79M5sQfBdfJc)oShHSN9ETIV{?bL}vz1+xc>$cP9D2@RfOVP+#F z)ccQ+P_M25z4?jelM{kJgq`QkHWT_M_`g#(&qec-fnFBNHwy_38ZP+%UP8xDNb|$~ zK6B>J>R%%=ptru<+T2$OIGzDSWf%#-7|+*SQ;^7m=3&uaFXjuz&9g<87r&KxTEi_qOs|NSvw+kby3zUw9UbC*~D|Nis; z_m_JO{`;MH(|>;$=l1#U&swqn{`mj><^O*1j$h&hTZ2PFLc0I{&aQMZ z|3@c=mom<6dz@=OG5)Epme#25d*f+Q49E_n)5(-tJmfE_y zx*8haYwqlsKYxCt{PaJyp$d9(&oWm(7&rg8&W?6Bz1cz?J-1(v61p{V%9JUQ3Nt== zUk@}&JAeNCj3c(QXV3QZ)C_S_S#V-sWTe6m4@XDG6)R-dcXWD}B+k%ORqarjn|1Eo zxhq#B3p0*tI)(%XJ4xTKYjs||dcuSWwOV~@^33)s3Dq=@kq}wI!_Kb5&nUvY{P}aG zVPWc0;g9RZ#Kg94+cwwIa{c=C$yX0uzC2MX{6%&3LgitSl9I78F&%yZuC8Y;UUc8Q zIXfq(y{1Oda*3jN%+aIK7Rr1hrPhBZik?1ws-dB=XU`rV{gsOsFE%wzzIs)vP>Y|5 z7V`Y%%i|6n6tg^?mX>z;@<9vb^A|2We)w>e^b&h}^)X`#*JNL}znq_+pOd2#UL+$e zedy@XrAwBK6CW`5hidhN=U!D3>M9z9|$hYi@vXOIe?b*}vFn>Q`K z`TO_pKXF1%EVQoGciy<{yu5fbCDvS9TC}zO@uNqlPoJ)M@nVwb1PzU`(Z$D)ANMu9 zb?eqiW5s#LwC>!!n|$rs(PEchKYvO}NIWbqPWmxkQ?vQ&*WusQ^+I|H4u6XQU-FvA zTk|BV0ewyV{coN-*YGHI!WbnbXJ_XpPoKW1sNi#j43?FWmVW)_&5gi+Jx7j6$wVx1 zbX=$ub=m&9(&*8(-@jivbEc@S^?rnGsL~$3?y+OXMj32%f0I-E_U($UBTrnkr+QY+ zQyw-)d02FDdu!{aZQF*3hzt`6p7F11a)?txLPFc8r+NN=bwixgr6#DV?lV@@(bctE zzTD&ejm5KOae#~N-ZfpZBJa|rOXtr&x_@6dv~UZ@*2ib>-o2u-y6%F*J#L=W(xsZ3 zn&+%kJ*)CtTUx|}4YoB``FC}+7KRQxWr`2S+qDH61*7FfriIx{j9GWWQg( zehK&O`?0=0?ZSmY!FSJ|9oOFZ^;LS|-Mh-l%37iu)~|n)vo1P3{Bc`D3E$#dQ_})P zaTbkJw`ccm+r^7D9dC6tmW`P*!;{^^=*&-XWeek_i@c=6)1XV2bV^US+`J*i7YS-Jh!uO*8Yd%V5+ z^zq|C2c3ijsjBPXHP31(Ny(c#ep;HEe&}p#=;-o1;PymrBH zopoMbhl-q8`~!DZRlIs-pszn}?AZAU`jUlOa$?*5w67aEGA=h)FUjiIi4%SwA4c8W z`TJRd!M2CuaU(PxY?myl%G(xco>Fx0Ue494KYa5YAIk&%q&^k(u*L`1}g+GmfCYHB*S-1Jiq~!X@uRD|`|sbs)^bu(QUpTp-n|!`G#p5x7o`4x#LJgAE?s(*!1z!Uer|sJ zC(e&iqTZ$t8BT>-qS=|5WzV1Q4-Yr+s@b(?&&&-QHtg83W8=n&kq`Lm%t0l`Ce0)f zzW>QVy6Rr}fd|ijP&YwMZR5M#JGEX~qVWdXE=NQ>y}U9{*Zp;dSFP#NrAze_7AQpp zXm}EQtlz2f&-{gxdj0x!jv*&}$&w{3t#rt`4IAR)<4K_unB~itw|?dTDG%t|JIqWe zy13w2ZbHKQD;r+j$jf8X4l2wraxwO?v3dSFbM+`WIW5u6Ten`#$+@ScP7cb-dj8_v-)7iLIKSYn^I^-^+j??Nr%Jx&`o%uOlN(b16>HQ%=Fclo)6oqj#1=R^pNlc?|z|MvVBt1_4HAj2Mg_bIZCK!#8_qJn3x#3={s^RT=-fPKEmyrScuceks}S| zheTwIpD~0=^YP)F>2M4e1`mV{S2>Z8aP#I(Ss9tF-`{Lu7e-EAv(Hp&%$O9x3vby{_Sr`_e&w=d zdQNxy`uqD23v2J`-l@~~9LH&}OVgJxf4WpCET4V;@V(3KHdo!OYwMi(;n=|8oAy865Y3#zVqyaf-)%-8NK7MfD#Mv5q`YXz!E|4_bu z;J`dDFE3wTUk8UOzCdhjY<+#b%EFUfwS}Slx8rqh*T+XR4j(@3<}@yPd(*pVSZj^QUq3?Cg`Os$fgy3vnRyJpQNc7Oi7dG}=Z8&01-P43z8 zDH|xOe*XUCWi8S2va;n@-IrTgmBvn3y0QA!O}8>7Sy|adi^?6lwvYofG%nTD)U>y^ zgO?~#K3ld79{4I`h=Rk?rRTsW`)}U3as1Az12R)A{YI+AzNxATsJlP-*!E#UZttlW zCR7zCC#P4JSMF@73ChgeWNt3}B{$B<|Hjp;zd&{B8X8o27Z;Z$7nhY)R$gCQB2T5Q ze|+@omoHbZTp2oGuTxuwbgLE-=SOzq{(@PBOj1jDVbL*G6=|qkMrBWjK_Q zl2g1tKD>R~nAh0e_GvLVH=t|ToH?HX6MXz5F)=P$(v;{!(b1c?ZCko%QTV=n(vzI~ zhB#5fl2vZ7i)D3{eS4dL{gy0|@=r}o-RE?SI&pS>{K*-CM&M6t)iIiy>FMbpK27^v zFXEEz0TnqqRx1Q{SsNa)S+wXc)r0lq>+!s^Nl8hmsTPXjR1!&2yQJh;@c~EFmuCfZ z?@)_DXD?qi@_3^| zGLsV{Xumc#re3%ZZg-5}EHz78uwX&)a`|oNN zZrWhLlP+|3cVBa_?47L^sL<1YZRzZ6wd(xsZPkx%_jLaqBqGxE?s}h41tFoX$)_fN zO?+8ZB`+>cu!a^Wj}Hv|2Si9;*-8qyu;kKRt&5~a%3a1H`Vg9u9!o=c%^5Qk1~2J| z&b8mjDORx9Z>E%Fl0dPb&h1Y4F=UXa=-#vRKOHJ}UMFZiudL+I-CmP@=JaWfZcJ<} zN0TJZUpRgP`|f7lfI&b`@9V_Eym8SF>c*?7QAI(+;DRGZkJfqR{`$3*HX!D#m8whT zvf$w0moHx?5AD;~cJ}Pq4U4;`X#uxhl`;>E76u7otj&M2Tu^>|Li(=`-` z-@kv8896)DbgCP$736sTzWMm^X;EWlzkKoH#mkpAsuEODnTR!RZhC85(IoOh(g_nI>sXn6Tl_?tT-NTvSz4v%9(DN<_rqxOom4t9n^xU%9e{#Pt06 zbJFF9f)}Nw95<&VD^;?ty}kRYRpP^j9p)sH4Ri(>*|pvd>eDzf5lC5l@1B{e#1dQE zj?T_$1_m<%dkodo9#vI2%$imI^Xn^pef?p>hS3b9I~H8Mc5TRj0lu_i02&$viUH-# z($X^Iqo6h2<<$`QEPR9}UEgGnH#vM~LLw-rmr|64@-P7k`Sj_2trx#-`%pA!`t+Nm z30`OCjvXJoaw#w>qFrK1hy73AdP#Km_x7?i$$Pc7c4v-&N~CM}uHlqn-c%`Ge* z0v*ft`{?_>%iTnoA?mrtMG zTUgsvb(6ZeLPB!;&Ycs+kKevy$AV*8P9N@t%{@A?`+NR&U*A*xW9ce@=Jtyg?QuF5 z7B+Cl-eo2xCUfQ#-@Yx>{r=S}mE%wQg%;Mf`pSv@UL5^6=-(-h-^-UvIFMy0^aF(w zOEkxiw_dtb;m7(`aK^ak;##lL$8Q6_UmBVS-YPFGJroo3(fj(maf@hv+s9cGUZHnh zG=$3;`T0^64dt5!wf^O$$B!S+tlgbz-8ZywVBg+JR=(h10D8V(=i#So1lZtY+idG7 zRg=}L>&Tp8%OfYzPz|F zL{2Ybo>8Q`*$3s8({S7k9z?Bw zd1=Mb_;`a&AIvE3OO~uzwd!_V>+|yRhyw@ogLq3H?a z^}H0vk6-sFW;{8`OsVy0{M5?I${F45B*BPIXOA|_U!|`Aa`ExiemWX=vcI2dVZi4Pr`bncvrhDLYckQq8t zr>6Xp9P!^{-MY^}TNCXr(IMY$_3Ax^wHvGQ$7^Ve8ak9>0VIkr zg217}n-w%5Zr;`G?3pvSo;YzrN_|<&$Kv6w)!LIL4IDIRjQ9W=bLC-SR69Ys(bg6b z6Z_QayJ^!VH*bFuj&_{8r)MDD?!$)<69aoHtE;OE3;PsmamM*(F~^S_dAWDs@X~5E zH8qF?UJf3mapvaDn-vri`M7rn?dg%6HfFMh`fN%5i491QvdinBKW%gRi0k}NZ z=ks$=!0Oej+03T*d1Y>w4jd4jJUJVhKx5?%8n5;j|HjHF-LLm1pZUR*05;jN$~$+Q zo}ZrM;NY14YjD zA3vV5ZUg3kyaQQ|*(_3Aa#NLhS9w93L?ZEeHg$t{z9V<5G5JaPkDLA9UPF2eoY*iP@))F25(=W;X>B;FU{AvY_LC;CU z?IgVDvTABD59yHkC+MB;g-M5$x#f)R>>qduY<>Q`nrLCF?fL!tEoaT@_ea1?H0FA5 z*|K=)(nOtg&pA%--@gZL%#M)x3HG^j$3%Ef=7v`jH*aohYT7t!)+~`=o8`;bxw;nB zd$Xm32M_l5^Sg25hVt040QzDV?e+Hgn;QoP#EJca%?TMpPK8aVzQis-d{PByme_xVY2JPP1AaHH zq|uRy2swKVknQd5?dCH`K63V`Q$U60>H^2Sts0?)Wv^Z}fBW{at&=Ym6cj|>Gu-^? z!2bP@-dx!bI#^a$`Su^O9QjpC6i7@v3~;kmTkPNB0@EC)J44FZ`BZrjQ_HL}4P8Z7>0m(;&7Pk5t>c)Gx zy3$uDTPEIF_3hpD?Ck7CUDryzbVLhHO-<=z#N-Vdo0=|Pya?S0wFKj-B^n(Qlarar zF4eU95)t;~YRttVJg%pW0B;ykz$k&hy~T?qZfc zo#0-Oq_kp_dj35xD>JE9Q&oL(Y*OK!JH6%yuUfY*E;g2Aymay6SSrw=Ls|q!M@I)+ znH-l)qkQJfnMtCYX=>^QPtWwUG;3R1da2$)ih%wtZ(-4Ey+|x!H`)N`iH6&$CH;1e z6ly8?k3~pv=cudIcg9o3{ zbP(f5G#vwo(Uo~y;WcC;uDTQtlQ`%%ji7^#nLYbKON$q67l(!21F9Ix7kJk{niVbu zjeYjaneUa^{R_2}qNL^ISfoR-u~&BfZUP;rB`2HCof~d>hH86+uEW3}#pUf=sL$$K z8{g~u`*%Zh^RgWszB)6<(G0!!nz=rp;KGd?Z>p;UX~O?M`D2fRhwv<->fe}ClLrZU zfd3Pb#_!S$z2;{e+oW%Tr?mKo80g4&RHRxrdhD)gE~U)Bey!*JTB~=E(3#e$QL(YX zrkw46qa3F{RqV(ctD*vTmRg<(7Xh9uqf>Xld|3pms-%8W}iI^lv8G^_EC{6$S z`9l9&Y5L#4+@*<>((#HzRj|FyyuF&93e@@i%@j0{_XcK4o5;sxDXS&_^Xk>g^z0u$ zx&#?3_BdHu?r-#5w{9KOm8vSguZawLb}=v!N+meB$WvVlP6vSe->-!3CU_pBXZq7e zMcCg|DeC^gQG;gnWp;Az~H<8vUj%l za@PRh9a`pA|}< zOXJ>Hzy~7Ct<(iS-)m4kgcsTV!=)?`j?}D~{PsUT9%x`-AQHU0u$K1T{D5??eggfI zcen}3Z=xCF$;ENAM?YfV}^whZMJG#=#8vo_~C8G%{#l_NJ4SY(H|-fyyIxW zy3-n>LP4fj)-{W?wffGPGe;FkvO^*0@0;7f#V*4A`T@60DuX;nBrKqb3iTEo$)S$# zZxlAP{r&R?6jr1VCtM>-JIbR^J?|vm)FrlP(V{V9#;6*#Xg~|@5vZI!=WbiM{?DiD zb35>Gqlc{EKlU0Fq+hr&K~YiBz~FE7Q}b1;YDl^TF2yuM6oP_)?mrivBl_H=bSg-% zG1AlPfVn69X%ki2LJ$GjRR1zo1+;-%QXJTe5g_=4jaGW}H6l0vb&;@|Vfl zpO^O+XR-zBlrD#k`fugLe!Y}9J}{CS{=2=MQ&);ED7&`|R4 z!K3o0Pdz?AKl3kb<&(PBs~g|{0eA#+l>^ z2~Ztf{10D4)`(3X7FO~4^|7Nz&z(JsL^s3xqlJEggPq+Y7wz#9oOMeoNL*YTs+sCg zvCu_Jmb`rZ8t4P_!zx1uefj!zx%$hRnpG=S#1pyvQ=q6==<4iBXiW;Hub~4iDeqvR z%-(}Ve*Ew@N=!e0e!a7^(t0fmOUp0K%}|$9wY3i)IwUR5HLe7+ROdI zQ6ng3bQT=%>*qZFA1^=_;Rwc}?^abkWs}A;?I(^n1KZc>cgyW99S58mp$C{nCVlAQ zO}(JYF)Ky|Jbmum*RNk?rKM}U^vW7gdBN$1@(X!%ozhfOJD;BZpsp3P4T6^rIdkq@ zxf_JaMBR7|O-;{@8;x8Jeu7`&WTMf+HLY1=ey*GrA0pK3)s_5w!-+>Xcz9@>?B(+o zPIZa3by-5F$@&)Ol81?37&zK7R6Ko}HacVFs)gNDw(U$qKd+ z_+s^Jj-CDTf9Su8B`QI&dq~!wNR_Chx z?T%Sq3l==1J18tHq+>S=HRG%}JAW8)w&nfTrlwnP12blrxb57z)5Ya%K) z*}P!uCE5^R_`kNDPVaAQp$`o-O0Z5b2l@f-?%uu2fe%GYnR|gIoVKCohdbCb&KSvW zZ~FN&ppKLa6FN$Q4Z_00c>h<=pC68idGhGdkCv8YSJp2!H@|;=@rAPs6Bj5C;|NZk zJh|lL^b6LlbxYpLzf`^mM=n6Z3)X$wcvyk16fA&5J((w)$RdXg20 zc~aBT;9TS*1!+P{tHtBQI?fi#;MsU`V%l*3#Vb}IxOLDEBiM@I6CALsgAu!xp?REj3n(V#C-Sc>d%wXM)9!0WXin z#tsd;x2f*_^&2-R5(zT`9TdgUyu9^JkWF|6aIWBoPz5ookd)oszrp1|Wzmb^X@K~M zl$-Ya%^R8F!^enR*=|@xKZ=q`^{M;*J)*Zj06ly9v{jldo7?{U8nBo*<#iUNRrDJq zb;?wF*mlQ)fSYI$OO|Y{y&F8n`~azu^0cF+#$>~W3nx!j{iuITHMI8IN>Cp^F0*07 zS2#ztShe+E!Vezg1(9j(vs{WBJRQeRoOp2m{`KqDP14r3%W?_J&-Wvc(eSr^dkx0} z=dkJ1lQ=dES`Q-H%-mcTx>j0RedP^X($$o32c+Hne0$Ou*!%YFn&)Tcf(a|LT%|+E zPw0(4kd{N_4aud@UXaCmv$HpE+SCetV`Y;(|KXD-`4Czt;&cm#$IfM~Lu?1o6=LOq zenwWBK7IN+9Xo`G;6WqSl$`8t|GX&4N^kS0C*v2Lr4O@?_OPetduELQby4eq#t_kC!J2%W) ziHIsNIfajy;oGu`2Yn0g6-EMxjPzW;UVle>BPqsA@-PL>U|Z7xtW8kyG}nVhOcd}! zJzdjcEm!7xCDDhSKacexW10H$<(1Pme|r7)?eGD6`ACq9^iCX|rzfUurgv+!-SP9w z;`Qs*#6nkOthxj7((3C+dEB;bYW&lx+)XKPKL8UP6MOa)!Z@HvTm1nRG&MG6WMs6o zw|8B6bT}(ZTYn~HO-odE@)}XKXo`=Is{D+A{N&`j2o7wBTLWDZNjJRMg~H6@CYM{; z6Xfg!1{4}uJR9l(m7W_mfHvWaAJkF8@t5VKr+eahp{7AwT)y1I83yl@+OP;gA8q#X z+7}Vw|MQDvm>ICKt*vd3+ox^an-iA1>k@OXpFB~AbAoqdweo#F58ENXOi|o-#|}Ol zYWR8>L5#2sMHhgk|{&Nb>XZOOvy;4y4i@`Ve3>AYY* z(3B1ryX2gXj4a_C#2-I?Z^CBMBs*S-(c$Fuo!`EFD~(f>5CLiFZhY7H^{WB=)67t@ z!J?uc8yX@<8Q{0$;nn(~DEW^GX0gOXN0M-K^l*{aBu5Bm3~Qci){LtX2+WHXFWv&S zBReAA4)R~Na-}~+?T8Ua>C{V0OUWq-dYgK??kQw`H0rm*8TeR`!ISnsej>& zZV72=Uu1vc1fqrF0urh!2vk^HkMTrmqP^bfbt+0qA8314T-zvFAk*{Vl0P*NtpZ;Y zTEQF|9Ve$+_FNYoh{C|z0?r}7Q-TrEVcVy=zox2Hl$O%vLxJAF6mW}%n{%f`w1*EZpv#n^a#ue%*xv3VT1Y+=88iq#3_Ih9 zcIWBoXlM7Ou5OW|qvoxRh}uDY`g0s1*k~=Fh~B<@iIMN!`}frafp~1vQ&V%%k}HaT z5N2a$A4Il>e<7zYTE2Wcb+K{x#P!qR35?h>3L^|1pTDK6b2DoOTHsXE<`7Fc%0b2% zu;`@BO`A-no@f2A3W4!l)~?OV%iDb2zM{k%oeGPAQDSpb)5#-8#+~d0A1uDGq`RxF z1P8?0%w^eYO9+Dn^XLEk@dI}y5EMtju_+st;8u)(>Og%zBgk=!RVW!FpNMyGBikoE%@2mYO?JsQ9WE`fzP&Wp6_xqFJFFk`}V1v93RpT ztquQf(xgdnv+lZi=gzGqK>-@c`xNt^?d=y)vFFd1bUk(E%vd$G^$CkJe?JSHYGa!CBsi4!Jl-m;~r)(aL7jg2Kx(3Wn3z9KDw#=-bHIzHqGAU243 z1jMsf?Om`N^66BMH!*Q>l;b`Yq(30_;GD5n*AK1%ZJ}myQT= zn$gu#2zn!XQxxEr6OSI9dfLF9GyS{iz3e1sGe7`rebLSrK(s`3&F$6#NJJT(I+gmK z{0Hz}3l|wpBR)i7qr3aXvuAZdIIW)_V|)^r!MZxz=)*u(_}pgAo}HJQ>+_$cl?ueo z%JK%^$%(Bcw}Rebp~2MBA((Uo1R7RS8yjyD7tBNINiR?yJS;m2@=TfgjzmG-fV>5c zT%|BU2@{pys|)_2PD+QcV-xLicoXu$H_8k-t!wq7Pbcv~BqfEe*|3vQ@8N=R&CICmeKo4Z?u8 zTxMgVG-k|9Z_unU<>8sn)K_}D<3@)iMXASljGObG3kb_fEe^ZBRg%AhWM1K`>wh= zF)^|E%a{GV`>Dn1Q?V)K1k1$bS;~YDS(n;MmKfTsU8qHuz7}Bzs{mUmv3O!&8N@jD zOVkA#j1wp9{rvKG@9rJqq$ek~G9M{BB_+k$)rTcX-5va(PSB#c+RmRJf!Vh58)6ea z2YRI(YwXd*#l%>P98~RFW9!EHJIkoihjH z3SQs-{rk7d4ht{ZK`r9@Jphz8w1t^{AQi89dfXKf`Peam->NKg|FbDr_SUXF2PRi~ z)!68!9rvxVQE&DA1AJ-S_)!W93Zq6v{24*OP%Dm~IC0>hgJ>aW2;GpPz^Zrn^?*ia z1a#qW!S~&`Z{GmXLSP4RO72WeNonJF;ejStz`H(=o4i z*eNSjehy!QWi-r6Rej<_vLKt#*4ipCOf6pgqG9JDEG`?r4jp6ewtoF(G7D}! zV@3v>W@cuNj1yHCMAxzf)5k%r?h1ktwxeIyevkC=l8;fqin-VSjnj` z0LG&3Qz8Jz$8dz=H_hEtCpvgAIsNs!cWl7-`uZynPDY6{fci8R6)#_&PffMI?p;sE z&l=fJTxxE9_U6q|gV(%f-SIzIsped{LYF-1dBhcn05G>`A#4CiIt$T}2%`N#keXv* z@g%{3cpN)wsZ++Z@H~^dMgv({ z-@pJLdYIX;ZYf+!h3~f;I=~^OCa>PGVM5g0 z57^YKwf`5hnMW+&cJK}xb zj-Th<`l!r1*3x}*!pQ~aQkD#71$j7pFR!RT3#&|9bk3@NCywl7amRb4EbKs6a^2Rg z<5Y4WwiO>9i@ncTJI;P0_F!*Pq^dvotW{fKcd-k+O3LaXFJ`i*Ex}_{1zxX%6W9l( zC@iph0;U@Y9`aLKp%SnXq@F*YyqKn)r(0*wh_b-X&@pnb!_=4{=nixsb; z!Ym$hYq)MGUEnfxC?6tvzgRQb%R09l*?HMAi*w~A2?i?qBdp%ve)kTink-pD?;&28!3K-1J0WdT zfgR}7DWR3EMER_szn;G}=UP~tx{JDp#5C$j>aV*EZRh62-K}d~a6+%_&6`KHUU=TA z1pxT6T{Ck29MM>zC7Kubk4c%Cn{>rOqhn*cyHq6?zlNQG5T#?po?v#aysS?KTS!-E zvEuUx?I}}cZIf%KHQfWsLPs<-l#5*t$Z~d0_RR(5M}}N*mWI&qY5O$4VwCzxck%{{ znwdFCM)J0ECfL8yUH47joE4bNgH1a_PCoB&6x~MUaMQ7~x0jP?S!`#AJ1>nvm>mw{ z>^Q6$JTZ)_QDUUskD}p-y#m5KX=W%z13EhwbD325*e@TQvnzo%)I@cu$7N-cpvqEH z8!6}re`GTvg%Dul7A=~LcMqVQT2MnZDd^~!8aEHa;9@JQ*BMTIqAc7uZoC!HeS?tb zP>ct-JWHHW&2#|GD*g^P_x)C?^A;{l)OHmKaiY7T;Q>LW_J8#DZM~Cs?zMef>WzwWg4bKu7w8_blxM^$)kqtaWDF7izhfuGXxJb(W zK;TmB6^Dg!4EXTLLuc?lP;7h)3J!cUrv=jT1oBjr1@bHNEMh~3yecj2$j@634#s4H zu`)GfaKAaUIHpO^eW0}cU(!JKWO`CovCGyiTO#DA-$+TBIe)&#gM*_Qe=r(0rb?qj z7CRvP6>Wm`6qd0WEP;FXuUo~?$nRGB4iwf}6FbEH~?voKK%A= zHO5@PH$~X;X&7FuFERDshvBMZJid3#8Vid7`XlaLmiErpXpeiaAz%PI(U3F zswFs`lE+$)G4t`s>1d|Pg1Exg{%sT67E^IZVbmn&y@wA!2Rv|;pmzXnDapwb9d8}b z-g7W$v@GJ(K1=z~!og_={5u`QR~k!;3g0;S1S<-e+AlF#r%a$vC(rI5a4i*Y;dh7v z7x530`PI+l*T+b9Pd}Z|!C9kw2Rnk7aDp^$n%w$rV6drFc<>kZnTP*$ZJY6rVnPRy zJRgaIir(`*pwp3bd~d_6#v$mYYO(u8wz8BTrVeV&o+IJBf5@@nn&Lox?^FPp`J7}G zYctoMdBLJR^_d3r2jHCPUNyI&1)%UJ(NBPAV5oLHEc;wnw;3A@ustL+RNxf({suOi z!bw0CKYlz!RFwV`Jwy(q27XMwFcU7FVVC5T6i${P&(58Pm7KGdos|Xg-O|(j7fw+> zq3PQ^~b6kHC)Rd)FB z&3}J+h!%>C(udpG{h*F2=7SgN!S)1my~Eu znk1NU0~P$u5Bo3zy8u&mEXQD;eBj_gha5L+l$||yR%uDYY0?eT%U1s$kH1Wpm)1`( zqI02D;86Ql3HLx|+Rot+qWF}Sm(QI4HB@ZWab1t&WGUcgL)&IQdAlT`8)U}pb zOG-aZ8{`)DQJ9S-|G}3~r)4z%GF7LcNU+V+wOajwYKD9oo^FEpK_^Y~L&K0W4?1lB zUwE++7i~B)h(LN|e{6Nm&d*Cr745X~GttGpduM21ARFe?C2a5q zlWk*?tSpkO(C|XK+<0Y19^BpC(a~$`>$fnsP*FJX(RoCkiz!$Qhl<38FGJoP zMkQd48LZNxtTUtcpBX}rSp{N2MtIAr81&@?B z>5f=G>`7jmqx*Z4Ug;qr=Fp({k0WRF>=XWVAvIOcz#v>k7=sRIm)46BM3^uud;0@w z(^KokLknk_nY}9rMBwuSd0|uA{`Z%LT^R@a(W5WWg~_vp2@;eQIa%4iwT1oZGy%gp z+B;BTT(moxt|0)K7_iIr8h!Nf70i)BZVxiT(#xs6R0Od<9pOACvRzkdZ|4A4y|t1yf= z{r_&Ib!N&DkE+@4Vb=1nv9M^JoON=rgv%+}nLa-4fMMRu&)*;E$!fxHrg=zyj4w}= z5W&cazVGbQ+1B1}yJ*qedGp%a+OXP=3^KM_vV`_hCd@1=BLhNp9iC&LAxyyTJ$qA~ab`LefYqwEWepbLF4ia_=(+LUV#KnG>K$)d(?`H}r)l$JRs}a_ZD6supHs zQ3WGN28Ev+<*3;pd|+>KnHv=~9spHSb1E%u{Q7Xyo4UzH^ulKsB%Buv`PLB-3L{5$ z;zXAdy9R~)R~2;V8ue!Cu?bCuwab& zkj4d~22+i9Rl$Al7l1bn)X6l=e>(LT%XCS?q_1{HLO^?nV!Pb=0JlWgl z<2P`8ED-HOLheJbGu6|&q3k?bE*nH9eOT|!8AIc~S2#*VHqs9Y-=uo)wOz0ey`bLkRNo-;&+dAP!Lq1fyEK5r_dpMTp^bn71m3 zSJ2c*S=k1LCR%;T2@r{()kK4B2)NtXBU1SkEv?slJdg>P8{(Bx0e1vVX1h53GS>_I z&%FSP7njM+z$FHLhs8NDdF??wx}@Hv7nhADqQ4AIvPr}6!v;lTGZ^c-e*N*p#K@3& z&@yCwXbNm8xv8mce7_HS<{`3B0L3LF_R70*YL{L2K6CSCcSF>JhUrPXe}BX-hRQT2 zYN6p0LhXA^)~;3AQDd9GWj1OW%p{pP>+02w>(|rm0V;vQAHw_`Dtz=uNmiu)bT^+h zixJe0KY#W|_UiDUq~D~wK^!4?SRmpWAP)UR^T;k&{+9aSD^Q6bEX1f4@1prci%X#KbDH*tI zSK8YfC3^fPqx^2hIm)#37t~4A@6SFrI#O~!%s8KtvKj3PNR9|&qSKal>&EqfBPxN zpFe*_#s2nq>F00Xy2ym=1X}5xJ^hOkUv@%l3m)${jgFl^G4M*n?>Eed!hG#x3_=%^a(JYX+mDm7G8AO-<^f;X}~&lnRNBmS<01trDBsHw2QY#!e_TCX7u=4~msL3U!49(8Lm{l}80i7?Vd3l7 zyVv?PO`ku{v9x4<7dsI(mzXxoc8CBxIXVVf^mpC1Z5tjLt{5OkL0nT0!OJw&(b0=Y z+yE@N@DPbB(19EP-UB5D&7f@8Z~UObdnT=~P=&r6F+yeR*eNVvS<5fd7B59e(KTmO zegoH00)_XCesn%2#`eE9ad?C|9tOz&#Wpr<9CtRLUSCX0i^3M1kpb$$v;rE7o3vW^ zhC%+0pN|+|T)40qs*kZKdP2CbIQ=bZGAewd7q4DTUGqd*PHd@zg8;jUg))vPFj(0Z z`t@{O&(H5_$W}geCMx_*&e+gsjy~S`>uv(2AtlZs&GrlN8PHMNzhaSC?X&7pO*}s-BpMNV zbjj_RxowVDX((7k?4P+OdI{Z5AKmjx+r`uKU;j@ReP9_jZ~m+K(Q(0orUvFFUS$jD zt@M?%=)cE~?YAJz^C0SjOb_nOn)rc?=Q_^vyXB6luwa10P{FM+|Bp-cmTPI;&*quw zzo>t5($d^^LPJmD)swrNmnVC3^K3 z8q>l9joKVnI6L0zx|XLY)G=iD%Y?v@j8qL7Qhn89g_+qsFs z51liq#7as^oYq@v^@xbTp6>e+i)-S4fuUMjS_-`WCI9X@b#yj5$ER20Hg$TMYBrdb zfB!djRwgx-XH>7uN=c~$BRZb+X1s$c(yUta&-uQ-dZC4#@%x+TpA#20|3!FI6U=7Fj^rP5 z)yBvZ=WAy4zA>#wTTZNjhLRr7CA*S1!2Iy4yqqAXl|%`aNi4aRojqvqV9CTjm%v5( z`gkE;H1>0l4k>>4(7h^u+Vtrw^M~e9eV-nm!mH?LYySy+ZBUYN_Q?0zW}W92y^}&G z-W+c+d}!ZH1zC|_A##~cefmk5%o0DnG*fQpb+unrvu5p=@0+POe@CctjIrsPzfL_R z0Y8L)#_VjGUjM9l&yc2XV74dUDmT2e%YI3j0Y}10nTV(nN6g(Z5d{?s56iq^|NfyY zb^yAPwziF;_(i)b^s9@q#;1=C{4n5iVvtDT+H;2=DUR-Av0{RR zNU;k|rS`*dkL$fnyp{A5#B4(F5FM16A`)5%Ns5DImsOFXtQ*d2)n#AE$O{(fj-Y!702*LSz=(6!~CJPP)79PB*uGrb*mepdB~wj14;9 z9fjKm9@n)R=if;`Qp$u=dHEQr@c%AcVV;N50|A(s8CB$5Z0CunhxnCP@?ez~MC0Dy z^LDCp(T9)y9Ozz zU;qB-?$hZIrcLWiJUXMx(-I;0N!(-tgyEZfw2~-`n5d{;+Mc_#TEM)B&6PKIa-G7I zSj+Ft&5l>rzaW)UeHoKAY2xmkTv`VXH>V#t-iTKXRS2-4U1D%h{3G!T0fUsLs;LR!Ms0PTK0v9_bH1ja&83| zV*Jt=7Dfh!Q=5D!`Xg@}uj}!nwYBo9M?BC6QG1d!cQZVre&d9D!#FM4thKdCl9icK z83fG#$X+MtZ=OHjt;p2^x9LWXOkBAG*vmFhFnGy5jn@Y3=8_~5A?(c|V?{ia>QeE1 z)C(76B_$UxU!Inpt|>l1;8*eY$E-piQ!c4%lw*!9moVA?rg(mTpLg_eJw)Whqj^a4 zkV#~DYP1>GT4CNyEsf3gaif2u8Ee6)~^ds>K%8zN3vs{Hv(YF*H13 zrqpYsj*`+{?(eyYtZA_ z8lW{wUcRZOW|zod001UXEWO4t^aby-Tn7ai>8@D?EQPKP9CYUvTP^Ymh z10BE*FKuI;J>3X}-b~3_c^In3EOYaNT-89|_dl&--`-dl2bC6y|40;;$?lPcVPie9 zZuGKA+*p})2E#oihVl-QLpfo_S4YQ(JCw4P!lXu7^j%U_5W=Zphtij2-UY&b6jD=H zmk2Y%g$Tg|6h#q*k>cqz%jPLTIuE@n+h3U|igN)K2ogq)Py$9_4JA)aZjI?mXUcRBLnTr*XA9%ySj{T?BWB$Ti`;$yN~B4i5{PO zf*4X_1Q=Wk+ijEPliPO4R*gt{`tTuy$bH@u<{AMyJ~s9zgSc|~K8%b3$yVO*S;yAV zK+%V%j$5baoY_HtPh9LLVtF!FQTSPk1Q#EWTER2GS!@DBKAyzoOLyMP5U=iZ!6Z6^ zW|*L{j)VM{sRL@pb%Z@#v5 zQ#(n37;8CDM~*DPyAff&lP(4`7lA|5v`*&@8Pn>!yN{0#tttl?pp0FHTt+v_RXbco z6Lb9dm)crhZ}iS|mrNsN466pYoB3CJc4c*DYA=g06i+B!Oh-X8}%yxYgh))vdk zZS7$BZ%iXFJk%EZh;3xVd01sMiJ-q!ck1#W3+3IK3g2Q1#wp%$fi5R6F)>Qf+^vXU zO8-o2M^V5ww*}vuxC^{DC)>aK&lboPQ`6mCGPYZh8|MZN7~r;H!{36RB2syLf2L2E z1F<~w9D6)#fda#}@Wh~xX;>q8Wlk=pG+Io;3`&F8&#FwS3lcWtZ4uKyei-gjr=-J zaYYJan90f9bA{iv<5CEs*ABeWOek}aNyf5oh%DGJe{mfX6cL;}aE)|dg1^$1YdJts zB&FrihcJ{8Fc^lQ;M&_tX5dxoT{*x+oGf-|2NMMQpRQ@^?81Qr#dK)OI>c(2jw2T$ zxjv79QCLH09+Lku3+x9OIW!#*l#>71uGJ<^-00yk%VRpRimlEqkl`VvD7TFp18Qyu zk*_b?T~TAK;;N>2Uwj}WAq(Zh7Rp{n@iXsE(ssVL4-N=i$W>nSoQK|SA#}N<3ovLr zWd&=*9ge4Gugq9w0&Biok$l6t8=r9f@0OGr7a+hPa}UrzqP1&wC4?tFj+0zGuv?Kt zHF?bw%v{_db>Z^mU%!7N!wD&ULLCa&bQ=ob^rcH4fNkSpKY#yT2)p3vIa?gvzT?8W z^NQlCOvI zP`E-4y-=|mYJ)kY>@%&*J!rP0!!yph$hJihPlmLvFmX?o}DkR|(g778- zk)nESw<%k3YI&$hH_X4WqVWo8iv`|inA3Np%_8;$CmSO$maC(frFsB*7|Ql*SZOY6c~$eHd9b^h>#(qx#Ka-)+u8Rk zSy}tB^?&^u{bKB9Jvl4&KN-f$`qpzR*Sr&YL#J1kmk;-iOzNI+!JfNVuhg>X+MQ1s6|WWwMusLmH>|EIs_MI z#Ef5r7R|LXyT;~_<8yPDO_XffFFAqRBcH#2KMDo#&MJILTwg&}HmUjsTq0xQ@`Xtopn!GnJU+U`UaGebJ$~;nj6#Q^u?Q8d6I7%T* z@p*DuQB%3?PB8w$kOgYRG~=tiaZRwmlol@^9}+!EPRGk~EFV(g7!Ux+8Oz)`_FmNq zq+*&Ca>T@=JF%~_va}fT-B)sR1#>MRO!+D6C4~1Z=a#~y*AAw{3e5KMRMIDmCCq?( z0vZ#n{d2~per!#+g=zhEtyg2J8`foY^}de`jad}{G`X210(7QY{`>M`0#+*sO94On z*(b_E;20-w1#Ix6$3?i4+qVe&2_Q>%^5jD2OpsJRwKMc~wkh1QO0rG0RXh6ldt>9F z2_k#$z|F$n^)@+#z)86mEwLLCBpSAIOrZ0f_ebx1RQax0KG9NkB;CtCw6H+3Dlt)-Kb;7U}KMJtRP|jy7b~Y zmri!Dl zctI2~$_xD@x3zLN9amGTL>D8i3$D~b{6va{53(E=O{GO`I-iz?oa|eBw;ztze*JLAeID@5iK+lh`TPnir<`3b*Z_}0xq3C9RrA3 zwhXUb=uvL?EFk0KEctJ~QE>g6vvU@=4pjCNn9RKH1uz{;%SZ2nU6kRt-uGV<9`O3N zV!fWCwBzp~Me)6aO{*}uvhTRrB}R^AxRNFKq!0b=yCJC zw{AUn-~b)WO=CsMv;&&SJBKnta9p(DGM=@G#kP+D6wl_Q*<>MOmY!oki*dRMMSGxNxD&Z3u9ht#bp3dwMIb@+sUS6e% zE9!PRH$+I89Ab3kqGQ3ydE?%`ehr7ur4l$J*3q+q%kho;UzDA9T+aRf_8VtIX-F!G zP9z~@M^R^2NJ3_rg(NN`GBVndWRxT!6%s3S%phIfJ!K8NQi7BY+@hxFJp(xYNpT|*`yGBh;FPcOJIiM@*LB`NDf{|{~Hrz-4 zE8~Zw=p7pOxtDbR@}d(54&}#@Ii4YOxDZQIr^;8~$(pGk67A8q=y7dNRedAzf>Ds}Ho>O62=lq4cK zIcx?5H>F>{u3x)mJ$`)Ng-MJ!TK)4625eYSQV|4Q^dVXx8P*D!GHFt<(8xTbBc{71 zBBbu@Pa5LJ-^%bkl|~Ecfy~9J`dLDv z4jeQ{RaqHM^(gbvc6JY`MY5)3om{bUrR%a~73m&;f8e{=g_#l-scK=A&YV#kp2P%b zge?z;LZ^b#k3j<8N=I5!MkZ&^9^d^!%>s>{q3xo>l^-9^LtIG+f<5CDyx1O^(OZzL zf$7tMFP_x4XvET1v?>niaoAkwMOvdl^qjrVxJhaB~t7InM*Pl%4@%ChtAAM^JkwycT$_Q*+wvX@*cR-7PT*I6|PLfvmR7-|HsKmLXo z!mUmA`tK>deq7(P zOwx9nXG&R?wTUAh+TO|AW2j*MN!I4w>$ItU9~Do+I(xQ*aK?oSM*A?Vc~n%_Rpahg zhCz`?{+_nB1mD~LbJ+f0NAtc$9mrf6r>mVlXJ;)J!AI zp;3F#cY2x!B3Ct{fkDMetl*mcQLAVs3J^3idVAg_mf>$)O#5@8?~$R)@==Vor><@T zdV^ewvv2%&Dxhq|wRKBTaOuxiNO8nW$GxR-UHTg?IjF;DymUL4B@kRtNBP{(r1s5! z_z;e`r`sy;|9=!j7>K{o#m)#DWRq zo2czFe&D5XpO8z2TAxc1iZSeQo~6Y)bLe{#Qj@-Uhc{Ha-(!Mo43H+0ICfc|Aoib~ zb$!KSLy?4VYI$h*ef>xLrCF8~p?5BNa6#(Bc^-+eNm%gvq4G^I{eI=L>_$A_<#+O#>6uw)i*iTt%ayX;@ z=+0c)Ais@ua6JYZd1@9dQr=`%^^hE9@n}Fb6-9K`b>F|wVB%O&M8UPSrIBlX3?vYj zMrfW@QQUtjvi^ey;{}VQt1d&i2)oy(6P&5ZI?6AlXBQV6#}sCOynB0(3$wx7s|TJ8 zOo`Qdf~I*gQQKobM)OxOc|#|zC=z}Xj4L3^WXKTdQG)j~PxblvxhKz`F9$_Ncn|PF z|8RfhR%EO#%|HL`6(&gf?4NKJATUXfRLQUt#Yz0d!RY5PkAOvvu#INW;?}LP{$k2U zD(&;<&QTM=(6@m}@d@TET7-ivFBoA}JkYr%s;itf#kN{`_rV4ZC+MfpmTNFdA)4uP}Tf#-})9jjSM&(B?3v zE=Y}%7D4xK=cw?d07^hOE+^O{cHGXtM>3C{&f+^exEc?l?a|Xv0;>k$X*xW15rjOH zg}_(x^O8~kB0%cdd%(Qnu(hwuAss~$<~4Aw09K#~1QGv3>z(4r*}qOXDUy&KsbtNN zKXSzAguq|?w+Z49Up^TNKaQhKCR+lz(oP-hZDAqpOHRiHN)v#FgjsWE6^?swGXAhhScGQ z;B}*Dr&%h98kUP)Ag()PRGtq9`6K!sde*W1FP5&;!3PE2cezk9|aNhoTj?o=Gp&XWOve9fTZGn}5#6f)`A zJU@xa0?=Y;;QF+w*Jz0$;TyNNWz0p~%*GUZN5{sa7q3$&fAc(((Fy34>h>A!(3^Tf zA8@4ex~>`;Rtb+M?$PN)e;ppqz`aWnS~9>o#&dY3}5&sbABhogW=tOpdZiT$i>DW+1tBt!TVH^HEB=^mQ zc~@)=G?=ENMSKTXz;i{X1R@N*fJ~1tKr~131c*R01t`@yB!?#yk(fzG#zh3=LTG=9 z-y+Z)#>bOj;^;`S2U!8?LWaQ28>X{|(93jhNX~1-LkyLh_eeT_9{J}(XA?YTx!n9L zhr`3))Yd{;)BdA_>$@!ptVaOKn!e{eecDA5%75RFw&!?af8F$Bxw3KyDkAFXSq~Sc zERkbQV&+WQ6aB+%kRP{l5$)ruVYx^R43vWQgS}7{fa!^4JUMj#*D#e&CvAY13wJjdVrT*YMs|ifI&6f;h=;nO zeuU)(Pc2<_)l$s7wH4#e%p=hL0#|^&xFqS0UVjQy96YK~1D{p>F&pkfQ-Izw@<;(< zr0MG7!k6Yv%`iF$;+!AhE4Jan;ibY=VYLQ;2CaHg%(Vqy8RzG-`|A9rn&wk-`RVfD z8Zpf0Iz#MWrw#vq$G(p z%fY0;N5Uqli`WeVez+UZDzjbv*8svc65ibF2hY03(uv#_Y^;bE%&lDn9$UQ{JBkj` zQ-A;X!4{FP1Z2=JMl$iSjIyI4W=x0$7IBv@QH#ZF{_?WV{3_aP9OM}*J8|liu-3^i z{4D44ubne?!JHXE98d1RnHtOr>zs%~hX~k{<4d5J_#l|t(EBI_zW(?TS1~>)KMopp zrpz#B?UUBbC>$NPf@>1zlbWs${U>IbPlN3Rz@EhGqamTq;(y zGSFCFaHI=%xoa!O0f6y=|1>qNGd)Ioi;i8|BLROYr1Pef1$BGPG0Aj48pbpbKQiAJ zPup3<@Ve@M_^0Rs^v7=1TtH7C5S~D3g5N-TItQ*(ur+$o>ZGTXopX)nK<|a;Vc^h? zxI>tpr}P4)*!T%|h_%QgVq$#AvvkiV95B$~%P|lS*E(wW@FKU%7;rEca)Nb4Wo3UT zU%)2_rP!i~?LYc1x_Is!Vf;+nJ90At2^qeXx-}EwNE`9&txTIgV&uq+97K{C#E5ME zWJ(?Sq~*(Vm!xnXp=Mz4pjV04X%tDtqT9;>Urvsc2^wg$&V*IVdg_ zsa;~AfmnqR9|XAn6n%DH$vHkYe8miSTz@gJDmShQcrTUumcb5@#CH`UAVG>o*TE%V zrwdILBfB@ZHx2FE_dPH*FLR>03s@)7|MI0vBr5VWBr@)@7Y{H_Hy^DG?}2bA7z|K& z@%PBmP)X8fvM3FaY-;Q2ZRgDLWubvL660{##l>m2na@LQj2sN%F%l9cIyh`begIxb zTBp<^0MIz_zg7DYGXj=fzi|VSgF5NKMdeN+xZ}4RWfl>E_eVxX5(^?92%cH|M^Pwm zd+xk>OHy92tpd$Ebtx{TLJXlI0>q8sKznLwHtf=fKhf zic+kO!rW3KspeR7fi;3fnq|m&!55LN#)eIJY5^IAvy)b^HBT4b14UG?urmT*nrtst z_ua-g&e>U@J)0Y9r(;J);r;QWZJhnxCG}J)#6;vq%h#;=T3szUS&W`;1+CY~CqHjg{YD1NW9{nR;ttb8S!$%eW*DBw?Mi3uq7mjaSh8?B%{`Rw+B z6L#J86X1SNpbyFZ##%HQdLcQy5d=yV%$nc2wGcXz!^{yQ#ljzOu6NRrqd=zE!2beA z0&K33VekC3w|$>czkYC5vf(500X&Y+LiV#PgSz2l z?VOr`Cwy;3tN0nzwCbYKK5aKwHIeSf8zc~F3M$<57ca^y1=CWJGao&g>g))G&gbIp zXE-(?V%| zqtXfyWV=i%f%Nr{XhyyP%Dj+3OMBWU*0zA1XLJtiLSYG!&>&oV$w})w<;VsK>UZy4 z|IT;@stV&-bEkALEk#XC!8drY+Cjt3Le{35!FoQH;k|$?B!c-Dg9EL4dTP~G;-{7%P^@B}yX3B-`=p5z6_Q*1+6ir)YRBL9 z&?b$OWa*@c%Yfn(j#8aXw%#EfP_-baV9LAb6S924qh@LXX{YtJLzgR3^oE*RTDX(2 zz%9(q5QtlUM4sPfB~;r#NKceFvEK)Ip6n!=&cc_+kHZJ}zH%-4g5b~GmNk^b>-U%7g< zin8*y`qy&7*4MZGT*+*8W~Q%oFG?^~llA(<6+j+b570N!5%9t8Ih!zk28?}hra zBOh8$D1fCyqxc&Y`35)Qv|ImSynV$@$#lwZx9PMTnF*{$dvJC_+uB!QqZK~}o-^It(+)edHi-7c#qfJ+bpB|17Ga%vS=60&M*s`_;`9t zbZ@+%`}d8EjUVD;b9HNm?1rV%p3@|wYdvrq0Ff79kBj|Wa%Hclj<2*Gy?EZFNoP}2 zzZb>*!yq){AuQGQ0woI{P5huiCYG#1MjMD><;z9w?mQgKOU~`B$ zb$-6Bd_XazELNspz6?xUd3*g^C6j!yBA!$c4EDvI{J=fDf5vf!#GAfPHbnT zg5a8l0D%CP#UsV+*3U%lb7S8<53seIE`Ioj2OE;5~cpI-5{{P*-5L&x3R{zDhhEH0bM} zfSc%!1`jR--Zj#s+H(O4AiHV@TZ7QgyS;t1zZlUYkCYu#9AaPrKIDcm{D|AH)!%vg z^d~GvoLiO@_MD2MuDI!A2g)EAcH+GAcpQt+mz(f zs(iQ(*HT^vs~4oaG1AUlqV1xO>R3RV!AUJ%1jcR_?Xj;oX2& zqn1C?WAQK*7JW_Z$cdLnF$5QE-F(iz?O%W>zN@7irpC-#$jHjlg%x$K-Ef_4nSg47 zzgKoO@~7{u_Z7LHqzF{6z`NKT=*kDI!|!iN3dKlf%T}b;blcpEZ<HS?_;LUwQ%d+kIVMngN~`tO>?pxK;q^IlttuNGmM|^u!4d(<6;w zjJ_HO0tiu3OFtvm67PW)!S+n2b}BGJhr5GWmIvZ_*$2{GF!IsT?$Skwuu=Q6tEIZc zpT9PY;-06QH1A42;geq!pCLKW2b6>++5y2w3*R#k&G<}xTk9V_H@^_RID4P6H{vkW z9uQVDS`BXf&%<|Nj16=2MQoccb)Y{kX>;Xchmm+bn#c zRQ63^2b;q6BH9F~bxsyTf>f|Yn!?wgu>ac+2D^b1==oT2!~97{`5%;8zzM}}!wGw% zA8h4fLAKj_WP>w%57EtKLIWBGC*m(3KdwZkmDbdG)h0Q~2;cNmjFag1`zlt{=+Wc& z;lqwIXJ#{ZiWAJM|Z+qCHyB{6k*1`Dk=e3-%D4Mzo~00aUM5w2hKp|NS* z#w7wQgR`H%e0dfPCF%sg1x9nQn?#8rr+R=J6FbfC^kV8b4kx$b;N$buXk4D|=AOxf z;gB4hRaWDab4~ZnAAFVRQP-d{hLxniuHdG>~30)8)4heB%hpn_hbg^<6vK2Zan`9RSpKJa4;x+NJB1x zaY?wq{>vu2k`kGRyC|V5D=k35&Yx%1F&UL+S1oPp{h=l%p==Q*7=s$nF2Y^RUAU0# zw=H}@0AG~#%ra8tT*57i5a7?6O!Bq#(nQkWwQHUTzu8_v0^&MB-SWS5H9%0n z;%wQVaS^~hvH&L*G@zxhPNngO*XGc6pWZ3wS86usHGVv)5EN22<1CXf0!jvl_vFcj zySq9$%ieL19~8Pua91@kse1QLUeqb}y=F+xnP&V%NuJUUQnLo^nqz6Kg>L7B>-~xb zR#*P#3lNQCS$+YzV-Nw8w@KR%Ga@)SC67{jew!3x06zlNgw1@;66sGPbV^fu{G<)w zjsVbMIxiJ2GcTT~w|v<$>CUSV1vBBy(i7`ml0p!PH}0(I8pCkh>`?zrQq6G;tToK# z4A$IL zCg$yTI4mGvpgaI4)ao@ZyHFc4ng-g0lQw2&;$$SHuZoKiHw4?p6S&|{N$^!x(zEJ$ zYnCgWI0Q5h1TkjOnCZB3Wkiqtn>{>G@W^$34N#4jSM=1r8o?-YDSKc%V1}4oXB^zd z0H8oNGK^LmRG6K>JRO{7xKzwoe)%$%-!wnC3y!0CJkH|5d-skQHjLnnDz~ZWeGDA} zBqAYO65yO7_~pa#)omb35ophB?vxKPh$|;dYafj~W=9C9^m_b9Rsi2%xr+-Wp#P9d z%$73nzK3n%8MR$m&NzuI8?|bw39i$T@EJ6LGJYl~AOYbp z)h^BFPO{KWTT_$aI0*Q)3^*9Gw-zRnFy%)VATZ)j6XDM}0$i*~V9COm-V;&^>oW)?H0Jy~ ztnGAk)X^D+{@2-wi39KwpiSaA<*CiIX+G>C;TX zt$s>{0ZV%_W5(GhI82{hG#4ZTtt;lGPo6Z8A1&HjJe-_j_!^+R$Cieor=$%K*uF3u zDk1827dJPXUSUv9McP@IMPsVe&-Dfl$ zu=W`%x-C$0CQGT5`OAW3B8V?2D(8*1priaHWC2C*sHzwU$rv}ai{|Fr5F}VZQ&P&b zeI&&ZH-uTSvAGNfS;e77d{|VZ?)LnxQuX?D0@}2xQx{#|(67h-4EJi8<^pCQ!4wgS z>*3=KX-K`#^d223^dbj?)62if%A-Wwj+_7H&1u)KZ>5!CMDm55=`kV=L$p=To&%P> zzS|-F%0}#Gao3u&c@p_t9cMi7BoIj97Adnoq0cVF0FdzgB{4{=+p1_V&aTz~_5ep6lQftl+ zt3{wrc7XB2QgMWMXq*!;btu=vD_Klel!eLyF+pCzznl?L6nP}B*o`iZda&i$m;kFd zEY)$;l9?>%QgUW@=4Qbo~F4O*RlcIM0ZOpea7E%Bk}Rl&*wJRJy&Y%b$`T-W0M=P2`@Su zZYhqfpK4=J!V9o;?opca)?w{R7Z-!X6B9S~cbiVL;k`-G__Y)(I$)lvp@Czt+~G96rpHXyHem(F-%-wj>dBIhe(#7qC?=lR&hUV3vANuwpr8lF> zsBOtJye9&^^R2+Cn|M=X)6CUPWCL!xbM*-Fd7sILd@WbPg(Bj2PE_5sdxBv&T`i3s zbtqO1d9IgW%Fj@N!?Mw969uSw)eEeQwc6{l)reSZ_+~%?_uODy;L0Bjo*U52MFx(f z{9$2TKXpZ3`aNf?5wJKbwrr)@f*PwfdYwj!tfY3 z&+`5?uRm993;-yzXWw=-o+FVg=FZddkZu_UM-7r}V<40N0{QW}bo@7-!VAj$`3;Gv zqosvqJvr%XWrmt&qm$Pmga@dFp^q5@#WEaRBsu$8em-Mh<&Pf^BavLas@3p>QE(<9 z5PX4LhwR>6^Q3|l&vKTR$g#OlTxBpD!D;Hob<)E=#-YE?1$yD<)LoTV#L~TdAjTBrlzDwmjN4d9;Dq*le0;G^n_&Lzs5NjIpdk_+d2ns zifn-803APW7X%-%qq3qVLVrkLwO-K>Do{TZilpSFOCMv+^6J%^zaC$@ggd1`^0>l+ zcFQUyNI7~`b*qzqR{=Pqks@?(%~;Ga;Ur@mww%DMEWhk(Km@+E2tE0V71G^=-8Z3% z;LCJe{0aF94AgI~7xg>A6AU(UQf`3PO(HIOw~;T#4x#7|v*7?+8-_EFtj9ox zBSDtQ@mRP80u(rwFqD^0p%R^vQoq(K1=f#W#hE(l_bYW7yySg&RrVHr&kztSJw3hH z#y?#O-mb0KCK&c|XaVBZo}KM8YDtPG%08Y9RrTKuJne{wG5pn7*+7uV`1{hxYevj4uI5l79J8c)Cd!PmSB9^j8^OQ(2P~ zVSR@R0>V4vXMYs0PM!b!%m_F?V36p0P{4$de=5(vuY6nIvt2p*XYT1iZ3+ffZ6UHV zcUXyiy4m7{Da!Ah=P9r5Q1IFqsQD$fRmgTTFh*3*fQ z+_$!+F${mhb*Hi}dwyR8ytKpLj@1RR1DF9Fluv1^%Qi6-ibaD{$CMvO*#}Eia3>=s z(qU7}j=;cHGr8S4uL&^tVYIv!ju;vMiTWva(?Q0hZt2qY$5%4}4XRUx#sDB{szcqQ z1@G3UBkZfR?cE10UuA|x%ZMhLQ~G9jL#Lab!VUqas129!D5EA27w3ZNqo(YbU?!;J z+i!e+dF9ie*N=GaK?+5GBEb0tD2a&3c|$7E+0ZUDqyyZA#58qn1X%5V1`5a+*s|Q1 zJI-u>F0re5UveL<5Sojcl1D8QCrz2MFx7=sBcsFf1nVxOfGkz@E>QUCzygk63_Dqv zOxj3&1xoSzWOLgIc~G&=6!#gP?C=d$+=HFoB51j$N$;MjGpFZ_^KQN$W zjWZ()H5s|&3U;JJejzY}xsR}ou#Lyn23-w>o9Qn?Pw(j8XcsMRk$~2Iat-gxT;s zk@U#o>*)DzBW6aOz}F!G)1zQYHpoUIg&V9mrvb!+Zv>XgS{X-U*P&6&m<41R&%fF+ zC>w}@77D&RA~KS(#QUyti0`=Slzw)GaPX8*>QX{BFxEoF#|PF^eO_8hmb838t4Eks z3M^r~Zpm5XC+sU+mtIaog`slp8_8=sr9cf`T~rJFbs9hpBQF9ZCm@S46uD{mMi-q% zO)6J9i5i2qXf|dHI9Q++I2gART!Ym+^QCLmMd_&g5bV&!K{Tn%h!~4N2t1tnkw5~g z&nE%Ofe#GIhE@si)rRq+aZE^15RKLEad%FyFxF}jrpceBdmNyP_&IpeF2_L(VQ?97 z5uB`S48S^tS;68LEC)g5VrjnFyocILdoIY=xs}C|fSY(1ovFs&hCt0RSGOlovyv;SY}Kz~6<2bB(p!^K0BqdGDZO_%UDw zKS=hVy%Eb-^wCrsp$fLI?F4Z!I~JDxQ7njIU+{hC8biuNf(ooN}}m zNj}6jsK!O^Flg8IX!Up-w5;NNX~nm%9}JK z1P@+x%pX+EWQr(uP?rX!`&+JFxx$J|nGdz3L+}vsAdE`NOG;9K?I6$W=P5U6`RN+z zNP{$*nN?1j>`fti#i==}Zm^#rPq)e?PsPS0F4x|4>6%&OiG*sk}FFoK4$ zE7!6zs->gHdnbRNWl}%auAkZdCRp3L*ASq7m~Bki`Fm6#M1Ll|L44`oYz)@By52+N z`RLIgzI@M~5~_@4%j9%$Dj5R01BrnN2?}|*J$QDO8|99O_>4xx<_}gg+y;tb96-xgD)5p%UE?1OHrr2yaArKt&JMFnh_&WOC-3Ulukp7#pO}cnO!}4|r`dI|cj7jj z3k?n5QDL(hg@lhkFd|JR;{`tL#}AC9tb27Iy&l5JfDCM5;RN1fXNV#br{f_82B#R{ z;KNewG7dxCOyw(hM~KyVJf7w_`*aQM6oaH6>sYub%xe&x6X{p2`p<4Cg<%6$Au!2C zm}5;FZ3(;=daGzUZxDL$If4hp30F-j@HA2pcmxM+FQvfs0B{s{JYwDMf`v3lf{?XQ z(IM6#d17dQ)&pdMPYtuhGjDDFl!RX%Q{Hxl%r&7Xb&aUXMDYqik3va4ASxzCVR$|& zPm&Y+%0<6Q`2^fRek>-`p*vHMjsW2xo)Z=g~|ri46-PmD|1hDE)>MjlPm}nZ9g@9yl5A|=Y z+s^w0w@tZpsj#4c7b-A1{cQ-52m=IinC-o35K{aG`x)&6dUWlI;p+x!+HZny8k-i5 zMMcF;D&^OA&HMZ|e z|KU_!&5H3UQvhP4;4sNNJgPt=Vb|?$3Tsmd9|)c0qpYmz z+@mxSIM&5!%e9OnrZEEq8~UcceqGO=ag=bVNxsZC-`%cAR+krvfVb`0Scc0m8Ufm3 z%EjY=g!}g|U%Z$F6=v&Ri^S_!uU(tLmJNS@JT7CXCU~n65wqbzPM>~9)du&4yDQ$G zxrK!myakRC7sF>#L^p!xtlpGuseD)35B?oCOGH4JeB543j+$Y}#sD~z26WlX5XFat zJlK4gh0sG8SQ(trA~Q=B;MW^_)+412kEdA>ZO|u`#54+9B{?Fui!c2rNRsev=TNeB0hFqg_ysvg$56;Tv)YKDw zqsS+Wi(n5;7YL&Z4$t4jb{o=`tI1!cB_j4YpJ5{YzyX1=M19HpHf;jYjP)IePN#|0 zZ?F7n#|pAhqoBBp%nY1OUeHbIkKHOt9ZLZQ>_`ty<+2RJb2m3eEIy%l|MUrI7^+O# z9ik0_SGR3XA3of}S)^!!ui_hVu>8_71g4HS8XB5eSm-o=KDsYqJ@K+-Ke50dOR}y* zBFQ4dbG51@?B!reK~Gngs1EQ#N6Ae{==)L5bqdsYkNt#}R|EmP)JKlI%@G971GEN& zLN!D(<4p9>)orFq2UR{%T+2x;G4$XJ5|uc!Bu5H6L}PxxW&C4#(}*%qvk7_xk&Lvz znCB=S!3Zz|JA6h8H^DdyLr-I^`566m>ox#K@1$~Y>kU-*tT3cQyqlYg!416$$OuCS z_?{tCM>4e(*S2WxWuU z5@twXSdP<>;PPnOUI~XvafJL`X7+g!54)m?)LeJZty}5RvFH=LLcVpFMxhcWf#B9f65?d3npW40fC?S+(j|c=$;HI6Sc)h@4Kt{cBTQwXE~vwslaC z?H7$-@&f3CnDhJa@gvuQ;Vk`K;(!y6$9abrM<_~N*0G&MG&o*?8> zUv`kF>+8pWV~_?Im)G6fqgyvzNBpy^u~gVQfdr10Y`Trj8I+%+Mr{VS;I2M>D=Z4n zeM9?CRde^Q2#d9d2o4Ii>8#xI@bqjXzTW;!<&lVw_l+CdP%iT-H|uT0On`>`=FOY1 zwa#;-q@Iq1j9~KAfCVj3)u>h4~nmGY%z4>lQ9lk%V&7 zIkernm0|H>nWS*}8p1B;yaVl(<# z7ri`Ew7k6eE@*G=Rr85oo zXP>O?aP3z^15?Prlaz4M&jlNk`==d<9i2PN-n^YTjrKJ#X9>ZVB8r`%_)(#>(8)UA zE~2mFh@sA+sN}WtR1)u8Mq0r);cjJaN9-V%dj#l@FHDf~o&AAqMbXO{;O~mRj~O-U zH+vq~6Ls&NA?$HM!IWNMzB3jg>xWjNKm=RGP@0J%5C`1m*$uwdnNF3f5MkQ|>2_!+ zVGGqw)X}4YOjpqrA15Xn?>L(f;%H+;G7`gZjt!*^{xSI0;S28&z7Ufb zm@*y2U=7^E5n22p&fUC;i6kqGADfr)Nx@l2CH!m@HE8lJ#ROVan=xz$#CU^wbEQC; zV-KmtIAYEM1OwSQ*U{0%#l_bMM9`Va6g&ciiy>`TWrPMbHRa4z-|s<;#^!{~4YJ7{ zV>$%QILUz;PI}Xc25pk5cvKjqL9dgccuEmCrL+ zhXotIb*pYj4uH^Og?H_bIJ>dm`ukZjcS!%;Fm7Vb)Bn{1_=qk)Q5O|&OIR@@so~HW zJVn{10#Z-xr0R^^9(P&G(zQc`?cQCx7(1rg8BNhsUVe<(TsA!y^x}0LJ${@W=rX!v zc)}2`Y01frhYWSS{$xbOU~5Zv_`&N9F^j|0u=}c5PI}d>%l9O|Ys)CTtInHSSPV8Y zvPoLm@boJyUw8@68W3z$V(ihf5I>q{`k^;`C!C}LfHV;_X4J?=M##Nytb8w9RbAal zON&{5*(kfyPSO>+#Y)95IgQxlMkR6Y4nnCO+|CxeeCL&Q~XM=B`3o|(!#8Gxr-O|%2_7sJe?s> zJ!mUHAW0Qz00Hae4(pS}nP2hbv$eBxadp+=&h`+yUt*fbTb=s~>y69~-2R%aw{ne% zUyNT-^|?p5=F3#NK!>7H0W@w^sNHsDrG~#N%^MGVijVpRwa{XpfA1oj5qx%%+|-!f zDtW!LvZs=K>C(#=w1&$yTRzS^eanfzj?4+<>GZArPAhAMJdDO zM&4qsRnh-Wb~c}pc?^oD(0ihnvq3Xr`=X11e%})uOgTgx24w{2pxY!iad~NJDSY@w zpk?ANYqoBEnVl`LDSRhRre*tL(jHCF<3fJBS#qqMY4FAA3uz!=W+?jksc0l_AVp&U7#m)0?U51xGRsS4 zOVI@)jmrhiFhs>0`~gOfX8#Xk`L-VKXwD~&A77kwlotccLrSXBnMXaXqr+-IHfnd_ z4q!DwIV2cDTQcB?0}f-wSe>)RLy(RNMg#)~vdx2d+pphQ@F7ZVs$R4u7idjMIzYGt zc-SM$V`{sU_wC)gkU2D_zHDsp?R2CBr>@hlPo=S{VACjl7y&&3$~HdrJOlqV{PriB9nue+x2;MoLg4UV3$TKVZl0WJUDR<`7#9* zh{XfA@G}Yz{LAZY2-eI{c5-!-9{3eSBmp5PJK^+cic;P_0hw#<8@rT38^u2J@%+c; zH9q82hC&oYuPNPGpH5ASY$qd6b$`y1^lR6ECaHKce;opDVpeBsdK&MHMupxt+I$OR zCr|I)J56Lye5WeohrI>Gz!o$GR0>(IesI3Id33g3lhd}-_?hS!4;`ve`_wnP+pSXR z1<0MJh{qn9GYhjVP*o+3GhhY?0ZML3K%YRFtf46({_GL8SCD(fJJLnb{5aiFoyimi@zr6;&%)-xIsP zulI_Rj{~5KX!gZ*`}dPInBf62LAK@j>vI$XyZ}XzxR!T? z2_PCiw1Y&yf`Tt}RCo;lp;27?`8XcL2{4OOLsCQ>^r80!cBxDKv=^`bclMSoFchf3 zIo_CQa2jyS6-JiiZ(G#6GNpF<^tJSK1P%h^c`4K?j-td$T{N_d+&(7Hq)Dkn!a)o5NwmNxTV%CXb6Zc5Tbt{U#jA`9llO9x% zE3sxojz4$Vx_{JXu{`k0%MsW#(IEO+esdTC;>Qv}vA77$2$B+B6JQpoh5(P)+q)Ax zXuU^_=wfitWZ1BPJsAv#D*9vi5XTUufd*5)ApAKDN>ErZBEY63osNf)VsRw9cSo4Y zO@zNh$F20}HcX7Vs1IL0J|6Ra-&v_bLwWWIPi}BuQIs7mp#0e*yn)0=1DsfKBXTR^ zLX?$t+jEbTh51#nTf=|z#95n2J5>{b2HoV?+3*>Y42kUgqKo`T9g~_3V7-3_ zeJCU$uvRZwa_8Q?t1Q^1gAnf#;_2MEKpNDLg1>2(E~wjn>^JH~i#K}nSNg&5=xBx> z^MpBIeTy|KSL)&)X({G?b6t4xjPO!k-hL^=7%j&QB^d#fLq*lWhZwx@`f$G9(YN`O zZ#8n@yIR+Vz=YJQuplxISdedzqM4Av^F5uAkTm99n;6*z<7(|x^@7jF+b;3}RMj|= z>6>jr!W8Oyo*M(#NN6PcGuOl6R#DlTbObO-KF|tT15c9MCs+%!VDa*0z=V^?y6Wm+BgDn)*U=JSih1 zgKq%XM(ARoj`11OWviLU#-JPPwY?KKuAEU$oEqL__fG7$p6BTJxh$=a6-8jnWOfcX zu^w1OJ^&g|tiFo|n@RAi_LjQ1LEc@C+b8^P>To6Y9eqGGpjFmjVk55Bw zCogXuFMVe7LB(_K-X+dbvJ3&9&|jG2Y{Cbj4}|Mb@`M)gfQ4e z-*5CI+-MfG-C$qEo;@5(aGR2&Zi(i(6dWj35}uobEa5Bv066h=bu|JeO>J#@)n6Ab z@{iZKyVEM0nwg=&#n_=fX8oR)qwl`5W-pLOO+(*SCv;5;3gGy1AdG7Vc9!C_(vs6x zaf4YzI$-c%BoyzNNfxWe8qA(E=PI64zZO%GxetWYIdy8sgp+VLTRc4he`xe+v?-8S zWS@KQo&W%rl|gKXre0SRoj`lx>dKs|EAMgDDmyDHK?=Tg>pYhAJa_WhxOuaX!+5Z} zQj3?mx}FBxGc=rFW%U{bO=Zd}H+HAK2fV7OVHXKC5yS?r66}amh`$;;RFKJ4WfN1W zs56&}%+q-AAP^W+Y}t~YoE%D$a&+|MwE(vO$#Qvst*j$EDO%&i#DrkAJGCOaIml#) z@zq7(`a5>+8Vd@R+@8vp70|IzWFkma8Se>xKQOAot3>^@)H&`USiKO5HH964`2#!LU z4%{N%S)(Ru=#U}NINuPRGTNFyk$zbX25t)Rs-vzh?%jLl!iB_%0uW-K5}vwTu?Qn0 z&M&f6#?QgzAa@LnjBo!l9%D@Pyyv zY{>;=LEKUn143o?#+|oes-e9RQ`xkKESw?Y07?OnqZuSVu-rnZ35ivqzc>}sU0vTn z+WVbLq?Z%6GExnZOsT)Q>=REGlf^kcn2ot_-1z0g2Vs2JeQvQcLmkG(FjS+>%m#30 z$`bsJs)&BluKBe7gb#P*Sg|gKdHE&0O8FmxRpiJSm8^hUMi|#jB`7VQgtk8*W8}hf_W?ip#i@;GcA3 z^I%7lESUjfu{*bnav#$XP~la+vo{wK|x6^9{|Kd{X}a*n)E-CuRnM| zQ$p3?1B&eN*aQrO5you24d~<;m^g+DJ9H?e`^$8XzGTuxYXEl?{kZ}B2*{qvHucQb z&U3_`!k%9X7MS<$Eqm8g3R7{aU2Izz1(8~2gR?KSuuupuD5WDslaKU~mrFQFm_FFu z!$W`U)`Nk8%Xrk(nxHZe!gLbx<9}zBJj}^q+7(3Nad|mror0erPoz0h599=kVC;MV z{)q5!KZ%KWNk0b6M6>Ca7;IyJSfBXe$gDrk*`r2jpVB%M5y5xB0m1^8A=`NWj#72+ zy6NaBBH?Cu_1QE3=uG_5t$UHFBJY_)*vfw;dlq?H|4RsPE}R;MN0O~Ojimf z^PqMnZNv>6;9&DHNW$c`d$(>#Tb)s?hv#t_=sPy&Nro9%b#<n=O zSUS6|20V1r-jq#;?-XZQrj5M6OaE z?_poS9EHv)u$yn+rHdC=bBsBx7O&^fzZef3h{%?}$YUT1P$pxZ>{@Qrzdyq%Bo9nk zv`@Jzcjy2{2?)+iGy_%0Vo3LtixjSWe)7I7H!q%=3;}M@&E_g8k>0Rb{aFI2;*Vnb zU{q9xV?a$PeE{9TtD4gnXtz_ zA45;WgYB_jVM0evk)A5_XU4`)j5bGNlTWohr-+gAwbwG1145gCDW}x1_X`{z`E4Oiv$ec>rApw|e%>nQ&D@Ei7UpBE}nrlW~m(PY=ba$3fG`6inBjM%}bj`3`1rB=|E~!`MaQzW(?F2j(8^K7WeL6EOm1`934GHtgdraRkmZxVBeUS z_R2_$KS5G~R>c!X>ooRP7l4&d%A6*m5ie}vc#}FW-&Aj^A8BbnevdwyJ-*=6Y$xp> zaqhwe3_HM|Vekb{bk4X&FeX#Ga^DCd=BOy7p(D-B@l2hksT9HHu2kK1rh4>j$p47Q zwEN7UF%C(@&vRLiQ6sOF4q7B3+>(93N~Df%K_$=KbczBHb_E4 z0!9l=&p#3#u&pgRW`1Qod7_Eu1QRg3cT@2I_@AZa?#r87hQ$t4I5vtyis+V{%hpq=!dBoT zU{yK?@(j`gJ`-&e%Xk4O#gB5bv)O1o!@(f|)VfCx)dPR%uK6}l!N0$hDFkHE?NTfW z_1K3G9fGo%Ji3dxXmSu4kuuKKoNnNl3A{Z)hWD;E3uWvJ<-n{_nd_uBt;HahY54~75N(IZF*Fr68RwVu}e zy>cn2Dw9GPD;A>)!S8mI#f117emA8=66i;Z?-BYp$Jp4TCW}jQZkOQT=hLqbW*qIk zC$07QJQERGb80too4N>W8KG~R;iQuOaG#b|tv7C8NlvcIotoS(4<_TQ*fwpYzR?cZ z=J?Xmxm#E4Yws+zalU2W#-olX;Ba%DJ6BqxzOwlRKfHre9b;*h6Uf;uZXoS5!zpZ$i%ET{FPK zbED+}!Jf48=uAb4J?gp0yKW?PRG>a@m!jFZLA!I#f{Bl=?i1K#OBES8$E7)d&xM`X zAFa7{X>5Me)7Nhw74-!UV*jpPjXxTs!QnvFq$JBWZMsUa)BdFuj28{3?3_!LckzY- zUg0XBfobb*m`VKenr%>4P%yAxKfI>-gLl>0K^89rr_9Xe`gaB0@H!d1y1_?3R9%*r zmibK@4@r;-K-czDzktksb|3O3(WY^9&{!#F`OI`G8imq@|&;C786OQ z{m;F&&;qprVyv9u)Rd43=WZXfy$e?}bK*56V$AUAm-Hv1q&~BU%pocn(%u{xQvLVH zsKNzsZHdhduR>^{X|$*~DJD=j_}Ll490lPTd%gmh2=y)R-!>zu=Q)oN;-Qe7A62zw0%J^1Pq?cL^0QaRB~YvE8l(O)}hL}S3T+|J}cw^*TP9Phiil7 zu)VQUpIgE%I|~boms#4O)-0D(IaWcEV8^zUs5ddJ@YrbCo2K~ZLkjrLIQ9&(IdJb@ z6_JElHu4GRAl0*#Rj;vQyZgt2I*|xabFN>vuC%P|abmt~zYcn;*hMd1wW{L;VYwpB zZfR*(NI_I?LpCY;Q~WT=iR6JsIMVDZl?^U=9PM=hkz~X z)6cO=jCzWw56445tosEw0fWtJK|x-8a|Ey~PzynWnNCPC(FI^A08+u*4?##tXFVVv zSeRe*nGKOAc|I-Qm9hNA%FTY}PH`=_i9Le&*g8|eGjEH)B z3sc+h6axl4xO;bp>-l45n=jJ*gT2xoL#&X^PhAmed*&;FgaMBUW{Uodh*aL%78w=> zW$z*ESXmomGokg68^om2}? zAy~H4FFEG}uM0#Yfd+H={DJ~G-9&CCG#zK6=|JxQ+jxpy)?hR>H}fAvL=?gPqYl~_ z(53w5@l#)1?*0AF9eWrG-a|?96xU2QCx`ms(ax6Cs}8ruF|{ zEkMZpE76ac6RZGEq;ACg#8l7wZGM-`JSL%e4nWiJ4Hy{o(9lTrY4--$p~XIOTwWxh ztU}4k&jfYUSzDXY_(@nnWEt)h zWSp3BkY^be0ocRxyjPe`j~;!gS) zoP+7|)y5=JS;7x+w+Ah_X8HK3bltCCI8kq@tCHUJ9@bjMee2@UQR&^c-xFU{N8e zY|iO%bj_5f*dU&k7D>1Xh}|hB;21jME_X|=!l$!||6Ct)q9<-y(Z`OV(enAWS^DRr zv^#0wI^P#9v;|IF|wxFvwuQ7 z(dV*RD=js3D|G>TLI7C+K#Gpw^CmYYtg2{|$7MbuUxrn_vRUKNyucx#c<9TGy07j;> zh6b0F#-H;jkw}ML#mg{w-@bA9$xa!cN-sy5$^whpi3sv&SHMtd3rQ$cLo2 zqQa|7ht6z-@bH-F;K05r+C?gkwOBAuzKH~YQ7@ct0ZDnF3|PHr$K}zXF>f zVBj3RXp!KY-L>oVnKLKb*#RZ^=je1gVlycn^?8sUb+V()|z(twInCaO+o2aja42 zNIXhm5S3w>b!j{(XdhB28>f%h7yeGcHy8Snufq?9`x9ED=3T&7edPn_L1jH)a<700 zGS$LU2CYQI0Ml{h3Q}jRm^J)knQ5)7a|a7}DVF_{3SoS< zvg-9~upj_`PO2k#`v>X_Dp!a+{BCE?{K3;j79q8!iVjUS&=oY1o6tYPcJ0z354+2= zPPARJc5M=6z3F7;CS>28W9);X^~m>@Lj&hSXYvbz{y=y*lriy<#0@^oj$~$!Sl`s6 z>QHQyDmk>lRuJ+r$Y!(UA9b8v5G_IG@`DNfX4mikg50D0ZQPI&ISj(dR@E5{Y+0w z(`I+mwHuZ@*AvFPF1ZX%hngf`(}p{*ZiVs3E!cjp!VOJq+~e#s=GKY=vdGQU1$eSa z;%C5;gKDk#8a}wo1tcxbCWjdqY-}{u*RO18G1uKI|4vmkNOng&wjQcMyg`%%MB`W& zCh7npGAg?gq@9N#%q1f+Tq0Jdg)NLWQ@FVy)4l1?<{(O57Qum!1!WImgI~3K{|LOG zJ8NmlpPF}dO+|Bjn(t1!b9zSc4~3c?er6DoUET8OnEA2Q4}ALk`6kxRbxmlU>Cl)m zDSp_U_gh=bkilDIB?thCYi!#lm~ym0>wGWQ4-WYZ)`Ij0@I){wHZrQMsS!0g`*em{ z6=xti24xfod^3G{+i{ss*trLQ!)KuQAaq%-Lx`}FfD3luXrV4*2?Ed(+tJvNuBfP} zXSS)M4vr|VEhQkkV8_^U(H!Go%A~8Z{dENIB{iPh!xfvPXxmt*q7-;S?LdiDJgaI( z&Zaz=6J2A|K1g z__At7AAk`2;J+2}GAF1L!WJ z#Ze|o=LHguO0?F14}libTPXVXU_%=Fl$(DualuO!wmYLqTCl(zxgaAY>`i949~K{! z4*nkL$?jKoM(y z&@2*-bUF%FGE5?H-pHKJF|qsAi(AGv?39bJMeWNOhq25wV*ZUzsNwD@t||j<2#_-$ z!>fea)bv*yjoHfb<#eyy>b{MG(U-n`yKvE>(-w2v?)M9~Gh|8kSd5v*j$McW`pJ{n zKpp&5Is3Pqq=?)??h)i6`s45`C?RP|B+e54>Bqya2RW*p07&qQ&0*`(>BQ2V3j*=hxOi-v$j4AovfdSo?q`Mma^P<{ZFh1(@g z^BZX0??dTcvWD9Q`?ZrNKXQDC?xv;)mA*>opw6tF_4zV@mda-M zu5iX{f&2KO)cdtiiGcVF=LMZS&gavQXUE87%@|bCbM5UdQ)lMx8Hpc*EjOMRL`a4K zKlOD+=})|+l2`UUPw~zhHuG!T^9h=%$vpAaGVbX2j>me zR|m}j!pIH$#dNJpmp(yOPTLZPdBJ3ns@7qun*2qOT;MWWG~^&MCII-SyIvgEltmfR zN+NRyhduN}h|3}SSy~f1uoVPMtrdV?7Qg~{!S%5wrsfxlZo((OBuZH3RBeqsVBE=5hg@9M@STuBc5Lk$l0^!(cmDu_|F(=qU5ES z#BX5!ZSUU4z)JI_pJNgFGbGKR3-4myfG*oDHE9#k3k{+=0Drn|=ZkZi0?r?25`=yv zZkEcM3qp3s0Hh%~SAIT3jNu?~-aCXFxCTx{R>EAgtF{CFXZ5zeb2$apmfT~mU(eZHS1B0NqPoz9cQ}Fy;ky%UhgAIVelV}-+qF7G!H^1`=yQwBNrg|rpB4> ztqJ+}@BFF;P-j98%|7v!uZC6?RvC%?S1ibXjVl0kGt*!8{VxYh5XDI0i#IO_{oa(b1q+hWf@)5kg!V&!i)^=hh3p)R26B9#U; zT-|l>ps@E5+TlijH@|hNO)~&&lxyG(sV8)zryxvbz7L;!J^;4lV-M1nV&+t?sMAxQ z`Q1C2Qh%au<5n{V+$tf*I21Wkhb3XOj530PdW4G5tm?n6b~x>5oK*m=!Yp!xr8|U! zzrRV0^>f;FObZcONW>JY+vm>1BRwVMHCk(a570vr6|vQl#CMTs31x+33YMJix1`ry zC0b*VOb>c3Ufh&=R#rx0Sl`fqfc?Joh@f5uD_?dVzp-;mr@ZTr*ku9kVCC~LulINd@@6b==_fw@-3Iiw1(I3x!I11f%IVu8St zfH&9y6Y!hd%QmfDnEAoTgxBk7meO*55it>656);-y$=Uel7-Hf{<85p(m@9&Yc zW2wp*2KqviT3Z8QGzspxUSsio`MuX|=s4~c(URLK`7|u`=ut$O{)5JU55tTnC1REY z>ZxHoVCLW^Nv>>+H9kc!eECW%HlS~UnlL*PO7iaBx9^|R67{jZ9((i#kE70o&|)NC z%EE5Lh8*ao`wO87p`(9Cty~YE%ua%1>ru5(jDE=Nl?OZ!@bK}u|9UK32uf%fGZbt{ ztMSkJGC#(uaX#-}Am4>}&h0>QDwDg@`Y_m;0ao@REJ07iGKnK&CZNm&6$U@;Eo%Xk zfUtoGK$pl2?(d!9lxB`!*$QIkTZrkC z_WfJEY0Al%zLuzR1bM>TcZge(A)M&g`q|I__U&8EFw8@eR1(JR+qFa zb9dJqG$`Vw^wrhxA5goHZlFSG7}Jo?wTi#1w~p5zV%C+Km12^Y38P+VZ!~r+djI%6 zd;VI0brp%yZ?EY}x01UKKf(e$=y&~imzgt#l?s&3jKbzwF{jR`{dr&;|AlT6MYzKg zaD5|g0}QWXD31v5%a=`J3@91q`$G4g3YlOJ;HrFB{_GhNLx8I%ZIzXS2Hx+FXb5W$ z`f-$#u76svq~u`n%n(C;d3mMNnem!2h>cI4KFzgWu`fY|DEj$x08Sp90OBlqROuI* z(jk>>jHNOLT~Wr>)GU+_MOjFO707=EPuB1hfk-s+e=^=z7I*tS`HMX{T`cYs8<6lb zi0UY)D7~D{*F3+xzKkP86r`RyX&paV>j8O%Rw|+e)_)Irie*7$h+{Bq#^_$I3r|sa z@$L|ZJ%=JDicyXdrjZ{N6>a7r1peWP|4aRZxdq9Xes%o5eP4h6#Ls^#bY6TslRfew z=6djmGe-iQkQ#86V6jqq?!}p;w5&`3^bJdx>SF~t!ax*3%!<y*qC1O!s)MVmlBuJsD>Ny)6IQHT`KqEK`@~86!9SqMK)JLG0iUD#WfJF@KYsWA zJ$+7e{HCPS&?MHwJTStsfK)0h0UtAlu`@wRfw#CIKN@rr`W^xjIXNOhBy3<&K0L#s zNO$=zU^#dJB6-?_X{cIZlBrGvw{e;!`F$uKz}1WpK@ABaF?Ms|>&Jm+&f~s%4JZ_! zRcg9i`z}o0vcm$QMQTJaScQYZ=jNZdjRT9NB*Pd7DK4fY8a4STNe=BV8YyN9nF}S( zkpmxi_5U@Mz)M7Cji5r@6m(ov`_x`@=f1$5e)uP!_zuhrig!NpHO%o3hQeh?tWBgx`0_{CIrCb1de!LwRtTun9VOtNs zkbH@FBjss5;huA}u2}iHgQ%-T*GyG&drPZOXFVy+8>8K4ceNwrN?NpJeq z>e+Z6N$k1C*6dr++(y&J?6QA9{Js|TeKpG2mPczG8XXt}jgDsHwK|J>sy8zfMty>m zkY6)q&NN!{;bG&Mcq&vrEV1sz-=RR_ZV!cUdLp1S6gcN&V`p)D5FBnzL3Tis0ZZVQ zm)C5USRO+h$G$LcpNO!Jl`*4X0Z|7jy3mNtVI z7W}{z+K6Ckiy7akxgg9TF!_&^%Ww}WQPmh~?*pKhmin~W<2DkJsL$53?THE*|6!pm zjf|Jdkjq399SsJ=4n0Zsi@+32nR0vQfT>8rgq6&A7B$t+r)vZP!2b&L#wHWYe;46V zhJO=E9s1do+1Pn>FNyLZ>F zXXlsolnVjySb z{`uJ_-ew*qj5Ov_tRhi_V0W~(?&UP?UF2=3KkD&boj3V;a_&hMQ&VS>E?0PZGTkbt zy{n>a{Pih4X_6MPhx(N>%(g4CAbl&k_xIvQ4n*ZUK3NTV(?!K3#9V|9{PpWlt9cV< zoWtRr<#NX~C?J`T5Fa700>rT;7z}Mc`9o>x44C}-^8vI9_~1ToC$K&nRfBX0Tbv9; zdl=dje&Ngwk%pyx^qxI)q4BcXeKPA%KQc^oGj|ILY7hlNK~|>n@@Z#tH?V-|Y!OX2 zQRHu^r6qf+Yesz~Y%d@d<_?TNsOPe9_$zlU1H^J!$oXJybpPB1S!vq83xyleLzlv& zXKoI)Jma)@f3TIA<`zLCFLo%sIUI#(Ve)nX^ek9DSMX9q9$$w?B%4HuC2T~16GP*_i&-+{0ns}+C`c!{qgNZJP?{q| z~*EBN>>-aVJ65v~5W~Tp%7Q%=3v8Q;wE3`?n$l;1h?B@3%FD zb~=(rc26pAO`**Qr7`;03N9&4RpkiIAA{z5qoaexcv%-PzRBqzfpgMmdo!NG=UlR2 z!MCH!=_w1F8hKx0GVegl)ER_?jnxobgx*3 zUe+pR6Sg=V+__Uav=6fbUN0|Kpy+V%Pj}KaO2GO_qeqXoU%uF;&MfL)iQPY351(nT zuJcZmtaulxNAdMz_w8e00Xme&(3@njy?fi68mG%<1S-c1EnP(6xp zGsn9rZtfqrk8dAUgwD4k4_Q(1Ag|l4+T<*w)M4$=4s+KBbLX&!q~t7t4JkZ&7niPY+A#T5|$&vpq$MyAki)G69M%Gntz&sFAk$JrZ?^hWfAfTf;>CmBr zT`^?a%8ora9o+JI$T!_Vg!cRQ+r7CLm9H6(YK4f-JGgMcz->R9F|Tdx$e$=01d$!` zcHr>g$h8>0BY-}A_H5QECn?Wg1S|Tj9mB_KHsl^T0q_QS@!3LuDmivaI#2)DO;H>S zDo!lHuK-^R7+**^7zA2sq@-t{nBaQEGkD&bYb+Qqwc!l%D3AfOHOOz#F2V1#&bUac zfM0_#K&$fR>r_@SfC}t{ypmg6+w{0-svs-Pvfkah z53_Ha5q+%mwAj3YvR6LTZSUMl=RUKv{Y+j$hi)4`syBVUk@0un=ulkWPsaL`O*W68 zv`#PXYob0~OF4JlaU6yF|mpU4OD|EG$l9$_h6ZHScZ@jNqI*;D1|lCWkcrnXUg zIms%+@ZUQZCYFya#tfQCkCNI|*)D9G)!vnFu5`nWYfA(n%-8s4WbZAhmAEY+uvNoh z14uHS;aULdU&jG@U62Z{rxS$86j}vLg5Dji1PuZ@bQw!*c&xbeTZleU2BA?H%Q_(7 z9}03Js-vN-G&=+%jpL{mvRsD6SZ~|0xm+ldl4QFDDqUHtM@! zT125c8-YZA#|eDARfwH|f!X+wftr~V1argrUf@rXjO@~#W<+h@9ulEeTu-ou*&eHQ zk2^unJEWirPWSrJOOP>`89gg5KAD?qrm6W_qH!pqdeRcj7;*|r?>1Jfga?8ja&mAO zuwr|+#QqZbwzYcHBwjdv?m)B_Ima;2r013~vq3|wWV^c3=5lCY(68pYNd-$CQDqa1 zV3c3~JKp~qg-YSP9jribiU-%~nyJ#@A`l~yWE2zRf*)T|HC#AuVKxU0yXM#-<}z60f#X+|XR~F~WQU1es za`kVsESU6xI}yY8r~V1rM(+h32}h-Fuo94#wK%|T9OU2%0qew*A|hw?r!>0tcuVgh z+W6)8aob0F-eJiY?T`r2Hcy!^g8x3zBf=70=;-Ye*`h*q=Vr@j5{VJI{Ki{#6)U>= zc;$kL(l;t^ z3=%(OPczdYRHh7w#nQTcr zouaYM^%z4!v`-(RcNDhHLOEb^32x@2a;A`w?>%@>v9}X3jA!H5g#`vyco&{oVuAFK zP7lpOBBR{o)FVf@cRicBQ1CeG?xlFubNy4EGJL=Q^cTX)19AxOlvP}k4cQa4cUfu9 zz=-NdW)b$gzO)sSY!QS4?|~S@4}PN~f`jW104K za3$1jgs-06yE7%sOC}5bZf*`p9D!)tO%G8OId5|T+wD7U7%2g>^7JTDXu-!p?I57! z>yT0C@lsiM-aGKVx;pGu1u{>X4WXhFbQ$O;u{eUg6A-|-)~FA3VWU0GUhhb#5!9Gv z@4rJ_@I=(VXYpCVxWvKjAEP8aYZUeRQMl#}`j#kLzW;b?>gaoC6`D^{_+$qD`Gpml zQGzMMM)R+2q}>#ltMx~n+>bpUucEbd^Grr?*YN#lRO7YHL@5h{t_-YoUhJ-2l;KyY7twE9Yq33=Z7CQ-7_WYJAFVZ%W=Kyj#$R8R z9ARq#18xjVGn}WksPQR)639Yo0EENwpeTon#SAczZ7|5?P&mDOd6hD@Zp<99ygS}S zpFeY_Ifsa!-@JXRJ!~2%mc0_lGUv`sAWTwIGrhZP*^oE`jPPmY18K+u3q3rjdx=aC ze>s?e1Eb)6h&jLr;7UZ!NoK5>j}7|$`64y4bae$y;aeVOY0UCQ%e^kJVGX<|_>=3OtG(eT`vaF%%Mp>ro@~=wH4Rc0tm!aUsOcAK@_;^h) z4AYO6Q5_NU&m2CC{kGTq`57L!cGmT|z+-?O;9)PKw0yGz;74jP6vP&t6P zpM!h_6*05kHb<{*x+40P2d~ub1rrMxBS}4Ll9kRUdEXeizroL+88nbujI|dV8?$|h zGujEZi5NDsfdFNXw;`}H?tI-M=?-R}ni6ktASu{3K+Nc@3pB;yoO7`ME~A}poerCe zfrk*RG6uq`CH1JIsMbxdXI6aTXA&1#YeA_CU8iS`aXsBe4y1}zNQ}9T%1j*FK43V7b zSPnV(CBz(g0=vy9aGdVZ1;L*Kk~vtC!398?#mKX{n7SG3U8an1vgW+CtwrJY2Y>FR zrgqUs%4-U{c9=T2awVa-1`B&+ralG+ikr*mdwXW`4c@Q*m*k&d8@qES+g%Jsj9>%> z>d`GXhC{_!rwZGeLN3gUY{+rQt8QSvJ>EuJaWm!PcN@4jTh44IY!Nc#_vmtc)Q&Tez^ChXe-mNymWVCG|0RI7^1gt#q z5E~kpC<^uc2;{4)!;~F<2OBmOh4y}vQDq@mDr|fpJ*yaL?v5Szz{*=wlFl^y2Vyu# z3Id@&*0#_<3iUcb%4&w4aV3hKWKJG#37CrU3GFWOh{(2Xzb4H%cl`JgFRwc{Z^|F; zucBbF$YO@dLl8clTxRun%&ZNr{?dNlEY=}ct}k3O5kSl@x;k4^^7?ezy5p{=OhFt^ znODH*i9$H|B`&p{)eM$1nV7I0Me*mY?J_PqoJTq9(r-R#E`|dsd-zaope8FfQy_=2 zZ|jwsx9Ae$zU#lWld`jWS(QRb8|**L*347cm6O?<5IjQZ(;>+}MMk@gMLO%m5+6)- zzBRXyM1qQs47iyai-wYxCPiLL=E~j7x)Weyc}89!FmeDP4g_Oi^9uiJ$JP+h&APAC*Tja^-DurxqWxU8 zU5I(MOo(}F0>*wM8m9R(olb!p_!VW`Z|Km^4A+}A1%G{Yv>nx6pvYsCnOP7~wfgl% zn#UyfEi`6Rdd%FBMO z``Zs65K4E_)DCR^YNew>Hl|V2UtanR z>t#kZ86~35YqIuXZh)6r#MTs|2Ju*Dt8JW~g~>44Q1SR=tVn)B z@cNs+qGm-j@O;As{v$;Fi4$83-I3i=oXj9$!Eyl!k*HwKSX|RXMFsgO$BJB`AR7ur zekZ~BBB_QyRZw;R{c0ve>a)%0SWzXZD4>XBDc6>iV@;n)VWxpC2-#8AEkJ@zWqxM6 zz>}7iHhiiWSwD2PoeH5Ec8ZS9F@&kP4xPG>6irKGNH)7x^7Z ziq+TMqKXg$Vuph2U255TVe$%5X;cv;BQx0hsHW|7T%e3mgF!?>rZQ(lkCX4#6YVu! zZWy!pc+?vh88onmhTfb^M_=TKbkGnEfYSikt~x4LY0Qlo^Q5SVv0`*nfh>??QzRTC z!S(qU1ZEU>TnWkuY6$)m6fTWU8P|t*Z{J3N3wiNc^AMv9BxMc*Ody>YKsSX1wE)a^ z{rB&OOBeICAXuro>0r{;r}BhwqIM^J@~8N}6gyNW%tWroz)jHpEh(cx%GPBp9-gvC zhPw!mM6bcn1D^stL08M>^{p-0bcmX2qX-%Z2v4)^VbLP0nsvZ>n%cB;!Sjv|4pgCx z%Cpy^8B=bZtyFWE;ornku~Vp&$^&>#k`iEos)9DQKsP>nHi!8p6%~8hsBdn@&#;k#hhg%-~|Vv5Hl4OiJ-!$B(}Tu0lkxJ>?y6 z%fgAl{j-cIQ6~~j*b<2*qY`BePyDaNH7l7qcIoot>(_r9UWen6Bn)DRc`ItFs_bY8 z;%ouFP(x8kv4qIr-kQSNZ&WNW3b)-egdQ*t4_Nu>NqIS~p=ZER{dhs@LpxU{Q(^6R z{Ca7*Kt?i$aS%aSw{Gm941EBMl563l{LK7D$L zHDMH+=KCpchq21o!C?)3Kl{O0urZL7oRs8-MntTGT#oh$oUnS&C(>jWvPg?PN8kne zJU|Y>l|D&ZA0eQnDWpgc3Y)0^p&`{QbMne%3R_oTSgZzgv=d!|A==R}Y;UWz0$FJ4i-Xzp( zfovKlcJS!ZPM@|CF+G9Xi=vWJWN`w=i#D97N)S`coqGk(%yoyBSGV?o?E%iJDDcAN zeMb*_yo9z8OpE+7hI3D*rXnBXx$!e+sPNP+MV2#W_&hfdvnRfwDlb<)l&Y8t1t1RS ziO3P#wyj>ZYX67@ zH<Qv2|0dn|FWf*B_~U|MU;QOQP}&9u%t8>5)sL>n$$C&>m|Uz za)&nBu~YbX%TLq1(53CzLUi?^qT)QWP6@3$h!VG%zQ5UqwDoAm=UG5m>~^35obo%peK$j3~WLa*$u&XN^YD^f$LSXm^! zr5@yJ%34rR5@=w-P=aFl)=_pmgWI9)J7IuDK`8j16Jlj8VAjx1!{i24U29L5f_F|V zrFdi__CHPv5{LxN%ofbl29!6fP)qlsrl2D)Gp_XEL#?Y(7`ziyu!R||tII|(fDeqw zwvpC&aEGi*f|-P7Ls-lfNIw(x*y5UR&GMnp{K3I@^3KVO5~d}Pd5e1WiYTa3Hw&ck zFr@X%9z=;yo3X`wTwDx0tV||}$4oqg)A|kcRcvYa_xqOOn!uIjzw)1XkhoSG1)$k+ zlljUMqaEDTk*p)$XqaHzGs04^)?j0lrbsqs_ih@Nm zM?SrK$HZEZuRCa9@!~98Gxg(f0y&tFK+gI4=cM;v#P_Dw($zJW@|IGTzXw7GcVu`X z#KG(kpx~!7xRAZgs3L>Zd-SSCY+8KwCefkB45^#}>qh#PzQn_Nr6M+AMjWP?% z8-M(OVPJ&L<+5K)p}Uk-6I26oM5rNpT3j{|V^MXs%NH8>G{sPNIi7L~5pSW-fYNUN zU92HhVvT`7u+t2CGx-smAT8Y-EA{HmuPB+T@jo(o!;FIP@+h(4cfP!RyD8-uum*jr zxmah|u!*8K$kvJbbddo~ylY+(WHcC;E<8yX1u>DF&kW^f^zZj$glmk05^Q1A{~#`T z@%*`PmoYBFl(xznJcNx4v2E0v5B8>s05_;1O-=%?Ys;2x;6?BwNe8HjP1Nxr!xeNP zGF&e&L?m-DL?DF?`saujqp#rfmO<#D7??pLNjn4XWemGnX~zHp-Q8!py0UY55jqgA zouFzdha|E$7J4Y$X2L9xA0>&{g)Yc8}pSHY-d()x?-I1B}Q3p}-C# zBxJj+U>d1y&gC%v#aU+9nDX}NyhET!txQ0r&ysiFYZJ^LIc-D`msD=3~-Pb3;Vk$wIJtU zrqF9`8V#I5(cvk|oY0w%M~`Caxm5;DrMhkQw6Foooj|MavKR@cN%UBKbFo zUAAp$)#$Koiz4-s=qBZq!@~bVCE50`Lh;y^W@`S{o&VQz^Z)rbQ7{X~J;Xfe`G0>r z#J9uWYPGFCZ2Q;$m*4b%E@_3I)Y87+HO;1rOe+%wj6apo*bH+XV)8#)02M3DBp@9P zDKBAX;eOqmaUSx+r6l0yFeZ}aSLBlQ;ER(X@V zPUnygZ=6xG`5FJczAS%A2@ zUHEHR0F*se436tUeu}SP2wOcnl)|5=WCEc z*1G==-T(bOwXq^?AL9Sd!4^1;|GwXU5BmS(V4E51Y`r!RdD2t!QVsUP;9nhYE?P#)c438cZgY*3h90JGcD!IEx_%IyylQPIEwrM2N&% zkM?BX5|JQAx{}gv(-KYpK!{7qJgAx%-QD+Notf?W_03c;B%juP;gs5R8@_71^dl%D z_!HDo1L4)ySDalaxoylu zmq22D+jJO~N;xK<~%bqto%*B56iYbjj0n`;7jOnQ}5y0S{GRDXVp<7qP zOGnGlH^Bcd;=*FhO12)i;pditOvUX)aqQ@bGTw9P(o0Jp4MwEF17Q4*sec%LogqWg zT(TeBzb`tqSy-#fkH9`}!GcVWTgt=|SqmOuBdQ&SVd1pdF*5TE8Yy%z`tk6Yyk&s1 zMKqooY$=bv>xq133A@YDtX1#5AZ*6_TEotnhqS2y!hB(P4I@Z5;yHU|(Ld615JgzZ zT0q(Vq!LAzh0Y937Pf?dPyx#yG->`j*XwkaYyn&1n&pkHMt_FSJ8*oc%{d-EhR=5o z`iJQzD7XA;-vM9pp?(`SK#+&+qCg;FmwWw$d0_rDKa-{rSkhW7LZ=5E)zhSypUN5H z0Pz#jJCk$gqr;?!TS+}^dTMt&7$mSa2#Uvu6)rBX@7SkC{}-!os&zz)RNZ-y;3M%! zr&{IfBbbCO$XC9Fmlq9TWThfY9TiBGiER@o1dzFGi=oDuG-(m8s!Xm!R1P?h^e`9PNl;lZ5qDViSozUq+A{+7{x%!VEzP6 z!+AgDl0Eaejy~F16!=a~%=PtBQIXxr#YJUA5dn3_X-Y18U*631hB4qTpdjXx;ttLN zYQj199P=>rvqeRV_>1L33HoUJK7xglu3x{a#O37e@Qx_;z+|%9ff_IP`G}8J8(l$$ z3pxk8B~3zRGIZYMI40=*E>m~H^Ri@j1|#en7Uff{Rd}sMm$LsZgD=8X4&q15B#BGHQXhu9&0ugE0SlA;q!DMiw^2X6&VM$`s2M2 zVW}W5U;pC=y&NvY{LBjZP=-m^hA@;Djp{S!Fc_BBC@d^Y8rOT0=r9c_YCN`Lprhsm zkZB=X83G_4AvULdiq59m#*uFhU4{7})5%m90IoJ#nIp>ZAlz$KPR>+oYXqjVot^t4 zb|sF$&dG=3>&%pUEsPB$3|A7y0&=QL$Bvjy3>3*z0I&~I!ncI$1%@+vnEq@W9~w<4 zf+*qjIG&2rn}3aiNKFA7vIdJHgZVnTrHyp|`E%^`a=j|_ zIpoQ9S?+~fQI+i3yLVrFJToH6SFgUjd2=xbljmikUI+7!pj9&l&$Y28CTdZVAd`OF zFf^H5QI0lRSvHMO5gP}#WN|V5{0w7@p_q_Aagji(0B;5$cSc9c<2co9DZ(pDk2L}q zb92-wn{|79BxX^B?$?_y9V+J2o<4bU4ko0W;a;6@5RQc@lGCRXVqzK~A{bk#t?i(# z*l^(zN@cx2As1Fd*`au$NlZn}Yzx3SSd**qF9e~T&H>Yaf01JVU97bR>*_)d_f9Nn ztgp{g|Aw%ff)E5w4MCZHXy);tCE4GG`+APUlZv}G@lbY74rp1!#4oy}mi7VDRKlnf zVW0Ls$IRGR&^H0e`}GsL*(U9IoXB?2v*E7K8KKkKeoHcGg!TaNjiU!Wi~bZ?8e*Of zD=5hF<3`{%FD&rr+>L??H_S->4YPGZKwQ-|3W>5I-Ew-iWS3-z2yONRpG(- zvCxlt=57d9puatCZ0iLbj5wB3lg=t-G=>#%BNj|#p)i0Ue;T(n7vVe!FepgEU&GKj zbd=)eRm2-=Z`!vQ+qzdj#VDWAdGp93Lq-y~!VHUZEWy-_$%Kh-B79_pA>BEYTIadQ z_~m;0vdNn)kLV7y3uJSbgGPe~PhqPK?fZWHsyOxZU^6pUaS?5_Vc*wN)gqIw{K}Vh zKX`~r2CN2kq&pDq3?3N%fnE%UhjQ18($tfQ^Unk;0Nm?;W~2V%>VY1KV(2+=%T7+O_)H^e?^%5hK zLU#l*a_(IG>&e>d!PdnVTZuN_^<)!`Kw#Dif0#HJKWfR&g9qpFF_`J!nnD~EJbq`e zM|uCmLlKEW@L@<4q7)IWoUCkmg)rrU?GNn~#OE+7{P(L2|64>;rIWb{N(4Eg_7IdS zQ+aqE6zU`{?m2ECq!YE?z1jNo82LGcCDKc(TLv$Ip~2D&J|J73G-DLt2|+ER7ZWB_ z!^mSl;0=m3mAFBOhd0Vp?rY1$PKaO*6E<;Wb&{k0E zW#pKQ*l}8sc7E zeWSH(^4!MXjQ*du0f_k3)C5f6+E8tSV3}heyM}MxZ1(SW`OXV5sUUWc1IVw{A3|Zz z500ZBJVH+|ntx~0BBDJvw@UH}IUCouIQl_Q$8?r=h;IsrlQr8QgX-#4HFR1T~hFLtxUn;pt;v-TSCB zsgjJ3=k%!)CoUmYVNMQf7LxLk1_z&s)=jS`j)k$1 zEpv%gG~Hn1nFYpyPsT9?sWh(_9}wp`b6A|5Fm40xH>dDlro4bFA%?=YwZC_rQWb7* zTAW^fQBf_N4cuS1#7;?TYL$dm14J!yd8$(yAfOk1@Vbx|E=lxhP-*nPNA%Y}JU$rw~ z0|6K6EnMQlI$;J+h4LC7a3-UC_8xBQkjzC+4l~Y=qx9oVFapdkzK5*E;!#fpDLRs) z2*eInGrn_{xJUa8eFzb+5sA`HW3u)9`5(A4yuFveQcNqR^%<;NK|C;N-I^fnn8z_wv&A(8$)zR{#OQxoHlnl+rI2{#r zLvJx$8ZeKagS#2Ofq41)oM(y$(hL7Y=|uIw0N-4U0}Bem?s}Fj9;LWBEag~8 z2rj)lq2kFp7cY(p%eS+$%L3CaUHai#c&Dzih(#+ZDu}CKH%? z&d2RfSZfjQ_UkX)N(D{bKqs9phJYgopJ1J2o!iw7xTNL==}<9)e6WXsMQbSyLzIxl zj!BDXj?rAIO@=O0QpU4<6egYPOd_hbk>(0RH1G|u%(N6%-ZGrdf9NOg4n5AuEp;LN zOCi=FzZrR&=f$uOA1G{|xn_(bR-1HiAbw#^SzuG$5G)KU(8Q6sahlD^ZqsC$E8uR_ zwRy>d=R$t9_l%=RZf7jz6hak+pBdTx31$v$8VnX49Y1=sgS8QAxnw&-1B12i?kAIh zI5=`8QLHypEL?T-5!y5Jt43H z*&IZu0>QCRj{Pj`ieqtOHO(>J1`8mWb`i;TR8W|mvvz2x7$iG~zjVWi7lm_?X+ZKm1DLcU64EphMEQ7iz?&1X1*fVD!az z@E+Veu<7B~Pv95%iUutW7{QOZb1_JiYm|T-5&4$L2s}WgNq?(@GBkB58+gIGcz__8 z)KQUMC$-x+NMDQ`N2+88NP5qd)5nj)3()pz>d&8}bLu3`5uxnb_AG-=wH{B6TSDdj zwYh~RShAh`n;m2*ntrMYtRDb3651#kg;kT}8vYli!eB;Pt{(abbM+>T7NyIF2Kz{D zE+8JAt@y;xB5CtaFx!F)zX%LI*|^0rFnX1ugyz`&A$dM`RE?KBIE*e9C~%+8%gtKW%kA`E69 zQY=t{pus>#){hbKHaEP!LRrwQK}%p}9mFhv5J9Z11#?cJ zd)3XpAcOm=^p(?7>54E|jMiI;hUAMr0NVCHyKf`&gCa;Eg5wrGmgAH88=-)a8-xVP zKEBO4ar#A2#Z0xN7VzO~cz_cV5*|n|hSWmKCn(6wQ7{jtMHIBJ85xJ-=5G|V{7bYw zU-#w-f;8ipjj_<}p||NNV_-%1)rqla7IV@6z}QrA^T)b6q-aSGMx<$o#*r3vdkh&p zdQV9$BSxV!1bqn3```-3bBb#yqS2nf3L*sv6&%x~?%s9h0Q{nZWU&N_bH}$aQnXNV z6U}Z&Js;;buPf6$$=;7nH~=k7X-4y2(0_5kM<4@ z79kLq#CHu{!4HOm6%vialbux5IItWjQq3=ph`&S4;Nx4%u}AmymriF9WuNslandtB z<{pvNBvruhG75o=j9uBRbRdkJXBh}Efr#9p+kA);Lk(T)R*hIGbu)4(w~FYy{g#LH z)b5_@&|-3uve9zhALByYmYf#+oJPBvk?aQlacILe?h z{7n#n;`m#C(w$O>xhsCR(8B&C$Z_$%c>M68;1B|#a@>A!`8K4iN}XqzPeH5C*F4Yp zWiaqr0ejo{#Ux@w^$mnbLl{$Y!0DW%{yW5>HR|aJ3WqpFK-=ii%Q2Q1G)NF&SbEN% zpI_rki36GtM%j+2gCUWG%32^+C;tfQOIQoiI{vxJ5wfR^gUW8SKP40P(^4CeIlZrT ze|oG|y%V9hZP;rnNh)rRE2xO^16~;eo!t%$H|-GIU2xs!FQ}@mWuM;RloVecoq%xr zpL)WzOFEEeC$ESDI}xA^jGJ$nM_4Z`!rumEXf?*CFJHC--MKfEX6R(Pe!7$Q05-`| za|A|}#Bpj$QYVBF<1G2=-TG*=gpC;Y#d zsn80=!Y5oF zI)C|en!ucE3>G+~te?Q3-aIGll>6t);E^j==4vV9Jj34r^$bqrr#yJS8WGU(;}IQ4 z^Mc(hy5`3CMdi(YtKqDc;n3lx%-Y`N9$}+>8+PX}6^aL_l)oS>kZKD|HDb6@jI~PL zYE}yLX(#zEw2JL1qO6Fp!h`IA{<(g=ur<;5ym+ z3KOb&po5b-?SWF^wALNJs1-LZRXXh>YZ=@m3Oj{0RvPf8YUisXk#yD z3GN)o+diFX3S5Sw9lm59L4G-4r4-xS1La{Ypt~V`JmWXBVcq%!{l>?gsaI z;)D@GLBZMc=1u&Qgf{e@JE)Q6&sD)t=+&dT;5c?xbg)}~`hT_* zZz@pG$jTYqW5Ghyi(?UUw?^v}Rb~=a4-t+&tK}iYB z8+G>Ty=Ui6)~#$o+M1TuVTf1y`5*!v2#~rQ5Kb}E3d2^9oLd<0A=K9uogdq0njO_ z@c$u$-+pV3TT{D%U_e!Q&cE4L?_Ee^t;E_9l#I>}{O+6WGAwq;ux;%~`TRD>T>ABF zSpYI6DcGr83HWUgRU~6OgcP8?s~N_YM%MZb8?bt6H$i1S;^t+`mvdmSrjj83 zV-SmBB9bUbn7d>#9E)6TF)els^GcA2{rjnWgmlP# zc7my@W_eeQM7My^Uvo;z^g)$YvO1)Ftuu7>~SPSYY7pIraV(ilpe zMSBc}8Ae!QhC!i)`|!@71AGH0e@x4v9GO-aIAq94oZ&ItF0R3%5|-P$oZ)1CF=0>i zAOPn*qPK~e73+-Iw2F=nr%mi*NM92*V@RbCK)hbTfDC30-V{`TsxZwt?DyKSjDYeu zFl0jN1B=yTn2X_xMUSbC_yuKvO76J*2PLzXq?DAiFp#8S0SOI9!F2XkK~_yBO-K$F znf$eP7Qq8dgarq-pe)}G8J|pb!e5rI>;X(zu|l0jA(I$S>$AdfRdNz-<#%xZaVrW8 z5?s>DDH>TEMLs0K@Ls=jN*R2iA%ayO0E)NDXrOe6s7H^rye<9sZ4(bsNkJ3U$OK>T zs=#f*V8M-T(2#iX3vCA|wKZ#KM+jooDXD>hOZW+V!hzb_D=N;mk19eK9#ksPs{0U! zQyG*oZs2h?M_LXZFR94#S-^~3+f3(7s99oP%gh*pAT9>bk2t5~{qnQ1x%1@HTB&&# zfl|P}poU=bBfmI2{M8J*Ag+s=<3cHjECWs!iieC-?Vpj@O-)T$s>oz%<$wp|WQY~Y zbtEQMnq6^aPd3P1bMe0+dOSI;wcPdN4b=$#b&=Cqe!H6M0T@SlQQbhJ?Mun$WV%Qg zi2%AG3MU?-o1p0L1s9mNsi18`6+x6YX7=ZM87~WA2AkHcwWLS6AF7DA-)-T)RGhV%K0OVLhGrICf_n(kvaz+5oBnJnFWZ+a6&TgEguX5!sRZ z4$$a}dxflvO|JYvY5)WPjQUsRNkQc#+ff*I*=vvNANHui>jITuPR<6Hu6lK{I?AlR zmu*qAlAE}W*WE{yXMcYC*3r%mjU5ZSuDkmzTsWZo41L;9ENFN+fC?Q7Mn?s2E`EU@ zFEavwNT>1{Gz`x$WTG=Gv?$V_(>1V$ItZboYCcQ~>RX-?M-am>uLAUY* z_5+N4*At&4vM7yo4k5#+{51&b`KNl+EIT{BfM2WmIz|ce937FCQ7{cqQ2whEwOmI?1eHekLsqQ)^fdUJ0+7FO9&MM1my?X!p^hJ?7#qg|Cra;gk$KCq#07 zpzPQwio}Ff)GpV9c0)q5VA=9Q?C0rnCNaG0uU~ggyT;x@WYV3+n}d~U-jM;UwfO5+ zu2{k9cJZcm_6Ir-i=6}M`0N=Dc(i8u=HZ3zc+)};afIne5POI%G|DsjBarz-+C4=|CKl7;Rc^tKgm%g@#97HfrcRH1j(06lz zl*JMEjI2wSsC>fKeyOZvB2t*$0PCX%Bh|rp!s&6&6vAi1msfe4pF4}kU2HKgDuKJA zdKEcb2kCz}IG{H=oIN5(j~IbqwNK9jOal!xO;q|>&q{2VDsT^v<|X@74dxa*&PWCqKh>!I6Y4wEUh z--PpWW-ELr6FE2CS;TP;(F5|D_g}u8NluOdneAMzDcV9kL17}=Fbf~lls{ZD;3HW) z_@6F_Z6LGmLt0%3X4MOvg6`~dIv9A4PiA2Uw?VW5s~DMsNPqE&_yJeorqK;2h+p52 zBTJxcC2X(()nNHPG<()s?48=qyOBReC_`5d{d?YiFif}94+PBE&fMtLgGM_C_^g~@ zh3TI0@@fEzb?&nPBQ#%74WzO_Y*NEhnjLEYMCbME|#Ww4Bi9PE~%;=qJtv4HKTWz>#J<`#T> zHGE3v8~Bh$o?IXZz?)YhT`_^L56aeRoGvFT2`Kx0L-Mi3;j*#lW}@8pwxMaWF30|= z;939lKjYN*pEnyfOq$hLIU(TBxR#;CKNHJ_yZ!i+`s<0tC7n|_l63Rdm+hOB-oKDc zUB$yh2=KXamA@43t?b1dY5V<4u3};dv`tHp?;Zk4lAZ!i1$_r@9~5OSX5ELJ=*1ti zfHT}RT6KRmwBN}^XX=*1d*428zQQG8`zg}`zc$;B=C)%(=W}UiZkUs9AWh8>nGT@q z9vT|#K58k{+WJZaqNSN{=6@r8kF6bk*15!>%dy1Hlh!5{^N_l=bZ z0z)eL9lz^gA?Vq(Tr9a&-2h4r)+y?lJo7lO?APr~q+`<44$_9C#0|I`{o9xfiS};C z$6~aeM1{VwL0?O1>4%xeH8;-Ba0*&1OMUgnL{gQz9Zi;w4!#_(hzmfYpulUi@S&Jn z2L=qKHMB)AQJQ6XzWb8s_ItLT)A{Rn{&2(lWVEbFlqD(IQ4Z`o^AR zuLu*7tdIkJB-_!cfM!pfIViSxVP=K-;R%{C022(427&X(`iPCMx#=N!MU+Z_qM}uf z09H}4xyd1Ux$B`%8#1p8^&JCK?UJz_HI3el5@U|5YY@vTh#`k3pm?E9#5tlv?{sPy z@%aHwkFbh>fcWR1k$QSfuMNy1EFYjxY*Vyef>P!-6Uou2@jyZMG1gcZg9I%4$%l`R zHgq9qS!;!!-b0l*sy8A>m=lUvKmrY6gefU0d}Iv03f)JjN8y@EGKHFmv6Yo3&ps|I zBMoe9!-WudaW*)jEXu4v1Fxmr@tBTGL@$bElD3{bv&|R+jOxwEx{R+H!H?cYEkv4I zr;tSwTe(?vLLZv-Fs>eX--aoF1fJ)!A0Y%tz`7ktAyX*@ZS zyi#W#2O|*Vtuxx|?`Adv8ie~|HE9x}1R0l)^xzp`Ztsjq)TH0PelfDpBN7J=Qi&vX z=8mBDSVi7{CpB9Z_hf?>{&p6xxVjFqkG+WK18 z-EmR6z~9PN#$Vwuw5pvq6}mH*2-`1?u@-m1kkfnvzjx;G)ewmnFRt_O5Spv!&JpD~ za?$rT7FE;A;JArfcv*sVM^X~mkB_}QAO1>4&wF5dG9`b!u!&G zutVF(X-FE}+NmL+_E7DO=2)NFu3R}>N48>tuc2DfT4p~b2Q7p8XA0Cf4 zA4@BM@`zmN@g265$?k&DOWVpK`dZk*Nmp#A8$inKAr^ zBg};OKIU5#k=bcjo)|cBQGv@LBM7 zVgd|ngk|W5b_eAEbtuAo;*Sha?#N-waAMn(K|k5-%gpy4`%c2;@QSyNg=xiYF0lQZ zt<=Vhp*Qw{(h*S6?LrOwIV zDe4w(T4KqTyl#AGZx9bd zTeFQet9YmYK{4qSUfd|UX?T~@Ogwq=*^3uLMe<~eylC|vJ#fU4J-ip(MK6}`4bsjR zi$6Lt!;J?8F8RFmZ7r31SenCdGlrBQ!wx&?9+`QZ$N;Wp^y081nlM;R{)zTEdbH=_ zFaGf~FkpmtSZdGF$n|NSwTzBgAcFCp&00rYLR zJ@lyx8$VTq6;%t16wwU^?> z+PmV}B7&1BS}Wcj+DEbmx=B=WR4BNO;E&FRAEN{ocd+M$LxOHsZuby0$Mu+G@un0v zlL6Snhl+{`Y6hQSl*yv2=rf)@YkqqH91ULyJ&q^U{+l<*`a%YcNB6{L68Js#1L#a0 zlXT|HRfHr&n+oMLOU>-{-?aBk!P!x zqvPJl$f=06C>QQi5N(H=dPdM?Oqbyuij)Y;6a-FuENC&@?CiE8wQ@SYbdPSv58f@5 ztE|Tr{*HA}nqPj@554B0j|#5S;p=1t3P{8=b5TFC4INy7NGKPq&3N~h*VWCFm2QB|qY=WjmJ5s7wX6E`=ezjC^&YDbj^~vN!W%Lv zRTXkXurM}yP({PKxiZu3de)=k&91sC3cze==y#Qs^^h6-D6SY2rUv`FT$)KSjXy_N zKH`L!?WjwZ!O!yvNtaVn*|msCQSm?*{)Y04D>Yci>fnHd3VsgzVb`E@0Vd|Ny;T&t zW9ZEWbXpi~LtMscilm-eXqDZ&1+0Md5RMs)BNs-F_CZ%<*93J-zM&ePc<3{_Iqn$) z<}?jaG4do-6c$op$Vw;Ydvm|>Y9n-s*h~GRqr#vvl46FcU{+z@1sPva zHRS`&ql$fD@Uq4L<)CffToe zIarO*jW)%Iy3@eQ0-?8>B!3WK4fQ3agBlEXiK|UU_jM zx9nArB!Esm6*&sZ`1oFOVf;Svu3PkoL0Cx%!6Vs!2L>6A>j#g2N-BF`pe&)IiOsYc z+q`f3;l&5b+)Gszvxu;Kw{|aJr5yNCbSD%yW9)m`Sa6m_nnjYF-sO$+XcB9FMY>>y8&14JK#-Y4IYQ3`VF zVQX|=yV_=F24p&;z8R(*f%;idb^ zJ><$RZv3uC;9-$r%{J3DZD^PWY66rTvBu~>`u(Xqt_>mExA>0>g( zh`}THok=Y?A0ePN#sSA(1vDn0Tn`N$KYxDmC)UDG#DtqZKTT|f@Ux7dpu_FGy@#^0 zJ|zzY5r7mL9}WrO0Ti_H*s;`HOqAo}!welW&k(bL%$aa(A!dOyW-awH_a`&0OMo)a z*QY^p=iWWiG3gN<0Cr5PF(+nCS!CgCg!ItKoC(J3xzoHjOazE96k&{$U{Q^xleEE- zNoQ;#ustDrB6e;!@_XcY9J{jUMjGtz%j^;wJLtOV2J-Gwx+c{P7|)TgARWlOE1zfi z5bf2Y@Qi@#o8#;ZG^RixQ@Z@G-rhVe$A4}AElGnYAxSDpl8_V`8YMJILQ)!#nT#3I zs6t3Ghsv0-5=zqetz@dqbEU|T3TdEWzpt$IJo~rz+Ry9xYxl?cTGZWrU)Se4hvPhs z<8*VIrWqwL*sH1vD!xOnfj?hITg5G)r{NR8p=ImpW}oTt+la~Xh3S_p`aL>|CqS@JID3}i5OgC6adA9a z^%Qpr{$Gbj2HTxub9vV5*MErEq!DZ9e2T%kUCi@5;nj&1IWa%;FA1k|h<{gjl_I3q9yK*R~w%rN+3 z0K|lWu87!4gNKFesS*?-xE=_F5R?aI5zr~9KyVU+SJnJ8+~Oq2KK{QPP{JydUM9zM z!6k?bc=kJ5tASctxX*Td5zb=(mI}=PS{o6OkX6?I7!xBlQ=TeedbUT^1Ya~V3N@32 zbUTkX%$;vvzI3|b%xwnj#2-sw-=?KeklpfY-3)ijcW?l2miNVjA^O%6+2P9*yRZG5 z3&67~9rC`qx&=YGUI!5mECLTD`v?N`0==8ebKrYGBc)%V{ltM8+2tqdB}#7J+rfp3 zPF=v?;r*dd#Y&g=Lbeaa5~JmNr@*Z2>~icy&Es?j50=>eGvJ`tiWOT4*BmgeB6cx| z%!smVm-B~IsH0%rwv!O}E@}Q`D&7!tGAbspFb$}eOtTJ{$1!~XJL5TJ5YJr!pRv-A zuk#@MPMQ>g2$Du6I9o(}3CA2qT6$TThd&O{5gUZ<=SFUaijHL_5;2{ zG^H`_>CEP;S+l$rEkdgt&I7^OWvGFQrg=OQqqD49f5X`g#c)hwoyQueW8UQ{Hz<-%6kR(V7|bx zg^0)JmROHn(=Eb9S>@xB5}>Z`yDi}#AVZmAWz}hozPpu3Wfj#Vh9`UWY=GZ1MFZv- zg*xu2XG=(yOK5{=Bd(qk2~EH>4b`3fRk66P{4%Kd{0xH>{+Tlj%@JT30oc#_`ul~2 zk$%A~o07)U?{I$rRs9~Zf-;FETvV|9WP8ThSoT{$)BwQe({Rkgckbk6_&Rp{I8PLv z9J>KFP8}#GNpg_tsmX2_u>;p3cQLP~jKTX-)&{rTf zP_F{2GPLg0)4uFo3@ai6O(AS|3KeMW;5-4V?lX4gi2`bY(xWDaiU}Jt-UxOoJk8h& zQ%G|s5cJ_(1?sr5g*(e62R;%BjFO$hfIc6SYm`=);LtrzV)9X;j&Zuo1Q;evbbzp6F%lhG7*3wUtGHuW2snsEy>Old^vCkZ338? zkXwn+MogOd6LlInnBd0|qGpBV$?1Wc#iPK^&6aKo2k>q=J1xdeLtZdRA+2<*9ze;+ zGvdg@2S*!vW^u9Z%8)ifuBn{#ZcAbeq$V;p6)WNTI-55wEh$}|y?dv!!{V=YSgijU zjusCq4;Q1bC66?y&MBe>Ss!G3&E2p1g}^5w>+Vd~a^&DakGzXi5HtmQW#|R-JZm;pzu01I%JP&_>&1x4fh8I)>nwIcwWCE

yXN8Sa2fwgl-x*wPMyB&$A1HEL?^XFp*3=rK9 z?8S8ri?8T;k=xJC%1o^j80%Dy=gN>KGGQ z{5M{Xq{{+ZZy0RI$Tzk6KJqlJUSE*3Q;ho3@lpFEIaMRN!hRYww@&|#pPgnD4}B_$ zvZ(*I*sEFo%orJwf@IWF*CVfBY$Q1H{RrQRjUYX?-ik$hl-N}|{YD0sc@jSea0d?H z5RN8z9@H2t%3#=fCS~c0#|`GU_j8?cP-3@bur$mNwS#a)S;*1f#Lwl%V+#yk*NOIB zK6?DHVcnc^-2=L0i*gq^;qI#~)FX+ngj)C>#E3#A`P6NQcNgO~vAw7&B1-MRstG?- z?|Ny*4mZ^eTb`;cflTVSLPUY}nMZ)<&7X`2p>+ecU)g)RBs-NUB`b#fK_zIA=&3M+ zAv)dN{F_;)%!_|Xe)Z)RYwTZ%)drruQJJ*u( zM1=F^%S`n=bmI}T-lRV7*JProK-b95O@DR?>sbb#*;nP=OAGFzh^Qi2K2yYBo#uHCb-6_>FXzUQK*H|anRS8kI| zB&uDnb3QYLo|Vl_T}NMu2ok#7%M^kP3fG%k|J>!M1QsgMx=Pf!>1x=eY~ymf4dgE#$VE9>neARDArM z;^MuK5ya5bVqgFsK}ly;CYT$WIqtPkCoD1$g4n%E>S$Q*S=U+h6k#5RT)UpeOQ2C(oyiAYVF!Yyrr_NJyh6+aPiH84?*nM@ z4P0hvs;-`Z%FHqy-7>tYE@#oH-}5Ip)L(_PyM<0s4MheQT>;y z{yFXNnHmvvgCc~I7>tZ)WUmFSqpxH#9VL{0LK;Okff$5__X%iHR24kyH-I!cV7|uS zryOH!1eXgZ8NDdN(Z$3>Ig5+FEODZkv_7aqs^S4IYi)&i1tQ)&<%Pf0i) zX724ufk!T+=?=)jXv~Eu z3k+9)rE+Pr)(a~aPkk+C9cp%V#U%yr>5PKp)_2_yAc!tW1r^&y1$JwkE4ysv$}92l zf|aG*ju%{LXJ={s%7s_h7W)lf8M=5m)t`UHC!!LgnBk@}k$?>Z%K)!_njR`|cB!&u zYH^p$Uq2}cgzCnU)t{jSoBA3TynE+| z;98J-R!l=LiZ*zaQ7^{u{xfnpWnct&CwKbF!b@SN5LWS!7ztVAyT8 zQ_9%4ng0H;YUv63s;Dp>6|O4kU@k#|dDkoi#sdc;;=Uwkfc6ae|y@S}1n za(Jd8f!V^egL{VkH=y4P=+mP{vGV6GZ8~gVwgkL5yaMxMOt$bDIdy6VJQCnxit<9Q zV6sbk9x@K%83 zd2Ts`Ha-~c2WF5OU%mQ`+DWr% z=~Ieu1<>6>R3!n7`unn=-$kdvjys^ZVNYV}0;o zX8_BhWt7}NTflykEG@}a!WtPsJU}pgr*_!R_>-;xc7@*6a{m^b0jM_|HM?okRQvVo zDkIZE3+t%8wXvpuOi>Ynl}neNJbsM00M!T0GW#uvtxS<1Lb_)=V-3b2MlJs#tvgT!TBP9OUEZwfKLJlk^~hAV0`b`0-t;rnMl)iR4=Zz^bt6KE`7UTM>QaSCn?-5H{(ref1!TNa)f7mJM9T6<604aD4$bv*)4lj?h z3fmEXeR+y4x!|9a`U z<%Mk{F#ba^$odAY$SKh9tmEuihLSbQ8!Qr7ie8^G32zs)ln`p@H7KTO?=7_t05`)w z5S9H#9LhHCni$P6k$p0N4!42Xi9{xCO#>CezkT|| z!a%k{G5`WsL9p|?ug7#ha7UF+i2^2(hSaA3O^l-NBK-fmS3XHqxLDg$m674x*e; z$MA(OENwfoiHXIzxdX?IbN1WXInWd}JZpj(Jdv7-?)}HDn?f(Q z{FxK8;unJiX)4ncaAt@}l%~Q;kgP2H(zBWCF~#gqEjCj3Ql>y?gF*N3AsU7e;MLW- ziv9bupd5l0Kg)n+?S>5}Fbt8j$1)-W;xxljWa1Hmv3uRmpZyli)ID-vFcl$cFnRYdWT_7@wu#*KCBg@n4xtWu zOrFoa&&7WN`bX-9J~eqV74{z18yd$27F41AoQY9oLpjoT0 z%|y&Y%+yiXLOn7;CFaBl9Hv3FV4QIiXFEBeHBjl>cdBL-U?u!;ngLi6WNs{660z!m zV@xy#Q9`>3lal>HYe1F==U30#sYjA}AQ6LC;=OU-D;XRPx`m*Mnx)k2%(_AxKRQcY zW1>0H+&pXiEk4S_hbzgZbScP8fe@&`q$DL#9jc_%=7T(PLKx;;zux;F9XmEgtXsz@ zv#_+(&DK^R_N1jzs`gtR%;{!2_l89cAtB<4?o2ble@;Y3mOOu+!W{&N(pPe&%%VtJ zy=RR3O|OLusb9ze^?zzJ6E0nHxv{+O3rRBU^5TU|YM6-9SBzt}%Vh;->J(V^`7^Ec zkj#+}IJ4|Fsi7Cp+(_*8oWphHP-r7W&7owV+bN%#2eZRc2jDF*_@ z(&w;e`_%Sk8bc2?<|0EOXkVy*pLNTPwfNS9x1q|u?bkXE0m0$P?OEEnS+lU4g&|!k z50DD7b=fU*9##3*uM|J6DGHZkJ50OVH7PPk$cf5oSZ`c*9%TKjadtlbF4Q@R8FKbL z0R4CsN&Wxw>i)Oa_m4gwi)L6L)SwObUtE;vsQvBpXUa#6LBl1yZ@P2|{(ivhBYg*s zIt!L|DSQG39I*ZO^HTXc8{oDy&7EqOsS<0BG%QeSfpf*;LNK(-_Fc`GZw7Y-lAQd3bYh&+Sw<* z0m_~GuW|uFnv~+&OZ)7$u-o`k%_nT zjRc!8;mpe(A4R<974!<13H_=}vA{ZEYl-x76Ype(M;IPm zS)#Zgju|~##ixx-EuiCHzR0;71tkZs%03(-_kNF2`dq@?N;hzXfHS98%K1Sbp9 zB=}H1-rls3^=z)~JEe!D1tSiI&0ra5;&{H(b>*7+&(7~tBI zYO2P)W=wySi3`^cautFaGG%rVlMVp<9TxAIzyPH!4{BEi900i_>3{{p+3Hf63hi34 zM~{(&1e!LU?0au^^FV?~*b_$#48+Y3kk_a&)Uq3y_bfQaJ6Pl6BiprWRO4Ohbfi_d z2D5I=R!1Q?x%~S^!X+hGr;&lAHMWbQ*n{AY%ii5m?K|cD>p%K7uf z){dfNU0l=Bm1`%6-DxGjn&%hn!T%n_RoT$n9T0*m%-HA^9UVHR;Oy-*zmZdjOw9Et zBi+^tuKM&V3?~IhfQp1QO(JQIrYFDRTm)%SZP;(5XK?N@82!LEvf!C%l7@UU%^`0T zi`Z)bU!|qOw$pXvj?-tnxrrH@z@aytah5fN>DPdEu(+)*FBfKm6I4`t^qB7AB9uUX z?Q?z;)w15Bjs}Hd!~?Rz>kf)4WQL5}EnpQ#nlUKvIZ>)@Z7M|$0Z|YE;5&zMw&%`F zCbXnqEd9CQY>_lO;{ZE`T~=+7cBsrX)zv+THUzcK5@e$v05PzHMEv&^R@P?xc?j9z zT#Vnoq8i=^7W5anf^_>#>X?D>T>arga~**BNFWPz6=_!HrL1&az4|+n9n)xuOl()l zJE$l*N{rDMIZ;NFa0MNojm>^rG$xXj{oWVNy^pD{q?8md1}*^lwarQA;k|pw*RRti zUgst<2jgV1iEEmgoT2hVauiJxUliGY41`VEBr(8mvO0hlA5HMCAlYy-X+s0=-zAxY zYOG&>9g2yQj#md=9Y*SuLgjJm?mT(2-|igqk)WWsx#{*&)15&C0q z2BRcGoa%1ft1j|dN(#drk>D;e=S{_?QDW{&rvqVn?KW_du;UBw3?2ns@)lL+%I^)1vb`m2dnH>`(psks+W}S_XCw#%S&icj(s8h^d?s{gXi6JOWbcex$ z+WRo`I1)Nb1i5J}J7*Y5Xi$j=GgTvj2l^Sx@A$`ESANIoq$FIZ#nW}+p#UR!^lYh0 zq|j0b7Z4qcJmhyEp&q~qrelQ<`06sh38OyfP@H~lF~LiigeWTqobPO>p1F%I?zF=Y zrQL0<#(D_uIZA>ZPk0_79ri4tF(H9MP}r}_Jb|eotClg2RX9AJnFX};FY ztP*>6nW{1Bpx+a}D9>eGTQw#!MC@K0J35o-m!W>cmrruK|6vXB7^W9xY`9s+u5^4e zz>A@AjNSl`$vDPj<(MeO16ZEor^H8a)W(KQub?eecLDAEk<;IhJoDFd-SFE^J1p6- z;l}mr22NXuj6@wq3}toMMQY{n22F=BKcjLI9Uljd62+#he?15@w(L0lZltFLoH&(oN>PLR(O((0)%pm|~fE zKV}+mV9n!BpE*M-%Cq5~^a4nMiWTIJ=>@b4_)q|)lxjTqGiUyS3hf?pC*ltdN#=Mw z$3t}BnFRS8F;9@ea3^>GQd66F%e)E~UZzVfb3?emHk!nXfILs`8a-!{6KKz4QVfwO z#_#tID3b}h@NcEvV@wA~1cXbjC6s2(ro4nYp~@zQ_fK!9r&ALn-<#{~Os51x!<~ce z4YUcMz`p>(-%L%tbHB#Tztw<%0SrO?sVKvcY7vD1TYu2-L7YGs21F1(RT;r8-r!7I z`53~MD^)Amj1|k4&DT2u_5fvtZ%=-f#(q5zR7hhD&Qo(*9q#=GPG4|oTlJ~JE~Fp$DX7Zi?p zE+8lk41F1;FjXjhnmwB~cb@=&qRB(4HBCK|R*MGv!?X{$BvwS22(h0hmd>4T608tC zHypz*p{xZ)A87`f(@Y14SAavjUvMvwNM;uJ+BoSPpnIG@!6Ae-m{+d`+1AD6a-lf*EZT;8 zijo1)55$K_Ty*qnc%dW-s$O~%G_=fYev_hxxn)2@EQOtiGLdwUVn<1Si(-*?%(vf5 zi6;Yhk5^?`S}i9UN*MJuEfRYW7?ki$yrJ>#q3L{PhP!JQc$g& z%h?fxu+8Ds=ijBS=DC9F)#Z|~S*1BsY^m4ZT`bsjWKbG#7#5ss(F(^->S%G9779wcWQD+)4i{H7^3URs4_&X`#pcOiuh0jwJu;1!T zVy`v_TnycCx8hFHhW&fzOi^IczZ9PT|9pi1`oRC8)A}DS&wu!C;a>cIUFrYO#U1J* z7GCBLL2h7F;Z+l;!e<;l({LkeZ(kO5(jM^e$)iU>3&lHbjJ{9CR1LOqM2}hMj}o1= zf(+-#O~wdrwns#SiT~HP&1CunKOJpYOBfZ#)pi;P_-k3{ErsH*R4l)yWsvvk)n;+_ zyxSoXO|YpL31q2v{5$g6($Sz%%C|@07Tm}^*;Z;R=)FF zOuoHc=;HXf-K1@lZNc70Z>LPPO1L7rQ+h_v{ami8BZ zt?3)#E%9dgOTJc1bo}Ri_m2Pk)A21_s*e9B=hN|@|HZec)pUI4zh3*lUnQWnM@&(e zd6R0_I=6YD{v|!y{**SQY`G@F?<-BD(`IDX+ItJTo`sozTUJj}EvZT3+(+3L>6_L< z0S)y4za)8N>FOZ!g$DzCm9S+x99-BRt?w4m6(!&~mYYaROw9{nskL@;j#o0XGM3PX zc)S1tf?#paE7||K3O1CGWMQX%&l^Spp^o_l{%E4Sj(BOf7cI&?-z-K0 za$y*~mez<71-h3g5=a${KNGf;&`ojNO8;B}$z$Krd_`1^h#F$n2xg5mX0!rYsmNaf z3o2T#7{LI*Gvv#%tF!so^yZZQZFNV|gyHs|#!VCE@wc?FVc%F%@&OzMys324nO_YJ z8Mw#5{!pC|$`pesoBv}~Cw1RNqIHcIQM)TEYeW??!>9akbBnP$)BQD8p8_pC+a1;^ z%>#86#CM=Chx1QhTk@X3_Q3(k-qgnBs%L3)DWE`K(W`M(X){<_!wXE&p!((cLa2f( zWv_i?SfF{V)IkLg-Mf6@0$ft$#~fK8{ucw*Z`gndDbi80B`*o9a@|nqBoq+)_wMCs zlJDI+Pj^?ICxZ*F1f>d$J&#&;X2UyRDlKe??Qpt-9*)5WdJ>q20=j<|SD2{6OF|u1 z%}3=)fK9=SB)1%BA4rldlyKjn>f%_y8%D5(B2+VK3G~4M1Hj%nC!}6T!$6!PWgv6& zUp_4FoT7<}wx|mKBn}2XK+s$u5b}|;&0qzhxCZ6fxnl>WKbT1FQZYjFe+fvAkqs7e zc)##os5iOS01cEb=#L62pkJfQq03sg$9{U(T5Lx{Stqe$2NSN?i*;7zlVq1bfdyA(c8bswX(P&Eeh{4owJ^8J|9i%~4)b9(4k9eo zJ$+p#(Cu(B(cR$^y<`b>Ck+Kwg{>oTZQ!$+%$%K8@%FiMgSo=&w_-YX`7*8|keILHc%r5jU>?UezF_s_*RQ8WJO&$N z_X;{gdwZ0lDt-Di!a+sAiol6Jo}v~-X4mfL;lC4pd48?a#BnS<9F$1G(3k#`pO0>3 zJfH#;sU(5{qg+BMnd2507t|6y8q25-?%yAK3pSOtqvL)sl$kRRgofrb*CI4+o7T!7 zrY|8T>{c>Z%Jh$G1JP>imIlEmDEfghl(n?9-#}~Vu3n-kKYbdF0B_nGgk)XCL&=mEwZE z0;hrL{8gd`A*+-`@)7;V8wkgJIKU7*JcS`PAvWM^{(EGa3Cq22nSjiP9T zipVIG3`hqEDn>&;dd+O+MC4ccJ}eyYE60Tj*pD4*fT8Rw=73HtX~CbA#+)ocHV|fo zjG6zsq0mS2XfTrnuV79G^&F4{A15vv|I2x&-6jDq(RT(mLG@FWYzGM{D=XmGF_#r9 z0Prh#b3_2U7~|t7PEcR&d3532xkXM+FF?Yn*J#KIHNqSU>dew7TIH@QUS2)JnDj|J z9@{f%$y}2>^B{CX^E}zXrv%$nWk~2#_$$Vd7pa>O21;^!kK7Q7;DSq7E4qvJKBE$l~gEuh=ERs7~~}0K6Bj+ z5>HtItQi~4pY-$`43Gde4qWE9;0hubw$~qng~OsHX0H^KC;(uluo~UYu6w8lm;t|y zzoyrRJV#v3^r=13VE`uV0f+qs*rn3$fB6OVKZgciAxxjCy!ho-wzeSjXq6#6P@2%G ztI4<5&OOPtA4nFdsn~F_BS^RyNQw;%u=2eHNCaPE1)hc`e(C&Hm~a zV~%v#fdGI&e7ZtCpnmXsuHLwjuggkqVhHv$OfDcjjU@m8QxGILZ{$K?Vmf6OM!mbS zq8|=E-en?beLNK*)`IXb5g}OPk(B`|$ml)kWi=2V2sn!{1{fJ_MYc=tj3mj>uVo`x zEc~CMA`|d$_!z+G+}se^X<$&6mS;)S1Oz%R3-u(1{!Dhbp$LpHR$p-g=ziejJvPWE zkW3^cMViN}x=R4D*5^P*r6m_g?1W5>tR>5T~o_K07#WvOMDeT@tA_b}EO^b^??#qhvJk z5x9^63~`k4otdc53qLX>Vd^HMGjvUek&CV^LdGFbF`z;*Ii}VjUm0=rcs`kW@PwP2 z=MX{}ZyY;v1W@`GH7Uzaq0oO~vcl8Xg0Pm11LTWBpy@~_Fl7XMHXZ8^sP`b52(=_@ zP+%w_*MG-2g_?Q^V9eqQ=y4o)VeuQ~9Q??>&t9x!azkap^TFg=sE-9i`K--NZVx;k z`%3m)f8q*rwsmECLM}2`h7GuO$76*gJp4tGpJsT{M zoN^;*JPAz@QTQ_jK>SJC9VmPhY`UIkyM$Y?2E(=z>)PYw~!eDI{66LK4E+mb5J$f)KBb8;Af1#wrV! zC~8M3?m|V$Zz*1U&0~`uW+G8Ay7DKPQkn3BJOl*P=;(f5T&FuY zuo8%^DDhmN@^Y4is2NTmZ(zhuDH0ecibM(X3jV+hoxIMnBv^kKZVF*NHda_!&&S|e z5VCn52nz$`cThnn7*F6zFgfGe;3tRy9VG21Rg7`(Qw0|=0k1p{|J_)X-~ul z{7QkO_L39~;o){McbVB6EBe&5wZGKV;8F+7V8TVbmJ}CPPIbfLoab-eD9g#kUcNjm z{UY7i$ocuycN9cy-x&g@W8y@?j)5^Lq#kHKtW!_{ifFw>pCepo3kz)SIHN%2iDDQ8 z0UI5RjO46v?)`p!p+sc3Nkd#qU;6DElbzu@IxTCuNknp6HW58x8FKldl#}pJxx$U5 zex~6<^M2||BXN~ygpbKqcZ30{K?+NS7QO4Q7}LVafw-0v-F~T+g+AW z=8SQzzjrv>owrq8nyLEqF70K{o;|Z4d>&PRe2t-9=l0$`d*Z2+^VXR(2ugsTAcl=8 zY*NDy4oig5(XP(UG~+9;?LKzw7-Jh$8Tf0JnaSkBjhp=Lg7~0-%~G*@t_0|Kr>SqF?=|@L$V1pKoZKjajw?ZU&r{Hz5o7cD+WXOKUBN2G=pbR z_)UF%Hf*r$txXS14B?si>zK)5c|lt39`=}a0Q}6+QF%^4&}2k@+A&~CWFsy(7J6h* zgd1&3&41q)56vy_{9*KuCKG?7Qx~V#J;Pgx8i6AFcLG?~g$rNq3>3_QiMM12Cdu|C zQu7>B=U?Y{#nwz_n1%tx#&kXsvR2BC`X&aoFPD!F&y#m4q;I8fNzCp;Tb^CjwPT?= zt-WUw*rm1tzNB&X_R3=^cP%aRdHgYPwpEX$(juvJUgeMFwYTE((SUc+Hk#}Jsv=SY z>T?D7Uvk)|2R!8EE5~I1_ArwO^*cs&4|s%59)Ja!QtS%LXUC*wcbCdAW%MgJ7BD-( z*NAbbmaeYcAYoV{DZP{c{b|m?Ob-D7YtxVS8w5y&rDu_U-&5Yg);o18Ya-bWYKyUdMzF&oxKubyW}hYah^dIduXyC~iSe0Nr&- z51cMKfY=VB@ioNjUIy?v&eG-$8G@@d#r5EU1A{#PGnBYgd-p0fJfb}3Qz?eR!Lgq@ zb&1or+w3*~b>k(OQgE+eQ-M4JAIzF{yV}~ zUwl^a@ijFm3X>>@!0_fP61~?z>!3WxGakih{U!P;_}6h?e=hs@meoX%K3Y*>VX1c9 zZp6@`5I9vvdD+mZY~6aBZLOH((0P!0A-aP?!?+tw^8)Y=#)5hP+fT_23s^Kh{ z^H{2HbD-p`m|Yg?{f&J$!U4pGf=oq;KEvMLd)+$aXL?sRet!=f}|guAQgS;{C8?vS0J#I&@Kp_R(me0UADh z<%>_l;ML;$M|(isM*&g>bA>3{r+4q;Oi+QM2{GNnm;{#I`6$Kr5)JYErj_mzoFveq z)_w$zf-wfQfSb6^-JON!gV_*js)h&Wtl6_^`QZB=E?j_DuT-rSlyqC$Z)}ePv!?6C zvjX~%m#=TS_XB0^Tw)j3+i$f-1FjmJ22R3Fl&CZ^CorlWIkHDY0ei9yJ4{weva>~> z=2P!e#0yU~lYTxNP!vPn@(22yKP9MDD0@CajZu1Nl6@ZvD5ndCTlA1&@*8wOhg90Fj#n3IIkduv|7yS z^>3`iW}Fg%MwfDdB91Eel;v*uo;?KyA+ZI8h4I=TI$a7{Zi-+N3rdHlI|3f+ekuUW zSuVTW6xQbPL2%N7qKEXgBJw%{O6k@$3Nu27sF`(8GdHX;M7IrD3jVj^r#LP7pL=ykcm# z?%m-IP>q_Yo?Rs_`s-HPW~dk7Zag?{$`2*7GH1Xufj^>T_hZZ!R&4WA?DSGv!LmDz z`i)Qw5srGt%)*j}1HcKTHiz5Cg90<@l;5SEnl^i*q9$hErmbG}Hcgm%@CdSb5%kNB zw}KTD$fS`GvRc}2c6kHA(u6>Ai2b7A!KDOU;WIG#!xJA7hoI8^*$izR8b!`r+I4;z zkR>&WlezhO)K_2#6zELQX$PskF-zc^nTfGG07AGT0$t;>})RIZ^3(|!`j32NL_Ahn2oW4d4O!%A?73WEN`%}w=bH%`tQ4jqNLKnT%Wc$b3l%$bES zQg}ND4?d%B6lniz)==$wJv@dN>{8a1tpCDP_z|xRa16%Evjov{`YFq)Q>H+_iuEYM z8XhXquYkTAgeRKoX?x7D$JX0eHUPRKBLin3??dN|?ONNuQl)bzl?WgZWDTZ1E<`!j zk5HaGc~S|7oYJkb(h-36;6Y||y_J+G;k14GLY`#^d}^mOidc{sBoMqTEF%!L7|Ti1YMPE+=QS`zh-9wDib!c zkt4`F5p)dBIPZfi4P6|I48b>`yup)M0nH`9bHbK}`)}WFC3^64oNsul{y!>r&}~7Q zq&%Y0i7Mo<#OQk98VC?TtmU4^iy7jLU9>=ta1P#i*v1PT@{S!+Y?!}x4YF(o;0QOE zHFv0NLnJa7m?55`niup=l%Sh75&B@)qJ8pE+(4${#!{GppUlP{8ozq;xHmKzv~w`C zNcaF4f*J%c1xN$?8cON2SZ8CBLJp@FfCA}wzM)%o>$}K$fBhtCHRc&_{ z>K#D?dK9~QD?grON)393-?vxzmr7tyK+wft7Du5K{m1HiiHc+#AEBd8sv5umV#p9W zZVFLUp$a=cBxv$RDk?^;(Y)U-sdt182@pY^V@Wzi;49Lat&6unv z%#yQoW&S)}ZRsW!QGh6A>*_T<>Z%CNJy+)cmVH+yxBwtPPL5jD{XncI*$?mHGGXxGMV0El#8Nqu6vzhqJB|i#qxElT>S5b=k5`0Ty~;+) zLe$EHKykNul*bH9S%BA5Wt$8Wa^GoC0p@Umd9HZS>*URK6z&mwdD!T^skMlZdAbaT zb(jh;CM2Qp+mf5<=NfRcS-)Nnf^BF2mw)o*mx%)@lIjEF-$ zP*agNKtj}xHo$m^>-}RdEM^xr?INE7fQTqaep$`5zvgA~WF9LB7`eJEQ$*ZOVJsUh z){AT1G%s6S-YvYnJuH9E$w6JfyZnEDK8HVu-VxurGP7(8wcHay9*XQOt*OCBoEq&N(3s@&lmv;{xQcnFI;S9naGOp z5$S~}@BWRIZ?)5xiZ=Bz9~I1!hVEhHD7EyD&7~1`=V<(-r(JD1KoEoa#uh4#ji)m+ zRj+rVpXg|}>wj&*!W2yfDXnht>rwvl7$=8kkMbw6W&{}^zF7)z)V#MYwy{m0b6ZAK^plW~!0O=6fd&T5 zP#Vh4-DGM*H9<4utSH_x9?kR~msvkmZCU`#+FxB=1I;2Ssqam@yxFhdY%Y2g+1`!c zMgy6-rpvn#ziD&%#!N43rQHn%zvlVN0%C!DN)=@`-bNSDl-paIO|@69*fp*ECD9-U zwijm)TNn)riK2l{8;rYXM4H6)@ugBEQAm{{1A^vA(8uziqwU6+JQ=6R@^J!{qx{@` ze4amOP|BHo*jqN+diCm&!P%x?e@gkE2a07=-I0wNGV8rpu3DwqFdZ15BDLo~YPb-R zA_x=k;3P+E>elAqGMu*72WwJjCRAC}OsZ5QOO~*0`4xFCBZD=l+uSRNK*YW}IKSY+ z*}A(Pc)>C3aoMH6oLqoH0!u}V5fc#Lsj!Y3ETgR5=C%?k59Tg_TgUa6?1Q_8W}kYn z3?nvocU_?QYok2~vuwD6zC{--#QX|M9b`cy3mUQetVt(g6SvgV)If;Yq67_x2aLuB zq=^!o7-xPx6oj0b0AeZ{Z$ZPNI((4+GH*)yn4)PN)(X7HKf2YNQhfDjw2P`xbGER0 z%^pI-&z}?5xA7d&ud=QK1<$y(A9ul4KwgZDAGBlB1eMKqw@LLxBXQQ|2;KPTYO(jV zwG4uBtX47h{Zv_bn)QT>7WtqhkNkcTkOv7j>6r!t>M0N*^GTdP>Eqz}p-S3F&$@j1 zB`PLLd7QB6g*Q!jM+wfbws_8;rthbMgJCMsn=-9x<)Tq8@K5X|6ZRL@YZKl<5Q?8Y z1NV?|e$hgwLlon(FtZCNb0@!WUXJbE8_>4p^P@peVrr>Qc&M235Nxh7Qvy&5ds=}% zd8{<+&;(OG^871M0ayY6Ua1Qpb1;;_&YbN_8(M#_uV0_C{G}e83t(%|0OJ@wIXy`F z?b|d4V1i74^)5;8>%fWd*L_!(*eGO99^OYaIO`16dm7B1c2kq$S|s@jh)jI`h(< zz~N7ywy1js_&meenIic2?_bO6PZBw~xgcW5zkZ6=03!RO>j0B zc3^hvMv-ER3I4h?E+qgt&a#x3Fb-a`B}}^qh%yPkp)rsDKar%x-@NV6=qmRz-5jd6a1Hf zkYL|4`~l#f3XuSjIPU?o9qd_)idIn)1o;?U{UF>5`T+ngI9q!u&J`7#-gg=TuM#E+ zfW$Wtbf9UT#7R%+1WjO^azpp^@|s+>&i&xz^0&atT`B@p+SG&`S~nhw@?rI zEw{B37Qeqv4OYIsCx8Nvcp0jHdj9n9T&K_LxD(@!6=Y8StEey7)psr86WANz%eY-g z9sk78j3dODQ))EWP6Ho+lDS{cp26xmX=xY~w~!1tj_!)W$di)}2Ydv3RFTNA+U&(p zHm38h`a5NC@^TTvM(%?}EtB0F?935df%A=D;tVoyJU`L&nNDAU&1@l3^@& zE7AUzBUXi9gAXvM3IT-^%`O!ZDmSivO@xZkQC@>q8NNlE_Yh1H?iCioBLKW3kuyL9 zH~iV$B24r7Rzy18x(TM!kYHfXF+AmebEXyDxdSn{lub4~up}e+>F`Y076I8Bx#{}m z9}CAF_rnD1NyLf;`;P!;A>xEEk6ra6@^ATQ!_yfVfR9X6HlcYY^H~ntg`qhY#MoF6 z-|$<6F)sWNJ{LEgnMK>&>z>?dCiFaXoR^r9^DV(8)U|5u3kT zc4tBeU?=?ZUKM;Ms!)7vm3s6@;Am3+4}#T;e|=a?QA_=V%P z%i{$@DS8|K<`0J$Md`R@9J4&d)7RY8#Kg!ILw7(!njzq}vvqBh%oVk@nWQ_y3>Yhy zURQA!RU4hx@8)K~B?asg{9obP)00KQFq?2)yjj!N*u=nK=EmQ{ zdHa@b_zf#zfzB9|z9sVm3xY~en+P{5)4!c6CgX)E&%guJ&J)nF<>z|=Pv!K5?}>2D z#leAx@fSY@pg(N*aC(4-@89RpSut$`HXXg<)dK8eS+xtfg7*r%sc6&)*MtwW5jy3* zeUJEE5-?NUbh0XFuPBm)F&T%PoWzKC|LM#nLILz?-ETfX$f(XoY+(CMqc^{M7XkYv znti6C(5bmdbn(So;dOohj?VeRPI(>x7nY*(n3Y_rO+VAp+jJlz{Hd`O!neV=z?z7mtv79cdRtC zfFIQw@DsH-5K`yps%dNU~Qi`FtalHjADO^A}+gG>%V9PirAt8at8_J@KmhXZ? zH3N5q%fOEm+A0x~;o#`Mg5QBp9FG@(D|%XTiStqcssnCjnNCSbkD0o$<1}P)2?IP} zZ6*&#Ut~hQWXTbiseJO%?}C-d-!c;c?7x{ zklv{p4iuIwpbIf)pw5F6O_!n)5kowpumZ}Od}WxG!1>Q1MDk>CPJyS2EoSA=ZKpsL z8!*6Q#fswx4*Wus%cdmBV|0dHTwg5AWW^tCEBS(8NJ!JVhMm z6ZgDU&7@wrdn4X)a4YDPP^C^#@&9nQH~z8y{{CQCOroiHxC9{M4o%5rwYC1}F`lf& zLlub6v94R_g#ldgp~HvoxiEoOrkd29pD$BCulVkD-g?ejG3Ycrv&2Z`8Nj|>_Xxdy z{E}5P&sy0~MVT+qq{ZI2q3~e;EOuSKe=mYBfe0IzyQ3l^yvj+Vw}>^q7m;|7Z~!)Sx^P8|I+Kw zz1|wK`lhKJ5?ElwUmy~w(FkLFahc$}0NQaVNu9JRjGgHxnf3CpaP1-KPzS)cA(2If zpKEArWNQ%QCCB8ZuzukKYJ9B$2v*&jX2{cm{ofex646uxP{8?-4PxXW4!n z4V2mXQJu&LZb_aYNlD+S10$5jLYe@YWT0__?HAM}v9+b-Rf++w73ywb!^Wmf0#nM( z|2Rw9xT-PCPPDbLROFg75&YfMw5w$eC3EZDzB%WSZ8*5qLep|`Va*y#>c} zY&$7+bO@w_n{P_fPM5Tf7&q?d1243IG_1o64TWWeZO zxi?ib=fKyveRfnl9gO$zX5U6NaIgAjM9b;@mI*G=%PzXl>U!;hFERr)8zL_}6}Y#g zdwAQvJasf3G*TOXR7!n$xP7?xHQft!i#z~Yc<;mLsI0BAq(O{1@p7PIC}T>PL$UMO zhAk{XNHb?($U=^(*_Ssv{j6aK&K(1g1^jYw{VedRq|`c^ z$%Y|*D|^$J-cT38=Af?@AZW^hhY!2=hAOE)ZW$Y}jvFY1ha4O@X;4IHe`t+FJ}57z z`m|pDRVm*2kirog)?((Y4ztl@9>t6%BrFUbf1HAoqt2)CammS)>9Z5-h6QKK$!zbT z9Kb+-=~U(BN0c^19#{mdgRTZSm>wIJ`qKyi<sAv$GoM~p zi6{V`YI&w80p*As9&7pDjTk?C@gs#NkHP;XH$aWW5NGVwC1P94LfUEEh34?%{l13x z{Keon(KRPK={y+<66gqR)_a=LO{2~Bx*>se!9jCZnVM#iPFEXZZ|{#Yy>#b+K6*b& z6+>BS#vWsu-&VHrQV8&5(AbszY>rRJr)PsMjBxoXlU>IHLESAO%uJrVh!7E8u-v{1_qNGC!XXn&7luz$G=zC-Y0Klx|S?UoPK&AAixKV&*u@X1mAM zDvXYP3x`c_sm%Clk0IRzCu_qeoA-pOc3pIy^)Mi{5X>ALf*$6q>fO}G+OjTfV8TSO zW#E1gs*@MfI%uJi>b-tno^r+knJp4OVWo-ClL-qnepA?#_)%VP$U?u(adCkX*+h#= zT@5IebIO+a9jzoX4@M`T-u`V!{nvI!4}e!^BfxG)5utJ$^72J_LQ+Zz1)j(q5q%r0DNR{Q5y1o4$qQYV8)abn~H zx?y8>j$Al~x~O+Ae7J`Ec7YB^!BB3H2nGdzn+&lSBnP6~(!{n)Q{~#sihU#kGIu~g z2Y&-Ar(oWPzMpm;S#{Ah+wD7dVo>6Hc1bD5TO&p!Kq3jtHNrTWW0ARRV~-1R=5Jq9 z*AvGAZ>*V9lu&9L&JHf5}{{HE- zk6`*VOf<_@8#XjB6Q%SYLkKBSrb@X0;KtCA@JmL8(?YXAwUy~LR$ws*&i)vtFg4*3 zrqXWueIb&?9l&C6g^<4NQZle|a9FdT_Y*!J$Oc}6Ygq$;=r8i}eeMHhIM0|I65~d= zrdh<okwTyz|SKaSZHf z^tKJn;isxt`bu3f?)VjCQc8EgdhM6MBwchn)9e&b)0fAFAPF@uDlJjqjqJ`RV!v>yU@>fpos>#p9_b3^B z$qtr~=`x=lm2o{73vDO?kx)c%H}9#{Ql7A%_8Bq}9^y*n87b33gGNTa1sRNZQW$9j~8jN z&EL~M8vbn0o@-%f!9-vyGV=nN=y}UuG(cKGJ1!8E6ILeDMgebLa?3oC=ShizQB<2x zUVk{Sgmz_0#vwz5Wp|u{ zAlty39CG~MDEs|e$Gu}XN*v%BfW||51vKm9xJ>F?3Y<0gi5M?I=%a>`{j;8h@>X4f zdk~7trf$@(CCz`Doy|y>hV*=gEs5F~~Kj zhcDG8NDkqS$D2jFiB!ZIDx_G=sGAT6q@;F0zkJeMAq&h1wjd4HO;)!3X*Y~WFarbE zId=Fk9wBTnxXDuH@}um7ehxat@EvaPoH;_LEbK~ka!Pzz4WmES+LP@Lv>~})Wa0h7 zuK=mQ(r)RGo=A{P7(i{@*hK$--f<}b8*l}BC&?c~iHih&VgKMv-Fp5ypOq&BWtKfL zn{xAO7I#gd<={gx^-&@~1DSyP+b4YEQH4GXRspx?lxP5_nkKlH^dtZb3yTF{CYYv- z8sfrJ2Y&&bjrl^)6)XDp=z)A2BL(KqnJZq8M$-KGb8}dx1Gj=x-$z9y!@JyECKs89 ztg<1z26lT8t&3B)QFWiRc1AJ9S>gsxF6z%h8g9|rwaL48KNykOqTX^Ve2PZswr$E| z*DRne!&L(ZB_gfc@^yb2b6ekKxBmBRIretXfK}F<3uot7s}@R%w5fp1P$qEyh_v9w zz}}d#VA|eR+Q^arzzKmVr&U?R>g#ZYVM>%ewMW1H7@t}vxIz#s+FvQv+TTK!+%BRL z(MT}@Py=rP#Et%v65c%NuUt7pb@Aof^ck4VlP@VS&3xXrM*>GfJKbN`2irhO?$R#? z?ZS=@o|yOwlMY3)|1fBOzfxB=!>{u;grzKK0SF|Hw@kwYPKm%AK?lyDghl&bDcyP*7d5)>JAWVVA z0bd+1TmYU)fu}KeaM4i%4hk)mWWs(8nrg&W>}&o20RV7(zZb{w96eA%g5Zcn!0pb& zjn7Vf^r;l$5@kw1!b4G!thDqa+->osW^Wg|h(q@t+`ISl*Dt^oA`=MCYj99VcQa;0 zVT|c~Yyu%5BSXb*s7P6~NXMH8vux&-nX*ySBfjzH0VYKNd@*kjra_R;!ySO4JQVZ2 zlBXRX2V;NP{P{gqRjClL0UEhH4Dr$8&la2sk{?qt*iq~G%|z3GbfF_OqH2Rx?RG5v zN^C4EfBnco+&~HmrfADYZ$XQ|vrWT@K|^8T67Dw9ioS(jV{%s=v?Nrty?Mie;JvKf ztZT1Rg*HGiG_@mTSiU^P+WIwET~?MUPEs*NRo;e*p+LT;6b2D4q(fMoIk~1dFHgL6 zKED?-{llUnWG7@jIAw~V#A!Y(`)~Y5)(G3}IRqdmK%*2vKN=dml3#LLStqdAz-rv_ zcU%R^3T_Zk)3tpFxj*_OE>-*c&Wg6>mfP<)L&!(_^tW(GS^88PeHQ~aK&SfI72!mskxTXeoxm7Gk^0u1h@uRX(Z8bP;QvkM@rI$WV!yK6MdA zsq2&}lK7`gpKf+5M|U~FpNoi0JI_;U!!L|1yNCq%OA9KQRRlRKFxh8`hZl221UwH} zAOq=3PEMD0iHQBAx24?V3@sCrn0p^s9fsPFHh4U%%7_P3rw*HQbpdGuA7-YxgIH?e zw-X)>f-koJ>MAnrGLsDr3E|G&d-KK@2O6Av&ts{UtFF~A!fMU%q5#q#9$A>?nN3jaGPTjHt87~qv6D{;WsIU$^f!+AHTbxHo$NA z!B~O|LIK7%Z@%JanmUR>Xs%Od&NLV?Vif@$*@N)FU?0NB7 zH^v+~6qdi#L-F9=y~VFz-@;~}%Vk)?l^^h?w$)6Sdc#0rX&gG*|Mrh~g`;MB+aVRu zYA(d0Mduhz(6Ak_B4QLhc))`Itu!S$`QH8e*OHULeF%$nzkW?Bva+*7SKqrvi-^*M zsU52vu`lO=gNkK#5wrN|n>L}GVv$G6jT@GMmLlD5UbuYu0HS|bdY3PAun+CsE6jO@ z4P$d2LmbkhRsk(I+Yv`8%1}H)TBClH$)%&ht#jJB#~e6N;Eo}L6=4yR4+llKb6y Sue~gMsHv0eEUuWl2mN0!2Sf<~ literal 0 HcmV?d00001 From ec59033917649941a0602f18692d3be9768b5441 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Fri, 7 Oct 2016 20:02:46 -0700 Subject: [PATCH 08/14] Update README.md --- kubernetes/README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kubernetes/README.md b/kubernetes/README.md index ec4adc526d51e..5f4d2df014159 100644 --- a/kubernetes/README.md +++ b/kubernetes/README.md @@ -24,3 +24,5 @@ * `--master k8s://default` ensures that it picks up the correct APIServer the default from the current context. * Check for pods being created. Watch the master logs using kubectl log -f . * If on a service that allows external load balancers to be provisioned, an external IP will be allocated to the service associated with the driver. The spark-master UI can be accessed from that IP address on port 4040. + +![spark-submit](spark-submit.png) From f336d4dfd95f288fa37c16715412043add9b08f1 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Wed, 9 Nov 2016 12:40:16 -0700 Subject: [PATCH 09/14] Use images with spark pre-installed (#1) * Use images with spark pre-installed * simplify staging for client.jar * Remove some tarball-uri code. Fix kube client URI in scheduler backend. Number executors default to 1 * tweak client again, works across my testing environments * use executor.sh shim * allow configuration of service account name for driver pod * spark image as a configuration setting instead of env var * namespace from spark.kubernetes.namespace * configure client with namespace; smooths out cases when not logged in as admin * Assume a download jar to /opt/spark/kubernetes to avoid dropping protections on /opt --- .../KubernetesClusterScheduler.scala | 130 ++++-------------- .../KubernetesClusterSchedulerBackend.scala | 76 +--------- 2 files changed, 36 insertions(+), 170 deletions(-) diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala index 626f776b32664..c169f88e77d02 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala @@ -48,7 +48,9 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) var client = setupKubernetesClient() val driverName = s"spark-driver-${Random.alphanumeric take 5 mkString("")}".toLowerCase() val svcName = s"spark-svc-${Random.alphanumeric take 5 mkString("")}".toLowerCase() - val instances = conf.get(EXECUTOR_INSTANCES).getOrElse(2) //TODO: default 2??? + val instances = conf.get(EXECUTOR_INSTANCES).getOrElse(1) + val serviceAccountName = conf.get("spark.kubernetes.serviceAccountName", "default") + val nameSpace = conf.get("spark.kubernetes.namespace", "default") logWarning("instances: " + instances) @@ -57,10 +59,10 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) } def stop(): Unit = { - client.pods().inNamespace(getNamespace()).withName(driverName).delete() + client.pods().inNamespace(nameSpace).withName(driverName).delete() client .services() - .inNamespace(getNamespace()) + .inNamespace(nameSpace) .withName(svcName) .delete() } @@ -70,121 +72,50 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) logInfo("Starting spark driver on kubernetes cluster") val driverDescription = buildDriverDescription(args) - // This is the URL of the spark distro. - val sparkDistUri = Option(System.getenv("SPARK_DISTRO_URI")).getOrElse { - throw new SparkException("Spark distribution not set, please set the SPARK_DISTRO_URI environment variable to " + - "a runnable spark archive.") - } - - // This is the URL of the driver pod's image. - // Any image may be supplied as long as it contains a - // ./install.sh file which is executable and sets up the - // spark environment in /opt/spark. - val sparkDriverImage = Option(System.getenv("SPARK_DRIVER_IMG")).getOrElse { - throw new SparkException("Spark driver image not set, please set the SPARK_DRIVER_IMG environment variable to " + - "a spark driver image.") + // image needs to support shim scripts "/opt/driver.sh" and "/opt/executor.sh" + val sparkDriverImage = conf.getOption("spark.kubernetes.sparkImage").getOrElse { + // TODO: this needs to default to some standard Apache Spark image + throw new SparkException("Spark image not set. Please configure spark.kubernetes.sparkImage") } // This is the URL of the client jar. val clientJarUri = args.userJar conf.setExecutorEnv("spark.executor.jar", clientJarUri) - conf.setExecutorEnv("spark.kubernetes.namespace", getNamespace()) + conf.setExecutorEnv("spark.kubernetes.namespace", nameSpace) conf.setExecutorEnv("spark.kubernetes.driver.image", sparkDriverImage) - conf.setExecutorEnv("spark.kubernetes.distribution.uri", sparkDistUri) // This is the kubernetes master we're launching on. val kubernetesHost = "k8s://" + client.getMasterUrl().getHost() logInfo("Using as kubernetes-master: " + kubernetesHost.toString()) - var annotationMap = Map("pod.beta.kubernetes.io/init-containers" -> raw"""[ - { - "name": "client-fetch", - "image": "busybox", - "command": ["wget", "-O", "/work-dir/client.jar", "$clientJarUri"], - "volumeMounts": [ - { - "name": "workdir", - "mountPath": "/work-dir" - } - ] - }, - { - "name": "distro-fetch", - "image": "busybox", - "command": ["wget", "-O", "/work-dir/spark.tgz", "$sparkDistUri"], - "volumeMounts": [ - { - "name": "workdir", - "mountPath": "/work-dir" - } - ] - }, - { - "name": "setup", - "image": "$sparkDriverImage", - "command": ["./install.sh"], - "volumeMounts": [ - { - "name": "workdir", - "mountPath": "/work-dir" - }, - { - "name": "opt", - "mountPath": "/opt" - } - ] - } - ]""") - - val labelMap = Map("type" -> "spark-driver") val pod = new PodBuilder() .withNewMetadata() .withLabels(labelMap.asJava) .withName(driverName) - .withAnnotations(annotationMap.asJava) .endMetadata() .withNewSpec() .withRestartPolicy("OnFailure") + .withServiceAccount(serviceAccountName) .addNewContainer() .withName("spark-driver") .withImage(sparkDriverImage) .withImagePullPolicy("Always") - .withCommand("/opt/spark/bin/spark-submit") - .withArgs(s"--class=${args.userClass}", + .withCommand(s"/opt/driver.sh") + .withArgs(s"$clientJarUri", + s"--class=${args.userClass}", s"--master=$kubernetesHost", s"--executor-memory=${driverDescription.mem}", s"--conf=spark.executor.jar=$clientJarUri", s"--conf=spark.executor.instances=$instances", - s"--conf=spark.kubernetes.namespace=${getNamespace()}", + s"--conf=spark.kubernetes.namespace=$nameSpace", s"--conf=spark.kubernetes.driver.image=$sparkDriverImage", - s"--conf=spark.kubernetes.distribution.uri=$sparkDistUri", - "/work-dir/client.jar", + "/opt/spark/kubernetes/client.jar", args.userArgs.mkString(" ")) - .withVolumeMounts() - .addNewVolumeMount() - .withName("workdir") - .withMountPath("/work-dir") - .endVolumeMount() - .addNewVolumeMount() - .withName("opt") - .withMountPath("/opt") - .endVolumeMount() .endContainer() - .withVolumes() - .addNewVolume() - .withName("workdir") - .withNewEmptyDir() - .endEmptyDir() - .endVolume() - .addNewVolume() - .withName("opt") - .withNewEmptyDir() - .endEmptyDir() - .endVolume() .endSpec() .build() - client.pods().inNamespace(getNamespace()).withName(driverName).create(pod) + client.pods().inNamespace(nameSpace).withName(driverName).create(pod) var svc = new ServiceBuilder() .withNewMetadata() @@ -205,7 +136,7 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) client .services() - .inNamespace(getNamespace()) + .inNamespace(nameSpace) .withName(svcName) .create(svc) @@ -225,22 +156,19 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) } def setupKubernetesClient(): KubernetesClient = { - val sparkMaster = new java.net.URI(conf.get("spark.master")) - if (sparkMaster.getHost() == "default") { - return new DefaultKubernetesClient() - } else { - var config = new ConfigBuilder().withMasterUrl(sparkMaster.getHost()).build - var client = new DefaultKubernetesClient(config) - return client - } - } + val sparkHost = new java.net.URI(conf.get("spark.master")).getHost() - def getNamespace(): String = { - var kubernetesNamespace = System.getenv("K8S_NAMESPACE") - if (kubernetesNamespace == null) { - kubernetesNamespace = "default" + var config = new ConfigBuilder().withNamespace(nameSpace) + if (sparkHost != "default") { + config = config.withMasterUrl(sparkHost) } - return kubernetesNamespace + + // TODO: support k8s user and password options: + // .withTrustCerts(true) + // .withUsername("admin") + // .withPassword("admin") + + new DefaultKubernetesClient(config.build()) } private def buildDriverDescription(args: ClientArguments): KubernetesDriverDescription = { diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 3765eeb063cb7..b5e312824f777 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -35,13 +35,13 @@ private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { - val config = new ConfigBuilder().withMasterUrl("https://kubernetes").build - val client = new DefaultKubernetesClient(config) + + val client = new DefaultKubernetesClient() + val DEFAULT_NUMBER_EXECUTORS = 2 val sparkExecutorName = s"spark-executor-${Random.alphanumeric take 5 mkString("")}".toLowerCase() var executorPods = mutable.ArrayBuffer[String]() - val sparkDistUri = sc.getConf.get("spark.kubernetes.distribution.uri") val sparkDriverImage = sc.getConf.get("spark.kubernetes.driver.image") val clientJarUri = sc.getConf.get("spark.executor.jar") val ns = sc.getConf.get("spark.kubernetes.namespace") @@ -97,94 +97,32 @@ private[spark] class KubernetesClusterSchedulerBackend( def createExecutorPod(executorNum: Int): String = { // create a single k8s executor pod. - var annotationMap = Map( - "pod.beta.kubernetes.io/init-containers" -> raw"""[ - { - "name": "client-fetch", - "image": "busybox", - "command": ["wget", "-O", "/work-dir/client.jar", "$clientJarUri"], - "volumeMounts": [ - { - "name": "workdir", - "mountPath": "/work-dir" - } - ] - }, - { - "name": "distro-fetch", - "image": "busybox", - "command": ["wget", "-O", "/work-dir/spark.tgz", "$sparkDistUri"], - "volumeMounts": [ - { - "name": "workdir", - "mountPath": "/work-dir" - } - ] - }, - { - "name": "setup", - "image": "$sparkDriverImage", - "command": ["./install.sh"], - "volumeMounts": [ - { - "name": "workdir", - "mountPath": "/work-dir" - }, - { - "name": "opt", - "mountPath": "/opt" - } - ] - } - ]""") - - val labelMap = Map("type" -> "spark-executor") val podName = s"$sparkExecutorName-$executorNum" var pod = new PodBuilder() .withNewMetadata() .withLabels(labelMap.asJava) .withName(podName) - .withAnnotations(annotationMap.asJava) .endMetadata() .withNewSpec() .withRestartPolicy("OnFailure") + .addNewContainer().withName("spark-executor").withImage(sparkDriverImage) - .withImagePullPolicy("Always") - .withCommand("/opt/spark/bin/spark-class") + .withImagePullPolicy("IfNotPresent") + .withCommand("/opt/executor.sh") .withArgs("org.apache.spark.executor.CoarseGrainedExecutorBackend", "--driver-url", s"$driverURL", "--executor-id", s"$executorNum", "--hostname", "localhost", "--cores", "1", "--app-id", "1") //TODO: change app-id per application and pass from driver. - .withVolumeMounts() - .addNewVolumeMount() - .withName("workdir") - .withMountPath("/work-dir") - .endVolumeMount() - .addNewVolumeMount() - .withName("opt") - .withMountPath("/opt") - .endVolumeMount() .endContainer() - .withVolumes() - .addNewVolume() - .withName("workdir") - .withNewEmptyDir() - .endEmptyDir() - .endVolume() - .addNewVolume() - .withName("opt") - .withNewEmptyDir() - .endEmptyDir() - .endVolume() + .endSpec().build() client.pods().inNamespace(ns).withName(podName).create(pod) return podName } - protected def driverURL: String = { if (conf.contains("spark.testing")) { "driverURL" From 4cab6e4317abfbbd3bb2c1aafa3352d7018339b2 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Wed, 9 Nov 2016 11:52:24 -0800 Subject: [PATCH 10/14] Update README.md --- kubernetes/README.md | 21 +++++++-------------- 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/kubernetes/README.md b/kubernetes/README.md index 5f4d2df014159..99a523e326ec5 100644 --- a/kubernetes/README.md +++ b/kubernetes/README.md @@ -1,7 +1,5 @@ # Pre-requisites * maven, JDK and all other pre-requisites for building Spark. -* An accessible kubernetes cluster which is running 1.4 or a 1.3.x cluster with alpha features enabled. - * We use init-containers, which require the above. # Steps to compile @@ -9,20 +7,15 @@ * Build the project * ./build/mvn -Pkubernetes -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests package * Ensure that you are pointing to a k8s cluster (kubectl config current-context), which you want to use with spark. -* Set the appropriate environment variables: - * `export SPARK_DISTRO_URI=http://storage.googleapis.com/foxish-spark-distro/spark.tgz` - * This should be a runnable distribution of spark. - * The tgz specified above was build as follows: - * ./dev/make-distribution.sh --name custom-spark --tgz -Pkubernetes -Phadoop-2.4 -* `export SPARK_DRIVER_IMG=foxish/k8s-spark-driver` - * This is the runnable docker image we will use. The docker image sources can be found here. -* [OPTIONAL] `export K8S_NAMESPACE=` - * This namespace needs to exist before hand. This env var can be left unset and it will use the default namespace. * Launch a spark-submit job: - * `./bin/spark-submit --deploy-mode cluster --class org.apache.spark.examples.SparkPi --master k8s://default --conf spark.executor.instances=5 http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar 10000` - * The implementation is such that it is interactive, and will clean the drivers and executors up upon termination. + * `./bin/spark-submit --deploy-mode cluster --class org.apache.spark.examples.SparkPi --master k8s://default --conf spark.executor.instances=5 --conf spark.kubernetes.sparkImage=manyangled/kube-spark:snapshot http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar 10000` + * Optionally, the following config parameters can be supplied to spark-submit with additional `--conf` arguments (or a configuration file). + * spark.kubernetes.serviceAccountName (defaults to "default") + * spark.kubernetes.namespace (defaults to "default"). The namespace must exist prior to launching spark-submit. + * The image is built from https://github.com/erikerlandson/openshift-spark. * `--master k8s://default` ensures that it picks up the correct APIServer the default from the current context. * Check for pods being created. Watch the master logs using kubectl log -f . -* If on a service that allows external load balancers to be provisioned, an external IP will be allocated to the service associated with the driver. The spark-master UI can be accessed from that IP address on port 4040. +* If on a cloud/infrastructure provider that allows external load balancers to be provisioned, an external IP will be allocated to the service associated with the driver. The spark-master UI can be accessed from that IP address on port 4040. + ![spark-submit](spark-submit.png) From 70a6f3f19ef26910b68b15dcf05a79b72e7805cd Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Tue, 15 Nov 2016 09:26:38 -0700 Subject: [PATCH 11/14] Add support for dynamic executors (#4) * Add support for dynamic executors * fill in some sane logic for doKillExecutors * doRequestTotalExecutors signals graceful executor shutdown, and favors idle executors --- .../KubernetesClusterScheduler.scala | 35 +++-- .../KubernetesClusterSchedulerBackend.scala | 128 +++++++++++++++--- 2 files changed, 128 insertions(+), 35 deletions(-) diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala index c169f88e77d02..a677d8725f9f5 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala @@ -80,14 +80,32 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) // This is the URL of the client jar. val clientJarUri = args.userJar - conf.setExecutorEnv("spark.executor.jar", clientJarUri) - conf.setExecutorEnv("spark.kubernetes.namespace", nameSpace) - conf.setExecutorEnv("spark.kubernetes.driver.image", sparkDriverImage) // This is the kubernetes master we're launching on. val kubernetesHost = "k8s://" + client.getMasterUrl().getHost() logInfo("Using as kubernetes-master: " + kubernetesHost.toString()) + val submitArgs = scala.collection.mutable.ArrayBuffer.empty[String] + submitArgs ++= Vector( + clientJarUri, + s"--class=${args.userClass}", + s"--master=$kubernetesHost", + s"--executor-memory=${driverDescription.mem}", + s"--conf=spark.executor.jar=$clientJarUri", + s"--conf=spark.executor.instances=$instances", + s"--conf=spark.kubernetes.namespace=$nameSpace", + s"--conf=spark.kubernetes.driver.image=$sparkDriverImage") + + if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + submitArgs ++= Vector( + "--conf spark.dynamicAllocation.enabled=true", + "--conf spark.shuffle.service.enabled=true") + } + + // these have to come at end of arg list + submitArgs ++= Vector("/opt/spark/kubernetes/client.jar", + args.userArgs.mkString(" ")) + val labelMap = Map("type" -> "spark-driver") val pod = new PodBuilder() .withNewMetadata() @@ -102,16 +120,7 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) .withImage(sparkDriverImage) .withImagePullPolicy("Always") .withCommand(s"/opt/driver.sh") - .withArgs(s"$clientJarUri", - s"--class=${args.userClass}", - s"--master=$kubernetesHost", - s"--executor-memory=${driverDescription.mem}", - s"--conf=spark.executor.jar=$clientJarUri", - s"--conf=spark.executor.instances=$instances", - s"--conf=spark.kubernetes.namespace=$nameSpace", - s"--conf=spark.kubernetes.driver.image=$sparkDriverImage", - "/opt/spark/kubernetes/client.jar", - args.userArgs.mkString(" ")) + .withArgs(submitArgs :_*) .endContainer() .endSpec() .build() diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index b5e312824f777..8d4f67a173071 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -22,7 +22,9 @@ import io.fabric8.kubernetes.api.model.PodBuilder import io.fabric8.kubernetes.api.model.extensions.JobBuilder import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} import org.apache.spark.internal.config._ +import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.TaskSchedulerImpl @@ -30,6 +32,7 @@ import org.apache.spark.util.Utils import scala.collection.mutable import scala.util.Random +import scala.concurrent.Future private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -40,41 +43,111 @@ private[spark] class KubernetesClusterSchedulerBackend( val DEFAULT_NUMBER_EXECUTORS = 2 val sparkExecutorName = s"spark-executor-${Random.alphanumeric take 5 mkString("")}".toLowerCase() - var executorPods = mutable.ArrayBuffer[String]() + + // TODO: do these need mutex guarding? + // key is executor id, value is pod name + var executorToPod = mutable.Map.empty[String, String] // active executors + var shutdownToPod = mutable.Map.empty[String, String] // pending shutdown + var executorID = 0 val sparkDriverImage = sc.getConf.get("spark.kubernetes.driver.image") val clientJarUri = sc.getConf.get("spark.executor.jar") val ns = sc.getConf.get("spark.kubernetes.namespace") + val dynamicExecutors = Utils.isDynamicAllocationEnabled(sc.getConf) + + // executor back-ends take their configuration this way + if (dynamicExecutors) { + sc.getConf.setExecutorEnv("spark.dynamicAllocation.enabled", "true") + sc.getConf.setExecutorEnv("spark.shuffle.service.enabled", "true") + } - override def start() { + override def start(): Unit = { super.start() - var i = 0 - for(i <- 1 to getInitialTargetExecutorNumber(sc.conf)){ - executorPods += createExecutorPod(i) - } - None + createExecutorPods(getInitialTargetExecutorNumber(sc.getConf)) } override def stop(): Unit = { - for (i <- 0 to executorPods.length) { - client.pods().inNamespace(ns).withName(executorPods(i)).delete() - } + // Kill all executor pods indiscriminately + killExecutorPods(executorToPod.toVector) + killExecutorPods(shutdownToPod.toVector) super.stop() } // Dynamic allocation interfaces - override def doRequestTotalExecutors(requestedTotal: Int): scala.concurrent.Future[Boolean] = { - return super.doRequestTotalExecutors(requestedTotal) + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + logInfo(s"Received doRequestTotalExecutors: $requestedTotal") + val n = executorToPod.size + val delta = requestedTotal - n + if (delta > 0) { + logInfo(s"Adding $delta new executors") + createExecutorPods(delta) + } else if (delta < 0) { + val d = -delta + val idle = executorToPod.toVector.filter { case (id, _) => !scheduler.isExecutorBusy(id) } + if (idle.length > 0) { + logInfo(s"Shutting down ${idle.length} idle executors") + shutdownExecutors(idle.take(d)) + } + val r = math.max(0, d - idle.length) + if (r > 0) { + logInfo(s"Shutting down $r non-idle executors") + shutdownExecutors(executorToPod.toVector.slice(n - r, n)) + } + } + // TODO: are there meaningful failure modes here? + Future.successful(true) } - override def doKillExecutors(executorIds: Seq[String]): scala.concurrent.Future[Boolean] = { - return super.doKillExecutors(executorIds) + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { + logInfo(s"Received doKillExecutors") + killExecutorPods(executorIds.map { id => (id, executorToPod(id)) }) + Future.successful(true) + } + + private def createExecutorPods(n: Int) { + for (i <- 1 to n) { + executorID += 1 + executorToPod += ((executorID.toString, createExecutorPod(executorID))) + } + } + + def shutdownExecutors(idPodPairs: Seq[(String, String)]) { + val active = getExecutorIds.toSet + + // Check for any finished shutting down and kill the pods + val shutdown = shutdownToPod.toVector.filter { case (e, _) => !active.contains(e) } + killExecutorPods(shutdown) + + // Now request shutdown for the new ones. + // Move them from executor list to list pending shutdown + for ((id, pod) <- idPodPairs) { + try { + // TODO: 'ask' returns a future - can it be used to check eventual success? + Option(driverEndpoint).foreach(_.ask[Boolean](RemoveExecutor(id, ExecutorKilled))) + executorToPod -= id + shutdownToPod += ((id, pod)) + } catch { + case e: Exception => logError(s"Error shutting down executor $id", e) + } + } + } + + private def killExecutorPods(idPodPairs: Seq[(String, String)]) { + for ((id, pod) <- idPodPairs) { + try { + client.pods().inNamespace(ns).withName(pod).delete() + executorToPod -= id + shutdownToPod -= id + } catch { + case e: Exception => logError(s"Error killing executor pod $pod", e) + } + } } def getInitialTargetExecutorNumber(conf: SparkConf, numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = { - if (Utils.isDynamicAllocationEnabled(conf)) { + if (dynamicExecutors) { val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS) val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) @@ -99,6 +172,21 @@ private[spark] class KubernetesClusterSchedulerBackend( // create a single k8s executor pod. val labelMap = Map("type" -> "spark-executor") val podName = s"$sparkExecutorName-$executorNum" + + val submitArgs = mutable.ArrayBuffer.empty[String] + + if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + submitArgs ++= Vector( + "dynamic-executors") + } + + submitArgs ++= Vector("org.apache.spark.executor.CoarseGrainedExecutorBackend", + "--driver-url", s"$driverURL", + "--executor-id", s"$executorNum", + "--hostname", "localhost", + "--app-id", "1", // TODO: change app-id per application and pass from driver. + "--cores", "1") + var pod = new PodBuilder() .withNewMetadata() .withLabels(labelMap.asJava) @@ -110,17 +198,13 @@ private[spark] class KubernetesClusterSchedulerBackend( .addNewContainer().withName("spark-executor").withImage(sparkDriverImage) .withImagePullPolicy("IfNotPresent") .withCommand("/opt/executor.sh") - .withArgs("org.apache.spark.executor.CoarseGrainedExecutorBackend", - "--driver-url", s"$driverURL", - "--executor-id", s"$executorNum", - "--hostname", "localhost", - "--cores", "1", - "--app-id", "1") //TODO: change app-id per application and pass from driver. + .withArgs(submitArgs :_*) .endContainer() .endSpec().build() client.pods().inNamespace(ns).withName(podName).create(pod) - return podName + + podName } protected def driverURL: String = { From 63bc5327c21c58131fa760cb94f82462b4ac1b5d Mon Sep 17 00:00:00 2001 From: Anirudh Date: Thu, 10 Nov 2016 08:48:49 -0800 Subject: [PATCH 12/14] [revert] adding new yamls for TPR --- sbin/driver.yaml | 7 +++++++ sbin/kubernetes-resource.yaml | 10 ++++++++++ 2 files changed, 17 insertions(+) create mode 100644 sbin/driver.yaml create mode 100644 sbin/kubernetes-resource.yaml diff --git a/sbin/driver.yaml b/sbin/driver.yaml new file mode 100644 index 0000000000000..de806599c2c5c --- /dev/null +++ b/sbin/driver.yaml @@ -0,0 +1,7 @@ +apiVersion: "kubernetes.io/v1" +kind: SparkJob +metadata: + name: spark-driver-1924 +image: "driver-image" +state: "completed" +num-executors: 10 \ No newline at end of file diff --git a/sbin/kubernetes-resource.yaml b/sbin/kubernetes-resource.yaml new file mode 100644 index 0000000000000..58d2072c0622b --- /dev/null +++ b/sbin/kubernetes-resource.yaml @@ -0,0 +1,10 @@ +metadata: + name: spark-job.kubernetes.io + labels: + resource: spark-job + object: spark +apiVersion: extensions/v1beta1 +kind: ThirdPartyResource +description: "A resource that manages a spark job" +versions: + - name: v1 \ No newline at end of file From 7876892620e9dc73c5cc075b65af9e448b3a3fff Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Mon, 21 Nov 2016 12:32:26 -0700 Subject: [PATCH 13/14] propagate user-set Spark configurations to the driver pod (#5) --- .../KubernetesClusterScheduler.scala | 39 +++++++++++++------ .../KubernetesClusterSchedulerBackend.scala | 23 +++++------ 2 files changed, 38 insertions(+), 24 deletions(-) diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala index a677d8725f9f5..a9d53c386d0dc 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterScheduler.scala @@ -35,6 +35,11 @@ import org.apache.spark.util.Utils import scala.util.Random +private[spark] object KubernetesClusterScheduler { + def defaultNameSpace = "default" + def defaultServiceAccountName = "default" +} + /** * This is a simple extension to ClusterScheduler * */ @@ -45,14 +50,26 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) private val DEFAULT_CORES = 1.0 logInfo("Created KubernetesClusterScheduler instance") + var client = setupKubernetesClient() val driverName = s"spark-driver-${Random.alphanumeric take 5 mkString("")}".toLowerCase() val svcName = s"spark-svc-${Random.alphanumeric take 5 mkString("")}".toLowerCase() - val instances = conf.get(EXECUTOR_INSTANCES).getOrElse(1) - val serviceAccountName = conf.get("spark.kubernetes.serviceAccountName", "default") - val nameSpace = conf.get("spark.kubernetes.namespace", "default") - - logWarning("instances: " + instances) + val nameSpace = conf.get( + "spark.kubernetes.namespace", + KubernetesClusterScheduler.defaultNameSpace) + val serviceAccountName = conf.get( + "spark.kubernetes.serviceAccountName", + KubernetesClusterScheduler.defaultServiceAccountName) + + // Anything that should either not be passed to driver config in the cluster, or + // that is going to be explicitly managed as command argument to the driver pod + val confBlackList = scala.collection.Set( + "spark.master", + "spark.app.name", + "spark.submit.deployMode", + "spark.executor.jar", + "spark.dynamicAllocation.enabled", + "spark.shuffle.service.enabled") def start(args: ClientArguments): Unit = { startDriver(client, args) @@ -73,7 +90,7 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) val driverDescription = buildDriverDescription(args) // image needs to support shim scripts "/opt/driver.sh" and "/opt/executor.sh" - val sparkDriverImage = conf.getOption("spark.kubernetes.sparkImage").getOrElse { + val sparkImage = conf.getOption("spark.kubernetes.sparkImage").getOrElse { // TODO: this needs to default to some standard Apache Spark image throw new SparkException("Spark image not set. Please configure spark.kubernetes.sparkImage") } @@ -91,10 +108,10 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) s"--class=${args.userClass}", s"--master=$kubernetesHost", s"--executor-memory=${driverDescription.mem}", - s"--conf=spark.executor.jar=$clientJarUri", - s"--conf=spark.executor.instances=$instances", - s"--conf=spark.kubernetes.namespace=$nameSpace", - s"--conf=spark.kubernetes.driver.image=$sparkDriverImage") + s"--conf spark.executor.jar=$clientJarUri") + + submitArgs ++= conf.getAll.filter { case (name, _) => !confBlackList.contains(name) } + .map { case (name, value) => s"--conf ${name}=${value}" } if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { submitArgs ++= Vector( @@ -117,7 +134,7 @@ private[spark] class KubernetesClusterScheduler(conf: SparkConf) .withServiceAccount(serviceAccountName) .addNewContainer() .withName("spark-driver") - .withImage(sparkDriverImage) + .withImage(sparkImage) .withImagePullPolicy("Always") .withCommand(s"/opt/driver.sh") .withArgs(submitArgs :_*) diff --git a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 8d4f67a173071..bf83189e5e2f6 100644 --- a/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/kubernetes/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -50,15 +50,17 @@ private[spark] class KubernetesClusterSchedulerBackend( var shutdownToPod = mutable.Map.empty[String, String] // pending shutdown var executorID = 0 - val sparkDriverImage = sc.getConf.get("spark.kubernetes.driver.image") - val clientJarUri = sc.getConf.get("spark.executor.jar") - val ns = sc.getConf.get("spark.kubernetes.namespace") - val dynamicExecutors = Utils.isDynamicAllocationEnabled(sc.getConf) + val sparkImage = conf.get("spark.kubernetes.sparkImage") + val clientJarUri = conf.get("spark.executor.jar") + val ns = conf.get( + "spark.kubernetes.namespace", + KubernetesClusterScheduler.defaultNameSpace) + val dynamicExecutors = Utils.isDynamicAllocationEnabled(conf) // executor back-ends take their configuration this way if (dynamicExecutors) { - sc.getConf.setExecutorEnv("spark.dynamicAllocation.enabled", "true") - sc.getConf.setExecutorEnv("spark.shuffle.service.enabled", "true") + conf.setExecutorEnv("spark.dynamicAllocation.enabled", "true") + conf.setExecutorEnv("spark.shuffle.service.enabled", "true") } override def start(): Unit = { @@ -159,12 +161,7 @@ private[spark] class KubernetesClusterSchedulerBackend( initialNumExecutors } else { - val targetNumExecutors = - sys.env - .get("SPARK_EXECUTOR_INSTANCES") - .map(_.toInt) - .getOrElse(numExecutors) - conf.get(EXECUTOR_INSTANCES).getOrElse(targetNumExecutors) + conf.get(EXECUTOR_INSTANCES).getOrElse(numExecutors) } } @@ -195,7 +192,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .withNewSpec() .withRestartPolicy("OnFailure") - .addNewContainer().withName("spark-executor").withImage(sparkDriverImage) + .addNewContainer().withName("spark-executor").withImage(sparkImage) .withImagePullPolicy("IfNotPresent") .withCommand("/opt/executor.sh") .withArgs(submitArgs :_*) From 8584913d51c1f68c8f9d3cdf142947cd35cdfcc2 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Tue, 22 Nov 2016 15:48:20 -0800 Subject: [PATCH 14/14] Updated image to use. --- kubernetes/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kubernetes/README.md b/kubernetes/README.md index 99a523e326ec5..a93a6cd2c6e27 100644 --- a/kubernetes/README.md +++ b/kubernetes/README.md @@ -8,7 +8,7 @@ * ./build/mvn -Pkubernetes -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests package * Ensure that you are pointing to a k8s cluster (kubectl config current-context), which you want to use with spark. * Launch a spark-submit job: - * `./bin/spark-submit --deploy-mode cluster --class org.apache.spark.examples.SparkPi --master k8s://default --conf spark.executor.instances=5 --conf spark.kubernetes.sparkImage=manyangled/kube-spark:snapshot http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar 10000` + * `./bin/spark-submit --deploy-mode cluster --class org.apache.spark.examples.SparkPi --master k8s://default --conf spark.executor.instances=5 --conf spark.kubernetes.sparkImage=manyangled/kube-spark:dynamic http://storage.googleapis.com/foxish-spark-distro/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar 10000` * Optionally, the following config parameters can be supplied to spark-submit with additional `--conf` arguments (or a configuration file). * spark.kubernetes.serviceAccountName (defaults to "default") * spark.kubernetes.namespace (defaults to "default"). The namespace must exist prior to launching spark-submit.